### What changes were proposed in this pull request?
Add failureaccess shade.
### Why are the changes needed?
When test main branch, client got error like below:
```
Caused by: java.lang.NoClassDefFoundError: org/apache/celeborn/shaded/com/google/common/util/concurrent/internal/InternalFutureFailureAccess
at org.apache.celeborn.shaded.com.google.common.cache.LocalCache$LoadingValueReference.<init>(LocalCache.java:3517)
at org.apache.celeborn.shaded.com.google.common.cache.LocalCache$LoadingValueReference.<init>(LocalCache.java:3521)
at org.apache.celeborn.shaded.com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2170)
at org.apache.celeborn.shaded.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2081)
at org.apache.celeborn.shaded.com.google.common.cache.LocalCache.get(LocalCache.java:4019)
at org.apache.celeborn.shaded.com.google.common.cache.LocalCache$LocalManualCache.get(LocalCache.java:4933)
at org.apache.celeborn.client.commit.ReducePartitionCommitHandler.replyGetReducerFileGroup(ReducePartitionCommitHandler.scala:283)
at org.apache.celeborn.client.commit.ReducePartitionCommitHandler.handleGetReducerFileGroup(ReducePartitionCommitHandler.scala:300)
at org.apache.celeborn.client.CommitManager.handleGetReducerFileGroup(CommitManager.scala:266)
at org.apache.celeborn.client.LifecycleManager.org$apache$celeborn$client$LifecycleManager$$handleGetReducerFileGroup(LifecycleManager.scala:628)
at org.apache.celeborn.client.LifecycleManager$$anonfun$receiveAndReply$1.applyOrElse(LifecycleManager.scala:314)
at org.apache.celeborn.common.rpc.netty.Inbox.$anonfun$process$1(Inbox.scala:115)
at org.apache.celeborn.common.rpc.netty.Inbox.safelyCall(Inbox.scala:222)
at org.apache.celeborn.common.rpc.netty.Inbox.process(Inbox.scala:110)
at org.apache.celeborn.common.rpc.netty.Dispatcher$MessageLoop.run(Dispatcher.scala:227)
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)
```
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Manual test.
Closes#2116 from onebox-li/shade-add-failureaccess.
Authored-by: onebox-li <lyh-36@163.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
Celeborn Flink client validates whether `execution.batch-shuffle-mode` is `ALL_EXCHANGES_BLOCKING`.
### Why are the changes needed?
The config option `execution.batch-shuffle-mode` of Flink is `ALL_EXCHANGES_BLOCKING` by default. Celeborn Flink client should validate whether `execution.batch-shuffle-mode` is `ALL_EXCHANGES_BLOCKING`. If `execution.batch-shuffle-mode` is set as `ALL_EXCHANGES_PIPELINED`, there is `NullPointerException` for `ReducePartitionCommitHandler#handleGetReducerFileGroup`, which exception is as follows:
```
023-11-16 14:40:55,984 ERROR org.apache.celeborn.common.rpc.netty.Inbox - Ignoring error
java.lang.NullPointerException: Cannot invoke "java.util.Set.add(Object)" because the return value of "java.util.concurrent.ConcurrentHashMap.get(Object)" is null
at org.apache.celeborn.client.commit.ReducePartitionCommitHandler.handleGetReducerFileGroup(ReducePartitionCommitHandler.scala:307)
at org.apache.celeborn.client.CommitManager.handleGetReducerFileGroup(CommitManager.scala:266)
at org.apache.celeborn.client.LifecycleManager.org$apache$celeborn$client$LifecycleManager$$handleGetReducerFileGroup(LifecycleManager.scala:559)
at org.apache.celeborn.client.LifecycleManager$$anonfun$receiveAndReply$1.applyOrElse(LifecycleManager.scala:297)
at org.apache.celeborn.common.rpc.netty.Inbox.$anonfun$process$1(Inbox.scala:115)
at org.apache.celeborn.common.rpc.netty.Inbox.safelyCall(Inbox.scala:222)
at org.apache.celeborn.common.rpc.netty.Inbox.process(Inbox.scala:110)
at org.apache.celeborn.common.rpc.netty.Dispatcher$MessageLoop.run(Dispatcher.scala:227)
at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
at java.base/java.lang.Thread.run(Thread.java:833)
```
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
`RemoteShuffleServiceFactorySuitJ#testInvalidShuffleServiceConfig`.
Closes#2106 from SteNicholas/CELEBORN-1134.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
Update log level for `TransferBufferPool#reserveBuffers` from warn to debug.
### Why are the changes needed?
The log level of `TransferBufferPool#reserveBuffers` should not be warn, which is normal credit-based shuffle read log.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
No.
Closes#2076 from SteNicholas/reserve-buffers-log.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
### Why are the changes needed?
```bash
flink-1.18.0
./bin/start-cluster.sh
./bin/flink run examples/streaming/WordCount.jar --execution-mode BATCH
```
```java
Caused by: java.lang.NoSuchMethodError: org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate.<init>(Ljava/lang/String;ILorg/apache/flink/runtime/jobgraph/IntermediateDataSetID;Lorg/apache/flink/runtime/io/network/partition/ResultPartitionType;Lorg/apache/flink/runtime/executiongraph/IndexRange;ILorg/apache/flink/runtime/io/network/partition/PartitionProducerStateProvider;Lorg/apache/flink/util/function/SupplierWithException;Lorg/apache/flink/runtime/io/network/buffer/BufferDecompressor;Lorg/apache/flink/core/memory/MemorySegmentProvider;ILorg/apache/flink/runtime/throughput/ThroughputCalculator;Lorg/apache/flink/runtime/throughput/BufferDebloater;)V
at org.apache.celeborn.plugin.flink.RemoteShuffleInputGate$FakedRemoteInputChannel.<init>(RemoteShuffleInputGate.java:225)
at org.apache.celeborn.plugin.flink.RemoteShuffleInputGate.getChannel(RemoteShuffleInputGate.java:179)
at org.apache.flink.runtime.io.network.partition.consumer.InputGate.setChannelStateWriter(InputGate.java:90)
at org.apache.flink.runtime.taskmanager.InputGateWithMetrics.setChannelStateWriter(InputGateWithMetrics.java:120)
at org.apache.flink.streaming.runtime.tasks.StreamTask.injectChannelStateWriterIntoChannels(StreamTask.java:524)
at org.apache.flink.streaming.runtime.tasks.StreamTask.<init>(StreamTask.java:496)
```
Flink 1.18.0 release
https://flink.apache.org/2023/10/24/announcing-the-release-of-apache-flink-1.18/
Interface `org.apache.flink.runtime.io.network.buffer.Buffer` adds `setRecycler` method.
[[FLINK-32549](https://issues.apache.org/jira/browse/FLINK-32549)][network] Tiered storage memory manager supports ownership transfer for buffers
`org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate` constructor adds parameters.
[[FLINK-31638](https://issues.apache.org/jira/browse/FLINK-31638)][network] Introduce the TieredStorageConsumerClient to SingleInputGate
[[FLINK-31642](https://issues.apache.org/jira/browse/FLINK-31642)][network] Introduce the MemoryTierConsumerAgent to TieredStorageConsumerClient
### Does this PR introduce _any_ user-facing change?
### How was this patch tested?
```bash
flink-1.18.0 ./bin/flink run examples/streaming/WordCount.jar --execution-mode BATCH
Executing example with default input data.
Use --input to specify file input.
Printing result to stdout. Use --output to specify output path.
Job has been submitted with JobID d7fc5f0ca018a54e9453c4d35f7c598a
Program execution finished
Job with JobID d7fc5f0ca018a54e9453c4d35f7c598a has finished.
Job Runtime: 1635 ms
```
<img width="1297" alt="image" src="https://github.com/apache/incubator-celeborn/assets/3898450/6a5266bf-2386-4386-b98b-a60d2570fa99">
Closes#2063 from cxzl25/CELEBORN-1105.
Authored-by: sychen <sychen@ctrip.com>
Signed-off-by: Shuang <lvshuang.tb@gmail.com>
### What changes were proposed in this pull request?
Optimize the retrieval of configuration in the internalCreateClient
### Why are the changes needed?
Directly accessing configuration information through 'conf.xx' in 'internalCreateClient' is time-consuming.

### Does this PR introduce _any_ user-facing change?
### How was this patch tested?
Closes#2055 from kerwin-zk/client-factory-conf.
Authored-by: xiyu.zk <xiyu.zk@alibaba-inc.com>
Signed-off-by: Fu Chen <cfmcgrady@gmail.com>
* Fix use of `GuardedBy` on nonexistant lock.
* Annotate methods, which are expected to be called with lock held, with `GuardedBy` so that error prone can analyze all invocations
There is no functional change, but it helps errorprone analysis.
No
Unit tests
Closes#2029 from mridulm/fix-flink-guarded-by-annotation.
Authored-by: Mridul Muralidharan <mridulatgmail.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
`StreamChunkSlice`, `ChunkFetchRequest` and `TransportableError` should merge to transport messages to enhance celeborn's compatibility.
### Why are the changes needed?
1. Improves celeborn's transport flexibility to change RPC.
2. Makes Compatible with 0.2 client.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
- `FetchHandlerSuiteJ`
- `RequestTimeoutIntegrationSuiteJ`
- `ChunkFetchIntegrationSuiteJ`
Closes#1982 from SteNicholas/CELEBORN-772.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: Shuang <lvshuang.tb@gmail.com>
### What changes were proposed in this pull request?
`batchHandleChangePartitionExecutors` could not handle uncaught exception in `ChangePartitionRequest`, which causes that the uncaught exception of the thread could not get for troubleshooting. Thread factory should set `UncaughtExceptionHandler` to handle uncaught exception.
### Why are the changes needed?
Thread factory sets `UncaughtExceptionHandler` to handle uncaught exception in `ThreadUtils`.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Unit tests.
Closes#1962 from SteNicholas/CELEBORN-1024.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: Cheng Pan <chengpan@apache.org>
### 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>
### 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`

### 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>
### What changes were proposed in this pull request?
Correct the `LICENSE` and `NOTICE` for the following shaded client jars
- `celeborn-client-flink-1.14-shaded_2.12-<version>.jar`
- `celeborn-client-flink-1.15-shaded_2.12-<version>.jar`
- `celeborn-client-flink-1.17-shaded_2.12-<version>.jar`
- `celeborn-client-mr-shaded_2.12-<version>.jar`
- `celeborn-client-spark-2-shaded_2.11-<version>.jar`
- `celeborn-client-spark-3-shaded_2.12-<version>.jar`
### Why are the changes needed?
The `LICENSE` and `NOTICE` shipped in a jar should match the content of the jar, for shaded jars, it should acknowledge all the third-party classes that are bundled.
See more discussion at https://lists.apache.org/thread/8v4wy5o132rpsjync6465zztgjlf6h5p
For how to determine which third-party jars are bundled, take `celeborn-client-spark-3-shaded_2.12-<version>.jar` as an example, the following command performs the packaging, and we can find them out by looking at logs like `Including ... in the shaded jar`
```
build/mvn clean package -DskipTests -pl :celeborn-client-spark-3-shaded_2.12 -am -Pspark-3.3
```
```
[INFO] --- maven-shade-plugin:3.4.0:shade (default) celeborn-client-spark-3-shaded_2.12 ---
[INFO] Including org.apache.celeborn:celeborn-client-spark-3_2.12🫙0.4.0-SNAPSHOT in the shaded jar.
[INFO] Including org.apache.celeborn:celeborn-common_2.12🫙0.4.0-SNAPSHOT in the shaded jar.
[INFO] Including org.apache.commons:commons-lang3:jar:3.12.0 in the shaded jar.
[INFO] Including io.netty:netty-all:jar:4.1.93.Final in the shaded jar.
[INFO] Including io.netty:netty-buffer:jar:4.1.93.Final in the shaded jar.
...
[INFO] Excluding org.apache.ratis:ratis-common:jar:2.5.1 from the shaded jar.
[INFO] Excluding org.apache.ratis:ratis-thirdparty-misc:jar:1.0.4 from the shaded jar.
[INFO] Excluding org.apache.ratis:ratis-proto:jar:2.5.1 from the shaded jar.
...
```
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Manually.
Closes#1933 from pan3793/CELEBORN-1003.
Authored-by: Cheng Pan <chengpan@apache.org>
Signed-off-by: Cheng Pan <chengpan@apache.org>
### 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?
GA
Closes#1941 from cxzl25/CELEBORN-986.
Authored-by: sychen <sychen@ctrip.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
`BacklogAnnouncement`, `BufferStreamEnd`, and `ReadAddCredit` should merge to transport messages to enhance celeborn's compatibility.
### Why are the changes needed?
1. Improves celeborn's transport flexibility to change RPC.
2. Makes Compatible with 0.2 client.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
- `TransportFrameDecoderWithBufferSupplierSuiteJ`
Closes#1905 from SteNicholas/CELEBORN-770.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: Shuang <lvshuang.tb@gmail.com>
### What changes were proposed in this pull request?
`PushDataHandShake`, `RegionFinish`, and `RegionStart` should merge to transport messages to enhance celeborn's compatibility.
### Why are the changes needed?
1. Improves celeborn's transport flexibility to change RPC.
2. Makes Compatible with 0.2 client.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
- `RemoteShuffleOutputGateSuiteJ`
Closes#1910 from SteNicholas/CELEBORN-771.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### 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?
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?
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?
1. Log offer slots results from LifecycleManager.
2. Log change partition results from LifecycleManager.
3. Log reserve slots results.
4. Log fetch file group failure instead of data lost.
### Why are the changes needed?
If data lost happened, we need to find out what worker cause this failure. So we need to check reserve slots result from LifecycleManager.
### Does this PR introduce _any_ user-facing change?
NO.
### How was this patch tested?
GA.
Closes#1798 from FMX/CELEBORN-876.
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?
1. Wrap IOException to PartitionUnRetryAbleException when fetch
2. Improve message logging when open stream/read data error
### Why are the changes needed?
When open stream, there would be encounter many different IOExceptions such as NoSuchFileException, FileNotFoundException,FileCorruptedException etc, for these checked exception should wrap to PartitionUnRetryAbleException to let client choose to regenerate the data.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
UT & Manual test
Closes#1796 from RexXiong/CELEBORN-878-IO-Exception.
Authored-by: Shuang <lvshuang.tb@gmail.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
After discussion, we make sure that `shuffleManager.unregisterShuffle()` will be triggered by Spark both in driver and executor. In this pr:
1. Add shuffle client both in driver and executor side in ShuffleManager
2. ShuffleClient call cleanupShuffle() when trigger `unregisterShuffle`.
This replaced https://github.com/apache/incubator-celeborn/pull/1719
### Why are the changes needed?
### Does this PR introduce _any_ user-facing change?
### How was this patch tested?
Closes#1726 from AngersZhuuuu/CELEBORN-804.
Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Cheng Pan <chengpan@apache.org>
### 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>
…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):

After this PR, the number of PartitionLocation objects decreases to 275 thousands

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>
### 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>
### 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>
### 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>
### What changes were proposed in this pull request?
Metics update logic need align with Flink 1.17/1.15
### Why are the changes needed?
See [1626](https://github.com/apache/incubator-celeborn/pull/1626) And metics update logic need align with Flink 1.17/1.15
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Tpcds Manual
Closes#1631 from RexXiong/CELEBORN-717-FOLLOWUP.
Authored-by: Shuang <lvshuang.tb@gmail.com>
Signed-off-by: zhongqiang.czq <zhongqiang.czq@alibaba-inc.com>
### What changes were proposed in this pull request?
Reset numBytesOut/numBuffersOut metrics for RemoteShuffleResultPartition
### Why are the changes needed?
Currently ResultPartition lost numBytesOut/numBuffersOut metrics, this will cause Flink AdaptiveScheduler can not dynamically adjust the task parallelism based on the input amount of data
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Manual test.
Closes#1626 from RexXiong/CELEBORN-717.
Authored-by: Shuang <lvshuang.tb@gmail.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
As title
### Why are the changes needed?
before this PR the `liborg_apache_celeborn_shaded_netty_transport_native_epoll_aarch_64.so` can't correctly be loaded.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Manually tested
```shell
> tar zxf celeborn-client-spark-3-shaded_2.12-0.4.0-SNAPSHOT.jar
> find * -name "*.so"
META-INF/native/liborg_apache_celeborn_shaded_netty_transport_native_epoll_aarch_64.so
META-INF/native/liborg_apache_celeborn_shaded_netty_transport_native_epoll_x86_64.so
```
Closes#1625 from cfmcgrady/typo.
Authored-by: Fu Chen <cfmcgrady@gmail.com>
Signed-off-by: Cheng Pan <chengpan@apache.org>
### 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>
### What changes were proposed in this pull request?
ShuffleClientImpl::mapperEnded should not consider attemptId, speculation tasks will update attemptId.
### Why are the changes needed?
ditto
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Cluster test
Closes#1591 from onebox-li/fix-mapend.
Authored-by: onebox-li <lyh-36@163.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
Use System.currentTimeMillis() + JobID.generate() as CelebornAppId.
### Why are the changes needed?
Flink Application mode with HA may use fixed id(00000000000000000000000000000000) as jobId. see [FLINK-19358](https://issues.apache.org/jira/browse/FLINK-19358).
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Manual
Closes#1572 from RexXiong/CELEBORN-660.
Authored-by: Shuang <lvshuang.tb@gmail.com>
Signed-off-by: zhongqiang.czq <zhongqiang.czq@alibaba-inc.com>
### 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>
### 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>