Commit Graph

303 Commits

Author SHA1 Message Date
mingji
d0ecf83fec [CELEBORN-764] Fix celeborn on HDFS might clean using app directories
### What changes were proposed in this pull request?
Make Celeborn leader clean expired app dirs on HDFS when an application is Lost.

### Why are the changes needed?
If Celeborn is working on HDFS, the storage manager starts and cleans expired app directories, and the newly created worker will want to delete any unknown app directories.
This will cause using app directories to be deleted unexpectedly.

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

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

Closes #1678 from FMX/CELEBORN-764.

Lead-authored-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
Co-authored-by: Cheng Pan <pan3793@gmail.com>
Co-authored-by: Ethan Feng <fengmingxiao.fmx@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-07-05 23:11:50 +08:00
Angerszhuuuu
693172d0bd [CELEBORN-751] Rename remain rss related class name and filenames etc
### What changes were proposed in this pull request?
Rename remain rss related class name and filenames etc...

### Why are the changes needed?

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

### How was this patch tested?

Closes #1664 from AngersZhuuuu/CELEBORN-751.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Angerszhuuuu <angers.zhu@gmail.com>
2023-07-04 10:20:08 +08:00
zky.zhoukeyong
af0f5e5784 [CELEBORN-755][FOLLOWUP] Avoid unnecessary memory copy when compression disabled
…sion disabled

### What changes were proposed in this pull request?
Avoid memory copy for code path where compression is disabled. Followup of https://github.com/apache/incubator-celeborn/pull/1669

### Why are the changes needed?
ditto

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

### How was this patch tested?
Pass GA

Closes #1671 from waitinfuture/755.

Authored-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-07-01 18:27:33 +08:00
xiyu.zk
381165d4e7
[CELEBORN-755] Support disable shuffle compression
### What changes were proposed in this pull request?
Support to decide whether to compress shuffle data through configuration.

### Why are the changes needed?
Currently, Celeborn compresses all shuffle data, but for example, the shuffle data of Gluten has already been compressed. In this case, no additional compression is required. Therefore, configuration needs to be provided for users to decide whether to use Celeborn’s compression according to the actual situation.

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

Closes #1669 from kerwin-zk/celeborn-755.

Authored-by: xiyu.zk <xiyu.zk@alibaba-inc.com>
Signed-off-by: Cheng Pan <chengpan@apache.org>
2023-07-01 00:03:50 +08:00
Fu Chen
adbd38a926
[CELEBORN-726][FOLLOWUP] Update data replication terminology from master/slave to primary/replica in the codebase
### What changes were proposed in this pull request?

As title

### Why are the changes needed?

In order to distinguish it from the existing master/worker, refactor data replication terminology to 'primary/replica' for improved clarity and inclusivity in the codebase

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

No

### How was this patch tested?

existing tests

Closes #1639 from cfmcgrady/primary-replica.

Lead-authored-by: Fu Chen <cfmcgrady@gmail.com>
Co-authored-by: Cheng Pan <chengpan@apache.org>
Signed-off-by: Cheng Pan <chengpan@apache.org>
2023-06-29 17:07:26 +08:00
Angerszhuuuu
1fd8833756
[CELEBORN-748] Rename RssHARetryClient to MasterClient
### What changes were proposed in this pull request?

Rename RssHARetryClient to MasterClient

### Why are the changes needed?

Code refactor

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

No.

### How was this patch tested?

Pass GA.

Closes #1661 from AngersZhuuuu/CELEBORN-748.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Cheng Pan <chengpan@apache.org>
2023-06-29 16:47:15 +08:00
Fu Chen
17c1e01874
[CELEBORN-726] Update data replication terminology from master/slave to primary/replica for configurations and metrics
### What changes were proposed in this pull request?

This pull PR is an integral component of #1639 . It primarily focuses on updating configuration settings and metrics terminology, while ensuring compatibility with older client versions by refraining from introducing changes related to RPC.

### Why are the changes needed?

In order to distinguish it from the existing master/worker, refactor data replication terminology to 'primary/replica' for improved clarity and inclusivity in the codebase

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

No

### How was this patch tested?

existing tests.

Closes #1650 from cfmcgrady/primary-replica-metrics.

Authored-by: Fu Chen <cfmcgrady@gmail.com>
Signed-off-by: Cheng Pan <chengpan@apache.org>
2023-06-29 09:47:02 +08:00
onebox-li
1b74d85fb1 [CELEBORN-725][MINOR] Refine congestion code
### What changes were proposed in this pull request?
Refine the congestion relevant code/log/comments

### Why are the changes needed?
ditto

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

### How was this patch tested?
manually test

Closes #1637 from onebox-li/improve-congestion.

Authored-by: onebox-li <lyh-36@163.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-06-28 18:31:40 +08:00
Cheng Pan
3d7c1fa0ae [CELEBORN-729] Fix typo PbRegisterShuffle#numMappers
### What changes were proposed in this pull request?

Fix typo `numMapppers`, should be `numMappers`

### Why are the changes needed?

Fix typo

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

No.

### How was this patch tested?

Protobuf serde depends on message field seq no, not name.

Closes #1642 from pan3793/CELEBORN-729.

Authored-by: Cheng Pan <chengpan@apache.org>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-06-28 18:28:34 +08:00
Angerszhuuuu
afab4a0a3b [CELEBORN-696][FOLLOWUP] Remove new allocated peer workers from pushExecludedWrkers
### What changes were proposed in this pull request?
Remove new allocated location's workers from pushExecludedWrkers should also remove peers

### Why are the changes needed?

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

### How was this patch tested?

Closes #1636 from AngersZhuuuu/CELEBORN-696-FOLLOWUP.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-06-28 17:38:36 +08:00
Angerszhuuuu
3985a5cbd7 [CELEBORN-666][FOLLOWUP] Unify all blacklist related code and comment
### What changes were proposed in this pull request?
Unify all blacklist related code and comment

### Why are the changes needed?

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

### How was this patch tested?

Closes #1638 from AngersZhuuuu/CELEBORN-666-FOLLOWUP.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-06-28 16:28:03 +08:00
zhongqiang.czq
374d735ae5
[CELEBORN-724] Fix the compatibility of HeartbeatFromApplicationRespo…
…nse with lower versions

### What changes were proposed in this pull request?
The master side will check HeartbeatFromApplication's reply field. if reply is true then it replies HeartbeatFromApplicationResponse otherwise OneWayMessageResponse.

The reply field is default false before the version 0.2.1, so master can be compatible with older client version

### Why are the changes needed?
Before the version `0.2.1`, the response of HeartbeatFromApplication is` OneWayMessageResponse`, but from `0.3.0`, the response of HeartbeatFromApplication is modified to `HeartbeatFromApplicationResponse`.
if the version of `client side `is `0.2.1` and the version of `server side is 0.3.0`, the `compatiblity issue `will occur.
The following compatiblity error will be printted.

``` java
java.io.InvalidObjectException: enum constant HEARTBEAT_FROM_APPLICATION_RESPONSE does not exist in class org.apache.celeborn.common.protocol.MessageType
	at java.io.ObjectInputStream.readEnum(ObjectInputStream.java:2157) ~[?:1.8.0_362]
	at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1662) ~[?:1.8.0_362]
	at java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:2430) ~[?:1.8.0_362]
	at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:2354) ~[?:1.8.0_362]
	at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:2212) ~[?:1.8.0_362]
	at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1668) ~[?:1.8.0_362]
	at java.io.ObjectInputStream.readObject(ObjectInputStream.java:502) ~[?:1.8.0_362]
	at java.io.ObjectInputStream.readObject(ObjectInputStream.java:460) ~[?:1.8.0_362]
	at org.apache.celeborn.common.serializer.JavaDeserializationStream.readObject(JavaSerializer.scala:76) ~[celeborn-client-spark-3-shaded_2.12-0.2.1-incubating.jar:?]
```
``` java
Caused by: java.lang.ClassCastException: Cannot cast org.apache.celeborn.common.protocol.message.ControlMessages$HeartbeatFromApplicationResponse to org.apache.celeborn.common.protocol.message.ControlMessages$OneWayMessageResponse$
	at java.lang.Class.cast(Class.java:3369) ~[?:1.8.0_362]
	at scala.concurrent.Future.$anonfun$mapTo$1(Future.scala:500) ~[scala-library-2.12.15.jar:?]
	at scala.util.Success.$anonfun$map$1(Try.scala:255) ~[scala-library-2.12.15.jar:?]
	at scala.util.Success.map(Try.scala:213) ~[scala-library-2.12.15.jar:?]
	at scala.concurrent.Future.$anonfun$map$1(Future.scala:292) ~[scala-library-2.12.15.jar:?]
	at scala.concurrent.impl.Promise.liftedTree1$1(Promise.scala:33) ~[scala-library-2.12.15.jar:?]
	at scala.concurrent.impl.Promise.$anonfun$transform$1(Promise.scala:33) ~[scala-library-2.12.15.jar:?]
	at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:64) ~[scala-library-2.12.15.jar:?]
	at scala.concurrent.BatchingExecutor$Batch.processBatch$1(BatchingExecutor.scala:67) ~[scala-library-2.12.15.jar:?]
	at scala.concurrent.BatchingExecutor$Batch.$anonfun$run$1(BatchingExecutor.scala:82) ~[scala-library-2.12.15.jar:?]
	at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23) ~[scala-library-2.12.15.jar:?]
	at scala.concurrent.BlockContext$.withBlockContext(BlockContext.scala:85) ~[scala-library-2.12.15.jar:?]
	at scala.concurrent.BatchingExecutor$Batch.run(BatchingExecutor.scala:59) ~[scala-library-2.12.15.jar:?]
	at scala.concurrent.Future$InternalCallbackExecutor$.unbatchedExecute(Future.scala:875) ~[scala-library-2.12.15.jar:?]
	at scala.concurrent.BatchingExecutor.execute(BatchingExecutor.scala:110) ~[scala-library-2.12.15.jar:?]
	at scala.concurrent.BatchingExecutor.execute$(BatchingExecutor.scala:107) ~[scala-library-2.12.15.jar:?]
	at scala.concurrent.Future$InternalCallbackExecutor$.execute(Future.scala:873) ~[scala-library-2.12.15.jar:?]
	at scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:72) ~[scala-library-2.12.15.jar:?]
	at scala.concurrent.impl.Promise$DefaultPromise.$anonfun$tryComplete$1(Promise.scala:288) ~[scala-library-2.12.15.jar:?]
	at scala.concurrent.impl.Promise$DefaultPromise.$anonfun$tryComplete$1$adapted(Promise.scala:288) ~[scala-library-2.12.15.jar:?]
	at scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:288) ~[scala-library-2.12.15.jar:?]
	at scala.concurrent.Promise.trySuccess(Promise.scala:94) ~[scala-library-2.12.15.jar:?]
	at scala.concurrent.Promise.trySuccess$(Promise.scala:94) ~[scala-library-2.12.15.jar:?]
	at scala.concurrent.impl.Promise$DefaultPromise.trySuccess(Promise.scala:187) ~[scala-library-2.12.15.jar:?]
	at org.apache.celeborn.common.rpc.netty.NettyRpcEnv.onSuccess$1(NettyRpcEnv.scala:218) ~[celeborn-client-spark-3-shaded_2.12-0.2.1-incubating.jar:?]
```

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

No

### How was this patch tested?
The pr is tested manually and the testing process is as follows:
1. server side is deploy using the code of latest branch-0.3.
2. spark client is deploy the version of 0.2.1, then run spark-sql to execute  3 tpcds queries( query1.sql/querey2/quere3.sql whose datasize is 1T), finnally verify that the queries are executed successfully and no above compatiblity error printted
3. spark client is deploy the version of 0.3.0,  then run spark-sql to execute 3 tpcds queries( query1.sql/querey2/quere3.sql whose datasize is 1T), finnally verify that the queries are executed successfully and no above compatiblity error printted

This patch had conflicts when merged, resolved by
Committer: Cheng Pan <chengpan@apache.org>

Closes #1635 from zhongqiangczq/heartbeat2.

Authored-by: zhongqiang.czq <zhongqiang.czq@alibaba-inc.com>
Signed-off-by: Cheng Pan <chengpan@apache.org>
2023-06-28 16:04:18 +08:00
Angerszhuuuu
33cf343d20 [CELEBORN-666][REFACTOR] Unify exclude and blacklist related configuration
### What changes were proposed in this pull request?
Unify exclude and blacklist related configuration

### Why are the changes needed?

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

### How was this patch tested?

Closes #1633 from AngersZhuuuu/CELEBORN-666-NEW.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Angerszhuuuu <angers.zhu@gmail.com>
2023-06-28 10:59:58 +08:00
zky.zhoukeyong
57b0e815cf [CELEBORN-656] Batch revive RPCs in client to avoid too many requests
### What changes were proposed in this pull request?
This PR batches revive requests and periodically send to LifecycleManager to reduce number or RPC requests.

To be more detailed. This PR changes Revive message to support multiple unique partitions, and also passes a set unique mapIds for checking MapEnd. Each time ShuffleClientImpl wants to revive, it adds a ReviveRquest to ReviveManager and wait for result. ReviveManager batches revive requests and periodically send to LifecycleManager (deduplicated by partitionId). LifecycleManager constructs ChangeLocationsCallContext and after all locations are notified, it replies to ShuffleClientImpl.

### Why are the changes needed?
In my test 3T TPCDS q23a with 3 Celeborn workers, when kill a worker, the LifecycleManger will receive 4.8w Revive requests:
```
[emr-usermaster-1-1 logs]$ cat spark-emr-user-org.apache.spark.sql.hive.thriftserver.HiveThriftServer2-1-master-1-1.c-fa08904e94c028d1.out.1 |grep -i revive |wc -l
64364
```
After this PR, number of ReviveBatch requests reduces to 708:
```
[emr-usermaster-1-1 logs]$ cat spark-emr-user-org.apache.spark.sql.hive.thriftserver.HiveThriftServer2-1-master-1-1.c-fa08904e94c028d1.out |grep -i revive |wc -l
2573
```

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

### How was this patch tested?
Manual test. I have tested:

1. Disable graceful shutdown, kill one worker, job succeeds
2. Disable graceful shutdown, kill two workers successively, job fails as expected
3. Enable graceful shutdown, restart two workers successively, job succeeds
4. Enable graceful shutdown, restart two workers successively, then kill the third one, job succeeds

Closes #1588 from waitinfuture/656-2.

Lead-authored-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
Co-authored-by: Keyong Zhou <zhouky@apache.org>
Co-authored-by: Keyong Zhou <waitinfuture@gmail.com>
Signed-off-by: Shuang <lvshuang.tb@gmail.com>
2023-06-27 22:11:04 +08:00
zky.zhoukeyong
ebff17ec3c
[CELEBORN-721] Fix concurrent bug in ChangePartitionManager
### What changes were proposed in this pull request?
Fixes concurrent bug in ChangePartitionManager.

### Why are the changes needed?
Before this PR, ```ChangePartitionManager.start``` tries to synchronize on ```requests``` in the body
of ```run()```, but the synchronized keyword was put outside of the ```batchHandleChangePartitionExecutors.submit```,
which has no effect.

When I was testing https://github.com/apache/incubator-celeborn/pull/1588 , I encountered unexpected situations that
when all ```rss-lifecycle-manager-change-partition-executor``` threads are idle, the ```inBatchPartitions``` is still not
empty:
```
23/06/27 20:35:55 INFO ChangePartitionManager: Inside run, shuffleId 0 inBatchPartitions size 834
```

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

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

Closes #1634 from waitinfuture/721.

Authored-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
Signed-off-by: Cheng Pan <chengpan@apache.org>
2023-06-27 21:30:47 +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
zky.zhoukeyong
809c76a2e4 [CELEBORN-718] Decrease RemainingReviveTimes regardless worker is excluded or not
…s excluded or not

### What changes were proposed in this pull request?
This PR makes ReviveTimes decrease regardless of the partition location is excluded or not.

### Why are the changes needed?
In such testing setup:

- 3 Celeborn workers
- Client side blacklist enabled ```spark.celeborn.client.push.blacklist.enabled=true```
- Replication is on ```spark.celeborn.client.push.replicate.enabled=true```
- Successively kill 2 workers

I expect the task fail because of revive failure (When replication is on, we need at least 2 workers), but in stead
the tasks hang forever. When digging into the logs I found the ```remain revive times``` does not decrease, leading
to infinite revive loop.
```
23/06/27 14:00:57 ERROR ShuffleClientImpl: Push data to xxx:xxx failed for shuffle 0 map 998 attempt 1 partition 666 batch 1, remain revive times 5.
23/06/27 14:01:01 ERROR ShuffleClientImpl: Push data to xxx:xxx failed for shuffle 0 map 998 attempt 1 partition 666 batch 1, remain revive times 5.
23/06/27 14:01:05 ERROR ShuffleClientImpl: Push data to xxx:xxx failed for shuffle 0 map 998 attempt 1 partition 666 batch 1, remain revive times 5.
23/06/27 14:01:09 ERROR ShuffleClientImpl: Push data to xxx:xxx failed for shuffle 0 map 998 attempt 1 partition 666 batch 1, remain revive times 5.
23/06/27 14:01:13 ERROR ShuffleClientImpl: Push data to xxx:xxx failed for shuffle 0 map 998 attempt 1 partition 666 batch 1, remain revive times 5.
23/06/27 14:01:17 ERROR ShuffleClientImpl: Push data to xxx:xxx failed for shuffle 0 map 998 attempt 1 partition 666 batch 1, remain revive times 5.
23/06/27 14:01:21 ERROR ShuffleClientImpl: Push data to xxx:xxx failed for shuffle 0 map 998 attempt 1 partition 666 batch 1, remain revive times 5.
23/06/27 14:01:25 ERROR ShuffleClientImpl: Push data to xxx:xxx failed for shuffle 0 map 998 attempt 1 partition 666 batch 1, remain revive times 5.
23/06/27 14:01:29 ERROR ShuffleClientImpl: Push data to xxx:xxx failed for shuffle 0 map 998 attempt 1 partition 666 batch 1, remain revive times 5.
23/06/27 14:01:33 ERROR ShuffleClientImpl: Push data to xxx:xxx failed for shuffle 0 map 998 attempt 1 partition 666 batch 1, remain revive times 5.
23/06/27 14:01:37 ERROR ShuffleClientImpl: Push data to xxx:xxx failed for shuffle 0 map 998 attempt 1 partition 666 batch 1, remain revive times 5.
23/06/27 14:01:41 ERROR ShuffleClientImpl: Push data to xxx:xxx failed for shuffle 0 map 998 attempt 1 partition 666 batch 1, remain revive times 5.
23/06/27 14:01:45 ERROR ShuffleClientImpl: Push data to xxx:xxx failed for shuffle 0 map 998 attempt 1 partition 666 batch 1, remain revive times 5.
23/06/27 14:01:49 ERROR ShuffleClientImpl: Push data to xxx:xxx failed for shuffle 0 map 998 attempt 1 partition 666 batch 1, remain revive times 5.
23/06/27 14:01:53 ERROR ShuffleClientImpl: Push data to xxx:xxx failed for shuffle 0 map 998 attempt 1 partition 666 batch 1, remain revive times 5.
23/06/27 14:01:57 ERROR ShuffleClientImpl: Push data to xxx:xxx failed for shuffle 0 map 998 attempt 1 partition 666 batch 1, remain revive times 5.
23/06/27 14:02:01 ERROR ShuffleClientImpl: Push data to xxx:xxx failed for shuffle 0 map 998 attempt 1 partition 666 batch 1, remain revive times 5.
23/06/27 14:02:05 ERROR ShuffleClientImpl: Push data to xxx:xxx failed for shuffle 0 map 998 attempt 1 partition 666 batch 1, remain revive times 5.
23/06/27 14:02:09 ERROR ShuffleClientImpl: Push data to xxx:xxx failed for shuffle 0 map 998 attempt 1 partition 666 batch 1, remain revive times 5.
23/06/27 14:02:13 ERROR ShuffleClientImpl: Push data to xxx:xxx failed for shuffle 0 map 998 attempt 1 partition 666 batch 1, remain revive times 5.
```

The reason is before this PR, the revive times will not decrease if the partition location is excluded, which I don't see a
reason for that.

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

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

Closes #1628 from waitinfuture/718.

Authored-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-06-27 15:21:09 +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
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
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
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
zky.zhoukeyong
222ed267b0
[CELEBORN-692] WorkerStatusTracker should handle WORKER_SHUTDOWN properly
### What changes were proposed in this pull request?
This PR put workers with WORKER_SHUTDOWN status into shuttingWorkers instead of blacklist.

### Why are the changes needed?
If WORKER_SHUTDOWN workers are put into blacklist, it will not trigger commit files, see ```CommitHandler::commitFiles```

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

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

Closes #1603 from waitinfuture/692.

Authored-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
Signed-off-by: Cheng Pan <chengpan@apache.org>
2023-06-19 15:54:45 +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
Shuang
28a99ded8d [CELEBORN-687] Fix shuffleResourceExists, reduce unexpected slot release request
### What changes were proposed in this pull request?
Check ShufflePartitionLocationInfo whether empty or not for every worker

### Why are the changes needed?
Actually shuffleResources would only remove related partitionLocations after stageEnd , then workers with empty partitionLocations will left(for speculative task), so shuffleResourceExists need check ShufflePartitionLocationInfo for every worker otherwise it would print wrong message and send release slot requests twice.

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

### How was this patch tested?
Manual test

### Before this pr

<img width="1252" alt="image" src="https://github.com/apache/incubator-celeborn/assets/28799061/06b71162-e78b-4163-8f52-24b50bc6c540">

![image](https://github.com/apache/incubator-celeborn/assets/28799061/fec263e0-9641-4d17-a837-ab03c36c5e6d)

### After this pr

![image](https://github.com/apache/incubator-celeborn/assets/28799061/8f2f9653-ff58-4a6e-ae06-1023922ca5bf)

Closes #1599 from RexXiong/CELEBORN-687.

Authored-by: Shuang <lvshuang.tb@gmail.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-06-16 18:37:48 +08:00
Angerszhuuuu
c1c46398d5 [CELEBORN-682] Master and client should handle blacklist worker and shutting down worker separately
### 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 #1594 from AngersZhuuuu/CELEBORN-682.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-06-16 18:29:03 +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
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
onebox-li
47f66a87a1 [CELEBORN-678] ShuffleClientImpl::mapperEnded should not consider attemptId
### What changes were proposed in this pull request?
ShuffleClientImpl::mapperEnded should not consider attemptId, speculation tasks will update attemptId.

### Why are the changes needed?
ditto

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

### How was this patch tested?
Cluster test

Closes #1591 from onebox-li/fix-mapend.

Authored-by: onebox-li <lyh-36@163.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-06-14 21:01:06 +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
Angerszhuuuu
6177476b18 [CELEBORN-547][FOLLOWUP] Fix wrong method name
### What changes were proposed in this pull request?
Fix wrong method name

### Why are the changes needed?

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

### How was this patch tested?

Closes #1582 from AngersZhuuuu/CELEBORN-547-FOLLOWUP.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Angerszhuuuu <angers.zhu@gmail.com>
2023-06-13 12:22:47 +08:00
Angerszhuuuu
0bf841006a [CELEBORN-663][REFACTOR] Refine RssInputStream logs to help admin debug
### What changes were proposed in this pull request?
Refine RssInputStream logs to indicate the failed location's info to help admin debug

### Why are the changes needed?

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

### How was this patch tested?

Closes #1576 from AngersZhuuuu/CELEBORN-663.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Angerszhuuuu <angers.zhu@gmail.com>
2023-06-12 18:49:41 +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
Angerszhuuuu
45503238b3 [CELEBORN-657][BUG] DataPushQueue return task should always remove iterator
### What changes were proposed in this pull request?
 DataPushQueue return task should always remove iterator
Related to
251b923b5b
cb19ed1c66

### Why are the changes needed?

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

### How was this patch tested?

Closes #1568 from AngersZhuuuu/CELEBORN-657.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Shuang <lvshuang.tb@gmail.com>
2023-06-09 13:37:07 +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
Angerszhuuuu
9502b1f26d [CELEBORN-639][BUG] ShuffleClient get push exception cause should handle NPE
### What changes were proposed in this pull request?
If we meet some unexpected exception, `getPushDataFailCause ` will throw NPE and broke the process of revive and remove push states. Here we should handle the NPE

### Why are the changes needed?

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

### How was this patch tested?

Closes #1551 from AngersZhuuuu/CELEBORN-639.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Angerszhuuuu <angers.zhu@gmail.com>
2023-06-07 15:15:58 +08:00
Angerszhuuuu
4f1ca8c960
[CELEBORN-621][BUG] Push merged data task timeout and mapended should also remove push states (#1526) 2023-06-02 14:04:39 +08:00
Angerszhuuuu
cf308aa057
[CLEBORN-595] Refine code frame of CelebornConf (#1525) 2023-06-01 10:37:58 +08:00
Angerszhuuuu
62681ba85d
[CELEBORN-595] Rename and refactor the configuration doc. (#1501) 2023-05-30 15:14:12 +08:00
Angerszhuuuu
4f85d80687
[CELEBORN-606] Refine CommitHandler's noisy log (#1511) 2023-05-24 15:25:10 +08:00
Angerszhuuuu
811e192bbd
[CELEBORN-446] Support rack aware during assign slots for ROUNDROBIN (#1370) 2023-05-18 13:58:51 +08:00
Angerszhuuuu
a22c61e479
[CELEBORN-582] Celeborn should handle InterruptedException during kill task properly (#1486) 2023-05-17 18:17:41 +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
Angerszhuuuu
c0a9578d9f
[CELEBORN-563] Remove unnecessary code (#1469) 2023-05-06 11:25:31 +08:00
Angerszhuuuu
783d4e5dc5
[CELEBORN-551] Remove unnecessary ShuffleClient.get() (#1456) 2023-05-04 20:47:45 +08:00
Angerszhuuuu
a108d6f837
[CELEBORN-559][IMPROVEMENT] createReader should also wait for retry when change to same peer (#1465) 2023-05-04 10:51:15 +08:00