leixm opened a new issue, #926:
URL: https://github.com/apache/incubator-uniffle/issues/926

   ### Code of Conduct
   
   - [X] I agree to follow this project's [Code of 
Conduct](https://www.apache.org/foundation/policies/conduct)
   
   
   ### Search before asking
   
   - [X] I have searched in the 
[issues](https://github.com/apache/incubator-uniffle/issues?q=is%3Aissue) and 
found no similar issues.
   
   
   ### Describe the bug
   
   The following exception occurred in the four attempts of the same task,the 
probability of this kind of problem recurring is relatively low. About 
thousands of apps will have this type of exception in one app, causing the app 
attempt to retry, and every time this exception occurs, the app is also 
different.
   ```
   org.apache.spark.SparkException: Task failed while writing rows.
        at 
org.apache.spark.sql.execution.datasources.FileFormatWriter$.org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask(FileFormatWriter.scala:321)
        at 
org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$write$1.apply(FileFormatWriter.scala:209)
        at 
org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$write$1.apply(FileFormatWriter.scala:208)
        at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
        at org.apache.spark.scheduler.Task.run(Task.scala:121)
        at 
org.apache.spark.executor.Executor$TaskRunner$$anonfun$10.apply(Executor.scala:408)
        at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1363)
        at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:414)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
        at java.lang.Thread.run(Thread.java:745)
   Caused by: java.lang.StackOverflowError
        at io.grpc.internal.RetriableStream.cancel(RetriableStream.java:493)
        at 
io.grpc.internal.ClientCallImpl.sendMessageInternal(ClientCallImpl.java:528)
        at io.grpc.internal.ClientCallImpl.sendMessage(ClientCallImpl.java:506)
        at io.grpc.stub.ClientCalls.asyncUnaryRequestCall(ClientCalls.java:317)
        at io.grpc.stub.ClientCalls.futureUnaryCall(ClientCalls.java:227)
        at io.grpc.stub.ClientCalls.blockingUnaryCall(ClientCalls.java:154)
        at 
org.apache.uniffle.proto.ShuffleServerGrpc$ShuffleServerBlockingStub.getMemoryShuffleData(ShuffleServerGrpc.java:768)
        at 
org.apache.uniffle.client.impl.grpc.ShuffleServerGrpcClient.getInMemoryShuffleData(ShuffleServerGrpcClient.java:627)
        at 
org.apache.uniffle.storage.handler.impl.MemoryClientReadHandler.readShuffleData(MemoryClientReadHandler.java:94)
        at 
org.apache.uniffle.storage.handler.impl.ComposedClientReadHandler.readShuffleData(ComposedClientReadHandler.java:93)
        at 
org.apache.uniffle.client.impl.ShuffleReadClientImpl.read(ShuffleReadClientImpl.java:224)
        at 
org.apache.uniffle.client.impl.ShuffleReadClientImpl.readShuffleBlockData(ShuffleReadClientImpl.java:160)
        at 
org.apache.uniffle.client.impl.ShuffleReadClientImpl.readShuffleBlockData(ShuffleReadClientImpl.java:214)
        at 
org.apache.uniffle.client.impl.ShuffleReadClientImpl.readShuffleBlockData(ShuffleReadClientImpl.java:214)
        at 
org.apache.uniffle.client.impl.ShuffleReadClientImpl.readShuffleBlockData(ShuffleReadClientImpl.java:214)
        at 
org.apache.uniffle.client.impl.ShuffleReadClientImpl.readShuffleBlockData(ShuffleReadClientImpl.java:214)
        at 
org.apache.uniffle.client.impl.ShuffleReadClientImpl.readShuffleBlockData(ShuffleReadClientImpl.java:214)
        at 
org.apache.uniffle.client.impl.ShuffleReadClientImpl.readShuffleBlockData(ShuffleReadClientImpl.java:214)
        at 
org.apache.uniffle.client.impl.ShuffleReadClientImpl.readShuffleBlockData(ShuffleReadClientImpl.java:214)
        at 
org.apache.uniffle.client.impl.ShuffleReadClientImpl.readShuffleBlockData(ShuffleReadClientImpl.java:214)
        at 
org.apache.uniffle.client.impl.ShuffleReadClientImpl.readShuffleBlockData(ShuffleReadClientImpl.java:214)
        at 
org.apache.uniffle.client.impl.ShuffleReadClientImpl.readShuffleBlockData(ShuffleReadClientImpl.java:214)
        at 
org.apache.uniffle.client.impl.ShuffleReadClientImpl.readShuffleBlockData(ShuffleReadClientImpl.java:214)
        at 
org.apache.uniffle.client.impl.ShuffleReadClientImpl.readShuffleBlockData(ShuffleReadClientImpl.java:214)
        at 
org.apache.uniffle.client.impl.ShuffleReadClientImpl.readShuffleBlockData(ShuffleReadClientImpl.java:214)```
   
   ### Affects Version(s)
   
   0.6.0
   
   ### Uniffle Server Log Output
   
   ```logtalk
   Corresponding ShuffleServer has been repeatedly printing the following log
   `[INFO] 2023-06-04 20:42:40,023 Grpc-839 ShuffleServerGrpcService 
getMemoryShuffleData - Successfully getInMemoryShuffleData cost 0 ms with 
318133 bytes shuffle data for appId[xxxxx], shuffleId[0], partitionId[12]`
   ```
   
   
   ### Uniffle Engine Log Output
   
   ```logtalk
   Before the exception is thrown, the following logs have been repeatedly 
appearing, indicating that the client has been requesting the 
GetInMemoryShuffleData interface of ShuffleServer.
   `23/06/04 20:42:43 INFO grpc.ShuffleServerGrpcClient: GetInMemoryShuffleData 
from xxxxx:19999 for appId[xxxxx], shuffleId[0], partitionId[12] cost 2 ms`
   ```
   
   
   ### Uniffle Server Configurations
   
   ```yaml
   rss.rpc.server.port 19999
   rss.jetty.http.port 19998
   rss.storage.basePath /aa/bb/cc,/aa1/bb/cc
   rss.storage.type MEMORY_LOCALFILE_HDFS
   rss.coordinator.quorum xxxxxxx
   rss.server.buffer.capacity 40gb
   rss.server.read.buffer.capacity 20gb
   rss.server.flush.thread.alive 50
   rss.server.flush.threadPool.size 4
   rss.server.disk.capacity 1t
   rss.server.heartbeat.timeout 60000
   rss.server.heartbeat.interval 10000
   rss.rpc.message.max.size 1073741824
   rss.server.preAllocation.expired 120000
   rss.server.commit.timeout 600000
   rss.server.app.expired.withoutHeartbeat 300000
   rss.server.buffer.flush.enabled true
   rss.server.tags high_priority
   rss.server.health.check.enable true
   rss.server.multistorage.fallback.strategy.class 
org.apache.uniffle.server.storage.LocalStorageManagerFallbackStrategy
   rss.server.flush.cold.storage.threshold.size 50gb
   rss.server.buffer.flush.threshold 64m
   rss.server.max.concurrency.of.single.partition.writer 20
   rss.server.huge-partition.size.threshold 50g
   rss.server.huge-partition.memory.limit.ratio 0.2
   ```
   ```
   
   
   ### Uniffle Engine Configurations
   
   ```yaml
   --conf spark.rss.client.shuffle.data.distribution.type LOCAL_ORDER
   ```
   
   
   ### Additional context
   
   _No response_
   
   ### Are you willing to submit PR?
   
   - [X] Yes I am willing to submit a PR!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to