aboutsummaryrefslogtreecommitdiffstats
path: root/node-repository
diff options
context:
space:
mode:
authorArnstein Ressem <aressem@gmail.com>2018-12-18 00:32:18 +0100
committerGitHub <noreply@github.com>2018-12-18 00:32:18 +0100
commit69308ac41225a79c44c4242c3b36eac43808b787 (patch)
tree300df69aee29215f4dd106f71b439e2c33fb3114 /node-repository
parenta3ed6b7671b7e1c4da494be714922366f7ac96ea (diff)
Revert "Jvenstad/fix node repo cache"
Diffstat (limited to 'node-repository')
-rw-r--r--node-repository/src/main/java/com/yahoo/vespa/hosted/provision/NodeRepository.java14
-rw-r--r--node-repository/src/main/java/com/yahoo/vespa/hosted/provision/flag/FlagId.java5
-rw-r--r--node-repository/src/main/java/com/yahoo/vespa/hosted/provision/maintenance/Maintainer.java2
-rw-r--r--node-repository/src/main/java/com/yahoo/vespa/hosted/provision/persistence/CountingCuratorTransaction.java61
-rw-r--r--node-repository/src/main/java/com/yahoo/vespa/hosted/provision/persistence/CuratorDatabase.java73
-rw-r--r--node-repository/src/main/java/com/yahoo/vespa/hosted/provision/persistence/CuratorDatabaseClient.java40
-rw-r--r--node-repository/src/test/java/com/yahoo/vespa/hosted/provision/persistence/CuratorDatabaseTest.java96
-rw-r--r--node-repository/src/test/java/com/yahoo/vespa/hosted/provision/restapi/v2/responses/flags1.json7
-rw-r--r--node-repository/src/test/java/com/yahoo/vespa/hosted/provision/restapi/v2/responses/flags2.json6
9 files changed, 100 insertions, 204 deletions
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 <em>after</em>
- * 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<String> getChildren(Path path) { return getCache().getChildren(path); }
+ public List<String> getChildren(Path path) { return getCache().getChildren(path); }
- Optional<byte[]> getData(Path path) { return getCache().getData(path); }
+ public Optional<byte[]> 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<Path, Optional<byte[]>> 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<String> 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<byte[]> 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<String> 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<HostName> 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<Node> addNodesInState(List<Node> 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<Node> 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<String> 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<NodeType, Version> 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<NodeType, Version> 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<LoadBalancer> 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<String> children1Call0 = database.getChildren(Path.fromString("/1")); // prime the db; this call returns a different instance
List<String> children1Call1 = database.getChildren(Path.fromString("/1"));
List<String> 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<String> children1Call0 = database.getChildren(Path.fromString("/1")); // prime the db; this call returns a different instance
List<String> 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": []
}
]
}