[
https://issues.apache.org/jira/browse/IGNITE-21566?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Alexander Lapin updated IGNITE-21566:
-------------------------------------
Description:
Multiple tests fail with either `Failed to get the primary replica` or becauase
corresponding lease is not accepted.
{code:java}
Caused by:
org.apache.ignite.internal.placementdriver.PrimaryReplicaAwaitTimeoutException:
IGN-PLACEMENTDRIVER-1 TraceId:11dd6a63-2686-4dfe-b1d4-ea85f3859816 The primary
replica await timed out [replicationGroupId=7_part_0,
referenceTimestamp=HybridTimestamp [physical=2024-02-19 11:58:08:158 +0000,
logical=3, composite=111958025054322691], currentLease=Lease
[leaseholder=itrst_sirot_0, leaseholderId=f43432e9-f374-41e1-b8b6-9813bb59b8a1,
accepted=false, startTime=HybridTimestamp [physical=2024-02-19 11:58:07:936
+0000, logical=2, composite=111958025039773698], expirationTime=HybridTimestamp
[physical=2024-02-19 12:00:07:936 +0000, logical=0,
composite=111958032904093696], prolongable=false,
replicationGroupId=7_part_0]]{code}
[ItTableRaftSnapshotsTest#snapshotInstallationRepeatsOnTimeout|https://ci.ignite.apache.org/buildConfiguration/ApacheIgnite3xGradle_Test_RunAllTests/7866010?expandBuildDeploymentsSection=false&hideTestsFromDependencies=false&hideProblemsFromDependencies=false&expandCode+Inspection=true&expandBuildProblemsSection=true&expandBuildChangesSection=true&expandBuildTestsSection=true]
h3. Upd#1
While starting replicas we have following code
{code:java}
if (localMemberAssignment == null || !startedRaftNode ||
replicaMgr.isReplicaStarted(replicaGrpId)) {
return;
}
try {
startReplicaWithNewListener(
... {code}
that actually won't start a replica if
{{replicaMgr.isReplicaStarted(replicaGrpId)}} that basically checks
{code:java}
public boolean isReplicaStarted(ReplicationGroupId replicaGrpId) {
return replicas.containsKey(replicaGrpId);
} {code}
where {{replicas }}is
{code:java}
private final ConcurrentHashMap<ReplicationGroupId, CompletableFuture<Replica>>
replicas = new ConcurrentHashMap<>(); {code}
On the other hand common replica message processing pattern assumes that
replica may not be ready and in that case it populates {{replicas}} with future
and awaits it. E.g. for leaseGrantMessage
{code:java}
CompletableFuture<Replica> replicaFut = replicas.computeIfAbsent(msg.groupId(),
k -> new CompletableFuture<>());
replicaFut.thenCompose(replica -> replica.processPlacementDriverMessage(msg))
{code}
All in all that means, that if leaseGrantMessage is a bit faster than replica
start (which is fine), it will populate replicas with a future and replica
start flow won't actually start the one and complete given future but just will
incorrectly assume that replica is already started.
was:
Multiple tests fail with either `Failed to get the primary replica` or becauase
corresponding lease is not accepted.
{code:java}
Caused by:
org.apache.ignite.internal.placementdriver.PrimaryReplicaAwaitTimeoutException:
IGN-PLACEMENTDRIVER-1 TraceId:11dd6a63-2686-4dfe-b1d4-ea85f3859816 The primary
replica await timed out [replicationGroupId=7_part_0,
referenceTimestamp=HybridTimestamp [physical=2024-02-19 11:58:08:158 +0000,
logical=3, composite=111958025054322691], currentLease=Lease
[leaseholder=itrst_sirot_0, leaseholderId=f43432e9-f374-41e1-b8b6-9813bb59b8a1,
accepted=false, startTime=HybridTimestamp [physical=2024-02-19 11:58:07:936
+0000, logical=2, composite=111958025039773698], expirationTime=HybridTimestamp
[physical=2024-02-19 12:00:07:936 +0000, logical=0,
composite=111958032904093696], prolongable=false,
replicationGroupId=7_part_0]]{code}
[ItTableRaftSnapshotsTest#snapshotInstallationRepeatsOnTimeout|https://ci.ignite.apache.org/buildConfiguration/ApacheIgnite3xGradle_Test_RunAllTests/7866010?expandBuildDeploymentsSection=false&hideTestsFromDependencies=false&hideProblemsFromDependencies=false&expandCode+Inspection=true&expandBuildProblemsSection=true&expandBuildChangesSection=true&expandBuildTestsSection=true]
> Multiple tests fail with Failed to get the primary replica caused by
> PrimaryReplicaAwaitTimeoutException because of unaccepted lease
> ------------------------------------------------------------------------------------------------------------------------------------
>
> Key: IGNITE-21566
> URL: https://issues.apache.org/jira/browse/IGNITE-21566
> Project: Ignite
> Issue Type: Bug
> Reporter: Alexander Lapin
> Assignee: Alexander Lapin
> Priority: Blocker
> Labels: ignite-3
> Time Spent: 10m
> Remaining Estimate: 0h
>
> Multiple tests fail with either `Failed to get the primary replica` or
> becauase corresponding lease is not accepted.
> {code:java}
> Caused by:
> org.apache.ignite.internal.placementdriver.PrimaryReplicaAwaitTimeoutException:
> IGN-PLACEMENTDRIVER-1 TraceId:11dd6a63-2686-4dfe-b1d4-ea85f3859816 The
> primary replica await timed out [replicationGroupId=7_part_0,
> referenceTimestamp=HybridTimestamp [physical=2024-02-19 11:58:08:158 +0000,
> logical=3, composite=111958025054322691], currentLease=Lease
> [leaseholder=itrst_sirot_0,
> leaseholderId=f43432e9-f374-41e1-b8b6-9813bb59b8a1, accepted=false,
> startTime=HybridTimestamp [physical=2024-02-19 11:58:07:936 +0000, logical=2,
> composite=111958025039773698], expirationTime=HybridTimestamp
> [physical=2024-02-19 12:00:07:936 +0000, logical=0,
> composite=111958032904093696], prolongable=false,
> replicationGroupId=7_part_0]]{code}
> [ItTableRaftSnapshotsTest#snapshotInstallationRepeatsOnTimeout|https://ci.ignite.apache.org/buildConfiguration/ApacheIgnite3xGradle_Test_RunAllTests/7866010?expandBuildDeploymentsSection=false&hideTestsFromDependencies=false&hideProblemsFromDependencies=false&expandCode+Inspection=true&expandBuildProblemsSection=true&expandBuildChangesSection=true&expandBuildTestsSection=true]
>
> h3. Upd#1
> While starting replicas we have following code
> {code:java}
> if (localMemberAssignment == null || !startedRaftNode ||
> replicaMgr.isReplicaStarted(replicaGrpId)) {
> return;
> }
> try {
> startReplicaWithNewListener(
> ... {code}
> that actually won't start a replica if
> {{replicaMgr.isReplicaStarted(replicaGrpId)}} that basically checks
> {code:java}
> public boolean isReplicaStarted(ReplicationGroupId replicaGrpId) {
> return replicas.containsKey(replicaGrpId);
> } {code}
> where {{replicas }}is
> {code:java}
> private final ConcurrentHashMap<ReplicationGroupId,
> CompletableFuture<Replica>> replicas = new ConcurrentHashMap<>(); {code}
> On the other hand common replica message processing pattern assumes that
> replica may not be ready and in that case it populates {{replicas}} with
> future and awaits it. E.g. for leaseGrantMessage
>
> {code:java}
> CompletableFuture<Replica> replicaFut =
> replicas.computeIfAbsent(msg.groupId(), k -> new CompletableFuture<>());
> replicaFut.thenCompose(replica -> replica.processPlacementDriverMessage(msg))
> {code}
> All in all that means, that if leaseGrantMessage is a bit faster than replica
> start (which is fine), it will populate replicas with a future and replica
> start flow won't actually start the one and complete given future but just
> will incorrectly assume that replica is already started.
>
--
This message was sent by Atlassian Jira
(v8.20.10#820010)