Commit Graph

493 Commits

Author SHA1 Message Date
mingji
40760ede3a [CELEBORN-568] Support storage type selection
### What changes were proposed in this pull request?
1. Celeborn supports storage type selection. HDD, SSD, and HDFS are available for now.
2. Add new buffer size for HDFS file writers.
3. Worker support empty working dirs.

### Why are the changes needed?
Support HDFS only scenario.

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

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

Closes #1619 from FMX/CELEBORN-568.

Lead-authored-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
Co-authored-by: Ethan Feng <fengmingxiao.fmx@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-06-27 18:07:08 +08:00
Angerszhuuuu
a2b215bd47 [CELEBORN-718] Support override Hadoop Conf by Celeborn Conf with celeborn.hadoop. prefix
### What changes were proposed in this pull request?
 Celeborn generate hadoop configuration should respect Celeborn conf

### Why are the changes needed?

In spark client side we should write like `spark.celeborn.hadoop.xxx.xx`
In server side we should write like `celeborn.hadoop.xxx.xxx`

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

### How was this patch tested?

Closes #1629 from AngersZhuuuu/CELEBORN-719.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Angerszhuuuu <angers.zhu@gmail.com>
2023-06-27 17:00:47 +08:00
Cheng Pan
1753556565
[CELEBORN-713] Local network binding support IP or FQDN
### What changes were proposed in this pull request?

This PR aims to make network local address binding support both IP and FQDN strategy.

Additional, it refactors the `ShuffleClientImpl#genAddressPair`, from `${hostAndPort}-${hostAndPort}` to `Pair<String, String>`, which works properly when using IP but may not on FQDN because FQDN may contain `-`

### Why are the changes needed?

Currently, when the bind hostname is not set explicitly, Celeborn will find the first non-loopback address and always uses the IP to bind, this is not suitable for K8s cases, as the STS has a stable FQDN but Pod IP will be changed once Pod restarting.

For `ShuffleClientImpl#genAddressPair`, it must be changed otherwise may cause

```
java.lang.RuntimeException: org.apache.spark.SparkException: Job aborted due to stage failure: Task 11657 in stage 0.0 failed 4 times, most recent failure: Lost task 11657.3 in stage 0.0 (TID 12747) (10.153.253.198 executor 157): java.lang.ArrayIndexOutOfBoundsException: 1
	at org.apache.celeborn.client.ShuffleClientImpl.doPushMergedData(ShuffleClientImpl.java:874)
	at org.apache.celeborn.client.ShuffleClientImpl.pushOrMergeData(ShuffleClientImpl.java:735)
	at org.apache.celeborn.client.ShuffleClientImpl.mergeData(ShuffleClientImpl.java:827)
	at org.apache.spark.shuffle.celeborn.SortBasedPusher.pushData(SortBasedPusher.java:140)
	at org.apache.spark.shuffle.celeborn.SortBasedPusher.insertRecord(SortBasedPusher.java:192)
	at org.apache.spark.shuffle.celeborn.SortBasedShuffleWriter.fastWrite0(SortBasedShuffleWriter.java:192)
	at org.apache.spark.shuffle.celeborn.SortBasedShuffleWriter.write(SortBasedShuffleWriter.java:145)
	at org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:59)
	at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:99)
	at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:52)
	at org.apache.spark.scheduler.Task.run(Task.scala:136)
	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:548)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1508)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:551)
	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:750)
```

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

Yes, a new configuration `celeborn.network.bind.preferIpAddress` is introduced, and the default value is `true` to preserve the existing behavior.

### How was this patch tested?

Manually testing with `celeborn.network.bind.preferIpAddress=false`
```
Server:		10.178.96.64
Address:	10.178.96.64#53

Name:	celeborn-master-0.celeborn-master-svc.spark.svc.cluster.local
Address: 10.153.143.252

Server:		10.178.96.64
Address:	10.178.96.64#53

Name:	celeborn-master-1.celeborn-master-svc.spark.svc.cluster.local
Address: 10.153.173.94

Server:		10.178.96.64
Address:	10.178.96.64#53

Name:	celeborn-master-2.celeborn-master-svc.spark.svc.cluster.local
Address: 10.153.149.42

starting org.apache.celeborn.service.deploy.worker.Worker, logging to /opt/celeborn/logs/celeborn--org.apache.celeborn.service.deploy.worker.Worker-1-celeborn-worker-4.out
2023-06-25 23:49:52 [INFO] [main] org.apache.celeborn.common.rpc.netty.Dispatcher#51 - Dispatcher numThreads: 4
2023-06-25 23:49:52 [INFO] [main] org.apache.celeborn.common.network.client.TransportClientFactory#91 - mode NIO threads 64
2023-06-25 23:49:52 [INFO] [main] org.apache.celeborn.common.rpc.netty.NettyRpcEnvFactory#51 - Starting RPC Server [WorkerSys] on celeborn-worker-4.celeborn-worker-svc.spark.svc.cluster.local:0 with advisor endpoint celeborn-worker-4.celeborn-worker-svc.spark.svc.cluster.local:0
2023-06-25 23:49:52 [INFO] [main] org.apache.celeborn.common.util.Utils#51 - Successfully started service 'WorkerSys' on port 38303.
```

Closes #1622 from pan3793/CELEBORN-713.

Authored-by: Cheng Pan <chengpan@apache.org>
Signed-off-by: Cheng Pan <chengpan@apache.org>
2023-06-27 09:42:11 +08:00
Cheng Pan
2b82194ce0 [CELEBORN-715] Change master URL schema from rss to celeborn
### What changes were proposed in this pull request?

Change Celeborn Master URL from `rss://<host>:<port>` to `celeborn://<host>:<port>`

### Why are the changes needed?

Respect the project name.

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

Yes, migration guide is updated accordingly.

### How was this patch tested?

Pass GA.

Closes #1624 from pan3793/CELEBORN-715.

Authored-by: Cheng Pan <chengpan@apache.org>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-06-26 22:30:20 +08:00
zky.zhoukeyong
6b82ecdfa0 [CELEBORN-712] Make appUniqueId a member of ShuffleClientImpl and refactor code
### What changes were proposed in this pull request?
Make appUniqueId a member of ShuffleClientImpl and remove applicationId from RPC messages across client side, so it won't cause compatibility issues.

### Why are the changes needed?
Currently Celeborn Client is bound to a single application id, so there's no need to pass applicationId around in many RPC messages in client side.

### Does this PR introduce _any_ user-facing change?
In some logs the application id will not be printed, which should not be a problem.

### How was this patch tested?
UTs.

Closes #1621 from waitinfuture/appid.

Authored-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-06-25 21:37:16 +08:00
zky.zhoukeyong
e2eeafd4bf [CELEBORN-709] Increase default fetch timeout
### What changes were proposed in this pull request?
30s for fetch timeout is too short and easy to exceed. This PR increases the default value to 600s.

### Why are the changes needed?
When I was testing 3T TPCDS with three workers, I encountered fetch timeout:
```
23/06/21 16:46:41,771 INFO [fetch-server-11-7] FetchHandler: Sending chunk 28856864163, 1, 0, 2147483647
...
23/06/21 16:47:16,870 INFO [fetch-server-11-7] FetchHandler: Sent chunk 28856864163, 1, 0, 2147483647
```
And I remember from some users' monitoring, the max fetch time can reach several minutes on heavy load without error.

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

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

Closes #1618 from waitinfuture/709.

Authored-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-06-23 21:06:43 +08:00
Cheng Pan
679f9cbf58 [CELEBORN-708] Fix commit metrics in application heartbeat
### What changes were proposed in this pull request?

- Fix commit metrics in application heartbeat
- Change client side application heartbeat message log level to info
- Improve heartbeat log by unify the word "heartbeat"

### Why are the changes needed?

`commitHandler.commitMetrics()` has side effects, multiple calls to get values independently is incorrect.

```
def commitMetrics(): (Long, Long) = (totalWritten.sumThenReset(), fileCount.sumThenReset())
```

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

Yes, bug fix.

### How was this patch tested?

Review.

Closes #1617 from pan3793/CELEBORN-708.

Authored-by: Cheng Pan <chengpan@apache.org>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-06-21 22:34:24 +08:00
Cheng Pan
8194407558 [CELEBORN-704] Print host and port on starting Netty RPC Server
### What changes were proposed in this pull request?

Print host and port on starting Netty RPC Server

### Why are the changes needed?

Sometimes, the Master/Worker may fail on bootstrap because `BindException: Cannot assign requested address`, but there is no clue which addresses it tried.

```
2023-06-21 14:28:12 [INFO] [main] org.apache.celeborn.service.deploy.worker.Worker#51 - Metrics system enabled.
2023-06-21 14:28:12 [ERROR] [main] org.apache.celeborn.service.deploy.worker.Worker#80 - Initialize worker failed.
java.net.BindException: Cannot assign requested address
	at sun.nio.ch.Net.bind0(Native Method) ~[?:1.8.0_372]
	at sun.nio.ch.Net.bind(Net.java:461) ~[?:1.8.0_372]
	at sun.nio.ch.Net.bind(Net.java:453) ~[?:1.8.0_372]
	at sun.nio.ch.ServerSocketChannelImpl.bind(ServerSocketChannelImpl.java:222) ~[?:1.8.0_372]
	at io.netty.channel.socket.nio.NioServerSocketChannel.doBind(NioServerSocketChannel.java:141) ~[netty-transport-4.1.93.Final.jar:4.1.93.Final]
	at io.netty.channel.AbstractChannel$AbstractUnsafe.bind(AbstractChannel.java:562) ~[netty-transport-4.1.93.Final.jar:4.1.93.Final]
	at io.netty.channel.DefaultChannelPipeline$HeadContext.bind(DefaultChannelPipeline.java:1334) ~[netty-transport-4.1.93.Final.jar:4.1.93.Final]
	at io.netty.channel.AbstractChannelHandlerContext.invokeBind(AbstractChannelHandlerContext.java:600) ~[netty-transport-4.1.93.Final.jar:4.1.93.Final]
	at io.netty.channel.AbstractChannelHandlerContext.bind(AbstractChannelHandlerContext.java:579) ~[netty-transport-4.1.93.Final.jar:4.1.93.Final]
	at io.netty.handler.logging.LoggingHandler.bind(LoggingHandler.java:230) ~[netty-handler-4.1.93.Final.jar:4.1.93.Final]
	at io.netty.channel.AbstractChannelHandlerContext.invokeBind(AbstractChannelHandlerContext.java:602) ~[netty-transport-4.1.93.Final.jar:4.1.93.Final]
	at io.netty.channel.AbstractChannelHandlerContext.bind(AbstractChannelHandlerContext.java:579) ~[netty-transport-4.1.93.Final.jar:4.1.93.Final]
	at io.netty.channel.DefaultChannelPipeline.bind(DefaultChannelPipeline.java:973) ~[netty-transport-4.1.93.Final.jar:4.1.93.Final]
	at io.netty.channel.AbstractChannel.bind(AbstractChannel.java:260) ~[netty-transport-4.1.93.Final.jar:4.1.93.Final]
	at io.netty.bootstrap.AbstractBootstrap$2.run(AbstractBootstrap.java:356) ~[netty-transport-4.1.93.Final.jar:4.1.93.Final]
	at io.netty.util.concurrent.AbstractEventExecutor.runTask(AbstractEventExecutor.java:174) ~[netty-common-4.1.93.Final.jar:4.1.93.Final]
	at io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:167) ~[netty-common-4.1.93.Final.jar:4.1.93.Final]
	at io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:470) ~[netty-common-4.1.93.Final.jar:4.1.93.Final]
	at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:569) ~[netty-transport-4.1.93.Final.jar:4.1.93.Final]
	at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:997) ~[netty-common-4.1.93.Final.jar:4.1.93.Final]
	at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74) ~[netty-common-4.1.93.Final.jar:4.1.93.Final]
	at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) ~[netty-common-4.1.93.Final.jar:4.1.93.Final]
	at java.lang.Thread.run(Thread.java:750) ~[?:1.8.0_372]
```

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

No.

### How was this patch tested?

Manually review.

Closes #1614 from pan3793/CELEBORN-704.

Authored-by: Cheng Pan <chengpan@apache.org>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-06-21 15:16:30 +08:00
zky.zhoukeyong
fdb126112e [CELEBORN-700][COMPATIBILITY] Fix compatibility issue caused by WorkerInfo
### What changes were proposed in this pull request?
Fixes compatibility issue introduced by change of WorkerInfo.

### Why are the changes needed?
When testing with branch-0.2 client and main server, I got the following error:
```
Caused by: scala.MatchError: [Ljava.lang.String;414ca35a (of class [Ljava.lang.String;)
        at org.apache.celeborn.common.util.PbSerDeUtils$.$anonfun$fromPbWorkerResource$1(PbSerDeUtils.scala:298)
        at scala.collection.Iterator.foreach(Iterator.scala:943)
        at scala.collection.Iterator.foreach$(Iterator.scala:943)
        at scala.collection.AbstractIterator.foreach(Iterator.scala:1431)
        at scala.collection.IterableLike.foreach(IterableLike.scala:74)
        at scala.collection.IterableLike.foreach$(IterableLike.scala:73)
        at scala.collection.AbstractIterable.foreach(Iterable.scala:56)
        at org.apache.celeborn.common.util.PbSerDeUtils$.fromPbWorkerResource(PbSerDeUtils.scala:297)
        at org.apache.celeborn.common.protocol.message.ControlMessages$.fromTransportMessage(ControlMessages.scala:863)
        at org.apache.celeborn.common.util.Utils$.fromTransportMessage(Utils.scala:828)
        at org.apache.celeborn.common.serializer.JavaSerializerInstance.deserialize(JavaSerializer.scala:110)
        at org.apache.celeborn.common.rpc.netty.NettyRpcEnv.$anonfun$deserialize$2(NettyRpcEnv.scala:276)
        at scala.util.DynamicVariable.withValue(DynamicVariable.scala:62)
        at org.apache.celeborn.common.rpc.netty.NettyRpcEnv.deserialize(NettyRpcEnv.scala:323)
        at org.apache.celeborn.common.rpc.netty.NettyRpcEnv.$anonfun$deserialize$1(NettyRpcEnv.scala:275)
        at scala.util.DynamicVariable.withValue(DynamicVariable.scala:62)
        at org.apache.celeborn.common.rpc.netty.NettyRpcEnv.deserialize(NettyRpcEnv.scala:275)
        at org.apache.celeborn.common.rpc.netty.NettyRpcEnv.$anonfun$ask$6(NettyRpcEnv.scala:235)
        at org.apache.celeborn.common.rpc.netty.NettyRpcEnv.$anonfun$ask$6$adapted(NettyRpcEnv.scala:235)
        at org.apache.celeborn.common.rpc.netty.RpcOutboxMessage.onSuccess(Outbox.scala:82)
        at org.apache.celeborn.common.network.client.TransportResponseHandler.handle(TransportResponseHandler.java:180)
        at org.apache.celeborn.common.network.server.TransportChannelHandler.channelRead(TransportChannelHandler.java:119)
        at org.apache.celeborn.shaded.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)
        at org.apache.celeborn.shaded.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)
        at org.apache.celeborn.shaded.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357)
        at org.apache.celeborn.shaded.io.netty.handler.timeout.IdleStateHandler.channelRead(IdleStateHandler.java:286)
        at org.apache.celeborn.shaded.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)
        at org.apache.celeborn.shaded.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)
        at org.apache.celeborn.shaded.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357)
        at org.apache.celeborn.common.network.util.TransportFrameDecoder.channelRead(TransportFrameDecoder.java:74)
```
And this is introduced by 811e192bbd (diff-b61712c3683306f65cd2ca051b54075952897a899951f3e37ec3968e7ba75710)

### Does this PR introduce _any_ user-facing change?
Yes, it fixes compatibility error when using branch-0.2 client and main server.

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

Closes #1610 from waitinfuture/700.

Authored-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-06-20 20:17:00 +08:00
zky.zhoukeyong
255661bbb7 [CELEBORN-696] Fix bugs related with shutting down and excluded workers
### What changes were proposed in this pull request?
1. Foreach PartitionLocation returned from Register Shuffle, remove from worker's local excluded list to refresh the local information.
2. ChangeLocationResponse will also return whether oldPartition is excluded in LifecycleManager. If so, remove it from Executor's excluded list.
3. Always trigger commit files for shutting down workers returned from HeartbeatFromApplicationResponse.
4. HeartbeatFromWorker sends the correct disk infos regardless of shutting down or not.

After this PR, the priority of excluded list is Master > LifecycleManager > Executor.

### Why are the changes needed?
During test with graceful turned on(workers have static rpc/push/fetch/replicate ports) and consistently restart one out of three workers, I encountered several bugs.
1. First I killed worker A, then Executor's client's local excluded list will contain A, after A stopped, I started it again, then master will offer slots on A, so we should remove from the executor's excluded list then.
2. When I kill-and-start a worker twice in a short time smaller than the app heartbeat interval, the second time WorkerStatusTracker will not trigger commit files because the local cache for the worker has not been refreshed.
3. When a worker is shutting down, in its heartbeat it passes empty diskInfos, and master blindly added to excluded list. We want a worker be either in the excluded list, or in the shutting down list, exclusively. If a worker is in excluded list, then LIfecycleManager will not trigger commit files when handle heartbeat response; on the other hand, if a worker is in the shutting down list, LifecycleManager will trigger commit files on it. So we must make it correct that a shutting down worker be in the shutting down list.

### Does this PR introduce _any_ user-facing change?
Yes, it fixes several bugs described above.

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

Closes #1606 from waitinfuture/696.

Authored-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
Signed-off-by: Shuang <lvshuang.tb@gmail.com>
2023-06-20 16:47:07 +08:00
onebox-li
88586d6c15 [CELEBORN-697] Fix assignment of DeviceInfo deviceStatAvailable
### What changes were proposed in this pull request?
DeviceInfo `deviceStatAvailable` 's variable name and assignment does not match

### Why are the changes needed?
ditto

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

### How was this patch tested?
Cluster test

Closes #1607 from onebox-li/fix-deviceStatAvailable.

Authored-by: onebox-li <lyh-36@163.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-06-20 15:53:15 +08:00
zky.zhoukeyong
7d634db547 [CELEBORN-695] Fix UnsupportedOperationException by refactoring WorkerInfo
### What changes were proposed in this pull request?
Refactor WorkerInfo

1. make ```diskInfos```, ```userResourceConsumption``` new maps instead of using the passed in reference
2. remove ```endpoint``` from the constructor

### Why are the changes needed?

When manually test stop-worker.sh with graceful turned on, I got the following Exception
```
23/06/19 11:04:25,665 INFO [worker-forward-message-scheduler] RssHARetryClient: connect to master master-1-1:9097.
23/06/19 11:04:27,168 ERROR [worker-forward-message-scheduler] RssHARetryClient: Send rpc with failure, has tried 15, max try 15!
org.apache.celeborn.common.exception.CelebornException: Exception thrown in awaitResult:
        at org.apache.celeborn.common.util.ThreadUtils$.awaitResult(ThreadUtils.scala:231)
        at org.apache.celeborn.common.rpc.RpcTimeout.awaitResult(RpcTimeout.scala:74)
        at org.apache.celeborn.common.haclient.RssHARetryClient.sendMessageInner(RssHARetryClient.java:150)
        at org.apache.celeborn.common.haclient.RssHARetryClient.askSync(RssHARetryClient.java:118)
        at org.apache.celeborn.service.deploy.worker.Worker.org$apache$celeborn$service$deploy$worker$Worker$$heartBeatToMaster(Worker.scala:306)
        at org.apache.celeborn.service.deploy.worker.Worker$$anon$1.$anonfun$run$1(Worker.scala:332)
        at org.apache.celeborn.common.util.Utils$.tryLogNonFatalError(Utils.scala:186)
        at org.apache.celeborn.service.deploy.worker.Worker$$anon$1.run(Worker.scala:332)
        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:750)
Caused by: org.apache.celeborn.common.exception.CelebornIOException: remove
        at org.apache.celeborn.service.deploy.master.clustermeta.ha.HAHelper.sendFailure(HAHelper.java:65)
        at org.apache.celeborn.service.deploy.master.Master.executeWithLeaderChecker(Master.scala:210)
        at org.apache.celeborn.service.deploy.master.Master$$anonfun$receiveAndReply$1.applyOrElse(Master.scala:315)
        at org.apache.celeborn.common.rpc.netty.Inbox.$anonfun$process$1(Inbox.scala:115)
        at org.apache.celeborn.common.rpc.netty.Inbox.safelyCall(Inbox.scala:222)
        at org.apache.celeborn.common.rpc.netty.Inbox.process(Inbox.scala:110)
        at org.apache.celeborn.common.rpc.netty.Dispatcher$MessageLoop.run(Dispatcher.scala:229)
        ... 3 more
Caused by: java.lang.UnsupportedOperationException: remove
        at scala.collection.convert.Wrappers$MapWrapper$$anon$2$$anon$3.remove(Wrappers.scala:236)
        at java.util.AbstractMap.remove(AbstractMap.java:254)
        at org.apache.celeborn.common.meta.WorkerInfo.$anonfun$updateThenGetDiskInfos$2(WorkerInfo.scala:225)
        at scala.collection.Iterator.foreach(Iterator.scala:943)
        at scala.collection.Iterator.foreach$(Iterator.scala:943)
        at scala.collection.AbstractIterator.foreach(Iterator.scala:1431)
        at scala.collection.IterableLike.foreach(IterableLike.scala:74)
        at scala.collection.IterableLike.foreach$(IterableLike.scala:73)
        at scala.collection.AbstractIterable.foreach(Iterable.scala:56)
        at org.apache.celeborn.common.meta.WorkerInfo.updateThenGetDiskInfos(WorkerInfo.scala:224)
        at org.apache.celeborn.service.deploy.master.clustermeta.AbstractMetaManager.lambda$updateWorkerHeartbeatMeta$5(AbstractMetaManager.java:205)
        at java.util.Optional.ifPresent(Optional.java:159)
        at org.apache.celeborn.service.deploy.master.clustermeta.AbstractMetaManager.updateWorkerHeartbeatMeta(AbstractMetaManager.java:203)
        at org.apache.celeborn.service.deploy.master.clustermeta.SingleMasterMetaManager.handleWorkerHeartbeat(SingleMasterMetaManager.java:105)
        at org.apache.celeborn.service.deploy.master.Master.org$apache$celeborn$service$deploy$master$Master$$handleHeartbeatFromWorker(Master.scala:428)
        at org.apache.celeborn.service.deploy.master.Master$$anonfun$receiveAndReply$1.$anonfun$applyOrElse$20(Master.scala:326)
        at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
        at org.apache.celeborn.service.deploy.master.Master.executeWithLeaderChecker(Master.scala:207)
        ... 8 more
```

According to the suggestion from https://github.com/apache/incubator-celeborn/pull/1602#issuecomment-1596722991

### Does this PR introduce _any_ user-facing change?
Yes, it fixes bug described  in https://github.com/apache/incubator-celeborn/pull/1602

### How was this patch tested?
UTs and manual test.

Closes #1605 from waitinfuture/695.

Authored-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-06-19 19:38:55 +08:00
sychen
e734ceb558 [MINOR] Cleanup code
### What changes were proposed in this pull request?
1. Use `<arg>-Ywarn-unused-import</arg>` to remove some unused imports
There is no way to use `<arg>-Ywarn-unused-import</arg>` at this stage
Because we have the following code
```
// Can Remove this if celeborn don't support scala211 in future
import org.apache.celeborn.common.util.FunctionConverter._
```
2. Fix scala case match not fully covered, avoid `scala.MatchError`
3. Fixed some scala compilation warnings

### Why are the changes needed?

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

### How was this patch tested?

Closes #1600 from cxzl25/cleanup_code.

Authored-by: sychen <sychen@ctrip.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-06-19 11:31:51 +08:00
Cheng Pan
e22379c3ab [CELEBORN-638] Migrate configurations celeborn.ha.master.* to celeborn.master.ha.*
### What changes were proposed in this pull request?

It was discussed during the last meeting, but abandoned due to the complication.

### Why are the changes needed?

Make the configuration unified.

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

Yes, but the legacy configurations still take effect.

### How was this patch tested?

New UTs.

Closes #1549 from pan3793/CELEBORN-638.

Authored-by: Cheng Pan <chengpan@apache.org>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-06-16 18:18:26 +08:00
zky.zhoukeyong
fae96f73cb [CELEBORN-686] Include ConnectException when exclude worker for fetch
### What changes were proposed in this pull request?
Include ConnectException when exclude worker for fetch

### Why are the changes needed?
Currently RssInputStream.isCriticalCause does not include ConnectException
```
java.io.IOException: Failed to connect to /192.168.1.17:46197
	at org.apache.celeborn.common.network.client.TransportClientFactory.internalCreateClient(TransportClientFactory.java:232)
	at org.apache.celeborn.common.network.client.TransportClientFactory.createClient(TransportClientFactory.java:176)
	at org.apache.celeborn.common.network.client.TransportClientFactory.createClient(TransportClientFactory.java:114)
	at org.apache.celeborn.common.network.client.TransportClientFactory.createClient(TransportClientFactory.java:183)
	at org.apache.celeborn.client.read.WorkerPartitionReader.<init>(WorkerPartitionReader.java:103)
	at org.apache.celeborn.client.read.RssInputStream$RssInputStreamImpl.createReader(RssInputStream.java:399)
	at org.apache.celeborn.client.read.RssInputStream$RssInputStreamImpl.createReaderWithRetry(RssInputStream.java:301)
	at org.apache.celeborn.client.read.RssInputStream$RssInputStreamImpl.moveToNextReader(RssInputStream.java:229)
	at org.apache.celeborn.client.read.RssInputStream$RssInputStreamImpl.<init>(RssInputStream.java:178)
	at org.apache.celeborn.client.read.RssInputStream.create(RssInputStream.java:63)
	at org.apache.celeborn.client.ShuffleClientImpl.readPartition(ShuffleClientImpl.java:1599)
	at org.apache.spark.shuffle.celeborn.RssShuffleReader.$anonfun$read$1(RssShuffleReader.scala:88)
	at org.apache.spark.shuffle.celeborn.RssShuffleReader.$anonfun$read$1$adapted(RssShuffleReader.scala:79)
	at scala.collection.Iterator$$anon$10.next(Iterator.scala:461)
	at scala.collection.Iterator$$anon$11.nextCur(Iterator.scala:486)
	at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:492)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
	at org.apache.spark.util.CompletionIterator.hasNext(CompletionIterator.scala:31)
	at org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:37)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage17.sort_addToSorter_0$(Unknown Source)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage17.sort_doSort_0$(Unknown Source)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage17.processNext(Unknown Source)
	at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:35)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage17.hasNext(Unknown Source)
	at org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:954)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage18.smj_findNextJoinRows_0$(Unknown Source)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage18.processNext(Unknown Source)
	at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:35)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage18.hasNext(Unknown Source)
	at org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$2.hasNext(WholeStageCodegenExec.scala:973)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
	at org.apache.spark.shuffle.celeborn.HashBasedShuffleWriter.fastWrite0(HashBasedShuffleWriter.java:251)
	at org.apache.spark.shuffle.celeborn.HashBasedShuffleWriter.write(HashBasedShuffleWriter.java:180)
	at org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:59)
	at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:99)
	at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:52)
	at org.apache.spark.scheduler.Task.run(Task.scala:133)
	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:506)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1474)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:509)
	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:750)
Caused by: org.apache.celeborn.shaded.io.netty.channel.AbstractChannel$AnnotatedConnectException: Connection refused: /192.168.1.17:46197
Caused by: java.net.ConnectException: Connection refused
	at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
	at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:716)
	at org.apache.celeborn.shaded.io.netty.channel.socket.nio.NioSocketChannel.doFinishConnect(NioSocketChannel.java:337)
	at org.apache.celeborn.shaded.io.netty.channel.nio.AbstractNioChannel$AbstractNioUnsafe.finishConnect(AbstractNioChannel.java:334)
	at org.apache.celeborn.shaded.io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:776)
	at org.apache.celeborn.shaded.io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:724)
	at org.apache.celeborn.shaded.io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:650)
	at org.apache.celeborn.shaded.io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:562)
	at org.apache.celeborn.shaded.io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:997)
	at org.apache.celeborn.shaded.io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
	at org.apache.celeborn.shaded.io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
	at java.lang.Thread.run(Thread.java:750)
```

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

### How was this patch tested?
Manual test

Closes #1598 from waitinfuture/686.

Authored-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-06-16 16:21:57 +08:00
Angerszhuuuu
1ba6dee324 [CELEBORN-680][DOC] Refresh celeborn configurations in doc
### What changes were proposed in this pull request?
Refresh celeborn configurations in doc

### Why are the changes needed?

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

### How was this patch tested?

Closes #1592 from AngersZhuuuu/CELEBORN-680.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Angerszhuuuu <angers.zhu@gmail.com>
2023-06-15 13:59:38 +08:00
Angerszhuuuu
0aa13832b5 [CELEBORN-676] Celeborn fetch chunk also should support check timeout
### What changes were proposed in this pull request?
Celeborn fetch chunk also should support check timeout

#### Test case
```
executor instance 20

SQL:
SELECT count(1) from (select /*+ REPARTITION(100) */ * from spark_auxiliary.t50g) tmp;

--conf spark.celeborn.client.spark.shuffle.writer=sort \
--conf spark.celeborn.client.fetch.excludeWorkerOnFailure.enabled=true \
--conf spark.celeborn.client.push.timeout=10s \
--conf spark.celeborn.client.push.replicate.enabled=true \
--conf spark.celeborn.client.push.revive.maxRetries=10 \
--conf spark.celeborn.client.reserveSlots.maxRetries=10 \
--conf spark.celeborn.client.registerShuffle.maxRetries=3 \
--conf spark.celeborn.client.push.blacklist.enabled=true \
--conf spark.celeborn.client.blacklistSlave.enabled=true \
--conf spark.celeborn.client.fetch.timeout=30s \
--conf spark.celeborn.client.push.data.timeout=30s \
--conf spark.celeborn.client.push.limit.inFlight.timeout=600s \
--conf spark.celeborn.client.push.maxReqsInFlight=32 \
--conf spark.celeborn.client.shuffle.compression.codec=ZSTD \
--conf spark.celeborn.rpc.askTimeout=30s \
--conf spark.celeborn.client.rpc.reserveSlots.askTimeout=30s \
--conf spark.celeborn.client.shuffle.batchHandleChangePartition.enabled=true \
--conf spark.celeborn.client.shuffle.batchHandleCommitPartition.enabled=true \
--conf spark.celeborn.client.shuffle.batchHandleReleasePartition.enabled=true
```

Test with 3 worker and add a `Thread.sleep(100s)` before worker handle `ChunkFetchRequest`

Before patch
<img width="1783" alt="截屏2023-06-14 上午11 20 55" src="https://github.com/apache/incubator-celeborn/assets/46485123/182dff7d-a057-4077-8368-d1552104d206">

After patch
<img width="1792" alt="image" src="https://github.com/apache/incubator-celeborn/assets/46485123/3c8b7933-8ace-426d-8e9f-04e0aabfac8e">

The log shows the fetch timeout checker workers
```
23/06/14 11:14:54 ERROR WorkerPartitionReader: Fetch chunk 0 failed.
org.apache.celeborn.common.exception.CelebornIOException: FETCH_DATA_TIMEOUT
	at org.apache.celeborn.common.network.client.TransportResponseHandler.failExpiredFetchRequest(TransportResponseHandler.java:147)
	at org.apache.celeborn.common.network.client.TransportResponseHandler.lambda$new$1(TransportResponseHandler.java:103)
	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)
23/06/14 11:14:54 WARN RssInputStream: Fetch chunk failed 1/6 times for location PartitionLocation[
  id-epoch:35-0
  host-rpcPort-pushPort-fetchPort-replicatePort:10.169.48.203-9092-9094-9093-9095
  mode:MASTER
  peer:(host-rpcPort-pushPort-fetchPort-replicatePort:10.169.48.202-9092-9094-9093-9095)
  storage hint:StorageInfo{type=HDD, mountPoint='/mnt/ssd/0', finalResult=true, filePath=}
  mapIdBitMap:null], change to peer
org.apache.celeborn.common.exception.CelebornIOException: Fetch chunk 0 failed.
	at org.apache.celeborn.client.read.WorkerPartitionReader$1.onFailure(WorkerPartitionReader.java:98)
	at org.apache.celeborn.common.network.client.TransportResponseHandler.failExpiredFetchRequest(TransportResponseHandler.java:146)
	at org.apache.celeborn.common.network.client.TransportResponseHandler.lambda$new$1(TransportResponseHandler.java:103)
	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)
Caused by: org.apache.celeborn.common.exception.CelebornIOException: FETCH_DATA_TIMEOUT
	at org.apache.celeborn.common.network.client.TransportResponseHandler.failExpiredFetchRequest(TransportResponseHandler.java:147)
	... 8 more
23/06/14 11:14:54 INFO SortBasedShuffleWriter: Memory used 72.0 MB
```

### Why are the changes needed?

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

### How was this patch tested?

Closes #1587 from AngersZhuuuu/CELEBORN-676.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Angerszhuuuu <angers.zhu@gmail.com>
2023-06-15 13:54:09 +08:00
Angerszhuuuu
8a0b7d80d6 [CELEBORN-681][DOC] Add celeborn.metrics.conf to conf entity
### What changes were proposed in this pull request?
Add celeborn.metrics.conf to conf entity

### Why are the changes needed?

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

### How was this patch tested?

Closes #1593 from AngersZhuuuu/CELEBORN-681.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Angerszhuuuu <angers.zhu@gmail.com>
2023-06-14 18:06:03 +08:00
Fu Chen
aa3bb0ac3b
[CELEBORN-679] Optimize Utils#bytesToString
### What changes were proposed in this pull request?

refer to https://github.com/apache/spark/pull/40301

1. Optimize `Utils.bytesToString`. Arithmetic ops on BigInt and BigDecimal are order(s) of magnitude slower than the ops on primitive types. Division is an especially slow operation and it is used en masse here.

2. According to the information sourced from [Wikipedia](https://en.wikipedia.org/wiki/Kilobyte), it is established that 1000 is the appropriate factor for representing kilobytes (KB), while 1024 is the correct factor for kibibytes (KiB). In alignment with this understanding, changing the size unit from "KB" to "KiB".

### Why are the changes needed?

the Utils#bytesToString method is frequently employed in memory-related log messages.

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

No, only perf improvement.

### How was this patch tested?

existing UT and manually tested.

Closes #1590 from cfmcgrady/bytesToString.

Authored-by: Fu Chen <cfmcgrady@gmail.com>
Signed-off-by: Cheng Pan <chengpan@apache.org>
2023-06-14 17:42:16 +08:00
Shuang
da85347330 [CELEBORN-675] Fix decode heartbeat message
### What changes were proposed in this pull request?
Give Heartbeat one byte message and skip this byte when decode.

### Why are the changes needed?
Heartbeat message may split in to two netty buffer,  then the `empty buffer` (which don't need actually, but need keep) be wrong removed, then decodeNext would throw NPE. see
```  java
while (headerBuf.readableBytes() < HEADER_SIZE) {
      ByteBuf next = buffers.getFirst();
      int toRead = Math.min(next.readableBytes(), HEADER_SIZE - headerBuf.readableBytes());
      headerBuf.writeBytes(next, toRead);
      if (!next.isReadable()) {
        buffers.removeFirst().release();
      }
    }
```

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

### How was this patch tested?
UT & MANUAL

Closes #1589 from RexXiong/CELEBORN-675.

Authored-by: Shuang <lvshuang.tb@gmail.com>
Signed-off-by: zhongqiang.czq <zhongqiang.czq@alibaba-inc.com>
2023-06-14 14:37:13 +08:00
Angerszhuuuu
f2357bf75c [CELEBORN-671] Add hasPeer method to PartitionLocation
### What changes were proposed in this pull request?
Add hasPeer method to PartitionLocation

### Why are the changes needed?

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

### How was this patch tested?

Closes #1583 from AngersZhuuuu/CELEBORN-671.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Angerszhuuuu <angers.zhu@gmail.com>
2023-06-14 10:29:16 +08:00
zky.zhoukeyong
47cded835f [CELEBORN-669] Avoid commit files on excluded worker list
### What changes were proposed in this pull request?
CommitHandler will check whether the target worker is in WorkerStatusTracker's excluded list. If so, skip calling commit files on it.

### Why are the changes needed?
Avoid unnecessary commit files to excluded worker.

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

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

Closes #1581 from waitinfuture/669.

Lead-authored-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
Co-authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Co-authored-by: Keyong Zhou <zhouky@apache.org>
Signed-off-by: Shuang <lvshuang.tb@gmail.com>
2023-06-13 22:31:02 +08:00
Ethan Feng
1fca5da553 [CELEBORN-674] Support revive for empty locations
### What changes were proposed in this pull request?
If some task retry or scheduled by speculation, the executors will keep failing because revive does not support old partition is empty.
Celeborn will trigger stage end if all mapper task calls mapper end, this is not what spark thinks a stage ends.  So in this moment, kill spark executors will cause the spark task to rerun the current stage. Shuffle client will need to register shuffle first but get empty partition locations, and it will need to revive to get the latest location with empty locations.

Here are logs example

```
23/06/13 08:32:11 ERROR ShuffleClientImpl: Exception raised while reviving for shuffle 12 map 970 attempt 27 partition 152 epoch -1.
java.lang.NullPointerException
	at org.apache.celeborn.common.util.PbSerDeUtils$.toPbPartitionLocation(PbSerDeUtils.scala:258)
	at org.apache.celeborn.common.protocol.message.ControlMessages$Revive$.apply(ControlMessages.scala:207)
	at org.apache.celeborn.client.ShuffleClientImpl.revive(ShuffleClientImpl.java:573)
	at org.apache.celeborn.client.ShuffleClientImpl.pushOrMergeData(ShuffleClientImpl.java:656)
	at org.apache.celeborn.client.ShuffleClientImpl.pushData(ShuffleClientImpl.java:984)
	at org.apache.celeborn.client.write.DataPusher.pushData(DataPusher.java:197)
	at org.apache.celeborn.client.write.DataPusher.access$500(DataPusher.java:38)
	at org.apache.celeborn.client.write.DataPusher$1.run(DataPusher.java:123)
23/06/13 08:32:11 ERROR Executor: Exception in task 21.27 in stage 54.1 (TID 17255)
org.apache.celeborn.common.exception.CelebornIOException: Revive for shuffle spark-3df9647407f14c39868a17b7950899c5-12 partition 152 failed.
	at org.apache.celeborn.client.ShuffleClientImpl.pushOrMergeData(ShuffleClientImpl.java:666)
	at org.apache.celeborn.client.ShuffleClientImpl.pushData(ShuffleClientImpl.java:984)
	at org.apache.celeborn.client.write.DataPusher.pushData(DataPusher.java:197)
	at org.apache.celeborn.client.write.DataPusher.access$500(DataPusher.java:38)
	at org.apache.celeborn.client.write.DataPusher$1.run(DataPusher.java:123)
23/06/13 08:32:11 INFO CoarseGrainedExecutorBackend: Got assigned task 17309
```

### Why are the changes needed?
To make shuffle client able to revive with empty locations.

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

### How was this patch tested?
K8S cluster.

Closes #1586 from FMX/CELEBORN-674.

Authored-by: Ethan Feng <ethanfeng@apache.org>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-06-13 20:35:04 +08:00
Angerszhuuuu
357add5b00 [CELEBORN-494][PERF] RssInputStream fetch side support blacklist to avoid client side timeout in same worker multiple times during fetch
### What changes were proposed in this pull request?
####Test case
```
executor instance 20

SQL:
SELECT count(1) from (select /*+ REPARTITION(100) */ * from spark_auxiliary.t50g) tmp;

create connection timeout 10s

Fetch chunk timeout 30s
```
In the graph, the shuffle read time of `before` and `after` is always the same delay time.

##### Worker can't connect
Before
![image](https://user-images.githubusercontent.com/46485123/229465520-9d751b40-2b8f-49d2-b350-a2278e3dd89e.png)

After
![image](https://user-images.githubusercontent.com/46485123/229465552-88ac1ca4-24ad-4c30-9a46-0cdcae6bbfd5.png)

##### OpenStream stuck
Before
![image](https://user-images.githubusercontent.com/46485123/229465629-68765a6a-2503-4018-8917-d49e47d5dccc.png)

After
![image](https://user-images.githubusercontent.com/46485123/229465683-2f57b374-1c66-4819-93dd-cabee7ccb788.png)

##### Fetch chunk stuck
Before
![image](https://user-images.githubusercontent.com/46485123/229465735-8d2f694b-1b4a-4984-b069-c4a308f41008.png)

After
![image](https://user-images.githubusercontent.com/46485123/229465754-c2237d5a-6fb6-4d5b-819e-b7d86a1e88d7.png)

### Why are the changes needed?

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

### How was this patch tested?

Closes #1406 from AngersZhuuuu/CELEBORN-494.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Shuang <lvshuang.tb@gmail.com>
2023-06-13 20:06:31 +08:00
Fu Chen
3fb896b11f [CELEBORN-666] Define protobuf-maven-plugin in the root pom.xml
### What changes were proposed in this pull request?

Define `protobuf-maven-plugin` in the root pom.xml

### Why are the changes needed?

to fix

```bash
build/mvn protobuf:compile -am -pl common
```

```
[ERROR] No plugin found for prefix 'protobuf' in the current project and in the plugin groups [org.apache.maven.plugins, org.codehaus.mojo] available from the repositories [local (/Users/fchen/.m2/repository), apache.snapshots (https://repository.apache.org/snapshots), central (https://repo.maven.apache.org/maven2)] -> [Help 1]
org.apache.maven.plugin.prefix.NoPluginFoundForPrefixException: No plugin found for prefix 'protobuf' in the current project and in the plugin groups [org.apache.maven.plugins, org.codehaus.mojo] available from the repositories [local (/Users/fchen/.m2/repository), apache.snapshots (https://repository.apache.org/snapshots), central (https://repo.maven.apache.org/maven2)]
    at org.apache.maven.plugin.prefix.internal.DefaultPluginPrefixResolver.resolve (DefaultPluginPrefixResolver.java:95)
    at org.apache.maven.lifecycle.internal.MojoDescriptorCreator.findPluginForPrefix (MojoDescriptorCreator.java:266)
    at org.apache.maven.lifecycle.internal.MojoDescriptorCreator.getMojoDescriptor (MojoDescriptorCreator.java:220)
    at org.apache.maven.lifecycle.internal.DefaultLifecycleTaskSegmentCalculator.calculateTaskSegments (DefaultLifecycleTaskSegmentCalculator.java:104)
    at org.apache.maven.lifecycle.internal.DefaultLifecycleTaskSegmentCalculator.calculateTaskSegments (DefaultLifecycleTaskSegmentCalculator.java:83)
    at org.apache.maven.lifecycle.internal.LifecycleStarter.execute (LifecycleStarter.java:89)
    at org.apache.maven.DefaultMaven.doExecute (DefaultMaven.java:298)
    at org.apache.maven.DefaultMaven.doExecute (DefaultMaven.java:192)
    at org.apache.maven.DefaultMaven.execute (DefaultMaven.java:105)
    at org.apache.maven.cli.MavenCli.execute (MavenCli.java:960)
    at org.apache.maven.cli.MavenCli.doMain (MavenCli.java:293)
    at org.apache.maven.cli.MavenCli.main (MavenCli.java:196)
    at sun.reflect.NativeMethodAccessorImpl.invoke0 (Native Method)
    at sun.reflect.NativeMethodAccessorImpl.invoke (NativeMethodAccessorImpl.java:62)
    at sun.reflect.DelegatingMethodAccessorImpl.invoke (DelegatingMethodAccessorImpl.java:43)
    at java.lang.reflect.Method.invoke (Method.java:498)
    at org.codehaus.plexus.classworlds.launcher.Launcher.launchEnhanced (Launcher.java:282)
    at org.codehaus.plexus.classworlds.launcher.Launcher.launch (Launcher.java:225)
    at org.codehaus.plexus.classworlds.launcher.Launcher.mainWithExitCode (Launcher.java:406)
    at org.codehaus.plexus.classworlds.launcher.Launcher.main (Launcher.java:347)
[ERROR]
[ERROR]
[ERROR] For more information about the errors and possible solutions, please read the following articles:
[ERROR] [Help 1] http://cwiki.apache.org/confluence/display/MAVEN/NoPluginFoundForPrefixException
```

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

No.

### How was this patch tested?

tested locally.

Closes #1579 from cfmcgrady/protobuf-plugin.

Authored-by: Fu Chen <cfmcgrady@gmail.com>
Signed-off-by: Cheng Pan <chengpan@apache.org>
2023-06-12 19:46:46 +08:00
Angerszhuuuu
6b725202a2 [CELEBORN-640][WORKER] DataPushQueue should not keep waiting take tasks
### What changes were proposed in this pull request?
In our prod meet many times of push queue stuck caused by PushState's status was not being removed.
Caused DataPushQueue to keep waiting for taking task.

Although have resolved some bugs, here we'd better add a max wait time for taking tasks since we already have the `PUSH_DATA_TIMEOUT` check method. If the target worker is really stuck, we can retry our task finally.

### Why are the changes needed?

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

### How was this patch tested?

Closes #1552 from AngersZhuuuu/CELEBORN-640.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Angerszhuuuu <angers.zhu@gmail.com>
2023-06-09 14:06:47 +08:00
Cheng Pan
588dbdfbe0
[CELEBORN-653][TEST] Fix invalid configuration key in SparkTestBase
### What changes were proposed in this pull request?

Dot is missing after `spark`

### Why are the changes needed?

Correct the configuration key.

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

No.

### How was this patch tested?

Pass GA.

Closes #1563 from pan3793/CELEBORN-653.

Authored-by: Cheng Pan <chengpan@apache.org>
Signed-off-by: Cheng Pan <chengpan@apache.org>
2023-06-09 10:31:25 +08:00
Cheng Pan
76533d7324
[CELEBORN-650][TEST] Upgrade scalatest and unify mockito version
### What changes were proposed in this pull request?

This PR upgrades

- `mockito` from 1.10.19 and 3.6.0 to 4.11.0
- `scalatest` from 3.2.3 to 3.2.16
- `mockito-scalatest` from 1.16.37 to 1.17.14

### Why are the changes needed?

Housekeeping, making test dependencies up-to-date and unified.

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

No, it only affects test.

### How was this patch tested?

Pass GA.

Closes #1562 from pan3793/CELEBORN-650.

Authored-by: Cheng Pan <chengpan@apache.org>
Signed-off-by: Cheng Pan <chengpan@apache.org>
2023-06-09 10:04:14 +08:00
onebox-li
0c869ac9a0
[CELEBORN-642] Improve metrics and update grafana
### What changes were proposed in this pull request?
Change in grafana

(ALL)
add:
JVMCPUTime
LastMinuteSystemLoad
AvailableProcessors
(For Master)
add:
LostWorkers
IsActiveMaster
PartitionSize
(For Worker)
add:
PushDataFailCount -> WriteDataFailCount
ReplicateDataFailCount
ReplicateDataWriteFailCount
ReplicateDataCreateConnectionFailCount
ReplicateDataConnectionExceptionCount
ReplicateDataTimeoutCount
SortedFileSize
PushDataHandshakeFailCount
RegionStartFailCount
RegionFinishFailCount
MasterPushDataHandshakeTime
SlavePushDataHandshakeTime
MasterRegionStartTime
SlaveRegionStartTime
MasterRegionFinishTime
SlaveRegionFinishTime
PotentialConsumeSpeed
UserProduceSpeed
WorkerConsumeSpeed
DeviceOSFreeBytes
DeviceCelebornFreeBytes
push usedHeapMemory/usedDirectMemory
fetch usedHeapMemory/usedDirectMemory
replicate usedHeapMemory/usedDirectMemory
remove:
dup ReserveSlotsTime

Change dashboard layout.

Fix support for multiple labels.

Modify some metrics docs.

### Why are the changes needed?
For better use of metrics.

### Does this PR introduce _any_ user-facing change?
Below metrics change name, extract some value to the label.
DeviceOSFreeCapacity(B) -> DeviceOSFreeBytes
DeviceOSTotalCapacity(B) -> DeviceOSTotalBytes
DeviceCelebornFreeCapacity(B) -> DeviceCelebornFreeBytes
DeviceCelebornTotalCapacity(B) -> DeviceCelebornTotalBytes
push usedHeapMemory/usedDirectMemory
fetch usedHeapMemory/usedDirectMemory
replicate usedHeapMemory/usedDirectMemory

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

Closes #1557 from onebox-li/improve-metrics.

Authored-by: onebox-li <lyh-36@163.com>
Signed-off-by: Cheng Pan <chengpan@apache.org>
2023-06-08 18:10:06 +08:00
Angerszhuuuu
2f054cd7d5 [CELEBORN-647][BUG] Fix potential NPE when remove push status
### What changes were proposed in this pull request?
Fix potential NPE when remove push status

### Why are the changes needed?

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

### How was this patch tested?

Closes #1559 from AngersZhuuuu/CELEBORN-647.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Angerszhuuuu <angers.zhu@gmail.com>
2023-06-08 17:22:48 +08:00
zhongqiang.czq
586785c88d [CELEBORN-617][FLINK] MapPartitionFileWriter updates flushing file length
…ngth

### What changes were proposed in this pull request?

### Why are the changes needed?

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

### How was this patch tested?

Closes #1519 from zhongqiangczq/mapfilelength.

Authored-by: zhongqiang.czq <zhongqiang.czq@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-06-08 10:47:36 +08:00
Cheng Pan
5bc37f1286
[CELEBORN-637] Remove support for rss.* configuration alias
### What changes were proposed in this pull request?

Remove support for `rss.*` configuration alias

### Why are the changes needed?

The legacy `rss.*` configuration alias was added during Celeborn entering Apache Incubator, to simplify users' migration from RSS to Celeborn.

Lots of configuration changes happened after Celeborn 0.2, the `rss.*` configuration alias become less helpful, so remove it to clean up the code.

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

Yes, but it's expected, the `rss.*` compatibility has never been documented.

### How was this patch tested?

Pass GA.

Closes #1547 from pan3793/CELEBORN-637.

Authored-by: Cheng Pan <chengpan@apache.org>
Signed-off-by: Cheng Pan <chengpan@apache.org>
2023-06-07 22:28:36 +08:00
Ethan Feng
76a42beab0
[CELEBORN-610][FLINK] Eliminate pluginconf and merge its content to CelebornConf
### What changes were proposed in this pull request?
Pluginconf might be hard to understand why Celeborn needs to config class.

### Why are the changes needed?
Ditto.

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

### How was this patch tested?
UT.

Closes #1524 from FMX/CELEBORN-610.

Authored-by: Ethan Feng <ethanfeng@apache.org>
Signed-off-by: Ethan Feng <ethanfeng@apache.org>
2023-06-05 14:08:53 +08:00
Angerszhuuuu
4df4775524
[CELEBORN-632][DOC] Add spark name space to spark specify properties (#1538) 2023-06-02 21:48:56 +08:00
Angerszhuuuu
e18a5ea769
[CELEBORN-624] StorageManager should only remove expired app dirs (#1531) 2023-06-02 11:33:33 +08:00
Ethan Feng
d33916e571
[CELEBORN-625] Add a config to enable/disable UnsafeRow fast write. (#1532) 2023-06-01 20:55:45 +08:00
Angerszhuuuu
cf308aa057
[CLEBORN-595] Refine code frame of CelebornConf (#1525) 2023-06-01 10:37:58 +08:00
Angerszhuuuu
6d5dd50915
[CELEBORN-595][FOLLOWUP] Fix change version to 0.3.0. (#1522) 2023-05-30 20:12:56 +08:00
Angerszhuuuu
62681ba85d
[CELEBORN-595] Rename and refactor the configuration doc. (#1501) 2023-05-30 15:14:12 +08:00
zhongqiangchen
f117cff776
[CELEBORN-618] [FLINK] worker side adds partition split configuration options (#1520) 2023-05-30 14:13:31 +08:00
Angerszhuuuu
07011f5a4d
[CELEBORN-601] Consolidate configsWithAlternatives with ConfigBuilder.withAlternative (#1506) 2023-05-28 09:13:05 +08:00
Leo Li
de97ad26ce
[CELEBORN-599] Consolidate calculation of mount point (#1505)
* [CELEBORN-599] Fix worker dirs get mount point

* update

* update

---------

Co-authored-by: liyihe <liyihe@bigo.sg>
2023-05-23 14:06:02 +08:00
Angerszhuuuu
6619015a63
[CELEBORN-596] Worker don't need to update disk max slots (#1502) 2023-05-23 10:30:35 +08:00
Angerszhuuuu
d244f44518
[CELEBORN-593] Refine some RPC related default configurations (#1498) 2023-05-19 18:23:12 +08:00
Angerszhuuuu
615d9a111f
[CELEBORN-487] Remove wrong space of config SHUFFLE_CLIENT_PUSH_BLACK (#1500) 2023-05-19 14:27:57 +08:00
Ethan Feng
ac78afdc4e
[CELEBORN-594] Eliminate Ratis noisy logs. (#1499) 2023-05-19 14:05:52 +08:00
Angerszhuuuu
42219aeb2a
[CELEBORN-592][REFACTOR] Refactor PbSerDeUtils's some foreach code format (#1497) 2023-05-18 16:22:14 +08:00
Shuang
6eabc519b3
[CELEBORN-591] RatisSystem need decrease no leader timeout configuration. (#1495) 2023-05-18 14:49:06 +08:00
Angerszhuuuu
811e192bbd
[CELEBORN-446] Support rack aware during assign slots for ROUNDROBIN (#1370) 2023-05-18 13:58:51 +08:00
Ethan Feng
7015d2463a
[CELEBORN-583] Merge pooled memory allocators. (#1490) 2023-05-18 10:37:30 +08:00
Angerszhuuuu
791d72d45f
[CELEBORN-590] Remove hadoop prefix of WORKER_WORKING_DIR (#1494) 2023-05-17 17:57:27 +08:00
Angerszhuuuu
7c6cb2f3bb
[CELEBORN-588] Remove test conf's category (#1491) 2023-05-17 17:37:28 +08:00
Angerszhuuuu
64a3534f71
[CELEBORN-584] Worker side should expose push/replicate/fetch Netty allocator metrics (#1489) 2023-05-16 17:51:33 +08:00
Shuang
f83304c337
[CELEBORN-581][Flink] Support JobManager failover. (#1485) 2023-05-16 14:51:53 +08:00
Angerszhuuuu
d657f8268a
[CELEBORN-586] Add SystemMiscSource to indicate system running status (#1488) 2023-05-16 14:03:07 +08:00
zhongqiangchen
5769c3fdc7
[CELEBORN-552] Add HeartBeat between the client and worker to keep alive (#1457) 2023-05-10 19:35:51 +08:00
Shuang
fb753fd48e
[CELEBORN-573] Guarantee resource/app/worker change persistent to raft in Ha Mode. (#1477) 2023-05-10 14:28:52 +08:00
Angerszhuuuu
778b5440bc
[CELEBORN-556][BUG] ReserveSlot should not use default RPC time out since register shuffle max timeout is network timeout (#1461) 2023-05-10 12:29:06 +08:00
Shuang
2fea818fa8
[CELEBORN-579] revert Destroy Message rename for compatibility. (#1482) 2023-05-09 15:24:02 +08:00
Ethan Feng
3e0d779962
[CELEBORN-576] Add static identity provider and manually settable identity provider for non-hadoop environment. (#1480) 2023-05-08 17:29:01 +08:00
Angerszhuuuu
a315a2eb41
[CELEBORN-575] PartitionLocationInfo change cause quick upgrade impacted (#1479) 2023-05-08 16:56:42 +08:00
Angerszhuuuu
ef4c12e0fe
[CELEBORN-565] FETCH_MAX_RETRIES should double when enable replicates (#1471) 2023-04-28 14:27:35 +08:00
Angerszhuuuu
bfce6052d7
[CELEBORN-560][FOLLOWUP] Follow the original design for handling rerun & speculative task after handleStageEnd (#1468) 2023-04-28 11:18:42 +08:00
Angerszhuuuu
7a4f2ebd8a
[CELEBORN-547] Refactor request related API (#1452) 2023-04-27 16:25:41 +08:00
Angerszhuuuu
be84e8ba0d
[CELEBORN-562][REFACTOR] Rename Destroy and DestroyResponse to make it more clear (#1467) 2023-04-27 12:31:32 +08:00
Shuang
64a4f7274c
[CELEBORN-554][Tuning] Improve For LM to avoid reserve/commit empty worker resources (#1459) 2023-04-26 18:04:50 +08:00
Angerszhuuuu
13ce04f8a1
[CELEBORN-557] HA_CLIENT_RPC_ASK_TIMEOUT should fallback to RPC_ASK_TIMEOUT (#1462)
* [CELEBORN-557] HA_CLIENT_RPC_ASK_TIMEOUT should fallback to RPC_ASK_TIMEOUT
2023-04-26 15:19:34 +08:00
Shuang
0b2e4877bd
[CELEBORN-553] Improve IO (#1458) 2023-04-25 21:14:06 +08:00
Shuang
d68deecaaa
[CELEBORN-546][FLINK] Use autoIncrement partitionId replace encode(mapId, attemptId) for generating partitionId (#1447) 2023-04-22 16:33:22 +08:00
Angerszhuuuu
181c1bfcd6
[CELEBORN-524][PERF] CongestionControl call too much ChannelsLimiter onTrim cause CPU stuck or occupy too much CPU cause no cpu for handlePushData (#1428) 2023-04-21 15:44:56 +08:00
Angerszhuuuu
6830cb61ef
[CELEBORN-540][Refactor] Add config entity of celeborn.rpc.io.threads (#1443)
* [CELEBORN-540][CONF] Add config entity of celeborn.rpc.io.threads
2023-04-21 11:21:41 +08:00
Shuang
62d60de8c5
[CELEBORN-537] Improve blacklist compute & minor fix for Flink (#1441)
[CELEBORN-537] improve blacklist compute & minor fix for flink
2023-04-20 18:30:10 +08:00
Ethan Feng
6378a386d0
[CELEBORN-530][REFACTOR] Move stream manager and memory manager to worker module. (#1439) 2023-04-20 10:17:26 +08:00
Ethan Feng
8be82548e1
[CELEBORN-520][FLINK] Tune map partition reading performance. (#1424) 2023-04-17 16:47:09 +08:00
Shuang
412d10b7dc
[CELEBORN-479][FLINK] support stopTrackingAndReleasePartitions when worker is not available (#1405) 2023-04-17 14:44:24 +08:00
Angerszhuuuu
938aec0e9f
[CELEBORN-528][REFACTOR] limitZeroInFlight should show inflight target (#1433) 2023-04-17 11:53:34 +08:00
Angerszhuuuu
932ccd0841
[CELEBORN-523][REFACTOR] Remove unnecessary code in WorkerPartitionLocationInfo (#1427) 2023-04-15 22:36:48 +08:00
Shuang
a22c6ca749
[CELEBORN-521] correct exception and unify unRetryableException (#1425) 2023-04-15 22:27:28 +08:00
Angerszhuuuu
3a21362265
[CELEBORN-511][IMPROVE] Move onTrim tag to StorageManager to avoid frequent trim action (#1415)
* [CELEBORN-511][IMPROVE] Move onTrim tag to StorageManager to avoid frequent trim action
2023-04-14 10:35:51 +08:00
Angerszhuuuu
480d7ac0d9
[CELEBORN-519][PERF] getMaster/SlaveLocation directly use uniqueId as key (#1421) 2023-04-13 21:53:33 +08:00
Ethan Feng
9cccfc9872
[CELEBORN-431][FLINK] Support dynamic buffer allocation in reading map partition. (#1407) 2023-04-13 10:37:47 +08:00
Angerszhuuuu
32b497973e
[CELEBORN-517][IMPROVEMENT] Optimize stopTimer/startTimer cpu cost (#1419) 2023-04-12 20:12:01 +08:00
Angerszhuuuu
da98ed9bea
[CELEBORN-516][PERF] Remove RPCSource since it cost too much CPU (#1420) 2023-04-12 18:47:06 +08:00
Angerszhuuuu
e5722126e9
[CELEBORN-502][REFACTOR] Merge GetBlacklistResponse to HeartbeatFromApplication (#1408)
* [CELEBORN-502][REFACTOR] Merge GetBlacklistResponse to HeartbeatFromApplication
2023-04-12 14:59:32 +08:00
Keyong Zhou
7dd2230a04
[CELEBORN-510][FLINK] DataPartitionReader.addBuffer should not call s… (#1413) 2023-04-07 18:17:55 +08:00
Shuang
9b2b8a01ec
[CELEBORN-507] don't set up worker endpoint when update meta and remove compare worker meta with workers (#1412) 2023-04-07 11:46:24 +08:00
Angerszhuuuu
cad2836e85
[CELEBORN-505] Fix typo of SHUFFLE_CHUCK_SIZE (#1411) 2023-04-04 19:15:30 +08:00
Keyong Zhou
2e1598c011
[CELEBORN-485] Make celeborn.push.replicate.enabled default to false (#1394) 2023-04-03 16:36:29 +08:00
Angerszhuuuu
bf46336d54
[CELEBORN-487][PERF] ShuffleClientSide support blacklist to avoid client side timeout in same worker multiple times (#1399) 2023-04-03 11:50:04 +08:00
Angerszhuuuu
b4f8ab19bd
[CELEBORN-484][PERF] Master trigger LifecycleManager commit shutdown worker's partition location. (#1395)
* [CELEBORN-484][PERF] Master trigger LifecycleManager commit shutdown worker's  partition location.
2023-04-02 09:18:12 +08:00
Keyong Zhou
61416a828d
[CELEBORN-497]Fix and enable JDK 11 for CI (#1401) 2023-03-31 13:39:02 +08:00
Shuang
45013b8bae
[CELEBORN-489][FLINK]fix retry client for open stream (#1397) 2023-03-30 11:44:19 +08:00
zhongqiangchen
cd92c423cd
[CELEBORN-475] Support extra tags for prometheus metrics (#1385)
[CELEBORN-475] Support extra tags for prometheus metrics
2023-03-28 21:22:28 +08:00
Ethan Feng
6cee85748d
[CELEBORN-477][FLINK] Report failed partition to flink framework. (#1391) 2023-03-28 15:54:37 +08:00
Keyong Zhou
cb19ed1c66
[CELEBORN-479][PERF] Refactor DataPushQueue.takePushTask to avoid busy wait (#1386) 2023-03-27 16:18:55 +08:00
Fei Wang
b40c573069
[CELEBORN-474][FOLLOWUP] Using inner static ConcurrentHashMap class and only apply for JDK8 (#1384) 2023-03-27 16:16:23 +08:00
Fei Wang
7c444cb0c5
[CELEBORN-474] Speed up ConcurrentHashMap#computeIfAbsent (#1383) 2023-03-26 09:41:59 +08:00
Fei Wang
c609c0ebaa
[MINOR] Fix typo and remove unused code (#1381)
* fix typo

* remove unused
2023-03-25 23:20:33 +08:00
Angerszhuuuu
acf6fd3bd2
[CELEBORN-345] TransportResponseHandler create too much thread (#1373) 2023-03-24 17:16:26 +08:00
Shuang
89b3f3887d
[CELEBORN-356] [FLINK] Support release single partition resource (#1314) 2023-03-24 17:15:28 +08:00