aboutsummaryrefslogtreecommitdiffstats
path: root/node-repository
diff options
context:
space:
mode:
authorJon Bratseth <bratseth@oath.com>2018-12-18 06:45:53 +0100
committerGitHub <noreply@github.com>2018-12-18 06:45:53 +0100
commit5ee0d59f455441bd4529b5b11810e38d8a09df36 (patch)
treebf1f83f188189d7506a4355189c3930ed80edc7f /node-repository
parent90403c2cec511da980afc44fbd6aef3d1ff0957e (diff)
Revert "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, 204 insertions, 100 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 852d47a4882..8b66b7f3fea 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,7 +81,6 @@ 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.
@@ -98,28 +97,19 @@ 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, useCacheIn(zone, useCuratorClientCache));
+ this.db = new CuratorDatabaseClient(flavors, curator, clock, 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; }
@@ -134,7 +124,7 @@ public class NodeRepository extends AbstractComponent {
/** Returns feature flags of this node repository */
public Flags flags() {
- return flags;
+ return db.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 1b798c14588..f0714111cea 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,7 +11,10 @@ import java.util.Arrays;
public enum FlagId {
/** Indicates whether a exclusive load balancer should be provisioned */
- exclusiveLoadBalancer("exclusive-load-balancer");
+ 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");
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 89f77d02b6e..f5576ae00fc 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().curator().cluster(), hostname, nodeRepository.clock().instant(), interval);
+ long delay = staggeredDelay(nodeRepository.database().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
new file mode 100644
index 00000000000..de803281a87
--- /dev/null
+++ b/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/persistence/CountingCuratorTransaction.java
@@ -0,0 +1,61 @@
+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 d8c1323151d..dd1b58dee8e 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,9 +23,10 @@ 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 2pc transaction containing the counter update.
+ * operations in a try block, with the counter increment in a finally block. Locks must be used to ensure consistency.
*
* @author bratseth
+ * @author jonmv
*/
public class CuratorDatabase {
@@ -86,6 +87,15 @@ 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);
@@ -94,60 +104,35 @@ public class CuratorDatabase {
}
/** Creates a path in curator and all its parents as necessary. If the path already exists this does nothing. */
- // As this operation does not depend on the prior state we do not need to increment the write counter
- public void create(Path path) {
+ void create(Path path) {
curator.create(path);
+ changeGenerationCounter.next(); // Increment counter to ensure getChildren sees any change.
}
/** Returns whether given path exists */
- public boolean exists(Path path) {
+ 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 */
- public List<String> getChildren(Path path) { return getCache().getChildren(path); }
+ List<String> getChildren(Path path) { return getCache().getChildren(path); }
- public Optional<byte[]> getData(Path path) { return getCache().getData(path); }
+ Optional<byte[]> getData(Path path) { return getCache().getData(path); }
- 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());
- }
+ /** Invalidates the current cache if outdated. */
private CuratorDatabaseCache getCache() {
- CacheAndGeneration cacheAndGeneration = getCacheSnapshot();
- while (cacheAndGeneration.expired()) {
- synchronized (cacheCreationLock) { // Prevent a race for creating new caches
- cacheAndGeneration = getCacheSnapshot();
- if (cacheAndGeneration.expired()) {
+ if (changeGenerationCounter.get() != cache.get().generation)
+ synchronized (cacheCreationLock) {
+ while (changeGenerationCounter.get() != cache.get().generation)
cache.set(newCache(changeGenerationCounter.get()));
- cacheAndGeneration = getCacheSnapshot();
- }
}
- }
- return cacheAndGeneration.validCache();
+
+ return cache.get();
}
/** Caches must only be instantiated using this method */
@@ -174,7 +159,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) */
- public CuratorDatabaseCache(long generation, Curator curator) {
+ private CuratorDatabaseCache(long generation, Curator curator) {
this.generation = generation;
this.curator = curator;
}
@@ -183,18 +168,16 @@ 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 content of this child - which may be empty.
- * Returns null only if it is not present in this state mirror
+ * Returns the a copy of the content of this child - which may be empty.
*/
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));
}
}
@@ -202,7 +185,7 @@ public class CuratorDatabase {
/** An implementation of the curator database cache which does no caching */
private static class DeactivatedCache extends CuratorDatabaseCache {
- public DeactivatedCache(long generation, Curator curator) { super(generation, curator); }
+ private 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 fb3f801cf65..9bcfdd8b494 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,6 +5,7 @@ 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;
@@ -19,6 +20,7 @@ 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;
@@ -67,19 +69,17 @@ 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,12 +91,20 @@ 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 = curatorDatabase.newCuratorTransactionIn(transaction);
+ CuratorTransaction curatorTransaction = newCuratorTransactionIn(transaction);
for (Node node : nodes) {
if (node.state() != expectedState)
throw new IllegalArgumentException(node + " is not in the " + node.state() + " state");
@@ -131,7 +139,7 @@ public class CuratorDatabaseClient {
for (Node node : nodes) {
Path path = toPath(node.state(), node.hostname());
- CuratorTransaction curatorTransaction = curatorDatabase.newCuratorTransactionIn(transaction);
+ CuratorTransaction curatorTransaction = newCuratorTransactionIn(transaction);
curatorTransaction.add(CuratorOperations.delete(path.getAbsolute()));
}
@@ -201,7 +209,7 @@ public class CuratorDatabaseClient {
List<Node> writtenNodes = new ArrayList<>(nodes.size());
- CuratorTransaction curatorTransaction = curatorDatabase.newCuratorTransactionIn(transaction);
+ CuratorTransaction curatorTransaction = newCuratorTransactionIn(transaction);
for (Node node : nodes) {
Node newNode = new Node(node.openStackId(), node.ipAddresses(), node.ipAddressPool().asSet(), node.hostname(),
node.parentHostname(), node.flavor(),
@@ -360,7 +368,7 @@ public class CuratorDatabaseClient {
public void writeInactiveJobs(Set<String> inactiveJobs) {
NestedTransaction transaction = new NestedTransaction();
- CuratorTransaction curatorTransaction = curatorDatabase.newCuratorTransactionIn(transaction);
+ CuratorTransaction curatorTransaction = newCuratorTransactionIn(transaction);
curatorTransaction.add(CuratorOperations.setData(inactiveJobsPath().getAbsolute(),
stringSetSerializer.toJson(inactiveJobs)));
transaction.commit();
@@ -380,7 +388,7 @@ public class CuratorDatabaseClient {
public void writeInfrastructureVersions(Map<NodeType, Version> infrastructureVersions) {
NestedTransaction transaction = new NestedTransaction();
- CuratorTransaction curatorTransaction = curatorDatabase.newCuratorTransactionIn(transaction);
+ CuratorTransaction curatorTransaction = newCuratorTransactionIn(transaction);
curatorTransaction.add(CuratorOperations.setData(infrastructureVersionsPath().getAbsolute(),
NodeTypeVersionsSerializer.toJson(infrastructureVersions)));
transaction.commit();
@@ -400,7 +408,7 @@ public class CuratorDatabaseClient {
public void writeOsVersions(Map<NodeType, Version> versions) {
NestedTransaction transaction = new NestedTransaction();
- CuratorTransaction curatorTransaction = curatorDatabase.newCuratorTransactionIn(transaction);
+ CuratorTransaction curatorTransaction = newCuratorTransactionIn(transaction);
curatorTransaction.add(CuratorOperations.setData(osVersionsPath().getAbsolute(),
NodeTypeVersionsSerializer.toJson(versions)));
transaction.commit();
@@ -441,7 +449,7 @@ public class CuratorDatabaseClient {
}
public void writeLoadBalancers(Collection<LoadBalancer> loadBalancers, NestedTransaction transaction) {
- CuratorTransaction curatorTransaction = curatorDatabase.newCuratorTransactionIn(transaction);
+ CuratorTransaction curatorTransaction = newCuratorTransactionIn(transaction);
loadBalancers.forEach(loadBalancer -> {
curatorTransaction.add(createOrSet(loadBalancerPath(loadBalancer.id()),
LoadBalancerSerializer.toJson(loadBalancer)));
@@ -450,7 +458,7 @@ public class CuratorDatabaseClient {
public void removeLoadBalancer(LoadBalancer loadBalancer) {
NestedTransaction transaction = new NestedTransaction();
- CuratorTransaction curatorTransaction = curatorDatabase.newCuratorTransactionIn(transaction);
+ CuratorTransaction curatorTransaction = newCuratorTransactionIn(transaction);
curatorTransaction.add(CuratorOperations.delete(loadBalancerPath(loadBalancer.id()).getAbsolute()));
transaction.commit();
}
@@ -466,7 +474,7 @@ public class CuratorDatabaseClient {
public void writeFlag(Flag flag) {
Path path = flagPath(flag.id());
NestedTransaction transaction = new NestedTransaction();
- CuratorTransaction curatorTransaction = curatorDatabase.newCuratorTransactionIn(transaction);
+ CuratorTransaction curatorTransaction = newCuratorTransactionIn(transaction);
curatorTransaction.add(createOrSet(path, FlagSerializer.toJson(flag)));
transaction.commit();
}
@@ -490,4 +498,10 @@ 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 bf82128dfb7..149510bdc97 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,14 +3,17 @@ 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;
@@ -26,7 +29,7 @@ import static org.junit.Assert.fail;
public class CuratorDatabaseTest {
@Test
- public void testTransactionsIncreaseTimer() throws Exception {
+ public void testTransactionsIncreaseCounter() throws Exception {
MockCurator curator = new MockCurator();
CuratorDatabase database = new CuratorDatabase(curator, Path.fromString("/"), true);
@@ -36,9 +39,8 @@ public class CuratorDatabaseTest {
commitCreate("/2", database);
commitCreate("/1/1", database);
commitCreate("/2/1", database);
- assertEquals(4L, (long)curator.counter("/changeCounter").get().get().postValue());
+ assertEquals(8L, (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);
@@ -49,7 +51,24 @@ 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
@@ -63,7 +82,7 @@ public class CuratorDatabaseTest {
commitCreate("/2", database);
commitCreate("/1/1", database);
commitCreate("/2/1", database);
- assertEquals(4L, (long)curator.counter("/changeCounter").get().get().postValue());
+ assertEquals(8L, (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"));
@@ -72,7 +91,7 @@ public class CuratorDatabaseTest {
}
@Test
- public void testThatCounterIncreasesAlsoOnCommitFailure() throws Exception {
+ public void testThatCounterIncreasesExactlyOnCommitFailure() throws Exception {
MockCurator curator = new MockCurator();
CuratorDatabase database = new CuratorDatabase(curator, Path.fromString("/"), true);
@@ -85,32 +104,16 @@ 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 {
- 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();
+ commitFailing(database); // fail during commit
fail("Expected exception");
}
- catch (Exception expected) {
- // expected because the parent does not exist
- }
- assertEquals(1L, (long)curator.counter("/changeCounter").get().get().postValue());
+ catch (Exception expected) { }
+ // Counter increased, even though commit failed.
+ assertEquals(3L, (long)curator.counter("/changeCounter").get().get().postValue());
}
private void commitCreate(String path, CuratorDatabase database) {
@@ -120,4 +123,41 @@ 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 8fd09b4a274..f27545f6094 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,6 +5,13 @@
"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 78de52e4e85..a0e9954bec4 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,6 +9,12 @@
"enabledApplications": [
"foo:bar:default"
]
+ },
+ {
+ "id": "new-cache-counting",
+ "enabled": false,
+ "enabledHostnames": [],
+ "enabledApplications": []
}
]
}