summaryrefslogtreecommitdiffstats
path: root/config-proxy
diff options
context:
space:
mode:
authorHarald Musum <musum@verizonmedia.com>2019-07-11 08:35:37 +0200
committerHarald Musum <musum@verizonmedia.com>2019-07-11 08:35:37 +0200
commit2a68cfb9cc5b78ca20efa54854661915ed0b4a64 (patch)
treed6485d34ba86f911ccd97526a8b92c06cc67a1db /config-proxy
parent5b9f5335e2256180e84eace69f08afdf16e74803 (diff)
Refactoring, no functional changes
Untangle class hierarchy a bit, move ClientUpdater code into RpcConfigSourceClient. Remove a complicated and unnecessary unit test in ProxyServerTest that depended on mocks behaving correctly.
Diffstat (limited to 'config-proxy')
-rw-r--r--config-proxy/src/main/java/com/yahoo/vespa/config/proxy/ClientUpdater.java69
-rw-r--r--config-proxy/src/main/java/com/yahoo/vespa/config/proxy/ConfigSourceClient.java2
-rw-r--r--config-proxy/src/main/java/com/yahoo/vespa/config/proxy/DelayedResponses.java7
-rw-r--r--config-proxy/src/main/java/com/yahoo/vespa/config/proxy/MemoryCacheConfigClient.java9
-rw-r--r--config-proxy/src/main/java/com/yahoo/vespa/config/proxy/ProxyServer.java15
-rw-r--r--config-proxy/src/main/java/com/yahoo/vespa/config/proxy/RpcConfigSourceClient.java64
-rw-r--r--config-proxy/src/main/java/com/yahoo/vespa/config/proxy/UpstreamConfigSubscriber.java8
-rw-r--r--config-proxy/src/test/java/com/yahoo/vespa/config/proxy/ConfigProxyRpcServerTest.java2
-rw-r--r--config-proxy/src/test/java/com/yahoo/vespa/config/proxy/DelayedResponseHandlerTest.java5
-rw-r--r--config-proxy/src/test/java/com/yahoo/vespa/config/proxy/DelayedResponsesTest.java3
-rw-r--r--config-proxy/src/test/java/com/yahoo/vespa/config/proxy/MockClientUpdater.java52
-rw-r--r--config-proxy/src/test/java/com/yahoo/vespa/config/proxy/MockConfigSource.java11
-rw-r--r--config-proxy/src/test/java/com/yahoo/vespa/config/proxy/MockConfigSourceClient.java5
-rw-r--r--config-proxy/src/test/java/com/yahoo/vespa/config/proxy/ProxyServerTest.java22
-rw-r--r--config-proxy/src/test/java/com/yahoo/vespa/config/proxy/RpcConfigSourceClientTest.java (renamed from config-proxy/src/test/java/com/yahoo/vespa/config/proxy/ClientUpdaterTest.java)12
15 files changed, 92 insertions, 194 deletions
diff --git a/config-proxy/src/main/java/com/yahoo/vespa/config/proxy/ClientUpdater.java b/config-proxy/src/main/java/com/yahoo/vespa/config/proxy/ClientUpdater.java
deleted file mode 100644
index e57758f3eec..00000000000
--- a/config-proxy/src/main/java/com/yahoo/vespa/config/proxy/ClientUpdater.java
+++ /dev/null
@@ -1,69 +0,0 @@
-// Copyright 2017 Yahoo Holdings. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
-package com.yahoo.vespa.config.proxy;
-
-import com.yahoo.log.LogLevel;
-import com.yahoo.vespa.config.RawConfig;
-import com.yahoo.vespa.config.protocol.JRTServerConfigRequest;
-
-import java.util.concurrent.DelayQueue;
-import java.util.logging.Logger;
-
-/**
- * Updates clients subscribing to config when config changes or the
- * timeout they have specified has elapsed. Not used when in 'memorycache' mode.
- *
- * @author hmusum
- */
-class ClientUpdater {
-
- private final static Logger log = Logger.getLogger(ClientUpdater.class.getName());
-
- private final ConfigProxyStatistics statistics;
- private final RpcServer rpcServer;
- private final DelayedResponses delayedResponses;
-
- ClientUpdater(RpcServer rpcServer, ConfigProxyStatistics statistics, DelayedResponses delayedResponses) {
- this.rpcServer = rpcServer;
- this.statistics = statistics;
- this.delayedResponses = delayedResponses;
- }
-
- /**
- * This method will be called when a response with changed config is received from upstream
- * (content or generation has changed) or the server timeout has elapsed.
- *
- * @param config new config
- */
- void updateSubscribers(RawConfig config) {
- log.log(LogLevel.DEBUG, () -> "Config updated for " + config.getKey() + "," + config.getGeneration());
- if (config.isError()) { statistics.incErrorCount(); }
- DelayQueue<DelayedResponse> responseDelayQueue = delayedResponses.responses();
- log.log(LogLevel.SPAM, () -> "Delayed response queue: " + responseDelayQueue);
- if (responseDelayQueue.size() == 0) {
- log.log(LogLevel.DEBUG, () -> "There exists no matching element on delayed response queue for " + config.getKey());
- return;
- } else {
- log.log(LogLevel.DEBUG, () -> "Delayed response queue has " + responseDelayQueue.size() + " elements");
- }
- boolean found = false;
- for (DelayedResponse response : responseDelayQueue.toArray(new DelayedResponse[0])) {
- JRTServerConfigRequest request = response.getRequest();
- if (request.getConfigKey().equals(config.getKey())
- // Generation 0 is special, used when returning empty sentinel config
- && (config.getGeneration() >= request.getRequestGeneration() || config.getGeneration() == 0)) {
- if (delayedResponses.remove(response)) {
- found = true;
- log.log(LogLevel.DEBUG, () -> "Call returnOkResponse for " + config.getKey() + "," + config.getGeneration());
- rpcServer.returnOkResponse(request, config);
- } else {
- log.log(LogLevel.INFO, "Could not remove " + config.getKey() + " from delayedResponses queue, already removed");
- }
- }
- }
- if (!found) {
- log.log(LogLevel.DEBUG, () -> "Found no recipient for " + config.getKey() + " in delayed response queue");
- }
- log.log(LogLevel.DEBUG, () -> "Finished updating config for " + config.getKey() + "," + config.getGeneration());
- }
-
-}
diff --git a/config-proxy/src/main/java/com/yahoo/vespa/config/proxy/ConfigSourceClient.java b/config-proxy/src/main/java/com/yahoo/vespa/config/proxy/ConfigSourceClient.java
index 5bab27c27a5..2f8a1b463e3 100644
--- a/config-proxy/src/main/java/com/yahoo/vespa/config/proxy/ConfigSourceClient.java
+++ b/config-proxy/src/main/java/com/yahoo/vespa/config/proxy/ConfigSourceClient.java
@@ -24,4 +24,6 @@ interface ConfigSourceClient {
List<String> getSourceConnections();
+ void updateSubscribers(RawConfig config);
+
}
diff --git a/config-proxy/src/main/java/com/yahoo/vespa/config/proxy/DelayedResponses.java b/config-proxy/src/main/java/com/yahoo/vespa/config/proxy/DelayedResponses.java
index 953aeefbfd1..b0ed3721f81 100644
--- a/config-proxy/src/main/java/com/yahoo/vespa/config/proxy/DelayedResponses.java
+++ b/config-proxy/src/main/java/com/yahoo/vespa/config/proxy/DelayedResponses.java
@@ -11,19 +11,12 @@ import java.util.concurrent.DelayQueue;
class DelayedResponses {
private final DelayQueue<DelayedResponse> delayedResponses = new DelayQueue<>();
- private final ConfigProxyStatistics statistics;
-
- DelayedResponses(ConfigProxyStatistics statistics) {
- this.statistics = statistics;
- }
void add(DelayedResponse response) {
delayedResponses.add(response);
- statistics.delayedResponses(delayedResponses.size());
}
boolean remove(DelayedResponse response) {
- statistics.decDelayedResponses();
return delayedResponses.remove(response);
}
diff --git a/config-proxy/src/main/java/com/yahoo/vespa/config/proxy/MemoryCacheConfigClient.java b/config-proxy/src/main/java/com/yahoo/vespa/config/proxy/MemoryCacheConfigClient.java
index 4b6e9172f8b..f9f5c475723 100644
--- a/config-proxy/src/main/java/com/yahoo/vespa/config/proxy/MemoryCacheConfigClient.java
+++ b/config-proxy/src/main/java/com/yahoo/vespa/config/proxy/MemoryCacheConfigClient.java
@@ -43,12 +43,10 @@ class MemoryCacheConfigClient implements ConfigSourceClient {
}
@Override
- public void cancel() {
- }
+ public void cancel() {}
@Override
- public void shutdownSourceConnections() {
- }
+ public void shutdownSourceConnections() {}
@Override
public String getActiveSourceConnection() {
@@ -60,4 +58,7 @@ class MemoryCacheConfigClient implements ConfigSourceClient {
return Collections.singletonList("N/A");
}
+ @Override
+ public void updateSubscribers(RawConfig config) {}
+
}
diff --git a/config-proxy/src/main/java/com/yahoo/vespa/config/proxy/ProxyServer.java b/config-proxy/src/main/java/com/yahoo/vespa/config/proxy/ProxyServer.java
index 55e546072fc..4bf3bd5a786 100644
--- a/config-proxy/src/main/java/com/yahoo/vespa/config/proxy/ProxyServer.java
+++ b/config-proxy/src/main/java/com/yahoo/vespa/config/proxy/ProxyServer.java
@@ -44,7 +44,6 @@ public class ProxyServer implements Runnable {
// Scheduled executor that periodically checks for requests that have timed out and response should be returned to clients
private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(1, new DaemonThreadFactory());
private final Supervisor supervisor = new Supervisor(new Transport(JRT_TRANSPORT_THREADS));
- private final ClientUpdater clientUpdater;
private ScheduledFuture<?> delayedResponseScheduler;
private final ConfigProxyRpcServer rpcServer;
@@ -85,8 +84,7 @@ public class ProxyServer implements Runnable {
this.delayedResponseHandling = delayedResponseHandling;
this.memoryCache = memoryCache;
this.rpcServer = createRpcServer(spec);
- clientUpdater = new ClientUpdater(rpcServer, statistics, delayedResponses);
- this.configClient = createClient(clientUpdater, delayedResponses, source, timingValues, memoryCache, configClient);
+ this.configClient = createClient(rpcServer, statistics, delayedResponses, source, timingValues, memoryCache, configClient);
this.fileDistributionAndUrlDownload = new FileDistributionAndUrlDownload(supervisor, source);
}
@@ -99,7 +97,7 @@ public class ProxyServer implements Runnable {
MemoryCache memoryCache,
ConfigProxyStatistics statistics) {
final boolean delayedResponseHandling = false;
- return new ProxyServer(null, new DelayedResponses(statistics),
+ return new ProxyServer(null, new DelayedResponses(),
source, statistics, defaultTimingValues(), delayedResponseHandling,
memoryCache, configSourceClient);
}
@@ -157,11 +155,12 @@ public class ProxyServer implements Runnable {
}
}
- private ConfigSourceClient createClient(ClientUpdater clientUpdater, DelayedResponses delayedResponses,
+ private ConfigSourceClient createClient(RpcServer rpcServer, ConfigProxyStatistics statistics,
+ DelayedResponses delayedResponses,
ConfigSourceSet source, TimingValues timingValues,
MemoryCache memoryCache, ConfigSourceClient client) {
return (client == null)
- ? new RpcConfigSourceClient(source, clientUpdater, memoryCache, timingValues, delayedResponses)
+ ? new RpcConfigSourceClient(rpcServer, source, statistics, memoryCache, timingValues, delayedResponses)
: client;
}
@@ -170,7 +169,7 @@ public class ProxyServer implements Runnable {
}
private RpcConfigSourceClient createRpcClient() {
- return new RpcConfigSourceClient(configSource, clientUpdater, memoryCache, timingValues, delayedResponses);
+ return new RpcConfigSourceClient(rpcServer, configSource, statistics, memoryCache, timingValues, delayedResponses);
}
private void setupSignalHandler() {
@@ -210,7 +209,7 @@ public class ProxyServer implements Runnable {
t.start();
ConfigSourceSet configSources = new ConfigSourceSet(properties.configSources);
- DelayedResponses delayedResponses = new DelayedResponses(statistics);
+ DelayedResponses delayedResponses = new DelayedResponses();
ProxyServer proxyServer = new ProxyServer(new Spec(null, port), delayedResponses, configSources, statistics,
defaultTimingValues(), true, new MemoryCache(), null);
// catch termination and interrupt signal
diff --git a/config-proxy/src/main/java/com/yahoo/vespa/config/proxy/RpcConfigSourceClient.java b/config-proxy/src/main/java/com/yahoo/vespa/config/proxy/RpcConfigSourceClient.java
index 9306220f518..c9f43ac48e2 100644
--- a/config-proxy/src/main/java/com/yahoo/vespa/config/proxy/RpcConfigSourceClient.java
+++ b/config-proxy/src/main/java/com/yahoo/vespa/config/proxy/RpcConfigSourceClient.java
@@ -14,7 +14,11 @@ import com.yahoo.log.LogLevel;
import com.yahoo.vespa.config.*;
import com.yahoo.vespa.config.protocol.JRTServerConfigRequest;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.DelayQueue;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.logging.Logger;
@@ -29,25 +33,28 @@ class RpcConfigSourceClient implements ConfigSourceClient {
private final static Logger log = Logger.getLogger(RpcConfigSourceClient.class.getName());
private final Supervisor supervisor = new Supervisor(new Transport());
+ private final RpcServer rpcServer;
private final ConfigSourceSet configSourceSet;
private final HashMap<ConfigCacheKey, Subscriber> activeSubscribers = new HashMap<>();
private final Object activeSubscribersLock = new Object();
private final MemoryCache memoryCache;
- private final ClientUpdater clientUpdater;
+ private final ConfigProxyStatistics statistics;
private final DelayedResponses delayedResponses;
private final TimingValues timingValues;
- private ExecutorService exec;
- private Map<ConfigSourceSet, JRTConfigRequester> requesterPool;
+ private final ExecutorService exec;
+ private final Map<ConfigSourceSet, JRTConfigRequester> requesterPool;
- RpcConfigSourceClient(ConfigSourceSet configSourceSet,
- ClientUpdater clientUpdater,
+ RpcConfigSourceClient(RpcServer rpcServer,
+ ConfigSourceSet configSourceSet,
+ ConfigProxyStatistics statistics,
MemoryCache memoryCache,
TimingValues timingValues,
DelayedResponses delayedResponses) {
+ this.rpcServer = rpcServer;
this.configSourceSet = configSourceSet;
- this.clientUpdater = clientUpdater;
+ this.statistics = statistics;
this.memoryCache = memoryCache;
this.delayedResponses = delayedResponses;
this.timingValues = timingValues;
@@ -115,6 +122,7 @@ class RpcConfigSourceClient implements ConfigSourceClient {
// happens at the same time
DelayedResponse delayedResponse = new DelayedResponse(request);
delayedResponses.add(delayedResponse);
+ statistics.delayedResponses(delayedResponses.size());
final ConfigCacheKey configCacheKey = new ConfigCacheKey(input.getKey(), input.getDefMd5());
RawConfig cachedConfig = memoryCache.get(configCacheKey);
@@ -131,6 +139,7 @@ class RpcConfigSourceClient implements ConfigSourceClient {
// unless another thread already did it
ret = cachedConfig;
}
+ statistics.decDelayedResponses();
}
if (!cachedConfig.isError() && cachedConfig.getGeneration() > 0) {
needToGetConfig = false;
@@ -148,7 +157,7 @@ class RpcConfigSourceClient implements ConfigSourceClient {
log.log(LogLevel.DEBUG, () -> "Already a subscriber running for: " + configCacheKey);
} else {
log.log(LogLevel.DEBUG, () -> "Could not find good config in cache, creating subscriber for: " + configCacheKey);
- UpstreamConfigSubscriber subscriber = new UpstreamConfigSubscriber(input, clientUpdater, configSourceSet,
+ UpstreamConfigSubscriber subscriber = new UpstreamConfigSubscriber(input, this, configSourceSet,
timingValues, requesterPool, memoryCache);
try {
subscriber.subscribe();
@@ -202,4 +211,43 @@ class RpcConfigSourceClient implements ConfigSourceClient {
}
return ret;
}
+
+ /**
+ * This method will be called when a response with changed config is received from upstream
+ * (content or generation has changed) or the server timeout has elapsed.
+ *
+ * @param config new config
+ */
+ public void updateSubscribers(RawConfig config) {
+ log.log(LogLevel.DEBUG, () -> "Config updated for " + config.getKey() + "," + config.getGeneration());
+ if (config.isError()) { statistics.incErrorCount(); }
+ DelayQueue<DelayedResponse> responseDelayQueue = delayedResponses.responses();
+ log.log(LogLevel.SPAM, () -> "Delayed response queue: " + responseDelayQueue);
+ if (responseDelayQueue.size() == 0) {
+ log.log(LogLevel.DEBUG, () -> "There exists no matching element on delayed response queue for " + config.getKey());
+ return;
+ } else {
+ log.log(LogLevel.DEBUG, () -> "Delayed response queue has " + responseDelayQueue.size() + " elements");
+ }
+ boolean found = false;
+ for (DelayedResponse response : responseDelayQueue.toArray(new DelayedResponse[0])) {
+ JRTServerConfigRequest request = response.getRequest();
+ if (request.getConfigKey().equals(config.getKey())
+ // Generation 0 is special, used when returning empty sentinel config
+ && (config.getGeneration() >= request.getRequestGeneration() || config.getGeneration() == 0)) {
+ if (delayedResponses.remove(response)) {
+ found = true;
+ log.log(LogLevel.DEBUG, () -> "Call returnOkResponse for " + config.getKey() + "," + config.getGeneration());
+ rpcServer.returnOkResponse(request, config);
+ } else {
+ log.log(LogLevel.INFO, "Could not remove " + config.getKey() + " from delayedResponses queue, already removed");
+ }
+ }
+ }
+ if (!found) {
+ log.log(LogLevel.DEBUG, () -> "Found no recipient for " + config.getKey() + " in delayed response queue");
+ }
+ log.log(LogLevel.DEBUG, () -> "Finished updating config for " + config.getKey() + "," + config.getGeneration());
+ }
+
}
diff --git a/config-proxy/src/main/java/com/yahoo/vespa/config/proxy/UpstreamConfigSubscriber.java b/config-proxy/src/main/java/com/yahoo/vespa/config/proxy/UpstreamConfigSubscriber.java
index b99b7d679ca..f8df16cb3d2 100644
--- a/config-proxy/src/main/java/com/yahoo/vespa/config/proxy/UpstreamConfigSubscriber.java
+++ b/config-proxy/src/main/java/com/yahoo/vespa/config/proxy/UpstreamConfigSubscriber.java
@@ -23,7 +23,7 @@ public class UpstreamConfigSubscriber implements Subscriber {
private final static Logger log = Logger.getLogger(UpstreamConfigSubscriber.class.getName());
private final RawConfig config;
- private final ClientUpdater clientUpdater;
+ private final ConfigSourceClient configSourceClient;
private final ConfigSource configSourceSet;
private final TimingValues timingValues;
private final Map<ConfigSourceSet, JRTConfigRequester> requesterPool;
@@ -31,11 +31,11 @@ public class UpstreamConfigSubscriber implements Subscriber {
private GenericConfigSubscriber subscriber;
private GenericConfigHandle handle;
- UpstreamConfigSubscriber(RawConfig config, ClientUpdater clientUpdater, ConfigSource configSourceSet,
+ UpstreamConfigSubscriber(RawConfig config, ConfigSourceClient configSourceClient, ConfigSource configSourceSet,
TimingValues timingValues, Map<ConfigSourceSet, JRTConfigRequester> requesterPool,
MemoryCache memoryCache) {
this.config = config;
- this.clientUpdater = clientUpdater;
+ this.configSourceClient = configSourceClient;
this.configSourceSet = configSourceSet;
this.timingValues = timingValues;
this.requesterPool = requesterPool;
@@ -70,7 +70,7 @@ public class UpstreamConfigSubscriber implements Subscriber {
"', generation=" + newConfig.getGeneration() +
", payload=" + newConfig.getPayload());
memoryCache.update(newConfig);
- clientUpdater.updateSubscribers(newConfig);
+ configSourceClient.updateSubscribers(newConfig);
}
@Override
diff --git a/config-proxy/src/test/java/com/yahoo/vespa/config/proxy/ConfigProxyRpcServerTest.java b/config-proxy/src/test/java/com/yahoo/vespa/config/proxy/ConfigProxyRpcServerTest.java
index 48456d8ac23..b810c6c7e4c 100644
--- a/config-proxy/src/test/java/com/yahoo/vespa/config/proxy/ConfigProxyRpcServerTest.java
+++ b/config-proxy/src/test/java/com/yahoo/vespa/config/proxy/ConfigProxyRpcServerTest.java
@@ -46,7 +46,7 @@ public class ConfigProxyRpcServerTest {
@Test
public void basic() {
- ProxyServer proxy = ProxyServer.createTestServer(new MockConfigSource(new MockClientUpdater()));
+ ProxyServer proxy = ProxyServer.createTestServer(new MockConfigSource());
Spec spec = new Spec("localhost", 12345);
ConfigProxyRpcServer server = new ConfigProxyRpcServer(proxy, new Supervisor(new Transport()), spec);
assertThat(server.getSpec(), is(spec));
diff --git a/config-proxy/src/test/java/com/yahoo/vespa/config/proxy/DelayedResponseHandlerTest.java b/config-proxy/src/test/java/com/yahoo/vespa/config/proxy/DelayedResponseHandlerTest.java
index 3ac1af6c4d8..af83a86db41 100644
--- a/config-proxy/src/test/java/com/yahoo/vespa/config/proxy/DelayedResponseHandlerTest.java
+++ b/config-proxy/src/test/java/com/yahoo/vespa/config/proxy/DelayedResponseHandlerTest.java
@@ -14,7 +14,7 @@ import static org.junit.Assert.assertThat;
*/
public class DelayedResponseHandlerTest {
- private final MockConfigSource source = new MockConfigSource(new MockClientUpdater());
+ private final MockConfigSource source = new MockConfigSource();
@Rule
public TemporaryFolder temporaryFolder = new TemporaryFolder();
@@ -28,8 +28,7 @@ public class DelayedResponseHandlerTest {
@Test
public void basic() {
ConfigTester tester = new ConfigTester();
- ConfigProxyStatistics statistics = new ConfigProxyStatistics();
- DelayedResponses delayedResponses = new DelayedResponses(statistics);
+ DelayedResponses delayedResponses = new DelayedResponses();
final MockRpcServer mockRpcServer = new MockRpcServer();
final MemoryCache memoryCache = new MemoryCache();
memoryCache.update(ConfigTester.fooConfig);
diff --git a/config-proxy/src/test/java/com/yahoo/vespa/config/proxy/DelayedResponsesTest.java b/config-proxy/src/test/java/com/yahoo/vespa/config/proxy/DelayedResponsesTest.java
index 4ea5f796ff5..3f960d2c37c 100644
--- a/config-proxy/src/test/java/com/yahoo/vespa/config/proxy/DelayedResponsesTest.java
+++ b/config-proxy/src/test/java/com/yahoo/vespa/config/proxy/DelayedResponsesTest.java
@@ -8,14 +8,13 @@ import static org.junit.Assert.assertThat;
/**
* @author hmusum
- * @since 5.1.9
*/
public class DelayedResponsesTest {
@Test
public void basic() throws InterruptedException {
ConfigTester tester = new ConfigTester();
- DelayedResponses responses = new DelayedResponses(new ConfigProxyStatistics());
+ DelayedResponses responses = new DelayedResponses();
DelayedResponse delayedResponse = new DelayedResponse(tester.createRequest("foo", "id", "bar", 10));
responses.add(delayedResponse);
diff --git a/config-proxy/src/test/java/com/yahoo/vespa/config/proxy/MockClientUpdater.java b/config-proxy/src/test/java/com/yahoo/vespa/config/proxy/MockClientUpdater.java
deleted file mode 100644
index f399d1bb085..00000000000
--- a/config-proxy/src/test/java/com/yahoo/vespa/config/proxy/MockClientUpdater.java
+++ /dev/null
@@ -1,52 +0,0 @@
-// Copyright 2017 Yahoo Holdings. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
-package com.yahoo.vespa.config.proxy;
-
-import com.yahoo.vespa.config.ConfigKey;
-import com.yahoo.vespa.config.RawConfig;
-
-import java.time.Duration;
-import java.time.Instant;
-
-class MockClientUpdater extends ClientUpdater {
- private RawConfig lastConfig;
-
- MockClientUpdater() {
- this(new ConfigProxyStatistics());
- }
-
- private MockClientUpdater(ConfigProxyStatistics statistics) {
- super(new MockRpcServer(), statistics, new DelayedResponses(statistics));
- }
-
- @Override
- public synchronized void updateSubscribers(RawConfig newConfig) {
- lastConfig = newConfig;
- }
-
- synchronized RawConfig getLastConfig() {
- return lastConfig;
- }
-
- long waitForConfigGeneration(ConfigKey<?> configKey, long expectedGeneration) {
- Instant end = Instant.now().plus(Duration.ofSeconds(60));
- RawConfig lastConfig;
- do {
- lastConfig = getLastConfig();
- System.out.println("config=" + lastConfig + (lastConfig == null ? "" : ",generation=" + lastConfig.getGeneration()));
- if (lastConfig != null && lastConfig.getKey().equals(configKey) &&
- lastConfig.getGeneration() == expectedGeneration) {
- break;
- } else {
- try {
- Thread.sleep(10);
- } catch (InterruptedException e) {
- throw new RuntimeException(e);
- }
- }
- } while (Instant.now().isBefore(end));
-
- if (lastConfig == null || ! lastConfig.getKey().equals(configKey) || lastConfig.getGeneration() != expectedGeneration)
- throw new RuntimeException("Did not get config " + configKey + " with generation " + expectedGeneration);
- return lastConfig.getGeneration();
- }
-}
diff --git a/config-proxy/src/test/java/com/yahoo/vespa/config/proxy/MockConfigSource.java b/config-proxy/src/test/java/com/yahoo/vespa/config/proxy/MockConfigSource.java
index 2b26996fbdc..3457cbcf576 100644
--- a/config-proxy/src/test/java/com/yahoo/vespa/config/proxy/MockConfigSource.java
+++ b/config-proxy/src/test/java/com/yahoo/vespa/config/proxy/MockConfigSource.java
@@ -5,7 +5,6 @@ import com.yahoo.config.subscription.ConfigSourceSet;
import com.yahoo.vespa.config.ConfigKey;
import com.yahoo.vespa.config.RawConfig;
-import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.Set;
@@ -15,20 +14,12 @@ import java.util.Set;
* source.
*
* @author hmusum
- * @since 5.1.10
*/
class MockConfigSource extends ConfigSourceSet {
private final HashMap<ConfigKey<?>, RawConfig> backing = new HashMap<>();
- private final ClientUpdater clientUpdater;
- MockConfigSource(ClientUpdater clientUpdater) {
- this.clientUpdater = clientUpdater;
- }
-
- MockConfigSource put(ConfigKey<?> key, RawConfig config) {
+ void put(ConfigKey<?> key, RawConfig config) {
backing.put(key, config);
- clientUpdater.updateSubscribers(config);
- return this;
}
RawConfig getConfig(ConfigKey<?> key) {
diff --git a/config-proxy/src/test/java/com/yahoo/vespa/config/proxy/MockConfigSourceClient.java b/config-proxy/src/test/java/com/yahoo/vespa/config/proxy/MockConfigSourceClient.java
index 5a15ade6968..963c922d5b5 100644
--- a/config-proxy/src/test/java/com/yahoo/vespa/config/proxy/MockConfigSourceClient.java
+++ b/config-proxy/src/test/java/com/yahoo/vespa/config/proxy/MockConfigSourceClient.java
@@ -51,4 +51,9 @@ public class MockConfigSourceClient implements ConfigSourceClient{
public List<String> getSourceConnections() {
return Collections.singletonList("N/A");
}
+
+ @Override
+ public void updateSubscribers(RawConfig config) {
+
+ }
}
diff --git a/config-proxy/src/test/java/com/yahoo/vespa/config/proxy/ProxyServerTest.java b/config-proxy/src/test/java/com/yahoo/vespa/config/proxy/ProxyServerTest.java
index 19732e4b5b5..9d6d0ca2a39 100644
--- a/config-proxy/src/test/java/com/yahoo/vespa/config/proxy/ProxyServerTest.java
+++ b/config-proxy/src/test/java/com/yahoo/vespa/config/proxy/ProxyServerTest.java
@@ -21,8 +21,7 @@ import static org.junit.Assert.*;
public class ProxyServerTest {
private final MemoryCache memoryCache = new MemoryCache();
- private MockClientUpdater clientUpdater = new MockClientUpdater();
- private final MockConfigSource source = new MockConfigSource(clientUpdater);
+ private final MockConfigSource source = new MockConfigSource();
private MockConfigSourceClient client = new MockConfigSourceClient(source, memoryCache);
private final ConfigProxyStatistics statistics = new ConfigProxyStatistics();
private ProxyServer proxy;
@@ -227,25 +226,6 @@ public class ProxyServerTest {
}
@Test
- public void testReconfigurationAsClient() {
- long generation = 1;
- RawConfig fooConfig = ConfigTester.fooConfig;
- source.put(fooConfig.getKey(), fooConfig);
-
- clientUpdater.waitForConfigGeneration(fooConfig.getKey(), generation);
- assertThat(clientUpdater.getLastConfig(), is(fooConfig));
-
- // Update payload in config
- generation++;
- final ConfigPayload payload = ConfigTester.createConfigPayload("bar", "value2");
- RawConfig fooConfig2 = createConfigWithNextConfigGeneration(fooConfig, 0, Payload.from(payload));
- source.put(fooConfig2.getKey(), fooConfig2);
-
- clientUpdater.waitForConfigGeneration(fooConfig2.getKey(), generation);
- assertFalse(clientUpdater.getLastConfig().equals(fooConfig));
- }
-
- @Test
public void testReadingSystemProperties() {
ProxyServer.Properties properties = ProxyServer.getSystemProperties();
assertThat(properties.eventInterval, is(ConfigProxyStatistics.defaultEventInterval));
diff --git a/config-proxy/src/test/java/com/yahoo/vespa/config/proxy/ClientUpdaterTest.java b/config-proxy/src/test/java/com/yahoo/vespa/config/proxy/RpcConfigSourceClientTest.java
index 25140afd4cc..7f762955b92 100644
--- a/config-proxy/src/test/java/com/yahoo/vespa/config/proxy/ClientUpdaterTest.java
+++ b/config-proxy/src/test/java/com/yahoo/vespa/config/proxy/RpcConfigSourceClientTest.java
@@ -15,12 +15,12 @@ import static org.junit.Assert.assertEquals;
/**
* @author hmusum
*/
-public class ClientUpdaterTest {
+public class RpcConfigSourceClientTest {
private MockRpcServer rpcServer;
private ConfigProxyStatistics statistics;
private DelayedResponses delayedResponses;
- private ClientUpdater clientUpdater;
+ private RpcConfigSourceClient rpcConfigSourceClient;
@Rule
public TemporaryFolder tempFolder = new TemporaryFolder();
@@ -30,8 +30,10 @@ public class ClientUpdaterTest {
public void setup() {
rpcServer = new MockRpcServer();
statistics = new ConfigProxyStatistics();
- delayedResponses = new DelayedResponses(statistics);
- clientUpdater = new ClientUpdater(rpcServer, statistics, delayedResponses);
+ delayedResponses = new DelayedResponses();
+ rpcConfigSourceClient =
+ new RpcConfigSourceClient(rpcServer, new MockConfigSource(), statistics,
+ new MemoryCache(), ProxyServer.defaultTimingValues(), delayedResponses);
}
@Test
@@ -103,7 +105,7 @@ public class ClientUpdaterTest {
}
private void configUpdatedSendResponse(RawConfig config) {
- clientUpdater.updateSubscribers(config);
+ rpcConfigSourceClient.updateSubscribers(config);
}
private RawConfig createConfigWithNextConfigGeneration(RawConfig config) {