Commit Graph

75 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
sychen
3054813a0f
[CELEBORN-856] Add mapreduce integration test
### 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 #2073 from cxzl25/CELEBORN-856.

Authored-by: sychen <sychen@ctrip.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
2023-11-22 14:36:29 +08:00
onebox-li
cd8acf89c9 [CELEBORN-1059] Fix callback not update if push worker excluded during retry
### What changes were proposed in this pull request?
When retry push data and revive succeed in ShuffleClientImpl#submitRetryPushData, if new location is excluded, the callback's `lastest` location has not been updated when wrappedCallback.onFailure is called in ShuffleClientImpl#isPushTargetWorkerExcluded. Therefore there may be problems with subsequent revive.

### Why are the changes needed?
Ditto

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

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

Closes #2005 from onebox-li/improve-push-exclude.

Authored-by: onebox-li <lyh-36@163.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-11-01 10:23:50 +08:00
Jiaan Geng
640007100a [CELEBORN-1019][FOLLOWUP] Remove duplicate line
### What changes were proposed in this pull request?
https://github.com/apache/incubator-celeborn/pull/1968 introduced duplicate lines of code, this is my fault.

### Why are the changes needed?
Remove duplicate line.

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

### How was this patch tested?
N/A

Closes #1990 from beliefer/CELEBORN-1019_followup.

Authored-by: Jiaan Geng <beliefer@163.com>
Signed-off-by: Fu Chen <cfmcgrady@gmail.com>
2023-10-16 13:48:36 +08:00
sychen
dd65e74f99 [CELEBORN-983] Rename PrometheusMetric configuration
### What changes were proposed in this pull request?
Replace
```properties
celeborn.metrics.master.prometheus.host
celeborn.metrics.master.prometheus.port
celeborn.metrics.worker.prometheus.host
celeborn.metrics.worker.prometheus.port
```
With
```properties
celeborn.master.http.host
celeborn.master.http.port
celeborn.worker.http.host
celeborn.worker.http.port
```

### Why are the changes needed?
The `celeborn.master.metrics.prometheus.port` and `celeborn.metrics.worker.prometheus.port` bind port not only serve prometheus metrics, but also provide some useful API services.

https://celeborn.apache.org/docs/latest/monitoring/#rest-api

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

### How was this patch tested?

Closes #1919 from cxzl25/CELEBORN-983.

Lead-authored-by: sychen <sychen@ctrip.com>
Co-authored-by: Keyong Zhou <zhouky@apache.org>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-10-13 13:28:58 +08:00
Jiaan Geng
a776e7e315 [CELEBORN-1019] Avoid magic strings copy from Spark SQLConf
### What changes were proposed in this pull request?
Avoid magic strings copy from Spark `SQLConf`.

### Why are the changes needed?
Currently, the spark integration uses many magic strings copy from Spark `SQLConf`.
Since we have already depend on Spark, references the variable from Spark is better.

### Does this PR introduce _any_ user-facing change?
No.
Just update the inner implementation.

### How was this patch tested?
Exists test cases.

Closes #1968 from beliefer/CELEBORN-1019.

Lead-authored-by: Jiaan Geng <beliefer@163.com>
Co-authored-by: Keyong Zhou <zhouky@apache.org>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-10-10 19:52:24 +08:00
Fu Chen
b2412d0774 [CELEBORN-1022][TEST] Update log level from FATAL to ERROR for console output in unit tests
### What changes were proposed in this pull request?

As title

### Why are the changes needed?

1. this is developer-friendly for debugging unit tests in IntelliJ IDEA, for example: Netty's memory leak reports are logged at the error level and won't cause unit tests to be marked as fatal.

```
23/10/09 09:57:26,422 ERROR [fetch-server-52-2] ResourceLeakDetector: LEAK: ByteBuf.release() was not called before it's garbage-collected. See https://netty.io/wiki/reference-counted-objects.html for more information.
Recent access records:
Created at:
	io.netty.buffer.PooledByteBufAllocator.newDirectBuffer(PooledByteBufAllocator.java:403)
	io.netty.buffer.AbstractByteBufAllocator.directBuffer(AbstractByteBufAllocator.java:188)
	io.netty.buffer.AbstractByteBufAllocator.directBuffer(AbstractByteBufAllocator.java:179)
	io.netty.buffer.AbstractByteBufAllocator.ioBuffer(AbstractByteBufAllocator.java:140)
	io.netty.channel.DefaultMaxMessagesRecvByteBufAllocator$MaxMessageHandle.allocate(DefaultMaxMessagesRecvByteBufAllocator.java:120)
	io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:150)
	io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:788)
	io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:724)
	io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:650)
	io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:562)
	io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:997)
	io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
	io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
	java.lang.Thread.run(Thread.java:750)
```

2. this won't increase console output and affect the stability of CI.

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

No

### How was this patch tested?

Pass GA

Closes #1958 from cfmcgrady/ut-console-log-level.

Authored-by: Fu Chen <cfmcgrady@gmail.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-10-09 15:56:05 +08:00
Fu Chen
c4135dc1b1 [CELEBORN-980] Asynchronously delete original files to fix ReusedExchange bug
### What changes were proposed in this pull request?

The `ReusedExchange` operator has the potential to generate different types of fetch requests, including both non-range and range requests. Currently, an issue arises due to the synchronous deletion of the original file by the Celeborn worker upon completion of sorting. This issue leads to the failure of non-range requests following a range request for the same partition.

the snippets to reproduce this bug
```scala
  val sparkConf = new SparkConf().setAppName("celeborn-test").setMaster("local[2]")
    .set("spark.shuffle.manager", "org.apache.spark.shuffle.celeborn.SparkShuffleManager")
    .set(s"spark.${CelebornConf.MASTER_ENDPOINTS.key}", masterInfo._1.rpcEnv.address.toString)
    .set("spark.sql.autoBroadcastJoinThreshold", "-1")
    .set("spark.sql.adaptive.skewJoin.skewedPartitionThresholdInBytes", "100")
    .set("spark.sql.adaptive.advisoryPartitionSizeInBytes", "100")
  val spark = SparkSession.builder()
    .config(sparkConf)
    .getOrCreate()
  spark.range(0, 1000, 1, 10)
    .selectExpr("id as k1", "id as v1")
    .createOrReplaceTempView("ta")
  spark.range(0, 1000, 1, 10)
    .selectExpr("id % 1 as k21", "id % 1 as k22", "id as v2")
    .createOrReplaceTempView("tb")
  spark.range(140)
    .select(
      col("id").cast("long").as("k3"),
      concat(col("id").cast("string"), lit("a")).as("v3"))
    .createOrReplaceTempView("tc")

  spark.sql(
    """
      |SELECT *
      |FROM ta
      |LEFT JOIN tb ON ta.k1 = tb.k21
      |LEFT JOIN tc ON tb.k22 = tc.k3
      |""".stripMargin)
    .createOrReplaceTempView("v1")

  spark.sql(
    """
      |SELECT * FROM v1 WHERE v3 IS NOT NULL
      |UNION
      |SELECT * FROM v1
      |""".stripMargin)
    .collect()
```

This PR proposes a solution to address this problem. It introduces an asynchronous thread for the removal of the original file. Once the sorted file is generated for a given partition, this modification ensures that both non-range and range fetch requests will be able to and only fetch the sorted file once it is generated for a given partition.

this activity diagram of `openStream`

![openStream](https://github.com/apache/incubator-celeborn/assets/8537877/633cc5b8-e673-45a0-860e-e1f7e50c8965)

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

No, only bug fix

### How was this patch tested?

UT

Closes #1932 from cfmcgrady/fix-partition-sort-bug-v4.

Authored-by: Fu Chen <cfmcgrady@gmail.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-10-09 11:04:41 +08:00
zhongqiang.czq
b1e3d661e6 [CELEBORN-627][FLINK][FOLLOWUP] Support split partitions
### What changes were proposed in this pull request?
fix duplicated sending commitFiles for MapPartition and fix not sending BufferStreamEnd while opening MapPartition split.

### Why are the changes needed?
After open partition split  for MapPartition, there are 2 errors.
- ERROR1 : Worker don't send streamend to client because concurrent thread sync problem . After idle timeout, client will close the channel and throws the Exception **" xx is lost, notify related stream xx"**
```java
2023-09-06T04:40:47.7549935Z 23/09/06 04:40:47,753 WARN [Keyed Aggregation -> Map -> Sink: Unnamed (5/8)#0] Task: Keyed Aggregation -> Map -> Sink: Unnamed (5/8)#0 (c1cade728ddb3a32e0bf72acb1d87588_c27dcf7b54ef6bfd6cff02ca8870b681_4_0) switched from RUNNING to FAILED with failure cause:
2023-09-06T04:40:47.7550644Z java.io.IOException: Client localhost/127.0.0.1:38485 is lost, notify related stream 256654410004
2023-09-06T04:40:47.7551219Z 	at org.apache.celeborn.plugin.flink.RemoteBufferStreamReader.errorReceived(RemoteBufferStreamReader.java:142)
2023-09-06T04:40:47.7551886Z 	at org.apache.celeborn.plugin.flink.RemoteBufferStreamReader.lambda$new$0(RemoteBufferStreamReader.java:77)
2023-09-06T04:40:47.7552576Z 	at org.apache.celeborn.plugin.flink.network.ReadClientHandler.processMessageInternal(ReadClientHandler.java:57)
2023-09-06T04:40:47.7553250Z 	at org.apache.celeborn.plugin.flink.network.ReadClientHandler.lambda$channelInactive$0(ReadClientHandler.java:119)
2023-09-06T04:40:47.7553806Z 	at java.util.concurrent.ConcurrentHashMap.forEach(ConcurrentHashMap.java:1597)
2023-09-06T04:40:47.7554564Z 	at org.apache.celeborn.plugin.flink.network.ReadClientHandler.channelInactive(ReadClientHandler.java:110)
2023-09-06T04:40:47.7555270Z 	at org.apache.celeborn.common.network.server.TransportRequestHandler.channelInactive(TransportRequestHandler.java:71)
2023-09-06T04:40:47.7556005Z 	at org.apache.celeborn.common.network.server.TransportChannelHandler.channelInactive(TransportChannelHandler.java:136)
2023-09-06T04:40:47.7556710Z 	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:305)
2023-09-06T04:40:47.7557370Z 	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:281)
2023-09-06T04:40:47.7558172Z 	at io.netty.channel.AbstractChannelHandlerContext.fireChannelInactive(AbstractChannelHandlerContext.java:274)
2023-09-06T04:40:47.7558803Z 	at io.netty.channel.ChannelInboundHandlerAdapter.channelInactive(ChannelInboundHandlerAdapter.java:81)
2023-09-06T04:40:47.7559368Z 	at io.netty.handler.timeout.IdleStateHandler.channelInactive(IdleStateHandler.java:277)
2023-09-06T04:40:47.7559954Z 	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:303)
2023-09-06T04:40:47.7560589Z 	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:281)
2023-09-06T04:40:47.7561222Z 	at io.netty.channel.AbstractChannelHandlerContext.fireChannelInactive(AbstractChannelHandlerContext.java:274)
2023-09-06T04:40:47.7561829Z 	at io.netty.channel.ChannelInboundHandlerAdapter.channelInactive(ChannelInboundHandlerAdapter.java:81)
2023-09-06T04:40:47.7562620Z 	at org.apache.celeborn.plugin.flink.network.TransportFrameDecoderWithBufferSupplier.channelInactive(TransportFrameDecoderWithBufferSupplier.java:206)
2023-09-06T04:40:47.7563506Z 	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:305)
2023-09-06T04:40:47.7564207Z 	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:281)
2023-09-06T04:40:47.7564829Z 	at io.netty.channel.AbstractChannelHandlerContext.fireChannelInactive(AbstractChannelHandlerContext.java:274)
2023-09-06T04:40:47.7565417Z 	at io.netty.channel.DefaultChannelPipeline$HeadContext.channelInactive(DefaultChannelPipeline.java:1405)
2023-09-06T04:40:47.7566014Z 	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:301)
2023-09-06T04:40:47.7566654Z 	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:281)
2023-09-06T04:40:47.7567317Z 	at io.netty.channel.DefaultChannelPipeline.fireChannelInactive(DefaultChannelPipeline.java:901)
2023-09-06T04:40:47.7567813Z 	at io.netty.channel.AbstractChannel$AbstractUnsafe$7.run(AbstractChannel.java:813)
2023-09-06T04:40:47.7568297Z 	at io.netty.util.concurrent.AbstractEventExecutor.runTask(AbstractEventExecutor.java:174)
2023-09-06T04:40:47.7568830Z 	at io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:167)
2023-09-06T04:40:47.7569402Z 	at io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:470)
2023-09-06T04:40:47.7569894Z 	at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:569)
2023-09-06T04:40:47.7570356Z 	at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:997)
2023-09-06T04:40:47.7570841Z 	at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
2023-09-06T04:40:47.7571319Z 	at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
2023-09-06T04:40:47.7571721Z 	at java.lang.Thread.run(Thread.java:750)
```
- ERROR2: Client will send duplicated commitFiles to worker. Becuase of inconsistency unHandledPartiitions , both batchCommit and finalCommit send commitFiles
``` java
2023-09-06T04:36:48.3146773Z 23/09/06 04:36:48,314 WARN [Worker-CommitFiles-1] Controller: Get Partition Location for 1693975002919-61094c8156f918062a5fae12d551bc90-0 0-1 but didn't exist.
```

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

### How was this patch tested?
ci

Closes #1881 from zhongqiangczq/fix-split-test.

Authored-by: zhongqiang.czq <zhongqiang.czq@alibaba-inc.com>
Signed-off-by: zhongqiang.czq <zhongqiang.czq@alibaba-inc.com>
2023-09-06 22:33:56 +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
SteNicholas
4918b8e216 [CELEBORN-830] Add spark integration test to verify fallback with workers unavailable
### What changes were proposed in this pull request?

Adds spark integration test to verify fallback with workers unavailable for #1814.

### Why are the changes needed?

None.

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

No.

### How was this patch tested?

- `ShuffleFallbackSuite` adds `eleborn spark integration test - fallback with workers unavailable` to verify whether to fallback with workers unavailable.

Closes #1856 from SteNicholas/CELEBORN-830.

Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-08-29 21:35:20 +08:00
zhouyifan279
d701d3ae2c [CELEBORN-912] Support build with Spark 3.5
### What changes were proposed in this pull request?

Support build with Spark 3.5

### Why are the changes needed?

Keep up with upstream.

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

No

### How was this patch tested?

Build with `mvn` and `sbt` locally.

Closes #1850 from zhouyifan279/build-spark-3.5.

Authored-by: zhouyifan279 <zhouyifan279@gmail.com>
Signed-off-by: Cheng Pan <chengpan@apache.org>
2023-08-29 03:15:12 +00: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
mingji
ea39a9372a [CELEBORN-760] Convert OpenStream and StreamHandler to Pb
### What changes were proposed in this pull request?
Merge OpenStream and StreamHandler to transport messages to enhance celeborn's compatibility.

### Why are the changes needed?
1. Improve flexibility to change RPC.
2. Compatible with 0.2 client.

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

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

Closes #1750 from FMX/CELEBORN-760.

Lead-authored-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
Co-authored-by: Ethan Feng <fengmingxiao.fmx@alibaba-inc.com>
Co-authored-by: Keyong Zhou <zhouky@apache.org>
Co-authored-by: Keyong Zhou <waitinfuture@gmail.com>
Co-authored-by: Keyong Zhou <zky.zhoukeyong@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-08-05 13:58:08 +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
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
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
Angerszhuuuu
5c7ecb8302
[CELEBORN-754][IMPORTANT] Provide a new SparkShuffleManager to replace RssShuffleManager in the future
### What changes were proposed in this pull request?
Provide a new SparkShuffleManager to replace RssShuffleManager in the future

### Why are the changes needed?

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

### How was this patch tested?

Closes #1667 from AngersZhuuuu/CELEBORN-754.

Lead-authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Co-authored-by: Cheng Pan <pan3793@gmail.com>
Signed-off-by: Cheng Pan <chengpan@apache.org>
2023-06-30 17:27:33 +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
Cheng Pan
b821349c4a
[CELEBORN-727][TEST] Fix flaky test RssHashCheckDiskSuite
### What changes were proposed in this pull request?

Fix the flaky test by enlarging `celeborn.client.shuffle.expired.checkInterval`

### Why are the changes needed?

```
RssHashCheckDiskSuite:
- celeborn spark integration test - hash-checkDiskFull *** FAILED ***
  868 was not less than 0 (RssHashCheckDiskSuite.scala:83)
```

https://github.com/apache/incubator-celeborn/actions/runs/5396767745/jobs/9800766633

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

No.

### How was this patch tested?

Pass GA, and should observe CI,

Closes #1640 from pan3793/CELEBORN-727.

Authored-by: Cheng Pan <chengpan@apache.org>
Signed-off-by: Cheng Pan <chengpan@apache.org>
2023-06-28 17:59:54 +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
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
Cheng Pan
1753556565
[CELEBORN-713] Local network binding support IP or FQDN
### What changes were proposed in this pull request?

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

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

### Why are the changes needed?

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

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

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

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

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

### How was this patch tested?

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

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

Server:		10.178.96.64
Address:	10.178.96.64#53

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

Server:		10.178.96.64
Address:	10.178.96.64#53

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

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

Closes #1622 from pan3793/CELEBORN-713.

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

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

### Why are the changes needed?

Respect the project name.

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

Yes, migration guide is updated accordingly.

### How was this patch tested?

Pass GA.

Closes #1624 from pan3793/CELEBORN-715.

Authored-by: Cheng Pan <chengpan@apache.org>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-06-26 22:30:20 +08:00
Fu Chen
1b3ec61690 [CELEBORN-711][TEST] Rework PushDataTimeoutTest
### What changes were proposed in this pull request?

1. separated push data timeout tests and push merge data timeout tests in `PushDataTimeoutTest`
2. updated the test results assertion
3. rework `pushdata timeout will add to blacklist`

### Why are the changes needed?

ensure that the timeout behavior is correctly implemented

https://github.com/apache/incubator-celeborn/pull/1613#discussion_r1236423721

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

No

### How was this patch tested?

existing tests

Closes #1620 from cfmcgrady/push-timeout-test.

Authored-by: Fu Chen <cfmcgrady@gmail.com>
Signed-off-by: Cheng Pan <chengpan@apache.org>
2023-06-26 13:45:27 +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
sychen
e734ceb558 [MINOR] Cleanup code
### What changes were proposed in this pull request?
1. Use `<arg>-Ywarn-unused-import</arg>` to remove some unused imports
There is no way to use `<arg>-Ywarn-unused-import</arg>` at this stage
Because we have the following code
```
// Can Remove this if celeborn don't support scala211 in future
import org.apache.celeborn.common.util.FunctionConverter._
```
2. Fix scala case match not fully covered, avoid `scala.MatchError`
3. Fixed some scala compilation warnings

### Why are the changes needed?

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

### How was this patch tested?

Closes #1600 from cxzl25/cleanup_code.

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

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

### Why are the changes needed?

Make the configuration unified.

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

Yes, but the legacy configurations still take effect.

### How was this patch tested?

New UTs.

Closes #1549 from pan3793/CELEBORN-638.

Authored-by: Cheng Pan <chengpan@apache.org>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-06-16 18:18:26 +08:00
zwangsheng
859d021eeb [CELEBORN-644][HELM] Support Helm Deploy Celeborn with HostNetwork And DnsPolicy
### What changes were proposed in this pull request?
Support service type `NodePort` in `values.yaml`

### Why are the changes needed?
For Kubernetes IT, should start celeborn cluster in minikube with nodePort for discover

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

### How was this patch tested?
CI Test & Local Test

Closes #1554 from zwangsheng/CELEBORN-644.

Authored-by: zwangsheng <2213335496@qq.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-06-16 17:59:39 +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
Cheng Pan
588dbdfbe0
[CELEBORN-653][TEST] Fix invalid configuration key in SparkTestBase
### What changes were proposed in this pull request?

Dot is missing after `spark`

### Why are the changes needed?

Correct the configuration key.

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

No.

### How was this patch tested?

Pass GA.

Closes #1563 from pan3793/CELEBORN-653.

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

This PR upgrades

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

### Why are the changes needed?

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

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

No, it only affects test.

### How was this patch tested?

Pass GA.

Closes #1562 from pan3793/CELEBORN-650.

Authored-by: Cheng Pan <chengpan@apache.org>
Signed-off-by: Cheng Pan <chengpan@apache.org>
2023-06-09 10:04:14 +08:00
zhongqiang.czq
586785c88d [CELEBORN-617][FLINK] MapPartitionFileWriter updates flushing file length
…ngth

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

### Why are the changes needed?

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

### How was this patch tested?

Closes #1519 from zhongqiangczq/mapfilelength.

Authored-by: zhongqiang.czq <zhongqiang.czq@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
2023-06-08 10:47:36 +08:00
zwangsheng
5068d6e897
[CELEBORN-105][TEST] Kubernetes Integration Test
### What changes were proposed in this pull request?
Add Kubernetes Integration Test
- [x] test helm install deploy
- [ ] test shuffle

### Why are the changes needed?
Add integration test

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

### How was this patch tested?
Ci test

Closes #1484 from zwangsheng/CELEBORN-105.

Authored-by: zwangsheng <2213335496@qq.com>
Signed-off-by: Cheng Pan <chengpan@apache.org>
2023-06-05 12:11:29 +08:00
Angerszhuuuu
4df4775524
[CELEBORN-632][DOC] Add spark name space to spark specify properties (#1538) 2023-06-02 21:48:56 +08:00
Angerszhuuuu
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
Cheng Pan
ef8e556202
[CELEBORN-604][SPARK] Support Spark 3.4 (#1509) 2023-05-24 23:10:13 +08:00
Shuang
f83304c337
[CELEBORN-581][Flink] Support JobManager failover. (#1485) 2023-05-16 14:51:53 +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
Ethan Feng
114b1b4d62
[CELEBORN-548][FLINK] Support flink 1.17. (#1472) 2023-05-05 23:00:49 +08:00
Ethan Feng
93d2f106e0
[CELEBORN-548][FLINK] Support flink 1.15. (#1463) 2023-05-04 15:23:59 +08:00
Shuang
d68deecaaa
[CELEBORN-546][FLINK] Use autoIncrement partitionId replace encode(mapId, attemptId) for generating partitionId (#1447) 2023-04-22 16:33:22 +08:00
cxzl25
13f772e0c0
[CELEBORN-525] Fix wrong parameter celeborn.push.buffer.size 2023-04-14 20:45:25 +08:00
Keyong Zhou
13c610838b
[CELEBORN-455] Use 4 bytes instead of 16 to read mapId in FileWriter.… (#1369) 2023-03-21 19:21:50 +08:00
Keyong Zhou
7adf1fca41
[CELEBORN-295] Optimize data push (#1232)
* [CELEBORN-295] Add double buffer for sort pusher
2023-02-28 10:35:55 +08:00
Ethan Feng
e9b33751d3
[CELEBORN-289] Add flink integration test module. (#1229) 2023-02-21 12:25:23 +08:00