### What changes were proposed in this pull request?
Now worker will send WorkLost too, master should also reply WorkerLostResponse when it's unknown worker
### Why are the changes needed?
### Does this PR introduce _any_ user-facing change?
### How was this patch tested?
Closes#1584 from AngersZhuuuu/CELEBORN-668-FOLLOWUP.
Lead-authored-by: Angerszhuuuu <angers.zhu@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?
Include ConnectException when exclude worker for fetch
### Why are the changes needed?
Currently RssInputStream.isCriticalCause does not include ConnectException
```
java.io.IOException: Failed to connect to /192.168.1.17:46197
at org.apache.celeborn.common.network.client.TransportClientFactory.internalCreateClient(TransportClientFactory.java:232)
at org.apache.celeborn.common.network.client.TransportClientFactory.createClient(TransportClientFactory.java:176)
at org.apache.celeborn.common.network.client.TransportClientFactory.createClient(TransportClientFactory.java:114)
at org.apache.celeborn.common.network.client.TransportClientFactory.createClient(TransportClientFactory.java:183)
at org.apache.celeborn.client.read.WorkerPartitionReader.<init>(WorkerPartitionReader.java:103)
at org.apache.celeborn.client.read.RssInputStream$RssInputStreamImpl.createReader(RssInputStream.java:399)
at org.apache.celeborn.client.read.RssInputStream$RssInputStreamImpl.createReaderWithRetry(RssInputStream.java:301)
at org.apache.celeborn.client.read.RssInputStream$RssInputStreamImpl.moveToNextReader(RssInputStream.java:229)
at org.apache.celeborn.client.read.RssInputStream$RssInputStreamImpl.<init>(RssInputStream.java:178)
at org.apache.celeborn.client.read.RssInputStream.create(RssInputStream.java:63)
at org.apache.celeborn.client.ShuffleClientImpl.readPartition(ShuffleClientImpl.java:1599)
at org.apache.spark.shuffle.celeborn.RssShuffleReader.$anonfun$read$1(RssShuffleReader.scala:88)
at org.apache.spark.shuffle.celeborn.RssShuffleReader.$anonfun$read$1$adapted(RssShuffleReader.scala:79)
at scala.collection.Iterator$$anon$10.next(Iterator.scala:461)
at scala.collection.Iterator$$anon$11.nextCur(Iterator.scala:486)
at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:492)
at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
at org.apache.spark.util.CompletionIterator.hasNext(CompletionIterator.scala:31)
at org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:37)
at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage17.sort_addToSorter_0$(Unknown Source)
at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage17.sort_doSort_0$(Unknown Source)
at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage17.processNext(Unknown Source)
at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:35)
at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage17.hasNext(Unknown Source)
at org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:954)
at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage18.smj_findNextJoinRows_0$(Unknown Source)
at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage18.processNext(Unknown Source)
at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:35)
at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage18.hasNext(Unknown Source)
at org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$2.hasNext(WholeStageCodegenExec.scala:973)
at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
at org.apache.spark.shuffle.celeborn.HashBasedShuffleWriter.fastWrite0(HashBasedShuffleWriter.java:251)
at org.apache.spark.shuffle.celeborn.HashBasedShuffleWriter.write(HashBasedShuffleWriter.java:180)
at org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:59)
at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:99)
at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:52)
at org.apache.spark.scheduler.Task.run(Task.scala:133)
at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:506)
at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1474)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:509)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
at java.lang.Thread.run(Thread.java:750)
Caused by: org.apache.celeborn.shaded.io.netty.channel.AbstractChannel$AnnotatedConnectException: Connection refused: /192.168.1.17:46197
Caused by: java.net.ConnectException: Connection refused
at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:716)
at org.apache.celeborn.shaded.io.netty.channel.socket.nio.NioSocketChannel.doFinishConnect(NioSocketChannel.java:337)
at org.apache.celeborn.shaded.io.netty.channel.nio.AbstractNioChannel$AbstractNioUnsafe.finishConnect(AbstractNioChannel.java:334)
at org.apache.celeborn.shaded.io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:776)
at org.apache.celeborn.shaded.io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:724)
at org.apache.celeborn.shaded.io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:650)
at org.apache.celeborn.shaded.io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:562)
at org.apache.celeborn.shaded.io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:997)
at org.apache.celeborn.shaded.io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
at org.apache.celeborn.shaded.io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
at java.lang.Thread.run(Thread.java:750)
```
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Manual test
Closes#1598 from waitinfuture/686.
Authored-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### 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>
### What changes were proposed in this pull request?
Celeborn fetch chunk also should support check timeout
#### Test case
```
executor instance 20
SQL:
SELECT count(1) from (select /*+ REPARTITION(100) */ * from spark_auxiliary.t50g) tmp;
--conf spark.celeborn.client.spark.shuffle.writer=sort \
--conf spark.celeborn.client.fetch.excludeWorkerOnFailure.enabled=true \
--conf spark.celeborn.client.push.timeout=10s \
--conf spark.celeborn.client.push.replicate.enabled=true \
--conf spark.celeborn.client.push.revive.maxRetries=10 \
--conf spark.celeborn.client.reserveSlots.maxRetries=10 \
--conf spark.celeborn.client.registerShuffle.maxRetries=3 \
--conf spark.celeborn.client.push.blacklist.enabled=true \
--conf spark.celeborn.client.blacklistSlave.enabled=true \
--conf spark.celeborn.client.fetch.timeout=30s \
--conf spark.celeborn.client.push.data.timeout=30s \
--conf spark.celeborn.client.push.limit.inFlight.timeout=600s \
--conf spark.celeborn.client.push.maxReqsInFlight=32 \
--conf spark.celeborn.client.shuffle.compression.codec=ZSTD \
--conf spark.celeborn.rpc.askTimeout=30s \
--conf spark.celeborn.client.rpc.reserveSlots.askTimeout=30s \
--conf spark.celeborn.client.shuffle.batchHandleChangePartition.enabled=true \
--conf spark.celeborn.client.shuffle.batchHandleCommitPartition.enabled=true \
--conf spark.celeborn.client.shuffle.batchHandleReleasePartition.enabled=true
```
Test with 3 worker and add a `Thread.sleep(100s)` before worker handle `ChunkFetchRequest`
Before patch
<img width="1783" alt="截屏2023-06-14 上午11 20 55" src="https://github.com/apache/incubator-celeborn/assets/46485123/182dff7d-a057-4077-8368-d1552104d206">
After patch
<img width="1792" alt="image" src="https://github.com/apache/incubator-celeborn/assets/46485123/3c8b7933-8ace-426d-8e9f-04e0aabfac8e">
The log shows the fetch timeout checker workers
```
23/06/14 11:14:54 ERROR WorkerPartitionReader: Fetch chunk 0 failed.
org.apache.celeborn.common.exception.CelebornIOException: FETCH_DATA_TIMEOUT
at org.apache.celeborn.common.network.client.TransportResponseHandler.failExpiredFetchRequest(TransportResponseHandler.java:147)
at org.apache.celeborn.common.network.client.TransportResponseHandler.lambda$new$1(TransportResponseHandler.java:103)
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
at java.lang.Thread.run(Thread.java:748)
23/06/14 11:14:54 WARN RssInputStream: Fetch chunk failed 1/6 times for location PartitionLocation[
id-epoch:35-0
host-rpcPort-pushPort-fetchPort-replicatePort:10.169.48.203-9092-9094-9093-9095
mode:MASTER
peer:(host-rpcPort-pushPort-fetchPort-replicatePort:10.169.48.202-9092-9094-9093-9095)
storage hint:StorageInfo{type=HDD, mountPoint='/mnt/ssd/0', finalResult=true, filePath=}
mapIdBitMap:null], change to peer
org.apache.celeborn.common.exception.CelebornIOException: Fetch chunk 0 failed.
at org.apache.celeborn.client.read.WorkerPartitionReader$1.onFailure(WorkerPartitionReader.java:98)
at org.apache.celeborn.common.network.client.TransportResponseHandler.failExpiredFetchRequest(TransportResponseHandler.java:146)
at org.apache.celeborn.common.network.client.TransportResponseHandler.lambda$new$1(TransportResponseHandler.java:103)
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
at java.lang.Thread.run(Thread.java:748)
Caused by: org.apache.celeborn.common.exception.CelebornIOException: FETCH_DATA_TIMEOUT
at org.apache.celeborn.common.network.client.TransportResponseHandler.failExpiredFetchRequest(TransportResponseHandler.java:147)
... 8 more
23/06/14 11:14:54 INFO SortBasedShuffleWriter: Memory used 72.0 MB
```
### Why are the changes needed?
### Does this PR introduce _any_ user-facing change?
### How was this patch tested?
Closes#1587 from AngersZhuuuu/CELEBORN-676.
Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Angerszhuuuu <angers.zhu@gmail.com>
### 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?
Add celeborn.metrics.conf to conf entity
### Why are the changes needed?
### Does this PR introduce _any_ user-facing change?
### How was this patch tested?
Closes#1593 from AngersZhuuuu/CELEBORN-681.
Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Angerszhuuuu <angers.zhu@gmail.com>
### What changes were proposed in this pull request?
refer to https://github.com/apache/spark/pull/40301
1. Optimize `Utils.bytesToString`. Arithmetic ops on BigInt and BigDecimal are order(s) of magnitude slower than the ops on primitive types. Division is an especially slow operation and it is used en masse here.
2. According to the information sourced from [Wikipedia](https://en.wikipedia.org/wiki/Kilobyte), it is established that 1000 is the appropriate factor for representing kilobytes (KB), while 1024 is the correct factor for kibibytes (KiB). In alignment with this understanding, changing the size unit from "KB" to "KiB".
### Why are the changes needed?
the Utils#bytesToString method is frequently employed in memory-related log messages.
### Does this PR introduce _any_ user-facing change?
No, only perf improvement.
### How was this patch tested?
existing UT and manually tested.
Closes#1590 from cfmcgrady/bytesToString.
Authored-by: Fu Chen <cfmcgrady@gmail.com>
Signed-off-by: Cheng Pan <chengpan@apache.org>
### What changes were proposed in this pull request?
Give Heartbeat one byte message and skip this byte when decode.
### Why are the changes needed?
Heartbeat message may split in to two netty buffer, then the `empty buffer` (which don't need actually, but need keep) be wrong removed, then decodeNext would throw NPE. see
``` java
while (headerBuf.readableBytes() < HEADER_SIZE) {
ByteBuf next = buffers.getFirst();
int toRead = Math.min(next.readableBytes(), HEADER_SIZE - headerBuf.readableBytes());
headerBuf.writeBytes(next, toRead);
if (!next.isReadable()) {
buffers.removeFirst().release();
}
}
```
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
UT & MANUAL
Closes#1589 from RexXiong/CELEBORN-675.
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?
Add hasPeer method to PartitionLocation
### Why are the changes needed?
### Does this PR introduce _any_ user-facing change?
### How was this patch tested?
Closes#1583 from AngersZhuuuu/CELEBORN-671.
Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Angerszhuuuu <angers.zhu@gmail.com>
### What changes were proposed in this pull request?
CommitHandler will check whether the target worker is in WorkerStatusTracker's excluded list. If so, skip calling commit files on it.
### Why are the changes needed?
Avoid unnecessary commit files to excluded worker.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Manual test.
Closes#1581 from waitinfuture/669.
Lead-authored-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
Co-authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Co-authored-by: Keyong Zhou <zhouky@apache.org>
Signed-off-by: Shuang <lvshuang.tb@gmail.com>
### What changes were proposed in this pull request?
If some task retry or scheduled by speculation, the executors will keep failing because revive does not support old partition is empty.
Celeborn will trigger stage end if all mapper task calls mapper end, this is not what spark thinks a stage ends. So in this moment, kill spark executors will cause the spark task to rerun the current stage. Shuffle client will need to register shuffle first but get empty partition locations, and it will need to revive to get the latest location with empty locations.
Here are logs example
```
23/06/13 08:32:11 ERROR ShuffleClientImpl: Exception raised while reviving for shuffle 12 map 970 attempt 27 partition 152 epoch -1.
java.lang.NullPointerException
at org.apache.celeborn.common.util.PbSerDeUtils$.toPbPartitionLocation(PbSerDeUtils.scala:258)
at org.apache.celeborn.common.protocol.message.ControlMessages$Revive$.apply(ControlMessages.scala:207)
at org.apache.celeborn.client.ShuffleClientImpl.revive(ShuffleClientImpl.java:573)
at org.apache.celeborn.client.ShuffleClientImpl.pushOrMergeData(ShuffleClientImpl.java:656)
at org.apache.celeborn.client.ShuffleClientImpl.pushData(ShuffleClientImpl.java:984)
at org.apache.celeborn.client.write.DataPusher.pushData(DataPusher.java:197)
at org.apache.celeborn.client.write.DataPusher.access$500(DataPusher.java:38)
at org.apache.celeborn.client.write.DataPusher$1.run(DataPusher.java:123)
23/06/13 08:32:11 ERROR Executor: Exception in task 21.27 in stage 54.1 (TID 17255)
org.apache.celeborn.common.exception.CelebornIOException: Revive for shuffle spark-3df9647407f14c39868a17b7950899c5-12 partition 152 failed.
at org.apache.celeborn.client.ShuffleClientImpl.pushOrMergeData(ShuffleClientImpl.java:666)
at org.apache.celeborn.client.ShuffleClientImpl.pushData(ShuffleClientImpl.java:984)
at org.apache.celeborn.client.write.DataPusher.pushData(DataPusher.java:197)
at org.apache.celeborn.client.write.DataPusher.access$500(DataPusher.java:38)
at org.apache.celeborn.client.write.DataPusher$1.run(DataPusher.java:123)
23/06/13 08:32:11 INFO CoarseGrainedExecutorBackend: Got assigned task 17309
```
### Why are the changes needed?
To make shuffle client able to revive with empty locations.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
K8S cluster.
Closes#1586 from FMX/CELEBORN-674.
Authored-by: Ethan Feng <ethanfeng@apache.org>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
Fix wrong method name
### Why are the changes needed?
### Does this PR introduce _any_ user-facing change?
### How was this patch tested?
Closes#1582 from AngersZhuuuu/CELEBORN-547-FOLLOWUP.
Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Angerszhuuuu <angers.zhu@gmail.com>
…eful is disabled
### What changes were proposed in this pull request?
Worker should report WorkerLost instead of WorkerUnavailable in it's shutdown hook if graceful shutdown is disabled.
### Why are the changes needed?
To avoid unnecessary commit file requests from lifecycle manager since it's not graceful shutdown.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Closes#1580 from waitinfuture/668.
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?
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?
Throw exception when raft client request not success.
### Why are the changes needed?
Ratis client may not throw exception when submit request not success. Current only LeaderNotReadyException,NotLeaderException throw out exception this may cause some inconsistent.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
UT
Closes#1556 from RexXiong/CELEBORN-646.
Authored-by: Shuang <lvshuang.tb@gmail.com>
Signed-off-by: Shuang <lvshuang.tb@gmail.com>
### What changes were proposed in this pull request?
Define `protobuf-maven-plugin` in the root pom.xml
### Why are the changes needed?
to fix
```bash
build/mvn protobuf:compile -am -pl common
```
```
[ERROR] No plugin found for prefix 'protobuf' in the current project and in the plugin groups [org.apache.maven.plugins, org.codehaus.mojo] available from the repositories [local (/Users/fchen/.m2/repository), apache.snapshots (https://repository.apache.org/snapshots), central (https://repo.maven.apache.org/maven2)] -> [Help 1]
org.apache.maven.plugin.prefix.NoPluginFoundForPrefixException: No plugin found for prefix 'protobuf' in the current project and in the plugin groups [org.apache.maven.plugins, org.codehaus.mojo] available from the repositories [local (/Users/fchen/.m2/repository), apache.snapshots (https://repository.apache.org/snapshots), central (https://repo.maven.apache.org/maven2)]
at org.apache.maven.plugin.prefix.internal.DefaultPluginPrefixResolver.resolve (DefaultPluginPrefixResolver.java:95)
at org.apache.maven.lifecycle.internal.MojoDescriptorCreator.findPluginForPrefix (MojoDescriptorCreator.java:266)
at org.apache.maven.lifecycle.internal.MojoDescriptorCreator.getMojoDescriptor (MojoDescriptorCreator.java:220)
at org.apache.maven.lifecycle.internal.DefaultLifecycleTaskSegmentCalculator.calculateTaskSegments (DefaultLifecycleTaskSegmentCalculator.java:104)
at org.apache.maven.lifecycle.internal.DefaultLifecycleTaskSegmentCalculator.calculateTaskSegments (DefaultLifecycleTaskSegmentCalculator.java:83)
at org.apache.maven.lifecycle.internal.LifecycleStarter.execute (LifecycleStarter.java:89)
at org.apache.maven.DefaultMaven.doExecute (DefaultMaven.java:298)
at org.apache.maven.DefaultMaven.doExecute (DefaultMaven.java:192)
at org.apache.maven.DefaultMaven.execute (DefaultMaven.java:105)
at org.apache.maven.cli.MavenCli.execute (MavenCli.java:960)
at org.apache.maven.cli.MavenCli.doMain (MavenCli.java:293)
at org.apache.maven.cli.MavenCli.main (MavenCli.java:196)
at sun.reflect.NativeMethodAccessorImpl.invoke0 (Native Method)
at sun.reflect.NativeMethodAccessorImpl.invoke (NativeMethodAccessorImpl.java:62)
at sun.reflect.DelegatingMethodAccessorImpl.invoke (DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke (Method.java:498)
at org.codehaus.plexus.classworlds.launcher.Launcher.launchEnhanced (Launcher.java:282)
at org.codehaus.plexus.classworlds.launcher.Launcher.launch (Launcher.java:225)
at org.codehaus.plexus.classworlds.launcher.Launcher.mainWithExitCode (Launcher.java:406)
at org.codehaus.plexus.classworlds.launcher.Launcher.main (Launcher.java:347)
[ERROR]
[ERROR]
[ERROR] For more information about the errors and possible solutions, please read the following articles:
[ERROR] [Help 1] http://cwiki.apache.org/confluence/display/MAVEN/NoPluginFoundForPrefixException
```
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
tested locally.
Closes#1579 from cfmcgrady/protobuf-plugin.
Authored-by: Fu Chen <cfmcgrady@gmail.com>
Signed-off-by: Cheng Pan <chengpan@apache.org>
### What changes were proposed in this pull request?
Refine RssInputStream logs to indicate the failed location's info to help admin debug
### Why are the changes needed?
### Does this PR introduce _any_ user-facing change?
### How was this patch tested?
Closes#1576 from AngersZhuuuu/CELEBORN-663.
Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Angerszhuuuu <angers.zhu@gmail.com>
### What changes were proposed in this pull request?
1. renamed `RssShuffleWriterSuiteJ` to `CelebornShuffleWriterSuiteBase`, which now serves as an abstract base class.
2. two new classes, `HashBasedShuffleWriterSuiteJ` and `SortBasedShuffleWriterSuiteJ`, have been added. These classes extend `CelebornShuffleWriterSuiteBase` and provide suites for testing hash-based and sort-based shuffle writers.
### Why are the changes needed?
### Does this PR introduce _any_ user-facing change?
### How was this patch tested?
Closes#1570 from cfmcgrady/sort-based-writer-suite.
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?
In this PR, worker always report node unavailable regardless graceful shutdown is turned on or off.
### Why are the changes needed?
To inform master the shutting down worker as soon as possible.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Manual test.
Closes#1575 from waitinfuture/662.
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?
In our prod meet many times of push queue stuck caused by PushState's status was not being removed.
Caused DataPushQueue to keep waiting for taking task.
Although have resolved some bugs, here we'd better add a max wait time for taking tasks since we already have the `PUSH_DATA_TIMEOUT` check method. If the target worker is really stuck, we can retry our task finally.
### Why are the changes needed?
### Does this PR introduce _any_ user-facing change?
### How was this patch tested?
Closes#1552 from AngersZhuuuu/CELEBORN-640.
Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Angerszhuuuu <angers.zhu@gmail.com>
### What changes were proposed in this pull request?
DataPushQueue return task should always remove iterator
Related to
251b923b5bcb19ed1c66
### Why are the changes needed?
### Does this PR introduce _any_ user-facing change?
### How was this patch tested?
Closes#1568 from AngersZhuuuu/CELEBORN-657.
Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Shuang <lvshuang.tb@gmail.com>
### 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>
### 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?
- Replace index-based item access with an iterator for LinkedList.
- Always try to remove a buffer if SendBufferPool does not have a matched candidate, this change makes the total buffer number from `capacity+N-1` to `capacity` in worst cases.
- Some logs and code polish.
### Why are the changes needed?
Improve performance and logs, reduce memory consumption.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Pass GA.
Closes#1560 from pan3793/CELEBORN-648.
Authored-by: Cheng Pan <chengpan@apache.org>
Signed-off-by: Cheng Pan <chengpan@apache.org>
### What changes were proposed in this pull request?
`mapStatusRecords` is required in Spark 2 for constructing `MapStatus` when AQE is enabled, but not in Spark 3, so remove it to save memory and compute resources.
This PR also simplifies the `for loop` code.
### Why are the changes needed?
Remove unnecessary variables to save resources and clean up code.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Pass GA.
Closes#1564 from pan3793/CELEBORN-654.
Authored-by: Cheng Pan <chengpan@apache.org>
Signed-off-by: Cheng Pan <chengpan@apache.org>
### What changes were proposed in this pull request?
Rename variable `newAppId` to `appUniqueId` in Spark client.
### Why are the changes needed?
Make the variable name intuitive.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Pass GA.
Closes#1565 from pan3793/CELEBORN-655.
Authored-by: Cheng Pan <chengpan@apache.org>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
Fix potential NPE when remove push status
### Why are the changes needed?
### Does this PR introduce _any_ user-facing change?
### How was this patch tested?
Closes#1559 from AngersZhuuuu/CELEBORN-647.
Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Angerszhuuuu <angers.zhu@gmail.com>
### What changes were proposed in this pull request?
This PR aims to improve `build/make-distribution.sh` by
- skip building javadoc and source artifacts
- skip building unnecessary modules
- allow skipping client modules
### Why are the changes needed?
Speed up the packaging process.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Tested with
```
build/make-distribution.sh
```
```
build/make-distribution.sh -Pspark-3.3
```
```
build/make-distribution.sh -Pflink-1.17
```
Closes#1561 from pan3793/CELEBORN-649.
Authored-by: Cheng Pan <chengpan@apache.org>
Signed-off-by: Cheng Pan <chengpan@apache.org>
…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>
### What changes were proposed in this pull request?
Remove support for `rss.*` configuration alias
### Why are the changes needed?
The legacy `rss.*` configuration alias was added during Celeborn entering Apache Incubator, to simplify users' migration from RSS to Celeborn.
Lots of configuration changes happened after Celeborn 0.2, the `rss.*` configuration alias become less helpful, so remove it to clean up the code.
### Does this PR introduce _any_ user-facing change?
Yes, but it's expected, the `rss.*` compatibility has never been documented.
### How was this patch tested?
Pass GA.
Closes#1547 from pan3793/CELEBORN-637.
Authored-by: Cheng Pan <chengpan@apache.org>
Signed-off-by: Cheng Pan <chengpan@apache.org>
### What changes were proposed in this pull request?
Use scala 2.12.15 as default scala version for flink.
### Why are the changes needed?
There is incompatible serialize problem between scala 2.12.7 to scala 2.12.15/scala 2.11.12, when use different scala version, the generated serialVersionUID is different, Then we may encounter deserialize problem between client/server rpc, refer [scala ](https://users.scala-lang.org/t/serialversionuid-change-between-scala-2-12-6-and-2-12-7/3478/3)

### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Manual test use Flink scala2.12.7 runtime with Celeborn scala 2.12.15 compiled Flink client
Closes#1553 from RexXiong/CELEBORN-641.
Authored-by: Shuang <lvshuang.tb@gmail.com>
Signed-off-by: Ethan Feng <ethanfeng@apache.org>
### What changes were proposed in this pull request?
Refine the logic here to make it easier understand.
### Why are the changes needed?
### Does this PR introduce _any_ user-facing change?
### How was this patch tested?
Closes#1555 from AngersZhuuuu/CELEBORN-645.
Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Angerszhuuuu <angers.zhu@gmail.com>
### What changes were proposed in this pull request?
If we meet some unexpected exception, `getPushDataFailCause ` will throw NPE and broke the process of revive and remove push states. Here we should handle the NPE
### Why are the changes needed?
### Does this PR introduce _any_ user-facing change?
### How was this patch tested?
Closes#1551 from AngersZhuuuu/CELEBORN-639.
Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Angerszhuuuu <angers.zhu@gmail.com>
### What changes were proposed in this pull request?
`SimpleDateFormat` is not thread-safe, replace it with a thread-safe `FastDateFormat`
### Why are the changes needed?
`FastDateFormat` is a fast and thread-safe version of `java.text.SimpleDateFormat`.
### Does this PR introduce _any_ user-facing change?
Yes, it's a bug fix.
### How was this patch tested?
Manually review.
Closes#1545 from pan3793/CELEBORN-636.
Authored-by: Cheng Pan <chengpan@apache.org>
Signed-off-by: Ethan Feng <ethanfeng@apache.org>
### What changes were proposed in this pull request?
Adapt Spark DRA patch for spark 3.4
### Why are the changes needed?
To support enabling DRA w/ Celeborn on Spark 3.4
### Does this PR introduce _any_ user-facing change?
Yes, this PR provides a DRA patch for Spark 3.4
### How was this patch tested?
Compiled with Spark 3.4
Closes#1546 from FMX/CELEBORN-619.
Lead-authored-by: Ethan Feng <ethanfeng@apache.org>
Co-authored-by: Ethan Feng <fengmingxiao.fmx@alibaba-inc.com>
Signed-off-by: Ethan Feng <ethanfeng@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>
### 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>
### What changes were proposed in this pull request?
Fix columnar shuffle codegen exception. This is a refactoring of #1523。
Closes#1543 from kerwin-zk/issue-620.
Authored-by: xiyu.zk <xiyu.zk@alibaba-inc.com>
Signed-off-by: xiyu.zk <xiyu.zk@alibaba-inc.com>
### What changes were proposed in this pull request?
Add doc about enabling rac-awareness
### Why are the changes needed?
Document new features.
### Does this PR introduce _any_ user-facing change?
Yes, the docs are updated.
### How was this patch tested?
<img width="1085" alt="截屏2023-06-02 下午3 19 10" src="https://github.com/apache/incubator-celeborn/assets/46485123/c8c51a4c-40be-40ea-befd-3c369b9f7600">
Closes#1536 from AngersZhuuuu/CELEBORN-629.
Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Cheng Pan <chengpan@apache.org>
### What changes were proposed in this pull request?
Adapt Spark DRA patch for spark 3.4
### Why are the changes needed?
To support enabling DRA w/ Celeborn on Spark 3.4
### Does this PR introduce _any_ user-facing change?
Yes, this PR provides a DRA patch for Spark 3.4
### How was this patch tested?
Compiled with Spark 3.4
Closes#1529 from FMX/CELEBORN-619.
Authored-by: Ethan Feng <ethanfeng@apache.org>
Signed-off-by: Ethan Feng <ethanfeng@apache.org>
### What changes were proposed in this pull request?
Refine this doc since:
1. It didn't mention our cluster default RPC type is `NETTY`
2. If the user use the ratis shell with `GRPC` but didn't know the ratis cluster is `NETTY`, the error is not clear and hard to debug.
### Why are the changes needed?
### Does this PR introduce _any_ user-facing change?
### How was this patch tested?
Closes#1542 from AngersZhuuuu/CELEBORN-623-FOLLOWUP.
Lead-authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Co-authored-by: Cheng Pan <pan3793@gmail.com>
Signed-off-by: Cheng Pan <chengpan@apache.org>
### What changes were proposed in this pull request?
Ratis-shell use GRPC by default. Celeborn support Netty for ratis, if `raft.rpc.type` is not specified, commands may fail.
e.g.
```
org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException: DEADLINE_EXCEEDED: deadline exceeded after 14.947369960s. [closed=[], open=[[buffered_nanos=14962358255, waiting_for_connection]]]
```
So I think we should update the document to mention how to change the RPC type to in `celeborn-ratis`.
### Why are the changes needed?
Improve user experience
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Manually test
Closes#1530 from onebox-li/ratis-shell-default-rpc.
Lead-authored-by: liyihe <liyihe@bigo.sg>
Co-authored-by: Cheng Pan <pan3793@gmail.com>
Signed-off-by: Cheng Pan <chengpan@apache.org>