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

Denis Chudov updated IGNITE-23702:
----------------------------------
    Description: 
*Stack trace:*
{code:java}
java.lang.AssertionError: Unexpected replica reservation with STOPPING state 
[groupId=18_part_1].
    at 
org.apache.ignite.internal.replicator.ReplicaManager$ReplicaStateManager.reserveReplica(ReplicaManager.java:1560)
 ~[ignite-replicator-9.0.127-SNAPSHOT.jar:?]
    at 
org.apache.ignite.internal.replicator.ReplicaImpl.waitForActualState(ReplicaImpl.java:306)
 ~[ignite-replicator-9.0.127-SNAPSHOT.jar:?]
    at 
org.apache.ignite.internal.replicator.ReplicaImpl.lambda$processLeaseGrantedMessage$6(ReplicaImpl.java:240)
 ~[ignite-replicator-9.0.127-SNAPSHOT.jar:?]
    at 
java.base/java.util.concurrent.CompletableFuture.uniComposeStage(CompletableFuture.java:1106)
 ~[?:?]
    at 
java.base/java.util.concurrent.CompletableFuture.thenCompose(CompletableFuture.java:2235)
 ~[?:?]
    at 
org.apache.ignite.internal.replicator.ReplicaImpl.lambda$processLeaseGrantedMessage$7(ReplicaImpl.java:209)
 ~[ignite-replicator-9.0.127-SNAPSHOT.jar:?]
    at 
java.base/java.util.concurrent.CompletableFuture.uniComposeStage(CompletableFuture.java:1106)
 ~[?:?]
    at 
java.base/java.util.concurrent.CompletableFuture.thenCompose(CompletableFuture.java:2235)
 ~[?:?]
    at 
org.apache.ignite.internal.replicator.ReplicaImpl.processLeaseGrantedMessage(ReplicaImpl.java:209)
 ~[ignite-replicator-9.0.127-SNAPSHOT.jar:?]
    at 
org.apache.ignite.internal.replicator.ReplicaImpl.processPlacementDriverMessage(ReplicaImpl.java:178)
 ~[ignite-replicator-9.0.127-SNAPSHOT.jar:?]
    at 
org.apache.ignite.internal.replicator.ReplicaManager.lambda$onPlacementDriverMessageReceived$7(ReplicaManager.java:539)
 ~[ignite-replicator-9.0.127-SNAPSHOT.jar:?]
    at 
java.base/java.util.concurrent.CompletableFuture.uniComposeStage(CompletableFuture.java:1106)
 ~[?:?]
    at 
java.base/java.util.concurrent.CompletableFuture.thenCompose(CompletableFuture.java:2235)
 ~[?:?]
    at 
org.apache.ignite.internal.replicator.ReplicaManager.onPlacementDriverMessageReceived(ReplicaManager.java:539)
 ~[ignite-replicator-9.0.127-SNAPSHOT.jar:?]
    at 
org.apache.ignite.internal.network.TrackableNetworkMessageHandler.onReceived(TrackableNetworkMessageHandler.java:52)
 ~[ignite-network-api-9.0.127-SNAPSHOT.jar:?]
    at 
org.apache.ignite.internal.network.DefaultMessagingService.handleStartingWithFirstHandler(DefaultMessagingService.java:549)
 ~[ignite-network-9.0.127-SNAPSHOT.jar:?]
    at 
org.apache.ignite.internal.network.DefaultMessagingService.lambda$handleMessageFromNetwork$5(DefaultMessagingService.java:440)
 ~[ignite-network-9.0.127-SNAPSHOT.jar:?]
    at 
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
 [?:?]
    at 
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
 [?:?]
    at java.base/java.lang.Thread.run(Thread.java:834) [?:?]{code}
*Scenario:*
 * Replica should be stopped due to partition restart;
 * replica appears to be a primary one, stop lease prolongation message is sent;
 * right after stop lease prolongation handling (9 ms later) LeaseGrantMessage 
is received on the same node where the primary replica was located;
 * 1 ms after LeaseGrantMessage is received, replica is stopped (the stop of 
raft node begins);
 * In parallel, the handling of LeaseGrantMessage continues, and the replica 
should be reserved as primary. The state is STOPPED already, and 
reservedForPrimary is true (due to previous primary state), which is incorrect, 
so the assertion is thrown. 

The correct scenario would be: reservedForPrimary becomes false during the 
handling of PRIMARY_REPLICA_EXPIRED event and reserveReplica() returns false 
due to STOPPING state. No error is thrown, replica declines the lease.

*What needs to be fixed:*
 * missed synchronized block, which should have wrapped stopReplica method in 
ReplicaStateManager#planDeferredReplicaStop
 * incorrect lease expiration waiting: reservation flag may be not removed when 
expiration timestamp comes. This means only that lease is expired, but doesn't 
mean that PRIMARY_REPLICA_EXPIRED was handled, so reservedForPrimary is still 
true and replica stop process still cannot begin. The deferred replica stop 
should wait not for this timestamp, but the event of expiration of the lease 
having the corresponding start time, or election of a new lease having the 
start time greater than expiration timestamp.

 

 

  was:
*Stack trace:*
{code:java}
java.lang.AssertionError: Unexpected replica reservation with STOPPING state 
[groupId=18_part_1].
    at 
org.apache.ignite.internal.replicator.ReplicaManager$ReplicaStateManager.reserveReplica(ReplicaManager.java:1560)
 ~[ignite-replicator-9.0.127-SNAPSHOT.jar:?]
    at 
org.apache.ignite.internal.replicator.ReplicaImpl.waitForActualState(ReplicaImpl.java:306)
 ~[ignite-replicator-9.0.127-SNAPSHOT.jar:?]
    at 
org.apache.ignite.internal.replicator.ReplicaImpl.lambda$processLeaseGrantedMessage$6(ReplicaImpl.java:240)
 ~[ignite-replicator-9.0.127-SNAPSHOT.jar:?]
    at 
java.base/java.util.concurrent.CompletableFuture.uniComposeStage(CompletableFuture.java:1106)
 ~[?:?]
    at 
java.base/java.util.concurrent.CompletableFuture.thenCompose(CompletableFuture.java:2235)
 ~[?:?]
    at 
org.apache.ignite.internal.replicator.ReplicaImpl.lambda$processLeaseGrantedMessage$7(ReplicaImpl.java:209)
 ~[ignite-replicator-9.0.127-SNAPSHOT.jar:?]
    at 
java.base/java.util.concurrent.CompletableFuture.uniComposeStage(CompletableFuture.java:1106)
 ~[?:?]
    at 
java.base/java.util.concurrent.CompletableFuture.thenCompose(CompletableFuture.java:2235)
 ~[?:?]
    at 
org.apache.ignite.internal.replicator.ReplicaImpl.processLeaseGrantedMessage(ReplicaImpl.java:209)
 ~[ignite-replicator-9.0.127-SNAPSHOT.jar:?]
    at 
org.apache.ignite.internal.replicator.ReplicaImpl.processPlacementDriverMessage(ReplicaImpl.java:178)
 ~[ignite-replicator-9.0.127-SNAPSHOT.jar:?]
    at 
org.apache.ignite.internal.replicator.ReplicaManager.lambda$onPlacementDriverMessageReceived$7(ReplicaManager.java:539)
 ~[ignite-replicator-9.0.127-SNAPSHOT.jar:?]
    at 
java.base/java.util.concurrent.CompletableFuture.uniComposeStage(CompletableFuture.java:1106)
 ~[?:?]
    at 
java.base/java.util.concurrent.CompletableFuture.thenCompose(CompletableFuture.java:2235)
 ~[?:?]
    at 
org.apache.ignite.internal.replicator.ReplicaManager.onPlacementDriverMessageReceived(ReplicaManager.java:539)
 ~[ignite-replicator-9.0.127-SNAPSHOT.jar:?]
    at 
org.apache.ignite.internal.network.TrackableNetworkMessageHandler.onReceived(TrackableNetworkMessageHandler.java:52)
 ~[ignite-network-api-9.0.127-SNAPSHOT.jar:?]
    at 
org.apache.ignite.internal.network.DefaultMessagingService.handleStartingWithFirstHandler(DefaultMessagingService.java:549)
 ~[ignite-network-9.0.127-SNAPSHOT.jar:?]
    at 
org.apache.ignite.internal.network.DefaultMessagingService.lambda$handleMessageFromNetwork$5(DefaultMessagingService.java:440)
 ~[ignite-network-9.0.127-SNAPSHOT.jar:?]
    at 
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
 [?:?]
    at 
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
 [?:?]
    at java.base/java.lang.Thread.run(Thread.java:834) [?:?]{code}
*Scenario:*

 

*What needs to be fixed:*
 * missed synchronized block, which should have wrapped stopReplica method in 
ReplicaStateManager#planDeferredReplicaStop
 * incorrect lease expiration waiting: reservation flah may be not removed when 
expiration timestamp comes

 

 


> Incorrect HB in deferred replica stop on partition restart
> ----------------------------------------------------------
>
>                 Key: IGNITE-23702
>                 URL: https://issues.apache.org/jira/browse/IGNITE-23702
>             Project: Ignite
>          Issue Type: Bug
>            Reporter: Denis Chudov
>            Priority: Major
>              Labels: ignite-3
>         Attachments: _Integration_Tests_Integration_CLI_10340.log
>
>
> *Stack trace:*
> {code:java}
> java.lang.AssertionError: Unexpected replica reservation with STOPPING state 
> [groupId=18_part_1].
>     at 
> org.apache.ignite.internal.replicator.ReplicaManager$ReplicaStateManager.reserveReplica(ReplicaManager.java:1560)
>  ~[ignite-replicator-9.0.127-SNAPSHOT.jar:?]
>     at 
> org.apache.ignite.internal.replicator.ReplicaImpl.waitForActualState(ReplicaImpl.java:306)
>  ~[ignite-replicator-9.0.127-SNAPSHOT.jar:?]
>     at 
> org.apache.ignite.internal.replicator.ReplicaImpl.lambda$processLeaseGrantedMessage$6(ReplicaImpl.java:240)
>  ~[ignite-replicator-9.0.127-SNAPSHOT.jar:?]
>     at 
> java.base/java.util.concurrent.CompletableFuture.uniComposeStage(CompletableFuture.java:1106)
>  ~[?:?]
>     at 
> java.base/java.util.concurrent.CompletableFuture.thenCompose(CompletableFuture.java:2235)
>  ~[?:?]
>     at 
> org.apache.ignite.internal.replicator.ReplicaImpl.lambda$processLeaseGrantedMessage$7(ReplicaImpl.java:209)
>  ~[ignite-replicator-9.0.127-SNAPSHOT.jar:?]
>     at 
> java.base/java.util.concurrent.CompletableFuture.uniComposeStage(CompletableFuture.java:1106)
>  ~[?:?]
>     at 
> java.base/java.util.concurrent.CompletableFuture.thenCompose(CompletableFuture.java:2235)
>  ~[?:?]
>     at 
> org.apache.ignite.internal.replicator.ReplicaImpl.processLeaseGrantedMessage(ReplicaImpl.java:209)
>  ~[ignite-replicator-9.0.127-SNAPSHOT.jar:?]
>     at 
> org.apache.ignite.internal.replicator.ReplicaImpl.processPlacementDriverMessage(ReplicaImpl.java:178)
>  ~[ignite-replicator-9.0.127-SNAPSHOT.jar:?]
>     at 
> org.apache.ignite.internal.replicator.ReplicaManager.lambda$onPlacementDriverMessageReceived$7(ReplicaManager.java:539)
>  ~[ignite-replicator-9.0.127-SNAPSHOT.jar:?]
>     at 
> java.base/java.util.concurrent.CompletableFuture.uniComposeStage(CompletableFuture.java:1106)
>  ~[?:?]
>     at 
> java.base/java.util.concurrent.CompletableFuture.thenCompose(CompletableFuture.java:2235)
>  ~[?:?]
>     at 
> org.apache.ignite.internal.replicator.ReplicaManager.onPlacementDriverMessageReceived(ReplicaManager.java:539)
>  ~[ignite-replicator-9.0.127-SNAPSHOT.jar:?]
>     at 
> org.apache.ignite.internal.network.TrackableNetworkMessageHandler.onReceived(TrackableNetworkMessageHandler.java:52)
>  ~[ignite-network-api-9.0.127-SNAPSHOT.jar:?]
>     at 
> org.apache.ignite.internal.network.DefaultMessagingService.handleStartingWithFirstHandler(DefaultMessagingService.java:549)
>  ~[ignite-network-9.0.127-SNAPSHOT.jar:?]
>     at 
> org.apache.ignite.internal.network.DefaultMessagingService.lambda$handleMessageFromNetwork$5(DefaultMessagingService.java:440)
>  ~[ignite-network-9.0.127-SNAPSHOT.jar:?]
>     at 
> java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
>  [?:?]
>     at 
> java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
>  [?:?]
>     at java.base/java.lang.Thread.run(Thread.java:834) [?:?]{code}
> *Scenario:*
>  * Replica should be stopped due to partition restart;
>  * replica appears to be a primary one, stop lease prolongation message is 
> sent;
>  * right after stop lease prolongation handling (9 ms later) 
> LeaseGrantMessage is received on the same node where the primary replica was 
> located;
>  * 1 ms after LeaseGrantMessage is received, replica is stopped (the stop of 
> raft node begins);
>  * In parallel, the handling of LeaseGrantMessage continues, and the replica 
> should be reserved as primary. The state is STOPPED already, and 
> reservedForPrimary is true (due to previous primary state), which is 
> incorrect, so the assertion is thrown. 
> The correct scenario would be: reservedForPrimary becomes false during the 
> handling of PRIMARY_REPLICA_EXPIRED event and reserveReplica() returns false 
> due to STOPPING state. No error is thrown, replica declines the lease.
> *What needs to be fixed:*
>  * missed synchronized block, which should have wrapped stopReplica method in 
> ReplicaStateManager#planDeferredReplicaStop
>  * incorrect lease expiration waiting: reservation flag may be not removed 
> when expiration timestamp comes. This means only that lease is expired, but 
> doesn't mean that PRIMARY_REPLICA_EXPIRED was handled, so reservedForPrimary 
> is still true and replica stop process still cannot begin. The deferred 
> replica stop should wait not for this timestamp, but the event of expiration 
> of the lease having the corresponding start time, or election of a new lease 
> having the start time greater than expiration timestamp.
>  
>  



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

Reply via email to