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

Mikhail Efremov updated IGNITE-22315:
-------------------------------------
    Description: 
*Description*

The goal is to make sure that when partition starting on an ignite-node, if the 
node in assignments, then there should be only ones raft-node, raft-client and 
replica and no any started entities otherwise.

*Motivation*

Now on the one hand in {{TableManager}} there is a path when raft-node and 
replica didn't start, but raft-client is:

In {{TableManager#startPartitionAndStartClient}} there is a place where 
{{startGroupFut}} returns {{{}false{}}}:\{{ localMemberAssignment}} is {{null}} 
or {{PartitionReplicatorNodeRecovery#shouldStartGroup}} returns {{{}false{}}}. 
Then, raft-node and replica will not be started, but regardless of 
{{startGroupFut}} result raft client will be started:
{code:java}
startGroupFut
        .thenComposeAsync(v -> inBusyLock(busyLock, () -> {
            TableRaftService tableRaftService = 
table.internalTable().tableRaftService();

            try {
                // Return existing service if it's already started.
                return completedFuture(
                        (TopologyAwareRaftGroupService) 
tableRaftService.partitionRaftGroupService(replicaGrpId.partitionId())
                );
            } catch (IgniteInternalException e) {
                // We use "IgniteInternalException" in accordance with the 
javadoc of "partitionRaftGroupService" method.
                try {
                    // TODO IGNITE-19614 This procedure takes 10 seconds if 
there's no majority online.
                    return raftMgr
                            .startRaftGroupService(replicaGrpId, 
newConfiguration, raftGroupServiceFactory, raftCommandsMarshaller);
                } catch (NodeStoppingException ex) {
                    return failedFuture(ex);
                }
            }
        }), ioExecutor)
        .thenAcceptAsync(updatedRaftGroupService -> inBusyLock(busyLock, () -> {
            ((InternalTableImpl) internalTbl).tableRaftService()
                    .updateInternalTableRaftGroupService(partId, 
updatedRaftGroupService);

            boolean startedRaftNode = startGroupFut.join();
            if (localMemberAssignment == null || !startedRaftNode || 
replicaMgr.isReplicaStarted(replicaGrpId)) {
                return;
            } {code}
the code shows that {{v}} argument ({{{}startGroupFut{}}}'s result) in the 
lambda doesn't affect on raft-client's starting, and also the client is put 
into {{TableRaftService}} then regardless of replica's starting.

On the other hand, there is a place that rely on raft-client creation on every 
node with table's ID in {{TableManager#tables}} map: inside 
{{TableManager#handleChangePendingAssignmentEvent}} there are two calls:
 # the same name method 
{{{}TableManager#handleChangePendingAssignmentEvent{}}}{{{{}}{}}}
 # {{TableManager#changePeersOnRebalance}}

Both are asking for internal table's 
{{{}tableRaftService().partitionRaftGroupService(partId){}}}, but there no any 
checks about is the local node is suitable for raft-entities and replica 
starting or they are already started.
 
Then, when we would try to fix instant raft-client starting it will lead to 
instant {{IgniteInternalException}} with _"No such partition P in table T"_ 
from {{{}TableRaftService#{}}}{{{}partitionRaftGroupService{}}}. This is a 
problem that should be solved.

{color:#383838}*Definition of done*{color}
{color:#383838}1. Raft-client must be started only together with raft-node and 
replica or shouldn't be started otherwise.{color}
{color:#383838}2. {{TableRaftService}} shouldn't be requested for raft-client 
if the local node isn't in a raft group.{color}
{color:#383838}3. Some tests may rely on described behavior, then failed tests 
should be investigated and, probably fixed.{color}
{color:#383838}4. New tests that check single starting of raft-node, 
raft-client and replica together are required{color}
{color:#0f54d6} {color}

{color:#383838} {color}

{color:#00855f} {color}

  was:
*Description*

The goal is to make sure that when partition starting on an ignite-node, if the 
node in assignments, then there should be only ones raft-node, raft-client and 
replica and no any started entities otherwise.

*Motivation*

Now on the one hand in {{TableManager}} there is a path when raft-node and 
replica didn't start, but raft-client is:

In {{TableManager#startPartitionAndStartClient}} there is a place where 
{{startGroupFut}} returns {{{}false{}}}:{{ localMemberAssignment}} is {{null}} 
or {{PartitionReplicatorNodeRecovery#shouldStartGroup}} returns {{{}false{}}}. 
Then, raft-node and replica will not be started, but regardless of 
{{startGroupFut}} result raft client will be started:
{code:java}
startGroupFut
        .thenComposeAsync(v -> inBusyLock(busyLock, () -> {
            TableRaftService tableRaftService = 
table.internalTable().tableRaftService();

            try {
                // Return existing service if it's already started.
                return completedFuture(
                        (TopologyAwareRaftGroupService) 
tableRaftService.partitionRaftGroupService(replicaGrpId.partitionId())
                );
            } catch (IgniteInternalException e) {
                // We use "IgniteInternalException" in accordance with the 
javadoc of "partitionRaftGroupService" method.
                try {
                    // TODO IGNITE-19614 This procedure takes 10 seconds if 
there's no majority online.
                    return raftMgr
                            .startRaftGroupService(replicaGrpId, 
newConfiguration, raftGroupServiceFactory, raftCommandsMarshaller);
                } catch (NodeStoppingException ex) {
                    return failedFuture(ex);
                }
            }
        }), ioExecutor)
        .thenAcceptAsync(updatedRaftGroupService -> inBusyLock(busyLock, () -> {
            ((InternalTableImpl) internalTbl).tableRaftService()
                    .updateInternalTableRaftGroupService(partId, 
updatedRaftGroupService);

            boolean startedRaftNode = startGroupFut.join();
            if (localMemberAssignment == null || !startedRaftNode || 
replicaMgr.isReplicaStarted(replicaGrpId)) {
                return;
            } {code}
{color:#00855f}the code shows that {{v}} argument ({{{}startGroupFut{}}}'s 
result) in the lambda doesn't affect on raft-client's starting, and also the 
client is put into {{TableRaftService}} then regardless of replica's 
starting.{color}

On the other hand, there is a place that rely on raft-client creation on every 
node with table's ID in {{TableManager#tables}} map: inside 
{{TableManager#handleChangePendingAssignmentEvent}} there are two calls: 
 # the same name method 
{{TableManager#handleChangePendingAssignmentEvent}}{{{}{}}}
 # {{TableManager#changePeersOnRebalance}}

Both are asking for internal table's 
{{{}tableRaftService().partitionRaftGroupService(partId){}}}, but there no any 
checks about is the local node is suitable for raft-entities and replica 
starting or they are already started.
 
Then, when we would try to fix instant raft-client starting it will lead to 
instant {{IgniteInternalException}} with _"No such partition P in table T"_ 
from {{{}TableRaftService#{}}}{{{}partitionRaftGroupService{}}}. This is a 
problem that should be solved.


{color:#383838}*Definition of done*{color}
{color:#383838}1. Raft-client must be started only together with raft-node and 
replica or shouldn't be started otherwise.{color}
{color:#383838}2. {{TableRaftService}} shouldn't be requested for raft-client 
if the local node isn't in a raft group.{color}
{color:#383838}3. Some tests may rely on described behavior, then failed tests 
should be investigated and, probably fixed.{color}
{color:#383838}4. New tests that check single starting of raft-node, 
raft-client and replica together are required{color}
{color:#0f54d6} {color}

{color:#383838} {color}

{color:#00855f} {color}


> Make raft-client starting only once and only with raft-client and replica 
> together
> ----------------------------------------------------------------------------------
>
>                 Key: IGNITE-22315
>                 URL: https://issues.apache.org/jira/browse/IGNITE-22315
>             Project: Ignite
>          Issue Type: Improvement
>            Reporter: Mikhail Efremov
>            Assignee: Mikhail Efremov
>            Priority: Major
>              Labels: ignite-3
>
> *Description*
> The goal is to make sure that when partition starting on an ignite-node, if 
> the node in assignments, then there should be only ones raft-node, 
> raft-client and replica and no any started entities otherwise.
> *Motivation*
> Now on the one hand in {{TableManager}} there is a path when raft-node and 
> replica didn't start, but raft-client is:
> In {{TableManager#startPartitionAndStartClient}} there is a place where 
> {{startGroupFut}} returns {{{}false{}}}:\{{ localMemberAssignment}} is 
> {{null}} or {{PartitionReplicatorNodeRecovery#shouldStartGroup}} returns 
> {{{}false{}}}. Then, raft-node and replica will not be started, but 
> regardless of {{startGroupFut}} result raft client will be started:
> {code:java}
> startGroupFut
>         .thenComposeAsync(v -> inBusyLock(busyLock, () -> {
>             TableRaftService tableRaftService = 
> table.internalTable().tableRaftService();
>             try {
>                 // Return existing service if it's already started.
>                 return completedFuture(
>                         (TopologyAwareRaftGroupService) 
> tableRaftService.partitionRaftGroupService(replicaGrpId.partitionId())
>                 );
>             } catch (IgniteInternalException e) {
>                 // We use "IgniteInternalException" in accordance with the 
> javadoc of "partitionRaftGroupService" method.
>                 try {
>                     // TODO IGNITE-19614 This procedure takes 10 seconds if 
> there's no majority online.
>                     return raftMgr
>                             .startRaftGroupService(replicaGrpId, 
> newConfiguration, raftGroupServiceFactory, raftCommandsMarshaller);
>                 } catch (NodeStoppingException ex) {
>                     return failedFuture(ex);
>                 }
>             }
>         }), ioExecutor)
>         .thenAcceptAsync(updatedRaftGroupService -> inBusyLock(busyLock, () 
> -> {
>             ((InternalTableImpl) internalTbl).tableRaftService()
>                     .updateInternalTableRaftGroupService(partId, 
> updatedRaftGroupService);
>             boolean startedRaftNode = startGroupFut.join();
>             if (localMemberAssignment == null || !startedRaftNode || 
> replicaMgr.isReplicaStarted(replicaGrpId)) {
>                 return;
>             } {code}
> the code shows that {{v}} argument ({{{}startGroupFut{}}}'s result) in the 
> lambda doesn't affect on raft-client's starting, and also the client is put 
> into {{TableRaftService}} then regardless of replica's starting.
> On the other hand, there is a place that rely on raft-client creation on 
> every node with table's ID in {{TableManager#tables}} map: inside 
> {{TableManager#handleChangePendingAssignmentEvent}} there are two calls:
>  # the same name method 
> {{{}TableManager#handleChangePendingAssignmentEvent{}}}{{{{}}{}}}
>  # {{TableManager#changePeersOnRebalance}}
> Both are asking for internal table's 
> {{{}tableRaftService().partitionRaftGroupService(partId){}}}, but there no 
> any checks about is the local node is suitable for raft-entities and replica 
> starting or they are already started.
>  
> Then, when we would try to fix instant raft-client starting it will lead to 
> instant {{IgniteInternalException}} with _"No such partition P in table T"_ 
> from {{{}TableRaftService#{}}}{{{}partitionRaftGroupService{}}}. This is a 
> problem that should be solved.
> {color:#383838}*Definition of done*{color}
> {color:#383838}1. Raft-client must be started only together with raft-node 
> and replica or shouldn't be started otherwise.{color}
> {color:#383838}2. {{TableRaftService}} shouldn't be requested for raft-client 
> if the local node isn't in a raft group.{color}
> {color:#383838}3. Some tests may rely on described behavior, then failed 
> tests should be investigated and, probably fixed.{color}
> {color:#383838}4. New tests that check single starting of raft-node, 
> raft-client and replica together are required{color}
> {color:#0f54d6} {color}
> {color:#383838} {color}
> {color:#00855f} {color}



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

Reply via email to