[ https://issues.apache.org/jira/browse/IGNITE-20646?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Vladislav Pyatkov updated IGNITE-20646: --------------------------------------- Description: h3. Motivation [Vladislav Pyatkov|https://issues.apache.org/jira/secure/ViewProfile.jspa?name=vpyatkov] has strong point that current CLOCK_SKEW aware implementation of await/getPrimaryReplica should be removed, meaning that we should compare expirationTimestamp with propagated one without error(CLOCK_SKEW). Same as for waiting. Current CLOC_SKEW aware approach is specified in org.apache.ignite.internal.placementdriver.PlacementDriver javadocs and guarantees that if primary replica is visible from await on node A it'll be visible thought get on any other node. h3. Definition of Done We should prepare full set of pros and cons for each option. Whether they are correct? Which one is better from the performance point of view? Which one is simpler? h3. UPD The issue is deeper than we described it here. The root cause of the issue is the placement driver API, which is inappropriate to use for someone who has never seen the implementation. I mean the two methods, awaitPrimaryReplica and getPrimaryReplica, because both methods get a timestamp from parameters but have the internal proposal that the timestamp is the current timestamp for this node. Nevertheless, functionality is required. My advise: {code} public CompletableFuture<ReplicaMeta> awaitPrimaryReplica( ReplicationGroupId groupId, long timeout, TimeUnit unit ) { return inBusyLockAsync(busyLock, () -> { CompletableFuture<ReplicaMeta> fut = new CompletableFuture<>(); EventListener<PrimaryReplicaEventParameters> lnsr = (parameters, exception) -> { if (parameters.groupId().equals(groupId)) { fut.complete(leases.leaseByGroupId().get(groupId)); return completedFuture(true); } return completedFuture(false); }; listen(PRIMARY_REPLICA_ELECTED, lnsr); // It is a pseudocode, basically. You should get the lease from the event parameter in the project code. Lease lease = leases.leaseByGroupId().getOrDefault(groupId, EMPTY_LEASE); if (lease.isAccepted()) { removeListener(PRIMARY_REPLICA_ELECTED, lnsr); fut.complete(lease); } return fut; }); } public CompletableFuture<ReplicaMeta> getPrimaryReplica(ReplicationGroupId replicationGroupId, HybridTimestamp timestamp) { return inBusyLockAsync(busyLock, () -> { Lease lease = leases.leaseByGroupId().getOrDefault(replicationGroupId, EMPTY_LEASE); if (lease.isAccepted() && lease.getExpirationTime().compareTo(timestamp) > 0 && lease.getStartTime().compareTo(timestamp) < 0 ) { return completedFuture(lease); } // TODO: Here we have to compare with the last update placement driver timestamp. Unfortunately, we do not save the // timestamp in the lease updater (Updater#updateLeaseBatchInternal) thread. It must be done in the future. // Based on Meta storage, safe time is the wrong solution because the lease timestamp does not depend on the safe time. return msManager .clusterTime() .waitFor(timestamp.addPhysicalTime(CLOCK_SKEW)) .thenApply(ignored -> inBusyLock(busyLock, () -> { Lease lease0 = leases.leaseByGroupId().getOrDefault(replicationGroupId, EMPTY_LEASE); if (lease.isAccepted() && lease.getExpirationTime().compareTo(timestamp) > 0 && lease.getStartTime().compareTo(timestamp) < 0 ) { return lease0; } else { return null; } })); }); } {code} was: h3. Motivation [Vladislav Pyatkov|https://issues.apache.org/jira/secure/ViewProfile.jspa?name=vpyatkov] has strong point that current CLOCK_SKEW aware implementation of await/getPrimaryReplica should be removed, meaning that we should compare expirationTimestamp with propagated one without error(CLOCK_SKEW). Same as for waiting. Current CLOC_SKEW aware approach is specified in org.apache.ignite.internal.placementdriver.PlacementDriver javadocs and guarantees that if primary replica is visible from await on node A it'll be visible thought get on any other node. h3. Definition of Done We should prepare full set of pros and cons for each option. Whether they are correct? Which one is better from the performance point of view? Which one is simpler? > Consider using CLOCK_SKEW unaware await/getPrimaryReplica() > ----------------------------------------------------------- > > Key: IGNITE-20646 > URL: https://issues.apache.org/jira/browse/IGNITE-20646 > Project: Ignite > Issue Type: Improvement > Reporter: Alexander Lapin > Priority: Major > > h3. Motivation > [Vladislav > Pyatkov|https://issues.apache.org/jira/secure/ViewProfile.jspa?name=vpyatkov] > has strong point that current CLOCK_SKEW aware implementation of > await/getPrimaryReplica should be removed, meaning that we should compare > expirationTimestamp with propagated one without error(CLOCK_SKEW). Same as > for waiting. Current CLOC_SKEW aware approach is specified in > org.apache.ignite.internal.placementdriver.PlacementDriver javadocs and > guarantees that if primary replica is visible from await on node A it'll be > visible thought get on any other node. > h3. Definition of Done > We should prepare full set of pros and cons for each option. Whether they are > correct? Which one is better from the performance point of view? Which one is > simpler? > h3. UPD > The issue is deeper than we described it here. The root cause of the issue is > the placement driver API, which is inappropriate to use for someone who has > never seen the implementation. I mean the two methods, awaitPrimaryReplica > and getPrimaryReplica, because both methods get a timestamp from parameters > but have the internal proposal that the timestamp is the current timestamp > for this node. Nevertheless, functionality is required. > My advise: > {code} > public CompletableFuture<ReplicaMeta> awaitPrimaryReplica( > ReplicationGroupId groupId, > long timeout, > TimeUnit unit > ) { > return inBusyLockAsync(busyLock, () -> { > CompletableFuture<ReplicaMeta> fut = new CompletableFuture<>(); > EventListener<PrimaryReplicaEventParameters> lnsr = (parameters, > exception) -> { > if (parameters.groupId().equals(groupId)) { > fut.complete(leases.leaseByGroupId().get(groupId)); > return completedFuture(true); > } > return completedFuture(false); > }; > listen(PRIMARY_REPLICA_ELECTED, lnsr); > // It is a pseudocode, basically. You should get the lease from > the event parameter in the project code. > Lease lease = leases.leaseByGroupId().getOrDefault(groupId, > EMPTY_LEASE); > if (lease.isAccepted()) { > removeListener(PRIMARY_REPLICA_ELECTED, lnsr); > fut.complete(lease); > } > return fut; > }); > } > public CompletableFuture<ReplicaMeta> > getPrimaryReplica(ReplicationGroupId replicationGroupId, HybridTimestamp > timestamp) { > return inBusyLockAsync(busyLock, () -> { > Lease lease = > leases.leaseByGroupId().getOrDefault(replicationGroupId, EMPTY_LEASE); > if (lease.isAccepted() && > lease.getExpirationTime().compareTo(timestamp) > 0 && > lease.getStartTime().compareTo(timestamp) < 0 > ) { > return completedFuture(lease); > } > // TODO: Here we have to compare with the last update placement > driver timestamp. Unfortunately, we do not save the > // timestamp in the lease updater > (Updater#updateLeaseBatchInternal) thread. It must be done in the future. > // Based on Meta storage, safe time is the wrong solution because > the lease timestamp does not depend on the safe time. > return msManager > .clusterTime() > .waitFor(timestamp.addPhysicalTime(CLOCK_SKEW)) > .thenApply(ignored -> inBusyLock(busyLock, () -> { > Lease lease0 = > leases.leaseByGroupId().getOrDefault(replicationGroupId, EMPTY_LEASE); > if (lease.isAccepted() && > > lease.getExpirationTime().compareTo(timestamp) > 0 && > lease.getStartTime().compareTo(timestamp) < 0 > ) { > return lease0; > } else { > return null; > } > })); > }); > } > {code} -- This message was sent by Atlassian Jira (v8.20.10#820010)