summaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
-rw-r--r--clustercontroller-apps/src/main/java/com/yahoo/vespa/clustercontroller/apps/clustercontroller/ClusterController.java5
-rw-r--r--clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/FleetController.java16
-rw-r--r--clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/NodeLookup.java7
-rw-r--r--clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/rpc/SlobrokClient.java5
-rw-r--r--clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/status/StatusHandler.java9
-rw-r--r--clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/status/statuspage/StatusPageServerInterface.java12
-rw-r--r--clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/ClusterFeedBlockTest.java4
-rw-r--r--clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/DatabaseTest.java18
-rw-r--r--clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/DummyCommunicator.java4
-rw-r--r--clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/DummyVdsNode.java34
-rw-r--r--clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/FleetControllerTest.java16
-rw-r--r--clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/StateChangeTest.java4
-rw-r--r--config-model/src/main/java/com/yahoo/vespa/model/admin/monitoring/VespaMetricSet.java7
-rw-r--r--container-search/src/main/java/com/yahoo/prelude/statistics/StatisticsSearcher.java3
-rw-r--r--controller-api/src/main/java/com/yahoo/vespa/hosted/controller/api/integration/billing/BillingController.java1
-rw-r--r--controller-api/src/main/java/com/yahoo/vespa/hosted/controller/api/integration/billing/MockBillingController.java5
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/billing/BillingApiHandler.java6
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/billing/BillingApiHandlerV2.java7
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/billing/BillingApiHandlerTest.java4
-rw-r--r--flags/src/main/java/com/yahoo/vespa/flags/Flags.java17
-rw-r--r--searchcore/src/vespa/searchcore/proton/common/scheduled_forward_executor.cpp34
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/disk_mem_usage_sampler.cpp2
-rw-r--r--vespaclient/CMakeLists.txt1
-rw-r--r--vespaclient/src/vespa/vespaclient/vdsstates/.gitignore5
-rw-r--r--vespaclient/src/vespa/vespaclient/vdsstates/CMakeLists.txt8
-rw-r--r--vespaclient/src/vespa/vespaclient/vdsstates/statesapp.cpp463
26 files changed, 87 insertions, 610 deletions
diff --git a/clustercontroller-apps/src/main/java/com/yahoo/vespa/clustercontroller/apps/clustercontroller/ClusterController.java b/clustercontroller-apps/src/main/java/com/yahoo/vespa/clustercontroller/apps/clustercontroller/ClusterController.java
index 4395240acd3..1ac870c45de 100644
--- a/clustercontroller-apps/src/main/java/com/yahoo/vespa/clustercontroller/apps/clustercontroller/ClusterController.java
+++ b/clustercontroller-apps/src/main/java/com/yahoo/vespa/clustercontroller/apps/clustercontroller/ClusterController.java
@@ -53,10 +53,9 @@ public class ClusterController extends AbstractComponent
synchronized (controllers) {
FleetController controller = controllers.get(options.clusterName());
if (controller == null) {
- StatusHandler.ContainerStatusPageServer statusPageServer = new StatusHandler.ContainerStatusPageServer();
- controller = FleetController.create(options, statusPageServer, metricWrapper);
+ controller = FleetController.create(options, metricWrapper);
controllers.put(options.clusterName(), controller);
- status.put(options.clusterName(), statusPageServer);
+ status.put(options.clusterName(), controller.statusPageServer());
} else {
controller.updateOptions(options);
}
diff --git a/clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/FleetController.java b/clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/FleetController.java
index 3a9ebde9da1..2952d3bf9ee 100644
--- a/clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/FleetController.java
+++ b/clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/FleetController.java
@@ -21,9 +21,9 @@ import com.yahoo.vespa.clustercontroller.core.status.ClusterStateRequestHandler;
import com.yahoo.vespa.clustercontroller.core.status.LegacyIndexPageRequestHandler;
import com.yahoo.vespa.clustercontroller.core.status.LegacyNodePageRequestHandler;
import com.yahoo.vespa.clustercontroller.core.status.NodeHealthRequestHandler;
+import com.yahoo.vespa.clustercontroller.core.status.StatusHandler;
import com.yahoo.vespa.clustercontroller.core.status.statuspage.StatusPageResponse;
import com.yahoo.vespa.clustercontroller.core.status.statuspage.StatusPageServer;
-import com.yahoo.vespa.clustercontroller.core.status.statuspage.StatusPageServerInterface;
import com.yahoo.vespa.clustercontroller.utils.util.MetricReporter;
import java.io.FileNotFoundException;
import java.time.Duration;
@@ -35,7 +35,6 @@ import java.util.HashSet;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
-import java.util.Objects;
import java.util.Queue;
import java.util.Set;
import java.util.TimeZone;
@@ -61,7 +60,7 @@ public class FleetController implements NodeListener, SlobrokListener, SystemSta
private final StateChangeHandler stateChangeHandler;
private final SystemStateBroadcaster systemStateBroadcaster;
private final StateVersionTracker stateVersionTracker;
- private final StatusPageServerInterface statusPageServer;
+ private final StatusHandler.ContainerStatusPageServer statusPageServer;
private final RpcServer rpcServer;
private final DatabaseHandler database;
private final MasterElectionHandler masterElectionHandler;
@@ -106,7 +105,6 @@ public class FleetController implements NodeListener, SlobrokListener, SystemSta
ContentCluster cluster,
NodeStateGatherer nodeStateGatherer,
Communicator communicator,
- StatusPageServerInterface statusPage,
RpcServer server,
NodeLookup nodeLookup,
DatabaseHandler database,
@@ -130,7 +128,7 @@ public class FleetController implements NodeListener, SlobrokListener, SystemSta
this.systemStateBroadcaster = systemStateBroadcaster;
this.stateVersionTracker = new StateVersionTracker(options.minMergeCompletionRatio());
this.metricUpdater = metricUpdater;
- this.statusPageServer = Objects.requireNonNull(statusPage, "statusPage cannot be null");
+ this.statusPageServer = new StatusHandler.ContainerStatusPageServer();
this.rpcServer = server;
this.masterElectionHandler = masterElectionHandler;
this.statusRequestRouter.addHandler(
@@ -150,9 +148,7 @@ public class FleetController implements NodeListener, SlobrokListener, SystemSta
propagateOptions();
}
- public static FleetController create(FleetControllerOptions options,
- StatusPageServerInterface statusPageServer,
- MetricReporter metricReporter) throws Exception {
+ public static FleetController create(FleetControllerOptions options, MetricReporter metricReporter) throws Exception {
var context = new FleetControllerContextImpl(options);
var timer = new RealTimer();
var metricUpdater = new MetricUpdater(metricReporter, options.fleetControllerIndex(), options.clusterName());
@@ -173,7 +169,7 @@ public class FleetController implements NodeListener, SlobrokListener, SystemSta
var stateBroadcaster = new SystemStateBroadcaster(context, timer, timer);
var masterElectionHandler = new MasterElectionHandler(context, options.fleetControllerIndex(), options.fleetControllerCount(), timer, timer);
var controller = new FleetController(context, timer, log, cluster, stateGatherer, communicator,
- statusPageServer, null, lookUp, database, stateGenerator,
+ null, lookUp, database, stateGenerator,
stateBroadcaster, masterElectionHandler, metricUpdater, options);
controller.start();
return controller;
@@ -1215,4 +1211,6 @@ public class FleetController implements NodeListener, SlobrokListener, SystemSta
return eventLog;
}
+ public StatusHandler.ContainerStatusPageServer statusPageServer() { return statusPageServer; }
+
}
diff --git a/clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/NodeLookup.java b/clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/NodeLookup.java
index 882ae8894fa..b0e7cafd396 100644
--- a/clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/NodeLookup.java
+++ b/clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/NodeLookup.java
@@ -12,11 +12,4 @@ public interface NodeLookup {
boolean updateCluster(ContentCluster cluster, SlobrokListener listener);
- /**
- * Returns whether the lookup instance has been able to bootstrap itself with information about nodes.
- *
- * Calling updateCluster() _before_ isReady has returned true may not provide any useful data.
- */
- boolean isReady();
-
}
diff --git a/clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/rpc/SlobrokClient.java b/clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/rpc/SlobrokClient.java
index d87e0e20908..559690e99e2 100644
--- a/clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/rpc/SlobrokClient.java
+++ b/clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/rpc/SlobrokClient.java
@@ -80,11 +80,6 @@ public class SlobrokClient implements NodeLookup {
public Mirror getMirror() { return mirror; }
@Override
- public boolean isReady() {
- return mirror != null && mirror.ready();
- }
-
- @Override
public boolean updateCluster(ContentCluster cluster, SlobrokListener listener) {
if (mirror == null) return false;
diff --git a/clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/status/StatusHandler.java b/clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/status/StatusHandler.java
index 79e6a91f561..302832e4542 100644
--- a/clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/status/StatusHandler.java
+++ b/clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/status/StatusHandler.java
@@ -3,11 +3,9 @@ package com.yahoo.vespa.clustercontroller.core.status;
import com.yahoo.vespa.clustercontroller.core.status.statuspage.StatusPageResponse;
import com.yahoo.vespa.clustercontroller.core.status.statuspage.StatusPageServer;
-import com.yahoo.vespa.clustercontroller.core.status.statuspage.StatusPageServerInterface;
import com.yahoo.vespa.clustercontroller.utils.communication.http.HttpRequest;
import com.yahoo.vespa.clustercontroller.utils.communication.http.HttpRequestHandler;
import com.yahoo.vespa.clustercontroller.utils.communication.http.HttpResult;
-
import java.io.IOException;
import java.io.StringWriter;
import java.nio.charset.StandardCharsets;
@@ -27,7 +25,7 @@ public class StatusHandler implements HttpRequestHandler {
}
- public static class ContainerStatusPageServer implements StatusPageServerInterface {
+ public static class ContainerStatusPageServer {
StatusPageServer.HttpRequest request;
StatusPageResponse response;
@@ -36,13 +34,9 @@ public class StatusHandler implements HttpRequestHandler {
// Lock safety with fleetcontroller. Wait until completion
private final Object answerMonitor = new Object();
- @Override
public int getPort() { return 0; }
- @Override
public void shutdown() throws InterruptedException, IOException {}
- @Override
public void setPort(int port) {}
- @Override
public StatusPageServer.HttpRequest getCurrentHttpRequest() {
synchronized (answerMonitor) {
StatusPageServer.HttpRequest r = request;
@@ -50,7 +44,6 @@ public class StatusHandler implements HttpRequestHandler {
return r;
}
}
- @Override
public void answerCurrentStatusRequest(StatusPageResponse r) {
synchronized (answerMonitor) {
response = r;
diff --git a/clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/status/statuspage/StatusPageServerInterface.java b/clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/status/statuspage/StatusPageServerInterface.java
deleted file mode 100644
index a06d069c59d..00000000000
--- a/clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/status/statuspage/StatusPageServerInterface.java
+++ /dev/null
@@ -1,12 +0,0 @@
-// Copyright Yahoo. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
-package com.yahoo.vespa.clustercontroller.core.status.statuspage;
-
-public interface StatusPageServerInterface {
-
- int getPort();
- void shutdown() throws InterruptedException, java.io.IOException;
- void setPort(int port) throws java.io.IOException, InterruptedException;
- StatusPageServer.HttpRequest getCurrentHttpRequest();
- void answerCurrentStatusRequest(StatusPageResponse r);
-
-}
diff --git a/clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/ClusterFeedBlockTest.java b/clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/ClusterFeedBlockTest.java
index 0485bd80aa0..413c8e7414c 100644
--- a/clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/ClusterFeedBlockTest.java
+++ b/clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/ClusterFeedBlockTest.java
@@ -7,7 +7,6 @@ import com.yahoo.vdslib.state.NodeType;
import com.yahoo.vdslib.state.State;
import com.yahoo.vespa.clustercontroller.core.database.DatabaseHandler;
import com.yahoo.vespa.clustercontroller.core.database.ZooKeeperDatabaseFactory;
-import com.yahoo.vespa.clustercontroller.core.status.StatusHandler;
import com.yahoo.vespa.clustercontroller.utils.util.NoMetricReporter;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.ExtendWith;
@@ -51,8 +50,7 @@ public class ClusterFeedBlockTest extends FleetControllerTest {
var stateGenerator = new StateChangeHandler(context, timer, eventLog);
var stateBroadcaster = new SystemStateBroadcaster(context, timer, timer);
var masterElectionHandler = new MasterElectionHandler(context, options.fleetControllerIndex(), options.fleetControllerCount(), timer, timer);
- var status = new StatusHandler.ContainerStatusPageServer();
- ctrl = new FleetController(context, timer, eventLog, cluster, stateGatherer, communicator, status, null, communicator, database,
+ ctrl = new FleetController(context, timer, eventLog, cluster, stateGatherer, communicator, null, communicator, database,
stateGenerator, stateBroadcaster, masterElectionHandler, metricUpdater, options);
ctrl.tick();
diff --git a/clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/DatabaseTest.java b/clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/DatabaseTest.java
index 666b4f63801..a66294851b8 100644
--- a/clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/DatabaseTest.java
+++ b/clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/DatabaseTest.java
@@ -1,12 +1,7 @@
// Copyright Yahoo. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
package com.yahoo.vespa.clustercontroller.core;
-import com.yahoo.jrt.ErrorCode;
-import com.yahoo.jrt.Request;
-import com.yahoo.jrt.Spec;
-import com.yahoo.jrt.StringValue;
import com.yahoo.jrt.Supervisor;
-import com.yahoo.jrt.Target;
import com.yahoo.jrt.Transport;
import com.yahoo.vdslib.state.Node;
import com.yahoo.vdslib.state.NodeState;
@@ -21,7 +16,6 @@ import java.util.TreeMap;
import java.util.logging.Logger;
import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertTrue;
@ExtendWith(CleanupZookeeperLogsOnSuccess.class)
public class DatabaseTest extends FleetControllerTest {
@@ -156,18 +150,8 @@ public class DatabaseTest extends FleetControllerTest {
}
}
- // Note: different semantics than FleetControllerTest.setWantedState
private void setWantedState(Node n, NodeState ns, Map<Node, NodeState> wantedStates) {
- int rpcPort = fleetController().getRpcPort();
- Target connection = supervisor.connect(new Spec("localhost", rpcPort));
- assertTrue(connection.isValid());
-
- Request req = new Request("setNodeState");
- req.parameters().add(new StringValue("storage/cluster.mycluster/" + n.getType().toString() + "/" + n.getIndex()));
- req.parameters().add(new StringValue(ns.serialize(true)));
- connection.invokeSync(req, timeout());
- assertEquals(ErrorCode.NONE, req.errorCode(), req.toString());
- assertTrue(req.checkReturnTypes("s"), req.toString());
+ setWantedState(ns, ns.getDescription(), "storage/cluster.mycluster/" + n.getType().toString() + "/" + n.getIndex(), supervisor);
wantedStates.put(n, ns);
}
diff --git a/clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/DummyCommunicator.java b/clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/DummyCommunicator.java
index 3127201a342..b82d4a135ae 100644
--- a/clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/DummyCommunicator.java
+++ b/clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/DummyCommunicator.java
@@ -154,8 +154,4 @@ public class DummyCommunicator implements Communicator, NodeLookup {
return false;
}
- @Override
- public boolean isReady() {
- return true;
- }
}
diff --git a/clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/DummyVdsNode.java b/clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/DummyVdsNode.java
index 36fe5321788..f2494ea0301 100644
--- a/clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/DummyVdsNode.java
+++ b/clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/DummyVdsNode.java
@@ -31,7 +31,6 @@ import java.util.Optional;
import java.util.TreeMap;
import java.util.logging.Level;
import java.util.logging.Logger;
-import java.util.stream.Collectors;
/**
*
@@ -63,11 +62,11 @@ public class DummyVdsNode {
static class Req {
Request request;
- long timeout;
+ long timeToReply;
- Req(Request r, long timeout) {
+ Req(Request r, long timeToReply) {
request = r;
- this.timeout = timeout;
+ this.timeToReply = timeToReply;
}
}
static class BackOff implements BackOffPolicy {
@@ -76,6 +75,8 @@ public class DummyVdsNode {
public boolean shouldWarn(double v) { return false; }
public boolean shouldInform(double v) { return false; }
}
+
+ /** List of requests that should be replied to after a specified time */
private final List<Req> waitingRequests = new LinkedList<>();
/**
@@ -94,7 +95,7 @@ public class DummyVdsNode {
long currentTime = timer.getCurrentTimeInMillis();
for (Iterator<Req> it = waitingRequests.iterator(); it.hasNext(); ) {
Req r = it.next();
- if (r.timeout <= currentTime) {
+ if (currentTime >= r.timeToReply) {
log.log(Level.FINE, () -> "Dummy node " + DummyVdsNode.this + ": Responding to node state request at time " + currentTime);
r.request.returnValues().add(new StringValue(nodeState.serialize()));
if (r.request.methodName().equals("getnodestate3")) {
@@ -294,10 +295,11 @@ public class DummyVdsNode {
m.returnDesc(0, "returnCode", "Returncode of request. Should be 0 = OK");
supervisor.addMethod(m);
- m = new Method("getnodestate3", "sii", "ss", this::rpc_getNodeState2);
+ m = new Method("getnodestate3", "sii", "ss", this::rpc_getNodeState3);
m.methodDesc("Get nodeState of a node, answer when state changes from given state.");
m.paramDesc(0, "nodeStateIn", "The node state of the given node");
m.paramDesc(1, "timeout", "Time timeout in milliseconds set by the state requester.");
+ m.paramDesc(2, "index", "Node index.");
m.returnDesc(0, "nodeStateOut", "The node state of the given node");
m.returnDesc(1, "hostinfo", "Information on the host node is running on");
supervisor.addMethod(m);
@@ -339,31 +341,27 @@ public class DummyVdsNode {
return false;
}
- private void rpc_getNodeState2(Request req) {
+ private void rpc_getNodeState3(Request req) {
log.log(Level.FINE, () -> "Dummy node " + this + ": Got " + req.methodName() + " request");
try{
String oldState = req.parameters().get(0).asString();
int timeout = req.parameters().get(1).asInt32();
- int index = -1;
- if (req.parameters().size() > 2) {
- index = req.parameters().get(2).asInt32();
- }
+ int index = req.parameters().get(2).asInt32();
synchronized(timer) {
boolean sentReply = sendGetNodeStateReply(index);
NodeState givenState = (oldState.equals("unknown") ? null : NodeState.deserialize(type, oldState));
if (givenState != null && (givenState.equals(nodeState) || sentReply)) {
- log.log(Level.FINE, () -> "Dummy node " + this + ": Has same state as reported " + givenState + ". Queing request. Timeout is " + timeout + " ms. "
- + "Will be answered at time " + (timer.getCurrentTimeInMillis() + timeout * 800L / 1000));
+ long timeToReply = timer.getCurrentTimeInMillis() + timeout * 800L / 1000;
+ log.log(Level.FINE, () -> "Dummy node " + this + " has same state as reported (" + givenState + "). Queuing request. Timeout is " + timeout + " ms. "
+ + "Will be answered at time " + timeToReply);
req.detach();
- waitingRequests.add(new Req(req, timer.getCurrentTimeInMillis() + timeout * 800L / 1000));
- log.log(Level.FINE, () -> "Dummy node " + this + " has now " + waitingRequests.size() + " entries and is " + (waitingRequests.isEmpty() ? "empty" : "not empty"));
+ waitingRequests.add(new Req(req, timeToReply));
+ log.log(Level.FINE, () -> "Dummy node " + this + " has " + waitingRequests.size() + " requests waiting to be answered");
timer.notifyAll();
} else {
log.log(Level.FINE, () -> "Dummy node " + this + ": Request had " + (givenState == null ? "no state" : "different state(" + givenState +")") + ". Answering with " + nodeState);
req.returnValues().add(new StringValue(nodeState.serialize()));
- if (req.methodName().equals("getnodestate3")) {
- req.returnValues().add(new StringValue(hostInfo));
- }
+ req.returnValues().add(new StringValue(hostInfo));
++immediateStateReplies;
}
}
diff --git a/clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/FleetControllerTest.java b/clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/FleetControllerTest.java
index e8b56cbc142..07401534a7b 100644
--- a/clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/FleetControllerTest.java
+++ b/clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/FleetControllerTest.java
@@ -19,7 +19,6 @@ import com.yahoo.vespa.clustercontroller.core.database.ZooKeeperDatabaseFactory;
import com.yahoo.vespa.clustercontroller.core.rpc.RPCCommunicator;
import com.yahoo.vespa.clustercontroller.core.rpc.RpcServer;
import com.yahoo.vespa.clustercontroller.core.rpc.SlobrokClient;
-import com.yahoo.vespa.clustercontroller.core.status.StatusHandler;
import com.yahoo.vespa.clustercontroller.core.testutils.WaitCondition;
import com.yahoo.vespa.clustercontroller.core.testutils.WaitTask;
import com.yahoo.vespa.clustercontroller.core.testutils.Waiter;
@@ -138,8 +137,7 @@ public abstract class FleetControllerTest implements Waiter {
var stateBroadcaster = new SystemStateBroadcaster(context, timer, timer);
var masterElectionHandler = new MasterElectionHandler(context, options.fleetControllerIndex(), options.fleetControllerCount(), timer, timer);
- var status = new StatusHandler.ContainerStatusPageServer();
- var controller = new FleetController(context, timer, log, cluster, stateGatherer, communicator, status, rpcServer, lookUp,
+ var controller = new FleetController(context, timer, log, cluster, stateGatherer, communicator, rpcServer, lookUp,
database, stateGenerator, stateBroadcaster, masterElectionHandler, metricUpdater, options);
controller.start();
return controller;
@@ -298,12 +296,16 @@ public abstract class FleetControllerTest implements Waiter {
}
void setWantedState(DummyVdsNode node, State state, String reason, Supervisor supervisor) {
- NodeState ns = new NodeState(node.getType(), state);
- if (reason != null) ns.setDescription(reason);
+ setWantedState(new NodeState(node.getType(), state), reason, node.getSlobrokName(), supervisor);
+ }
+
+ void setWantedState(NodeState nodeState, String reason, String slobrokName, Supervisor supervisor) {
+ if (reason != null) nodeState.setDescription(reason);
Target connection = supervisor.connect(new Spec("localhost", fleetController().getRpcPort()));
+
Request req = new Request("setNodeState");
- req.parameters().add(new StringValue(node.getSlobrokName()));
- req.parameters().add(new StringValue(ns.serialize()));
+ req.parameters().add(new StringValue(slobrokName));
+ req.parameters().add(new StringValue(nodeState.serialize()));
connection.invokeSync(req, timeout());
if (req.isError()) {
fail("Failed to invoke setNodeState(): " + req.errorCode() + ": " + req.errorMessage());
diff --git a/clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/StateChangeTest.java b/clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/StateChangeTest.java
index 5868ad723a4..7d2cc9b8df2 100644
--- a/clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/StateChangeTest.java
+++ b/clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/StateChangeTest.java
@@ -9,7 +9,6 @@ import com.yahoo.vdslib.state.NodeType;
import com.yahoo.vdslib.state.State;
import com.yahoo.vespa.clustercontroller.core.database.DatabaseHandler;
import com.yahoo.vespa.clustercontroller.core.database.ZooKeeperDatabaseFactory;
-import com.yahoo.vespa.clustercontroller.core.status.StatusHandler;
import com.yahoo.vespa.clustercontroller.core.testutils.StateWaiter;
import com.yahoo.vespa.clustercontroller.utils.util.NoMetricReporter;
import org.junit.jupiter.api.Test;
@@ -52,8 +51,7 @@ public class StateChangeTest extends FleetControllerTest {
var stateGenerator = new StateChangeHandler(context, timer, eventLog);
var stateBroadcaster = new SystemStateBroadcaster(context, timer, timer);
var masterElectionHandler = new MasterElectionHandler(context, options.fleetControllerIndex(), options.fleetControllerCount(), timer, timer);
- var status = new StatusHandler.ContainerStatusPageServer();
- ctrl = new FleetController(context, timer, eventLog, cluster, stateGatherer, communicator, status, null, communicator, database,
+ ctrl = new FleetController(context, timer, eventLog, cluster, stateGatherer, communicator, null, communicator, database,
stateGenerator, stateBroadcaster, masterElectionHandler, metricUpdater, options);
ctrl.tick();
diff --git a/config-model/src/main/java/com/yahoo/vespa/model/admin/monitoring/VespaMetricSet.java b/config-model/src/main/java/com/yahoo/vespa/model/admin/monitoring/VespaMetricSet.java
index 3d2c3faa985..348c006e8a6 100644
--- a/config-model/src/main/java/com/yahoo/vespa/model/admin/monitoring/VespaMetricSet.java
+++ b/config-model/src/main/java/com/yahoo/vespa/model/admin/monitoring/VespaMetricSet.java
@@ -56,7 +56,6 @@ public class VespaMetricSet {
metrics.add(new Metric("slobrok.heartbeats.failed.count"));
metrics.add(new Metric("logd.processed.lines.count"));
metrics.add(new Metric("worker.connections.max"));
- metrics.add(new Metric("routing.config.is-stale"));
metrics.add(new Metric("endpoint.certificate.expiry.seconds"));
// Java (JRT) TLS metrics
@@ -327,6 +326,12 @@ public class VespaMetricSet {
metrics.add(new Metric("query_latency.count"));
metrics.add(new Metric("query_latency.95percentile"));
metrics.add(new Metric("query_latency.99percentile"));
+ metrics.add(new Metric("query_timeout.min"));
+ metrics.add(new Metric("query_timeout.max"));
+ metrics.add(new Metric("query_timeout.sum"));
+ metrics.add(new Metric("query_timeout.count"));
+ metrics.add(new Metric("query_timeout.95percentile"));
+ metrics.add(new Metric("query_timeout.99percentile"));
metrics.add(new Metric("failed_queries.rate"));
metrics.add(new Metric("degraded_queries.rate"));
metrics.add(new Metric("hits_per_query.max"));
diff --git a/container-search/src/main/java/com/yahoo/prelude/statistics/StatisticsSearcher.java b/container-search/src/main/java/com/yahoo/prelude/statistics/StatisticsSearcher.java
index 09f15b4c5c2..28b6d64a94e 100644
--- a/container-search/src/main/java/com/yahoo/prelude/statistics/StatisticsSearcher.java
+++ b/container-search/src/main/java/com/yahoo/prelude/statistics/StatisticsSearcher.java
@@ -63,6 +63,7 @@ public class StatisticsSearcher extends Searcher {
private static final String FAILED_QUERIES_METRIC = "failed_queries";
private static final String MEAN_QUERY_LATENCY_METRIC = "mean_query_latency";
private static final String QUERY_LATENCY_METRIC = "query_latency";
+ private static final String QUERY_TIMEOUT_METRIC = "query_timeout";
private static final String QUERY_HIT_OFFSET_METRIC = "query_hit_offset";
private static final String QUERIES_METRIC = "queries";
private static final String PEAK_QPS_METRIC = "peak_qps";
@@ -125,6 +126,7 @@ public class StatisticsSearcher extends Searcher {
this.peakQpsReporter = new PeakQpsReporter();
this.metric = metric;
+ metricReceiver.declareGauge(QUERY_TIMEOUT_METRIC, Optional.empty(), new MetricSettings.Builder().histogram(true).build());
metricReceiver.declareGauge(QUERY_LATENCY_METRIC, Optional.empty(), new MetricSettings.Builder().histogram(true).build());
metricReceiver.declareGauge(HITS_PER_QUERY_METRIC, Optional.empty(), new MetricSettings.Builder().histogram(true).build());
metricReceiver.declareGauge(TOTALHITS_PER_QUERY_METRIC, Optional.empty(), new MetricSettings.Builder().histogram(true).build());
@@ -223,6 +225,7 @@ public class StatisticsSearcher extends Searcher {
logQuery(query);
long start_ns = getStartNanoTime(query);
qps(metricContext);
+ metric.set(QUERY_TIMEOUT_METRIC, query.getTimeout(), metricContext);
Result result;
//handle exceptions thrown below in searchers
try {
diff --git a/controller-api/src/main/java/com/yahoo/vespa/hosted/controller/api/integration/billing/BillingController.java b/controller-api/src/main/java/com/yahoo/vespa/hosted/controller/api/integration/billing/BillingController.java
index 40bc9b27449..8b0f58c79d2 100644
--- a/controller-api/src/main/java/com/yahoo/vespa/hosted/controller/api/integration/billing/BillingController.java
+++ b/controller-api/src/main/java/com/yahoo/vespa/hosted/controller/api/integration/billing/BillingController.java
@@ -62,6 +62,7 @@ public interface BillingController {
* @return The ID of the new bill.
*/
Bill.Id createBillForPeriod(TenantName tenant, ZonedDateTime startTime, ZonedDateTime endTime, String agent);
+ Bill.Id createBillForPeriod(TenantName tenant, LocalDate startDate, LocalDate endDate, String agent);
/**
* Create an unpersisted bill of unbilled use for the given tenant from the end of last bill until the given date.
diff --git a/controller-api/src/main/java/com/yahoo/vespa/hosted/controller/api/integration/billing/MockBillingController.java b/controller-api/src/main/java/com/yahoo/vespa/hosted/controller/api/integration/billing/MockBillingController.java
index c9f9b7f50e4..6ea4c7442d8 100644
--- a/controller-api/src/main/java/com/yahoo/vespa/hosted/controller/api/integration/billing/MockBillingController.java
+++ b/controller-api/src/main/java/com/yahoo/vespa/hosted/controller/api/integration/billing/MockBillingController.java
@@ -77,6 +77,11 @@ public class MockBillingController implements BillingController {
}
@Override
+ public Bill.Id createBillForPeriod(TenantName tenant, LocalDate startDate, LocalDate endDate, String agent) {
+ return createBillForPeriod(tenant, startDate.atStartOfDay(ZoneOffset.UTC), endDate.plusDays(1).atStartOfDay(ZoneOffset.UTC), agent);
+ }
+
+ @Override
public Bill createUncommittedBill(TenantName tenant, LocalDate until) {
return uncommittedBills.getOrDefault(tenant, emptyBill());
}
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/billing/BillingApiHandler.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/billing/BillingApiHandler.java
index d1962b16120..307880682d9 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/billing/BillingApiHandler.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/billing/BillingApiHandler.java
@@ -246,10 +246,8 @@ public class BillingApiHandler extends ThreadedHttpRequestHandler {
LocalDate startDate = LocalDate.parse(getInspectorFieldOrThrow(inspector, "startTime"));
LocalDate endDate = LocalDate.parse(getInspectorFieldOrThrow(inspector, "endTime"));
- ZonedDateTime startTime = startDate.atStartOfDay(ZoneId.of("UTC"));
- ZonedDateTime endTime = endDate.plusDays(1).atStartOfDay(ZoneId.of("UTC"));
- var billId = billingController.createBillForPeriod(tenantName, startTime, endTime, userId);
+ var billId = billingController.createBillForPeriod(tenantName, startDate, endDate, userId);
Slime slime = new Slime();
Cursor root = slime.setObject();
@@ -476,7 +474,7 @@ public class BillingApiHandler extends ThreadedHttpRequestHandler {
private LocalDate untilParameter(String until) {
if (until == null || until.isEmpty() || until.isBlank())
- return LocalDate.now().plusDays(1);
+ return LocalDate.now();
return LocalDate.parse(until);
}
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/billing/BillingApiHandlerV2.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/billing/BillingApiHandlerV2.java
index 8722e588fa7..0ddaa409ef8 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/billing/BillingApiHandlerV2.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/billing/BillingApiHandlerV2.java
@@ -345,12 +345,7 @@ public class BillingApiHandlerV2 extends RestApiRequestHandler<BillingApiHandler
private LocalDate untilParameter(RestApi.RequestContext ctx) {
return ctx.queryParameters().getString("until")
.map(LocalDate::parse)
- .map(date -> date.plusDays(1))
- .orElseGet(this::tomorrow);
- }
-
- private LocalDate tomorrow() {
- return LocalDate.now(clock).plusDays(1);
+ .orElseGet(() -> LocalDate.now(clock));
}
private static String getInspectorFieldOrThrow(Inspector inspector, String field) {
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/billing/BillingApiHandlerTest.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/billing/BillingApiHandlerTest.java
index 73d4daf92da..79007a4439a 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/billing/BillingApiHandlerTest.java
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/billing/BillingApiHandlerTest.java
@@ -114,8 +114,8 @@ public class BillingApiHandlerTest extends ControllerContainerCloudTest {
bills = billingController.getBillsForTenant(tenant);
assertEquals(1, bills.size());
Bill bill = bills.get(0);
- assertEquals("2020-04-20T00:00Z[UTC]", bill.getStartTime().toString());
- assertEquals("2020-05-21T00:00Z[UTC]", bill.getEndTime().toString());
+ assertEquals("2020-04-20T00:00Z", bill.getStartTime().toString());
+ assertEquals("2020-05-21T00:00Z", bill.getEndTime().toString());
assertEquals("2020-04-20", bill.getStartDate().toString());
assertEquals("2020-05-20", bill.getEndDate().toString());
diff --git a/flags/src/main/java/com/yahoo/vespa/flags/Flags.java b/flags/src/main/java/com/yahoo/vespa/flags/Flags.java
index d8756a22506..e3021570953 100644
--- a/flags/src/main/java/com/yahoo/vespa/flags/Flags.java
+++ b/flags/src/main/java/com/yahoo/vespa/flags/Flags.java
@@ -226,7 +226,7 @@ public class Flags {
public static final UnboundBooleanFlag ENABLED_HORIZON_DASHBOARD = defineFeatureFlag(
"enabled-horizon-dashboard", false,
- List.of("olaa"), "2021-09-13", "2023-01-01",
+ List.of("olaa"), "2021-09-13", "2023-06-01",
"Enable Horizon dashboard",
"Takes effect immediately",
TENANT_ID, CONSOLE_USER_EMAIL
@@ -311,7 +311,7 @@ public class Flags {
public static final UnboundBooleanFlag SEPARATE_METRIC_CHECK_CONFIG = defineFeatureFlag(
"separate-metric-check-config", false,
- List.of("olaa"), "2022-07-04", "2023-01-01",
+ List.of("olaa"), "2022-07-04", "2023-06-01",
"Determines whether one metrics config check should be written per Vespa node",
"Takes effect on next tick",
HOSTNAME);
@@ -326,7 +326,7 @@ public class Flags {
public static final UnboundBooleanFlag CLEANUP_TENANT_ROLES = defineFeatureFlag(
"cleanup-tenant-roles", false,
- List.of("olaa"), "2022-08-10", "2023-01-01",
+ List.of("olaa"), "2022-08-10", "2023-02-01",
"Determines whether old tenant roles should be deleted",
"Takes effect next maintenance run"
);
@@ -359,7 +359,7 @@ public class Flags {
public static final UnboundBooleanFlag ENABLE_OTELCOL = defineFeatureFlag(
"enable-otel-collector", false,
- List.of("olaa"), "2022-09-23", "2023-01-01",
+ List.of("olaa"), "2022-09-23", "2023-06-01",
"Whether an OpenTelemetry collector should be enabled",
"Takes effect at next tick",
APPLICATION_ID);
@@ -393,7 +393,7 @@ public class Flags {
public static final UnboundBooleanFlag ENABLED_MAIL_VERIFICATION = defineFeatureFlag(
"enabled-mail-verification", false,
- List.of("olaa"), "2022-10-28", "2023-01-01",
+ List.of("olaa"), "2022-10-28", "2023-02-01",
"Enable mail verification",
"Takes effect immediately");
@@ -418,13 +418,6 @@ public class Flags {
"Takes effect on redeployment",
APPLICATION_ID);
- public static final UnboundBooleanFlag USE_LOCKS_IN_FILEDISTRIBUTION = defineFeatureFlag(
- "use-locks-in-filedistribution", true,
- List.of("hmusum"), "2022-11-16", "2023-01-31",
- "If true, use locks when writing and deleting file references.",
- "Takes effect immediately",
- ZONE_ID, APPLICATION_ID);
-
/** WARNING: public for testing: All flags should be defined in {@link Flags}. */
public static UnboundBooleanFlag defineFeatureFlag(String flagId, boolean defaultValue, List<String> owners,
String createdAt, String expiresAt, String description,
diff --git a/searchcore/src/vespa/searchcore/proton/common/scheduled_forward_executor.cpp b/searchcore/src/vespa/searchcore/proton/common/scheduled_forward_executor.cpp
index b38de7a2456..f19ff36dbfb 100644
--- a/searchcore/src/vespa/searchcore/proton/common/scheduled_forward_executor.cpp
+++ b/searchcore/src/vespa/searchcore/proton/common/scheduled_forward_executor.cpp
@@ -3,8 +3,8 @@
#include "scheduled_forward_executor.h"
#include <vespa/vespalib/util/lambdatask.h>
#include <vespa/vespalib/stllike/hash_map.hpp>
-#include <atomic>
#include <thread>
+#include <condition_variable>
#include <cassert>
using vespalib::makeLambdaTask;
@@ -14,44 +14,56 @@ namespace proton {
class ScheduledForwardExecutor::State {
public:
State() :
+ _mutex(),
+ _cond(),
_handle(),
_start_success(0),
_start_failed(0),
_running(false)
{}
~State() {
+ std::lock_guard guard(_mutex);
assert( !_handle );
- assert(!isRunning());
+ assert( ! _running);
}
/// Returns false if it was already running
bool start() {
- bool already_running = _running.exchange(true);
+ std::lock_guard guard(_mutex);
+ bool already_running = _running;
+ _running = true;
if (already_running) {
_start_failed++;
} else {
_start_success++;
}
+ _cond.notify_all();
return ! already_running;
}
void complete() {
- bool was_running = _running.exchange(false);
+ std::lock_guard guard(_mutex);
+ bool was_running = _running;
+ _running = false;
assert(was_running);
+ _cond.notify_all();
}
void setHandle(Handle handle) {
+ std::lock_guard guard(_mutex);
_handle = std::move(handle);
}
void cancel() {
+ std::unique_lock guard(_mutex);
_handle.reset();
- while(isRunning()) {
- std::this_thread::sleep_for(1ms);
+ while(_running) {
+ _cond.wait(guard);
}
}
private:
- bool isRunning() const { return _running.load(std::memory_order_relaxed); }
- Handle _handle;
- std::atomic<uint64_t> _start_success;
- std::atomic<uint64_t> _start_failed;
- std::atomic<bool> _running;
+ std::mutex _mutex;
+ std::condition_variable _cond;
+ Handle _handle;
+ uint64_t _start_success;
+ uint64_t _start_failed;
+ bool _running;
};
class ScheduledForwardExecutor::Registration : public vespalib::IDestructorCallback {
diff --git a/searchcore/src/vespa/searchcore/proton/server/disk_mem_usage_sampler.cpp b/searchcore/src/vespa/searchcore/proton/server/disk_mem_usage_sampler.cpp
index 9e09e968b49..1f3bb524f74 100644
--- a/searchcore/src/vespa/searchcore/proton/server/disk_mem_usage_sampler.cpp
+++ b/searchcore/src/vespa/searchcore/proton/server/disk_mem_usage_sampler.cpp
@@ -41,8 +41,8 @@ DiskMemUsageSampler::setConfig(const Config &config, IScheduledExecutor & execut
if (_periodicHandle && (_sampleInterval == config.sampleInterval) && !wasChanged) {
return;
}
- _sampleInterval = config.sampleInterval;
_periodicHandle.reset();
+ _sampleInterval = config.sampleInterval;
sampleAndReportUsage();
vespalib::duration maxInterval = std::min(vespalib::duration(1s), _sampleInterval);
_periodicHandle = executor.scheduleAtFixedRate(makeLambdaTask([this]() {
diff --git a/vespaclient/CMakeLists.txt b/vespaclient/CMakeLists.txt
index 9593304cccd..912b35fa763 100644
--- a/vespaclient/CMakeLists.txt
+++ b/vespaclient/CMakeLists.txt
@@ -14,6 +14,5 @@ vespa_define_module(
src/vespa/vespaclient/clusterlist
APPS
- src/vespa/vespaclient/vdsstates
src/vespa/vespaclient/vesparoute
)
diff --git a/vespaclient/src/vespa/vespaclient/vdsstates/.gitignore b/vespaclient/src/vespa/vespaclient/vdsstates/.gitignore
deleted file mode 100644
index 30187c17166..00000000000
--- a/vespaclient/src/vespa/vespaclient/vdsstates/.gitignore
+++ /dev/null
@@ -1,5 +0,0 @@
-.*.swp
-.depend
-Makefile
-vdsgetnodestate
-vdsgetnodestate-bin
diff --git a/vespaclient/src/vespa/vespaclient/vdsstates/CMakeLists.txt b/vespaclient/src/vespa/vespaclient/vdsstates/CMakeLists.txt
deleted file mode 100644
index 01367a788ae..00000000000
--- a/vespaclient/src/vespa/vespaclient/vdsstates/CMakeLists.txt
+++ /dev/null
@@ -1,8 +0,0 @@
-# Copyright Yahoo. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
-vespa_add_executable(vespaclient_vdsgetnodestate_app
- SOURCES
- statesapp.cpp
- OUTPUT_NAME vdsgetnodestate-bin
- DEPENDS
- vespaclient_clusterlist
-)
diff --git a/vespaclient/src/vespa/vespaclient/vdsstates/statesapp.cpp b/vespaclient/src/vespa/vespaclient/vdsstates/statesapp.cpp
deleted file mode 100644
index 017abfdf28c..00000000000
--- a/vespaclient/src/vespa/vespaclient/vdsstates/statesapp.cpp
+++ /dev/null
@@ -1,463 +0,0 @@
-// Copyright Yahoo. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
-
-#include <vespa/defaults.h>
-#include <vespa/slobrok/sbmirror.h>
-#include <vespa/fnet/frt/supervisor.h>
-#include <vespa/fnet/frt/target.h>
-#include <vespa/fnet/frt/rpcrequest.h>
-#include <vespa/vdslib/distribution/distribution.h>
-#include <vespa/vdslib/state/clusterstate.h>
-#include <vespa/vespalib/util/programoptions.h>
-#include <vespa/vespaclient/clusterlist/clusterlist.h>
-#include <vespa/vespalib/util/time.h>
-#include <vespa/vespalib/text/lowercase.h>
-#include <vespa/config-stor-distribution.h>
-#include <vespa/config/helper/configgetter.hpp>
-#include <vespa/vespalib/util/signalhandler.h>
-#include <sstream>
-#include <iostream>
-#include <thread>
-#include <cstdlib>
-#include <sys/time.h>
-
-#include <vespa/log/log.h>
-LOG_SETUP("vdsstatetool");
-
-namespace storage {
-
-enum Mode { SETNODESTATE, GETNODESTATE, GETCLUSTERSTATE };
-
-namespace {
- Mode getMode(std::string calledAs) {
- std::string::size_type pos = calledAs.rfind('/');
- if (pos != std::string::npos) {
- calledAs = calledAs.substr(pos + 1);
- }
- if (calledAs == "vdssetnodestate-bin") return SETNODESTATE;
- if (calledAs == "vdsgetclusterstate-bin") return GETCLUSTERSTATE;
- if (calledAs == "vdsgetsystemstate-bin") return GETCLUSTERSTATE;
- if (calledAs == "vdsgetnodestate-bin") return GETNODESTATE;
- std::cerr << "Tool called through unknown name '" << calledAs << "'. Assuming you want to "
- << "get node state.\n";
- return GETNODESTATE;
- }
-
- uint64_t getTimeInMillis() {
- struct timeval t;
- gettimeofday(&t, 0);
- return (t.tv_sec * uint64_t(1000)) + (t.tv_usec / uint64_t(1000));
- }
-
- struct Sorter {
- bool operator()(const std::pair<std::string, std::string>& first,
- const std::pair<std::string, std::string>& second)
- { return (first.first < second.first); }
- };
-
- const lib::State* getState(const std::string& s) {
- vespalib::string lower = vespalib::LowerCase::convert(s);
- if (lower == "up") return &lib::State::UP;
- if (lower == "down") return &lib::State::DOWN;
- if (lower == "retired") return &lib::State::RETIRED;
- if (lower == "maintenance") return &lib::State::MAINTENANCE;
- return 0;
- }
-
- template<typename T>
- struct ConfigReader : public T::Subscriber
- {
- T config;
-
- ConfigReader(const std::string& configId) {
- T::subscribe(configId, *this);
- }
- void configure(const T& c) { config = c; }
- };
-}
-
-struct Options : public vespalib::ProgramOptions {
- Mode _mode;
- bool _showSyntax;
- std::string _clusterName;
- vespaclient::ClusterList::Cluster _cluster;
- uint32_t _nodeIndex;
- std::string _slobrokConfigId;
- std::string _slobrokConnectionSpec;
- std::string _nodeType;
- bool _nonfriendlyOutput;
- std::string _state;
- std::string _message;
- std::string _doc;
- uint32_t _slobrokTimeout;
-
- Options(Mode mode);
- ~Options();
-
- bool validate() {
- if (_nodeType != ""
- && _nodeType != "storage" && _nodeType != "distributor")
- {
- std::cerr << "Illegal nodetype '" << _nodeType << "'.\n";
- return false;
- }
- if (_mode == SETNODESTATE) {
- const lib::State* state = getState(_state);
- if (state == 0) {
- std::cerr << "Illegal state '" << _state << "'.\n";
- return false;
- }
- if (*state == lib::State::RETIRED ||
- *state == lib::State::MAINTENANCE)
- {
- if (_nodeType != "storage") {
- std::cerr << "Given state is only valid for storage nodes. "
- << "Thus you need to specify only to\n"
- << "set state of storage nodes.\n";
- return false;
- }
- }
- if (*state != lib::State::UP && *state != lib::State::RETIRED
- && _message == "")
- {
- std::cerr << "You should always have a reason for setting the "
- "node in a non-available state.\n";
- return false;
- }
- }
-
- vespaclient::ClusterList clusterList;
- try {
- _cluster = clusterList.verifyContentCluster(_clusterName);
- _clusterName = _cluster.getName();
- } catch (const vespaclient::ClusterList::ClusterNotFoundException& e) {
- std::cerr << e.getMessage() << "\n";
- std::_Exit(1);
- }
- return true;
- }
-};
-
-Options::Options(Mode mode)
- : _mode(mode), _cluster("", ""), _nodeIndex(0xffffffff), _nonfriendlyOutput(false), _slobrokTimeout(0)
-{
- _doc = "https://yahoo.github.io/vespa/";
- if (_mode == SETNODESTATE) {
- setSyntaxMessage(
- "Set the wanted node state of a storage node. This will "
- "override the state the node is in in the cluster state, if "
- "the current state is \"better\" than the wanted state. "
- "For instance, a node that is currently in initializing state "
- "can be forced into down state, while a node that is currently"
- " down can not be forced into retired state, but can be forced"
- " into maintenance state.\n\n"
- "For more info on states refer to\n" + _doc
- );
- } else if (_mode == GETCLUSTERSTATE) {
- setSyntaxMessage(
- "Get the cluster state of a given cluster.\n\n"
- "For more info on states refer to\n" + _doc
- );
- } else {
- setSyntaxMessage(
- "Retrieve the state of a one or more storage services from the "
- "fleet controller. Will list the state of the locally running "
- "services, possibly restricted to less by options.\n\n"
- "The result will show the slobrok address of the service, and "
- "three states. The first state will show how the state of that "
- "given service looks in the current cluster state. This state "
- "is the state the fleetcontroller is reporting to all nodes "
- "in the cluster this service is in. The second state is the "
- "reported state, which is the state the given node is reporting"
- " to be in itself. The third state is the wanted state, which "
- "is the state we want the node to be in. In most cases this "
- "should be the up state, but in some cases the fleet controller"
- " or an administrator may have set the wanted state otherwise, "
- "in order to get problem nodes out of the cluster.\n\n"
- "For more info on states refer to\n" + _doc
- );
- }
- addOption("h help", _showSyntax, false,
- "Show this help page.");
-
- addOption("c cluster", _clusterName, std::string("storage"),
- "Which cluster to connect to. By default it will attempt to connect to cluster named 'storage'.");
- if (_mode != GETCLUSTERSTATE) {
- addOption("t type", _nodeType, std::string(""),
- "Node type to query. This can either be 'storage' or "
- "'distributor'. If not specified, the operation will "
- "affect both types.");
- addOption("i index", _nodeIndex, uint32_t(0xffffffff),
- "The node index of the distributor or storage node to "
- "contact. If not specified, all indexes running locally "
- "on this node will be queried");
- }
- if (_mode != SETNODESTATE) {
- addOption("r raw", _nonfriendlyOutput, false,
- "Show the serialized state formats directly instead of "
- "reformatting them to look more user friendly.");
- }
- if (_mode == SETNODESTATE) {
- addArgument("Wanted state", _state,
- "Wanted state to set node in. "
- "This must be one of up, down or maintenance. Or if "
- "it's not a distributor it can also be retired.");
- addArgument("Reason", _message, std::string(""),
- "Give a reason for why you're altering the wanted "
- "state, which will show up in various admin tools. "
- "(Use double quotes to give a reason with whitespace "
- "in it)");
- }
- addOptionHeader("Advanced options. Not needed for most usecases");
- addOption("l slobrokconfig", _slobrokConfigId,
- std::string("client"),
- "Config id of slobrok. Will use the default config id of client if not specified.");
- addOption("p slobrokspec", _slobrokConnectionSpec, std::string(""),
- "Slobrok connection spec. By setting this, this application "
- "will not need config at all, but will use the given "
- "connection spec to talk with slobrok.");
- addOption("s slobroktimeout", _slobrokTimeout, uint32_t(5 * 60),
- "Seconds to wait for slobrok client to connect to a slobrok server before failing.");
-}
-Options::~Options() {}
-
-
-struct StateApp {
- Options _options;
-
- StateApp(std::string calledAs) : _options(getMode(calledAs)) {}
-
- int main(int argc, char **argv) {
- _options.setCommandLineArguments(argc, argv);
- try{
- _options.parse();
- } catch (vespalib::InvalidCommandLineArgumentsException& e) {
- if (!_options._showSyntax) {
- std::cerr << e.getMessage() << "\n";
- _options.writeSyntaxPage(std::cerr, false);
- std::cerr << "\n";
- return 1;
- }
- }
- if (_options._showSyntax) {
- _options.writeSyntaxPage(std::cerr, false);
- std::cerr << "\n";
- return 0;
- }
- if (!_options.validate()) {
- _options.writeSyntaxPage(std::cerr, false);
- return 1;
- }
- return run();
- }
-
- int run() {
- fnet::frt::StandaloneFRT supervisor;
-
- std::unique_ptr<slobrok::api::MirrorAPI> slobrok;
- if (_options._slobrokConnectionSpec == "") {
- config::ConfigUri config(_options._slobrokConfigId);
- slobrok = std::make_unique<slobrok::api::MirrorAPI>(supervisor.supervisor(), slobrok::ConfiguratorFactory(config));
- } else {
- std::vector<std::string> specList;
- specList.push_back(_options._slobrokConnectionSpec);
- slobrok = std::make_unique<slobrok::api::MirrorAPI>(supervisor.supervisor(), slobrok::ConfiguratorFactory(specList));
- }
- LOG(debug, "Waiting for slobrok data to be available.");
- uint64_t startTime = getTimeInMillis();
- uint64_t warnTime = 5 * 1000;
- uint64_t timeout = _options._slobrokTimeout * 1000;
- while (true) {
- uint64_t currentTime = getTimeInMillis();
- if (currentTime >= startTime + timeout) break;
- if (slobrok->ready()) break;
- if (currentTime >= startTime + warnTime) {
- if (warnTime > 5000) {
- std::cerr << "Still waiting for slobrok to respond. Have "
- << "gotten no response in "
- << ((currentTime - startTime) / 1000)
- << " seconds.\n";
- } else {
- std::cerr << "Waiting for slobrok server to respond. Have "
- << "gotten no response in "
- << ((currentTime - startTime) / 1000) << "\n"
- << "seconds. Likely cause being one or more "
- << "slobrok server nodes being down.\n(Thus not "
- << "replying that socket is closed)\n";
- }
- warnTime *= 4;
- }
- std::this_thread::sleep_for(10ms);
- }
- if (!slobrok->ready()) {
- std::cerr << "Slobrok not ready.\n";
- return 1;
- }
-
- config::ConfigUri uri(_options._cluster.getConfigId());
- lib::Distribution distribution(*config::ConfigGetter<vespa::config::content::StorDistributionConfig>::getConfig(uri.getConfigId(), uri.getContext()));
-
- LOG(debug, "Got slobrok data");
- std::string mask = "storage/cluster." + _options._cluster.getName() + "/fleetcontroller/*";
- slobrok::api::MirrorAPI::SpecList specs = slobrok->lookup(mask);
- if (specs.size() == 0) {
- std::cerr << "No fleet controller could be found for '"
- << mask << ".\n";
- return 1;
- }
- std::sort(specs.begin(), specs.end(), Sorter());
- LOG(debug, "Found fleet controller %s - %s\n",
- specs.front().first.c_str(), specs.front().second.c_str());
- FRT_Target *target = supervisor.supervisor().GetTarget(specs.front().second.c_str());
- if (!_options._nonfriendlyOutput && _options._mode == GETNODESTATE)
- {
- std::cerr <<
-"Shows the various states of one or more nodes in a Vespa Storage cluster.\n"
-"There exist three different type of node states. They are:\n"
-"\n"
-" Reported state - The state reported to the fleet controller by the node.\n"
-" Wanted state - The state administrators want the node to be in.\n"
-" Current state - The state of a given node in the current cluster state.\n"
-" This is the state all the other nodes know about. This\n"
-" state is a product of the other two states and fleet\n"
-" controller logic to keep the cluster stable.\n"
-"\n"
-"For more information about states of Vespa storage nodes, refer to\n"
- << _options._doc << "\n\n";
- }
- bool failed = false;
- for (int i=0; i<2; ++i) {
- std::string nodeType(_options._nodeType);
- if ((_options._nodeType != "" || _options._mode == GETCLUSTERSTATE)
- && i > 0)
- {
- break;
- }
- if (_options._nodeType == "") {
- nodeType = (i == 0 ? "storage" : "distributor");
- }
- std::vector<uint32_t> indexes;
- if (_options._nodeIndex != 0xffffffff
- || _options._mode == GETCLUSTERSTATE)
- {
- indexes.push_back(_options._nodeIndex);
- } else {
- std::string hostname(vespa::Defaults::vespaHostname());
- FRT_RPCRequest* req = supervisor.supervisor().AllocRPCRequest();
- req->SetMethodName("getNodeList");
- target->InvokeSync(req, 10.0);
- std::string prefix = _options._cluster.getConfigId() + "/" + nodeType + "/";
- failed = (req->GetErrorCode() != FRTE_NO_ERROR);
- if (failed) {
- std::cerr << "Failed RPC call against "
- << specs.front().second << ".\nError "
- << req->GetErrorCode() << " : "
- << req->GetErrorMessage() << "\n";
- break;
- }
- uint32_t arraySize(
- req->GetReturn()->GetValue(0)._string_array._len);
- for (uint32_t j=0; j<arraySize; ++j) {
- std::string slobrokAddress(req->GetReturn()->GetValue(0)
- ._string_array._pt[j]._str);
- std::string rpcAddress(req->GetReturn()->GetValue(1)
- ._string_array._pt[j]._str);
- std::string::size_type pos = slobrokAddress.find(prefix);
- std::string::size_type match = rpcAddress.find(hostname);
- //std::cerr << "1. '" << slobrokAddress << "'.\n";
- //std::cerr << "2. '" << rpcAddress << "'.\n";
- if (pos != std::string::npos && match != std::string::npos)
- {
- uint32_t index = atoi(slobrokAddress.substr(
- pos + prefix.size()).c_str());
- indexes.push_back(index);
- }
- }
- }
- if (indexes.size() == 0) {
- std::cerr << "Could not find any storage or distributor "
- << "services on this node.\n"
- << "Specify node index with --index parameter.\n";
- failed = true;
- break;
- }
- for (uint32_t j=0; j<indexes.size(); ++j) {
- FRT_RPCRequest* req = supervisor.supervisor().AllocRPCRequest();
- if (_options._mode == GETNODESTATE) {
- req->SetMethodName("getNodeState");
- req->GetParams()->AddString(nodeType.c_str());
- req->GetParams()->AddInt32(indexes[j]);
- } else if (_options._mode == SETNODESTATE) {
- req->SetMethodName("setNodeState");
- std::ostringstream address;
- address << _options._cluster.getConfigId() << "/"
- << nodeType << "/" << indexes[j];
- lib::NodeState ns(lib::NodeType::get(nodeType),
- *getState(_options._state));
- ns.setDescription(_options._message);
- req->GetParams()->AddString(address.str().c_str());
- req->GetParams()->AddString(ns.toString(false).c_str());
- } else {
- req->SetMethodName("getSystemState");
- }
- target->InvokeSync(req, 10.0);
- failed = (req->GetErrorCode() != FRTE_NO_ERROR);
- if (failed) {
- std::cerr << "Failed RPC call against "
- << specs.front().second
- << ".\nError " << req->GetErrorCode() << " : "
- << req->GetErrorMessage() << "\n";
- break;
- } else {
- bool friendly = !_options._nonfriendlyOutput;
- if (_options._mode == GETNODESTATE) {
- lib::NodeState current(
- req->GetReturn()->GetValue(0)._string._str);
- lib::NodeState reported(
- req->GetReturn()->GetValue(1)._string._str);
- lib::NodeState wanted(
- req->GetReturn()->GetValue(2)._string._str);
- std::cout << "Node state of "
- << _options._cluster.getConfigId() << "/" << nodeType
- << "/" << indexes[j];
- std::cout << "\nCurrent state: ";
- current.print(std::cout, friendly, " ");
- std::cout << "\nReported state ";
- reported.print(std::cout, friendly, " ");
- std::cout << "\nWanted state: ";
- wanted.print(std::cout, friendly, " ");
- std::cout << "\n\n";
- } else if (_options._mode == SETNODESTATE) {
- std::string result(
- req->GetReturn()->GetValue(0)._string._str);
- if (result != "") {
- std::cout << result << "\n";
- }
- } else {
- std::string rawstate(
- req->GetReturn()->GetValue(1)._string._str);
- lib::ClusterState state(rawstate);
- if (friendly) {
- state.printStateGroupwise(std::cout, distribution,
- true, "");
- } else {
- std::cout << rawstate << "\n";
- }
- std::cout << "\n";
- }
- }
- req->SubRef();
- }
- }
- target->SubRef();
- return (failed ? 1 : 0);
- }
-};
-
-} // storage
-
-int main(int argc, char **argv) {
- vespalib::SignalHandler::PIPE.ignore();
- assert(argc > 0);
- storage::StateApp client(argv[0]);
- return client.main(argc, argv);
-}