Commit Graph

371 Commits

Author SHA1 Message Date
Fu Chen
fe623888bf [CELEBORN-1290] Fix NPE occurring prior to worker registration
### What changes were proposed in this pull request?

As title

### Why are the changes needed?

This PR addressed a NPE issue occurs when the `Worker#reigstered` member is accessed before it is initialized.

The problem occurs because the `TransportChannelHandler` might be served before the worker is registered.

```
24/02/01 15:07:32,090 WARN [push-server-6-6] TransportChannelHandler: Exception in connection from /xx.xx.xx.xx:xxx
java.lang.NullPointerException
        at org.apache.celeborn.service.deploy.worker.PushDataHandler.checkRegistered(PushDataHandler.scala:714)
        at org.apache.celeborn.common.network.server.TransportRequestHandler.checkRegistered(TransportRequestHandler.java:82)
        at org.apache.celeborn.common.network.server.TransportRequestHandler.handle(TransportRequestHandler.java:76)
        at org.apache.celeborn.common.network.server.TransportChannelHandler.channelRead(TransportChannelHandler.java:151)
        at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:444)
        at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
        at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:412)
        at io.netty.handler.timeout.IdleStateHandler.channelRead(IdleStateHandler.java:286)
        at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:442)
        at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
        at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:412)
        at org.apache.celeborn.common.network.util.TransportFrameDecoder.channelRead(TransportFrameDecoder.java:74)
        at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:444)
        at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
        at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:412)
        at io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1410)
        at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:440)
        at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
        at io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:919)
        at io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:166)
        at io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:788)
        at io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:724)
        at io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:650)
        at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:562)
        at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:997)
        at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
        at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
        at java.lang.Thread.run(Thread.java:750)
```

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

Pass GA

Closes #2274 from cfmcgrady/check-registered.

Authored-by: Fu Chen <cfmcgrady@gmail.com>
Signed-off-by: Fu Chen <cfmcgrady@gmail.com>
2024-02-27 19:14:19 +08:00
liangyongyuan
4ddc91afda [CELEBRON-1282] Optimize push data replica error message
### What changes were proposed in this pull request?
Optimize the handling of exceptions during the push of replica data, now only throwing PUSH_DATA_CONNECTION_EXCEPTION_REPLICA in specific scenarios.

### Why are the changes needed?
When handling exceptions related to pushing replica data in the worker, unmatched exceptions, such as 'file already closed,' are uniformly transformed into REPLICATE_DATA_CONNECTION_EXCEPTION_COUNT and returned to the client. The client then excludes the peer node based on this count, which may not be appropriate in certain scenarios. For instance, in the case of an exception like 'file already closed,' it typically occurs during multiple splits and commitFile operations. Excluding a large number of nodes under such circumstances is clearly not in line with expectations.
![image](https://github.com/apache/incubator-celeborn/assets/46274164/816d21ad-1f79-45f0-bbe7-e93e15389edd)

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
through exist uts

Closes #2323 from lyy-pineapple/CELEBORN-1282.

Authored-by: liangyongyuan <liangyongyuan@xiaomi.com>
Signed-off-by: waitinfuture <zky.zhoukeyong@alibaba-inc.com>
2024-02-26 12:55:26 +08:00
SteNicholas
b9bdea3c72
[CELEBORN-1280] Change default value of celeborn.worker.graceful.shutdown.recoverDbBackend to ROCKSDB
### What changes were proposed in this pull request?

Change the default value of `celeborn.worker.graceful.shutdown.recoverDbBackend` from `LEVELDB` to `ROCKSDB`.

### Why are the changes needed?

Because the LevelDB support will be removed, the default value of `celeborn.worker.graceful.shutdown.recoverDbBackend` could be changed to ROCKSDB instead of LEVELDB for preparation of LevelDB deprecation.

Backport:
 [[SPARK-45351][CORE] Change spark.shuffle.service.db.backend default value to ROCKSDB](https://github.com/apache/spark/pull/43142)
 [[SPARK-45413][CORE] Add warning for prepare drop LevelDB support](https://github.com/apache/spark/pull/43217)

### Does this PR introduce _any_ user-facing change?

The default value of `celeborn.worker.graceful.shutdown.recoverDbBackend` is changed from `LEVELDB` to `ROCKSDB`.

### How was this patch tested?

No.

Closes #2320 from SteNicholas/CELEBORN-1280.

Lead-authored-by: SteNicholas <programgeek@163.com>
Co-authored-by: Nicholas Jiang <programgeek@163.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
2024-02-23 14:53:24 +08:00
SteNicholas
d71f16f7bf [CELEBORN-1254][FOLLOWUP] Rename celeborn.worker.sortPartition.reservedMemory.enabled to celeborn.worker.sortPartition.prefetch.enabled
### What changes were proposed in this pull request?

Rename `celeborn.worker.sortPartition.reservedMemory.enabled` to `celeborn.worker.sortPartition.prefetch.enabled`. Address [r1469066327](https://github.com/apache/incubator-celeborn/pull/2264/files#r1469066327) of pan3793.

### Why are the changes needed?

`celeborn.worker.sortPartition.reservedMemory.enabled` is misleading, which should represent that prefetch the original partition files during the first sequential reading path to leverage the Linux PageCache mechanism to speed up the subsequent random reading of them. The config name could use `celeborn.worker.sortPartition.prefetch.enabled` which is is more accurate.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

No.

Closes #2312 from SteNicholas/CELEBORN-1254.

Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: SteNicholas <programgeek@163.com>
2024-02-21 14:05:40 +08:00
Fei Wang
e54a2b7e2f [CELEBORN-1230] Check working directory read and write error without init delay
### What changes were proposed in this pull request?
In this pr, when getting device disk info, we check the dir writable to make sure that the capacity reported to celeborn master are correct and does no include non-writable directories.
### Why are the changes needed?
To ignore bad disk when initializing the worker.

### Does this PR introduce _any_ user-facing change?
No.

### How was this patch tested?

Existing UT.

Closes #2233 from turboFei/check_disk_init.

Lead-authored-by: Fei Wang <fwang12@ebay.com>
Co-authored-by: Keyong Zhou <zky.zhoukeyong@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2024-02-16 00:07:01 +08:00
Chandni Singh
ab4c0bc85b [CELEBORN-1257] Adds a secured port in Celeborn Master for secure communication with LifecycleManager
### What changes were proposed in this pull request?
This adds a secured port to Celeborn Master which is used for secure communication with LifecycleManager.
This is part of adding authentication support in Celeborn (see CELEBORN-1011).

This change targets just adding the secured port to Master. The following items from the proposal are still pending:
1. Persisting the app secrets in Ratis.
2. Forwarding secrets to Workers and having ability for the workers to pull registration info from the Master.
3. Secured and internal port in Workers.
4. Secured communication between workers and clients.

In addition, since we are supporting both secured and unsecured communication for backward compatibility and seamless rolling upgrades, there is an additional change needed. An app which registers with the Master can try to talk to the workers on unsecured ports which is a security breach. So, the workers need to know whether an app registered with Master or not and for that Master has to propagate list of un-secured apps to Celeborn workers as well. We can discuss this more with https://issues.apache.org/jira/browse/CELEBORN-1261

### Why are the changes needed?
It is needed for adding authentication support to Celeborn (CELEBORN-1011)

### Does this PR introduce _any_ user-facing change?
Yes

### How was this patch tested?
Added a simple UT.

Closes #2281 from otterc/CELEBORN-1257.

Authored-by: Chandni Singh <singh.chandni@gmail.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2024-02-06 14:53:28 +08:00
SteNicholas
c3b129da1f
[CELEBORN-1252][FOLLOWUP] Fix Worker#computeResourceConsumption NullPointerException for userResourceConsumption that does not contain given userIdentifier
### What changes were proposed in this pull request?

Fix `Worker#computeResourceConsumption` `NullPointerException` for `userResourceConsumption` that does not contain given `userIdentifier`.

### Why are the changes needed?

When `userResourceConsumption` of `workerInfo` does not contain given `userIdentifier`, `Worker#computeResourceConsumption` causes `NullPointerException` for worker dimension resource consumption metrics.

```
24/02/05 17:36:15,983 ERROR [worker-forward-message-scheduler] Utils: Uncaught exception in thread worker-forward-message-scheduler
java.lang.NullPointerException
        at org.apache.celeborn.service.deploy.worker.Worker.$anonfun$gaugeResourceConsumption$1(Worker.scala:555)
        at scala.runtime.java8.JFunction0$mcJ$sp.apply(JFunction0$mcJ$sp.java:23)
        at org.apache.celeborn.common.metrics.source.GaugeSupplier$$anon$3.getValue(AbstractSource.scala:453)
        at org.apache.celeborn.common.metrics.source.AbstractSource.addGauge(AbstractSource.scala:79)
        at org.apache.celeborn.common.metrics.source.AbstractSource.addGauge(AbstractSource.scala:99)
        at org.apache.celeborn.service.deploy.worker.Worker.gaugeResourceConsumption(Worker.scala:554)
        at org.apache.celeborn.service.deploy.worker.Worker.$anonfun$handleResourceConsumption$1(Worker.scala:537)
        at org.apache.celeborn.service.deploy.worker.Worker.$anonfun$handleResourceConsumption$1$adapted(Worker.scala:536)
        at scala.collection.immutable.Map$Map1.foreach(Map.scala:128)
        at org.apache.celeborn.service.deploy.worker.Worker.handleResourceConsumption(Worker.scala:536)
        at org.apache.celeborn.service.deploy.worker.Worker.org$apache$celeborn$service$deploy$worker$Worker$$heartbeatToMaster(Worker.scala:362)
        at org.apache.celeborn.service.deploy.worker.Worker$$anon$1.$anonfun$run$1(Worker.scala:395)
        at org.apache.celeborn.common.util.Utils$.tryLogNonFatalError(Utils.scala:230)
        at org.apache.celeborn.service.deploy.worker.Worker$$anon$1.run(Worker.scala:395)
        at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
        at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
        at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
        at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)
```

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

GA and cluster.

Closes #2288 from SteNicholas/CELEBORN-1252.

Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
2024-02-05 19:55:01 +08:00
SteNicholas
277d060e48
[CELEBORN-1174][FOLLOWUP] Fix Worker#computeResourceConsumption NullPointerException with null subResourceConsumptions
### What changes were proposed in this pull request?

Fix `Worker#computeResourceConsumption` `NullPointerException` with null `subResourceConsumptions`.

### Why are the changes needed?

With null `subResourceConsumptions`, `Worker#computeResourceConsumption` causes `NullPointerException` for application dimension resource consumption metrics.

```
24/02/04 13:58:13,757 ERROR [worker-forward-message-scheduler] Utils: Uncaught exception in thread worker-forward-message-scheduler
java.lang.NullPointerException
        at org.apache.celeborn.service.deploy.worker.Worker.computeResourceConsumption(Worker.scala:581)
        at org.apache.celeborn.service.deploy.worker.Worker.$anonfun$gaugeResourceConsumption$1(Worker.scala:555)
        at scala.runtime.java8.JFunction0$mcJ$sp.apply(JFunction0$mcJ$sp.java:23)
        at org.apache.celeborn.common.metrics.source.GaugeSupplier$$anon$3.getValue(AbstractSource.scala:453)
        at org.apache.celeborn.common.metrics.source.AbstractSource.addGauge(AbstractSource.scala:79)
        at org.apache.celeborn.common.metrics.source.AbstractSource.addGauge(AbstractSource.scala:99)
        at org.apache.celeborn.service.deploy.worker.Worker.gaugeResourceConsumption(Worker.scala:554)
        at org.apache.celeborn.service.deploy.worker.Worker.$anonfun$handleResourceConsumption$1(Worker.scala:537)
        at org.apache.celeborn.service.deploy.worker.Worker.$anonfun$handleResourceConsumption$1$adapted(Worker.scala:536)
        at scala.collection.immutable.Map$Map1.foreach(Map.scala:128)
        at org.apache.celeborn.service.deploy.worker.Worker.handleResourceConsumption(Worker.scala:536)
        at org.apache.celeborn.service.deploy.worker.Worker.org$apache$celeborn$service$deploy$worker$Worker$$heartbeatToMaster(Worker.scala:362)
        at org.apache.celeborn.service.deploy.worker.Worker$$anon$1.$anonfun$run$1(Worker.scala:395)
        at org.apache.celeborn.common.util.Utils$.tryLogNonFatalError(Utils.scala:230)
        at org.apache.celeborn.service.deploy.worker.Worker$$anon$1.run(Worker.scala:395)
        at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
        at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
        at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
        at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)
```

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

GA and cluster.

Closes #2286 from SteNicholas/CELEBORN-1174.

Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
2024-02-04 20:24:12 +08:00
SteNicholas
4c5e6f065c
[CELEBORN-1182] Support application dimension ActiveConnectionCount metric to record the number of registered connections for each application
### What changes were proposed in this pull request?

`WorkerSource` supports application dimension `ActiveConnectionCount` metric to record the number of registered connections for each application.

### Why are the changes needed?

`ActiveConnectionCount` metric records the number of registered connections at present. It's recommended to support dimension ActiveConnectionCount metric to record the number of registered connections for each application in Worker. Application dimension `ActiveConnectionCount` metric could provide users with the actual number of registered connections for each application.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Internal tests.

Closes #2167 from SteNicholas/CELEBORN-1182.

Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
2024-02-02 16:29:10 +08:00
SteNicholas
05fa11b3a0 [CELEBORN-1174] Introduce application dimension resource consumption metrics
### What changes were proposed in this pull request?

Introduce application dimension resource consumption metrics for `ResourceConsumptionSource`.

### Why are the changes needed?

`ResourceConsumption` namespace metrics are generated for each user and they are identified using a metric tag at present. It's recommended to introduce application dimension resource consumption metrics that expose application dimension resource consumption of Master and Worker. By monitoring resource consumption in the application dimension, you can obtain the actual situation of application resource consumption.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

- `WorkerInfoSuite#WorkerInfo toString output`
- `PbSerDeUtilsTest#fromAndToPbResourceConsumption`
- `MasterStateMachineSuitej#testObjSerde`

Closes #2161 from SteNicholas/CELEBORN-1174.

Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: SteNicholas <programgeek@163.com>
2024-02-01 15:24:29 +08:00
Shuang
e71d912d50 [CELEBORN-1245] Support Celeborn Master(Leader) to manage workers
### What changes were proposed in this pull request?
1. Support Celeborn Master(Leader) to manage workers by sending event when heartbeat
2. Add Worker Status to Worker then we can know the status of the workers(such as during decommission...)
3. Add Http interface for master to handleWorkerEvent/getWorkerEvent

### Why are the changes needed?
Currently, we only support managing the status of workers on the worker side. This pr supports the master to manage the status of all workers. By sending events such as (Decommission/Graceful/Exit) when heartbeat, workers can be asynchronously execute the command from master. MeanWhile we can't know what the worker status during worker decommission so this pr add worker status to tell the exactly status of the worker.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Pass GA

Closes #2255 from RexXiong/CELEBORN-1245.

Authored-by: Shuang <lvshuang.xjs@alibaba-inc.com>
Signed-off-by: Shuang <lvshuang.xjs@alibaba-inc.com>
2024-02-01 09:44:59 +08:00
mingji
99da6e2028
[CELEBORN-1248] Improve flusher's robustness
### What changes were proposed in this pull request?
Improve flusher's robustness.

### Why are the changes needed?
Flusher's thread might be terminated due to uncaught exceptions.

### Does this PR introduce _any_ user-facing change?
NO.

### How was this patch tested?
GA and cluaster.

Closes #2254 from FMX/b1248.

Authored-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
2024-01-29 21:22:17 +08:00
CodingCat
916df8f6ec [CELEBORN-1235] Start test nodes in random ports to allow multiple builds run in the same ci server
### What changes were proposed in this pull request?

this PR is to improve the test implementations so that it starts test nodes in random ports instead of using the hardcoded ones

### Why are the changes needed?

currently the test nodes are started in the hard coded ports, this prevents to run multiple builds in the same CI/CD server (which is not uncommonly seen in many companies infra)

### Does this PR introduce _any_ user-facing change?

no

### How was this patch tested?

it runs in our private CI/CD infra with many parallel builds very well

Closes #2237 from CodingCat/enhance_test.

Authored-by: CodingCat <zhunansjtu@gmail.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2024-01-27 12:51:53 +08:00
SteNicholas
954277d07c [CELEBORN-1254] PartitionFilesSorter seeks to position of each block and does not warm up for non-hdfs files
### What changes were proposed in this pull request?

Introduce `celeborn.worker.sortPartition.reservedMemory.enabled` to support that `PartitionFilesSorter` seeks to position of each block and does not warmup for non-hdfs files.

### Why are the changes needed?

File sorting includes three steps: reading files, sorting MapIds, and writing files. The default block of Celeborn is 256k, and the number of blocks is about 1000, so the sorting process is very fast, and the main overhead is file reading and writing. There are roughly three options for the entire sorting process:

1. Memory of the file size is allocated in advance, the file is read in as a whole, MapId is parsed and sorted, and Blocks are written back to the disk in MapId order.
2. No memory is allocated, seek to the location of each block, parse and sort the MapId, and transfer the Blocks of the original file to the new file in the order of MapId.
3. Allocate a small block of memory (such as 256k), read the entire file sequentially, parse and sort the MapId, and transfer the block of the original file to the new file in the order of MapId.

From an IO perspective, at first glance, solution 1 uses sufficient memory and there is no sequential reading and writing; solution 2 has random reading and random writing; solution 3 has sequential writing. Intuitively solution 1 has better performance. Due to the existence of PageCache, when writing a file in solution 3, the original file is likely to be cached in PageCache. `PartitionFilesSorter` support solution3 with PageCache at present, which has better performance especially HDD disk. It's better to support solution2 with switch config that seeks to position of each block and does not warm up for non-hdfs files especially SDD disk.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

GA and cluster.

Closes #2264 from SteNicholas/CELEBORN-1254.

Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: SteNicholas <programgeek@163.com>
2024-01-26 19:20:06 +08:00
Angerszhuuuu
3ffed66c40 [CELEBORN-1236][METRICS] Celeborn add metrics about thread pool
### What changes were proposed in this pull request?
Add metrics about worker's thread pool, help admin to observe the thread pool's work status.

ThreadPool list as below:

1. celeborn-dispatcher
2. celeborn-netty-rpc-connection-executor
3. worker-disk-{mount_point}-cleaner
4. worker-device-checker
5. flusher-{mount_point}
6. worker-file-sorter-executor
7. worker-data-replicator
8. worker-files-committer
9. worker-expired-shuffle-cleaner

```
metrics_active_thread_count_Value{role="Worker",threadPool="celeborn-dispatcher"} 64 1706237338484
metrics_pending_task_count_Value{role="Worker",threadPool="celeborn-dispatcher"} 0 1706237338484
metrics_pool_size_Value{role="Worker",threadPool="celeborn-dispatcher"} 64 1706237338484
metrics_core_pool_size_Value{role="Worker",threadPool="celeborn-dispatcher"} 64 1706237338484
metrics_maximum_pool_size_Value{role="Worker",threadPool="celeborn-dispatcher"} 64 1706237338484
metrics_largest_pool_size_Value{role="Worker",threadPool="celeborn-dispatcher"} 64 1706237338484
metrics_active_thread_count_Value{role="Worker",threadPool="celeborn-netty-rpc-connection-executor"} 0 1706237338484
metrics_pending_task_count_Value{role="Worker",threadPool="celeborn-netty-rpc-connection-executor"} 0 1706237338484
metrics_pool_size_Value{role="Worker",threadPool="celeborn-netty-rpc-connection-executor"} 0 1706237338484
metrics_core_pool_size_Value{role="Worker",threadPool="celeborn-netty-rpc-connection-executor"} 64 1706237338484
metrics_maximum_pool_size_Value{role="Worker",threadPool="celeborn-netty-rpc-connection-executor"} 64 1706237338484
metrics_largest_pool_size_Value{role="Worker",threadPool="celeborn-netty-rpc-connection-executor"} 1 1706237338484
metrics_active_thread_count_Value{role="Worker",threadPool="worker-disk-/-cleaner"} 0 1706237338484
metrics_pending_task_count_Value{role="Worker",threadPool="worker-disk-/-cleaner"} 0 1706237338484
metrics_pool_size_Value{role="Worker",threadPool="worker-disk-/-cleaner"} 0 1706237338484
metrics_core_pool_size_Value{role="Worker",threadPool="worker-disk-/-cleaner"} 4 1706237338484
metrics_maximum_pool_size_Value{role="Worker",threadPool="worker-disk-/-cleaner"} 4 1706237338484
metrics_largest_pool_size_Value{role="Worker",threadPool="worker-disk-/-cleaner"} 0 1706237338485
metrics_active_thread_count_Value{role="Worker",threadPool="worker-device-checker"} 0 1706237338485
metrics_pending_task_count_Value{role="Worker",threadPool="worker-device-checker"} 0 1706237338485
metrics_pool_size_Value{role="Worker",threadPool="worker-device-checker"} 2 1706237338485
metrics_core_pool_size_Value{role="Worker",threadPool="worker-device-checker"} 5 1706237338485
metrics_maximum_pool_size_Value{role="Worker",threadPool="worker-device-checker"} 5 1706237338485
metrics_largest_pool_size_Value{role="Worker",threadPool="worker-device-checker"} 2 1706237338485
metrics_thread_count_Value{role="Worker",threadPool="LocalFlusher1441328175-/"} 2 1706237338485
metrics_thread_is_terminated_count_Value{role="Worker",threadPool="LocalFlusher1441328175-/"} 0 1706237338485
metrics_thread_is_shutdown_count_Value{role="Worker",threadPool="LocalFlusher1441328175-/"} 0 1706237338485
metrics_active_thread_count_Value{role="Worker",threadPool="worker-file-sorter-executor"} 0 1706237338485
metrics_pending_task_count_Value{role="Worker",threadPool="worker-file-sorter-executor"} 0 1706237338485
metrics_pool_size_Value{role="Worker",threadPool="worker-file-sorter-executor"} 0 1706237338485
metrics_core_pool_size_Value{role="Worker",threadPool="worker-file-sorter-executor"} 24 1706237338485
metrics_maximum_pool_size_Value{role="Worker",threadPool="worker-file-sorter-executor"} 24 1706237338485
metrics_largest_pool_size_Value{role="Worker",threadPool="worker-file-sorter-executor"} 0 1706237338485
metrics_active_thread_count_Value{role="Worker",threadPool="worker-data-replicator"} 0 1706237338485
metrics_pending_task_count_Value{role="Worker",threadPool="worker-data-replicator"} 0 1706237338485
metrics_pool_size_Value{role="Worker",threadPool="worker-data-replicator"} 0 1706237338485
metrics_core_pool_size_Value{role="Worker",threadPool="worker-data-replicator"} 64 1706237338485
metrics_maximum_pool_size_Value{role="Worker",threadPool="worker-data-replicator"} 64 1706237338485
metrics_largest_pool_size_Value{role="Worker",threadPool="worker-data-replicator"} 0 1706237338485
metrics_active_thread_count_Value{role="Worker",threadPool="worker-files-committer"} 0 1706237338485
metrics_pending_task_count_Value{role="Worker",threadPool="worker-files-committer"} 0 1706237338485
metrics_pool_size_Value{role="Worker",threadPool="worker-files-committer"} 0 1706237338485
metrics_core_pool_size_Value{role="Worker",threadPool="worker-files-committer"} 32 1706237338485
metrics_maximum_pool_size_Value{role="Worker",threadPool="worker-files-committer"} 32 1706237338485
metrics_largest_pool_size_Value{role="Worker",threadPool="worker-files-committer"} 0 1706237338485
metrics_active_thread_count_Value{role="Worker",threadPool="worker-expired-shuffle-cleaner"} 0 1706237338485
metrics_pending_task_count_Value{role="Worker",threadPool="worker-expired-shuffle-cleaner"} 0 1706237338485
metrics_pool_size_Value{role="Worker",threadPool="worker-expired-shuffle-cleaner"} 2 1706237338485
metrics_core_pool_size_Value{role="Worker",threadPool="worker-expired-shuffle-cleaner"} 64 1706237338485
metrics_maximum_pool_size_Value{role="Worker",threadPool="worker-expired-shuffle-cleaner"} 64 1706237338485
metrics_largest_pool_size_Value{role="Worker",threadPool="worker-expired-shuffle-cleaner"} 2 1706237338485
```
### Why are the changes needed?
Help observe server status

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
MT

Closes #2239 from AngersZhuuuu/CLEBORN-1236.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Angerszhuuuu <angers.zhu@gmail.com>
2024-01-26 18:14:05 +08:00
Angerszhuuuu
5c54388bc2
[CELEBORN-1252] Fix resource consumption of worker does not update when update interval is greater than heartbeat interval
### What changes were proposed in this pull request?

 Resource consumption of worker does not update when update interval of resource consumpution is greater than heartbeat interval.

<img width="1741" alt="截屏2024-01-24 14 49 50" src="https://github.com/apache/incubator-celeborn/assets/46485123/21cfd412-c69e-4955-8bc8-155ee470697d">

This pull request introduces below changes:

1. Avoid master repeat add gauge for same user
2. For worker, user resource consumption can directly get from worker's snapshot, didn't need update interval

### Why are the changes needed?

No.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

No.

Closes #2260 from AngersZhuuuu/CELEBORN-1252.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
2024-01-25 20:28:19 +08:00
Angerszhuuuu
4709251bb4
[CELEBORN-1246] Introduce OpenStreamSuccessCount, FetchChunkSuccessCount and WriteDataSuccessCount metric to expose the count of opening stream, fetching chunk and writing data successfully
### What changes were proposed in this pull request?

Introduce `OpenStreamSuccessCount`, `FetchChunkSuccessCount` and `WriteDataSuccessCount` metric to expose the count of opening stream, fetching chunk and writing data successfully in current worker.

### Why are the changes needed?

The ratio of opening stream, fetching chunk and writing data failed is important for Celeborn performance to balance the healty of cluster, which is lack of the count of opening stream, fetching chunk and writing data successfully.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

No.

Closes #2252 from AngersZhuuuu/CELEBORN-1246.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
2024-01-24 10:44:28 +08:00
Angerszhuuuu
67e6cbfb51
[CELEBORN-1242] Unify celeborn thread name format
### What changes were proposed in this pull request?

Unify celeborn thread name format with the following pattern:

- client: `celeborn-client-[component]-[function]er`
- service: `[master|worker]-[component]-[function]er`
- other: `celeborn-[component]-[function]er`

### Why are the changes needed?

It's recommended to unify celeborn thread name format especially client side for application.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

No.

Closes #2248 from AngersZhuuuu/CELEBORN-1242.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
2024-01-23 16:56:40 +08:00
Angerszhuuuu
a224f713b4 [CELEBORN-1226] Unify creation of thread using ThreadUtils
### What changes were proposed in this pull request?
Make all single thread use standard ThreadUtils to simplify the code

### Why are the changes needed?

### Does this PR introduce _any_ user-facing change?

### How was this patch tested?

Closes #2229 from AngersZhuuuu/CELEBORN-1226.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Angerszhuuuu <angers.zhu@gmail.com>
2024-01-22 12:09:22 +08:00
xianminglei
ef47645b7a
[CELEBORN-1238] deviceCheckThreadPool is only initialized when diskCheck is enabled
### What changes were proposed in this pull request?
deviceCheckThreadPool is only initialized when diskCheck is enabled

### Why are the changes needed?
deviceCheckThreadPool is only initialized when diskCheck is enabled

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Existing UTs.

Closes #2242 from leixm/issue_1238.

Authored-by: xianminglei <xianming.lei@shopee.com>
Signed-off-by: Cheng Pan <chengpan@apache.org>
2024-01-19 17:36:28 +08:00
xianminglei
b90fb1fdb2 [CELEBORN-1237][METRICS] Refactor metrics name
### What changes were proposed in this pull request?
Refactor metrics name.

### Why are the changes needed?
Easier to understand the meaning of metrics

### Does this PR introduce _any_ user-facing change?
METRICS.md
migration.md
monitoring.md

### How was this patch tested?
Existing UTs.

Closes #2240 from leixm/metrics_name.

Authored-by: xianminglei <xianming.lei@shopee.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2024-01-18 18:15:43 +08:00
zky.zhoukeyong
749a0fa439 [CELEBORN-1233] Treat unfound PartitionLocation as failed in Controller#commitFiles
### What changes were proposed in this pull request?
I tested 1T TPCDS with the following Celeborn 8-worker cluster setup:
1. Workers have fixed ports for rpc/push/replicate
2. `spark.celeborn.client.spark.fetch.throwsFetchFailure` is enabled
3. graceful shutdown is enabled

I randomly kill -9 and ./sbin/stop-worker.sh (both graceful shutdown and non-graceful shutdown) some workers and start it immediately. Then I encountered result incorrectness with low probability (1 out of 99 queries).

After digging into it, I found the reason is as follows:
1. At time T1, all workers are serving shuffle 602
2. At time T2, I run stop-worker.sh for worker2, and then run kill -9 and start worker1. Since the workers are configured with fixed ports, clients think they are OK and Master let them re-register, which will also success. And worker2 is clean in memory.
4. At time T3, push requests to worker2 fails and revives on worker1, so worker1 has reservation for shuffle 602. Then I start worker2.
5. At time T4, LifecycleManager sends CommitFiles to all workers, on worker1, it just logs that some PartitionLocations
    don't exist and ignores them.
6. CommitFiles success, but worker1 loses some data before restarting, and no error happens.

The following snapshot shows the process.

![image](https://github.com/apache/incubator-celeborn/assets/948245/9ef1a1ff-bb26-420a-929c-70c9476ec700)

This PR fixes this by treating unfound PartitionLocations as failed when handling CommitFiles.

### Why are the changes needed?
ditto

### Does this PR introduce _any_ user-facing change?
No.

### How was this patch tested?
Manual test

Closes #2235 from waitinfuture/1233.

Authored-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2024-01-18 11:29:44 +08:00
SteNicholas
30608ea698
[CELEBORN-1225] Worker should build replicate factory to get client for sending replicate data
### What changes were proposed in this pull request?

`PushDataHandler` should build replicate factory to get client for sending replicate data instead of push client factory.

### Why are the changes needed?

`PushDataHandler` uses push client factory to create client for replicating, which should use replicate factory, otherwise replicate module configuration does not take effect for replicating of worker server.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

GA and cluster.

Closes #2232 from SteNicholas/CELEBORN-1225.

Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
2024-01-17 16:40:46 +08:00
Fei Wang
d46b6623b3
[CELEBORN-1228] Format the timestamp when recording worker failure
### What changes were proposed in this pull request?

Format the timestamp when recoding worker failure inforamtion.

### Why are the changes needed?

Now the long type timestamp is difficult to view and confuse without reading source code.

### Does this PR introduce _any_ user-facing change?

### How was this patch tested?

Closes #2230 from turboFei/date_format.

Authored-by: Fei Wang <fwang12@ebay.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
2024-01-17 14:04:30 +08:00
mingji
413e4f3b80 [CELEBORN-1222] Fix Celeborn worker won't record HDFS writer
### What changes were proposed in this pull request?
To record hdfs writer in worker.
To fix a bug introduced by https://github.com/apache/incubator-celeborn/pull/2130.

### Why are the changes needed?
If the hdfs writer won't be recorded, the worker won't clean the HDFS shuffle file if a partition is broken until the master cleans the HDFS.

### Does this PR introduce _any_ user-facing change?
NO.

### How was this patch tested?
GA and cluster.

Closes #2227 from FMX/b1222.

Authored-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2024-01-16 10:08:03 +08:00
Angerszhuuuu
1495fca69b
[CELEBORN-1219] takeBuffer() avoid checking source.metricsCollectCriticalEnabled twice
### What changes were proposed in this pull request?
takeBuffer() avoid checking source.metricsCollectCriticalEnabled twice

### Why are the changes needed?

### Does this PR introduce _any_ user-facing change?

### How was this patch tested?

Closes #2223 from AngersZhuuuu/CELEBORN-1219.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
2024-01-15 16:50:40 +08:00
Angerszhuuuu
cbdabf4ae6
[CELEBORN-1220][IMPROVEMENT] Make trim logic more robust
### What changes were proposed in this pull request?
We meet a case that trim action stoped but didn't set trimInProcess back, then the worker won't trigger a new trim and pause push data, already pushed data(replicate data can work well) won't release, then won't recover receive push data request.

This pr make the logic more robust

### Why are the changes needed?
Make logic more robust

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
No

Closes #2224 from AngersZhuuuu/CELEBORN-1220.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
2024-01-15 14:22:03 +08:00
SteNicholas
7bc34972ac [CELEBORN-1177] OpenStream should register stream via ChunkStreamManager to close stream for ReusedExchange
### What changes were proposed in this pull request?

`OpenStream` should register stream via `ChunkStreamManager`, which is served to obtain disk file to close stream for `ReusedExchange` operator.

Follow up #1932.

### Why are the changes needed?

`OpenStream` does not register chunk stream for reading local or dfs shuffle. Therefore `LocalPartitionReader` and `DfsPartitionReader` could not obtain the disk file from `ChunkStreamManager` that causes the below `NullPointerException` for closing stream.
```
ERROR [fetch-server-11-11] TransportRequestHandler: Error while invoking handler#receive() on RPC id 4
java.lang.NullPointerException
        at org.apache.celeborn.service.deploy.worker.storage.ChunkStreamManager.getShuffleKeyAndFileName(ChunkStreamManager.java:188)
        at org.apache.celeborn.service.deploy.worker.FetchHandler.handleEndStreamFromClient(FetchHandler.scala:344)
        at org.apache.celeborn.service.deploy.worker.FetchHandler.handleRpcRequest(FetchHandler.scala:137)
        at org.apache.celeborn.service.deploy.worker.FetchHandler.receive(FetchHandler.scala:94)
        at org.apache.celeborn.common.network.server.TransportRequestHandler.processRpcRequest(TransportRequestHandler.java:96)
        at org.apache.celeborn.common.network.server.TransportRequestHandler.handle(TransportRequestHandler.java:84)
        at org.apache.celeborn.common.network.server.TransportChannelHandler.channelRead(TransportChannelHandler.java:151)
        at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:444)
        at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
        at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:412)
        at io.netty.handler.timeout.IdleStateHandler.channelRead(IdleStateHandler.java:286)
        at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:442)
        at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
        at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:412)
        at org.apache.celeborn.common.network.util.TransportFrameDecoder.channelRead(TransportFrameDecoder.java:74)
        at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:444)
        at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
        at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:412)
        at io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1410)
        at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:440)
        at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
        at io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:919)
        at io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:166)
        at io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:788)
        at io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:724)
        at io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:650)
        at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:562)
        at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:997)
        at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
        at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
        at java.lang.Thread.run(Thread.java:745)
```
In summary, `FetchHandler` only closes stream registered via `ChunkStreamManager`. `LocalPartitionReader` and `DfsPartitionReader` should use `ChunkStreamManager#registerStream` to close stream for deleting original unsorted disk file in `ReusedExchange` operator.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

- `FetchHandlerSuiteJ#testLocalReadSortFileOnceOriginalFileBeDeleted`
- `FetchHandlerSuiteJ#testDoNotDeleteOriginalFileWhenNonRangeLocalReadWorkInProgress`
- `ReusedExchangeSuite`

Closes #2209 from SteNicholas/CELEBORN-1177.

Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: Fu Chen <cfmcgrady@gmail.com>
2024-01-12 00:26:59 +08:00
sychen
5f0a8406c6
[CELEBORN-1133][FOLLOWUP] Refactor FileInfo
### What changes were proposed in this pull request?

### Why are the changes needed?
```
common/src/main/java/org/apache/celeborn/common/meta/DiskFileInfo.java:[110,14] [MissingOverride] getFileLength implements method in FileInfo; expected Override
common/src/main/java/org/apache/celeborn/common/meta/MapFileMeta.java:[40,38] [InconsistentCapitalization] Found the field 'numSubPartitions' with the same name as the parameter 'numSubpartitions' but with different capitalization.
worker/src/main/java/org/apache/celeborn/service/deploy/worker/storage/MapPartitionDataWriter.java:[164,65] [UnnecessaryParentheses] These grouping parentheses are unnecessary; it is unlikely the code will be misinterpreted without them
```

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
GA

Closes #2222 from cxzl25/CELEBORN-1133-FOLLOWUP.

Authored-by: sychen <sychen@ctrip.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
2024-01-11 11:09:15 +08:00
SteNicholas
4b5e23db37
[CELEBORN-1215] Introduce PausePushDataAndReplicateTime metric to record time for a worker to stop receiving pushData from clients and other workers
### What changes were proposed in this pull request?

Introduce `PausePushDataAndReplicateTime` metric to record time for a worker to stop receiving pushData from clients and other workers.

### Why are the changes needed?

`PausePushData` means the count for a worker to stop receiving pushData from clients because of back pressure. Meanwhile, `PausePushDataAndReplicate` means the count for a worker to stop receiving pushData from clients and other workers because of back pressure. Therefore,`PausePushDataTime` records the time for a worker to stop receiving pushData from clients or other workers, of which definition is confusing for users. It's recommended that `PausePushDataAndReplicateTime` metric is introduced that means the time for a worker to stop receiving pushData from clients and other workers because of back pressure.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

- [Celeborn Dashboard](https://stenicholas.grafana.net/d/U_qgru_7z/celeborn?orgId=1&refresh=5s)
- `MemoryManagerSuite#[CELEBORN-882] Test MemoryManager check memory thread logic`

Closes #2221 from SteNicholas/CELEBORN-1215.

Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
2024-01-10 19:55:04 +08:00
SteNicholas
0cd1291f6c
[CELEBORN-1214] Introduce WriteDataHardSplitCount metric to record HARD_SPLIT partitions of PushData and PushMergedData
### What changes were proposed in this pull request?

Introduce `WriteDataHardSplitCount` metric to record `HARD_SPLIT` partitions of PushData and PushMergedData.

### Why are the changes needed?

As the log level of `PushDataHandler#handlePushData` and `PushDataHandler#handlePushMergedData` use the DEBUG level, `WriteDataHardSplitCount` metric shoud be introduced to record HARD_SPLIT partitions of PushData and PushMergedData for `PushDataHandler`.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

[Celeborn Dashboard](https://stenicholas.grafana.net/d/U_qgru_7z/celeborn?orgId=1&refresh=5s)

Closes #2217 from SteNicholas/CELEBORN-1214.

Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
2024-01-09 21:54:53 +08:00
SteNicholas
9201cc79b0
[MINOR] Update log level of PushData and PushMergedData for PushDataHandler from info to debug
### What changes were proposed in this pull request?

Update log level of `PushData` and `PushMergedData` for `PushDataHandler` from info to debug.

### Why are the changes needed?

In production practice, there are many logs for `PushDataHandler#handlePushData` and `PushDataHandler#handlePushMergedData`, which are unnecessary to use info level to print.

```
# cat worker-stderr.log.2024_01_09_08 |grep "INFO"|grep "hard split"|wc -l
10914811
```

```
2024-01-09 11:01:16,082 [INFO] [push-server-6-37] - org.apache.celeborn.service.deploy.worker.PushDataHandler -Logging.scala(51) -[Case1] Receive push merged data for committed hard split partition of (shuffle application_1650016801129_24552188_1-1, map 40707 attempt 0)
2024-01-09 11:01:16,082 [INFO] [push-server-6-10] - org.apache.celeborn.service.deploy.worker.PushDataHandler -Logging.scala(51) -[Case1] Receive push merged data for committed hard split partition of (shuffle application_1703497903483_735656_1-1, map 23739 attempt 0)
2024-01-09 11:01:16,083 [INFO] [push-server-6-37] - org.apache.celeborn.service.deploy.worker.PushDataHandler -Logging.scala(51) -[Case1] Receive push merged data for committed hard split partition of (shuffle application_1650016801129_24552188_1-1, map 40775 attempt 0)
2024-01-09 11:01:16,084 [INFO] [push-server-6-25] - org.apache.celeborn.service.deploy.worker.PushDataHandler -Logging.scala(51) -[Case1] Receive push merged data for committed hard split partition of (shuffle application_1650016801129_24552188_1-1, map 41286 attempt 0)
2024-01-09 11:01:16,085 [INFO] [push-server-6-19] - org.apache.celeborn.service.deploy.worker.PushDataHandler -Logging.scala(51) -[Case1] Receive push merged data for committed hard split partition of (shuffle application_1703497903483_735656_1-1, map 24123 attempt 0)
2024-01-09 11:01:16,085 [INFO] [push-server-6-37] - org.apache.celeborn.service.deploy.worker.PushDataHandler -Logging.scala(51) -[Case1] Receive push merged data for committed hard split partition of (shuffle application_1650016801129_24552188_1-1, map 41429 attempt 0)
2024-01-09 11:01:16,086 [INFO] [push-server-6-8] - org.apache.celeborn.service.deploy.worker.PushDataHandler -Logging.scala(51) -[Case1] Receive push merged data for committed hard split partition of (shuffle application_1650016801129_24552188_1-1, map 41512 attempt 0)
2024-01-09 11:01:16,088 [INFO] [push-server-6-59] - org.apache.celeborn.service.deploy.worker.PushDataHandler -Logging.scala(51) -[Case1] Receive push merged data for committed hard split partition of (shuffle application_1650016801129_24552188_1-1, map 41512 attempt 0)
2024-01-09 11:01:16,088 [INFO] [push-server-6-21] - org.apache.celeborn.service.deploy.worker.PushDataHandler -Logging.scala(51) -[Case1] Receive push merged data for committed hard split partition of (shuffle application_1703497903483_735656_1-1, map 24075 attempt 0)
2024-01-09 11:01:16,088 [INFO] [push-server-6-8] - org.apache.celeborn.service.deploy.worker.PushDataHandler -Logging.scala(51) -[Case1] Receive push merged data for committed hard split partition of (shuffle application_1650016801129_24552188_1-1, map 41512 attempt 0)
2024-01-09 11:01:16,090 [INFO] [push-server-6-21] - org.apache.celeborn.service.deploy.worker.PushDataHandler -Logging.scala(51) -[Case1] Receive push merged data for committed hard split partition of (shuffle application_1703497903483_735656_1-1, map 24075 attempt 0)
2024-01-09 11:01:16,090 [INFO] [push-server-6-35] - org.apache.celeborn.service.deploy.worker.PushDataHandler -Logging.scala(51) -[Case1] Receive push merged data for committed hard split partition of (shuffle application_1650016801129_24552188_1-1, map 41523 attempt 0)
2024-01-09 11:01:16,090 [INFO] [push-server-6-62] - org.apache.celeborn.service.deploy.worker.PushDataHandler -Logging.scala(51) -[Case1] Receive push merged data for committed hard split partition of (shuffle application_1703497903483_735656_1-1, map 22060 attempt 0)
2024-01-09 11:01:16,091 [INFO] [push-server-6-56] - org.apache.celeborn.service.deploy.worker.PushDataHandler -Logging.scala(51) -[Case1] Receive push merged data for committed hard split partition of (shuffle application_1703497903483_735656_1-1, map 21910 attempt 0)
2024-01-09 11:01:16,091 [INFO] [push-server-6-55] - org.apache.celeborn.service.deploy.worker.PushDataHandler -Logging.scala(51) -[Case1] Receive push merged data for committed hard split partition of (shuffle application_1703497903483_735656_1-1, map 21910 attempt 0)
2024-01-09 11:01:16,092 [INFO] [push-server-6-56] - org.apache.celeborn.service.deploy.worker.PushDataHandler -Logging.scala(51) -[Case1] Receive push merged data for committed hard split partition of (shuffle application_1703497903483_735656_1-1, map 21910 attempt 0)
2024-01-09 11:01:16,092 [INFO] [push-server-6-55] - org.apache.celeborn.service.deploy.worker.PushDataHandler -Logging.scala(51) -[Case1] Receive push merged data for committed hard split partition of (shuffle application_1703497903483_735656_1-1, map 21910 attempt 0)
2024-01-09 11:01:16,092 [INFO] [push-server-6-55] - org.apache.celeborn.service.deploy.worker.PushDataHandler -Logging.scala(51) -[Case1] Receive push merged data for committed hard split partition of (shuffle application_1703497903483_735656_1-1, map 21910 attempt 0)
2024-01-09 11:01:16,092 [INFO] [push-server-6-56] - org.apache.celeborn.service.deploy.worker.PushDataHandler -Logging.scala(51) -[Case1] Receive push merged data for committed hard split partition of (shuffle application_1703497903483_735656_1-1, map 21910 attempt 0)
2024-01-09 11:01:16,092 [INFO] [push-server-6-23] - org.apache.celeborn.service.deploy.worker.PushDataHandler -Logging.scala(51) -[Case1] Receive push merged data for committed hard split partition of (shuffle application_1703497903483_735656_1-1, map 21747 attempt 0)
2024-01-09 11:01:16,092 [INFO] [push-server-6-56] - org.apache.celeborn.service.deploy.worker.PushDataHandler -Logging.scala(51) -[Case1] Receive push merged data for committed hard split partition of (shuffle application_1703497903483_735656_1-1, map 21910 attempt 0)
2024-01-09 11:01:16,093 [INFO] [push-server-6-56] - org.apache.celeborn.service.deploy.worker.PushDataHandler -Logging.scala(51) -[Case1] Receive push merged data for committed hard split partition of (shuffle application_1703497903483_735656_1-1, map 21910 attempt 0)
2024-01-09 11:01:16,093 [INFO] [push-server-6-21] - org.apache.celeborn.service.deploy.worker.PushDataHandler -Logging.scala(51) -[Case1] Receive push merged data for committed hard split partition of (shuffle application_1703497903483_735656_1-1, map 21329 attempt 0)
2024-01-09 11:01:16,093 [INFO] [push-server-6-23] - org.apache.celeborn.service.deploy.worker.PushDataHandler -Logging.scala(51) -[Case1] Receive push merged data for committed hard split partition of (shuffle application_1703497903483_735656_1-1, map 21747 attempt 0)
2024-01-09 11:01:16,093 [INFO] [push-server-6-21] - org.apache.celeborn.service.deploy.worker.PushDataHandler -Logging.scala(51) -[Case1] Receive push merged data for committed hard split partition of (shuffle application_1703497903483_735656_1-1, map 21327 attempt 0)
2024-01-09 11:01:16,093 [INFO] [push-server-6-23] - org.apache.celeborn.service.deploy.worker.PushDataHandler -Logging.scala(51) -[Case1] Receive push merged data for committed hard split partition of (shuffle application_1703497903483_735656_1-1, map 21747 attempt 0)
2024-01-09 11:01:16,094 [INFO] [push-server-6-41] - org.apache.celeborn.service.deploy.worker.PushDataHandler -Logging.scala(51) -[Case1] Receive push merged data for committed hard split partition of (shuffle application_1703497903483_735656_1-1, map 23648 attempt 0)
2024-01-09 11:01:16,095 [INFO] [push-server-6-41] - org.apache.celeborn.service.deploy.worker.PushDataHandler -Logging.scala(51) -[Case1] Receive push merged data for committed hard split partition of (shuffle application_1703497903483_735656_1-1, map 21747 attempt 0)
2024-01-09 11:01:16,096 [INFO] [push-server-6-22] - org.apache.celeborn.service.deploy.worker.PushDataHandler -Logging.scala(51) -[Case1] Receive push merged data for committed hard split partition of (shuffle application_1650016801129_24552188_1-1, map 41716 attempt 0)
2024-01-09 11:01:16,096 [INFO] [push-server-6-8] - org.apache.celeborn.service.deploy.worker.PushDataHandler -Logging.scala(51) -[Case1] Receive push merged data for committed hard split partition of (shuffle application_1650016801129_24552188_1-1, map 40178 attempt 0)
2024-01-09 11:01:16,096 [INFO] [push-server-6-23] - org.apache.celeborn.service.deploy.worker.PushDataHandler -Logging.scala(51) -[Case1] Receive push merged data for committed hard split partition of (shuffle application_1703497903483_735656_1-1, map 23648 attempt 0)
2024-01-09 11:01:16,097 [INFO] [push-server-6-2] - org.apache.celeborn.service.deploy.worker.PushDataHandler -Logging.scala(51) -[Case1] Receive push merged data for committed hard split partition of (shuffle application_1703497903483_735656_1-1, map 21329 attempt 0)
2024-01-09 11:01:16,098 [INFO] [push-server-6-56] - org.apache.celeborn.service.deploy.worker.PushDataHandler -Logging.scala(51) -[Case1] Receive push merged data for committed hard split partition of (shuffle application_1703497903483_735656_1-1, map 21910 attempt 0)
2024-01-09 11:01:16,098 [INFO] [push-server-6-23] - org.apache.celeborn.service.deploy.worker.PushDataHandler -Logging.scala(51) -[Case1] Receive push merged data for committed hard split partition of (shuffle application_1703497903483_735656_1-1, map 23827 attempt 1)
2024-01-09 11:01:16,099 [INFO] [push-server-6-41] - org.apache.celeborn.service.deploy.worker.PushDataHandler -Logging.scala(51) -[Case1] Receive push merged data for committed hard split partition of (shuffle application_1703497903483_735656_1-1, map 23827 attempt 1)
2024-01-09 11:01:16,099 [INFO] [push-server-6-14] - org.apache.celeborn.service.deploy.worker.PushDataHandler -Logging.scala(51) -[Case1] Receive push merged data for committed hard split partition of (shuffle application_1650016801129_24552188_1-1, map 40057 attempt 0)
2024-01-09 11:01:16,103 [INFO] [push-server-6-8] - org.apache.celeborn.service.deploy.worker.PushDataHandler -Logging.scala(51) -[Case1] Receive push merged data for committed hard split partition of (shuffle application_1650016801129_24552188_1-1, map 41276 attempt 0)
2024-01-09 11:01:16,104 [INFO] [push-server-6-55] - org.apache.celeborn.service.deploy.worker.PushDataHandler -Logging.scala(51) -[Case1] Receive push merged data for committed hard split partition of (shuffle application_1703497903483_735656_1-1, map 22326 attempt 0)
2024-01-09 11:01:16,105 [INFO] [push-server-6-37] - org.apache.celeborn.service.deploy.worker.PushDataHandler -Logging.scala(51) -[Case1] Receive push merged data for committed hard split partition of (shuffle application_1650016801129_24552188_1-1, map 40371 attempt 0)
2024-01-09 11:01:16,106 [INFO] [push-server-6-14] - org.apache.celeborn.service.deploy.worker.PushDataHandler -Logging.scala(51) -[Case1] Receive push merged data for committed hard split partition of (shuffle application_1703497903483_735656_1-1, map 22140 attempt 0)
2024-01-09 11:01:16,106 [INFO] [push-server-6-19] - org.apache.celeborn.service.deploy.worker.PushDataHandler -Logging.scala(51) -[Case1] Receive push merged data for committed hard split partition of (shuffle application_1650016801129_24552188_1-1, map 41825 attempt 0)
2024-01-09 11:01:16,109 [INFO] [push-server-6-5] - org.apache.celeborn.service.deploy.worker.PushDataHandler -Logging.scala(51) -[Case1] Receive push merged data for committed hard split partition of (shuffle application_1703497903483_735656_1-1, map 22033 attempt 0)
2024-01-09 11:01:16,109 [INFO] [push-server-6-53] - org.apache.celeborn.service.deploy.worker.PushDataHandler -Logging.scala(51) -[Case1] Receive push merged data for committed hard split partition of (shuffle application_1703497903483_735656_1-1, map 22033 attempt 0)
2024-01-09 11:01:16,109 [INFO] [push-server-6-21] - org.apache.celeborn.service.deploy.worker.PushDataHandler -Logging.scala(51) -[Case1] Receive push merged data for committed hard split partition of (shuffle application_1703497903483_735656_1-1, map 21327 attempt 0)
```

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

No.

Closes #2215 from SteNicholas/handle-push-data-log.

Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: Cheng Pan <chengpan@apache.org>
2024-01-09 15:27:50 +08:00
Cheng Pan
4fae840513
[CELEBORN-1210] Fix potential memory leak in PartitionFilesCleaner
### What changes were proposed in this pull request?

1. let `cleanupExpiredShuffleKey` holds the same lock with add and cleaner thread
2. fix the removal during the iteration
3. simply call `condition.await()` instead of while loop with `condition.await(500ms)`

### Why are the changes needed?

The usage of `LinkedBlockingQueue queue` in `PartitionFilesCleaner` is not a typical producer-consumer model, but an order-agnostic buffer, which confuses me on the first round code reading.

Though `LinkedBlockingQueue` is a thread-safe collection, it means won't get a `ConcurrentModificationException` if the queue is modified while it is iterating, but the iteration is not guaranteed to see all queue entries. As `cleanupExpiredShuffleKey` is not guarded by the lock, elements removal from the cleaner thread may break the iteration and eventually cause a memory leak.

Ref: https://stackoverflow.com/questions/37945981/concurrently-iterating-over-a-blockingqueue

Another issue is the removal inside iteration. The typical usage is
```
Iterator itr = collection.listIterator();
while (itr.hasNext()) {
    if (itr.next() xxx condition) {
        itr.remove();
    }
}
```
The keypoint is that `itr.remove()` should be called instead of `collection.remove(x)`

### Does this PR introduce _any_ user-facing change?

I suppose there is a potential memory leak issue on the Worker without this patch.

### How was this patch tested?

Pass GA.

Closes #2205 from pan3793/CELEBORN-1210.

Authored-by: Cheng Pan <chengpan@apache.org>
Signed-off-by: Cheng Pan <chengpan@apache.org>
2024-01-05 18:12:18 +08:00
SteNicholas
29e930488b
[CELEBORN-1100] Introduce ChunkStreamCount, OpenStreamFailCount metrics about opening stream of FetchHandler
### What changes were proposed in this pull request?

Introduces `ChunkStreamCount`, `OpenStreamFailCount` metrics about opening stream of `FetchHandler`:

- `WorkerSource` adds `ChunkStreamCount`, `OpenStreamFailCount` metrics.
- Corrects the grafana dashboard of `celeborn-dashboard.json`. `celeborn-dashboard.json` has been verified via [Celeborn Dashboard](https://stenicholas.grafana.net/d/U_qgru_7z/celeborn?orgId=1&refresh=5s). For example:
  1. `"expr": "metrics_RunningApplicationCount_Value"`
  2. Moves the panel positition of `FetchChunkFailCount` to `FetchRelatives` instead of `PushRelatives`.
  3. Updates the `gridPos` of some panels.

### Why are the changes needed?

There are no any metrics about opening stream of `FetchHandler` for Celeborn Worker.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

[Celeborn Dashboard](https://stenicholas.grafana.net/d/U_qgru_7z/celeborn?orgId=1&refresh=5s)

Closes #2212 from SteNicholas/CELEBORN-1100.

Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
2024-01-05 17:05:35 +08:00
mingji
8b6aae04d1 [CELEBORN-1201] Optimize memory usage of cache in partition sorter
### What changes were proposed in this pull request?
Add a cache in partition sorted and limit its max size.

### Why are the changes needed?
To reduce memory consumption in partition sort by tweak the index cache.

### Does this PR introduce _any_ user-facing change?
NO.

### How was this patch tested?
GA and cluster.

Closes #2194 from FMX/B1201.

Lead-authored-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
Co-authored-by: Keyong Zhou <zky.zhoukeyong@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2024-01-04 11:34:48 +08:00
sychen
9f8a6bcb79 [CELEBORN-1190][FOLLOWUP] Apply error prone patch and suppress some problems
### What changes were proposed in this pull request?
1. Fix IntLongMath, InconsistentCapitalization, UnnecessaryAssignment
2. disable StringSplitter, EmptyBlockTag, EqualsGetClass, MissingSummary, BadImport

### Why are the changes needed?

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
GA

```bash
./build/make-distribution.sh --release
```

PR

total  202

```
  43 SynchronizeOnNonFinalField
  24 StaticAssignmentInConstructor
  21 JdkObsolete
  16 ThreadLocalUsage
  16 MutableConstantField
  16 MissingCasesInEnumSwitch
  11 UnusedMethod
  11 NonAtomicVolatileUpdate
   8 UnusedNestedClass
   8 NonOverridingEquals
   8 Finally
   4 MixedMutabilityReturnType
   4 DoubleBraceInitialization
   4 CatchAndPrintStackTrace
   4 CanonicalDuration
   2 ReferenceEquality
   1 ClassCanBeStatic
   1 ByteBufferBackingArray
```

Closes #2180 from cxzl25/error_prone_patch_followup.

Authored-by: sychen <sychen@ctrip.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2024-01-03 15:59:50 +08:00
mingji
7be05b430b [CELEBORN-1133] Refactor fileinfo
### What changes were proposed in this pull request?
Rename FileWriter to PartitionLocationDataWriter, add storageManager, delete fileinfo, and flusher in the constructor.

FileInfo(userIdentifier,partitionSplitEnabled,fileMeta)
– NonMemoryFileInfo(streams,filePath,storageType,bytesFlushed)
– MemoryFileInfo(length,buffer)

FileMeta
– reduceFileMeta(chunkOffsets,sorted)
– mapFileMeta(bufferSize,numSubPartitions)

### Why are the changes needed?
1. To make concepts more clear.
2. To support memory storage and HDFS slot management.

### Does this PR introduce _any_ user-facing change?
NO.

### How was this patch tested?
GA and cluster test with worker kill.

Closes #2130 from FMX/b1133.

Lead-authored-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
Co-authored-by: Keyong Zhou <zky.zhoukeyong@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2024-01-02 21:26:10 +08:00
SteNicholas
3097ffe33b [CELEBORN-678][FOLLOWUP] MapperAttempts for a shuffle should reply MAP_ENDED when mapper has already been ended from speculative task
### What changes were proposed in this pull request?

MapperAttempts for a shuffle replies the `MAP_ENDED` when mapper has already been ended for receving push data or push merged data from speculative task.

Follow up #1591.

### Why are the changes needed?

When mapper has already been ended for receving push data or push merged data from speculative task, `PushDataHandler` should trigger MapEnd instead of StageEnd for worker. Meanwhile, the `ShuffleClientImpl` should handle `STAGE_ENDED` as MapEnd, otherwise causes that other tasks of the stage could not send shuffle data for data lost.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Internal test.

Closes #2190 from SteNicholas/CELEBORN-678.

Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-12-27 20:40:40 +08:00
SteNicholas
e7e39a51be
[CELEBORN-1189] Introduce RunningApplicationCount metric and /applications API to record running applications of worker
### What changes were proposed in this pull request?

Introduce `RunningApplicationCount` metric and `/applications` API to record running applications for Celeborn worker.

### Why are the changes needed?

`RunningApplicationCount` metrics only monitors the count of running applications in the cluster for master. Meanwhile, `/listTopDiskUsedApps` API lists the top disk usage application ids for master and worker. Therefore `RunningApplicationCount` metric and `/applications` API could be introduced to record running applications of worker.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Internal tests.

Closes #2172 from SteNicholas/CELEBORN-1189.

Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
2023-12-27 09:51:16 +08:00
zwangsheng
6c2fdf7477
[CELEBORN-1188][TEST] Using JUnit function instead of java assert
### What changes were proposed in this pull request?
Using Junit function instead of java assert.

### Why are the changes needed?
When java assert fail, will throw AssertException, which is hard to find diff.

![截屏2023-12-20 10 34 52](https://github.com/apache/incubator-celeborn/assets/52876270/b36421a5-64e1-4717-a6d4-3b08db403293)

Instead, when we use junit assert, we can clearly find diff.

![截屏2023-12-20 11 17 21](https://github.com/apache/incubator-celeborn/assets/52876270/ce39fa20-e9ab-4419-a4ca-62c4157e4b2c)

### Does this PR introduce _any_ user-facing change?
NO, only test changed

### How was this patch tested?
Run CI

Closes #2173 from zwangsheng/CELEBORN-1188.

Authored-by: zwangsheng <binjieyang@apache.org>
Signed-off-by: Cheng Pan <chengpan@apache.org>
2023-12-20 21:20:38 +08:00
sychen
7f653ce7d6 [CELEBORN-1190] Apply error prone patch and suppress some problems
### What changes were proposed in this pull request?
1.  Fix MissingOverride, DefaultCharset, UnnecessaryParentheses Rule
2. Exclude generated sources, FutureReturnValueIgnored, TypeParameterUnusedInFormals, UnusedVariable

### Why are the changes needed?
```
./build/make-distribution.sh --release
```
We get a lot of WARNINGs.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
GA

Closes #2177 from cxzl25/error_prone_patch.

Authored-by: sychen <sychen@ctrip.com>
Signed-off-by: Fu Chen <cfmcgrady@gmail.com>
2023-12-20 20:54:18 +08:00
zky.zhoukeyong
e361788e48 [CELEBORN-1178] Destroy fail reserved slots in LifecycleManager#reserveSlotsWithRetry
### What changes were proposed in this pull request?
I'm testing main branch and encountered the following scenario.
I run `sbin/stop-worker.sh` near simultaneously on 3 out of 6 workers, and I'm expecting the 3 workers
will soon shutdown because I enabled graceful shutdown. However, only the first worker I stopped
shutdown in 15s as expected, the other two won't shutdown until shutdown timeout.

After digging into it, I found `LifecycleManager#reserveSlotsWithRetry` will reserve for the same location
twice:
1. At T1, only worker1 shutdown, pushes receive HARD_SPLIT and goes to revive
2. At T2, LifecycleManager handles revive requests in batch, and try to reallocate the locs to other workers
3. At T3, reserve to worker3 succeeds because it's not shutdown yet, but reserve to worker2 fails because it's shutdown
4. At T4, LifecycleManager will re-allocate the failed slots to other workers except worker1 and worker2. However, at this time Worker3 is also shutdown, so it fails to reserve on worker3
5. At T5, it re-allocates slots that failed to worker3. However, `getFailedPartitionLocations` will return slots allocated to worker3 in step 3, and increment the epoch to 2. At this time, worker3 has slots of epoch 1, but they will never to pushed to because newer epoch 3 is generated at the same time
6. Since the epoch 2 locs in worker3 will never be pushed to, it will never get a chance to return HARD_SPLIT, as a result it can't fast shutdown untile timeout.

This PR fixes this by destroying failed to be reserved slots in the process of `reserveSlotsWithRetry`

### Why are the changes needed?
ditto

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Manual test.

Before:
![image](https://github.com/apache/incubator-celeborn/assets/948245/50c55524-d37f-494e-a5aa-fba682438cda)
After:
![image](https://github.com/apache/incubator-celeborn/assets/948245/8c90a869-b388-46f3-a86b-a37fd0f4ce0f)

Closes #2163 from waitinfuture/1178.

Authored-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-12-17 14:28:04 +08:00
zky.zhoukeyong
309153a99b [CELEBORN-1175] Add UT for commit files
### What changes were proposed in this pull request?
As title.

### Why are the changes needed?
As title.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Passes UTs.

Closes #2162 from waitinfuture/1175-2.

Authored-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-12-16 01:36:29 +08:00
zky.zhoukeyong
01feb93abb [CELEBORN-1167] Avoid calling parmap when destroy slots
### What changes were proposed in this pull request?
As title

### Why are the changes needed?
One user reported that LifecycleManager's parmap can create huge number of threads and causes OOM.

![image](https://github.com/apache/incubator-celeborn/assets/948245/1e9a0b83-32fe-40d5-8739-2b370e030fc8)

There are four places where parmap is called:

1. When LifecycleManager commits files
2. When LifecycleManager reserves slots
3. When LifecycleManager setup connection to workers
4. When LifecycleManager call destroy slots

This PR fixes the fourth one. To be more detail, this PR eliminates `parmap` when destroying slots, and also replaces `askSync` with `ask`.

### Does this PR introduce _any_ user-facing change?
No.

### How was this patch tested?
Manual test and GA.

Closes #2156 from waitinfuture/1167.

Lead-authored-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
Co-authored-by: cxzl25 <cxzl25@users.noreply.github.com>
Co-authored-by: Keyong Zhou <waitinfuture@gmail.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-12-15 18:30:31 +08:00
SteNicholas
850d3199ef [CELEBORN-1164] Introduce FetchChunkFailCount metric to expose the count of fetching chunk failed in current worker
### What changes were proposed in this pull request?

Introduce `FetchChunkFailCount` metric to expose the count of fetching chunk failed in current worker.

### Why are the changes needed?

The metrics about the count of PushData or PushMergedData failed in current worker is supported at present. It's better to support `FetchChunkFailCount` metric to expose the count of fetching chunk failed in current worker.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Internal test.

Closes #2151 from SteNicholas/CELEBORN-1164.

Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-12-13 23:01:16 +08:00
zky.zhoukeyong
6b86b58f9e
[CELEBORN-1162][BUG] Fix refCnt 0 Exception in FetchHandler#handleChunkFetchRequest
### What changes were proposed in this pull request?
When I'm testing main branch I encountered exception below:
```
23/12/12 16:03:03,262 WARN [fetch-server-11-52] DefaultPromise: An exception was thrown by org.apache.celeborn.service.deploy.worker.FetchHandler$$anon$2.operationComplete()
io.netty.util.IllegalReferenceCountException: refCnt: 0
	at io.netty.buffer.AbstractByteBuf.ensureAccessible(AbstractByteBuf.java:1454)
	at io.netty.buffer.AbstractByteBuf.checkIndex(AbstractByteBuf.java:1383)
	at io.netty.buffer.AbstractByteBuf.getInt(AbstractByteBuf.java:433)
	at io.netty.buffer.ByteBufUtil.hashCode(ByteBufUtil.java:208)
	at io.netty.buffer.AbstractByteBuf.hashCode(AbstractByteBuf.java:1342)
	at java.util.WeakHashMap.hash(WeakHashMap.java:298)
	at java.util.WeakHashMap.getEntry(WeakHashMap.java:426)
	at java.util.WeakHashMap.containsKey(WeakHashMap.java:417)
	at org.apache.commons.lang3.builder.ToStringStyle.isRegistered(ToStringStyle.java:207)
	at org.apache.commons.lang3.builder.ToStringStyle.appendInternal(ToStringStyle.java:492)
	at org.apache.commons.lang3.builder.ToStringStyle.append(ToStringStyle.java:466)
	at org.apache.commons.lang3.builder.ToStringBuilder.append(ToStringBuilder.java:845)
	at org.apache.celeborn.common.network.buffer.NettyManagedBuffer.toString(NettyManagedBuffer.java:82)
	at java.lang.String.valueOf(String.java:2994)
	at java.lang.StringBuffer.append(StringBuffer.java:269)
	at org.apache.commons.lang3.builder.ToStringStyle.appendDetail(ToStringStyle.java:614)
	at org.apache.commons.lang3.builder.ToStringStyle.appendInternal(ToStringStyle.java:579)
	at org.apache.commons.lang3.builder.ToStringStyle.append(ToStringStyle.java:466)
	at org.apache.commons.lang3.builder.ToStringBuilder.append(ToStringBuilder.java:845)
	at org.apache.celeborn.common.network.protocol.RpcRequest.toString(RpcRequest.java:96)
	at org.apache.celeborn.service.deploy.worker.FetchHandler$$anon$2.operationComplete(FetchHandler.scala:403)
	at io.netty.util.concurrent.DefaultPromise.notifyListener0(DefaultPromise.java:590)
	at io.netty.util.concurrent.DefaultPromise.notifyListeners0(DefaultPromise.java:583)
	at io.netty.util.concurrent.DefaultPromise.notifyListenersNow(DefaultPromise.java:559)
	at io.netty.util.concurrent.DefaultPromise.notifyListeners(DefaultPromise.java:492)
	at io.netty.util.concurrent.DefaultPromise.setValue0(DefaultPromise.java:636)
	at io.netty.util.concurrent.DefaultPromise.setSuccess0(DefaultPromise.java:625)
	at io.netty.util.concurrent.DefaultPromise.trySuccess(DefaultPromise.java:105)
	at io.netty.util.internal.PromiseNotificationUtil.trySuccess(PromiseNotificationUtil.java:48)
	at io.netty.channel.ChannelOutboundBuffer.safeSuccess(ChannelOutboundBuffer.java:728)
	at io.netty.channel.ChannelOutboundBuffer.remove(ChannelOutboundBuffer.java:283)
	at io.netty.channel.nio.AbstractNioByteChannel.doWriteInternal(AbstractNioByteChannel.java:242)
	at io.netty.channel.nio.AbstractNioByteChannel.doWrite0(AbstractNioByteChannel.java:212)
	at io.netty.channel.socket.nio.NioSocketChannel.doWrite(NioSocketChannel.java:407)
	at io.netty.channel.AbstractChannel$AbstractUnsafe.flush0(AbstractChannel.java:931)
	at io.netty.channel.nio.AbstractNioChannel$AbstractNioUnsafe.forceFlush(AbstractNioChannel.java:361)
	at io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:782)
	at io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:724)
	at io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:650)
	at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:562)
	at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:997)
	at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
	at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
	at java.lang.Thread.run(Thread.java:750)
```

This is because in https://github.com/apache/incubator-celeborn/pull/2123 the `release` is called in `TransportRequestHandler#processRpcRequest`, but `FetchHandler#handleChunkFetchRequest` refererences `req` in callback, which is later.

### Why are the changes needed?
ditto

### Does this PR introduce _any_ user-facing change?
No.

### How was this patch tested?
Manual test

Closes #2148 from waitinfuture/1162.

Authored-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
2023-12-12 16:40:08 +08:00
Chandni Singh
074597d05e [CELEBORN-1147] Added a dedicated API for RPC messages which also accepts an RpcResponseCallback instance
### What changes were proposed in this pull request?

Currently in `BaseMessageHandler` there is a single API for receive which is used for all messages. This makes handling messages when multiple handlers are added messy.

- req.body.release() is only invoked when the handler actually process the message and not delegates it.
- every handler will have to create an instance of RpcResponseCallback for Rpc messages which is exactly the same.

Instead, releasing the message body and creating a callback for Rpc messages can be done in TransportRequestHandler. This avoids:

- code duplication related to RpcResponseCallback in every RPC handler
- every new request handler doesn't need to release the request body. It will be always be done in TransportRequestHandler.

Please note that this is how it is in Apache Spark and with Sasl Authentication, we will add a SaslRpcHandler (https://github.com/apache/incubator-celeborn/pull/2105) which wraps the underlying message handler.

### Why are the changes needed?

The changes are needed for adding authentication to Celeborn. See [CELEBORN-1011](https://issues.apache.org/jira/browse/CELEBORN-1011).

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Existing UTs and added some more UTs.

Closes #2123 from otterc/CELEBORN-1147.

Authored-by: Chandni Singh <singh.chandni@gmail.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-12-09 02:02:15 +08:00
jiang13021
c9db95386c [CELEBORN-1154] Fix NPE in DeviceMonitor#readWriteError
### What changes were proposed in this pull request?
Avoid NPE in DeviceMonitor#readWriteError

### Why are the changes needed?
The NullPointerException in the finally block will mask the real exception.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
manually test

Closes #2137 from jiang13021/celeborn-1154.

Authored-by: jiang13021 <jiangyanze.jyz@antgroup.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-12-07 23:20:43 +08:00
SteNicholas
b79dcd78ba
[MINOR] Update log level of ChunkFetchSuccess failed for FetchHandler#handleChunkFetchRequest from error to warn
### What changes were proposed in this pull request?

Update log level of sending ChunkFetchSuccess operation failed for `FetchHandler#handleChunkFetchRequest` from error to warn.

### Why are the changes needed?

The log level of sending ChunkFetchSuccess operation failed for `FetchHandler#handleChunkFetchRequest` is warn enough, because sending ChunkFetchSuccess operation failed doen not take much effect.

```
| 2023-12-04 13:33:07.393 | - | 2023-12-04 13:33:07,375 [ERROR] [fetch-server-11-20] - org.apache.celeborn.service.deploy.worker.FetchHandler -Logging.scala(80) -Sending ChunkFetchSuccess operation failed, chunk StreamChunkSlice{streamId=578269344491, chunkIndex=2, offset=0, len=2147483647}

| 2023-12-04 13:23:48.546 | - | 2023-12-04 13:23:48,448 [ERROR] [fetch-server-11-34] - org.apache.celeborn.service.deploy.worker.FetchHandler -Logging.scala(80) -Sending ChunkFetchSuccess operation failed, chunk StreamChunkSlice{streamId=1211783796302, chunkIndex=22, offset=0, len=2147483647}

| 2023-12-04 13:14:23.164 | - | 2023-12-04 13:14:23,080 [ERROR] [fetch-server-11-31] - org.apache.celeborn.service.deploy.worker.FetchHandler -Logging.scala(80) -Sending ChunkFetchSuccess operation failed, chunk StreamChunkSlice{streamId=143400144965, chunkIndex=0, offset=0, len=2147483647}

| 2023-12-04 13:14:23.119 | - | 2023-12-04 13:14:23,079 [ERROR] [fetch-server-11-8] - org.apache.celeborn.service.deploy.worker.FetchHandler -Logging.scala(80) -Sending ChunkFetchSuccess operation failed, chunk StreamChunkSlice{streamId=2016474250418, chunkIndex=1, offset=0, len=2147483647}

| 2023-12-04 13:05:38.157 | - | 2023-12-04 13:05:38,145 [ERROR] [fetch-server-11-18] - org.apache.celeborn.service.deploy.worker.FetchHandler -Logging.scala(80) -Sending ChunkFetchSuccess operation failed, chunk StreamChunkSlice{streamId=786116367486, chunkIndex=2, offset=0, len=2147483647}

| 2023-12-04 13:05:34.162 | - | 2023-12-04 13:05:34,112 [ERROR] [fetch-server-11-3] - org.apache.celeborn.service.deploy.worker.FetchHandler -Logging.scala(80) -Sending ChunkFetchSuccess operation failed, chunk StreamChunkSlice{streamId=1942979937404, chunkIndex=1, offset=0, len=2147483647}

| 2023-12-04 13:03:53.660 | - | 2023-12-04 13:03:53,585 [ERROR] [fetch-server-11-24] - org.apache.celeborn.service.deploy.worker.FetchHandler -Logging.scala(80) -Sending ChunkFetchSuccess operation failed, chunk StreamChunkSlice{streamId=1897602579272, chunkIndex=3, offset=0, len=2147483647}

| 2023-12-04 12:46:28.090 | - | 2023-12-04 12:46:28,039 [ERROR] [fetch-server-11-42] - org.apache.celeborn.service.deploy.worker.FetchHandler -Logging.scala(80) -Sending ChunkFetchSuccess operation failed, chunk StreamChunkSlice{streamId=1018072081537, chunkIndex=1, offset=0, len=2147483647}
```

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

No.

Closes #2128 from SteNicholas/chunk-fetch-success-failed.

Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
2023-12-07 10:48:00 +08:00
qinrui
04a1e90207 [CELEBORN-1122] Metrics supports json format
### What changes were proposed in this pull request?
If the user does not use prometheus to collect monitoring metrics, but rather some other ones. Using metrics in JSON format would be more user-friendly.The PR supports JSON format for metrics.

### Why are the changes needed?
Ditto.

### Does this PR introduce _any_ user-facing change?
Metrics supports JSON format

### How was this patch tested?
Cluster test.

Closes #2089 from suizhe007/CELEBORN-1122.

Authored-by: qinrui <qr7972@gmail.com>
Signed-off-by: Shuang <lvshuang.tb@gmail.com>
2023-12-06 09:24:28 +08:00