)]}'
{
  "log": [
    {
      "commit": "95419e14a0d1986a4582a6364057193dce296992",
      "tree": "50733b57b8b1f97dc32ec2fcde5d606baac8bcc6",
      "parents": [
        "149f3b98b8500627c2fd6432d5fb0cabf314fd9b"
      ],
      "author": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Wed Apr 15 19:10:11 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Wed Apr 15 19:10:11 2026 +0800"
      },
      "message": "[CELEBORN-2301] MessageEncoder enables zero-copy sendfile for FileRegion in Netty native transports\n\n### What changes were proposed in this pull request?\n\n`MessageEncoder` enables zero-copy sendfile for `FileRegion` in Netty native transports, which emits the header `ByteBuf` and `FileRegion` as **separate objects** in the outbound message list when the body is a `FileRegion` backed by `FileSegmentManagedBuffer`, instead of wrapping them together in a `MessageWithHeader`.\n\nPreviously, all messages with a body were unconditionally wrapped in `MessageWithHeader`. This caused native transports (EPOLL, KQUEUE) to fall into a generic `FileRegion.transferTo()` fallback path that copies data through user-space, bypassing the optimized `sendfile()` / `splice()` zero-copy path that Netty\u0027s native transports provide.\n\nThe split is only applied when the `ManagedBuffer` is a `FileSegmentManagedBuffer`, whose `release()` is a no-op, making it safe to emit the `FileRegion` independently of write lifecycle management. Other `ManagedBuffer` types (e.g., `BlockManagerManagedBuffer`) still use the `MessageWithHeader` wrapper because they perform resource cleanup in `release()` that must be tied to `MessageWithHeader.deallocate()`.\n\nBackport: https://github.com/apache/spark/pull/55087.\n\n### Why are the changes needed?\n\nWhen using native transports (AUTO/EPOLL on Linux), file-backed shuffle fetch performance was severely degraded compared to NIO mode. The root cause lies in how Netty\u0027s native transports dispatch `FileRegion` writes.\n\nIn `AbstractEpollStreamChannel.doWriteSingle()` (and the analogous KQueue path), Netty uses an `instanceof` check to choose between two write strategies:\n\nhttps://github.com/netty/netty/blob/eeb5674526f0b49a142580686a5a9a7147ddadec/transport-classes-epoll/src/main/java/io/netty/channel/epoll/AbstractEpollStreamChannel.java#L474-L493\n\n```java\n} else if (msg instanceof DefaultFileRegion) {\n    return writeDefaultFileRegion(in, (DefaultFileRegion) msg);  // → socket.sendFile() (zero-copy)\n} else if (msg instanceof FileRegion) {\n    return writeFileRegion(in, (FileRegion) msg);                // → region.transferTo() (user-space copy)\n}\n```\n\n- **`writeDefaultFileRegion()`** calls `socket.sendFile()`, which maps directly to the Linux `sendfile()` syscall — a true zero-copy path where data is transferred from the file page cache to the socket buffer entirely within the kernel, with no user-space copy.\n\nhttps://github.com/netty/netty/blob/eeb5674526f0b49a142580686a5a9a7147ddadec/transport-classes-epoll/src/main/java/io/netty/channel/epoll/AbstractEpollStreamChannel.java#L367-L386\n\n```java\n    private int writeDefaultFileRegion(ChannelOutboundBuffer in, DefaultFileRegion region) throws Exception {\n        final long offset \u003d region.transferred();\n        final long regionCount \u003d region.count();\n        if (offset \u003e\u003d regionCount) {\n            in.remove();\n            return 0;\n        }\n\n        final long flushedAmount \u003d socket.sendFile(region, region.position(), offset, regionCount - offset);\n        if (flushedAmount \u003e 0) {\n            in.progress(flushedAmount);\n            if (region.transferred() \u003e\u003d regionCount) {\n                in.remove();\n            }\n            return 1;\n        } else if (flushedAmount \u003d\u003d 0) {\n            validateFileRegion(region, offset);\n        }\n        return WRITE_STATUS_SNDBUF_FULL;\n    }\n```\n\n- **`writeFileRegion()`** falls back to `region.transferTo(WritableByteChannel)`, which writes data through a `SocketWritableByteChannel` wrapper — effectively a user-space copy path.\n\nhttps://github.com/netty/netty/blob/eeb5674526f0b49a142580686a5a9a7147ddadec/transport-classes-epoll/src/main/java/io/netty/channel/epoll/AbstractEpollStreamChannel.java#L402-L420\n\n```java\nprivate int writeFileRegion(ChannelOutboundBuffer in, FileRegion region) throws Exception {\n        if (region.transferred() \u003e\u003d region.count()) {\n            in.remove();\n            return 0;\n        }\n\n        if (byteChannel \u003d\u003d null) {\n            byteChannel \u003d new EpollSocketWritableByteChannel();\n        }\n        final long flushedAmount \u003d region.transferTo(byteChannel, region.transferred());\n        if (flushedAmount \u003e 0) {\n            in.progress(flushedAmount);\n            if (region.transferred() \u003e\u003d region.count()) {\n                in.remove();\n            }\n            return 1;\n        }\n        return WRITE_STATUS_SNDBUF_FULL;\n    }\n```\n\nSpark\u0027s `MessageWithHeader extends AbstractFileRegion` (not `DefaultFileRegion`). When `MessageEncoder` wraps a `DefaultFileRegion` body inside `MessageWithHeader`, the resulting object is a generic `FileRegion` from Netty\u0027s perspective. This means Netty dispatches it to the `writeFileRegion()` fallback, which calls `MessageWithHeader.transferTo()`:\n\n```java\n// MessageWithHeader.java, line 121\nif (body instanceof FileRegion fileRegion) {\n    writtenBody \u003d fileRegion.transferTo(target, totalBytesTransferred - headerLength);\n}\n```\n\nHere, even though the inner body is a `DefaultFileRegion`, its `transferTo()` is invoked with a `WritableByteChannel` (not a file descriptor), so the data is read from the file into a user-space buffer and then written to the socket — **the zero-copy opportunity is lost**.\n\nBy emitting the `DefaultFileRegion` directly into Netty\u0027s outbound buffer (instead of wrapping it in `MessageWithHeader`), Netty\u0027s native transport recognizes it via `instanceof DefaultFileRegion` and routes it to `socket.sendFile()`, restoring the zero-copy `sendfile()` path.\n\n**Benchmark results (File-Backed Shuffle Fetch) show dramatic improvement:**\n\n| Scenario | Before (ms) | After (ms) | Improvement |\n|---|---|---|---|\n| EPOLL, sequential fetch (JDK8) | 524 | 134 | **~3.9x faster** |\n| EPOLL, parallel fetch (JDK8) | 191 | 55 | **~3.4x faster** |\n\n### Does this PR resolve a correctness bug?\n\nNo.\n\n### Does this PR introduce _any_ user-facing change?\n\nNo. This is an internal optimization to the Netty transport layer. Users benefit from improved shuffle fetch performance when using native transports (the default on Linux) without any configuration changes.\n\n### How was this patch tested?\n\n- Re-ran `NettyTransportBenchmark` with JDK8 to confirm the performance improvement. Updated benchmark result files accordingly.\n\nCloses #3649 from SteNicholas/CELEBORN-2301.\n\nLead-authored-by: SteNicholas \u003cprogramgeek@163.com\u003e\nCo-authored-by: Cheng Pan \u003cchengpan@apache.org\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "149f3b98b8500627c2fd6432d5fb0cabf314fd9b",
      "tree": "276c06da6a28836edc202846d30e9ffb497bd48b",
      "parents": [
        "c456df3ec6a4611895dcc7dfe4167ef1e261560c"
      ],
      "author": {
        "name": "Sanskar Modi",
        "email": "sanskarmodi97@gmail.com",
        "time": "Wed Apr 15 15:07:35 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Wed Apr 15 15:07:35 2026 +0800"
      },
      "message": "[CELEBORN-1577][BUG] Quota cancel shuffle should use app shuffle id\n\n### What changes were proposed in this pull request?\n\n- Added a new mapping for celebornShuffleId -\u003e appShuffleId\n- cancelAllActiveStages should passing appShuffleId not celebornShuffleId\n\n### Why are the changes needed?\n\n`shuffleAllocatedWorkers` worker contains celebornShuffleId, we need to use `appShuffleId` because DAGScheduler only understand app shuffle id.\n\n### Does this PR resolve a correctness bug?\n\nNo\n\n### Does this PR introduce _any_ user-facing change?\n\nNo\n\n### How was this patch tested?\n\nNA\n\nCloses #3662 from s0nskar/fix_quota_shuffle_id.\n\nAuthored-by: Sanskar Modi \u003csanskarmodi97@gmail.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "c456df3ec6a4611895dcc7dfe4167ef1e261560c",
      "tree": "222b50934eec1eb54bd9e91a7fdf21dd03abfc02",
      "parents": [
        "913d027efd1e56d080ec8080818e05ccf8eaa025"
      ],
      "author": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Mon Apr 13 10:26:56 2026 +0700"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Mon Apr 13 10:26:56 2026 +0700"
      },
      "message": "[CELEBORN-2298] Introduce NettyTransportBenchmark for Netty transport layer performance evaluation\n\n### What changes were proposed in this pull request?\n\nIntroduce `NettyTransportBenchmark` for Netty transport layer performance evaluation.\n\nAll suites measure performance through the actual Celeborn transport pipeline\n(`TransportServer` + `TransportClientFactory` + `TransportContext`).\n\nSuite overview:\n1. RPC Latency            - server-client RPC overhead at different payload sizes\n2. Concurrent Throughput  - multi-client pressure on the transport layer\n3. IOMode Comparison      - NIO vs native transport (Automatically selects EPOLL/KQUEUE)\n4. Server Thread Scaling  - validates MAX_DEFAULT_NETTY_THREADS\u003d8 cap\n5. Multi-Connection       - numConnectionsPerPeer\u003d1 vs 2 vs 4\n6. Async Write Pressure   - fire-and-forget RPCs to saturate the write path\n7. Large Block Transfer   - shuffle-like 16MB block transfers (in-memory payload)\n8. File-Backed Shuffle    - ChunkFetch from disk, NIO vs native transport (EPOLL sendfile bypass detection)\n\nBackport: https://github.com/apache/spark/pull/55061.\n\n### Why are the changes needed?\n\nNetty is a crucial third-party component for Celeborn. Introduce a micro-benchmark test facilitates:\n\n- Verify performance during subsequent Netty upgrades;\n- Validate performance after changes to relevant code in Celeborn.\n\n### Does this PR resolve a correctness bug?\n\nNo.\n\n### Does this PR introduce _any_ user-facing change?\n\nNo.\n\n### How was this patch tested?\n\nGenerated `NettyTransportBenchmark-results.txt`.\n\nCloses #3647 from SteNicholas/CELEBORN-2298.\n\nLead-authored-by: SteNicholas \u003cprogramgeek@163.com\u003e\nCo-authored-by: Cheng Pan \u003cchengpan@apache.org\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "913d027efd1e56d080ec8080818e05ccf8eaa025",
      "tree": "9a4cb97d0b6087452de4264ca509e447f806deca",
      "parents": [
        "163bcb36edc11d1b999827a4e06bfcdd3bc7b3ea"
      ],
      "author": {
        "name": "zhengtao",
        "email": "shuaizhentao.szt@alibaba-inc.com",
        "time": "Sat Apr 11 11:44:19 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Sat Apr 11 11:44:19 2026 +0800"
      },
      "message": "[CELEBORN-2287] Split mode should be HARD_SPLIT when disk is full\n\n### What changes were proposed in this pull request?\n\nChange the split mode to `HARD_SPLIT` when disk is full.\n\n### Why are the changes needed?\n\nWhen the disk is already in a full state, continuous writing data in `SOFT_SPLIT` mode may cause the reserved space to be filled up as well.\n\n### Does this PR resolve a correctness bug?\n\nNo.\n\n### Does this PR introduce _any_ user-facing change?\n\nNo.\n\n### How was this patch tested?\n\nManual test and UT.\n\nCloses #3653 from zaynt4606/clb2287.\n\nAuthored-by: zhengtao \u003cshuaizhentao.szt@alibaba-inc.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "163bcb36edc11d1b999827a4e06bfcdd3bc7b3ea",
      "tree": "61a031efca759baf3b9a72d7e137dd8b6ff3c7e2",
      "parents": [
        "a30166df08f884869ef3c0ef03829cd21bed8814"
      ],
      "author": {
        "name": "sychen",
        "email": "sychen@ctrip.com",
        "time": "Fri Apr 10 14:41:33 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Fri Apr 10 14:41:33 2026 +0800"
      },
      "message": "[CELEBORN-2297] Update workflow to manually install Helm and chart-testing\n\n### What changes were proposed in this pull request?\n\n### Why are the changes needed?\n```\nThe actions azure/setup-helmv4.2.0, docker/setup-buildx-actionv1, and docker/build-push-actionv2 are not allowed in apache/celeborn\n```\nhttps://github.com/apache/celeborn/actions/runs/23730849688\n\n```yml\n      - name: Setup chart-testing\n        uses: ./.github/actions/chart-testing-action\n```\n\n```\nThe action sigstore/cosign-installer11086d25041f77fe8fe7b9ea4e48e3b9192b8f19 is not allowed in apache/celebor\n```\n### Does this PR resolve a correctness bug?\nNo\n\n### Does this PR introduce _any_ user-facing change?\nNo\n\n### How was this patch tested?\nGHA\nhttps://github.com/apache/celeborn/actions/runs/23884713736/job/69645124360?pr\u003d3639\n\nCloses #3639 from cxzl25/fix_it_test.\n\nAuthored-by: sychen \u003csychen@ctrip.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "a30166df08f884869ef3c0ef03829cd21bed8814",
      "tree": "973a7f4d526ed2802804873e13bf46f90542f60e",
      "parents": [
        "c688c76d3c8de471afbf8c5fd1857b0a5926a721"
      ],
      "author": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Fri Apr 10 14:39:51 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Fri Apr 10 14:39:51 2026 +0800"
      },
      "message": "[CELEBORN-2305] Bump Ratis version from 3.2.1 to 3.2.2\n\n### What changes were proposed in this pull request?\n\nBump Ratis version from 3.2.1 to 3.2.2.\n\n### Why are the changes needed?\n\nBump Ratis version from 3.2.1 to 3.2.2. Ratis has released v3.2.2 of which release note refers to [3.2.2](https://ratis.apache.org/post/3.2.2.html). The 3.2.2 version is a maintenance release with multiple improvements and bugfixes.\n\n### Does this PR resolve a correctness bug?\n\nNo.\n\n### Does this PR introduce _any_ user-facing change?\n\nNo.\n\n### How was this patch tested?\n\nCI.\n\nCloses #3658 from SteNicholas/CELEBORN-2305.\n\nAuthored-by: SteNicholas \u003cprogramgeek@163.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "c688c76d3c8de471afbf8c5fd1857b0a5926a721",
      "tree": "6f36aa996d8ad62c0644e384f0492f4b05c96a04",
      "parents": [
        "c246031889495c45c3e96cf682f2378382411e0e"
      ],
      "author": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Thu Apr 09 20:35:35 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Thu Apr 09 20:35:35 2026 +0800"
      },
      "message": "[CELEBORN-2063][FOLLOWUP] Fix timeout unit for parallel creation of partition writer\n\n### What changes were proposed in this pull request?\n\nFix timeout unit for parallel creation of partition writer in `Utils#tryFuturesWithTimeout`.\n\nFollow up #3387, #3656.\n\n### Why are the changes needed?\n\n`Utils#tryFuturesWithTimeout` uses wrong timeout unit which does not match the config option `celeborn.worker.writer.create.parallel.timeout` as follows:\n\n```\nval WORKER_WRITER_CREATE_PARALLEL_TIMEOUT: ConfigEntry[Long] \u003d\n    buildConf(\"celeborn.worker.writer.create.parallel.timeout\")\n      .categories(\"worker\")\n      .version(\"0.6.3\")\n      .doc(\"Timeout for a worker to create a file writer in parallel.\")\n      .timeConf(TimeUnit.MILLISECONDS)\n      .createWithDefaultString(\"120s\")\n```\n\n### Does this PR resolve a correctness bug?\n\nNo.\n\n### Does this PR introduce _any_ user-facing change?\n\nNo.\n\n### How was this patch tested?\n\nCI.\n\nCloses #3657 from SteNicholas/CELEBORN-2063.\n\nAuthored-by: SteNicholas \u003cprogramgeek@163.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "c246031889495c45c3e96cf682f2378382411e0e",
      "tree": "e424bf054b33dd198e3b8dfb00c8835e387d792e",
      "parents": [
        "84830d9fcfe1067c050774d8f21a121b4b22911a"
      ],
      "author": {
        "name": "Xianming Lei",
        "email": "xianming.lei@shopee.com",
        "time": "Thu Apr 09 14:49:01 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Thu Apr 09 14:49:01 2026 +0800"
      },
      "message": "[CELEBORN-2294][FOLLOWUP] Fix flaky test SparkUtilsSuite\n\n### What changes were proposed in this pull request?\nRemove racy assertions in the \"check if fetch failure task another attempt is running or successful\" test in SparkUtilsSuite.\n\n### Why are the changes needed?\n\nAfter CELEBORN-2294 added a zombie TaskSetManager check in shouldReportShuffleFetchFailure, the test became flaky due to a race condition. The test calls shouldReportShuffleFetchFailure a second time from the test thread, but by that point the FetchFailed has already been processed by Spark\u0027s DAGScheduler — **the TaskSetManager is either marked as zombie** or the task has been removed from taskIdToTaskSetManager. This causes the second call to return false, failing the assertion.\n\n### Does this PR resolve a correctness bug?\n\nNo.\n\n### Does this PR introduce _any_ user-facing change?\n\nNo.\n\n### How was this patch tested?\n\nThe affected UT: SparkUtilsSuite - check if fetch failure task another attempt is running or successful.\n\nCloses #3655 from leixm/FOLLOW-CELEBORN-2294.\n\nAuthored-by: Xianming Lei \u003cxianming.lei@shopee.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "84830d9fcfe1067c050774d8f21a121b4b22911a",
      "tree": "dc44136c990291c6f63c631bd1d609e8aac4f62a",
      "parents": [
        "7f1bac3443f191d2bc5dcebebb88cd1744b53cd7"
      ],
      "author": {
        "name": "sychen",
        "email": "sychen@ctrip.com",
        "time": "Thu Apr 09 08:24:21 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Thu Apr 09 08:24:21 2026 +0800"
      },
      "message": "[CELEBORN-2304] Fix timeout unit mismatch in disk monitor check\n\n### What changes were proposed in this pull request?\nChanged tryWithTimeoutAndCallback and tryFutureWithTimeoutAndCallback in Utils.scala to accept timeout in milliseconds instead of seconds.\n\n### Why are the changes needed?\nWORKER_DEVICE_STATUS_CHECK_TIMEOUT is configured in milliseconds (e.g. 30s → 30000), but was passed directly to Future.get(..., TimeUnit.SECONDS)\n\nhttps://github.com/apache/celeborn/blob/7f1bac3443f191d2bc5dcebebb88cd1744b53cd7/common/src/main/scala/org/apache/celeborn/common/CelebornConf.scala#L4100-L4107\n\nhttps://github.com/apache/celeborn/blob/7f1bac3443f191d2bc5dcebebb88cd1744b53cd7/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/storage/DeviceMonitor.scala#L287-L291\n\n### Does this PR resolve a correctness bug?\nNo\n\n### Does this PR introduce _any_ user-facing change?\nNo\n\n### How was this patch tested?\nGHA\n\nCloses #3656 from cxzl25/CELEBORN-2304.\n\nAuthored-by: sychen \u003csychen@ctrip.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "7f1bac3443f191d2bc5dcebebb88cd1744b53cd7",
      "tree": "ee96a224a2ce2d92e94077cdb95006889bf12149",
      "parents": [
        "234ff2d705bd5a6c4a6603d3976af8a3615bcbf9"
      ],
      "author": {
        "name": "sychen",
        "email": "sychen@ctrip.com",
        "time": "Wed Apr 08 11:06:35 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Wed Apr 08 11:06:35 2026 +0800"
      },
      "message": "[CELEBORN-2302] Fix NPE in MemoryManager.close() when readBufferDispatcher is not initialized\n\n### What changes were proposed in this pull request?\nAdd a null check for readBufferDispatcher before calling close() in MemoryManager.close().\n\n### Why are the changes needed?\nreadBufferDispatcher is only initialized when readBufferThreshold \u003e 0.\n\n### Does this PR resolve a correctness bug?\nNo\n\n### Does this PR introduce _any_ user-facing change?\nNo\n\n### How was this patch tested?\nGHA\n\nCloses #3654 from cxzl25/CELEBORN-2302.\n\nAuthored-by: sychen \u003csychen@ctrip.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "234ff2d705bd5a6c4a6603d3976af8a3615bcbf9",
      "tree": "bd828e62cf9201f8e95a60c352375cb4c1d7d2ac",
      "parents": [
        "ca8533c8390bfcdcb418144e9fc95c3561fe641a"
      ],
      "author": {
        "name": "afterincomparableyum",
        "email": "224495379+afterincomparableyum@users.noreply.github.com",
        "time": "Wed Apr 08 11:04:32 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Wed Apr 08 11:04:32 2026 +0800"
      },
      "message": "[CELEBORN-2219][CIP-14] Support PushMergedData in CppClient\n\n### What changes were proposed in this pull request?\n\nImplement PushMergedData functionality in the C++ client, enabling batch merging and pushing of shuffle data grouped by worker address.\n\nKey changes:\n  - Add mergeData() and pushMergedData() to ShuffleClient, which accumulateper partition data batches and push them as merged payloads when the buffer threshold is exceeded or at mapper end.\n  - Introduce DataBatches class to manage batch accumulation thread-safe add/take operations and size-bounded requireBatches().\n  - Add PushMergedDataCallback to handle success responses (split handling, congestion control, MAP_ENDED) and failure responses with revive-based retry via submitRetryPushMergedData().\n  - Add PushMergedData network message type with encoding for partitionUniqueIds and batchOffsets arrays.\n  - Extend Encoders with encode/decode support for vector\u003cstring\u003e and vector\u003cint32_t\u003e.\n  - Add pushMergedDataAsync() to TransportClient.\n  - Add unit tests for DataBatches, PushMergedData message encoding, and array encoders.\n\n### Why are the changes needed?\n\nThis is needed to extend the functionality of C++ Client and there is a Bolt dependency on it with https://github.com/bytedance/bolt/issues/370\n\n### Does this PR resolve a correctness bug?\n\nNo\n\n### Does this PR introduce _any_ user-facing change?\n\nYes because it is a new functionality in the c++ client\n\n### How was this patch tested?\n\nTested through running unit tests and compiling locally.\n\nCloses #3611 from afterincomparableyum/cpp-client/celeborn-2219.\n\nAuthored-by: afterincomparableyum \u003c224495379+afterincomparableyum@users.noreply.github.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "ca8533c8390bfcdcb418144e9fc95c3561fe641a",
      "tree": "05ca0cca878a6e1c7208e0dcee9be35ab8c365c5",
      "parents": [
        "37aed7ea5421e0de9bfdfbd452746103a579232e"
      ],
      "author": {
        "name": "sychen",
        "email": "sychen@ctrip.com",
        "time": "Wed Apr 08 10:55:07 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Wed Apr 08 10:55:07 2026 +0800"
      },
      "message": "[CELEBORN-2300] Change the default value of celeborn.port.maxRetries from 1 to 16\n\n### What changes were proposed in this pull request?\n\n### Why are the changes needed?\n\nAlign with spark.port.maxRetries, default to 16 retries.\n\n```java\nERROR Executor: Exception in task 1153.0 in stage 11763.0 (TID 516674)\njava.lang.RuntimeException: java.net.BindException: Address already in use: Service \u0027ShuffleClient\u0027 failed after 1 retries (on a random free port)! Consider explicitly setting the appropriate binding address for the service \u0027ShuffleClient\u0027 to the correct binding address.\n\tat org.apache.spark.shuffle.celeborn.SparkShuffleManager.getWriter(SparkShuffleManager.java:310)\n```\n\n### Does this PR resolve a correctness bug?\nNo\n\n### Does this PR introduce _any_ user-facing change?\nYes\n\n### How was this patch tested?\nGHA\n\nCloses #3648 from cxzl25/CELEBORN-2300.\n\nAuthored-by: sychen \u003csychen@ctrip.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "37aed7ea5421e0de9bfdfbd452746103a579232e",
      "tree": "fb701374002a3929cc7008693d851ed9e2d8d546",
      "parents": [
        "37a27bc4cb3c993289e5b94439db224572cd3e66"
      ],
      "author": {
        "name": "Xianming Lei",
        "email": "jerrylei@apache.org",
        "time": "Fri Apr 03 18:00:01 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Fri Apr 03 18:00:01 2026 +0800"
      },
      "message": "[CELEBORN-2295] CommitHandler should support retry interval\n\n### What changes were proposed in this pull request?\n\n`CommitHandler` should support retry interval for retry of committing file.\n\n### Why are the changes needed?\n\nWhen commitFiles RPC fails, the current implementation retries immediately without any backoff. If the worker is experiencing transient network issues, immediate retries are likely to fail again. Adding a configurable retry interval (`celeborn.client.requestCommitFiles.retryInterval`, default 10s) gives the worker time to recover before the next attempt, significantly improving the success rate of retries. A dedicated `ScheduledExecutorService` is used to avoid blocking threads in the shared RPC pool during the wait.\n\n### Does this PR resolve a correctness bug?\n\nNo.\n\n### Does this PR introduce _any_ user-facing change?\n\nNo.\n\n### How was this patch tested?\n\nCI.\n\nCloses #3642 from leixm/CELEBORN-2295.\n\nLead-authored-by: Xianming Lei \u003cjerrylei@apache.org\u003e\nCo-authored-by: Xianming Lei \u003cxianming.lei@shopee.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "37a27bc4cb3c993289e5b94439db224572cd3e66",
      "tree": "727807a0fb7165ebcdf3eb56dd1873bac849ba37",
      "parents": [
        "097f1df172e7ed8f2d32c3c5a8c04b22a595fe25"
      ],
      "author": {
        "name": "sychen",
        "email": "sychen@ctrip.com",
        "time": "Thu Apr 02 10:56:26 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Thu Apr 02 10:56:26 2026 +0800"
      },
      "message": "[CELEBORN-2296] Fix race condition in MemoryManager singleton initialization\n\n### What changes were proposed in this pull request?\nApply double-checked locking to MemoryManager.initialize() and add synchronized to MemoryManager.reset() to make the singleton lifecycle thread-safe.\n\n### Why are the changes needed?\n\n```java\n26/04/01 10:32:20,050 ERROR [worker 3 starter thread] WordCountTestWithAuthentication: create worker failed, detail:\njava.lang.NullPointerException\n        at org.apache.celeborn.service.deploy.worker.memory.ChannelsLimiter.\u003cinit\u003e(ChannelsLimiter.java:52)\n        at org.apache.celeborn.service.deploy.worker.Worker.\u003cinit\u003e(Worker.scala:240)\n        at org.apache.celeborn.service.deploy.MiniClusterFeature.createWorker(MiniClusterFeature.scala:172)\n        at org.apache.celeborn.service.deploy.MiniClusterFeature.createWorker$(MiniClusterFeature.scala:153)\n        at org.apache.celeborn.tests.flink.WordCountTestBase.createWorker(WordCountTest.scala:44)\n        at org.apache.celeborn.service.deploy.MiniClusterFeature.createWorker(MiniClusterFeature.scala:150)\n        at org.apache.celeborn.service.deploy.MiniClusterFeature.createWorker$(MiniClusterFeature.scala:149)\n        at org.apache.celeborn.tests.flink.WordCountTestBase.createWorker(WordCountTest.scala:44)\n        at org.apache.celeborn.service.deploy.MiniClusterFeature.$anonfun$setUpWorkers$2(MiniClusterFeature.scala:221)\n        at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)\n        at org.apache.celeborn.service.deploy.MiniClusterFeature$RunnerWrap.$anonfun$run$1(MiniClusterFeature.scala:50)\n        at org.apache.celeborn.common.util.Utils$.tryLogNonFatalError(Utils.scala:234)\n        at org.apache.celeborn.service.deploy.MiniClusterFeature$RunnerWrap.run(MiniClusterFeature.scala:50)\n```\n\n### Does this PR resolve a correctness bug?\nNo\n\n### Does this PR introduce _any_ user-facing change?\nNo\n\n### How was this patch tested?\nGHA\n\nCloses #3643 from cxzl25/CELEBORN-2296.\n\nAuthored-by: sychen \u003csychen@ctrip.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "097f1df172e7ed8f2d32c3c5a8c04b22a595fe25",
      "tree": "f6dc3c19226cdf0132931530f905a7dcb0236f6b",
      "parents": [
        "c3adbca3cd02f5f34ba120f86bffbee71fe78e03"
      ],
      "author": {
        "name": "Xianming Lei",
        "email": "jerrylei@apache.org",
        "time": "Thu Apr 02 10:52:28 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Thu Apr 02 10:52:28 2026 +0800"
      },
      "message": "[CELEBORN-2294] The shuffle fetch failed report from the zombie stage should be ignored\n\n### What changes were proposed in this pull request?\nThe shuffle fetch failed report from the zombie stage should be ignored\n\n### Why are the changes needed?\nWithout this PR, if a stage attempt has already triggered FetchFailed, there will still be running tasks reporting fetch failed to LifeCycleManager, which will cause the current Stage Attempt to mistakenly trigger a stage rerun.\n\nSpark also ignores FetchFailed from the previous stage attempt, and Celeborn should keep the same logic.\n\u003cimg width\u003d\"1760\" height\u003d\"664\" alt\u003d\"image\" src\u003d\"https://github.com/user-attachments/assets/201cd849-475b-442f-bec1-0b8ef1048036\" /\u003e\n\n### Does this PR resolve a correctness bug?\nNo.\n\n### Does this PR introduce _any_ user-facing change?\nNo.\n\n### How was this patch tested?\nExisting UTs.\n\nCloses #3640 from leixm/main.\n\nAuthored-by: Xianming Lei \u003cjerrylei@apache.org\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "c3adbca3cd02f5f34ba120f86bffbee71fe78e03",
      "tree": "379b2a992c9f7abe7206a3f38ffccaa972aead0d",
      "parents": [
        "d3b75132daeaa0484ae2562fe2a9c9fadfa70c92"
      ],
      "author": {
        "name": "Kartikay Bhutani",
        "email": "kbhutani0001@gmail.com",
        "time": "Thu Apr 02 10:50:09 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Thu Apr 02 10:50:09 2026 +0800"
      },
      "message": "[CELEBORN-2293] Fix ConcurrentModificationException in WorkerStatusTracker.shuttingWorkers\n\n### What changes were proposed in this pull request?\n\nReplace `HashSet` with `ConcurrentHashMap.newKeySet()` for `shuttingWorkers` in `WorkerStatusTracker`.\n\n### Why are the changes needed?\n\nWhen multiple shuffles hit a shutting-down worker simultaneously, one thread iterates `shuttingWorkers` in `currentFailedWorkers()` (for logging) while another modifies it in `recordWorkerFailure()`.\n\n### Does this PR resolve a correctness bug?\n\nYes.\n\n### Does this PR introduce _any_ user-facing change?\n\nNo.\n\n### How was this patch tested?\n\nAdded a unit test in `WorkerStatusTrackerSuite` and ran the same. Was able to reproduce only once (without the fix) out of multiple runs.\n\nCloses #3638 from kaybhutani/fix-concurrent-shuttingworkers.\n\nLead-authored-by: Kartikay Bhutani \u003ckbhutani0001@gmail.com\u003e\nCo-authored-by: kartikay \u003ckbhutani0001@gmail.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "d3b75132daeaa0484ae2562fe2a9c9fadfa70c92",
      "tree": "f3c10bf4c3e9baa4af5cde522c24052379c37a6f",
      "parents": [
        "42f1a08c1bcf0ba117d05d91a766d1214e5b881f"
      ],
      "author": {
        "name": "luogen.lg",
        "email": "luogen.lg@alibaba-inc.com",
        "time": "Wed Apr 01 15:48:42 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Wed Apr 01 15:48:42 2026 +0800"
      },
      "message": "[CELEBORN-2292] Fix ArithmeticException when PUSH_DATA_HAND_SHAKE fails before any data written\n\n### What changes were proposed in this pull request?\n\nNOTE: This is the same patch with #3637 pushing to main branch. Because some code has been refactored, the original patch can not be simply cherry-picked.\n\nHandle the case where numSubpartitions is zero in MapPartitionDataReader.open(). When the partition is empty, treat it as a normal empty partition and notify consumers accordingly.\n\n### Why are the changes needed?\n\nWhen the first PUSH_DATA_HAND_SHAKE request fails (e.g., timeout), client triggers revive with reason HARD_SPLIT. Manager adds the failed partition to partition locations, but numSubpartitions remains uninitialized (zero). Reading such partition causes ArithmeticException: / by zero.\nSince this is caused by client-side behavior, we handle it on worker side first for cross-version compatibility. The issue that flink shuffle client revives with fixed reason HARD_SPLIT can be addressed in later PRs.\n\n### Does this PR resolve a correctness bug?\n\nNo.\n\n### Does this PR introduce any user-facing change?\n\nNo.\n\n### How was this patch tested?\n\nManually tested with a hacked version that throws exception on the first handshake invocation. But the test code is too hacky to included into this PR. Advices are welcomed on how to add a proper unit test for this scenario without introducing too much complexity.\n\nCloses #3641 from pltbkd/CELEBORN-2292-on-main.\n\nAuthored-by: luogen.lg \u003cluogen.lg@alibaba-inc.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "42f1a08c1bcf0ba117d05d91a766d1214e5b881f",
      "tree": "a0a969901d16522734ca7c9e80e34e302a9f1166",
      "parents": [
        "235f07de49339d4cc2d5ac94deb55106a79ca7b4"
      ],
      "author": {
        "name": "Aravind Patnam",
        "email": "akpatnam25@gmail.com",
        "time": "Mon Mar 30 14:20:21 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Mon Mar 30 14:20:21 2026 +0800"
      },
      "message": "[CELEBORN-2284] Fix TLS Memory Leak\n\n### What changes were proposed in this pull request?\nWhile running jobs with TLS enabled, we encountered memory leaks which cause worker OOMs.\n```\n26/02/13 21:02:52,779 ERROR [push-server-9-9] ResourceLeakDetector: LEAK: ByteBuf.release() was not called before it\u0027s garbage-collected. See https://netty.io/wiki/reference-counted-objects.html for more information.\nRecent access records:\nCreated at:\n\tio.netty.buffer.AbstractByteBufAllocator.compositeDirectBuffer(AbstractByteBufAllocator.java:224)\n\tio.netty.buffer.AbstractByteBufAllocator.compositeBuffer(AbstractByteBufAllocator.java:202)\n\torg.apache.celeborn.common.network.util.TransportFrameDecoder.decodeNext(TransportFrameDecoder.java:143)\n\torg.apache.celeborn.common.network.util.TransportFrameDecoder.channelRead(TransportFrameDecoder.java:66)\n\tio.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:444)\n\tio.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)\n\tio.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:412)\n\tio.netty.handler.ssl.SslHandler.unwrap(SslHandler.java:1475)\n\tio.netty.handler.ssl.SslHandler.decodeJdkCompatible(SslHandler.java:1338)\n\tio.netty.handler.ssl.SslHandler.decode(SslHandler.java:1387)\n\tio.netty.handler.codec.ByteToMessageDecoder.decodeRemovalReentryProtection(ByteToMessageDecoder.java:529)\n\tio.netty.handler.codec.ByteToMessageDecoder.callDecode(ByteToMessageDecoder.java:468)\n\tio.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:290)\n\tio.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:444)\n\tio.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)\n\tio.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:412)\n\tio.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1410)\n\tio.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:440)\n\tio.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)\n\tio.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:919)\n\tio.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:166)\n\tio.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:788)\n\tio.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:724)\n\tio.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:650)\n\tio.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:562)\n\tio.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:997)\n\tio.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)\n\tio.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)\n\tjava.base/java.lang.Thread.run(Thread.java:840)\n\n```\n\nWhen a Celeborn worker receives a PushData or PushMergedData message, it replicates that frame to a secondary worker for fault tolerance. On an SSL-enabled cluster this replication goes through SslMessageEncoder.encode(). Here is the flow of what happens inside SslMessageEncoder.encode():\n\n- The encoder asks the message body for an SSL-friendly copy by calling convertToNettyForSsl(). For shuffle data, the body is a NettyManagedBuffer — data already loaded in off-heap memory. This call runs buf.duplicate().retain(), which creates a second reference to the same memory and increments the reference count from 1 to 2.\n\n- The encoder places this second reference inside a composite buffer and hands it to Netty for writing.\n\n- Netty writes the composite to the network, then releases it — decrementing the count from 2 to 1.\n\n- Nothing releases the original NettyManagedBuffer\u0027s hold on the data, so the count stays at 1 forever.\n\n- This results in every replicated PushData frame leaking a chunk of off-heap memory, eventually causing OOM and worker crash.\n\nThe fix for this issue is to release the original message body, so that the net reference count is preserved. The second reference — now living inside the composite buffer in out — keeps the memory alive while Netty writes it to the network. When Netty finishes and releases the composite, the count reaches 0 and the memory is freed cleanly.\n\nThis is exactly what the non-SSL MessageEncoder already does via MessageWithHeader.deallocate() — the SSL path simply needed to replicate that behavior explicitly.\n\n### Why are the changes needed?\nfix memory leak\n\n### Does this PR resolve a correctness bug?\n\n### Does this PR introduce _any_ user-facing change?\nno\n\n### How was this patch tested?\nalready internally in production and tested.\nAlso added unit tests\n\nCloses #3630 from akpatnam25/CELEBORN-2284.\n\nAuthored-by: Aravind Patnam \u003cakpatnam25@gmail.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "235f07de49339d4cc2d5ac94deb55106a79ca7b4",
      "tree": "5070577cb04f73b7a0887f4809a90f912104a227",
      "parents": [
        "6fc5565319c6622aef8637a5cf5fa348fc763745"
      ],
      "author": {
        "name": "sychen",
        "email": "sychen@ctrip.com",
        "time": "Mon Mar 30 10:37:43 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Mon Mar 30 10:37:43 2026 +0800"
      },
      "message": "[CELEBORN-2277] Replace synchronized in Flusher.getWorkerIndex with AtomicInteger\n\n### What changes were proposed in this pull request?\n\nReplace the synchronized block in getWorkerIndex with an AtomicInteger.updateAndGet call using a CAS-based atomic operation.\n\n### Why are the changes needed?\n\nThe synchronized keyword locks the entire object and causes thread contention under high concurrency. Using AtomicInteger reduces lock scope to a single variable and avoids blocking overhead for this lightweight index increment operation.\n\n### Does this PR resolve a correctness bug?\nNo.\n\n### Does this PR introduce _any_ user-facing change?\nNo.\n\n### How was this patch tested?\n\nGHA.\n\nCloses #3621 from cxzl25/CELEBORN-2277.\n\nLead-authored-by: sychen \u003csychen@ctrip.com\u003e\nCo-authored-by: cxzl25 \u003c3898450+cxzl25@users.noreply.github.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "6fc5565319c6622aef8637a5cf5fa348fc763745",
      "tree": "9f614a4b7cfcddecbe4e95e2687f0ce6eb1b65d2",
      "parents": [
        "3773c6568a3041bab2294e4a2e31ce6a42a05ab6"
      ],
      "author": {
        "name": "Kartikay Bhutani",
        "email": "kbhutani0001@gmail.com",
        "time": "Fri Mar 27 10:35:55 2026 +0800"
      },
      "committer": {
        "name": "子懿",
        "email": "ziyi.jxf@antgroup.com",
        "time": "Fri Mar 27 10:35:55 2026 +0800"
      },
      "message": "[CELEBORN-2291] Support fsync on commit to ensure shuffle data durability\n\n### What changes were proposed in this pull request?\n  Add a new configuration `celeborn.worker.commitFiles.fsync` (default `false`) that calls `FileChannel.force(false)` (fdatasync) before closing the channel in\n   `LocalTierWriter.closeStreams()`.\n\n  ### Why are the changes needed?\n\n  Without this, committed shuffle data can sit in the OS page cache before the kernel flushes it to disk. A hard crash in that window loses data even though Celeborn considers it committed. This option lets operators opt into stronger durability guarantees.\n\n  ### Does this PR resolve a correctness bug?\n\n  No. It adds an optional durability enhancement.\n\n  ### Does this PR introduce _any_ user-facing change?\n\n  Yes. New configuration key `celeborn.worker.commitFiles.fsync` (boolean, default `false`).\n\n  ### How was this patch tested?\n\n  Existing unit tests. Configuration verified via `ConfigurationSuite` and for LocalTierWriter added a new test with fsync enabled and ran `TierWriterSuite`.\n\nAdditional context: [slack](https://apachecelebor-kw08030.slack.com/archives/C04B1FYS6SY/p1774259245973229)\n\nCloses #3635 from kaybhutani/kartikay/fsync-on-commit.\n\nAuthored-by: Kartikay Bhutani \u003ckbhutani0001@gmail.com\u003e\nSigned-off-by: 子懿 \u003cziyi.jxf@antgroup.com\u003e\n"
    },
    {
      "commit": "3773c6568a3041bab2294e4a2e31ce6a42a05ab6",
      "tree": "e026d2ebb409123e181430dafaa2507b7548a211",
      "parents": [
        "28a0733bb2441f98531f531b26800c0d3ea06e99"
      ],
      "author": {
        "name": "sychen",
        "email": "sychen@ctrip.com",
        "time": "Mon Mar 23 14:02:54 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Mon Mar 23 14:02:54 2026 +0800"
      },
      "message": "[CELEBORN-2285] Bump maven 3.9.14\n\n### What changes were proposed in this pull request?\n\n### Why are the changes needed?\nhttps://maven.apache.org/docs/3.9.14/release-notes.html\n\n### Does this PR resolve a correctness bug?\nNo\n\n### Does this PR introduce _any_ user-facing change?\nNo\n\n### How was this patch tested?\nGHA\n\nCloses #3634 from cxzl25/CELEBORN-2285.\n\nAuthored-by: sychen \u003csychen@ctrip.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "28a0733bb2441f98531f531b26800c0d3ea06e99",
      "tree": "397acb36495a4f97d5b4677df79c952f49b7f82d",
      "parents": [
        "5fc0f199a4080c407bcb2bd5225b6e1edafaadf8"
      ],
      "author": {
        "name": "sychen",
        "email": "sychen@ctrip.com",
        "time": "Thu Mar 19 11:12:12 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Thu Mar 19 11:12:12 2026 +0800"
      },
      "message": "[CELEBORN-2276] Fix race condition in MemoryManager.releaseSortMemory\n\n### What changes were proposed in this pull request?\nUse `updateAndGet`\n\n### Why are the changes needed?\n`reserveSortMemory` calls `sortMemoryCounter.addAndGet` without any synchronized, so the lock in `releaseSortMemory` doesn\u0027t actually protect against races with `reserveSortMemory` anyway.\n\n### Does this PR resolve a correctness bug?\nNo\n\n### Does this PR introduce _any_ user-facing change?\nNo\n\n### How was this patch tested?\nGHA\n\nCloses #3620 from cxzl25/CELEBORN-2276.\n\nAuthored-by: sychen \u003csychen@ctrip.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "5fc0f199a4080c407bcb2bd5225b6e1edafaadf8",
      "tree": "79b2c6b2efc645803668eaca45a08812b2f97b0b",
      "parents": [
        "b4cb5a0b1ac097d33baf8dded1b5be2afd0578a4"
      ],
      "author": {
        "name": "yew1eb",
        "email": "yew1eb@gmail.com",
        "time": "Wed Mar 18 22:22:55 2026 +0800"
      },
      "committer": {
        "name": "Shuang",
        "email": "lvshuang.xjs@alibaba-inc.com",
        "time": "Wed Mar 18 22:22:55 2026 +0800"
      },
      "message": "[CELEBORN-2282] Eliminate redundant HashMap lookups in CelebornInputStream#fillBuffer\n\n### What changes were proposed in this pull request?\n\nReplace three redundant HashMap operations (`containsKey` + `put` + `get`) with a single `computeIfAbsent` call in `CelebornInputStream#fillBuffer`.\n\n### Why are the changes needed?\n\n`fillBuffer` is on the hot shuffle-read path and called for every batch read. The original code performs up to three HashMap lookups per batch instead of one, causing unnecessary CPU overhead at scale.\n\n### Does this PR resolve a correctness bug?\n\nNo.\n\n### Does this PR introduce _any_ user-facing change?\n\nNo.\n\n### How was this patch tested?\n\nExisting unit tests.\n\nCloses #3627 from yew1eb/CELEBORN-2282.\n\nAuthored-by: yew1eb \u003cyew1eb@gmail.com\u003e\nSigned-off-by: Shuang \u003clvshuang.xjs@alibaba-inc.com\u003e\n"
    },
    {
      "commit": "b4cb5a0b1ac097d33baf8dded1b5be2afd0578a4",
      "tree": "d24110ceb4550b5f3dbb920f48673316e9f11f79",
      "parents": [
        "7b25797fb8cf08c3495419b1d872663b08f829fb"
      ],
      "author": {
        "name": "yew1eb",
        "email": "yew1eb@gmail.com",
        "time": "Wed Mar 18 22:20:47 2026 +0800"
      },
      "committer": {
        "name": "Shuang",
        "email": "lvshuang.xjs@alibaba-inc.com",
        "time": "Wed Mar 18 22:20:47 2026 +0800"
      },
      "message": "[CELEBORN-2283][BUG] Fix missing return in Master.handleRequestSlots when all workers are excluded\n\n### What changes were proposed in this pull request?\n  Add a missing `return` statement after `context.reply()` in `Master#handleRequestSlots`\n  when `numAvailableWorkers \u003d\u003d 0`.\n\n### Why are the changes needed?\n  When all workers are excluded, the code replies with `WORKER_EXCLUDED` but continues\n  executing to `Random.nextInt(numAvailableWorkers)` (i.e. `Random.nextInt(0)`), which\n  throws `IllegalArgumentException`. This results in a duplicate response being sent to\n  the client and misleading error logs on the Master side.\n\n### Does this PR resolve a correctness bug?\n No.\n\n### Does this PR introduce _any_ user-facing change?\n No.\n\n### How was this patch tested?\nExisting unit tests.\n\nCloses #3628 from yew1eb/CELEBORN-2283.\n\nAuthored-by: yew1eb \u003cyew1eb@gmail.com\u003e\nSigned-off-by: Shuang \u003clvshuang.xjs@alibaba-inc.com\u003e\n"
    },
    {
      "commit": "7b25797fb8cf08c3495419b1d872663b08f829fb",
      "tree": "f698d78b036c5534c79d58601d729d6c99d3440a",
      "parents": [
        "15cba469467b6038c5c40ee50c6420726a7c1a2e"
      ],
      "author": {
        "name": "Shuai Lu",
        "email": "lushuainada@gmail.com",
        "time": "Wed Mar 18 22:18:21 2026 +0800"
      },
      "committer": {
        "name": "Shuang",
        "email": "lvshuang.xjs@alibaba-inc.com",
        "time": "Wed Mar 18 22:18:21 2026 +0800"
      },
      "message": "[CELEBORN-2274] Fix replicate channels not resumed when transitioning from PUSH_AND_REPLICATE_PAUSED to PUSH_PAUSED\n\n### What changes were proposed in this pull request?\n\nFix a bug in `MemoryManager.switchServingState()` where replicate channels permanently lose `autoRead\u003dtrue` after a memory pressure event.\n\nWhen the serving state transitions from `PUSH_AND_REPLICATE_PAUSED` to `PUSH_PAUSED`, `resumeReplicate()` was only called inside the `!tryResumeByPinnedMemory()` guard. If `tryResumeByPinnedMemory()` returned `true`, the entire block was skipped and replicate channels were never resumed.\n\nThe fix moves `resumeReplicate()` outside the `tryResumeByPinnedMemory()` guard so it is always called when stepping down from `PUSH_AND_REPLICATE_PAUSED` to `PUSH_PAUSED`. This is a state machine invariant: `PUSH_PAUSED` means only push is paused; replicate must always be resumed.\n\n### Why are the changes needed?\n\nOnce replicate channels are stuck with `autoRead\u003dfalse`, Netty I/O threads stop reading from all replicate connections. Remote workers writing to the affected worker see their TCP send buffers fill up (zero window), causing pending writes to accumulate in `ChannelOutboundBuffer`. Each pending write holds a reference to a direct memory `ByteBuf`, causing direct memory to grow indefinitely on the remote workers.\n\nThe failure sequence:\n1. Worker hits memory pressure → state \u003d `PUSH_AND_REPLICATE_PAUSED` → all channels paused\n2. Pinned memory is low → `tryResumeByPinnedMemory()` returns `true` → `resumeByPinnedMemory(PUSH_PAUSED)` resumes push only, replicate not resumed\n3. Memory drops to push-only range → state \u003d `PUSH_PAUSED`, but `resumeReplicate()` is never called\n4. Replicate channels permanently stuck with `autoRead\u003dfalse`, causing unbounded direct memory growth on remote workers\n\n### Does this PR resolve a correctness bug?\n\nYes.\n\n### Does this PR introduce _any_ user-facing change?\n\nNo.\n\n### How was this patch tested?\n\nAdded a new unit test `Test MemoryManager resume replicate by pinned memory` in `MemoryManagerSuite` that reproduces the exact failure scenario:\n1. Enter `PUSH_AND_REPLICATE_PAUSED` with low pinned memory (channels resumed by pinned memory path)\n2. Raise pinned memory so both push and replicate get paused\n3. Drop memory to `PUSH_PAUSED` range with low pinned memory\n4. Assert replicate listener is resumed — this assertion fails without the fix\n\nCloses #3616 from sl3635/CELEBORN-2274.\n\nAuthored-by: Shuai Lu \u003clushuainada@gmail.com\u003e\nSigned-off-by: Shuang \u003clvshuang.xjs@alibaba-inc.com\u003e\n"
    },
    {
      "commit": "15cba469467b6038c5c40ee50c6420726a7c1a2e",
      "tree": "f60554c3990fbe0d826ae6a149306847fa6d3e50",
      "parents": [
        "af0ba1a5ec0e1faf3d4a0d189058c755aeb6b18c"
      ],
      "author": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Wed Mar 18 20:41:01 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Wed Mar 18 20:41:01 2026 +0800"
      },
      "message": "[MINOR] Update repository references from incubator-gluten to gluten for TLP graduation of gluten\n\n### What changes were proposed in this pull request?\n\nUpdate repository references from incubator-gluten to gluten for TLP graduation of gluten.\n\n- Backport https://github.com/apache/gluten/pull/11735.\n- Close #3631.\n\n### Why are the changes needed?\n\nApache Gluten has already TLP graduated, which should update repository references.\n\n### Does this PR resolve a correctness bug?\n\nNo.\n\n### Does this PR introduce _any_ user-facing change?\n\nNo.\n\n### How was this patch tested?\n\nNo.\n\nCloses #3633 from SteNicholas/gluten-graduation.\n\nAuthored-by: SteNicholas \u003cprogramgeek@163.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "af0ba1a5ec0e1faf3d4a0d189058c755aeb6b18c",
      "tree": "c991f7586da7fe2cded1fb9dd0b498deffd69afd",
      "parents": [
        "400e9518d7f19cacf8797971e169afa6bdea5be6"
      ],
      "author": {
        "name": "sychen",
        "email": "sychen@ctrip.com",
        "time": "Wed Mar 18 15:55:32 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Wed Mar 18 15:55:32 2026 +0800"
      },
      "message": "[CELEBORN-2281] Improve error logging and null checks in CreditStreamManager\n\n### What changes were proposed in this pull request?\n\n- Initialize `AtomicReference\u003cIOException\u003e` with proper syntax.\n- Add exception to `logger.error` for better error context.\n- Simplify and improve null checks and logging in `addCredit` and `cleanResource` methods.\n\n### Why are the changes needed?\n\nnit.\n\n### Does this PR resolve a correctness bug?\n\nNo.\n\n### Does this PR introduce _any_ user-facing change?\n\nNo.\n\n### How was this patch tested?\n\nGHA.\n\nCloses #3626 from cxzl25/CELEBORN-2281.\n\nLead-authored-by: sychen \u003csychen@ctrip.com\u003e\nCo-authored-by: cxzl25 \u003c3898450+cxzl25@users.noreply.github.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "400e9518d7f19cacf8797971e169afa6bdea5be6",
      "tree": "4cf6016fecd2cbcc7f539a8b3a796b71234acc8f",
      "parents": [
        "8c2c9523d07a38a6c750834e8d9598d1e8571157"
      ],
      "author": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Sat Mar 14 17:27:00 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Sat Mar 14 17:27:00 2026 +0800"
      },
      "message": "[CELEBORN-2280] Support celeborn.network.memory.allocator.type to specify netty memory allocator\n\n### What changes were proposed in this pull request?\n\nSupport `celeborn.network.memory.allocator.type` to specify netty memory allocator including `AdaptiveByteBufAllocator `.\n\n### Why are the changes needed?\n\nNetty 4.2 introduces `AdaptiveByteBufAllocator` an auto-tuning pooling `ByteBufAllocator` which uses `AdaptivePoolingAllocator` added in https://github.com/netty/netty/pull/13075.\n\n### Does this PR resolve a correctness bug?\n\nNo.\n\n### Does this PR introduce _any_ user-facing change?\n\nIntroduce `celeborn.network.memory.allocator.type` to specify netty memory allocator.\n\n### How was this patch tested?\n\nCI.\n\nCloses #3625 from SteNicholas/CELEBORN-2280.\n\nAuthored-by: SteNicholas \u003cprogramgeek@163.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "8c2c9523d07a38a6c750834e8d9598d1e8571157",
      "tree": "1662f58628e70605270ca37536b07a14d9e53023",
      "parents": [
        "4b157c68a5f7e82f4a50b8a2c8ae8989aff843c1"
      ],
      "author": {
        "name": "sychen",
        "email": "sychen@ctrip.com",
        "time": "Tue Mar 10 17:22:51 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Wed Mar 11 19:43:16 2026 +0800"
      },
      "message": "[CELEBORN-2279] Update log level from `INFO` to `ERROR` for console output in spark-it tests\n\n### What changes were proposed in this pull request?\n\nUpdate log level from `INFO` to `ERROR` for console output in spark-it tests.\n\n### Why are the changes needed?\n\n`spark-it` outputs too many INFO level logs to stdout.\n\n\u003cimg width\u003d\"1017\" height\u003d\"143\" alt\u003d\"image\" src\u003d\"https://github.com/user-attachments/assets/41667a37-050b-4174-afe6-6e4afcda8fcc\" /\u003e\n\n\u003cimg width\u003d\"783\" height\u003d\"188\" alt\u003d\"image\" src\u003d\"https://github.com/user-attachments/assets/6bb4f3e3-223d-4f8c-bea6-4c8d451bf4b9\" /\u003e\n\n### Does this PR resolve a correctness bug?\n\nNo.\n\n### Does this PR introduce _any_ user-facing change?\n\nNo.\n\n### How was this patch tested?\n\nGHA.\n\nCloses #3623 from cxzl25/CELEBORN-2279.\n\nAuthored-by: sychen \u003csychen@ctrip.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "4b157c68a5f7e82f4a50b8a2c8ae8989aff843c1",
      "tree": "630c63b237000990298bee7089d64a1a7e789784",
      "parents": [
        "b78177f3ac7adceb1f0510d2111943702e726eba"
      ],
      "author": {
        "name": "Aravind Patnam",
        "email": "akpatnam25@gmail.com",
        "time": "Tue Mar 10 10:36:54 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Tue Mar 10 10:36:54 2026 +0800"
      },
      "message": "[CELEBORN-2278] Make HTTP auth bypass API paths configurable\n\n### What changes were proposed in this pull request?\n\nAllow http paths that should be bypassed from auth to be configured. This is particularly useful when one of the read endpoints is used for health checks, and should not require auth each time for a high frequency operation.\n\n### Why are the changes needed?\n\nSee above.\n\n### Does this PR resolve a correctness bug?\n\nNo.\n\n### Does this PR introduce _any_ user-facing change?\n\nNo.\n\n### How was this patch tested?\n\nTrivial change, already added in our cluster for certain endpoints.\n\nCloses #3622 from akpatnam25/CELEBORN-2278.\n\nAuthored-by: Aravind Patnam \u003cakpatnam25@gmail.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "b78177f3ac7adceb1f0510d2111943702e726eba",
      "tree": "37c12a2779ffe43aca621e06fcc2d74b01350d08",
      "parents": [
        "391ef4bfc42b4c121d6a029d60689af15ab16b5b"
      ],
      "author": {
        "name": "Shuai Lu",
        "email": "lushuainada@gmail.com",
        "time": "Mon Mar 09 10:34:10 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Mon Mar 09 10:34:10 2026 +0800"
      },
      "message": "[CELEBORN-2238] Fix RuntimeException during stream cleanup preventing peer failover\n\n### What changes were proposed in this pull request?\n\nFix a bug in `CelebornInputStream` where a `RuntimeException` thrown during best-effort stream cleanup prevents peer failover when a primary worker becomes unregistered.\n\nIn `createReaderWithRetry`, when reader creation fails on the primary, the code tries to close the existing stream by calling `clientFactory.createClient()` before switching to the peer. This cleanup was wrapped in `catch (InterruptedException | IOException ex)`. When SASL authentication is configured, `SaslClientBootstrap` wraps `IOException` in `RuntimeException`, so the cleanup call also throws `RuntimeException`. This uncaught exception escapes the retry loop entirely, bypassing `location \u003d location.getPeer()` and causing the executor to exhaust retries on the same failed primary worker.\n\nThe fix adds `RuntimeException` to the cleanup catch clause — `catch (InterruptedException | IOException | RuntimeException ex)` — so that any exception during best-effort cleanup is logged and swallowed, allowing the peer switch to proceed.\n\n### Why are the changes needed?\n\nWithout this fix, when a worker pod is rotated or becomes unregistered and SASL authentication is enabled, the replica retry mechanism silently fails. The executor retries multiple times on the same dead primary worker and eventually fails the task, even though a healthy replica exists.\n\n### Does this PR resolve a correctness bug?\n\nYes.\n\n### Does this PR introduce _any_ user-facing change?\n\nNo.\n\n### How was this patch tested?\n\nAdded `CelebornInputStreamPeerFailoverTest` with three unit tests:\n- `testPeerFailoverWithRuntimeExceptionDuringCleanup`: primary fails, cleanup throws `RuntimeException` (simulates SASL wrapping), replica succeeds — verifies the fix\n- `testPeerFailoverWithIOExceptionDuringCleanup`: same scenario with plain `IOException` during cleanup — verifies existing behavior is preserved\n- `testFailureWithoutPeer`: no replica configured, verifies retries are exhausted and `CelebornIOException` is thrown\n\nCloses #3617 from sl3635/CELEBORN-2238.\n\nAuthored-by: Shuai Lu \u003clushuainada@gmail.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "391ef4bfc42b4c121d6a029d60689af15ab16b5b",
      "tree": "da484e8735734efdb0ab4ba99e469141ff7d9c17",
      "parents": [
        "dca37496ce59bd67526548957d2f607af8eee6cc"
      ],
      "author": {
        "name": "ShlomiTubul",
        "email": "shlomi.tubul@placer.ai",
        "time": "Thu Mar 05 14:32:49 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Thu Mar 05 14:32:49 2026 +0800"
      },
      "message": "[CELEBORN-2273] Fix cache mutation in TagsManager.getTaggedWorkers()\n\nWhat changes were proposed in this pull request?\ngetTaggedWorkers() obtains a direct reference to the cached Set from getWorkersWithTag()and then calls retainAll() on it to intersect with other tags and available workers. Since retainAll() mutates the Set in-place, this permanently corrupts the cached entry. When multiple applications with different tag combinations share the same master, one app\u0027s intersection shrinks the cached Set, causing subsequent lookups by other apps to find fewer or zero workers. Once corrupted to an empty Set, all future slot requests fail with WORKER_EXCLUDED until the cache is refreshed.\n\nWhy are the changes needed?\nDoes this PR resolve a correctness bug?\nYes\n\nDoes this PR introduce any user-facing change?\nNo\n\nHow was this patch tested?\ncustom image in my dev env + local test\n\nCloses #3615 from shlomitubul/main.\n\nAuthored-by: ShlomiTubul \u003cshlomi.tubul@placer.ai\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "dca37496ce59bd67526548957d2f607af8eee6cc",
      "tree": "a05960521f264643b17529754b19b185f20b7d00",
      "parents": [
        "13ea40c3d086a483f9913628b80640591c223508"
      ],
      "author": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Tue Mar 03 11:24:45 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Tue Mar 03 11:24:45 2026 +0800"
      },
      "message": "[CELEBORN-2218] Bump lz4-java version from 1.8.0 to 1.10.4 to resolve CVE‐2025‐12183 and CVE-2025-66566\n\n### What changes were proposed in this pull request?\n\n- Bump lz4-java version from 1.8.0 to 1.10.4 to resolve CVE‐2025‐12183 and CVE-2025-66566.\n- `Lz4Decompressor` follows the [suggestion](https://github.com/apache/spark/pull/53290#issuecomment-3607045004) to move from `fastDecompressor` to `safeDecompressor` to mitigate the performance.\n\nBackport:\n\n- https://github.com/apache/spark/pull/53327\n- https://github.com/apache/spark/pull/53347\n- https://github.com/apache/spark/pull/53971\n- https://github.com/apache/spark/pull/53454\n- https://github.com/apache/spark/pull/54585\n\n### Why are the changes needed?\n\n- [CVE‐2025‐12183](https://sites.google.com/sonatype.com/vulnerabilities/cve-2025-12183): Various lz4-java compression and decompression implementations do not guard against out-of-bounds memory access. Untrusted input may lead to denial of service and information disclosure. Vulnerable Maven coordinates: org.lz4:lz4-java up to and including 1.8.0.\n\n- [CVE-2025-66566](https://github.com/advisories/GHSA-cmp6-m4wj-q63q): Insufficient clearing of the output buffer in Java-based decompressor implementations in lz4-java 1.10.0 and earlier allows remote attackers to read previous buffer contents via crafted compressed input. In applications where the output buffer is reused without being cleared, this may lead to disclosure of sensitive data. JNI-based implementations are not affected.\n\nTherefore, lz4-java version should upgrade to 1.10.4.\n\n### Does this PR resolve a correctness bug?\n\nNo.\n\n### Does this PR introduce _any_ user-facing change?\n\nNo.\n\n### How was this patch tested?\n\nCI.\n\nCloses #3555 from SteNicholas/CELEBORN-2218.\n\nLead-authored-by: SteNicholas \u003cprogramgeek@163.com\u003e\nCo-authored-by: Cheng Pan \u003cchengpan@apache.org\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "13ea40c3d086a483f9913628b80640591c223508",
      "tree": "e1ddbcf4de0ce6d76ec258a77b19989f323b3057",
      "parents": [
        "ffdb41674596970c9a3ad7a85a08f81b37ae622d"
      ],
      "author": {
        "name": "Cheng Pan",
        "email": "chengpan@apache.org",
        "time": "Mon Mar 02 20:01:28 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Mon Mar 02 20:01:28 2026 +0800"
      },
      "message": "[CELEBORN-2272] Add LZ4TPCDSDataBenchmark\n\n### What changes were proposed in this pull request?\n\nAdd LZ4TPCDSDataBenchmark, use TPC-DS data to measure compression/decompression perf.\n\n### Why are the changes needed?\n\nProvide benchmark reports to measure performance change when upgrading lz4-java.\n\n### Does this PR resolve a correctness bug?\n\nNo.\n\n### Does this PR introduce _any_ user-facing change?\n\nNo.\n\n### How was this patch tested?\n\nSee benchmark reports.\n\nCloses #3613 from pan3793/lz4-benchmark.\n\nAuthored-by: Cheng Pan \u003cchengpan@apache.org\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "ffdb41674596970c9a3ad7a85a08f81b37ae622d",
      "tree": "7c0f5fc1daddaa34ba73b853071435ff9fa75519",
      "parents": [
        "0a67e2b304f464bd6199982582f06324a00beca4"
      ],
      "author": {
        "name": "Enrico Olivelli",
        "email": "eolivelli@gmail.com",
        "time": "Mon Mar 02 18:07:39 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Mon Mar 02 18:07:39 2026 +0800"
      },
      "message": "[CELEBORN-2270] Fix problem with eviction to tiered storage during partition split\n\nNOTE: this PR is stacked on top of https://github.com/apache/celeborn/pull/3608\n\nPlease consider only 756d25e49ef5f0321b90002d319b72924b9f4196\n\n### What changes were proposed in this pull request?\n\nHandle the eviction to a different location type.\n\n### Why are the changes needed?\n\nBecause it may happen that a MEMORY file is to be evicted to another storage type (i.e. S3). This does not work.\n\nUsually, as described in tests in #3608 when you have tiered storage, the primary partition type is usually not MEMORY, but it may happen that during a partition split the client decides to use MEMORY.\n\nThis patch fixes the problem on the worker side.\nAn alternative fix would be to change the behavior of the client, and simulate what the master does when offering slots.\nSuch a change would be more involved and it won\u0027t make the server side resilient to this scenario.\n\n### Does this PR resolve a correctness bug?\n\nNo.\n\n### Does this PR introduce _any_ user-facing change?\n\nNo.\n\n### How was this patch tested?\n\n- New integration tests\n- Manual testing on real k8s cluster with S3\n\nCloses #3610 from eolivelli/CELEBORN-2270-fix-partition-split.\n\nAuthored-by: Enrico Olivelli \u003ceolivelli@gmail.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "0a67e2b304f464bd6199982582f06324a00beca4",
      "tree": "8511d5ed7e616b454e4c1795b7070fc0b271a06b",
      "parents": [
        "fddb81754b03326f15df0b84ea1568a0621b7b88"
      ],
      "author": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Mon Mar 02 18:04:40 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Mon Mar 02 18:04:40 2026 +0800"
      },
      "message": "[CELEBORN-2049] Bump Ratis version from 3.1.3 to 3.2.1\n\n### What changes were proposed in this pull request?\n\nBump Ratis version from 3.1.3 to 3.2.1 including adding options of `celeborn-ratis sh peer add -peers` in `celeborn_ratis_shell.md` to follow up https://github.com/apache/ratis/pull/1282.\n\n### Why are the changes needed?\n\nBump Ratis version from 3.1.3 to 3.2.1. Ratis has released v3.2.1, of which release note refers to [3.2.1](https://ratis.apache.org/post/3.2.1.html). The 3.2.1 version is a maintenance release with multiple improvements and bugfixes. The usage of `celeborn-ratis` is as follows:\n\n```\n$ celeborn-ratis sh\nUsage: ratis sh [generic options]\n\t [election [transfer] [stepDown] [pause] [resume]]\n\t [group [info] [list]]\n\t [local [raftMetaConf]]\n\t [peer [add] [remove] [setPriority]]\n\t [snapshot [create]]\n\n$ celeborn-ratis sh election transfer\nUsage: transfer -address \u003cHOSTNAME:PORT\u003e -peers \u003cPEER0_HOST:PEER0_PORT,PEER1_HOST:PEER1_PORT,PEER2_HOST:PEER2_PORT\u003e [-groupid \u003cRAFT_GROUP_ID\u003e] [-timeout \u003cTIMEOUT_IN_SECONDS\u003e]\n\n$ celeborn-ratis sh election stepDown\nUsage: stepDown -peers \u003cPEER0_HOST:PEER0_PORT,PEER1_HOST:PEER1_PORT,PEER2_HOST:PEER2_PORT\u003e [-groupid \u003cRAFT_GROUP_ID\u003e]\n\n$ celeborn-ratis sh election pause\nUsage: pause -address \u003cHOSTNAME:PORT\u003e -peers \u003cPEER0_HOST:PEER0_PORT,PEER1_HOST:PEER1_PORT,PEER2_HOST:PEER2_PORT\u003e [-groupid \u003cRAFT_GROUP_ID\u003e]\n\n$ celeborn-ratis sh election resume\nUsage: resume -address \u003cHOSTNAME:PORT\u003e -peers \u003cPEER0_HOST:PEER0_PORT,PEER1_HOST:PEER1_PORT,PEER2_HOST:PEER2_PORT\u003e [-groupid \u003cRAFT_GROUP_ID\u003e]\n\n$ celeborn-ratis sh group info\nUsage: info -peers \u003cPEER0_HOST:PEER0_PORT,PEER1_HOST:PEER1_PORT,PEER2_HOST:PEER2_PORT\u003e [-groupid \u003cRAFT_GROUP_ID\u003e]\n\n$ celeborn-ratis sh group list\nUsage: list -peers \u003cPEER0_HOST:PEER0_PORT,PEER1_HOST:PEER1_PORT,PEER2_HOST:PEER2_PORT\u003e [-groupid \u003cRAFT_GROUP_ID\u003e] \u003c[-serverAddress \u003cPEER0_HOST:PEER0_PORT\u003e]|[-peerId \u003cpeerId\u003e]\u003e\n\n$ celeborn-ratis sh peer add -peers\nUsage: add -peers \u003cPEER0_HOST:PEER0_PORT,PEER1_HOST:PEER1_PORT,PEER2_HOST:PEER2_PORT\u003e [-groupid \u003cRAFT_GROUP_ID\u003e] \u003c[-address \u003cPEER0_HOST:PEER0_PORT\u003e]|[-peerId \u003cpeerId\u003e]\u003e [-clientAddress \u003cCLIENT_ADDRESS1,CLIENT_ADDRESS2,...\u003e] [-adminAddress \u003cADMIN_ADDRESS1,ADMIN_ADDRESS2,...\u003e]\n\n$ celeborn-ratis sh peer remove -peers\nUsage: remove -peers \u003cPEER0_HOST:PEER0_PORT,PEER1_HOST:PEER1_PORT,PEER2_HOST:PEER2_PORT\u003e [-groupid \u003cRAFT_GROUP_ID\u003e] \u003c[-address \u003cPEER0_HOST:PEER0_PORT\u003e]|[-peerId \u003cpeerId\u003e]\u003e\n\n$ celeborn-ratis sh peer setPriority\nUsage: setPriority -peers \u003cPEER0_HOST:PEER0_PORT,PEER1_HOST:PEER1_PORT,PEER2_HOST:PEER2_PORT\u003e [-groupid \u003cRAFT_GROUP_ID\u003e] -addressPriority \u003cPEER_HOST:PEER_PORT|PRIORITY\u003e\n\n$ celeborn-ratis sh snapshot create\nUsage: create -peers \u003cPEER0_HOST:PEER0_PORT,PEER1_HOST:PEER1_PORT,PEER2_HOST:PEER2_PORT\u003e [-groupid \u003cRAFT_GROUP_ID\u003e] [-snapshotTimeout \u003ctimeoutInMs\u003e] [-peerId \u003craftPeerId\u003e]\n\n$ celeborn-ratis sh local raftMetaConf\nUsage: raftMetaConf -peers \u003c[P0_ID|]P0_HOST:P0_PORT,[P1_ID|]P1_HOST:P1_PORT,[P2_ID|]P2_HOST:P2_PORT\u003e -path \u003cPARENT_PATH_OF_RAFT_META_CONF\u003e\n```\n### Does this PR resolve a correctness bug?\n\nNo.\n\n### Does this PR introduce _any_ user-facing change?\n\nNo.\n\n### How was this patch tested?\n\nCI.\n\nCloses #3612 from SteNicholas/CELEBORN-2049.\n\nAuthored-by: SteNicholas \u003cprogramgeek@163.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "fddb81754b03326f15df0b84ea1568a0621b7b88",
      "tree": "0c6bd54b3d26fda2922716b97c122c51fb147501",
      "parents": [
        "deb7538f23c739292030748abd99001f4aede225"
      ],
      "author": {
        "name": "afterincomparableyum",
        "email": "224495379+afterincomparableyum@users.noreply.github.com",
        "time": "Sun Mar 01 09:54:10 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Sun Mar 01 09:54:10 2026 +0800"
      },
      "message": "[CELEBORN-2226][CIP-14] Support RetryFetchChunk functionality for Cel…\n\nImplement chunk-fetch retry logic in CelebornInputStream::getNextChunk(), matching the Java CelebornInputStream behavior. When a chunk fetch fails, the retry loop excludes the failed worker, switches to the peer replica (if available), and sleeps between retry rounds before creating a new reader.\n\nAdded getLocation() to PartitionReader interface and WorkerPartitionReader\n\nReplaced the stub getNextChunk() with full retry logic: excluded worker checks, peer switching, configurable retry count, sleep between retries\n\nUpdated moveToNextChunk() and moveToNextReader() to handle nullable returns from getNextChunk()\n\nAdded unit test for WorkerPartitionReader::getLocation()\n\nAdded unit tests for getNextChunk() retry logic\n\nCI and build passes\n\nCloses #3605 from afterincomparableyum/cpp-client/celeborn-2226.\n\nAuthored-by: afterincomparableyum \u003c224495379+afterincomparableyum@users.noreply.github.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "deb7538f23c739292030748abd99001f4aede225",
      "tree": "1417c6a61e30c9e371743bb11370b25e8e77a300",
      "parents": [
        "bc4dc12cae0fee63b5951658f2c89dc5725eec22"
      ],
      "author": {
        "name": "zhenghuan",
        "email": "zhenghuan@weidian.com",
        "time": "Sat Feb 28 13:16:58 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Sat Feb 28 13:16:58 2026 +0800"
      },
      "message": "[CELEBORN-2248] Implement lazy loading for columnar shuffle classes and skew shuffle method using static holder pattern\n\n### What changes were proposed in this pull request?\n\nThis PR converts the static initialization of columnar shuffle class constructors\nand skew shuffle method to lazy initialization using the initialization-on-demand\nholder idiom (static inner class pattern) in SparkUtils.java.\n\nSpecifically, the following changes were made:\n\n1. Introduced `ColumnarHashBasedShuffleWriterConstructorHolder` static inner class\n   to lazily initialize the constructor for ColumnarHashBasedShuffleWriter\n\n2. Introduced `ColumnarShuffleReaderConstructorHolder` static inner class to lazily\n   initialize the constructor for CelebornColumnarShuffleReader\n\n3. Introduced `CelebornSkewShuffleMethodHolder` static inner class to lazily\n   initialize the `isCelebornSkewedShuffle` method reference\n\n4. Modified `createColumnarHashBasedShuffleWriter()`, `createColumnarShuffleReader()`,\n   and `isCelebornSkewShuffleOrChildShuffle()` methods to use the holder pattern for\n   lazy initialization\n\n5. Added JavaDoc comments explaining the lazy loading mechanism\n\n### Why are the changes needed?\n\nThe current implementation statically initializes columnar shuffle class constructors\nand the skew shuffle method at SparkUtils class loading time, which means these\nclasses/methods are loaded regardless of whether they are actually used.\n\nThis lazy loading approach ensures that:\n- Columnar shuffle classes are only loaded when actually needed (when\n  `celeborn.columnarShuffle.enabled` is true and the create methods are called)\n- CelebornShuffleState class is only loaded when skew shuffle detection is needed\n- Reduces unnecessary class loading overhead for users not using these features\n- Improves startup performance and memory footprint\n- Aligns with the conditional usage pattern already present in SparkShuffleManager\n\nThe static holder pattern (initialization-on-demand holder idiom) provides several\nadvantages:\n- Thread-safe without explicit synchronization (guaranteed by JVM class loading mechanism)\n- No synchronization overhead at runtime (no volatile reads or lock acquisition)\n- Simpler and more concise code compared to double-checked locking\n- Recommended by Effective Java (Item 83) for lazy initialization\n\n### Does this PR resolve a correctness bug?\n\nNo, this is a performance optimization.\n\n### Does this PR introduce any user-facing change?\n\nNo. This change only affects when certain classes are loaded internally.\nThe functionality and API remain unchanged.\n\n### How was this patch tested?\n\n- Code review to verify correct implementation of the initialization-on-demand holder pattern\n- Verified that JVM class loading guarantees thread safety\n- The changes are backward compatible and don\u0027t alter functionality, only initialization timing\n\nCloses #3581 from ever4Kenny/CELEBORN-2248.\n\nAuthored-by: zhenghuan \u003czhenghuan@weidian.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "bc4dc12cae0fee63b5951658f2c89dc5725eec22",
      "tree": "057c9900527589055798fb46be9bdd37bd31b38f",
      "parents": [
        "e119902b6c538789679a277056ea58f91e19f455"
      ],
      "author": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Fri Feb 27 19:59:26 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Fri Feb 27 19:59:26 2026 +0800"
      },
      "message": "[CELEBORN-2239] Support Spark 4.1\n\n### What changes were proposed in this pull request?\n\nSupport Spark 4.1.\n\n### Why are the changes needed?\n\nSpark 4.1.1 has already released, which release notes refer to [Spark Release 4.1.1](https://spark.apache.org/releases/spark-release-4.1.1.html).\n\n### Does this PR resolve a correctness bug?\n\nNo.\n\n### Does this PR introduce _any_ user-facing change?\n\nNo.\n\n### How was this patch tested?\n\nCI.\n\nCloses #3571 from SteNicholas/CELEBORN-2239.\n\nAuthored-by: SteNicholas \u003cprogramgeek@163.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "e119902b6c538789679a277056ea58f91e19f455",
      "tree": "0d1c4b21bea2d3b44594f29f1e5d4a9ef0ff1dde",
      "parents": [
        "9d48e6cc5f2da1a7612972b06b61f448fac9edab"
      ],
      "author": {
        "name": "Enrico Olivelli",
        "email": "eolivelli@gmail.com",
        "time": "Fri Feb 27 15:19:22 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Fri Feb 27 15:19:22 2026 +0800"
      },
      "message": "[CELEBORN-2268] Improve test coverage for MEMORY and S3 storage\n\n### What changes were proposed in this pull request?\n\nThis commit adds only tests and some useful debug information about using MEMORY and S3 storage.\n\n### Why are the changes needed?\n\nBecause there is not enough code coverage on some configurations that may happen in production,  in particular about:\n- using MEMORY storage\n- using only S3 storage\n- using MEMORY with eviction to S3\n\nThere is an interesting case to test: when you configure MEMORY to S3 eviction and the dataset is small.\nIt is important to ensure that no file is created in S3\n\n### Does this PR resolve a correctness bug?\n\nNo.\n\n### Does this PR introduce _any_ user-facing change?\n\nNo.\n\n### How was this patch tested?\n\nIt adds new integration tests.\n\nCloses #3608 from eolivelli/fix-eviction-apache.\n\nAuthored-by: Enrico Olivelli \u003ceolivelli@gmail.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "9d48e6cc5f2da1a7612972b06b61f448fac9edab",
      "tree": "dbf6aa8556846a52dfa746ddb85d39c8635f9363",
      "parents": [
        "ad5a88161cdd6febfb499f00dfbedd8ecb9d9d2a"
      ],
      "author": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Fri Feb 27 11:39:59 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Fri Feb 27 11:39:59 2026 +0800"
      },
      "message": "[CELEBORN-2258] Bump Netty version from 4.1.118.Final to 4.2.10.Final\n\n### What changes were proposed in this pull request?\n\nBump Netty version from 4.1.118.Final to 4.2.10.Final, which follows the official community migration guide: [Netty-4.2-Migration-Guide](https://github.com/netty/netty/wiki/Netty-4.2-Migration-Guide).\n\n### Why are the changes needed?\n\nThe Netty 4.2.10.Final version has been released, which netty version is 4.1.118.Final at present.\n\nBackport:\n\n- https://github.com/apache/spark/pull/34881\n- https://github.com/apache/spark/pull/52552\n- https://github.com/apache/spark/pull/53499\n\n### Does this PR resolve a correctness bug?\n\nNo.\n\n### Does this PR introduce _any_ user-facing change?\n\nNo.\n\n### How was this patch tested?\n\nCI.\n\nCloses #3596 from SteNicholas/CELEBORN-2258.\n\nAuthored-by: SteNicholas \u003cprogramgeek@163.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "ad5a88161cdd6febfb499f00dfbedd8ecb9d9d2a",
      "tree": "322694754278624d7320eb6875b8d943df972c2b",
      "parents": [
        "2460efd95e59dfa58a53f334fe2d25c3073cc013"
      ],
      "author": {
        "name": "afterincomparableyum",
        "email": "224495379+afterincomparableyum@users.noreply.github.com",
        "time": "Thu Feb 26 11:24:37 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Thu Feb 26 11:24:37 2026 +0800"
      },
      "message": "[CELEBORN-2267][FOLLOWUP] Add Cpp-Write Java-Read integration tests for LZ4 and ZSTD\n\nThis is a follow up to https://github.com/apache/celeborn/pull/3575\n\n- Add compression codec argument to C++ DataSumWithWriterClient and set it in CelebornConf so the writer uses LZ4/ZSTD when enabled\n- Pass codec from runCppWriteJavaRead to the C++ writer command\n- Add CppWriteJavaReadTestWithLZ4 and CppWriteJavaReadTestWithZSTD (mirroring CppWriteJavaReadTestWithNONE)\n\n### How was this patch tested?\n\nI compiled and ran tests locally, all passed.\n\nCloses #3606 from afterincomparableyum/cpp-client/celeborn-2267.\n\nAuthored-by: afterincomparableyum \u003c224495379+afterincomparableyum@users.noreply.github.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "2460efd95e59dfa58a53f334fe2d25c3073cc013",
      "tree": "f2582dba611ab6212f4a1c78e4824bde03557c2a",
      "parents": [
        "4d97a8560a3bf5a839c14282a111eaf54bdac35f"
      ],
      "author": {
        "name": "afterincomparableyum",
        "email": "224495379+afterincomparableyum@users.noreply.github.com",
        "time": "Thu Feb 26 11:18:48 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Thu Feb 26 11:18:48 2026 +0800"
      },
      "message": "[CELEBORN-2222][CIP-14] Support Retrying when createReader failed for CelebornInputStream in CppClient\n\nThis PR implements retry support for createReader failures in the C++ client, matching the behavior of the Java implementation. The implementation includes:\n\n- Added configuration properties:\n  * clientFetchMaxRetriesForEachReplica (default: 3)\n  * dataIoRetryWait (default: 5s)\n  * clientPushReplicateEnabled (default: false)\n  * excludeWorkerOnFailure (default: false)\n  * excludedWorker.expireTimeout (default: 60s)\n  * optimizeSkewedPartitionRead (default: false)\n\n- Added peer location support methods to PartitionLocation:\n  * hasPeer() - Check if location has a peer replica\n  * getPeer() - Get the peer location\n  * hostAndFetchPort() - Get host:port string for logging\n\n- Implemented retry logic in createReaderWithRetry():\n  * Retries up to fetchChunkMaxRetry_ times (doubled if replication enabled)[which is why I added this parameter in this PR]\n  * Switches to peer location on failure when available\n  * Sleeps between retries when both replicas tried or no peer exists\n  * Resets retry counter when moving to new location or on success\n\n- Added unit tests for new functionality\n\n### How was this patch tested?\n\nUnit tests and compiling\n\nCloses #3583 from afterincomparableyum/cpp-client/celeborn-2222.\n\nAuthored-by: afterincomparableyum \u003c224495379+afterincomparableyum@users.noreply.github.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "4d97a8560a3bf5a839c14282a111eaf54bdac35f",
      "tree": "fd334fda3cfc982c7f5c49a481beb51ccc3590a9",
      "parents": [
        "feb3ed90c36d924dbfab8e2bec1972c4ef162486"
      ],
      "author": {
        "name": "Prateek Srivastava",
        "email": "me@prateek.io",
        "time": "Thu Feb 26 11:15:53 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Thu Feb 26 11:15:53 2026 +0800"
      },
      "message": "[CELEBORN-2271] StorageManager#saveCommittedFileInfosExecutor should call shutdown before awaitTermination\n\n### What changes were proposed in this pull request?\nCall saveCommittedFileInfosExecutor.shutdown() before awaitTermination() in saveAllCommittedFileInfosToDB() so the executor shuts down correctly during worker shutdown.\n\n### Why are the changes needed?\nawaitTermination() only waits for the executor to finish after a shutdown has been requested; without shutdown(), the executor keeps running and can schedule more work.\n\nFrom [ExecutorService documentation](https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/ExecutorService.html#awaitTermination-long-java.util.concurrent.TimeUnit-):\n\n\u003e Blocks until all tasks have completed execution \"after a shutdown request\", ...\n\n### Does this PR resolve a correctness bug?\n\nPossibly, as it could lead to race conditions writing to RocksDB during shutdown, which could cause data loss or correctness issues.\n\n### Does this PR introduce _any_ user-facing change?\nNo.\n\n### How was this patch tested?\nShould be exercised by  existing tests to ensure this doesn\u0027t introduce a regression.\n\nCloses #3607 from f2prateek/fix-shutdown.\n\nAuthored-by: Prateek Srivastava \u003cme@prateek.io\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "feb3ed90c36d924dbfab8e2bec1972c4ef162486",
      "tree": "b164a6587eeb65bd015739768fb16c8ff55900eb",
      "parents": [
        "ac6d1cf5d399618f934bd82983ef9632d278b340"
      ],
      "author": {
        "name": "Enrico Olivelli",
        "email": "eolivelli@gmail.com",
        "time": "Wed Feb 25 09:46:54 2026 +0800"
      },
      "committer": {
        "name": "Shuang",
        "email": "lvshuang.xjs@alibaba-inc.com",
        "time": "Wed Feb 25 09:46:54 2026 +0800"
      },
      "message": "[CELEBORN-2262] Prepare S3 directory only once and cache s3 client for MultiPartUploader\n\n### What changes were proposed in this pull request?\n\n- Create only one S3 client for all MultiPartUploaders\n- Create S3 worker directory only once and not for every slot\n\n### Why are the changes needed?\n- Because on S3 AWS creating connections is slow (due to credentials handshaking and TLS handshaking)\n- Because \"mkdirs\" in S3 AWS is very slow (and it needs several S3 calls)\n\nSample CPU flamegraph about the need of Connection pooling:\n\u003cimg width\u003d\"2248\" height\u003d\"1275\" alt\u003d\"image\" src\u003d\"https://github.com/user-attachments/assets/5fb46d8f-5a1e-41a0-a8ca-01c92a2a3eb0\" /\u003e\n\nSample CPU flamegraph about the need of pooling the client due to AssumeRoleWithWebIdentity\n\u003cimg width\u003d\"2248\" height\u003d\"1275\" alt\u003d\"image\" src\u003d\"https://github.com/user-attachments/assets/e9efbadd-ef68-40d3-8fb5-d8fe43f56752\" /\u003e\n\n### Does this PR resolve a correctness bug?\n\nNo\n\n### Does this PR introduce _any_ user-facing change?\n\nNo\n\n### How was this patch tested?\n\nManual testing.\n\nThere is one end-to-end integration test with S3 that exercise this code\n\nCloses #3604 from eolivelli/improve-s3-apache.\n\nAuthored-by: Enrico Olivelli \u003ceolivelli@gmail.com\u003e\nSigned-off-by: Shuang \u003clvshuang.xjs@alibaba-inc.com\u003e\n"
    },
    {
      "commit": "ac6d1cf5d399618f934bd82983ef9632d278b340",
      "tree": "17379cc92c3ed2163f40098d9d65c16fe1fdf7a7",
      "parents": [
        "e4836b75527ace35837ee0c87b6585b1d8701b66"
      ],
      "author": {
        "name": "jay.narale",
        "email": "jay.narale@uber.com",
        "time": "Tue Feb 24 13:55:59 2026 +0800"
      },
      "committer": {
        "name": "Shuang",
        "email": "lvshuang.xjs@alibaba-inc.com",
        "time": "Tue Feb 24 13:55:59 2026 +0800"
      },
      "message": "[CELEBORN-2269] Update Cpp TransportClient to resolve hostnames via DNS\n\n### What changes were proposed in this pull request?\n\nThis PR changes the folly::SocketAddress constructor calls in TransportClientFactory::createClient to pass true for the allowNameLookup parameter. This affects two call sites: the address used as the connection pool key, and the address used when connecting the bootstrap to the server.\n\nFolly code - https://github.com/facebook/folly/blob/main/folly/SocketAddress.h#L80\n\n### Why are the changes needed?\n\nWithout allowNameLookup \u003d true, folly::SocketAddress only accepts numeric IP addresses. When a Celeborn worker is addressed by hostnamehe constructor throws an \"invalid address\" exception, causing all connections to that worker to fail.\n\nSetting the parameter to true makes folly::SocketAddress use getaddrinfo, which transparently handles both hostnames (via DNS resolution) and numeric IPs. This is safe and backward-compatible since getaddrinfo recognizes numeric addresses without issuing a DNS query.\n\n### Does this PR resolve a correctness bug?\n\nNo.\n\n### Does this PR introduce _any_ user-facing change?\n\nNo.\n\n### How was this patch tested?\n\nCI.\n\nCloses #3609 from jaystarshot/u-c.\n\nAuthored-by: jay.narale \u003cjay.narale@uber.com\u003e\nSigned-off-by: Shuang \u003clvshuang.xjs@alibaba-inc.com\u003e\n"
    },
    {
      "commit": "e4836b75527ace35837ee0c87b6585b1d8701b66",
      "tree": "7ad51baebe30e80abd02e114cb78cfbb2fff60d7",
      "parents": [
        "13cd4a9232b7f771e2314943d8614598f7d62283"
      ],
      "author": {
        "name": "jay.narale",
        "email": "jay.narale@uber.com",
        "time": "Thu Feb 19 16:23:11 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Thu Feb 19 16:23:11 2026 +0800"
      },
      "message": "[CELEBORN-2266] Modernize Protobuf CMake usage and add install rules\n\n### What changes were proposed in this pull request?\n\n- Switched Protobuf CMake integration from the legacy FindProtobuf module to modern CONFIG mode with imported targets (protobuf::protoc, protobuf::libprotobuf).\n\n- Added install() rules for public headers, generated proto headers, and static libraries so the C++ client can be consumed as an installed package.\n\n- Added missing #include \u003cset\u003e in CelebornUtils.h.\n\n### Why are the changes needed?\n\nThe legacy FindProtobuf module-variable style (${PROTOBUF_LIBRARY}, bare protoc command) is fragile and does not work reliably with package managers like vcpkg or Conan that provide Protobuf via CMake config files. Switching to CONFIG mode and imported targets is the modern CMake best practice and ensures the correct protoc binary and library are used regardless of the build environment.\n\nThe install rules are needed so that downstream projects can consume the Celeborn C++ client from a clean install prefix rather than pointing directly at the source and build trees.\n\nThe missing \u003cset\u003e include was causing compilation failures in certain toolchain configurations where the header was not transitively included.\n\n### Does this PR resolve a correctness bug?\n\nNo.\n\n### Does this PR introduce _any_ user-facing change?\n\nNo.\n\n### How was this patch tested?\n\nCI.\n\nCloses #3602 from jaystarshot/u-c.\n\nAuthored-by: jay.narale \u003cjay.narale@uber.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "13cd4a9232b7f771e2314943d8614598f7d62283",
      "tree": "2421822ef06a5e9a037d070468b61c7451931402",
      "parents": [
        "eb7a720ac9c60c14c9f2b7091888911429ab1c7b"
      ],
      "author": {
        "name": "Enrico Olivelli",
        "email": "eolivelli@gmail.com",
        "time": "Tue Feb 17 16:37:08 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Tue Feb 17 16:37:08 2026 +0800"
      },
      "message": "[CELEBORN-2265] Do not waste resources on hotpath for debug logging in HashBasedShuffleWriter and SortBasedShuffleWriter\n\n### What changes were proposed in this pull request?\nCompute logging message only when needed in order to save CPU cycles on the hotpath.\n\n### Why are the changes needed?\n\n\u003cimg width\u003d\"2561\" height\u003d\"687\" alt\u003d\"image\" src\u003d\"https://github.com/user-attachments/assets/53733848-ebcd-4079-a77a-9aa38ed1e90a\" /\u003e\n\n### Does this PR resolve a correctness bug?\n\nNo\n\n### Does this PR introduce _any_ user-facing change?\n\nNo\n\n### How was this patch tested?\n\nManual testing\n\nCloses #3603 from eolivelli/flushbuffer-logger.\n\nAuthored-by: Enrico Olivelli \u003ceolivelli@gmail.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "eb7a720ac9c60c14c9f2b7091888911429ab1c7b",
      "tree": "a511e48a3fbf41ab0981529e02c45c00977dce55",
      "parents": [
        "8e6f4d5f95f58238913bf6f5bc769e5508d64efe"
      ],
      "author": {
        "name": "Dzeri96",
        "email": "13813363+Dzeri96@users.noreply.github.com",
        "time": "Tue Feb 17 16:32:46 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Tue Feb 17 16:32:46 2026 +0800"
      },
      "message": "[CELEBORN-2259] The S3MultipartUploadHandler uses fs.s3a.aws.credentials.provider\n\n### What changes were proposed in this pull request?\n\nThe S3 Client in `S3MultipartUploadHandler` now uses the dynamic config `fs.s3a.aws.credentials.provider` in order to set its provider chain up.\n\n### Why are the changes needed?\n\nBefore this, it was only possible to use the hard-coded provider configuration.\n\n### Does this PR resolve a correctness bug?\n\nSort of.\n\n### Does this PR introduce _any_ user-facing change?\n\nYes, in the sense that `celeborn.hadoop.fs.s3a.aws.credentials.provider` will now work correctly in the MultiPartHandler.\n\n### How was this patch tested?\n\nUnit tests and a manual test.\n**Note**: I don\u0027t like having to change the class in order to make it testable, but I\u0027m planning to get rid of this whole logic in another PR, where we will use the same hadoop-created S3 client everywhere.\n\nCloses #3599 from Dzeri96/CELEBORN-2259-cherrypicked.\n\nLead-authored-by: Dzeri96 \u003c13813363+Dzeri96@users.noreply.github.com\u003e\nCo-authored-by: Filip Darmanovic \u003cdzeri96@proton.me\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "8e6f4d5f95f58238913bf6f5bc769e5508d64efe",
      "tree": "20aaf3ddcc8b21d89da75177577dbb1b232f047f",
      "parents": [
        "b1cbdabdf70230b7d2fff3ee0f7c44fa5a829f92"
      ],
      "author": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Tue Feb 17 16:03:13 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Tue Feb 17 16:03:13 2026 +0800"
      },
      "message": "[CELEBORN-2063] Parallelize the create partition writer in handleReserveSlots to speed up the reserveSlots RPC process time\n\n### What changes were proposed in this pull request?\n\nParallelize the create partition writer in `handleReserveSlots` to speed up the reserveSlots RPC process time。\n\n### Why are the changes needed?\n\nThe creation of partition writer in `handleReserveSlots` could use parallelize way to speed up the reserveSlots RPC process time.\n\n### Does this PR introduce _any_ user-facing change?\n\nIntroduce `celeborn.worker.writer.create.parallel.enabled`, `celeborn.worker.writer.create.parallel.threads` and `eleborn.worker.writer.create.parallel.timeout` to config parallelize the creation of file writer.\n\n### How was this patch tested?\n\nCI.\n\nCloses #3387 from SteNicholas/CELEBORN-2063.\n\nAuthored-by: SteNicholas \u003cprogramgeek@163.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "b1cbdabdf70230b7d2fff3ee0f7c44fa5a829f92",
      "tree": "56cc0669aaa7b5dd65765128d17b9507c723bff1",
      "parents": [
        "4ed9cbd2a6f2019b79f248e13cc99b14dc6e23e0"
      ],
      "author": {
        "name": "afterincomparableyum",
        "email": "afterincomparableyum",
        "time": "Mon Feb 16 14:59:18 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Mon Feb 16 14:59:18 2026 +0800"
      },
      "message": "[CELEBORN-2221][CIP-14] Support writing with compression in C++ client\n\nIntegrate existing compression infrastructure (LZ4 and ZSTD) into the C++ client write path. This enables compression during pushData operations, matching the functionality available in the Java client.\n\nChanges:\n- Add compression support to ShuffleClientImpl:\n  * Add shuffleCompressionEnabled_ flag and compressor_ member\n  * Initialize compressor from CelebornConf in constructor\n  * Compress data in pushData() when compression is enabled\n  * Use compressed size for batchBytesSize tracking\n\n- Configuration integration:\n  * Read compression codec from celeborn.client.shuffle.compression.codec\n  * Read ZSTD compression level from celeborn.client.shuffle.compression.zstd.level\n  * Default to NONE (compression disabled)\n\n- Retry/revive support:\n  * Retry path correctly uses pre-compressed body buffer\n  * No re-compression needed during retries\n\n- Testing:\n  * Add CompressorFactoryTest for factory pattern and config integration\n  * Add compression config tests to CelebornConfTest\n  * Test offset compression support for both LZ4 and ZSTD\n\n### How was this patch tested?\n\nUnit Tests, as well as compiling code\n\nCloses #3575 from afterincomparableyum/cpp-client/celeborn-2221.\n\nAuthored-by: afterincomparableyum \u003cafterincomparableyum\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "4ed9cbd2a6f2019b79f248e13cc99b14dc6e23e0",
      "tree": "93f16d1d9ba6007f6603c81e8f644ea2059acfc2",
      "parents": [
        "81d89f3ecac24fe22b29a2965a740c8bf93ce186"
      ],
      "author": {
        "name": "Enrico Olivelli",
        "email": "eolivelli@gmail.com",
        "time": "Sun Feb 15 23:16:29 2026 +0800"
      },
      "committer": {
        "name": "Shuang",
        "email": "lvshuang.xjs@alibaba-inc.com",
        "time": "Sun Feb 15 23:16:29 2026 +0800"
      },
      "message": "[CELEBORN-2263] Fix IndexOutOfBoundsException while reading from S3\n\n### What changes were proposed in this pull request?\n\nProperly pass the size of the array to the InputStream that feeds the flush.\n\n### Why are the changes needed?\n\nBecause without this change if the array is bigger than the buffer, then the inputstream returns garbage, resulting in corrupted data on S3.\n\n### Does this PR resolve a correctness bug?\n\nYes.\n\n### Does this PR introduce _any_ user-facing change?\n\nNo.\n\n### How was this patch tested?\n\nNew unit test + Manual testing.\n\nCloses #3600 from eolivelli/CELEBORN-2263-apache.\n\nAuthored-by: Enrico Olivelli \u003ceolivelli@gmail.com\u003e\nSigned-off-by: Shuang \u003clvshuang.xjs@alibaba-inc.com\u003e\n"
    },
    {
      "commit": "81d89f3ecac24fe22b29a2965a740c8bf93ce186",
      "tree": "552647c3ab78d890ae3818e990e3d9c5cff39649",
      "parents": [
        "b659439edc10f6b4f1768c4810222cd8e909151a"
      ],
      "author": {
        "name": "Enrico Olivelli",
        "email": "eolivelli@gmail.com",
        "time": "Thu Feb 05 14:12:04 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Thu Feb 05 14:12:04 2026 +0800"
      },
      "message": "[CELEBORN-2256] Helm chart: add support for setting annotations on the service account (to support eks.amazonaws.com/role-arn)\n\n### What changes were proposed in this pull request?\nAdding support for setting \"annotations\" on the Celeborn Service Account.\n\nPatch originally contributed by Filip Darmanovic Dzeri96\n\n### Why are the changes needed?\nThis is needed to support AWS IAM roles in k8s EKS\n\n### Does this PR resolve a correctness bug?\n\nNo\n\n### Does this PR introduce _any_ user-facing change?\n\nYes, you can now configure the annotations.\n\n### How was this patch tested?\n\nManual testing + unit tests\n\nCloses #3595 from eolivelli/CELEBORN-2256.\n\nAuthored-by: Enrico Olivelli \u003ceolivelli@gmail.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "b659439edc10f6b4f1768c4810222cd8e909151a",
      "tree": "8d876d32f37c5ff7ba5d14ec61dea685356b9ae0",
      "parents": [
        "2097ad0a347b982e68ac07779ee1ab11815b524b"
      ],
      "author": {
        "name": "Enrico Olivelli",
        "email": "enrico@beast.io",
        "time": "Thu Feb 05 13:39:12 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Thu Feb 05 13:44:59 2026 +0800"
      },
      "message": "[CELEBORN-2254] Fix support for S3 and add a simple integration test\n\n### What changes were proposed in this pull request?\n\n* Fix creating files to S3 (and other DFS)\n* Add integration test for Spark and S3 (using Minio)\n* in CI some job will run with the AWS profile because this way we can activate the new integration test (that needs the S3 client dependencies)\n\n### Why are the changes needed?\n\nSee https://issues.apache.org/jira/browse/CELEBORN-2254.\n\n### Does this PR resolve a correctness bug?\n\nNo\n\n### Does this PR introduce _any_ user-facing change?\n\nNo\n\n### How was this patch tested?\n\n* I have added an integration test\n* I have this patch on out internal fork, to make Celeborn run on k8s with S3\n\nCloses #3592 from eolivelli/CELEBORN-2254-apache.\n\nAuthored-by: Enrico Olivelli \u003cenrico@beast.io\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "2097ad0a347b982e68ac07779ee1ab11815b524b",
      "tree": "d5defbf92becdc6041c54191129e7667db2f39e6",
      "parents": [
        "475293663c7519c4f4e4fee9a37d4fd1900003a1"
      ],
      "author": {
        "name": "xxx",
        "email": "953396112@qq.com",
        "time": "Wed Feb 04 19:38:06 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Wed Feb 04 19:38:06 2026 +0800"
      },
      "message": "[CELEBORN-2205] Introduce metrics to fetch chunk for memory and local disk\n\n### What changes were proposed in this pull request?\n\nIntroduce metrics to fetch chunk time for memory and local disk.\n\n### Why are the changes needed?\n\nIntroduce metrics to fetch chunk time for memory and local disk.\n\n### Does this PR resolve a correctness bug?\n\nNo.\n\n### Does this PR introduce _any_ user-facing change?\n\nNo.\n\n### How was this patch tested?\n\n[Grafana](https://xy2953396112.grafana.net/public-dashboards/979279524ef74b6b92d0c08c39aa7c9e)\n\nCloses #3546 from xy2953396112/CELEBORN-2205.\n\nAuthored-by: xxx \u003c953396112@qq.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "475293663c7519c4f4e4fee9a37d4fd1900003a1",
      "tree": "b1f0e96e70fbc01723b546d8b215d4455459e64d",
      "parents": [
        "46a49a8285b3a77f51b1a3223760cab7d8182667"
      ],
      "author": {
        "name": "Sanskar Modi",
        "email": "sanskarmodi97@gmail.com",
        "time": "Wed Feb 04 19:34:26 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Wed Feb 04 19:34:26 2026 +0800"
      },
      "message": "[CELEBORN-1577][FOLLOWUP] Fix master resource consumption metrics\n\n### What changes were proposed in this pull request?\n\nFix master resource consumption metrics. https://github.com/apache/celeborn/pull/2819/ introduced a bug in master resource consumption metrics where we passed a local variable as GaugeSupplier leading to static values for user resource consumption.\n\n### Why are the changes needed?\n\nCurrently the code is buggy and gives a static value\n\n### Does this PR resolve a correctness bug?\n\nNo\n\n### Does this PR introduce _any_ user-facing change?\n\nNo\n\n### How was this patch tested?\nGA in our cluster.\n\nCloses #3591 from s0nskar/CELEBORN-1577.\n\nAuthored-by: Sanskar Modi \u003csanskarmodi97@gmail.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "46a49a8285b3a77f51b1a3223760cab7d8182667",
      "tree": "9cb17a828013b27b24808c92a1b95721a6bd40ee",
      "parents": [
        "f04ddddc2d4a7f5311e4ef802cb49d84cb0eea95"
      ],
      "author": {
        "name": "yew1eb",
        "email": "yew1eb@gmail.com",
        "time": "Tue Jan 27 21:15:44 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Tue Jan 27 21:15:44 2026 +0800"
      },
      "message": "[CELEBORN-2250] Fix lock contention in ReducePartitionCommitHandler.finishMapperAttempt via fine-grained locks\n\n### What changes were proposed in this pull request?\nAdd `shuffleIdLocks` (fine-grained locks per shuffleId), replace global `shuffleMapperAttempts` lock in `initMapperAttempts` and `finishMapperAttempt`.\n\n### Why are the changes needed?\nHigh concurrency causes lock contention on `shuffleMapperAttempts` in `finishMapperAttempt`, leading to abnormally long shuffle write time for small queries in Kyuubi Shared Mode. Fine-grained locks eliminate cross-shuffle blocking and improve concurrency.\n\n### Does this PR resolve a correctness bug?\nNo.\n### Does this PR introduce _any_ user-facing change?\nNo.\n\n### How was this patch tested?\nCI.\n\nCloses #3586 from yew1eb/CELEBORN-2250.\n\nAuthored-by: yew1eb \u003cyew1eb@gmail.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "f04ddddc2d4a7f5311e4ef802cb49d84cb0eea95",
      "tree": "b3a544f48fbb574816357662bd5f255a9ddc98c7",
      "parents": [
        "e81cea069644fd9f35f953666a174e1b6a9763a0"
      ],
      "author": {
        "name": "luogen.lg",
        "email": "luogen.lg@alibaba-inc.com",
        "time": "Thu Jan 22 10:35:44 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Thu Jan 22 10:35:44 2026 +0800"
      },
      "message": "[CELEBORN-2251] Introducing a shim layer and a common-tiered submodule for Flink clients\n\n### What changes were proposed in this pull request?\n\n1. Introduce a client-flink-common-tiered submodule to host the shared tiered shuffle logic.\n2. Introduce a shim layer to further unify the implementations and make version-specific changes more explicit.\n3. Unify the tests as well, so that versioned clients can simply run the common test suite with their own shims.\n\n### Why are the changes needed?\n\nThough Celeborn now has a client-flink-common submodule, clients still have to copy a lot of code from version to version, with small but necessary changes buried in the duplicates. The tests don’t share a common implementation either. Even worse, with Flink introducing tiered shuffle from 1.20 onward, all tiered shuffle implementations must be placed inside the versioned clients to ensure that client-flink-common can be compiled against all Flink versions. This makes it much harder to evolve the tiered shuffle implementations.\nFor the maintenance of Flink clients in the future, we need better organization of the submodules, consolidating the implementations and tests.\n\n### Does this PR resolve a correctness bug?\n\nNo.\n\n### Does this PR introduce _any_ user-facing change?\n\nNo.\n\n### How was this patch tested?\n\nThe patch consolidates all current Flink client tests, and can be tested with them.\n\nCloses #3589 from pltbkd/flink-shim-0.7.\n\nLead-authored-by: luogen.lg \u003cluogen.lg@alibaba-inc.com\u003e\nCo-authored-by: SteNicholas \u003cprogramgeek@163.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "e81cea069644fd9f35f953666a174e1b6a9763a0",
      "tree": "37091e1632e794c9805eb60e8f4c8506926444fa",
      "parents": [
        "908346128643d218c88537695037d623d76659a0"
      ],
      "author": {
        "name": "sychen",
        "email": "sychen@ctrip.com",
        "time": "Wed Jan 21 16:31:44 2026 +0800"
      },
      "committer": {
        "name": "Shaoyun Chen",
        "email": "csy@apache.org",
        "time": "Wed Jan 21 16:31:44 2026 +0800"
      },
      "message": "[CELEBORN-2245] Bump maven 3.9.12\n\n### What changes were proposed in this pull request?\n\n### Why are the changes needed?\n\nAvoid downloading maven every time.\n```\n Version: 20260105.207.1\n  Included Software: https://github.com/actions/runner-images/blob/ubuntu22/20260105.207/images/ubuntu/Ubuntu2204-Readme.md\n\nMaven 3.9.12\n```\n\n### Does this PR resolve a correctness bug?\n\n### Does this PR introduce _any_ user-facing change?\n\n### How was this patch tested?\nCurrent\n```\n  env:\n    JAVA_HOME: /opt/hostedtoolcache/Java_Zulu_jdk/8.0.472-8/x64\n    JAVA_HOME_8_X64: /opt/hostedtoolcache/Java_Zulu_jdk/8.0.472-8/x64\nexec: curl --progress-bar -L https://www.apache.org/dyn/closer.lua/maven/maven-3/3.9.9/binaries/apache-maven-3.9.9-bin.tar.gz?action\u003ddownload\n#\u003d#\u003d#\n##O#-#\n```\n\nPR\n```\n  env:\n    JAVA_HOME: /opt/hostedtoolcache/Java_Zulu_jdk/8.0.472-8/x64\n    JAVA_HOME_8_X64: /opt/hostedtoolcache/Java_Zulu_jdk/8.0.472-8/x64\nUsing `mvn` from path: /usr/bin/mvn\n```\n\nCloses #3577 from cxzl25/CELEBORN-2245.\n\nAuthored-by: sychen \u003csychen@ctrip.com\u003e\nSigned-off-by: Shaoyun Chen \u003ccsy@apache.org\u003e\n"
    },
    {
      "commit": "908346128643d218c88537695037d623d76659a0",
      "tree": "3683d23e1973cb346c723da91d1df903f5c0e89b",
      "parents": [
        "4749382f929a45c3491c3d4beda0aca253132fcc"
      ],
      "author": {
        "name": "yew1eb",
        "email": "yew1eb@gmail.com",
        "time": "Tue Jan 20 10:40:08 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Tue Jan 20 10:43:33 2026 +0800"
      },
      "message": "[CELEBORN-2249] Bump Spark from 3.5.7 to 3.5.8\n\n### What changes were proposed in this pull request?\n\nBump Spark from 3.5.7 to 3.5.8.\n\n### Why are the changes needed?\n\nSpark 3.5.8 has been announced to release: [Spark 3.5.8 released](https://spark.apache.org/news/spark-3-5-8-released.html). The profile spark-3.5 could bump Spark from 3.5.7 to 3.5.8.\n\n### Does this PR resolve a correctness bug?\n\nNo.\n\n### Does this PR introduce _any_ user-facing change?\n\nNo.\n\n### How was this patch tested?\n\nCI.\n\nCloses #3585 from yew1eb/CELEBORN_2249.\n\nAuthored-by: yew1eb \u003cyew1eb@gmail.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "4749382f929a45c3491c3d4beda0aca253132fcc",
      "tree": "9c09cb00b95943c4ab0d0ac382a074747574ab2d",
      "parents": [
        "7e4bea0e7e09ec57edb030745c13a41f4f96868a"
      ],
      "author": {
        "name": "xxx",
        "email": "953396112@qq.com",
        "time": "Thu Jan 15 17:03:29 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Thu Jan 15 17:03:29 2026 +0800"
      },
      "message": "[CELEBORN-2211] Avoid allocating additional buffers When HdfsFlushTask writes data\n\n### What changes were proposed in this pull request?\n\nInitialize the `FSDataOutputStream` in the `DfsTierWriter`, use it for data writing when the `HdfsFlushTask` performs flushing, and close the `FSDataOutputStream` when executing `closeStreams`.\n\n### Why are the changes needed?\n\nAvoid allocating additional buffers When HdfsFlushTask writes data.\n\n### Does this PR resolve a correctness bug?\n\nNO\n\n### Does this PR introduce _any_ user-facing change?\n\nNO\n\n### How was this patch tested?\n\nCI\n\nCloses #3548 from xy2953396112/CELEBORN-2211.\n\nAuthored-by: xxx \u003c953396112@qq.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "7e4bea0e7e09ec57edb030745c13a41f4f96868a",
      "tree": "22512d76df9be99ffd28aeefb2b906c2258af6b8",
      "parents": [
        "42ff69f9fdc9610daef4baa8919016bed5c36334"
      ],
      "author": {
        "name": "jiucheng",
        "email": "zdd467967@antgroup.com",
        "time": "Tue Jan 13 11:41:00 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Tue Jan 13 11:41:00 2026 +0800"
      },
      "message": "[CELEBORN-2243][FOLLOWUP][SPARK-2] During the close phase of hashWriter, pushData and mergeData are sent in parallel\n\n### What changes were proposed in this pull request?\n\nsame as https://github.com/apache/celeborn/pull/3574\nalso do on spark-2\n\n### Why are the changes needed?\n\n### Does this PR resolve a correctness bug?\n\n### Does this PR introduce _any_ user-facing change?\n\n### How was this patch tested?\n\nCloses #3584 from mcdull-zhang/CELEBORN-2243.\n\nAuthored-by: jiucheng \u003czdd467967@antgroup.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "42ff69f9fdc9610daef4baa8919016bed5c36334",
      "tree": "1e5ee2ad348070bebf225d17b9ca9a7f78bed4ed",
      "parents": [
        "9fca94bac6a7d35f7d83735e6a754a9f3f84d05a"
      ],
      "author": {
        "name": "afterincomparableyum",
        "email": "afterincomparableyum",
        "time": "Mon Jan 12 18:00:55 2026 +0800"
      },
      "committer": {
        "name": "Shuang",
        "email": "lvshuang.xjs@alibaba-inc.com",
        "time": "Mon Jan 12 18:00:55 2026 +0800"
      },
      "message": "[CELEBORN-2229][CIP-14] Add support for celeborn.client.push.maxBytesSizeInFlight in CppClient\n\n### What changes were proposed in this pull request?\n\nAdd support for celeborn.client.push.maxBytesSizeInFlight in CppClient, similar to InFlightRequestTracker.java\n\n### Does this PR resolve a correctness bug?\n\nNo\n\n### How was this patch tested?\n\nCompile locally, CI/CD will run unit tests\n\nCloses #3568 from afterincomparableyum/cpp-client/celeborn-2229.\n\nAuthored-by: afterincomparableyum \u003cafterincomparableyum\u003e\nSigned-off-by: Shuang \u003clvshuang.xjs@alibaba-inc.com\u003e\n"
    },
    {
      "commit": "9fca94bac6a7d35f7d83735e6a754a9f3f84d05a",
      "tree": "e1d23629926ca625bcac7ca4ab4d665a7d4f16ee",
      "parents": [
        "d30c02e3690a5ecfe3cbb32f2d21ba101d37c679"
      ],
      "author": {
        "name": "jiucheng",
        "email": "zdd467967@antgroup.com",
        "time": "Mon Jan 12 15:51:16 2026 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Mon Jan 12 15:51:16 2026 +0800"
      },
      "message": "[CELEBORN-2243] During the close phase of hashWriter, pushData and mergeData are sent in parallel\n\n### What changes were proposed in this pull request?\nOptimize the close process of HashBasedShuffleWriter：\n\nBeforePR：First, synchronously wait for pushData in the dataPusher queue to complete, then send mergeData and synchronously wait.\nAfterPR：During the process of sending pushData by dataPusher, mergeData is also constructed and sent simultaneously.\n\nOptimization results: The table below shows the shuffle write time for TPCH 1TB.\n| | SortBasedShuffleWriter | HashBasedShuffleWriter |\n|--------|--------|--------|\n| Before | 615.44W ms | 502.31W ms |\n| After |  442.47W ms | 434.77W ms |\n\nNote: This is the result of multiple optimizations combined;\nother optimizations will be submitted as pull requests in the future.\n\n### Why are the changes needed?\nImprove performance by increasing the parallelism of pushing/merging data.\n\n### Does this PR resolve a correctness bug?\n\nNO\n\n### Does this PR introduce _any_ user-facing change?\n\nNO\n\n### How was this patch tested?\n\nExists Unit Test.\n\nCloses #3574 from mcdull-zhang/hash_close_opt.\n\nAuthored-by: jiucheng \u003czdd467967@antgroup.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "d30c02e3690a5ecfe3cbb32f2d21ba101d37c679",
      "tree": "61ba1d388914fc5b65115e576cf9510a214e7bb8",
      "parents": [
        "2dd1b7aac4d488c0496da1fc77cab9c79bccd69b"
      ],
      "author": {
        "name": "HolyLow",
        "email": "jiaming.xie7@gmail.com",
        "time": "Mon Jan 05 22:24:22 2026 +0800"
      },
      "committer": {
        "name": "Shuang",
        "email": "lvshuang.xjs@alibaba-inc.com",
        "time": "Mon Jan 05 22:24:22 2026 +0800"
      },
      "message": "[CELEBORN-2235][CIP-14] Adapt Java end\u0027s serialization to CppWriterClient\n\n### What changes were proposed in this pull request?\nThis PR adapts Java end\u0027s serialization to CppWriterClient, including RegisterShuffle/Response, Revive/Response, MapperEnd/Response. Joint test for cpp-write java-read procedure is included as well.\n\n### Why are the changes needed?\nSupport writing to Celeborn server with CppWriterClient.\n\n### Does this PR resolve a correctness bug?\nNo.\n\n### Does this PR introduce _any_ user-facing change?\nNo.\n\n### How was this patch tested?\nCompilation and integration tests.\n\nCloses #3561 from HolyLow/issue/celeborn-2235-adapt-java-to-cpp-writer-serialization.\n\nAuthored-by: HolyLow \u003cjiaming.xie7@gmail.com\u003e\nSigned-off-by: Shuang \u003clvshuang.xjs@alibaba-inc.com\u003e\n"
    },
    {
      "commit": "2dd1b7aac4d488c0496da1fc77cab9c79bccd69b",
      "tree": "a5e947af31b1272cd9a3ec8d86db2c5ad4637136",
      "parents": [
        "783a7cc4fcedd57d62f6c876057700319163c7ee"
      ],
      "author": {
        "name": "xxx",
        "email": "953396112@qq.com",
        "time": "Fri Jan 02 20:04:50 2026 +0800"
      },
      "committer": {
        "name": "Shuang",
        "email": "lvshuang.xjs@alibaba-inc.com",
        "time": "Fri Jan 02 20:04:50 2026 +0800"
      },
      "message": "[CELEBORN-2210] When a flushBuffer consolidation OOM exception occurs…\n\n…, support setting the Buffer for fileInfo.\n\n### What changes were proposed in this pull request?\n\nWhen a flushBuffer consolidation OOM exception occurs, support setting the Buffer for fileInfo.\n\n### Why are the changes needed?\n\nWhen a flushBuffer consolidation OOM exception occurs, the current logic does not allow setting the Buffer for fileInfo.\n\n### Does this PR resolve a correctness bug?\n\nNO\n\n### Does this PR introduce _any_ user-facing change?\n\nNO\n\n### How was this patch tested?\n\nCI\n\nCloses #3547 from xy2953396112/CELEBORN-2210.\n\nAuthored-by: xxx \u003c953396112@qq.com\u003e\nSigned-off-by: Shuang \u003clvshuang.xjs@alibaba-inc.com\u003e\n"
    },
    {
      "commit": "783a7cc4fcedd57d62f6c876057700319163c7ee",
      "tree": "a01c83e616ca75a619fe02564cdbc9e68458e8b8",
      "parents": [
        "38532d7070e7d588250f2b69287729ed3d9cd3bf"
      ],
      "author": {
        "name": "xxx",
        "email": "953396112@qq.com",
        "time": "Fri Jan 02 20:03:23 2026 +0800"
      },
      "committer": {
        "name": "Shuang",
        "email": "lvshuang.xjs@alibaba-inc.com",
        "time": "Fri Jan 02 20:03:23 2026 +0800"
      },
      "message": "[CELEBORN-2236] Avoiding regular expressions for DiskFileInfo storage type determination\n\n### What changes were proposed in this pull request?\n\nAvoiding regular expressions for DiskFileInfo storage type determination.\n\n### Why are the changes needed?\n\nWhen sending a heartbeat, the Worker iterates all FileInfo objects and uses regex matching on a large number of them to check if the file is an HDFS file, thus reducing processing efficiency.\n\n\u003cimg width\u003d\"2766\" height\u003d\"1456\" alt\u003d\"image\" src\u003d\"https://github.com/user-attachments/assets/2dc075f6-562d-4467-a75c-4b6682ed866d\" /\u003e\n\n### Does this PR resolve a correctness bug?\n\nNo.\n\n### Does this PR introduce _any_ user-facing change?\n\nNo.\n\n### How was this patch tested?\n\nCI.\n\nCloses #3562 from xy2953396112/CELEBORN-2236.\n\nAuthored-by: xxx \u003c953396112@qq.com\u003e\nSigned-off-by: Shuang \u003clvshuang.xjs@alibaba-inc.com\u003e\n"
    },
    {
      "commit": "38532d7070e7d588250f2b69287729ed3d9cd3bf",
      "tree": "eab899e9a83b0f0db823dcc3b71d9d1ab165a023",
      "parents": [
        "526f7bb4fd31c3e0d362f643da43c92072d71e84"
      ],
      "author": {
        "name": "Wang, Fei",
        "email": "fwang12@ebay.com",
        "time": "Mon Dec 29 21:23:22 2025 -0800"
      },
      "committer": {
        "name": "Wang, Fei",
        "email": "fwang12@ebay.com",
        "time": "Mon Dec 29 21:23:22 2025 -0800"
      },
      "message": "[CELEBORN-2166][FOLLOWUP] Update config celeborn.client.shuffleDataLostOnUnknownWorker.enabled version to 0.6.3\n\n### What changes were proposed in this pull request?\n\nUpdate config celeborn.client.shuffleDataLostOnUnknownWorker.enabled version to 0.6.3\n\n### Why are the changes needed?\n\nFollowup for https://github.com/apache/celeborn/pull/3496, it is better to merge into branch-0.6 as well.\n### Does this PR resolve a correctness bug?\n\nNo.\n### Does this PR introduce _any_ user-facing change?\n\nNo, it has not been releases yet.\n\n### How was this patch tested?\n\nGA.\n\nCloses #3576 from turboFei/update_conf.\n\nAuthored-by: Wang, Fei \u003cfwang12@ebay.com\u003e\nSigned-off-by: Wang, Fei \u003cfwang12@ebay.com\u003e\n"
    },
    {
      "commit": "526f7bb4fd31c3e0d362f643da43c92072d71e84",
      "tree": "366ce326ead3262aa415929f7abe5566b4d7b438",
      "parents": [
        "ac8cffbb060a3144bff36e7d1de13565f1b157fa"
      ],
      "author": {
        "name": "Cheng Pan",
        "email": "chengpan@apache.org",
        "time": "Sat Dec 20 23:27:43 2025 +0800"
      },
      "committer": {
        "name": "Cheng Pan",
        "email": "chengpan@apache.org",
        "time": "Sat Dec 20 23:27:43 2025 +0800"
      },
      "message": "[CELEBORN-2240] Adapt to SPARK-51756 which add a new parameter `checksumValue` in `MapStatus.apply`\n\n### What changes were proposed in this pull request?\n\nAdapt to SPARK-51756, which changes the MapStatus API used by Celeborn.\n\n### Why are the changes needed?\n\nA necessary step to make Celeborn support Spark 4.1.\n\n### Does this PR resolve a correctness bug?\n\nNo.\n\n### Does this PR introduce _any_ user-facing change?\n\nYes, it makes\n\n### How was this patch tested?\n\nHave integrated with Spark 4.1 in the internal test env, and verified by some simple queries.\n\nCloses #3570 from pan3793/CELEBORN-2240.\n\nAuthored-by: Cheng Pan \u003cchengpan@apache.org\u003e\nSigned-off-by: Cheng Pan \u003cchengpan@apache.org\u003e\n"
    },
    {
      "commit": "ac8cffbb060a3144bff36e7d1de13565f1b157fa",
      "tree": "b2a3539b58f03738df2f741999a8eae1f4ed92d2",
      "parents": [
        "5789b4e35786bb62964042f818f5c32a76bf259e"
      ],
      "author": {
        "name": "Wang, Fei",
        "email": "fwang12@ebay.com",
        "time": "Tue Dec 16 20:08:27 2025 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Tue Dec 16 20:15:58 2025 +0800"
      },
      "message": "[CELEBORN-2231] Upgrade jersey version to 2.47 to fix CVE-2025-12383\n\n### What changes were proposed in this pull request?\n\nUpgrade jersey version to 2.47 to fix CVE-2025-12383\n\n### Why are the changes needed?\n\nto fix CVE-2025-12383\n\n### Does this PR resolve a correctness bug?\n\nNo.\n\n### Does this PR introduce _any_ user-facing change?\n\nNo.\n\n### How was this patch tested?\n\nGA\n\nCloses #3557 from turboFei/jersery.\n\nAuthored-by: Wang, Fei \u003cfwang12@ebay.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "5789b4e35786bb62964042f818f5c32a76bf259e",
      "tree": "0b8bafa26e18dd07be6f2c16fd0e3755ad76dcd0",
      "parents": [
        "0f663d0585e61b5079cb728f3631ebef317081e5"
      ],
      "author": {
        "name": "TheodoreLx",
        "email": "1548069580@qq.com",
        "time": "Sun Dec 14 21:29:17 2025 +0800"
      },
      "committer": {
        "name": "Shuang",
        "email": "lvshuang.xjs@alibaba-inc.com",
        "time": "Sun Dec 14 21:29:17 2025 +0800"
      },
      "message": "[CELEBORN-2217] Use a separate thread to handle RpcEndpointVerifier messages\n\n### What changes were proposed in this pull request?\n\nAn EndpointVerifierMessageLoop is introduced to specifically handle RpcEndpointVerifier messages, and a separate thread is used to execute the EndpointVerifierMessageLoop.\n\n### Why are the changes needed?\n\nRpcEndpointVerifier.CheckExistence is a high-priority message that needs to be processed first. In the original model, if the LifecycleManager\u0027s RPC message queue accumulates a large backlog, RpcEndpointVerifier.CheckExistence messages cannot be processed immediately, leading to numerous ShuffleClient initialization failures and causing task failures.\n\n### Does this PR resolve a correctness bug?\n\nNo\n\n### Does this PR introduce _any_ user-facing change?\n\nNo\n\n### How was this patch tested?\n\nCluster Test\n\nCloses #3554 from TheodoreLx/verifier-first.\n\nAuthored-by: TheodoreLx \u003c1548069580@qq.com\u003e\nSigned-off-by: Shuang \u003clvshuang.xjs@alibaba-inc.com\u003e\n"
    },
    {
      "commit": "0f663d0585e61b5079cb728f3631ebef317081e5",
      "tree": "0ab3864aeb99dbf9fc04ca7a88deeb1f281b6754",
      "parents": [
        "77a8921648c77a6fd073a09e07aa6f41a98153c1"
      ],
      "author": {
        "name": "xxx",
        "email": "953396112@qq.com",
        "time": "Thu Dec 11 13:24:15 2025 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Thu Dec 11 13:24:15 2025 +0800"
      },
      "message": "[CELEBORN-1577][FOLLOWUP] Fix addGauge error in registerUserResourceCons…\n\n…umptionMetrics\n\n### What changes were proposed in this pull request?\n\nFix addGauge error in registerUserResourceConsumptionMetrics\n\n### Why are the changes needed?\n\nFix addGauge error in registerUserResourceConsumptionMetrics\n\n### Does this PR resolve a correctness bug?\n\nNO\n\n### Does this PR introduce _any_ user-facing change?\n\nNO\n\n### How was this patch tested?\n\nCI\n\nCloses #3532 from xy2953396112/CELEBORN-1577.\n\nAuthored-by: xxx \u003c953396112@qq.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "77a8921648c77a6fd073a09e07aa6f41a98153c1",
      "tree": "35c596902ecac4c08d021d6215593b74c32646f9",
      "parents": [
        "36d6b0cba31d56aa45acfe921d877b33ff9637e8"
      ],
      "author": {
        "name": "xxx",
        "email": "953396112@qq.com",
        "time": "Thu Dec 11 13:21:47 2025 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Thu Dec 11 13:21:47 2025 +0800"
      },
      "message": "[CELEBORN-2212] Optimize the sorting efficiency of memoryWriters when evicting the largest memory file\n\n### What changes were proposed in this pull request?\n\nOptimize the sorting efficiency of memoryWriters when evicting the largest memory file.\n\n### Why are the changes needed?\n\nOptimize the sorting efficiency of memoryWriters when evicting the largest memory file.\n\n### Does this PR resolve a correctness bug?\n\nNo.\n\n### Does this PR introduce _any_ user-facing change?\n\nNo.\n\n### How was this patch tested?\n\nCI.\n\nCloses #3549 from xy2953396112/CELEBORN-2212.\n\nAuthored-by: xxx \u003c953396112@qq.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "36d6b0cba31d56aa45acfe921d877b33ff9637e8",
      "tree": "06ca274ddbb89c477713074f33b93cff595c4c0d",
      "parents": [
        "22336992ac4828a5957dff7f414a74993422d047"
      ],
      "author": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Thu Dec 11 11:23:57 2025 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Thu Dec 11 11:23:57 2025 +0800"
      },
      "message": "[CELEBORN-2237] Support Flink 2.2\n\n### What changes were proposed in this pull request?\n\nSupport Flink 2.2.\n\n### Why are the changes needed?\n\nFlink 2.2 has already released, which release notes refer to [Release notes - Flink 2.2](https://nightlies.apache.org/flink/flink-docs-master/release-notes/flink-2.2).\n\n### Does this PR resolve a correctness bug?\n\nNo.\n\n### Does this PR introduce _any_ user-facing change?\n\nNo.\n\n### How was this patch tested?\n\nCI.\n\nCloses #3563 from SteNicholas/CELEBORN-2237.\n\nAuthored-by: SteNicholas \u003cprogramgeek@163.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "22336992ac4828a5957dff7f414a74993422d047",
      "tree": "51c927af470347891509e47d988ae718b7cbbcc4",
      "parents": [
        "d546398a4bb86ac6d39c413d160bfb3bacd423e6"
      ],
      "author": {
        "name": "Ping Zhang",
        "email": "ping.goblue@gmail.com",
        "time": "Thu Dec 11 10:55:47 2025 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Thu Dec 11 10:55:47 2025 +0800"
      },
      "message": "[CELEBORN-2214] Add extraInitContainers for worker in helm\n\n### What changes were proposed in this pull request?\n\nAllow users to define more initContainers in the worker statefulset\n\n### Why are the changes needed?\n\nThis feature lets users define additional initContainers—for example, to wait for underlying NVMe disks to become ready.\nOn AKS LSv3 nodes, the local NVMe devices require setup steps (in our case, a DaemonSet mounts them under /mnt). By supporting extraInitContainers, we can delay starting the Celeborn workers until the NVMe disks are fully mounted.\n\n### Does this PR resolve a correctness bug?\n\nNo\n\n### Does this PR introduce _any_ user-facing change?\n\nNo\n\n### How was this patch tested?\n\nhelm tests:\n\n```\n❯ helm unittest . --file \"tests/**/*_test.yaml\" --strict --debug\n\n### Chart [ celeborn ] .\n\n PASS  Test Celeborn configmap\ttests/configmap_test.yaml\n PASS  Test Celeborn role\ttests/role_test.yaml\n PASS  Test Celeborn rolebinding\ttests/rolebinding_test.yaml\n PASS  Test Celeborn service account\ttests/serviceaccount_test.yaml\n PASS  Test Celeborn master pod monitor\ttests/master/podmonitor_test.yaml\n PASS  Test Celeborn master priority class\ttests/master/priorityclass_test.yaml\n PASS  Test Celeborn master service\ttests/master/service_test.yaml\n PASS  Test Celeborn master statefulset\ttests/master/statefulset_test.yaml\n PASS  Test Celeborn worker pod monitor\ttests/worker/podmonitor_test.yaml\n PASS  Test Celeborn worker priority class\ttests/worker/priorityclass_test.yaml\n PASS  Test Celeborn worker service\ttests/worker/service_test.yaml\n PASS  Test Celeborn worker statefulset\ttests/worker/statefulset_test.yaml\n\nCharts:      1 passed, 1 total\nTest Suites: 12 passed, 12 total\nTests:       77 passed, 77 total\nSnapshot:    0 passed, 0 total\nTime:        125.416ms\n```\n\nCloses #3550 from pingzh/dev/pingz-worker.extraInitContainers-CELEBORN-2214.\n\nLead-authored-by: Ping Zhang \u003cping.goblue@gmail.com\u003e\nCo-authored-by: Ping Zhang \u003cpingzh@umich.edu\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "d546398a4bb86ac6d39c413d160bfb3bacd423e6",
      "tree": "059c1e5429c59680026aa4fa2a260a44c65a88d8",
      "parents": [
        "f35b6b80ac13af3cb28cc8522bc209256a289f76"
      ],
      "author": {
        "name": "Wang, Fei",
        "email": "fwang12@ebay.com",
        "time": "Wed Dec 10 20:50:27 2025 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Wed Dec 10 21:04:01 2025 +0800"
      },
      "message": "[CELEBORN-2234] Bump jetty version to 9.4.58.v20250814 to fix GHSA-qh8g-58pp-2wxh\n\n### What changes were proposed in this pull request?\n\nBump jetty version to 9.4.58.v20250814 to fix GHSA-qh8g-58pp-2wxh\n### Why are the changes needed?\n\nTo fix CVE https://github.com/advisories/GHSA-qh8g-58pp-2wxh\n\n### Does this PR resolve a correctness bug?\n\n### Does this PR introduce _any_ user-facing change?\n\nNo.\n\n### How was this patch tested?\nGA.\n\nCloses #3560 from turboFei/jetty.\n\nAuthored-by: Wang, Fei \u003cfwang12@ebay.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "f35b6b80ac13af3cb28cc8522bc209256a289f76",
      "tree": "46477ebf9af3c53150c8413d60f742e30023444a",
      "parents": [
        "ef2586a96ab7aff87d7ea4aba171438faaf0e63e"
      ],
      "author": {
        "name": "HolyLow",
        "email": "jiaming.xie7@gmail.com",
        "time": "Mon Dec 08 14:00:36 2025 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Mon Dec 08 14:00:36 2025 +0800"
      },
      "message": "[CELEBORN-2206][CIP-14] Support PushData and Revive in Cpp\u0027s ShuffleClient\n\n### What changes were proposed in this pull request?\nThis PR supports PushData and Revive in Cpp\u0027s ShuffleClient so that the Cpp module is capable of writing to Celeborn Server.\n\n### Why are the changes needed?\nThis PR enables Cpp module to write to Celeborn Server.\n\n### Does this PR resolve a correctness bug?\nNo.\n\n### Does this PR introduce _any_ user-facing change?\nNo.\n\n### How was this patch tested?\nCompilation.\n\nCloses #3553 from HolyLow/issue/celeborn-2215-support-PushData-and-Revive-in-cpp-ShuffleClient.\n\nAuthored-by: HolyLow \u003cjiaming.xie7@gmail.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "ef2586a96ab7aff87d7ea4aba171438faaf0e63e",
      "tree": "98619ab42886a1b875862b17239d257630f68864",
      "parents": [
        "e5cef163646ff4592d981aad8eb714ccd3833414"
      ],
      "author": {
        "name": "Ping Zhang",
        "email": "ping.goblue@gmail.com",
        "time": "Fri Dec 05 11:49:15 2025 -0800"
      },
      "committer": {
        "name": "Wang, Fei",
        "email": "fwang12@ebay.com",
        "time": "Fri Dec 05 11:49:15 2025 -0800"
      },
      "message": "[MINOR] add makefile to do docs gen\n\n### What changes were proposed in this pull request?\n\ncreate a makefile to generate docs and later more targets can be added\n\n### Why are the changes needed?\n\ninside the makefile, it uses a python docker container to build / serve docs, so that contributors dont need to install a specific python version, venv etc. (e.g. right now the requirements.txt does not support python 3.12)\n\n### Does this PR resolve a correctness bug?\n\nNo\n\n### Does this PR introduce _any_ user-facing change?\n\nNo\n\n### How was this patch tested?\n\nlocally:\n\n```\n❯ make\nUsage: make \u003ctarget\u003e\n\nTargets:\n  docs-clean    Remove the generated site directory\n  docs-serve    Run mkdocs serve inside Docker and expose DOCS_PORT\n  docs          Build the MkDocs site inside a Python 3.11 container\n  help          Show help for available make targets\n```\n\n```\n❯ make docs\nINFO     -  DeprecationWarning: pkg_resources is deprecated as an API. See https://setuptools.pypa.io/en/latest/pkg_resources.html\n  File \"/usr/local/lib/python3.11/site-packages/mkdocs_macros/context.py\", line 19, in \u003cmodule\u003e\n    import pkg_resources\n  File \"/usr/local/lib/python3.11/site-packages/pkg_resources/__init__.py\", line 98, in \u003cmodule\u003e\n    warnings.warn(\n\nINFO     -  [macros] - Macros arguments: {\u0027module_name\u0027: \u0027main\u0027, \u0027modules\u0027: [], \u0027include_dir\u0027: \u0027\u0027, \u0027include_yaml\u0027: [], \u0027j2_block_start_string\u0027: \u0027\u0027, \u0027j2_block_end_string\u0027: \u0027\u0027, \u0027j2_variable_start_string\u0027: \u0027\u0027, \u0027j2_variable_end_string\u0027: \u0027\u0027, \u0027on_undefined\u0027: \u0027keep\u0027, \u0027on_error_fail\u0027: False, \u0027verbose\u0027: False}\nINFO     -  [macros] - Extra variables (config file): [\u0027social\u0027]\nINFO     -  [macros] - Extra filters (module): [\u0027pretty\u0027]\nINFO     -  Cleaning site directory\nINFO     -  Building documentation to directory: /workspace/site\nINFO     -  The following pages exist in the docs directory, but are not included in the \"nav\" configuration:\n  - configuration/client.md\n  - configuration/columnar-shuffle.md\n  - configuration/ha.md\n  - configuration/master.md\n  - configuration/metrics.md\n  - configuration/network-module.md\n  - configuration/network.md\n  - configuration/quota.md\n  - configuration/worker.md\n  - developers/release.md\nWARNING  -  Documentation file \u0027developers/faulttolerant.md\u0027 contains a link to \u0027../assets/img/revive.svg\u0027 which is not found in the documentation files.\nWARNING  -  Documentation file \u0027developers/faulttolerant.md\u0027 contains a link to \u0027../assets/img/batchrevive.svg\u0027 which is not found in the documentation files.\nWARNING  -  Documentation file \u0027developers/overview.md\u0027 contains a link to \u0027../assets/img/ess.svg\u0027 which is not found in the documentation files.\nWARNING  -  Documentation file \u0027developers/overview.md\u0027 contains a link to \u0027../assets/img/celeborn.svg\u0027 which is not found in the documentation files.\nWARNING  -  Documentation file \u0027developers/shuffleclient.md\u0027 contains a link to \u0027../assets/img/softsplit.svg\u0027 which is not found in the documentation files.\nWARNING  -  Documentation file \u0027developers/storage.md\u0027 contains a link to \u0027../assets/img/reducepartition.svg\u0027 which is not found in the documentation files.\nWARNING  -  Documentation file \u0027developers/storage.md\u0027 contains a link to \u0027../assets/img/mappartition.svg\u0027 which is not found in the documentation files.\nWARNING  -  Documentation file \u0027developers/storage.md\u0027 contains a link to \u0027../assets/img/multilayer.svg\u0027 which is not found in the documentation files.\nWARNING  -  Documentation file \u0027developers/trafficcontrol.md\u0027 contains a link to \u0027../assets/img/backpressure.svg\u0027 which is not found in the documentation files.\nINFO     -  Documentation built in 1.33 seconds\n\n~/code/apache-celeborn dev/pingz-makefile*                                                                                                                       5s apache-celeborn 14:52:43\n```\n\n`make docs-serve` locally\n\u003cimg width\u003d\"3222\" height\u003d\"1154\" alt\u003d\"image\" src\u003d\"https://github.com/user-attachments/assets/dddc02d1-fb4b-423c-be02-c433a766b5c5\" /\u003e\n\nCloses #3540 from pingzh/dev/pingz-makefile.\n\nAuthored-by: Ping Zhang \u003cping.goblue@gmail.com\u003e\nSigned-off-by: Wang, Fei \u003cfwang12@ebay.com\u003e\n"
    },
    {
      "commit": "e5cef163646ff4592d981aad8eb714ccd3833414",
      "tree": "cb722fcd614b62c9998f0d6521968d86a1a702ec",
      "parents": [
        "36cdc29eab749fe24fa23cb654c252bdbde1bd85"
      ],
      "author": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Mon Dec 01 12:06:40 2025 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Mon Dec 01 12:06:40 2025 +0800"
      },
      "message": "[CELEBORN-2209] Introduce ReadBufferUsageRatio metric to monitor credit stream read buffer usage\n\n### What changes were proposed in this pull request?\n\nIntroduce `ReadBufferUsageRatio` metric to monitor credit stream read buffer usage.\n\n### Why are the changes needed?\n\n`BufferStreamReadBuffer` metric is used to monitor the memory used by credit stream read buffer, which is not enough to monitor ratio of credit stream read buffer used and max direct memory.\n\n### Does this PR introduce _any_ user-facing change?\n\nNo.\n\n### How was this patch tested?\n\nCI and grafana manual test with [celeborn dashboard](https://stenicholas.grafana.net/public-dashboards/12f47ac2ba2f4c0c88f761f98ffcf51c).\n\nCloses #3545 from SteNicholas/CELEBORN-2209.\n\nAuthored-by: SteNicholas \u003cprogramgeek@163.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "36cdc29eab749fe24fa23cb654c252bdbde1bd85",
      "tree": "ddc54997d991b46382817c206580e3e5836e2d6a",
      "parents": [
        "8966c9b770490e30ab405e9543e24d7f2c4201eb"
      ],
      "author": {
        "name": "HolyLow",
        "email": "jiaming.xie7@gmail.com",
        "time": "Fri Nov 28 15:22:48 2025 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Fri Nov 28 15:22:48 2025 +0800"
      },
      "message": "[CELEBORN-2206][CIP-14] Support PushDataCallback in CppClient\n\n### What changes were proposed in this pull request?\nThis PR supports PushDataCallback in CppClient.\n\n### Why are the changes needed?\nPushDataCallback is the building block of PushData logic of CppClient\u0027s writing procedure.\n\n### Does this PR resolve a correctness bug?\nNo.\n\n### Does this PR introduce _any_ user-facing change?\nNo.\n\n### How was this patch tested?\nCompilation and UTs.\n\nCloses #3543 from HolyLow/issue/celeborn-2206-support-PushDataCallback-in-cpp-client.\n\nAuthored-by: HolyLow \u003cjiaming.xie7@gmail.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "8966c9b770490e30ab405e9543e24d7f2c4201eb",
      "tree": "d48f0d8e47b00fa8b7ddab57fcf3fe6f7131e854",
      "parents": [
        "5d6ce216a3267e66d86e52d2127fba75d898a1bf"
      ],
      "author": {
        "name": "Wang, Fei",
        "email": "fwang12@ebay.com",
        "time": "Fri Nov 21 16:33:57 2025 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Fri Nov 21 16:33:57 2025 +0800"
      },
      "message": "[CELEBORN-2208] Log the partition reader wait time if exceeds the threshold\n\n### What changes were proposed in this pull request?\nLog the partition reader wait time if exceeds the threshold.\n\n### Why are the changes needed?\n\nNow I see the task shuffle read wait time is very long, however there is no task log to indicate the slowness.\n\u003cimg width\u003d\"1702\" height\u003d\"130\" alt\u003d\"image\" src\u003d\"https://github.com/user-attachments/assets/47973563-13c7-4178-8954-3d3a23181a02\" /\u003e\n\n\u003cimg width\u003d\"1104\" height\u003d\"425\" alt\u003d\"image\" src\u003d\"https://github.com/user-attachments/assets/864448a8-de6b-47da-bb54-75b7b2f8a0c4\" /\u003e\n\n### Does this PR resolve a correctness bug?\n\nNo.\n### Does this PR introduce _any_ user-facing change?\n\nNo.\n\n### How was this patch tested?\n\nCode review.\n\nCloses #3544 from turboFei/log_time.\n\nAuthored-by: Wang, Fei \u003cfwang12@ebay.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "5d6ce216a3267e66d86e52d2127fba75d898a1bf",
      "tree": "0b39c0df60c240b9470ee791adc42b4159623b62",
      "parents": [
        "cc0d1ba70a4dd923f4cf69de5ec25bbd23a87c97"
      ],
      "author": {
        "name": "Xianming Lei",
        "email": "xianming.lei@shopee.com",
        "time": "Thu Nov 20 11:51:22 2025 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Thu Nov 20 11:51:22 2025 +0800"
      },
      "message": "[CELEBORN-1983][FOLLOWUP] Fix fetch fail not throw due to reach spark maxTaskFailures\n\n### What changes were proposed in this pull request?\nFix fetch fail not throw due to reach spark maxTaskFailures.\n\n### Why are the changes needed?\nThe condition `ti.attemptNumber() \u003e\u003d maxTaskFails - 1` may not be executed. Suppose that the current `taskAttempts` is index0, index1, index2, and index3, and that index0 and index1 have already failed while index2 and index3 are running, and the current `reportFetchFailed` is index3, then the final result will be false, while the expected result will be true.\nTherefore, we should check the attemptNumber of the current task separately before the loop starts.\n\n\u003cimg width\u003d\"3558\" height\u003d\"608\" alt\u003d\"image\" src\u003d\"https://github.com/user-attachments/assets/2a0af3e7-912e-420e-a864-4c525d07e251\" /\u003e\n\u003cimg width\u003d\"2332\" height\u003d\"814\" alt\u003d\"image\" src\u003d\"https://github.com/user-attachments/assets/bf832091-56d5-41b8-b58a-502e409d67a8\" /\u003e\n\n### Does this PR resolve a correctness bug?\n\nNo.\n\n### Does this PR introduce _any_ user-facing change?\nNo.\n\n### How was this patch tested?\nExisting UTs.\n\nCloses #3531 from leixm/follow_CELEBORN-1983.\n\nAuthored-by: Xianming Lei \u003cxianming.lei@shopee.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "cc0d1ba70a4dd923f4cf69de5ec25bbd23a87c97",
      "tree": "3c6e9047b44117a24303ad1c0e6efc9b1cb7eef9",
      "parents": [
        "3d61cede3f42155e3f895c52acc30e069655b77c"
      ],
      "author": {
        "name": "TheodoreLx",
        "email": "1548069580@qq.com",
        "time": "Thu Nov 20 11:19:37 2025 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Thu Nov 20 11:19:37 2025 +0800"
      },
      "message": "[CELEBORN-2152] Support merge buffers on the worker side to improve memory utilization\n\n### What changes were proposed in this pull request?\n\nProvides a configuration item that can copy the body buffer in pushdata to a newly requested buffer before writing on the worker, achieving 100% buffer internal space utilization, and ultimately significantly improving the overall utilization of NettyMemory.\n\n### Why are the changes needed?\nIn the worker, Netty uses AdaptiveRecvByteBufAllocator to determine the buffer size to allocate in advance when reading data from the socket. However, in certain network environments, there can be a significant discrepancy between the buffer size predicted and allocated by AdaptiveRecvByteBufAllocator and the actual data size read from the socket. This can result in a large buffer being allocated but only a small amount of data being read, ultimately leading to very low overall memory utilization in the worker. A clear metric is that NettyMemory is large but DiskBuffer is very small. This means that the worker may receive a small amount of data but quickly enter the Pause state due to excessive NettyMemory usage.\n\n### Does this PR introduce _any_ user-facing change?\n\nno\n\n### How was this patch tested?\n\ncluster test\n\n### Performance Test\n\u003cimg width\u003d\"1697\" height\u003d\"700\" alt\u003d\"image\" src\u003d\"https://github.com/user-attachments/assets/56495d08-6da7-4d43-8e8a-da87a33ccf90\" /\u003e\n\nCloses #3479 from TheodoreLx/merge-push-buffer.\n\nAuthored-by: TheodoreLx \u003c1548069580@qq.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "3d61cede3f42155e3f895c52acc30e069655b77c",
      "tree": "fb3511d7e4a02732fc217cd593fa53b85f41e596",
      "parents": [
        "ffff5bb94efd7c3a2c6c51aebccfd3c11b5d5198"
      ],
      "author": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Tue Nov 18 14:54:49 2025 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Tue Nov 18 14:54:49 2025 +0800"
      },
      "message": "[CELEBORN-2207] Master StatefulSet should add ratis-log4j.properties for ratis shell\n\n### What changes were proposed in this pull request?\n\nMaster StatefulSet should add `ratis-log4j.properties` for ratis shell.\n\n### Why are the changes needed?\n\nMaster StatefulSet does not include `ratis-log4j.properties`, which cause that `${CELEBORN_CONF_DIR}/ratis-log4j.properties` not exists for ratis shell. Therefore, StatefulSet should add r`atis-log4j.properties` for ratis shell of master.\n\n### Does this PR resolve a correctness bug?\n\nNo.\n\n### Does this PR introduce _any_ user-facing change?\n\nNo.\n\n### How was this patch tested?\n\n`Test Celeborn configmap#Should have correct data fields`\n\nCloses #3542 from SteNicholas/CELEBORN-2207.\n\nAuthored-by: SteNicholas \u003cprogramgeek@163.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "ffff5bb94efd7c3a2c6c51aebccfd3c11b5d5198",
      "tree": "3d1bcf107f89e3db99305da37d3561913fdecdb5",
      "parents": [
        "b6789915b8c20ab7457a16ccea290a360f49845e"
      ],
      "author": {
        "name": "HolyLow",
        "email": "jiaming.xie7@gmail.com",
        "time": "Mon Nov 17 11:58:16 2025 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Mon Nov 17 11:58:16 2025 +0800"
      },
      "message": "[CELEBORN-2196][CIP-14] Support ReviveManager in CppClient\n\n### What changes were proposed in this pull request?\nThis PR supports ReviveManager in CppClient.\n\n### Why are the changes needed?\nReviveManager is the building component for writing procedure of CppClient.\n\n### Does this PR resolve a correctness bug?\nNo.\n\n### Does this PR introduce _any_ user-facing change?\nNo.\n\n### How was this patch tested?\nCompilation and UTs.\n\nCloses #3538 from HolyLow/issue/celeborn-2196-support-ReviveManager-in-cpp-client.\n\nAuthored-by: HolyLow \u003cjiaming.xie7@gmail.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "b6789915b8c20ab7457a16ccea290a360f49845e",
      "tree": "8b66a546578fd2a57efc40d6bf94f0a9fa333a41",
      "parents": [
        "2874227291a1dd4fabd0fe7a0bd19534af08a2e8"
      ],
      "author": {
        "name": "Ping Zhang",
        "email": "ping.goblue@gmail.com",
        "time": "Mon Nov 17 10:54:29 2025 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Mon Nov 17 10:54:29 2025 +0800"
      },
      "message": "[CELEBORN-2203] Set celeborn.master.internal.endpoints in the configmap\n\n### What changes were proposed in this pull request?\n\nSet `celeborn.master.internal.endpoints` in the configmap.\n\n### Why are the changes needed?\n\nThe default value is `\u003clocalhost\u003e:8097` right now, which cause that workers fail to start as it cannot connect to that port.\n\n### Does this PR resolve a correctness bug?\n\nNo.\n\n### Does this PR introduce _any_ user-facing change?\n\nNo.\n\n### How was this patch tested?\n\n\u003cimg width\u003d\"2732\" height\u003d\"1104\" alt\u003d\"image\" src\u003d\"https://github.com/user-attachments/assets/542b6554-61c1-4c7b-a7bc-718cdcd50176\" /\u003e\n\nCloses #3536 from pingzh/dev/pingz-celeborn.master.internal.endpoints.\n\nAuthored-by: Ping Zhang \u003cping.goblue@gmail.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "2874227291a1dd4fabd0fe7a0bd19534af08a2e8",
      "tree": "2c59eb9783fdebf254b2244ff8b107380212cffe",
      "parents": [
        "79f0d319a17b838e15021b4abbb4040c710254ac"
      ],
      "author": {
        "name": "Ping Zhang",
        "email": "ping.goblue@gmail.com",
        "time": "Fri Nov 14 20:18:20 2025 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Fri Nov 14 20:18:20 2025 +0800"
      },
      "message": "[MINOR] Add calculation steps for tuning section\n\n### What changes were proposed in this pull request?\n\nper title\n\n### Why are the changes needed?\n\nso that readers can easily understand the process\n\n### Does this PR resolve a correctness bug?\n\nNo\n\n### Does this PR introduce _any_ user-facing change?\n\n### How was this patch tested?\n\nbefore:\n\u003cimg width\u003d\"2098\" height\u003d\"594\" alt\u003d\"image\" src\u003d\"https://github.com/user-attachments/assets/3b2a364b-1bf1-480f-b6cd-92c35a768a98\" /\u003e\n\nafter\n\n\u003cimg width\u003d\"2194\" height\u003d\"1122\" alt\u003d\"image\" src\u003d\"https://github.com/user-attachments/assets/8b37a50b-14ec-4926-bd52-e8fd5819ce29\" /\u003e\n\nCloses #3537 from pingzh/dev/pingz-doc-tuning.\n\nAuthored-by: Ping Zhang \u003cping.goblue@gmail.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "79f0d319a17b838e15021b4abbb4040c710254ac",
      "tree": "7f7f7dbfbaf994a4195b7ec728c88de181e18c85",
      "parents": [
        "835ee978c0acb1168754b1f4821011a77ba9fd80"
      ],
      "author": {
        "name": "jiang13021",
        "email": "jiangyanze.jyz@antgroup.com",
        "time": "Wed Nov 12 14:34:46 2025 +0800"
      },
      "committer": {
        "name": "Shuang",
        "email": "lvshuang.xjs@alibaba-inc.com",
        "time": "Wed Nov 12 14:34:46 2025 +0800"
      },
      "message": "[CELEBORN-2200] Throw IOException when compressed data header corrupted\n\n### What changes were proposed in this pull request?\nAs title.\n\n### Why are the changes needed?\nWe discovered that the corruption of the compressed data header may cause data loss. By throwing IOException, we can trigger a stage rerun to avoid data loss.\n\n### Does this PR resolve a correctness bug?\nYes\n\n### Does this PR introduce _any_ user-facing change?\nNo\n\n### How was this patch tested?\nAdd UT testLz4CodecCorrupted \u0026 testZstdCodecCorrupted.\n\nCloses #3534 from jiang13021/celeborn-2200.\n\nAuthored-by: jiang13021 \u003cjiangyanze.jyz@antgroup.com\u003e\nSigned-off-by: Shuang \u003clvshuang.xjs@alibaba-inc.com\u003e\n"
    },
    {
      "commit": "835ee978c0acb1168754b1f4821011a77ba9fd80",
      "tree": "d61dce68f35b1920939e304175db6d2051f050f2",
      "parents": [
        "5e4d80bb1e764b80f5d3462bb8ffb9061efc63b4"
      ],
      "author": {
        "name": "pingzh",
        "email": "ping.goblue@gmail.com",
        "time": "Wed Nov 12 10:19:38 2025 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Wed Nov 12 10:19:38 2025 +0800"
      },
      "message": "[CELEBORN-2202] Add configmap checksum to the statefulset annotation\n\n### What changes were proposed in this pull request?\n\n- Add the `checksum/celeborn_config` annotation to both master and worker\u0027s statefulset:\n```\n      annotations:\n        celeborn.apache.org/conf-hash: 0a0d909a0539b19a6b70f6a8df315a22eb12b5b4c3f59457d93b10405ae685f6\n```\n- sort the keys of celeborn config in the config map to produce consistent checksum\n\n### Why are the changes needed?\n\nWhen the configs in `celeborn` are changed, helm can restart the celeborn service.\n\n### Does this PR resolve a correctness bug?\n\nNo.\n\n### Does this PR introduce _any_ user-facing change?\n\nNo.\n\n### How was this patch tested?\n\n1. helm unittest\n```\n~/Documents/repos/opensource/celeborn/charts/celeborn\n❯ helm unittest . --file \"tests/**/*_test.yaml\" --strict --debug\n\n### Chart [ celeborn ] .\n\n PASS  Test Celeborn configmap\ttests/configmap_test.yaml\n PASS  Test Celeborn role\ttests/role_test.yaml\n PASS  Test Celeborn rolebinding\ttests/rolebinding_test.yaml\n PASS  Test Celeborn service account\ttests/serviceaccount_test.yaml\n PASS  Test Celeborn master pod monitor\ttests/master/podmonitor_test.yaml\n PASS  Test Celeborn master priority class\ttests/master/priorityclass_test.yaml\n PASS  Test Celeborn master service\ttests/master/service_test.yaml\n PASS  Test Celeborn master statefulset\ttests/master/statefulset_test.yaml\n PASS  Test Celeborn worker pod monitor\ttests/worker/podmonitor_test.yaml\n PASS  Test Celeborn worker priority class\ttests/worker/priorityclass_test.yaml\n PASS  Test Celeborn worker service\ttests/worker/service_test.yaml\n PASS  Test Celeborn worker statefulset\ttests/worker/statefulset_test.yaml\n\nCharts:      1 passed, 1 total\nTest Suites: 12 passed, 12 total\nTests:       75 passed, 75 total\nSnapshot:    0 passed, 0 total\nTime:        107.537875ms\n```\n\n2. `helm template --debug ../celeborn`\n\n\u003cimg width\u003d\"2316\" height\u003d\"1184\" alt\u003d\"image\" src\u003d\"https://github.com/user-attachments/assets/84297397-b008-40d2-a6a9-209664313b9b\" /\u003e\n\n\u003cimg width\u003d\"2532\" height\u003d\"1430\" alt\u003d\"image\" src\u003d\"https://github.com/user-attachments/assets/74f0b495-88a1-404f-ba09-647b1c306a69\" /\u003e\n\nCloses #3535 from pingzh/configmap-sha.\n\nAuthored-by: pingzh \u003cping.goblue@gmail.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "5e4d80bb1e764b80f5d3462bb8ffb9061efc63b4",
      "tree": "ba36b765612188efb2a6544fce3ced31129ea9a9",
      "parents": [
        "d12aafd542fe95e63c4f01747a99eb851576f788"
      ],
      "author": {
        "name": "pingzh",
        "email": "ping.goblue@gmail.com",
        "time": "Fri Nov 07 20:51:35 2025 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Fri Nov 07 20:51:35 2025 +0800"
      },
      "message": "[CELEBORN-2189] Allow config worker pod terminationGracePeriodSeconds in chart\n\n### What changes were proposed in this pull request?\n\nUpdate celeborn helm chart to expose worker pod\u0027s `terminationGracePeriodSeconds` for more flexibility.\n\n### Why are the changes needed?\n\nThe value of `terminationGracePeriodSeconds` need to expose for more flexibility.\n\n### Does this PR resolve a correctness bug?\n\nNo.\n\n### Does this PR introduce _any_ user-facing change?\n\nNo.\n\n### How was this patch tested?\n\nCI. and also local\n\n```\n❯ helm unittest .\n\n### Chart [ celeborn ] .\n\n PASS  Test Celeborn configmap\ttests/configmap_test.yaml\n PASS  Test Celeborn role\ttests/role_test.yaml\n PASS  Test Celeborn rolebinding\ttests/rolebinding_test.yaml\n PASS  Test Celeborn service account\ttests/serviceaccount_test.yaml\n\nCharts:      1 passed, 1 total\nTest Suites: 4 passed, 4 total\nTests:       9 passed, 9 total\nSnapshot:    0 passed, 0 total\nTime:        60.240792ms\n\n```\n\nCloses #3533 from pingzh/pingzh-add-terminationGracePeriodSeconds.\n\nAuthored-by: pingzh \u003cping.goblue@gmail.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "d12aafd542fe95e63c4f01747a99eb851576f788",
      "tree": "2d8f4900fd6fc8535cfbe664190cce6dc748df7f",
      "parents": [
        "00bcf8bc5379b5e43607655c0242e11112f1bf7f"
      ],
      "author": {
        "name": "dz",
        "email": "953396112@qq.com",
        "time": "Fri Nov 07 10:00:35 2025 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Fri Nov 07 10:00:35 2025 +0800"
      },
      "message": "[CELEBORN-2105] RpcMetricsTracker should clean up metrics for stopping Inbox\n\n### What changes were proposed in this pull request?\n\n`RpcMetricsTracker` should clean up metrics for stopping `Inbox` to avoid resource leak.\n\n### Why are the changes needed?\n\nWhen `Inbox` is closing, `RpcMetricsTracker` does not clean up metrics at present, which may cause resource leak.\n\n### Does this PR introduce _any_ user-facing change?\n\nNo.\n\n### How was this patch tested?\n\nCI.\n\nCloses #3419 from xy2953396112/CELEBORN-2105.\n\nAuthored-by: dz \u003c953396112@qq.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "00bcf8bc5379b5e43607655c0242e11112f1bf7f",
      "tree": "0c8f7155fa8fad5c823d5149516f1522c8e9f2a6",
      "parents": [
        "7efff4bdbe4d55f18a7e5cb41cecbb142c06bdf2"
      ],
      "author": {
        "name": "zhouhai",
        "email": "zhouhai@shizhuang-inc.com",
        "time": "Thu Nov 06 14:40:29 2025 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Thu Nov 06 14:40:29 2025 +0800"
      },
      "message": "[CELEBORN-2198] Fix NPE in tryWithTimeoutAndCallback test due to lazy deviceCheckThreadPool not initialized\n\n### What changes were proposed in this pull request?\n\nUse fallback thread pool in test when DeviceMonitor.deviceCheckThreadPool is uninitialized.\n\n### Why are the changes needed?\n\nThe unit test `tryWithTimeoutAndCallback` in `DeviceMonitorSuite` fails with `NullPointerException` when run in isolation. The root cause is as follows:\n\n`DeviceMonitor.deviceCheckThreadPool` is  lazily initialized, initialized only when DeviceMonitor.createDeviceMonitor() is first called.\n\n```\njava.lang.NullPointerException was thrown.\njava.lang.NullPointerException\n  at org.apache.celeborn.common.util.Utils$.tryWithTimeoutAndCallback(Utils.scala:1028)\n  at org.apache.celeborn.service.deploy.worker.storage.DeviceMonitorSuite.$anonfun$new$17(DeviceMonitorSuite.scala:371)\n  at org.apache.celeborn.service.deploy.worker.storage.DeviceMonitorSuite.$anonfun$new$17$adapted(DeviceMonitorSuite.scala:368)\n  at scala.collection.immutable.Range.foreach(Range.scala:158)\n  at org.apache.celeborn.service.deploy.worker.storage.DeviceMonitorSuite.$anonfun$new$14(DeviceMonitorSuite.scala:368)\n```\n\n### Does this PR resolve a correctness bug?\n\nNo.\n\n### Does this PR introduce _any_ user-facing change?\n\nNo.\n\n### How was this patch tested?\n\nRe-ran the test in isolation and full suite — all pass.\n\nCloses #3529 from yew1eb/CELEBORN-2198.\n\nAuthored-by: zhouhai \u003czhouhai@shizhuang-inc.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "7efff4bdbe4d55f18a7e5cb41cecbb142c06bdf2",
      "tree": "583ceef429e120221b13728ff6ef34545fcff515",
      "parents": [
        "da323ef69b0fc0ade5dea234a3831679ae354e57"
      ],
      "author": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Wed Nov 05 14:57:14 2025 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Wed Nov 05 14:57:14 2025 +0800"
      },
      "message": "[CELEBORN-1258][FOLLOWUP] Introduce --show-cluster-apps-info master command to show cluster application\u0027s info\n\n### What changes were proposed in this pull request?\n\nIntroduce `--show-cluster-apps-info` master command to show cluster application\u0027s info.\n\n### Why are the changes needed?\n\n#3428 introduces `/api/v1/applications/info` to list all running application\u0027s info of the cluster. Therefore, Cli should also introduce --show-cluster-apps-info master command.\n\n### Does this PR resolve a correctness bug?\n\nNo.\n\n### Does this PR introduce _any_ user-facing change?\n\nCli adds `--show-cluster-apps-info` master command to show cluster application\u0027s info.\n\n### How was this patch tested?\n\n`TestCelebornCliCommands#master --show-cluster-apps-info`\n\nCloses #3530 from SteNicholas/CELEBORN-1258.\n\nAuthored-by: SteNicholas \u003cprogramgeek@163.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "da323ef69b0fc0ade5dea234a3831679ae354e57",
      "tree": "9d055bd145f8bd3ece9f10ca3d119d2e28de4bcf",
      "parents": [
        "76199454ca28f1d0f390b06afc2bc87e61a3b3ed"
      ],
      "author": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Wed Nov 05 14:31:17 2025 +0800"
      },
      "committer": {
        "name": "子懿",
        "email": "ziyi.jxf@antgroup.com",
        "time": "Wed Nov 05 14:31:17 2025 +0800"
      },
      "message": "[CELEBORN-2047][FOLLOWUP] MapPartitionData should close dataChannel, indexChannel, dataInputStream and indexInputStream\n\n### What changes were proposed in this pull request?\n\n`MapPartitionData` should close `dataChannel`, `indexChannel`, `dataInputStream` and `indexInputStream`.\n\nFollow up #3445.\n\n### Why are the changes needed?\n\n`dataChannel`, `indexChannel`, `dataInputStream` and `indexInputStream` should be closed in `MapPartitionData` instead of `PartitionDataReader`.\n\n### Does this PR resolve a correctness bug?\n\nNo.\n\n### Does this PR introduce _any_ user-facing change?\n\nNo.\n\n### How was this patch tested?\n\nCI.\n\nCloses #3521 from SteNicholas/CELEBORN-2047.\n\nAuthored-by: SteNicholas \u003cprogramgeek@163.com\u003e\nSigned-off-by: 子懿 \u003cziyi.jxf@antgroup.com\u003e\n"
    },
    {
      "commit": "76199454ca28f1d0f390b06afc2bc87e61a3b3ed",
      "tree": "aca91b91b5db5eb7fef6bf765a42ce4088b3971b",
      "parents": [
        "00dc8cb80b0ae66fc5e1c275cb0e8915b028d2b2"
      ],
      "author": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Wed Nov 05 10:24:38 2025 +0800"
      },
      "committer": {
        "name": "Shuang",
        "email": "lvshuang.xjs@alibaba-inc.com",
        "time": "Wed Nov 05 10:24:38 2025 +0800"
      },
      "message": "[CELEBORN-2192] ReadBufferDispatcher should add timeout constraints to fast fail in case of timeout\n\n### What changes were proposed in this pull request?\n\n`ReadBufferDispatcher` should add timeout constraints to fast fail in case of timeout.\n\n### Why are the changes needed?\n\nSetting `celeborn.worker.directMemoryRatioForReadBuffer` with small ratio may result in a backlog of read buffer requests for `ReadBufferDispatcher` at present, which causes running flink jobs to stall. `ReadBufferDispatcher` should add timeout constraints to fast fail in case of timeout, avoiding long wait times for client.\n\n### Does this PR resolve a correctness bug?\n\nNo.\n\n### Does this PR introduce _any_ user-facing change?\n\nIntroduce `celeborn.worker.readBuffer.processTimeout` config to specify timeout for buffer dispatcher to process a read buffer request.\n\n### How was this patch tested?\n\n`ReadBufferDispactherSuite#[CELEBORN-2192] ReadBufferDispatcher should add timeout constraints to fast fail in case of timeout`\n\nCloses #3525 from SteNicholas/CELEBORN-2192.\n\nLead-authored-by: SteNicholas \u003cprogramgeek@163.com\u003e\nCo-authored-by: 子懿 \u003cziyi.jxf@antgroup.com\u003e\nSigned-off-by: Shuang \u003clvshuang.xjs@alibaba-inc.com\u003e\n"
    },
    {
      "commit": "00dc8cb80b0ae66fc5e1c275cb0e8915b028d2b2",
      "tree": "a45ea42f5989714aa449521c851f73cd42f699c8",
      "parents": [
        "e81aa57e008819e0931fff27a97280baf4d0b6f7"
      ],
      "author": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Tue Nov 04 15:57:34 2025 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Tue Nov 04 15:57:34 2025 +0800"
      },
      "message": "[CELEBORN-2193] Bump Flink from 2.0.0, 2.1.0 to 2.0.1, 2.1.1\n\n### What changes were proposed in this pull request?\n\nBump Flink from 2.0.0, 2.1.0 to 2.0.1, 2.1.1.\n\n### Why are the changes needed?\n\nFlink has released v2.0.1 and v2.1.1, which release notes refer to:\n\n- [Apache Flink 2.0.1 Release](https://github.com/apache/flink/releases/tag/release-2.0.1)\n- [Apache Flink 2.1.1 Release](https://github.com/apache/flink/releases/tag/release-2.1.1)\n\nFlink v2.0.1 adds the `getConsumedPartitionType()` interface into `IndexedInputGate`, which refers to https://github.com/apache/flink/pull/26548.\n\n`HybridShuffleWordCountTest` could execute with parallelism in https://github.com/apache/flink/pull/26369 which has released in v2.0.1 and v2.1.1.\n\n### Does this PR resolve a correctness bug?\n\nNo.\n\n### Does this PR introduce _any_ user-facing change?\n\nNo.\n\n### How was this patch tested?\n\nCI.\n\nCloses #3526 from SteNicholas/CELEBORN-2193.\n\nAuthored-by: SteNicholas \u003cprogramgeek@163.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "e81aa57e008819e0931fff27a97280baf4d0b6f7",
      "tree": "3547190d8fbaa17af0f8c45f311dfc0b23ba0ed5",
      "parents": [
        "7244f181e1ebc3fb2633842c6cc5e1873a7c58c5"
      ],
      "author": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Tue Nov 04 15:54:45 2025 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Tue Nov 04 15:54:45 2025 +0800"
      },
      "message": "[CELEBORN-2195] Align log4j2.xml and metrics.properties of charts with templates\n\n### What changes were proposed in this pull request?\n\nAlign `log4j2.xml` and `metrics.properties` of charts with templates.\n\nFollow up:\n\n- https://github.com/apache/celeborn/pull/2895\n- https://github.com/apache/celeborn/pull/3265\n- https://github.com/apache/celeborn/pull/3346\n\n### Why are the changes needed?\n\nThe `log4j2.xml` and `metrics.properties` of charts are inconsistent with templates. We should align `log4j2.xml` and `metrics.properties` with conf templates.\n\n### Does this PR resolve a correctness bug?\n\nNo.\n\n### Does this PR introduce _any_ user-facing change?\n\nNo.\n\n### How was this patch tested?\n\nCI.\n\nCloses #3528 from SteNicholas/CELEBORN-2195.\n\nAuthored-by: SteNicholas \u003cprogramgeek@163.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "7244f181e1ebc3fb2633842c6cc5e1873a7c58c5",
      "tree": "283391a7172bd49036c81ff2f000501d9197d39f",
      "parents": [
        "04231e07dcaab6a75def6d28e3efe19d887beda3"
      ],
      "author": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Tue Nov 04 15:23:06 2025 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Tue Nov 04 15:35:55 2025 +0800"
      },
      "message": "[CELEBORN-2194] Change default value of celeborn.worker.directMemoryRatioForReadBuffer\n\n### What changes were proposed in this pull request?\n\nChange default value of `celeborn.worker.directMemoryRatioForReadBuffer` from 0.1 to 0.35.\n\n### Why are the changes needed?\n\nThe default value of `celeborn.worker.directMemoryRatioForReadBuffer` is 0.1, which is too small to cause a backlog of read buffer requests in `ReadBufferDispacther`. Therefore, `celeborn.worker.directMemoryRatioForReadBuffer` should be changed from `0.1` to `0.35` which is production practice value to raise read buffer threshold of `ReadBufferDispatcher`.\n\n### Does this PR resolve a correctness bug?\n\nNo.\n\n### Does this PR introduce _any_ user-facing change?\n\nThe default value of `celeborn.worker.directMemoryRatioForReadBuffer` is changed to 0.35.\n\n### How was this patch tested?\n\nCI.\n\nCloses #3527 from SteNicholas/CELEBORN-2194.\n\nAuthored-by: SteNicholas \u003cprogramgeek@163.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "04231e07dcaab6a75def6d28e3efe19d887beda3",
      "tree": "48990e5fda17ec65cdbaaf9e34309aa8d161f607",
      "parents": [
        "8ccd4dad10a53647d54231337b5c6966160ee236"
      ],
      "author": {
        "name": "xxx",
        "email": "953396112@qq.com",
        "time": "Mon Nov 03 20:40:37 2025 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Mon Nov 03 20:40:37 2025 +0800"
      },
      "message": "[CELEBORN-2188] Abort multipart upload for S3 and OSS in DfsTierWriter#handleException\n\n### What changes were proposed in this pull request?\n\nAbort multipart upload for S3 and OSS in `DfsTierWriter#handleException`.\n\n### Why are the changes needed?\n\nWhen handling exception of S3 or OSS, `DfsTierWriter` should abort multipart upload instead of complete.\n\n### Does this PR resolve a correctness bug?\n\nNo.\n\n### Does this PR introduce _any_ user-facing change?\n\nNo.\n\n### How was this patch tested?\n\nCI.\n\nCloses #3522 from xy2953396112/CELEBORN-2188.\n\nAuthored-by: xxx \u003c953396112@qq.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    },
    {
      "commit": "8ccd4dad10a53647d54231337b5c6966160ee236",
      "tree": "c3a0ef0a2734dfedd6b9d8a2d0975dbbaa926a55",
      "parents": [
        "005d76d3fe53d52bdaf88d4d222c114ecd310d7d"
      ],
      "author": {
        "name": "HolyLow",
        "email": "jiaming.xie7@gmail.com",
        "time": "Mon Nov 03 11:36:17 2025 +0800"
      },
      "committer": {
        "name": "SteNicholas",
        "email": "programgeek@163.com",
        "time": "Mon Nov 03 11:36:17 2025 +0800"
      },
      "message": "[CELEBORN-2182][CIP-14] Support PushState and PushStrategy in CppClient\n\n### What changes were proposed in this pull request?\nThis PR supports PushState and PushStrategy in CppClient.\n\n### Why are the changes needed?\nThese functionalities are used in the writing procedure in CppClient.\n\n### Does this PR introduce _any_ user-facing change?\nNo.\n\n### How was this patch tested?\nCompilation and UTs.\n\nCloses #3515 from HolyLow/issue/celeborn-2182-support-PushState-in-cpp-client.\n\nAuthored-by: HolyLow \u003cjiaming.xie7@gmail.com\u003e\nSigned-off-by: SteNicholas \u003cprogramgeek@163.com\u003e\n"
    }
  ],
  "next": "005d76d3fe53d52bdaf88d4d222c114ecd310d7d"
}
