From cb043b3e3b7866a3d8602a5b94763bc323b0c4c3 Mon Sep 17 00:00:00 2001 From: Alan Wang Date: Wed, 25 Feb 2026 01:01:50 -0800 Subject: [PATCH 1/8] initial --- .../apache/cassandra/tcm/sequences/Move.java | 4 + .../test/accord/AccordRegainRangesTest.java | 114 ++++++++++++++++++ 2 files changed, 118 insertions(+) create mode 100644 test/distributed/org/apache/cassandra/distributed/test/accord/AccordRegainRangesTest.java diff --git a/src/java/org/apache/cassandra/tcm/sequences/Move.java b/src/java/org/apache/cassandra/tcm/sequences/Move.java index a26d6ded4500..d755932c4885 100644 --- a/src/java/org/apache/cassandra/tcm/sequences/Move.java +++ b/src/java/org/apache/cassandra/tcm/sequences/Move.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.Collection; import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Set; @@ -53,6 +54,7 @@ import org.apache.cassandra.schema.Schema; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.service.accord.AccordService; +import org.apache.cassandra.service.accord.AccordTopology; import org.apache.cassandra.streaming.StreamOperation; import org.apache.cassandra.streaming.StreamPlan; import org.apache.cassandra.streaming.StreamResultFuture; @@ -203,6 +205,8 @@ public SequenceState executeNext() try { ClusterMetadata metadata = ClusterMetadata.current(); + List epochsWeAreRegainingRangesFor = AccordService.instance().topology().epochsWeAreRegainingRangesFor(AccordService.instance().topology().current(), AccordTopology.createAccordTopology(applyTo(metadata).success().metadata)); + AccordService.instance().topology().blockUntilAllEpochsRetired(epochsWeAreRegainingRangesFor); logger.info("Moving {} from {} to {}.", metadata.directory.endpoint(startMove.nodeId()), metadata.tokenMap.tokens(startMove.nodeId()), diff --git a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordRegainRangesTest.java b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordRegainRangesTest.java new file mode 100644 index 000000000000..00aa1bc5ed0b --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordRegainRangesTest.java @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test.accord; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; + +import accord.local.CommandStore; +import accord.local.PreLoadContext; +import accord.primitives.AbstractRanges; +import accord.primitives.Ranges; + +import org.apache.cassandra.distributed.api.ConsistencyLevel; +import org.apache.cassandra.distributed.api.Feature; +import org.apache.cassandra.distributed.api.SimpleQueryResult; +import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.service.accord.AccordService; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.apache.cassandra.service.accord.AccordService.getBlocking; +import static com.google.common.collect.Iterables.getOnlyElement; +import org.junit.BeforeClass; +import org.junit.Test; + +public class AccordRegainRangesTest extends AccordTestBase +{ + private static final Logger logger = LoggerFactory.getLogger(AccordRegainRangesTest.class); + + @Override + protected Logger logger() + { + return logger; + } + + @BeforeClass + public static void setupClass() throws IOException + { + AccordTestBase.setupCluster(builder -> builder + .withoutVNodes() + .withConfig(config -> + config + .set("accord.shard_durability_target_splits", "1") + .set("accord.shard_durability_cycle", "20s") + .with(Feature.NETWORK, Feature.GOSSIP)), 6); + } + + @Test + public void regainRangesTest() throws Throwable + { + List ddls = Arrays.asList("DROP KEYSPACE IF EXISTS " + KEYSPACE + ';', + "CREATE KEYSPACE " + KEYSPACE + " WITH REPLICATION={'class':'SimpleStrategy', 'replication_factor': 3}", + "CREATE TABLE " + qualifiedAccordTableName + " (k int PRIMARY KEY, v int) WITH transactional_mode='full'"); + test(ddls, cluster -> { + cluster.coordinator(2).execute(wrapInTxn("INSERT INTO " + qualifiedAccordTableName + " (k, v) VALUES (?, ?)"), ConsistencyLevel.SERIAL, 1, 2); + SimpleQueryResult result = cluster.coordinator(2).executeWithResult("SELECT token(k) FROM " + qualifiedAccordTableName + " WHERE k = 1 LIMIT 1", ConsistencyLevel.SERIAL); + + String originalToken = cluster.get(2).callOnInstance(() -> getOnlyElement(StorageService.instance.getTokens())); + + long token = (Long) result.toObjectArrays()[0][0]; + + assert(token < Long.parseLong(originalToken)); + + long epoch = cluster.get(2).callOnInstance(() -> { + long priorEpoch = AccordService.instance().topology().epoch(); + StorageService.instance.move(Long.toString(token - 1000)); + return priorEpoch; + }); + + cluster.coordinator(3).execute(wrapInTxn("INSERT INTO " + qualifiedAccordTableName + " (k, v) VALUES (?, ?)"), ConsistencyLevel.SERIAL, 1, 5); + + + cluster.get(2).runOnInstance(() -> { + StorageService.instance.move(originalToken); + }); + + // Ensure no overlapping safeToRead ranges + cluster.get(2).runOnInstance(() -> { + assert (AccordService.instance().topology().active().minEpoch() > epoch); + Ranges range = Ranges.EMPTY; + for (CommandStore commandStore : AccordService.instance().node().commandStores().all()) { + Ranges safeToReadRanges = getBlocking(commandStore.submit((PreLoadContext.Empty) () -> "No overlapping safeToReadRanges", safeCommandStore -> { + Ranges mergedRanges = Ranges.EMPTY; + for (Ranges r : safeCommandStore.safeToReadAt().values()) + mergedRanges = mergedRanges.union(AbstractRanges.UnionMode.MERGE_ADJACENT, r); + return mergedRanges; + })); + + assert(range.overlapping(safeToReadRanges).isEmpty()); + range = range.union(AbstractRanges.UnionMode.MERGE_ADJACENT, safeToReadRanges); + } + }); + }); + } + +} + From 6eddccdac75e359a4f4be0ba6fac28f79a6791c6 Mon Sep 17 00:00:00 2001 From: Alan Wang Date: Thu, 26 Feb 2026 17:19:05 -0800 Subject: [PATCH 2/8] use listener and only wait for ranges that are being regained rather than epoch --- .../apache/cassandra/tcm/sequences/Move.java | 57 ++++++++++++++++++- .../test/accord/AccordRegainRangesTest.java | 19 ++++++- 2 files changed, 72 insertions(+), 4 deletions(-) diff --git a/src/java/org/apache/cassandra/tcm/sequences/Move.java b/src/java/org/apache/cassandra/tcm/sequences/Move.java index d755932c4885..f754b355c503 100644 --- a/src/java/org/apache/cassandra/tcm/sequences/Move.java +++ b/src/java/org/apache/cassandra/tcm/sequences/Move.java @@ -28,12 +28,22 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicBoolean; +import javax.annotation.Nullable; + import com.google.common.annotations.VisibleForTesting; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import accord.api.TopologyListener; +import accord.primitives.Ranges; import accord.topology.EpochReady; +import accord.topology.Topology; +import accord.topology.TopologyException; +import accord.topology.TopologyManager; +import accord.topology.TopologyNotReadyException; +import accord.topology.TopologyRetiredException; +import accord.utils.Invariants; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.SystemKeyspace; @@ -75,6 +85,7 @@ import org.apache.cassandra.tcm.transformations.PrepareMove; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.JVMStabilityInspector; +import org.apache.cassandra.utils.concurrent.Condition; import org.apache.cassandra.utils.concurrent.Future; import org.apache.cassandra.utils.concurrent.FutureCombiner; import org.apache.cassandra.utils.vint.VIntCoding; @@ -86,6 +97,7 @@ import static org.apache.cassandra.tcm.Transformation.Kind.START_MOVE; import static org.apache.cassandra.tcm.sequences.SequenceState.continuable; import static org.apache.cassandra.tcm.sequences.SequenceState.error; +import static org.apache.cassandra.utils.concurrent.Condition.newOneTimeCondition; public class Move extends MultiStepOperation { @@ -205,8 +217,49 @@ public SequenceState executeNext() try { ClusterMetadata metadata = ClusterMetadata.current(); - List epochsWeAreRegainingRangesFor = AccordService.instance().topology().epochsWeAreRegainingRangesFor(AccordService.instance().topology().current(), AccordTopology.createAccordTopology(applyTo(metadata).success().metadata)); - AccordService.instance().topology().blockUntilAllEpochsRetired(epochsWeAreRegainingRangesFor); + TopologyManager.RegainingEpochRange regainingEpochRange = AccordService.instance().topology().epochAndRangeToBeRetired(AccordService.instance().topology().current(), AccordTopology.createAccordTopology(applyTo(metadata).success().metadata)); + if (regainingEpochRange != null) + { + Condition condition = newOneTimeCondition(); + + class waitForEpochAndRangeRetirement implements TopologyListener + { + final Condition condition; + final long waitingForEpoch; + final Ranges waitingForRange; + + public waitForEpochAndRangeRetirement(Condition condition, long waitingForEpoch, Ranges waitingForRange) + { + this.condition = condition; + this.waitingForEpoch = waitingForEpoch; + this.waitingForRange = waitingForRange; + } + + @Override + public void onEpochRetired(Ranges ranges, long epoch, @Nullable Topology topology) + { + try + { + if (AccordService.instance().topology().active().get(waitingForEpoch).retired().containsAll(waitingForRange)) + condition.signal(); + } + catch (TopologyRetiredException e) + { + condition.signal(); + } + catch (TopologyException e) + { + logger.info("Topology exception: ", e); + } + } + } + + waitForEpochAndRangeRetirement wait = new waitForEpochAndRangeRetirement(condition, regainingEpochRange.epoch(), regainingEpochRange.range()); + AccordService.instance().topology().addListener(wait); + condition.awaitThrowUncheckedOnInterrupt(); + AccordService.instance().topology().removeListener(wait); + } + logger.info("Moving {} from {} to {}.", metadata.directory.endpoint(startMove.nodeId()), metadata.tokenMap.tokens(startMove.nodeId()), diff --git a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordRegainRangesTest.java b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordRegainRangesTest.java index 00aa1bc5ed0b..823cdf35d3c1 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordRegainRangesTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordRegainRangesTest.java @@ -22,19 +22,27 @@ import java.util.Arrays; import java.util.List; +import accord.api.RoutingKey; import accord.local.CommandStore; import accord.local.PreLoadContext; import accord.primitives.AbstractRanges; import accord.primitives.Ranges; +import accord.topology.TopologyException; +import org.apache.cassandra.dht.Murmur3Partitioner; import org.apache.cassandra.distributed.api.ConsistencyLevel; import org.apache.cassandra.distributed.api.Feature; import org.apache.cassandra.distributed.api.SimpleQueryResult; +import org.apache.cassandra.schema.TableId; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.service.accord.AccordService; +import org.apache.cassandra.service.accord.TokenRange; +import org.apache.cassandra.service.accord.api.TokenKey; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static accord.primitives.Range.range; import static org.apache.cassandra.service.accord.AccordService.getBlocking; import static com.google.common.collect.Iterables.getOnlyElement; import org.junit.BeforeClass; @@ -86,14 +94,21 @@ public void regainRangesTest() throws Throwable cluster.coordinator(3).execute(wrapInTxn("INSERT INTO " + qualifiedAccordTableName + " (k, v) VALUES (?, ?)"), ConsistencyLevel.SERIAL, 1, 5); - cluster.get(2).runOnInstance(() -> { StorageService.instance.move(originalToken); }); // Ensure no overlapping safeToRead ranges cluster.get(2).runOnInstance(() -> { - assert (AccordService.instance().topology().active().minEpoch() > epoch); + RoutingKey start = TokenKey.parse(TableId.fromString("tid:11"), String.valueOf(token), Murmur3Partitioner.instance); + RoutingKey end = TokenKey.parse(TableId.fromString("tid:11"), originalToken, Murmur3Partitioner.instance); + Ranges regainedRange = Ranges.of(TokenRange.create(start, end)); + try + { + assert (AccordService.instance().topology().active().get(epoch).retired().containsAll(regainedRange)); + } catch (TopologyException e) { + assert(false); + } Ranges range = Ranges.EMPTY; for (CommandStore commandStore : AccordService.instance().node().commandStores().all()) { Ranges safeToReadRanges = getBlocking(commandStore.submit((PreLoadContext.Empty) () -> "No overlapping safeToReadRanges", safeCommandStore -> { From 398fffbd00366e61fe9da45caf1b4c6747f6e27f Mon Sep 17 00:00:00 2001 From: Alan Wang Date: Thu, 26 Feb 2026 17:20:39 -0800 Subject: [PATCH 3/8] additional check to avoid locking --- src/java/org/apache/cassandra/tcm/sequences/Move.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/java/org/apache/cassandra/tcm/sequences/Move.java b/src/java/org/apache/cassandra/tcm/sequences/Move.java index f754b355c503..c60451e9ade2 100644 --- a/src/java/org/apache/cassandra/tcm/sequences/Move.java +++ b/src/java/org/apache/cassandra/tcm/sequences/Move.java @@ -240,7 +240,7 @@ public void onEpochRetired(Ranges ranges, long epoch, @Nullable Topology topolog { try { - if (AccordService.instance().topology().active().get(waitingForEpoch).retired().containsAll(waitingForRange)) + if (epoch >= waitingForEpoch && AccordService.instance().topology().active().get(waitingForEpoch).retired().containsAll(waitingForRange)) condition.signal(); } catch (TopologyRetiredException e) From bd8209fe35f796215504ec37fb5f2aeb35a25c4b Mon Sep 17 00:00:00 2001 From: Alan Wang Date: Fri, 27 Feb 2026 00:53:00 -0800 Subject: [PATCH 4/8] added additional test + removed extra imports --- .gitmodules | 4 +- .../apache/cassandra/tcm/sequences/Move.java | 3 - .../test/accord/AccordRegainRangesTest.java | 60 ++++++++++++++++++- 3 files changed, 61 insertions(+), 6 deletions(-) diff --git a/.gitmodules b/.gitmodules index 616dacf610a7..8b3913ed184d 100644 --- a/.gitmodules +++ b/.gitmodules @@ -1,4 +1,4 @@ [submodule "modules/accord"] path = modules/accord - url = https://github.com/apache/cassandra-accord.git - branch = trunk + url = https://github.com/alanwang67/cassandra-accord.git + branch = CASSANDRA-21183 diff --git a/src/java/org/apache/cassandra/tcm/sequences/Move.java b/src/java/org/apache/cassandra/tcm/sequences/Move.java index c60451e9ade2..a1b6f48df73d 100644 --- a/src/java/org/apache/cassandra/tcm/sequences/Move.java +++ b/src/java/org/apache/cassandra/tcm/sequences/Move.java @@ -21,7 +21,6 @@ import java.io.IOException; import java.util.Collection; import java.util.HashSet; -import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Set; @@ -41,9 +40,7 @@ import accord.topology.Topology; import accord.topology.TopologyException; import accord.topology.TopologyManager; -import accord.topology.TopologyNotReadyException; import accord.topology.TopologyRetiredException; -import accord.utils.Invariants; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.SystemKeyspace; diff --git a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordRegainRangesTest.java b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordRegainRangesTest.java index 823cdf35d3c1..ca5359d8b90a 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordRegainRangesTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordRegainRangesTest.java @@ -24,10 +24,13 @@ import accord.api.RoutingKey; import accord.local.CommandStore; +import accord.local.CommandStores; import accord.local.PreLoadContext; +import accord.local.SafeCommandStore; import accord.primitives.AbstractRanges; import accord.primitives.Ranges; import accord.topology.TopologyException; +import accord.utils.LargeBitSet; import org.apache.cassandra.dht.Murmur3Partitioner; import org.apache.cassandra.distributed.api.ConsistencyLevel; @@ -42,9 +45,11 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static accord.primitives.Range.range; +import static accord.local.CommandStores.checkQueryDisjointRangesAcrossCommandStores; import static org.apache.cassandra.service.accord.AccordService.getBlocking; import static com.google.common.collect.Iterables.getOnlyElement; +import static org.junit.Assert.assertFalse; + import org.junit.BeforeClass; import org.junit.Test; @@ -125,5 +130,58 @@ public void regainRangesTest() throws Throwable }); } + @Test + public void querySameRangeOnDifferentCommandStoresTest() throws Throwable + { + List ddls = Arrays.asList("DROP KEYSPACE IF EXISTS " + KEYSPACE + ';', + "CREATE KEYSPACE " + KEYSPACE + " WITH REPLICATION={'class':'SimpleStrategy', 'replication_factor': 3}", + "CREATE TABLE " + qualifiedAccordTableName + " (k int PRIMARY KEY, v int) WITH transactional_mode='full'"); + test(ddls, cluster -> { + cluster.coordinator(2).execute(wrapInTxn("INSERT INTO " + qualifiedAccordTableName + " (k, v) VALUES (?, ?)"), ConsistencyLevel.SERIAL, 1, 2); + SimpleQueryResult result = cluster.coordinator(2).executeWithResult("SELECT token(k) FROM " + qualifiedAccordTableName + " WHERE k = 1 LIMIT 1", ConsistencyLevel.SERIAL); + + String originalToken = cluster.get(2).callOnInstance(() -> getOnlyElement(StorageService.instance.getTokens())); + + long token = (Long) result.toObjectArrays()[0][0]; + + assert(token < Long.parseLong(originalToken)); + + long epoch = cluster.get(2).callOnInstance(() -> { + long priorEpoch = AccordService.instance().topology().epoch(); + StorageService.instance.move(Long.toString(token - 1000)); + return priorEpoch; + }); + + cluster.coordinator(3).execute(wrapInTxn("INSERT INTO " + qualifiedAccordTableName + " (k, v) VALUES (?, ?)"), ConsistencyLevel.SERIAL, 1, 5); + + cluster.get(2).runOnInstance(() -> { + /* This call results in other nodes invoking this */ + StorageService.instance.move(originalToken); + }); + + cluster.get(2).runOnInstance(() -> { + RoutingKey start = TokenKey.parse(TableId.fromString("tid:11"), String.valueOf(token), Murmur3Partitioner.instance); + RoutingKey end = TokenKey.parse(TableId.fromString("tid:11"), originalToken, Murmur3Partitioner.instance); + Ranges regainedRange = Ranges.of(TokenRange.create(start, end)); + + int numberOfShards = AccordService.instance().node().commandStores().all().length; + LargeBitSet bitSet = new LargeBitSet(numberOfShards); + for (int i = 0; i < numberOfShards; i++) + bitSet.set(i); + + CommandStores.ShardHolder[] shardHolders = new CommandStores.ShardHolder[numberOfShards]; + int i = 0; + for (CommandStore commandStore : AccordService.instance().node().commandStores().all()) + { + CommandStores.RangesForEpoch rangesForEpoch = getBlocking(commandStore.submit((PreLoadContext.Empty) () -> "Get rangesForEpoch", SafeCommandStore::ranges)); + shardHolders[i] = new CommandStores.ShardHolder(commandStore, rangesForEpoch); + i += 1; + } + + assertFalse(checkQueryDisjointRangesAcrossCommandStores(AccordService.instance().node().commandStores().overlappingCommandStores(), shardHolders, bitSet, regainedRange)); + }); + + }); + } } From a1edf9f3bba2819ed080f3fb5c7ca4b717b4ae24 Mon Sep 17 00:00:00 2001 From: Alan Wang Date: Tue, 3 Mar 2026 16:47:40 -0800 Subject: [PATCH 5/8] test updates --- .../test/accord/AccordRegainRangesTest.java | 79 +++++-------------- 1 file changed, 20 insertions(+), 59 deletions(-) diff --git a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordRegainRangesTest.java b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordRegainRangesTest.java index ca5359d8b90a..91a44cba1c9d 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordRegainRangesTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordRegainRangesTest.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.Arrays; import java.util.List; +import java.util.Map; import accord.api.RoutingKey; import accord.local.CommandStore; @@ -29,18 +30,24 @@ import accord.local.SafeCommandStore; import accord.primitives.AbstractRanges; import accord.primitives.Ranges; +import accord.topology.Shard; +import accord.topology.Topology; import accord.topology.TopologyException; +import accord.topology.TopologyManager; +import accord.topology.TopologyRetiredException; import accord.utils.LargeBitSet; import org.apache.cassandra.dht.Murmur3Partitioner; import org.apache.cassandra.distributed.api.ConsistencyLevel; import org.apache.cassandra.distributed.api.Feature; import org.apache.cassandra.distributed.api.SimpleQueryResult; +import org.apache.cassandra.schema.Schema; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.service.accord.AccordService; import org.apache.cassandra.service.accord.TokenRange; import org.apache.cassandra.service.accord.api.TokenKey; +import org.apache.cassandra.tcm.ClusterMetadataService; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -83,6 +90,7 @@ public void regainRangesTest() throws Throwable "CREATE TABLE " + qualifiedAccordTableName + " (k int PRIMARY KEY, v int) WITH transactional_mode='full'"); test(ddls, cluster -> { cluster.coordinator(2).execute(wrapInTxn("INSERT INTO " + qualifiedAccordTableName + " (k, v) VALUES (?, ?)"), ConsistencyLevel.SERIAL, 1, 2); + SimpleQueryResult result = cluster.coordinator(2).executeWithResult("SELECT token(k) FROM " + qualifiedAccordTableName + " WHERE k = 1 LIMIT 1", ConsistencyLevel.SERIAL); String originalToken = cluster.get(2).callOnInstance(() -> getOnlyElement(StorageService.instance.getTokens())); @@ -97,23 +105,30 @@ public void regainRangesTest() throws Throwable return priorEpoch; }); - cluster.coordinator(3).execute(wrapInTxn("INSERT INTO " + qualifiedAccordTableName + " (k, v) VALUES (?, ?)"), ConsistencyLevel.SERIAL, 1, 5); - cluster.get(2).runOnInstance(() -> { StorageService.instance.move(originalToken); }); + String tableName = accordTableName; + // Ensure no overlapping safeToRead ranges cluster.get(2).runOnInstance(() -> { - RoutingKey start = TokenKey.parse(TableId.fromString("tid:11"), String.valueOf(token), Murmur3Partitioner.instance); - RoutingKey end = TokenKey.parse(TableId.fromString("tid:11"), originalToken, Murmur3Partitioner.instance); + TableId tid = Schema.instance.getTableMetadata(KEYSPACE, tableName).id(); + RoutingKey start = TokenKey.parse(tid, String.valueOf(token), Murmur3Partitioner.instance); + RoutingKey end = TokenKey.parse(tid, originalToken, Murmur3Partitioner.instance); Ranges regainedRange = Ranges.of(TokenRange.create(start, end)); try { assert (AccordService.instance().topology().active().get(epoch).retired().containsAll(regainedRange)); - } catch (TopologyException e) { + } + catch (TopologyRetiredException ignored) + { + } + catch (TopologyException e) + { assert(false); } + Ranges range = Ranges.EMPTY; for (CommandStore commandStore : AccordService.instance().node().commandStores().all()) { Ranges safeToReadRanges = getBlocking(commandStore.submit((PreLoadContext.Empty) () -> "No overlapping safeToReadRanges", safeCommandStore -> { @@ -129,59 +144,5 @@ public void regainRangesTest() throws Throwable }); }); } - - @Test - public void querySameRangeOnDifferentCommandStoresTest() throws Throwable - { - List ddls = Arrays.asList("DROP KEYSPACE IF EXISTS " + KEYSPACE + ';', - "CREATE KEYSPACE " + KEYSPACE + " WITH REPLICATION={'class':'SimpleStrategy', 'replication_factor': 3}", - "CREATE TABLE " + qualifiedAccordTableName + " (k int PRIMARY KEY, v int) WITH transactional_mode='full'"); - test(ddls, cluster -> { - cluster.coordinator(2).execute(wrapInTxn("INSERT INTO " + qualifiedAccordTableName + " (k, v) VALUES (?, ?)"), ConsistencyLevel.SERIAL, 1, 2); - SimpleQueryResult result = cluster.coordinator(2).executeWithResult("SELECT token(k) FROM " + qualifiedAccordTableName + " WHERE k = 1 LIMIT 1", ConsistencyLevel.SERIAL); - - String originalToken = cluster.get(2).callOnInstance(() -> getOnlyElement(StorageService.instance.getTokens())); - - long token = (Long) result.toObjectArrays()[0][0]; - - assert(token < Long.parseLong(originalToken)); - - long epoch = cluster.get(2).callOnInstance(() -> { - long priorEpoch = AccordService.instance().topology().epoch(); - StorageService.instance.move(Long.toString(token - 1000)); - return priorEpoch; - }); - - cluster.coordinator(3).execute(wrapInTxn("INSERT INTO " + qualifiedAccordTableName + " (k, v) VALUES (?, ?)"), ConsistencyLevel.SERIAL, 1, 5); - - cluster.get(2).runOnInstance(() -> { - /* This call results in other nodes invoking this */ - StorageService.instance.move(originalToken); - }); - - cluster.get(2).runOnInstance(() -> { - RoutingKey start = TokenKey.parse(TableId.fromString("tid:11"), String.valueOf(token), Murmur3Partitioner.instance); - RoutingKey end = TokenKey.parse(TableId.fromString("tid:11"), originalToken, Murmur3Partitioner.instance); - Ranges regainedRange = Ranges.of(TokenRange.create(start, end)); - - int numberOfShards = AccordService.instance().node().commandStores().all().length; - LargeBitSet bitSet = new LargeBitSet(numberOfShards); - for (int i = 0; i < numberOfShards; i++) - bitSet.set(i); - - CommandStores.ShardHolder[] shardHolders = new CommandStores.ShardHolder[numberOfShards]; - int i = 0; - for (CommandStore commandStore : AccordService.instance().node().commandStores().all()) - { - CommandStores.RangesForEpoch rangesForEpoch = getBlocking(commandStore.submit((PreLoadContext.Empty) () -> "Get rangesForEpoch", SafeCommandStore::ranges)); - shardHolders[i] = new CommandStores.ShardHolder(commandStore, rangesForEpoch); - i += 1; - } - - assertFalse(checkQueryDisjointRangesAcrossCommandStores(AccordService.instance().node().commandStores().overlappingCommandStores(), shardHolders, bitSet, regainedRange)); - }); - - }); - } } From 30e1ba1fb7f7f545e54da569cb6398303889d40c Mon Sep 17 00:00:00 2001 From: Alan Wang Date: Tue, 3 Mar 2026 16:48:48 -0800 Subject: [PATCH 6/8] fix git modules --- .gitmodules | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.gitmodules b/.gitmodules index 8b3913ed184d..616dacf610a7 100644 --- a/.gitmodules +++ b/.gitmodules @@ -1,4 +1,4 @@ [submodule "modules/accord"] path = modules/accord - url = https://github.com/alanwang67/cassandra-accord.git - branch = CASSANDRA-21183 + url = https://github.com/apache/cassandra-accord.git + branch = trunk From 3ab3887ff60777c964140eb920e7d18aab9b5e7e Mon Sep 17 00:00:00 2001 From: Benedict Elliott Smith Date: Fri, 17 Apr 2026 16:40:14 +0100 Subject: [PATCH 7/8] feedback/edits for regaining ranges --- .../src/main/java/accord/api/Journal.java | 185 +++ .../accord/impl/AbstractSafeCommandStore.java | 308 ++++ .../main/java/accord/local/CommandStore.java | 1321 +++++++++++++++++ .../main/java/accord/local/CommandStores.java | 1219 +++++++++++++++ .../src/main/java/accord/local/Node.java | 886 +++++++++++ .../OverlappingCommandStoresException.java | 39 + .../java/accord/local/SafeCommandStore.java | 676 +++++++++ .../java/accord/topology/TopologyManager.java | 590 ++++++++ .../impl/basic/DelayedCommandStores.java | 502 +++++++ 9 files changed, 5726 insertions(+) create mode 100644 accord-core/src/main/java/accord/api/Journal.java create mode 100644 accord-core/src/main/java/accord/impl/AbstractSafeCommandStore.java create mode 100644 accord-core/src/main/java/accord/local/CommandStore.java create mode 100644 accord-core/src/main/java/accord/local/CommandStores.java create mode 100644 accord-core/src/main/java/accord/local/Node.java create mode 100644 accord-core/src/main/java/accord/local/OverlappingCommandStoresException.java create mode 100644 accord-core/src/main/java/accord/local/SafeCommandStore.java create mode 100644 accord-core/src/main/java/accord/topology/TopologyManager.java create mode 100644 accord-core/src/test/java/accord/impl/basic/DelayedCommandStores.java diff --git a/accord-core/src/main/java/accord/api/Journal.java b/accord-core/src/main/java/accord/api/Journal.java new file mode 100644 index 000000000000..cf9986c61fa3 --- /dev/null +++ b/accord-core/src/main/java/accord/api/Journal.java @@ -0,0 +1,185 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package accord.api; + +import java.util.List; +import java.util.NavigableMap; +import java.util.Objects; +import java.util.function.Supplier; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + +import accord.impl.CommandChange; +import accord.local.Command; +import accord.local.CommandStores; +import accord.local.CommandStores.PreviouslyOwned; +import accord.local.DurableBefore; +import accord.local.Node; +import accord.local.RedundantBefore; +import accord.primitives.EpochSupplier; +import accord.primitives.Ranges; +import accord.primitives.Timestamp; +import accord.primitives.TxnId; +import accord.topology.Topology; +import accord.utils.Invariants; +import accord.utils.PersistentField.Persister; +import accord.utils.async.AsyncChain; +import org.agrona.collections.Int2ObjectHashMap; + +/** + * Persisted journal for transactional recovery. + */ +public interface Journal +{ + enum Load + { + ALL, + MINIMAL, + MINIMAL_WITH_DEPS + } + + void start(Node node); + + Command loadCommand(int store, TxnId txnId, RedundantBefore redundantBefore, DurableBefore durableBefore); + default List> debugCommand(int commandStoreId, TxnId txnId) { throw new UnsupportedOperationException(); } + Command.Minimal loadMinimal(int commandStoreId, TxnId txnId, RedundantBefore redundantBefore, DurableBefore durableBefore); + Command.MinimalWithDeps loadMinimalWithDeps(int store, TxnId txnId, RedundantBefore redundantBefore, DurableBefore durableBefore); + void saveCommand(int store, CommandUpdate value, Runnable onFlush); + + List replayTopologies(); + void saveTopology(TopologyUpdate topologyUpdate, Runnable onFlush); + + void purge(CommandStores commandStores, EpochSupplier minEpoch); + + /** + * Replays all messages from journal to rehydrate CommandStores state. Returns whether it has seen (and ignored) + * any exceptions during replay. + */ + boolean replay(CommandStores commandStores); + + RedundantBefore loadRedundantBefore(int store); + NavigableMap loadBootstrapBeganAt(int store); + NavigableMap loadSafeToRead(int store); + CommandStores.RangesForEpoch loadRangesForEpoch(int store); + + Persister durableBeforePersister(); + + void saveStoreState(int store, FieldUpdates fieldUpdates, Runnable onFlush); + + class TopologyUpdate + { + public final Int2ObjectHashMap commandStores; + public final Topology global; + public final PreviouslyOwned previouslyOwned; + + public TopologyUpdate(@Nonnull Int2ObjectHashMap commandStores, @Nonnull Topology global, PreviouslyOwned previouslyOwned) + { + this.commandStores = commandStores; + this.global = global; + this.previouslyOwned = previouslyOwned; + } + + public boolean isEquivalent(TopologyUpdate other) + { + boolean equivalent = global.isEquivalent(other.global); + if (!equivalent) + return false; + Invariants.require(commandStores.equals(other.commandStores)); + return true; + } + + public TopologyUpdate cloneWithEquivalentEpoch(long epoch) + { + return new TopologyUpdate(commandStores, global.cloneEquivalentWithEpoch(epoch), previouslyOwned); + } + + @Override + public boolean equals(Object object) + { + if (this == object) return true; + if (object == null || getClass() != object.getClass()) return false; + TopologyUpdate update = (TopologyUpdate) object; + return Objects.equals(commandStores, update.commandStores) && Objects.equals(global, update.global); + } + + @Override + public int hashCode() + { + return Objects.hash(commandStores, global); + } + + @Override + public String toString() + { + return "TopologyUpdate{" + + "commandStores=" + commandStores + + ", global=" + global + + '}'; + } + } + + class CommandUpdate + { + public final TxnId txnId; + public final Command before; + public final Command after; + + public CommandUpdate(@Nullable Command before, @Nonnull Command after) + { + this.txnId = after.txnId(); + this.before = before; + this.after = after; + } + } + + class FieldUpdates + { + public RedundantBefore newRedundantBefore; + public NavigableMap newBootstrapBeganAt; + public NavigableMap newSafeToRead; + public Ranges newPermanentlyUnsafeToRead; + public CommandStores.RangesForEpoch newRangesForEpoch; + + public String toString() + { + StringBuilder builder = new StringBuilder("FieldUpdates{"); + if (newRedundantBefore != null) + builder.append("newRedundantBefore=").append(newRedundantBefore).append(", "); + if (newBootstrapBeganAt != null) + builder.append("newBootstrapBeganAt=").append(newBootstrapBeganAt).append(", "); + if (newSafeToRead != null) + builder.append("newSafeToRead=").append(newSafeToRead).append(", "); + if (newPermanentlyUnsafeToRead != null) + builder.append("newPermanentlyUnsafeToRead=").append(newPermanentlyUnsafeToRead).append(", "); + if (newRangesForEpoch != null) + builder.append("newRangesForEpoch=").append(newRangesForEpoch).append(", "); + builder.setLength(builder.length() - 2); + builder.append('}'); + return builder.toString(); + } + } + + /** + * Helper for CommandStore to restore Command states. + */ + interface Replayer + { + AsyncChain replay(TxnId txnId); + } +} diff --git a/accord-core/src/main/java/accord/impl/AbstractSafeCommandStore.java b/accord-core/src/main/java/accord/impl/AbstractSafeCommandStore.java new file mode 100644 index 000000000000..b684575126d3 --- /dev/null +++ b/accord-core/src/main/java/accord/impl/AbstractSafeCommandStore.java @@ -0,0 +1,308 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package accord.impl; + +import java.util.ArrayList; +import java.util.List; +import java.util.NavigableMap; + +import accord.api.RoutingKey; +import accord.local.CommandStore; +import accord.local.LoadKeys; +import accord.local.PreLoadContext; +import accord.local.RedundantBefore; +import accord.local.SafeCommand; +import accord.local.SafeCommandStore; +import accord.local.cfk.SafeCommandsForKey; +import accord.primitives.Ranges; +import accord.primitives.Routable; +import accord.primitives.RoutingKeys; +import accord.primitives.Timestamp; +import accord.primitives.TxnId; +import accord.primitives.Unseekables; + +import static accord.api.Journal.FieldUpdates; +import static accord.local.CommandStores.RangesForEpoch; + +public abstract class AbstractSafeCommandStore> +extends SafeCommandStore +{ + protected final PreLoadContext context; + + private final CommandStore commandStore; + private FieldUpdates fieldUpdates; + + protected AbstractSafeCommandStore(PreLoadContext context, CommandStore commandStore) + { + this.context = context; + this.commandStore = commandStore; + } + + @Override + public CommandStore commandStore() + { + return commandStore; + } + + public interface CommandStoreCaches extends AutoCloseable + { + void close(); + + C acquireIfLoaded(TxnId txnId); + CFK acquireIfLoaded(RoutingKey key); + } + + protected abstract Caches tryGetCaches(); + protected abstract C add(C safeCommand, Caches caches); + protected abstract CFK add(CFK safeCfk, Caches caches); + + @Override + public PreLoadContext canExecute(PreLoadContext with) + { + if (with.isEmpty()) return with; + if (with.keys().domain() == Routable.Domain.Range) + return with.isSubsetOf(this.context) ? with : null; + + LoadKeys require = with.loadKeys(); + if (require != LoadKeys.NONE) + { + PreLoadContext context = context(); + if (!context.loadKeys().satisfiesIfPresent(require)) + return null; + + if (with.loadKeysFor().compareTo(context.loadKeysFor()) > 0) + return null; + } + + try (Caches caches = tryGetCaches()) + { + for (TxnId txnId : with.txnIds()) + { + if (null != getInternal(txnId)) + continue; + + if (caches == null) + return null; + + C safeCommand = caches.acquireIfLoaded(txnId); + if (safeCommand == null) + return null; + + add(safeCommand, caches); + } + + LoadKeys loadKeys = with.loadKeys(); + if (loadKeys == LoadKeys.NONE) + return with; + + List unavailable = null; + Unseekables keys = with.keys(); + if (keys.isEmpty()) + return with; + + for (int i = 0 ; i < keys.size() ; ++i) + { + RoutingKey key = (RoutingKey) keys.get(i); + if (null != getInternal(key)) + continue; // already in working set + + if (caches != null) + { + CFK safeCfk = caches.acquireIfLoaded(key); + if (safeCfk != null) + { + add(safeCfk, caches); + continue; + } + } + if (unavailable == null) + unavailable = new ArrayList<>(); + unavailable.add(key); + } + + if (unavailable == null) + return with; + + if (unavailable.size() == keys.size()) + return null; + + return PreLoadContext.contextFor(with.primaryTxnId(), with.additionalTxnId(), keys.without(RoutingKeys.ofSortedUnique(unavailable)), loadKeys, context.loadKeysFor(), context.reason()); + } + } + + @Override + public PreLoadContext context() + { + return context; + } + + @Override + protected C ifLoadedInternal(TxnId txnId) + { + try (Caches caches = tryGetCaches()) + { + if (caches == null) + return null; + + C command = caches.acquireIfLoaded(txnId); + if (command == null) + return null; + + return add(command, caches); + } + } + + @Override + protected CFK ifLoadedInternal(RoutingKey txnId) + { + try (Caches caches = tryGetCaches()) + { + if (caches == null) + return null; + + CFK cfk = caches.acquireIfLoaded(txnId); + if (cfk == null) + return null; + + return add(cfk, caches); + } + } + + // TODO (expected): cleanup the integration hooks here; they're a bit byzantine. Also clearly document behaviour. + public void postExecute() + { + flushFieldUpdates(); + } + + protected void persistFieldUpdates() + { + flushFieldUpdates(); + } + + protected void flushFieldUpdates() + { + if (fieldUpdates == null) + return; + + if (fieldUpdates.newRedundantBefore != null) + super.unsafeSetRedundantBefore(fieldUpdates.newRedundantBefore); + + if (fieldUpdates.newBootstrapBeganAt != null) + super.setBootstrapBeganAt(fieldUpdates.newBootstrapBeganAt); + + if (fieldUpdates.newSafeToRead != null) + super.setSafeToRead(fieldUpdates.newSafeToRead); + + if (fieldUpdates.newRangesForEpoch != null) + super.setRangesForEpoch(fieldUpdates.newRangesForEpoch); + + fieldUpdates = null; + } + + /** + * Persistent field update logic + */ + + @Override + public final void upsertRedundantBefore(RedundantBefore addRedundantBefore) + { + // TODO (required): this is potentially unsafe: if the update is not persisted for some reason (due to some later exception) + // we can continue with a stale redundantBefore + // TODO (expected): fix RedundantBefore sorting issue and switch to upsert mode + ensureFieldUpdates().newRedundantBefore = RedundantBefore.merge(redundantBefore(), addRedundantBefore); + unsafeUpsertRedundantBefore(addRedundantBefore); + } + + @Override + public final void setBootstrapBeganAt(NavigableMap newBootstrapBeganAt) + { + ensureFieldUpdates().newBootstrapBeganAt = newBootstrapBeganAt; + } + + @Override + public final void setSafeToRead(NavigableMap newSafeToRead) + { + ensureFieldUpdates().newSafeToRead = newSafeToRead; + } + + @Override + public final void setPermanentlyUnsafeToRead(Ranges newPermanentlyUnsafeToRead) + { + ensureFieldUpdates().newPermanentlyUnsafeToRead = newPermanentlyUnsafeToRead; + } + + @Override + public void setRangesForEpoch(RangesForEpoch rangesForEpoch) + { + if (rangesForEpoch != null) + { + super.setRangesForEpoch(rangesForEpoch); + ensureFieldUpdates().newRangesForEpoch = rangesForEpoch; + } + } + + @Override + public RangesForEpoch ranges() + { + if (fieldUpdates != null && fieldUpdates.newRangesForEpoch != null) + return fieldUpdates.newRangesForEpoch; + + return commandStore.unsafeGetRangesForEpoch(); + } + + @Override + public NavigableMap bootstrapBeganAt() + { + if (fieldUpdates != null && fieldUpdates.newBootstrapBeganAt != null) + return fieldUpdates.newBootstrapBeganAt; + + return super.bootstrapBeganAt(); + } + + @Override + public NavigableMap safeToReadAt() + { + if (fieldUpdates != null && fieldUpdates.newSafeToRead != null) + return fieldUpdates.newSafeToRead; + + return super.safeToReadAt(); + } + + @Override + public RedundantBefore redundantBefore() + { + if (fieldUpdates != null && fieldUpdates.newRedundantBefore != null) + return fieldUpdates.newRedundantBefore; + + return super.redundantBefore(); + } + + private FieldUpdates ensureFieldUpdates() + { + if (fieldUpdates == null) fieldUpdates = new FieldUpdates(); + return fieldUpdates; + } + + public FieldUpdates fieldUpdates() + { + return fieldUpdates; + } +} diff --git a/accord-core/src/main/java/accord/local/CommandStore.java b/accord-core/src/main/java/accord/local/CommandStore.java new file mode 100644 index 000000000000..83966f192825 --- /dev/null +++ b/accord-core/src/main/java/accord/local/CommandStore.java @@ -0,0 +1,1321 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package accord.local; + +import java.util.AbstractMap.SimpleImmutableEntry; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BiConsumer; +import java.util.function.Consumer; +import java.util.function.Function; +import java.util.function.Supplier; +import javax.annotation.Nullable; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableSortedMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import accord.api.Agent; +import accord.topology.EpochReady; +import accord.api.DataStore; +import accord.api.DataStore.FetchKind; +import accord.api.Journal; +import accord.api.LocalListeners; +import accord.api.ProgressLog; +import accord.impl.AbstractAsyncExecutor; +import accord.coordinate.CoordinateMaxConflict; +import accord.local.CommandStores.BootstrapRangeAction; +import accord.local.CommandStores.RangesForEpoch; +import accord.local.Commands.NotifyWaitingOnPlus; +import accord.local.PreLoadContext.Empty; +import accord.local.RedundantBefore.Bounds; +import accord.local.RedundantStatus.SomeStatus; +import accord.primitives.Ranges; +import accord.primitives.Routables; +import accord.primitives.SaveStatus; +import accord.primitives.Status; +import accord.primitives.Status.Durability.HasOutcome; +import accord.primitives.Timestamp; +import accord.primitives.TxnId; +import accord.primitives.Unseekables; +import accord.utils.DeterministicIdentitySet; +import accord.utils.Invariants; +import accord.utils.Reduce; +import accord.utils.UnhandledEnum; +import accord.utils.async.AsyncChain; +import accord.utils.async.AsyncChains; +import accord.utils.async.AsyncResult; +import accord.utils.async.AsyncResults; +import accord.utils.async.AsyncResults.SettableByCallback; +import accord.utils.async.AsyncResults.SettableWithDescription; +import accord.utils.async.Cancellable; +import accord.utils.async.AsyncResults.SettableResult; +import org.agrona.collections.LongHashSet; + +import static accord.topology.EpochReady.DONE; +import static accord.topology.EpochReady.done; +import static accord.api.DataStore.FetchKind.Image; +import static accord.api.ProtocolModifiers.Toggles.requiresUniqueHlcs; +import static accord.local.RedundantStatus.SomeStatus.GC_BEFORE_AND_LOCALLY_DURABLE; +import static accord.local.RedundantStatus.SomeStatus.LOCALLY_APPLIED_ONLY; +import static accord.local.RedundantStatus.SomeStatus.LOCALLY_DURABLE_TO_COMMAND_STORE_ONLY; +import static accord.local.RedundantStatus.SomeStatus.LOCALLY_DURABLE_TO_DATA_STORE_ONLY; +import static accord.local.RedundantStatus.SomeStatus.LOCALLY_WITNESSED_ONLY; +import static accord.local.RedundantStatus.SomeStatus.LOG_UNAVAILABLE_ONLY; +import static accord.local.RedundantStatus.SomeStatus.QUORUM_APPLIED_ONLY; +import static accord.local.RedundantStatus.SomeStatus.UNREADY_ONLY; +import static accord.local.RedundantStatus.SomeStatus.SHARD_APPLIED_ONLY; +import static accord.primitives.AbstractRanges.UnionMode.MERGE_ADJACENT; +import static accord.primitives.Routables.Slice.Minimal; +import static accord.primitives.Timestamp.Flag.HLC_BOUND; +import static accord.primitives.Txn.Kind.VisibilitySyncPoint; +import static accord.utils.Invariants.nonNull; + +/** + * Single threaded internal shard of accord transaction metadata + */ +public abstract class CommandStore implements AbstractAsyncExecutor, SequentialAsyncExecutor +{ + private static final Logger logger = LoggerFactory.getLogger(CommandStore.class); + + public static class EpochUpdate + { + public final RangesForEpoch newRangesForEpoch; + public final RedundantBefore addRedundantBefore; + + EpochUpdate(RangesForEpoch newRangesForEpoch, RedundantBefore addRedundantBefore) + { + this.newRangesForEpoch = newRangesForEpoch; + this.addRedundantBefore = addRedundantBefore; + } + } + + // TODO (required): we only REMOVE ranges now, so it should be possible to simplify this + public static class EpochUpdateHolder extends AtomicReference + { + // TODO (desired): can better encapsulate by accepting only the newRangesForEpoch and deriving the add/remove ranges + public void add(long epoch, RangesForEpoch newRangesForEpoch, Ranges addRanges) + { + RedundantBefore addRedundantBefore = RedundantBefore.create(addRanges, epoch, Long.MAX_VALUE, TxnId.minForEpoch(epoch), UNREADY_ONLY); + update(newRangesForEpoch, addRedundantBefore); + } + + public void remove(long epoch, RangesForEpoch newRangesForEpoch, Ranges removeRanges) + { + RedundantBefore addRedundantBefore = RedundantBefore.create(removeRanges, Long.MIN_VALUE, epoch, TxnId.NONE, SomeStatus.NONE); + update(newRangesForEpoch, addRedundantBefore); + } + + private void update(RangesForEpoch newRangesForEpoch, RedundantBefore addRedundantBefore) + { + EpochUpdate baseUpdate = new EpochUpdate(newRangesForEpoch, addRedundantBefore); + EpochUpdate cur = get(); + if (cur == null || !compareAndSet(cur, new EpochUpdate(newRangesForEpoch, RedundantBefore.merge(cur.addRedundantBefore, addRedundantBefore)))) + set(baseUpdate); + } + } + + public interface Factory + { + CommandStore create(int id, + NodeCommandStoreService node, + Agent agent, + DataStore store, + ProgressLog.Factory progressLogFactory, + LocalListeners.Factory listenersFactory, + EpochUpdateHolder rangesForEpoch, + Journal journal); + } + + protected final int id; + protected final NodeCommandStoreService node; + protected final Agent agent; + protected final DataStore dataStore; + protected final ProgressLog progressLog; + protected final LocalListeners listeners; + protected final EpochUpdateHolder epochUpdateHolder; + + // Used in markShardStale to make sure the staleness includes in progress bootstraps + // TODO (desired): migrate to BTree + private transient NavigableMap bootstrapBeganAt = emptyBootstrapBeganAt(); // additive (i.e. once inserted, rolled-over until invalidated, and the floor entry contains additions) + protected boolean hasResumedBootstraps; + private RedundantBefore redundantBefore = RedundantBefore.EMPTY; + private MaxConflicts maxConflicts = MaxConflicts.EMPTY; + private MaxDecidedRX maxDecidedRX = MaxDecidedRX.EMPTY; + private int maxConflictsUpdates = 0; + protected RangesForEpoch rangesForEpoch; + protected @Nullable Ranges refuses; + List syncPointListeners; + + /** + * safeToRead is related to RedundantBefore, but a distinct concept. + * While readyAt defines the txnId bounds we expect to maintain data for locally, + * safeToRead defines executeAt bounds we can safely participate in transaction execution for. + * safeToRead is defined by the no-op transaction we execute after a bootstrap is initiated, + * and creates a global bound before which we know we have complete data from our bootstrap. + * + * There's a smearing period during bootstrap where some keys may be ahead of others, for instance, + * since we do not create a precise instant in the transaction log for bootstrap to avoid impeding execution. + * + * We also update safeToRead when we go stale, to remove ranges we may have bootstrapped but that are now known to + * be incomplete. In this case we permit transactions to execute in any order for the unsafe key ranges. + * But they may still be ordered for other key ranges they participate in. + */ + private NavigableMap safeToRead = emptySafeToRead(); + private Ranges permanentlyUnsafeToRead = Ranges.EMPTY; + private final Set bootstraps = Collections.synchronizedSet(new DeterministicIdentitySet<>()); + @Nullable private RejectBefore rejectBefore; + + static class WaitingOnVisibility + { + final SettableResult whenDone; + final Ranges allRanges; + Ranges waitingOn, waitingOnDurable; + + WaitingOnVisibility(SettableResult whenDone, Ranges ranges) + { + this.whenDone = whenDone; + this.allRanges = this.waitingOn = this.waitingOnDurable = ranges; + } + } + private final TreeMap waitingOnVisibility = new TreeMap<>(); + + protected CommandStore(int id, + NodeCommandStoreService node, + Agent agent, + DataStore dataStore, + ProgressLog.Factory progressLogFactory, + LocalListeners.Factory listenersFactory, + EpochUpdateHolder epochUpdateHolder) + { + this.id = id; + this.node = node; + this.agent = agent; + this.dataStore = dataStore; + this.progressLog = progressLogFactory.create(this); + this.listeners = listenersFactory.create(this); + this.epochUpdateHolder = epochUpdateHolder; + } + + public final int id() + { + return id; + } + + public void restore() {}; + + public abstract Journal.Replayer replayer(); + // expected to invoke safeStore.upsertRedundantBefore at some future point, when the commandStore state is durably persisted + protected abstract void ensureDurable(Ranges ranges, RedundantBefore onCommandStoreDurable); + + public Agent agent() + { + return agent; + } + + public void unsafeClearForTesting() + { + progressLog.clear(); + bootstraps.clear(); + rangesForEpoch = null; + bootstrapBeganAt = emptyBootstrapBeganAt(); + redundantBefore = RedundantBefore.EMPTY; + maxConflicts = MaxConflicts.EMPTY; + maxDecidedRX = MaxDecidedRX.EMPTY; + safeToRead = emptySafeToRead(); + listeners.clear(); + waitingOnVisibility.clear(); + } + + public void updateRangesForEpoch(SafeCommandStore safeStore) + { + EpochUpdate update = epochUpdateHolder.get(); + if (update == null) + return; + + update = epochUpdateHolder.getAndSet(null); + if (update.addRedundantBefore.size() > 0) + safeStore.upsertRedundantBefore(update.addRedundantBefore); + if (update.newRangesForEpoch != null) + safeStore.setRangesForEpoch(update.newRangesForEpoch); + + safeStore.persistFieldUpdates(); + } + + @VisibleForTesting + public void unsafeUpdateRangesForEpoch() + { + EpochUpdate update = epochUpdateHolder.getAndSet(null); + if (update == null) + return; + + if (update.addRedundantBefore.size() > 0) + unsafeUpsertRedundantBefore(update.addRedundantBefore); + if (update.newRangesForEpoch != null) + unsafeSetRangesForEpoch(update.newRangesForEpoch); + } + + public RangesForEpoch unsafeGetRangesForEpoch() + { + return rangesForEpoch; + } + + public MaxDecidedRX unsafeGetMaxDecidedRX() + { + return maxDecidedRX; + } + + @VisibleForTesting + public final void unsafeSetRangesForEpoch(RangesForEpoch newRangesForEpoch) + { + rangesForEpoch = nonNull(newRangesForEpoch); + } + + protected final void unsafeClearRangesForEpoch() + { + rangesForEpoch = null; + } + + protected void loadRangesForEpoch(RangesForEpoch newRangesForEpoch) + { + Invariants.require(this.rangesForEpoch == null); + unsafeSetRangesForEpoch(newRangesForEpoch); + } + + public abstract boolean inStore(); + + public boolean tryExecuteImmediately(Runnable run) + { + if (!inStore()) + return false; + + try { run.run(); } + catch (Throwable t) { agent.onException(t); } + return true; + } + + public abstract AsyncChain chain(PreLoadContext context, Consumer consumer); + public abstract AsyncChain chain(PreLoadContext context, Function apply); + + public Cancellable execute(PreLoadContext context, Consumer consumer, BiConsumer callback) + { + return chain(context, consumer).begin(callback); + } + + public AsyncResult execute(PreLoadContext context, Consumer consumer) + { + return chain(context, consumer).beginAsResult(); + } + + public Cancellable execute(PreLoadContext context, Function apply, BiConsumer callback) + { + return chain(context, apply).begin(callback); + } + + public AsyncResult submit(PreLoadContext context, Function apply) + { + return chain(context, apply).beginAsResult(); + } + + public abstract void shutdown(); + + protected void unsafeSetMaxDecidedRX(MaxDecidedRX newMaxDecidedRX) + { + this.maxDecidedRX = newMaxDecidedRX; + } + + protected void unsafeSetRejectBefore(RejectBefore newRejectBefore) + { + this.rejectBefore = newRejectBefore; + } + + final void unsafeSetRedundantBefore(RedundantBefore newRedundantBefore) + { + redundantBefore = newRedundantBefore; + } + + protected void unsafeClearRedundantBefore() + { + unsafeSetRedundantBefore(null); + } + + protected void loadRedundantBefore(RedundantBefore newRedundantBefore) + { + Invariants.require(redundantBefore == null || redundantBefore.equals(RedundantBefore.EMPTY)); + Invariants.require(newRedundantBefore != null); + unsafeSetRedundantBefore(newRedundantBefore); + } + + protected void unsafeUpsertRedundantBefore(RedundantBefore addRedundantBefore) + { + unsafeSetRedundantBefore(RedundantBefore.merge(redundantBefore, addRedundantBefore)); + } + + @VisibleForTesting + public boolean unsafeIsRefusingAny() + { + return refuses != null; + } + + protected void unsafeRefuseRequests(Ranges refuse) + { + Invariants.require(refuses == null || !refuses.intersects(refuse)); + if (refuses == null) refuses = refuse; + else refuses = refuses.with(refuse); + } + + protected void unsafeAcceptRequests(Ranges accept) + { + Invariants.require(refuses != null && refuses.containsAll(accept)); + refuses = refuses.without(accept); + if (refuses.isEmpty()) + refuses = null; + } + + /** + * This method may be invoked on a non-CommandStore thread + */ + final void unsafeSetSafeToRead(NavigableMap newSafeToRead) + { + if (newSafeToRead != null) + { + for (Map.Entry entry : newSafeToRead.entrySet()) + { + Ranges rangeExcluded = entry.getValue().without(this.permanentlyUnsafeToRead); + logger.info("{} is excluded from newSafeToRead because it is in the regained ranges", rangeExcluded); + } + } + + node.updateStamp(); + this.safeToRead = newSafeToRead; + } + + final void unsafeSetPermanentlyUnsafeToRead(Ranges newPermanentlyUnsafeToRead) + { + this.permanentlyUnsafeToRead = newPermanentlyUnsafeToRead; + } + + protected final void unsafeClearSafeToRead() + { + unsafeSetSafeToRead(null); + } + + protected void loadSafeToRead(NavigableMap newSafeToRead) + { + Invariants.require(safeToRead == null || safeToRead.equals(emptySafeToRead())); + Invariants.require(newSafeToRead != null); + unsafeSetSafeToRead(newSafeToRead); + updateMaxConflicts(newSafeToRead); + } + + final void unsafeSetBootstrapBeganAt(NavigableMap newBootstrapBeganAt) + { + this.bootstrapBeganAt = newBootstrapBeganAt; + } + + protected final void unsafeClearBootstrapBeganAt() + { + unsafeSetBootstrapBeganAt(null); + } + + protected synchronized void loadBootstrapBeganAt(NavigableMap newBootstrapBeganAt) + { + Invariants.require(bootstrapBeganAt == null || bootstrapBeganAt.equals(emptyBootstrapBeganAt())); + Invariants.require(newBootstrapBeganAt != null); + unsafeSetBootstrapBeganAt(newBootstrapBeganAt); + updateMaxConflicts(newBootstrapBeganAt); + } + + /** + * To be overridden by implementations, to ensure the new state is persisted. + */ + protected void setMaxConflicts(MaxConflicts maxConflicts) + { + this.maxConflicts = maxConflicts; + } + + protected int dumpCounter = 0; + + protected void updateMaxConflicts(Command prev, Command updated, boolean force) + { + Timestamp executeAt = updated.executeAt(); + if (executeAt == null) return; + if (prev != null && prev.executeAt() != null && prev.executeAt().compareToStrict(executeAt) >= 0 && !force) return; + executeAt = executeAt.flattenUniqueHlc(); // this is what guarantees a bootstrap recipient can compute uniqueHlc safely + MaxConflicts updatedMaxConflicts = maxConflicts.update(updated.participants().hasTouched(), executeAt); + updateMaxConflicts(executeAt, updatedMaxConflicts); + } + + protected void updateMaxConflicts(Ranges ranges, Timestamp executeAt) + { + updateMaxConflicts(executeAt, maxConflicts.update(ranges, executeAt)); + } + + protected void updateMaxConflicts(NavigableMap map) + { + Timestamp max = Timestamp.NONE; + MaxConflicts updated = maxConflicts; + for (Map.Entry e : map.entrySet()) + { + Timestamp at = e.getKey(); + if (at.compareTo(Timestamp.NONE) > 0) + { + updated = updated.update(e.getValue(), at); + max = Timestamp.max(max, at); + } + } + if (updated != maxConflicts) + updateMaxConflicts(max, updated); + } + + protected void updateMaxConflicts(Timestamp executeAt, MaxConflicts updatedMaxConflicts) + { + if (++maxConflictsUpdates >= agent.maxConflictsPruneInterval()) + { + int initialSize = updatedMaxConflicts.size(); + MaxConflicts initialConflicts = updatedMaxConflicts; + long pruneHlc = executeAt.hlc() - agent.maxConflictsHlcPruneDelta(); + Timestamp pruneBefore = pruneHlc > 0 ? Timestamp.fromValues(executeAt.epoch(), pruneHlc, executeAt.node) : null; + Ranges ranges = rangesForEpoch.all(); + if (pruneBefore != null) + updatedMaxConflicts = updatedMaxConflicts.update(ranges, pruneBefore); + + int prunedSize = updatedMaxConflicts.size(); + if (initialSize > 100 && prunedSize == initialSize) + { + logger.debug("Ineffective prune for {}. Initial size: {}, pruned size: {}, executeAt: {}, pruneBefore: {}", ranges, initialSize, prunedSize, executeAt, pruneBefore); + if (dumpCounter == 0) + { + logger.trace("initial MaxConflicts dump: {}", initialConflicts); + logger.trace("pruned MaxConflicts dump: {}", updatedMaxConflicts); + } + dumpCounter++; + dumpCounter %= 100; + } + else if (prunedSize != initialSize) + { + logger.trace("Successfully pruned {} to {}", initialSize, prunedSize); + } + + + maxConflictsUpdates = 0; + } + setMaxConflicts(updatedMaxConflicts); + } + + final void markExclusiveSyncPoint(SafeCommandStore safeStore, TxnId txnId, Ranges ranges) + { + // TODO (desired): narrow ranges to those that are owned + Invariants.requireArgument(txnId.isSyncPoint()); + RejectBefore newRejectBefore = rejectBefore != null ? rejectBefore : new RejectBefore(); + newRejectBefore = RejectBefore.add(newRejectBefore, ranges, txnId); + unsafeSetRejectBefore(newRejectBefore); + } + + final void markExclusiveSyncPointDecided(SafeCommandStore safeStore, TxnId txnId, Ranges ranges) + { + unsafeSetMaxDecidedRX(maxDecidedRX.update(ranges, txnId)); + } + + final void markExclusiveSyncPointLocallyApplied(SafeCommandStore safeStore, TxnId txnId, Ranges ranges) + { + // TODO (desired): narrow ranges to those that are owned + Invariants.requireArgument(txnId.isSyncPoint()); + RedundantBefore addNow = RedundantBefore.create(ranges, txnId, LOCALLY_APPLIED_ONLY); + safeStore.upsertRedundantBefore(addNow); + RedundantBefore addOnDataStoreDurable = RedundantBefore.create(ranges, txnId, LOCALLY_DURABLE_TO_DATA_STORE_ONLY); + RedundantBefore addOnCommandStoreDurable = RedundantBefore.create(ranges, txnId, LOCALLY_DURABLE_TO_COMMAND_STORE_ONLY); + dataStore.ensureDurable(this, ranges, addOnDataStoreDurable); + ensureDurable(ranges, addOnCommandStoreDurable); + } + + /** + * We expect keys to be sliced to those owned by the replica in the coordination epoch + */ + final Timestamp preaccept(TxnId txnId, Routables keys, SafeCommandStore safeStore, boolean permitFastPath) + { + NodeCommandStoreService node = safeStore.node(); + + boolean isExpired = safeStore.agent().rejectPreAccept(safeStore.node(), txnId) && !txnId.isSyncPoint(); + if (rejectBefore != null && !isExpired) + isExpired = rejectBefore.rejects(txnId, keys); + + if (isExpired) + return node.uniqueTimestamp(txnId).asRejected(); + + Timestamp min = TxnId.mergeMax(txnId, maxConflicts.get(keys)); + if (permitFastPath && txnId == min && txnId.epoch() >= node.epoch()) + return txnId; + + return node.uniqueTimestamp(min); + } + + /** + * We expect keys to be sliced to those owned by the replica in the coordination epoch + */ + public final Timestamp maxConflict(Routables keys) + { + return maxConflicts.get(keys); + } + + @Override + public String toString() + { + return getClass().getSimpleName() + "{id=" + id + ", node=" + node.id().id + '}'; + } + + public final AsyncResult cancelBootstraps() + { + return submit((Empty)() -> "Cancel Bootstraps", safeStore -> { + cancelBootstraps(safeStore, safeStore.ranges().all()); + return null; + }); + } + + public final void cancelBootstraps(SafeCommandStore safeStore, Ranges ranges) + { + Invariants.require(safeStore.commandStore() == this && inStore()); + bootstraps.forEach(b -> b.invalidate(ranges)); + } + + public final AsyncResult resumeBootstrap(Node node) + { + synchronized (this) + { + Invariants.require(!hasResumedBootstraps); + hasResumedBootstraps = true; + } + + return submit((Empty)() -> "Resume Bootstrap", safeStore -> { + Ranges unfinished = rangesForEpoch.all(); + unfinished = unfinished.without(safeToRead.lastEntry().getValue()); + unfinished = redundantBefore.removeLostOrStale(unfinished); + for (Bootstrap bootstrap : bootstraps) + unfinished = unfinished.without(bootstrap.all); + + long epoch = rangesForEpoch.epochAtIndex(0); + if (unfinished.isEmpty()) + return done(epoch); + + logger.info("{}: Resuming bootstrap of {}", this, unfinished); + return epochReadyAfterBootstrap(unfinished, epoch, startSafeBootstrapInternal(node, safeStore, unfinished, epoch)); + }); + } + + /** + * Defer submitting the work until we have wired up any changes to topology in memory, then first submit the work + * to setup any state in the command store, and finally submit the distributed work to bootstrap the data locally. + * So, the outer future's success is sufficient for the topology to be acknowledged, and the inner future for the + * bootstrap to be complete. + */ + final Supplier bootstrapper(Node node, Ranges newRanges, long epoch, BootstrapRangeAction action) + { + switch (action) + { + default: throw new UnhandledEnum(action); + case BOOTSTRAP_NOT_NEEDED: + return () -> { + AsyncResult done = execute((Empty) () -> "Initialise New Epoch", (safeStore) -> { + logger.info("{}: Initialising {} for epoch {}", this, newRanges, epoch); + // Merge in a base for any ranges that needs to be covered + Ranges newBootstrapRanges = newRanges; + for (Ranges existing : bootstrapBeganAt.values()) + newBootstrapRanges = newBootstrapRanges.without(existing); + if (!newBootstrapRanges.isEmpty()) + safeStore.setBootstrapBeganAt(bootstrap(TxnId.NONE, newBootstrapRanges, bootstrapBeganAt)); + safeStore.setSafeToRead(purgeAndInsert(safeToRead, TxnId.NONE, newRanges)); + markExclusiveSyncPointDecided(safeStore, TxnId.NONE, newRanges); + }); + + return EpochReady.all(epoch, done); + }; + case SAFE_BOOTSTRAP: + return () -> epochReadyAfterBootstrap(newRanges, epoch, startSafeBootstrap(node, newRanges, epoch)); + + case UNSAFE_BOOTSTRAP: + return () -> epochReadyAfterBootstrap(newRanges, epoch, startUnsafeBootstrap(node, newRanges, epoch, Image)); + } + } + + private EpochReady epochReadyAfterBootstrap(Ranges newRanges, long epoch, AsyncResult bootstrap) + { + return epochReadyAfterBootstrap(newRanges, epoch, EpochReady.wrap(epoch, bootstrap)); + } + + private EpochReady epochReadyAfterBootstrap(Ranges newRanges, long epoch, EpochReady bootstrap) + { + AsyncResult readyToCoordinate = readyToCoordinate(newRanges, epoch); + return new EpochReady(epoch, + bootstrap.active, + readyToCoordinate, + bootstrap.data, + bootstrap.reads); + } + + private AsyncResult startSafeBootstrap(Node node, Ranges newRanges, long epoch) + { + return submit((Empty) () -> "New Epoch", safeStore -> { + return startSafeBootstrapInternal(node, safeStore, newRanges, epoch); + }); + } + + private static final AsyncResult MUST_OVERWRITE = AsyncResults.failure(new IllegalStateException()); + private EpochReady startSafeBootstrapInternal(Node node, SafeCommandStore safeStore, Ranges newRanges, long epoch) + { + logger.info("{}: Starting Safe Bootstrap for {} for epoch {}", this, newRanges, epoch); + Bootstrap bootstrap = new Bootstrap(node, this, epoch, newRanges); + bootstraps.add(bootstrap); + bootstrap.start(safeStore); + return new EpochReady(epoch, + MUST_OVERWRITE, + MUST_OVERWRITE, + bootstrap.data, + bootstrap.reads); + } + + /** + * Rebootstraps some of the ranges for the command store. It follows steps similar to what + * bootstrap would go through, with two differences: + * + * * Marks pre-rebootstrap transactions with LOCALLY_LOST status, which means the node can not + * safely participate in pre-rebootstrap transactions, _even_ if they're coming after the node is + * done bootstrapping. + * * Marks the store as rebootstrapping, which will preclude rebootstrapping node from responding + * to PreAccept, Accept, and BeginRecovery and computing dependencies while node is being rebootstrapped, + * and ranges aren't ready to coordinate. + */ + protected AsyncResult startUnsafeBootstrap(Node node, Ranges ranges, long epoch, FetchKind fetch) + { + return submit((Empty) () -> "Refuse Requests for " + fetch + " Bootstrap", safeStore -> { + unsafeRefuseRequests(ranges); + safeStore.setSafeToRead(purgeHistory(safeToRead, ranges)); + // TODO (expected): rationalise with startSafeBootstrap + String description = "Bootstrap " + ranges + " for epoch " + epoch + " in " + this; + return new EpochReady(epoch, MUST_OVERWRITE, readyToCoordinate(ranges, epoch), new SettableWithDescription<>(description), new SettableWithDescription<>(description)); + }).invoke((success, fail) -> { + if (fail != null) logger.error("Fatal error initiating {} bootstrap for {}", this, fetch, fail); + else rebootstrap(node, ranges, epoch, 1, success, fetch); + }); + } + + private void rebootstrap(Node node, Ranges ranges, long epoch, int attempt, EpochReady ready, FetchKind fetch) + { + CoordinateMaxConflict + .maxConflict(node, ranges) + .recover(failure -> { + Runnable retry = () -> rebootstrap(node, ranges, epoch, attempt + 1, ready, fetch); + Runnable fail = () -> { + ((SettableByCallback)ready.data).tryFailure(failure); + ((SettableByCallback)ready.reads).tryFailure(failure); + }; + agent.ownershipEvents().onFailedBootstrap(attempt, "Fetch Max Conflict (to mark log safe at)", ranges, retry, fail, failure); + return AsyncChains.failure(failure); + }).flatMap(success -> chain((Empty) () -> "Initiate Unsafe " + fetch + " Bootstrap", safeStore -> { + node.uniqueNow(success.hlc()); // ensure we pick a higher timestamp than the maximum conflict we found globally + // Mark unsafe to read first + + Ranges remaining = ranges.slice(rangesForEpoch.currentRanges(), Minimal); + if (remaining.isEmpty()) + { + logger.info("Terminating unsafe {} bootstrap process for {} as no active ranges", fetch, this); + return AsyncChains.success(null); + } + + Bootstrap bootstrap = new Bootstrap(node, this, epoch, remaining, fetch); + bootstraps.add(bootstrap); + // If rebootstrap can grab a later timestamp for subsequent attempts, but this timestamp is enough for us + // to establish which transactions, for which ranges the node can safely participate in). + TxnId unreadyBefore = bootstrap.start(safeStore); + safeStore.unsafeUpsertRedundantBefore(RedundantBefore.create(ranges, unreadyBefore, LOG_UNAVAILABLE_ONLY)); + updateMaxConflicts(ranges, unreadyBefore); + // TODO (desired): we could start accepting non-dep requests here + bootstrap.data.invoke((SettableByCallback)ready.data); + bootstrap.reads.invoke((SettableByCallback)ready.reads); + ready.coordinate.invokeIfSuccess(() -> { + execute((Empty)() -> "Accept Dependency Requests", safeStore0 -> { + unsafeAcceptRequests(remaining); + }); + }); + return null; + })).begin(agent); + } + + /** + * Defer submitting the work until we have wired up any changes to topology in memory, then first submit the work + * to setup any state in the command store, and finally submit the distributed work to bootstrap the data locally. + * So, the outer future's success is sufficient for the topology to be acknowledged, and the inner future for the + * bootstrap to be complete. + */ + protected Supplier refreshReadyToCoordinate(Node node, Ranges ranges, long epoch) + { + return () -> { + AsyncResult readyToCoordinate = readyToCoordinate(ranges, epoch); + return new EpochReady(epoch, DONE, readyToCoordinate, DONE, DONE); + }; + } + + // may be invoked by any thread without holding the command store lock + private AsyncResult readyToCoordinate(Ranges ranges, long epoch) + { + if (redundantBefore.min(ranges, Bounds::locallyWitnessedBefore).epoch() >= epoch) + return DONE; + + SettableResult whenDone = new SettableWithDescription<>(this + " is ready to coordinate " + ranges + " on epoch " + epoch); + TxnId minForEpoch = TxnId.minForEpoch(epoch); + Ranges remaining = redundantBefore.removeWitnessed(minForEpoch, ranges); + WaitingOnVisibility sync = new WaitingOnVisibility(whenDone, remaining); + synchronized (waitingOnVisibility) + { + WaitingOnVisibility prev = waitingOnVisibility.putIfAbsent(epoch, sync); + Invariants.require(prev == null); + } + ensureReadyToCoordinate(epoch, ranges); + return whenDone; + } + + private void ensureReadyToCoordinate(long epoch, Ranges ranges) + { + TxnId minForEpoch = TxnId.minForEpoch(epoch); + node.durability().close("[" + this + " Epoch " + epoch + ']', VisibilitySyncPoint, minForEpoch, ranges, 1, TimeUnit.HOURS) + .invoke((success, fail) -> { + if (fail != null) + { + Ranges notRetired = redundantBefore.removeRetired(ranges); + Ranges retired = ranges.without(notRetired); + Ranges remaining = redundantBefore.removeWitnessed(minForEpoch, notRetired); + + if (!retired.isEmpty()) + { + logger.info("Failed to close epoch {} for ranges {} on store {}, but some are retired; marking these as synced.", epoch, ranges, id, fail); + execute((Empty)() -> "Mark Retired Ranges Synced", safeStore -> { + markVisibleInternal(safeStore, epoch, retired, "(Retired)"); + }); + } + else if (remaining.isEmpty()) + { + logger.info("Failed to close epoch {} for ranges {} on store {}, but none remaining. Aborting.", epoch, ranges, id, fail); + } + if (!remaining.isEmpty()) + { + logger.error("Failed to close epoch {} for ranges {} on store {}. Retrying.", epoch, remaining, id, fail); + node.someExecutor().execute(() -> ensureReadyToCoordinate(epoch, remaining)); + } + } + }); + } + + Supplier unbootstrap(long epoch, Ranges removedRanges) + { + return () -> { + AsyncResult done = submit((Empty) () -> "Unbootstrap", safeStore -> { + for (Bootstrap prev : bootstraps) + { + Ranges abort = prev.allValid.slice(removedRanges, Minimal); + if (!abort.isEmpty()) + prev.invalidate(abort); + } + return null; + }); + + return new EpochReady(epoch, done, done, done, done); + }; + } + + final void complete(Bootstrap bootstrap) + { + bootstraps.remove(bootstrap); + } + + final void markBootstrapping(SafeCommandStore safeStore, TxnId globalSyncId, Ranges ranges) + { + safeStore.setBootstrapBeganAt(bootstrap(globalSyncId, ranges, bootstrapBeganAt)); + safeStore.setSafeToRead(purgeHistory(safeToRead, ranges)); + updateMaxConflicts(ranges, globalSyncId); + RedundantBefore addRedundantBefore = RedundantBefore.create(ranges, Long.MIN_VALUE, Long.MAX_VALUE, globalSyncId, UNREADY_ONLY); + safeStore.upsertRedundantBefore(addRedundantBefore); + } + + // TODO (expected): we can immediately truncate dependencies locally once an exclusiveSyncPoint applies, we don't need to wait for the whole shard + public void markShardDurable(SafeCommandStore safeStore, TxnId globalSyncId, Ranges durableRanges, HasOutcome durability) + { + if (!durability.isDurable()) + return; + + SomeStatus status = durability.isUniversal() ? SHARD_APPLIED_ONLY : QUORUM_APPLIED_ONLY; + final Ranges slicedRanges = durableRanges.slice(safeStore.ranges().allUntil(globalSyncId.epoch()), Minimal); + TxnId locallyRedundantBefore = safeStore.redundantBefore().min(slicedRanges, Bounds::maxLocallyAppliedBefore); + RedundantBefore addNow = RedundantBefore.create(slicedRanges, globalSyncId, status); + safeStore.upsertRedundantBefore(addNow); + + if (status != SHARD_APPLIED_ONLY) + return; + + if (locallyRedundantBefore.compareTo(globalSyncId) < 0) + { + // TODO (expected): if bootstrapping only part of the range, mark the rest for GC; or relax this as can safely GC behind bootstrap + TxnId maxBootstrap = safeStore.redundantBefore().max(slicedRanges, Bounds::maxReadyAt); + if (maxBootstrap.compareTo(globalSyncId) >= 0) + logger.info("Ignoring markShardDurable for a point we are bootstrapping. Bootstrapping: {}, Global: {}, Ranges: {}", maxBootstrap, globalSyncId, slicedRanges); + else + logger.warn("Trying to markShardDurable a point we have not yet caught-up to locally. Local: {}, Global: {}, Ranges: {}", locallyRedundantBefore, globalSyncId, slicedRanges); + return; + } + + // TODO (desired): not all systems care about HLC_BOUND for GC, make configurable + if (globalSyncId.is(HLC_BOUND) || !requiresUniqueHlcs()) + { + RedundantBefore addOnDataStoreDurable = RedundantBefore.create(slicedRanges, globalSyncId, GC_BEFORE_AND_LOCALLY_DURABLE); + dataStore.ensureDurable(this, slicedRanges, addOnDataStoreDurable); + } + } + + protected void updatedRedundantBefore(SafeCommandStore safeStore, RedundantBefore added) + { + TxnId clearWaitingBefore = redundantBefore.minShardAndLocallyAppliedBefore(); + TxnId clearAllBefore = TxnId.min(clearWaitingBefore, durableBefore().min.quorumBefore); + progressLog.clearBefore(safeStore, clearWaitingBefore, clearAllBefore); + listeners.clearBefore(clearWaitingBefore); + } + + @VisibleForTesting + public AsyncResult awaitVisibility(long epoch, Ranges ranges) + { + synchronized (waitingOnVisibility) + { + if (waitingOnVisibility.isEmpty()) + return AsyncResults.success(null); + + List> awaiting = new ArrayList<>(); + for (Map.Entry e : waitingOnVisibility.entrySet()) + { + if (e.getKey() > epoch) + break; + + Ranges remaining = e.getValue().waitingOn; + Ranges intersecting = remaining.slice(ranges, Minimal); + if (!intersecting.isEmpty()) + { + awaiting.add(e.getValue().whenDone); + ranges = ranges.without(intersecting); + } + } + + if (awaiting.isEmpty()) + return AsyncResults.success(null); + return AsyncResults.debuggableReduce(awaiting, Reduce.toNull()); + } + } + + protected final Ranges isWaitingOnVisibility(TxnId syncId, Ranges ranges) + { + synchronized (waitingOnVisibility) + { + if (waitingOnVisibility.isEmpty()) + return Ranges.EMPTY; + + Ranges waitingOn = Ranges.EMPTY; + for (Map.Entry e : waitingOnVisibility.entrySet()) + { + if (e.getKey() > syncId.epoch()) + break; + + Ranges remaining = e.getValue().waitingOn; + Ranges intersecting = remaining.slice(ranges, Minimal); + if (!intersecting.isEmpty()) + { + ranges = ranges.without(intersecting); + waitingOn = waitingOn.with(intersecting); + } + } + + return waitingOn; + } + } + + protected final void markingVisible(TxnId syncId, Ranges ranges) + { + synchronized (waitingOnVisibility) + { + if (waitingOnVisibility.isEmpty()) + return; + + for (Map.Entry e : waitingOnVisibility.entrySet()) + { + if (e.getKey() > syncId.epoch()) + break; + + Ranges remaining = e.getValue().waitingOn.without(ranges); + if (e.getValue().waitingOn != remaining) + e.getValue().waitingOn = remaining; + } + } + } + + protected final void cancelMarkingVisible(TxnId syncId, Ranges ranges) + { + synchronized (waitingOnVisibility) + { + if (waitingOnVisibility.isEmpty()) + return; + + for (Map.Entry e : waitingOnVisibility.entrySet()) + { + if (e.getKey() > syncId.epoch()) + break; + + Ranges unmark = e.getValue().waitingOnDurable.slice(ranges, Minimal); + if (!unmark.isEmpty()) + e.getValue().waitingOn = e.getValue().waitingOn.with(unmark); + } + } + } + + protected final void markVisible(SafeCommandStore safeStore, TxnId syncId, Ranges ranges) + { + Invariants.require(syncId.is(VisibilitySyncPoint)); + RedundantBefore addRedundantBefore = RedundantBefore.create(ranges, syncId, LOCALLY_WITNESSED_ONLY); + safeStore.upsertRedundantBefore(addRedundantBefore); + markVisibleInternal(safeStore, syncId.epoch(), ranges, syncId); + } + + private void markVisibleInternal(SafeCommandStore safeStore, long epoch, Ranges ranges, Object describe) + { + synchronized (waitingOnVisibility) + { + if (waitingOnVisibility.isEmpty()) + return; + + LongHashSet remove = null; + for (Map.Entry e : waitingOnVisibility.entrySet()) + { + if (e.getKey() > epoch) + break; + + Ranges waitingOn = e.getValue().waitingOn; + Ranges waitingOnDurable = e.getValue().waitingOnDurable; + Ranges synced = waitingOnDurable.slice(ranges, Minimal); + boolean intersects = waitingOnDurable.intersects(ranges); + if (intersects) + { + e.getValue().waitingOn = waitingOn = waitingOn.without(ranges); + e.getValue().waitingOnDurable = waitingOnDurable = waitingOnDurable.without(ranges); + if (waitingOnDurable.isEmpty()) + { + SettableResult done = e.getValue().whenDone; + logger.debug("{} completed full visibility sync for {} on epoch {} using {}", this, e.getValue().allRanges, e.getKey(), describe); + done.trySuccess(null); + if (remove == null) + remove = new LongHashSet(); + remove.add(e.getKey()); + } + else + { + logger.debug("{} completed partial visibility sync for {} on epoch {} using {}; {} still to sync and {} to sync durably", this, synced, e.getKey(), describe, waitingOn, waitingOnDurable); + } + } + } + if (remove != null) + remove.forEach(waitingOnVisibility::remove); + } + } + + public void markShardStale(SafeCommandStore safeStore, Timestamp staleSince, Ranges ranges, boolean isSincePrecise) + { + Timestamp staleUntilAtLeast = staleSince; + if (isSincePrecise) + { + ranges = ranges.slice(safeStore.ranges().allAt(staleSince.epoch()), Minimal); + } + else + { + ranges = ranges.slice(safeStore.ranges().allSince(staleSince.epoch()), Minimal); + // make sure no in-progress bootstrap attempts will override the stale since for commands whose staleness bounds are unknown + staleUntilAtLeast = Timestamp.max(bootstrapBeganAt.lastKey(), staleUntilAtLeast); + } + + if (ranges.isEmpty()) + return; + + agent.ownershipEvents().onStale(staleSince, ranges); + + RedundantBefore addRedundantBefore = RedundantBefore.createStale(ranges, staleUntilAtLeast); + safeStore.upsertRedundantBefore(addRedundantBefore); + // find which ranges need to bootstrap, subtracting those already in progress that cover the id + + markUnsafeToRead(ranges); + } + + /** + * This is a heavy-handed operator action to unstick waiting transactions whose transitive dependencies + * may already be applied. + */ + public final AsyncResult operatorTryToExecuteListeningTxns() + { + SettableResult done = new SettableResult<>(); + execute((Empty)() -> "Try Execute Listening", safeStore -> { + tryExecuteListening(safeStore, listeners.txnsWaitingOn(SaveStatus.Applied).iterator(), done); + }); + return done; + } + + private void tryExecuteListening(SafeCommandStore safeStore, Iterator iterator, SettableResult done) + { + if (!iterator.hasNext()) + { + done.trySuccess(null); + return; + } + + try + { + TxnId waitingOn = iterator.next(); + PreLoadContext context = PreLoadContext.contextFor(waitingOn, "Try Execute Listening"); + if (!safeStore.canExecuteWith(context) || !safeStore.tryRecurse()) + { + //noinspection DataFlowIssue + safeStore = safeStore; + execute(context, safeStore0 -> tryExecuteListening(safeStore0, waitingOn, iterator, done)); + } + else + { + try { tryExecuteListening(safeStore, waitingOn, iterator, done); } + finally { safeStore.unrecurse(); } + } + } + catch (Throwable t) + { + done.tryFailure(t); + } + } + + private void tryExecuteListening(SafeCommandStore safeStore, TxnId waitingOn, Iterator iterator, SettableResult done) + { + try + { + SafeCommand safeCommand = safeStore.unsafeGet(waitingOn); + //noinspection DataFlowIssue + safeStore = safeStore; + //noinspection DataFlowIssue + safeCommand = safeCommand; + boolean wasApplied = safeCommand.current().hasBeen(Status.Applied); + Consumer continuation = safeStore0 -> { + if (!wasApplied) + { + SafeCommand safeCommand0 = safeStore0.ifLoadedAndInitialised(waitingOn); + if (safeCommand0 != null && safeCommand0.current().saveStatus().hasBeen(Status.Applied)) + logger.warn("{} was successfully applied by tryToExecuteListening", waitingOn); + } + tryExecuteListening(safeStore0, iterator, done); + }; + + Commands.maybeExecute(safeStore, safeCommand, safeCommand.current(), true, true, NotifyWaitingOnPlus.adapter(continuation, true, true)); + } + catch (Throwable t) + { + done.tryFailure(t); + } + } + + public final boolean isRejectedIfNotPreAccepted(TxnId txnId, Unseekables participants) + { + if (rejectBefore == null) + return false; + + return rejectBefore.rejects(txnId, participants); + } + + public final MaxConflicts unsafeGetMaxConflicts() + { + return maxConflicts; + } + + public final RedundantBefore unsafeGetRedundantBefore() + { + return redundantBefore; + } + + public final LocalListeners unsafeGetListeners() + { + return listeners; + } + + @Nullable + public final RejectBefore unsafeGetRejectBefore() + { + return rejectBefore; + } + + public final DurableBefore durableBefore() + { + return node.durableBefore(); + } + + public final ProgressLog unsafeProgressLog() + { + return progressLog; + } + + @VisibleForTesting + public final NavigableMap unsafeGetBootstrapBeganAt() { return bootstrapBeganAt; } + + @VisibleForTesting + public NavigableMap unsafeGetSafeToRead() { return safeToRead; } + + final void markUnsafeToRead(Ranges ranges) + { + if (safeToRead.values().stream().anyMatch(r -> r.intersects(ranges))) + { + execute((Empty) () -> "Mark Unsafe To Read", safeStore -> { + safeStore.setSafeToRead(purgeHistory(safeToRead, ranges)); + }, agent); + } + } + + final AsyncChain markPermanentlyUnsafeToRead(Ranges ranges) + { + return chain((Empty) () -> "Mark Range As Regained", safeStore -> { + safeStore.setSafeToRead(purgeHistory(safeToRead, ranges)); + safeStore.setPermanentlyUnsafeToRead(permanentlyUnsafeToRead.union(MERGE_ADJACENT, ranges)); + }); + } + + public final DataStore unsafeGetDataStore() + { + return dataStore; + } + + final synchronized AsyncResult markSafeToRead(Timestamp forBootstrapAt, Timestamp at, Ranges ranges) + { + return execute((Empty) () -> "Mark Safe To Read", safeStore -> { + // TODO (required): handle weird edge cases like newer at having a lower HLC than prior existing at, but higher epoch + Ranges validatedSafeToRead = redundantBefore.validateSafeToRead(forBootstrapAt, ranges); + safeStore.setSafeToRead(purgeAndInsert(safeToRead, at, validatedSafeToRead)); + updateMaxConflicts(ranges, at); + }); + } + + public static ImmutableSortedMap bootstrap(TxnId at, Ranges ranges, NavigableMap readyAt) + { + Invariants.requireArgument(!ranges.isEmpty()); + if (at == TxnId.NONE) + { + for (Ranges rs : readyAt.values()) + Invariants.require(!ranges.intersects(rs)); + } + // if we're bootstrapping these ranges, then any period we previously owned the ranges for is effectively invalidated + return purgeAndInsert(readyAt, at, ranges); + } + + private static ImmutableSortedMap purgeAndInsert(NavigableMap in, T insertAt, Ranges insert) + { + TreeMap build = new TreeMap<>(in); + build.headMap(insertAt, false).entrySet().forEach(e -> e.setValue(e.getValue().without(insert))); + build.tailMap(insertAt, true).entrySet().forEach(e -> e.setValue(e.getValue().union(MERGE_ADJACENT, insert))); + build.entrySet().removeIf(e -> e.getKey().compareTo(Timestamp.NONE) > 0 && e.getValue().isEmpty()); + Map.Entry prev = build.floorEntry(insertAt); + build.putIfAbsent(insertAt, prev.getValue().with(insert)); + return ImmutableSortedMap.copyOf(build); + } + + private static ImmutableSortedMap purgeHistory(NavigableMap in, Ranges remove) + { + return ImmutableSortedMap.copyOf(purgeHistoryIterator(in, remove)); + } + + private static Iterable> purgeHistoryIterator(NavigableMap in, Ranges removeRanges) + { + return () -> in.entrySet().stream() + .map(e -> without(e, removeRanges)) + .filter(e -> !e.getValue().isEmpty() || e.getKey().equals(TxnId.NONE)) + .iterator(); + } + + private static Map.Entry without(Map.Entry in, Ranges remove) + { + Ranges without = in.getValue().without(remove); + if (without == in.getValue()) + return in; + return new SimpleImmutableEntry<>(in.getKey(), without); + } + + @Override + public int hashCode() + { + return id; + } + + public boolean isBootstrapping() + { + return !bootstraps.isEmpty(); + } + + public void updateMinHlc(long minHlc) + { + Timestamp timestamp = Timestamp.fromValues(rangesForEpoch.epochs[rangesForEpoch.epochs.length - 1], minHlc, 0, node.id()); + MaxConflicts updated = maxConflicts.update(rangesForEpoch.all(), timestamp); + setMaxConflicts(updated); + } + + public static NavigableMap emptyBootstrapBeganAt() + { + return ImmutableSortedMap.of(TxnId.NONE, Ranges.EMPTY); + } + + public static NavigableMap emptySafeToRead() + { + return ImmutableSortedMap.of(Timestamp.NONE, Ranges.EMPTY); + } + + public NodeCommandStoreService node() + { + return node; + } + + void unsafeRegister(SyncPointListener listener) + { + Invariants.require(inStore()); + List newListeners = new ArrayList<>(); + if (syncPointListeners != null) + newListeners.addAll(syncPointListeners); + newListeners.add(listener); + syncPointListeners = newListeners; + } + + void unsafeUnregister(SyncPointListener listener) + { + Invariants.require(inStore()); + if (syncPointListeners != null) + { + List newListeners = new ArrayList<>(syncPointListeners); + newListeners.remove(listener); + if (newListeners.isEmpty()) + newListeners = null; + syncPointListeners = newListeners; + } + } +} diff --git a/accord-core/src/main/java/accord/local/CommandStores.java b/accord-core/src/main/java/accord/local/CommandStores.java new file mode 100644 index 000000000000..72aa60131c7a --- /dev/null +++ b/accord-core/src/main/java/accord/local/CommandStores.java @@ -0,0 +1,1219 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package accord.local; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.function.BiConsumer; +import java.util.function.Consumer; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import accord.api.Agent; +import accord.api.AsyncExecutorFactory; +import accord.api.AsyncExecutor; +import accord.topology.EpochReady; +import accord.api.DataStore; +import accord.api.Journal; +import accord.api.LocalListeners; +import accord.api.ProgressLog; +import accord.api.RoutingKey; +import accord.local.CommandStore.EpochUpdateHolder; +import accord.primitives.AbstractRanges; +import accord.primitives.AbstractUnseekableKeys; +import accord.primitives.EpochSupplier; +import accord.primitives.Participants; +import accord.primitives.Range; +import accord.primitives.Ranges; +import accord.primitives.RoutingKeys; +import accord.primitives.Timestamp; +import accord.primitives.TxnId; +import accord.primitives.Unseekables; +import accord.topology.Shard; +import accord.topology.Topology; +import accord.utils.IndexedQuadConsumer; +import accord.utils.IndexedRangeQuadConsumer; +import accord.utils.Invariants; +import accord.utils.RandomSource; +import accord.utils.Reduce; +import accord.utils.SearchableRangeList; +import accord.utils.LargeBitSet; +import accord.utils.UnhandledEnum; +import accord.utils.async.AsyncChain; +import accord.utils.async.AsyncChains; +import accord.utils.async.AsyncResult; +import accord.utils.async.AsyncResults; +import accord.utils.async.Cancellable; +import accord.utils.async.NestedAsyncResult; +import org.agrona.collections.Hashing; +import org.agrona.collections.Int2IntHashMap; +import org.agrona.collections.Int2ObjectHashMap; + +import static accord.topology.EpochReady.done; +import static accord.api.DataStore.FetchKind.Sync; +import static accord.local.CommandStores.BootstrapRangeAction.BOOTSTRAP_NOT_NEEDED; +import static accord.local.CommandStores.BootstrapRangeAction.SAFE_BOOTSTRAP; +import static accord.primitives.Routables.Slice.Minimal; +import static accord.utils.Invariants.illegalState; +import static java.util.stream.Collectors.toList; + +/** + * Manages the single threaded metadata shards + */ +public abstract class CommandStores implements AsyncExecutorFactory +{ + @SuppressWarnings("unused") + private static final Logger logger = LoggerFactory.getLogger(CommandStores.class); + + static final Iterator INVALID = new Iterator<>() + { + @Override public boolean hasNext() { throw new UnsupportedOperationException(); } + @Override public CommandStore next() { throw new UnsupportedOperationException(); } + }; + + public interface LatentStoreSelector + { + StoreSelector refine(TxnId txnId, @Nullable Timestamp executeAt, Participants participants); + + class StandardLatentStoreSelector implements LatentStoreSelector + { + private static final StandardLatentStoreSelector INSTANCE = new StandardLatentStoreSelector(); + + @Override + public StoreSelector refine(TxnId txnId, @Nullable Timestamp executeAt, Participants participants) + { + return snapshot -> StoreFinder.find(snapshot, participants) + .filter(snapshot, participants, txnId.epoch(), (executeAt != null ? executeAt : txnId).epoch()) + .iterator(snapshot); + } + } + + static LatentStoreSelector standard() + { + return StandardLatentStoreSelector.INSTANCE; + } + } + + public interface StoreSelector extends LatentStoreSelector + { + default StoreSelector refine(TxnId txnId, @Nullable Timestamp executeAt, Participants participants) { return this; } + Iterator select(Snapshot snapshot); + } + + public static class IncludingSpecificStoreSelector implements StoreSelector + { + final int storeId; + + public IncludingSpecificStoreSelector(int storeId) + { + this.storeId = storeId; + } + + @Override + public StoreSelector refine(TxnId txnId, @Nullable Timestamp executeAt, Participants participants) + { + return snapshot -> { + StoreFinder finder = StoreFinder.find(snapshot, participants) + .filter(snapshot, participants, txnId.epoch(), (executeAt != null ? executeAt : txnId).epoch()); + finder.set(snapshot.byId.get(storeId)); + return finder.iterator(snapshot); + }; + } + + @Override + public Iterator select(Snapshot snapshot) + { + return Collections.singletonList(snapshot.byId(storeId)).iterator(); + } + } + + // TODO (required): as we get more tables this will become expensive to allocate; we need to index first by prefix + public static class StoreFinder extends LargeBitSet implements IndexedQuadConsumer, IndexedRangeQuadConsumer + { + final int[] indexMap; + private boolean invalid; + + private StoreFinder(int size, int[] indexMap) + { + super(size); + this.indexMap = indexMap; + } + + public StoreFinder(Snapshot snapshot) + { + this(snapshot.shards.length, snapshot.indexForRange); + } + + public static StoreSelector selector(Unseekables unseekables, long minEpoch, long maxEpoch) + { + return snapshot -> { + StoreFinder finder = StoreFinder.find(snapshot, unseekables); + finder.filter(snapshot, unseekables, minEpoch, maxEpoch); + return finder.iterator(snapshot); + }; + } + + public static StoreFinder find(Snapshot snapshot, Unseekables unseekables) + { + StoreFinder finder = new StoreFinder(snapshot); + switch (unseekables.domain()) + { + default: throw new UnhandledEnum(unseekables.domain()); + case Range: + { + int minIndex = 0; + for (Range range : (AbstractRanges)unseekables) + minIndex = snapshot.lookupByRange.forEachRange(range, finder, finder, null, null, null, null, minIndex); + break; + } + case Key: + { + int minIndex = 0; + for (RoutingKey key : (AbstractUnseekableKeys)unseekables) + minIndex = snapshot.lookupByRange.forEachKey(key, finder, finder, null, null, null, null, minIndex); + break; + } + } + return finder; + } + + public StoreFinder filter(Snapshot snapshot, Unseekables unseekables, long minEpoch, long maxEpoch) + { + for (int i = firstSetBit(); i >= 0 ; i = nextSetBit(i + 1, -1)) + { + ShardHolder shard = snapshot.shards[i]; + Ranges shardRanges = shard.ranges().allBetween(minEpoch, maxEpoch); + if (shardRanges != shard.ranges.all() && !shardRanges.intersects(unseekables)) + { + unset(i); + } + else if (unsafelyTouchesRegainedRanges(snapshot, shard, unseekables, minEpoch)) + { + invalid = true; + break; + } + } + return this; + } + + public Iterator iterator(Snapshot snapshot) + { + if (invalid) + return INVALID; + + return new Iterator<>() + { + int i = firstSetBit(); + @Override + public boolean hasNext() + { + return i >= 0; + } + + @Override + public CommandStore next() + { + CommandStore next = snapshot.shards[i].store; + i = nextSetBit(i + 1, -1); + return next; + } + }; + } + + @Override + public void accept(Object p1, Object p2, Object p3, Object p4, int index) + { + set(indexMap[index]); + } + + @Override + public void accept(Object p1, Object p2, Object p3, Object p4, int fromIndex, int toIndex) + { + for (int i = fromIndex ; i < toIndex ; ++i) + set(indexMap[i]); + } + } + + public interface Factory + { + CommandStores create(NodeCommandStoreService node, + Agent agent, + DataStore store, + RandomSource random, + Journal journal, + ShardDistributor shardDistributor, + ProgressLog.Factory progressLogFactory, + LocalListeners.Factory listenersFactory); + } + + private static class StoreSupplier + { + private final NodeCommandStoreService node; + private final Agent agent; + private final DataStore store; + private final ProgressLog.Factory progressLogFactory; + private final LocalListeners.Factory listenersFactory; + private final CommandStore.Factory shardFactory; + private final RandomSource random; + private final Journal journal; + + StoreSupplier(NodeCommandStoreService node, Agent agent, DataStore store, RandomSource random, ProgressLog.Factory progressLogFactory, LocalListeners.Factory listenersFactory, CommandStore.Factory shardFactory, Journal journal) + { + this.node = node; + this.agent = agent; + this.store = store; + this.random = random; + this.progressLogFactory = progressLogFactory; + this.listenersFactory = listenersFactory; + this.shardFactory = shardFactory; + this.journal = journal; + } + + CommandStore create(int id, EpochUpdateHolder rangesForEpoch) + { + return shardFactory.create(id, node, agent, this.store, progressLogFactory, listenersFactory, rangesForEpoch, journal); + } + } + + public static class ShardHolder + { + public final CommandStore store; + @Nullable final Ranges regainsRanges; + RangesForEpoch ranges; + + ShardHolder(CommandStore store, @Nullable Ranges regainsRanges) + { + this.store = store; + this.regainsRanges = regainsRanges; + } + + public ShardHolder(CommandStore store, RangesForEpoch ranges, @Nullable Ranges regainsRanges) + { + this.store = store; + this.regainsRanges = regainsRanges; + this.ranges = ranges; + } + + public RangesForEpoch ranges() + { + return ranges; + } + + boolean filter(long minEpoch, long maxEpoch, Unseekables unseekables) + { + Ranges shardRanges = ranges.allBetween(minEpoch, maxEpoch); + return shardRanges != ranges.all() && !shardRanges.intersects(unseekables); + } + + public String toString() + { + return store.id() + " " + ranges; + } + } + + public interface RangesForEpochSupplier + { + RangesForEpoch ranges(); + } + + public static final class PreviouslyOwned + { + public static final PreviouslyOwned EMPTY = new PreviouslyOwned(0, RangesForEpoch.EMPTY.epochs, RangesForEpoch.EMPTY.ranges); + final long maxEpoch; + final long[] epochs; // the epoch upon which it was last owned + final Ranges[] ranges; + + public PreviouslyOwned(long maxEpoch, long[] epochs, Ranges[] ranges) + { + this.maxEpoch = maxEpoch; + this.epochs = epochs; + this.ranges = ranges; + } + + PreviouslyOwned prepend(long epoch, Ranges ranges) + { + Invariants.require(epochs.length == 0 || epoch > epochs[0]); + long[] newEpochs = new long[this.epochs.length + 1]; + Ranges[] newRanges = new Ranges[epochs.length]; + newEpochs[0] = epoch; + newRanges[0] = ranges; + System.arraycopy(this.epochs, 0, newEpochs, 1, this.epochs.length); + System.arraycopy(this.ranges, 0, newRanges, 1, this.ranges.length); + return new PreviouslyOwned(epoch, newEpochs, newRanges); + } + + public boolean overlaps(long epoch, Unseekables test) + { + if (epoch > maxEpoch) + return false; + + for (int i = 0 ; i < epochs.length && epoch <= epochs[i] ; ++i) + { + if (this.ranges[i].intersects(test)) + return true; + } + + return false; + } + + public Ranges regains(Ranges overlapping) + { + Ranges regains = Ranges.EMPTY; + for (Ranges rs : this.ranges) + regains = regains.without(rs.slice(overlapping, Minimal)); + return regains; + } + + public int size() + { + return epochs.length; + } + + public long epochs(int i) + { + return epochs[i]; + } + + public Ranges ranges(int i) + { + return ranges[i]; + } + } + + // We ONLY remove ranges to keep logic manageable; likely to only merge CommandStores into a new CommandStore via some kind of Bootstrap + public static class RangesForEpoch + { + public static final RangesForEpoch EMPTY = new RangesForEpoch(new long[0], new Ranges[0]); + + final long[] epochs; + final Ranges[] ranges; + + public RangesForEpoch(long epoch, Ranges ranges) + { + this.epochs = new long[] { epoch }; + this.ranges = new Ranges[] { ranges }; + } + + public RangesForEpoch(long[] epochs, Ranges[] ranges) + { + Invariants.require(epochs.length == ranges.length); + this.epochs = epochs; + this.ranges = ranges; + } + + public int size() + { + return epochs.length; + } + + public void forEach(BiConsumer forEach) + { + for (int i = 0; i < epochs.length; i++) + forEach.accept(epochs[i], ranges[i]); + } + + @Override + public boolean equals(Object object) + { + if (this == object) return true; + if (object == null || getClass() != object.getClass()) return false; + RangesForEpoch that = (RangesForEpoch) object; + return Arrays.equals(epochs, that.epochs) && Arrays.equals(ranges, that.ranges); + } + + @Override + public int hashCode() + { + throw new UnsupportedOperationException(); + } + + public RangesForEpoch withRanges(long epoch, Ranges latestRanges) + { + Invariants.requireArgument(epochs.length == 0 || epochs[epochs.length - 1] <= epoch); + int newLength = epochs.length == 0 || epochs[epochs.length - 1] < epoch ? epochs.length + 1 : epochs.length; + long[] newEpochs = Arrays.copyOf(epochs, newLength); + Ranges[] newRanges = Arrays.copyOf(ranges, newLength); + newEpochs[newLength - 1] = epoch; + newRanges[newLength - 1] = latestRanges; + Invariants.require(newEpochs[newLength - 1] == 0 || newEpochs[newLength - 1] == epoch, "Attempted to override historic epoch %d with %d", newEpochs[newLength - 1], epoch); + return new RangesForEpoch(newEpochs, newRanges); + } + + public @Nonnull Ranges coordinates(TxnId txnId) + { + return allAt(txnId); + } + + public @Nonnull Ranges allAt(Timestamp at) + { + return allAt(at.epoch()); + } + + public @Nonnull Ranges allAt(long epoch) + { + int i = floorIndex(epoch); + if (i < 0) return Ranges.EMPTY; + return ranges[i]; + } + + /** + * Extend a previously computed set of Ranges that included {@code fromInclusive} + * to include ranges up to {@code toInclusive} + */ + public @Nonnull Ranges extend(Ranges extend, long curFrom, long curTo, long extendFrom, long extendTo) + { + if (extend.isEmpty()) // this captures the case where curTo < epochs[0] + return allBetween(extendFrom, extendTo); + + if (extendFrom >= curFrom) + return extend; + + int startCurIndex = floorIndex(curFrom); + int startExtendIndex = Math.max(0, floorIndex(extendFrom)); + if (startCurIndex <= startExtendIndex) + return extend; + + return ranges[startExtendIndex]; + } + + public @Nonnull Ranges allBetween(long fromInclusive, EpochSupplier toInclusive) + { + return allBetween(fromInclusive, toInclusive.epoch()); + } + + public @Nonnull Ranges allBetween(long fromInclusive, long toInclusive) + { + if (fromInclusive > toInclusive) + throw new IndexOutOfBoundsException(); + + int since = floorIndex(fromInclusive); + if (since >= 0) return ranges[since]; + + int to = floorIndex(toInclusive); + if (to >= 0) return ranges[0]; + return Ranges.EMPTY; + } + + public @Nonnull Ranges all() + { + return ranges[0]; + } + + public @Nonnull Ranges notRetired(SafeCommandStore safeStore) + { + return safeStore.redundantBefore().removeRetired(ranges[0]); + } + + public @Nonnull Ranges allBefore(long toExclusive) + { + int to = ceilIndex(toExclusive); + return to <= 0 ? Ranges.EMPTY : ranges[0]; + } + + public @Nonnull Ranges allUntil(long toInclusive) + { + int to = floorIndex(toInclusive); + return to < 0 ? Ranges.EMPTY : ranges[0]; + } + + public @Nonnull Ranges allSince(long fromInclusive) + { + int since = floorIndex(fromInclusive); + return ranges[Math.max(since, 0)]; + } + + public Ranges rangesAtIndex(int index) + { + return ranges[index]; + } + + public long epochAtIndex(int index) + { + return epochs[index]; + } + + public int floorIndex(long epoch) + { + int i = Arrays.binarySearch(epochs, epoch); + if (i < 0) i = -2 - i; + return i; + } + + private int ceilIndex(long epoch) + { + int i = Arrays.binarySearch(epochs, epoch); + if (i < 0) i = -1 - i; + return i; + } + + public int indexOffset(long lowEpoch, long highEpoch) + { + if (lowEpoch == highEpoch) + return 0; + + int lowIndex = Math.max(0, floorIndex(lowEpoch)); + int highIndex = lowIndex; + while (highIndex + 1 < epochs.length && epochs[highIndex + 1] <= highEpoch) + ++highIndex; + return highIndex - lowIndex; + } + + public @Nonnull Ranges currentRanges() + { + return ranges[ranges.length - 1]; + } + + public String toString() + { + return IntStream.range(0, ranges.length).mapToObj(i -> epochs[i] + ": " + ranges[i]) + .collect(Collectors.joining(", ")); + } + + public long earliestLaterEpochThatFullyCovers(long sinceEpoch, Unseekables keysOrRanges) + { + return Math.max(sinceEpoch, epochs[0]); + } + + public long latestEarlierEpochThatFullyCovers(long beforeEpoch, Unseekables keysOrRanges) + { + int i = ceilIndex(beforeEpoch); + if (i == 0) + return beforeEpoch; + + long latest = beforeEpoch; + Ranges existing = Ranges.EMPTY; + long next = beforeEpoch; + if (i < epochs.length) + { + existing = ranges[i]; + next = Math.min(next, epochs[i]); + } + while (--i >= 0) + { + if (ranges[i].without(existing).intersects(keysOrRanges)) + latest = next - 1; + existing = existing.with(ranges[i]); + next = epochs[i]; + } + return latest; + } + + public Ranges removed(long presentIn, long removedByInclusive) + { + int i = Math.max(1, floorIndex(presentIn)); + int maxi = 1 + floorIndex(removedByInclusive); + Ranges removed = Ranges.EMPTY; + while (i < maxi) + { + removed = removed.with(ranges[i - 1].without(ranges[i])); + ++i; + } + return removed; + } + } + + protected void loadSnapshot(Snapshot toLoad) + { + current = toLoad; + } + + public static class Snapshot extends Journal.TopologyUpdate implements Iterable + { + public final Topology local; + final ShardHolder[] shards; + final Int2IntHashMap byId; + private final int[] indexForRange; + final SearchableRangeList lookupByRange; + + public Snapshot(ShardHolder[] shards, Topology local, Topology global, PreviouslyOwned previouslyOwned) + { + super(asMap(shards), global, previouslyOwned); + this.local = local; + this.shards = shards; + this.byId = new Int2IntHashMap(shards.length, Hashing.DEFAULT_LOAD_FACTOR, -1); + int count = 0; + int prevId = -1; + for (int i = 0 ; i < shards.length ; ++i) + { + ShardHolder shard = shards[i]; + int id = shard.store.id; + Invariants.require(id > prevId); + byId.put(id, i); + count += shard.ranges.all().size(); + prevId = id; + } + class RangeAndIndex + { + final Range range; + final int index; + + RangeAndIndex(Range range, int index) + { + this.range = range; + this.index = index; + } + } + RangeAndIndex[] rangesAndIndexes = new RangeAndIndex[count]; + count = 0; + for (int i = 0; i < shards.length ; ++i) + { + Ranges add = shards[i].ranges.all(); + for (Range range : add) + rangesAndIndexes[count++] = new RangeAndIndex(range, i); + } + + Arrays.sort(rangesAndIndexes, (a, b) -> a.range.compareTo(b.range)); + + Range[] ranges = new Range[count]; + indexForRange = new int[count]; + for (int i = 0 ; i < rangesAndIndexes.length ; ++i) + { + ranges[i] = rangesAndIndexes[i].range; + indexForRange[i] = rangesAndIndexes[i].index; + } + lookupByRange = SearchableRangeList.build(ranges); + } + + // This method exists to ensure we do not hold references to command stores + public Journal.TopologyUpdate asTopologyUpdate() + { + return new Journal.TopologyUpdate(commandStores, global, previouslyOwned); + } + + private static Int2ObjectHashMap asMap(ShardHolder[] shards) + { + Int2ObjectHashMap commandStores = new Int2ObjectHashMap<>(); + for (ShardHolder shard : shards) + commandStores.put(shard.store.id, shard.ranges); + return commandStores; + } + + public CommandStore byId(int id) + { + return shards[byId.get(id)].store; + } + + @Override + public Iterator iterator() + { + return Arrays.asList(shards).iterator(); + } + } + + final StoreSupplier supplier; + final ShardDistributor shardDistributor; + final Journal journal; + volatile Snapshot current; + int nextId; + + private CommandStores(StoreSupplier supplier, ShardDistributor shardDistributor, Journal journal) + { + this.supplier = supplier; + this.shardDistributor = shardDistributor; + + this.current = new Snapshot(new ShardHolder[0], Topology.EMPTY, Topology.EMPTY, PreviouslyOwned.EMPTY); + this.journal = journal; + } + + public CommandStores(NodeCommandStoreService time, Agent agent, DataStore store, RandomSource random, Journal journal, ShardDistributor shardDistributor, + ProgressLog.Factory progressLogFactory, LocalListeners.Factory listenersFactory, CommandStore.Factory shardFactory) + { + this(new StoreSupplier(time, agent, store, random, progressLogFactory, listenersFactory, shardFactory, journal), shardDistributor, journal); + } + + public Node.Id nodeId() + { + return supplier.node.id(); + } + + public Topology local() + { + return current.local; + } + + public DataStore dataStore() + { + return supplier.store; + } + + static class TopologyUpdate + { + final Snapshot snapshot; + final Supplier bootstrap; + + TopologyUpdate(Snapshot snapshot, Supplier bootstrap) + { + this.snapshot = snapshot; + this.bootstrap = bootstrap; + } + } + + public enum BootstrapRangeAction + { + BOOTSTRAP_NOT_NEEDED, SAFE_BOOTSTRAP, UNSAFE_BOOTSTRAP + } + + protected BootstrapRangeAction shouldBootstrap(Node node, Topology prevGlobal, Topology newLocalTopology, Range add) + { + if (newLocalTopology.epoch() == 1 || !prevGlobal.ranges().contains(add)) + return BOOTSTRAP_NOT_NEEDED; + + return SAFE_BOOTSTRAP; + } + + public AsyncResult rebootstrap(Node node) + { + List> results = new ArrayList<>(); + Snapshot snapshot = current; + for (ShardHolder shard : snapshot.shards) + results.add(shard.store.startUnsafeBootstrap(node, shard.ranges.all(), snapshot.global.epoch(), Sync)); + return AsyncResults.allOf(results).flatMap(list -> { + return AsyncChains.reduce(list.stream() + .flatMap(b -> Stream.of(b.reads.chain(), b.coordinate.chain())) + .collect(Collectors.toList()), + Reduce.toNull()).beginAsResult(); + }); + } + + private synchronized TopologyUpdate updateTopology(Node node, Snapshot prev, Topology newTopology) + { + Invariants.requireArgument(!newTopology.isSubset(), "Use full topology for CommandStores.updateTopology"); + + long epoch = newTopology.epoch(); + if (epoch <= prev.global.epoch()) + { + Invariants.require(node.isReplaying(), "Received topology with epoch %d <= %d, but we are not replaying", epoch, prev.global.epoch()); + return new TopologyUpdate(prev, () -> done(epoch)); + } + + Topology newLocalTopology = newTopology.forNode(supplier.node.id()).trim(); + Ranges addedGlobal = newTopology.ranges().without(prev.global.ranges()); + node.addNewRangesToDurableBefore(addedGlobal, epoch); + + Ranges added = newLocalTopology.ranges().without(prev.local.ranges()); + Ranges subtracted = prev.local.ranges().without(newLocalTopology.ranges()); + + List> bootstrapUpdates = new ArrayList<>(); + List result = new ArrayList<>(prev.shards.length + added.size()); + PreviouslyOwned previouslyOwned = prev.previouslyOwned; + + for (ShardHolder shard : prev.shards) + { + Ranges current = shard.ranges().currentRanges(); + Ranges removeRanges = subtracted.slice(current, Minimal); + if (!removeRanges.isEmpty()) + { + // TODO (required): This is updating the a non-volatile field in the previous Snapshot, why modify it at all, even with volatile the guaranteed visibility is weak even with mutual exclusion + shard.ranges = shard.ranges().withRanges(newTopology.epoch(), current.without(subtracted)); + shard.store.epochUpdateHolder.remove(epoch, shard.ranges, removeRanges); + + bootstrapUpdates.add(shard.store.unbootstrap(epoch, removeRanges)); + } + + Ranges regainedRanges = shard.ranges().all().slice(added, Minimal); + if (!regainedRanges.isEmpty()) + bootstrapUpdates.add(() -> EpochReady.all(epoch, shard.store.markPermanentlyUnsafeToRead(regainedRanges).beginAsResult())); + + // TODO (desired): only sync affected shards + Ranges ranges = shard.ranges().currentRanges(); + // ranges can be empty when ranges are lost or consolidated across epochs. + if (epoch > 1 && requiresSync(ranges, prev.global, newTopology)) + { + logger.debug("Epoch {} requires visibility sync for {}", epoch, ranges); + bootstrapUpdates.add(shard.store.refreshReadyToCoordinate(node, ranges, epoch)); + } + + result.add(shard); + } + + if (!added.isEmpty()) + { + logger.info("Epoch {} adding {} to local command stores", epoch, added); + for (Ranges addRanges : shardDistributor.split(added)) + { + EpochUpdateHolder updateHolder = new EpochUpdateHolder(); + RangesForEpoch rangesForEpoch = new RangesForEpoch(epoch, addRanges); + updateHolder.add(epoch, rangesForEpoch, addRanges); + ShardHolder shard = new ShardHolder(supplier.create(nextId++, updateHolder), previouslyOwned.regains(addRanges)); + shard.ranges = rangesForEpoch; + + Map partitioned = addRanges.partitioningBy(range -> shouldBootstrap(node, prev.global, newLocalTopology, range), BootstrapRangeAction.class); + for (Map.Entry entry : partitioned.entrySet()) + { + BootstrapRangeAction action = entry.getKey(); + bootstrapUpdates.add(shard.store.bootstrapper(node, entry.getValue(), newLocalTopology.epoch(), action)); + } + result.add(shard); + } + } + + Supplier bootstrap; + if (bootstrapUpdates.isEmpty()) + { + logger.debug("Epoch {} implies no change to local command stores", epoch); + bootstrap = () -> done(epoch); + } + else + { + if (!subtracted.isEmpty()) + logger.info("Epoch {} removes {} from local command stores", epoch, subtracted); + + bootstrap = () -> { + List list = bootstrapUpdates.stream().map(Supplier::get).collect(toList()); + return new EpochReady(epoch, + AsyncResults.debuggableReduce(Lists.transform(list, EpochReady::active), Reduce.toNull()), + AsyncResults.debuggableReduce(Lists.transform(list, EpochReady::coordinate), Reduce.toNull()), + AsyncResults.debuggableReduce(Lists.transform(list, EpochReady::data), Reduce.toNull()), + AsyncResults.debuggableReduce(Lists.transform(list, EpochReady::reads), Reduce.toNull()) + ); + }; + } + + if (!subtracted.isEmpty()) + previouslyOwned = previouslyOwned.prepend(epoch - 1, subtracted); + + return new TopologyUpdate(new Snapshot(result.toArray(new ShardHolder[0]), newLocalTopology, newTopology, previouslyOwned), bootstrap); + } + + private static boolean requiresSync(Ranges ranges, Topology oldTopology, Topology newTopology) + { + List oldShards = oldTopology.foldl(ranges, (oldShard, shards, i) -> { + shards.add(oldShard); + return shards; + }, new ArrayList<>()); + + List newShards = newTopology.foldl(ranges, (newShard, shards, i) -> { + shards.add(newShard); + return shards; + }, new ArrayList<>()); + + if (oldShards.size() != newShards.size()) + return true; + + for (int i = 0 ; i < oldShards.size() ; ++i) + { + Shard oldShard = oldShards.get(i); + Shard newShard = newShards.get(i); + if (!oldShard.notInFastPath.equals(newShard.notInFastPath)) + return true; + + if (!newShard.nodes.equals(oldShard.nodes)) + return true; + } + return false; + } + + public void forAllUnsafe(Consumer forEach) + { + Snapshot snapshot = current; + for (ShardHolder shard : snapshot.shards) + forEach.accept(shard.store); + } + + public AsyncChain forAll(String reason, Consumer forEach) + { + return mapReduce(snapshot -> Stream.of(snapshot.shards).map(shard -> shard.store).iterator(), new MapReduceCommandStores<>(RoutingKeys.EMPTY) + { + @Override public Void reduce(Void o1, Void o2) { return null; } + @Override public TxnId primaryTxnId() { return null; } + @Override public String reason() { return reason; } + @Override + protected Void applyInternal(SafeCommandStore safeStore) + { + forEach.accept(safeStore); + return null; + } + }); + } + + public AsyncChain forEach(String reason, Participants participants, long minEpoch, long maxEpoch, Consumer forEach) + { + return forEach(reason, null, participants, minEpoch, maxEpoch, forEach); + } + + public AsyncChain forEach(String reason, TxnId txnId, Participants participants, long minEpoch, long maxEpoch, Consumer forEach) + { + return forEach(reason, txnId, participants, LoadKeys.SYNC, LoadKeysFor.READ_WRITE, minEpoch, maxEpoch, forEach); + } + + public AsyncChain forEach(String reason, TxnId txnId, Participants participants, LoadKeys loadKeys, LoadKeysFor loadKeysFor, long minEpoch, long maxEpoch, Consumer forEach) + { + return mapReduce(StoreFinder.selector(participants, minEpoch, maxEpoch), new MapReduceCommandStores, Void>(participants) + { + @Override public LoadKeys loadKeys() { return loadKeys;} + @Override public LoadKeysFor loadKeysFor() { return loadKeysFor; } + @Override public Void reduce(Void o1, Void o2) { return null; } + @Override public TxnId primaryTxnId() { return txnId; } + @Override public String reason() { return reason; } + @Override + protected Void applyInternal(SafeCommandStore safeStore) + { + forEach.accept(safeStore); + return null; + } + }); + } + + public Cancellable mapReduceConsume(long minEpoch, long maxEpoch, MapReduceConsumeCommandStores mapReduceConsume) + { + AsyncChain reduced = mapReduce(StoreFinder.selector(mapReduceConsume.scope, minEpoch, maxEpoch), mapReduceConsume); + return reduced.begin(mapReduceConsume); + } + + /** + * Maybe asynchronously, {@code apply} the function to each applicable {@code CommandStore}, invoke {@code reduce} + * on pairs of responses until only one remains, then {@code accept} the result. + * + * Note that {@code reduce} and {@code accept} are invoked by only one thread, and never concurrently with {@code apply}, + * so they do not require mutual exclusion. + */ + public Cancellable mapReduceConsume(StoreSelector selector, MapReduceConsumeCommandStores mapReduceConsume) + { + AsyncChain reduced = mapReduce(selector, mapReduceConsume); + return reduced.begin(mapReduceConsume); + } + + public Cancellable mapReduceConsume(IntStream commandStoreIds, MapReduceConsumeCommandStores mapReduceConsume) + { + AsyncChain reduced = mapReduce(commandStoreIds, mapReduceConsume); + return reduced.begin(mapReduceConsume); + } + + public AsyncChain mapReduce(IntStream commandStoreIds, MapReduceCommandStores mapReduce) + { + return mapReduce(snapshot -> commandStoreIds.mapToObj(snapshot::byId).iterator(), mapReduce); + } + + public AsyncChain mapReduce(StoreSelector selector, MapReduceCommandStores mapReduceConsume) + { + Snapshot snapshot = current; + Iterator stores = selector.select(snapshot); + if (stores == INVALID) + return AsyncChains.failure(new OverlappingCommandStoresException()); + + AsyncChain chain = null; + while (stores.hasNext()) + { + CommandStore store = stores.next(); + AsyncChain next = mapReduceConsume.applyAsync(store); + if (next != null) + chain = chain != null ? AsyncChains.reduce(chain, next, mapReduceConsume) : next; + } + + return chain == null ? AsyncChains.success(null) : chain; + } + + private static boolean unsafelyTouchesRegainedRanges(Snapshot snapshot, ShardHolder shard, Unseekables unseekables, long minEpoch) + { + if (shard.regainsRanges == null) + return false; + + unseekables = unseekables.slice(shard.regainsRanges, Minimal); + if (unseekables.isEmpty()) + return false; + + return snapshot.previouslyOwned.overlaps(minEpoch, unseekables); + } + + /** + * Initialize topology from snapshot on boot. + */ + public synchronized void initializeTopologyUnsafe(Journal.TopologyUpdate update) + { + Invariants.require(current.global.epoch() == 0); + ShardHolder[] shards = new ShardHolder[update.commandStores.size()]; + int i = 0; + int maxId = -1; + for (Map.Entry e : update.commandStores.entrySet()) + { + RangesForEpoch rfe = e.getValue(); + Invariants.require(rfe != null); + EpochUpdateHolder holder = new EpochUpdateHolder(); + holder.add(1, rfe, rfe.all()); + shards[i++] = new ShardHolder(supplier.create(e.getKey(), holder), rfe, update.previouslyOwned.regains(rfe.all())); + maxId = Math.max(maxId, e.getKey()); + } + Arrays.sort(shards, Comparator.comparingInt(shard -> shard.store.id)); + + nextId = maxId + 1; + loadSnapshot(new Snapshot(shards, update.global.forNode(supplier.node.id()).trim(), update.global, update.previouslyOwned)); + } + + public synchronized void resetTopology(Journal.TopologyUpdate update) + { + Snapshot current = this.current; + Invariants.require(update.global.epoch() == current.local.epoch()); + ShardHolder[] shards = new ShardHolder[current.commandStores.size()]; + int maxId = -1; + for (Map.Entry e : update.commandStores.entrySet()) + { + int storeId = e.getKey(); + RangesForEpoch rfe = e.getValue(); + Invariants.require(rfe != null); + ShardHolder shard = new ShardHolder(current.byId(storeId), rfe, update.previouslyOwned.regains(rfe.all())); + EpochUpdateHolder holder = shard.store.epochUpdateHolder; + rfe.forEach(new BiConsumer<>() + { + RangesForEpoch accumulator = null; + Ranges prev = null; + public void accept(Long epoch, Ranges ranges) + { + if (accumulator == null) + accumulator = new RangesForEpoch(epoch, ranges); + else + accumulator = accumulator.withRanges(epoch, ranges); + + Ranges additions = ranges; + Ranges removals = Ranges.EMPTY; + if (prev != null) + { + additions = ranges.without(prev); + removals = prev.without(ranges); + } + + if (!additions.isEmpty()) + holder.add(epoch, accumulator, additions); + if (!removals.isEmpty()) + holder.remove(epoch, accumulator, removals); + shard.store.unsafeUpdateRangesForEpoch(); + prev = ranges; + } + }); + + shards[storeId] = shard; + maxId = Math.max(maxId, storeId); + } + + nextId = maxId + 1; + loadSnapshot(new Snapshot(shards, current.local, current.global, update.previouslyOwned)); + } + + public synchronized Supplier updateTopology(Node node, Topology newTopology) + { + TopologyUpdate update = updateTopology(node, current, newTopology); + if (update.snapshot != current) + { + AsyncResults.SettableResult flush = new AsyncResults.SettableWithDescription<>("Write Topology To Journal"); + journal.saveTopology(update.snapshot.asTopologyUpdate(), () -> flush.setSuccess(null)); + current = update.snapshot; + return () -> { + EpochReady ready = update.bootstrap.get(); + return new EpochReady(ready.epoch, + ready.active, + NestedAsyncResult.flatMap(flush, ignore -> ready.coordinate), + NestedAsyncResult.flatMap(flush, ignore -> ready.data), + NestedAsyncResult.flatMap(flush, ignore -> ready.reads) + ); + }; + } + return update.bootstrap; + } + + public void shutdown() + { + for (ShardHolder shard : current.shards) + shard.store.shutdown(); + } + + @Override + public AsyncExecutor someExecutor() + { + return someSequentialExecutor(); + } + + @Override + public SequentialAsyncExecutor someSequentialExecutor() + { + return any(); + } + + @VisibleForTesting + public CommandStore any() + { + ShardHolder[] shards = current.shards; + if (shards.length == 0) throw illegalState("Unable to get CommandStore; non defined"); + return shards[supplier.random.nextInt(shards.length)].store; + } + + public CommandStore[] all() + { + ShardHolder[] shards = current.shards; + CommandStore[] all = new CommandStore[shards.length]; + for (int i = 0; i < shards.length; i++) + all[i] = shards[i].store; + return all; + } + + public CommandStore forId(int id) + { + Snapshot snapshot = current; + return snapshot.shards[snapshot.byId.get(id)].store; + } + + public int[] ids() + { + ShardHolder[] shards = current.shards; + int[] ids = new int[shards.length]; + for (int i = 0; i < ids.length; i++) + ids[i] = shards[i].store.id; + Arrays.sort(ids); + return ids; + } + + public int count() + { + return current.shards.length; + } + + public ShardDistributor shardDistributor() + { + return shardDistributor; + } + + @VisibleForTesting + public CommandStore unsafeForKey(RoutingKey key) + { + ShardHolder[] shards = current.shards; + for (ShardHolder shard : shards) + { + if (shard.ranges().currentRanges().contains(key)) + return shard.store; + } + throw new IllegalArgumentException(); + } + + protected Snapshot current() + { + return current; + } +} diff --git a/accord-core/src/main/java/accord/local/Node.java b/accord-core/src/main/java/accord/local/Node.java new file mode 100644 index 000000000000..9c4f1620c48a --- /dev/null +++ b/accord-core/src/main/java/accord/local/Node.java @@ -0,0 +1,886 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package accord.local; + +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicLongFieldUpdater; +import java.util.concurrent.locks.ReentrantLock; +import java.util.function.BiConsumer; +import java.util.function.Function; +import java.util.function.Supplier; +import java.util.stream.Stream; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + +import com.google.common.annotations.VisibleForTesting; + +import accord.api.Agent; +import accord.api.AsyncExecutor; +import accord.api.TopologyService; +import accord.topology.Topologies; +import accord.topology.ActiveEpoch; +import accord.topology.ActiveEpochs; +import accord.topology.EpochReady; +import accord.api.DataStore; +import accord.api.Journal; +import accord.api.LocalListeners; +import accord.api.MessageSink; +import accord.api.ProgressLog; +import accord.api.RemoteListeners; +import accord.api.Result; +import accord.api.RoutingKey; +import accord.api.Scheduler; +import accord.api.Timeouts; +import accord.api.TopologySorter; +import accord.coordinate.CoordinateEphemeralRead; +import accord.coordinate.CoordinateTransaction; +import accord.coordinate.Coordination; +import accord.coordinate.CoordinationAdapter; +import accord.coordinate.CoordinationAdapter.Factory.Kind; +import accord.coordinate.Coordinations; +import accord.coordinate.Infer.InvalidIf; +import accord.coordinate.Outcome; +import accord.coordinate.PrepareRecovery; +import accord.local.CommandStores.LatentStoreSelector; +import accord.local.CommandStores.StoreSelector; +import accord.local.cfk.CommandsForKey; +import accord.local.durability.DurabilityService; +import accord.messages.Callback; +import accord.messages.Reply; +import accord.messages.ReplyContext; +import accord.messages.Request; +import accord.primitives.Ballot; +import accord.primitives.EpochSupplier; +import accord.primitives.FullRoute; +import accord.primitives.Ranges; +import accord.primitives.Routable.Domain; +import accord.primitives.Routables; +import accord.primitives.RoutingKeys; +import accord.primitives.Seekables; +import accord.primitives.Timestamp; +import accord.primitives.Txn; +import accord.primitives.TxnId; +import accord.primitives.TxnId.Cardinality; +import accord.topology.TopologyException; +import accord.topology.TopologyManager; +import accord.topology.TopologyRetiredException; +import accord.utils.Invariants; +import accord.utils.PersistentField; +import accord.utils.PersistentField.Persister; +import accord.utils.RandomSource; +import accord.utils.SortedArrays.SortedArrayList; +import accord.utils.async.AsyncChain; +import accord.utils.async.AsyncChains; +import accord.utils.async.AsyncResult; +import accord.utils.async.AsyncResults; +import accord.utils.async.Cancellable; +import net.nicoulaj.compilecommand.annotations.Inline; + +import static accord.api.ProtocolModifiers.Toggles.defaultMediumPath; +import static accord.api.ProtocolModifiers.Toggles.ensurePermitted; +import static accord.api.ProtocolModifiers.Toggles.usePrivilegedCoordinator; +import static accord.coordinate.Coordination.CoordinationKind.COORDINATES_STATE_MACHINE; +import static accord.primitives.Routable.Domain.Key; +import static accord.primitives.Routable.Domain.Range; +import static accord.primitives.Txn.Kind.Read; +import static accord.primitives.Txn.Kind.Write; +import static accord.primitives.TxnId.Cardinality.Any; +import static accord.primitives.TxnId.Cardinality.cardinality; +import static accord.primitives.TxnId.FastPath.Unoptimised; +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static java.util.concurrent.TimeUnit.NANOSECONDS; +import static java.util.concurrent.TimeUnit.SECONDS; + +public class Node implements NodeCommandStoreService +{ + public static class Id implements Comparable + { + public static final Id NONE = new Id(0); + public static final Id MAX = new Id(Integer.MAX_VALUE); + + public final int id; + + public Id(int id) + { + this.id = id; + } + + @Override + public int hashCode() + { + return Integer.hashCode(id); + } + + @Override + public boolean equals(Object that) + { + return that instanceof Id && equals((Id) that); + } + + public boolean equals(Id that) + { + if (that == null) return false; + return id == that.id; + } + + @Override + public int compareTo(Id that) + { + return Integer.compareUnsigned(this.id, that.id); + } + + public String toString() + { + return Integer.toString(id); + } + } + + private final Id id; + private final MessageSink messageSink; + private final TopologyManager topology; + private final RemoteListeners listeners; + private final Timeouts timeouts; + private final CommandStores commandStores; + private final CoordinationAdapter.Factory coordinationAdapters; + + private final TimeService time; + private final UniqueTimeService uniqueTime; + private final Agent agent; + private final RandomSource random; + + private final Scheduler scheduler; + private final DurabilityService durabilityService; + + private volatile DurableBefore durableBefore = DurableBefore.EMPTY; + private DurableBefore minDurableBefore = DurableBefore.EMPTY; + private final ReentrantLock durableBeforeLock = new ReentrantLock(); + private final PersistentField persistDurableBefore; + + private final Coordinations coordinations = new Coordinations(); + private final AtomicLong nextCoordinationId = new AtomicLong(); + + /** + * Used to guard some operations that should normally operate on consistent information, but in rare cases may need to repeat work. + * For simplicity we have a global stamp counter for this. + * At present, only used for managing unavailable() computations. + */ + private volatile long stamp; + private static final AtomicLongFieldUpdater stampUpdater = AtomicLongFieldUpdater.newUpdater(Node.class, "stamp"); + private volatile boolean replaying; + + public Node(Id id, MessageSink messageSink, + TopologyService topologyService, TimeService time, UniqueTimeService uniqueTime, + Supplier dataSupplier, ShardDistributor shardDistributor, Agent agent, RandomSource random, Scheduler scheduler, TopologySorter.Supplier topologySorter, + Function remoteListenersFactory, Function requestTimeoutsFactory, Function progressLogFactory, + Function localListenersFactory, CommandStores.Factory factory, CoordinationAdapter.Factory coordinationAdapters, + Persister durableBeforePersister, + Journal journal) + { + this.id = id; + this.scheduler = scheduler; // we set scheduler first so that e.g. requestTimeoutsFactory and progressLogFactory can take references to it + this.messageSink = messageSink; + this.coordinationAdapters = coordinationAdapters; + this.time = time; + this.uniqueTime = uniqueTime; + this.timeouts = requestTimeoutsFactory.apply(this); + this.listeners = remoteListenersFactory.apply(this); + this.agent = agent; + this.random = random; + this.persistDurableBefore = new PersistentField<>(() -> durableBefore, + (input, prev) -> { + DurableBefore next = DurableBefore.merge(input, prev); + if (next.equals(prev)) + return prev; + return next.equals(prev) ? prev : next; + }, + safeDurableBeforePersister(durableBeforePersister), + this::setPersistedDurableBefore); + this.commandStores = factory.create(this, agent, dataSupplier.get(), random.fork(), journal, shardDistributor, progressLogFactory.apply(this), localListenersFactory.apply(this)); + this.topology = new TopologyManager(topologySorter, this, topologyService, time, timeouts); + this.durabilityService = new DurabilityService(this); + // TODO (desired): make frequency configurable + scheduler.recurring(() -> commandStores.forAllUnsafe(store -> store.progressLog.maybeNotify()), 1, SECONDS); + scheduler.recurring(timeouts::maybeNotify, 100, MILLISECONDS); + } + + public void load() + { + persistDurableBefore.load(); + } + + public DurabilityService durability() + { + return durabilityService; + } + + /** + * This starts the node for tests and makes sure that the provided topology is acknowledged correctly. This method is not + * safe for production systems as it doesn't handle restarts and partially acknowledged histories + * @return {@link EpochReady#active} + */ + @VisibleForTesting + public AsyncResult unsafeStart() + { + topology.topologyService().onStartup(this); + ActiveEpochs epochs = topology.active(); + if (epochs.isEmpty()) + return AsyncResults.success(null); + + return epochs.epochReady(epochs.epoch()).active; + } + + public CommandStores commandStores() + { + return commandStores; + } + + public MessageSink messageSink() + { + return messageSink; + } + + public final DurableBefore durableBefore() + { + return durableBefore; + } + + public void addNewRangesToDurableBefore(Ranges ranges, long epoch) + { + durableBeforeLock.lock(); + try + { + TxnId from = TxnId.minForEpoch(epoch); + DurableBefore addDurableBefore = DurableBefore.create(ranges, from, from); + DurableBefore newDurableBefore = DurableBefore.merge(durableBefore, addDurableBefore); + // TODO (required): it is possible for this invariant to be breached if topologies are received out of order. + // We should not update min past the max known epoch. + Invariants.require(newDurableBefore.min.quorumBefore.compareTo(durableBefore.min.quorumBefore) >= 0, + "Previous durable before: %s, new: %s", durableBefore, newDurableBefore); + + minDurableBefore = DurableBefore.merge(minDurableBefore, addDurableBefore); + durableBefore = newDurableBefore; + } + finally + { + durableBeforeLock.unlock(); + } + } + + private Persister safeDurableBeforePersister(Persister wrap) + { + return new Persister<>() + { + @Override + public AsyncResult persist(DurableBefore addValue, DurableBefore newValue) + { + Invariants.require(addValue.maxEpoch() <= epoch()); + return wrap.persist(addValue, newValue); + } + @Override public DurableBefore load() { return wrap.load(); } + }; + } + + private void setPersistedDurableBefore(DurableBefore newDurableBefore) + { + durableBeforeLock.lock(); + try + { + // TODO (desired): do not re-merge any minDurableBefore that was already known when we created the update + durableBefore = DurableBefore.merge(newDurableBefore, minDurableBefore); + } + finally + { + durableBeforeLock.unlock(); + } + } + + public AsyncResult markDurable(Ranges ranges, TxnId majorityBefore, TxnId universalBefore) + { + return markDurable(DurableBefore.create(ranges, majorityBefore, universalBefore)); + } + + public AsyncResult markDurable(DurableBefore addDurableBefore) + { + return withEpochExact(addDurableBefore.maxEpoch(), (AsyncExecutor)null, () -> persistDurableBefore.mergeAndUpdate(addDurableBefore).chain()) + .beginAsResult(); + } + + @Override + public long epoch() + { + return topology().epoch(); + } + + // TODO (required): audit use of withEpochAtLeast vs withEpochExact + // TODO (expected): provide a deadline + public void withEpochAtLeast(EpochSupplier epochSupplier, @Nullable AsyncExecutor executor, BiConsumer callback) + { + if (epochSupplier == null) + callback.accept(null, null); + else + withEpochAtLeast(epochSupplier.epoch(), executor, callback); + } + + public void withEpochAtLeast(long epoch, @Nullable AsyncExecutor ifAsync, BiConsumer callback) + { + ActiveEpochs epochs = topology().active(); + if (epochs.hasAtLeastEpoch(epoch)) + { + callback.accept(null, null); + } + else + { + topology.await(epoch, ifAsync).begin(callback); + } + } + + public Object withEpochAtLeast(long epoch, @Nullable AsyncExecutor ifAsync, BiConsumer ifFailure, Runnable ifSuccess) + { + ActiveEpochs epochs = topology().active(); + if (epochs.hasAtLeastEpoch(epoch)) + { + ifSuccess.run(); + return ifSuccess; + } + else + { + return topology.await(epoch, ifAsync).begin((success, fail) -> { + if (fail != null) ifFailure.accept(null, fail); + else ifSuccess.run(); + }); + } + } + + public void withEpochExact(long epoch, @Nullable AsyncExecutor ifAsync, BiConsumer ifFailure, Function onFailure, Runnable ifSuccess) + { + ActiveEpochs epochs = topology().active(); + if (epoch < epochs.minEpoch()) + { + ifFailure.accept(null, onFailure.apply(new TopologyRetiredException(epoch, epochs.minEpoch()))); + } + else if (epochs.hasEpoch(epoch)) + { + ifSuccess.run(); + } + else + { + topology.await(epoch, ifAsync).begin((success, fail) -> { + if (fail != null) ifFailure.accept(null, onFailure.apply(fail)); + else ifSuccess.run(); + }); + } + } + + @Inline + public AsyncChain withEpochExact(long epoch, @Nullable AsyncExecutor executor, Supplier> supplier) + { + ActiveEpochs epochs = topology().active(); + if (epoch < epochs.minEpoch()) + { + return AsyncChains.failure(new TopologyRetiredException(epoch, epochs.minEpoch())); + } + else if (epochs.hasEpoch(epoch)) + { + return supplier.get(); + } + else + { + return topology.await(epoch, executor).flatMapOverride(supplier); + } + } + + @Inline + public AsyncChain withEpochAtLeast(long epoch, @Nullable AsyncExecutor executor, Supplier> supplier) + { + ActiveEpochs epochs = topology().active(); + if (epochs.hasAtLeastEpoch(epoch)) + { + return supplier.get(); + } + else + { + return topology.await(epoch, executor).flatMapOverride(supplier); + } + } + + public void withEpochAtLeast(long epoch, @Nullable AsyncExecutor ifAsync, BiConsumer ifFailure, Function onFailure, Runnable ifSuccess) + { + ActiveEpochs epochs = topology().active(); + if (epochs.hasAtLeastEpoch(epoch)) + { + ifSuccess.run(); + } + else + { + topology.await(epoch, ifAsync).begin((success, fail) -> { + if (fail != null) ifFailure.accept(null, onFailure.apply(fail)); + else ifSuccess.run(); + }); + } + } + + + public TopologyManager topology() + { + return topology; + } + + @Override + public AsyncExecutor someExecutor() + { + return commandStores.someExecutor(); + } + + @Override + public SequentialAsyncExecutor someSequentialExecutor() + { + return commandStores.someSequentialExecutor(); + } + + public void shutdown() + { + commandStores.shutdown(); + } + + public long uniqueNow() + { + return uniqueTime.uniqueNow(); + } + + @Override + public long uniqueNow(long greaterThan) + { + return uniqueTime.uniqueNow(greaterThan); + } + + @Override + public long uniqueStale(long greaterThan) + { + return uniqueTime.uniqueStale(greaterThan); + } + + @Override + public long now() + { + return time.now(); + } + + @Override + public long elapsed(TimeUnit timeUnit) + { + return time.elapsed(timeUnit); + } + + public void send(Topologies topologies, Request send) + { + SortedArrayList nodes = topologies.nodes(); + for (int i = 0 ; i < nodes.size() ; ++i) + { + Node.Id to = nodes.get(i); + if (!topologies.isFaulty(nodes.get(i))) + send(to, send); + } + } + + public void send(Topologies topologies, Function requestFactory) + { + SortedArrayList nodes = topologies.nodes(); + for (int i = 0 ; i < nodes.size() ; ++i) + { + Node.Id to = nodes.get(i); + if (!topologies.isFaulty(nodes.get(i))) + send(to, requestFactory.apply(to)); + } + } + + public void send(Topologies topologies, Request send, @Nonnull AsyncExecutor executor, Callback callback) + { + SortedArrayList nodes = topologies.nodes(); + for (int i = 0 ; i < nodes.size() ; ++i) + { + Node.Id to = nodes.get(i); + if (!topologies.isFaulty(nodes.get(i))) + messageSink.send(to, send, executor, callback); + } + } + + // TODO (required): callback must be invoked if for any reason send fails + public void send(Topologies topologies, Function requestFactory, @Nonnull AsyncExecutor executor, Callback callback) + { + SortedArrayList nodes = topologies.nodes(); + for (int i = 0 ; i < nodes.size() ; ++i) + { + Node.Id to = nodes.get(i); + if (!topologies.isFaulty(nodes.get(i))) + messageSink.send(to, requestFactory.apply(to), executor, callback); + } + } + + // send to a specific node + public Cancellable send(Id to, Request send, @Nonnull AsyncExecutor executor, Callback callback) + { + return messageSink.send(to, send, executor, callback); + } + + // send to a specific node + public void send(Id to, Request send) + { + messageSink.send(to, send); + } + + public void reply(Id replyingToNode, ReplyContext replyContext, Reply send, Throwable failure) + { + if (failure != null) + { + agent.onException(failure); + if (send != null) + agent().onException(new IllegalArgumentException(String.format("fail (%s) and send (%s) are both not null", failure, send))); + messageSink.replyWithUnknownFailure(replyingToNode, replyContext, failure); + return; + } + else if (send == null) + { + NullPointerException e = new NullPointerException(); + agent.onException(e); + throw e; + } + messageSink.reply(replyingToNode, replyContext, send); + } + + public TxnId nextTxnIdWithDefaultFlags(Seekables keys, Txn.Kind kind, Domain domain) + { + return nextTxnIdWithFlags(keys, kind, domain, Any, defaultMediumPath().bit()); + } + + public TxnId nextStaleTxnIdWithDefaultFlags(long minEpoch, long minHlc, Seekables keys, Txn.Kind kind, Domain domain) + { + return nextStaleTxnIdWithFlags(minEpoch, minHlc, keys, kind, domain, Any, defaultMediumPath().bit()); + } + + public TxnId nextTxnIdWithDefaultFlags(Seekables keys, Txn.Kind kind, Domain domain, Cardinality cardinality) + { + return nextTxnIdWithFlags(keys, kind, domain, cardinality, defaultMediumPath().bit()); + } + + private long epoch(long minEpoch, Seekables keys, Txn.Kind kind) + { + if (!kind.isSyncPoint()) + return Math.max(minEpoch, epoch()); + + return topology.active().maxEpoch(minEpoch, ActiveEpoch::all, keys); + } + + public TxnId nextTxnIdWithDefaultFlags(long minEpoch, long minHlc, Seekables keys, Txn.Kind kind, Domain domain, Cardinality cardinality) + { + long epoch = epoch(minEpoch, keys, kind); + return newTxnId(epoch, uniqueNow(minHlc), kind, domain, cardinality, defaultMediumPath().bit(), id); + } + + /** + * TODO (required): Make sure we cannot re-issue the same txnid on startup + * TODO (required): Don't use new epoch for TxnId until a quorum is ready to coordinate it + */ + public TxnId nextTxnIdWithFlags(Seekables keys, Txn.Kind kind, Domain domain, Cardinality cardinality, int flags) + { + return newTxnId(epoch(Long.MIN_VALUE, keys, kind), uniqueNow(), kind, domain, cardinality, flags, id); + } + + public TxnId nextStaleTxnIdWithFlags(long minEpoch, long minHlc, Seekables keys, Txn.Kind kind, Domain domain, Cardinality cardinality, int flags) + { + long epoch = epoch(minEpoch, keys, kind); + long hlc = uniqueStale(minHlc); + return newTxnId(epoch, hlc, kind, domain, cardinality, flags, id); + } + + private static TxnId newTxnId(long epoch, long now, Txn.Kind kind, Domain domain, Cardinality cardinality, int flags, Node.Id node) + { + Invariants.require(domain == Key || kind != Write, "Range writes not supported without forwarding uniqueHlc information to WaitingOn for direct dependencies"); + Invariants.require(domain == Range || !kind.isSyncPoint, "Key ExclusiveSyncPoint not supported without improvements to CommandsForKey for managing execution"); + TxnId txnId = new TxnId(epoch, now, flags, kind, domain, cardinality, node); + Invariants.require((txnId.lsb & (0xffff & ~TxnId.IDENTITY_FLAGS)) == 0); + return txnId; + } + + public TxnId nextTxnId(Txn txn) + { + return nextTxnId(0, 0, txn); + } + + public TxnId nextTxnId(long minEpoch, long minHlc, Txn txn) + { + Seekables keys = txn.keys(); + Txn.Kind kind = txn.kind(); + return nextTxnId(minEpoch, minHlc, keys, kind); + } + + public TxnId nextTxnId(@Nullable Timestamp min, Seekables keys, Txn.Kind kind) + { + return nextTxnId(min == null ? 0 : min.epoch(), min == null ? 0 : min.hlc(), keys, kind); + } + + public TxnId nextTxnId(long minEpoch, long minHlc, Seekables keys, Txn.Kind kind) + { + Domain domain = keys.domain(); + Cardinality cardinality = cardinality(domain, keys); + + if (!usePrivilegedCoordinator() || (kind != Read && kind != Write)) + return nextTxnIdWithDefaultFlags(minEpoch, minHlc, keys, kind, domain, cardinality); + + long epoch = epoch(minEpoch, keys, kind); + long hlc = uniqueNow(minHlc); + int flags = computeBestDefaultTxnIdFlags(keys, epoch); + TxnId txnId = new TxnId(epoch, hlc, flags, kind, domain, cardinality, id); + Invariants.require((txnId.lsb & (0xffff & ~TxnId.IDENTITY_FLAGS)) == 0); + return txnId; + } + + private int computeBestDefaultTxnIdFlags(Routables keys, long epoch) + { + ActiveEpochs epochs = topology().active(); + if (!epochs.hasEpoch(epoch) || !usePrivilegedCoordinator()) + return defaultMediumPath().bit(); + + TxnId.FastPath fastPath = ensurePermitted(epochs.selectFastPath(keys, epoch)); + return fastPath.bits | defaultMediumPath().bit(); + } + + public TxnId nextTxnId(Txn txn, TxnId.FastPath fastPath, TxnId.MediumPath mediumPath) + { + ActiveEpochs epochs = topology().active(); + Seekables keys = txn.keys(); + Txn.Kind kind = txn.kind(); + Domain domain = keys.domain(); + + long epoch = epoch(Long.MIN_VALUE, keys, kind); + long now = uniqueNow(); + fastPath = ensurePermitted(fastPath); + if (fastPath != Unoptimised && (!epochs.hasEpoch(epoch) || !epochs.supportsPrivilegedFastPath(keys, epoch))) + fastPath = Unoptimised; + + Cardinality cardinality = cardinality(domain, keys); + return newTxnId(epoch, now, kind, domain, cardinality, fastPath.bits | mediumPath.bit(), id); + } + + public AsyncChain coordinate(Txn txn) + { + TxnId txnId = nextTxnId(txn); + return coordinate(txnId, txn); + } + + public AsyncChain coordinate(TxnId txnId, Txn txn) + { + return coordinate(txnId, txn, txnId.epoch(), Long.MAX_VALUE); + } + + // TODO (required): plumb deadlineNanos in (perhaps on integration side, but maybe introduce some context we can pass through for the MessageSink) + public AsyncChain coordinate(TxnId txnId, Txn txn, long minEpoch, long deadlineNanos) + { + return withEpochExact(Math.max(txnId.epoch(), minEpoch), (AsyncExecutor) null, () -> initiateCoordination(txnId, txn)); + } + + private AsyncChain initiateCoordination(TxnId txnId, Txn txn) + { + if (txnId.kind() == Txn.Kind.EphemeralRead) + return CoordinateEphemeralRead.coordinate(this, txnId, txn); + else + return CoordinateTransaction.coordinate(this, txnId, txn); + } + + public FullRoute computeRoute(TxnId txnId, Routables keysOrRanges) throws TopologyException + { + return computeRoute(txnId.epoch(), keysOrRanges, topology.active()); + } + + public FullRoute computeRoute(long epoch, Routables keysOrRanges, ActiveEpochs active) throws TopologyException + { + Invariants.requireArgument(!keysOrRanges.isEmpty(), "Attempted to compute a route from empty keys or ranges"); + + RoutingKey homeKey = selectHomeKey(active.get(epoch), keysOrRanges); + + return keysOrRanges.toRoute(homeKey); + } + + private RoutingKey selectHomeKey(ActiveEpoch e, Routables keysOrRanges) + { + Ranges owned = e.local().ranges(); + int i = (int)keysOrRanges.findNextIntersection(0, owned, 0); + if (i >= 0) + return keysOrRanges.get(i).someIntersectingRoutingKey(owned); + + return keysOrRanges.get(random.nextInt(keysOrRanges.size())).someIntersectingRoutingKey(null); + } + + public AsyncChain recover(TxnId txnId, InvalidIf invalidIf, FullRoute route, LatentStoreSelector reportTo) + { + SequentialAsyncExecutor executor = someSequentialExecutor(); + return withEpochExact(txnId.epoch(), executor, () -> new AsyncChains.Head<>() + { + @Override + protected Cancellable start(BiConsumer callback) + { + PrepareRecovery.recover(Node.this, executor, txnId, invalidIf, route, null, reportTo, callback); + return null; + } + }); + } + + public void receive(Request request, Id from, ReplyContext replyContext) + { + long waitForEpoch = request.waitForEpoch(); + withEpochAtLeast(waitForEpoch, null, agent, () -> { + try + { + request.process(this, from, replyContext); + } + catch (Throwable t) + { + reply(from, replyContext, null, t); + } + }); + } + + public CoordinationAdapter coordinationAdapter(TxnId txnId, Kind kind) + { + return coordinationAdapters.get(txnId, kind); + } + + public AsyncChain updateMinHlc(long minHlc) + { + // TODO (required): command stores that are not ready due to bootstrap need to refresh their min HLC on bootstrap completion + StoreSelector selector = snapshot -> Stream.of(snapshot.shards).map(sh -> sh.store).iterator(); + return commandStores().mapReduce(selector, new MapReduceCommandStores<>(RoutingKeys.EMPTY) + { + @Override public Void reduce(Void o1, Void o2) { return null; } + @Override public TxnId primaryTxnId() { return null; } + @Override public String reason() { return "Update Min HLC"; } + @Override protected Void applyInternal(SafeCommandStore safeStore) + { + safeStore.commandStore().updateMinHlc(minHlc); + return null; + } + }); + } + + public Scheduler scheduler() + { + return scheduler; + } + + public Agent agent() + { + return agent; + } + + public RemoteListeners remoteListeners() + { + return listeners; + } + + @Override + public Timeouts timeouts() + { + return timeouts; + } + + @Override + public Id id() + { + return id; + } + + @Override + public String toString() + { + return "Node{" + id + '}'; + } + + @VisibleForTesting + public CommandStore unsafeForKey(RoutingKey key) + { + return commandStores.unsafeForKey(key); + } + + public CommandStore unsafeByIndex(int index) + { + return commandStores.current.shards[index].store; + } + + public TimeService time() + { + return time; + } + + public final long currentStamp() + { + return stamp; + } + + public long nextCoordinationId() + { + long startedAtNanos = time.elapsed(NANOSECONDS); + long nextId = nextCoordinationId.get(); + if (startedAtNanos >= nextId && nextCoordinationId.compareAndSet(nextId, startedAtNanos)) + return startedAtNanos; + return nextCoordinationId.incrementAndGet(); + } + + public void register(Coordination coordination) + { + coordinations.register(coordination); + } + + public void unregister(Coordination coordination) + { + coordinations.unregister(coordination); + } + + public Coordinations coordinations() + { + return coordinations; + } + + public boolean isCoordinatingWithBallot(TxnId txnId, Ballot ballot) + { + long mostRecent = coordinations.mostRecent(txnId, COORDINATES_STATE_MACHINE, ballot); + if (mostRecent < 0) + return false; + long ageNanos = Math.max(recentElapsed(NANOSECONDS) - mostRecent, 0); + return !agent.isSlowCoordinator(ageNanos, NANOSECONDS, txnId, 1); + } + + public void updateStamp() + { + stampUpdater.incrementAndGet(this); + } + + @Override + public boolean isReplaying() + { + return replaying; + } + + public void unsafeSetReplaying(boolean replaying) + { + this.replaying = replaying; + if (replaying) CommandsForKey.disableLinearizabilityViolationsReporting(); + else CommandsForKey.enableLinearizabilityViolationsReporting(); + } +} \ No newline at end of file diff --git a/accord-core/src/main/java/accord/local/OverlappingCommandStoresException.java b/accord-core/src/main/java/accord/local/OverlappingCommandStoresException.java new file mode 100644 index 000000000000..f1e358d655e2 --- /dev/null +++ b/accord-core/src/main/java/accord/local/OverlappingCommandStoresException.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package accord.local; + +import accord.utils.Rethrowable; + +public class OverlappingCommandStoresException extends RuntimeException implements Rethrowable +{ + public OverlappingCommandStoresException() + { + } + + private OverlappingCommandStoresException(Throwable cause) + { + super(cause); + } + + @Override + public OverlappingCommandStoresException rethrowable() + { + return new OverlappingCommandStoresException(this); + } +} diff --git a/accord-core/src/main/java/accord/local/SafeCommandStore.java b/accord-core/src/main/java/accord/local/SafeCommandStore.java new file mode 100644 index 000000000000..8d0b06f6b7b5 --- /dev/null +++ b/accord-core/src/main/java/accord/local/SafeCommandStore.java @@ -0,0 +1,676 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package accord.local; + +import java.util.ArrayList; +import java.util.List; +import java.util.NavigableMap; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + +import accord.api.Agent; +import accord.api.DataStore; +import accord.api.LocalListeners; +import accord.api.ProgressLog; +import accord.api.RoutingKey; +import accord.local.CommandStores.RangesForEpoch; +import accord.local.CommandStores.RangesForEpochSupplier; +import accord.local.RedundantBefore.RedundantBeforeSupplier; +import accord.local.cfk.CommandsForKey; +import accord.local.cfk.SafeCommandsForKey; +import accord.local.cfk.UpdateUnmanagedMode; +import accord.primitives.AbstractUnseekableKeys; +import accord.primitives.KeyDeps; +import accord.primitives.Participants; +import accord.primitives.RangeDeps; +import accord.primitives.Ranges; +import accord.primitives.RoutingKeys; +import accord.primitives.SaveStatus; +import accord.primitives.Status; +import accord.primitives.Timestamp; +import accord.primitives.TxnId; +import accord.primitives.Unseekables; +import accord.utils.Invariants; +import accord.utils.Reduce; +import accord.utils.LargeBitSet; +import accord.utils.SortedList; +import accord.utils.async.AsyncChain; +import accord.utils.async.AsyncChains; + +import static accord.local.LoadKeys.INCR; +import static accord.local.LoadKeys.NONE; +import static accord.local.LoadKeysFor.WRITE; +import static accord.local.RedundantStatus.Property.LOCALLY_APPLIED; +import static accord.local.RedundantStatus.SomeStatus.LOCALLY_WITNESSED_ONLY; +import static accord.local.RedundantStatus.Property.LOCALLY_REDUNDANT; +import static accord.local.RedundantStatus.Property.SHARD_APPLIED; +import static accord.local.cfk.UpdateUnmanagedMode.REGISTER; +import static accord.primitives.Known.KnownRoute.MaybeRoute; +import static accord.primitives.Routable.Domain.Range; +import static accord.primitives.Routables.Slice.Minimal; +import static accord.primitives.SaveStatus.Applied; +import static accord.primitives.SaveStatus.Committed; +import static accord.primitives.SaveStatus.TruncatedApply; +import static accord.primitives.SaveStatus.Uninitialised; +import static accord.primitives.Timestamp.Flag.SHARD_BOUND; +import static accord.primitives.Txn.Kind.VisibilitySyncPoint; +import static accord.utils.Invariants.illegalArgument; +import static accord.utils.Invariants.illegalState; + +/** + * A CommandStore with exclusive access; a reference to this should not be retained outside of the scope of the method + * that it is passed to. For the duration of the method invocation only, the methods on this interface are safe to invoke. + * + * Method implementations may therefore be single threaded, without volatile access or other concurrency control + */ +public abstract class SafeCommandStore implements RangesForEpochSupplier, RedundantBeforeSupplier, CommandSummaries +{ + private static final int MAX_REENTRANCY = 50; + private int reentrancyCounter; + public boolean tryRecurse() + { + if (reentrancyCounter == MAX_REENTRANCY) + return false; + ++reentrancyCounter; + return true; + } + public void unrecurse() + { + --reentrancyCounter; + Invariants.require(reentrancyCounter >= 0); + } + + public final boolean refusesAnyOf(Participants participants) + { + Ranges refuses = commandStore().refuses; + return refuses != null && participants.intersects(refuses); + } + + public final boolean refusesAllOwnedOf(Participants participants) + { + Ranges refuses = commandStore().refuses; + if (refuses == null) + return false; + + // TODO (required): memoize this, and expose it as a standard method as we want it elsewhere + Ranges notRetired = redundantBefore().removeLocallyRetired(ranges().all()); + return refuses.containsAll(participants.slice(notRetired, Minimal)); + } + + /** + * If the transaction exists (with some associated data) in the CommandStore, return it. Otherwise return null. + * + * This is useful for operations that do not retain a route, but do expect to operate on existing local state; + * this guards against recreating a previously truncated command when we do not otherwise have enough information + * to prevent it. + */ + public @Nullable SafeCommand ifInitialised(TxnId txnId) + { + SafeCommand safeCommand = getInternal(txnId); + if (safeCommand == null) + return null; + Command command = safeCommand.current(); + if (command.saveStatus().isUninitialised()) + return null; + return maybeCleanup(safeCommand); + } + + // decidedExecuteAt == null if not yet PreCommitted + + /** + * Retrieve a SafeCommand. If it is initialised, optionally use its present contents to determine if it should be + * truncated, and apply the truncation before returning the command. + * This behaviour may be overridden by implementations if they know any truncation would already have been applied. + * + * If it is not initialised, use the provided parameters to determine if the record may have been expunged; + * if not, create it. + * + * We do not distinguish between participants, home keys, and non-participating home keys for now, even though + * these fundamentally have different implications. Logically, we may erase a home shard's record as soon as + * the transaction has been made durable at a majority of replicas of every shard, and state for any participating + * keys may be erased as soon as their non-faulty peers have recorded the outcome. + * + * However if in some cases we don't know which commands are home keys or participants we need to wait to erase + * a transaction until both of these criteria are met for every key. + * + * TODO (desired): Introduce static types that permit us to propagate this information safely. + */ + public SafeCommand get(TxnId txnId, StoreParticipants participants) + { + SafeCommand safeCommand = getInternal(txnId); + if (safeCommand == null) + throw notFound(txnId); + + return maybeCleanup(safeCommand, participants); + } + + protected SafeCommand get(TxnId txnId) + { + SafeCommand safeCommand = getInternal(txnId); + if (safeCommand == null) + throw notFound(txnId); + + return maybeCleanup(safeCommand); + } + + public SafeCommand unsafeGet(TxnId txnId) + { + return get(txnId); + } + + public SafeCommand unsafeGetNoCleanup(TxnId txnId) + { + return getInternal(txnId); + } + + private RuntimeException notFound(TxnId txnId) + { + if (context().txnIds().contains(txnId)) throw illegalState("%s was specified in %s but was not returned by getInternal(key)", txnId, context().txnIds()); + else throw illegalArgument("%s was not specified in %s", txnId, context().txnIds()); + } + + protected SafeCommand maybeCleanup(SafeCommand safeCommand) + { + Command command = safeCommand.current(); + Commands.maybeCleanup(this, safeCommand, command, command.participants()); + return safeCommand; + } + + protected SafeCommand maybeCleanup(SafeCommand safeCommand, @Nonnull StoreParticipants supplemental) + { + Command command = safeCommand.current(); + StoreParticipants participants = command.participants().supplementOrMerge(command.saveStatus(), supplemental); + Commands.maybeCleanup(this, safeCommand, command, participants); + return safeCommand; + } + + /** + * If the transaction is in memory, return it (and make it visible to future invocations of {@code command}, {@code ifPresent} etc). + * Otherwise return null. + * + * This permits efficient operation when a transaction involved in processing another transaction happens to be in memory. + */ + public SafeCommand ifLoadedAndInitialised(TxnId txnId) + { + SafeCommand safeCommand = getInternal(txnId); + if (safeCommand == null) + { + safeCommand = ifLoadedInternal(txnId); + if (safeCommand == null) + return null; + } + + if (safeCommand.isUnset() || safeCommand.current().saveStatus() == Uninitialised) + return null; + + return maybeCleanup(safeCommand); + } + + protected SafeCommandsForKey maybeCleanup(SafeCommandsForKey safeCfk) + { + RedundantBefore.Bounds bounds = redundantBefore().get(safeCfk.key().toUnseekable()); + if (bounds != null) + safeCfk.updateRedundantBefore(this, bounds); + return safeCfk; + } + + /** + * If the transaction is in memory, return it (and make it visible to future invocations of {@code command}, {@code ifPresent} etc). + * Otherwise return null. + * + * This permits efficient operation when a transaction involved in processing another transaction happens to be in memory. + */ + public final SafeCommandsForKey ifLoadedAndInitialised(RoutingKey key) + { + SafeCommandsForKey safeCfk = getInternal(key); + if (safeCfk != null) + return safeCfk; + + safeCfk = ifLoadedInternal(key); + if (safeCfk == null) + return null; + return maybeCleanup(safeCfk); + } + + public SafeCommandsForKey get(RoutingKey key) + { + SafeCommandsForKey safeCfk = getInternal(key); + if (safeCfk != null) + return maybeCleanup(safeCfk); + + if (context().loadKeys() != NONE && context().keys().contains(key)) throw illegalState("%s was specified in %s but was not returned by getInternal(key)", key, context().keys()); + else throw illegalArgument("%s was not specified in %s", key, context()); + } + + /** Get anything already referenced (should include anything in PreLoadContext). If returned, should be initialised. */ + protected abstract SafeCommand getInternal(TxnId txnId); + /** Get if available */ + protected abstract SafeCommand ifLoadedInternal(TxnId txnId); + /** Get anything already referenced (should include anything in PreLoadContext) */ + protected abstract SafeCommandsForKey getInternal(RoutingKey key); + /** Get if available */ + protected abstract SafeCommandsForKey ifLoadedInternal(RoutingKey key); + + public final boolean canExecuteWith(PreLoadContext context) { return canExecute(context) == context; } + + /** + * Attempt to ready the provided PreLoadContext; if this can only be achieved partially, a new PreLoadContext + * will be returned containing the readily available data. If nothing is available, null will be returned. + */ + public abstract @Nullable PreLoadContext canExecute(PreLoadContext context); + + /** + * The current PreLoadContext, excluding any upgrade. + */ + public abstract PreLoadContext context(); + + protected void update(Command prev, Command updated, boolean force) + { + updateMaxConflicts(prev, updated, force); + updateCommandsForKey(prev, updated, force); + updateExclusiveSyncPoint(prev, updated, force); + } + + public void updateExclusiveSyncPoint(Command prev, Command updated, boolean force) + { + if (!updated.txnId().isSyncPoint() || updated.txnId().domain() != Range) return; + if (updated.route() == null) return; + + List listeners = commandStore().syncPointListeners; + if (listeners != null) + { + for (SyncPointListener listener : listeners) + listener.update(this, updated); + } + + SaveStatus oldSaveStatus = prev == null ? SaveStatus.Uninitialised : prev.saveStatus(); + SaveStatus newSaveStatus = updated.saveStatus(); + + if (newSaveStatus.known.isDefinitionKnown() && (force || !oldSaveStatus.known.isDefinitionKnown())) + { + Ranges ranges = updated.participants().touches().toRanges(); + commandStore().markExclusiveSyncPoint(this, updated.txnId(), ranges); + } + + if (newSaveStatus.compareTo(Committed) >= 0 && newSaveStatus.compareTo(TruncatedApply) <= 0 && (force || oldSaveStatus.compareTo(Committed) < 0)) + { + Ranges ranges = updated.participants().owns().toRanges(); + commandStore().markExclusiveSyncPointDecided(this, updated.txnId(), ranges); + } + + if (newSaveStatus == Applied && (force || oldSaveStatus != Applied)) + { + Ranges ranges = updated.participants().touches().toRanges(); + TxnId txnIdWithFlags = (TxnId)updated.executeAt(); + commandStore().markExclusiveSyncPointLocallyApplied(this, txnIdWithFlags, ranges); + } + + if (updated.partialDeps() != null) + { + RedundantBefore addRedundantBefore = RedundantBefore.EMPTY; + RangeDeps deps = updated.partialDeps().rangeDeps; + for (int i = 0 ; i < deps.txnIdCount() ; ++i) + { + TxnId txnId = deps.txnIdWithFlags(i); + if (txnId.is(SHARD_BOUND)) + { + Ranges ranges = deps.ranges(txnId).slice(ranges().all(), Minimal); + addRedundantBefore = RedundantBefore.merge(addRedundantBefore, RedundantBefore.create(ranges, txnId, LOCALLY_WITNESSED_ONLY)); + } + } + // TODO (expected): we should be able to use unsafeUpsertRedundantBefore here as the implementation will replay deps + // BUT the implementation may not replay all versions of the partialDeps we report here. + // This is likely to still be fine, as we don't imply anything for GC, but we won't do it for the moment. + // We might instead prefer to report these deps only once we are certain they won't change (i.e. when Stable), + // BUT in this case we cannot update waitingOnSync until then either (also probably fine) + if (addRedundantBefore != RedundantBefore.EMPTY) + upsertRedundantBefore(addRedundantBefore); + } + } + + public void updateMaxConflicts(Command prev, Command updated, boolean force) + { + SaveStatus oldSaveStatus = prev == null ? SaveStatus.Uninitialised : prev.saveStatus(); + SaveStatus newSaveStatus = updated.saveStatus(); + if (newSaveStatus.status.equals(oldSaveStatus.status) && oldSaveStatus.known.definition().isKnown() && !force) + return; + + TxnId txnId = updated.txnId(); + if (!txnId.isVisible()) + return; + + commandStore().updateMaxConflicts(prev, updated, force); + } + + /** + * Methods that implementors can use to capture changes to auxiliary collections: + */ + + public abstract void upsertRedundantBefore(RedundantBefore addRedundantBefore); + + protected void unsafeSetRedundantBefore(RedundantBefore newRedundantBefore) + { + commandStore().unsafeSetRedundantBefore(newRedundantBefore); + } + + protected void unsafeUpsertRedundantBefore(RedundantBefore addRedundantBefore) + { + commandStore().unsafeUpsertRedundantBefore(addRedundantBefore); + commandStore().updatedRedundantBefore(this, addRedundantBefore); + } + + public void setBootstrapBeganAt(NavigableMap newBootstrapBeganAt) + { + commandStore().unsafeSetBootstrapBeganAt(newBootstrapBeganAt); + } + + public void setSafeToRead(NavigableMap newSafeToRead) + { + commandStore().unsafeSetSafeToRead(newSafeToRead); + } + + public void setPermanentlyUnsafeToRead(Ranges newPermanentlyUnsafeToRead) + { + commandStore().unsafeSetPermanentlyUnsafeToRead(newPermanentlyUnsafeToRead); + } + + public void setRangesForEpoch(RangesForEpoch rangesForEpoch) + { + commandStore().unsafeSetRangesForEpoch(rangesForEpoch); + } + + public void updateCommandsForKey(Command prev, Command next, boolean force) + { + if (!CommandsForKey.needsUpdate(this, prev, next) && !force) + return; + + TxnId txnId = next.txnId(); + if (CommandsForKey.manages(txnId)) updateManagedCommandsForKey(this, prev, next, force); + if (!CommandsForKey.managesExecution(txnId) && next.hasBeen(Status.Stable) && !next.hasBeen(Status.Truncated) && (force || !prev.hasBeen(Status.Stable))) + updateUnmanagedCommandsForKey(this, next, REGISTER); + // TODO (expected): register deps during Accept phase to more quickly sync epochs +// else if (txnId.is(Range) && next.known().deps.hasProposedOrDecidedDeps()) +// updateUnmanagedCommandsForKey(this, next, REGISTER_DEPS_ONLY); + } + + abstract protected void persistFieldUpdates(); + + private static void updateManagedCommandsForKey(SafeCommandStore safeStore, Command prev, Command next, boolean forceNotify) + { + StoreParticipants participants = next.participants().supplement(prev.participants()); + Participants update = next.hasBeen(Status.Committed) ? participants.hasTouched() : participants.stillTouches(); + if (update.isEmpty()) + return; + + // TODO (expected): we don't want to insert any dependencies for those we only touch; we just need to record them as decided/applied for execution + PreLoadContext context = PreLoadContext.contextFor(next.txnId(), update, INCR, WRITE, "Update CommandsForKey"); + PreLoadContext execute = safeStore.canExecute(context); + if (execute != null) + { + updateManagedCommandsForKey(safeStore, execute.keys(), next.txnId(), forceNotify); + } + if (execute != context) + { + if (execute != null) + context = PreLoadContext.contextFor(next.txnId(), update.without(execute.keys()), INCR, WRITE, "Update CommandsForKey"); + + Invariants.require(!context.keys().isEmpty()); + safeStore = safeStore; // prevent accidental usage inside lambda + safeStore.commandStore().execute(context, safeStore0 -> { + PreLoadContext ctx = safeStore0.context(); + TxnId txnId = ctx.primaryTxnId(); + Unseekables keys = ctx.keys(); + updateManagedCommandsForKey(safeStore0, keys, txnId, forceNotify); + }, safeStore.commandStore().agent); + } + } + + private static void updateManagedCommandsForKey(SafeCommandStore safeStore, Unseekables update, TxnId txnId, boolean forceNotify) + { + // TODO (expected): avoid reentrancy / recursion + SafeCommand safeCommand = safeStore.get(txnId); + for (RoutingKey key : (AbstractUnseekableKeys)update) + { + // we use callback and re-fetch current to guard against reentrancy causing + // us to interact with "future" or stale information (respectively) + safeStore.get(key).callback(safeStore, safeCommand.current(), forceNotify); + } + } + + private static void updateUnmanagedCommandsForKey(SafeCommandStore safeStore, Command next, UpdateUnmanagedMode mode) + { + TxnId txnId = next.txnId(); + RoutingKeys keys; + + if (!txnId.isSyncPoint()) keys = next.asCommitted().waitingOn().keys; + else + { + Command.WaitingOn waitingOn = next.asCommitted().waitingOn; + RedundantBefore redundantBefore = safeStore.redundantBefore(); + KeyDeps deps = next.partialDeps().keyDeps; + keys = deps.keys(); + LargeBitSet select = new LargeBitSet(keys.size()); + for (int i = 0 ; i < keys.size() ; ++i) + { + if (waitingOn.isWaitingOnKey(i)) + { + select.set(i); + continue; + } + + SortedList txnIdsForKey = deps.txnIdsForKeyIndex(i); + RoutingKey key = keys.get(i); + TxnId maxTxnId = txnIdsForKey.get(txnIdsForKey.size() - 1); + // TODO (desired): convert to O(n) merge + RedundantStatus status = redundantBefore.status(maxTxnId, null, key); + if (!status.all(SHARD_APPLIED, LOCALLY_APPLIED) || !status.all(LOCALLY_REDUNDANT)) // TODO (required): should be a new property for correct merge? + select.set(i); + } + if (select.getSetBitCount() != keys.size()) + { + RoutingKey[] array = new RoutingKey[select.getSetBitCount()]; + int count = 0; + for (int i = 0 ; i < keys.size() ; ++i) + { + if (select.get(i)) + array[count++] = keys.get(i); + } + keys = RoutingKeys.ofSortedUnique(array); + } + } + // TODO (required): use StoreParticipants.executes() + // TODO (required): consider how execution works for transactions that await future deps and where the command store inherits additional keys in execution epoch + PreLoadContext context = PreLoadContext.contextFor(txnId, keys, INCR, WRITE, "Update Unmanaged CommandsForKey"); + PreLoadContext execute = safeStore.canExecute(context); + // TODO (expected): execute immediately for any keys we already have loaded, and save only those we haven't for later + if (execute != null) + { + updateUnmanagedCommandsForKey(safeStore, execute.keys(), txnId, mode); + } + if (execute == context) + { + if (next.txnId().is(Range) && next.txnId().is(VisibilitySyncPoint)) + registerTransitiveRangeDeps(safeStore, txnId, next); + } + else + { + if (execute != null) + context = PreLoadContext.contextFor(txnId, keys.without(execute.keys()), INCR, WRITE, "Update Unmanaged CommandsForKey"); + + safeStore = safeStore; + CommandStore unsafeStore = safeStore.commandStore(); + AsyncChain submit = unsafeStore.chain(context, safeStore0 -> { updateUnmanagedCommandsForKey(safeStore0, safeStore0.context().keys() , txnId, mode); }); + if (next.txnId().is(Range)) + submit = submit.flatMap(success -> unsafeStore.chain(PreLoadContext.contextFor(txnId, "Register Transitive Dependencies"), safeStore0 -> { registerTransitiveRangeDeps(safeStore0, txnId, next); })); + submit.begin(safeStore.commandStore().agent); + } + } + + private static TxnId maxTxnId(KeyDeps keyDeps, RoutingKey key) + { + int i = keyDeps.keys().indexOf(key); + if (i < 0) + return TxnId.NONE; + SortedList txnIdsForKey = keyDeps.txnIdsForKeyIndex(i); + return txnIdsForKey.get(txnIdsForKey.size() - 1); + } + + private static void updateUnmanagedCommandsForKey(SafeCommandStore safeStore, Unseekables update, TxnId txnId, UpdateUnmanagedMode mode) + { + SafeCommand safeCommand = safeStore.get(txnId); + for (RoutingKey key : (AbstractUnseekableKeys)update) + { + safeStore.get(key).registerUnmanaged(safeStore, safeCommand, mode); + } + } + + private static void registerTransitiveRangeDeps(SafeCommandStore safeStore, TxnId syncId, Command syncCommand) + { + if (!syncId.is(VisibilitySyncPoint)) + return; + + CommandStore commandStore = safeStore.commandStore(); + Ranges touches = syncCommand.participants().touches().toRanges(); + Ranges waitingOn = commandStore.isWaitingOnVisibility(syncId, touches); + if (waitingOn.isEmpty()) + return; + + List> async = new ArrayList<>(); + RangeDeps rangeDeps = syncCommand.partialDeps().rangeDeps; + rangeDeps.forEachUniqueTxnId(waitingOn, null, (ignore, txnIdWithFlags) -> { + TxnId txnId = txnIdWithFlags.withoutNonIdentityFlags(); + PreLoadContext context = PreLoadContext.contextFor(txnId, "Register Transitive Range Deps"); + Ranges ranges = rangeDeps.ranges(txnId); + if (safeStore.canExecuteWith(context)) registerTransitive(safeStore, txnId, ranges); + else async.add(safeStore.commandStore().chain(context, safeStore0 -> { + registerTransitive(safeStore0, txnId, ranges); + })); + }); + + AsyncChains.chain(() -> commandStore.markingVisible(syncId, waitingOn)) + .flatMap(ignore -> AsyncChains.reduce(async, Reduce.toNull(), null)) + .begin((success, fail) -> { + if (fail == null) commandStore.execute((PreLoadContext.Empty)() -> "Mark Synced", safeStore0 -> commandStore.markVisible(safeStore0, syncId, waitingOn)); + else commandStore.execute((PreLoadContext.Empty)() -> "Unmark Syncing", safeStore0 -> commandStore.cancelMarkingVisible(syncId, waitingOn)); + }); + } + + private static void registerTransitive(SafeCommandStore safeStore, TxnId txnId, Ranges witnessedBy) + { + SafeCommand safeCommand = safeStore.unsafeGet(txnId); + if (safeCommand != null && safeCommand.current().known().route() != MaybeRoute) + return; + + RangesForEpoch rangesForEpoch = safeStore.ranges(); + // TODO (required): this is incompatible with rebootstrap - we need to use some additional condition + witnessedBy = witnessedBy.without(rangesForEpoch.coordinates(txnId)); // already coordinates, no need to replicate + if (witnessedBy.isEmpty()) + return; + + witnessedBy = witnessedBy.slice(rangesForEpoch.allSince(txnId.epoch()), Minimal); // never coordinated, no need to replicate for dependency or recovery calculations + if (witnessedBy.isEmpty()) + return; + + safeCommand.updateParticipants(safeStore, safeCommand.current().participants().supplement(null, witnessedBy)); + } + + public abstract CommandStore commandStore(); + public abstract DataStore dataStore(); + public abstract Agent agent(); + public abstract ProgressLog progressLog(); + public abstract NodeCommandStoreService node(); + public abstract RangesForEpoch ranges(); + + protected NavigableMap bootstrapBeganAt() + { + return commandStore().unsafeGetBootstrapBeganAt(); + } + + public NavigableMap safeToReadAt() + { + return commandStore().unsafeGetSafeToRead(); + } + + public RedundantBefore redundantBefore() + { + return commandStore().unsafeGetRedundantBefore(); + } + + public MaxDecidedRX maxDecidedRX() + { + return commandStore().unsafeGetMaxDecidedRX(); + } + + public DurableBefore durableBefore() + { + return commandStore().node.durableBefore(); + } + + public Ranges futureRanges(TxnId txnId) + { + return ranges().allBefore(txnId.epoch()); + } + + public Ranges coordinateRanges(TxnId txnId) + { + return ranges().allAt(txnId.epoch()); + } + + public Ranges ranges(TxnId txnId, long untilLocalEpoch) + { + return ranges().allBetween(txnId.epoch(), untilLocalEpoch); + } + + public final Ranges safeToReadAt(Timestamp at) + { + return safeToReadAt().lowerEntry(at).getValue(); + } + + public @Nonnull Ranges unsafeToReadAt(Timestamp at) + { + return ranges().allAt(at).without(safeToReadAt(at)); + } + + public void registerListener(SafeCommand listeningTo, SaveStatus await, TxnId waiting) + { + Invariants.require(listeningTo.current().saveStatus().compareTo(await) < 0); + Invariants.require(!CommandsForKey.managesExecution(listeningTo.txnId())); + commandStore().listeners.register(listeningTo.txnId(), await, waiting); + } + + public LocalListeners.Registered register(TxnId txnId, LocalListeners.ComplexListener listener) + { + return commandStore().listeners.register(txnId, listener); + } + + public void notifyListeners(SafeCommand safeCommand, Command prev) + { + commandStore().listeners.notify(this, safeCommand, prev); + } + + public void register(SyncPointListener syncPointListener) + { + commandStore().unsafeRegister(syncPointListener); + } + + public void unregister(SyncPointListener syncPointListener) + { + commandStore().unsafeUnregister(syncPointListener); + } +} diff --git a/accord-core/src/main/java/accord/topology/TopologyManager.java b/accord-core/src/main/java/accord/topology/TopologyManager.java new file mode 100644 index 000000000000..282ae9ef3481 --- /dev/null +++ b/accord-core/src/main/java/accord/topology/TopologyManager.java @@ -0,0 +1,590 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package accord.topology; + +import java.util.IdentityHashMap; +import java.util.Map; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.Executor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Function; +import java.util.function.Supplier; +import javax.annotation.Nullable; + +import com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import accord.api.AsyncExecutor; +import accord.api.TopologyListener; +import accord.api.Timeouts; +import accord.api.TopologyService; +import accord.api.TopologySorter; +import accord.api.VisibleForImplementation; +import accord.local.Node; +import accord.local.Node.Id; +import accord.local.TimeService; +import accord.primitives.Ranges; +import accord.primitives.TxnId; +import accord.topology.TopologyCollector.BestFastPath; +import accord.topology.TopologyCollector.Simple; +import accord.topology.TopologyCollector.SupportsPrivilegedFastPath; +import accord.utils.Invariants; +import accord.utils.async.AsyncChain; +import accord.utils.async.AsyncChains; +import accord.utils.async.AsyncResult; +import accord.utils.async.AsyncResults; +import accord.utils.async.NestedAsyncResult; + +import static accord.primitives.AbstractRanges.UnionMode.MERGE_ADJACENT; +import static accord.primitives.Routables.Slice.Minimal; + +/** + * Manages topology state changes and update bookkeeping + * + * Each time the topology changes we need to: + * * confirm previous owners of ranges we replicate are aware of the new config + * * learn of any outstanding operations for ranges we replicate + * * clean up obsolete data + * + * Assumes a topology service that won't report epoch n without having n-1 etc also available + * + * TODO (desired, efficiency/clarity): make TopologyManager a Topologies and copy-on-write update to it, + * so we can always just take a reference for transactions instead of copying every time (and index into it by the txnId.epoch) + */ +public class TopologyManager +{ + private static final Logger logger = LoggerFactory.getLogger(TopologyManager.class); + private static final PendingEpoch SUCCESS; + + static + { + SUCCESS = new PendingEpoch(-1L, null); + SUCCESS.setActive(); + } + + final TopologySorter.Supplier sorter; + final Simple liveCollector, allCollector; + final BestFastPath bestFastPath; + final SupportsPrivilegedFastPath supportsPrivilegedFastPath; + final Node node; + final TopologyService topologyService; + final TimeService time; + final Timeouts timeouts; + private volatile ActiveEpochs active; + private final PendingEpochs pending; + private final CopyOnWriteArrayList listeners = new CopyOnWriteArrayList<>(); + + public TopologyManager(TopologySorter.Supplier sorter, Node node, TopologyService topologyService, TimeService time, Timeouts timeouts) + { + this.sorter = sorter; + this.liveCollector = new Simple(sorter, SelectShards.LIVE); + this.allCollector = new Simple(sorter, SelectShards.ALL); + this.bestFastPath = new BestFastPath(node.id()); + this.supportsPrivilegedFastPath = new SupportsPrivilegedFastPath(node.id()); + this.node = node; + this.time = time; + this.timeouts = timeouts; + this.topologyService = topologyService; + this.active = new ActiveEpochs(this, new ActiveEpoch[0], -1); + this.pending = new PendingEpochs(this); + } + + public void onReadyToCoordinate(Id node, long epoch) + { + synchronized (this) + { + if (epoch >= active.minEpoch()) + active.onReadyToCoordinate(node, epoch); + if (epoch > active.currentEpoch) + pending.remoteReadyToCoordinate(node, epoch); + } + for (TopologyListener listener : listeners) + listener.onRemoteReadyToCoordinate(node, epoch); + } + + public void onEpochClosed(Ranges ranges, long epoch) + { + onEpochClosed(ranges, epoch, null); + } + + public void onEpochClosed(Ranges ranges, TxnId txnId) + { + onEpochClosed(ranges, txnId.epoch(), txnId); + } + + private void onEpochClosed(Ranges ranges, long epoch, @Nullable TxnId txnId) + { + Topology topology = null; + synchronized (this) + { + ActiveEpoch e = active.ifExists(epoch); + if (txnId != null) + { + if (e != null) + { + ranges = ranges.without(e.addedRanges); + if (ranges.isEmpty()) + return; + } + e = active.ifExists(--epoch); + } + + if (e != null) + topology = e.all(); + + if (epoch > active.currentEpoch) + ranges = pending.closed(ranges, epoch); + ranges = active.closed(ranges, epoch); + } + if (!ranges.isEmpty()) + { + for (TopologyListener listener : listeners) + listener.onEpochClosed(ranges, epoch, topology); + } + } + + public void onEpochRetired(Ranges ranges, long epoch) + { + onEpochRetired(ranges, epoch, null); + } + + public void onEpochRetired(Ranges ranges, TxnId txnId) + { + onEpochRetired(ranges, txnId.epoch(), txnId); + } + + private void onEpochRetired(Ranges ranges, long epoch, @Nullable TxnId txnId) + { + Topology topology = null; + synchronized (this) + { + ActiveEpoch e = active.ifExists(epoch); + if (txnId != null) + { + if (e != null) + { + ranges = ranges.without(e.addedRanges); + if (ranges.isEmpty()) + return; + } + + // if we're retiring only ranges that are no longer live, we can retire the declaration epoch; otherwise we only retire the prior epoch + if (e == null || e.live.ranges.intersects(ranges)) + e = active.ifExists(--epoch); + } + + if (e != null) + topology = e.all; + + if (epoch > active.currentEpoch) + ranges = pending.retired(ranges, epoch); + ranges = active.retired(ranges, epoch); + } + if (!ranges.isEmpty()) + { + for (TopologyListener listener : listeners) + listener.onEpochRetired(ranges, epoch, topology); + } + } + + public synchronized void truncateTopologiesUntil(long epoch) + { + ActiveEpochs current = active; + Invariants.requireArgument(current.epoch() >= epoch, "Unable to truncate; epoch %d is > current epoch %d", epoch, current.epoch()); + + if (current.minEpoch() >= epoch) + return; + + int newLen = current.epochs.length - (int) (epoch - current.minEpoch()); + Invariants.require(current.epochs[newLen - 1].isQuorumReady(), "Epoch %d is not ready to coordinate", current.epochs[newLen - 1].epoch()); + + ActiveEpoch[] nextEpochs = new ActiveEpoch[newLen]; + System.arraycopy(current.epochs, 0, nextEpochs, 0, newLen); + active = new ActiveEpochs(this, nextEpochs, current.firstNonEmptyEpoch); + } + + public TopologySorter.Supplier sorter() + { + return sorter; + } + + public Topology current() + { + return active.current(); + } + + public Topology currentLocal() + { + return active.currentLocal(); + } + + public boolean isEmpty() + { + return active.isEmpty() && pending.isEmpty(); + } + + public long epoch() + { + return current().epoch; + } + + @VisibleForImplementation + public synchronized long pendingEpoch() + { + return pending.maxEpoch(); + } + + // TODO (desired): add tests for epoch GC and tracking + @VisibleForImplementation + public long firstNonEmpty() + { + return active.firstNonEmptyEpoch; + } + + public long minEpoch() + { + ActiveEpochs epochs = this.active; + return epochs.minEpoch(); + } + + // TODO (testing): test all of these methods when asking for epochs that have been cleaned up (and other code paths) + + public ActiveEpochs active() + { + return active; + } + + public void addListener(TopologyListener listener) + { + listeners.add(listener); + } + + public void removeListener(TopologyListener listener) + { + listeners.remove(listener); + } + + protected Executor executor() + { + return Runnable::run; + } + + public void reportTopology(Topology topology) + { + PendingEpoch e; + synchronized (this) + { + long epoch = topology.epoch; + // if active is empty, treat the earliest pending epoch as our low bound to avoid race conditions where we begin updating active but discover an earlier epoch + long currentEpoch = !active.isEmpty() ? active.currentEpoch : !pending.isEmpty() ? pending.atIndex(0).epoch - 1 : 0; + if (epoch <= currentEpoch) + { + logger.debug("Ignoring topology for epoch {} which is behind our latest epoch {}", epoch, currentEpoch); + return; + } + + e = pending.getOrCreate(epoch); + e.setTopology(topology); + } + + logger.debug("Epoch {} received", topology.epoch()); + for (TopologyListener listener : listeners) + listener.onReceived(topology); + + updateActive(); + } + + public static class RegainingEpochRange + { + public final long epoch; + public final Ranges ranges; + + public RegainingEpochRange(long epoch, Ranges ranges) + { + this.epoch = epoch; + this.ranges = ranges; + } + + public long epoch() + { + return epoch; + } + + public Ranges ranges() + { + return ranges; + } + } + + @Nullable + public RegainingEpochRange computeRegaining(Topology current, Topology next) + { + Map additions = Topology.computeNodeAdditions(current, next); + long greatestEpoch = -1; + Ranges ranges = Ranges.EMPTY; + + ActiveEpochs active = this.active; + for (Map.Entry entry : additions.entrySet()) + { + Ranges addingForNode = entry.getValue(); + for (ActiveEpoch e : active) + { + addingForNode = addingForNode.without(e.removedRanges).without(e.retired()); + if (addingForNode.isEmpty()) + break; + + Ranges existingForNode = e.all().rangesForNode(entry.getKey()); + Ranges regainingForNode = addingForNode.slice(existingForNode, Minimal); + if (!regainingForNode.isEmpty()) + { + greatestEpoch = Math.max(greatestEpoch, e.epoch()); + ranges = ranges.union(MERGE_ADJACENT, regainingForNode); + addingForNode = addingForNode.without(regainingForNode); + } + addingForNode = addingForNode.without(e.addedRanges); + } + } + + if (greatestEpoch != -1) + return new RegainingEpochRange(greatestEpoch, ranges); + + return null; + } + + private final AtomicBoolean updatingActive = new AtomicBoolean(); + private void updateActive() + { + if (!updatingActive.compareAndSet(false, true)) + return; + + try + { + while (true) + { + Topology topology; + PendingEpoch pending; + synchronized (this) + { + if (this.pending.isEmpty() || (!this.active.isEmpty() && this.pending.atIndex(0).epoch > 1 + current().epoch())) + return; + + pending = this.pending.atIndex(0); + topology = pending.topology(); + if (topology == null) + return; + } + + Supplier bootstrap = node.commandStores().updateTopology(node, topology); + AsyncResult.Settable whenSetup = new AsyncResults.SettableWithDescription<>("Publishing Active Epoch"); + EpochReady epochReady = new EpochReady(topology.epoch, + NestedAsyncResult.flatMap(whenSetup, ignore -> AsyncResults.success(null)), + NestedAsyncResult.flatMap(whenSetup, EpochReady::coordinate), + NestedAsyncResult.flatMap(whenSetup, EpochReady::data), + NestedAsyncResult.flatMap(whenSetup, EpochReady::reads)); + + if (!this.active.isEmpty()) + { + ActiveEpoch prev = this.active.epochs[0]; + Invariants.require(prev.epoch() == topology.epoch - 1); + epochReady = orderReporting(prev.epochReady(), epochReady); + } + + ActiveEpoch active = new ActiveEpoch(node.id(), topology, epochReady, sorter.get(topology), this.active.current().ranges); + + synchronized (this) + { + active.recordClosed(pending.closed); + active.recordRetired(pending.retired); + pending.ready.forEach(active::onReadyToCoordinate); + + ActiveEpochs prev = this.active; + ActiveEpoch[] next = new ActiveEpoch[prev.epochs.length + 1]; + System.arraycopy(prev.epochs, 0, next, 1, prev.epochs.length); + next[0] = active; + + if (!prev.isEmpty() && !prev.epochs[0].all.hardRemoved.containsAll(topology.hardRemoved)) + { + IdentityHashMap cache = new IdentityHashMap<>(); + for (int i = next.length - 1 ; i >= 0 ; --i) + { + ActiveEpoch e = next[i]; + Topology newGlobal = next[i].all.withHardRemoved(topology.hardRemoved, cache); + if (newGlobal != e.all) + { + next[i] = new ActiveEpoch(node.id(), newGlobal, e.shardQuorumReady, e.receivedNodeReady, e.quorumReadyTracker, + e.addedRanges, e.removedRanges, e.epochReady(), e.quorumReady(), e.closed(), e.retired()); + } + } + } + + this.active = new ActiveEpochs(this, next, prev.firstNonEmptyEpoch); + this.pending.removeFirst(topology.epoch); + } + + EpochReady innerReady = bootstrap(bootstrap); + whenSetup.setSuccess(innerReady); + + pending.setActive(); + listeners.forEach(listener -> listener.onActive(active)); + + long epoch = topology.epoch; + Node.Id self = node.id(); + innerReady.coordinate.invokeIfSuccess(() -> { + listeners.forEach(listener -> listener.onReadyToCoordinate(topology)); + onReadyToCoordinate(self, epoch); + }); + } + } + finally + { + updatingActive.set(false); + } + } + + @VisibleForTesting + protected EpochReady bootstrap(Supplier bootstrap) + { + return bootstrap.get(); + } + + private static EpochReady orderReporting(EpochReady previous, EpochReady next) + { + if (previous.epoch + 1 != next.epoch) + throw new IllegalArgumentException("Attempted to order epochs but they are not next to each other... previous=" + previous.epoch + ", next=" + next.epoch); + if (previous.coordinate.isDone() && previous.data.isDone() && previous.reads.isDone()) + return next; + + return new EpochReady(next.epoch, + next.active, + NestedAsyncResult.flatMap(previous.coordinate, ignore -> next.coordinate), + NestedAsyncResult.flatMap(previous.data, ignore -> next.data), + NestedAsyncResult.flatMap(previous.reads, ignore -> next.reads) + ); + } + + public AsyncChain await(long epoch, @Nullable AsyncExecutor ifAsync) + { + PendingEpoch pendingEpoch; + boolean fetch; + synchronized (this) + { + if (epoch <= active.currentEpoch) + return AsyncChains.success(null); + + pendingEpoch = pending.getOrCreate(epoch); + fetch = pendingEpoch.fetching == null; + } + + node.agent().systemEvents().onWaitingForEpoch(epoch); + AsyncChain result = pendingEpoch.whenActive().chainImmediatelyElse(ifAsync); + if (fetch) + { + while (true) + { + fetch(pendingEpoch); + --epoch; + synchronized (this) + { + if (epoch <= active.currentEpoch) + break; + + pendingEpoch = pending.getOrCreate(epoch); + if (pendingEpoch.fetching != null) + break; + } + } + } + return result; + } + + private void fetch(PendingEpoch pending) + { + synchronized (this) + { + if (pending.topology() != null || pending.epoch < active.currentEpoch) + return; + + if (pending.fetching != null && !pending.fetching.isDone()) + return; + + pending.fetching = topologyService.fetchTopologyForEpoch(pending.epoch); + } + + pending.fetching.invoke((success, fail) -> { + if (fail == null) reportTopology(success); + else if (active.currentEpoch < pending.epoch && pending.topology() == null) + { + // TODO (expected): special casing of TopologyRetiredException? + logger.warn("Failed to fetch epoch {}. Retrying.", pending.epoch, fail); + node.agent().onException(fail, "Fetch epoch " + pending.epoch); + long retryInMicros = node.agent().retryTopologyDelay(node, 1 + ++pending.fetchAttempts, TimeUnit.MICROSECONDS); + node.scheduler().once(() -> fetch(pending), retryInMicros, TimeUnit.MICROSECONDS); + } + }); + } + + @VisibleForImplementation + public AsyncResult epochReady(long epoch, Function> get) + { + // synchronized for state.ready visibility + synchronized (this) + { + if (active.hasAtLeastEpoch(epoch)) + { + if (!active.hasEpoch(epoch)) + return get.apply(EpochReady.done(epoch)); + return get.apply(active.getKnown(epoch).epochReady()); + } + + return pending.getOrCreate(epoch).whenActive().get().flatMap(r -> get.apply(active.epochReady(epoch))); + } + } + + @VisibleForTesting + ActiveEpoch unsafeGetActiveEpoch(long epoch) + { + return active.getKnown(epoch); + } + + @VisibleForTesting + public void unsafeSetActive(ActiveEpochs newActive) + { + active = newActive; + } + + @VisibleForTesting + public Ranges unsafeQuorumReady(long epoch) + { + ActiveEpoch e = active.ifExists(epoch); + return e == null ? Ranges.EMPTY : e.quorumReady(); + } + + public boolean unsafeIsQuorumReady(long epoch) + { + ActiveEpoch e = active.ifExists(epoch); + return e != null && e.isQuorumReady(); + } + + public TopologyService topologyService() + { + return topologyService; + } +} diff --git a/accord-core/src/test/java/accord/impl/basic/DelayedCommandStores.java b/accord-core/src/test/java/accord/impl/basic/DelayedCommandStores.java new file mode 100644 index 000000000000..a9b47c36ed66 --- /dev/null +++ b/accord-core/src/test/java/accord/impl/basic/DelayedCommandStores.java @@ -0,0 +1,502 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package accord.impl.basic; + +import java.util.*; +import java.util.concurrent.Callable; +import java.util.function.BiConsumer; +import java.util.function.Consumer; +import java.util.function.Function; + +import com.google.common.collect.ImmutableSortedMap; + +import accord.api.Agent; +import accord.api.DataStore; +import accord.api.Journal; +import accord.api.LocalListeners; +import accord.api.ProgressLog; +import accord.api.RoutingKey; +import accord.impl.InMemoryCommandStore; +import accord.impl.InMemoryCommandStores; +import accord.impl.InMemorySafeCommand; +import accord.impl.InMemorySafeCommandsForKey; +import accord.impl.PrefixedIntHashKey; +import accord.impl.basic.TaskExecutorService.Task; +import accord.local.Command; +import accord.local.CommandStore; +import accord.local.CommandStores; +import accord.local.NodeCommandStoreService; +import accord.local.PreLoadContext; +import accord.local.RedundantBefore; +import accord.local.SafeCommandStore; +import accord.local.ShardDistributor; +import accord.local.cfk.CommandsForKey; +import accord.primitives.Range; +import accord.primitives.Ranges; +import accord.primitives.RoutableKey; +import accord.primitives.Timestamp; +import accord.primitives.TxnId; +import accord.topology.Topology; +import accord.utils.Invariants; +import accord.utils.RandomSource; +import accord.utils.async.AsyncChain; +import accord.utils.async.AsyncChains; +import accord.utils.async.Cancellable; + +import static accord.api.Journal.CommandUpdate; +import static accord.utils.Invariants.Paranoia.LINEAR; +import static accord.utils.Invariants.ParanoiaCostFactor.HIGH; + +public class DelayedCommandStores extends InMemoryCommandStores.SingleThread +{ + private DelayedCommandStores(NodeCommandStoreService time, Agent agent, DataStore store, RandomSource random, ShardDistributor shardDistributor, ProgressLog.Factory progressLogFactory, LocalListeners.Factory listenersFactory, SimulatedDelayedExecutorService executorService, CacheLoading isLoadedCheck, Journal journal) + { + super(time, agent, store, random, journal, shardDistributor, progressLogFactory, listenersFactory, DelayedCommandStore.factory(executorService, isLoadedCheck)); + } + + public static CommandStores.Factory factory(PendingQueue pending, CacheLoading isLoadedCheck) + { + return (time, agent, store, random, journal, shardDistributor, progressLogFactory, listenersFactory) -> + new DelayedCommandStores(time, agent, store, random, shardDistributor, progressLogFactory, listenersFactory, new SimulatedDelayedExecutorService(pending, agent, time.id()), isLoadedCheck, journal); + } + + public void validateShardStateForTesting(Journal.TopologyUpdate lastUpdate) + { + PreviouslyOwned previouslyOwned = lastUpdate.previouslyOwned; + ShardHolder[] shards = new ShardHolder[lastUpdate.commandStores.size()]; + int i = 0; + for (Map.Entry e : lastUpdate.commandStores.entrySet()) + { + Snapshot current = current(); + RangesForEpoch ranges = e.getValue(); + CommandStore commandStore = null; + for (ShardHolder shard : current) + { + if (shard.ranges().equals(ranges)) + { + Invariants.require(commandStore == null); + commandStore = shard.store; + } + } + Invariants.nonNull(commandStore, "Each set of ranges should have a corresponding command store, but %d did not:(%s)", + ranges, Arrays.toString(shards)) + .restore(); + + ShardHolder shard = new ShardHolder(commandStore, ranges, previouslyOwned.regains(ranges.all())); + shards[i++] = shard; + } + Arrays.sort(shards, Comparator.comparingInt(shard -> shard.store.id())); + + loadSnapshot(new Snapshot(shards, lastUpdate.global.forNode(nodeId()).trim(), lastUpdate.global, lastUpdate.previouslyOwned)); + } + + protected void loadSnapshot(Snapshot nextSnapshot) + { + Snapshot current = current(); + // These checks are only applicable to delayed command stores. + for (ShardHolder shard : current) + { + CommandStore prev = current.byId(shard.store.id()); + CommandStore next = nextSnapshot.byId(shard.store.id()); + { + RedundantBefore orig = prev.unsafeGetRedundantBefore(); + RedundantBefore loaded = next.unsafeGetRedundantBefore(); + Invariants.require(orig.equals(loaded), "%s should equal %s", loaded, orig); + } + + { + NavigableMap orig = prev.unsafeGetBootstrapBeganAt(); + NavigableMap loaded = next.unsafeGetBootstrapBeganAt(); + Invariants.require(orig.equals(loaded), "%s should equal %s", loaded, orig); + } + + { + NavigableMap orig = prev.unsafeGetSafeToRead(); + NavigableMap loaded = next.unsafeGetSafeToRead(); + Invariants.require(orig.equals(loaded), "%s should equal %s", loaded, orig); + } + + { + RangesForEpoch orig = prev.unsafeGetRangesForEpoch(); + RangesForEpoch loaded = next.unsafeGetRangesForEpoch(); + Invariants.require(orig.equals(loaded), "%s should equal %s", loaded, orig); + } + } + + super.loadSnapshot(nextSnapshot); + } + + private static boolean contains(Topology previous, int searchPrefix) + { + for (Range range : previous.ranges()) + { + int prefix = ((PrefixedIntHashKey) range.start()).prefix; + if (prefix == searchPrefix) + return true; + } + return false; + } + + public static class DelayedCommandStore extends InMemoryCommandStore + { + public class DelayedTask extends Task + { + private DelayedTask(Callable call) + { + super(call); + } + + private DelayedTask(Callable call, Pending origin) + { + super(call, origin); + } + + public DelayedCommandStore owner() + { + return DelayedCommandStore.this; + } + + @Override + public void run() + { + Invariants.require(active == null); + Invariants.require(activeThread == null); + active = this; + activeThread = Thread.currentThread(); + try + { + super.run(); + } + finally + { + Invariants.require(active == this); + Invariants.require(activeThread == Thread.currentThread()); + active = null; + activeThread = null; + } + } + + public Callable callable() + { + return callable; + } + + public String toString() + { + return callable.toString(); + } + } + + private final SimulatedDelayedExecutorService executor; + private final Queue> pending = new ArrayDeque<>(); + private final CacheLoading cacheLoading; + private final Journal journal; + private Task active; + private Thread activeThread; + + public DelayedCommandStore(int id, NodeCommandStoreService time, Agent agent, DataStore store, ProgressLog.Factory progressLogFactory, LocalListeners.Factory listenersFactory, EpochUpdateHolder epochUpdateHolder, SimulatedDelayedExecutorService executor, CacheLoading cacheLoading, Journal journal) + { + super(id, time, agent, store, progressLogFactory, listenersFactory, epochUpdateHolder, journal); + this.executor = executor; + this.cacheLoading = cacheLoading; + this.journal = journal; + restore(); + } + + protected void loadRedundantBefore(RedundantBefore redundantBefore) + { + if (redundantBefore == null) + { + Invariants.require(unsafeGetRedundantBefore().size() == 0); + } + else + { + unsafeClearRedundantBefore(); + super.loadRedundantBefore(redundantBefore); + } + } + + protected void loadBootstrapBeganAt(NavigableMap bootstrapBeganAt) + { + unsafeClearBootstrapBeganAt(); + if (bootstrapBeganAt == null) bootstrapBeganAt = ImmutableSortedMap.of(TxnId.NONE, Ranges.EMPTY); + super.loadBootstrapBeganAt(bootstrapBeganAt); + } + + protected void loadSafeToRead(NavigableMap safeToRead) + { + unsafeClearSafeToRead(); + if (safeToRead == null) safeToRead = ImmutableSortedMap.of(Timestamp.NONE, Ranges.EMPTY); + super.loadSafeToRead(safeToRead); + } + + @Override + protected void loadRangesForEpoch(RangesForEpoch newRangesForEpoch) + { + if (newRangesForEpoch == null) Invariants.require(super.rangesForEpoch == null); + else + { + unsafeClearRangesForEpoch(); + super.loadRangesForEpoch(newRangesForEpoch); + } + } + + public void restore() + { + loadRangesForEpoch(journal.loadRangesForEpoch(id())); + loadRedundantBefore(journal.loadRedundantBefore(id())); + loadBootstrapBeganAt(journal.loadBootstrapBeganAt(id())); + loadSafeToRead(journal.loadSafeToRead(id())); + } + + @Override + public void onRead(Command current) + { + cacheLoading.validateRead(this, current); + } + + @Override + public void onWrite(Command current) + { + cacheLoading.validateWrite(this, current); + } + + @Override + public void onRead(CommandsForKey current) + { + cacheLoading.validateRead(this, current); + } + + @Override + protected boolean canExposeUnloaded() + { + return !cacheLoading.cacheEmpty(); + } + + private static CommandStore.Factory factory(SimulatedDelayedExecutorService executor, CacheLoading isLoadedCheck) + { + return (id, node, agent, store, progressLogFactory, listenersFactory, rangesForEpoch, journal) -> new DelayedCommandStore(id, node, agent, store, progressLogFactory, listenersFactory, rangesForEpoch, executor, isLoadedCheck, journal); + } + + @Override + public boolean inStore() + { + return Thread.currentThread() == activeThread; + } + + @Override + public AsyncChain chain(PreLoadContext context, Consumer consumer) + { + return submit(newTask(context, i -> { consumer.accept(i); return null; })); + } + + @Override + public AsyncChain chain(PreLoadContext context, Function function) + { + return submit(newTask(context, function)); + } + + @Override + public AsyncChain chain(Callable call) + { + return submit(new DelayedTask<>(call)); + } + + @Override + public void execute(Runnable run) + { + execute(new DelayedTask<>(() -> { + run.run(); + return null; + })); + } + + private void execute(DelayedTask task) + { + boolean wasEmpty = pending.isEmpty(); + executor.preregister(task); + pending.add(task); + if (wasEmpty) + runNextTask(); + } + + private DelayedTask newTask(PreLoadContext context, Function function) + { + Pending origin = Pending.Global.activeOrigin(); + if (RecurringPendingRunnable.isRecurring(origin) && context.primaryTxnId() != null && !context.primaryTxnId().isSystemTxn()) + origin = null; + return new DelayedTask<>(() -> executeInContext(this, context, function), origin); + } + + private AsyncChain submit(DelayedTask task) + { + if (Invariants.testParanoia(LINEAR, LINEAR, HIGH)) + { + return AsyncChains.detectLeak(agent::onException, () -> { + boolean wasEmpty = pending.isEmpty(); + pending.add(task); + if (wasEmpty) + runNextTask(); + }).flatMap(ignore -> task.chain()); + } + else + { + return new AsyncChains.Head<>() + { + @Override + protected Cancellable start(BiConsumer callback) + { + execute(task); + task.invoke(callback); + return () -> { + if (pending.peek() != task) + { + pending.remove(task); + executor.cancel(task); + task.cancel(false); + } + }; + } + }; + } + } + + private void runNextTask() + { + Task next = pending.peek(); + if (next == null) + return; + + next.invoke(this.agent()); // used to track unexpected exceptions and notify simulations + next.invoke(this::afterExecution); + executor.executePreregistered(next); + } + + private void afterExecution() + { + pending.poll(); + runNextTask(); + } + + @Override + public void shutdown() + { + + } + + @Override + protected InMemorySafeStore createSafeStore(PreLoadContext context, RangesForEpoch ranges, Map commands, Map commandsForKeys) + { + return new DelayedSafeStore(this, ranges, context, commands, commandsForKeys, cacheLoading); + } + } + + static int counter = 0; + public static class DelayedSafeStore extends InMemoryCommandStore.InMemorySafeStore + { + private final DelayedCommandStore commandStore; + private final CacheLoading cacheLoading; + + public DelayedSafeStore(DelayedCommandStore commandStore, + RangesForEpoch ranges, + PreLoadContext context, + Map commands, + Map commandsForKey, + CacheLoading cacheLoading) + { + super(commandStore, ranges, context, commands, commandsForKey); + this.commandStore = commandStore; + this.cacheLoading = cacheLoading; + ++counter; + } + + @Override + public void postExecute() + { + persistFieldUpdates(); + commands.entrySet().forEach(e -> { + InMemorySafeCommand safe = e.getValue(); + if (!safe.isModified()) return; + + Command before = safe.original(); + Command after = safe.current(); + commandStore.journal.saveCommand(commandStore.id(), new CommandUpdate(before, after), () -> {}); + commandStore.onWrite(safe.current()); + }); + super.postExecute(); + } + + @Override + protected void persistFieldUpdates() + { + Journal.FieldUpdates fieldUpdates = fieldUpdates(); + if (fieldUpdates != null) + commandStore.journal.saveStoreState(commandStore.id(), fieldUpdates, () -> {}); + super.persistFieldUpdates(); + } + + @Override + protected InMemoryCommandStore.InMemoryCommandStoreCaches tryGetCaches() + { + if (!cacheLoading.cacheEmpty()) + { + boolean cacheFull = cacheLoading.cacheFull(); + return commandStore.new InMemoryCommandStoreCaches() { + @Override + public InMemorySafeCommand acquireIfLoaded(TxnId txnId) + { + if (cacheFull || cacheLoading.isLoaded(txnId)) + return super.acquireIfLoaded(txnId); + return null; + } + + @Override + public InMemorySafeCommandsForKey acquireIfLoaded(RoutingKey key) + { + if (cacheFull || cacheLoading.isLoaded(key)) + return super.acquireIfLoaded(key); + return null; + } + }; + } + return null; + + } + } + + public List unsafeStores() + { + List stores = new ArrayList<>(); + for (ShardHolder holder : current()) + stores.add((DelayedCommandStore) holder.store); + return stores; + } + + public interface CacheLoading + { + boolean cacheEmpty(); + boolean cacheFull(); + boolean isLoaded(TxnId txnId); + boolean isLoaded(RoutingKey key); + boolean tfkLoaded(); + void validateRead(CommandStore commandStore, Command command); + void validateWrite(CommandStore commandStore, Command command); + void validateRead(CommandStore commandStore, CommandsForKey cfk); + } +} From 12ed273124120e5fbef12933c3da21a2bb3c6b0a Mon Sep 17 00:00:00 2001 From: Alan Wang Date: Fri, 17 Apr 2026 16:11:32 -0700 Subject: [PATCH 8/8] Revert "feedback/edits for regaining ranges" This reverts commit 3ab3887ff60777c964140eb920e7d18aab9b5e7e. --- .../src/main/java/accord/api/Journal.java | 185 --- .../accord/impl/AbstractSafeCommandStore.java | 308 ---- .../main/java/accord/local/CommandStore.java | 1321 ----------------- .../main/java/accord/local/CommandStores.java | 1219 --------------- .../src/main/java/accord/local/Node.java | 886 ----------- .../OverlappingCommandStoresException.java | 39 - .../java/accord/local/SafeCommandStore.java | 676 --------- .../java/accord/topology/TopologyManager.java | 590 -------- .../impl/basic/DelayedCommandStores.java | 502 ------- 9 files changed, 5726 deletions(-) delete mode 100644 accord-core/src/main/java/accord/api/Journal.java delete mode 100644 accord-core/src/main/java/accord/impl/AbstractSafeCommandStore.java delete mode 100644 accord-core/src/main/java/accord/local/CommandStore.java delete mode 100644 accord-core/src/main/java/accord/local/CommandStores.java delete mode 100644 accord-core/src/main/java/accord/local/Node.java delete mode 100644 accord-core/src/main/java/accord/local/OverlappingCommandStoresException.java delete mode 100644 accord-core/src/main/java/accord/local/SafeCommandStore.java delete mode 100644 accord-core/src/main/java/accord/topology/TopologyManager.java delete mode 100644 accord-core/src/test/java/accord/impl/basic/DelayedCommandStores.java diff --git a/accord-core/src/main/java/accord/api/Journal.java b/accord-core/src/main/java/accord/api/Journal.java deleted file mode 100644 index cf9986c61fa3..000000000000 --- a/accord-core/src/main/java/accord/api/Journal.java +++ /dev/null @@ -1,185 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package accord.api; - -import java.util.List; -import java.util.NavigableMap; -import java.util.Objects; -import java.util.function.Supplier; -import javax.annotation.Nonnull; -import javax.annotation.Nullable; - -import accord.impl.CommandChange; -import accord.local.Command; -import accord.local.CommandStores; -import accord.local.CommandStores.PreviouslyOwned; -import accord.local.DurableBefore; -import accord.local.Node; -import accord.local.RedundantBefore; -import accord.primitives.EpochSupplier; -import accord.primitives.Ranges; -import accord.primitives.Timestamp; -import accord.primitives.TxnId; -import accord.topology.Topology; -import accord.utils.Invariants; -import accord.utils.PersistentField.Persister; -import accord.utils.async.AsyncChain; -import org.agrona.collections.Int2ObjectHashMap; - -/** - * Persisted journal for transactional recovery. - */ -public interface Journal -{ - enum Load - { - ALL, - MINIMAL, - MINIMAL_WITH_DEPS - } - - void start(Node node); - - Command loadCommand(int store, TxnId txnId, RedundantBefore redundantBefore, DurableBefore durableBefore); - default List> debugCommand(int commandStoreId, TxnId txnId) { throw new UnsupportedOperationException(); } - Command.Minimal loadMinimal(int commandStoreId, TxnId txnId, RedundantBefore redundantBefore, DurableBefore durableBefore); - Command.MinimalWithDeps loadMinimalWithDeps(int store, TxnId txnId, RedundantBefore redundantBefore, DurableBefore durableBefore); - void saveCommand(int store, CommandUpdate value, Runnable onFlush); - - List replayTopologies(); - void saveTopology(TopologyUpdate topologyUpdate, Runnable onFlush); - - void purge(CommandStores commandStores, EpochSupplier minEpoch); - - /** - * Replays all messages from journal to rehydrate CommandStores state. Returns whether it has seen (and ignored) - * any exceptions during replay. - */ - boolean replay(CommandStores commandStores); - - RedundantBefore loadRedundantBefore(int store); - NavigableMap loadBootstrapBeganAt(int store); - NavigableMap loadSafeToRead(int store); - CommandStores.RangesForEpoch loadRangesForEpoch(int store); - - Persister durableBeforePersister(); - - void saveStoreState(int store, FieldUpdates fieldUpdates, Runnable onFlush); - - class TopologyUpdate - { - public final Int2ObjectHashMap commandStores; - public final Topology global; - public final PreviouslyOwned previouslyOwned; - - public TopologyUpdate(@Nonnull Int2ObjectHashMap commandStores, @Nonnull Topology global, PreviouslyOwned previouslyOwned) - { - this.commandStores = commandStores; - this.global = global; - this.previouslyOwned = previouslyOwned; - } - - public boolean isEquivalent(TopologyUpdate other) - { - boolean equivalent = global.isEquivalent(other.global); - if (!equivalent) - return false; - Invariants.require(commandStores.equals(other.commandStores)); - return true; - } - - public TopologyUpdate cloneWithEquivalentEpoch(long epoch) - { - return new TopologyUpdate(commandStores, global.cloneEquivalentWithEpoch(epoch), previouslyOwned); - } - - @Override - public boolean equals(Object object) - { - if (this == object) return true; - if (object == null || getClass() != object.getClass()) return false; - TopologyUpdate update = (TopologyUpdate) object; - return Objects.equals(commandStores, update.commandStores) && Objects.equals(global, update.global); - } - - @Override - public int hashCode() - { - return Objects.hash(commandStores, global); - } - - @Override - public String toString() - { - return "TopologyUpdate{" + - "commandStores=" + commandStores + - ", global=" + global + - '}'; - } - } - - class CommandUpdate - { - public final TxnId txnId; - public final Command before; - public final Command after; - - public CommandUpdate(@Nullable Command before, @Nonnull Command after) - { - this.txnId = after.txnId(); - this.before = before; - this.after = after; - } - } - - class FieldUpdates - { - public RedundantBefore newRedundantBefore; - public NavigableMap newBootstrapBeganAt; - public NavigableMap newSafeToRead; - public Ranges newPermanentlyUnsafeToRead; - public CommandStores.RangesForEpoch newRangesForEpoch; - - public String toString() - { - StringBuilder builder = new StringBuilder("FieldUpdates{"); - if (newRedundantBefore != null) - builder.append("newRedundantBefore=").append(newRedundantBefore).append(", "); - if (newBootstrapBeganAt != null) - builder.append("newBootstrapBeganAt=").append(newBootstrapBeganAt).append(", "); - if (newSafeToRead != null) - builder.append("newSafeToRead=").append(newSafeToRead).append(", "); - if (newPermanentlyUnsafeToRead != null) - builder.append("newPermanentlyUnsafeToRead=").append(newPermanentlyUnsafeToRead).append(", "); - if (newRangesForEpoch != null) - builder.append("newRangesForEpoch=").append(newRangesForEpoch).append(", "); - builder.setLength(builder.length() - 2); - builder.append('}'); - return builder.toString(); - } - } - - /** - * Helper for CommandStore to restore Command states. - */ - interface Replayer - { - AsyncChain replay(TxnId txnId); - } -} diff --git a/accord-core/src/main/java/accord/impl/AbstractSafeCommandStore.java b/accord-core/src/main/java/accord/impl/AbstractSafeCommandStore.java deleted file mode 100644 index b684575126d3..000000000000 --- a/accord-core/src/main/java/accord/impl/AbstractSafeCommandStore.java +++ /dev/null @@ -1,308 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package accord.impl; - -import java.util.ArrayList; -import java.util.List; -import java.util.NavigableMap; - -import accord.api.RoutingKey; -import accord.local.CommandStore; -import accord.local.LoadKeys; -import accord.local.PreLoadContext; -import accord.local.RedundantBefore; -import accord.local.SafeCommand; -import accord.local.SafeCommandStore; -import accord.local.cfk.SafeCommandsForKey; -import accord.primitives.Ranges; -import accord.primitives.Routable; -import accord.primitives.RoutingKeys; -import accord.primitives.Timestamp; -import accord.primitives.TxnId; -import accord.primitives.Unseekables; - -import static accord.api.Journal.FieldUpdates; -import static accord.local.CommandStores.RangesForEpoch; - -public abstract class AbstractSafeCommandStore> -extends SafeCommandStore -{ - protected final PreLoadContext context; - - private final CommandStore commandStore; - private FieldUpdates fieldUpdates; - - protected AbstractSafeCommandStore(PreLoadContext context, CommandStore commandStore) - { - this.context = context; - this.commandStore = commandStore; - } - - @Override - public CommandStore commandStore() - { - return commandStore; - } - - public interface CommandStoreCaches extends AutoCloseable - { - void close(); - - C acquireIfLoaded(TxnId txnId); - CFK acquireIfLoaded(RoutingKey key); - } - - protected abstract Caches tryGetCaches(); - protected abstract C add(C safeCommand, Caches caches); - protected abstract CFK add(CFK safeCfk, Caches caches); - - @Override - public PreLoadContext canExecute(PreLoadContext with) - { - if (with.isEmpty()) return with; - if (with.keys().domain() == Routable.Domain.Range) - return with.isSubsetOf(this.context) ? with : null; - - LoadKeys require = with.loadKeys(); - if (require != LoadKeys.NONE) - { - PreLoadContext context = context(); - if (!context.loadKeys().satisfiesIfPresent(require)) - return null; - - if (with.loadKeysFor().compareTo(context.loadKeysFor()) > 0) - return null; - } - - try (Caches caches = tryGetCaches()) - { - for (TxnId txnId : with.txnIds()) - { - if (null != getInternal(txnId)) - continue; - - if (caches == null) - return null; - - C safeCommand = caches.acquireIfLoaded(txnId); - if (safeCommand == null) - return null; - - add(safeCommand, caches); - } - - LoadKeys loadKeys = with.loadKeys(); - if (loadKeys == LoadKeys.NONE) - return with; - - List unavailable = null; - Unseekables keys = with.keys(); - if (keys.isEmpty()) - return with; - - for (int i = 0 ; i < keys.size() ; ++i) - { - RoutingKey key = (RoutingKey) keys.get(i); - if (null != getInternal(key)) - continue; // already in working set - - if (caches != null) - { - CFK safeCfk = caches.acquireIfLoaded(key); - if (safeCfk != null) - { - add(safeCfk, caches); - continue; - } - } - if (unavailable == null) - unavailable = new ArrayList<>(); - unavailable.add(key); - } - - if (unavailable == null) - return with; - - if (unavailable.size() == keys.size()) - return null; - - return PreLoadContext.contextFor(with.primaryTxnId(), with.additionalTxnId(), keys.without(RoutingKeys.ofSortedUnique(unavailable)), loadKeys, context.loadKeysFor(), context.reason()); - } - } - - @Override - public PreLoadContext context() - { - return context; - } - - @Override - protected C ifLoadedInternal(TxnId txnId) - { - try (Caches caches = tryGetCaches()) - { - if (caches == null) - return null; - - C command = caches.acquireIfLoaded(txnId); - if (command == null) - return null; - - return add(command, caches); - } - } - - @Override - protected CFK ifLoadedInternal(RoutingKey txnId) - { - try (Caches caches = tryGetCaches()) - { - if (caches == null) - return null; - - CFK cfk = caches.acquireIfLoaded(txnId); - if (cfk == null) - return null; - - return add(cfk, caches); - } - } - - // TODO (expected): cleanup the integration hooks here; they're a bit byzantine. Also clearly document behaviour. - public void postExecute() - { - flushFieldUpdates(); - } - - protected void persistFieldUpdates() - { - flushFieldUpdates(); - } - - protected void flushFieldUpdates() - { - if (fieldUpdates == null) - return; - - if (fieldUpdates.newRedundantBefore != null) - super.unsafeSetRedundantBefore(fieldUpdates.newRedundantBefore); - - if (fieldUpdates.newBootstrapBeganAt != null) - super.setBootstrapBeganAt(fieldUpdates.newBootstrapBeganAt); - - if (fieldUpdates.newSafeToRead != null) - super.setSafeToRead(fieldUpdates.newSafeToRead); - - if (fieldUpdates.newRangesForEpoch != null) - super.setRangesForEpoch(fieldUpdates.newRangesForEpoch); - - fieldUpdates = null; - } - - /** - * Persistent field update logic - */ - - @Override - public final void upsertRedundantBefore(RedundantBefore addRedundantBefore) - { - // TODO (required): this is potentially unsafe: if the update is not persisted for some reason (due to some later exception) - // we can continue with a stale redundantBefore - // TODO (expected): fix RedundantBefore sorting issue and switch to upsert mode - ensureFieldUpdates().newRedundantBefore = RedundantBefore.merge(redundantBefore(), addRedundantBefore); - unsafeUpsertRedundantBefore(addRedundantBefore); - } - - @Override - public final void setBootstrapBeganAt(NavigableMap newBootstrapBeganAt) - { - ensureFieldUpdates().newBootstrapBeganAt = newBootstrapBeganAt; - } - - @Override - public final void setSafeToRead(NavigableMap newSafeToRead) - { - ensureFieldUpdates().newSafeToRead = newSafeToRead; - } - - @Override - public final void setPermanentlyUnsafeToRead(Ranges newPermanentlyUnsafeToRead) - { - ensureFieldUpdates().newPermanentlyUnsafeToRead = newPermanentlyUnsafeToRead; - } - - @Override - public void setRangesForEpoch(RangesForEpoch rangesForEpoch) - { - if (rangesForEpoch != null) - { - super.setRangesForEpoch(rangesForEpoch); - ensureFieldUpdates().newRangesForEpoch = rangesForEpoch; - } - } - - @Override - public RangesForEpoch ranges() - { - if (fieldUpdates != null && fieldUpdates.newRangesForEpoch != null) - return fieldUpdates.newRangesForEpoch; - - return commandStore.unsafeGetRangesForEpoch(); - } - - @Override - public NavigableMap bootstrapBeganAt() - { - if (fieldUpdates != null && fieldUpdates.newBootstrapBeganAt != null) - return fieldUpdates.newBootstrapBeganAt; - - return super.bootstrapBeganAt(); - } - - @Override - public NavigableMap safeToReadAt() - { - if (fieldUpdates != null && fieldUpdates.newSafeToRead != null) - return fieldUpdates.newSafeToRead; - - return super.safeToReadAt(); - } - - @Override - public RedundantBefore redundantBefore() - { - if (fieldUpdates != null && fieldUpdates.newRedundantBefore != null) - return fieldUpdates.newRedundantBefore; - - return super.redundantBefore(); - } - - private FieldUpdates ensureFieldUpdates() - { - if (fieldUpdates == null) fieldUpdates = new FieldUpdates(); - return fieldUpdates; - } - - public FieldUpdates fieldUpdates() - { - return fieldUpdates; - } -} diff --git a/accord-core/src/main/java/accord/local/CommandStore.java b/accord-core/src/main/java/accord/local/CommandStore.java deleted file mode 100644 index 83966f192825..000000000000 --- a/accord-core/src/main/java/accord/local/CommandStore.java +++ /dev/null @@ -1,1321 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package accord.local; - -import java.util.AbstractMap.SimpleImmutableEntry; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.NavigableMap; -import java.util.Set; -import java.util.TreeMap; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicReference; -import java.util.function.BiConsumer; -import java.util.function.Consumer; -import java.util.function.Function; -import java.util.function.Supplier; -import javax.annotation.Nullable; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.ImmutableSortedMap; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import accord.api.Agent; -import accord.topology.EpochReady; -import accord.api.DataStore; -import accord.api.DataStore.FetchKind; -import accord.api.Journal; -import accord.api.LocalListeners; -import accord.api.ProgressLog; -import accord.impl.AbstractAsyncExecutor; -import accord.coordinate.CoordinateMaxConflict; -import accord.local.CommandStores.BootstrapRangeAction; -import accord.local.CommandStores.RangesForEpoch; -import accord.local.Commands.NotifyWaitingOnPlus; -import accord.local.PreLoadContext.Empty; -import accord.local.RedundantBefore.Bounds; -import accord.local.RedundantStatus.SomeStatus; -import accord.primitives.Ranges; -import accord.primitives.Routables; -import accord.primitives.SaveStatus; -import accord.primitives.Status; -import accord.primitives.Status.Durability.HasOutcome; -import accord.primitives.Timestamp; -import accord.primitives.TxnId; -import accord.primitives.Unseekables; -import accord.utils.DeterministicIdentitySet; -import accord.utils.Invariants; -import accord.utils.Reduce; -import accord.utils.UnhandledEnum; -import accord.utils.async.AsyncChain; -import accord.utils.async.AsyncChains; -import accord.utils.async.AsyncResult; -import accord.utils.async.AsyncResults; -import accord.utils.async.AsyncResults.SettableByCallback; -import accord.utils.async.AsyncResults.SettableWithDescription; -import accord.utils.async.Cancellable; -import accord.utils.async.AsyncResults.SettableResult; -import org.agrona.collections.LongHashSet; - -import static accord.topology.EpochReady.DONE; -import static accord.topology.EpochReady.done; -import static accord.api.DataStore.FetchKind.Image; -import static accord.api.ProtocolModifiers.Toggles.requiresUniqueHlcs; -import static accord.local.RedundantStatus.SomeStatus.GC_BEFORE_AND_LOCALLY_DURABLE; -import static accord.local.RedundantStatus.SomeStatus.LOCALLY_APPLIED_ONLY; -import static accord.local.RedundantStatus.SomeStatus.LOCALLY_DURABLE_TO_COMMAND_STORE_ONLY; -import static accord.local.RedundantStatus.SomeStatus.LOCALLY_DURABLE_TO_DATA_STORE_ONLY; -import static accord.local.RedundantStatus.SomeStatus.LOCALLY_WITNESSED_ONLY; -import static accord.local.RedundantStatus.SomeStatus.LOG_UNAVAILABLE_ONLY; -import static accord.local.RedundantStatus.SomeStatus.QUORUM_APPLIED_ONLY; -import static accord.local.RedundantStatus.SomeStatus.UNREADY_ONLY; -import static accord.local.RedundantStatus.SomeStatus.SHARD_APPLIED_ONLY; -import static accord.primitives.AbstractRanges.UnionMode.MERGE_ADJACENT; -import static accord.primitives.Routables.Slice.Minimal; -import static accord.primitives.Timestamp.Flag.HLC_BOUND; -import static accord.primitives.Txn.Kind.VisibilitySyncPoint; -import static accord.utils.Invariants.nonNull; - -/** - * Single threaded internal shard of accord transaction metadata - */ -public abstract class CommandStore implements AbstractAsyncExecutor, SequentialAsyncExecutor -{ - private static final Logger logger = LoggerFactory.getLogger(CommandStore.class); - - public static class EpochUpdate - { - public final RangesForEpoch newRangesForEpoch; - public final RedundantBefore addRedundantBefore; - - EpochUpdate(RangesForEpoch newRangesForEpoch, RedundantBefore addRedundantBefore) - { - this.newRangesForEpoch = newRangesForEpoch; - this.addRedundantBefore = addRedundantBefore; - } - } - - // TODO (required): we only REMOVE ranges now, so it should be possible to simplify this - public static class EpochUpdateHolder extends AtomicReference - { - // TODO (desired): can better encapsulate by accepting only the newRangesForEpoch and deriving the add/remove ranges - public void add(long epoch, RangesForEpoch newRangesForEpoch, Ranges addRanges) - { - RedundantBefore addRedundantBefore = RedundantBefore.create(addRanges, epoch, Long.MAX_VALUE, TxnId.minForEpoch(epoch), UNREADY_ONLY); - update(newRangesForEpoch, addRedundantBefore); - } - - public void remove(long epoch, RangesForEpoch newRangesForEpoch, Ranges removeRanges) - { - RedundantBefore addRedundantBefore = RedundantBefore.create(removeRanges, Long.MIN_VALUE, epoch, TxnId.NONE, SomeStatus.NONE); - update(newRangesForEpoch, addRedundantBefore); - } - - private void update(RangesForEpoch newRangesForEpoch, RedundantBefore addRedundantBefore) - { - EpochUpdate baseUpdate = new EpochUpdate(newRangesForEpoch, addRedundantBefore); - EpochUpdate cur = get(); - if (cur == null || !compareAndSet(cur, new EpochUpdate(newRangesForEpoch, RedundantBefore.merge(cur.addRedundantBefore, addRedundantBefore)))) - set(baseUpdate); - } - } - - public interface Factory - { - CommandStore create(int id, - NodeCommandStoreService node, - Agent agent, - DataStore store, - ProgressLog.Factory progressLogFactory, - LocalListeners.Factory listenersFactory, - EpochUpdateHolder rangesForEpoch, - Journal journal); - } - - protected final int id; - protected final NodeCommandStoreService node; - protected final Agent agent; - protected final DataStore dataStore; - protected final ProgressLog progressLog; - protected final LocalListeners listeners; - protected final EpochUpdateHolder epochUpdateHolder; - - // Used in markShardStale to make sure the staleness includes in progress bootstraps - // TODO (desired): migrate to BTree - private transient NavigableMap bootstrapBeganAt = emptyBootstrapBeganAt(); // additive (i.e. once inserted, rolled-over until invalidated, and the floor entry contains additions) - protected boolean hasResumedBootstraps; - private RedundantBefore redundantBefore = RedundantBefore.EMPTY; - private MaxConflicts maxConflicts = MaxConflicts.EMPTY; - private MaxDecidedRX maxDecidedRX = MaxDecidedRX.EMPTY; - private int maxConflictsUpdates = 0; - protected RangesForEpoch rangesForEpoch; - protected @Nullable Ranges refuses; - List syncPointListeners; - - /** - * safeToRead is related to RedundantBefore, but a distinct concept. - * While readyAt defines the txnId bounds we expect to maintain data for locally, - * safeToRead defines executeAt bounds we can safely participate in transaction execution for. - * safeToRead is defined by the no-op transaction we execute after a bootstrap is initiated, - * and creates a global bound before which we know we have complete data from our bootstrap. - * - * There's a smearing period during bootstrap where some keys may be ahead of others, for instance, - * since we do not create a precise instant in the transaction log for bootstrap to avoid impeding execution. - * - * We also update safeToRead when we go stale, to remove ranges we may have bootstrapped but that are now known to - * be incomplete. In this case we permit transactions to execute in any order for the unsafe key ranges. - * But they may still be ordered for other key ranges they participate in. - */ - private NavigableMap safeToRead = emptySafeToRead(); - private Ranges permanentlyUnsafeToRead = Ranges.EMPTY; - private final Set bootstraps = Collections.synchronizedSet(new DeterministicIdentitySet<>()); - @Nullable private RejectBefore rejectBefore; - - static class WaitingOnVisibility - { - final SettableResult whenDone; - final Ranges allRanges; - Ranges waitingOn, waitingOnDurable; - - WaitingOnVisibility(SettableResult whenDone, Ranges ranges) - { - this.whenDone = whenDone; - this.allRanges = this.waitingOn = this.waitingOnDurable = ranges; - } - } - private final TreeMap waitingOnVisibility = new TreeMap<>(); - - protected CommandStore(int id, - NodeCommandStoreService node, - Agent agent, - DataStore dataStore, - ProgressLog.Factory progressLogFactory, - LocalListeners.Factory listenersFactory, - EpochUpdateHolder epochUpdateHolder) - { - this.id = id; - this.node = node; - this.agent = agent; - this.dataStore = dataStore; - this.progressLog = progressLogFactory.create(this); - this.listeners = listenersFactory.create(this); - this.epochUpdateHolder = epochUpdateHolder; - } - - public final int id() - { - return id; - } - - public void restore() {}; - - public abstract Journal.Replayer replayer(); - // expected to invoke safeStore.upsertRedundantBefore at some future point, when the commandStore state is durably persisted - protected abstract void ensureDurable(Ranges ranges, RedundantBefore onCommandStoreDurable); - - public Agent agent() - { - return agent; - } - - public void unsafeClearForTesting() - { - progressLog.clear(); - bootstraps.clear(); - rangesForEpoch = null; - bootstrapBeganAt = emptyBootstrapBeganAt(); - redundantBefore = RedundantBefore.EMPTY; - maxConflicts = MaxConflicts.EMPTY; - maxDecidedRX = MaxDecidedRX.EMPTY; - safeToRead = emptySafeToRead(); - listeners.clear(); - waitingOnVisibility.clear(); - } - - public void updateRangesForEpoch(SafeCommandStore safeStore) - { - EpochUpdate update = epochUpdateHolder.get(); - if (update == null) - return; - - update = epochUpdateHolder.getAndSet(null); - if (update.addRedundantBefore.size() > 0) - safeStore.upsertRedundantBefore(update.addRedundantBefore); - if (update.newRangesForEpoch != null) - safeStore.setRangesForEpoch(update.newRangesForEpoch); - - safeStore.persistFieldUpdates(); - } - - @VisibleForTesting - public void unsafeUpdateRangesForEpoch() - { - EpochUpdate update = epochUpdateHolder.getAndSet(null); - if (update == null) - return; - - if (update.addRedundantBefore.size() > 0) - unsafeUpsertRedundantBefore(update.addRedundantBefore); - if (update.newRangesForEpoch != null) - unsafeSetRangesForEpoch(update.newRangesForEpoch); - } - - public RangesForEpoch unsafeGetRangesForEpoch() - { - return rangesForEpoch; - } - - public MaxDecidedRX unsafeGetMaxDecidedRX() - { - return maxDecidedRX; - } - - @VisibleForTesting - public final void unsafeSetRangesForEpoch(RangesForEpoch newRangesForEpoch) - { - rangesForEpoch = nonNull(newRangesForEpoch); - } - - protected final void unsafeClearRangesForEpoch() - { - rangesForEpoch = null; - } - - protected void loadRangesForEpoch(RangesForEpoch newRangesForEpoch) - { - Invariants.require(this.rangesForEpoch == null); - unsafeSetRangesForEpoch(newRangesForEpoch); - } - - public abstract boolean inStore(); - - public boolean tryExecuteImmediately(Runnable run) - { - if (!inStore()) - return false; - - try { run.run(); } - catch (Throwable t) { agent.onException(t); } - return true; - } - - public abstract AsyncChain chain(PreLoadContext context, Consumer consumer); - public abstract AsyncChain chain(PreLoadContext context, Function apply); - - public Cancellable execute(PreLoadContext context, Consumer consumer, BiConsumer callback) - { - return chain(context, consumer).begin(callback); - } - - public AsyncResult execute(PreLoadContext context, Consumer consumer) - { - return chain(context, consumer).beginAsResult(); - } - - public Cancellable execute(PreLoadContext context, Function apply, BiConsumer callback) - { - return chain(context, apply).begin(callback); - } - - public AsyncResult submit(PreLoadContext context, Function apply) - { - return chain(context, apply).beginAsResult(); - } - - public abstract void shutdown(); - - protected void unsafeSetMaxDecidedRX(MaxDecidedRX newMaxDecidedRX) - { - this.maxDecidedRX = newMaxDecidedRX; - } - - protected void unsafeSetRejectBefore(RejectBefore newRejectBefore) - { - this.rejectBefore = newRejectBefore; - } - - final void unsafeSetRedundantBefore(RedundantBefore newRedundantBefore) - { - redundantBefore = newRedundantBefore; - } - - protected void unsafeClearRedundantBefore() - { - unsafeSetRedundantBefore(null); - } - - protected void loadRedundantBefore(RedundantBefore newRedundantBefore) - { - Invariants.require(redundantBefore == null || redundantBefore.equals(RedundantBefore.EMPTY)); - Invariants.require(newRedundantBefore != null); - unsafeSetRedundantBefore(newRedundantBefore); - } - - protected void unsafeUpsertRedundantBefore(RedundantBefore addRedundantBefore) - { - unsafeSetRedundantBefore(RedundantBefore.merge(redundantBefore, addRedundantBefore)); - } - - @VisibleForTesting - public boolean unsafeIsRefusingAny() - { - return refuses != null; - } - - protected void unsafeRefuseRequests(Ranges refuse) - { - Invariants.require(refuses == null || !refuses.intersects(refuse)); - if (refuses == null) refuses = refuse; - else refuses = refuses.with(refuse); - } - - protected void unsafeAcceptRequests(Ranges accept) - { - Invariants.require(refuses != null && refuses.containsAll(accept)); - refuses = refuses.without(accept); - if (refuses.isEmpty()) - refuses = null; - } - - /** - * This method may be invoked on a non-CommandStore thread - */ - final void unsafeSetSafeToRead(NavigableMap newSafeToRead) - { - if (newSafeToRead != null) - { - for (Map.Entry entry : newSafeToRead.entrySet()) - { - Ranges rangeExcluded = entry.getValue().without(this.permanentlyUnsafeToRead); - logger.info("{} is excluded from newSafeToRead because it is in the regained ranges", rangeExcluded); - } - } - - node.updateStamp(); - this.safeToRead = newSafeToRead; - } - - final void unsafeSetPermanentlyUnsafeToRead(Ranges newPermanentlyUnsafeToRead) - { - this.permanentlyUnsafeToRead = newPermanentlyUnsafeToRead; - } - - protected final void unsafeClearSafeToRead() - { - unsafeSetSafeToRead(null); - } - - protected void loadSafeToRead(NavigableMap newSafeToRead) - { - Invariants.require(safeToRead == null || safeToRead.equals(emptySafeToRead())); - Invariants.require(newSafeToRead != null); - unsafeSetSafeToRead(newSafeToRead); - updateMaxConflicts(newSafeToRead); - } - - final void unsafeSetBootstrapBeganAt(NavigableMap newBootstrapBeganAt) - { - this.bootstrapBeganAt = newBootstrapBeganAt; - } - - protected final void unsafeClearBootstrapBeganAt() - { - unsafeSetBootstrapBeganAt(null); - } - - protected synchronized void loadBootstrapBeganAt(NavigableMap newBootstrapBeganAt) - { - Invariants.require(bootstrapBeganAt == null || bootstrapBeganAt.equals(emptyBootstrapBeganAt())); - Invariants.require(newBootstrapBeganAt != null); - unsafeSetBootstrapBeganAt(newBootstrapBeganAt); - updateMaxConflicts(newBootstrapBeganAt); - } - - /** - * To be overridden by implementations, to ensure the new state is persisted. - */ - protected void setMaxConflicts(MaxConflicts maxConflicts) - { - this.maxConflicts = maxConflicts; - } - - protected int dumpCounter = 0; - - protected void updateMaxConflicts(Command prev, Command updated, boolean force) - { - Timestamp executeAt = updated.executeAt(); - if (executeAt == null) return; - if (prev != null && prev.executeAt() != null && prev.executeAt().compareToStrict(executeAt) >= 0 && !force) return; - executeAt = executeAt.flattenUniqueHlc(); // this is what guarantees a bootstrap recipient can compute uniqueHlc safely - MaxConflicts updatedMaxConflicts = maxConflicts.update(updated.participants().hasTouched(), executeAt); - updateMaxConflicts(executeAt, updatedMaxConflicts); - } - - protected void updateMaxConflicts(Ranges ranges, Timestamp executeAt) - { - updateMaxConflicts(executeAt, maxConflicts.update(ranges, executeAt)); - } - - protected void updateMaxConflicts(NavigableMap map) - { - Timestamp max = Timestamp.NONE; - MaxConflicts updated = maxConflicts; - for (Map.Entry e : map.entrySet()) - { - Timestamp at = e.getKey(); - if (at.compareTo(Timestamp.NONE) > 0) - { - updated = updated.update(e.getValue(), at); - max = Timestamp.max(max, at); - } - } - if (updated != maxConflicts) - updateMaxConflicts(max, updated); - } - - protected void updateMaxConflicts(Timestamp executeAt, MaxConflicts updatedMaxConflicts) - { - if (++maxConflictsUpdates >= agent.maxConflictsPruneInterval()) - { - int initialSize = updatedMaxConflicts.size(); - MaxConflicts initialConflicts = updatedMaxConflicts; - long pruneHlc = executeAt.hlc() - agent.maxConflictsHlcPruneDelta(); - Timestamp pruneBefore = pruneHlc > 0 ? Timestamp.fromValues(executeAt.epoch(), pruneHlc, executeAt.node) : null; - Ranges ranges = rangesForEpoch.all(); - if (pruneBefore != null) - updatedMaxConflicts = updatedMaxConflicts.update(ranges, pruneBefore); - - int prunedSize = updatedMaxConflicts.size(); - if (initialSize > 100 && prunedSize == initialSize) - { - logger.debug("Ineffective prune for {}. Initial size: {}, pruned size: {}, executeAt: {}, pruneBefore: {}", ranges, initialSize, prunedSize, executeAt, pruneBefore); - if (dumpCounter == 0) - { - logger.trace("initial MaxConflicts dump: {}", initialConflicts); - logger.trace("pruned MaxConflicts dump: {}", updatedMaxConflicts); - } - dumpCounter++; - dumpCounter %= 100; - } - else if (prunedSize != initialSize) - { - logger.trace("Successfully pruned {} to {}", initialSize, prunedSize); - } - - - maxConflictsUpdates = 0; - } - setMaxConflicts(updatedMaxConflicts); - } - - final void markExclusiveSyncPoint(SafeCommandStore safeStore, TxnId txnId, Ranges ranges) - { - // TODO (desired): narrow ranges to those that are owned - Invariants.requireArgument(txnId.isSyncPoint()); - RejectBefore newRejectBefore = rejectBefore != null ? rejectBefore : new RejectBefore(); - newRejectBefore = RejectBefore.add(newRejectBefore, ranges, txnId); - unsafeSetRejectBefore(newRejectBefore); - } - - final void markExclusiveSyncPointDecided(SafeCommandStore safeStore, TxnId txnId, Ranges ranges) - { - unsafeSetMaxDecidedRX(maxDecidedRX.update(ranges, txnId)); - } - - final void markExclusiveSyncPointLocallyApplied(SafeCommandStore safeStore, TxnId txnId, Ranges ranges) - { - // TODO (desired): narrow ranges to those that are owned - Invariants.requireArgument(txnId.isSyncPoint()); - RedundantBefore addNow = RedundantBefore.create(ranges, txnId, LOCALLY_APPLIED_ONLY); - safeStore.upsertRedundantBefore(addNow); - RedundantBefore addOnDataStoreDurable = RedundantBefore.create(ranges, txnId, LOCALLY_DURABLE_TO_DATA_STORE_ONLY); - RedundantBefore addOnCommandStoreDurable = RedundantBefore.create(ranges, txnId, LOCALLY_DURABLE_TO_COMMAND_STORE_ONLY); - dataStore.ensureDurable(this, ranges, addOnDataStoreDurable); - ensureDurable(ranges, addOnCommandStoreDurable); - } - - /** - * We expect keys to be sliced to those owned by the replica in the coordination epoch - */ - final Timestamp preaccept(TxnId txnId, Routables keys, SafeCommandStore safeStore, boolean permitFastPath) - { - NodeCommandStoreService node = safeStore.node(); - - boolean isExpired = safeStore.agent().rejectPreAccept(safeStore.node(), txnId) && !txnId.isSyncPoint(); - if (rejectBefore != null && !isExpired) - isExpired = rejectBefore.rejects(txnId, keys); - - if (isExpired) - return node.uniqueTimestamp(txnId).asRejected(); - - Timestamp min = TxnId.mergeMax(txnId, maxConflicts.get(keys)); - if (permitFastPath && txnId == min && txnId.epoch() >= node.epoch()) - return txnId; - - return node.uniqueTimestamp(min); - } - - /** - * We expect keys to be sliced to those owned by the replica in the coordination epoch - */ - public final Timestamp maxConflict(Routables keys) - { - return maxConflicts.get(keys); - } - - @Override - public String toString() - { - return getClass().getSimpleName() + "{id=" + id + ", node=" + node.id().id + '}'; - } - - public final AsyncResult cancelBootstraps() - { - return submit((Empty)() -> "Cancel Bootstraps", safeStore -> { - cancelBootstraps(safeStore, safeStore.ranges().all()); - return null; - }); - } - - public final void cancelBootstraps(SafeCommandStore safeStore, Ranges ranges) - { - Invariants.require(safeStore.commandStore() == this && inStore()); - bootstraps.forEach(b -> b.invalidate(ranges)); - } - - public final AsyncResult resumeBootstrap(Node node) - { - synchronized (this) - { - Invariants.require(!hasResumedBootstraps); - hasResumedBootstraps = true; - } - - return submit((Empty)() -> "Resume Bootstrap", safeStore -> { - Ranges unfinished = rangesForEpoch.all(); - unfinished = unfinished.without(safeToRead.lastEntry().getValue()); - unfinished = redundantBefore.removeLostOrStale(unfinished); - for (Bootstrap bootstrap : bootstraps) - unfinished = unfinished.without(bootstrap.all); - - long epoch = rangesForEpoch.epochAtIndex(0); - if (unfinished.isEmpty()) - return done(epoch); - - logger.info("{}: Resuming bootstrap of {}", this, unfinished); - return epochReadyAfterBootstrap(unfinished, epoch, startSafeBootstrapInternal(node, safeStore, unfinished, epoch)); - }); - } - - /** - * Defer submitting the work until we have wired up any changes to topology in memory, then first submit the work - * to setup any state in the command store, and finally submit the distributed work to bootstrap the data locally. - * So, the outer future's success is sufficient for the topology to be acknowledged, and the inner future for the - * bootstrap to be complete. - */ - final Supplier bootstrapper(Node node, Ranges newRanges, long epoch, BootstrapRangeAction action) - { - switch (action) - { - default: throw new UnhandledEnum(action); - case BOOTSTRAP_NOT_NEEDED: - return () -> { - AsyncResult done = execute((Empty) () -> "Initialise New Epoch", (safeStore) -> { - logger.info("{}: Initialising {} for epoch {}", this, newRanges, epoch); - // Merge in a base for any ranges that needs to be covered - Ranges newBootstrapRanges = newRanges; - for (Ranges existing : bootstrapBeganAt.values()) - newBootstrapRanges = newBootstrapRanges.without(existing); - if (!newBootstrapRanges.isEmpty()) - safeStore.setBootstrapBeganAt(bootstrap(TxnId.NONE, newBootstrapRanges, bootstrapBeganAt)); - safeStore.setSafeToRead(purgeAndInsert(safeToRead, TxnId.NONE, newRanges)); - markExclusiveSyncPointDecided(safeStore, TxnId.NONE, newRanges); - }); - - return EpochReady.all(epoch, done); - }; - case SAFE_BOOTSTRAP: - return () -> epochReadyAfterBootstrap(newRanges, epoch, startSafeBootstrap(node, newRanges, epoch)); - - case UNSAFE_BOOTSTRAP: - return () -> epochReadyAfterBootstrap(newRanges, epoch, startUnsafeBootstrap(node, newRanges, epoch, Image)); - } - } - - private EpochReady epochReadyAfterBootstrap(Ranges newRanges, long epoch, AsyncResult bootstrap) - { - return epochReadyAfterBootstrap(newRanges, epoch, EpochReady.wrap(epoch, bootstrap)); - } - - private EpochReady epochReadyAfterBootstrap(Ranges newRanges, long epoch, EpochReady bootstrap) - { - AsyncResult readyToCoordinate = readyToCoordinate(newRanges, epoch); - return new EpochReady(epoch, - bootstrap.active, - readyToCoordinate, - bootstrap.data, - bootstrap.reads); - } - - private AsyncResult startSafeBootstrap(Node node, Ranges newRanges, long epoch) - { - return submit((Empty) () -> "New Epoch", safeStore -> { - return startSafeBootstrapInternal(node, safeStore, newRanges, epoch); - }); - } - - private static final AsyncResult MUST_OVERWRITE = AsyncResults.failure(new IllegalStateException()); - private EpochReady startSafeBootstrapInternal(Node node, SafeCommandStore safeStore, Ranges newRanges, long epoch) - { - logger.info("{}: Starting Safe Bootstrap for {} for epoch {}", this, newRanges, epoch); - Bootstrap bootstrap = new Bootstrap(node, this, epoch, newRanges); - bootstraps.add(bootstrap); - bootstrap.start(safeStore); - return new EpochReady(epoch, - MUST_OVERWRITE, - MUST_OVERWRITE, - bootstrap.data, - bootstrap.reads); - } - - /** - * Rebootstraps some of the ranges for the command store. It follows steps similar to what - * bootstrap would go through, with two differences: - * - * * Marks pre-rebootstrap transactions with LOCALLY_LOST status, which means the node can not - * safely participate in pre-rebootstrap transactions, _even_ if they're coming after the node is - * done bootstrapping. - * * Marks the store as rebootstrapping, which will preclude rebootstrapping node from responding - * to PreAccept, Accept, and BeginRecovery and computing dependencies while node is being rebootstrapped, - * and ranges aren't ready to coordinate. - */ - protected AsyncResult startUnsafeBootstrap(Node node, Ranges ranges, long epoch, FetchKind fetch) - { - return submit((Empty) () -> "Refuse Requests for " + fetch + " Bootstrap", safeStore -> { - unsafeRefuseRequests(ranges); - safeStore.setSafeToRead(purgeHistory(safeToRead, ranges)); - // TODO (expected): rationalise with startSafeBootstrap - String description = "Bootstrap " + ranges + " for epoch " + epoch + " in " + this; - return new EpochReady(epoch, MUST_OVERWRITE, readyToCoordinate(ranges, epoch), new SettableWithDescription<>(description), new SettableWithDescription<>(description)); - }).invoke((success, fail) -> { - if (fail != null) logger.error("Fatal error initiating {} bootstrap for {}", this, fetch, fail); - else rebootstrap(node, ranges, epoch, 1, success, fetch); - }); - } - - private void rebootstrap(Node node, Ranges ranges, long epoch, int attempt, EpochReady ready, FetchKind fetch) - { - CoordinateMaxConflict - .maxConflict(node, ranges) - .recover(failure -> { - Runnable retry = () -> rebootstrap(node, ranges, epoch, attempt + 1, ready, fetch); - Runnable fail = () -> { - ((SettableByCallback)ready.data).tryFailure(failure); - ((SettableByCallback)ready.reads).tryFailure(failure); - }; - agent.ownershipEvents().onFailedBootstrap(attempt, "Fetch Max Conflict (to mark log safe at)", ranges, retry, fail, failure); - return AsyncChains.failure(failure); - }).flatMap(success -> chain((Empty) () -> "Initiate Unsafe " + fetch + " Bootstrap", safeStore -> { - node.uniqueNow(success.hlc()); // ensure we pick a higher timestamp than the maximum conflict we found globally - // Mark unsafe to read first - - Ranges remaining = ranges.slice(rangesForEpoch.currentRanges(), Minimal); - if (remaining.isEmpty()) - { - logger.info("Terminating unsafe {} bootstrap process for {} as no active ranges", fetch, this); - return AsyncChains.success(null); - } - - Bootstrap bootstrap = new Bootstrap(node, this, epoch, remaining, fetch); - bootstraps.add(bootstrap); - // If rebootstrap can grab a later timestamp for subsequent attempts, but this timestamp is enough for us - // to establish which transactions, for which ranges the node can safely participate in). - TxnId unreadyBefore = bootstrap.start(safeStore); - safeStore.unsafeUpsertRedundantBefore(RedundantBefore.create(ranges, unreadyBefore, LOG_UNAVAILABLE_ONLY)); - updateMaxConflicts(ranges, unreadyBefore); - // TODO (desired): we could start accepting non-dep requests here - bootstrap.data.invoke((SettableByCallback)ready.data); - bootstrap.reads.invoke((SettableByCallback)ready.reads); - ready.coordinate.invokeIfSuccess(() -> { - execute((Empty)() -> "Accept Dependency Requests", safeStore0 -> { - unsafeAcceptRequests(remaining); - }); - }); - return null; - })).begin(agent); - } - - /** - * Defer submitting the work until we have wired up any changes to topology in memory, then first submit the work - * to setup any state in the command store, and finally submit the distributed work to bootstrap the data locally. - * So, the outer future's success is sufficient for the topology to be acknowledged, and the inner future for the - * bootstrap to be complete. - */ - protected Supplier refreshReadyToCoordinate(Node node, Ranges ranges, long epoch) - { - return () -> { - AsyncResult readyToCoordinate = readyToCoordinate(ranges, epoch); - return new EpochReady(epoch, DONE, readyToCoordinate, DONE, DONE); - }; - } - - // may be invoked by any thread without holding the command store lock - private AsyncResult readyToCoordinate(Ranges ranges, long epoch) - { - if (redundantBefore.min(ranges, Bounds::locallyWitnessedBefore).epoch() >= epoch) - return DONE; - - SettableResult whenDone = new SettableWithDescription<>(this + " is ready to coordinate " + ranges + " on epoch " + epoch); - TxnId minForEpoch = TxnId.minForEpoch(epoch); - Ranges remaining = redundantBefore.removeWitnessed(minForEpoch, ranges); - WaitingOnVisibility sync = new WaitingOnVisibility(whenDone, remaining); - synchronized (waitingOnVisibility) - { - WaitingOnVisibility prev = waitingOnVisibility.putIfAbsent(epoch, sync); - Invariants.require(prev == null); - } - ensureReadyToCoordinate(epoch, ranges); - return whenDone; - } - - private void ensureReadyToCoordinate(long epoch, Ranges ranges) - { - TxnId minForEpoch = TxnId.minForEpoch(epoch); - node.durability().close("[" + this + " Epoch " + epoch + ']', VisibilitySyncPoint, minForEpoch, ranges, 1, TimeUnit.HOURS) - .invoke((success, fail) -> { - if (fail != null) - { - Ranges notRetired = redundantBefore.removeRetired(ranges); - Ranges retired = ranges.without(notRetired); - Ranges remaining = redundantBefore.removeWitnessed(minForEpoch, notRetired); - - if (!retired.isEmpty()) - { - logger.info("Failed to close epoch {} for ranges {} on store {}, but some are retired; marking these as synced.", epoch, ranges, id, fail); - execute((Empty)() -> "Mark Retired Ranges Synced", safeStore -> { - markVisibleInternal(safeStore, epoch, retired, "(Retired)"); - }); - } - else if (remaining.isEmpty()) - { - logger.info("Failed to close epoch {} for ranges {} on store {}, but none remaining. Aborting.", epoch, ranges, id, fail); - } - if (!remaining.isEmpty()) - { - logger.error("Failed to close epoch {} for ranges {} on store {}. Retrying.", epoch, remaining, id, fail); - node.someExecutor().execute(() -> ensureReadyToCoordinate(epoch, remaining)); - } - } - }); - } - - Supplier unbootstrap(long epoch, Ranges removedRanges) - { - return () -> { - AsyncResult done = submit((Empty) () -> "Unbootstrap", safeStore -> { - for (Bootstrap prev : bootstraps) - { - Ranges abort = prev.allValid.slice(removedRanges, Minimal); - if (!abort.isEmpty()) - prev.invalidate(abort); - } - return null; - }); - - return new EpochReady(epoch, done, done, done, done); - }; - } - - final void complete(Bootstrap bootstrap) - { - bootstraps.remove(bootstrap); - } - - final void markBootstrapping(SafeCommandStore safeStore, TxnId globalSyncId, Ranges ranges) - { - safeStore.setBootstrapBeganAt(bootstrap(globalSyncId, ranges, bootstrapBeganAt)); - safeStore.setSafeToRead(purgeHistory(safeToRead, ranges)); - updateMaxConflicts(ranges, globalSyncId); - RedundantBefore addRedundantBefore = RedundantBefore.create(ranges, Long.MIN_VALUE, Long.MAX_VALUE, globalSyncId, UNREADY_ONLY); - safeStore.upsertRedundantBefore(addRedundantBefore); - } - - // TODO (expected): we can immediately truncate dependencies locally once an exclusiveSyncPoint applies, we don't need to wait for the whole shard - public void markShardDurable(SafeCommandStore safeStore, TxnId globalSyncId, Ranges durableRanges, HasOutcome durability) - { - if (!durability.isDurable()) - return; - - SomeStatus status = durability.isUniversal() ? SHARD_APPLIED_ONLY : QUORUM_APPLIED_ONLY; - final Ranges slicedRanges = durableRanges.slice(safeStore.ranges().allUntil(globalSyncId.epoch()), Minimal); - TxnId locallyRedundantBefore = safeStore.redundantBefore().min(slicedRanges, Bounds::maxLocallyAppliedBefore); - RedundantBefore addNow = RedundantBefore.create(slicedRanges, globalSyncId, status); - safeStore.upsertRedundantBefore(addNow); - - if (status != SHARD_APPLIED_ONLY) - return; - - if (locallyRedundantBefore.compareTo(globalSyncId) < 0) - { - // TODO (expected): if bootstrapping only part of the range, mark the rest for GC; or relax this as can safely GC behind bootstrap - TxnId maxBootstrap = safeStore.redundantBefore().max(slicedRanges, Bounds::maxReadyAt); - if (maxBootstrap.compareTo(globalSyncId) >= 0) - logger.info("Ignoring markShardDurable for a point we are bootstrapping. Bootstrapping: {}, Global: {}, Ranges: {}", maxBootstrap, globalSyncId, slicedRanges); - else - logger.warn("Trying to markShardDurable a point we have not yet caught-up to locally. Local: {}, Global: {}, Ranges: {}", locallyRedundantBefore, globalSyncId, slicedRanges); - return; - } - - // TODO (desired): not all systems care about HLC_BOUND for GC, make configurable - if (globalSyncId.is(HLC_BOUND) || !requiresUniqueHlcs()) - { - RedundantBefore addOnDataStoreDurable = RedundantBefore.create(slicedRanges, globalSyncId, GC_BEFORE_AND_LOCALLY_DURABLE); - dataStore.ensureDurable(this, slicedRanges, addOnDataStoreDurable); - } - } - - protected void updatedRedundantBefore(SafeCommandStore safeStore, RedundantBefore added) - { - TxnId clearWaitingBefore = redundantBefore.minShardAndLocallyAppliedBefore(); - TxnId clearAllBefore = TxnId.min(clearWaitingBefore, durableBefore().min.quorumBefore); - progressLog.clearBefore(safeStore, clearWaitingBefore, clearAllBefore); - listeners.clearBefore(clearWaitingBefore); - } - - @VisibleForTesting - public AsyncResult awaitVisibility(long epoch, Ranges ranges) - { - synchronized (waitingOnVisibility) - { - if (waitingOnVisibility.isEmpty()) - return AsyncResults.success(null); - - List> awaiting = new ArrayList<>(); - for (Map.Entry e : waitingOnVisibility.entrySet()) - { - if (e.getKey() > epoch) - break; - - Ranges remaining = e.getValue().waitingOn; - Ranges intersecting = remaining.slice(ranges, Minimal); - if (!intersecting.isEmpty()) - { - awaiting.add(e.getValue().whenDone); - ranges = ranges.without(intersecting); - } - } - - if (awaiting.isEmpty()) - return AsyncResults.success(null); - return AsyncResults.debuggableReduce(awaiting, Reduce.toNull()); - } - } - - protected final Ranges isWaitingOnVisibility(TxnId syncId, Ranges ranges) - { - synchronized (waitingOnVisibility) - { - if (waitingOnVisibility.isEmpty()) - return Ranges.EMPTY; - - Ranges waitingOn = Ranges.EMPTY; - for (Map.Entry e : waitingOnVisibility.entrySet()) - { - if (e.getKey() > syncId.epoch()) - break; - - Ranges remaining = e.getValue().waitingOn; - Ranges intersecting = remaining.slice(ranges, Minimal); - if (!intersecting.isEmpty()) - { - ranges = ranges.without(intersecting); - waitingOn = waitingOn.with(intersecting); - } - } - - return waitingOn; - } - } - - protected final void markingVisible(TxnId syncId, Ranges ranges) - { - synchronized (waitingOnVisibility) - { - if (waitingOnVisibility.isEmpty()) - return; - - for (Map.Entry e : waitingOnVisibility.entrySet()) - { - if (e.getKey() > syncId.epoch()) - break; - - Ranges remaining = e.getValue().waitingOn.without(ranges); - if (e.getValue().waitingOn != remaining) - e.getValue().waitingOn = remaining; - } - } - } - - protected final void cancelMarkingVisible(TxnId syncId, Ranges ranges) - { - synchronized (waitingOnVisibility) - { - if (waitingOnVisibility.isEmpty()) - return; - - for (Map.Entry e : waitingOnVisibility.entrySet()) - { - if (e.getKey() > syncId.epoch()) - break; - - Ranges unmark = e.getValue().waitingOnDurable.slice(ranges, Minimal); - if (!unmark.isEmpty()) - e.getValue().waitingOn = e.getValue().waitingOn.with(unmark); - } - } - } - - protected final void markVisible(SafeCommandStore safeStore, TxnId syncId, Ranges ranges) - { - Invariants.require(syncId.is(VisibilitySyncPoint)); - RedundantBefore addRedundantBefore = RedundantBefore.create(ranges, syncId, LOCALLY_WITNESSED_ONLY); - safeStore.upsertRedundantBefore(addRedundantBefore); - markVisibleInternal(safeStore, syncId.epoch(), ranges, syncId); - } - - private void markVisibleInternal(SafeCommandStore safeStore, long epoch, Ranges ranges, Object describe) - { - synchronized (waitingOnVisibility) - { - if (waitingOnVisibility.isEmpty()) - return; - - LongHashSet remove = null; - for (Map.Entry e : waitingOnVisibility.entrySet()) - { - if (e.getKey() > epoch) - break; - - Ranges waitingOn = e.getValue().waitingOn; - Ranges waitingOnDurable = e.getValue().waitingOnDurable; - Ranges synced = waitingOnDurable.slice(ranges, Minimal); - boolean intersects = waitingOnDurable.intersects(ranges); - if (intersects) - { - e.getValue().waitingOn = waitingOn = waitingOn.without(ranges); - e.getValue().waitingOnDurable = waitingOnDurable = waitingOnDurable.without(ranges); - if (waitingOnDurable.isEmpty()) - { - SettableResult done = e.getValue().whenDone; - logger.debug("{} completed full visibility sync for {} on epoch {} using {}", this, e.getValue().allRanges, e.getKey(), describe); - done.trySuccess(null); - if (remove == null) - remove = new LongHashSet(); - remove.add(e.getKey()); - } - else - { - logger.debug("{} completed partial visibility sync for {} on epoch {} using {}; {} still to sync and {} to sync durably", this, synced, e.getKey(), describe, waitingOn, waitingOnDurable); - } - } - } - if (remove != null) - remove.forEach(waitingOnVisibility::remove); - } - } - - public void markShardStale(SafeCommandStore safeStore, Timestamp staleSince, Ranges ranges, boolean isSincePrecise) - { - Timestamp staleUntilAtLeast = staleSince; - if (isSincePrecise) - { - ranges = ranges.slice(safeStore.ranges().allAt(staleSince.epoch()), Minimal); - } - else - { - ranges = ranges.slice(safeStore.ranges().allSince(staleSince.epoch()), Minimal); - // make sure no in-progress bootstrap attempts will override the stale since for commands whose staleness bounds are unknown - staleUntilAtLeast = Timestamp.max(bootstrapBeganAt.lastKey(), staleUntilAtLeast); - } - - if (ranges.isEmpty()) - return; - - agent.ownershipEvents().onStale(staleSince, ranges); - - RedundantBefore addRedundantBefore = RedundantBefore.createStale(ranges, staleUntilAtLeast); - safeStore.upsertRedundantBefore(addRedundantBefore); - // find which ranges need to bootstrap, subtracting those already in progress that cover the id - - markUnsafeToRead(ranges); - } - - /** - * This is a heavy-handed operator action to unstick waiting transactions whose transitive dependencies - * may already be applied. - */ - public final AsyncResult operatorTryToExecuteListeningTxns() - { - SettableResult done = new SettableResult<>(); - execute((Empty)() -> "Try Execute Listening", safeStore -> { - tryExecuteListening(safeStore, listeners.txnsWaitingOn(SaveStatus.Applied).iterator(), done); - }); - return done; - } - - private void tryExecuteListening(SafeCommandStore safeStore, Iterator iterator, SettableResult done) - { - if (!iterator.hasNext()) - { - done.trySuccess(null); - return; - } - - try - { - TxnId waitingOn = iterator.next(); - PreLoadContext context = PreLoadContext.contextFor(waitingOn, "Try Execute Listening"); - if (!safeStore.canExecuteWith(context) || !safeStore.tryRecurse()) - { - //noinspection DataFlowIssue - safeStore = safeStore; - execute(context, safeStore0 -> tryExecuteListening(safeStore0, waitingOn, iterator, done)); - } - else - { - try { tryExecuteListening(safeStore, waitingOn, iterator, done); } - finally { safeStore.unrecurse(); } - } - } - catch (Throwable t) - { - done.tryFailure(t); - } - } - - private void tryExecuteListening(SafeCommandStore safeStore, TxnId waitingOn, Iterator iterator, SettableResult done) - { - try - { - SafeCommand safeCommand = safeStore.unsafeGet(waitingOn); - //noinspection DataFlowIssue - safeStore = safeStore; - //noinspection DataFlowIssue - safeCommand = safeCommand; - boolean wasApplied = safeCommand.current().hasBeen(Status.Applied); - Consumer continuation = safeStore0 -> { - if (!wasApplied) - { - SafeCommand safeCommand0 = safeStore0.ifLoadedAndInitialised(waitingOn); - if (safeCommand0 != null && safeCommand0.current().saveStatus().hasBeen(Status.Applied)) - logger.warn("{} was successfully applied by tryToExecuteListening", waitingOn); - } - tryExecuteListening(safeStore0, iterator, done); - }; - - Commands.maybeExecute(safeStore, safeCommand, safeCommand.current(), true, true, NotifyWaitingOnPlus.adapter(continuation, true, true)); - } - catch (Throwable t) - { - done.tryFailure(t); - } - } - - public final boolean isRejectedIfNotPreAccepted(TxnId txnId, Unseekables participants) - { - if (rejectBefore == null) - return false; - - return rejectBefore.rejects(txnId, participants); - } - - public final MaxConflicts unsafeGetMaxConflicts() - { - return maxConflicts; - } - - public final RedundantBefore unsafeGetRedundantBefore() - { - return redundantBefore; - } - - public final LocalListeners unsafeGetListeners() - { - return listeners; - } - - @Nullable - public final RejectBefore unsafeGetRejectBefore() - { - return rejectBefore; - } - - public final DurableBefore durableBefore() - { - return node.durableBefore(); - } - - public final ProgressLog unsafeProgressLog() - { - return progressLog; - } - - @VisibleForTesting - public final NavigableMap unsafeGetBootstrapBeganAt() { return bootstrapBeganAt; } - - @VisibleForTesting - public NavigableMap unsafeGetSafeToRead() { return safeToRead; } - - final void markUnsafeToRead(Ranges ranges) - { - if (safeToRead.values().stream().anyMatch(r -> r.intersects(ranges))) - { - execute((Empty) () -> "Mark Unsafe To Read", safeStore -> { - safeStore.setSafeToRead(purgeHistory(safeToRead, ranges)); - }, agent); - } - } - - final AsyncChain markPermanentlyUnsafeToRead(Ranges ranges) - { - return chain((Empty) () -> "Mark Range As Regained", safeStore -> { - safeStore.setSafeToRead(purgeHistory(safeToRead, ranges)); - safeStore.setPermanentlyUnsafeToRead(permanentlyUnsafeToRead.union(MERGE_ADJACENT, ranges)); - }); - } - - public final DataStore unsafeGetDataStore() - { - return dataStore; - } - - final synchronized AsyncResult markSafeToRead(Timestamp forBootstrapAt, Timestamp at, Ranges ranges) - { - return execute((Empty) () -> "Mark Safe To Read", safeStore -> { - // TODO (required): handle weird edge cases like newer at having a lower HLC than prior existing at, but higher epoch - Ranges validatedSafeToRead = redundantBefore.validateSafeToRead(forBootstrapAt, ranges); - safeStore.setSafeToRead(purgeAndInsert(safeToRead, at, validatedSafeToRead)); - updateMaxConflicts(ranges, at); - }); - } - - public static ImmutableSortedMap bootstrap(TxnId at, Ranges ranges, NavigableMap readyAt) - { - Invariants.requireArgument(!ranges.isEmpty()); - if (at == TxnId.NONE) - { - for (Ranges rs : readyAt.values()) - Invariants.require(!ranges.intersects(rs)); - } - // if we're bootstrapping these ranges, then any period we previously owned the ranges for is effectively invalidated - return purgeAndInsert(readyAt, at, ranges); - } - - private static ImmutableSortedMap purgeAndInsert(NavigableMap in, T insertAt, Ranges insert) - { - TreeMap build = new TreeMap<>(in); - build.headMap(insertAt, false).entrySet().forEach(e -> e.setValue(e.getValue().without(insert))); - build.tailMap(insertAt, true).entrySet().forEach(e -> e.setValue(e.getValue().union(MERGE_ADJACENT, insert))); - build.entrySet().removeIf(e -> e.getKey().compareTo(Timestamp.NONE) > 0 && e.getValue().isEmpty()); - Map.Entry prev = build.floorEntry(insertAt); - build.putIfAbsent(insertAt, prev.getValue().with(insert)); - return ImmutableSortedMap.copyOf(build); - } - - private static ImmutableSortedMap purgeHistory(NavigableMap in, Ranges remove) - { - return ImmutableSortedMap.copyOf(purgeHistoryIterator(in, remove)); - } - - private static Iterable> purgeHistoryIterator(NavigableMap in, Ranges removeRanges) - { - return () -> in.entrySet().stream() - .map(e -> without(e, removeRanges)) - .filter(e -> !e.getValue().isEmpty() || e.getKey().equals(TxnId.NONE)) - .iterator(); - } - - private static Map.Entry without(Map.Entry in, Ranges remove) - { - Ranges without = in.getValue().without(remove); - if (without == in.getValue()) - return in; - return new SimpleImmutableEntry<>(in.getKey(), without); - } - - @Override - public int hashCode() - { - return id; - } - - public boolean isBootstrapping() - { - return !bootstraps.isEmpty(); - } - - public void updateMinHlc(long minHlc) - { - Timestamp timestamp = Timestamp.fromValues(rangesForEpoch.epochs[rangesForEpoch.epochs.length - 1], minHlc, 0, node.id()); - MaxConflicts updated = maxConflicts.update(rangesForEpoch.all(), timestamp); - setMaxConflicts(updated); - } - - public static NavigableMap emptyBootstrapBeganAt() - { - return ImmutableSortedMap.of(TxnId.NONE, Ranges.EMPTY); - } - - public static NavigableMap emptySafeToRead() - { - return ImmutableSortedMap.of(Timestamp.NONE, Ranges.EMPTY); - } - - public NodeCommandStoreService node() - { - return node; - } - - void unsafeRegister(SyncPointListener listener) - { - Invariants.require(inStore()); - List newListeners = new ArrayList<>(); - if (syncPointListeners != null) - newListeners.addAll(syncPointListeners); - newListeners.add(listener); - syncPointListeners = newListeners; - } - - void unsafeUnregister(SyncPointListener listener) - { - Invariants.require(inStore()); - if (syncPointListeners != null) - { - List newListeners = new ArrayList<>(syncPointListeners); - newListeners.remove(listener); - if (newListeners.isEmpty()) - newListeners = null; - syncPointListeners = newListeners; - } - } -} diff --git a/accord-core/src/main/java/accord/local/CommandStores.java b/accord-core/src/main/java/accord/local/CommandStores.java deleted file mode 100644 index 72aa60131c7a..000000000000 --- a/accord-core/src/main/java/accord/local/CommandStores.java +++ /dev/null @@ -1,1219 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package accord.local; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.Comparator; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.function.BiConsumer; -import java.util.function.Consumer; -import java.util.function.Supplier; -import java.util.stream.Collectors; -import java.util.stream.IntStream; -import java.util.stream.Stream; -import javax.annotation.Nonnull; -import javax.annotation.Nullable; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.Lists; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import accord.api.Agent; -import accord.api.AsyncExecutorFactory; -import accord.api.AsyncExecutor; -import accord.topology.EpochReady; -import accord.api.DataStore; -import accord.api.Journal; -import accord.api.LocalListeners; -import accord.api.ProgressLog; -import accord.api.RoutingKey; -import accord.local.CommandStore.EpochUpdateHolder; -import accord.primitives.AbstractRanges; -import accord.primitives.AbstractUnseekableKeys; -import accord.primitives.EpochSupplier; -import accord.primitives.Participants; -import accord.primitives.Range; -import accord.primitives.Ranges; -import accord.primitives.RoutingKeys; -import accord.primitives.Timestamp; -import accord.primitives.TxnId; -import accord.primitives.Unseekables; -import accord.topology.Shard; -import accord.topology.Topology; -import accord.utils.IndexedQuadConsumer; -import accord.utils.IndexedRangeQuadConsumer; -import accord.utils.Invariants; -import accord.utils.RandomSource; -import accord.utils.Reduce; -import accord.utils.SearchableRangeList; -import accord.utils.LargeBitSet; -import accord.utils.UnhandledEnum; -import accord.utils.async.AsyncChain; -import accord.utils.async.AsyncChains; -import accord.utils.async.AsyncResult; -import accord.utils.async.AsyncResults; -import accord.utils.async.Cancellable; -import accord.utils.async.NestedAsyncResult; -import org.agrona.collections.Hashing; -import org.agrona.collections.Int2IntHashMap; -import org.agrona.collections.Int2ObjectHashMap; - -import static accord.topology.EpochReady.done; -import static accord.api.DataStore.FetchKind.Sync; -import static accord.local.CommandStores.BootstrapRangeAction.BOOTSTRAP_NOT_NEEDED; -import static accord.local.CommandStores.BootstrapRangeAction.SAFE_BOOTSTRAP; -import static accord.primitives.Routables.Slice.Minimal; -import static accord.utils.Invariants.illegalState; -import static java.util.stream.Collectors.toList; - -/** - * Manages the single threaded metadata shards - */ -public abstract class CommandStores implements AsyncExecutorFactory -{ - @SuppressWarnings("unused") - private static final Logger logger = LoggerFactory.getLogger(CommandStores.class); - - static final Iterator INVALID = new Iterator<>() - { - @Override public boolean hasNext() { throw new UnsupportedOperationException(); } - @Override public CommandStore next() { throw new UnsupportedOperationException(); } - }; - - public interface LatentStoreSelector - { - StoreSelector refine(TxnId txnId, @Nullable Timestamp executeAt, Participants participants); - - class StandardLatentStoreSelector implements LatentStoreSelector - { - private static final StandardLatentStoreSelector INSTANCE = new StandardLatentStoreSelector(); - - @Override - public StoreSelector refine(TxnId txnId, @Nullable Timestamp executeAt, Participants participants) - { - return snapshot -> StoreFinder.find(snapshot, participants) - .filter(snapshot, participants, txnId.epoch(), (executeAt != null ? executeAt : txnId).epoch()) - .iterator(snapshot); - } - } - - static LatentStoreSelector standard() - { - return StandardLatentStoreSelector.INSTANCE; - } - } - - public interface StoreSelector extends LatentStoreSelector - { - default StoreSelector refine(TxnId txnId, @Nullable Timestamp executeAt, Participants participants) { return this; } - Iterator select(Snapshot snapshot); - } - - public static class IncludingSpecificStoreSelector implements StoreSelector - { - final int storeId; - - public IncludingSpecificStoreSelector(int storeId) - { - this.storeId = storeId; - } - - @Override - public StoreSelector refine(TxnId txnId, @Nullable Timestamp executeAt, Participants participants) - { - return snapshot -> { - StoreFinder finder = StoreFinder.find(snapshot, participants) - .filter(snapshot, participants, txnId.epoch(), (executeAt != null ? executeAt : txnId).epoch()); - finder.set(snapshot.byId.get(storeId)); - return finder.iterator(snapshot); - }; - } - - @Override - public Iterator select(Snapshot snapshot) - { - return Collections.singletonList(snapshot.byId(storeId)).iterator(); - } - } - - // TODO (required): as we get more tables this will become expensive to allocate; we need to index first by prefix - public static class StoreFinder extends LargeBitSet implements IndexedQuadConsumer, IndexedRangeQuadConsumer - { - final int[] indexMap; - private boolean invalid; - - private StoreFinder(int size, int[] indexMap) - { - super(size); - this.indexMap = indexMap; - } - - public StoreFinder(Snapshot snapshot) - { - this(snapshot.shards.length, snapshot.indexForRange); - } - - public static StoreSelector selector(Unseekables unseekables, long minEpoch, long maxEpoch) - { - return snapshot -> { - StoreFinder finder = StoreFinder.find(snapshot, unseekables); - finder.filter(snapshot, unseekables, minEpoch, maxEpoch); - return finder.iterator(snapshot); - }; - } - - public static StoreFinder find(Snapshot snapshot, Unseekables unseekables) - { - StoreFinder finder = new StoreFinder(snapshot); - switch (unseekables.domain()) - { - default: throw new UnhandledEnum(unseekables.domain()); - case Range: - { - int minIndex = 0; - for (Range range : (AbstractRanges)unseekables) - minIndex = snapshot.lookupByRange.forEachRange(range, finder, finder, null, null, null, null, minIndex); - break; - } - case Key: - { - int minIndex = 0; - for (RoutingKey key : (AbstractUnseekableKeys)unseekables) - minIndex = snapshot.lookupByRange.forEachKey(key, finder, finder, null, null, null, null, minIndex); - break; - } - } - return finder; - } - - public StoreFinder filter(Snapshot snapshot, Unseekables unseekables, long minEpoch, long maxEpoch) - { - for (int i = firstSetBit(); i >= 0 ; i = nextSetBit(i + 1, -1)) - { - ShardHolder shard = snapshot.shards[i]; - Ranges shardRanges = shard.ranges().allBetween(minEpoch, maxEpoch); - if (shardRanges != shard.ranges.all() && !shardRanges.intersects(unseekables)) - { - unset(i); - } - else if (unsafelyTouchesRegainedRanges(snapshot, shard, unseekables, minEpoch)) - { - invalid = true; - break; - } - } - return this; - } - - public Iterator iterator(Snapshot snapshot) - { - if (invalid) - return INVALID; - - return new Iterator<>() - { - int i = firstSetBit(); - @Override - public boolean hasNext() - { - return i >= 0; - } - - @Override - public CommandStore next() - { - CommandStore next = snapshot.shards[i].store; - i = nextSetBit(i + 1, -1); - return next; - } - }; - } - - @Override - public void accept(Object p1, Object p2, Object p3, Object p4, int index) - { - set(indexMap[index]); - } - - @Override - public void accept(Object p1, Object p2, Object p3, Object p4, int fromIndex, int toIndex) - { - for (int i = fromIndex ; i < toIndex ; ++i) - set(indexMap[i]); - } - } - - public interface Factory - { - CommandStores create(NodeCommandStoreService node, - Agent agent, - DataStore store, - RandomSource random, - Journal journal, - ShardDistributor shardDistributor, - ProgressLog.Factory progressLogFactory, - LocalListeners.Factory listenersFactory); - } - - private static class StoreSupplier - { - private final NodeCommandStoreService node; - private final Agent agent; - private final DataStore store; - private final ProgressLog.Factory progressLogFactory; - private final LocalListeners.Factory listenersFactory; - private final CommandStore.Factory shardFactory; - private final RandomSource random; - private final Journal journal; - - StoreSupplier(NodeCommandStoreService node, Agent agent, DataStore store, RandomSource random, ProgressLog.Factory progressLogFactory, LocalListeners.Factory listenersFactory, CommandStore.Factory shardFactory, Journal journal) - { - this.node = node; - this.agent = agent; - this.store = store; - this.random = random; - this.progressLogFactory = progressLogFactory; - this.listenersFactory = listenersFactory; - this.shardFactory = shardFactory; - this.journal = journal; - } - - CommandStore create(int id, EpochUpdateHolder rangesForEpoch) - { - return shardFactory.create(id, node, agent, this.store, progressLogFactory, listenersFactory, rangesForEpoch, journal); - } - } - - public static class ShardHolder - { - public final CommandStore store; - @Nullable final Ranges regainsRanges; - RangesForEpoch ranges; - - ShardHolder(CommandStore store, @Nullable Ranges regainsRanges) - { - this.store = store; - this.regainsRanges = regainsRanges; - } - - public ShardHolder(CommandStore store, RangesForEpoch ranges, @Nullable Ranges regainsRanges) - { - this.store = store; - this.regainsRanges = regainsRanges; - this.ranges = ranges; - } - - public RangesForEpoch ranges() - { - return ranges; - } - - boolean filter(long minEpoch, long maxEpoch, Unseekables unseekables) - { - Ranges shardRanges = ranges.allBetween(minEpoch, maxEpoch); - return shardRanges != ranges.all() && !shardRanges.intersects(unseekables); - } - - public String toString() - { - return store.id() + " " + ranges; - } - } - - public interface RangesForEpochSupplier - { - RangesForEpoch ranges(); - } - - public static final class PreviouslyOwned - { - public static final PreviouslyOwned EMPTY = new PreviouslyOwned(0, RangesForEpoch.EMPTY.epochs, RangesForEpoch.EMPTY.ranges); - final long maxEpoch; - final long[] epochs; // the epoch upon which it was last owned - final Ranges[] ranges; - - public PreviouslyOwned(long maxEpoch, long[] epochs, Ranges[] ranges) - { - this.maxEpoch = maxEpoch; - this.epochs = epochs; - this.ranges = ranges; - } - - PreviouslyOwned prepend(long epoch, Ranges ranges) - { - Invariants.require(epochs.length == 0 || epoch > epochs[0]); - long[] newEpochs = new long[this.epochs.length + 1]; - Ranges[] newRanges = new Ranges[epochs.length]; - newEpochs[0] = epoch; - newRanges[0] = ranges; - System.arraycopy(this.epochs, 0, newEpochs, 1, this.epochs.length); - System.arraycopy(this.ranges, 0, newRanges, 1, this.ranges.length); - return new PreviouslyOwned(epoch, newEpochs, newRanges); - } - - public boolean overlaps(long epoch, Unseekables test) - { - if (epoch > maxEpoch) - return false; - - for (int i = 0 ; i < epochs.length && epoch <= epochs[i] ; ++i) - { - if (this.ranges[i].intersects(test)) - return true; - } - - return false; - } - - public Ranges regains(Ranges overlapping) - { - Ranges regains = Ranges.EMPTY; - for (Ranges rs : this.ranges) - regains = regains.without(rs.slice(overlapping, Minimal)); - return regains; - } - - public int size() - { - return epochs.length; - } - - public long epochs(int i) - { - return epochs[i]; - } - - public Ranges ranges(int i) - { - return ranges[i]; - } - } - - // We ONLY remove ranges to keep logic manageable; likely to only merge CommandStores into a new CommandStore via some kind of Bootstrap - public static class RangesForEpoch - { - public static final RangesForEpoch EMPTY = new RangesForEpoch(new long[0], new Ranges[0]); - - final long[] epochs; - final Ranges[] ranges; - - public RangesForEpoch(long epoch, Ranges ranges) - { - this.epochs = new long[] { epoch }; - this.ranges = new Ranges[] { ranges }; - } - - public RangesForEpoch(long[] epochs, Ranges[] ranges) - { - Invariants.require(epochs.length == ranges.length); - this.epochs = epochs; - this.ranges = ranges; - } - - public int size() - { - return epochs.length; - } - - public void forEach(BiConsumer forEach) - { - for (int i = 0; i < epochs.length; i++) - forEach.accept(epochs[i], ranges[i]); - } - - @Override - public boolean equals(Object object) - { - if (this == object) return true; - if (object == null || getClass() != object.getClass()) return false; - RangesForEpoch that = (RangesForEpoch) object; - return Arrays.equals(epochs, that.epochs) && Arrays.equals(ranges, that.ranges); - } - - @Override - public int hashCode() - { - throw new UnsupportedOperationException(); - } - - public RangesForEpoch withRanges(long epoch, Ranges latestRanges) - { - Invariants.requireArgument(epochs.length == 0 || epochs[epochs.length - 1] <= epoch); - int newLength = epochs.length == 0 || epochs[epochs.length - 1] < epoch ? epochs.length + 1 : epochs.length; - long[] newEpochs = Arrays.copyOf(epochs, newLength); - Ranges[] newRanges = Arrays.copyOf(ranges, newLength); - newEpochs[newLength - 1] = epoch; - newRanges[newLength - 1] = latestRanges; - Invariants.require(newEpochs[newLength - 1] == 0 || newEpochs[newLength - 1] == epoch, "Attempted to override historic epoch %d with %d", newEpochs[newLength - 1], epoch); - return new RangesForEpoch(newEpochs, newRanges); - } - - public @Nonnull Ranges coordinates(TxnId txnId) - { - return allAt(txnId); - } - - public @Nonnull Ranges allAt(Timestamp at) - { - return allAt(at.epoch()); - } - - public @Nonnull Ranges allAt(long epoch) - { - int i = floorIndex(epoch); - if (i < 0) return Ranges.EMPTY; - return ranges[i]; - } - - /** - * Extend a previously computed set of Ranges that included {@code fromInclusive} - * to include ranges up to {@code toInclusive} - */ - public @Nonnull Ranges extend(Ranges extend, long curFrom, long curTo, long extendFrom, long extendTo) - { - if (extend.isEmpty()) // this captures the case where curTo < epochs[0] - return allBetween(extendFrom, extendTo); - - if (extendFrom >= curFrom) - return extend; - - int startCurIndex = floorIndex(curFrom); - int startExtendIndex = Math.max(0, floorIndex(extendFrom)); - if (startCurIndex <= startExtendIndex) - return extend; - - return ranges[startExtendIndex]; - } - - public @Nonnull Ranges allBetween(long fromInclusive, EpochSupplier toInclusive) - { - return allBetween(fromInclusive, toInclusive.epoch()); - } - - public @Nonnull Ranges allBetween(long fromInclusive, long toInclusive) - { - if (fromInclusive > toInclusive) - throw new IndexOutOfBoundsException(); - - int since = floorIndex(fromInclusive); - if (since >= 0) return ranges[since]; - - int to = floorIndex(toInclusive); - if (to >= 0) return ranges[0]; - return Ranges.EMPTY; - } - - public @Nonnull Ranges all() - { - return ranges[0]; - } - - public @Nonnull Ranges notRetired(SafeCommandStore safeStore) - { - return safeStore.redundantBefore().removeRetired(ranges[0]); - } - - public @Nonnull Ranges allBefore(long toExclusive) - { - int to = ceilIndex(toExclusive); - return to <= 0 ? Ranges.EMPTY : ranges[0]; - } - - public @Nonnull Ranges allUntil(long toInclusive) - { - int to = floorIndex(toInclusive); - return to < 0 ? Ranges.EMPTY : ranges[0]; - } - - public @Nonnull Ranges allSince(long fromInclusive) - { - int since = floorIndex(fromInclusive); - return ranges[Math.max(since, 0)]; - } - - public Ranges rangesAtIndex(int index) - { - return ranges[index]; - } - - public long epochAtIndex(int index) - { - return epochs[index]; - } - - public int floorIndex(long epoch) - { - int i = Arrays.binarySearch(epochs, epoch); - if (i < 0) i = -2 - i; - return i; - } - - private int ceilIndex(long epoch) - { - int i = Arrays.binarySearch(epochs, epoch); - if (i < 0) i = -1 - i; - return i; - } - - public int indexOffset(long lowEpoch, long highEpoch) - { - if (lowEpoch == highEpoch) - return 0; - - int lowIndex = Math.max(0, floorIndex(lowEpoch)); - int highIndex = lowIndex; - while (highIndex + 1 < epochs.length && epochs[highIndex + 1] <= highEpoch) - ++highIndex; - return highIndex - lowIndex; - } - - public @Nonnull Ranges currentRanges() - { - return ranges[ranges.length - 1]; - } - - public String toString() - { - return IntStream.range(0, ranges.length).mapToObj(i -> epochs[i] + ": " + ranges[i]) - .collect(Collectors.joining(", ")); - } - - public long earliestLaterEpochThatFullyCovers(long sinceEpoch, Unseekables keysOrRanges) - { - return Math.max(sinceEpoch, epochs[0]); - } - - public long latestEarlierEpochThatFullyCovers(long beforeEpoch, Unseekables keysOrRanges) - { - int i = ceilIndex(beforeEpoch); - if (i == 0) - return beforeEpoch; - - long latest = beforeEpoch; - Ranges existing = Ranges.EMPTY; - long next = beforeEpoch; - if (i < epochs.length) - { - existing = ranges[i]; - next = Math.min(next, epochs[i]); - } - while (--i >= 0) - { - if (ranges[i].without(existing).intersects(keysOrRanges)) - latest = next - 1; - existing = existing.with(ranges[i]); - next = epochs[i]; - } - return latest; - } - - public Ranges removed(long presentIn, long removedByInclusive) - { - int i = Math.max(1, floorIndex(presentIn)); - int maxi = 1 + floorIndex(removedByInclusive); - Ranges removed = Ranges.EMPTY; - while (i < maxi) - { - removed = removed.with(ranges[i - 1].without(ranges[i])); - ++i; - } - return removed; - } - } - - protected void loadSnapshot(Snapshot toLoad) - { - current = toLoad; - } - - public static class Snapshot extends Journal.TopologyUpdate implements Iterable - { - public final Topology local; - final ShardHolder[] shards; - final Int2IntHashMap byId; - private final int[] indexForRange; - final SearchableRangeList lookupByRange; - - public Snapshot(ShardHolder[] shards, Topology local, Topology global, PreviouslyOwned previouslyOwned) - { - super(asMap(shards), global, previouslyOwned); - this.local = local; - this.shards = shards; - this.byId = new Int2IntHashMap(shards.length, Hashing.DEFAULT_LOAD_FACTOR, -1); - int count = 0; - int prevId = -1; - for (int i = 0 ; i < shards.length ; ++i) - { - ShardHolder shard = shards[i]; - int id = shard.store.id; - Invariants.require(id > prevId); - byId.put(id, i); - count += shard.ranges.all().size(); - prevId = id; - } - class RangeAndIndex - { - final Range range; - final int index; - - RangeAndIndex(Range range, int index) - { - this.range = range; - this.index = index; - } - } - RangeAndIndex[] rangesAndIndexes = new RangeAndIndex[count]; - count = 0; - for (int i = 0; i < shards.length ; ++i) - { - Ranges add = shards[i].ranges.all(); - for (Range range : add) - rangesAndIndexes[count++] = new RangeAndIndex(range, i); - } - - Arrays.sort(rangesAndIndexes, (a, b) -> a.range.compareTo(b.range)); - - Range[] ranges = new Range[count]; - indexForRange = new int[count]; - for (int i = 0 ; i < rangesAndIndexes.length ; ++i) - { - ranges[i] = rangesAndIndexes[i].range; - indexForRange[i] = rangesAndIndexes[i].index; - } - lookupByRange = SearchableRangeList.build(ranges); - } - - // This method exists to ensure we do not hold references to command stores - public Journal.TopologyUpdate asTopologyUpdate() - { - return new Journal.TopologyUpdate(commandStores, global, previouslyOwned); - } - - private static Int2ObjectHashMap asMap(ShardHolder[] shards) - { - Int2ObjectHashMap commandStores = new Int2ObjectHashMap<>(); - for (ShardHolder shard : shards) - commandStores.put(shard.store.id, shard.ranges); - return commandStores; - } - - public CommandStore byId(int id) - { - return shards[byId.get(id)].store; - } - - @Override - public Iterator iterator() - { - return Arrays.asList(shards).iterator(); - } - } - - final StoreSupplier supplier; - final ShardDistributor shardDistributor; - final Journal journal; - volatile Snapshot current; - int nextId; - - private CommandStores(StoreSupplier supplier, ShardDistributor shardDistributor, Journal journal) - { - this.supplier = supplier; - this.shardDistributor = shardDistributor; - - this.current = new Snapshot(new ShardHolder[0], Topology.EMPTY, Topology.EMPTY, PreviouslyOwned.EMPTY); - this.journal = journal; - } - - public CommandStores(NodeCommandStoreService time, Agent agent, DataStore store, RandomSource random, Journal journal, ShardDistributor shardDistributor, - ProgressLog.Factory progressLogFactory, LocalListeners.Factory listenersFactory, CommandStore.Factory shardFactory) - { - this(new StoreSupplier(time, agent, store, random, progressLogFactory, listenersFactory, shardFactory, journal), shardDistributor, journal); - } - - public Node.Id nodeId() - { - return supplier.node.id(); - } - - public Topology local() - { - return current.local; - } - - public DataStore dataStore() - { - return supplier.store; - } - - static class TopologyUpdate - { - final Snapshot snapshot; - final Supplier bootstrap; - - TopologyUpdate(Snapshot snapshot, Supplier bootstrap) - { - this.snapshot = snapshot; - this.bootstrap = bootstrap; - } - } - - public enum BootstrapRangeAction - { - BOOTSTRAP_NOT_NEEDED, SAFE_BOOTSTRAP, UNSAFE_BOOTSTRAP - } - - protected BootstrapRangeAction shouldBootstrap(Node node, Topology prevGlobal, Topology newLocalTopology, Range add) - { - if (newLocalTopology.epoch() == 1 || !prevGlobal.ranges().contains(add)) - return BOOTSTRAP_NOT_NEEDED; - - return SAFE_BOOTSTRAP; - } - - public AsyncResult rebootstrap(Node node) - { - List> results = new ArrayList<>(); - Snapshot snapshot = current; - for (ShardHolder shard : snapshot.shards) - results.add(shard.store.startUnsafeBootstrap(node, shard.ranges.all(), snapshot.global.epoch(), Sync)); - return AsyncResults.allOf(results).flatMap(list -> { - return AsyncChains.reduce(list.stream() - .flatMap(b -> Stream.of(b.reads.chain(), b.coordinate.chain())) - .collect(Collectors.toList()), - Reduce.toNull()).beginAsResult(); - }); - } - - private synchronized TopologyUpdate updateTopology(Node node, Snapshot prev, Topology newTopology) - { - Invariants.requireArgument(!newTopology.isSubset(), "Use full topology for CommandStores.updateTopology"); - - long epoch = newTopology.epoch(); - if (epoch <= prev.global.epoch()) - { - Invariants.require(node.isReplaying(), "Received topology with epoch %d <= %d, but we are not replaying", epoch, prev.global.epoch()); - return new TopologyUpdate(prev, () -> done(epoch)); - } - - Topology newLocalTopology = newTopology.forNode(supplier.node.id()).trim(); - Ranges addedGlobal = newTopology.ranges().without(prev.global.ranges()); - node.addNewRangesToDurableBefore(addedGlobal, epoch); - - Ranges added = newLocalTopology.ranges().without(prev.local.ranges()); - Ranges subtracted = prev.local.ranges().without(newLocalTopology.ranges()); - - List> bootstrapUpdates = new ArrayList<>(); - List result = new ArrayList<>(prev.shards.length + added.size()); - PreviouslyOwned previouslyOwned = prev.previouslyOwned; - - for (ShardHolder shard : prev.shards) - { - Ranges current = shard.ranges().currentRanges(); - Ranges removeRanges = subtracted.slice(current, Minimal); - if (!removeRanges.isEmpty()) - { - // TODO (required): This is updating the a non-volatile field in the previous Snapshot, why modify it at all, even with volatile the guaranteed visibility is weak even with mutual exclusion - shard.ranges = shard.ranges().withRanges(newTopology.epoch(), current.without(subtracted)); - shard.store.epochUpdateHolder.remove(epoch, shard.ranges, removeRanges); - - bootstrapUpdates.add(shard.store.unbootstrap(epoch, removeRanges)); - } - - Ranges regainedRanges = shard.ranges().all().slice(added, Minimal); - if (!regainedRanges.isEmpty()) - bootstrapUpdates.add(() -> EpochReady.all(epoch, shard.store.markPermanentlyUnsafeToRead(regainedRanges).beginAsResult())); - - // TODO (desired): only sync affected shards - Ranges ranges = shard.ranges().currentRanges(); - // ranges can be empty when ranges are lost or consolidated across epochs. - if (epoch > 1 && requiresSync(ranges, prev.global, newTopology)) - { - logger.debug("Epoch {} requires visibility sync for {}", epoch, ranges); - bootstrapUpdates.add(shard.store.refreshReadyToCoordinate(node, ranges, epoch)); - } - - result.add(shard); - } - - if (!added.isEmpty()) - { - logger.info("Epoch {} adding {} to local command stores", epoch, added); - for (Ranges addRanges : shardDistributor.split(added)) - { - EpochUpdateHolder updateHolder = new EpochUpdateHolder(); - RangesForEpoch rangesForEpoch = new RangesForEpoch(epoch, addRanges); - updateHolder.add(epoch, rangesForEpoch, addRanges); - ShardHolder shard = new ShardHolder(supplier.create(nextId++, updateHolder), previouslyOwned.regains(addRanges)); - shard.ranges = rangesForEpoch; - - Map partitioned = addRanges.partitioningBy(range -> shouldBootstrap(node, prev.global, newLocalTopology, range), BootstrapRangeAction.class); - for (Map.Entry entry : partitioned.entrySet()) - { - BootstrapRangeAction action = entry.getKey(); - bootstrapUpdates.add(shard.store.bootstrapper(node, entry.getValue(), newLocalTopology.epoch(), action)); - } - result.add(shard); - } - } - - Supplier bootstrap; - if (bootstrapUpdates.isEmpty()) - { - logger.debug("Epoch {} implies no change to local command stores", epoch); - bootstrap = () -> done(epoch); - } - else - { - if (!subtracted.isEmpty()) - logger.info("Epoch {} removes {} from local command stores", epoch, subtracted); - - bootstrap = () -> { - List list = bootstrapUpdates.stream().map(Supplier::get).collect(toList()); - return new EpochReady(epoch, - AsyncResults.debuggableReduce(Lists.transform(list, EpochReady::active), Reduce.toNull()), - AsyncResults.debuggableReduce(Lists.transform(list, EpochReady::coordinate), Reduce.toNull()), - AsyncResults.debuggableReduce(Lists.transform(list, EpochReady::data), Reduce.toNull()), - AsyncResults.debuggableReduce(Lists.transform(list, EpochReady::reads), Reduce.toNull()) - ); - }; - } - - if (!subtracted.isEmpty()) - previouslyOwned = previouslyOwned.prepend(epoch - 1, subtracted); - - return new TopologyUpdate(new Snapshot(result.toArray(new ShardHolder[0]), newLocalTopology, newTopology, previouslyOwned), bootstrap); - } - - private static boolean requiresSync(Ranges ranges, Topology oldTopology, Topology newTopology) - { - List oldShards = oldTopology.foldl(ranges, (oldShard, shards, i) -> { - shards.add(oldShard); - return shards; - }, new ArrayList<>()); - - List newShards = newTopology.foldl(ranges, (newShard, shards, i) -> { - shards.add(newShard); - return shards; - }, new ArrayList<>()); - - if (oldShards.size() != newShards.size()) - return true; - - for (int i = 0 ; i < oldShards.size() ; ++i) - { - Shard oldShard = oldShards.get(i); - Shard newShard = newShards.get(i); - if (!oldShard.notInFastPath.equals(newShard.notInFastPath)) - return true; - - if (!newShard.nodes.equals(oldShard.nodes)) - return true; - } - return false; - } - - public void forAllUnsafe(Consumer forEach) - { - Snapshot snapshot = current; - for (ShardHolder shard : snapshot.shards) - forEach.accept(shard.store); - } - - public AsyncChain forAll(String reason, Consumer forEach) - { - return mapReduce(snapshot -> Stream.of(snapshot.shards).map(shard -> shard.store).iterator(), new MapReduceCommandStores<>(RoutingKeys.EMPTY) - { - @Override public Void reduce(Void o1, Void o2) { return null; } - @Override public TxnId primaryTxnId() { return null; } - @Override public String reason() { return reason; } - @Override - protected Void applyInternal(SafeCommandStore safeStore) - { - forEach.accept(safeStore); - return null; - } - }); - } - - public AsyncChain forEach(String reason, Participants participants, long minEpoch, long maxEpoch, Consumer forEach) - { - return forEach(reason, null, participants, minEpoch, maxEpoch, forEach); - } - - public AsyncChain forEach(String reason, TxnId txnId, Participants participants, long minEpoch, long maxEpoch, Consumer forEach) - { - return forEach(reason, txnId, participants, LoadKeys.SYNC, LoadKeysFor.READ_WRITE, minEpoch, maxEpoch, forEach); - } - - public AsyncChain forEach(String reason, TxnId txnId, Participants participants, LoadKeys loadKeys, LoadKeysFor loadKeysFor, long minEpoch, long maxEpoch, Consumer forEach) - { - return mapReduce(StoreFinder.selector(participants, minEpoch, maxEpoch), new MapReduceCommandStores, Void>(participants) - { - @Override public LoadKeys loadKeys() { return loadKeys;} - @Override public LoadKeysFor loadKeysFor() { return loadKeysFor; } - @Override public Void reduce(Void o1, Void o2) { return null; } - @Override public TxnId primaryTxnId() { return txnId; } - @Override public String reason() { return reason; } - @Override - protected Void applyInternal(SafeCommandStore safeStore) - { - forEach.accept(safeStore); - return null; - } - }); - } - - public Cancellable mapReduceConsume(long minEpoch, long maxEpoch, MapReduceConsumeCommandStores mapReduceConsume) - { - AsyncChain reduced = mapReduce(StoreFinder.selector(mapReduceConsume.scope, minEpoch, maxEpoch), mapReduceConsume); - return reduced.begin(mapReduceConsume); - } - - /** - * Maybe asynchronously, {@code apply} the function to each applicable {@code CommandStore}, invoke {@code reduce} - * on pairs of responses until only one remains, then {@code accept} the result. - * - * Note that {@code reduce} and {@code accept} are invoked by only one thread, and never concurrently with {@code apply}, - * so they do not require mutual exclusion. - */ - public Cancellable mapReduceConsume(StoreSelector selector, MapReduceConsumeCommandStores mapReduceConsume) - { - AsyncChain reduced = mapReduce(selector, mapReduceConsume); - return reduced.begin(mapReduceConsume); - } - - public Cancellable mapReduceConsume(IntStream commandStoreIds, MapReduceConsumeCommandStores mapReduceConsume) - { - AsyncChain reduced = mapReduce(commandStoreIds, mapReduceConsume); - return reduced.begin(mapReduceConsume); - } - - public AsyncChain mapReduce(IntStream commandStoreIds, MapReduceCommandStores mapReduce) - { - return mapReduce(snapshot -> commandStoreIds.mapToObj(snapshot::byId).iterator(), mapReduce); - } - - public AsyncChain mapReduce(StoreSelector selector, MapReduceCommandStores mapReduceConsume) - { - Snapshot snapshot = current; - Iterator stores = selector.select(snapshot); - if (stores == INVALID) - return AsyncChains.failure(new OverlappingCommandStoresException()); - - AsyncChain chain = null; - while (stores.hasNext()) - { - CommandStore store = stores.next(); - AsyncChain next = mapReduceConsume.applyAsync(store); - if (next != null) - chain = chain != null ? AsyncChains.reduce(chain, next, mapReduceConsume) : next; - } - - return chain == null ? AsyncChains.success(null) : chain; - } - - private static boolean unsafelyTouchesRegainedRanges(Snapshot snapshot, ShardHolder shard, Unseekables unseekables, long minEpoch) - { - if (shard.regainsRanges == null) - return false; - - unseekables = unseekables.slice(shard.regainsRanges, Minimal); - if (unseekables.isEmpty()) - return false; - - return snapshot.previouslyOwned.overlaps(minEpoch, unseekables); - } - - /** - * Initialize topology from snapshot on boot. - */ - public synchronized void initializeTopologyUnsafe(Journal.TopologyUpdate update) - { - Invariants.require(current.global.epoch() == 0); - ShardHolder[] shards = new ShardHolder[update.commandStores.size()]; - int i = 0; - int maxId = -1; - for (Map.Entry e : update.commandStores.entrySet()) - { - RangesForEpoch rfe = e.getValue(); - Invariants.require(rfe != null); - EpochUpdateHolder holder = new EpochUpdateHolder(); - holder.add(1, rfe, rfe.all()); - shards[i++] = new ShardHolder(supplier.create(e.getKey(), holder), rfe, update.previouslyOwned.regains(rfe.all())); - maxId = Math.max(maxId, e.getKey()); - } - Arrays.sort(shards, Comparator.comparingInt(shard -> shard.store.id)); - - nextId = maxId + 1; - loadSnapshot(new Snapshot(shards, update.global.forNode(supplier.node.id()).trim(), update.global, update.previouslyOwned)); - } - - public synchronized void resetTopology(Journal.TopologyUpdate update) - { - Snapshot current = this.current; - Invariants.require(update.global.epoch() == current.local.epoch()); - ShardHolder[] shards = new ShardHolder[current.commandStores.size()]; - int maxId = -1; - for (Map.Entry e : update.commandStores.entrySet()) - { - int storeId = e.getKey(); - RangesForEpoch rfe = e.getValue(); - Invariants.require(rfe != null); - ShardHolder shard = new ShardHolder(current.byId(storeId), rfe, update.previouslyOwned.regains(rfe.all())); - EpochUpdateHolder holder = shard.store.epochUpdateHolder; - rfe.forEach(new BiConsumer<>() - { - RangesForEpoch accumulator = null; - Ranges prev = null; - public void accept(Long epoch, Ranges ranges) - { - if (accumulator == null) - accumulator = new RangesForEpoch(epoch, ranges); - else - accumulator = accumulator.withRanges(epoch, ranges); - - Ranges additions = ranges; - Ranges removals = Ranges.EMPTY; - if (prev != null) - { - additions = ranges.without(prev); - removals = prev.without(ranges); - } - - if (!additions.isEmpty()) - holder.add(epoch, accumulator, additions); - if (!removals.isEmpty()) - holder.remove(epoch, accumulator, removals); - shard.store.unsafeUpdateRangesForEpoch(); - prev = ranges; - } - }); - - shards[storeId] = shard; - maxId = Math.max(maxId, storeId); - } - - nextId = maxId + 1; - loadSnapshot(new Snapshot(shards, current.local, current.global, update.previouslyOwned)); - } - - public synchronized Supplier updateTopology(Node node, Topology newTopology) - { - TopologyUpdate update = updateTopology(node, current, newTopology); - if (update.snapshot != current) - { - AsyncResults.SettableResult flush = new AsyncResults.SettableWithDescription<>("Write Topology To Journal"); - journal.saveTopology(update.snapshot.asTopologyUpdate(), () -> flush.setSuccess(null)); - current = update.snapshot; - return () -> { - EpochReady ready = update.bootstrap.get(); - return new EpochReady(ready.epoch, - ready.active, - NestedAsyncResult.flatMap(flush, ignore -> ready.coordinate), - NestedAsyncResult.flatMap(flush, ignore -> ready.data), - NestedAsyncResult.flatMap(flush, ignore -> ready.reads) - ); - }; - } - return update.bootstrap; - } - - public void shutdown() - { - for (ShardHolder shard : current.shards) - shard.store.shutdown(); - } - - @Override - public AsyncExecutor someExecutor() - { - return someSequentialExecutor(); - } - - @Override - public SequentialAsyncExecutor someSequentialExecutor() - { - return any(); - } - - @VisibleForTesting - public CommandStore any() - { - ShardHolder[] shards = current.shards; - if (shards.length == 0) throw illegalState("Unable to get CommandStore; non defined"); - return shards[supplier.random.nextInt(shards.length)].store; - } - - public CommandStore[] all() - { - ShardHolder[] shards = current.shards; - CommandStore[] all = new CommandStore[shards.length]; - for (int i = 0; i < shards.length; i++) - all[i] = shards[i].store; - return all; - } - - public CommandStore forId(int id) - { - Snapshot snapshot = current; - return snapshot.shards[snapshot.byId.get(id)].store; - } - - public int[] ids() - { - ShardHolder[] shards = current.shards; - int[] ids = new int[shards.length]; - for (int i = 0; i < ids.length; i++) - ids[i] = shards[i].store.id; - Arrays.sort(ids); - return ids; - } - - public int count() - { - return current.shards.length; - } - - public ShardDistributor shardDistributor() - { - return shardDistributor; - } - - @VisibleForTesting - public CommandStore unsafeForKey(RoutingKey key) - { - ShardHolder[] shards = current.shards; - for (ShardHolder shard : shards) - { - if (shard.ranges().currentRanges().contains(key)) - return shard.store; - } - throw new IllegalArgumentException(); - } - - protected Snapshot current() - { - return current; - } -} diff --git a/accord-core/src/main/java/accord/local/Node.java b/accord-core/src/main/java/accord/local/Node.java deleted file mode 100644 index 9c4f1620c48a..000000000000 --- a/accord-core/src/main/java/accord/local/Node.java +++ /dev/null @@ -1,886 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package accord.local; - -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.atomic.AtomicLongFieldUpdater; -import java.util.concurrent.locks.ReentrantLock; -import java.util.function.BiConsumer; -import java.util.function.Function; -import java.util.function.Supplier; -import java.util.stream.Stream; - -import javax.annotation.Nonnull; -import javax.annotation.Nullable; - -import com.google.common.annotations.VisibleForTesting; - -import accord.api.Agent; -import accord.api.AsyncExecutor; -import accord.api.TopologyService; -import accord.topology.Topologies; -import accord.topology.ActiveEpoch; -import accord.topology.ActiveEpochs; -import accord.topology.EpochReady; -import accord.api.DataStore; -import accord.api.Journal; -import accord.api.LocalListeners; -import accord.api.MessageSink; -import accord.api.ProgressLog; -import accord.api.RemoteListeners; -import accord.api.Result; -import accord.api.RoutingKey; -import accord.api.Scheduler; -import accord.api.Timeouts; -import accord.api.TopologySorter; -import accord.coordinate.CoordinateEphemeralRead; -import accord.coordinate.CoordinateTransaction; -import accord.coordinate.Coordination; -import accord.coordinate.CoordinationAdapter; -import accord.coordinate.CoordinationAdapter.Factory.Kind; -import accord.coordinate.Coordinations; -import accord.coordinate.Infer.InvalidIf; -import accord.coordinate.Outcome; -import accord.coordinate.PrepareRecovery; -import accord.local.CommandStores.LatentStoreSelector; -import accord.local.CommandStores.StoreSelector; -import accord.local.cfk.CommandsForKey; -import accord.local.durability.DurabilityService; -import accord.messages.Callback; -import accord.messages.Reply; -import accord.messages.ReplyContext; -import accord.messages.Request; -import accord.primitives.Ballot; -import accord.primitives.EpochSupplier; -import accord.primitives.FullRoute; -import accord.primitives.Ranges; -import accord.primitives.Routable.Domain; -import accord.primitives.Routables; -import accord.primitives.RoutingKeys; -import accord.primitives.Seekables; -import accord.primitives.Timestamp; -import accord.primitives.Txn; -import accord.primitives.TxnId; -import accord.primitives.TxnId.Cardinality; -import accord.topology.TopologyException; -import accord.topology.TopologyManager; -import accord.topology.TopologyRetiredException; -import accord.utils.Invariants; -import accord.utils.PersistentField; -import accord.utils.PersistentField.Persister; -import accord.utils.RandomSource; -import accord.utils.SortedArrays.SortedArrayList; -import accord.utils.async.AsyncChain; -import accord.utils.async.AsyncChains; -import accord.utils.async.AsyncResult; -import accord.utils.async.AsyncResults; -import accord.utils.async.Cancellable; -import net.nicoulaj.compilecommand.annotations.Inline; - -import static accord.api.ProtocolModifiers.Toggles.defaultMediumPath; -import static accord.api.ProtocolModifiers.Toggles.ensurePermitted; -import static accord.api.ProtocolModifiers.Toggles.usePrivilegedCoordinator; -import static accord.coordinate.Coordination.CoordinationKind.COORDINATES_STATE_MACHINE; -import static accord.primitives.Routable.Domain.Key; -import static accord.primitives.Routable.Domain.Range; -import static accord.primitives.Txn.Kind.Read; -import static accord.primitives.Txn.Kind.Write; -import static accord.primitives.TxnId.Cardinality.Any; -import static accord.primitives.TxnId.Cardinality.cardinality; -import static accord.primitives.TxnId.FastPath.Unoptimised; -import static java.util.concurrent.TimeUnit.MILLISECONDS; -import static java.util.concurrent.TimeUnit.NANOSECONDS; -import static java.util.concurrent.TimeUnit.SECONDS; - -public class Node implements NodeCommandStoreService -{ - public static class Id implements Comparable - { - public static final Id NONE = new Id(0); - public static final Id MAX = new Id(Integer.MAX_VALUE); - - public final int id; - - public Id(int id) - { - this.id = id; - } - - @Override - public int hashCode() - { - return Integer.hashCode(id); - } - - @Override - public boolean equals(Object that) - { - return that instanceof Id && equals((Id) that); - } - - public boolean equals(Id that) - { - if (that == null) return false; - return id == that.id; - } - - @Override - public int compareTo(Id that) - { - return Integer.compareUnsigned(this.id, that.id); - } - - public String toString() - { - return Integer.toString(id); - } - } - - private final Id id; - private final MessageSink messageSink; - private final TopologyManager topology; - private final RemoteListeners listeners; - private final Timeouts timeouts; - private final CommandStores commandStores; - private final CoordinationAdapter.Factory coordinationAdapters; - - private final TimeService time; - private final UniqueTimeService uniqueTime; - private final Agent agent; - private final RandomSource random; - - private final Scheduler scheduler; - private final DurabilityService durabilityService; - - private volatile DurableBefore durableBefore = DurableBefore.EMPTY; - private DurableBefore minDurableBefore = DurableBefore.EMPTY; - private final ReentrantLock durableBeforeLock = new ReentrantLock(); - private final PersistentField persistDurableBefore; - - private final Coordinations coordinations = new Coordinations(); - private final AtomicLong nextCoordinationId = new AtomicLong(); - - /** - * Used to guard some operations that should normally operate on consistent information, but in rare cases may need to repeat work. - * For simplicity we have a global stamp counter for this. - * At present, only used for managing unavailable() computations. - */ - private volatile long stamp; - private static final AtomicLongFieldUpdater stampUpdater = AtomicLongFieldUpdater.newUpdater(Node.class, "stamp"); - private volatile boolean replaying; - - public Node(Id id, MessageSink messageSink, - TopologyService topologyService, TimeService time, UniqueTimeService uniqueTime, - Supplier dataSupplier, ShardDistributor shardDistributor, Agent agent, RandomSource random, Scheduler scheduler, TopologySorter.Supplier topologySorter, - Function remoteListenersFactory, Function requestTimeoutsFactory, Function progressLogFactory, - Function localListenersFactory, CommandStores.Factory factory, CoordinationAdapter.Factory coordinationAdapters, - Persister durableBeforePersister, - Journal journal) - { - this.id = id; - this.scheduler = scheduler; // we set scheduler first so that e.g. requestTimeoutsFactory and progressLogFactory can take references to it - this.messageSink = messageSink; - this.coordinationAdapters = coordinationAdapters; - this.time = time; - this.uniqueTime = uniqueTime; - this.timeouts = requestTimeoutsFactory.apply(this); - this.listeners = remoteListenersFactory.apply(this); - this.agent = agent; - this.random = random; - this.persistDurableBefore = new PersistentField<>(() -> durableBefore, - (input, prev) -> { - DurableBefore next = DurableBefore.merge(input, prev); - if (next.equals(prev)) - return prev; - return next.equals(prev) ? prev : next; - }, - safeDurableBeforePersister(durableBeforePersister), - this::setPersistedDurableBefore); - this.commandStores = factory.create(this, agent, dataSupplier.get(), random.fork(), journal, shardDistributor, progressLogFactory.apply(this), localListenersFactory.apply(this)); - this.topology = new TopologyManager(topologySorter, this, topologyService, time, timeouts); - this.durabilityService = new DurabilityService(this); - // TODO (desired): make frequency configurable - scheduler.recurring(() -> commandStores.forAllUnsafe(store -> store.progressLog.maybeNotify()), 1, SECONDS); - scheduler.recurring(timeouts::maybeNotify, 100, MILLISECONDS); - } - - public void load() - { - persistDurableBefore.load(); - } - - public DurabilityService durability() - { - return durabilityService; - } - - /** - * This starts the node for tests and makes sure that the provided topology is acknowledged correctly. This method is not - * safe for production systems as it doesn't handle restarts and partially acknowledged histories - * @return {@link EpochReady#active} - */ - @VisibleForTesting - public AsyncResult unsafeStart() - { - topology.topologyService().onStartup(this); - ActiveEpochs epochs = topology.active(); - if (epochs.isEmpty()) - return AsyncResults.success(null); - - return epochs.epochReady(epochs.epoch()).active; - } - - public CommandStores commandStores() - { - return commandStores; - } - - public MessageSink messageSink() - { - return messageSink; - } - - public final DurableBefore durableBefore() - { - return durableBefore; - } - - public void addNewRangesToDurableBefore(Ranges ranges, long epoch) - { - durableBeforeLock.lock(); - try - { - TxnId from = TxnId.minForEpoch(epoch); - DurableBefore addDurableBefore = DurableBefore.create(ranges, from, from); - DurableBefore newDurableBefore = DurableBefore.merge(durableBefore, addDurableBefore); - // TODO (required): it is possible for this invariant to be breached if topologies are received out of order. - // We should not update min past the max known epoch. - Invariants.require(newDurableBefore.min.quorumBefore.compareTo(durableBefore.min.quorumBefore) >= 0, - "Previous durable before: %s, new: %s", durableBefore, newDurableBefore); - - minDurableBefore = DurableBefore.merge(minDurableBefore, addDurableBefore); - durableBefore = newDurableBefore; - } - finally - { - durableBeforeLock.unlock(); - } - } - - private Persister safeDurableBeforePersister(Persister wrap) - { - return new Persister<>() - { - @Override - public AsyncResult persist(DurableBefore addValue, DurableBefore newValue) - { - Invariants.require(addValue.maxEpoch() <= epoch()); - return wrap.persist(addValue, newValue); - } - @Override public DurableBefore load() { return wrap.load(); } - }; - } - - private void setPersistedDurableBefore(DurableBefore newDurableBefore) - { - durableBeforeLock.lock(); - try - { - // TODO (desired): do not re-merge any minDurableBefore that was already known when we created the update - durableBefore = DurableBefore.merge(newDurableBefore, minDurableBefore); - } - finally - { - durableBeforeLock.unlock(); - } - } - - public AsyncResult markDurable(Ranges ranges, TxnId majorityBefore, TxnId universalBefore) - { - return markDurable(DurableBefore.create(ranges, majorityBefore, universalBefore)); - } - - public AsyncResult markDurable(DurableBefore addDurableBefore) - { - return withEpochExact(addDurableBefore.maxEpoch(), (AsyncExecutor)null, () -> persistDurableBefore.mergeAndUpdate(addDurableBefore).chain()) - .beginAsResult(); - } - - @Override - public long epoch() - { - return topology().epoch(); - } - - // TODO (required): audit use of withEpochAtLeast vs withEpochExact - // TODO (expected): provide a deadline - public void withEpochAtLeast(EpochSupplier epochSupplier, @Nullable AsyncExecutor executor, BiConsumer callback) - { - if (epochSupplier == null) - callback.accept(null, null); - else - withEpochAtLeast(epochSupplier.epoch(), executor, callback); - } - - public void withEpochAtLeast(long epoch, @Nullable AsyncExecutor ifAsync, BiConsumer callback) - { - ActiveEpochs epochs = topology().active(); - if (epochs.hasAtLeastEpoch(epoch)) - { - callback.accept(null, null); - } - else - { - topology.await(epoch, ifAsync).begin(callback); - } - } - - public Object withEpochAtLeast(long epoch, @Nullable AsyncExecutor ifAsync, BiConsumer ifFailure, Runnable ifSuccess) - { - ActiveEpochs epochs = topology().active(); - if (epochs.hasAtLeastEpoch(epoch)) - { - ifSuccess.run(); - return ifSuccess; - } - else - { - return topology.await(epoch, ifAsync).begin((success, fail) -> { - if (fail != null) ifFailure.accept(null, fail); - else ifSuccess.run(); - }); - } - } - - public void withEpochExact(long epoch, @Nullable AsyncExecutor ifAsync, BiConsumer ifFailure, Function onFailure, Runnable ifSuccess) - { - ActiveEpochs epochs = topology().active(); - if (epoch < epochs.minEpoch()) - { - ifFailure.accept(null, onFailure.apply(new TopologyRetiredException(epoch, epochs.minEpoch()))); - } - else if (epochs.hasEpoch(epoch)) - { - ifSuccess.run(); - } - else - { - topology.await(epoch, ifAsync).begin((success, fail) -> { - if (fail != null) ifFailure.accept(null, onFailure.apply(fail)); - else ifSuccess.run(); - }); - } - } - - @Inline - public AsyncChain withEpochExact(long epoch, @Nullable AsyncExecutor executor, Supplier> supplier) - { - ActiveEpochs epochs = topology().active(); - if (epoch < epochs.minEpoch()) - { - return AsyncChains.failure(new TopologyRetiredException(epoch, epochs.minEpoch())); - } - else if (epochs.hasEpoch(epoch)) - { - return supplier.get(); - } - else - { - return topology.await(epoch, executor).flatMapOverride(supplier); - } - } - - @Inline - public AsyncChain withEpochAtLeast(long epoch, @Nullable AsyncExecutor executor, Supplier> supplier) - { - ActiveEpochs epochs = topology().active(); - if (epochs.hasAtLeastEpoch(epoch)) - { - return supplier.get(); - } - else - { - return topology.await(epoch, executor).flatMapOverride(supplier); - } - } - - public void withEpochAtLeast(long epoch, @Nullable AsyncExecutor ifAsync, BiConsumer ifFailure, Function onFailure, Runnable ifSuccess) - { - ActiveEpochs epochs = topology().active(); - if (epochs.hasAtLeastEpoch(epoch)) - { - ifSuccess.run(); - } - else - { - topology.await(epoch, ifAsync).begin((success, fail) -> { - if (fail != null) ifFailure.accept(null, onFailure.apply(fail)); - else ifSuccess.run(); - }); - } - } - - - public TopologyManager topology() - { - return topology; - } - - @Override - public AsyncExecutor someExecutor() - { - return commandStores.someExecutor(); - } - - @Override - public SequentialAsyncExecutor someSequentialExecutor() - { - return commandStores.someSequentialExecutor(); - } - - public void shutdown() - { - commandStores.shutdown(); - } - - public long uniqueNow() - { - return uniqueTime.uniqueNow(); - } - - @Override - public long uniqueNow(long greaterThan) - { - return uniqueTime.uniqueNow(greaterThan); - } - - @Override - public long uniqueStale(long greaterThan) - { - return uniqueTime.uniqueStale(greaterThan); - } - - @Override - public long now() - { - return time.now(); - } - - @Override - public long elapsed(TimeUnit timeUnit) - { - return time.elapsed(timeUnit); - } - - public void send(Topologies topologies, Request send) - { - SortedArrayList nodes = topologies.nodes(); - for (int i = 0 ; i < nodes.size() ; ++i) - { - Node.Id to = nodes.get(i); - if (!topologies.isFaulty(nodes.get(i))) - send(to, send); - } - } - - public void send(Topologies topologies, Function requestFactory) - { - SortedArrayList nodes = topologies.nodes(); - for (int i = 0 ; i < nodes.size() ; ++i) - { - Node.Id to = nodes.get(i); - if (!topologies.isFaulty(nodes.get(i))) - send(to, requestFactory.apply(to)); - } - } - - public void send(Topologies topologies, Request send, @Nonnull AsyncExecutor executor, Callback callback) - { - SortedArrayList nodes = topologies.nodes(); - for (int i = 0 ; i < nodes.size() ; ++i) - { - Node.Id to = nodes.get(i); - if (!topologies.isFaulty(nodes.get(i))) - messageSink.send(to, send, executor, callback); - } - } - - // TODO (required): callback must be invoked if for any reason send fails - public void send(Topologies topologies, Function requestFactory, @Nonnull AsyncExecutor executor, Callback callback) - { - SortedArrayList nodes = topologies.nodes(); - for (int i = 0 ; i < nodes.size() ; ++i) - { - Node.Id to = nodes.get(i); - if (!topologies.isFaulty(nodes.get(i))) - messageSink.send(to, requestFactory.apply(to), executor, callback); - } - } - - // send to a specific node - public Cancellable send(Id to, Request send, @Nonnull AsyncExecutor executor, Callback callback) - { - return messageSink.send(to, send, executor, callback); - } - - // send to a specific node - public void send(Id to, Request send) - { - messageSink.send(to, send); - } - - public void reply(Id replyingToNode, ReplyContext replyContext, Reply send, Throwable failure) - { - if (failure != null) - { - agent.onException(failure); - if (send != null) - agent().onException(new IllegalArgumentException(String.format("fail (%s) and send (%s) are both not null", failure, send))); - messageSink.replyWithUnknownFailure(replyingToNode, replyContext, failure); - return; - } - else if (send == null) - { - NullPointerException e = new NullPointerException(); - agent.onException(e); - throw e; - } - messageSink.reply(replyingToNode, replyContext, send); - } - - public TxnId nextTxnIdWithDefaultFlags(Seekables keys, Txn.Kind kind, Domain domain) - { - return nextTxnIdWithFlags(keys, kind, domain, Any, defaultMediumPath().bit()); - } - - public TxnId nextStaleTxnIdWithDefaultFlags(long minEpoch, long minHlc, Seekables keys, Txn.Kind kind, Domain domain) - { - return nextStaleTxnIdWithFlags(minEpoch, minHlc, keys, kind, domain, Any, defaultMediumPath().bit()); - } - - public TxnId nextTxnIdWithDefaultFlags(Seekables keys, Txn.Kind kind, Domain domain, Cardinality cardinality) - { - return nextTxnIdWithFlags(keys, kind, domain, cardinality, defaultMediumPath().bit()); - } - - private long epoch(long minEpoch, Seekables keys, Txn.Kind kind) - { - if (!kind.isSyncPoint()) - return Math.max(minEpoch, epoch()); - - return topology.active().maxEpoch(minEpoch, ActiveEpoch::all, keys); - } - - public TxnId nextTxnIdWithDefaultFlags(long minEpoch, long minHlc, Seekables keys, Txn.Kind kind, Domain domain, Cardinality cardinality) - { - long epoch = epoch(minEpoch, keys, kind); - return newTxnId(epoch, uniqueNow(minHlc), kind, domain, cardinality, defaultMediumPath().bit(), id); - } - - /** - * TODO (required): Make sure we cannot re-issue the same txnid on startup - * TODO (required): Don't use new epoch for TxnId until a quorum is ready to coordinate it - */ - public TxnId nextTxnIdWithFlags(Seekables keys, Txn.Kind kind, Domain domain, Cardinality cardinality, int flags) - { - return newTxnId(epoch(Long.MIN_VALUE, keys, kind), uniqueNow(), kind, domain, cardinality, flags, id); - } - - public TxnId nextStaleTxnIdWithFlags(long minEpoch, long minHlc, Seekables keys, Txn.Kind kind, Domain domain, Cardinality cardinality, int flags) - { - long epoch = epoch(minEpoch, keys, kind); - long hlc = uniqueStale(minHlc); - return newTxnId(epoch, hlc, kind, domain, cardinality, flags, id); - } - - private static TxnId newTxnId(long epoch, long now, Txn.Kind kind, Domain domain, Cardinality cardinality, int flags, Node.Id node) - { - Invariants.require(domain == Key || kind != Write, "Range writes not supported without forwarding uniqueHlc information to WaitingOn for direct dependencies"); - Invariants.require(domain == Range || !kind.isSyncPoint, "Key ExclusiveSyncPoint not supported without improvements to CommandsForKey for managing execution"); - TxnId txnId = new TxnId(epoch, now, flags, kind, domain, cardinality, node); - Invariants.require((txnId.lsb & (0xffff & ~TxnId.IDENTITY_FLAGS)) == 0); - return txnId; - } - - public TxnId nextTxnId(Txn txn) - { - return nextTxnId(0, 0, txn); - } - - public TxnId nextTxnId(long minEpoch, long minHlc, Txn txn) - { - Seekables keys = txn.keys(); - Txn.Kind kind = txn.kind(); - return nextTxnId(minEpoch, minHlc, keys, kind); - } - - public TxnId nextTxnId(@Nullable Timestamp min, Seekables keys, Txn.Kind kind) - { - return nextTxnId(min == null ? 0 : min.epoch(), min == null ? 0 : min.hlc(), keys, kind); - } - - public TxnId nextTxnId(long minEpoch, long minHlc, Seekables keys, Txn.Kind kind) - { - Domain domain = keys.domain(); - Cardinality cardinality = cardinality(domain, keys); - - if (!usePrivilegedCoordinator() || (kind != Read && kind != Write)) - return nextTxnIdWithDefaultFlags(minEpoch, minHlc, keys, kind, domain, cardinality); - - long epoch = epoch(minEpoch, keys, kind); - long hlc = uniqueNow(minHlc); - int flags = computeBestDefaultTxnIdFlags(keys, epoch); - TxnId txnId = new TxnId(epoch, hlc, flags, kind, domain, cardinality, id); - Invariants.require((txnId.lsb & (0xffff & ~TxnId.IDENTITY_FLAGS)) == 0); - return txnId; - } - - private int computeBestDefaultTxnIdFlags(Routables keys, long epoch) - { - ActiveEpochs epochs = topology().active(); - if (!epochs.hasEpoch(epoch) || !usePrivilegedCoordinator()) - return defaultMediumPath().bit(); - - TxnId.FastPath fastPath = ensurePermitted(epochs.selectFastPath(keys, epoch)); - return fastPath.bits | defaultMediumPath().bit(); - } - - public TxnId nextTxnId(Txn txn, TxnId.FastPath fastPath, TxnId.MediumPath mediumPath) - { - ActiveEpochs epochs = topology().active(); - Seekables keys = txn.keys(); - Txn.Kind kind = txn.kind(); - Domain domain = keys.domain(); - - long epoch = epoch(Long.MIN_VALUE, keys, kind); - long now = uniqueNow(); - fastPath = ensurePermitted(fastPath); - if (fastPath != Unoptimised && (!epochs.hasEpoch(epoch) || !epochs.supportsPrivilegedFastPath(keys, epoch))) - fastPath = Unoptimised; - - Cardinality cardinality = cardinality(domain, keys); - return newTxnId(epoch, now, kind, domain, cardinality, fastPath.bits | mediumPath.bit(), id); - } - - public AsyncChain coordinate(Txn txn) - { - TxnId txnId = nextTxnId(txn); - return coordinate(txnId, txn); - } - - public AsyncChain coordinate(TxnId txnId, Txn txn) - { - return coordinate(txnId, txn, txnId.epoch(), Long.MAX_VALUE); - } - - // TODO (required): plumb deadlineNanos in (perhaps on integration side, but maybe introduce some context we can pass through for the MessageSink) - public AsyncChain coordinate(TxnId txnId, Txn txn, long minEpoch, long deadlineNanos) - { - return withEpochExact(Math.max(txnId.epoch(), minEpoch), (AsyncExecutor) null, () -> initiateCoordination(txnId, txn)); - } - - private AsyncChain initiateCoordination(TxnId txnId, Txn txn) - { - if (txnId.kind() == Txn.Kind.EphemeralRead) - return CoordinateEphemeralRead.coordinate(this, txnId, txn); - else - return CoordinateTransaction.coordinate(this, txnId, txn); - } - - public FullRoute computeRoute(TxnId txnId, Routables keysOrRanges) throws TopologyException - { - return computeRoute(txnId.epoch(), keysOrRanges, topology.active()); - } - - public FullRoute computeRoute(long epoch, Routables keysOrRanges, ActiveEpochs active) throws TopologyException - { - Invariants.requireArgument(!keysOrRanges.isEmpty(), "Attempted to compute a route from empty keys or ranges"); - - RoutingKey homeKey = selectHomeKey(active.get(epoch), keysOrRanges); - - return keysOrRanges.toRoute(homeKey); - } - - private RoutingKey selectHomeKey(ActiveEpoch e, Routables keysOrRanges) - { - Ranges owned = e.local().ranges(); - int i = (int)keysOrRanges.findNextIntersection(0, owned, 0); - if (i >= 0) - return keysOrRanges.get(i).someIntersectingRoutingKey(owned); - - return keysOrRanges.get(random.nextInt(keysOrRanges.size())).someIntersectingRoutingKey(null); - } - - public AsyncChain recover(TxnId txnId, InvalidIf invalidIf, FullRoute route, LatentStoreSelector reportTo) - { - SequentialAsyncExecutor executor = someSequentialExecutor(); - return withEpochExact(txnId.epoch(), executor, () -> new AsyncChains.Head<>() - { - @Override - protected Cancellable start(BiConsumer callback) - { - PrepareRecovery.recover(Node.this, executor, txnId, invalidIf, route, null, reportTo, callback); - return null; - } - }); - } - - public void receive(Request request, Id from, ReplyContext replyContext) - { - long waitForEpoch = request.waitForEpoch(); - withEpochAtLeast(waitForEpoch, null, agent, () -> { - try - { - request.process(this, from, replyContext); - } - catch (Throwable t) - { - reply(from, replyContext, null, t); - } - }); - } - - public CoordinationAdapter coordinationAdapter(TxnId txnId, Kind kind) - { - return coordinationAdapters.get(txnId, kind); - } - - public AsyncChain updateMinHlc(long minHlc) - { - // TODO (required): command stores that are not ready due to bootstrap need to refresh their min HLC on bootstrap completion - StoreSelector selector = snapshot -> Stream.of(snapshot.shards).map(sh -> sh.store).iterator(); - return commandStores().mapReduce(selector, new MapReduceCommandStores<>(RoutingKeys.EMPTY) - { - @Override public Void reduce(Void o1, Void o2) { return null; } - @Override public TxnId primaryTxnId() { return null; } - @Override public String reason() { return "Update Min HLC"; } - @Override protected Void applyInternal(SafeCommandStore safeStore) - { - safeStore.commandStore().updateMinHlc(minHlc); - return null; - } - }); - } - - public Scheduler scheduler() - { - return scheduler; - } - - public Agent agent() - { - return agent; - } - - public RemoteListeners remoteListeners() - { - return listeners; - } - - @Override - public Timeouts timeouts() - { - return timeouts; - } - - @Override - public Id id() - { - return id; - } - - @Override - public String toString() - { - return "Node{" + id + '}'; - } - - @VisibleForTesting - public CommandStore unsafeForKey(RoutingKey key) - { - return commandStores.unsafeForKey(key); - } - - public CommandStore unsafeByIndex(int index) - { - return commandStores.current.shards[index].store; - } - - public TimeService time() - { - return time; - } - - public final long currentStamp() - { - return stamp; - } - - public long nextCoordinationId() - { - long startedAtNanos = time.elapsed(NANOSECONDS); - long nextId = nextCoordinationId.get(); - if (startedAtNanos >= nextId && nextCoordinationId.compareAndSet(nextId, startedAtNanos)) - return startedAtNanos; - return nextCoordinationId.incrementAndGet(); - } - - public void register(Coordination coordination) - { - coordinations.register(coordination); - } - - public void unregister(Coordination coordination) - { - coordinations.unregister(coordination); - } - - public Coordinations coordinations() - { - return coordinations; - } - - public boolean isCoordinatingWithBallot(TxnId txnId, Ballot ballot) - { - long mostRecent = coordinations.mostRecent(txnId, COORDINATES_STATE_MACHINE, ballot); - if (mostRecent < 0) - return false; - long ageNanos = Math.max(recentElapsed(NANOSECONDS) - mostRecent, 0); - return !agent.isSlowCoordinator(ageNanos, NANOSECONDS, txnId, 1); - } - - public void updateStamp() - { - stampUpdater.incrementAndGet(this); - } - - @Override - public boolean isReplaying() - { - return replaying; - } - - public void unsafeSetReplaying(boolean replaying) - { - this.replaying = replaying; - if (replaying) CommandsForKey.disableLinearizabilityViolationsReporting(); - else CommandsForKey.enableLinearizabilityViolationsReporting(); - } -} \ No newline at end of file diff --git a/accord-core/src/main/java/accord/local/OverlappingCommandStoresException.java b/accord-core/src/main/java/accord/local/OverlappingCommandStoresException.java deleted file mode 100644 index f1e358d655e2..000000000000 --- a/accord-core/src/main/java/accord/local/OverlappingCommandStoresException.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package accord.local; - -import accord.utils.Rethrowable; - -public class OverlappingCommandStoresException extends RuntimeException implements Rethrowable -{ - public OverlappingCommandStoresException() - { - } - - private OverlappingCommandStoresException(Throwable cause) - { - super(cause); - } - - @Override - public OverlappingCommandStoresException rethrowable() - { - return new OverlappingCommandStoresException(this); - } -} diff --git a/accord-core/src/main/java/accord/local/SafeCommandStore.java b/accord-core/src/main/java/accord/local/SafeCommandStore.java deleted file mode 100644 index 8d0b06f6b7b5..000000000000 --- a/accord-core/src/main/java/accord/local/SafeCommandStore.java +++ /dev/null @@ -1,676 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package accord.local; - -import java.util.ArrayList; -import java.util.List; -import java.util.NavigableMap; -import javax.annotation.Nonnull; -import javax.annotation.Nullable; - -import accord.api.Agent; -import accord.api.DataStore; -import accord.api.LocalListeners; -import accord.api.ProgressLog; -import accord.api.RoutingKey; -import accord.local.CommandStores.RangesForEpoch; -import accord.local.CommandStores.RangesForEpochSupplier; -import accord.local.RedundantBefore.RedundantBeforeSupplier; -import accord.local.cfk.CommandsForKey; -import accord.local.cfk.SafeCommandsForKey; -import accord.local.cfk.UpdateUnmanagedMode; -import accord.primitives.AbstractUnseekableKeys; -import accord.primitives.KeyDeps; -import accord.primitives.Participants; -import accord.primitives.RangeDeps; -import accord.primitives.Ranges; -import accord.primitives.RoutingKeys; -import accord.primitives.SaveStatus; -import accord.primitives.Status; -import accord.primitives.Timestamp; -import accord.primitives.TxnId; -import accord.primitives.Unseekables; -import accord.utils.Invariants; -import accord.utils.Reduce; -import accord.utils.LargeBitSet; -import accord.utils.SortedList; -import accord.utils.async.AsyncChain; -import accord.utils.async.AsyncChains; - -import static accord.local.LoadKeys.INCR; -import static accord.local.LoadKeys.NONE; -import static accord.local.LoadKeysFor.WRITE; -import static accord.local.RedundantStatus.Property.LOCALLY_APPLIED; -import static accord.local.RedundantStatus.SomeStatus.LOCALLY_WITNESSED_ONLY; -import static accord.local.RedundantStatus.Property.LOCALLY_REDUNDANT; -import static accord.local.RedundantStatus.Property.SHARD_APPLIED; -import static accord.local.cfk.UpdateUnmanagedMode.REGISTER; -import static accord.primitives.Known.KnownRoute.MaybeRoute; -import static accord.primitives.Routable.Domain.Range; -import static accord.primitives.Routables.Slice.Minimal; -import static accord.primitives.SaveStatus.Applied; -import static accord.primitives.SaveStatus.Committed; -import static accord.primitives.SaveStatus.TruncatedApply; -import static accord.primitives.SaveStatus.Uninitialised; -import static accord.primitives.Timestamp.Flag.SHARD_BOUND; -import static accord.primitives.Txn.Kind.VisibilitySyncPoint; -import static accord.utils.Invariants.illegalArgument; -import static accord.utils.Invariants.illegalState; - -/** - * A CommandStore with exclusive access; a reference to this should not be retained outside of the scope of the method - * that it is passed to. For the duration of the method invocation only, the methods on this interface are safe to invoke. - * - * Method implementations may therefore be single threaded, without volatile access or other concurrency control - */ -public abstract class SafeCommandStore implements RangesForEpochSupplier, RedundantBeforeSupplier, CommandSummaries -{ - private static final int MAX_REENTRANCY = 50; - private int reentrancyCounter; - public boolean tryRecurse() - { - if (reentrancyCounter == MAX_REENTRANCY) - return false; - ++reentrancyCounter; - return true; - } - public void unrecurse() - { - --reentrancyCounter; - Invariants.require(reentrancyCounter >= 0); - } - - public final boolean refusesAnyOf(Participants participants) - { - Ranges refuses = commandStore().refuses; - return refuses != null && participants.intersects(refuses); - } - - public final boolean refusesAllOwnedOf(Participants participants) - { - Ranges refuses = commandStore().refuses; - if (refuses == null) - return false; - - // TODO (required): memoize this, and expose it as a standard method as we want it elsewhere - Ranges notRetired = redundantBefore().removeLocallyRetired(ranges().all()); - return refuses.containsAll(participants.slice(notRetired, Minimal)); - } - - /** - * If the transaction exists (with some associated data) in the CommandStore, return it. Otherwise return null. - * - * This is useful for operations that do not retain a route, but do expect to operate on existing local state; - * this guards against recreating a previously truncated command when we do not otherwise have enough information - * to prevent it. - */ - public @Nullable SafeCommand ifInitialised(TxnId txnId) - { - SafeCommand safeCommand = getInternal(txnId); - if (safeCommand == null) - return null; - Command command = safeCommand.current(); - if (command.saveStatus().isUninitialised()) - return null; - return maybeCleanup(safeCommand); - } - - // decidedExecuteAt == null if not yet PreCommitted - - /** - * Retrieve a SafeCommand. If it is initialised, optionally use its present contents to determine if it should be - * truncated, and apply the truncation before returning the command. - * This behaviour may be overridden by implementations if they know any truncation would already have been applied. - * - * If it is not initialised, use the provided parameters to determine if the record may have been expunged; - * if not, create it. - * - * We do not distinguish between participants, home keys, and non-participating home keys for now, even though - * these fundamentally have different implications. Logically, we may erase a home shard's record as soon as - * the transaction has been made durable at a majority of replicas of every shard, and state for any participating - * keys may be erased as soon as their non-faulty peers have recorded the outcome. - * - * However if in some cases we don't know which commands are home keys or participants we need to wait to erase - * a transaction until both of these criteria are met for every key. - * - * TODO (desired): Introduce static types that permit us to propagate this information safely. - */ - public SafeCommand get(TxnId txnId, StoreParticipants participants) - { - SafeCommand safeCommand = getInternal(txnId); - if (safeCommand == null) - throw notFound(txnId); - - return maybeCleanup(safeCommand, participants); - } - - protected SafeCommand get(TxnId txnId) - { - SafeCommand safeCommand = getInternal(txnId); - if (safeCommand == null) - throw notFound(txnId); - - return maybeCleanup(safeCommand); - } - - public SafeCommand unsafeGet(TxnId txnId) - { - return get(txnId); - } - - public SafeCommand unsafeGetNoCleanup(TxnId txnId) - { - return getInternal(txnId); - } - - private RuntimeException notFound(TxnId txnId) - { - if (context().txnIds().contains(txnId)) throw illegalState("%s was specified in %s but was not returned by getInternal(key)", txnId, context().txnIds()); - else throw illegalArgument("%s was not specified in %s", txnId, context().txnIds()); - } - - protected SafeCommand maybeCleanup(SafeCommand safeCommand) - { - Command command = safeCommand.current(); - Commands.maybeCleanup(this, safeCommand, command, command.participants()); - return safeCommand; - } - - protected SafeCommand maybeCleanup(SafeCommand safeCommand, @Nonnull StoreParticipants supplemental) - { - Command command = safeCommand.current(); - StoreParticipants participants = command.participants().supplementOrMerge(command.saveStatus(), supplemental); - Commands.maybeCleanup(this, safeCommand, command, participants); - return safeCommand; - } - - /** - * If the transaction is in memory, return it (and make it visible to future invocations of {@code command}, {@code ifPresent} etc). - * Otherwise return null. - * - * This permits efficient operation when a transaction involved in processing another transaction happens to be in memory. - */ - public SafeCommand ifLoadedAndInitialised(TxnId txnId) - { - SafeCommand safeCommand = getInternal(txnId); - if (safeCommand == null) - { - safeCommand = ifLoadedInternal(txnId); - if (safeCommand == null) - return null; - } - - if (safeCommand.isUnset() || safeCommand.current().saveStatus() == Uninitialised) - return null; - - return maybeCleanup(safeCommand); - } - - protected SafeCommandsForKey maybeCleanup(SafeCommandsForKey safeCfk) - { - RedundantBefore.Bounds bounds = redundantBefore().get(safeCfk.key().toUnseekable()); - if (bounds != null) - safeCfk.updateRedundantBefore(this, bounds); - return safeCfk; - } - - /** - * If the transaction is in memory, return it (and make it visible to future invocations of {@code command}, {@code ifPresent} etc). - * Otherwise return null. - * - * This permits efficient operation when a transaction involved in processing another transaction happens to be in memory. - */ - public final SafeCommandsForKey ifLoadedAndInitialised(RoutingKey key) - { - SafeCommandsForKey safeCfk = getInternal(key); - if (safeCfk != null) - return safeCfk; - - safeCfk = ifLoadedInternal(key); - if (safeCfk == null) - return null; - return maybeCleanup(safeCfk); - } - - public SafeCommandsForKey get(RoutingKey key) - { - SafeCommandsForKey safeCfk = getInternal(key); - if (safeCfk != null) - return maybeCleanup(safeCfk); - - if (context().loadKeys() != NONE && context().keys().contains(key)) throw illegalState("%s was specified in %s but was not returned by getInternal(key)", key, context().keys()); - else throw illegalArgument("%s was not specified in %s", key, context()); - } - - /** Get anything already referenced (should include anything in PreLoadContext). If returned, should be initialised. */ - protected abstract SafeCommand getInternal(TxnId txnId); - /** Get if available */ - protected abstract SafeCommand ifLoadedInternal(TxnId txnId); - /** Get anything already referenced (should include anything in PreLoadContext) */ - protected abstract SafeCommandsForKey getInternal(RoutingKey key); - /** Get if available */ - protected abstract SafeCommandsForKey ifLoadedInternal(RoutingKey key); - - public final boolean canExecuteWith(PreLoadContext context) { return canExecute(context) == context; } - - /** - * Attempt to ready the provided PreLoadContext; if this can only be achieved partially, a new PreLoadContext - * will be returned containing the readily available data. If nothing is available, null will be returned. - */ - public abstract @Nullable PreLoadContext canExecute(PreLoadContext context); - - /** - * The current PreLoadContext, excluding any upgrade. - */ - public abstract PreLoadContext context(); - - protected void update(Command prev, Command updated, boolean force) - { - updateMaxConflicts(prev, updated, force); - updateCommandsForKey(prev, updated, force); - updateExclusiveSyncPoint(prev, updated, force); - } - - public void updateExclusiveSyncPoint(Command prev, Command updated, boolean force) - { - if (!updated.txnId().isSyncPoint() || updated.txnId().domain() != Range) return; - if (updated.route() == null) return; - - List listeners = commandStore().syncPointListeners; - if (listeners != null) - { - for (SyncPointListener listener : listeners) - listener.update(this, updated); - } - - SaveStatus oldSaveStatus = prev == null ? SaveStatus.Uninitialised : prev.saveStatus(); - SaveStatus newSaveStatus = updated.saveStatus(); - - if (newSaveStatus.known.isDefinitionKnown() && (force || !oldSaveStatus.known.isDefinitionKnown())) - { - Ranges ranges = updated.participants().touches().toRanges(); - commandStore().markExclusiveSyncPoint(this, updated.txnId(), ranges); - } - - if (newSaveStatus.compareTo(Committed) >= 0 && newSaveStatus.compareTo(TruncatedApply) <= 0 && (force || oldSaveStatus.compareTo(Committed) < 0)) - { - Ranges ranges = updated.participants().owns().toRanges(); - commandStore().markExclusiveSyncPointDecided(this, updated.txnId(), ranges); - } - - if (newSaveStatus == Applied && (force || oldSaveStatus != Applied)) - { - Ranges ranges = updated.participants().touches().toRanges(); - TxnId txnIdWithFlags = (TxnId)updated.executeAt(); - commandStore().markExclusiveSyncPointLocallyApplied(this, txnIdWithFlags, ranges); - } - - if (updated.partialDeps() != null) - { - RedundantBefore addRedundantBefore = RedundantBefore.EMPTY; - RangeDeps deps = updated.partialDeps().rangeDeps; - for (int i = 0 ; i < deps.txnIdCount() ; ++i) - { - TxnId txnId = deps.txnIdWithFlags(i); - if (txnId.is(SHARD_BOUND)) - { - Ranges ranges = deps.ranges(txnId).slice(ranges().all(), Minimal); - addRedundantBefore = RedundantBefore.merge(addRedundantBefore, RedundantBefore.create(ranges, txnId, LOCALLY_WITNESSED_ONLY)); - } - } - // TODO (expected): we should be able to use unsafeUpsertRedundantBefore here as the implementation will replay deps - // BUT the implementation may not replay all versions of the partialDeps we report here. - // This is likely to still be fine, as we don't imply anything for GC, but we won't do it for the moment. - // We might instead prefer to report these deps only once we are certain they won't change (i.e. when Stable), - // BUT in this case we cannot update waitingOnSync until then either (also probably fine) - if (addRedundantBefore != RedundantBefore.EMPTY) - upsertRedundantBefore(addRedundantBefore); - } - } - - public void updateMaxConflicts(Command prev, Command updated, boolean force) - { - SaveStatus oldSaveStatus = prev == null ? SaveStatus.Uninitialised : prev.saveStatus(); - SaveStatus newSaveStatus = updated.saveStatus(); - if (newSaveStatus.status.equals(oldSaveStatus.status) && oldSaveStatus.known.definition().isKnown() && !force) - return; - - TxnId txnId = updated.txnId(); - if (!txnId.isVisible()) - return; - - commandStore().updateMaxConflicts(prev, updated, force); - } - - /** - * Methods that implementors can use to capture changes to auxiliary collections: - */ - - public abstract void upsertRedundantBefore(RedundantBefore addRedundantBefore); - - protected void unsafeSetRedundantBefore(RedundantBefore newRedundantBefore) - { - commandStore().unsafeSetRedundantBefore(newRedundantBefore); - } - - protected void unsafeUpsertRedundantBefore(RedundantBefore addRedundantBefore) - { - commandStore().unsafeUpsertRedundantBefore(addRedundantBefore); - commandStore().updatedRedundantBefore(this, addRedundantBefore); - } - - public void setBootstrapBeganAt(NavigableMap newBootstrapBeganAt) - { - commandStore().unsafeSetBootstrapBeganAt(newBootstrapBeganAt); - } - - public void setSafeToRead(NavigableMap newSafeToRead) - { - commandStore().unsafeSetSafeToRead(newSafeToRead); - } - - public void setPermanentlyUnsafeToRead(Ranges newPermanentlyUnsafeToRead) - { - commandStore().unsafeSetPermanentlyUnsafeToRead(newPermanentlyUnsafeToRead); - } - - public void setRangesForEpoch(RangesForEpoch rangesForEpoch) - { - commandStore().unsafeSetRangesForEpoch(rangesForEpoch); - } - - public void updateCommandsForKey(Command prev, Command next, boolean force) - { - if (!CommandsForKey.needsUpdate(this, prev, next) && !force) - return; - - TxnId txnId = next.txnId(); - if (CommandsForKey.manages(txnId)) updateManagedCommandsForKey(this, prev, next, force); - if (!CommandsForKey.managesExecution(txnId) && next.hasBeen(Status.Stable) && !next.hasBeen(Status.Truncated) && (force || !prev.hasBeen(Status.Stable))) - updateUnmanagedCommandsForKey(this, next, REGISTER); - // TODO (expected): register deps during Accept phase to more quickly sync epochs -// else if (txnId.is(Range) && next.known().deps.hasProposedOrDecidedDeps()) -// updateUnmanagedCommandsForKey(this, next, REGISTER_DEPS_ONLY); - } - - abstract protected void persistFieldUpdates(); - - private static void updateManagedCommandsForKey(SafeCommandStore safeStore, Command prev, Command next, boolean forceNotify) - { - StoreParticipants participants = next.participants().supplement(prev.participants()); - Participants update = next.hasBeen(Status.Committed) ? participants.hasTouched() : participants.stillTouches(); - if (update.isEmpty()) - return; - - // TODO (expected): we don't want to insert any dependencies for those we only touch; we just need to record them as decided/applied for execution - PreLoadContext context = PreLoadContext.contextFor(next.txnId(), update, INCR, WRITE, "Update CommandsForKey"); - PreLoadContext execute = safeStore.canExecute(context); - if (execute != null) - { - updateManagedCommandsForKey(safeStore, execute.keys(), next.txnId(), forceNotify); - } - if (execute != context) - { - if (execute != null) - context = PreLoadContext.contextFor(next.txnId(), update.without(execute.keys()), INCR, WRITE, "Update CommandsForKey"); - - Invariants.require(!context.keys().isEmpty()); - safeStore = safeStore; // prevent accidental usage inside lambda - safeStore.commandStore().execute(context, safeStore0 -> { - PreLoadContext ctx = safeStore0.context(); - TxnId txnId = ctx.primaryTxnId(); - Unseekables keys = ctx.keys(); - updateManagedCommandsForKey(safeStore0, keys, txnId, forceNotify); - }, safeStore.commandStore().agent); - } - } - - private static void updateManagedCommandsForKey(SafeCommandStore safeStore, Unseekables update, TxnId txnId, boolean forceNotify) - { - // TODO (expected): avoid reentrancy / recursion - SafeCommand safeCommand = safeStore.get(txnId); - for (RoutingKey key : (AbstractUnseekableKeys)update) - { - // we use callback and re-fetch current to guard against reentrancy causing - // us to interact with "future" or stale information (respectively) - safeStore.get(key).callback(safeStore, safeCommand.current(), forceNotify); - } - } - - private static void updateUnmanagedCommandsForKey(SafeCommandStore safeStore, Command next, UpdateUnmanagedMode mode) - { - TxnId txnId = next.txnId(); - RoutingKeys keys; - - if (!txnId.isSyncPoint()) keys = next.asCommitted().waitingOn().keys; - else - { - Command.WaitingOn waitingOn = next.asCommitted().waitingOn; - RedundantBefore redundantBefore = safeStore.redundantBefore(); - KeyDeps deps = next.partialDeps().keyDeps; - keys = deps.keys(); - LargeBitSet select = new LargeBitSet(keys.size()); - for (int i = 0 ; i < keys.size() ; ++i) - { - if (waitingOn.isWaitingOnKey(i)) - { - select.set(i); - continue; - } - - SortedList txnIdsForKey = deps.txnIdsForKeyIndex(i); - RoutingKey key = keys.get(i); - TxnId maxTxnId = txnIdsForKey.get(txnIdsForKey.size() - 1); - // TODO (desired): convert to O(n) merge - RedundantStatus status = redundantBefore.status(maxTxnId, null, key); - if (!status.all(SHARD_APPLIED, LOCALLY_APPLIED) || !status.all(LOCALLY_REDUNDANT)) // TODO (required): should be a new property for correct merge? - select.set(i); - } - if (select.getSetBitCount() != keys.size()) - { - RoutingKey[] array = new RoutingKey[select.getSetBitCount()]; - int count = 0; - for (int i = 0 ; i < keys.size() ; ++i) - { - if (select.get(i)) - array[count++] = keys.get(i); - } - keys = RoutingKeys.ofSortedUnique(array); - } - } - // TODO (required): use StoreParticipants.executes() - // TODO (required): consider how execution works for transactions that await future deps and where the command store inherits additional keys in execution epoch - PreLoadContext context = PreLoadContext.contextFor(txnId, keys, INCR, WRITE, "Update Unmanaged CommandsForKey"); - PreLoadContext execute = safeStore.canExecute(context); - // TODO (expected): execute immediately for any keys we already have loaded, and save only those we haven't for later - if (execute != null) - { - updateUnmanagedCommandsForKey(safeStore, execute.keys(), txnId, mode); - } - if (execute == context) - { - if (next.txnId().is(Range) && next.txnId().is(VisibilitySyncPoint)) - registerTransitiveRangeDeps(safeStore, txnId, next); - } - else - { - if (execute != null) - context = PreLoadContext.contextFor(txnId, keys.without(execute.keys()), INCR, WRITE, "Update Unmanaged CommandsForKey"); - - safeStore = safeStore; - CommandStore unsafeStore = safeStore.commandStore(); - AsyncChain submit = unsafeStore.chain(context, safeStore0 -> { updateUnmanagedCommandsForKey(safeStore0, safeStore0.context().keys() , txnId, mode); }); - if (next.txnId().is(Range)) - submit = submit.flatMap(success -> unsafeStore.chain(PreLoadContext.contextFor(txnId, "Register Transitive Dependencies"), safeStore0 -> { registerTransitiveRangeDeps(safeStore0, txnId, next); })); - submit.begin(safeStore.commandStore().agent); - } - } - - private static TxnId maxTxnId(KeyDeps keyDeps, RoutingKey key) - { - int i = keyDeps.keys().indexOf(key); - if (i < 0) - return TxnId.NONE; - SortedList txnIdsForKey = keyDeps.txnIdsForKeyIndex(i); - return txnIdsForKey.get(txnIdsForKey.size() - 1); - } - - private static void updateUnmanagedCommandsForKey(SafeCommandStore safeStore, Unseekables update, TxnId txnId, UpdateUnmanagedMode mode) - { - SafeCommand safeCommand = safeStore.get(txnId); - for (RoutingKey key : (AbstractUnseekableKeys)update) - { - safeStore.get(key).registerUnmanaged(safeStore, safeCommand, mode); - } - } - - private static void registerTransitiveRangeDeps(SafeCommandStore safeStore, TxnId syncId, Command syncCommand) - { - if (!syncId.is(VisibilitySyncPoint)) - return; - - CommandStore commandStore = safeStore.commandStore(); - Ranges touches = syncCommand.participants().touches().toRanges(); - Ranges waitingOn = commandStore.isWaitingOnVisibility(syncId, touches); - if (waitingOn.isEmpty()) - return; - - List> async = new ArrayList<>(); - RangeDeps rangeDeps = syncCommand.partialDeps().rangeDeps; - rangeDeps.forEachUniqueTxnId(waitingOn, null, (ignore, txnIdWithFlags) -> { - TxnId txnId = txnIdWithFlags.withoutNonIdentityFlags(); - PreLoadContext context = PreLoadContext.contextFor(txnId, "Register Transitive Range Deps"); - Ranges ranges = rangeDeps.ranges(txnId); - if (safeStore.canExecuteWith(context)) registerTransitive(safeStore, txnId, ranges); - else async.add(safeStore.commandStore().chain(context, safeStore0 -> { - registerTransitive(safeStore0, txnId, ranges); - })); - }); - - AsyncChains.chain(() -> commandStore.markingVisible(syncId, waitingOn)) - .flatMap(ignore -> AsyncChains.reduce(async, Reduce.toNull(), null)) - .begin((success, fail) -> { - if (fail == null) commandStore.execute((PreLoadContext.Empty)() -> "Mark Synced", safeStore0 -> commandStore.markVisible(safeStore0, syncId, waitingOn)); - else commandStore.execute((PreLoadContext.Empty)() -> "Unmark Syncing", safeStore0 -> commandStore.cancelMarkingVisible(syncId, waitingOn)); - }); - } - - private static void registerTransitive(SafeCommandStore safeStore, TxnId txnId, Ranges witnessedBy) - { - SafeCommand safeCommand = safeStore.unsafeGet(txnId); - if (safeCommand != null && safeCommand.current().known().route() != MaybeRoute) - return; - - RangesForEpoch rangesForEpoch = safeStore.ranges(); - // TODO (required): this is incompatible with rebootstrap - we need to use some additional condition - witnessedBy = witnessedBy.without(rangesForEpoch.coordinates(txnId)); // already coordinates, no need to replicate - if (witnessedBy.isEmpty()) - return; - - witnessedBy = witnessedBy.slice(rangesForEpoch.allSince(txnId.epoch()), Minimal); // never coordinated, no need to replicate for dependency or recovery calculations - if (witnessedBy.isEmpty()) - return; - - safeCommand.updateParticipants(safeStore, safeCommand.current().participants().supplement(null, witnessedBy)); - } - - public abstract CommandStore commandStore(); - public abstract DataStore dataStore(); - public abstract Agent agent(); - public abstract ProgressLog progressLog(); - public abstract NodeCommandStoreService node(); - public abstract RangesForEpoch ranges(); - - protected NavigableMap bootstrapBeganAt() - { - return commandStore().unsafeGetBootstrapBeganAt(); - } - - public NavigableMap safeToReadAt() - { - return commandStore().unsafeGetSafeToRead(); - } - - public RedundantBefore redundantBefore() - { - return commandStore().unsafeGetRedundantBefore(); - } - - public MaxDecidedRX maxDecidedRX() - { - return commandStore().unsafeGetMaxDecidedRX(); - } - - public DurableBefore durableBefore() - { - return commandStore().node.durableBefore(); - } - - public Ranges futureRanges(TxnId txnId) - { - return ranges().allBefore(txnId.epoch()); - } - - public Ranges coordinateRanges(TxnId txnId) - { - return ranges().allAt(txnId.epoch()); - } - - public Ranges ranges(TxnId txnId, long untilLocalEpoch) - { - return ranges().allBetween(txnId.epoch(), untilLocalEpoch); - } - - public final Ranges safeToReadAt(Timestamp at) - { - return safeToReadAt().lowerEntry(at).getValue(); - } - - public @Nonnull Ranges unsafeToReadAt(Timestamp at) - { - return ranges().allAt(at).without(safeToReadAt(at)); - } - - public void registerListener(SafeCommand listeningTo, SaveStatus await, TxnId waiting) - { - Invariants.require(listeningTo.current().saveStatus().compareTo(await) < 0); - Invariants.require(!CommandsForKey.managesExecution(listeningTo.txnId())); - commandStore().listeners.register(listeningTo.txnId(), await, waiting); - } - - public LocalListeners.Registered register(TxnId txnId, LocalListeners.ComplexListener listener) - { - return commandStore().listeners.register(txnId, listener); - } - - public void notifyListeners(SafeCommand safeCommand, Command prev) - { - commandStore().listeners.notify(this, safeCommand, prev); - } - - public void register(SyncPointListener syncPointListener) - { - commandStore().unsafeRegister(syncPointListener); - } - - public void unregister(SyncPointListener syncPointListener) - { - commandStore().unsafeUnregister(syncPointListener); - } -} diff --git a/accord-core/src/main/java/accord/topology/TopologyManager.java b/accord-core/src/main/java/accord/topology/TopologyManager.java deleted file mode 100644 index 282ae9ef3481..000000000000 --- a/accord-core/src/main/java/accord/topology/TopologyManager.java +++ /dev/null @@ -1,590 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package accord.topology; - -import java.util.IdentityHashMap; -import java.util.Map; -import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.Executor; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.function.Function; -import java.util.function.Supplier; -import javax.annotation.Nullable; - -import com.google.common.annotations.VisibleForTesting; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import accord.api.AsyncExecutor; -import accord.api.TopologyListener; -import accord.api.Timeouts; -import accord.api.TopologyService; -import accord.api.TopologySorter; -import accord.api.VisibleForImplementation; -import accord.local.Node; -import accord.local.Node.Id; -import accord.local.TimeService; -import accord.primitives.Ranges; -import accord.primitives.TxnId; -import accord.topology.TopologyCollector.BestFastPath; -import accord.topology.TopologyCollector.Simple; -import accord.topology.TopologyCollector.SupportsPrivilegedFastPath; -import accord.utils.Invariants; -import accord.utils.async.AsyncChain; -import accord.utils.async.AsyncChains; -import accord.utils.async.AsyncResult; -import accord.utils.async.AsyncResults; -import accord.utils.async.NestedAsyncResult; - -import static accord.primitives.AbstractRanges.UnionMode.MERGE_ADJACENT; -import static accord.primitives.Routables.Slice.Minimal; - -/** - * Manages topology state changes and update bookkeeping - * - * Each time the topology changes we need to: - * * confirm previous owners of ranges we replicate are aware of the new config - * * learn of any outstanding operations for ranges we replicate - * * clean up obsolete data - * - * Assumes a topology service that won't report epoch n without having n-1 etc also available - * - * TODO (desired, efficiency/clarity): make TopologyManager a Topologies and copy-on-write update to it, - * so we can always just take a reference for transactions instead of copying every time (and index into it by the txnId.epoch) - */ -public class TopologyManager -{ - private static final Logger logger = LoggerFactory.getLogger(TopologyManager.class); - private static final PendingEpoch SUCCESS; - - static - { - SUCCESS = new PendingEpoch(-1L, null); - SUCCESS.setActive(); - } - - final TopologySorter.Supplier sorter; - final Simple liveCollector, allCollector; - final BestFastPath bestFastPath; - final SupportsPrivilegedFastPath supportsPrivilegedFastPath; - final Node node; - final TopologyService topologyService; - final TimeService time; - final Timeouts timeouts; - private volatile ActiveEpochs active; - private final PendingEpochs pending; - private final CopyOnWriteArrayList listeners = new CopyOnWriteArrayList<>(); - - public TopologyManager(TopologySorter.Supplier sorter, Node node, TopologyService topologyService, TimeService time, Timeouts timeouts) - { - this.sorter = sorter; - this.liveCollector = new Simple(sorter, SelectShards.LIVE); - this.allCollector = new Simple(sorter, SelectShards.ALL); - this.bestFastPath = new BestFastPath(node.id()); - this.supportsPrivilegedFastPath = new SupportsPrivilegedFastPath(node.id()); - this.node = node; - this.time = time; - this.timeouts = timeouts; - this.topologyService = topologyService; - this.active = new ActiveEpochs(this, new ActiveEpoch[0], -1); - this.pending = new PendingEpochs(this); - } - - public void onReadyToCoordinate(Id node, long epoch) - { - synchronized (this) - { - if (epoch >= active.minEpoch()) - active.onReadyToCoordinate(node, epoch); - if (epoch > active.currentEpoch) - pending.remoteReadyToCoordinate(node, epoch); - } - for (TopologyListener listener : listeners) - listener.onRemoteReadyToCoordinate(node, epoch); - } - - public void onEpochClosed(Ranges ranges, long epoch) - { - onEpochClosed(ranges, epoch, null); - } - - public void onEpochClosed(Ranges ranges, TxnId txnId) - { - onEpochClosed(ranges, txnId.epoch(), txnId); - } - - private void onEpochClosed(Ranges ranges, long epoch, @Nullable TxnId txnId) - { - Topology topology = null; - synchronized (this) - { - ActiveEpoch e = active.ifExists(epoch); - if (txnId != null) - { - if (e != null) - { - ranges = ranges.without(e.addedRanges); - if (ranges.isEmpty()) - return; - } - e = active.ifExists(--epoch); - } - - if (e != null) - topology = e.all(); - - if (epoch > active.currentEpoch) - ranges = pending.closed(ranges, epoch); - ranges = active.closed(ranges, epoch); - } - if (!ranges.isEmpty()) - { - for (TopologyListener listener : listeners) - listener.onEpochClosed(ranges, epoch, topology); - } - } - - public void onEpochRetired(Ranges ranges, long epoch) - { - onEpochRetired(ranges, epoch, null); - } - - public void onEpochRetired(Ranges ranges, TxnId txnId) - { - onEpochRetired(ranges, txnId.epoch(), txnId); - } - - private void onEpochRetired(Ranges ranges, long epoch, @Nullable TxnId txnId) - { - Topology topology = null; - synchronized (this) - { - ActiveEpoch e = active.ifExists(epoch); - if (txnId != null) - { - if (e != null) - { - ranges = ranges.without(e.addedRanges); - if (ranges.isEmpty()) - return; - } - - // if we're retiring only ranges that are no longer live, we can retire the declaration epoch; otherwise we only retire the prior epoch - if (e == null || e.live.ranges.intersects(ranges)) - e = active.ifExists(--epoch); - } - - if (e != null) - topology = e.all; - - if (epoch > active.currentEpoch) - ranges = pending.retired(ranges, epoch); - ranges = active.retired(ranges, epoch); - } - if (!ranges.isEmpty()) - { - for (TopologyListener listener : listeners) - listener.onEpochRetired(ranges, epoch, topology); - } - } - - public synchronized void truncateTopologiesUntil(long epoch) - { - ActiveEpochs current = active; - Invariants.requireArgument(current.epoch() >= epoch, "Unable to truncate; epoch %d is > current epoch %d", epoch, current.epoch()); - - if (current.minEpoch() >= epoch) - return; - - int newLen = current.epochs.length - (int) (epoch - current.minEpoch()); - Invariants.require(current.epochs[newLen - 1].isQuorumReady(), "Epoch %d is not ready to coordinate", current.epochs[newLen - 1].epoch()); - - ActiveEpoch[] nextEpochs = new ActiveEpoch[newLen]; - System.arraycopy(current.epochs, 0, nextEpochs, 0, newLen); - active = new ActiveEpochs(this, nextEpochs, current.firstNonEmptyEpoch); - } - - public TopologySorter.Supplier sorter() - { - return sorter; - } - - public Topology current() - { - return active.current(); - } - - public Topology currentLocal() - { - return active.currentLocal(); - } - - public boolean isEmpty() - { - return active.isEmpty() && pending.isEmpty(); - } - - public long epoch() - { - return current().epoch; - } - - @VisibleForImplementation - public synchronized long pendingEpoch() - { - return pending.maxEpoch(); - } - - // TODO (desired): add tests for epoch GC and tracking - @VisibleForImplementation - public long firstNonEmpty() - { - return active.firstNonEmptyEpoch; - } - - public long minEpoch() - { - ActiveEpochs epochs = this.active; - return epochs.minEpoch(); - } - - // TODO (testing): test all of these methods when asking for epochs that have been cleaned up (and other code paths) - - public ActiveEpochs active() - { - return active; - } - - public void addListener(TopologyListener listener) - { - listeners.add(listener); - } - - public void removeListener(TopologyListener listener) - { - listeners.remove(listener); - } - - protected Executor executor() - { - return Runnable::run; - } - - public void reportTopology(Topology topology) - { - PendingEpoch e; - synchronized (this) - { - long epoch = topology.epoch; - // if active is empty, treat the earliest pending epoch as our low bound to avoid race conditions where we begin updating active but discover an earlier epoch - long currentEpoch = !active.isEmpty() ? active.currentEpoch : !pending.isEmpty() ? pending.atIndex(0).epoch - 1 : 0; - if (epoch <= currentEpoch) - { - logger.debug("Ignoring topology for epoch {} which is behind our latest epoch {}", epoch, currentEpoch); - return; - } - - e = pending.getOrCreate(epoch); - e.setTopology(topology); - } - - logger.debug("Epoch {} received", topology.epoch()); - for (TopologyListener listener : listeners) - listener.onReceived(topology); - - updateActive(); - } - - public static class RegainingEpochRange - { - public final long epoch; - public final Ranges ranges; - - public RegainingEpochRange(long epoch, Ranges ranges) - { - this.epoch = epoch; - this.ranges = ranges; - } - - public long epoch() - { - return epoch; - } - - public Ranges ranges() - { - return ranges; - } - } - - @Nullable - public RegainingEpochRange computeRegaining(Topology current, Topology next) - { - Map additions = Topology.computeNodeAdditions(current, next); - long greatestEpoch = -1; - Ranges ranges = Ranges.EMPTY; - - ActiveEpochs active = this.active; - for (Map.Entry entry : additions.entrySet()) - { - Ranges addingForNode = entry.getValue(); - for (ActiveEpoch e : active) - { - addingForNode = addingForNode.without(e.removedRanges).without(e.retired()); - if (addingForNode.isEmpty()) - break; - - Ranges existingForNode = e.all().rangesForNode(entry.getKey()); - Ranges regainingForNode = addingForNode.slice(existingForNode, Minimal); - if (!regainingForNode.isEmpty()) - { - greatestEpoch = Math.max(greatestEpoch, e.epoch()); - ranges = ranges.union(MERGE_ADJACENT, regainingForNode); - addingForNode = addingForNode.without(regainingForNode); - } - addingForNode = addingForNode.without(e.addedRanges); - } - } - - if (greatestEpoch != -1) - return new RegainingEpochRange(greatestEpoch, ranges); - - return null; - } - - private final AtomicBoolean updatingActive = new AtomicBoolean(); - private void updateActive() - { - if (!updatingActive.compareAndSet(false, true)) - return; - - try - { - while (true) - { - Topology topology; - PendingEpoch pending; - synchronized (this) - { - if (this.pending.isEmpty() || (!this.active.isEmpty() && this.pending.atIndex(0).epoch > 1 + current().epoch())) - return; - - pending = this.pending.atIndex(0); - topology = pending.topology(); - if (topology == null) - return; - } - - Supplier bootstrap = node.commandStores().updateTopology(node, topology); - AsyncResult.Settable whenSetup = new AsyncResults.SettableWithDescription<>("Publishing Active Epoch"); - EpochReady epochReady = new EpochReady(topology.epoch, - NestedAsyncResult.flatMap(whenSetup, ignore -> AsyncResults.success(null)), - NestedAsyncResult.flatMap(whenSetup, EpochReady::coordinate), - NestedAsyncResult.flatMap(whenSetup, EpochReady::data), - NestedAsyncResult.flatMap(whenSetup, EpochReady::reads)); - - if (!this.active.isEmpty()) - { - ActiveEpoch prev = this.active.epochs[0]; - Invariants.require(prev.epoch() == topology.epoch - 1); - epochReady = orderReporting(prev.epochReady(), epochReady); - } - - ActiveEpoch active = new ActiveEpoch(node.id(), topology, epochReady, sorter.get(topology), this.active.current().ranges); - - synchronized (this) - { - active.recordClosed(pending.closed); - active.recordRetired(pending.retired); - pending.ready.forEach(active::onReadyToCoordinate); - - ActiveEpochs prev = this.active; - ActiveEpoch[] next = new ActiveEpoch[prev.epochs.length + 1]; - System.arraycopy(prev.epochs, 0, next, 1, prev.epochs.length); - next[0] = active; - - if (!prev.isEmpty() && !prev.epochs[0].all.hardRemoved.containsAll(topology.hardRemoved)) - { - IdentityHashMap cache = new IdentityHashMap<>(); - for (int i = next.length - 1 ; i >= 0 ; --i) - { - ActiveEpoch e = next[i]; - Topology newGlobal = next[i].all.withHardRemoved(topology.hardRemoved, cache); - if (newGlobal != e.all) - { - next[i] = new ActiveEpoch(node.id(), newGlobal, e.shardQuorumReady, e.receivedNodeReady, e.quorumReadyTracker, - e.addedRanges, e.removedRanges, e.epochReady(), e.quorumReady(), e.closed(), e.retired()); - } - } - } - - this.active = new ActiveEpochs(this, next, prev.firstNonEmptyEpoch); - this.pending.removeFirst(topology.epoch); - } - - EpochReady innerReady = bootstrap(bootstrap); - whenSetup.setSuccess(innerReady); - - pending.setActive(); - listeners.forEach(listener -> listener.onActive(active)); - - long epoch = topology.epoch; - Node.Id self = node.id(); - innerReady.coordinate.invokeIfSuccess(() -> { - listeners.forEach(listener -> listener.onReadyToCoordinate(topology)); - onReadyToCoordinate(self, epoch); - }); - } - } - finally - { - updatingActive.set(false); - } - } - - @VisibleForTesting - protected EpochReady bootstrap(Supplier bootstrap) - { - return bootstrap.get(); - } - - private static EpochReady orderReporting(EpochReady previous, EpochReady next) - { - if (previous.epoch + 1 != next.epoch) - throw new IllegalArgumentException("Attempted to order epochs but they are not next to each other... previous=" + previous.epoch + ", next=" + next.epoch); - if (previous.coordinate.isDone() && previous.data.isDone() && previous.reads.isDone()) - return next; - - return new EpochReady(next.epoch, - next.active, - NestedAsyncResult.flatMap(previous.coordinate, ignore -> next.coordinate), - NestedAsyncResult.flatMap(previous.data, ignore -> next.data), - NestedAsyncResult.flatMap(previous.reads, ignore -> next.reads) - ); - } - - public AsyncChain await(long epoch, @Nullable AsyncExecutor ifAsync) - { - PendingEpoch pendingEpoch; - boolean fetch; - synchronized (this) - { - if (epoch <= active.currentEpoch) - return AsyncChains.success(null); - - pendingEpoch = pending.getOrCreate(epoch); - fetch = pendingEpoch.fetching == null; - } - - node.agent().systemEvents().onWaitingForEpoch(epoch); - AsyncChain result = pendingEpoch.whenActive().chainImmediatelyElse(ifAsync); - if (fetch) - { - while (true) - { - fetch(pendingEpoch); - --epoch; - synchronized (this) - { - if (epoch <= active.currentEpoch) - break; - - pendingEpoch = pending.getOrCreate(epoch); - if (pendingEpoch.fetching != null) - break; - } - } - } - return result; - } - - private void fetch(PendingEpoch pending) - { - synchronized (this) - { - if (pending.topology() != null || pending.epoch < active.currentEpoch) - return; - - if (pending.fetching != null && !pending.fetching.isDone()) - return; - - pending.fetching = topologyService.fetchTopologyForEpoch(pending.epoch); - } - - pending.fetching.invoke((success, fail) -> { - if (fail == null) reportTopology(success); - else if (active.currentEpoch < pending.epoch && pending.topology() == null) - { - // TODO (expected): special casing of TopologyRetiredException? - logger.warn("Failed to fetch epoch {}. Retrying.", pending.epoch, fail); - node.agent().onException(fail, "Fetch epoch " + pending.epoch); - long retryInMicros = node.agent().retryTopologyDelay(node, 1 + ++pending.fetchAttempts, TimeUnit.MICROSECONDS); - node.scheduler().once(() -> fetch(pending), retryInMicros, TimeUnit.MICROSECONDS); - } - }); - } - - @VisibleForImplementation - public AsyncResult epochReady(long epoch, Function> get) - { - // synchronized for state.ready visibility - synchronized (this) - { - if (active.hasAtLeastEpoch(epoch)) - { - if (!active.hasEpoch(epoch)) - return get.apply(EpochReady.done(epoch)); - return get.apply(active.getKnown(epoch).epochReady()); - } - - return pending.getOrCreate(epoch).whenActive().get().flatMap(r -> get.apply(active.epochReady(epoch))); - } - } - - @VisibleForTesting - ActiveEpoch unsafeGetActiveEpoch(long epoch) - { - return active.getKnown(epoch); - } - - @VisibleForTesting - public void unsafeSetActive(ActiveEpochs newActive) - { - active = newActive; - } - - @VisibleForTesting - public Ranges unsafeQuorumReady(long epoch) - { - ActiveEpoch e = active.ifExists(epoch); - return e == null ? Ranges.EMPTY : e.quorumReady(); - } - - public boolean unsafeIsQuorumReady(long epoch) - { - ActiveEpoch e = active.ifExists(epoch); - return e != null && e.isQuorumReady(); - } - - public TopologyService topologyService() - { - return topologyService; - } -} diff --git a/accord-core/src/test/java/accord/impl/basic/DelayedCommandStores.java b/accord-core/src/test/java/accord/impl/basic/DelayedCommandStores.java deleted file mode 100644 index a9b47c36ed66..000000000000 --- a/accord-core/src/test/java/accord/impl/basic/DelayedCommandStores.java +++ /dev/null @@ -1,502 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package accord.impl.basic; - -import java.util.*; -import java.util.concurrent.Callable; -import java.util.function.BiConsumer; -import java.util.function.Consumer; -import java.util.function.Function; - -import com.google.common.collect.ImmutableSortedMap; - -import accord.api.Agent; -import accord.api.DataStore; -import accord.api.Journal; -import accord.api.LocalListeners; -import accord.api.ProgressLog; -import accord.api.RoutingKey; -import accord.impl.InMemoryCommandStore; -import accord.impl.InMemoryCommandStores; -import accord.impl.InMemorySafeCommand; -import accord.impl.InMemorySafeCommandsForKey; -import accord.impl.PrefixedIntHashKey; -import accord.impl.basic.TaskExecutorService.Task; -import accord.local.Command; -import accord.local.CommandStore; -import accord.local.CommandStores; -import accord.local.NodeCommandStoreService; -import accord.local.PreLoadContext; -import accord.local.RedundantBefore; -import accord.local.SafeCommandStore; -import accord.local.ShardDistributor; -import accord.local.cfk.CommandsForKey; -import accord.primitives.Range; -import accord.primitives.Ranges; -import accord.primitives.RoutableKey; -import accord.primitives.Timestamp; -import accord.primitives.TxnId; -import accord.topology.Topology; -import accord.utils.Invariants; -import accord.utils.RandomSource; -import accord.utils.async.AsyncChain; -import accord.utils.async.AsyncChains; -import accord.utils.async.Cancellable; - -import static accord.api.Journal.CommandUpdate; -import static accord.utils.Invariants.Paranoia.LINEAR; -import static accord.utils.Invariants.ParanoiaCostFactor.HIGH; - -public class DelayedCommandStores extends InMemoryCommandStores.SingleThread -{ - private DelayedCommandStores(NodeCommandStoreService time, Agent agent, DataStore store, RandomSource random, ShardDistributor shardDistributor, ProgressLog.Factory progressLogFactory, LocalListeners.Factory listenersFactory, SimulatedDelayedExecutorService executorService, CacheLoading isLoadedCheck, Journal journal) - { - super(time, agent, store, random, journal, shardDistributor, progressLogFactory, listenersFactory, DelayedCommandStore.factory(executorService, isLoadedCheck)); - } - - public static CommandStores.Factory factory(PendingQueue pending, CacheLoading isLoadedCheck) - { - return (time, agent, store, random, journal, shardDistributor, progressLogFactory, listenersFactory) -> - new DelayedCommandStores(time, agent, store, random, shardDistributor, progressLogFactory, listenersFactory, new SimulatedDelayedExecutorService(pending, agent, time.id()), isLoadedCheck, journal); - } - - public void validateShardStateForTesting(Journal.TopologyUpdate lastUpdate) - { - PreviouslyOwned previouslyOwned = lastUpdate.previouslyOwned; - ShardHolder[] shards = new ShardHolder[lastUpdate.commandStores.size()]; - int i = 0; - for (Map.Entry e : lastUpdate.commandStores.entrySet()) - { - Snapshot current = current(); - RangesForEpoch ranges = e.getValue(); - CommandStore commandStore = null; - for (ShardHolder shard : current) - { - if (shard.ranges().equals(ranges)) - { - Invariants.require(commandStore == null); - commandStore = shard.store; - } - } - Invariants.nonNull(commandStore, "Each set of ranges should have a corresponding command store, but %d did not:(%s)", - ranges, Arrays.toString(shards)) - .restore(); - - ShardHolder shard = new ShardHolder(commandStore, ranges, previouslyOwned.regains(ranges.all())); - shards[i++] = shard; - } - Arrays.sort(shards, Comparator.comparingInt(shard -> shard.store.id())); - - loadSnapshot(new Snapshot(shards, lastUpdate.global.forNode(nodeId()).trim(), lastUpdate.global, lastUpdate.previouslyOwned)); - } - - protected void loadSnapshot(Snapshot nextSnapshot) - { - Snapshot current = current(); - // These checks are only applicable to delayed command stores. - for (ShardHolder shard : current) - { - CommandStore prev = current.byId(shard.store.id()); - CommandStore next = nextSnapshot.byId(shard.store.id()); - { - RedundantBefore orig = prev.unsafeGetRedundantBefore(); - RedundantBefore loaded = next.unsafeGetRedundantBefore(); - Invariants.require(orig.equals(loaded), "%s should equal %s", loaded, orig); - } - - { - NavigableMap orig = prev.unsafeGetBootstrapBeganAt(); - NavigableMap loaded = next.unsafeGetBootstrapBeganAt(); - Invariants.require(orig.equals(loaded), "%s should equal %s", loaded, orig); - } - - { - NavigableMap orig = prev.unsafeGetSafeToRead(); - NavigableMap loaded = next.unsafeGetSafeToRead(); - Invariants.require(orig.equals(loaded), "%s should equal %s", loaded, orig); - } - - { - RangesForEpoch orig = prev.unsafeGetRangesForEpoch(); - RangesForEpoch loaded = next.unsafeGetRangesForEpoch(); - Invariants.require(orig.equals(loaded), "%s should equal %s", loaded, orig); - } - } - - super.loadSnapshot(nextSnapshot); - } - - private static boolean contains(Topology previous, int searchPrefix) - { - for (Range range : previous.ranges()) - { - int prefix = ((PrefixedIntHashKey) range.start()).prefix; - if (prefix == searchPrefix) - return true; - } - return false; - } - - public static class DelayedCommandStore extends InMemoryCommandStore - { - public class DelayedTask extends Task - { - private DelayedTask(Callable call) - { - super(call); - } - - private DelayedTask(Callable call, Pending origin) - { - super(call, origin); - } - - public DelayedCommandStore owner() - { - return DelayedCommandStore.this; - } - - @Override - public void run() - { - Invariants.require(active == null); - Invariants.require(activeThread == null); - active = this; - activeThread = Thread.currentThread(); - try - { - super.run(); - } - finally - { - Invariants.require(active == this); - Invariants.require(activeThread == Thread.currentThread()); - active = null; - activeThread = null; - } - } - - public Callable callable() - { - return callable; - } - - public String toString() - { - return callable.toString(); - } - } - - private final SimulatedDelayedExecutorService executor; - private final Queue> pending = new ArrayDeque<>(); - private final CacheLoading cacheLoading; - private final Journal journal; - private Task active; - private Thread activeThread; - - public DelayedCommandStore(int id, NodeCommandStoreService time, Agent agent, DataStore store, ProgressLog.Factory progressLogFactory, LocalListeners.Factory listenersFactory, EpochUpdateHolder epochUpdateHolder, SimulatedDelayedExecutorService executor, CacheLoading cacheLoading, Journal journal) - { - super(id, time, agent, store, progressLogFactory, listenersFactory, epochUpdateHolder, journal); - this.executor = executor; - this.cacheLoading = cacheLoading; - this.journal = journal; - restore(); - } - - protected void loadRedundantBefore(RedundantBefore redundantBefore) - { - if (redundantBefore == null) - { - Invariants.require(unsafeGetRedundantBefore().size() == 0); - } - else - { - unsafeClearRedundantBefore(); - super.loadRedundantBefore(redundantBefore); - } - } - - protected void loadBootstrapBeganAt(NavigableMap bootstrapBeganAt) - { - unsafeClearBootstrapBeganAt(); - if (bootstrapBeganAt == null) bootstrapBeganAt = ImmutableSortedMap.of(TxnId.NONE, Ranges.EMPTY); - super.loadBootstrapBeganAt(bootstrapBeganAt); - } - - protected void loadSafeToRead(NavigableMap safeToRead) - { - unsafeClearSafeToRead(); - if (safeToRead == null) safeToRead = ImmutableSortedMap.of(Timestamp.NONE, Ranges.EMPTY); - super.loadSafeToRead(safeToRead); - } - - @Override - protected void loadRangesForEpoch(RangesForEpoch newRangesForEpoch) - { - if (newRangesForEpoch == null) Invariants.require(super.rangesForEpoch == null); - else - { - unsafeClearRangesForEpoch(); - super.loadRangesForEpoch(newRangesForEpoch); - } - } - - public void restore() - { - loadRangesForEpoch(journal.loadRangesForEpoch(id())); - loadRedundantBefore(journal.loadRedundantBefore(id())); - loadBootstrapBeganAt(journal.loadBootstrapBeganAt(id())); - loadSafeToRead(journal.loadSafeToRead(id())); - } - - @Override - public void onRead(Command current) - { - cacheLoading.validateRead(this, current); - } - - @Override - public void onWrite(Command current) - { - cacheLoading.validateWrite(this, current); - } - - @Override - public void onRead(CommandsForKey current) - { - cacheLoading.validateRead(this, current); - } - - @Override - protected boolean canExposeUnloaded() - { - return !cacheLoading.cacheEmpty(); - } - - private static CommandStore.Factory factory(SimulatedDelayedExecutorService executor, CacheLoading isLoadedCheck) - { - return (id, node, agent, store, progressLogFactory, listenersFactory, rangesForEpoch, journal) -> new DelayedCommandStore(id, node, agent, store, progressLogFactory, listenersFactory, rangesForEpoch, executor, isLoadedCheck, journal); - } - - @Override - public boolean inStore() - { - return Thread.currentThread() == activeThread; - } - - @Override - public AsyncChain chain(PreLoadContext context, Consumer consumer) - { - return submit(newTask(context, i -> { consumer.accept(i); return null; })); - } - - @Override - public AsyncChain chain(PreLoadContext context, Function function) - { - return submit(newTask(context, function)); - } - - @Override - public AsyncChain chain(Callable call) - { - return submit(new DelayedTask<>(call)); - } - - @Override - public void execute(Runnable run) - { - execute(new DelayedTask<>(() -> { - run.run(); - return null; - })); - } - - private void execute(DelayedTask task) - { - boolean wasEmpty = pending.isEmpty(); - executor.preregister(task); - pending.add(task); - if (wasEmpty) - runNextTask(); - } - - private DelayedTask newTask(PreLoadContext context, Function function) - { - Pending origin = Pending.Global.activeOrigin(); - if (RecurringPendingRunnable.isRecurring(origin) && context.primaryTxnId() != null && !context.primaryTxnId().isSystemTxn()) - origin = null; - return new DelayedTask<>(() -> executeInContext(this, context, function), origin); - } - - private AsyncChain submit(DelayedTask task) - { - if (Invariants.testParanoia(LINEAR, LINEAR, HIGH)) - { - return AsyncChains.detectLeak(agent::onException, () -> { - boolean wasEmpty = pending.isEmpty(); - pending.add(task); - if (wasEmpty) - runNextTask(); - }).flatMap(ignore -> task.chain()); - } - else - { - return new AsyncChains.Head<>() - { - @Override - protected Cancellable start(BiConsumer callback) - { - execute(task); - task.invoke(callback); - return () -> { - if (pending.peek() != task) - { - pending.remove(task); - executor.cancel(task); - task.cancel(false); - } - }; - } - }; - } - } - - private void runNextTask() - { - Task next = pending.peek(); - if (next == null) - return; - - next.invoke(this.agent()); // used to track unexpected exceptions and notify simulations - next.invoke(this::afterExecution); - executor.executePreregistered(next); - } - - private void afterExecution() - { - pending.poll(); - runNextTask(); - } - - @Override - public void shutdown() - { - - } - - @Override - protected InMemorySafeStore createSafeStore(PreLoadContext context, RangesForEpoch ranges, Map commands, Map commandsForKeys) - { - return new DelayedSafeStore(this, ranges, context, commands, commandsForKeys, cacheLoading); - } - } - - static int counter = 0; - public static class DelayedSafeStore extends InMemoryCommandStore.InMemorySafeStore - { - private final DelayedCommandStore commandStore; - private final CacheLoading cacheLoading; - - public DelayedSafeStore(DelayedCommandStore commandStore, - RangesForEpoch ranges, - PreLoadContext context, - Map commands, - Map commandsForKey, - CacheLoading cacheLoading) - { - super(commandStore, ranges, context, commands, commandsForKey); - this.commandStore = commandStore; - this.cacheLoading = cacheLoading; - ++counter; - } - - @Override - public void postExecute() - { - persistFieldUpdates(); - commands.entrySet().forEach(e -> { - InMemorySafeCommand safe = e.getValue(); - if (!safe.isModified()) return; - - Command before = safe.original(); - Command after = safe.current(); - commandStore.journal.saveCommand(commandStore.id(), new CommandUpdate(before, after), () -> {}); - commandStore.onWrite(safe.current()); - }); - super.postExecute(); - } - - @Override - protected void persistFieldUpdates() - { - Journal.FieldUpdates fieldUpdates = fieldUpdates(); - if (fieldUpdates != null) - commandStore.journal.saveStoreState(commandStore.id(), fieldUpdates, () -> {}); - super.persistFieldUpdates(); - } - - @Override - protected InMemoryCommandStore.InMemoryCommandStoreCaches tryGetCaches() - { - if (!cacheLoading.cacheEmpty()) - { - boolean cacheFull = cacheLoading.cacheFull(); - return commandStore.new InMemoryCommandStoreCaches() { - @Override - public InMemorySafeCommand acquireIfLoaded(TxnId txnId) - { - if (cacheFull || cacheLoading.isLoaded(txnId)) - return super.acquireIfLoaded(txnId); - return null; - } - - @Override - public InMemorySafeCommandsForKey acquireIfLoaded(RoutingKey key) - { - if (cacheFull || cacheLoading.isLoaded(key)) - return super.acquireIfLoaded(key); - return null; - } - }; - } - return null; - - } - } - - public List unsafeStores() - { - List stores = new ArrayList<>(); - for (ShardHolder holder : current()) - stores.add((DelayedCommandStore) holder.store); - return stores; - } - - public interface CacheLoading - { - boolean cacheEmpty(); - boolean cacheFull(); - boolean isLoaded(TxnId txnId); - boolean isLoaded(RoutingKey key); - boolean tfkLoaded(); - void validateRead(CommandStore commandStore, Command command); - void validateWrite(CommandStore commandStore, Command command); - void validateRead(CommandStore commandStore, CommandsForKey cfk); - } -}