[ 
https://issues.apache.org/jira/browse/IGNITE-21394?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Denis Chudov updated IGNITE-21394:
----------------------------------
    Description: 
*Motivation*

The handler of pending assignments event change ( 
TableManager#handleChangePendingAssignmentEvent() ) tries to do changePeerAsync 
after starting the partition and client. In order to know whether the calling 
of changePeerAsync is needed, it tries to get the current leader of 
corresponding raft group. This call of 
RaftGroupService#refreshAndGetLeaderWithTerm can fail with TimeoutException. 
For example, there is no known leader on the node that the GetLeader request is 
sent to, or that node is no more in the raft group, etc., and in the same time 
that node is the only known peer of the raft group: in these cases the 
GetLeader request will be constantly retried in hope to get a response with 
leader finally, when it's elected, but this can never happen. So, the 
TimeoutException is expected in this case.

This exception should be handled within the mentioned listener of pending 
assignments event change. otherwise it fails the watch processor, making it 
unable to handle the further meta storage updates (and making the node 
inoperable). This means that, most likely, the current node is not a leader of 
the raft group, and changePeers shouln't be done, or it has not caught up with 
the current assignments events, this means that some client requests for this 
node for this partition will fail, but the node will remain operable.

*Definition of done*

TimeoutException in the listener of pending assignments change doesn't fail the 
watch processor and doesn't lead to multiple exceptions like this:
{code:java}
[2024-01-29T22:00:58,658][ERROR][%isckvt_tmccd_3344%Raft-Group-Client-5][WatchProcessor]
 Error occurred when notifying safe time advanced callback
 java.util.concurrent.CompletionException: java.util.concurrent.TimeoutException
    at 
java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:331)
 ~[?:?]
    at 
java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:346)
 ~[?:?]
    at 
java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:632)
 ~[?:?]
    at 
java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:506) 
~[?:?]
    at 
java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:2088)
 ~[?:?]
    at 
org.apache.ignite.internal.raft.RaftGroupServiceImpl.sendWithRetry(RaftGroupServiceImpl.java:546)
 ~[ignite-raft-3.0.0-SNAPSHOT.jar:?]
    at 
org.apache.ignite.internal.raft.RaftGroupServiceImpl.lambda$handleErrorResponse$42(RaftGroupServiceImpl.java:635)
 ~[ignite-raft-3.0.0-SNAPSHOT.jar:?]
    at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515) 
[?:?]
    at java.util.concurrent.FutureTask.run(FutureTask.java:264) [?:?]
    at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:304)
 [?:?]
    at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) 
[?:?]
    at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) 
[?:?]
    at java.lang.Thread.run(Thread.java:834) [?:?]
Caused by: java.util.concurrent.TimeoutException
    ... 8 more{code}
Implementation notes

 

It can be reproduced in integration tests like 
ItSchemaChangeKvViewTest#testMergeChangesColumnDefault when there are 3 nodes 
starting, then a table with 25 partitions/1 replica created. During the table 
start the rebalance is possible, like this:
 * a replication group is moved from node A to node B
 * some node C tries to perform GetLeader, and has only node A in local peers
 * node A thinks it is the only member of the replication group, and is not 
leader, sends "Unknown leader" response to C
 * node C constatnly retries the request to node A.

  was:
*Motivation*

The handler of pending assignments event change ( 
TableManager#handleChangePendingAssignmentEvent() ) tries to do changePeerAsync 
after starting the partition and client. In order to know whether the calling 
of changePeerAsync is needed, it tries to get the current leader of 
corresponding raft group. This call of 
RaftGroupService#refreshAndGetLeaderWithTerm can fail with TimeoutException. 
For example, there is no known leader on the node that the GetLeader request is 
sent to, or that node is no more in the raft group, etc., and in the same time 
that node is the only known peer of the raft group: in these cases the 
GetLeader request will be constantly retried in hope to get a response with 
leader finally, when it's elected, but this can never happen. So, the 
TimeoutException is expected in this case.

This exception should be handled within the mentioned listener of pending 
assignments event change. otherwise it fails the watch processor, making it 
unable to handle the further meta storage updates (and making the node 
inoperable). This means that, most likely, the current node is not a leader of 
the raft group, and changePeers shouln't be done, or it has not caught up with 
the current assignments events, this means that some client requests for this 
node for this partition will fail, but the node will remain operable.

*Definition of done*

TimeoutException in the listener of pending assignments change doesn't fail the 
watch processor and doesn't lead to multiple exceptions like this:

 
{code:java}
024-01-29T22:00:58,658][ERROR][%isckvt_tmccd_3344%Raft-Group-Client-5][WatchProcessor]
 Error occurred when notifying safe time advanced callback 
java.util.concurrent.CompletionException: java.util.concurrent.TimeoutException 
 at 
java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:331)
 ~[?:?]  at 
java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:346)
 ~[?:?]  at 
java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:632)
 ~[?:?]  at 
java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:506) 
~[?:?]  at 
java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:2088)
 ~[?:?]  at 
org.apache.ignite.internal.raft.RaftGroupServiceImpl.sendWithRetry(RaftGroupServiceImpl.java:546)
 ~[ignite-raft-3.0.0-SNAPSHOT.jar:?]  at 
org.apache.ignite.internal.raft.RaftGroupServiceImpl.lambda$handleErrorResponse$42(RaftGroupServiceImpl.java:635)
 ~[ignite-raft-3.0.0-SNAPSHOT.jar:?]  at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515) [?:?]  
at java.util.concurrent.FutureTask.run(FutureTask.java:264) [?:?]  at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:304)
 [?:?]  at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) 
[?:?]  at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) 
[?:?]  at java.lang.Thread.run(Thread.java:834) [?:?]Caused by: 
java.util.concurrent.TimeoutException  ... 8 more {code}
Implementation notes

 

It can be reproduced in integration tests like 
ItSchemaChangeKvViewTest#testMergeChangesColumnDefault when there are 3 nodes 
starting, then a table with 25 partitions/1 replica created. During the table 
start the rebalance is possible, like this:
 * a replication group is moved from node A to node B
 * some node C tries to perform GetLeader, and has only node A in local peers
 * node A thinks it is the only member of the replication group, and is not 
leader, sends "Unknown leader" response to C
 * node C constatnly retries the request to node A.


> TimeoutException in the listener of pending assignments change shouldn't fail 
> the watch processor
> -------------------------------------------------------------------------------------------------
>
>                 Key: IGNITE-21394
>                 URL: https://issues.apache.org/jira/browse/IGNITE-21394
>             Project: Ignite
>          Issue Type: Bug
>            Reporter: Denis Chudov
>            Priority: Major
>              Labels: ignite-3
>
> *Motivation*
> The handler of pending assignments event change ( 
> TableManager#handleChangePendingAssignmentEvent() ) tries to do 
> changePeerAsync after starting the partition and client. In order to know 
> whether the calling of changePeerAsync is needed, it tries to get the current 
> leader of corresponding raft group. This call of 
> RaftGroupService#refreshAndGetLeaderWithTerm can fail with TimeoutException. 
> For example, there is no known leader on the node that the GetLeader request 
> is sent to, or that node is no more in the raft group, etc., and in the same 
> time that node is the only known peer of the raft group: in these cases the 
> GetLeader request will be constantly retried in hope to get a response with 
> leader finally, when it's elected, but this can never happen. So, the 
> TimeoutException is expected in this case.
> This exception should be handled within the mentioned listener of pending 
> assignments event change. otherwise it fails the watch processor, making it 
> unable to handle the further meta storage updates (and making the node 
> inoperable). This means that, most likely, the current node is not a leader 
> of the raft group, and changePeers shouln't be done, or it has not caught up 
> with the current assignments events, this means that some client requests for 
> this node for this partition will fail, but the node will remain operable.
> *Definition of done*
> TimeoutException in the listener of pending assignments change doesn't fail 
> the watch processor and doesn't lead to multiple exceptions like this:
> {code:java}
> [2024-01-29T22:00:58,658][ERROR][%isckvt_tmccd_3344%Raft-Group-Client-5][WatchProcessor]
>  Error occurred when notifying safe time advanced callback
>  java.util.concurrent.CompletionException: 
> java.util.concurrent.TimeoutException
>     at 
> java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:331)
>  ~[?:?]
>     at 
> java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:346)
>  ~[?:?]
>     at 
> java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:632)
>  ~[?:?]
>     at 
> java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:506)
>  ~[?:?]
>     at 
> java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:2088)
>  ~[?:?]
>     at 
> org.apache.ignite.internal.raft.RaftGroupServiceImpl.sendWithRetry(RaftGroupServiceImpl.java:546)
>  ~[ignite-raft-3.0.0-SNAPSHOT.jar:?]
>     at 
> org.apache.ignite.internal.raft.RaftGroupServiceImpl.lambda$handleErrorResponse$42(RaftGroupServiceImpl.java:635)
>  ~[ignite-raft-3.0.0-SNAPSHOT.jar:?]
>     at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515) [?:?]
>     at java.util.concurrent.FutureTask.run(FutureTask.java:264) [?:?]
>     at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:304)
>  [?:?]
>     at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
>  [?:?]
>     at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
>  [?:?]
>     at java.lang.Thread.run(Thread.java:834) [?:?]
> Caused by: java.util.concurrent.TimeoutException
>     ... 8 more{code}
> Implementation notes
>  
> It can be reproduced in integration tests like 
> ItSchemaChangeKvViewTest#testMergeChangesColumnDefault when there are 3 nodes 
> starting, then a table with 25 partitions/1 replica created. During the table 
> start the rebalance is possible, like this:
>  * a replication group is moved from node A to node B
>  * some node C tries to perform GetLeader, and has only node A in local peers
>  * node A thinks it is the only member of the replication group, and is not 
> leader, sends "Unknown leader" response to C
>  * node C constatnly retries the request to node A.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to