summaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
authorØyvind Grønnesby <oyving@verizonmedia.com>2019-07-17 10:33:47 +0200
committerGitHub <noreply@github.com>2019-07-17 10:33:47 +0200
commitc183d42c7255bab7c9d05d5a87bcbd4646761c32 (patch)
tree0daa9885cf3bc3f3984b40a613bb59cd0f622286
parent8f62c7d636a04c0b2e18b735548f76cfedc813ae (diff)
parent7eaaa6eb110da07a2d0e01b0e29183043e7fcddd (diff)
Merge pull request #9396 from vespa-engine/olaa/cfg-server-metric-aggregation
olaa/cfg server metric aggregation
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/ApplicationRepository.java57
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/http/v2/ApplicationHandler.java10
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/http/v2/MetricsResponse.java56
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/metrics/ClusterInfo.java44
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/metrics/MetricsAggregator.java86
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/metrics/MetricsRetriever.java89
-rw-r--r--configserver/src/main/resources/configserver-app/services.xml1
-rw-r--r--configserver/src/test/java/com/yahoo/vespa/config/server/metrics/MetricsRetrieverTest.java99
-rw-r--r--configserver/src/test/resources/metrics/container_metrics41
-rw-r--r--configserver/src/test/resources/metrics/content_metrics16
-rw-r--r--controller-api/src/main/java/com/yahoo/vespa/hosted/controller/api/application/v4/model/ClusterMetrics.java40
-rw-r--r--controller-api/src/main/java/com/yahoo/vespa/hosted/controller/api/integration/configserver/ConfigServer.java3
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/LockedApplication.java2
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/DeploymentMetricsMaintainer.java4
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/persistence/ApplicationSerializer.java1
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/integration/ConfigServerMock.java6
16 files changed, 545 insertions, 10 deletions
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/ApplicationRepository.java b/configserver/src/main/java/com/yahoo/vespa/config/server/ApplicationRepository.java
index 091620a72f9..f65eaaf3fa3 100644
--- a/configserver/src/main/java/com/yahoo/vespa/config/server/ApplicationRepository.java
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/ApplicationRepository.java
@@ -42,7 +42,11 @@ import com.yahoo.vespa.config.server.deploy.Deployment;
import com.yahoo.vespa.config.server.deploy.InfraDeployerProvider;
import com.yahoo.vespa.config.server.http.LogRetriever;
import com.yahoo.vespa.config.server.http.SimpleHttpFetcher;
+import com.yahoo.vespa.config.server.http.v2.MetricsResponse;
import com.yahoo.vespa.config.server.http.v2.PrepareResult;
+import com.yahoo.vespa.config.server.metrics.ClusterInfo;
+import com.yahoo.vespa.config.server.metrics.MetricsAggregator;
+import com.yahoo.vespa.config.server.metrics.MetricsRetriever;
import com.yahoo.vespa.config.server.provision.HostProvisionerProvider;
import com.yahoo.vespa.config.server.session.LocalSession;
import com.yahoo.vespa.config.server.session.LocalSessionRepo;
@@ -67,10 +71,14 @@ import java.nio.file.attribute.BasicFileAttributes;
import java.time.Clock;
import java.time.Duration;
import java.time.Instant;
+import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
+import java.util.HashMap;
import java.util.HashSet;
+import java.util.LinkedHashMap;
import java.util.List;
+import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.logging.Level;
@@ -80,6 +88,7 @@ import java.util.stream.Collectors;
import static com.yahoo.config.model.api.container.ContainerServiceType.CLUSTERCONTROLLER_CONTAINER;
import static com.yahoo.config.model.api.container.ContainerServiceType.CONTAINER;
import static com.yahoo.config.model.api.container.ContainerServiceType.LOGSERVER_CONTAINER;
+import static com.yahoo.config.model.api.container.ContainerServiceType.METRICS_PROXY_CONTAINER;
import static com.yahoo.vespa.config.server.tenant.TenantRepository.HOSTED_VESPA_TENANT;
import static java.nio.file.Files.readAttributes;
@@ -633,6 +642,21 @@ public class ApplicationRepository implements com.yahoo.config.provision.Deploye
return tenantRepository.getTenant(tenantName).getApplicationRepo().activeApplications();
}
+ // ---------------- Metrics ------------------------------------------------------------------------
+
+ public MetricsResponse getMetrics(ApplicationId applicationId) {
+ var metricsRetriever = new MetricsRetriever();
+ var clusters = getClustersOfApplication(applicationId);
+ var clusterMetrics = new LinkedHashMap<ClusterInfo, MetricsAggregator>();
+
+ clusters.forEach(cluster -> {
+ var metrics = metricsRetriever.requestMetricsForCluster(cluster);
+ clusterMetrics.put(cluster, metrics);
+ });
+
+ return new MetricsResponse(200, applicationId, clusterMetrics);
+ }
+
// ---------------- Misc operations ----------------------------------------------------------------
public ApplicationMetaData getMetadataFromSession(Tenant tenant, long sessionId) {
@@ -750,18 +774,41 @@ public class ApplicationRepository implements com.yahoo.config.provision.Deploye
.anyMatch(serviceInfo -> serviceInfo.getServiceType().equalsIgnoreCase("logserver")))
.findFirst().orElseThrow(() -> new IllegalArgumentException("Could not find HostInfo for LogServer"));
- ServiceInfo containerServiceInfo = logServerHostInfo.getServices().stream()
- .filter(service -> List.of(LOGSERVER_CONTAINER.serviceName, CONTAINER.serviceName).contains(service.getServiceType()))
+ ServiceInfo serviceInfo = logServerHostInfo.getServices().stream().filter(service -> List.of(LOGSERVER_CONTAINER.serviceName, CONTAINER.serviceName).contains(service.getServiceType()))
.findFirst().orElseThrow(() -> new IllegalArgumentException("No container running on logserver host"));
+ int port = servicePort(serviceInfo);
+ return "http://" + logServerHostInfo.getHostname() + ":" + port + "/logs";
+ }
- int port = containerServiceInfo.getPorts().stream()
+ private int servicePort(ServiceInfo serviceInfo) {
+ int port = serviceInfo.getPorts().stream()
.filter(portInfo -> portInfo.getTags().stream().anyMatch(tag -> tag.equalsIgnoreCase("http")))
.findFirst().orElseThrow(() -> new IllegalArgumentException("Could not find HTTP port"))
.getPort();
-
- return "http://" + logServerHostInfo.getHostname() + ":" + port + "/logs";
+ return port;
}
+ /** Finds the hosts of an application, grouped by cluster name */
+ private Collection<ClusterInfo> getClustersOfApplication(ApplicationId applicationId) {
+ Application application = getApplication(applicationId);
+ Map<String, List<URI>> clusterHosts = new HashMap<>();
+ Map<String, ClusterInfo> clusters = new HashMap<>();
+ application.getModel().getHosts().stream()
+ .filter(host -> host.getServices().stream().noneMatch(serviceInfo -> serviceInfo.getServiceType().equalsIgnoreCase("logserver")))
+ .forEach(hostInfo -> {
+ ServiceInfo serviceInfo = hostInfo.getServices().stream().filter(service -> METRICS_PROXY_CONTAINER.serviceName.equals(service.getServiceType()))
+ .findFirst().orElseThrow(() -> new IllegalArgumentException("Unable to find services " + METRICS_PROXY_CONTAINER.serviceName.toString()));
+ String clusterName = serviceInfo.getProperty("clusterid").orElse("");
+ String clusterTypeString = serviceInfo.getProperty("clustertype").orElse("");
+ ClusterInfo.ClusterType clusterType = ClusterInfo.ClusterType.valueOf(clusterTypeString);
+ URI host = URI.create("http://" + hostInfo.getHostname() + ":" + servicePort(serviceInfo) + "/metrics/v1/values");
+ clusterHosts.computeIfAbsent(clusterName, l -> new ArrayList<URI>()).add(host);
+ clusters.computeIfAbsent(clusterName, c -> new ClusterInfo(clusterName, clusterType)).addHost(host);
+ }
+ );
+ return clusters.values();
+
+ }
/** Returns version to use when deploying application in given environment */
static Version decideVersion(ApplicationId application, Environment environment, Version sessionVersion, boolean bootstrap) {
if ( environment.isManuallyDeployed()
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/http/v2/ApplicationHandler.java b/configserver/src/main/java/com/yahoo/vespa/config/server/http/v2/ApplicationHandler.java
index 865805d1258..e18c6ad6c56 100644
--- a/configserver/src/main/java/com/yahoo/vespa/config/server/http/v2/ApplicationHandler.java
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/http/v2/ApplicationHandler.java
@@ -104,6 +104,10 @@ public class ApplicationHandler extends HttpHandler {
return applicationRepository.getLogs(applicationId, hostname, apiParams);
}
+ if (isMetricsRequest(request)) {
+ return applicationRepository.getMetrics(applicationId);
+ }
+
if (isIsSuspendedRequest(request)) {
return new ApplicationSuspendedResponse(applicationRepository.isSuspended(applicationId));
}
@@ -144,6 +148,7 @@ public class ApplicationHandler extends HttpHandler {
"http://*/application/v2/tenant/*/application/*/environment/*/region/*/instance/*/serviceconverge",
"http://*/application/v2/tenant/*/application/*/environment/*/region/*/instance/*/serviceconverge/*",
"http://*/application/v2/tenant/*/application/*/environment/*/region/*/instance/*/clustercontroller/*/status/*",
+ "http://*/application/v2/tenant/*/application/*/environment/*/region/*/instance/*/metrics",
"http://*/application/v2/tenant/*/application/*/environment/*/region/*/instance/*",
"http://*/application/v2/tenant/*/application/*/logs",
"http://*/application/v2/tenant/*/application/*");
@@ -154,6 +159,11 @@ public class ApplicationHandler extends HttpHandler {
request.getUri().getPath().endsWith("/suspended");
}
+ private static boolean isMetricsRequest(HttpRequest request) {
+ return getBindingMatch(request).groupCount() == 7 &&
+ request.getUri().getPath().endsWith("/metrics");
+ }
+
private static boolean isLogRequest(HttpRequest request) {
return getBindingMatch(request).groupCount() == 4 &&
request.getUri().getPath().endsWith("/logs");
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/http/v2/MetricsResponse.java b/configserver/src/main/java/com/yahoo/vespa/config/server/http/v2/MetricsResponse.java
new file mode 100644
index 00000000000..88971433a01
--- /dev/null
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/http/v2/MetricsResponse.java
@@ -0,0 +1,56 @@
+// Copyright 2019 Oath Inc. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+package com.yahoo.vespa.config.server.http.v2;
+
+import com.yahoo.config.provision.ApplicationId;
+import com.yahoo.container.jdisc.HttpResponse;
+import com.yahoo.slime.Cursor;
+import com.yahoo.slime.JsonFormat;
+import com.yahoo.slime.Slime;
+import com.yahoo.vespa.config.server.http.HttpConfigResponse;
+import com.yahoo.vespa.config.server.metrics.ClusterInfo;
+import com.yahoo.vespa.config.server.metrics.MetricsAggregator;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.Map;
+
+/**
+ * @author olaa
+ */
+public class MetricsResponse extends HttpResponse {
+
+ private final Slime slime = new Slime();
+
+ public MetricsResponse(int status, ApplicationId applicationId, Map<ClusterInfo, MetricsAggregator> aggregatedMetrics) {
+ super(status);
+
+ Cursor application = slime.setObject();
+ application.setString("applicationId", applicationId.serializedForm());
+
+ Cursor clusters = application.setArray("clusters");
+
+ for (var entry : aggregatedMetrics.entrySet()) {
+ Cursor cluster = clusters.addObject();
+ cluster.setString("clusterId", entry.getKey().getClusterId());
+ cluster.setString("clusterType", entry.getKey().getClusterType().name());
+
+ MetricsAggregator aggregator = entry.getValue();
+ Cursor metrics = cluster.setObject("metrics");
+ aggregator.aggregateQueryRate().ifPresent(queryrate -> metrics.setDouble("queriesPerSecond", queryrate));
+ aggregator.aggregateFeedRate().ifPresent(feedRate -> metrics.setDouble("feedPerSecond", feedRate));
+ aggregator.aggregateDocumentCount().ifPresent(documentCount -> metrics.setDouble("documentCount", documentCount));
+ aggregator.aggregateQueryLatency().ifPresent(queryLatency -> metrics.setDouble("queryLatency",queryLatency));
+ aggregator.aggregateFeedLatency().ifPresent(feedLatency -> metrics.setDouble("feedLatency", feedLatency));
+ }
+ }
+
+ @Override
+ public void render(OutputStream outputStream) throws IOException {
+ new JsonFormat(false).encode(outputStream, slime);
+ }
+
+ @Override
+ public String getContentType() {
+ return HttpConfigResponse.JSON_CONTENT_TYPE;
+ }
+}
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/metrics/ClusterInfo.java b/configserver/src/main/java/com/yahoo/vespa/config/server/metrics/ClusterInfo.java
new file mode 100644
index 00000000000..ef9a73fedd4
--- /dev/null
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/metrics/ClusterInfo.java
@@ -0,0 +1,44 @@
+// Copyright 2019 Oath Inc. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+package com.yahoo.vespa.config.server.metrics;
+
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * @author olaa
+ */
+public class ClusterInfo {
+
+ private final String clusterId;
+ private final ClusterType clusterType;
+ private final List<URI> hostnames;
+
+ public ClusterInfo(String clusterId, ClusterType clusterType) {
+ this(clusterId, clusterType, new ArrayList<>());
+ }
+
+ public ClusterInfo(String clusterId, ClusterType clusterType, List<URI> hostnames) {
+ this.clusterId = clusterId;
+ this.clusterType = clusterType;
+ this.hostnames = hostnames;
+ }
+
+ public String getClusterId() {
+ return clusterId;
+ }
+
+ public ClusterType getClusterType() {
+ return clusterType;
+ }
+
+ public List<URI> getHostnames() {
+ return hostnames;
+ }
+
+ public void addHost(URI host) {
+ hostnames.add(host);
+ }
+
+ public enum ClusterType {content, container};
+}
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/metrics/MetricsAggregator.java b/configserver/src/main/java/com/yahoo/vespa/config/server/metrics/MetricsAggregator.java
new file mode 100644
index 00000000000..c6b2131863d
--- /dev/null
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/metrics/MetricsAggregator.java
@@ -0,0 +1,86 @@
+// Copyright 2019 Oath Inc. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+package com.yahoo.vespa.config.server.metrics;
+
+import java.time.Instant;
+import java.util.Optional;
+
+/**
+ * @author olaa
+ * @author ogronnesby
+ */
+public class MetricsAggregator {
+
+ private LatencyMetrics feed;
+ private LatencyMetrics qr;
+ private LatencyMetrics container;
+ private Double documentCount;
+ private Instant timestamp;
+
+ public MetricsAggregator addFeedLatency(double sum, double count) {
+ this.feed = combineLatency(this.feed, sum, count);
+ return this;
+ }
+
+ public MetricsAggregator addQrLatency(double sum, double count) {
+ this.qr = combineLatency(this.qr, sum, count);
+ return this;
+ }
+
+ public MetricsAggregator addContainerLatency(double sum, double count) {
+ this.container = combineLatency(this.container, sum, count);
+ return this;
+ }
+
+ public MetricsAggregator addDocumentCount(double count) {
+ this.documentCount = (this.documentCount == null ? 0.0 : this.documentCount) + count;
+ return this;
+ }
+
+ public MetricsAggregator setTimestamp(Instant timestamp) {
+ this.timestamp = timestamp;
+ return this;
+ }
+
+ public Optional<Double> aggregateFeedLatency() {
+ return Optional.ofNullable(feed).map(m -> m.latencySum / m.latencyCount);
+
+ }
+
+ public Optional<Double> aggregateFeedRate() {
+ return Optional.ofNullable(feed).map(m -> m.latencyCount / 60);
+ }
+
+ public Optional<Double> aggregateQueryLatency() {
+ if (container == null && qr == null) return Optional.empty();
+ var c = Optional.ofNullable(container).orElseGet(LatencyMetrics::new);
+ var q = Optional.ofNullable(qr).orElseGet(LatencyMetrics::new);
+ return Optional.of((c.latencySum + q.latencySum) / (c.latencyCount + q.latencyCount));
+ }
+
+ public Optional<Double> aggregateQueryRate() {
+ if (container == null && qr == null) return Optional.empty();
+ var c = Optional.ofNullable(container).orElseGet(LatencyMetrics::new);
+ var q = Optional.ofNullable(qr).orElseGet(LatencyMetrics::new);
+ return Optional.of((c.latencyCount + q.latencyCount) / 60);
+ }
+
+ public Optional<Double> aggregateDocumentCount() {
+ return Optional.ofNullable(documentCount);
+ }
+
+ public Instant getTimestamp() {
+ return timestamp;
+ }
+
+ private LatencyMetrics combineLatency(LatencyMetrics metricsOrNull, double sum, double count) {
+ var metrics = Optional.ofNullable(metricsOrNull).orElseGet(LatencyMetrics::new);
+ metrics.latencyCount += count;
+ metrics.latencySum += sum;
+ return metrics;
+ }
+
+ private static class LatencyMetrics {
+ double latencySum;
+ double latencyCount;
+ }
+}
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/metrics/MetricsRetriever.java b/configserver/src/main/java/com/yahoo/vespa/config/server/metrics/MetricsRetriever.java
new file mode 100644
index 00000000000..0881d32b21e
--- /dev/null
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/metrics/MetricsRetriever.java
@@ -0,0 +1,89 @@
+// Copyright 2019 Oath Inc. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+package com.yahoo.vespa.config.server.metrics;
+
+import com.yahoo.slime.ArrayTraverser;
+import com.yahoo.slime.Inspector;
+import com.yahoo.slime.Slime;
+import com.yahoo.vespa.config.SlimeUtils;
+import org.apache.http.HttpResponse;
+import org.apache.http.client.HttpClient;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.impl.client.HttpClientBuilder;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.UncheckedIOException;
+import java.net.URI;
+import java.time.Instant;
+
+
+/**
+ * Client for reaching out to nodes in an application instance and get their
+ * metrics.
+ *
+ * @author olaa
+ * @author ogronnesby
+ */
+public class MetricsRetriever {
+ private final HttpClient httpClient = HttpClientBuilder.create().build();
+
+ /**
+ * Call the metrics API on each host in the cluster and aggregate the metrics
+ * into a single value.
+ */
+ public MetricsAggregator requestMetricsForCluster(ClusterInfo clusterInfo) {
+ var aggregator = new MetricsAggregator();
+ clusterInfo.getHostnames().forEach(host -> getHostMetrics(host, aggregator));
+ return aggregator;
+ }
+
+ private void getHostMetrics(URI hostURI, MetricsAggregator metrics) {
+ Slime responseBody = doMetricsRequest(hostURI);
+ Inspector services = responseBody.get().field("services");
+ services.traverse((ArrayTraverser) (i, servicesInspector) -> {
+ parseService(servicesInspector, metrics);
+ });
+ }
+
+ private Slime doMetricsRequest(URI hostURI) {
+ HttpGet get = new HttpGet(hostURI);
+ try {
+ HttpResponse response = httpClient.execute(get);
+ InputStream is = response.getEntity().getContent();
+ Slime slime = SlimeUtils.jsonToSlime(is.readAllBytes());
+ is.close();
+ return slime;
+ } catch (IOException e) {
+ throw new UncheckedIOException(e);
+ }
+ }
+
+ private void parseService(Inspector service, MetricsAggregator metrics) {
+ String serviceName = service.field("name").asString();
+ Instant timestamp = Instant.ofEpochSecond(service.field("timestamp").asLong());
+ metrics.setTimestamp(timestamp);
+ service.field("metrics").traverse((ArrayTraverser) (i, m) -> {
+ Inspector values = m.field("values");
+ switch (serviceName) {
+ case "container":
+ metrics.addContainerLatency(
+ values.field("query_latency.sum").asDouble(),
+ values.field("query_latency.count").asDouble());
+ metrics.addFeedLatency(
+ values.field("feed_latency.sum").asDouble(),
+ values.field("feed_latency.count").asDouble());
+ break;
+ case "qrserver":
+ metrics.addQrLatency(
+ values.field("query_latency.sum").asDouble(),
+ values.field("query_latency.count").asDouble());
+ break;
+ case "distributor":
+ metrics.addDocumentCount(values.field("vds.distributor.docsstored.average").asDouble());
+ break;
+ }
+ });
+
+ }
+
+}
diff --git a/configserver/src/main/resources/configserver-app/services.xml b/configserver/src/main/resources/configserver-app/services.xml
index e57024413c9..97b2156e8ca 100644
--- a/configserver/src/main/resources/configserver-app/services.xml
+++ b/configserver/src/main/resources/configserver-app/services.xml
@@ -125,6 +125,7 @@
<binding>http://*/application/v2/tenant/*/application/*/environment/*/region/*/instance/*/serviceconverge</binding>
<binding>http://*/application/v2/tenant/*/application/*/environment/*/region/*/instance/*/serviceconverge/*</binding>
<binding>http://*/application/v2/tenant/*/application/*/environment/*/region/*/instance/*/clustercontroller/*/status/*</binding>
+ <binding>http://*/application/v2/tenant/*/application/*/environment/*/region/*/instance/*/metrics</binding>
<binding>http://*/application/v2/tenant/*/application/*/environment/*/region/*/instance/*</binding>
<binding>http://*/application/v2/tenant/*/application/*</binding>
<binding>http://*/application/v2/tenant/*/application/*/logs</binding>
diff --git a/configserver/src/test/java/com/yahoo/vespa/config/server/metrics/MetricsRetrieverTest.java b/configserver/src/test/java/com/yahoo/vespa/config/server/metrics/MetricsRetrieverTest.java
new file mode 100644
index 00000000000..1b878a432c9
--- /dev/null
+++ b/configserver/src/test/java/com/yahoo/vespa/config/server/metrics/MetricsRetrieverTest.java
@@ -0,0 +1,99 @@
+package com.yahoo.vespa.config.server.metrics;
+
+import com.github.tomakehurst.wiremock.junit.WireMockRule;
+import junit.framework.AssertionFailedError;
+import org.junit.Rule;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.net.URI;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.List;
+import java.util.Optional;
+import java.util.function.BiConsumer;
+
+import static com.github.tomakehurst.wiremock.client.WireMock.aResponse;
+import static com.github.tomakehurst.wiremock.client.WireMock.get;
+import static com.github.tomakehurst.wiremock.client.WireMock.stubFor;
+import static com.github.tomakehurst.wiremock.client.WireMock.urlEqualTo;
+import static com.github.tomakehurst.wiremock.core.WireMockConfiguration.options;
+import static org.junit.Assert.*;
+
+
+/**
+ * @author olaa
+ */
+public class MetricsRetrieverTest {
+
+ @Rule
+ public final WireMockRule wireMock = new WireMockRule(options().port(8080), true);
+
+ @Test
+ public void testMetricAggregation() throws IOException {
+ var metricsRetriever = new MetricsRetriever();
+
+ var clusters = List.of(
+ new ClusterInfo("cluster1", ClusterInfo.ClusterType.content, List.of(URI.create("http://localhost:8080/1"), URI.create("http://localhost:8080/2"))),
+ new ClusterInfo("cluster2", ClusterInfo.ClusterType.container, List.of(URI.create("http://localhost:8080/3")))
+ );
+
+ stubFor(get(urlEqualTo("/1"))
+ .willReturn(aResponse()
+ .withStatus(200)
+ .withBody(contentMetrics())));
+
+ stubFor(get(urlEqualTo("/2"))
+ .willReturn(aResponse()
+ .withStatus(200)
+ .withBody(contentMetrics())));
+
+ stubFor(get(urlEqualTo("/3"))
+ .willReturn(aResponse()
+ .withStatus(200)
+ .withBody(containerMetrics())));
+
+ compareAggregators(
+ new MetricsAggregator().addDocumentCount(6000.0),
+ metricsRetriever.requestMetricsForCluster(clusters.get(0))
+ );
+
+ compareAggregators(
+ new MetricsAggregator()
+ .addContainerLatency(3000, 43)
+ .addContainerLatency(2000, 0)
+ .addQrLatency(3000, 43)
+ .addFeedLatency(3000, 43),
+ metricsRetriever.requestMetricsForCluster(clusters.get(1))
+ );
+
+ wireMock.stop();
+ }
+
+ private String containerMetrics() throws IOException {
+ return Files.readString(Path.of("src/test/resources/metrics/container_metrics"));
+ }
+
+ private String contentMetrics() throws IOException {
+ return Files.readString(Path.of("src/test/resources/metrics/content_metrics"));
+ }
+
+ // Same tolerance value as used internally in MetricsAggregator.isZero
+ private static final double metricsTolerance = 0.001;
+
+ private void compareAggregators(MetricsAggregator expected, MetricsAggregator actual) {
+ BiConsumer<Double, Double> assertDoubles = (a, b) -> assertEquals(a.doubleValue(), b.doubleValue(), metricsTolerance);
+
+ compareOptionals(expected.aggregateDocumentCount(), actual.aggregateDocumentCount(), assertDoubles);
+ compareOptionals(expected.aggregateQueryRate(), actual.aggregateQueryRate(), assertDoubles);
+ compareOptionals(expected.aggregateFeedRate(), actual.aggregateFeedRate(), assertDoubles);
+ compareOptionals(expected.aggregateQueryLatency(), actual.aggregateQueryLatency(), assertDoubles);
+ compareOptionals(expected.aggregateFeedLatency(), actual.aggregateFeedLatency(), assertDoubles);
+ }
+
+ @SuppressWarnings("OptionalUsedAsFieldOrParameterType")
+ private static <T> void compareOptionals(Optional<T> a, Optional<T> b, BiConsumer<T, T> comparer) {
+ if (a.isPresent() != b.isPresent()) throw new AssertionFailedError("Both optionals are not present: " + a + ", " + b);
+ a.ifPresent(x -> b.ifPresent(y -> comparer.accept(x, y)));
+ }
+} \ No newline at end of file
diff --git a/configserver/src/test/resources/metrics/container_metrics b/configserver/src/test/resources/metrics/container_metrics
new file mode 100644
index 00000000000..09232fe0f93
--- /dev/null
+++ b/configserver/src/test/resources/metrics/container_metrics
@@ -0,0 +1,41 @@
+{
+ "services": [
+ {
+ "name":"container",
+ "timestamp": 1557306075,
+ "metrics": [
+ {
+ "values": {
+ "queries.rate": 23.0,
+ "query_latency.sum": 2000,
+ "document.count": 300000,
+ "feed.rate": 23.0,
+ "write_latency.sum": 2000
+ }
+ },
+ {
+ "values": {
+ "query_latency.count": 43.0,
+ "query_latency.sum": 3000,
+ "feed_latency.count": 43.0,
+ "feed_latency.sum": 3000
+
+ }
+ }
+ ]
+ },
+
+ {
+ "name":"qrserver",
+ "timestamp": 1557306075,
+ "metrics": [
+ {
+ "values": {
+ "query_latency.count": 43.0,
+ "query_latency.sum": 3000
+ }
+ }
+ ]
+ }
+ ]
+} \ No newline at end of file
diff --git a/configserver/src/test/resources/metrics/content_metrics b/configserver/src/test/resources/metrics/content_metrics
new file mode 100644
index 00000000000..a239aeea5ca
--- /dev/null
+++ b/configserver/src/test/resources/metrics/content_metrics
@@ -0,0 +1,16 @@
+{
+ "services": [
+ {
+ "name":"distributor",
+ "timestamp": 1557306075,
+ "metrics": [
+ {
+ "values": {
+ "vds.distributor.docsstored.average": 3000
+
+ }
+ }
+ ]
+ }
+ ]
+} \ No newline at end of file
diff --git a/controller-api/src/main/java/com/yahoo/vespa/hosted/controller/api/application/v4/model/ClusterMetrics.java b/controller-api/src/main/java/com/yahoo/vespa/hosted/controller/api/application/v4/model/ClusterMetrics.java
new file mode 100644
index 00000000000..1377a333335
--- /dev/null
+++ b/controller-api/src/main/java/com/yahoo/vespa/hosted/controller/api/application/v4/model/ClusterMetrics.java
@@ -0,0 +1,40 @@
+// Copyright 2019 Oath Inc. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+package com.yahoo.vespa.hosted.controller.api.application.v4.model;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * @author olaa
+ */
+public class ClusterMetrics {
+
+ private final String clusterId;
+ private final ClusterType clusterType;
+ private final Map<String, Double> metrics;
+
+ public ClusterMetrics(String clusterId, ClusterType clusterType) {
+ this.clusterId = clusterId;
+ this.clusterType = clusterType;
+ this.metrics = new HashMap<>();
+ }
+
+ public String getClusterId() {
+ return clusterId;
+ }
+
+ public ClusterType getClusterType() {
+ return clusterType;
+ }
+
+ public Map<String, Double> getMetrics() {
+ return Collections.unmodifiableMap(metrics);
+ }
+
+ public void addMetric(String name, double value) {
+ metrics.put(name, value);
+ }
+
+ public enum ClusterType {content, container};
+}
diff --git a/controller-api/src/main/java/com/yahoo/vespa/hosted/controller/api/integration/configserver/ConfigServer.java b/controller-api/src/main/java/com/yahoo/vespa/hosted/controller/api/integration/configserver/ConfigServer.java
index 9eae2965c45..688cf275892 100644
--- a/controller-api/src/main/java/com/yahoo/vespa/hosted/controller/api/integration/configserver/ConfigServer.java
+++ b/controller-api/src/main/java/com/yahoo/vespa/hosted/controller/api/integration/configserver/ConfigServer.java
@@ -6,6 +6,7 @@ import com.yahoo.config.provision.ApplicationId;
import com.yahoo.config.provision.zone.ZoneId;
import com.yahoo.vespa.hosted.controller.api.application.v4.model.DeployOptions;
import com.yahoo.vespa.hosted.controller.api.application.v4.model.EndpointStatus;
+import com.yahoo.vespa.hosted.controller.api.application.v4.model.ClusterMetrics;
import com.yahoo.vespa.hosted.controller.api.identifiers.DeploymentId;
import com.yahoo.vespa.hosted.controller.api.identifiers.Hostname;
import com.yahoo.vespa.hosted.controller.api.integration.certificates.ApplicationCertificate;
@@ -44,6 +45,8 @@ public interface ConfigServer {
InputStream getLogs(DeploymentId deployment, Map<String, String> queryParameters);
+ List<ClusterMetrics> getMetrics(DeploymentId deployment);
+
List<String> getContentClusters(DeploymentId deployment);
/**
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/LockedApplication.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/LockedApplication.java
index 294dc10d0bd..8722c3defeb 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/LockedApplication.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/LockedApplication.java
@@ -25,7 +25,6 @@ import com.yahoo.vespa.hosted.controller.application.DeploymentJobs;
import com.yahoo.vespa.hosted.controller.application.DeploymentMetrics;
import com.yahoo.vespa.hosted.controller.application.JobStatus;
import com.yahoo.vespa.hosted.controller.application.RotationStatus;
-import com.yahoo.vespa.hosted.controller.rotation.RotationId;
import java.time.Instant;
import java.util.LinkedHashMap;
@@ -279,7 +278,6 @@ public class LockedApplication {
metrics, pemDeployKey, rotations, rotationStatus, applicationCertificate);
}
-
/** Don't expose non-leaf sub-objects. */
private LockedApplication with(Deployment deployment) {
Map<ZoneId, Deployment> deployments = new LinkedHashMap<>(this.deployments);
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/DeploymentMetricsMaintainer.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/DeploymentMetricsMaintainer.java
index 4ad5940f8f2..ab587fc4078 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/DeploymentMetricsMaintainer.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/DeploymentMetricsMaintainer.java
@@ -41,8 +41,7 @@ public class DeploymentMetricsMaintainer extends Maintainer {
private final ApplicationController applications;
public DeploymentMetricsMaintainer(Controller controller, Duration duration, JobControl jobControl) {
- super(controller, duration, jobControl, DeploymentMetricsMaintainer.class.getSimpleName(),
- SystemName.allOf(Predicate.not(SystemName::isPublic)));
+ super(controller, duration, jobControl, DeploymentMetricsMaintainer.class.getSimpleName(), SystemName.all());
this.applications = controller.applications();
}
@@ -122,5 +121,4 @@ public class DeploymentMetricsMaintainer extends Maintainer {
default: throw new IllegalArgumentException("Unknown API value for rotation status: " + status);
}
}
-
}
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/persistence/ApplicationSerializer.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/persistence/ApplicationSerializer.java
index d49587963dc..c1962e8d17c 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/persistence/ApplicationSerializer.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/persistence/ApplicationSerializer.java
@@ -109,6 +109,7 @@ public class ApplicationSerializer {
private final String lastWrittenField = "lastWritten";
private final String lastQueriesPerSecondField = "lastQueriesPerSecond";
private final String lastWritesPerSecondField = "lastWritesPerSecond";
+ private final String clusterMetricsField = "clusterMetrics";
// DeploymentJobs fields
private final String projectIdField = "projectId";
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/integration/ConfigServerMock.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/integration/ConfigServerMock.java
index a89c5988396..fbc7bf20a24 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/integration/ConfigServerMock.java
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/integration/ConfigServerMock.java
@@ -11,6 +11,7 @@ import com.yahoo.config.provision.zone.ZoneId;
import com.yahoo.vespa.hosted.controller.api.application.v4.model.DeployOptions;
import com.yahoo.vespa.hosted.controller.api.application.v4.model.EndpointStatus;
import com.yahoo.vespa.hosted.controller.api.application.v4.model.configserverbindings.ConfigChangeActions;
+import com.yahoo.vespa.hosted.controller.api.application.v4.model.ClusterMetrics;
import com.yahoo.vespa.hosted.controller.api.identifiers.DeploymentId;
import com.yahoo.vespa.hosted.controller.api.identifiers.Hostname;
import com.yahoo.vespa.hosted.controller.api.identifiers.Identifier;
@@ -330,6 +331,11 @@ public class ConfigServerMock extends AbstractComponent implements ConfigServer
return applicationView;
}
+ @Override
+ public List<ClusterMetrics> getMetrics(DeploymentId deployment) {
+ return List.of();
+ }
+
// Returns a canned example response
@Override
public Map<?,?> getServiceApiResponse(String tenantName, String applicationName, String instanceName,