From 69308ac41225a79c44c4242c3b36eac43808b787 Mon Sep 17 00:00:00 2001 From: Arnstein Ressem Date: Tue, 18 Dec 2018 00:32:18 +0100 Subject: Revert "Jvenstad/fix node repo cache" --- .../vespa/hosted/provision/NodeRepository.java | 14 +++- .../yahoo/vespa/hosted/provision/flag/FlagId.java | 5 +- .../hosted/provision/maintenance/Maintainer.java | 2 +- .../persistence/CountingCuratorTransaction.java | 61 -------------- .../provision/persistence/CuratorDatabase.java | 73 +++++++++------- .../persistence/CuratorDatabaseClient.java | 40 +++------ .../provision/persistence/CuratorDatabaseTest.java | 96 +++++++--------------- .../provision/restapi/v2/responses/flags1.json | 7 -- .../provision/restapi/v2/responses/flags2.json | 6 -- 9 files changed, 100 insertions(+), 204 deletions(-) delete mode 100644 node-repository/src/main/java/com/yahoo/vespa/hosted/provision/persistence/CountingCuratorTransaction.java (limited to 'node-repository') diff --git a/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/NodeRepository.java b/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/NodeRepository.java index 8b66b7f3fea..852d47a4882 100644 --- a/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/NodeRepository.java +++ b/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/NodeRepository.java @@ -81,6 +81,7 @@ public class NodeRepository extends AbstractComponent { private final NameResolver nameResolver; private final DockerImage dockerImage; private final OsVersions osVersions; + private final Flags flags; /** * Creates a node repository from a zookeeper provider. @@ -97,19 +98,28 @@ public class NodeRepository extends AbstractComponent { */ public NodeRepository(NodeFlavors flavors, Curator curator, Clock clock, Zone zone, NameResolver nameResolver, DockerImage dockerImage, boolean useCuratorClientCache) { - this.db = new CuratorDatabaseClient(flavors, curator, clock, zone, useCuratorClientCache); + this.db = new CuratorDatabaseClient(flavors, curator, clock, zone, useCacheIn(zone, useCuratorClientCache)); this.zone = zone; this.clock = clock; this.flavors = flavors; this.nameResolver = nameResolver; this.dockerImage = dockerImage; this.osVersions = new OsVersions(this.db); + this.flags = new Flags(this.db); // read and write all nodes to make sure they are stored in the latest version of the serialized format for (Node.State state : Node.State.values()) db.writeTo(state, db.getNodes(state), Agent.system, Optional.empty()); } + private static boolean useCacheIn(Zone zone, boolean useCache) { + if (zone.region().value().equals("cd-us-central-1")) { + // TODO: Temporarily disabled in CD to see if allocation conflict is related to caching + return false; + } + return useCache; + } + /** Returns the curator database client used by this */ public CuratorDatabaseClient database() { return db; } @@ -124,7 +134,7 @@ public class NodeRepository extends AbstractComponent { /** Returns feature flags of this node repository */ public Flags flags() { - return db.flags(); + return flags; } // ---------------- Query API ---------------------------------------------------------------- diff --git a/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/flag/FlagId.java b/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/flag/FlagId.java index f0714111cea..1b798c14588 100644 --- a/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/flag/FlagId.java +++ b/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/flag/FlagId.java @@ -11,10 +11,7 @@ import java.util.Arrays; public enum FlagId { /** Indicates whether a exclusive load balancer should be provisioned */ - exclusiveLoadBalancer("exclusive-load-balancer"), - - /** Temporary. Indicates whether to use the new cache generation counting, or the old one (with a known bug) */ - newCacheCounting("new-cache-counting"); + exclusiveLoadBalancer("exclusive-load-balancer"); private final String serializedValue; diff --git a/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/maintenance/Maintainer.java b/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/maintenance/Maintainer.java index f5576ae00fc..89f77d02b6e 100644 --- a/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/maintenance/Maintainer.java +++ b/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/maintenance/Maintainer.java @@ -35,7 +35,7 @@ public abstract class Maintainer extends AbstractComponent implements Runnable { this.jobControl = jobControl; HostName hostname = HostName.from(com.yahoo.net.HostName.getLocalhost()); - long delay = staggeredDelay(nodeRepository.database().cluster(), hostname, nodeRepository.clock().instant(), interval); + long delay = staggeredDelay(nodeRepository.database().curator().cluster(), hostname, nodeRepository.clock().instant(), interval); service = new ScheduledThreadPoolExecutor(1); service.scheduleAtFixedRate(this, delay, interval.toMillis(), TimeUnit.MILLISECONDS); jobControl.started(name()); diff --git a/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/persistence/CountingCuratorTransaction.java b/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/persistence/CountingCuratorTransaction.java deleted file mode 100644 index de803281a87..00000000000 --- a/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/persistence/CountingCuratorTransaction.java +++ /dev/null @@ -1,61 +0,0 @@ -package com.yahoo.vespa.hosted.provision.persistence; - -import com.yahoo.path.Path; -import com.yahoo.vespa.curator.Curator; -import com.yahoo.vespa.curator.Lock; -import com.yahoo.vespa.curator.recipes.CuratorCounter; -import com.yahoo.vespa.curator.transaction.CuratorOperation; -import com.yahoo.vespa.curator.transaction.CuratorTransaction; -import com.yahoo.vespa.curator.transaction.TransactionChanges; - -/** - * CuratorTransaction wrapper which increments a counter, to signal invalidation of node repository caches. - * - * This class ensures a CuratorTransaction against the cached data (the node repository data) is - * accompanied by an increment of the data generation counter. An increment must occur after - * the write has completed, successfully or not. It is therefore placed in a {@code finally} block, - * wrapping the super class' {@link #commit()}. - * Likewise, {@link #prepare()} is also wrapped with an increment, in case it fails due to an inconsistent cache. - * The cache is invalid whenever the generation counter is higher than what the cache contents were read with. - * The usual locking for modifications of shared data is then enough to ensure the cache provides a - * consistent view of the shared data, with one exception: when incrementing the counter fails. This is - * assumed to be extremely rare, and the consequence is temporary neglect of cache invalidation. - * - * @author jonmv - */ -class CountingCuratorTransaction extends CuratorTransaction { - - private final CuratorCounter counter; - - public CountingCuratorTransaction(Curator curator, CuratorCounter counter) { - super(curator); - this.counter = counter; - } - - @Override - public void prepare() { - try { - counter.get(); - super.prepare(); - } - finally { - counter.next(); - } - } - - @Override - public void commit() { - try { - super.commit(); - } - finally { - counter.next(); - } - } - - @Override - public String toString() { - return "(" + super.toString() + "), INCREMENT " + counter; - } - -} diff --git a/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/persistence/CuratorDatabase.java b/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/persistence/CuratorDatabase.java index dd1b58dee8e..d8c1323151d 100644 --- a/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/persistence/CuratorDatabase.java +++ b/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/persistence/CuratorDatabase.java @@ -23,10 +23,9 @@ import java.util.stream.Collectors; * This encapsulated the curator database of the node repo. * It serves reads from an in-memory cache of the content which is invalidated when changed on another node * using a global, shared counter. The counter is updated on all write operations, ensured by wrapping write - * operations in a try block, with the counter increment in a finally block. Locks must be used to ensure consistency. + * operations in a 2pc transaction containing the counter update. * * @author bratseth - * @author jonmv */ public class CuratorDatabase { @@ -87,15 +86,6 @@ public class CuratorDatabase { * Important: It is the nested transaction which must be committed - never the curator transaction directly. */ public CuratorTransaction newCuratorTransactionIn(NestedTransaction transaction) { - // Wrap the curator transaction with an increment of the generation counter. - CountingCuratorTransaction curatorTransaction = new CountingCuratorTransaction(curator, changeGenerationCounter); - transaction.add(curatorTransaction); - return curatorTransaction; - } - - // TODO jvenstad: remove. - /** Kept for now to be able to revert to old caching behaviour. */ - CuratorTransaction newEagerCuratorTransactionIn(NestedTransaction transaction) { // Add a counting transaction first, to make sure we always invalidate the current state on any transaction commit transaction.add(new EagerCountingCuratorTransaction(changeGenerationCounter), CuratorTransaction.class); CuratorTransaction curatorTransaction = new CuratorTransaction(curator); @@ -104,35 +94,60 @@ public class CuratorDatabase { } /** Creates a path in curator and all its parents as necessary. If the path already exists this does nothing. */ - void create(Path path) { + // As this operation does not depend on the prior state we do not need to increment the write counter + public void create(Path path) { curator.create(path); - changeGenerationCounter.next(); // Increment counter to ensure getChildren sees any change. } /** Returns whether given path exists */ - boolean exists(Path path) { + public boolean exists(Path path) { return curator.exists(path); } // --------- Read operations ------------------------------------------------------------------------------- // These can read from the memory file system, which accurately mirrors the ZooKeeper content IF - // the current generation counter is the same as it was when data was put into the cache, AND - // the data to read is protected by a lock which is held now, and during any writes of the data. /** Returns the immediate, local names of the children under this node in any order */ - List getChildren(Path path) { return getCache().getChildren(path); } + public List getChildren(Path path) { return getCache().getChildren(path); } - Optional getData(Path path) { return getCache().getData(path); } + public Optional getData(Path path) { return getCache().getData(path); } - /** Invalidates the current cache if outdated. */ + private static class CacheAndGeneration { + public CacheAndGeneration(CuratorDatabaseCache cache, long generation) + { + this.cache = cache; + this.generation = generation; + } + public boolean expired() { + return generation != cache.generation(); + } + public CuratorDatabaseCache validCache() { + if (expired()) { + throw new IllegalStateException("The cache has generation " + cache.generation() + + " while the root genration counter in zookeeper says " + generation + + ". That is totally unacceptable and must be a sever programming error in my close vicinity."); + } + return cache; + } + + private CuratorDatabaseCache cache; + private long generation; + } + private CacheAndGeneration getCacheSnapshot() { + return new CacheAndGeneration(cache.get(), changeGenerationCounter.get()); + } private CuratorDatabaseCache getCache() { - if (changeGenerationCounter.get() != cache.get().generation) - synchronized (cacheCreationLock) { - while (changeGenerationCounter.get() != cache.get().generation) + CacheAndGeneration cacheAndGeneration = getCacheSnapshot(); + while (cacheAndGeneration.expired()) { + synchronized (cacheCreationLock) { // Prevent a race for creating new caches + cacheAndGeneration = getCacheSnapshot(); + if (cacheAndGeneration.expired()) { cache.set(newCache(changeGenerationCounter.get())); + cacheAndGeneration = getCacheSnapshot(); + } } - - return cache.get(); + } + return cacheAndGeneration.validCache(); } /** Caches must only be instantiated using this method */ @@ -159,7 +174,7 @@ public class CuratorDatabase { private final Map> data = new ConcurrentHashMap<>(); /** Create an empty snapshot at a given generation (as an empty snapshot is a valid partial snapshot) */ - private CuratorDatabaseCache(long generation, Curator curator) { + public CuratorDatabaseCache(long generation, Curator curator) { this.generation = generation; this.curator = curator; } @@ -168,16 +183,18 @@ public class CuratorDatabase { /** * Returns the children of this path, which may be empty. + * Returns null only if it is not present in this state mirror */ public List getChildren(Path path) { return children.computeIfAbsent(path, key -> ImmutableList.copyOf(curator.getChildren(path))); } /** - * Returns the a copy of the content of this child - which may be empty. + * Returns the content of this child - which may be empty. + * Returns null only if it is not present in this state mirror */ public Optional getData(Path path) { - return data.computeIfAbsent(path, key -> curator.getData(path)).map(data -> Arrays.copyOf(data, data.length)); + return data.computeIfAbsent(path, key -> curator.getData(path).map(data -> Arrays.copyOf(data, data.length))); } } @@ -185,7 +202,7 @@ public class CuratorDatabase { /** An implementation of the curator database cache which does no caching */ private static class DeactivatedCache extends CuratorDatabaseCache { - private DeactivatedCache(long generation, Curator curator) { super(generation, curator); } + public DeactivatedCache(long generation, Curator curator) { super(generation, curator); } @Override public List getChildren(Path path) { return curator.getChildren(path); } diff --git a/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/persistence/CuratorDatabaseClient.java b/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/persistence/CuratorDatabaseClient.java index 9bcfdd8b494..fb3f801cf65 100644 --- a/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/persistence/CuratorDatabaseClient.java +++ b/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/persistence/CuratorDatabaseClient.java @@ -5,7 +5,6 @@ import com.google.common.util.concurrent.UncheckedTimeoutException; import com.yahoo.component.Version; import com.yahoo.config.provision.ApplicationId; import com.yahoo.config.provision.ApplicationLockException; -import com.yahoo.config.provision.HostName; import com.yahoo.config.provision.NodeFlavors; import com.yahoo.config.provision.NodeType; import com.yahoo.config.provision.Zone; @@ -20,7 +19,6 @@ import com.yahoo.vespa.curator.transaction.CuratorTransaction; import com.yahoo.vespa.hosted.provision.Node; import com.yahoo.vespa.hosted.provision.flag.Flag; import com.yahoo.vespa.hosted.provision.flag.FlagId; -import com.yahoo.vespa.hosted.provision.flag.Flags; import com.yahoo.vespa.hosted.provision.lb.LoadBalancer; import com.yahoo.vespa.hosted.provision.lb.LoadBalancerId; import com.yahoo.vespa.hosted.provision.node.Agent; @@ -69,17 +67,19 @@ public class CuratorDatabaseClient { private final CuratorDatabase curatorDatabase; private final Clock clock; private final Zone zone; - private final Flags flags; public CuratorDatabaseClient(NodeFlavors flavors, Curator curator, Clock clock, Zone zone, boolean useCache) { this.nodeSerializer = new NodeSerializer(flavors); this.zone = zone; - this.flags = new Flags(this); this.curatorDatabase = new CuratorDatabase(curator, root, useCache); this.clock = clock; initZK(); } + public CuratorDatabase curator() { + return curatorDatabase; + } + private void initZK() { curatorDatabase.create(root); for (Node.State state : Node.State.values()) @@ -91,20 +91,12 @@ public class CuratorDatabaseClient { curatorDatabase.create(flagsRoot); } - public List cluster() { - return curatorDatabase.cluster(); - } - - public Flags flags() { - return flags; - } - /** * Adds a set of nodes. Rollbacks/fails transaction if any node is not in the expected state. */ public List addNodesInState(List nodes, Node.State expectedState) { NestedTransaction transaction = new NestedTransaction(); - CuratorTransaction curatorTransaction = newCuratorTransactionIn(transaction); + CuratorTransaction curatorTransaction = curatorDatabase.newCuratorTransactionIn(transaction); for (Node node : nodes) { if (node.state() != expectedState) throw new IllegalArgumentException(node + " is not in the " + node.state() + " state"); @@ -139,7 +131,7 @@ public class CuratorDatabaseClient { for (Node node : nodes) { Path path = toPath(node.state(), node.hostname()); - CuratorTransaction curatorTransaction = newCuratorTransactionIn(transaction); + CuratorTransaction curatorTransaction = curatorDatabase.newCuratorTransactionIn(transaction); curatorTransaction.add(CuratorOperations.delete(path.getAbsolute())); } @@ -209,7 +201,7 @@ public class CuratorDatabaseClient { List writtenNodes = new ArrayList<>(nodes.size()); - CuratorTransaction curatorTransaction = newCuratorTransactionIn(transaction); + CuratorTransaction curatorTransaction = curatorDatabase.newCuratorTransactionIn(transaction); for (Node node : nodes) { Node newNode = new Node(node.openStackId(), node.ipAddresses(), node.ipAddressPool().asSet(), node.hostname(), node.parentHostname(), node.flavor(), @@ -368,7 +360,7 @@ public class CuratorDatabaseClient { public void writeInactiveJobs(Set inactiveJobs) { NestedTransaction transaction = new NestedTransaction(); - CuratorTransaction curatorTransaction = newCuratorTransactionIn(transaction); + CuratorTransaction curatorTransaction = curatorDatabase.newCuratorTransactionIn(transaction); curatorTransaction.add(CuratorOperations.setData(inactiveJobsPath().getAbsolute(), stringSetSerializer.toJson(inactiveJobs))); transaction.commit(); @@ -388,7 +380,7 @@ public class CuratorDatabaseClient { public void writeInfrastructureVersions(Map infrastructureVersions) { NestedTransaction transaction = new NestedTransaction(); - CuratorTransaction curatorTransaction = newCuratorTransactionIn(transaction); + CuratorTransaction curatorTransaction = curatorDatabase.newCuratorTransactionIn(transaction); curatorTransaction.add(CuratorOperations.setData(infrastructureVersionsPath().getAbsolute(), NodeTypeVersionsSerializer.toJson(infrastructureVersions))); transaction.commit(); @@ -408,7 +400,7 @@ public class CuratorDatabaseClient { public void writeOsVersions(Map versions) { NestedTransaction transaction = new NestedTransaction(); - CuratorTransaction curatorTransaction = newCuratorTransactionIn(transaction); + CuratorTransaction curatorTransaction = curatorDatabase.newCuratorTransactionIn(transaction); curatorTransaction.add(CuratorOperations.setData(osVersionsPath().getAbsolute(), NodeTypeVersionsSerializer.toJson(versions))); transaction.commit(); @@ -449,7 +441,7 @@ public class CuratorDatabaseClient { } public void writeLoadBalancers(Collection loadBalancers, NestedTransaction transaction) { - CuratorTransaction curatorTransaction = newCuratorTransactionIn(transaction); + CuratorTransaction curatorTransaction = curatorDatabase.newCuratorTransactionIn(transaction); loadBalancers.forEach(loadBalancer -> { curatorTransaction.add(createOrSet(loadBalancerPath(loadBalancer.id()), LoadBalancerSerializer.toJson(loadBalancer))); @@ -458,7 +450,7 @@ public class CuratorDatabaseClient { public void removeLoadBalancer(LoadBalancer loadBalancer) { NestedTransaction transaction = new NestedTransaction(); - CuratorTransaction curatorTransaction = newCuratorTransactionIn(transaction); + CuratorTransaction curatorTransaction = curatorDatabase.newCuratorTransactionIn(transaction); curatorTransaction.add(CuratorOperations.delete(loadBalancerPath(loadBalancer.id()).getAbsolute())); transaction.commit(); } @@ -474,7 +466,7 @@ public class CuratorDatabaseClient { public void writeFlag(Flag flag) { Path path = flagPath(flag.id()); NestedTransaction transaction = new NestedTransaction(); - CuratorTransaction curatorTransaction = newCuratorTransactionIn(transaction); + CuratorTransaction curatorTransaction = curatorDatabase.newCuratorTransactionIn(transaction); curatorTransaction.add(createOrSet(path, FlagSerializer.toJson(flag))); transaction.commit(); } @@ -498,10 +490,4 @@ public class CuratorDatabaseClient { return CuratorOperations.create(path.getAbsolute(), data); } - private CuratorTransaction newCuratorTransactionIn(NestedTransaction transaction) { - return flags.get(FlagId.newCacheCounting).isEnabled() - ? curatorDatabase.newCuratorTransactionIn(transaction) - : curatorDatabase.newEagerCuratorTransactionIn(transaction); - } - } diff --git a/node-repository/src/test/java/com/yahoo/vespa/hosted/provision/persistence/CuratorDatabaseTest.java b/node-repository/src/test/java/com/yahoo/vespa/hosted/provision/persistence/CuratorDatabaseTest.java index 149510bdc97..bf82128dfb7 100644 --- a/node-repository/src/test/java/com/yahoo/vespa/hosted/provision/persistence/CuratorDatabaseTest.java +++ b/node-repository/src/test/java/com/yahoo/vespa/hosted/provision/persistence/CuratorDatabaseTest.java @@ -3,17 +3,14 @@ package com.yahoo.vespa.hosted.provision.persistence; import com.yahoo.path.Path; import com.yahoo.transaction.NestedTransaction; -import com.yahoo.vespa.curator.Curator; import com.yahoo.vespa.curator.mock.MockCurator; -import com.yahoo.vespa.curator.transaction.CuratorOperation; import com.yahoo.vespa.curator.transaction.CuratorOperations; import com.yahoo.vespa.curator.transaction.CuratorTransaction; -import com.yahoo.vespa.curator.transaction.TransactionChanges; import org.junit.Test; import java.util.List; +import java.util.Optional; -import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertFalse; @@ -29,7 +26,7 @@ import static org.junit.Assert.fail; public class CuratorDatabaseTest { @Test - public void testTransactionsIncreaseCounter() throws Exception { + public void testTransactionsIncreaseTimer() throws Exception { MockCurator curator = new MockCurator(); CuratorDatabase database = new CuratorDatabase(curator, Path.fromString("/"), true); @@ -39,8 +36,9 @@ public class CuratorDatabaseTest { commitCreate("/2", database); commitCreate("/1/1", database); commitCreate("/2/1", database); - assertEquals(8L, (long)curator.counter("/changeCounter").get().get().postValue()); + assertEquals(4L, (long)curator.counter("/changeCounter").get().get().postValue()); + List children1Call0 = database.getChildren(Path.fromString("/1")); // prime the db; this call returns a different instance List children1Call1 = database.getChildren(Path.fromString("/1")); List children1Call2 = database.getChildren(Path.fromString("/1")); assertTrue("We reuse cached data when there are no commits", children1Call1 == children1Call2); @@ -51,24 +49,7 @@ public class CuratorDatabaseTest { assertEquals(2, database.getChildren(Path.fromString("/2")).size()); assertFalse("We do not reuse cached data in different parts of the tree when there are commits", children1Call3 == children1Call2); - } - - @Test - public void testCacheInvalidation() throws Exception { - MockCurator curator = new MockCurator(); - CuratorDatabase database = new CuratorDatabase(curator, Path.fromString("/"), true); - - assertEquals(0L, (long)curator.counter("/changeCounter").get().get().postValue()); - commitCreate("/1", database); - assertArrayEquals(new byte[0], database.getData(Path.fromString("/1")).get()); - commitReadingWrite("/1", "hello".getBytes(), database); - // Data cached during commit of write transaction. Should be invalid now, and re-read. - assertEquals(4L, (long)curator.counter("/changeCounter").get().get().postValue()); - assertArrayEquals("hello".getBytes(), database.getData(Path.fromString("/1")).get()); - - assertEquals(0, database.getChildren(Path.fromString("/1")).size()); - commitCreate("/1/1", database); - assertEquals(1, database.getChildren(Path.fromString("/1")).size()); + } @Test @@ -82,7 +63,7 @@ public class CuratorDatabaseTest { commitCreate("/2", database); commitCreate("/1/1", database); commitCreate("/2/1", database); - assertEquals(8L, (long)curator.counter("/changeCounter").get().get().postValue()); + assertEquals(4L, (long)curator.counter("/changeCounter").get().get().postValue()); List children1Call0 = database.getChildren(Path.fromString("/1")); // prime the db; this call returns a different instance List children1Call1 = database.getChildren(Path.fromString("/1")); @@ -91,7 +72,7 @@ public class CuratorDatabaseTest { } @Test - public void testThatCounterIncreasesExactlyOnCommitFailure() throws Exception { + public void testThatCounterIncreasesAlsoOnCommitFailure() throws Exception { MockCurator curator = new MockCurator(); CuratorDatabase database = new CuratorDatabase(curator, Path.fromString("/"), true); @@ -104,16 +85,32 @@ public class CuratorDatabaseTest { catch (Exception expected) { // expected because the parent does not exist } - // Counter increased once, since prepare failed. assertEquals(1L, (long)curator.counter("/changeCounter").get().get().postValue()); + } + + @Test + public void testThatCounterIncreasesAlsoOnCommitFailureFromExistingTransaction() throws Exception { + MockCurator curator = new MockCurator(); + CuratorDatabase database = new CuratorDatabase(curator, Path.fromString("/"), true); + + assertEquals(0L, (long)curator.counter("/changeCounter").get().get().postValue()); try { - commitFailing(database); // fail during commit + NestedTransaction t = new NestedTransaction(); + CuratorTransaction separateC = new CuratorTransaction(curator); + separateC.add(CuratorOperations.create("/1/2")); // fail as parent does not exist + t.add(separateC); + + CuratorTransaction c = database.newCuratorTransactionIn(t); + c.add(CuratorOperations.create("/1")); // does not fail + + t.commit(); fail("Expected exception"); } - catch (Exception expected) { } - // Counter increased, even though commit failed. - assertEquals(3L, (long)curator.counter("/changeCounter").get().get().postValue()); + catch (Exception expected) { + // expected because the parent does not exist + } + assertEquals(1L, (long)curator.counter("/changeCounter").get().get().postValue()); } private void commitCreate(String path, CuratorDatabase database) { @@ -123,41 +120,4 @@ public class CuratorDatabaseTest { t.commit(); } - private void commitReadingWrite(String path, byte[] data, CuratorDatabase database) { - NestedTransaction transaction = new NestedTransaction(); - byte[] oldData = database.getData(Path.fromString(path)).get(); - CuratorTransaction curatorTransaction = database.newCuratorTransactionIn(transaction); - // Add a dummy operation which reads the data and populates the cache during commit of the write. - curatorTransaction.add(new DummyOperation(() -> assertArrayEquals(oldData, database.getData(Path.fromString(path)).get()))); - curatorTransaction.add(CuratorOperations.setData(path, data)); - transaction.commit(); - } - - /** Commit an operation which fails during commit. */ - private void commitFailing(CuratorDatabase database) { - NestedTransaction t = new NestedTransaction(); - CuratorTransaction c = database.newCuratorTransactionIn(t); - c.add(new DummyOperation(() -> { throw new RuntimeException(); })); - t.commit(); - } - - static class DummyOperation implements CuratorOperation { - - private final Runnable task; - - public DummyOperation(Runnable task) { - this.task = task; - } - - @Override - public org.apache.curator.framework.api.transaction.CuratorTransaction and(org.apache.curator.framework.api.transaction.CuratorTransaction transaction) { - task.run(); - return transaction; - } - - @Override - public void check(Curator curator, TransactionChanges changes) { } - - } - } diff --git a/node-repository/src/test/java/com/yahoo/vespa/hosted/provision/restapi/v2/responses/flags1.json b/node-repository/src/test/java/com/yahoo/vespa/hosted/provision/restapi/v2/responses/flags1.json index f27545f6094..8fd09b4a274 100644 --- a/node-repository/src/test/java/com/yahoo/vespa/hosted/provision/restapi/v2/responses/flags1.json +++ b/node-repository/src/test/java/com/yahoo/vespa/hosted/provision/restapi/v2/responses/flags1.json @@ -5,13 +5,6 @@ "enabled": false, "enabledHostnames": [], "enabledApplications": [] - }, - { - "id": "new-cache-counting", - "enabled": false, - "enabledHostnames": [], - "enabledApplications": [] } - ] } diff --git a/node-repository/src/test/java/com/yahoo/vespa/hosted/provision/restapi/v2/responses/flags2.json b/node-repository/src/test/java/com/yahoo/vespa/hosted/provision/restapi/v2/responses/flags2.json index a0e9954bec4..78de52e4e85 100644 --- a/node-repository/src/test/java/com/yahoo/vespa/hosted/provision/restapi/v2/responses/flags2.json +++ b/node-repository/src/test/java/com/yahoo/vespa/hosted/provision/restapi/v2/responses/flags2.json @@ -9,12 +9,6 @@ "enabledApplications": [ "foo:bar:default" ] - }, - { - "id": "new-cache-counting", - "enabled": false, - "enabledHostnames": [], - "enabledApplications": [] } ] } -- cgit v1.2.3