Commit Graph

92 Commits

Author SHA1 Message Date
Erik.fang
aee41555c6 [CELEBORN-955] Re-run Spark Stage for Celeborn Shuffle Fetch Failure
### What changes were proposed in this pull request?
Currently, Celeborn uses replication to handle shuffle data lost for celeborn shuffle reader, this PR implements an alternative solution by Spark stage resubmission.

Design doc:
https://docs.google.com/document/d/1dkG6fww3g99VAb1wkphNlUES_MpngVPNg8601chmVp8/edit

### Why are the changes needed?
Spark stage resubmission uses less resources compared with replication, and some Celeborn users are also asking for it

### Does this PR introduce _any_ user-facing change?
a new config celeborn.client.fetch.throwsFetchFailure is introduced to enable this feature

### How was this patch tested?
two UTs are attached, and we also tested it in Ant Group's Dev spark cluster

Closes #1924 from ErikFang/Re-run-Spark-Stage-for-Celeborn-Shuffle-Fetch-Failure.

Lead-authored-by: Erik.fang <fmerik@gmail.com>
Co-authored-by: Cheng Pan <pan3793@gmail.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-11-26 16:47:58 +08:00
Shuang
931880a82d [CELEBORN-1112] Inform celeborn application is shutdown, then celeborn cluster can release resource immediately
### What changes were proposed in this pull request?
Unregister application to Celeborn master After Application stopped, then master will expire the related shuffle resource immediately, resulting in resource savings.

### Why are the changes needed?
Currently Celeborn master expires the related application shuffle resource only when application is being checked timeout,
this would greatly delay the release of resources, which is not conducive to saving resources.

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

### How was this patch tested?
PASS GA

Closes #2075 from RexXiong/CELEBORN-1112.

Authored-by: Shuang <lvshuang.tb@gmail.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-11-08 20:46:51 +08:00
mingji
5e77b851c9 [CELEBORN-1081] Client support celeborn.storage.activeTypes config
### What changes were proposed in this pull request?
1.To support `celeborn.storage.activeTypes` in Client.
2.Master will ignore slots for "UNKNOWN_DISK".

### Why are the changes needed?
Enable client application to select storage types to use.

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

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

Closes #2045 from FMX/B1081.

Authored-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
Signed-off-by: Shuang <lvshuang.tb@gmail.com>
2023-11-03 20:03:11 +08:00
SteNicholas
3092644168 [CELEBORN-1095] Support configuration of fastest available XXHashFactory instance for checksum of Lz4Decompressor
### What changes were proposed in this pull request?

`CelebornConf` adds `celeborn.client.shuffle.decompression.lz4.xxhash.instance` to configure fastest available `XXHashFactory` instance for checksum of `Lz4Decompressor`. Fix #2043.

### Why are the changes needed?

`Lz4Decompressor` creates the checksum with `XXHashFactory#fastestInstance`, which returns the fastest available `XXHashFactory` instance that uses nativeInstance at default. The fastest available `XXHashFactory` instance for checksum of `Lz4Decompressor` could be supported to configure instead of dependency on the class loader is the system class loader, which method is as follows:
```
/**
 * Returns the fastest available {link XXHashFactory} instance. If the class
 * loader is the system class loader and if the
 * {link #nativeInstance() native instance} loads successfully, then the
 * {link #nativeInstance() native instance} is returned, otherwise the
 * {link #fastestJavaInstance() fastest Java instance} is returned.
 * <p>
 * Please read {link #nativeInstance() javadocs of nativeInstance()} before
 * using this method.
 *
 * return the fastest available {link XXHashFactory} instance.
 */
public static XXHashFactory fastestInstance() {
  if (Native.isLoaded()
      || Native.class.getClassLoader() == ClassLoader.getSystemClassLoader()) {
    try {
      return nativeInstance();
    } catch (Throwable t) {
      return fastestJavaInstance();
    }
  } else {
    return fastestJavaInstance();
  }
}
```

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

No.

### How was this patch tested?

- `CelebornConfSuite`
- `ConfigurationSuite`

Closes #2050 from SteNicholas/CELEBORN-1095.

Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: xiyu.zk <xiyu.zk@alibaba-inc.com>
2023-10-31 14:57:31 +08:00
mingji
e0c00ecd38 [CELEBORN-839][MR] Support Hadoop MapReduce
### What changes were proposed in this pull request?
1. Map side merge and push.
2. Support hadoop2 & 3.
3. Reduce in-memory merge.
4. Integrate LifecycleManager to RmApplicationMaster.

### Why are the changes needed?
Ditto.

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

### How was this patch tested?
Cluster.

I tested this PR on a cluster with a 4x 16 CPU 64G Mem 4ESSD cluster.
Hadoop 2.8.5

1TB Terasort, 8400 mappers, 1000 reducers
Celeborn 81min vs MR shuffle 89min
![mr1](https://github.com/apache/incubator-celeborn/assets/4150993/a3cf6493-b6ff-4c03-9936-4558cf22761d)
![mr2](https://github.com/apache/incubator-celeborn/assets/4150993/9119ffb4-6996-4b77-bcdf-cbd6db5c096f)

1GB wordcount, 8 mappers, 8 reducers
Celeborn 35s VS MR shuffle 38s
![mr3](https://github.com/apache/incubator-celeborn/assets/4150993/907dce24-16b7-4788-ab5d-5b784fd07d47)
![mr4](https://github.com/apache/incubator-celeborn/assets/4150993/8e8065b9-6c46-4c8d-9e71-45eed8e63877)

Closes #1830 from FMX/CELEBORN-839.

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-09-14 14:12:53 +08:00
zky.zhoukeyong
a42ec85a6e [CELEBORN-943][PERF] Pre-create CelebornInputStreams in CelebornShuffleReader
### What changes were proposed in this pull request?
This PR fixes performance degradation when Spark's coalescePartitions takes effect caused
by RPC latency.

### Why are the changes needed?
I encountered a performance degradation when testing  tpcds 10T q10:
||Time|
|---|---|
|ESS|14s|
|Celeborn| 24s|

After digging into it I found out that q10 triggers partition coalescence:
![image](https://github.com/apache/incubator-celeborn/assets/948245/0b4745da-8d57-4661-a35d-683d97f56e1d)

As I configured `spark.sql.adaptive.coalescePartitions.initialPartitionNum` to 1000, `CelebornShuffleReader`
will call `shuffleClient.readPartition` sequentially 1000 times, causing the delay.

This PR optimizes by calling `shuffleClient.readPartition` in parallel. After this PR q10 time becomes 14s.

### Does this PR introduce _any_ user-facing change?
No, but introduced a new client side configuration `celeborn.client.streamCreatorPool.threads`
which defaults to 32.

### How was this patch tested?
TPCDS 1T and passes GA.

Closes #1876 from waitinfuture/943.

Lead-authored-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
Co-authored-by: Keyong Zhou <waitinfuture@gmail.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-09-04 21:46:11 +08:00
zhongqiang.czq
b66eaff880 [CELEBORN-627][FLINK] Support split partitions
### What changes were proposed in this pull request?
In MapPartiitoin, datas are split into regions.

1. Unlike ReducePartition whose partition split can occur on pushing data
to keep MapPartition data ordering,  PartitionSplit only be done on the time of sending PushDataHandShake or RegionStart messages (As shown in the following image). That's to say that the partition split only appear at the beginnig of a region but not inner a region.
> Notice: if the client side think that it's failed to push HandShake or RegionStart messages. but the worker side can still receive normal HandShake/RegionStart message. After client revive succss, it don't push any messages to old partition, so the worker having the old partition will create a empty file. After committing files, the worker will return empty commitids. That's to say that empty file will be filterd after committing files and ReduceTask will not read any empty files.

![image](https://github.com/apache/incubator-celeborn/assets/96606293/468fd660-afbc-42c1-b111-6643f5c1e944)

2. PushData/RegioinFinish don't care the following cases:
 - Diskfull
 - ExceedPartitionSplitThreshold
 - Worker ShuttingDown
so if one of the above three conditions appears, PushData and RegionFinish cant still do as normal. Workers should consider the ShuttingDown case and  try best to wait all the regions finished before shutting down.

if PushData or RegionFinish failed like network timeout and so on, then MapTask will failed and start another attempte maptask.

![image](https://github.com/apache/incubator-celeborn/assets/96606293/db9f9166-2085-4be1-b09e-cf73b469c55b)

3. how shuffle read supports partition split?
ReduceTask should get split paritions by order and open the stream by partition epoc orderly

### Why are the changes needed?
PartiitonSplit is not supported by MapPartition from now.
There still a risk that  a partition file'size is too large to store the file on worker disk.
To avoid this risk, this pr introduces partition split in shuffle read and shuffle write.

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

### How was this patch tested?
UT and manual TPCDS test

Closes #1550 from FMX/CELEBORN-627.

Lead-authored-by: zhongqiang.czq <zhongqiang.czq@alibaba-inc.com>
Co-authored-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
Co-authored-by: Ethan Feng <ethanfeng@apache.org>
Signed-off-by: zhongqiang.czq <zhongqiang.czq@alibaba-inc.com>
2023-09-01 19:25:51 +08:00
mingji
505ba804c7 [CELEBORN-752] Support read local shuffle file for spark
### What changes were proposed in this pull request?
For spark clusters, support read local shuffle file if Celeborn is co-deployed with yarn node managers. This PR help to reduce the number of active connections.

### Why are the changes needed?
Ditto.

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

### How was this patch tested?
GA and cluster.  The performance is identical whether you enable local reader, but the active connection number may vary according to your connections per peer.
<img width="951" alt="截屏2023-08-16 20 20 14" src="https://github.com/apache/incubator-celeborn/assets/4150993/9106e731-28fc-4e78-9c05-ae6a269d249a">
The active connection number changed from 3745 to 2894. This PR will help to improve cluster stability.

Closes #1812 from FMX/CELEBORN-752.

Authored-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-08-30 18:52:18 +08:00
lishiyucn
57a35ca349 [CELEBORN-498] Add new config for DfsPartitionReader's chunk size
### What changes were proposed in this pull request?
As title

### Why are the changes needed?
Make `celeborn.shuffle.chunk.size` worker side only config.
Add a new client side config `celeborn.client.fetch.dfsReadChunkSize` for DfsPartitionReader

### Does this PR introduce _any_ user-facing change?
Yes, the chunks size of DfsPartitionReader is changed from client side config `celeborn.shuffle.chunk.size`
to `celeborn.client.fetch.dfsReadChunkSize`

### How was this patch tested?
Passes GA

Closes #1834 from lishiyucn/main.

Lead-authored-by: lishiyucn <675590586@qq.com>
Co-authored-by: shiyu li <675590586@qq.com>
Co-authored-by: Keyong Zhou <zky.zhoukeyong@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-08-24 21:31:34 +08:00
e
4a4a37ed17 [MINOR] Fix typo in CelebornConf
### What changes were proposed in this pull request?

Fix typo in CelebornConf

### Why are the changes needed?

Fix typo in CelebornConf

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

NO

### How was this patch tested?

Passing GA

Closes #1813 from jiaoqingbo/typo-conf.

Authored-by: e <1178404354@qq.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-08-15 10:32:08 +08:00
Fu Chen
516bdc7e08
[CELEBORN-877][DOC] Document on SBT
### What changes were proposed in this pull request?

As title

### Why are the changes needed?

As title

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

No

### How was this patch tested?

Manual test

Closes #1795 from cfmcgrady/sbt-docs.

Authored-by: Fu Chen <cfmcgrady@gmail.com>
Signed-off-by: Cheng Pan <chengpan@apache.org>
2023-08-11 12:17:55 +08:00
zky.zhoukeyong
6ea1ee2ec4 [CELEBORN-152] Add config to limit max workers when offering slots
### What changes were proposed in this pull request?
Add config to limit max workers when offering slots, the config can be set both
in server side and client side. Celeborn will choose the smaller positive configs from client and master.

### Why are the changes needed?
For large Celeborn clusters, users may want to limit the number of workers that
a shuffle can spread, reasons are:

1. One worker failure will not affect all applications
2. One huge shuffle will not affect all applications
3. It's more efficient to limit a shuffle within a restricted number of workers, say 100, than
    spreading across a large number of workers, say 1000, because the network connections
   in pushing data is `number of ShuffleClient` * `number of allocated Workers`

The recommended number of Workers should depend on workload and Worker hardware,
and this can be configured per application, so it's relatively flexible.

### Does this PR introduce _any_ user-facing change?
No, added a new configuration.

### How was this patch tested?
Added ITs and passes GA.

Closes #1790 from waitinfuture/152.

Authored-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-08-07 10:13:53 +08:00
zky.zhoukeyong
6a5e3ed794 [CELEBORN-812] Cleanup SendBufferPool if idle for long
### What changes were proposed in this pull request?
Cleans up the pooled send buffers and push tasks if the SendBufferPool has been idle for more than
`celeborn.client.push.sendbufferpool.expireTimeout`.

### Why are the changes needed?
Before this PR the SendBufferPool will cache the send buffers and push tasks forever. If they are large
and will not be reused in the future, it wastes memory and causes GC.

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

### How was this patch tested?
Passes GA and manual tests.

Closes #1735 from waitinfuture/812-1.

Authored-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-07-20 00:34:55 +08:00
onebox-li
405b2801fa [CELEBORN-810] Fix some typos and grammar
### What changes were proposed in this pull request?
Fix some typos and grammar

### Why are the changes needed?
Ditto

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

### How was this patch tested?
manually test

Closes #1733 from onebox-li/fix-typo.

Authored-by: onebox-li <lyh-36@163.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-07-19 18:35:38 +08:00
Cheng Pan
0db919403e Revert "[CELEBORN-798] Add heartbeat from client to LifecycleManager to clean…"
This reverts commit e56a8a8bed.
2023-07-19 15:08:45 +08:00
zky.zhoukeyong
1109e2c8f4 [CELEBORN-803][FOLLOWUP] Make ``rpcAskTimeout`` default to 60s
### What changes were proposed in this pull request?
As title.

### Why are the changes needed?
Timeout of ```RpcEndpointRef.ask``` is controlled by ```celeborn.rpc.askTimeout```,
so we also need to increase ```celeborn.rpc.askTimeout``` to extend the timeout of commit files.

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

### How was this patch tested?
Passes GA and manual test.

Closes #1725 from waitinfuture/803-fu.

Authored-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-07-17 23:53:52 +08:00
zky.zhoukeyong
9ec223edd7 [CELEBORN-803] Increase default timeout for commit files
### What changes were proposed in this pull request?
As title.

### Why are the changes needed?
In 0.2.1-incubating, commit files default timeout is ```NETWORK_TIMEOUT```, which is 240s.
It's more reasonable because commit files costs relatively long time. In my testing with tough disks,
30s timeout with 2 retires is not enough.

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

### How was this patch tested?
Passes GA and manual test.

Closes #1724 from waitinfuture/803.

Authored-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-07-17 22:31:36 +08:00
zky.zhoukeyong
e56a8a8bed [CELEBORN-798] Add heartbeat from client to LifecycleManager to clean…
…up client

### What changes were proposed in this pull request?
Add heartbeat from client to lifecycle manager. In this PR heartbeat request contains local shuffle ids from
client, lifecycle manager checks with it's local set and returns ids it doesn't know. Upon receiving response,
client calls ```unregisterShuffle``` for cleanup.

### Why are the changes needed?
Before this PR, client side ```unregisterShuffle``` is never called. When running TPCDS 3T with spark thriftserver
without DRA, I found the Executor's heap contains 1.6 million PartitionLocation objects (and StorageInfo):
![image](https://github.com/apache/incubator-celeborn/assets/948245/43658369-7763-4511-a5b0-9b3fbdf02005)

After this PR, the number of PartitionLocation objects decreases to 275 thousands
![image](https://github.com/apache/incubator-celeborn/assets/948245/45f8f849-186d-4cad-83c8-64bd6d18debc)

This heartbeat can be extended in the future for other purposes, i.e. reporting client's metrics.

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

### How was this patch tested?
Passes GA and  manual test.

Closes #1719 from waitinfuture/798.

Authored-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-07-17 18:14:10 +08:00
zky.zhoukeyong
95119b1e4b [CELEBORN-799][FOLLOWUP] Fix doc of celeborn.client.push.maxReqsInFlight.total
…Flight.total```

### What changes were proposed in this pull request?
Refer to https://github.com/apache/incubator-celeborn/pull/1720#discussion_r1265092164

### Why are the changes needed?
ditto

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

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

Closes #1723 from waitinfuture/799-fu.

Authored-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-07-17 18:01:03 +08:00
zky.zhoukeyong
4b3a47c9db [CELEBORN-799] Limit total inflight push requests
### What changes were proposed in this pull request?
As title.

### Why are the changes needed?
In case where worker instances is very large, say 1000, then before this PR total memory consumed
by inflight requests is 64K * 1000 * ```celeborn.client.push.maxReqsInFlight(16)``` = 1G. This PR
limits total inflight push requests, as 0.2.1-incubating does.

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

### How was this patch tested?
Passes GA and manual test.

Closes #1720 from waitinfuture/799.

Lead-authored-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
Co-authored-by: Cheng Pan <pan3793@gmail.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-07-17 16:17:24 +08:00
Angerszhuuuu
9f09ac6ce9 [CELEBORN-780] Change SPARK_SHUFFLE_FORCE_FALLBACK_PARTITION_THRESHOLD default to Int.MaxValue since slot's is not a bottleneck
### What changes were proposed in this pull request?
Now slots is not a bottleneck, change SPARK_SHUFFLE_FORCE_FALLBACK_PARTITION_THRESHOLD default value to Int.MaxValue.

### Why are the changes needed?

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

### How was this patch tested?

Closes #1695 from AngersZhuuuu/CELEBORN-780.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Angerszhuuuu <angers.zhu@gmail.com>
2023-07-10 18:50:10 +08:00
zky.zhoukeyong
09881f5cff [CELEBORN-769] Change default value of celeborn.client.push.maxReqsInFlight to 16
…Flight to 16

### What changes were proposed in this pull request?
Change default value of celeborn.client.push.maxReqsInFlight to 16.

### Why are the changes needed?
Previous value 4 is too small, 16 is more reasonable.

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

### How was this patch tested?
Pass GA.

Closes #1683 from waitinfuture/769.

Authored-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-07-06 10:22:06 +08:00
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
zky.zhoukeyong
4300835363 [CELEBORN-768] Change default config values for batch rpcs and netty …
…memory allocator

### What changes were proposed in this pull request?
Changes the following configs' default values
| config  | previous value | current value |
| ------------- | ------------- | ------------- |
| celeborn.network.memory.allocator.share  | false | true |
| celeborn.client.shuffle.batchHandleChangePartition.enabled  | false | true |
| celeborn.client.shuffle.batchHandleCommitPartition.enabled | false | true |

### Why are the changes needed?
In my test, when graceful shutdown is enabled but ```celeborn.client.shuffle.batchHandleChangePartition.enabled``` and ```celeborn.client.shuffle.batchHandleCommitPartition.enabled``` disabled, the worker takes much longer to stop than the two configs enabled.
In another test where worker size is quite small(2 cores 4 G) and replication is on, if shared allocator is disabled, the netty's onTrim fails to release memory, and further causes push data timeout.

### Does this PR introduce _any_ user-facing change?
No, these conifgs are introduces from 0.3.0.

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

Closes #1682 from waitinfuture/768.

Authored-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-07-05 18:16:41 +08:00
Fu Chen
3af5c231c7 [CELEBORN-767][DOC] Update the docs of celeborn.client.spark.push.sort.memory.threshold
### What changes were proposed in this pull request?

As title

### Why are the changes needed?

To clarify the usage of conf `celeborn.client.spark.push.sort.memory.threshold`

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

No

### How was this patch tested?

Pass GA

Closes #1680 from cfmcgrady/docs.

Authored-by: Fu Chen <cfmcgrady@gmail.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-07-05 18:07:09 +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
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
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
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
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
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
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
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
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
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
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
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
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
7c6cb2f3bb
[CELEBORN-588] Remove test conf's category (#1491) 2023-05-17 17:37:28 +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