[ 
https://issues.apache.org/jira/browse/RATIS-1578?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17538628#comment-17538628
 ] 

mango commented on RATIS-1578:
------------------------------

[~szetszwo]  Hi, we hava the problem during testing.

I re-tested the environment:

     Ratis version: ratis master branch
     Ozone version: HDDS-4454 branch

Using ozone for a long time to write tests, I found this problem and here is 
the error stack:
{code:java}
2022-05-18 14:19:41,011 
[8df14bca-0194-4bd9-9001-7bd34845aaaa-DataStreamManagement-request--thread23] 
WARN org.apache.ratis.netty.server.DataStreamManagement: Failed to process 
DataStreamRequestByteBuf:cl
ientId=client-D242BCE5C75D,type=STREAM_DATA,id=7,offset=0,length=392
java.util.concurrent.CompletionException: java.lang.IllegalStateException: 
org.apache.ratis.util.Preconditions$$Lambda$164/609879567@195f7afa
        at 
java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:273)
        at 
java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:280)
        at 
java.util.concurrent.CompletableFuture.biApply(CompletableFuture.java:1107)
        at 
java.util.concurrent.CompletableFuture$BiApply.tryFire(CompletableFuture.java:1070)
        at 
java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:442)
        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: java.lang.IllegalStateException: 
org.apache.ratis.util.Preconditions$$Lambda$164/609879567@195f7afa
        at org.apache.ratis.util.Preconditions.assertTrue(Preconditions.java:45)
        at 
org.apache.ratis.util.Preconditions.assertNotNull(Preconditions.java:91)
        at 
org.apache.ratis.util.Preconditions.assertNotNull(Preconditions.java:96)
        at 
org.apache.ratis.protocol.RaftClientMessage.<init>(RaftClientMessage.java:31)
        at 
org.apache.ratis.protocol.RaftClientRequest.<init>(RaftClientRequest.java:344)
        at 
org.apache.ratis.protocol.RaftClientRequest.<init>(RaftClientRequest.java:31)
        at 
org.apache.ratis.protocol.RaftClientRequest$Builder.build(RaftClientRequest.java:244)
        at 
org.apache.ratis.client.impl.RaftClientImpl.newRaftClientRequest(RaftClientImpl.java:241)
        at 
org.apache.ratis.client.impl.OrderedAsync.lambda$null$1(OrderedAsync.java:169)
        at 
org.apache.ratis.client.impl.OrderedAsync$PendingOrderedRequest.lambda$newRequestImpl$0(OrderedAsync.java:76)
        at java.util.Optional.map(Optional.java:215)
        at 
org.apache.ratis.client.impl.OrderedAsync$PendingOrderedRequest.newRequestImpl(OrderedAsync.java:76)
        at 
org.apache.ratis.client.impl.OrderedAsync.sendRequestWithRetry(OrderedAsync.java:187)
        at 
org.apache.ratis.util.SlidingWindow$Client.sendOrDelayRequest(SlidingWindow.java:280)
        at 
org.apache.ratis.util.SlidingWindow$Client.submitNewRequest(SlidingWindow.java:259)
        at org.apache.ratis.client.impl.OrderedAsync.send(OrderedAsync.java:170)
        at 
org.apache.ratis.client.impl.OrderedAsync.newInstance(OrderedAsync.java:118)
        at 
org.apache.ratis.client.impl.RaftClientImpl.lambda$new$0(RaftClientImpl.java:169)
        at org.apache.ratis.util.MemoizedSupplier.get(MemoizedSupplier.java:62)
        at 
org.apache.ratis.client.impl.RaftClientImpl.getOrderedAsync(RaftClientImpl.java:223)
        at org.apache.ratis.client.impl.AsyncImpl.send(AsyncImpl.java:41)
        at org.apache.ratis.client.impl.AsyncImpl.sendForward(AsyncImpl.java:67)
        at 
org.apache.ratis.netty.server.DataStreamManagement.startTransaction(DataStreamManagement.java:361)
        at 
org.apache.ratis.netty.server.DataStreamManagement.lambda$null$11(DataStreamManagement.java:460)
        at 
java.util.concurrent.CompletableFuture.biApply(CompletableFuture.java:1105)
        ... 5 more{code}
I observed that before this error occurred, pipeline conducted an internal 
re-election, the following is the re-election log:
{code:java}
2022-05-18 14:12:02,035 [grpc-default-executor-0] WARN 
org.apache.ratis.grpc.server.GrpcLogAppender: 
8df14bca-0194-4bd9-9001-7bd34845aaaa@group-B0435BCFE4F7->fcdcd8d3-8dec-4663-a334-30fedee963c9-AppendLog
ResponseHandler: Failed appendEntries: 
org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException: UNAVAILABLE: io 
exception
2022-05-18 14:12:02,037 [grpc-default-executor-0] WARN 
org.apache.ratis.grpc.server.GrpcLogAppender: 
8df14bca-0194-4bd9-9001-7bd34845aaaa@group-B0435BCFE4F7->fcdcd8d3-8dec-4663-a334-30fedee963c9-GrpcLogAp
pender: Leader has not got in touch with Follower 
8df14bca-0194-4bd9-9001-7bd34845aaaa@group-B0435BCFE4F7->fcdcd8d3-8dec-4663-a334-30fedee963c9(c-1,m0,n299690,
 attendVote=true, lastRpcSendTime=1, lastRpcR
esponseTime=24250) yet, just keep nextIndex unchanged and retry.{code}

> The following assertion error occurs when transferring large files
> ------------------------------------------------------------------
>
>                 Key: RATIS-1578
>                 URL: https://issues.apache.org/jira/browse/RATIS-1578
>             Project: Ratis
>          Issue Type: Sub-task
>            Reporter: GuoHao
>            Assignee: Tsz-wo Sze
>            Priority: Major
>
>  
> The following assertion error occurs when transferring large files.
>  
> Please look at length, the packet size I actually transmitted is 1MB, and the 
> size here is Length =1460. I suspect it is caused by sticky packet. Could you 
> help me solve this problem?
>  
> {code:java}
> 2022-05-09 22:03:14,050 
> [97c45065-e803-4377-9a2f-db1e53410b77-NettyServerStreamRpc-channelRead--thread9]
>  WARN org.apache.ratis.netty.server.DataStreamManagement: Failed to process 
> DataStreamRequestByteBuf:clientId=client-B557FB683D6A,type=STREAM_DATA,id=45,offset=667799,length=1460
> java.util.concurrent.ExecutionException: java.lang.IllegalStateException
>         at 
> java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:357)
>         at 
> java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1915)
>         at 
> org.apache.ratis.netty.server.DataStreamManagement.read(DataStreamManagement.java:419)
>         at 
> org.apache.ratis.netty.server.NettyServerStreamRpc$1.lambda$channelRead$0(NettyServerStreamRpc.java:230)
>         at 
> org.apache.ratis.thirdparty.io.netty.util.concurrent.PromiseTask.runTask(PromiseTask.java:98)
>         at 
> org.apache.ratis.thirdparty.io.netty.util.concurrent.PromiseTask.run(PromiseTask.java:106)
>         at 
> org.apache.ratis.thirdparty.io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:164)
>         at 
> org.apache.ratis.thirdparty.io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:472)
>         at 
> org.apache.ratis.thirdparty.io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:384)
>         at 
> org.apache.ratis.thirdparty.io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
>         at 
> org.apache.ratis.thirdparty.io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
>         at java.lang.Thread.run(Thread.java:748)
> Caused by: java.lang.IllegalStateException
>         at 
> org.apache.ratis.util.Preconditions.assertTrue(Preconditions.java:33)
>         at 
> org.apache.ratis.netty.server.DataStreamManagement.assertReplyCorrespondingToRequest(DataStreamManagement.java:499)
>         at 
> org.apache.ratis.netty.server.DataStreamManagement.checkSuccessRemoteWrite(DataStreamManagement.java:506)
>         at 
> org.apache.ratis.netty.server.DataStreamManagement.sendReply(DataStreamManagement.java:352)
>         at 
> org.apache.ratis.netty.server.DataStreamManagement.lambda$null$10(DataStreamManagement.java:470)
>         at 
> java.util.concurrent.CompletableFuture.biApply(CompletableFuture.java:1105)
>         at 
> java.util.concurrent.CompletableFuture$BiApply.tryFire(CompletableFuture.java:1070)
>         at 
> java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:442)
>         at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>         at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>         ... 1 more {code}



--
This message was sent by Atlassian Jira
(v8.20.7#820007)

Reply via email to