[ https://issues.apache.org/jira/browse/IGNITE-19532?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Alexander Lapin updated IGNITE-19532: ------------------------------------- Description: h3. Motivation Let's assume that we have a component named LeaseTracker that locally stores leases in a map though meta storage updates {code:java} public class LeaseTracker implements PlacementDriver { /** Leases cache. */ private final Map<ReplicationGroupId, Lease> leases; ... private class UpdateListener implements WatchListener { @Override public CompletableFuture<Void> onUpdate(WatchEvent event) { for (EntryEvent entry : event.entryEvents()) { ... Lease lease = fromBytes(msEntry.value()); leases.put(grpId, lease); ... return completedFuture(null); } } ...{code} and we want to await lease in a meta storage safe time bounded way. {code:java} public CompletableFuture<LeaseMeta> getPrimaryReplica(ReplicationGroupId replicationGroupId, HybridTimestamp timestamp) { ... return msManager.clusterTime().waitFor(timestamp).thenApply(() -> { ... Lease lease0 = leases.get(replicationGroupId); if (lease.getExpirationTime().after(timestamp)) { return lease0; } else { return null; } ... } } {code} Currently that won't work properly because {code:java} msManager.clusterTime().waitFor(timestamp) {code} will be completed before local leases will be populated with data through meta storage events. Thus it'll be nice to complete clusterTime().waitFor(timestamp) futute (and generally speaking publish safe time) after all corresponding ms listeners are completed. was: h3. Motivation Let's assume that we have a component named LeaseTracker that locally stores leases in a map though meta storage updates {code:java} public class LeaseTracker implements PlacementDriver { /** Leases cache. */ private final Map<ReplicationGroupId, Lease> leases; ... private class UpdateListener implements WatchListener { @Override public CompletableFuture<Void> onUpdate(WatchEvent event) { for (EntryEvent entry : event.entryEvents()) { ... Lease lease = fromBytes(msEntry.value()); leases.put(grpId, lease); ... return completedFuture(null); } } ...{code} and we want to await lease in a meta storage safe time bounded way. {code:java} public CompletableFuture<LeaseMeta> getPrimaryReplica(ReplicationGroupId replicationGroupId, HybridTimestamp timestamp) { ... return msManager.clusterTime().waitFor(timestamp).thenApply(() -> { ... Lease lease0 = leases.get(replicationGroupId); if (lease.getExpirationTime().after(timestamp)) { return lease0; } else { return null; } ... })); } finally { busyLock.leaveBusy(); } } {code} Currently we may > Introduce happends before relation between local meta storage safe time > publication and completion of corresponding meta storage listners > ----------------------------------------------------------------------------------------------------------------------------------------- > > Key: IGNITE-19532 > URL: https://issues.apache.org/jira/browse/IGNITE-19532 > Project: Ignite > Issue Type: Improvement > Reporter: Alexander Lapin > Priority: Major > > h3. Motivation > Let's assume that we have a component named LeaseTracker that locally stores > leases in a map though meta storage updates > {code:java} > public class LeaseTracker implements PlacementDriver { > /** Leases cache. */ > private final Map<ReplicationGroupId, Lease> leases; > ... > private class UpdateListener implements WatchListener { > @Override > public CompletableFuture<Void> onUpdate(WatchEvent event) { > for (EntryEvent entry : event.entryEvents()) { > ... > Lease lease = fromBytes(msEntry.value()); > leases.put(grpId, lease); > ... > return completedFuture(null); > } > } > ...{code} > and we want to await lease in a meta storage safe time bounded way. > {code:java} > public CompletableFuture<LeaseMeta> getPrimaryReplica(ReplicationGroupId > replicationGroupId, HybridTimestamp timestamp) { > ... > return msManager.clusterTime().waitFor(timestamp).thenApply(() -> { > ... > Lease lease0 = leases.get(replicationGroupId); > if (lease.getExpirationTime().after(timestamp)) { > return lease0; > } else { > return null; > } > ... > } > } {code} > Currently that won't work properly > because > {code:java} > msManager.clusterTime().waitFor(timestamp) {code} > will be completed before local leases will be populated with data through > meta storage events. > Thus it'll be nice to complete clusterTime().waitFor(timestamp) futute (and > generally speaking publish safe time) after all corresponding ms listeners > are completed. -- This message was sent by Atlassian Jira (v8.20.10#820010)