### What changes were proposed in this pull request?
Worker throw FileNotFoundException while fetch chunk:
```
java.io.FileNotFoundException: /xxx/celeborn-worker/shuffle_data/subdir_0/application_1693206141914_540726_1/1/871-0-0 (No such file or directory
```
before commit shuffle files, files are deleted in storage-scheduler thread
```
2023-09-07 19:38:16,506 [INFO] [dispatcher-event-loop-44] - org.apache.celeborn.service.deploy.worker.storage.StorageManager -Logging.scala(51) -Create file /xxx/celeborn-worker/shuffle_data/subdir_0/application_1693206141914_540726_1/1/986-0-0 success
2023-09-07 19:38:16,506 [INFO] [dispatcher-event-loop-44] - org.apache.celeborn.service.deploy.worker.Controller -Logging.scala(51) -Reserved 29 primary location and 0 replica location for application_1693206141914_540726_1-1
2023-09-07 19:38:16,537 [INFO] [storage-scheduler] - org.apache.celeborn.service.deploy.worker.storage.StorageManager -Logging.scala(51) -Delete expired app dir /xxx/celeborn-worker/shuffle_data/subdir_0/application_1693206141914_540726_1.
2023-09-07 19:38:16,580 [INFO] [storage-scheduler] - org.apache.celeborn.service.deploy.worker.storage.StorageManager -Logging.scala(51) -Delete expired app dir /xxx/celeborn-worker/shuffle_data/subdir_0/application_1693206141914_540726_1.
2023-09-07 19:38:16,629 [INFO] [storage-scheduler] - org.apache.celeborn.service.deploy.worker.storage.StorageManager -Logging.scala(51) -Delete expired app dir /xxx/celeborn-worker/shuffle_data/subdir_0/application_1693206141914_540726_1.
2023-09-07 19:38:16,661 [INFO] [storage-scheduler] - org.apache.celeborn.service.deploy.worker.storage.StorageManager -Logging.scala(51) -Delete expired app dir /xxx/celeborn-worker/shuffle_data/subdir_0/application_1693206141914_540726_1.
2023-09-07 19:38:16,681 [INFO] [storage-scheduler] - org.apache.celeborn.service.deploy.worker.storage.StorageManager -Logging.scala(51) -Delete expired app dir /xxx/celeborn-worker/shuffle_data/subdir_0/application_1693206141914_540726_1.
2023-09-07 19:38:17,355 [INFO] [dispatcher-event-loop-12] - org.apache.celeborn.service.deploy.worker.Controller -Logging.scala(51) -Start commitFiles for application_1693206141914_540726_1-1
2023-09-07 19:38:17,362 [INFO] [async-reply] - org.apache.celeborn.service.deploy.worker.Controller -Logging.scala(51) -CommitFiles for application_1693206141914_540726_1-1 success with 29 committed primary partitions, 0 empty primary partitions, 0 failed primary partitions, 0 committed replica partitions, 0 empty replica partitions, 0 failed replica partitions.
java.io.FileNotFoundException: /xxx/celeborn-worker/shuffle_data/subdir_0/application_1693206141914_540726_1/1/976-0-0 (No such file or directory)
java.io.FileNotFoundException: /xxx/celeborn-worker/shuffle_data/subdir_0/application_1693206141914_540726_1/1/482-0-0 (No such file or directory)
java.io.FileNotFoundException: /xxx/celeborn-worker/shuffle_data/subdir_0/application_1693206141914_540726_1/1/658-0-0 (No such file or directory)
```
it may have concurrent problem in this method.
``` scala
private def cleanupExpiredAppDirs(): Unit = {
val appIds = shuffleKeySet().asScala.map(key => Utils.splitShuffleKey(key)._1)
disksSnapshot().filter(_.status != DiskStatus.IO_HANG).foreach { diskInfo =>
diskInfo.dirs.foreach {
case workingDir if workingDir.exists() =>
workingDir.listFiles().foreach { appDir =>
// Don't delete shuffleKey's data that exist correct shuffle file info.
if (!appIds.contains(appDir.getName)) {
val threadPool = diskOperators.get(diskInfo.mountPoint)
deleteDirectory(appDir, threadPool)
logInfo(s"Delete expired app dir $appDir.")
}
}
// workingDir not exist when initializing worker on new disk
case _ => // do nothing
}
}
}
```
We should find all app directories first, then get the active shuffle keys.
https://issues.apache.org/jira/browse/CELEBORN-881
### Why are the changes needed?
Bugfix.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Passes GA and manual test.
Closes#1889 from zy-jordan/CELEBORN-881.
Lead-authored-by: hongzhaoyang <15316036153@163.com>
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>
### What changes were proposed in this pull request?
In some scenarios, DNS resolution may fail. We can record the DNS resolution results like Spark.
fd424caf6c/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java (L185-L192)
### Why are the changes needed?
### Does this PR introduce _any_ user-facing change?
### How was this patch tested?
Closes#1891 from cxzl25/CELEBORN-958.
Authored-by: sychen <sychen@ctrip.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
Only the Dockfile needs to change in this pr.
### Why are the changes needed?
When deploying celeborn for flink on kubernetes, Introducing jemalloc can improve pod memory usage.
### Does this PR introduce _any_ user-facing change?
None
### How was this patch tested?
Maybe starting a production job to test the memory usage improvement is needed.
Closes#1824 from mddxhj/feature/introduce_jemalloc.
Authored-by: Jun He <xuehaijuxian@gmail.com>
Signed-off-by: Cheng Pan <chengpan@apache.org>
### What changes were proposed in this pull request?
use `TimeUnit.NANOSECONDS.toMillis` instead of `/1000_000`
### Why are the changes needed?
### Does this PR introduce _any_ user-facing change?
### How was this patch tested?
Closes#1888 from cxzl25/CELEBORN-957.
Authored-by: sychen <sychen@ctrip.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### 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?
PASS GA
Closes#1887 from jiaoqingbo/956.
Authored-by: jiaoqingbo <1178404354@qq.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
As title
### Why are the changes needed?
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
GA
Closes#1886 from zwangsheng/CELEBORN-953.
Lead-authored-by: zwangsheng <2213335496@qq.com>
Co-authored-by: zwangsheng <binjieyang@apache.org>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
As title
### Why are the changes needed?
See https://issues.apache.org/jira/browse/SPARK-42689
### Does this PR introduce _any_ user-facing change?
Yes. User need to set `spark.shuffle.sort.io.plugin.class` to `org.apache.spark.shuffle.celeborn.CelebornShuffleDataIO` to enable this feature.
### How was this patch tested?
Add a new matrix dimension, shuffle-plugin-class, in github ci, to run spark tests over `LocalDiskShuffleDataIO` and `CelebornShuffleDataIO` respectively.
Closes#1884 from zhouyifan279/spark-driver-component.
Authored-by: zhouyifan279 <zhouyifan279@gmail.com>
Signed-off-by: Cheng Pan <chengpan@apache.org>
### 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?
Pass GA
Closes#1852 from cfmcgrady/audit-deps-ci.
Authored-by: Fu Chen <cfmcgrady@gmail.com>
Signed-off-by: Fu Chen <cfmcgrady@gmail.com>
### 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>
### What changes were proposed in this pull request?
Add sbt project `sparkColumnarShuffle` to `sparkGroup`
### Why are the changes needed?
Add the project `sparkColumnarShuffle` to the spark tests group `sparkGroup` to enable the columnar-related tests for SBT.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Run tests locally.
Closes#1854 from zhouyifan279/columnar-shuffle-sbt.
Authored-by: zhouyifan279 <zhouyifan279@gmail.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### 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?
PASS GA
Closes#1882 from jiaoqingbo/950.
Authored-by: jiaoqingbo <1178404354@qq.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
Currently judging whether it is a Gluten shuffle through serde is only applicable to Velox Backend. In order to adapt to ClickHouse Backend at the same time, it is more generic to use ColumnarShuffleDependency as the judgment basis.
### Why are the changes needed?
### Does this PR introduce _any_ user-facing change?
### How was this patch tested?
Closes#1878 from kerwin-zk/gluten.
Authored-by: xiyu.zk <xiyu.zk@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
Update the script to start word count demo.
### Why are the changes needed?
A user reported that he could not run the demo while following the quick start docs.
### Does this PR introduce _any_ user-facing change?
NO.
### How was this patch tested?
Cluster.
Closes#1880 from FMX/CELEBORN-948.
Authored-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
Add a link to expose cluster planning doc.
### Why are the changes needed?
Ditto.
### Does this PR introduce _any_ user-facing change?
NO.
### How was this patch tested?
Not necessary.
Closes#1879 from FMX/CELEBORN-944.
Authored-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
When shutdown type is decommission, we should change the `ShutdownHookManager#HookEntry`'s
timeout to `celeborn.worker.decommission.forceExitTimeout`.
### Why are the changes needed?
ditto
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Manual test
Closes#1877 from waitinfuture/945.
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>
### 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:

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>
### 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.

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.

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>
### What changes were proposed in this pull request?
This PR automatically iterates to the next branch to be merged instead of using the latest all the time
### Why are the changes needed?
anti-misoperation
### Does this PR introduce _any_ user-facing change?
no
### How was this patch tested?
manully
Closes#1870 from yaooqinn/CELEBORN-937.
Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
`build/release/release.sh` supports uploading client jars to Nexus.
### Why are the changes needed?
We should deploy the client jars to maven central.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Manually verification.
Closes#1875 from pan3793/CELEBORN-942.
Authored-by: Cheng Pan <chengpan@apache.org>
Signed-off-by: Cheng Pan <chengpan@apache.org>
### 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?
PASS GA
Closes#1872 from jiaoqingbo/939.
Authored-by: jiaoqingbo <1178404354@qq.com>
Signed-off-by: Shuang <lvshuang.tb@gmail.com>
### 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?
PASS GA
Closes#1873 from jiaoqingbo/940.
Authored-by: jiaoqingbo <1178404354@qq.com>
Signed-off-by: Shuang <lvshuang.tb@gmail.com>
### What changes were proposed in this pull request?
Fix the incorrect deploy doc about using HDFS only.
### Why are the changes needed?
Ditto.
### Does this PR introduce _any_ user-facing change?
NO.
### How was this patch tested?
Just docs.
Closes#1874 from FMX/CELEBORN-941.
Authored-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
Put map index args after partition index args in CelebornShuffleReader constructor
### Why are the changes needed?
#1853 changed the args order in CelebornShuffleReader constructor. It will break gluten celeborn shuffle manager.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Run test locally.
Closes#1869 from zhouyifan279/shuffle-reader-ctor.
Authored-by: zhouyifan279 <zhouyifan279@gmail.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
Separate local read test.
### Why are the changes needed?
Ditto.
### Does this PR introduce _any_ user-facing change?
NO.
### How was this patch tested?
Pass GA.
Closes#1871 from FMX/CELEBORN-752-FOLLOWUP.
Authored-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
Introduce a quick start guide for running Apache Flink with Apache Celeborn to help Flink users to run with Celeborn.
### Why are the changes needed?
There is no quick start guide for running Apache Flink with Apache Celeborn.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
None.
Closes#1868 from SteNicholas/CELEBORN-822.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### 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>
### What changes were proposed in this pull request?
Add metrics about active shuffle data size in every worker and update Grafana dashboard. The metric value will decrease when shuffle is expired.
### Why are the changes needed?
Ditto.
### Does this PR introduce _any_ user-facing change?
NO.
### How was this patch tested?
Cluster.
<img width="733" alt="截屏2023-08-30 17 00 11" src="https://github.com/apache/incubator-celeborn/assets/4150993/48e28c1c-2b49-45d7-b3ba-358674ff3f3d">
Closes#1867 from FMX/CELEBORN-933.
Authored-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
Shuffle master urls to avoid always connect first master first time
### Why are the changes needed?
### Does this PR introduce _any_ user-facing change?
### How was this patch tested?
Closes#1866 from zy-jordan/CELEBORN-936.
Authored-by: hongzhaoyang <15316036153@163.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
Correct `celeborn.metrics.conf.*.sink.csv.class` configuration example for a CSV sink.
### Why are the changes needed?
`celeborn.metrics.conf.*.sink.csv.class` configuration example for a CSV sink is wrong, which value should be `org.apache.celeborn.common.metrics.sink.CsvSink`.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
None.
Closes#1865 from SteNicholas/CELEBORN-927.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
Modify log content and location to accurately describe state changes
### Why are the changes needed?
In the previous implementation, when servingState was PUSH_PAUSED and lastState was PUSH_AND_REPLICATE_PAUSED, the code only triggered the Resume of REPLICATE_MODULE, but the log showed "Trigger action: PAUSE PUSH, RESUME REPLICATE"
The above log content is not accurate
### Does this PR introduce _any_ user-facing change?
NO
### How was this patch tested?
PASS GA
Closes#1864 from jiaoqingbo/934.
Authored-by: jiaoqingbo <1178404354@qq.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
This PR fixes the resolution for merged pull requests. It appears that the user "asfgit" is no longer closing pull requests, but rather the committers are.
### Why are the changes needed?
Bugfix, make the merge script re-runnable again if you accidentally abort cherry-pick or change you mind later for backporting
### Does this PR introduce _any_ user-facing change?
no
### How was this patch tested?
tested locally
Closes#1862 from yaooqinn/CELEBORN-931.
Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Cheng Pan <chengpan@apache.org>
### What changes were proposed in this pull request?
Replace JIRA_USERNAME and JIRA_PASSWORD with ASF_*
### Why are the changes needed?
hotfix
### Does this PR introduce _any_ user-facing change?
no
### How was this patch tested?
manually
Closes#1861 from yaooqinn/CELEBORN-930_F.
Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Cheng Pan <chengpan@apache.org>
### What changes were proposed in this pull request?
Don't stop LocalFlusher when notify error.
### Why are the changes needed?
If LocalDeviceMonitor find non-critical error(e.g. disk full usage) count exceeds threshold and notify error to observed device. LocalFlusher will do stopAndCleanFlusher, which will interrupt flush threads. Then if the disk recover from the error, new disk buffer pushed into the flush queue cannot be flushed. It always causes high memory usage, which would cause pausePushDataAndReplicate, and can't recover from it .
### Does this PR introduce _any_ user-facing change?
no
### How was this patch tested?
Manual.
1. Run a spark job with shuffle.
2. Copy data in a worker to trigger full disk error.
3. See logs as below

5. Confirm flush threads are still running.
6. Submit another spark job with shuffle, and found disk buffer increase and decrease while flushing disk.

Closes#1860 from liying919/opt-disk-buffer.
Authored-by: 宪英 <xianying.ly@antgroup.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
Run `dev/reformat` before you create a new pull request for code style in `CONTRIBUTING.md`.
### Why are the changes needed?
Affects the contribution of creating pull request.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
No.
Closes#1859 from SteNicholas/CELEBORN-927.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### 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>
### What changes were proposed in this pull request?
Previously, we allowed for token authentication when resolving Jira issues in pull request merging. However, the token auth is kinda lazy during the initial handshake, maintainers might get confused someday.
This pull request promptly calls the current_user() function to initiate authentication and provides clear instructions for token expiration.
see also 8523ee5d90
### Why are the changes needed?
make it easy for maintainers to update their expired Jira tokens.
### Does this PR introduce _any_ user-facing change?
no
### How was this patch tested?
a maintainer can test this with invalid tokens
Closes#1857 from yaooqinn/CELEBORN-930.
Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
As title
### Why are the changes needed?
This PR
1. Strengthening assertion conditions.
2. Enabling the previously ignored `testLargeFile` scenario.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Pass GA
Closes#1848 from cfmcgrady/refine-partition-files-sorter-suite.
Authored-by: Fu Chen <cfmcgrady@gmail.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
…ion index args in ShuffleReader related methods
### What changes were proposed in this pull request?
Unify the order of map index args and partition index args in ShuffleReader related methods.
### Why are the changes needed?
The order of map index args and partition index args in CelebornShuffleReader constructor is different the order in `SparkShuffleManager#getReader`.
It can messed up easily.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Run tests locally.
Closes#1853 from zhouyifan279/columnar-shuffle-followup.
Authored-by: zhouyifan279 <zhouyifan279@gmail.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
`CelebornShuffleFallbackPolicyRunner` could not only check quota, but also check whether cluster has available workers. If there is no available workers, fallback to external shuffle.
### Why are the changes needed?
`CelebornShuffleFallbackPolicyRunner` adds a check for available workers.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
- `SparkShuffleManagerSuite#testClusterNotAvailableWithAvailableWorkers`
Closes#1814 from SteNicholas/CELEBORN-830.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
As title.
### Why are the changes needed?
As title.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Passes GA.
Closes#1851 from waitinfuture/927.
Authored-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
Using `awaitTermination` instead of `shutdownNow`.
### Why are the changes needed?
When we call `wait` function without get the object's monitor will meet `IllegalMonitorStateException`.
And In `saveAllCommittedFileInfosToDB` this function, we are hard to get threshold monitor.
`wait` method metioned:
>IllegalMonitorStateException – if the current thread is not the owner of the object's monitor.
We can using `awaitTermination` replace of `shutdownNow`.
According to desc about `shutdownNow` method:
> This method does not wait for actively executing tasks to terminate. Use awaitTermination to do that.
And `awaitTermination` metions:
> Blocks until all tasks have completed execution after a shutdown request, or the timeout occurs, or the current thread is interrupted, whichever happens first.
At all, `awaitTermination` is applicable to the current scenario.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
UT
Closes#1849 from zwangsheng/CELEBORN-926.
Authored-by: zwangsheng <2213335496@qq.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### 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>
### What changes were proposed in this pull request?
Move Columnar Shuffle code into an individual module
### Why are the changes needed?
Spark 3.5 made a lot of changes to AtomicType in https://issues.apache.org/jira/browse/SPARK-42887.
This causes compilation error when building columnar shuffle code.
As columnar shuffle is a configurable feature, I think it's better to move related code into a individual module. Then we can exclude this module when build with Spark 3.5 for now.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Add test `ColumnarHashBasedShuffleWriterSuiteJ` and `CelebornColumnarShuffleReaderSuite`
Closes#1843 from zhouyifan279/columnar-shuffle.
Authored-by: zhouyifan279 <zhouyifan279@gmail.com>
Signed-off-by: Cheng Pan <chengpan@apache.org>
### 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?
PASS GA
Closes#1847 from jiaoqingbo/CELEBORN-924.
Authored-by: jiaoqingbo <1178404354@qq.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
As title
### Why are the changes needed?
To address the CI failure introduced in https://github.com/apache/incubator-celeborn/pull/1831, this pull request resolves the issue by removing the `log4j-slf4j2-impl` dependency from SBT profile `spark-3.4`. This change is prompted by the pinning of `slf4j-api` to version 1.7.36, rendering `log4j-slf4j2-impl` unnecessary.
```
[error] Test org.apache.spark.shuffle.celeborn.SortBasedPusherSuiteJ failed: java.lang.NoSuchMethodError: org.apache.logging.slf4j.Log4jLoggerFactory: method <init>()V not found, took 0.0 sec
[error] at org.slf4j.impl.StaticLoggerBinder.<init>(StaticLoggerBinder.java:53)
[error] at org.slf4j.impl.StaticLoggerBinder.<clinit>(StaticLoggerBinder.java:41)
[error] at org.slf4j.LoggerFactory.bind(LoggerFactory.java:150)
[error] at org.slf4j.LoggerFactory.performInitialization(LoggerFactory.java:124)
[error] at org.slf4j.LoggerFactory.getILoggerFactory(LoggerFactory.java:417)
[error] at org.slf4j.LoggerFactory.getLogger(LoggerFactory.java:362)
[error] at org.slf4j.LoggerFactory.getLogger(LoggerFactory.java:3[88](https://github.com/apache/incubator-celeborn/actions/runs/5974971986/job/16210071148#step:4:89))
[error] at org.apache.spark.shuffle.celeborn.SortBasedPusherSuiteJ.<clinit>(SortBasedPusherSuiteJ.java:51)
[error] ...
[error] Test org.apache.spark.shuffle.celeborn.SortBasedPusherSuiteJ failed: java.lang.NoClassDefFoundError: Could not initialize class org.apache.spark.shuffle.celeborn.SortBasedPusherSuiteJ, took 0.0 sec
[error] at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
[error] ...
```
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Pass GA
Closes#1844 from cfmcgrady/celeborn-906-followup.
Authored-by: Fu Chen <cfmcgrady@gmail.com>
Signed-off-by: Cheng Pan <chengpan@apache.org>
### What changes were proposed in this pull request?
Fix a broken link in docs/developers/overview.md.
### Why are the changes needed?
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Locally tested.
Closes#1845 from zhouyifan279/upgrade-page-link.
Authored-by: zhouyifan279 <zhouyifan279@gmail.com>
Signed-off-by: Cheng Pan <chengpan@apache.org>
### What changes were proposed in this pull request?
As title
### Why are the changes needed?
This PR ensures dependency alignment between SBT and Maven, based on the audit results implemented in https://github.com/apache/incubator-celeborn/pull/1797
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Pass GA and Review
Closes#1831 from cfmcgrady/align-deps-2.
Authored-by: Fu Chen <cfmcgrady@gmail.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### 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?
Pass GA
Closes#1797 from cfmcgrady/audit-deps.
Authored-by: Fu Chen <cfmcgrady@gmail.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>