summaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
-rw-r--r--clustercontroller-apps/src/main/java/com/yahoo/vespa/clustercontroller/apps/clustercontroller/ClusterController.java2
-rw-r--r--clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/FleetController.java48
-rw-r--r--clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/FleetControllerOptions.java16
-rw-r--r--clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/status/LegacyIndexPageRequestHandler.java6
-rw-r--r--clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/status/statuspage/StatusPageServer.java270
-rw-r--r--clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/FleetControllerTest.java13
-rw-r--r--clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/MasterElectionTest.java12
-rw-r--r--clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/StatusPagesTest.java374
-rw-r--r--clustercontroller-reindexer/src/main/java/ai/vespa/reindexing/Reindexer.java92
-rw-r--r--clustercontroller-reindexer/src/main/java/ai/vespa/reindexing/Reindexing.java2
-rw-r--r--clustercontroller-reindexer/src/main/java/ai/vespa/reindexing/ReindexingMaintainer.java7
-rw-r--r--clustercontroller-reindexer/src/main/java/ai/vespa/reindexing/ReindexingMetrics.java48
-rw-r--r--clustercontroller-reindexer/src/test/java/ai/vespa/reindexing/ReindexerTest.java37
-rw-r--r--config-model/src/main/java/com/yahoo/vespa/model/application/validation/change/search/AttributeChangeValidator.java38
-rw-r--r--config-model/src/main/java/com/yahoo/vespa/model/application/validation/change/search/DocumentDatabaseChangeValidator.java6
-rw-r--r--config-model/src/main/java/com/yahoo/vespa/model/container/http/AccessControl.java14
-rw-r--r--config-model/src/main/java/com/yahoo/vespa/model/container/http/Http.java4
-rw-r--r--config-model/src/main/java/com/yahoo/vespa/model/container/http/xml/HttpBuilder.java4
-rw-r--r--config-model/src/main/java/com/yahoo/vespa/model/container/xml/ContainerModelBuilder.java8
-rw-r--r--config-model/src/test/java/com/yahoo/vespa/model/application/validation/change/search/AttributeChangeValidatorTest.java32
-rw-r--r--config-model/src/test/java/com/yahoo/vespa/model/application/validation/change/search/DocumentTypeChangeValidatorTest.java19
-rw-r--r--config-model/src/test/java/com/yahoo/vespa/model/container/http/StrictFilteringTest.java41
-rw-r--r--config-model/src/test/java/com/yahoo/vespa/model/container/xml/AccessControlTest.java23
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/http/v2/ApplicationHandler.java15
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/session/SessionRepository.java6
-rw-r--r--configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/ApplicationHandlerTest.java8
-rwxr-xr-xcontainer-core/src/main/java/com/yahoo/container/Container.java10
-rw-r--r--container-core/src/main/java/com/yahoo/container/core/config/ApplicationBundleLoader.java4
-rw-r--r--container-core/src/main/java/com/yahoo/container/core/config/FileAcquirerBundleInstaller.java7
-rw-r--r--container-core/src/main/java/com/yahoo/container/core/config/HandlersConfigurerDi.java1
-rw-r--r--container-core/src/main/java/com/yahoo/container/core/config/PlatformBundleLoader.java1
-rw-r--r--container-core/src/main/java/com/yahoo/container/core/config/testutil/HandlersConfigurerTestWrapper.java10
-rw-r--r--container-core/src/main/java/com/yahoo/container/core/config/testutil/MockOsgiWrapper.java1
-rw-r--r--container-core/src/main/java/com/yahoo/container/handler/Coverage.java26
-rw-r--r--container-core/src/main/java/com/yahoo/container/handler/Prefix.java1
-rw-r--r--container-core/src/main/java/com/yahoo/container/handler/ThreadPoolProvider.java2
-rw-r--r--container-core/src/main/java/com/yahoo/container/handler/Timing.java1
-rw-r--r--container-core/src/main/java/com/yahoo/container/handler/VipStatusHandler.java4
-rw-r--r--container-core/src/main/java/com/yahoo/container/handler/metrics/ErrorResponse.java6
-rw-r--r--container-core/src/main/java/com/yahoo/container/handler/metrics/JsonResponse.java2
-rw-r--r--container-core/src/main/java/com/yahoo/container/handler/metrics/MetricsV2Handler.java1
-rw-r--r--container-core/src/main/java/com/yahoo/container/handler/metrics/PrometheusV1Handler.java3
-rw-r--r--container-core/src/main/java/com/yahoo/container/handler/test/MockService.java23
-rw-r--r--container-core/src/main/java/com/yahoo/container/handler/test/MockServiceHandler.java18
-rw-r--r--container-core/src/main/java/com/yahoo/container/handler/threadpool/DefaultContainerThreadpool.java2
-rw-r--r--container-core/src/main/java/com/yahoo/container/handler/threadpool/ExecutorServiceWrapper.java10
-rw-r--r--container-core/src/main/java/com/yahoo/container/handler/threadpool/WorkerCompletionTimingThreadPoolExecutor.java18
-rw-r--r--container-core/src/main/java/com/yahoo/container/http/filter/FilterChainRepository.java2
-rw-r--r--container-core/src/main/java/com/yahoo/container/jdisc/AsyncHttpResponse.java10
-rw-r--r--container-core/src/main/java/com/yahoo/container/jdisc/ContentChannelOutputStream.java29
-rw-r--r--container-core/src/main/java/com/yahoo/container/jdisc/EmptyResponse.java4
-rw-r--r--container-core/src/main/java/com/yahoo/container/jdisc/HttpRequest.java39
-rw-r--r--container-core/src/main/java/com/yahoo/container/jdisc/LoggingRequestHandler.java23
-rw-r--r--container-core/src/main/java/com/yahoo/container/jdisc/RequestHandlerTestDriver.java4
-rw-r--r--container-core/src/main/java/com/yahoo/container/jdisc/ThreadedHttpRequestHandler.java39
-rw-r--r--container-core/src/main/java/com/yahoo/container/jdisc/ThreadedRequestHandler.java1
-rw-r--r--container-core/src/main/java/com/yahoo/container/jdisc/VespaHeaders.java30
-rw-r--r--container-core/src/main/java/com/yahoo/container/jdisc/state/FileWrapper.java1
-rw-r--r--container-core/src/main/java/com/yahoo/container/jdisc/state/GaugeMetric.java2
-rw-r--r--container-core/src/main/java/com/yahoo/container/jdisc/state/HostLifeGatherer.java1
-rw-r--r--container-core/src/main/java/com/yahoo/container/jdisc/state/JSONObjectWithLegibleException.java5
-rw-r--r--container-core/src/main/java/com/yahoo/container/jdisc/state/MetricGatherer.java6
-rw-r--r--container-core/src/main/java/com/yahoo/container/jdisc/state/MetricsPacketsHandler.java1
-rw-r--r--container-core/src/main/java/com/yahoo/container/jdisc/state/SnapshotProvider.java2
-rw-r--r--container-core/src/main/java/com/yahoo/container/jdisc/state/StateMonitor.java1
-rw-r--r--container-core/src/main/java/com/yahoo/container/servlet/ServletProvider.java7
-rw-r--r--container-core/src/main/java/com/yahoo/container/xml/providers/DatatypeFactoryProvider.java4
-rw-r--r--container-core/src/main/java/com/yahoo/container/xml/providers/DocumentBuilderFactoryProvider.java2
-rw-r--r--container-core/src/main/java/com/yahoo/container/xml/providers/SAXParserFactoryProvider.java2
-rw-r--r--container-core/src/main/java/com/yahoo/container/xml/providers/SchemaFactoryProvider.java2
-rw-r--r--container-core/src/main/java/com/yahoo/container/xml/providers/TransformerFactoryProvider.java2
-rw-r--r--container-core/src/main/java/com/yahoo/container/xml/providers/XMLEventFactoryProvider.java2
-rw-r--r--container-core/src/main/java/com/yahoo/container/xml/providers/XMLInputFactoryProvider.java2
-rw-r--r--container-core/src/main/java/com/yahoo/container/xml/providers/XMLOutputFactoryProvider.java2
-rw-r--r--container-core/src/main/java/com/yahoo/container/xml/providers/XPathFactoryProvider.java2
-rw-r--r--container-core/src/main/java/com/yahoo/language/provider/DefaultLinguisticsProvider.java4
-rw-r--r--container-core/src/main/java/com/yahoo/osgi/Osgi.java1
-rw-r--r--container-core/src/main/java/com/yahoo/osgi/OsgiImpl.java1
-rw-r--r--container-core/src/main/java/com/yahoo/processing/handler/ProcessingResponse.java4
-rw-r--r--container-core/src/main/java/com/yahoo/processing/handler/ProcessingTestDriver.java18
-rw-r--r--container-core/src/main/java/com/yahoo/processing/handler/ResponseHeaders.java1
-rw-r--r--container-core/src/main/java/com/yahoo/processing/processors/RequestPropertyTracer.java3
-rw-r--r--container-core/src/main/java/com/yahoo/processing/rendering/AsynchronousRenderer.java3
-rw-r--r--container-core/src/main/java/com/yahoo/processing/rendering/AsynchronousSectionedRenderer.java6
-rw-r--r--container-core/src/main/java/com/yahoo/processing/rendering/ProcessingRenderer.java3
-rw-r--r--container-core/src/main/java/com/yahoo/processing/rendering/Renderer.java22
-rw-r--r--container-core/src/main/java/com/yahoo/restapi/JacksonJsonResponse.java5
-rw-r--r--container-disc/src/main/java/com/yahoo/container/FilterConfigProvider.java1
-rw-r--r--container-disc/src/main/java/com/yahoo/container/jdisc/AthenzIdentityProviderProvider.java2
-rw-r--r--container-disc/src/main/java/com/yahoo/container/jdisc/RestrictedBundleContext.java44
-rw-r--r--container-disc/src/main/java/com/yahoo/container/jdisc/SecretStoreProvider.java1
-rw-r--r--container-disc/src/main/java/com/yahoo/container/jdisc/SystemInfoProvider.java7
-rw-r--r--container-disc/src/main/java/com/yahoo/container/jdisc/athenz/AthenzIdentityProvider.java2
-rw-r--r--container-disc/src/main/java/com/yahoo/container/jdisc/athenz/AthenzIdentityProviderException.java1
-rw-r--r--container-disc/src/main/java/com/yahoo/container/jdisc/metric/DisableGuiceMetric.java1
-rw-r--r--container-disc/src/main/java/com/yahoo/container/jdisc/metric/ForwardingMetricConsumer.java1
-rw-r--r--container-disc/src/main/java/com/yahoo/container/jdisc/metric/GarbageCollectionMetrics.java4
-rw-r--r--container-disc/src/main/java/com/yahoo/container/jdisc/metric/JrtMetrics.java1
-rw-r--r--container-disc/src/main/java/com/yahoo/container/jdisc/metric/MetricUpdater.java2
-rw-r--r--container-disc/src/main/java/com/yahoo/container/jdisc/secretstore/SecretStore.java2
-rw-r--r--container-disc/src/main/java/com/yahoo/container/usability/BindingsOverviewHandler.java24
-rw-r--r--container-messagebus/src/main/java/com/yahoo/container/jdisc/messagebus/MbusClientProvider.java1
-rw-r--r--container-messagebus/src/main/java/com/yahoo/container/jdisc/messagebus/SessionCache.java23
-rw-r--r--controller-api/src/main/java/com/yahoo/vespa/hosted/controller/api/integration/noderepository/NodeRepositoryNode.java12
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/application/ApplicationApiHandler.java4
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/billing/BillingApiHandler.java19
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/ApplicationApiTest.java6
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/billing/BillingApiHandlerTest.java10
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/billing/responses/billing-all-tenants30
-rw-r--r--document/abi-spec.json2
-rw-r--r--document/src/main/java/com/yahoo/document/DataType.java4
-rw-r--r--document/src/main/java/com/yahoo/document/TensorDataType.java16
-rw-r--r--eval/src/tests/eval/aggr/aggr_test.cpp61
-rw-r--r--eval/src/tests/tensor/dense_single_reduce_function/dense_single_reduce_function_test.cpp106
-rw-r--r--eval/src/tests/tensor/instruction_benchmark/instruction_benchmark.cpp31
-rw-r--r--eval/src/vespa/eval/eval/CMakeLists.txt1
-rw-r--r--eval/src/vespa/eval/eval/aggr.cpp1
-rw-r--r--eval/src/vespa/eval/eval/aggr.h46
-rw-r--r--eval/src/vespa/eval/eval/cell_type.cpp3
-rw-r--r--eval/src/vespa/eval/eval/cell_type.h39
-rw-r--r--eval/src/vespa/eval/eval/value_type.h31
-rw-r--r--eval/src/vespa/eval/instruction/generic_peek.cpp4
-rw-r--r--eval/src/vespa/eval/tensor/dense/dense_remove_dimension_optimizer.cpp11
-rw-r--r--eval/src/vespa/eval/tensor/dense/dense_single_reduce_function.cpp194
-rw-r--r--eval/src/vespa/eval/tensor/dense/dense_single_reduce_function.h38
-rw-r--r--jdisc_http_service/abi-spec.json2
-rw-r--r--jdisc_http_service/src/main/java/com/yahoo/jdisc/http/server/jetty/FilterResolver.java33
-rw-r--r--jdisc_http_service/src/main/java/com/yahoo/jdisc/http/server/jetty/JDiscContext.java2
-rw-r--r--jdisc_http_service/src/main/resources/configdefinitions/jdisc.http.jdisc.http.server.def3
-rw-r--r--jdisc_http_service/src/test/java/com/yahoo/jdisc/http/server/jetty/FilterTestCase.java35
-rw-r--r--node-repository/src/main/java/com/yahoo/vespa/hosted/provision/NodeRepository.java20
-rw-r--r--node-repository/src/main/java/com/yahoo/vespa/hosted/provision/autoscale/Autoscaler.java7
-rw-r--r--node-repository/src/main/java/com/yahoo/vespa/hosted/provision/maintenance/AutoscalingMaintainer.java2
-rw-r--r--node-repository/src/main/java/com/yahoo/vespa/hosted/provision/provisioning/NodeRepositoryProvisioner.java3
-rw-r--r--node-repository/src/main/java/com/yahoo/vespa/hosted/provision/provisioning/ProvisionedHost.java32
-rw-r--r--node-repository/src/main/java/com/yahoo/vespa/hosted/provision/restapi/NodesV2ApiHandler.java7
-rw-r--r--node-repository/src/test/java/com/yahoo/vespa/hosted/provision/autoscale/AutoscalingTester.java3
-rw-r--r--node-repository/src/test/java/com/yahoo/vespa/hosted/provision/maintenance/DynamicProvisioningMaintainerTest.java3
-rw-r--r--node-repository/src/test/java/com/yahoo/vespa/hosted/provision/provisioning/DynamicDockerProvisionTest.java3
-rw-r--r--node-repository/src/test/java/com/yahoo/vespa/hosted/provision/provisioning/ProvisioningTest.java7
-rw-r--r--node-repository/src/test/java/com/yahoo/vespa/hosted/provision/provisioning/ProvisioningTester.java4
-rw-r--r--node-repository/src/test/java/com/yahoo/vespa/hosted/provision/restapi/NodesV2ApiTest.java27
-rw-r--r--node-repository/src/test/java/com/yahoo/vespa/hosted/provision/restapi/responses/node9.json3
-rw-r--r--vespajlib/src/main/java/com/yahoo/collections/AbstractFilteringList.java4
-rw-r--r--zkfacade/src/main/java/com/yahoo/vespa/curator/Curator.java9
145 files changed, 1234 insertions, 1355 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 4091363128e..b04f04abfb6 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
@@ -58,7 +58,7 @@ public class ClusterController extends AbstractComponent
if (controller == null) {
StatusHandler.ContainerStatusPageServer statusPageServer = new StatusHandler.ContainerStatusPageServer();
- controller = FleetController.createForContainer(options, statusPageServer, metricWrapper);
+ controller = FleetController.create(options, statusPageServer, metricWrapper);
controllers.put(clusterName, controller);
status.put(clusterName, statusPageServer);
} else {
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 658dd10f7e5..9c9e1042c79 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
@@ -1,10 +1,9 @@
-// Copyright 2017 Yahoo Holdings. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+// Copyright Verizon Media. 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.document.FixedBucketSpaces;
import com.yahoo.exception.ExceptionUtils;
import com.yahoo.jrt.ListenFailedException;
-import java.util.logging.Level;
import com.yahoo.vdslib.distribution.ConfiguredNode;
import com.yahoo.vdslib.state.ClusterState;
import com.yahoo.vdslib.state.Node;
@@ -28,7 +27,6 @@ import com.yahoo.vespa.clustercontroller.core.status.statuspage.StatusPageRespon
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 com.yahoo.vespa.clustercontroller.utils.util.NoMetricReporter;
import java.io.FileNotFoundException;
import java.util.ArrayDeque;
@@ -44,6 +42,7 @@ import java.util.Set;
import java.util.TimeZone;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.logging.Level;
import java.util.logging.Logger;
import java.util.stream.Collectors;
import java.util.stream.Stream;
@@ -51,7 +50,7 @@ import java.util.stream.Stream;
public class FleetController implements NodeStateOrHostInfoChangeHandler, NodeAddedOrRemovedListener, SystemStateListener,
Runnable, RemoteClusterControllerTaskScheduler {
- private static Logger log = Logger.getLogger(FleetController.class.getName());
+ private static final Logger log = Logger.getLogger(FleetController.class.getName());
private final Timer timer;
private final Object monitor;
@@ -68,7 +67,7 @@ public class FleetController implements NodeStateOrHostInfoChangeHandler, NodeAd
private final DatabaseHandler database;
private final MasterElectionHandler masterElectionHandler;
private Thread runner = null;
- private AtomicBoolean running = new AtomicBoolean(true);
+ private final AtomicBoolean running = new AtomicBoolean(true);
private FleetControllerOptions options;
private FleetControllerOptions nextOptions;
private final List<SystemStateListener> systemStateListeners = new CopyOnWriteArrayList<>();
@@ -79,12 +78,12 @@ public class FleetController implements NodeStateOrHostInfoChangeHandler, NodeAd
private Long controllerThreadId = null;
private boolean waitingForCycle = false;
- private StatusPageServer.PatternRequestRouter statusRequestRouter = new StatusPageServer.PatternRequestRouter();
+ private final StatusPageServer.PatternRequestRouter statusRequestRouter = new StatusPageServer.PatternRequestRouter();
private final List<ClusterStateBundle> newStates = new ArrayList<>();
private final List<ClusterStateBundle> convergedStates = new ArrayList<>();
private long configGeneration = -1;
private long nextConfigGeneration = -1;
- private Queue<RemoteClusterControllerTask> remoteTasks = new LinkedList<>();
+ private final Queue<RemoteClusterControllerTask> remoteTasks = new LinkedList<>();
private final MetricUpdater metricUpdater;
private boolean isMaster = false;
@@ -92,9 +91,9 @@ public class FleetController implements NodeStateOrHostInfoChangeHandler, NodeAd
private long firstAllowedStateBroadcast = Long.MAX_VALUE;
private long tickStartTime = Long.MAX_VALUE;
- private List<RemoteClusterControllerTask> tasksPendingStateRecompute = new ArrayList<>();
+ private final List<RemoteClusterControllerTask> tasksPendingStateRecompute = new ArrayList<>();
// Invariant: queued task versions are monotonically increasing with queue position
- private Queue<VersionDependentTaskCompletion> taskCompletionQueue = new ArrayDeque<>();
+ private final Queue<VersionDependentTaskCompletion> taskCompletionQueue = new ArrayDeque<>();
// Legacy behavior is an empty set of explicitly configured bucket spaces, which means that
// only a baseline cluster state will be sent from the controller and no per-space state
@@ -125,8 +124,7 @@ public class FleetController implements NodeStateOrHostInfoChangeHandler, NodeAd
SystemStateBroadcaster systemStateBroadcaster,
MasterElectionHandler masterElectionHandler,
MetricUpdater metricUpdater,
- FleetControllerOptions options) throws Exception
- {
+ FleetControllerOptions options) {
log.info("Starting up cluster controller " + options.fleetControllerIndex + " for cluster " + cluster.getName());
this.timer = timer;
this.monitor = timer;
@@ -166,26 +164,10 @@ public class FleetController implements NodeStateOrHostInfoChangeHandler, NodeAd
propagateOptions();
}
- public static FleetController createForContainer(FleetControllerOptions options,
- StatusPageServerInterface statusPageServer,
- MetricReporter metricReporter) throws Exception {
+ public static FleetController create(FleetControllerOptions options,
+ StatusPageServerInterface statusPageServer,
+ MetricReporter metricReporter) throws Exception {
Timer timer = new RealTimer();
- return create(options, timer, statusPageServer, null, metricReporter);
- }
-
- public static FleetController createForStandAlone(FleetControllerOptions options) throws Exception {
- Timer timer = new RealTimer();
- RpcServer rpcServer = new RpcServer(timer, timer, options.clusterName, options.fleetControllerIndex, options.slobrokBackOffPolicy);
- StatusPageServer statusPageServer = new StatusPageServer(timer, timer, options.httpPort);
- return create(options, timer, statusPageServer, rpcServer, new NoMetricReporter());
- }
-
- private static FleetController create(FleetControllerOptions options,
- Timer timer,
- StatusPageServerInterface statusPageServer,
- RpcServer rpcServer,
- MetricReporter metricReporter) throws Exception
- {
MetricUpdater metricUpdater = new MetricUpdater(metricReporter, options.fleetControllerIndex);
EventLog log = new EventLog(timer, metricUpdater);
ContentCluster cluster = new ContentCluster(
@@ -209,7 +191,7 @@ public class FleetController implements NodeStateOrHostInfoChangeHandler, NodeAd
SystemStateBroadcaster stateBroadcaster = new SystemStateBroadcaster(timer, timer);
MasterElectionHandler masterElectionHandler = new MasterElectionHandler(options.fleetControllerIndex, options.fleetControllerCount, timer, timer);
FleetController controller = new FleetController(
- timer, log, cluster, stateGatherer, communicator, statusPageServer, rpcServer, lookUp, database, stateGenerator, stateBroadcaster, masterElectionHandler, metricUpdater, options);
+ timer, log, cluster, stateGatherer, communicator, statusPageServer, null, lookUp, database, stateGenerator, stateBroadcaster, masterElectionHandler, metricUpdater, options);
controller.start();
return controller;
}
@@ -469,7 +451,7 @@ public class FleetController implements NodeStateOrHostInfoChangeHandler, NodeAd
}
/** This is called when the options field has been set to a new set of options */
- private void propagateOptions() throws java.io.IOException, ListenFailedException {
+ private void propagateOptions() {
verifyInControllerThread();
if (changesConfiguredNodeSet(options.nodes)) {
@@ -547,7 +529,7 @@ public class FleetController implements NodeStateOrHostInfoChangeHandler, NodeAd
} catch (Exception e) {
responseCode = StatusPageResponse.ResponseCode.INTERNAL_SERVER_ERROR;
message = "Internal Server Error";
- hiddenMessage = ExceptionUtils.getStackTraceAsString(e);;
+ hiddenMessage = ExceptionUtils.getStackTraceAsString(e);
log.log(Level.FINE, "Unknown exception thrown for request " + httpRequest.getRequest() +
": " + hiddenMessage);
}
diff --git a/clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/FleetControllerOptions.java b/clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/FleetControllerOptions.java
index 553b3332ee8..2044eb1eab0 100644
--- a/clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/FleetControllerOptions.java
+++ b/clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/FleetControllerOptions.java
@@ -1,16 +1,20 @@
-// Copyright 2017 Yahoo Holdings. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+// Copyright Verizone Media. 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.slobrok.api.BackOffPolicy;
import com.yahoo.vdslib.distribution.ConfiguredNode;
import com.yahoo.vdslib.distribution.Distribution;
import com.yahoo.vdslib.state.NodeType;
-import com.yahoo.vespa.clustercontroller.core.status.statuspage.StatusPageServer;
-import java.time.Duration;
-import java.util.*;
import java.text.DecimalFormat;
import java.text.DecimalFormatSymbols;
+import java.time.Duration;
+import java.util.Collection;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
/**
* This class represents all the options that can be set in the fleetcontroller.
@@ -30,7 +34,7 @@ public class FleetControllerOptions implements Cloneable {
public int stateGatherCount = 2;
// TODO: This cannot be null but nonnull is not verified
- public String slobrokConnectionSpecs[];
+ public String[] slobrokConnectionSpecs;
public int rpcPort = 0;
public int httpPort = 0;
public int distributionBits = 16;
@@ -189,7 +193,7 @@ public class FleetControllerOptions implements Cloneable {
static DecimalFormat DecimalDot2 = new DecimalFormat("0.00", new DecimalFormatSymbols(Locale.ENGLISH));
- public void writeHtmlState(StringBuilder sb, StatusPageServer.HttpRequest request) {
+ public void writeHtmlState(StringBuilder sb) {
String slobrokspecs = "";
for (int i=0; i<slobrokConnectionSpecs.length; ++i) {
if (i != 0) slobrokspecs += "<br>";
diff --git a/clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/status/LegacyIndexPageRequestHandler.java b/clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/status/LegacyIndexPageRequestHandler.java
index f799492d164..9d5e5e46f08 100644
--- a/clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/status/LegacyIndexPageRequestHandler.java
+++ b/clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/status/LegacyIndexPageRequestHandler.java
@@ -22,7 +22,7 @@ public class LegacyIndexPageRequestHandler implements StatusPageServer.RequestHa
private final EventLog eventLog;
private final long startedTime;
private final RunDataExtractor data;
- private boolean showLocalSystemStatesInLog = true;
+ private final boolean showLocalSystemStatesInLog;
public LegacyIndexPageRequestHandler(Timer timer, boolean showLocalSystemStatesInLog, ContentCluster cluster,
MasterElectionHandler masterElectionHandler,
@@ -72,12 +72,12 @@ public class LegacyIndexPageRequestHandler implements StatusPageServer.RequestHa
eventLog
);
// Overview of current config
- data.getOptions().writeHtmlState(content, request);
+ data.getOptions().writeHtmlState(content);
// Current cluster state and cluster state history
writeHtmlState(stateVersionTracker, content, request);
} else {
// Overview of current config
- data.getOptions().writeHtmlState(content, request);
+ data.getOptions().writeHtmlState(content);
}
// Event log
eventLog.writeHtmlState(content, null);
diff --git a/clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/status/statuspage/StatusPageServer.java b/clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/status/statuspage/StatusPageServer.java
index 3d3de32c356..5920a7d651a 100644
--- a/clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/status/statuspage/StatusPageServer.java
+++ b/clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/status/statuspage/StatusPageServer.java
@@ -1,25 +1,10 @@
// Copyright 2017 Yahoo Holdings. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
package com.yahoo.vespa.clustercontroller.core.status.statuspage;
-import com.yahoo.exception.ExceptionUtils;
-import java.util.logging.Level;
-
-import java.io.BufferedReader;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.io.OutputStream;
-import java.net.InetSocketAddress;
-import java.net.ServerSocket;
-import java.net.Socket;
-import java.net.SocketTimeoutException;
-import java.text.DateFormat;
-import java.text.SimpleDateFormat;
import java.util.ArrayList;
-import java.util.Date;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
-import java.util.TimeZone;
import java.util.logging.Logger;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
@@ -27,238 +12,10 @@ import java.util.regex.Pattern;
/**
* Shows status pages with debug information through a very simple HTTP interface.
*/
-public class StatusPageServer implements Runnable, StatusPageServerInterface {
+public class StatusPageServer {
public static Logger log = Logger.getLogger(StatusPageServer.class.getName());
- private final com.yahoo.vespa.clustercontroller.core.Timer timer;
- private final Object monitor;
- private ServerSocket ssocket;
- private final Thread runner;
- private int port = 0;
- private boolean running = true;
- private boolean shouldBeConnected = false;
- private HttpRequest currentHttpRequest = null;
- private StatusPageResponse currentResponse = null;
- private long lastConnectErrorTime = 0;
- private String lastConnectError = "";
- private PatternRequestRouter staticContentRouter = new PatternRequestRouter();
- private Date startTime = new Date();
-
- public StatusPageServer(com.yahoo.vespa.clustercontroller.core.Timer timer, Object monitor, int port) throws java.io.IOException, InterruptedException {
- this.timer = timer;
- this.monitor = monitor;
- this.port = port;
- connect();
- runner = new Thread(this);
- runner.start();
- }
-
- public boolean isConnected() {
- if (ssocket != null && ssocket.isBound() && (ssocket.getLocalPort() == port || port == 0)) {
- return true;
- } else {
- log.log(Level.FINEST, "Status page server socket is no longer connected: "+ (ssocket != null) + " " + ssocket.isBound() + " " + ssocket.getLocalPort() + " " + port);
- return false;
- }
- }
-
- public void connect() throws java.io.IOException, InterruptedException {
- synchronized(monitor) {
- if (ssocket != null) {
- if (ssocket.isBound() && ssocket.getLocalPort() == port) {
- return;
- }
- disconnect();
- }
- ssocket = new ServerSocket();
- if (port != 0) {
- ssocket.setReuseAddress(true);
- }
- ssocket.setSoTimeout(100);
- ssocket.bind(new InetSocketAddress(port));
- shouldBeConnected = true;
- for (int i=0; i<200; ++i) {
- if (isConnected()) break;
- Thread.sleep(10);
- }
- if (!isConnected()) {
- log.log(Level.INFO, "Fleetcontroller: Server Socket not ready after connect()");
- }
- log.log(Level.FINE, "Fleet controller status page viewer listening to " + ssocket.getLocalSocketAddress());
- monitor.notifyAll();
- }
- }
-
- public void disconnect() throws java.io.IOException {
- synchronized(monitor) {
- shouldBeConnected = false;
- if (ssocket != null) ssocket.close();
- ssocket = null;
- monitor.notifyAll();
- }
- }
-
- public void setPort(int port) throws java.io.IOException, InterruptedException {
- // Only bother to reconnect if we were connected to begin with, we care about what port it runs on, and it's not already running there
- if (port != 0 && isConnected() && port != ((InetSocketAddress) ssocket.getLocalSocketAddress()).getPort()) {
- log.log(Level.INFO, "Exchanging port used by status server. Moving from port "
- + ((InetSocketAddress) ssocket.getLocalSocketAddress()).getPort() + " to port " + port);
- disconnect();
- this.port = port;
- if (ssocket == null || !ssocket.isBound() || ssocket.getLocalPort() != port) {
- connect();
- }
- } else {
- this.port = port;
- }
- }
-
- public int getPort() {
- // Cannot use this.port, because of tests using port 0 to get any address
- if (ssocket == null || !ssocket.isBound()) {
- throw new IllegalStateException("Cannot ask for port before server socket is bound");
- }
- return ((InetSocketAddress) ssocket.getLocalSocketAddress()).getPort();
- }
-
- public void shutdown() throws InterruptedException, java.io.IOException {
- running = false;
- runner.interrupt();
- runner.join();
- disconnect();
- }
-
- public void run() {
- try{
- while (running) {
- Socket connection = null;
- ServerSocket serverSocket = null;
- synchronized(monitor) {
- if (ssocket == null || !ssocket.isBound()) {
- monitor.wait(1000);
- continue;
- }
- serverSocket = ssocket;
- }
- try{
- connection = serverSocket.accept();
- } catch (SocketTimeoutException e) {
- // Ignore, since timeout is set to 100 ms
- } catch (java.io.IOException e) {
- log.log(shouldBeConnected ? Level.WARNING : Level.FINE, "Caught IO exception in ServerSocket.accept(): " + e.getMessage());
- }
- if (connection == null) continue;
- log.log(Level.FINE, "Got a status page request.");
- String requestString = "";
- OutputStream output = null;
- try (BufferedReader br = new BufferedReader(new InputStreamReader(connection.getInputStream()))) {
- StringBuilder sb = new StringBuilder();
- while (true) {
- String s = br.readLine();
- if (s == null) throw new java.io.IOException("No data in HTTP request on socket " + connection.toString());
- if (s.length() > 4 && s.substring(0,4).equals("GET ")) {
- int nextSpace = s.indexOf(' ', 4);
- if (nextSpace == -1) {
- requestString = s.substring(4);
- } else {
- requestString = s.substring(4, nextSpace);
- }
- }
- if (s == null || s.equals("")) break;
- sb.append(s).append("\n");
- }
- log.log(Level.FINE, "Got HTTP request: " + sb.toString());
-
- HttpRequest httpRequest = null;
- StatusPageResponse response = null;
- try {
- httpRequest = new HttpRequest(requestString);
- // Static files are served directly by the HTTP server thread, since
- // it makes no sense to go via the fleetcontroller logic for these.
- RequestHandler contentHandler = staticContentRouter.resolveHandler(httpRequest);
- if (contentHandler != null) {
- response = contentHandler.handle(httpRequest);
- }
- } catch (Exception e) {
- response = new StatusPageResponse();
- response.setResponseCode(StatusPageResponse.ResponseCode.INTERNAL_SERVER_ERROR);
- StringBuilder content = new StringBuilder();
- response.writeHtmlHeader(content, "Internal Server Error");
- response.writeHtmlFooter(content, ExceptionUtils.getStackTraceAsString(e));
- response.writeContent(content.toString());
- }
- if (response == null) {
- synchronized(monitor) {
- currentHttpRequest = httpRequest;
- currentResponse = null;
- while (running) {
- if (currentResponse != null) {
- response = currentResponse;
- break;
- }
- monitor.wait(100);
- }
- }
- }
- if (response == null) {
- response = new StatusPageResponse();
- StringBuilder content = new StringBuilder();
- response.setContentType("text/html");
- response.writeHtmlHeader(content, "Failed to get response. Fleet controller probably in the process of shutting down.");
- response.writeHtmlFooter(content, "");
- response.writeContent(content.toString());
- }
-
- output = connection.getOutputStream();
- StringBuilder header = new StringBuilder();
- // TODO: per-response cache control
- header.append("HTTP/1.1 ")
- .append(response.getResponseCode().getCode())
- .append(" ")
- .append(response.getResponseCode().getMessage())
- .append("\r\n")
- .append("Date: ").append(new Date().toString()).append("\r\n")
- .append("Connection: Close\r\n")
- .append("Content-type: ").append(response.getContentType()).append("\r\n");
- if (response.isClientCachingEnabled()) {
- // TODO(vekterli): would be better to let HTTP handlers set header values in response
- DateFormat df = new SimpleDateFormat("EEE, d MMM yyyy HH:mm:ss z");
- df.setTimeZone(TimeZone.getTimeZone("GMT"));
- header.append("Last-Modified: ").append(df.format(startTime)).append("\r\n");
- } else {
- header.append("Expires: Fri, 01 Jan 1990 00:00:00 GMT\r\n")
- .append("Pragma: no-cache\r\n")
- .append("Cache-control: no-cache, must-revalidate\r\n");
- }
- header.append("\r\n");
- output.write(header.toString().getBytes());
- output.write(response.getOutputStream().toByteArray());
- } catch (java.io.IOException e) {
- log.log(e.getMessage().indexOf("Broken pipe") >= 0 ? Level.FINE : Level.INFO,
- "Failed to process HTTP request : " + e.getMessage());
- } catch (Exception e) {
- log.log(Level.WARNING, "Caught exception in HTTP server thread: "
- + e.getClass().getName() + ": " + e.getMessage());
- } finally {
- if (output != null) try {
- output.close();
- } catch (IOException e) {
- log.log(e.getMessage().indexOf("Broken pipe") >= 0 ? Level.FINE : Level.INFO,
- "Failed to close output stream on socket " + connection + ": " + e.getMessage());
- }
- if (connection != null) try{
- connection.close();
- } catch (IOException e) {
- log.log(Level.INFO, "Failed to close socket " + connection + ": " + e.getMessage());
- }
- }
- }
- } catch (InterruptedException e) {
- log.log(Level.FINE, "Status processing thread shut down by interrupt exception: " + e);
- }
- }
-
/**
* Very simple HTTP request class. This should be replaced the second
* the fleetcontroller e.g. moves into the container.
@@ -377,29 +134,4 @@ public class StatusPageServer implements Runnable, StatusPageServerInterface {
}
}
- public HttpRequest getCurrentHttpRequest() {
- synchronized (monitor) {
- return currentHttpRequest;
- }
- }
-
- public void answerCurrentStatusRequest(StatusPageResponse r) {
- if (!isConnected()) {
- long time = timer.getCurrentTimeInMillis();
- try{
- connect();
- } catch (Exception e) {
- if (!e.getMessage().equals(lastConnectError) || time - lastConnectErrorTime > 60 * 1000) {
- lastConnectError = e.getMessage();
- lastConnectErrorTime = time;
- log.log(Level.WARNING, "Failed to initialize HTTP status server server socket: " + e.getMessage());
- }
- }
- }
- synchronized (monitor) {
- currentResponse = r;
- currentHttpRequest = null; // Avoid fleetcontroller processing request more than once
- }
- }
-
}
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 1587e2696b8..f8bf387ce41 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
@@ -9,7 +9,6 @@ import com.yahoo.jrt.Target;
import com.yahoo.jrt.Transport;
import com.yahoo.jrt.slobrok.api.BackOffPolicy;
import com.yahoo.jrt.slobrok.server.Slobrok;
-import java.util.logging.Level;
import com.yahoo.log.LogSetup;
import com.yahoo.vdslib.distribution.ConfiguredNode;
import com.yahoo.vdslib.state.ClusterState;
@@ -22,7 +21,7 @@ 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.statuspage.StatusPageServer;
+import com.yahoo.vespa.clustercontroller.core.status.StatusHandler;
import com.yahoo.vespa.clustercontroller.core.status.statuspage.StatusPageServerInterface;
import com.yahoo.vespa.clustercontroller.core.testutils.WaitCondition;
import com.yahoo.vespa.clustercontroller.core.testutils.WaitTask;
@@ -40,9 +39,11 @@ import java.util.Collection;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
+import java.util.Objects;
import java.util.Set;
import java.util.TreeMap;
import java.util.TreeSet;
+import java.util.logging.Level;
import java.util.logging.Logger;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
@@ -149,6 +150,7 @@ public abstract class FleetControllerTest implements Waiter {
}
FleetController createFleetController(boolean useFakeTimer, FleetControllerOptions options, boolean startThread, StatusPageServerInterface status) throws Exception {
+ Objects.requireNonNull(status, "status server cannot be null");
Timer timer = useFakeTimer ? this.timer : new RealTimer();
MetricUpdater metricUpdater = new MetricUpdater(new NoMetricReporter(), options.fleetControllerIndex);
EventLog log = new EventLog(timer, metricUpdater);
@@ -169,9 +171,6 @@ public abstract class FleetControllerTest implements Waiter {
options.nodeStateRequestRoundTripTimeMaxSeconds);
SlobrokClient lookUp = new SlobrokClient(timer);
lookUp.setSlobrokConnectionSpecs(new String[0]);
- if (status == null) {
- status = new StatusPageServer(timer, timer, options.httpPort);
- }
RpcServer rpcServer = new RpcServer(timer, timer, options.clusterName, options.fleetControllerIndex, options.slobrokBackOffPolicy);
DatabaseHandler database = new DatabaseHandler(new ZooKeeperDatabaseFactory(), timer, options.zooKeeperServerAddress, options.fleetControllerIndex, timer);
StateChangeHandler stateGenerator = new StateChangeHandler(timer, log, metricUpdater);
@@ -189,7 +188,7 @@ public abstract class FleetControllerTest implements Waiter {
}
protected void setUpFleetController(boolean useFakeTimer, FleetControllerOptions options, boolean startThread) throws Exception {
- setUpFleetController(useFakeTimer, options, startThread, null);
+ setUpFleetController(useFakeTimer, options, startThread, new StatusHandler.ContainerStatusPageServer());
}
protected void setUpFleetController(boolean useFakeTimer, FleetControllerOptions options, boolean startThread, StatusPageServerInterface status) throws Exception {
if (slobrok == null) setUpSystem(useFakeTimer, options);
@@ -209,7 +208,7 @@ public abstract class FleetControllerTest implements Waiter {
void startFleetController() throws Exception {
if (fleetController == null) {
- fleetController = createFleetController(usingFakeTimer, options, true, null);
+ fleetController = createFleetController(usingFakeTimer, options, true, new StatusHandler.ContainerStatusPageServer());
} else {
log.log(Level.WARNING, "already started fleetcontroller, not starting another");
}
diff --git a/clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/MasterElectionTest.java b/clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/MasterElectionTest.java
index d14f6701288..896f73ce6bf 100644
--- a/clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/MasterElectionTest.java
+++ b/clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/MasterElectionTest.java
@@ -12,6 +12,7 @@ import com.yahoo.vdslib.state.ClusterState;
import com.yahoo.vdslib.state.NodeState;
import com.yahoo.vdslib.state.NodeType;
import com.yahoo.vdslib.state.State;
+import com.yahoo.vespa.clustercontroller.core.status.StatusHandler;
import org.junit.Ignore;
import org.junit.Rule;
import org.junit.Test;
@@ -60,7 +61,7 @@ public class MasterElectionTest extends FleetControllerTest {
for (int i=0; i<count; ++i) {
FleetControllerOptions nodeOptions = options.clone();
nodeOptions.fleetControllerIndex = i;
- fleetControllers.add(createFleetController(usingFakeTimer, nodeOptions, true, null));
+ fleetControllers.add(createFleetController(usingFakeTimer, nodeOptions, true, new StatusHandler.ContainerStatusPageServer()));
}
}
@@ -143,14 +144,15 @@ public class MasterElectionTest extends FleetControllerTest {
assertFalse("Fleet controller " + i, fleetControllers.get(i).isMaster());
}
+ StatusHandler.ContainerStatusPageServer statusPageServer = new StatusHandler.ContainerStatusPageServer();
log.log(Level.INFO, "STARTING FLEET CONTROLLER 2");
- fleetControllers.set(2, createFleetController(usingFakeTimer, fleetControllers.get(2).getOptions(), true, null));
+ fleetControllers.set(2, createFleetController(usingFakeTimer, fleetControllers.get(2).getOptions(), true, statusPageServer));
waitForMaster(2);
log.log(Level.INFO, "STARTING FLEET CONTROLLER 0");
- fleetControllers.set(0, createFleetController(usingFakeTimer, fleetControllers.get(0).getOptions(), true, null));
+ fleetControllers.set(0, createFleetController(usingFakeTimer, fleetControllers.get(0).getOptions(), true, statusPageServer));
waitForMaster(0);
log.log(Level.INFO, "STARTING FLEET CONTROLLER 1");
- fleetControllers.set(1, createFleetController(usingFakeTimer, fleetControllers.get(1).getOptions(), true, null));
+ fleetControllers.set(1, createFleetController(usingFakeTimer, fleetControllers.get(1).getOptions(), true, statusPageServer));
waitForMaster(0);
log.log(Level.INFO, "SHUTTING DOWN FLEET CONTROLLER 4");
@@ -538,7 +540,7 @@ public class MasterElectionTest extends FleetControllerTest {
waitForMaster(1);
waitForCompleteCycle(1);
- fleetControllers.set(0, createFleetController(usingFakeTimer, fleetControllers.get(0).getOptions(), true, null));
+ fleetControllers.set(0, createFleetController(usingFakeTimer, fleetControllers.get(0).getOptions(), true, new StatusHandler.ContainerStatusPageServer()));
waitForMaster(0);
waitForCompleteCycle(0);
diff --git a/clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/StatusPagesTest.java b/clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/StatusPagesTest.java
deleted file mode 100644
index f761538cf1e..00000000000
--- a/clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/StatusPagesTest.java
+++ /dev/null
@@ -1,374 +0,0 @@
-// Copyright 2017 Yahoo Holdings. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
-package com.yahoo.vespa.clustercontroller.core;
-
-import com.yahoo.vdslib.distribution.Distribution;
-import com.yahoo.vdslib.state.Node;
-import com.yahoo.vdslib.state.NodeState;
-import com.yahoo.vdslib.state.NodeType;
-import com.yahoo.vdslib.state.State;
-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.utils.communication.http.HttpRequest;
-import com.yahoo.vespa.clustercontroller.utils.communication.http.HttpResult;
-import org.codehaus.jettison.json.JSONObject;
-import org.junit.Test;
-
-import java.io.BufferedWriter;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStreamWriter;
-import java.net.Socket;
-import java.nio.charset.StandardCharsets;
-import java.text.DateFormat;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.TimeZone;
-import java.util.logging.Logger;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
-public class StatusPagesTest extends FleetControllerTest {
-
- public static Logger log = Logger.getLogger(StatusPagesTest.class.getName());
-
- private String doHttpGetRequest(String request, Date ifModifiedSince) throws IOException {
- int statusPort = fleetController.getHttpPort();
- Socket socket = new Socket("localhost", statusPort);
-
- BufferedWriter bw = new BufferedWriter(new OutputStreamWriter(socket.getOutputStream()));
- bw.write("GET " + request + " HTTP/1.1\r\n");
- if (ifModifiedSince != null) {
- DateFormat df = new SimpleDateFormat("EEE, d MMM yyyy HH:mm:ss z");
- df.setTimeZone(TimeZone.getTimeZone("GMT"));
- bw.write("If-Modified-Since: " + df.format(ifModifiedSince) + "\r\n");
- }
- bw.write("\r\n");
- bw.flush();
-
- InputStream stream = socket.getInputStream();
- ByteArrayOutputStream output = new ByteArrayOutputStream();
- try {
- byte [] buf = new byte[4096];
- while (true) {
- int read = stream.read(buf);
- if (read<=0) {
- break;
- }
- output.write(buf, 0, read);
- }
- output.close();
- return output.toString();
- } finally {
- stream.close();
- bw.close();
- }
- }
-
- private String doHttpGetRequest(String request) throws IOException {
- return doHttpGetRequest(request, null);
- }
-
- @Test
- public void testStatusThroughContainer() throws Exception {
- startingTest("StatusPagesTest::testStatusThroughContainer()");
- FleetControllerOptions options = defaultOptions("mycluster");
- options.setStorageDistribution(new Distribution(Distribution.getDefaultDistributionConfig(3, 10)));
- final StatusHandler.ContainerStatusPageServer statusServer = new StatusHandler.ContainerStatusPageServer();
- setUpFleetController(true, options, true, statusServer);
- setUpVdsNodes(true, new DummyVdsNodeOptions());
- waitForStableSystem();
-
- //ThreadPoolExecutor executor = new ThreadPoolExecutor(10, 100, 100, TimeUnit.SECONDS, new ArrayBlockingQueue<Runnable>(1000));
- //FleetControllerComponent fcComp = new FleetControllerComponent();
- //fcComp.addFleetController("mycluster", fleetController, statusServer);
- StatusHandler comp = new StatusHandler(new StatusHandler.ClusterStatusPageServerSet() {
- @Override
- public StatusHandler.ContainerStatusPageServer get(String cluster) {
- return ("mycluster".equals(cluster) ? statusServer : null);
- }
-
- @Override
- public Map<String, StatusHandler.ContainerStatusPageServer> getAll() {
- Map<String, StatusHandler.ContainerStatusPageServer> map = new HashMap<>();
- map.put("mycluster", statusServer);
- return map;
- }
- });
-
- {
- HttpRequest request = new HttpRequest().setPath("/clustercontroller-status/v1");
- HttpResult result = comp.handleRequest(request);
- assertEquals(result.toString(true), 200, result.getHttpReturnCode());
- assertEquals("<title>clusters</title>\n<a href=\"./mycluster\">mycluster</a><br>\n", result.getContent().toString());
- }
- {
- HttpRequest request = new HttpRequest().setPath("/clustercontroller-status/v1/");
- HttpResult result = comp.handleRequest(request);
- assertEquals(result.toString(true), 200, result.getHttpReturnCode());
- assertEquals("<title>clusters</title>\n<a href=\"./mycluster\">mycluster</a><br>\n", result.getContent().toString());
- }
- {
- HttpRequest request = new HttpRequest().setPath("/clustercontroller-status/v1/mycluster");
- HttpResult result = comp.handleRequest(request);
- assertEquals(result.toString(true), 200, result.getHttpReturnCode());
- assertTrue(result.toString(true), result.getContent().toString().contains(
- "mycluster Cluster Controller 0 Status Page"));
- }
- {
- HttpRequest request = new HttpRequest().setPath("/clustercontroller-status/v1/mycluster/");
- HttpResult result = comp.handleRequest(request);
- assertEquals(result.toString(true), 200, result.getHttpReturnCode());
- assertTrue(result.toString(true), result.getContent().toString().contains(
- "mycluster Cluster Controller 0 Status Page"));
- assertTrue(result.toString(true), result.getContent().toString().contains(
- "href=\"mycluster/node=distributor.0\""));
- assertTrue(result.toString(true), result.getContent().toString().contains(
- "href=\"mycluster/node=storage.0\""));
- }
- {
- HttpRequest request = new HttpRequest().setPath("/clustercontroller-status/v1/mycluster/node=storage.0");
- HttpResult result = comp.handleRequest(request);
- assertEquals(result.toString(true), 200, result.getHttpReturnCode());
- assertTrue(result.toString(true), result.getContent().toString().contains(
- "Node status for storage.0"));
- assertTrue(result.toString(true), result.getContent().toString().contains(
- "href=\"..\""));
- }
- {
- HttpRequest request = new HttpRequest().setPath("/clustercontroller-status/v1/foo");
- HttpResult result = comp.handleRequest(request);
- assertEquals(result.toString(true), 404, result.getHttpReturnCode());
- }
- {
- HttpRequest request = new HttpRequest().setPath("/foobar/v1/mycluster/");
- HttpResult result = comp.handleRequest(request);
- assertEquals(result.toString(true), 404, result.getHttpReturnCode());
- }
- {
- HttpRequest request = new HttpRequest().setPath("/clustercontroller-status/v2/");
- HttpResult result = comp.handleRequest(request);
- assertEquals(result.toString(true), 404, result.getHttpReturnCode());
- }
- //executor.shutdown();
- }
-
- @Test
- public void testZooKeeperAddressSplitting() {
- String rawAddress = "conc1.foo.yahoo.com:2181,conc2.foo.yahoo.com:2181,"
- + "dp1.foo.yahoo.com:2181,dp2.foo.yahoo.com:2181,"
- + "dp3.foo.yahoo.com:2181";
- String result = "conc1.foo.yahoo.com:2181, conc2.foo.yahoo.com:2181, "
- + "dp1.foo.yahoo.com:2181, dp2.foo.yahoo.com:2181, "
- + "dp3.foo.yahoo.com:2181";
- String split = FleetControllerOptions.splitZooKeeperAddress(rawAddress);
- assertEquals(result, split);
- }
-
- @Test
- public void testSimpleConnectionWithSomeContent() throws Exception {
- // Set this to true temporary if you want to check status page from browser. Should be false in checked in code always.
- boolean haltTestToViewStatusPage = false;
- startingTest("StatusPagesTest::testSimpleConnectionWithSomeContent()");
- FleetControllerOptions options = defaultOptions("mycluster");
- options.setStorageDistribution(new Distribution(Distribution.getDefaultDistributionConfig(3, 10)));
- //options.minRatioOfStorageNodesUp = 0.99;
- if (haltTestToViewStatusPage) {
- options.httpPort = 19234;
- }
- setUpFleetController(true, options);
- setUpVdsNodes(true, new DummyVdsNodeOptions());
- waitForStableSystem();
-
- nodes.get(2).disconnectBreakConnection();
- nodes.get(5).disconnectAsShutdown();
- nodes.get(7).disconnectSlobrok();
-
- fleetController.getCluster().getNodeInfo(new Node(NodeType.STORAGE, 3)).setWantedState(new NodeState(NodeType.STORAGE, State.MAINTENANCE).setDescription("Test&<>special"));
-
- String content = doHttpGetRequest("/");
-
- assertTrue(content, content.contains("<html>"));
- assertTrue(content, content.contains("</html>"));
- assertTrue(content, content.contains("Baseline cluster state"));
- assertTrue(content, content.contains("Cluster states"));
- assertTrue(content, content.contains("Event log"));
-
- if (haltTestToViewStatusPage) {
- System.err.println(content);
- try{
- Thread.sleep(1000000);
- } catch (InterruptedException e) {}
- }
- }
-
- @Test
- public void testNodePage() throws Exception {
- startingTest("StatusPagesTest::testNodePage()");
- FleetControllerOptions options = defaultOptions("mycluster");
- options.setStorageDistribution(new Distribution(Distribution.getDefaultDistributionConfig(3, 10)));
- setUpFleetController(true, options);
- setUpVdsNodes(true, new DummyVdsNodeOptions());
- waitForStableSystem();
-
- String content = doHttpGetRequest("/node=storage.0");
-
- assertTrue(content, content.contains("<html>"));
- assertTrue(content, content.contains("</html>"));
- assertTrue(content, content.contains("Node status for storage.0"));
- assertTrue(content, content.contains("REPORTED"));
- assertTrue(content, content.contains("Altered node state in cluster state from"));
- //System.err.println(sb.toString());
- }
-
- @Test
- public void testErrorResponseCode() throws Exception {
- startingTest("StatusPagesTest::testNodePage()");
- FleetControllerOptions options = defaultOptions("mycluster");
- options.setStorageDistribution(new Distribution(Distribution.getDefaultDistributionConfig(3, 10)));
- setUpFleetController(true, options);
- setUpVdsNodes(true, new DummyVdsNodeOptions());
- waitForStableSystem();
-
- String content = doHttpGetRequest("/fraggle/rock");
-
- assertTrue(content.contains("404 Not Found"));
- //System.err.println(sb.toString());
- }
-
- private StatusPageServer.HttpRequest makeHttpRequest(String request) {
- return new StatusPageServer.HttpRequest(request);
- }
-
- @Test
- public void testHttpRequestParsing() {
- {
- StatusPageServer.HttpRequest request = makeHttpRequest("/") ;
- assertEquals("/", request.getPath());
- assertFalse(request.hasQueryParameters());
- }
- {
- StatusPageServer.HttpRequest request = makeHttpRequest("/foo/bar");
- assertEquals("/foo/bar", request.getPath());
- assertFalse(request.hasQueryParameters());
- }
- {
- StatusPageServer.HttpRequest request = makeHttpRequest("/foo/bar?baz=baff");
- assertEquals("/foo/bar", request.getPath());
- assertTrue(request.hasQueryParameters());
- assertEquals("baff", request.getQueryParameter("baz"));
- }
- {
- StatusPageServer.HttpRequest request = makeHttpRequest("/?baz=baff&blarg=blee");
- assertEquals("/", request.getPath());
- assertTrue(request.hasQueryParameters());
- assertEquals("baff", request.getQueryParameter("baz"));
- assertEquals("blee", request.getQueryParameter("blarg"));
- }
- {
- StatusPageServer.HttpRequest request = makeHttpRequest("/node=storage.101?showlocal");
- assertEquals("/node=storage.101", request.getPath());
- assertTrue(request.hasQueryParameters());
- assertTrue(request.hasQueryParameter("showlocal"));
- assertNull(request.getQueryParameter("showlocal"));
- }
- }
-
- private static class DummyRequestHandler implements StatusPageServer.RequestHandler {
- private String returnData;
- DummyRequestHandler(String returnData) {
- this.returnData = returnData;
- }
-
- @Override
- public StatusPageResponse handle(StatusPageServer.HttpRequest request) {
- StatusPageResponse response = new StatusPageResponse();
- response.writeContent(returnData);
- return response;
- }
- }
-
- private String invokeHandler(StatusPageServer.RequestRouter router, String request) {
- StatusPageServer.HttpRequest httpRequest = makeHttpRequest(request);
- StatusPageServer.RequestHandler handler = router.resolveHandler(httpRequest);
- if (handler == null) {
- return null;
- }
- return handler.handle(httpRequest).getOutputStream().toString(StandardCharsets.UTF_8);
- }
-
- @Test
- public void testRequestRouting() {
- StatusPageServer.PatternRequestRouter router = new StatusPageServer.PatternRequestRouter();
- router.addHandler("^/alerts/red.*", new DummyRequestHandler("red alert!"));
- router.addHandler("^/alerts.*", new DummyRequestHandler("beige alert"));
- router.addHandler("^/$", new DummyRequestHandler("root"));
- assertEquals("root", invokeHandler(router, "/"));
- assertEquals("beige alert", invokeHandler(router, "/alerts"));
- assertEquals("beige alert", invokeHandler(router, "/alerts?foo"));
- assertEquals("red alert!", invokeHandler(router, "/alerts/red"));
- assertEquals("red alert!", invokeHandler(router, "/alerts/red/blue"));
- assertNull(invokeHandler(router, "/blarg"));
- }
-
- private String[] getResponseParts(String response) {
- int offset = response.indexOf("\r\n\r\n");
- if (offset == -1) {
- throw new IllegalStateException("No HTTP header delimiter found");
- }
- return new String[] {
- response.substring(0, offset + 2), // all header lines must have linebreaks
- response.substring(offset + 4)
- };
- }
-
- @Test
- public void testStateServing() throws Exception {
- startingTest("StatusPagesTest::testStateServing()");
- FleetControllerOptions options = defaultOptions("mycluster");
- setUpFleetController(true, options);
- fleetController.updateOptions(options, 5);
- waitForCompleteCycle();
- {
- String content = doHttpGetRequest("/state/v1/health");
- String[] parts = getResponseParts(content);
- String body = parts[1];
- String expected =
- "{\n" +
- " \"status\" : {\n" +
- " \"code\" : \"up\"\n" +
- " },\n" +
- " \"config\" : {\n" +
- " \"component\" : {\n" +
- " \"generation\" : 5\n" +
- " }\n" +
- " }\n" +
- "}";
- assertEquals(expected, body);
- // Check that it actually parses
- new JSONObject(expected);
- }
- }
-
- @Test
- public void testClusterStateServing() throws Exception {
- startingTest("StatusPagesTest::testClusterStateServing()");
- FleetControllerOptions options = defaultOptions("mycluster");
- setUpFleetController(true, options);
- fleetController.updateOptions(options, 5);
- waitForCompleteCycle();
- {
- String content = doHttpGetRequest("/clusterstate");
- String[] parts = getResponseParts(content);
- String body = parts[1];
- String expected = "version:2 cluster:d";
- assertEquals(expected, body);
- }
- }
-}
diff --git a/clustercontroller-reindexer/src/main/java/ai/vespa/reindexing/Reindexer.java b/clustercontroller-reindexer/src/main/java/ai/vespa/reindexing/Reindexer.java
index b2f1d833df7..13ed9800db3 100644
--- a/clustercontroller-reindexer/src/main/java/ai/vespa/reindexing/Reindexer.java
+++ b/clustercontroller-reindexer/src/main/java/ai/vespa/reindexing/Reindexer.java
@@ -11,6 +11,7 @@ import com.yahoo.documentapi.ProgressToken;
import com.yahoo.documentapi.VisitorControlHandler;
import com.yahoo.documentapi.VisitorParameters;
import com.yahoo.documentapi.messagebus.protocol.DocumentProtocol;
+import com.yahoo.jdisc.Metric;
import com.yahoo.vespa.curator.Lock;
import java.time.Clock;
@@ -44,15 +45,13 @@ public class Reindexer {
private final Map<DocumentType, Instant> ready;
private final ReindexingCurator database;
private final Function<VisitorParameters, Runnable> visitorSessions;
+ private final ReindexingMetrics metrics;
private final Clock clock;
private final Phaser phaser = new Phaser(2); // Reindexer and visitor.
- private Reindexing reindexing;
- private Status status;
-
@Inject
public Reindexer(Cluster cluster, Map<DocumentType, Instant> ready, ReindexingCurator database,
- DocumentAccess access, Clock clock) {
+ DocumentAccess access, Metric metric, Clock clock) {
this(cluster,
ready,
database,
@@ -64,11 +63,12 @@ public class Reindexer {
throw new IllegalStateException(e);
}
},
+ metric,
clock);
}
Reindexer(Cluster cluster, Map<DocumentType, Instant> ready, ReindexingCurator database,
- Function<VisitorParameters, Runnable> visitorSessions, Clock clock) {
+ Function<VisitorParameters, Runnable> visitorSessions, Metric metric, Clock clock) {
for (DocumentType type : ready.keySet())
cluster.bucketSpaceOf(type); // Verifies this is known.
@@ -76,10 +76,11 @@ public class Reindexer {
this.ready = new TreeMap<>(ready); // Iterate through document types in consistent order.
this.database = database;
this.visitorSessions = visitorSessions;
+ this.metrics = new ReindexingMetrics(metric, cluster.name);
this.clock = clock;
}
- /** Lets the reindexere abort any ongoing visit session, wait for it to complete normally, then exit. */
+ /** Lets the reindexer abort any ongoing visit session, wait for it to complete normally, then exit. */
public void shutdown() {
phaser.forceTermination(); // All parties waiting on this phaser are immediately allowed to proceed.
}
@@ -90,12 +91,16 @@ public class Reindexer {
throw new IllegalStateException("Already shut down");
try (Lock lock = database.lockReindexing()) {
+ Reindexing reindexing = updateWithReady(ready, database.readReindexing(), clock.instant());
+ database.writeReindexing(reindexing);
+ metrics.dump(reindexing);
+
for (DocumentType type : ready.keySet()) { // We consider only document types for which we have config.
if (ready.get(type).isAfter(clock.instant()))
log.log(INFO, "Received config for reindexing which is ready in the future — will process later " +
"(" + ready.get(type) + " is after " + clock.instant() + ")");
else
- progress(type);
+ progress(type, new AtomicReference<>(reindexing), new AtomicReference<>(reindexing.status().get(type)));
if (phaser.isTerminated())
break;
@@ -103,77 +108,86 @@ public class Reindexer {
}
}
+ static Reindexing updateWithReady(Map<DocumentType, Instant> ready, Reindexing reindexing, Instant now) {
+ for (DocumentType type : ready.keySet()) { // We consider update for document types for which we have config.
+ if ( ! ready.get(type).isAfter(now)) {
+ Status status = reindexing.status().getOrDefault(type, Status.ready(now)
+ .running()
+ .successful(now));
+ if (status.startedAt().isBefore(ready.get(type)))
+ status = Status.ready(now);
+
+ reindexing = reindexing.with(type, status);
+ }
+ }
+ return reindexing;
+ }
+
@SuppressWarnings("fallthrough") // (ノಠ ∩ಠ)ノ彡( \o°o)\
- private void progress(DocumentType type) {
- // If this is a new document type (or a new cluster), no reindexing is required.
- reindexing = database.readReindexing();
- status = reindexing.status().getOrDefault(type,
- Status.ready(clock.instant())
- .running()
- .successful(clock.instant()));
- if (ready.get(type).isAfter(status.startedAt()))
- status = Status.ready(clock.instant()); // Need to restart, as a newer reindexing is required.
-
- database.writeReindexing(reindexing = reindexing.with(type, status));
-
- switch (status.state()) {
+ private void progress(DocumentType type, AtomicReference<Reindexing> reindexing, AtomicReference<Status> status) {
+
+ database.writeReindexing(reindexing.updateAndGet(value -> value.with(type, status.get())));
+ metrics.dump(reindexing.get());
+
+ switch (status.get().state()) {
default:
- log.log(WARNING, "Unknown reindexing state '" + status.state() + "'");
+ log.log(WARNING, "Unknown reindexing state '" + status.get().state() + "'");
case FAILED:
log.log(FINE, () -> "Not continuing reindexing of " + type + " due to previous failure");
case SUCCESSFUL: // Intentional fallthrough — all three are done states.
return;
case RUNNING:
log.log(WARNING, "Unexpected state 'RUNNING' of reindexing of " + type);
- case READY: // Intentional fallthrough — must just assume we failed updating state when exiting previously.
+ case READY: // Intentional fallthrough — must just assume we failed updating state when exiting previously.
log.log(FINE, () -> "Running reindexing of " + type);
}
// Visit buckets until they're all done, or until we are interrupted.
- status = status.running();
+ status.updateAndGet(Status::running);
AtomicReference<Instant> progressLastStored = new AtomicReference<>(clock.instant());
VisitorControlHandler control = new VisitorControlHandler() {
@Override
public void onProgress(ProgressToken token) {
super.onProgress(token);
- status = status.progressed(token);
+ status.updateAndGet(value -> value.progressed(token));
if (progressLastStored.get().isBefore(clock.instant().minusSeconds(10))) {
progressLastStored.set(clock.instant());
- database.writeReindexing(reindexing = reindexing.with(type, status));
+ database.writeReindexing(reindexing.updateAndGet(value -> value.with(type, status.get())));
+ metrics.dump(reindexing.get());
}
}
@Override
public void onDone(CompletionCode code, String message) {
super.onDone(code, message);
- phaser.arriveAndAwaitAdvance(); // Synchronize with the reindex thread.
+ phaser.arriveAndAwaitAdvance(); // Synchronize with the reindexer control thread.
}
};
- VisitorParameters parameters = createParameters(type, status.progress().orElse(null));
+ VisitorParameters parameters = createParameters(type, status.get().progress().orElse(null));
parameters.setControlHandler(control);
- Runnable sessionShutdown = visitorSessions.apply(parameters);
+ Runnable sessionShutdown = visitorSessions.apply(parameters); // Also starts the visitor session.
- // Wait until done; or until termination is forced, in which case we abort the visit and wait for it to complete.
- phaser.arriveAndAwaitAdvance(); // Synchronize with the visitor completion thread.
- sessionShutdown.run();
+ // Wait until done; or until termination is forced, in which we shut down the visitor session immediately.
+ phaser.arriveAndAwaitAdvance(); // Synchronize with visitor completion.
+ sessionShutdown.run(); // Shutdown aborts the session, then waits for it to terminate normally.
- // If we were interrupted, the result may not yet be set in the control handler.
switch (control.getResult().getCode()) {
default:
log.log(WARNING, "Unexpected visitor result '" + control.getResult().getCode() + "'");
- case FAILURE: // Intentional fallthrough — this is an error.
+ case FAILURE: // Intentional fallthrough — this is an error.
log.log(WARNING, "Visiting failed: " + control.getResult().getMessage());
- status = status.failed(clock.instant(), control.getResult().getMessage());
+ status.updateAndGet(value -> value.failed(clock.instant(), control.getResult().getMessage()));
break;
case ABORTED:
- log.log(FINE, () -> "Halting reindexing of " + type + " due to shutdown — will continue later");
- status = status.halted();
+ log.log(FINE, () -> "Halting reindexing of " + type + " due to shutdown — will continue later");
+ status.updateAndGet(Status::halted);
break;
case SUCCESS:
- log.log(INFO, "Completed reindexing of " + type + " after " + Duration.between(status.startedAt(), clock.instant()));
- status = status.successful(clock.instant());
+ log.log(INFO, "Completed reindexing of " + type + " after " + Duration.between(status.get().startedAt(), clock.instant()));
+ status.updateAndGet(value -> value.successful(clock.instant()));
}
- database.writeReindexing(reindexing.with(type, status));
+ database.writeReindexing(reindexing.updateAndGet(value -> value.with(type, status.get())));
+ metrics.dump(reindexing.get());
}
VisitorParameters createParameters(DocumentType type, ProgressToken progress) {
diff --git a/clustercontroller-reindexer/src/main/java/ai/vespa/reindexing/Reindexing.java b/clustercontroller-reindexer/src/main/java/ai/vespa/reindexing/Reindexing.java
index 792889e4aa8..51322c37a7d 100644
--- a/clustercontroller-reindexer/src/main/java/ai/vespa/reindexing/Reindexing.java
+++ b/clustercontroller-reindexer/src/main/java/ai/vespa/reindexing/Reindexing.java
@@ -121,7 +121,7 @@ public class Reindexing {
public Status failed(Instant now, String message) {
if (state != State.RUNNING)
throw new IllegalStateException("Current state must be RUNNING when changing to FAILED");
- return new Status(startedAt, requireNonNull(now), null, State.FAILED, requireNonNull(message));
+ return new Status(startedAt, requireNonNull(now), progress, State.FAILED, requireNonNull(message));
}
public Instant startedAt() {
diff --git a/clustercontroller-reindexer/src/main/java/ai/vespa/reindexing/ReindexingMaintainer.java b/clustercontroller-reindexer/src/main/java/ai/vespa/reindexing/ReindexingMaintainer.java
index 740a04619d1..a336ad02f20 100644
--- a/clustercontroller-reindexer/src/main/java/ai/vespa/reindexing/ReindexingMaintainer.java
+++ b/clustercontroller-reindexer/src/main/java/ai/vespa/reindexing/ReindexingMaintainer.java
@@ -12,6 +12,7 @@ import com.yahoo.document.DocumentType;
import com.yahoo.document.DocumentTypeManager;
import com.yahoo.document.config.DocumentmanagerConfig;
import com.yahoo.documentapi.DocumentAccess;
+import com.yahoo.jdisc.Metric;
import com.yahoo.net.HostName;
import com.yahoo.vespa.config.content.AllClustersBucketSpacesConfig;
import com.yahoo.vespa.config.content.reindexing.ReindexingConfig;
@@ -52,13 +53,14 @@ public class ReindexingMaintainer extends AbstractComponent {
@Inject
public ReindexingMaintainer(@SuppressWarnings("unused") VespaZooKeeperServer ensureZkHasStarted,
+ Metric metric,
DocumentAccess access, ZookeepersConfig zookeepersConfig,
ClusterListConfig clusterListConfig, AllClustersBucketSpacesConfig allClustersBucketSpacesConfig,
ReindexingConfig reindexingConfig, DocumentmanagerConfig documentmanagerConfig) {
- this(Clock.systemUTC(), access, zookeepersConfig, clusterListConfig, allClustersBucketSpacesConfig, reindexingConfig, documentmanagerConfig);
+ this(Clock.systemUTC(), metric, access, zookeepersConfig, clusterListConfig, allClustersBucketSpacesConfig, reindexingConfig, documentmanagerConfig);
}
- ReindexingMaintainer(Clock clock, DocumentAccess access, ZookeepersConfig zookeepersConfig,
+ ReindexingMaintainer(Clock clock, Metric metric, DocumentAccess access, ZookeepersConfig zookeepersConfig,
ClusterListConfig clusterListConfig, AllClustersBucketSpacesConfig allClustersBucketSpacesConfig,
ReindexingConfig reindexingConfig, DocumentmanagerConfig documentmanagerConfig) {
DocumentTypeManager manager = new DocumentTypeManager(documentmanagerConfig);
@@ -68,6 +70,7 @@ public class ReindexingMaintainer extends AbstractComponent {
reindexingConfig.clusterName(),
manager),
access,
+ metric,
clock);
this.executor = new ScheduledThreadPoolExecutor(1, new DaemonThreadFactory("reindexer-"));
if (reindexingConfig.enabled())
diff --git a/clustercontroller-reindexer/src/main/java/ai/vespa/reindexing/ReindexingMetrics.java b/clustercontroller-reindexer/src/main/java/ai/vespa/reindexing/ReindexingMetrics.java
new file mode 100644
index 00000000000..5e536d1f2ee
--- /dev/null
+++ b/clustercontroller-reindexer/src/main/java/ai/vespa/reindexing/ReindexingMetrics.java
@@ -0,0 +1,48 @@
+// Copyright Verizon Media. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+package ai.vespa.reindexing;
+
+import com.yahoo.documentapi.ProgressToken;
+import com.yahoo.jdisc.Metric;
+
+import java.time.Clock;
+import java.util.Map;
+
+import static ai.vespa.reindexing.Reindexing.State.SUCCESSFUL;
+
+/**
+ * Metrics for reindexing in a content cluster.
+ *
+ * @author jonmv
+ */
+class ReindexingMetrics {
+
+ private final Metric metric;
+ private final String cluster;
+
+ ReindexingMetrics(Metric metric, String cluster) {
+ this.metric = metric;
+ this.cluster = cluster;
+ }
+
+ void dump(Reindexing reindexing) {
+ reindexing.status().forEach((type, status) -> {
+ metric.set("reindexing.progress",
+ status.progress().map(ProgressToken::percentFinished).map(percentage -> percentage * 1e-2)
+ .orElse(status.state() == SUCCESSFUL ? 1.0 : 0.0),
+ metric.createContext(Map.of("clusterid", cluster,
+ "documenttype", type.getName(),
+ "state", toString(status.state()))));
+ });
+ }
+
+ private static String toString(Reindexing.State state) {
+ switch (state) {
+ case READY: return "pending";
+ case RUNNING: return "running";
+ case FAILED: return "failed";
+ case SUCCESSFUL: return "successful";
+ default: throw new IllegalArgumentException("Unknown reindexing state '" + state + "'");
+ }
+ }
+
+}
diff --git a/clustercontroller-reindexer/src/test/java/ai/vespa/reindexing/ReindexerTest.java b/clustercontroller-reindexer/src/test/java/ai/vespa/reindexing/ReindexerTest.java
index a5ad2ba32f1..b0ffdf8ae60 100644
--- a/clustercontroller-reindexer/src/test/java/ai/vespa/reindexing/ReindexerTest.java
+++ b/clustercontroller-reindexer/src/test/java/ai/vespa/reindexing/ReindexerTest.java
@@ -4,7 +4,6 @@ package ai.vespa.reindexing;
import ai.vespa.reindexing.Reindexer.Cluster;
import ai.vespa.reindexing.Reindexing.Status;
import ai.vespa.reindexing.ReindexingCurator.ReindexingLockException;
-import com.yahoo.document.Document;
import com.yahoo.document.DocumentType;
import com.yahoo.document.DocumentTypeManager;
import com.yahoo.document.config.DocumentmanagerConfig;
@@ -12,6 +11,7 @@ import com.yahoo.documentapi.ProgressToken;
import com.yahoo.documentapi.VisitorControlHandler;
import com.yahoo.documentapi.VisitorParameters;
import com.yahoo.documentapi.messagebus.protocol.DocumentProtocol;
+import com.yahoo.jdisc.test.MockMetric;
import com.yahoo.searchdefinition.derived.Deriver;
import com.yahoo.test.ManualClock;
import com.yahoo.vespa.curator.mock.MockCurator;
@@ -40,13 +40,13 @@ import static org.junit.jupiter.api.Assertions.fail;
*/
class ReindexerTest {
- static final Function<VisitorParameters, Runnable> failIfCalled = __ -> () -> { fail("Not supposed to run"); };
+ static final Function<VisitorParameters, Runnable> failIfCalled = __ -> () -> fail("Not supposed to run");
final DocumentmanagerConfig musicConfig = Deriver.getDocumentManagerConfig("src/test/resources/schemas/music.sd").build();
final DocumentTypeManager manager = new DocumentTypeManager(musicConfig);
final DocumentType music = manager.getDocumentType("music");
- final Document document1 = new Document(music, "id:ns:music::one");
final Cluster cluster = new Cluster("cluster", "id", Map.of(music, "default"));
+ final MockMetric metric = new MockMetric();
final ManualClock clock = new ManualClock(Instant.EPOCH);
ReindexingCurator database;
@@ -63,12 +63,13 @@ class ReindexerTest {
Map.of(music, Instant.EPOCH),
database,
failIfCalled,
+ metric,
clock));
}
@Test
void throwsWhenLockHeldElsewhere() throws InterruptedException, ExecutionException {
- Reindexer reindexer = new Reindexer(cluster, Map.of(music, Instant.EPOCH), database, failIfCalled, clock);
+ Reindexer reindexer = new Reindexer(cluster, Map.of(music, Instant.EPOCH), database, failIfCalled, metric, clock);
Executors.newSingleThreadExecutor().submit(database::lockReindexing).get();
assertThrows(ReindexingLockException.class, reindexer::reindex);
}
@@ -76,12 +77,13 @@ class ReindexerTest {
@Test
@Timeout(10)
void nothingToDoWithEmptyConfig() throws ReindexingLockException {
- new Reindexer(cluster, Map.of(), database, failIfCalled, clock).reindex();
+ new Reindexer(cluster, Map.of(), database, failIfCalled, metric, clock).reindex();
+ assertEquals(Map.of(), metric.metrics());
}
@Test
void testParameters() {
- Reindexer reindexer = new Reindexer(cluster, Map.of(), database, failIfCalled, clock);
+ Reindexer reindexer = new Reindexer(cluster, Map.of(), database, failIfCalled, metric, clock);
ProgressToken token = new ProgressToken();
VisitorParameters parameters = reindexer.createParameters(music, token);
assertEquals("music:[document]", parameters.getFieldSet());
@@ -98,14 +100,19 @@ class ReindexerTest {
void testReindexing() throws ReindexingLockException {
// Reindexer is told to update "music" documents no earlier than EPOCH, which is just now.
// Since "music" is a new document type, it is stored as just reindexed, and nothing else happens.
- new Reindexer(cluster, Map.of(music, Instant.EPOCH), database, failIfCalled, clock).reindex();
+ new Reindexer(cluster, Map.of(music, Instant.EPOCH), database, failIfCalled, metric, clock).reindex();
Reindexing reindexing = Reindexing.empty().with(music, Status.ready(Instant.EPOCH).running().successful(Instant.EPOCH));
assertEquals(reindexing, database.readReindexing());
+ assertEquals(Map.of("reindexing.progress", Map.of(Map.of("documenttype", "music",
+ "clusterid", "cluster",
+ "state", "successful"),
+ 1.0)),
+ metric.metrics());
// New config tells reindexer to reindex "music" documents no earlier than at 10 millis after EPOCH, which isn't yet.
// Nothing happens, since it's not yet time. This isn't supposed to happen unless high clock skew.
clock.advance(Duration.ofMillis(5));
- new Reindexer(cluster, Map.of(music, Instant.ofEpochMilli(10)), database, failIfCalled, clock).reindex();
+ new Reindexer(cluster, Map.of(music, Instant.ofEpochMilli(10)), database, failIfCalled, metric, clock).reindex();
assertEquals(reindexing, database.readReindexing());
// It's time to reindex the "music" documents — let this complete successfully.
@@ -116,13 +123,14 @@ class ReindexerTest {
database.writeReindexing(Reindexing.empty()); // Wipe database to verify we write data from reindexer.
executor.execute(() -> parameters.getControlHandler().onDone(VisitorControlHandler.CompletionCode.SUCCESS, "OK"));
return () -> shutDown.set(true);
- }, clock).reindex();
+ }, metric, clock).reindex();
reindexing = reindexing.with(music, Status.ready(clock.instant()).running().successful(clock.instant()));
assertEquals(reindexing, database.readReindexing());
assertTrue(shutDown.get(), "Session was shut down");
// One more reindexing, this time shut down before visit completes, but after progress is reported.
clock.advance(Duration.ofMillis(10));
+ metric.metrics().clear();
shutDown.set(false);
AtomicReference<Reindexer> aborted = new AtomicReference<>();
aborted.set(new Reindexer(cluster, Map.of(music, Instant.ofEpochMilli(20)), database, parameters -> {
@@ -133,11 +141,16 @@ class ReindexerTest {
shutDown.set(true);
parameters.getControlHandler().onDone(VisitorControlHandler.CompletionCode.ABORTED, "Shut down");
};
- }, clock));
+ }, metric, clock));
aborted.get().reindex();
reindexing = reindexing.with(music, Status.ready(clock.instant()).running().progressed(new ProgressToken()).halted());
assertEquals(reindexing, database.readReindexing());
assertTrue(shutDown.get(), "Session was shut down");
+ assertEquals(Map.of("reindexing.progress", Map.of(Map.of("documenttype", "music",
+ "clusterid", "cluster",
+ "state", "pending"),
+ 1.0)), // new ProgressToken() is 100% done.
+ metric.metrics());
// Last reindexing fails.
clock.advance(Duration.ofMillis(10));
@@ -146,13 +159,13 @@ class ReindexerTest {
database.writeReindexing(Reindexing.empty()); // Wipe database to verify we write data from reindexer.
executor.execute(() -> parameters.getControlHandler().onDone(VisitorControlHandler.CompletionCode.FAILURE, "Error"));
return () -> shutDown.set(true);
- }, clock).reindex();
+ }, metric, clock).reindex();
reindexing = reindexing.with(music, Status.ready(clock.instant()).running().failed(clock.instant(), "Error"));
assertEquals(reindexing, database.readReindexing());
assertTrue(shutDown.get(), "Session was shut down");
// Document type is ignored in next run, as it has failed fatally.
- new Reindexer(cluster, Map.of(music, Instant.ofEpochMilli(30)), database, failIfCalled, clock).reindex();
+ new Reindexer(cluster, Map.of(music, Instant.ofEpochMilli(30)), database, failIfCalled, metric, clock).reindex();
assertEquals(reindexing, database.readReindexing());
}
diff --git a/config-model/src/main/java/com/yahoo/vespa/model/application/validation/change/search/AttributeChangeValidator.java b/config-model/src/main/java/com/yahoo/vespa/model/application/validation/change/search/AttributeChangeValidator.java
index a10aac30298..0aee0675ea7 100644
--- a/config-model/src/main/java/com/yahoo/vespa/model/application/validation/change/search/AttributeChangeValidator.java
+++ b/config-model/src/main/java/com/yahoo/vespa/model/application/validation/change/search/AttributeChangeValidator.java
@@ -6,13 +6,10 @@ import com.yahoo.documentmodel.NewDocumentType;
import com.yahoo.searchdefinition.derived.AttributeFields;
import com.yahoo.searchdefinition.derived.IndexSchema;
import com.yahoo.searchdefinition.document.Attribute;
-import com.yahoo.config.application.api.ValidationOverrides;
import com.yahoo.searchdefinition.document.HnswIndexParams;
import com.yahoo.vespa.model.application.validation.change.VespaConfigChangeAction;
-import com.yahoo.vespa.model.application.validation.change.VespaRefeedAction;
import com.yahoo.vespa.model.application.validation.change.VespaRestartAction;
-import java.time.Instant;
import java.util.ArrayList;
import java.util.List;
import java.util.Objects;
@@ -51,12 +48,11 @@ public class AttributeChangeValidator {
this.nextDocType = nextDocType;
}
- public List<VespaConfigChangeAction> validate(ValidationOverrides overrides, Instant now) {
+ public List<VespaConfigChangeAction> validate() {
List<VespaConfigChangeAction> result = new ArrayList<>();
result.addAll(validateAddAttributeAspect());
result.addAll(validateRemoveAttributeAspect());
result.addAll(validateAttributeSettings());
- result.addAll(validateTensorTypes(overrides, now));
return result;
}
@@ -144,36 +140,4 @@ public class AttributeChangeValidator {
}
}
- private List<VespaConfigChangeAction> validateTensorTypes(ValidationOverrides overrides, Instant now) {
- List<VespaConfigChangeAction> result = new ArrayList<>();
-
- for (Attribute nextAttr : nextFields.attributes()) {
- Attribute currentAttr = currentFields.getAttribute(nextAttr.getName());
-
- if (currentAttr != null && currentAttr.tensorType().isPresent()) {
- // If the tensor attribute is not present on the new attribute, it means that the data type of the attribute
- // has been changed. This is already handled by DocumentTypeChangeValidator, so we can ignore it here
- if (!nextAttr.tensorType().isPresent()) {
- continue;
- }
-
- // Tensor attribute has changed type
- if (!nextAttr.tensorType().get().equals(currentAttr.tensorType().get())) {
- result.add(createTensorTypeChangedRefeedAction(id, currentAttr, nextAttr, overrides, now));
- }
- }
- }
-
- return result;
- }
-
- private static VespaRefeedAction createTensorTypeChangedRefeedAction(ClusterSpec.Id id, Attribute currentAttr, Attribute nextAttr, ValidationOverrides overrides, Instant now) {
- return VespaRefeedAction.of(id,
- "tensor-type-change",
- overrides,
- new ChangeMessageBuilder(nextAttr.getName()).addChange("tensor type",
- currentAttr.tensorType().get().toString(),
- nextAttr.tensorType().get().toString()).build(), now);
- }
-
}
diff --git a/config-model/src/main/java/com/yahoo/vespa/model/application/validation/change/search/DocumentDatabaseChangeValidator.java b/config-model/src/main/java/com/yahoo/vespa/model/application/validation/change/search/DocumentDatabaseChangeValidator.java
index 68a97f33dfd..3dcfbe3629d 100644
--- a/config-model/src/main/java/com/yahoo/vespa/model/application/validation/change/search/DocumentDatabaseChangeValidator.java
+++ b/config-model/src/main/java/com/yahoo/vespa/model/application/validation/change/search/DocumentDatabaseChangeValidator.java
@@ -38,20 +38,20 @@ public class DocumentDatabaseChangeValidator {
public List<VespaConfigChangeAction> validate(ValidationOverrides overrides, Instant now) {
List<VespaConfigChangeAction> result = new ArrayList<>();
- result.addAll(validateAttributeChanges(overrides, now));
+ result.addAll(validateAttributeChanges());
result.addAll(validateStructFieldAttributeChanges(overrides, now));
result.addAll(validateIndexingScriptChanges(overrides, now));
result.addAll(validateDocumentTypeChanges(overrides, now));
return result;
}
- private List<VespaConfigChangeAction> validateAttributeChanges(ValidationOverrides overrides, Instant now) {
+ private List<VespaConfigChangeAction> validateAttributeChanges() {
return new AttributeChangeValidator(id,
currentDatabase.getDerivedConfiguration().getAttributeFields(),
currentDatabase.getDerivedConfiguration().getIndexSchema(), currentDocType,
nextDatabase.getDerivedConfiguration().getAttributeFields(),
nextDatabase.getDerivedConfiguration().getIndexSchema(), nextDocType)
- .validate(overrides, now);
+ .validate();
}
private List<VespaConfigChangeAction> validateStructFieldAttributeChanges(ValidationOverrides overrides, Instant now) {
diff --git a/config-model/src/main/java/com/yahoo/vespa/model/container/http/AccessControl.java b/config-model/src/main/java/com/yahoo/vespa/model/container/http/AccessControl.java
index 1e5c944f8dc..9bd12350f26 100644
--- a/config-model/src/main/java/com/yahoo/vespa/model/container/http/AccessControl.java
+++ b/config-model/src/main/java/com/yahoo/vespa/model/container/http/AccessControl.java
@@ -10,6 +10,7 @@ import com.yahoo.vespa.model.container.component.FileStatusHandlerComponent;
import com.yahoo.vespa.model.container.component.Handler;
import com.yahoo.vespa.model.container.component.SystemBindingPattern;
import com.yahoo.vespa.model.container.component.chain.Chain;
+import com.yahoo.vespa.model.container.http.ssl.HostedSslConnectorFactory;
import java.util.Collection;
import java.util.Collections;
@@ -114,6 +115,10 @@ public class AccessControl {
removeDuplicateBindingsFromAccessControlChain(http);
}
+ public void configureHostedConnector(HostedSslConnectorFactory connectorFactory) {
+ connectorFactory.setDefaultRequestFilterChain(ACCESS_CONTROL_CHAIN_ID);
+ }
+
/** returns the excluded bindings as specified in 'access-control' in services.xml **/
public Set<BindingPattern> excludedBindings() { return excludedBindings; }
@@ -127,7 +132,6 @@ public class AccessControl {
private void addAccessControlFilterChain(Http http) {
http.getFilterChains().add(createChain(ACCESS_CONTROL_CHAIN_ID));
- http.getBindings().addAll(List.of(createAccessControlBinding("/"), createAccessControlBinding("/*")));
}
private void addAccessControlExcludedChain(Http http) {
@@ -146,7 +150,6 @@ public class AccessControl {
// Remove bindings from access control chain that have binding pattern as a different filter chain
private void removeDuplicateBindingsFromAccessControlChain(Http http) {
- removeDuplicateBindingsFromChain(http, ACCESS_CONTROL_CHAIN_ID);
removeDuplicateBindingsFromChain(http, ACCESS_CONTROL_EXCLUDED_CHAIN_ID);
}
@@ -172,13 +175,6 @@ public class AccessControl {
}
- private static FilterBinding createAccessControlBinding(String path) {
- return FilterBinding.create(
- FilterBinding.Type.REQUEST,
- new ComponentSpecification(ACCESS_CONTROL_CHAIN_ID.stringValue()),
- SystemBindingPattern.fromHttpPortAndPath(Integer.toString(HOSTED_CONTAINER_PORT), path));
- }
-
private static FilterBinding createAccessControlExcludedBinding(BindingPattern excludedBinding) {
BindingPattern rewrittenBinding = SystemBindingPattern.fromHttpPortAndPath(
Integer.toString(HOSTED_CONTAINER_PORT), excludedBinding.path()); // only keep path from excluded binding
diff --git a/config-model/src/main/java/com/yahoo/vespa/model/container/http/Http.java b/config-model/src/main/java/com/yahoo/vespa/model/container/http/Http.java
index 1ed043857e2..b5c3cac1879 100644
--- a/config-model/src/main/java/com/yahoo/vespa/model/container/http/Http.java
+++ b/config-model/src/main/java/com/yahoo/vespa/model/container/http/Http.java
@@ -24,6 +24,7 @@ public class Http extends AbstractConfigProducer<AbstractConfigProducer<?>> impl
private final List<FilterBinding> bindings = new CopyOnWriteArrayList<>();
private volatile JettyHttpServer httpServer;
private volatile AccessControl accessControl;
+ private volatile boolean strictFiltering = false; // TODO Vespa 8: Enable strict filtering by default if filtering is enabled
public Http(FilterChains chains) {
super("http");
@@ -72,6 +73,8 @@ public class Http extends AbstractConfigProducer<AbstractConfigProducer<?>> impl
return Optional.ofNullable(accessControl);
}
+ public void setStrictFiltering(boolean enabled) { this.strictFiltering = enabled; }
+
@Override
public void getConfig(ServerConfig.Builder builder) {
for (FilterBinding binding : bindings) {
@@ -80,6 +83,7 @@ public class Http extends AbstractConfigProducer<AbstractConfigProducer<?>> impl
.binding(binding.binding().patternString()));
}
populateDefaultFiltersConfig(builder, httpServer);
+ builder.strictFiltering(strictFiltering);
}
@Override
diff --git a/config-model/src/main/java/com/yahoo/vespa/model/container/http/xml/HttpBuilder.java b/config-model/src/main/java/com/yahoo/vespa/model/container/http/xml/HttpBuilder.java
index 9b9ebedda6d..5417a522d6a 100644
--- a/config-model/src/main/java/com/yahoo/vespa/model/container/http/xml/HttpBuilder.java
+++ b/config-model/src/main/java/com/yahoo/vespa/model/container/http/xml/HttpBuilder.java
@@ -40,11 +40,14 @@ public class HttpBuilder extends VespaDomBuilder.DomConfigProducerBuilder<Http>
FilterChains filterChains;
List<FilterBinding> bindings = new ArrayList<>();
AccessControl accessControl = null;
+ Optional<Boolean> strictFiltering = Optional.empty();
Element filteringElem = XML.getChild(spec, "filtering");
if (filteringElem != null) {
filterChains = new FilterChainsBuilder().build(deployState, ancestor, filteringElem);
bindings = readFilterBindings(filteringElem);
+ strictFiltering = XmlHelper.getOptionalAttribute(filteringElem, "strict-mode")
+ .map(Boolean::valueOf);
Element accessControlElem = XML.getChild(filteringElem, "access-control");
if (accessControlElem != null) {
@@ -55,6 +58,7 @@ public class HttpBuilder extends VespaDomBuilder.DomConfigProducerBuilder<Http>
}
Http http = new Http(filterChains);
+ strictFiltering.ifPresent(http::setStrictFiltering);
http.getBindings().addAll(bindings);
ApplicationContainerCluster cluster = getContainerCluster(ancestor).orElse(null);
http.setHttpServer(new JettyHttpServerBuilder(cluster).build(deployState, ancestor, spec));
diff --git a/config-model/src/main/java/com/yahoo/vespa/model/container/xml/ContainerModelBuilder.java b/config-model/src/main/java/com/yahoo/vespa/model/container/xml/ContainerModelBuilder.java
index c8f2bd08ea5..0e23527c97c 100644
--- a/config-model/src/main/java/com/yahoo/vespa/model/container/xml/ContainerModelBuilder.java
+++ b/config-model/src/main/java/com/yahoo/vespa/model/container/xml/ContainerModelBuilder.java
@@ -327,6 +327,7 @@ public class ContainerModelBuilder extends ConfigModelBuilder<ContainerModel> {
String serverName = server.getComponentId().getName();
// If the deployment contains certificate/private key reference, setup TLS port
+ HostedSslConnectorFactory connectorFactory;
if (deployState.endpointCertificateSecrets().isPresent()) {
boolean authorizeClient = deployState.zone().system().isPublic();
if (authorizeClient && deployState.tlsClientAuthority().isEmpty()) {
@@ -340,13 +341,14 @@ public class ContainerModelBuilder extends ConfigModelBuilder<ContainerModel> {
.map(clientAuth -> clientAuth.equals(AccessControl.ClientAuthentication.need))
.orElse(false);
- HostedSslConnectorFactory connectorFactory = authorizeClient
+ connectorFactory = authorizeClient
? HostedSslConnectorFactory.withProvidedCertificateAndTruststore(serverName, endpointCertificateSecrets, deployState.tlsClientAuthority().get())
: HostedSslConnectorFactory.withProvidedCertificate(serverName, endpointCertificateSecrets, enforceHandshakeClientAuth);
- server.addConnector(connectorFactory);
} else {
- server.addConnector(HostedSslConnectorFactory.withDefaultCertificateAndTruststore(serverName));
+ connectorFactory = HostedSslConnectorFactory.withDefaultCertificateAndTruststore(serverName);
}
+ cluster.getHttp().getAccessControl().ifPresent(accessControl -> accessControl.configureHostedConnector(connectorFactory));
+ server.addConnector(connectorFactory);
}
private static boolean isHostedTenantApplication(ConfigModelContext context) {
diff --git a/config-model/src/test/java/com/yahoo/vespa/model/application/validation/change/search/AttributeChangeValidatorTest.java b/config-model/src/test/java/com/yahoo/vespa/model/application/validation/change/search/AttributeChangeValidatorTest.java
index 168ee797fbf..e89f0c0a9cd 100644
--- a/config-model/src/test/java/com/yahoo/vespa/model/application/validation/change/search/AttributeChangeValidatorTest.java
+++ b/config-model/src/test/java/com/yahoo/vespa/model/application/validation/change/search/AttributeChangeValidatorTest.java
@@ -1,15 +1,12 @@
// Copyright 2017 Yahoo Holdings. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
package com.yahoo.vespa.model.application.validation.change.search;
-import com.yahoo.config.application.api.ValidationOverrides;
import com.yahoo.config.provision.ClusterSpec;
import com.yahoo.vespa.model.application.validation.change.VespaConfigChangeAction;
import org.junit.Test;
-import java.time.Instant;
import java.util.List;
-import static com.yahoo.vespa.model.application.validation.change.ConfigChangeTestUtils.newRefeedAction;
import static com.yahoo.vespa.model.application.validation.change.ConfigChangeTestUtils.newRestartAction;
public class AttributeChangeValidatorTest {
@@ -30,7 +27,7 @@ public class AttributeChangeValidatorTest {
@Override
public List<VespaConfigChangeAction> validate() {
- return validator.validate(ValidationOverrides.empty, Instant.now());
+ return validator.validate();
}
}
@@ -111,33 +108,6 @@ public class AttributeChangeValidatorTest {
}
@Test
- public void changing_tensor_type_of_tensor_field_requires_refeed() throws Exception {
- new Fixture(
- "field f1 type tensor(x[2]) { indexing: attribute }",
- "field f1 type tensor(x[3]) { indexing: attribute }")
- .assertValidation(newRefeedAction(ClusterSpec.Id.from("test"),
- "tensor-type-change",
- ValidationOverrides.empty,
- "Field 'f1' changed: tensor type: 'tensor(x[2])' -> 'tensor(x[3])'", Instant.now()));
-
- new Fixture(
- "field f1 type tensor(x[5]) { indexing: attribute }",
- "field f1 type tensor(x[3]) { indexing: attribute }")
- .assertValidation(newRefeedAction(ClusterSpec.Id.from("test"),
- "tensor-type-change",
- ValidationOverrides.empty,
- "Field 'f1' changed: tensor type: 'tensor(x[5])' -> 'tensor(x[3])'", Instant.now()));
- }
-
- @Test
- public void not_changing_tensor_type_of_tensor_field_is_ok() throws Exception {
- new Fixture(
- "field f1 type tensor(x[2]) { indexing: attribute }",
- "field f1 type tensor(x[2]) { indexing: attribute }")
- .assertValidation();
- }
-
- @Test
public void adding_rank_filter_requires_restart() throws Exception {
new Fixture("field f1 type string { indexing: attribute }",
"field f1 type string { indexing: attribute \n rank: filter }").
diff --git a/config-model/src/test/java/com/yahoo/vespa/model/application/validation/change/search/DocumentTypeChangeValidatorTest.java b/config-model/src/test/java/com/yahoo/vespa/model/application/validation/change/search/DocumentTypeChangeValidatorTest.java
index 190c2c8c645..a074f961a53 100644
--- a/config-model/src/test/java/com/yahoo/vespa/model/application/validation/change/search/DocumentTypeChangeValidatorTest.java
+++ b/config-model/src/test/java/com/yahoo/vespa/model/application/validation/change/search/DocumentTypeChangeValidatorTest.java
@@ -208,6 +208,25 @@ public class DocumentTypeChangeValidatorTest {
action.toString());
}
+ @Test
+ public void changing_tensor_type_of_tensor_field_requires_refeed() throws Exception {
+ new Fixture(
+ "field f1 type tensor(x[2]) { indexing: attribute }",
+ "field f1 type tensor(x[3]) { indexing: attribute }")
+ .assertValidation(newRefeedAction(ClusterSpec.Id.from("test"),
+ "field-type-change",
+ ValidationOverrides.empty,
+ "Field 'f1' changed: data type: 'tensor(x[2])' -> 'tensor(x[3])'", Instant.now()));
+
+ new Fixture(
+ "field f1 type tensor(x[5]) { indexing: attribute }",
+ "field f1 type tensor(x[3]) { indexing: attribute }")
+ .assertValidation(newRefeedAction(ClusterSpec.Id.from("test"),
+ "field-type-change",
+ ValidationOverrides.empty,
+ "Field 'f1' changed: data type: 'tensor(x[5])' -> 'tensor(x[3])'", Instant.now()));
+ }
+
private static NewDocumentType createDocumentTypeWithReferenceField(String nameReferencedDocumentType) {
StructDataType headerfields = new StructDataType("headerfields");
headerfields.addField(new Field("ref", new ReferenceDataType(new DocumentType(nameReferencedDocumentType), 0)));
diff --git a/config-model/src/test/java/com/yahoo/vespa/model/container/http/StrictFilteringTest.java b/config-model/src/test/java/com/yahoo/vespa/model/container/http/StrictFilteringTest.java
new file mode 100644
index 00000000000..98383e77324
--- /dev/null
+++ b/config-model/src/test/java/com/yahoo/vespa/model/container/http/StrictFilteringTest.java
@@ -0,0 +1,41 @@
+// Copyright Verizon Media. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+package com.yahoo.vespa.model.container.http;
+
+import com.yahoo.config.model.builder.xml.test.DomBuilderTest;
+import com.yahoo.config.model.deploy.DeployState;
+import com.yahoo.jdisc.http.ServerConfig;
+import com.yahoo.vespa.model.container.xml.ContainerModelBuilder;
+import org.junit.Test;
+import org.w3c.dom.Element;
+
+import static org.junit.Assert.assertTrue;
+
+/**
+ * @author bjorncs
+ */
+public class StrictFilteringTest extends DomBuilderTest {
+
+ @Test
+ public void default_request_and_response_filters_in_services_xml_are_listen_in_server_config() {
+ Element xml = parse(
+ "<container version='1.0'>",
+ " <http>",
+ " <filtering strict-mode=\"true\">",
+ " <request-chain id='request-chain-with-binding'>",
+ " <filter id='my-filter' class='MyFilter'/>",
+ " <binding>http://*/my-chain-binding</binding>",
+ " </request-chain>",
+ " </filtering>",
+ " <server id='server1' port='8000' />",
+ " </http>",
+ "</container>");
+ buildContainerCluster(xml);
+ ServerConfig config = root.getConfig(ServerConfig.class, "container/http/jdisc-jetty/server1");
+ assertTrue(config.strictFiltering());
+ }
+
+ private void buildContainerCluster(Element containerElem) {
+ new ContainerModelBuilder(true, ContainerModelBuilder.Networking.enable).build(DeployState.createTestState(), null, null, root, containerElem);
+ root.freezeModelTopology();
+ }
+}
diff --git a/config-model/src/test/java/com/yahoo/vespa/model/container/xml/AccessControlTest.java b/config-model/src/test/java/com/yahoo/vespa/model/container/xml/AccessControlTest.java
index f21ab28be72..1ac95ac9a99 100644
--- a/config-model/src/test/java/com/yahoo/vespa/model/container/xml/AccessControlTest.java
+++ b/config-model/src/test/java/com/yahoo/vespa/model/container/xml/AccessControlTest.java
@@ -10,6 +10,7 @@ import com.yahoo.vespa.model.container.ApplicationContainer;
import com.yahoo.vespa.model.container.http.AccessControl;
import com.yahoo.vespa.model.container.http.FilterChains;
import com.yahoo.vespa.model.container.http.Http;
+import com.yahoo.vespa.model.container.http.ssl.HostedSslConnectorFactory;
import org.junit.Test;
import java.util.ArrayList;
@@ -22,6 +23,7 @@ import static com.yahoo.vespa.defaults.Defaults.getDefaults;
import static org.hamcrest.CoreMatchers.hasItem;
import static org.hamcrest.CoreMatchers.is;
import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.empty;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.hasItems;
import static org.hamcrest.Matchers.not;
@@ -152,15 +154,24 @@ public class AccessControlTest extends ContainerModelBuilderTestBase {
}
@Test
- public void access_control_filter_chain_contains_catchall_bindings() {
+ public void hosted_connector_for_port_4443_uses_access_control_filter_chain_as_default_request_filter_chain() {
Http http = createModelAndGetHttp(
" <http>",
" <filtering>",
" <access-control/>",
" </filtering>",
" </http>");
+
Set<String> actualBindings = getFilterBindings(http, AccessControl.ACCESS_CONTROL_CHAIN_ID);
- assertThat(actualBindings, containsInAnyOrder("http://*:4443/*"));
+ assertThat(actualBindings, empty());
+
+ HostedSslConnectorFactory hostedConnectorFactory = (HostedSslConnectorFactory)http.getHttpServer().get().getConnectorFactories().stream()
+ .filter(connectorFactory -> connectorFactory instanceof HostedSslConnectorFactory)
+ .findAny()
+ .get();
+ Optional<ComponentId> maybeDefaultChain = hostedConnectorFactory.getDefaultRequestFilterChain();
+ assertTrue(maybeDefaultChain.isPresent());
+ assertEquals(AccessControl.ACCESS_CONTROL_CHAIN_ID, maybeDefaultChain.get());
}
@Test
@@ -193,7 +204,7 @@ public class AccessControlTest extends ContainerModelBuilderTestBase {
}
@Test
- public void access_control_chains_does_not_contain_duplicate_bindings_to_user_request_filter_chain() {
+ public void access_control_chain_exclude_chain_does_not_contain_duplicate_bindings_to_user_request_filter_chain() {
Http http = createModelAndGetHttp(
" <http>",
" <handler id='custom.Handler'>",
@@ -221,9 +232,6 @@ public class AccessControlTest extends ContainerModelBuilderTestBase {
"http://*:4443/metrics/v2",
"http://*:4443/metrics/v2/*"));
- Set<String> actualAccessControlBindings = getFilterBindings(http, AccessControl.ACCESS_CONTROL_CHAIN_ID);
- assertThat(actualAccessControlBindings, containsInAnyOrder("http://*:4443/*"));
-
Set<String> actualCustomChainBindings = getFilterBindings(http, ComponentId.fromString("my-custom-request-chain"));
assertThat(actualCustomChainBindings, containsInAnyOrder("http://*/custom-handler/*", "http://*/"));
}
@@ -261,9 +269,6 @@ public class AccessControlTest extends ContainerModelBuilderTestBase {
"http://*:4443/",
"http://*:4443/custom-handler/*"));
- Set<String> actualAccessControlBindings = getFilterBindings(http, AccessControl.ACCESS_CONTROL_CHAIN_ID);
- assertThat(actualAccessControlBindings, containsInAnyOrder("http://*:4443/*"));
-
Set<String> actualCustomChainBindings = getFilterBindings(http, ComponentId.fromString("my-custom-response-chain"));
assertThat(actualCustomChainBindings, containsInAnyOrder("http://*/custom-handler/*"));
}
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 a49d98176d2..6fb315dc3b3 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
@@ -24,6 +24,7 @@ import com.yahoo.vespa.config.server.http.HttpErrorResponse;
import com.yahoo.vespa.config.server.http.HttpHandler;
import com.yahoo.vespa.config.server.http.JSONResponse;
import com.yahoo.vespa.config.server.http.NotFoundException;
+import com.yahoo.vespa.config.server.tenant.Tenant;
import java.io.IOException;
import java.time.Duration;
@@ -220,11 +221,11 @@ public class ApplicationHandler extends HttpHandler {
}
private void triggerReindexing(HttpRequest request, ApplicationId applicationId) {
- List<String> clusters = Optional.ofNullable(request.getProperty("cluster")).stream()
+ List<String> clusters = Optional.ofNullable(request.getProperty("clusterId")).stream()
.flatMap(value -> Stream.of(value.split(",")))
.filter(cluster -> ! cluster.isBlank())
.collect(toList());
- List<String> types = Optional.ofNullable(request.getProperty("type")).stream()
+ List<String> types = Optional.ofNullable(request.getProperty("documentType")).stream()
.flatMap(value -> Stream.of(value.split(",")))
.filter(type -> ! type.isBlank())
.collect(toList());
@@ -244,9 +245,13 @@ public class ApplicationHandler extends HttpHandler {
}
private HttpResponse getReindexingStatus(ApplicationId applicationId) {
- return new ReindexResponse(applicationRepository.getTenant(applicationId).getApplicationRepo().database()
- .readReindexingStatus(applicationId)
- .orElseThrow(() -> new NotFoundException("Reindexing status not found for " + applicationId)));
+ Tenant tenant = applicationRepository.getTenant(applicationId);
+ if (tenant == null)
+ throw new NotFoundException("Tenant '" + applicationId.tenant().value() + "' not found");
+
+ return new ReindexResponse(tenant.getApplicationRepo().database()
+ .readReindexingStatus(applicationId)
+ .orElseThrow(() -> new NotFoundException("Reindexing status not found for " + applicationId)));
}
private HttpResponse restart(HttpRequest request, ApplicationId applicationId) {
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/session/SessionRepository.java b/configserver/src/main/java/com/yahoo/vespa/config/server/session/SessionRepository.java
index 8679d7d678a..93c1e4c2b50 100644
--- a/configserver/src/main/java/com/yahoo/vespa/config/server/session/SessionRepository.java
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/session/SessionRepository.java
@@ -569,8 +569,10 @@ public class SessionRepository {
}
private void copyApp(File sourceDir, File destinationDir) throws IOException {
- if (destinationDir.exists())
- throw new RuntimeException("Destination dir " + destinationDir + " already exists");
+ if (destinationDir.exists()) {
+ log.log(Level.INFO, "Destination dir " + destinationDir + " already exists, app has already been copied");
+ return;
+ }
if (! sourceDir.isDirectory())
throw new IllegalArgumentException(sourceDir.getAbsolutePath() + " is not a directory");
diff --git a/configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/ApplicationHandlerTest.java b/configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/ApplicationHandlerTest.java
index 7612aa5e01f..a34de472d1e 100644
--- a/configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/ApplicationHandlerTest.java
+++ b/configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/ApplicationHandlerTest.java
@@ -225,25 +225,25 @@ public class ApplicationHandlerTest {
clock.advance(Duration.ofSeconds(1));
expected = expected.withReady(clock.instant());
- reindex(applicationId, "?cluster=");
+ reindex(applicationId, "?clusterId=");
assertEquals(expected,
database.readReindexingStatus(applicationId).orElseThrow());
clock.advance(Duration.ofSeconds(1));
expected = expected.withReady(clock.instant());
- reindex(applicationId, "?type=moo");
+ reindex(applicationId, "?documentType=moo");
assertEquals(expected,
database.readReindexingStatus(applicationId).orElseThrow());
clock.advance(Duration.ofSeconds(1));
- reindex(applicationId, "?cluster=foo,boo");
+ reindex(applicationId, "?clusterId=foo,boo");
expected = expected.withReady("foo", clock.instant())
.withReady("boo", clock.instant());
assertEquals(expected,
database.readReindexingStatus(applicationId).orElseThrow());
clock.advance(Duration.ofSeconds(1));
- reindex(applicationId, "?cluster=foo,boo&type=bar,baz");
+ reindex(applicationId, "?clusterId=foo,boo&documentType=bar,baz");
expected = expected.withReady("foo", "bar", clock.instant())
.withReady("foo", "baz", clock.instant())
.withReady("boo", "bar", clock.instant())
diff --git a/container-core/src/main/java/com/yahoo/container/Container.java b/container-core/src/main/java/com/yahoo/container/Container.java
index 031d4a26d05..70167453e21 100755
--- a/container-core/src/main/java/com/yahoo/container/Container.java
+++ b/container-core/src/main/java/com/yahoo/container/Container.java
@@ -32,7 +32,7 @@ public class Container {
private volatile FileAcquirer fileAcquirer;
private volatile UrlDownloader urlDownloader;
- private static Logger logger = Logger.getLogger(Container.class.getName());
+ private static final Logger logger = Logger.getLogger(Container.class.getName());
// TODO: Make this final again.
private static Container instance = new Container();
@@ -52,7 +52,7 @@ public class Container {
}
/**
- * Hack. For internal use only, will be removed later
+ * Hack. For internal use only, will be removed later.
*
* Used by Application to be able to repeatedly set up containers.
*/
@@ -93,7 +93,7 @@ public class Container {
this.componentRegistry = registry;
}
- //Only intended for use by the Server instance.
+ // Only intended for use by the Server instance.
public void setupFileAcquirer(QrConfig.Filedistributor filedistributorConfig) {
if (usingCustomFileAcquirer)
return;
@@ -109,9 +109,7 @@ public class Container {
setPathAcquirer(fileAcquirer);
}
- /**
- * Only for internal use.
- */
+ /** Only for internal use. */
public void setCustomFileAcquirer(FileAcquirer fileAcquirer) {
if (this.fileAcquirer != null) {
throw new RuntimeException("Can't change file acquirer. Is " +
diff --git a/container-core/src/main/java/com/yahoo/container/core/config/ApplicationBundleLoader.java b/container-core/src/main/java/com/yahoo/container/core/config/ApplicationBundleLoader.java
index 6ecb6c75f90..5bb506fbd6a 100644
--- a/container-core/src/main/java/com/yahoo/container/core/config/ApplicationBundleLoader.java
+++ b/container-core/src/main/java/com/yahoo/container/core/config/ApplicationBundleLoader.java
@@ -21,9 +21,11 @@ import java.util.stream.Collectors;
* @author Tony Vaagenes
*/
public class ApplicationBundleLoader {
+
private static final Logger log = Logger.getLogger(ApplicationBundleLoader.class.getName());
- /* Map of file refs of active bundles (not scheduled for uninstall) to the installed bundle.
+ /**
+ * Map of file refs of active bundles (not scheduled for uninstall) to the installed bundle.
*
* Used to:
* 1. Avoid installing already installed bundles. Just an optimization, installing the same bundle location is a NOP
diff --git a/container-core/src/main/java/com/yahoo/container/core/config/FileAcquirerBundleInstaller.java b/container-core/src/main/java/com/yahoo/container/core/config/FileAcquirerBundleInstaller.java
index 51d77462652..97a11ce7507 100644
--- a/container-core/src/main/java/com/yahoo/container/core/config/FileAcquirerBundleInstaller.java
+++ b/container-core/src/main/java/com/yahoo/container/core/config/FileAcquirerBundleInstaller.java
@@ -18,7 +18,8 @@ import java.util.logging.Logger;
* @author gjoranv
*/
public class FileAcquirerBundleInstaller {
- private static Logger log = Logger.getLogger(FileAcquirerBundleInstaller.class.getName());
+
+ private static final Logger log = Logger.getLogger(FileAcquirerBundleInstaller.class.getName());
private final FileAcquirer fileAcquirer;
@@ -39,8 +40,8 @@ public class FileAcquirerBundleInstaller {
retries++;
}
if (notReadable(file)) {
- com.yahoo.protect.Process.logAndDie("Shutting down - unable to read bundle file with reference '" + reference
- + "' and path " + file.getAbsolutePath());
+ com.yahoo.protect.Process.logAndDie("Shutting down - unable to read bundle file with reference '" +
+ reference + "' and path " + file.getAbsolutePath());
}
}
diff --git a/container-core/src/main/java/com/yahoo/container/core/config/HandlersConfigurerDi.java b/container-core/src/main/java/com/yahoo/container/core/config/HandlersConfigurerDi.java
index 9c0951c1c95..af163e88fee 100644
--- a/container-core/src/main/java/com/yahoo/container/core/config/HandlersConfigurerDi.java
+++ b/container-core/src/main/java/com/yahoo/container/core/config/HandlersConfigurerDi.java
@@ -37,7 +37,6 @@ import java.util.logging.Logger;
import static com.yahoo.collections.CollectionUtil.first;
-
/**
* For internal use only.
*
diff --git a/container-core/src/main/java/com/yahoo/container/core/config/PlatformBundleLoader.java b/container-core/src/main/java/com/yahoo/container/core/config/PlatformBundleLoader.java
index 0ab89e223f6..3951e656736 100644
--- a/container-core/src/main/java/com/yahoo/container/core/config/PlatformBundleLoader.java
+++ b/container-core/src/main/java/com/yahoo/container/core/config/PlatformBundleLoader.java
@@ -20,6 +20,7 @@ import java.util.logging.Logger;
* @author gjoranv
*/
public class PlatformBundleLoader {
+
private static final Logger log = Logger.getLogger(PlatformBundleLoader.class.getName());
private final Osgi osgi;
diff --git a/container-core/src/main/java/com/yahoo/container/core/config/testutil/HandlersConfigurerTestWrapper.java b/container-core/src/main/java/com/yahoo/container/core/config/testutil/HandlersConfigurerTestWrapper.java
index 2ae33347408..d98a865e1fb 100644
--- a/container-core/src/main/java/com/yahoo/container/core/config/testutil/HandlersConfigurerTestWrapper.java
+++ b/container-core/src/main/java/com/yahoo/container/core/config/testutil/HandlersConfigurerTestWrapper.java
@@ -35,12 +35,13 @@ import java.util.concurrent.Executors;
*
*/
public class HandlersConfigurerTestWrapper {
- private ConfigSourceSet configSources =
+
+ private final ConfigSourceSet configSources =
new ConfigSourceSet(this.getClass().getSimpleName() + ": " + new Random().nextLong());
- private HandlersConfigurerDi configurer;
+ private final HandlersConfigurerDi configurer;
// TODO: Remove once tests use ConfigSet rather than dir:
- private final static String testFiles[] = {
+ private final static String[] testFiles = {
"components.cfg",
"handlers.cfg",
"platform-bundles.cfg",
@@ -143,8 +144,11 @@ public class HandlersConfigurerTestWrapper {
}
private static class SimpleContainerThreadpool implements ContainerThreadPool {
+
private final Executor executor = Executors.newCachedThreadPool();
+
@Override public Executor executor() { return executor; }
+
}
}
diff --git a/container-core/src/main/java/com/yahoo/container/core/config/testutil/MockOsgiWrapper.java b/container-core/src/main/java/com/yahoo/container/core/config/testutil/MockOsgiWrapper.java
index 98c927b8efd..356f302f7c0 100644
--- a/container-core/src/main/java/com/yahoo/container/core/config/testutil/MockOsgiWrapper.java
+++ b/container-core/src/main/java/com/yahoo/container/core/config/testutil/MockOsgiWrapper.java
@@ -37,4 +37,5 @@ public class MockOsgiWrapper implements OsgiWrapper {
@Override
public void allowDuplicateBundles(Collection<Bundle> bundles) { }
+
}
diff --git a/container-core/src/main/java/com/yahoo/container/handler/Coverage.java b/container-core/src/main/java/com/yahoo/container/handler/Coverage.java
index 2ec5d34a0a6..7593e4dfecb 100644
--- a/container-core/src/main/java/com/yahoo/container/handler/Coverage.java
+++ b/container-core/src/main/java/com/yahoo/container/handler/Coverage.java
@@ -92,7 +92,7 @@ public class Coverage {
}
/**
- * The number of documents searched for this result. If the final result
+ * Returns the number of documents searched for this result. If the final result
* set is produced through several queries, this number will be the sum
* for all the queries.
*/
@@ -100,18 +100,12 @@ public class Coverage {
return docs;
}
- /**
- * Total number of documents that could be searched.
- *
- * @return Total number of active documents
- */
+ /** Returns the total number of documents that could be searched. */
public long getActive() { return active; }
/**
- * Total number of documents that will be searchable once redistribution has settled.
- * Still in beta, sematics not finalized yet.
- *
- * @return Total number of documents that will soon be available.
+ * Returns the total number of documents that will be searchable once redistribution has settled.
+ * Still in beta, semantics not finalized yet.
*/
@Beta
public long getSoonActive() { return soonActive; }
@@ -122,9 +116,7 @@ public class Coverage {
public boolean isDegradedByAdapativeTimeout() { return (degradedReason & DEGRADED_BY_ADAPTIVE_TIMEOUT) != 0; }
public boolean isDegradedByNonIdealState() { return (degradedReason == 0) && (getResultPercentage() != 100);}
- /**
- * @return whether the search had full coverage or not
- */
+ /** Returns whether the search had full coverage or not */
public boolean getFull() {
switch (fullReason) {
case EXPLICITLY_FULL:
@@ -138,16 +130,12 @@ public class Coverage {
}
}
- /**
- * @return the number of search instances which participated successfully in the search.
- */
+ /** Returns the number of search instances which participated successfully in the search. */
public int getNodes() {
return nodes;
}
- /**
- * @return the number of search instances which tried to participate in the search.
- */
+ /** Returns the number of search instances which tried to participate in the search. */
public int getNodesTried() {
return nodesTried;
}
diff --git a/container-core/src/main/java/com/yahoo/container/handler/Prefix.java b/container-core/src/main/java/com/yahoo/container/handler/Prefix.java
index 076e0b32a58..2fec5f07736 100644
--- a/container-core/src/main/java/com/yahoo/container/handler/Prefix.java
+++ b/container-core/src/main/java/com/yahoo/container/handler/Prefix.java
@@ -49,4 +49,5 @@ public final class Prefix implements Comparable<Prefix> {
public String toString() {
return prefix + ": " + handler;
}
+
}
diff --git a/container-core/src/main/java/com/yahoo/container/handler/ThreadPoolProvider.java b/container-core/src/main/java/com/yahoo/container/handler/ThreadPoolProvider.java
index ae313b1b04c..a07898cb1d1 100644
--- a/container-core/src/main/java/com/yahoo/container/handler/ThreadPoolProvider.java
+++ b/container-core/src/main/java/com/yahoo/container/handler/ThreadPoolProvider.java
@@ -58,7 +58,7 @@ public class ThreadPoolProvider extends AbstractComponent implements Provider<Ex
public Executor get() { return threadpool.executor(); }
/**
- * Shutdown the thread pool, give a grace period of 1 second before forcibly
+ * Shut down the thread pool, give a grace period of 1 second before forcibly
* shutting down all worker threads.
*/
@Override
diff --git a/container-core/src/main/java/com/yahoo/container/handler/Timing.java b/container-core/src/main/java/com/yahoo/container/handler/Timing.java
index e52928404be..0026854ce61 100644
--- a/container-core/src/main/java/com/yahoo/container/handler/Timing.java
+++ b/container-core/src/main/java/com/yahoo/container/handler/Timing.java
@@ -70,4 +70,5 @@ public class Timing {
public long getTimeout() {
return timeout;
}
+
}
diff --git a/container-core/src/main/java/com/yahoo/container/handler/VipStatusHandler.java b/container-core/src/main/java/com/yahoo/container/handler/VipStatusHandler.java
index f2df5ddb7c5..04d03a44fb6 100644
--- a/container-core/src/main/java/com/yahoo/container/handler/VipStatusHandler.java
+++ b/container-core/src/main/java/com/yahoo/container/handler/VipStatusHandler.java
@@ -70,8 +70,7 @@ public final class VipStatusHandler extends ThreadedHttpRequestHandler {
stream.write(data);
}
else {
- throw new IllegalStateException(
- "Neither file nor hardcoded data. This is a bug, please notify the Vespa team.");
+ throw new IllegalStateException("Neither file nor hardcoded data. This is a bug.");
}
stream.close();
}
@@ -158,7 +157,6 @@ public final class VipStatusHandler extends ThreadedHttpRequestHandler {
* out of capacity. This is the default behavior.
*/
@Inject
- @SuppressWarnings("unused") // injected
public VipStatusHandler(VipStatusConfig vipConfig, Metric metric, VipStatus vipStatus) {
// One thread should be enough for status handling - otherwise something else is completely wrong,
// in which case this will eventually start returning a 503 (due to work rejection) as the bounded
diff --git a/container-core/src/main/java/com/yahoo/container/handler/metrics/ErrorResponse.java b/container-core/src/main/java/com/yahoo/container/handler/metrics/ErrorResponse.java
index 1fcde746878..c59dc2939a5 100644
--- a/container-core/src/main/java/com/yahoo/container/handler/metrics/ErrorResponse.java
+++ b/container-core/src/main/java/com/yahoo/container/handler/metrics/ErrorResponse.java
@@ -13,9 +13,10 @@ import static java.util.logging.Level.WARNING;
* @author gjoranv
*/
public class ErrorResponse extends JsonResponse {
- private static Logger log = Logger.getLogger(ErrorResponse.class.getName());
- private static ObjectMapper objectMapper = new ObjectMapper();
+ private static final Logger log = Logger.getLogger(ErrorResponse.class.getName());
+
+ private static final ObjectMapper objectMapper = new ObjectMapper();
public ErrorResponse(int code, String message) {
super(code, asErrorJson(message != null ? message : "<null>"));
@@ -29,4 +30,5 @@ public class ErrorResponse extends JsonResponse {
return "Could not encode error message to json, check the log for details.";
}
}
+
}
diff --git a/container-core/src/main/java/com/yahoo/container/handler/metrics/JsonResponse.java b/container-core/src/main/java/com/yahoo/container/handler/metrics/JsonResponse.java
index def06ce9de3..fc817332079 100644
--- a/container-core/src/main/java/com/yahoo/container/handler/metrics/JsonResponse.java
+++ b/container-core/src/main/java/com/yahoo/container/handler/metrics/JsonResponse.java
@@ -11,6 +11,7 @@ import java.nio.charset.Charset;
* @author gjoranv
*/
public class JsonResponse extends HttpResponse {
+
private final byte[] data;
public JsonResponse(int code, String data) {
@@ -27,4 +28,5 @@ public class JsonResponse extends HttpResponse {
public void render(OutputStream outputStream) throws IOException {
outputStream.write(data);
}
+
}
diff --git a/container-core/src/main/java/com/yahoo/container/handler/metrics/MetricsV2Handler.java b/container-core/src/main/java/com/yahoo/container/handler/metrics/MetricsV2Handler.java
index 78ea62e1b3a..a4a092b02ad 100644
--- a/container-core/src/main/java/com/yahoo/container/handler/metrics/MetricsV2Handler.java
+++ b/container-core/src/main/java/com/yahoo/container/handler/metrics/MetricsV2Handler.java
@@ -74,4 +74,5 @@ public class MetricsV2Handler extends HttpHandlerBase {
static String consumerQuery(String consumer) {
return (consumer == null || consumer.isEmpty()) ? "" : "?consumer=" + consumer;
}
+
}
diff --git a/container-core/src/main/java/com/yahoo/container/handler/metrics/PrometheusV1Handler.java b/container-core/src/main/java/com/yahoo/container/handler/metrics/PrometheusV1Handler.java
index e33f2f47828..00fb488489e 100644
--- a/container-core/src/main/java/com/yahoo/container/handler/metrics/PrometheusV1Handler.java
+++ b/container-core/src/main/java/com/yahoo/container/handler/metrics/PrometheusV1Handler.java
@@ -20,6 +20,9 @@ import org.apache.http.impl.client.CloseableHttpClient;
import static com.yahoo.container.handler.metrics.MetricsV2Handler.consumerQuery;
import static com.yahoo.jdisc.Response.Status.INTERNAL_SERVER_ERROR;
+/**
+ * @author Oracien
+ */
public class PrometheusV1Handler extends HttpHandlerBase{
public static final String V1_PATH = "/prometheus/v1";
diff --git a/container-core/src/main/java/com/yahoo/container/handler/test/MockService.java b/container-core/src/main/java/com/yahoo/container/handler/test/MockService.java
index eef80e95b3d..0228bc06c51 100644
--- a/container-core/src/main/java/com/yahoo/container/handler/test/MockService.java
+++ b/container-core/src/main/java/com/yahoo/container/handler/test/MockService.java
@@ -43,18 +43,18 @@ import java.util.logging.Logger;
@Beta
public class MockService extends LoggingRequestHandler {
- private MockServiceHandler handler;
+ private final MockServiceHandler handler;
/**
* Create a mock service that mocks an external service using data provided via file distribution.
* A custom handler can be created by subclassing and overriding the createHandler method.
*
- * @param executor An {@link Executor} used to create threads.
- * @param accessLog An {@link AccessLog} where requests will be logged.
- * @param fileAcquirer A {@link FileAcquirer} which is used to fetch file from config.
- * @param config A {@link MockserviceConfig} for this service.
- * @throws InterruptedException if unable to get data file within timeout.
- * @throws IOException if unable to create handler due to some IO errors.
+ * @param executor used to create threads
+ * @param accessLog where requests will be logged
+ * @param fileAcquirer used to fetch file from config
+ * @param config the mock config for this service
+ * @throws InterruptedException if unable to get data file within timeout
+ * @throws IOException if unable to create handler due to some IO errors
*/
public MockService(Executor executor, AccessLog accessLog, FileAcquirer fileAcquirer, MockserviceConfig config, Metric metric) throws InterruptedException, IOException {
super(executor, accessLog, metric);
@@ -65,9 +65,9 @@ public class MockService extends LoggingRequestHandler {
/**
* Create a handler for a file. Override this method to handle a custom file syntax of your own.
*
- * @param dataFile A file to read.
- * @return a {@link MockServiceHandler} used to handle requests.
- * @throws IOException if errors occured when loading the file
+ * @param dataFile the file to read
+ * @return the handler used to handle requests
+ * @throws IOException if errors occurred when loading the file
*/
protected MockServiceHandler createHandler(File dataFile) throws IOException {
if (!dataFile.getName().endsWith(".txt")) {
@@ -210,7 +210,7 @@ public class MockService extends LoggingRequestHandler {
}
}
- private class ExceptionResponse extends HttpResponse {
+ private static class ExceptionResponse extends HttpResponse {
private final Exception e;
public ExceptionResponse(int code, Exception e) {
super(code);
@@ -224,4 +224,5 @@ public class MockService extends LoggingRequestHandler {
}
}
}
+
}
diff --git a/container-core/src/main/java/com/yahoo/container/handler/test/MockServiceHandler.java b/container-core/src/main/java/com/yahoo/container/handler/test/MockServiceHandler.java
index 0a246431e43..2ef3d66d501 100644
--- a/container-core/src/main/java/com/yahoo/container/handler/test/MockServiceHandler.java
+++ b/container-core/src/main/java/com/yahoo/container/handler/test/MockServiceHandler.java
@@ -8,17 +8,17 @@ import com.yahoo.container.jdisc.HttpRequest;
* A service handler that is able to map a request to a key and retrieve a value given a key.
*
* @author Ulf Lilleengen
- * @since 5.1.21
*/
@Beta
public interface MockServiceHandler {
+
/**
* Create a custom Key given a http request. This will be called for each request, and allows a handler
* to customize its key format.
* @param request The client http request.
* @return a {@link Key} used to query for the value.
*/
- public Key createKey(HttpRequest request);
+ Key createKey(HttpRequest request);
/**
* Lookup a {@link Value} for a {@link Key}. Returns null if the key is not found.
@@ -26,9 +26,10 @@ public interface MockServiceHandler {
* @param key The {@link Key} to look up.
* @return A {@link Value} used as response.
*/
- public Value get(Key key);
+ Value get(Key key);
+
+ final class Value {
- public final class Value {
public final int returnCode;
public final byte[] data;
public final String contentType;
@@ -38,10 +39,13 @@ public interface MockServiceHandler {
this.data = data;
this.contentType = contentType;
}
+
}
- public interface Key {
- public int hashCode();
- public boolean equals(Object other);
+ interface Key {
+
+ int hashCode();
+ boolean equals(Object other);
+
}
}
diff --git a/container-core/src/main/java/com/yahoo/container/handler/threadpool/DefaultContainerThreadpool.java b/container-core/src/main/java/com/yahoo/container/handler/threadpool/DefaultContainerThreadpool.java
index 46b3a86798b..6bed4a6f442 100644
--- a/container-core/src/main/java/com/yahoo/container/handler/threadpool/DefaultContainerThreadpool.java
+++ b/container-core/src/main/java/com/yahoo/container/handler/threadpool/DefaultContainerThreadpool.java
@@ -50,7 +50,9 @@ public class DefaultContainerThreadpool extends AbstractComponent implements Aut
}
@Override public Executor executor() { return threadpool; }
+
@Override public void close() { closeInternal(); }
+
@Override public void deconstruct() { closeInternal(); super.deconstruct(); }
/**
diff --git a/container-core/src/main/java/com/yahoo/container/handler/threadpool/ExecutorServiceWrapper.java b/container-core/src/main/java/com/yahoo/container/handler/threadpool/ExecutorServiceWrapper.java
index 771c1da82b6..8e0d11c3171 100644
--- a/container-core/src/main/java/com/yahoo/container/handler/threadpool/ExecutorServiceWrapper.java
+++ b/container-core/src/main/java/com/yahoo/container/handler/threadpool/ExecutorServiceWrapper.java
@@ -28,10 +28,12 @@ class ExecutorServiceWrapper extends ForwardingExecutorService {
private final Thread metricReporter;
private final AtomicBoolean closed = new AtomicBoolean(false);
- ExecutorServiceWrapper(
- WorkerCompletionTimingThreadPoolExecutor wrapped,
- ThreadPoolMetric metric, ProcessTerminator processTerminator,
- long maxThreadExecutionTimeMillis, String name, int queueCapacity) {
+ ExecutorServiceWrapper(WorkerCompletionTimingThreadPoolExecutor wrapped,
+ ThreadPoolMetric metric,
+ ProcessTerminator processTerminator,
+ long maxThreadExecutionTimeMillis,
+ String name,
+ int queueCapacity) {
this.wrapped = wrapped;
this.metric = metric;
this.processTerminator = processTerminator;
diff --git a/container-core/src/main/java/com/yahoo/container/handler/threadpool/WorkerCompletionTimingThreadPoolExecutor.java b/container-core/src/main/java/com/yahoo/container/handler/threadpool/WorkerCompletionTimingThreadPoolExecutor.java
index 56f8319c110..1f64cdbdc40 100644
--- a/container-core/src/main/java/com/yahoo/container/handler/threadpool/WorkerCompletionTimingThreadPoolExecutor.java
+++ b/container-core/src/main/java/com/yahoo/container/handler/threadpool/WorkerCompletionTimingThreadPoolExecutor.java
@@ -17,21 +17,18 @@ import java.util.concurrent.atomic.AtomicLong;
*/
class WorkerCompletionTimingThreadPoolExecutor extends ThreadPoolExecutor {
-
-
volatile long lastThreadAssignmentTimeMillis = System.currentTimeMillis();
private final AtomicLong startedCount = new AtomicLong(0);
private final AtomicLong completedCount = new AtomicLong(0);
private final ThreadPoolMetric metric;
- WorkerCompletionTimingThreadPoolExecutor(
- int corePoolSize,
- int maximumPoolSize,
- long keepAliveTime,
- TimeUnit unit,
- BlockingQueue<Runnable> workQueue,
- ThreadFactory threadFactory,
- ThreadPoolMetric metric) {
+ WorkerCompletionTimingThreadPoolExecutor(int corePoolSize,
+ int maximumPoolSize,
+ long keepAliveTime,
+ TimeUnit unit,
+ BlockingQueue<Runnable> workQueue,
+ ThreadFactory threadFactory,
+ ThreadPoolMetric metric) {
super(corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue, threadFactory);
this.metric = metric;
}
@@ -56,5 +53,6 @@ class WorkerCompletionTimingThreadPoolExecutor extends ThreadPoolExecutor {
public int getActiveCount() {
return (int)(startedCount.get() - completedCount.get());
}
+
}
diff --git a/container-core/src/main/java/com/yahoo/container/http/filter/FilterChainRepository.java b/container-core/src/main/java/com/yahoo/container/http/filter/FilterChainRepository.java
index 31bceca9337..688d2c1c5be 100644
--- a/container-core/src/main/java/com/yahoo/container/http/filter/FilterChainRepository.java
+++ b/container-core/src/main/java/com/yahoo/container/http/filter/FilterChainRepository.java
@@ -38,6 +38,7 @@ import static java.util.stream.Collectors.toSet;
* @author bjorncs
*/
public class FilterChainRepository extends AbstractComponent {
+
private static final Logger log = Logger.getLogger(FilterChainRepository.class.getName());
private final ComponentRegistry<Object> filterAndChains;
@@ -198,4 +199,5 @@ public class FilterChainRepository extends AbstractComponent {
throw new IllegalArgumentException("Unsupported filter type: " + filter.getClass().getName());
}
}
+
}
diff --git a/container-core/src/main/java/com/yahoo/container/jdisc/AsyncHttpResponse.java b/container-core/src/main/java/com/yahoo/container/jdisc/AsyncHttpResponse.java
index 592cbff8440..6ed467a465c 100644
--- a/container-core/src/main/java/com/yahoo/container/jdisc/AsyncHttpResponse.java
+++ b/container-core/src/main/java/com/yahoo/container/jdisc/AsyncHttpResponse.java
@@ -32,13 +32,9 @@ public abstract class AsyncHttpResponse extends HttpResponse {
* output (using the provided channel and completion handler) when (async)
* rendering is completed.
*
- * @param output
- * the stream to which content should be rendered
- * @param networkChannel
- * the channel which must be closed on completion
- * @param handler
- * the completion handler to submit when closing the channel, may
- * be null
+ * @param output the stream to which content should be rendered
+ * @param networkChannel the channel which must be closed on completion
+ * @param handler the completion handler to submit when closing the channel, may be null
*/
public abstract void render(OutputStream output, ContentChannel networkChannel, CompletionHandler handler)
throws IOException;
diff --git a/container-core/src/main/java/com/yahoo/container/jdisc/ContentChannelOutputStream.java b/container-core/src/main/java/com/yahoo/container/jdisc/ContentChannelOutputStream.java
index 329889e70c0..1d4c20efe5e 100644
--- a/container-core/src/main/java/com/yahoo/container/jdisc/ContentChannelOutputStream.java
+++ b/container-core/src/main/java/com/yahoo/container/jdisc/ContentChannelOutputStream.java
@@ -29,7 +29,7 @@ public class ContentChannelOutputStream extends OutputStream implements Writable
private boolean failed = false;
private final Object failLock = new Object();
- public ContentChannelOutputStream(final ContentChannel endpoint) {
+ public ContentChannelOutputStream(ContentChannel endpoint) {
this.endpoint = endpoint;
buffer = new BufferChain(this);
}
@@ -38,7 +38,7 @@ public class ContentChannelOutputStream extends OutputStream implements Writable
* Buffered write of a single byte.
*/
@Override
- public void write(final int b) throws IOException {
+ public void write(int b) throws IOException {
try {
buffer.append((byte) b);
} catch (RuntimeException e) {
@@ -74,8 +74,7 @@ public class ContentChannelOutputStream extends OutputStream implements Writable
* It is in other words safe to recycle the array {@code b}.
*/
@Override
- public void write(final byte[] b, final int off, final int len)
- throws IOException {
+ public void write(byte[] b, int off, int len) throws IOException {
nonCopyingWrite(Arrays.copyOfRange(b, off, off + len));
}
@@ -85,7 +84,7 @@ public class ContentChannelOutputStream extends OutputStream implements Writable
* It is in other words safe to recycle the array {@code b}.
*/
@Override
- public void write(final byte[] b) throws IOException {
+ public void write(byte[] b) throws IOException {
nonCopyingWrite(Arrays.copyOf(b, b.length));
}
@@ -94,8 +93,7 @@ public class ContentChannelOutputStream extends OutputStream implements Writable
* <i>transferring</i> ownership of that array to this stream. It is in
* other words <i>not</i> safe to recycle the array {@code b}.
*/
- public void nonCopyingWrite(final byte[] b, final int off, final int len)
- throws IOException {
+ public void nonCopyingWrite(byte[] b, int off, int len) throws IOException {
try {
buffer.append(b, off, len);
} catch (RuntimeException e) {
@@ -108,7 +106,7 @@ public class ContentChannelOutputStream extends OutputStream implements Writable
* <i>transferring</i> ownership of that array to this stream. It is in
* other words <i>not</i> safe to recycle the array {@code b}.
*/
- public void nonCopyingWrite(final byte[] b) throws IOException {
+ public void nonCopyingWrite(byte[] b) throws IOException {
try {
buffer.append(b);
} catch (RuntimeException e) {
@@ -125,27 +123,23 @@ public class ContentChannelOutputStream extends OutputStream implements Writable
* the ByteBuffer to this stream.
*/
@Override
- public void send(final ByteBuffer src) throws IOException {
- // Don't do a buffer.flush() from here, this method is used by the
- // buffer itself
+ public void send(ByteBuffer src) throws IOException {
+ // Don't do a buffer.flush() from here, this method is used by the buffer itself
try {
- byteBufferData += (long) src.remaining();
+ byteBufferData += src.remaining();
endpoint.write(src, new LoggingCompletionHandler());
} catch (RuntimeException e) {
throw new IOException(Exceptions.toMessageString(e), e);
}
}
- /**
- * Give the number of bytes written.
- *
- * @return the number of bytes written to this stream
- */
+ /** Returns the number of bytes written to this stream */
public long written() {
return buffer.appended() + byteBufferData;
}
class LoggingCompletionHandler implements CompletionHandler {
+
@Override
public void completed() {
}
@@ -166,4 +160,5 @@ public class ContentChannelOutputStream extends OutputStream implements Writable
}
}
}
+
}
diff --git a/container-core/src/main/java/com/yahoo/container/jdisc/EmptyResponse.java b/container-core/src/main/java/com/yahoo/container/jdisc/EmptyResponse.java
index 88c6291fb26..737017b7957 100644
--- a/container-core/src/main/java/com/yahoo/container/jdisc/EmptyResponse.java
+++ b/container-core/src/main/java/com/yahoo/container/jdisc/EmptyResponse.java
@@ -5,8 +5,7 @@ import java.io.IOException;
import java.io.OutputStream;
/**
- * Placeholder response when no content, only headers and status is to be
- * returned.
+ * Placeholder response when no content, only headers and status is to be returned.
*
* @author Steinar Knutsen
*/
@@ -19,4 +18,5 @@ public class EmptyResponse extends HttpResponse {
public void render(OutputStream outputStream) throws IOException {
// NOP
}
+
}
diff --git a/container-core/src/main/java/com/yahoo/container/jdisc/HttpRequest.java b/container-core/src/main/java/com/yahoo/container/jdisc/HttpRequest.java
index edd24fed515..e202442479f 100644
--- a/container-core/src/main/java/com/yahoo/container/jdisc/HttpRequest.java
+++ b/container-core/src/main/java/com/yahoo/container/jdisc/HttpRequest.java
@@ -55,7 +55,7 @@ public class HttpRequest {
InputStream requestData = null;
URI uri = null;
CurrentContainer container = null;
- private String nag = " must be set before the attempted operation.";
+ private final String nag = " must be set before the attempted operation.";
SocketAddress remoteAddress;
private void boom(Object ref, String what) {
@@ -99,9 +99,7 @@ public class HttpRequest {
* {@link #jdiscRequest(com.yahoo.jdisc.http.HttpRequest)} before
* instantiating any HTTP request.
*
- * @param request
- * source for defaults and parent JDisc request, may be null
- *
+ * @param request source for defaults and parent JDisc request, may be null
* @see HttpRequest#createTestRequest(String, com.yahoo.jdisc.http.HttpRequest.Method)
*/
public Builder(HttpRequest request) {
@@ -111,9 +109,7 @@ public class HttpRequest {
/**
* Instantiate a request builder with defaults from an existing request.
*
- * @param request
- * parent JDisc request
- *
+ * @param request parent JDisc request
* @see HttpRequest#createTestRequest(String, com.yahoo.jdisc.http.HttpRequest.Method)
*/
public Builder(com.yahoo.jdisc.http.HttpRequest request) {
@@ -216,8 +212,7 @@ public class HttpRequest {
}
/**
- * Start of API for synchronous HTTP request dispatch. Not yet ready for
- * use.
+ * Start of API for synchronous HTTP request dispatch. Not yet ready for use.
*
* @return a new client request
*/
@@ -244,8 +239,7 @@ public class HttpRequest {
}
/**
- * Start of API for synchronous HTTP request dispatch. Not yet ready for
- * use.
+ * Start of API for synchronous HTTP request dispatch. Not yet ready for use.
*
* @return a new server request
*/
@@ -277,8 +271,7 @@ public class HttpRequest {
return new HttpRequest(serverRequest, requestData, properties);
}
- private void setParameters(
- com.yahoo.jdisc.http.HttpRequest request) {
+ private void setParameters(com.yahoo.jdisc.http.HttpRequest request) {
for (Map.Entry<String, String> entry : properties.entrySet()) {
request.parameters().put(entry.getKey(), wrap(entry.getValue()));
}
@@ -290,10 +283,8 @@ public class HttpRequest {
* Wrap a JDisc HTTP request in a synchronous API. The properties from the
* JDisc request will be copied into the HTTP request.
*
- * @param jdiscHttpRequest
- * the JDisc request
- * @param requestData
- * the associated input stream, e.g. with POST request
+ * @param jdiscHttpRequest the JDisc request
+ * @param requestData the associated input stream, e.g. with POST request
*/
public HttpRequest(com.yahoo.jdisc.http.HttpRequest jdiscHttpRequest, InputStream requestData) {
this(jdiscHttpRequest, requestData, null);
@@ -308,13 +299,10 @@ public class HttpRequest {
* will obviously not be reflected by the request. The same applies for
* JDisc parameters.
*
- * @param jdiscHttpRequest
- * the JDisc request
- * @param requestData
- * the associated input stream, e.g. with POST request
- * @param propertyOverrides
- * properties which should not have the same settings as in the
- * parent JDisc request, may be null
+ * @param jdiscHttpRequest the JDisc request
+ * @param requestData the associated input stream, e.g. with POST request
+ * @param propertyOverrides properties which should not have the same settings as in the
+ * parent JDisc request, may be null
*/
public HttpRequest(com.yahoo.jdisc.http.HttpRequest jdiscHttpRequest,
InputStream requestData, Map<String, String> propertyOverrides) {
@@ -495,8 +483,7 @@ public class HttpRequest {
* Helper method to parse boolean request flags, using
* Boolean.parseBoolean(String). Unset values are regarded as false.
*
- * @param name
- * the name of a request property
+ * @param name the name of a request property
* @return whether the property has been explicitly set to true
*/
public boolean getBooleanProperty(String name) {
diff --git a/container-core/src/main/java/com/yahoo/container/jdisc/LoggingRequestHandler.java b/container-core/src/main/java/com/yahoo/container/jdisc/LoggingRequestHandler.java
index 19460053469..064b6cf6279 100644
--- a/container-core/src/main/java/com/yahoo/container/jdisc/LoggingRequestHandler.java
+++ b/container-core/src/main/java/com/yahoo/container/jdisc/LoggingRequestHandler.java
@@ -37,24 +37,30 @@ public abstract class LoggingRequestHandler extends ThreadedHttpRequestHandler {
}
public static class Context {
+
final Executor executor;
final AccessLog accessLog;
final Metric metric;
+
@Inject
public Context(Executor executor, AccessLog accessLog, Metric metric) {
this.executor = executor;
this.accessLog = accessLog;
this.metric = metric;
}
+
public Context(Context other) {
this.executor = other.executor;
this.accessLog = other.accessLog;
this.metric = other.metric;
}
+
public Executor getExecutor() { return executor; }
public AccessLog getAccessLog() { return accessLog; }
public Metric getMetric() { return metric; }
+
}
+
public static Context testOnlyContext() {
return new Context(new Executor() {
@Override
@@ -254,15 +260,14 @@ public abstract class LoggingRequestHandler extends ThreadedHttpRequestHandler {
} else {
// Not running on JDisc http layer (Jetty), e.g unit tests
AccessLogEntry accessLogEntry = new AccessLogEntry();
- populateAccessLogEntryNotCreatedByHttpServer(
- accessLogEntry,
- jdiscRequest,
- extendedResponse.getTiming(),
- httpRequest.getUri().toString(),
- commitStartTime,
- startTime,
- rendererWiring.written(),
- httpResponse.getStatus());
+ populateAccessLogEntryNotCreatedByHttpServer(accessLogEntry,
+ jdiscRequest,
+ extendedResponse.getTiming(),
+ httpRequest.getUri().toString(),
+ commitStartTime,
+ startTime,
+ rendererWiring.written(),
+ httpResponse.getStatus());
accessLog.log(accessLogEntry);
entry = accessLogEntry;
}
diff --git a/container-core/src/main/java/com/yahoo/container/jdisc/RequestHandlerTestDriver.java b/container-core/src/main/java/com/yahoo/container/jdisc/RequestHandlerTestDriver.java
index a3e264c16ee..faa30bd109d 100644
--- a/container-core/src/main/java/com/yahoo/container/jdisc/RequestHandlerTestDriver.java
+++ b/container-core/src/main/java/com/yahoo/container/jdisc/RequestHandlerTestDriver.java
@@ -28,7 +28,7 @@ import java.util.concurrent.TimeUnit;
@Beta
public class RequestHandlerTestDriver implements AutoCloseable {
- private TestDriver driver;
+ private final TestDriver driver;
private MockResponseHandler responseHandler = null;
@@ -152,7 +152,7 @@ public class RequestHandlerTestDriver implements AutoCloseable {
StringBuilder b = new StringBuilder();
while (content.available()>0) {
ByteBuffer nextBuffer = content.read();
- b.append(Charset.forName("utf-8").decode(nextBuffer).toString());
+ b.append(Charset.forName("utf-8").decode(nextBuffer));
}
return b.toString();
}
diff --git a/container-core/src/main/java/com/yahoo/container/jdisc/ThreadedHttpRequestHandler.java b/container-core/src/main/java/com/yahoo/container/jdisc/ThreadedHttpRequestHandler.java
index c46488694de..9687697d6f6 100644
--- a/container-core/src/main/java/com/yahoo/container/jdisc/ThreadedHttpRequestHandler.java
+++ b/container-core/src/main/java/com/yahoo/container/jdisc/ThreadedHttpRequestHandler.java
@@ -69,9 +69,7 @@ public abstract class ThreadedHttpRequestHandler extends ThreadedRequestHandler
@Override
public final void handleRequest(Request request, BufferedContentChannel requestContent, ResponseHandler responseHandler) {
- if (log.isLoggable(Level.FINE)) {
- log.log(Level.FINE, "In " + this.getClass() + ".handleRequest()");
- }
+ log.log(Level.FINE, () -> "In " + this.getClass() + ".handleRequest()");
com.yahoo.jdisc.http.HttpRequest jdiscRequest = asHttpRequest(request);
HttpRequest httpRequest = new HttpRequest(jdiscRequest, new UnsafeContentInputStream(requestContent.toReadable()));
LazyContentChannel channel = null;
@@ -95,8 +93,7 @@ public abstract class ThreadedHttpRequestHandler extends ThreadedRequestHandler
}
/** Render and return whether the channel was closed */
- private void render(HttpRequest request, HttpResponse httpResponse,
- LazyContentChannel channel, long startTime) {
+ private void render(HttpRequest request, HttpResponse httpResponse, LazyContentChannel channel, long startTime) {
LoggingCompletionHandler logOnCompletion = null;
ContentChannelOutputStream output = null;
try {
@@ -139,7 +136,7 @@ public abstract class ThreadedHttpRequestHandler extends ThreadedRequestHandler
private boolean closed = false;
// Fields needed to lazily create or close the channel */
- private HttpRequest httpRequest;
+ private final HttpRequest httpRequest;
private HttpResponse httpResponse;
private final ResponseHandler responseHandler;
private final Metric metric;
@@ -227,29 +224,27 @@ public abstract class ThreadedHttpRequestHandler extends ThreadedRequestHandler
/**
* Override this to implement custom access logging.
*
- * @param startTime
- * execution start
- * @param renderStartTime
- * start of output rendering
- * @param response
- * the response which the log entry regards
- * @param httpRequest
- * the incoming HTTP request
- * @param rendererWiring
- * the stream the rendered response is written to, used for
- * fetching length of rendered response
+ * @param startTime execution start
+ * @param renderStartTime start of output rendering
+ * @param response the response which the log entry regards
+ * @param httpRequest the incoming HTTP request
+ * @param rendererWiring the stream the rendered response is written to, used for
+ * fetching length of rendered response
*/
- protected LoggingCompletionHandler createLoggingCompletionHandler(
- long startTime, long renderStartTime, HttpResponse response,
- HttpRequest httpRequest, ContentChannelOutputStream rendererWiring) {
+ protected LoggingCompletionHandler createLoggingCompletionHandler(long startTime,
+ long renderStartTime,
+ HttpResponse response,
+ HttpRequest httpRequest,
+ ContentChannelOutputStream rendererWiring) {
return null;
}
protected com.yahoo.jdisc.http.HttpRequest asHttpRequest(Request request) {
if (!(request instanceof com.yahoo.jdisc.http.HttpRequest)) {
- throw new IllegalArgumentException("Expected "
- + com.yahoo.jdisc.http.HttpRequest.class.getName() + ", got " + request.getClass().getName());
+ throw new IllegalArgumentException("Expected " + com.yahoo.jdisc.http.HttpRequest.class.getName() +
+ ", got " + request.getClass().getName());
}
return (com.yahoo.jdisc.http.HttpRequest) request;
}
+
}
diff --git a/container-core/src/main/java/com/yahoo/container/jdisc/ThreadedRequestHandler.java b/container-core/src/main/java/com/yahoo/container/jdisc/ThreadedRequestHandler.java
index 4c93613603b..446ee90c205 100644
--- a/container-core/src/main/java/com/yahoo/container/jdisc/ThreadedRequestHandler.java
+++ b/container-core/src/main/java/com/yahoo/container/jdisc/ThreadedRequestHandler.java
@@ -269,6 +269,7 @@ public abstract class ThreadedRequestHandler extends AbstractRequestHandler {
private static class NullFeedContext implements Context {
private static final NullFeedContext INSTANCE = new NullFeedContext();
}
+
}
}
diff --git a/container-core/src/main/java/com/yahoo/container/jdisc/VespaHeaders.java b/container-core/src/main/java/com/yahoo/container/jdisc/VespaHeaders.java
index f6a089d6fd2..3236f7d2407 100644
--- a/container-core/src/main/java/com/yahoo/container/jdisc/VespaHeaders.java
+++ b/container-core/src/main/java/com/yahoo/container/jdisc/VespaHeaders.java
@@ -47,11 +47,7 @@ public final class VespaHeaders {
private static final Tuple2<Boolean, Integer> NO_MATCH = new Tuple2<>(false, Response.Status.OK);
public static boolean benchmarkCoverage(boolean benchmarkOutput, HeaderFields headers) {
- if (benchmarkOutput && headers.get(BenchmarkingHeaders.REQUEST_COVERAGE) != null) {
- return true;
- } else {
- return false;
- }
+ return benchmarkOutput && headers.get(BenchmarkingHeaders.REQUEST_COVERAGE) != null;
}
/** Returns true if this is a benchmarking request, according to headers */
@@ -60,14 +56,14 @@ public final class VespaHeaders {
}
/**
- * Add search benchmark output to the HTTP getHeaders
+ * Add search benchmark output to the HTTP getHeaders.
*
- * @param responseHeaders The response to write the headers to.
- * @param benchmarkCoverage True to include coverage headers.
- * @param t The Timing to read data from.
- * @param c The Counts to read data from.
- * @param errorCount The error count.
- * @param coverage The Coverage to read data from.
+ * @param responseHeaders the response to write the headers to
+ * @param benchmarkCoverage true to include coverage headers
+ * @param t the Timing to read data from
+ * @param c the Counts to read data from
+ * @param errorCount the error count
+ * @param coverage the Coverage to read data from
*/
public static void benchmarkOutput(HeaderFields responseHeaders, boolean benchmarkCoverage,
Timing t, HitCounts c, int errorCount, Coverage coverage) {
@@ -106,10 +102,10 @@ public final class VespaHeaders {
/**
* (during normal execution) return 200 unless this is not a success or a 4xx error is requested.
*
- * @param isSuccess Whether or not the response represents a success.
- * @param mainError The main error of the response, if any.
- * @param allErrors All the errors of the response, if any.
- * @return The status code of the given response.
+ * @param isSuccess whether or not the response represents a success
+ * @param mainError the main error of the response, if any
+ * @param allErrors all the errors of the response, if any
+ * @return the status code of the given response
*/
public static int getStatus(boolean isSuccess, ErrorMessage mainError, Iterator<? extends ErrorMessage> allErrors) {
// Do note, SearchResponse has its own implementation of isSuccess()
@@ -129,7 +125,7 @@ public final class VespaHeaders {
Iterator<? extends ErrorMessage> errorIterator = allErrors;
if (errorIterator != null && errorIterator.hasNext()) {
- for (; errorIterator.hasNext();) {
+ while (errorIterator.hasNext()) {
ErrorMessage error = errorIterator.next();
Tuple2<Boolean, Integer> status = chooseWebServiceStatus(error);
if (status.first) {
diff --git a/container-core/src/main/java/com/yahoo/container/jdisc/state/FileWrapper.java b/container-core/src/main/java/com/yahoo/container/jdisc/state/FileWrapper.java
index 6e22e02eb5b..3e127b87017 100644
--- a/container-core/src/main/java/com/yahoo/container/jdisc/state/FileWrapper.java
+++ b/container-core/src/main/java/com/yahoo/container/jdisc/state/FileWrapper.java
@@ -24,4 +24,5 @@ public class FileWrapper {
boolean isRegularFile(Path path) {
return Files.isRegularFile(path);
}
+
}
diff --git a/container-core/src/main/java/com/yahoo/container/jdisc/state/GaugeMetric.java b/container-core/src/main/java/com/yahoo/container/jdisc/state/GaugeMetric.java
index 9b89b8abe52..9a195710c8f 100644
--- a/container-core/src/main/java/com/yahoo/container/jdisc/state/GaugeMetric.java
+++ b/container-core/src/main/java/com/yahoo/container/jdisc/state/GaugeMetric.java
@@ -20,7 +20,7 @@ public final class GaugeMetric extends MetricValue {
private double min;
private double sum;
private long count;
- private Optional<List<Tuple2<String, Double>>> percentiles;
+ private final Optional<List<Tuple2<String, Double>>> percentiles;
private GaugeMetric(double last, double max, double min, double sum, long count, Optional<List<Tuple2<String, Double>>> percentiles) {
this.last = last;
diff --git a/container-core/src/main/java/com/yahoo/container/jdisc/state/HostLifeGatherer.java b/container-core/src/main/java/com/yahoo/container/jdisc/state/HostLifeGatherer.java
index 080a5a8dc32..730f7bc13cd 100644
--- a/container-core/src/main/java/com/yahoo/container/jdisc/state/HostLifeGatherer.java
+++ b/container-core/src/main/java/com/yahoo/container/jdisc/state/HostLifeGatherer.java
@@ -44,4 +44,5 @@ public class HostLifeGatherer {
return jsonObject;
}
+
}
diff --git a/container-core/src/main/java/com/yahoo/container/jdisc/state/JSONObjectWithLegibleException.java b/container-core/src/main/java/com/yahoo/container/jdisc/state/JSONObjectWithLegibleException.java
index dc1bfb89197..d22dd9d6f4b 100644
--- a/container-core/src/main/java/com/yahoo/container/jdisc/state/JSONObjectWithLegibleException.java
+++ b/container-core/src/main/java/com/yahoo/container/jdisc/state/JSONObjectWithLegibleException.java
@@ -13,6 +13,7 @@ import java.util.Map;
* @author gjoranv
*/
class JSONObjectWithLegibleException extends JSONObject {
+
@Override
public JSONObject put(String s, boolean b) {
try {
@@ -80,7 +81,7 @@ class JSONObjectWithLegibleException extends JSONObject {
private String getErrorMessage(String key, Object value, JSONException e) {
return "Trying to add invalid JSON object with key '" + key +
- "' and value '" + value +
- "' - " + e.getMessage();
+ "' and value '" + value + "' - " + e.getMessage();
}
+
}
diff --git a/container-core/src/main/java/com/yahoo/container/jdisc/state/MetricGatherer.java b/container-core/src/main/java/com/yahoo/container/jdisc/state/MetricGatherer.java
index 061ce7138ad..6a06a6362f5 100644
--- a/container-core/src/main/java/com/yahoo/container/jdisc/state/MetricGatherer.java
+++ b/container-core/src/main/java/com/yahoo/container/jdisc/state/MetricGatherer.java
@@ -7,8 +7,9 @@ import java.util.ArrayList;
import java.util.List;
/**
+ * Gathers metrics regarding currently processing coredumps and host life.
+ *
* @author olaa
- * Gathers metrics regarding currently processing coredumps and host life
*/
public class MetricGatherer {
@@ -17,7 +18,8 @@ public class MetricGatherer {
List<JSONObject> packetList = new ArrayList<>();
packetList.add(CoredumpGatherer.gatherCoredumpMetrics(fileWrapper));
if (System.getProperty("os.name").contains("nux"))
- packetList.add(HostLifeGatherer.getHostLifePacket(fileWrapper));
+ packetList.add(HostLifeGatherer.getHostLifePacket(fileWrapper));
return packetList;
}
+
}
diff --git a/container-core/src/main/java/com/yahoo/container/jdisc/state/MetricsPacketsHandler.java b/container-core/src/main/java/com/yahoo/container/jdisc/state/MetricsPacketsHandler.java
index d1036db5c6f..c1a6f650a9c 100644
--- a/container-core/src/main/java/com/yahoo/container/jdisc/state/MetricsPacketsHandler.java
+++ b/container-core/src/main/java/com/yahoo/container/jdisc/state/MetricsPacketsHandler.java
@@ -43,6 +43,7 @@ import static com.yahoo.container.jdisc.state.StateHandler.getSnapshotPreprocess
* @author gjoranv
*/
public class MetricsPacketsHandler extends AbstractRequestHandler {
+
static final String APPLICATION_KEY = "application";
static final String TIMESTAMP_KEY = "timestamp";
static final String STATUS_CODE_KEY = "status_code";
diff --git a/container-core/src/main/java/com/yahoo/container/jdisc/state/SnapshotProvider.java b/container-core/src/main/java/com/yahoo/container/jdisc/state/SnapshotProvider.java
index 4967fd1f162..d693bf97bd8 100644
--- a/container-core/src/main/java/com/yahoo/container/jdisc/state/SnapshotProvider.java
+++ b/container-core/src/main/java/com/yahoo/container/jdisc/state/SnapshotProvider.java
@@ -7,7 +7,7 @@ import java.io.PrintStream;
* An interface for components supplying a state snapshot where persistence and
* other pre-processing has been done.
*
- * @author <a href="mailto:steinar@yahoo-inc.com">Steinar Knutsen</a>
+ * @author Steinar Knutsen
*/
public interface SnapshotProvider {
diff --git a/container-core/src/main/java/com/yahoo/container/jdisc/state/StateMonitor.java b/container-core/src/main/java/com/yahoo/container/jdisc/state/StateMonitor.java
index ccd0864b3ab..40a0ef10fbc 100644
--- a/container-core/src/main/java/com/yahoo/container/jdisc/state/StateMonitor.java
+++ b/container-core/src/main/java/com/yahoo/container/jdisc/state/StateMonitor.java
@@ -136,4 +136,5 @@ public class StateMonitor extends AbstractComponent {
}
});
}
+
}
diff --git a/container-core/src/main/java/com/yahoo/container/servlet/ServletProvider.java b/container-core/src/main/java/com/yahoo/container/servlet/ServletProvider.java
index 903c01a27f7..213063a725d 100644
--- a/container-core/src/main/java/com/yahoo/container/servlet/ServletProvider.java
+++ b/container-core/src/main/java/com/yahoo/container/servlet/ServletProvider.java
@@ -11,14 +11,11 @@ import org.eclipse.jetty.servlet.ServletHolder;
*/
public class ServletProvider implements Provider<ServletHolder> {
- private ServletHolder servletHolder;
+ private final ServletHolder servletHolder;
public ServletProvider(Servlet servlet, ServletConfigConfig servletConfigConfig) {
servletHolder = new ServletHolder(servlet);
-
- servletConfigConfig.map().forEach( (key, value) ->
- servletHolder.setInitParameter(key, value)
- );
+ servletConfigConfig.map().forEach( (key, value) -> servletHolder.setInitParameter(key, value));
}
@Override
diff --git a/container-core/src/main/java/com/yahoo/container/xml/providers/DatatypeFactoryProvider.java b/container-core/src/main/java/com/yahoo/container/xml/providers/DatatypeFactoryProvider.java
index ffce3649419..d49c548d25c 100644
--- a/container-core/src/main/java/com/yahoo/container/xml/providers/DatatypeFactoryProvider.java
+++ b/container-core/src/main/java/com/yahoo/container/xml/providers/DatatypeFactoryProvider.java
@@ -10,8 +10,9 @@ import javax.xml.datatype.DatatypeFactory;
* @author Einar M R Rosenvinge
* @deprecated Do not use!
*/
-@Deprecated
+@Deprecated // TODO: Remove on Vespa 8
public class DatatypeFactoryProvider implements Provider<DatatypeFactory> {
+
public static final String FACTORY_CLASS = DatatypeFactory.DATATYPEFACTORY_IMPLEMENTATION_CLASS;
@Override
@@ -25,4 +26,5 @@ public class DatatypeFactoryProvider implements Provider<DatatypeFactory> {
@Override
public void deconstruct() { }
+
}
diff --git a/container-core/src/main/java/com/yahoo/container/xml/providers/DocumentBuilderFactoryProvider.java b/container-core/src/main/java/com/yahoo/container/xml/providers/DocumentBuilderFactoryProvider.java
index 37b8dff8bf4..c81d173e1ed 100644
--- a/container-core/src/main/java/com/yahoo/container/xml/providers/DocumentBuilderFactoryProvider.java
+++ b/container-core/src/main/java/com/yahoo/container/xml/providers/DocumentBuilderFactoryProvider.java
@@ -9,7 +9,7 @@ import javax.xml.parsers.DocumentBuilderFactory;
* @author Einar M R Rosenvinge
* @deprecated Do not use!
*/
-@Deprecated
+@Deprecated // TODO: Remove on Vespa 8
public class DocumentBuilderFactoryProvider implements Provider<DocumentBuilderFactory> {
public static final String FACTORY_CLASS = "com.sun.org.apache.xerces.internal.jaxp.DocumentBuilderFactoryImpl";
diff --git a/container-core/src/main/java/com/yahoo/container/xml/providers/SAXParserFactoryProvider.java b/container-core/src/main/java/com/yahoo/container/xml/providers/SAXParserFactoryProvider.java
index a8ac55a8aca..0d0b79d8ce7 100644
--- a/container-core/src/main/java/com/yahoo/container/xml/providers/SAXParserFactoryProvider.java
+++ b/container-core/src/main/java/com/yahoo/container/xml/providers/SAXParserFactoryProvider.java
@@ -9,7 +9,7 @@ import javax.xml.parsers.SAXParserFactory;
* @author Einar M R Rosenvinge
* @deprecated Do not use!
*/
-@Deprecated
+@Deprecated // TODO: Remove on Vespa 8
public class SAXParserFactoryProvider implements Provider<SAXParserFactory> {
public static final String FACTORY_CLASS = "com.sun.org.apache.xerces.internal.jaxp.SAXParserFactoryImpl";
diff --git a/container-core/src/main/java/com/yahoo/container/xml/providers/SchemaFactoryProvider.java b/container-core/src/main/java/com/yahoo/container/xml/providers/SchemaFactoryProvider.java
index bbcdf7c9553..0d69e129492 100644
--- a/container-core/src/main/java/com/yahoo/container/xml/providers/SchemaFactoryProvider.java
+++ b/container-core/src/main/java/com/yahoo/container/xml/providers/SchemaFactoryProvider.java
@@ -10,7 +10,7 @@ import javax.xml.validation.SchemaFactory;
* @author Einar M R Rosenvinge
* @deprecated Do not use!
*/
-@Deprecated
+@Deprecated // TODO: Remove on Vespa 8
public class SchemaFactoryProvider implements Provider<SchemaFactory> {
public static final String FACTORY_CLASS = "com.sun.org.apache.xerces.internal.jaxp.validation.XMLSchemaFactory";
diff --git a/container-core/src/main/java/com/yahoo/container/xml/providers/TransformerFactoryProvider.java b/container-core/src/main/java/com/yahoo/container/xml/providers/TransformerFactoryProvider.java
index 974d2e6a259..071a576abe8 100644
--- a/container-core/src/main/java/com/yahoo/container/xml/providers/TransformerFactoryProvider.java
+++ b/container-core/src/main/java/com/yahoo/container/xml/providers/TransformerFactoryProvider.java
@@ -9,7 +9,7 @@ import javax.xml.transform.TransformerFactory;
* @author Einar M R Rosenvinge
* @deprecated Do not use!
*/
-@Deprecated
+@Deprecated // TODO: Remove on Vespa 8
public class TransformerFactoryProvider implements Provider<TransformerFactory> {
public static final String FACTORY_CLASS = "com.sun.org.apache.xalan.internal.xsltc.trax.TransformerFactoryImpl";
diff --git a/container-core/src/main/java/com/yahoo/container/xml/providers/XMLEventFactoryProvider.java b/container-core/src/main/java/com/yahoo/container/xml/providers/XMLEventFactoryProvider.java
index 702ecedcc93..f5e1e666072 100644
--- a/container-core/src/main/java/com/yahoo/container/xml/providers/XMLEventFactoryProvider.java
+++ b/container-core/src/main/java/com/yahoo/container/xml/providers/XMLEventFactoryProvider.java
@@ -9,7 +9,7 @@ import javax.xml.stream.XMLEventFactory;
* @author Einar M R Rosenvinge
* @deprecated Do not use!
*/
-@Deprecated
+@Deprecated // TODO: Remove on Vespa 8
public class XMLEventFactoryProvider implements Provider<XMLEventFactory> {
public static final String FACTORY_CLASS = "com.sun.xml.internal.stream.events.XMLEventFactoryImpl";
diff --git a/container-core/src/main/java/com/yahoo/container/xml/providers/XMLInputFactoryProvider.java b/container-core/src/main/java/com/yahoo/container/xml/providers/XMLInputFactoryProvider.java
index 9f3518525de..99eb6df7093 100644
--- a/container-core/src/main/java/com/yahoo/container/xml/providers/XMLInputFactoryProvider.java
+++ b/container-core/src/main/java/com/yahoo/container/xml/providers/XMLInputFactoryProvider.java
@@ -9,7 +9,7 @@ import javax.xml.stream.XMLInputFactory;
* @author Einar M R Rosenvinge
* @deprecated Do not use!
*/
-@Deprecated
+@Deprecated // TODO: Remove on Vespa 8
public class XMLInputFactoryProvider implements Provider<XMLInputFactory> {
private static final String INPUT_FACTORY_INTERFACE = XMLInputFactory.class.getName();
diff --git a/container-core/src/main/java/com/yahoo/container/xml/providers/XMLOutputFactoryProvider.java b/container-core/src/main/java/com/yahoo/container/xml/providers/XMLOutputFactoryProvider.java
index ab28ba2e923..c5a21dae410 100644
--- a/container-core/src/main/java/com/yahoo/container/xml/providers/XMLOutputFactoryProvider.java
+++ b/container-core/src/main/java/com/yahoo/container/xml/providers/XMLOutputFactoryProvider.java
@@ -9,7 +9,7 @@ import javax.xml.stream.XMLOutputFactory;
* @author Einar M R Rosenvinge
* @deprecated Do not use!
*/
-@Deprecated
+@Deprecated // TODO: Remove on Vespa 8
public class XMLOutputFactoryProvider implements Provider<XMLOutputFactory> {
public static final String FACTORY_CLASS = "com.sun.xml.internal.stream.XMLOutputFactoryImpl";
diff --git a/container-core/src/main/java/com/yahoo/container/xml/providers/XPathFactoryProvider.java b/container-core/src/main/java/com/yahoo/container/xml/providers/XPathFactoryProvider.java
index 407369032cf..23ef9995caf 100644
--- a/container-core/src/main/java/com/yahoo/container/xml/providers/XPathFactoryProvider.java
+++ b/container-core/src/main/java/com/yahoo/container/xml/providers/XPathFactoryProvider.java
@@ -10,7 +10,7 @@ import javax.xml.xpath.XPathFactoryConfigurationException;
* @author Einar M R Rosenvinge
* @deprecated Do not use!
*/
-@Deprecated
+@Deprecated // TODO: Remove on Vespa 8
public class XPathFactoryProvider implements Provider<XPathFactory> {
public static final String FACTORY_CLASS = "com.sun.org.apache.xpath.internal.jaxp.XPathFactoryImpl";
diff --git a/container-core/src/main/java/com/yahoo/language/provider/DefaultLinguisticsProvider.java b/container-core/src/main/java/com/yahoo/language/provider/DefaultLinguisticsProvider.java
index ace5a7ab304..92cdfe8d918 100644
--- a/container-core/src/main/java/com/yahoo/language/provider/DefaultLinguisticsProvider.java
+++ b/container-core/src/main/java/com/yahoo/language/provider/DefaultLinguisticsProvider.java
@@ -17,8 +17,8 @@ import com.yahoo.language.opennlp.OpenNlpLinguistics;
@SuppressWarnings("unused") // Injected
public class DefaultLinguisticsProvider implements Provider<Linguistics> {
- // Use lazy initialization to avoid expensive (memory-wise) instantiation f
- private volatile Supplier<Linguistics> linguisticsSupplier = Suppliers.memoize(OpenNlpLinguistics::new);
+ // Use lazy initialization to avoid expensive (memory-wise) instantiation
+ private final Supplier<Linguistics> linguisticsSupplier = Suppliers.memoize(OpenNlpLinguistics::new);
@Inject
public DefaultLinguisticsProvider() { }
diff --git a/container-core/src/main/java/com/yahoo/osgi/Osgi.java b/container-core/src/main/java/com/yahoo/osgi/Osgi.java
index 513e7883594..54f9ad48703 100644
--- a/container-core/src/main/java/com/yahoo/osgi/Osgi.java
+++ b/container-core/src/main/java/com/yahoo/osgi/Osgi.java
@@ -27,4 +27,5 @@ public interface Osgi {
default boolean hasFelixFramework() {
return false;
}
+
}
diff --git a/container-core/src/main/java/com/yahoo/osgi/OsgiImpl.java b/container-core/src/main/java/com/yahoo/osgi/OsgiImpl.java
index b34442d50a9..97e2367bac2 100644
--- a/container-core/src/main/java/com/yahoo/osgi/OsgiImpl.java
+++ b/container-core/src/main/java/com/yahoo/osgi/OsgiImpl.java
@@ -19,6 +19,7 @@ import java.util.logging.Logger;
* @author gjoranv
*/
public class OsgiImpl implements Osgi {
+
private static final Logger log = Logger.getLogger(OsgiImpl.class.getName());
private final OsgiFramework jdiscOsgi;
diff --git a/container-core/src/main/java/com/yahoo/processing/handler/ProcessingResponse.java b/container-core/src/main/java/com/yahoo/processing/handler/ProcessingResponse.java
index 2727d111829..a8ccc28c751 100644
--- a/container-core/src/main/java/com/yahoo/processing/handler/ProcessingResponse.java
+++ b/container-core/src/main/java/com/yahoo/processing/handler/ProcessingResponse.java
@@ -31,8 +31,7 @@ import com.yahoo.processing.response.DataList;
* wrapper of the knowhow needed to render the Response from processing.
*
* @author bratseth
- * @author <a href="mailto:steinar@yahoo-inc.com">Steinar Knutsen</a>
- * @since 5.1.12
+ * @author Steinar Knutsen
*/
public class ProcessingResponse extends AsyncHttpResponse {
@@ -44,7 +43,6 @@ public class ProcessingResponse extends AsyncHttpResponse {
/** True if the return status has been set explicitly and should not be further changed */
private boolean explicitStatusSet = false;
- @SuppressWarnings("unchecked")
public ProcessingResponse(int status, com.yahoo.processing.Request processingRequest,
com.yahoo.processing.Response processingResponse,
Renderer renderer,
diff --git a/container-core/src/main/java/com/yahoo/processing/handler/ProcessingTestDriver.java b/container-core/src/main/java/com/yahoo/processing/handler/ProcessingTestDriver.java
index 8fc0b49c71b..d4e55dbc556 100644
--- a/container-core/src/main/java/com/yahoo/processing/handler/ProcessingTestDriver.java
+++ b/container-core/src/main/java/com/yahoo/processing/handler/ProcessingTestDriver.java
@@ -23,7 +23,6 @@ import java.util.concurrent.Executors;
* Create an instance of this to test making processing requests and get the response or response data.
*
* @author bratseth
- * @since 5.21
*/
@Beta
public class ProcessingTestDriver extends RequestHandlerTestDriver {
@@ -31,20 +30,20 @@ public class ProcessingTestDriver extends RequestHandlerTestDriver {
private final ProcessingHandler processingHandler;
public ProcessingTestDriver(Collection<Chain<Processor>> chains) {
- this(chains, new ComponentRegistry<Renderer>());
+ this(chains, new ComponentRegistry<>());
}
public ProcessingTestDriver(String binding, Collection<Chain<Processor>> chains) {
- this(chains, new ComponentRegistry<Renderer>());
+ this(chains, new ComponentRegistry<>());
}
@SafeVarargs
@SuppressWarnings("varargs")
public ProcessingTestDriver(Chain<Processor> ... chains) {
- this(Arrays.asList(chains), new ComponentRegistry<Renderer>());
+ this(Arrays.asList(chains), new ComponentRegistry<>());
}
@SafeVarargs
@SuppressWarnings("varargs")
public ProcessingTestDriver(String binding, Chain<Processor> ... chains) {
- this(binding, Arrays.asList(chains), new ComponentRegistry<Renderer>());
+ this(binding, Arrays.asList(chains), new ComponentRegistry<>());
}
public ProcessingTestDriver(Collection<Chain<Processor>> chains, ComponentRegistry<Renderer> renderers) {
this(createProcessingHandler(chains, renderers, AccessLog.voidAccessLog()));
@@ -64,7 +63,7 @@ public class ProcessingTestDriver extends RequestHandlerTestDriver {
public ProcessingTestDriver(Chain<Processor> chain, AccessLogInterface accessLogInterface) {
this(createProcessingHandler(
Collections.singleton(chain),
- new ComponentRegistry<Renderer>(),
+ new ComponentRegistry<>(),
createAccessLog(accessLogInterface)));
}
@@ -76,10 +75,9 @@ public class ProcessingTestDriver extends RequestHandlerTestDriver {
return new AccessLog(componentRegistry);
}
- private static ProcessingHandler createProcessingHandler(
- Collection<Chain<Processor>> chains,
- ComponentRegistry<Renderer> renderers,
- AccessLog accessLog) {
+ private static ProcessingHandler createProcessingHandler(Collection<Chain<Processor>> chains,
+ ComponentRegistry<Renderer> renderers,
+ AccessLog accessLog) {
Executor executor = Executors.newSingleThreadExecutor();
ChainRegistry<Processor> registry = new ChainRegistry<>();
diff --git a/container-core/src/main/java/com/yahoo/processing/handler/ResponseHeaders.java b/container-core/src/main/java/com/yahoo/processing/handler/ResponseHeaders.java
index 0267b892878..3a21c5a3ec9 100644
--- a/container-core/src/main/java/com/yahoo/processing/handler/ResponseHeaders.java
+++ b/container-core/src/main/java/com/yahoo/processing/handler/ResponseHeaders.java
@@ -14,7 +14,6 @@ import java.util.Map;
* A Response may contain multiple such data objects, and all of them will be added to the response.
*
* @author bratseth
- * @since 5.1.23
*/
public class ResponseHeaders extends AbstractData {
diff --git a/container-core/src/main/java/com/yahoo/processing/processors/RequestPropertyTracer.java b/container-core/src/main/java/com/yahoo/processing/processors/RequestPropertyTracer.java
index e08cf013c19..a42b027b795 100644
--- a/container-core/src/main/java/com/yahoo/processing/processors/RequestPropertyTracer.java
+++ b/container-core/src/main/java/com/yahoo/processing/processors/RequestPropertyTracer.java
@@ -12,8 +12,7 @@ import java.util.Map;
* A processor which adds the current content of the Request.properties() to
* the trace before calling the next processor, if traceLevel is 4 or more.
*
- * @author bratseth
- * @since 5.1.17
+ * @author bratseth
*/
public class RequestPropertyTracer extends Processor {
diff --git a/container-core/src/main/java/com/yahoo/processing/rendering/AsynchronousRenderer.java b/container-core/src/main/java/com/yahoo/processing/rendering/AsynchronousRenderer.java
index b6298558ff9..eb39c4c8117 100644
--- a/container-core/src/main/java/com/yahoo/processing/rendering/AsynchronousRenderer.java
+++ b/container-core/src/main/java/com/yahoo/processing/rendering/AsynchronousRenderer.java
@@ -20,8 +20,7 @@ public abstract class AsynchronousRenderer <RESPONSE extends Response> extends R
* Exposes JDisc wiring to ensure asynchronous cleanup.
*
* @param channel the channel to the client receiving the response
- * @param completionHandler the JDisc completion handler which will be invoked at the end
- * of the rendering
+ * @param completionHandler the JDisc completion handler which will be invoked at the end of the rendering
* @throws IllegalStateException if attempted invoked more than once
*/
public abstract void setNetworkWiring(ContentChannel channel, CompletionHandler completionHandler);
diff --git a/container-core/src/main/java/com/yahoo/processing/rendering/AsynchronousSectionedRenderer.java b/container-core/src/main/java/com/yahoo/processing/rendering/AsynchronousSectionedRenderer.java
index fdc3b63fc92..f86cad7c619 100644
--- a/container-core/src/main/java/com/yahoo/processing/rendering/AsynchronousSectionedRenderer.java
+++ b/container-core/src/main/java/com/yahoo/processing/rendering/AsynchronousSectionedRenderer.java
@@ -53,10 +53,8 @@ public abstract class AsynchronousSectionedRenderer<RESPONSE extends Response> e
* stream to be used throughput the rendering. Subsequent calls must use the
* same stream.
*
- * @param stream
- * the stream to render to in this and all subsequent calls.
- * @throws IOException
- * passed on from the stream
+ * @param stream the stream to render to in this and all subsequent calls.
+ * @throws IOException passed on from the stream
*/
public abstract void beginResponse(OutputStream stream) throws IOException;
diff --git a/container-core/src/main/java/com/yahoo/processing/rendering/ProcessingRenderer.java b/container-core/src/main/java/com/yahoo/processing/rendering/ProcessingRenderer.java
index 556eec9b460..052c1c3f7f5 100644
--- a/container-core/src/main/java/com/yahoo/processing/rendering/ProcessingRenderer.java
+++ b/container-core/src/main/java/com/yahoo/processing/rendering/ProcessingRenderer.java
@@ -167,7 +167,7 @@ public class ProcessingRenderer extends AsynchronousSectionedRenderer<Response>
private static class TraceRenderingVisitor extends TraceVisitor {
- private JSONWriter jsonWriter;
+ private final JSONWriter jsonWriter;
public TraceRenderingVisitor(JSONWriter jsonWriter) {
this.jsonWriter = jsonWriter;
@@ -226,4 +226,5 @@ public class ProcessingRenderer extends AsynchronousSectionedRenderer<Response>
return (IOException) super.getCause();
}
}
+
}
diff --git a/container-core/src/main/java/com/yahoo/processing/rendering/Renderer.java b/container-core/src/main/java/com/yahoo/processing/rendering/Renderer.java
index d04eda943af..a7fa557f71a 100644
--- a/container-core/src/main/java/com/yahoo/processing/rendering/Renderer.java
+++ b/container-core/src/main/java/com/yahoo/processing/rendering/Renderer.java
@@ -46,16 +46,11 @@ public abstract class Renderer<RESPONSE extends Response> extends AbstractCompon
* exception causing failure wrapped in an ExecutionException if rendering
* was not successful.
*
- * @param stream
- * a stream API bridge to JDisc
- * @param response
- * the response to render
- * @param execution
- * the execution which created this response
- * @param request
- * the request matching the response
- * @return a ListenableFuture containing a boolean where true indicates a
- * successful rendering
+ * @param stream a stream API bridge to JDisc
+ * @param response the response to render
+ * @param execution the execution which created this response
+ * @param request the request matching the response
+ * @return a ListenableFuture containing a boolean where true indicates a successful rendering
*/
public abstract ListenableFuture<Boolean> render(OutputStream stream, RESPONSE response,
Execution execution, Request request);
@@ -63,17 +58,14 @@ public abstract class Renderer<RESPONSE extends Response> extends AbstractCompon
/**
* Name of the output encoding, if applicable.
*
- *<p>TODO: ensure null is OK
- *
- * @return The encoding of the output if applicable, e.g. "utf-8"
+ * @return the encoding of the output if applicable, e.g. "utf-8"
*/
public abstract String getEncoding();
/**
* The MIME type of the rendered content sent to the client.
*
- * @return The mime type of the data written to the writer, e.g.
- * "text/plain"
+ * @return The mime type of the data written to the writer, e.g. "text/plain"
*/
public abstract String getMimeType();
diff --git a/container-core/src/main/java/com/yahoo/restapi/JacksonJsonResponse.java b/container-core/src/main/java/com/yahoo/restapi/JacksonJsonResponse.java
index 94785819aa6..0a2c08530aa 100644
--- a/container-core/src/main/java/com/yahoo/restapi/JacksonJsonResponse.java
+++ b/container-core/src/main/java/com/yahoo/restapi/JacksonJsonResponse.java
@@ -19,8 +19,8 @@ import java.util.logging.Logger;
public class JacksonJsonResponse<T> extends HttpResponse {
private static final Logger log = Logger.getLogger(JacksonJsonResponse.class.getName());
- private static final ObjectMapper defaultJsonMapper = new ObjectMapper()
- .registerModule(new JavaTimeModule()).registerModule(new Jdk8Module());
+ private static final ObjectMapper defaultJsonMapper =
+ new ObjectMapper().registerModule(new JavaTimeModule()).registerModule(new Jdk8Module());
private final ObjectMapper jsonMapper;
private final T entity;
@@ -48,4 +48,5 @@ public class JacksonJsonResponse<T> extends HttpResponse {
@Override public String getContentType() { return "application/json"; }
public T getEntity() { return entity; }
+
}
diff --git a/container-disc/src/main/java/com/yahoo/container/FilterConfigProvider.java b/container-disc/src/main/java/com/yahoo/container/FilterConfigProvider.java
index c17b9d445a2..b22a8314d2b 100644
--- a/container-disc/src/main/java/com/yahoo/container/FilterConfigProvider.java
+++ b/container-disc/src/main/java/com/yahoo/container/FilterConfigProvider.java
@@ -74,4 +74,5 @@ public final class FilterConfigProvider implements Provider<FilterConfig> {
@Override
public void deconstruct() {}
+
}
diff --git a/container-disc/src/main/java/com/yahoo/container/jdisc/AthenzIdentityProviderProvider.java b/container-disc/src/main/java/com/yahoo/container/jdisc/AthenzIdentityProviderProvider.java
index 0e3110e26a8..09febdabc60 100644
--- a/container-disc/src/main/java/com/yahoo/container/jdisc/AthenzIdentityProviderProvider.java
+++ b/container-disc/src/main/java/com/yahoo/container/jdisc/AthenzIdentityProviderProvider.java
@@ -14,6 +14,7 @@ import java.util.List;
* @author mortent
*/
public class AthenzIdentityProviderProvider implements Provider<AthenzIdentityProvider> {
+
private static final ThrowingAthenzIdentityProvider instance = new ThrowingAthenzIdentityProvider();
@Override
@@ -84,4 +85,5 @@ public class AthenzIdentityProviderProvider implements Provider<AthenzIdentityPr
throw new UnsupportedOperationException(message);
}
}
+
}
diff --git a/container-disc/src/main/java/com/yahoo/container/jdisc/RestrictedBundleContext.java b/container-disc/src/main/java/com/yahoo/container/jdisc/RestrictedBundleContext.java
index a8acaf7dd10..baf7ac8c4dc 100644
--- a/container-disc/src/main/java/com/yahoo/container/jdisc/RestrictedBundleContext.java
+++ b/container-disc/src/main/java/com/yahoo/container/jdisc/RestrictedBundleContext.java
@@ -22,25 +22,19 @@ public class RestrictedBundleContext implements BundleContext {
@Override
public ServiceRegistration<?> registerService(String[] strings, Object o, Dictionary<String, ?> stringDictionary) {
- if (wrapped == null) {
- return null;
- }
+ if (wrapped == null) return null;
return wrapped.registerService(strings, o, stringDictionary);
}
@Override
public ServiceRegistration<?> registerService(String localHostname, Object o, Dictionary<String, ?> stringDictionary) {
- if (wrapped == null) {
- return null;
- }
+ if (wrapped == null) return null;
return wrapped.registerService(localHostname, o, stringDictionary);
}
@Override
public <S> ServiceRegistration<S> registerService(Class<S> sClass, S s, Dictionary<String, ?> stringDictionary) {
- if (wrapped == null) {
- return null;
- }
+ if (wrapped == null) return null;
return wrapped.registerService(sClass, s, stringDictionary);
}
@@ -51,57 +45,43 @@ public class RestrictedBundleContext implements BundleContext {
@Override
public ServiceReference<?>[] getServiceReferences(String localHostname, String localHostname2) throws InvalidSyntaxException {
- if (wrapped == null) {
- return new ServiceReference<?>[0];
- }
+ if (wrapped == null) return new ServiceReference<?>[0];
return wrapped.getServiceReferences(localHostname, localHostname2);
}
@Override
public ServiceReference<?>[] getAllServiceReferences(String localHostname, String localHostname2) throws InvalidSyntaxException {
- if (wrapped == null) {
- return new ServiceReference<?>[0];
- }
+ if (wrapped == null) return new ServiceReference<?>[0];
return wrapped.getAllServiceReferences(localHostname, localHostname2);
}
@Override
public ServiceReference<?> getServiceReference(String localHostname) {
- if (wrapped == null) {
- return null;
- }
+ if (wrapped == null) return null;
return wrapped.getServiceReference(localHostname);
}
@Override
public <S> ServiceReference<S> getServiceReference(Class<S> sClass) {
- if (wrapped == null) {
- return null;
- }
+ if (wrapped == null) return null;
return wrapped.getServiceReference(sClass);
}
@Override
public <S> Collection<ServiceReference<S>> getServiceReferences(Class<S> sClass, String localHostname) throws InvalidSyntaxException {
- if (wrapped == null) {
- return Collections.<ServiceReference<S>>emptyList();
- }
+ if (wrapped == null) return Collections.<ServiceReference<S>>emptyList();
return wrapped.getServiceReferences(sClass, localHostname);
}
@Override
public <S> S getService(ServiceReference<S> sServiceReference) {
- if (wrapped == null) {
- return null;
- }
+ if (wrapped == null) return null;
return wrapped.getService(sServiceReference);
}
@Override
public boolean ungetService(ServiceReference<?> serviceReference) {
- if (wrapped == null) {
- return false;
- }
+ if (wrapped == null) return false;
return wrapped.ungetService(serviceReference);
}
@@ -110,10 +90,6 @@ public class RestrictedBundleContext implements BundleContext {
return null;
}
-
- //---------------------
-
-
@Override
public String getProperty(String localHostname) {
throw newException();
diff --git a/container-disc/src/main/java/com/yahoo/container/jdisc/SecretStoreProvider.java b/container-disc/src/main/java/com/yahoo/container/jdisc/SecretStoreProvider.java
index 9c1dd00fdd4..eca62bc6ae6 100644
--- a/container-disc/src/main/java/com/yahoo/container/jdisc/SecretStoreProvider.java
+++ b/container-disc/src/main/java/com/yahoo/container/jdisc/SecretStoreProvider.java
@@ -34,4 +34,5 @@ public class SecretStoreProvider implements Provider<SecretStore> {
throw new SecretNotFoundException("A secret store is not available");
}
}
+
}
diff --git a/container-disc/src/main/java/com/yahoo/container/jdisc/SystemInfoProvider.java b/container-disc/src/main/java/com/yahoo/container/jdisc/SystemInfoProvider.java
index 0bb3832ddf5..b25517ec1f7 100644
--- a/container-disc/src/main/java/com/yahoo/container/jdisc/SystemInfoProvider.java
+++ b/container-disc/src/main/java/com/yahoo/container/jdisc/SystemInfoProvider.java
@@ -19,9 +19,12 @@ public class SystemInfoProvider extends AbstractComponent implements Provider<Sy
private final SystemInfo instance;
- @Inject public SystemInfoProvider(ConfigserverConfig config) {
+ @Inject
+ public SystemInfoProvider(ConfigserverConfig config) {
this.instance = new SystemInfo(new Zone(Environment.valueOf(config.environment()), config.region()));
}
- @Override public SystemInfo get() { return instance; }
+ @Override
+ public SystemInfo get() { return instance; }
+
}
diff --git a/container-disc/src/main/java/com/yahoo/container/jdisc/athenz/AthenzIdentityProvider.java b/container-disc/src/main/java/com/yahoo/container/jdisc/athenz/AthenzIdentityProvider.java
index 10bf96749e8..26014dba08c 100644
--- a/container-disc/src/main/java/com/yahoo/container/jdisc/athenz/AthenzIdentityProvider.java
+++ b/container-disc/src/main/java/com/yahoo/container/jdisc/athenz/AthenzIdentityProvider.java
@@ -11,6 +11,7 @@ import java.util.List;
* @author mortent
*/
public interface AthenzIdentityProvider {
+
String domain();
String service();
SSLContext getIdentitySslContext();
@@ -22,4 +23,5 @@ public interface AthenzIdentityProvider {
List<X509Certificate> getIdentityCertificate();
PrivateKey getPrivateKey();
Path trustStorePath();
+
}
diff --git a/container-disc/src/main/java/com/yahoo/container/jdisc/athenz/AthenzIdentityProviderException.java b/container-disc/src/main/java/com/yahoo/container/jdisc/athenz/AthenzIdentityProviderException.java
index fd5839bfc45..039a0535c32 100644
--- a/container-disc/src/main/java/com/yahoo/container/jdisc/athenz/AthenzIdentityProviderException.java
+++ b/container-disc/src/main/java/com/yahoo/container/jdisc/athenz/AthenzIdentityProviderException.java
@@ -13,4 +13,5 @@ public class AthenzIdentityProviderException extends RuntimeException {
public AthenzIdentityProviderException(String message, Throwable cause) {
super(message, cause);
}
+
}
diff --git a/container-disc/src/main/java/com/yahoo/container/jdisc/metric/DisableGuiceMetric.java b/container-disc/src/main/java/com/yahoo/container/jdisc/metric/DisableGuiceMetric.java
index 5c205cd5157..b168b21ac1c 100644
--- a/container-disc/src/main/java/com/yahoo/container/jdisc/metric/DisableGuiceMetric.java
+++ b/container-disc/src/main/java/com/yahoo/container/jdisc/metric/DisableGuiceMetric.java
@@ -28,4 +28,5 @@ public class DisableGuiceMetric implements Metric {
private static RuntimeException newException() {
return new UnsupportedOperationException("The Metric framework is only available to components.");
}
+
}
diff --git a/container-disc/src/main/java/com/yahoo/container/jdisc/metric/ForwardingMetricConsumer.java b/container-disc/src/main/java/com/yahoo/container/jdisc/metric/ForwardingMetricConsumer.java
index c9caaa9d4b5..60b7b0335bf 100644
--- a/container-disc/src/main/java/com/yahoo/container/jdisc/metric/ForwardingMetricConsumer.java
+++ b/container-disc/src/main/java/com/yahoo/container/jdisc/metric/ForwardingMetricConsumer.java
@@ -55,4 +55,5 @@ public final class ForwardingMetricConsumer implements MetricConsumer {
this.contexts = contexts;
}
}
+
}
diff --git a/container-disc/src/main/java/com/yahoo/container/jdisc/metric/GarbageCollectionMetrics.java b/container-disc/src/main/java/com/yahoo/container/jdisc/metric/GarbageCollectionMetrics.java
index c452fb2435d..17effe2136a 100644
--- a/container-disc/src/main/java/com/yahoo/container/jdisc/metric/GarbageCollectionMetrics.java
+++ b/container-disc/src/main/java/com/yahoo/container/jdisc/metric/GarbageCollectionMetrics.java
@@ -17,6 +17,7 @@ import java.util.Map;
* @author ollivir
*/
public class GarbageCollectionMetrics {
+
private static final String GC_COUNT = "jdisc.gc.count";
private static final String GC_TIME = "jdisc.gc.ms";
private static final String DIMENSION_KEY = "gcName";
@@ -35,7 +36,7 @@ public class GarbageCollectionMetrics {
}
}
- private Map<String, LinkedList<GcStats>> gcStatistics;
+ private final Map<String, LinkedList<GcStats>> gcStatistics;
private final Clock clock;
@@ -92,4 +93,5 @@ public class GarbageCollectionMetrics {
Map<String, LinkedList<GcStats>> getGcStatistics() {
return gcStatistics;
}
+
}
diff --git a/container-disc/src/main/java/com/yahoo/container/jdisc/metric/JrtMetrics.java b/container-disc/src/main/java/com/yahoo/container/jdisc/metric/JrtMetrics.java
index b29d7fe1f21..22a335a8171 100644
--- a/container-disc/src/main/java/com/yahoo/container/jdisc/metric/JrtMetrics.java
+++ b/container-disc/src/main/java/com/yahoo/container/jdisc/metric/JrtMetrics.java
@@ -38,4 +38,5 @@ class JrtMetrics {
metric.add(metricName, countIncrement, null);
}
}
+
}
diff --git a/container-disc/src/main/java/com/yahoo/container/jdisc/metric/MetricUpdater.java b/container-disc/src/main/java/com/yahoo/container/jdisc/metric/MetricUpdater.java
index 05c3b88b788..0c26d6eefd9 100644
--- a/container-disc/src/main/java/com/yahoo/container/jdisc/metric/MetricUpdater.java
+++ b/container-disc/src/main/java/com/yahoo/container/jdisc/metric/MetricUpdater.java
@@ -20,7 +20,6 @@ import java.util.TimerTask;
*
* @author bjorncs
* @author vegardh
- *
*/
public class MetricUpdater extends AbstractComponent {
@@ -139,5 +138,6 @@ public class MetricUpdater extends AbstractComponent {
void schedule(Runnable runnable, Duration frequency);
void cancel();
}
+
}
diff --git a/container-disc/src/main/java/com/yahoo/container/jdisc/secretstore/SecretStore.java b/container-disc/src/main/java/com/yahoo/container/jdisc/secretstore/SecretStore.java
index 7cd8e11c677..8af1f9860bf 100644
--- a/container-disc/src/main/java/com/yahoo/container/jdisc/secretstore/SecretStore.java
+++ b/container-disc/src/main/java/com/yahoo/container/jdisc/secretstore/SecretStore.java
@@ -8,6 +8,7 @@ import java.util.List;
* @author mortent
*/
public interface SecretStore {
+
/** Returns the secret for this key */
String getSecret(String key);
@@ -18,4 +19,5 @@ public interface SecretStore {
default List<Integer> listSecretVersions(String key) {
throw new UnsupportedOperationException("Secret store does not support listing versions");
}
+
}
diff --git a/container-disc/src/main/java/com/yahoo/container/usability/BindingsOverviewHandler.java b/container-disc/src/main/java/com/yahoo/container/usability/BindingsOverviewHandler.java
index ae65fc3ad68..709441999d0 100644
--- a/container-disc/src/main/java/com/yahoo/container/usability/BindingsOverviewHandler.java
+++ b/container-disc/src/main/java/com/yahoo/container/usability/BindingsOverviewHandler.java
@@ -57,7 +57,7 @@ public class BindingsOverviewHandler extends AbstractRequestHandler {
@Override
protected com.yahoo.jdisc.Response newResponse() {
com.yahoo.jdisc.Response response = new com.yahoo.jdisc.Response(statusToReturn);
- response.headers().add("Content-Type", Arrays.asList(new String[]{"application/json"}));
+ response.headers().add("Content-Type", List.of("application/json"));
return response;
}
}.connect(handler));
@@ -110,12 +110,10 @@ public class BindingsOverviewHandler extends AbstractRequestHandler {
}
private static JSONArray renderBindings(List<String> bindings) {
- JSONArray ret = new JSONArray();
-
+ JSONArray array = new JSONArray();
for (String binding : bindings)
- ret.put(binding);
-
- return ret;
+ array.put(binding);
+ return array;
}
private static JSONObject renderComponent(Object component, ComponentId id) {
@@ -136,9 +134,9 @@ public class BindingsOverviewHandler extends AbstractRequestHandler {
try {
Bundle bundle = FrameworkUtil.getBundle(component.getClass());
- String bundleName = bundle != null ?
- bundle.getSymbolicName() + ":" + bundle.getVersion() :
- "From classpath";
+ String bundleName = bundle != null
+ ? bundle.getSymbolicName() + ":" + bundle.getVersion()
+ : "From classpath";
return new BundleInfo(component.getClass().getName(), bundleName);
} catch (Exception | NoClassDefFoundError e) {
return new BundleInfo("Unavailable, reconfiguration in progress.", "");
@@ -155,12 +153,15 @@ public class BindingsOverviewHandler extends AbstractRequestHandler {
}
static final class BundleInfo {
+
public final String className;
public final String bundleName;
+
BundleInfo(String className, String bundleName) {
this.className = className;
this.bundleName = bundleName;
}
+
}
static final class StatusResponse {
@@ -182,7 +183,8 @@ public class BindingsOverviewHandler extends AbstractRequestHandler {
}
- private class IgnoredContent implements ContentChannel {
+ private static class IgnoredContent implements ContentChannel {
+
@Override
public void write(ByteBuffer buf, CompletionHandler handler) {
handler.completed();
@@ -192,5 +194,7 @@ public class BindingsOverviewHandler extends AbstractRequestHandler {
public void close(CompletionHandler handler) {
handler.completed();
}
+
}
+
}
diff --git a/container-messagebus/src/main/java/com/yahoo/container/jdisc/messagebus/MbusClientProvider.java b/container-messagebus/src/main/java/com/yahoo/container/jdisc/messagebus/MbusClientProvider.java
index b90f96fb240..2b86d83547e 100644
--- a/container-messagebus/src/main/java/com/yahoo/container/jdisc/messagebus/MbusClientProvider.java
+++ b/container-messagebus/src/main/java/com/yahoo/container/jdisc/messagebus/MbusClientProvider.java
@@ -61,4 +61,5 @@ public class MbusClientProvider implements Provider<MbusClient> {
public void deconstruct() {
client.release();
}
+
}
diff --git a/container-messagebus/src/main/java/com/yahoo/container/jdisc/messagebus/SessionCache.java b/container-messagebus/src/main/java/com/yahoo/container/jdisc/messagebus/SessionCache.java
index e62f6a8a21a..68b1f5aa5db 100644
--- a/container-messagebus/src/main/java/com/yahoo/container/jdisc/messagebus/SessionCache.java
+++ b/container-messagebus/src/main/java/com/yahoo/container/jdisc/messagebus/SessionCache.java
@@ -159,7 +159,7 @@ public final class SessionCache extends AbstractComponent {
return sourcesCreator.retain(sourceLock, sources, p);
}
- private abstract class SessionCreator<PARAMS, KEY, SESSION extends SharedResource> {
+ private abstract static class SessionCreator<PARAMS, KEY, SESSION extends SharedResource> {
abstract SESSION create(PARAMS p);
@@ -352,6 +352,7 @@ public final class SessionCache extends AbstractComponent {
}
static class UnknownThrottlePolicySignature extends ThrottlePolicySignature {
+
private final ThrottlePolicy policy;
UnknownThrottlePolicySignature(final ThrottlePolicy policy) {
@@ -409,26 +410,16 @@ public final class SessionCache extends AbstractComponent {
@Override
public boolean equals(Object obj) {
- if (this == obj) {
- return true;
- }
- if (obj == null) {
- return false;
- }
- if (getClass() != obj.getClass()) {
- return false;
- }
+ if (this == obj) return true;
+ if (obj == null) return false;
+ if (getClass() != obj.getClass()) return false;
SourceSessionKey other = (SourceSessionKey) obj;
if (policy == null) {
- if (other.policy != null) {
- return false;
- }
+ if (other.policy != null) return false;
} else if (!policy.equals(other.policy)) {
return false;
}
- if (Double.doubleToLongBits(timeout) != Double.doubleToLongBits(other.timeout)) {
- return false;
- }
+ if (Double.doubleToLongBits(timeout) != Double.doubleToLongBits(other.timeout)) return false;
return true;
}
}
diff --git a/controller-api/src/main/java/com/yahoo/vespa/hosted/controller/api/integration/noderepository/NodeRepositoryNode.java b/controller-api/src/main/java/com/yahoo/vespa/hosted/controller/api/integration/noderepository/NodeRepositoryNode.java
index 7bb47185751..65d6f2a5fa6 100644
--- a/controller-api/src/main/java/com/yahoo/vespa/hosted/controller/api/integration/noderepository/NodeRepositoryNode.java
+++ b/controller-api/src/main/java/com/yahoo/vespa/hosted/controller/api/integration/noderepository/NodeRepositoryNode.java
@@ -8,7 +8,6 @@ import com.fasterxml.jackson.annotation.JsonInclude;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.databind.JsonNode;
-import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.Set;
@@ -29,6 +28,8 @@ public class NodeRepositoryNode {
private Set<String> ipAddresses;
@JsonProperty("additionalIpAddresses")
private Set<String> additionalIpAddresses;
+ @JsonProperty("additionalHostnames")
+ private List<String> additionalHostnames;
@JsonProperty("openStackId")
private String openStackId;
@JsonProperty("flavor")
@@ -142,6 +143,14 @@ public class NodeRepositoryNode {
this.additionalIpAddresses = additionalIpAddresses;
}
+ public List<String> getAdditionalHostnames() {
+ return additionalHostnames;
+ }
+
+ public void setAdditionalHostnames(List<String> additionalHostnames) {
+ this.additionalHostnames = additionalHostnames;
+ }
+
public String getOpenStackId() {
return openStackId;
}
@@ -397,6 +406,7 @@ public class NodeRepositoryNode {
", hostname='" + hostname + '\'' +
", ipAddresses=" + ipAddresses +
", additionalIpAddresses=" + additionalIpAddresses +
+ ", additionalHostnames=" + additionalHostnames +
", openStackId='" + openStackId + '\'' +
", flavor='" + flavor + '\'' +
", resources=" + resources +
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/application/ApplicationApiHandler.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/application/ApplicationApiHandler.java
index 1c0dca26f5a..4d928a6b8a7 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/application/ApplicationApiHandler.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/application/ApplicationApiHandler.java
@@ -1539,11 +1539,11 @@ public class ApplicationApiHandler extends LoggingRequestHandler {
private HttpResponse reindex(String tenantName, String applicationName, String instanceName, String environment, String region, HttpRequest request) {
ApplicationId id = ApplicationId.from(tenantName, applicationName, instanceName);
ZoneId zone = ZoneId.from(environment, region);
- List<String> clusterNames = Optional.ofNullable(request.getProperty("cluster")).stream()
+ List<String> clusterNames = Optional.ofNullable(request.getProperty("clusterId")).stream()
.flatMap(clusters -> Stream.of(clusters.split(",")))
.filter(cluster -> ! cluster.isBlank())
.collect(toUnmodifiableList());
- List<String> documentTypes = Optional.ofNullable(request.getProperty("type")).stream()
+ List<String> documentTypes = Optional.ofNullable(request.getProperty("documentType")).stream()
.flatMap(types -> Stream.of(types.split(",")))
.filter(type -> ! type.isBlank())
.collect(toUnmodifiableList());
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 199eee6d0c9..d6c6f5ff167 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
@@ -18,12 +18,14 @@ import com.yahoo.slime.Slime;
import com.yahoo.slime.SlimeUtils;
import com.yahoo.vespa.hosted.controller.ApplicationController;
import com.yahoo.vespa.hosted.controller.Controller;
+import com.yahoo.vespa.hosted.controller.TenantController;
import com.yahoo.vespa.hosted.controller.api.integration.billing.CollectionMethod;
import com.yahoo.vespa.hosted.controller.api.integration.billing.PaymentInstrument;
import com.yahoo.vespa.hosted.controller.api.integration.billing.Invoice;
import com.yahoo.vespa.hosted.controller.api.integration.billing.InstrumentOwner;
import com.yahoo.vespa.hosted.controller.api.integration.billing.BillingController;
import com.yahoo.vespa.hosted.controller.api.integration.billing.PlanId;
+import com.yahoo.vespa.hosted.controller.tenant.Tenant;
import com.yahoo.yolean.Exceptions;
import javax.ws.rs.BadRequestException;
@@ -37,6 +39,7 @@ import java.time.ZoneId;
import java.time.ZonedDateTime;
import java.time.format.DateTimeFormatter;
import java.time.format.DateTimeParseException;
+import java.util.Comparator;
import java.util.List;
import java.util.Optional;
import java.util.concurrent.Executor;
@@ -53,6 +56,7 @@ public class BillingApiHandler extends LoggingRequestHandler {
private final BillingController billingController;
private final ApplicationController applicationController;
+ private final TenantController tenantController;
public BillingApiHandler(Executor executor,
AccessLog accessLog,
@@ -60,6 +64,7 @@ public class BillingApiHandler extends LoggingRequestHandler {
super(executor, accessLog);
this.billingController = controller.serviceRegistry().billingController();
this.applicationController = controller.applications();
+ this.tenantController = controller.tenants();
}
@Override
@@ -175,15 +180,16 @@ public class BillingApiHandler extends LoggingRequestHandler {
root.setString("until", untilDate.format(DateTimeFormatter.ISO_DATE));
var tenants = root.setArray("tenants");
- uncommittedInvoices.forEach((tenant, invoice) -> {
+ tenantController.asList().stream().sorted(Comparator.comparing(Tenant::name)).forEach(tenant -> {
+ var invoice = uncommittedInvoices.get(tenant.name());
var tc = tenants.addObject();
- tc.setString("tenant", tenant.value());
- getPlanForTenant(tc, tenant);
- getCollectionForTenant(tc, tenant);
+ tc.setString("tenant", tenant.name().value());
+ getPlanForTenant(tc, tenant.name());
+ getCollectionForTenant(tc, tenant.name());
renderCurrentUsage(tc.setObject("current"), invoice);
- renderAdditionalItems(tc.setObject("additional").setArray("items"), billingController.getUnusedLineItems(tenant));
+ renderAdditionalItems(tc.setObject("additional").setArray("items"), billingController.getUnusedLineItems(tenant.name()));
- billingController.getDefaultInstrument(tenant).ifPresent(card ->
+ billingController.getDefaultInstrument(tenant.name()).ifPresent(card ->
renderInstrument(tc.setObject("payment"), card)
);
});
@@ -302,6 +308,7 @@ public class BillingApiHandler extends LoggingRequestHandler {
}
private void renderCurrentUsage(Cursor cursor, Invoice currentUsage) {
+ if (currentUsage == null) return;
cursor.setString("amount", currentUsage.sum().toPlainString());
cursor.setString("status", "accrued");
cursor.setString("from", currentUsage.getStartTime().format(DATE_TIME_FORMATTER));
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/ApplicationApiTest.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/ApplicationApiTest.java
index 7e097304790..5e98ac0d3ee 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/ApplicationApiTest.java
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/ApplicationApiTest.java
@@ -584,14 +584,14 @@ public class ApplicationApiTest extends ControllerContainerTest {
// POST a 'reindex application' command with cluster filter
tester.assertResponse(request("/application/v4/tenant/tenant1/application/application1/instance/instance1/environment/prod/region/us-central-1/reindex", POST)
- .properties(Map.of("cluster", "boo,moo"))
+ .properties(Map.of("clusterId", "boo,moo"))
.userIdentity(USER_ID),
"{\"message\":\"Requested reindexing of tenant1.application1.instance1 in prod.us-central-1, on clusters boo, moo\"}");
// POST a 'reindex application' command with cluster and document type filters
tester.assertResponse(request("/application/v4/tenant/tenant1/application/application1/instance/instance1/environment/prod/region/us-central-1/reindex", POST)
- .properties(Map.of("cluster", "boo,moo",
- "type", "foo,boo"))
+ .properties(Map.of("clusterId", "boo,moo",
+ "documentType", "foo,boo"))
.userIdentity(USER_ID),
"{\"message\":\"Requested reindexing of tenant1.application1.instance1 in prod.us-central-1, on clusters boo, moo, for types foo, boo\"}");
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 8493250d9a3..1d0f0935c05 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
@@ -2,6 +2,7 @@ package com.yahoo.vespa.hosted.controller.restapi.billing;
import com.yahoo.config.provision.SystemName;
import com.yahoo.config.provision.TenantName;
+import com.yahoo.vespa.hosted.controller.ControllerTester;
import com.yahoo.vespa.hosted.controller.api.integration.billing.CollectionMethod;
import com.yahoo.vespa.hosted.controller.api.integration.billing.Invoice;
import com.yahoo.vespa.hosted.controller.api.integration.billing.MockBillingController;
@@ -9,12 +10,17 @@ import com.yahoo.vespa.hosted.controller.api.integration.billing.PlanId;
import com.yahoo.vespa.hosted.controller.api.role.Role;
import com.yahoo.vespa.hosted.controller.restapi.ContainerTester;
import com.yahoo.vespa.hosted.controller.restapi.ControllerContainerCloudTest;
+import com.yahoo.vespa.hosted.controller.security.Auth0Credentials;
+import com.yahoo.vespa.hosted.controller.security.CloudTenantSpec;
+import com.yahoo.vespa.hosted.controller.security.Credentials;
+import com.yahoo.vespa.hosted.controller.security.TenantSpec;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import java.io.File;
import java.math.BigDecimal;
+import java.security.Principal;
import java.time.LocalDate;
import java.time.ZoneId;
import java.time.ZonedDateTime;
@@ -165,6 +171,9 @@ public class BillingApiHandlerTest extends ControllerContainerCloudTest {
@Test
public void list_all_uninvoiced_items() {
+ tester.controller().tenants().create(new CloudTenantSpec(tenant, ""), new Auth0Credentials(() -> "foo", Set.of(Role.hostedOperator())));
+ tester.controller().tenants().create(new CloudTenantSpec(tenant2, ""), new Auth0Credentials(() -> "foo", Set.of(Role.hostedOperator())));
+
var invoice = createInvoice();
billingController.setPlan(tenant, PlanId.from("some-plan"), true);
billingController.setPlan(tenant2, PlanId.from("some-plan"), true);
@@ -172,7 +181,6 @@ public class BillingApiHandlerTest extends ControllerContainerCloudTest {
billingController.addLineItem(tenant, "support", new BigDecimal("42"), "Smith");
billingController.addInvoice(tenant2, invoice, false);
-
var request = request("/billing/v1/billing?until=2020-05-28").roles(financeAdmin);
tester.assertResponse(request, new File("billing-all-tenants"));
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/billing/responses/billing-all-tenants b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/billing/responses/billing-all-tenants
index 81868a44e57..5c61dc6e86e 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/billing/responses/billing-all-tenants
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/billing/responses/billing-all-tenants
@@ -2,7 +2,7 @@
"until":"2020-05-28",
"tenants":[
{
- "tenant":"tenant2",
+ "tenant":"tenant1",
"plan":"some-plan",
"planName":"Plan with id: some-plan",
"collection": "AUTO",
@@ -20,10 +20,21 @@
}
]
},
- "additional":{"items":[]}
+ "additional":
+ {
+ "items":[
+ {
+ "id":"line-item-id",
+ "description":"support",
+ "amount":"42.00",
+ "plan":"some-plan",
+ "planName":"Plan with id: some-plan"
+ }
+ ]
+ }
},
{
- "tenant":"tenant1",
+ "tenant":"tenant2",
"plan":"some-plan",
"planName":"Plan with id: some-plan",
"collection": "AUTO",
@@ -41,18 +52,7 @@
}
]
},
- "additional":
- {
- "items":[
- {
- "id":"line-item-id",
- "description":"support",
- "amount":"42.00",
- "plan":"some-plan",
- "planName":"Plan with id: some-plan"
- }
- ]
- }
+ "additional":{"items":[]}
}
]
} \ No newline at end of file
diff --git a/document/abi-spec.json b/document/abi-spec.json
index b119f9991b3..903b7a897df 100644
--- a/document/abi-spec.json
+++ b/document/abi-spec.json
@@ -1911,6 +1911,8 @@
"public java.lang.Class getValueClass()",
"public boolean isValueCompatible(com.yahoo.document.datatypes.FieldValue)",
"public com.yahoo.tensor.TensorType getTensorType()",
+ "public boolean equals(java.lang.Object)",
+ "public int hashCode()",
"public bridge synthetic com.yahoo.document.DataType clone()",
"public bridge synthetic com.yahoo.vespa.objects.Identifiable clone()",
"public bridge synthetic java.lang.Object clone()"
diff --git a/document/src/main/java/com/yahoo/document/DataType.java b/document/src/main/java/com/yahoo/document/DataType.java
index fa5dffd042a..104d63cae96 100644
--- a/document/src/main/java/com/yahoo/document/DataType.java
+++ b/document/src/main/java/com/yahoo/document/DataType.java
@@ -84,6 +84,7 @@ public abstract class DataType extends Identifiable implements Serializable, Com
this.dataTypeId = dataTypeId;
}
+ @Override
public DataType clone() {
return (DataType)super.clone();
}
@@ -248,14 +249,17 @@ public abstract class DataType extends Identifiable implements Serializable, Com
manager.registerSingleType(this);
}
+ @Override
public int hashCode() {
return name.hashCode();
}
+ @Override
public boolean equals(Object other) {
return (other instanceof DataType) && (dataTypeId == ((DataType)other).dataTypeId);
}
+ @Override
public String toString() {
return "datatype " + name + " (code: " + dataTypeId + ")";
}
diff --git a/document/src/main/java/com/yahoo/document/TensorDataType.java b/document/src/main/java/com/yahoo/document/TensorDataType.java
index b21461597bf..c4fdff30f8b 100644
--- a/document/src/main/java/com/yahoo/document/TensorDataType.java
+++ b/document/src/main/java/com/yahoo/document/TensorDataType.java
@@ -6,6 +6,8 @@ import com.yahoo.document.datatypes.TensorFieldValue;
import com.yahoo.tensor.TensorType;
import com.yahoo.vespa.objects.Ids;
+import java.util.Objects;
+
/**
* A DataType containing a tensor type
*
@@ -23,6 +25,7 @@ public class TensorDataType extends DataType {
this.tensorType = tensorType;
}
+ @Override
public TensorDataType clone() {
return (TensorDataType)super.clone();
}
@@ -48,4 +51,17 @@ public class TensorDataType extends DataType {
/** Returns the type of the tensor this field can hold */
public TensorType getTensorType() { return tensorType; }
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) return true;
+ if (o == null || getClass() != o.getClass()) return false;
+ if (!super.equals(o)) return false;
+ TensorDataType that = (TensorDataType) o;
+ return Objects.equals(tensorType, that.tensorType);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(super.hashCode(), tensorType);
+ }
}
diff --git a/eval/src/tests/eval/aggr/aggr_test.cpp b/eval/src/tests/eval/aggr/aggr_test.cpp
index b3e9c625fd9..9045df68305 100644
--- a/eval/src/tests/eval/aggr/aggr_test.cpp
+++ b/eval/src/tests/eval/aggr/aggr_test.cpp
@@ -19,6 +19,36 @@ TEST("require that aggregator list returns appropriate entries") {
EXPECT_EQUAL(int(list[6]), int(Aggr::MIN));
}
+TEST("require that aggr::is_simple works as expected") {
+ EXPECT_FALSE(aggr::is_simple(Aggr::AVG));
+ EXPECT_FALSE(aggr::is_simple(Aggr::COUNT));
+ EXPECT_TRUE (aggr::is_simple(Aggr::PROD));
+ EXPECT_TRUE (aggr::is_simple(Aggr::SUM));
+ EXPECT_TRUE (aggr::is_simple(Aggr::MAX));
+ EXPECT_FALSE(aggr::is_simple(Aggr::MEDIAN));
+ EXPECT_TRUE (aggr::is_simple(Aggr::MIN));
+}
+
+TEST("require that aggr::is_ident works as expected") {
+ EXPECT_TRUE (aggr::is_ident(Aggr::AVG));
+ EXPECT_FALSE(aggr::is_ident(Aggr::COUNT));
+ EXPECT_TRUE (aggr::is_ident(Aggr::PROD));
+ EXPECT_TRUE (aggr::is_ident(Aggr::SUM));
+ EXPECT_TRUE (aggr::is_ident(Aggr::MAX));
+ EXPECT_TRUE (aggr::is_ident(Aggr::MEDIAN));
+ EXPECT_TRUE (aggr::is_ident(Aggr::MIN));
+}
+
+TEST("require that aggr::is_complex works as expected") {
+ EXPECT_FALSE(aggr::is_complex(Aggr::AVG));
+ EXPECT_FALSE(aggr::is_complex(Aggr::COUNT));
+ EXPECT_FALSE(aggr::is_complex(Aggr::PROD));
+ EXPECT_FALSE(aggr::is_complex(Aggr::SUM));
+ EXPECT_FALSE(aggr::is_complex(Aggr::MAX));
+ EXPECT_TRUE (aggr::is_complex(Aggr::MEDIAN));
+ EXPECT_FALSE(aggr::is_complex(Aggr::MIN));
+}
+
TEST("require that AVG aggregator works as expected") {
Stash stash;
Aggregator &aggr = Aggregator::create(Aggr::AVG, stash);
@@ -28,6 +58,7 @@ TEST("require that AVG aggregator works as expected") {
aggr.next(30.0), EXPECT_EQUAL(aggr.result(), 20.0);
aggr.first(100.0), EXPECT_EQUAL(aggr.result(), 100.0);
aggr.next(200.0), EXPECT_EQUAL(aggr.result(), 150.0);
+ EXPECT_TRUE(aggr.enum_value() == Aggr::AVG);
}
TEST("require that COUNT aggregator works as expected") {
@@ -39,6 +70,7 @@ TEST("require that COUNT aggregator works as expected") {
aggr.next(30.0), EXPECT_EQUAL(aggr.result(), 3.0);
aggr.first(100.0), EXPECT_EQUAL(aggr.result(), 1.0);
aggr.next(200.0), EXPECT_EQUAL(aggr.result(), 2.0);
+ EXPECT_TRUE(aggr.enum_value() == Aggr::COUNT);
}
TEST("require that PROD aggregator works as expected") {
@@ -50,6 +82,13 @@ TEST("require that PROD aggregator works as expected") {
aggr.next(30.0), EXPECT_EQUAL(aggr.result(), 6000.0);
aggr.first(100.0), EXPECT_EQUAL(aggr.result(), 100.0);
aggr.next(200.0), EXPECT_EQUAL(aggr.result(), 20000.0);
+ EXPECT_TRUE(aggr.enum_value() == Aggr::PROD);
+}
+
+TEST("require that Prod combine works as expected") {
+ using Type = Prod<double>;
+ EXPECT_EQUAL(Type::combine(3,7), 21.0);
+ EXPECT_EQUAL(Type::combine(5,4), 20.0);
}
TEST("require that SUM aggregator works as expected") {
@@ -61,6 +100,13 @@ TEST("require that SUM aggregator works as expected") {
aggr.next(30.0), EXPECT_EQUAL(aggr.result(), 60.0);
aggr.first(100.0), EXPECT_EQUAL(aggr.result(), 100.0);
aggr.next(200.0), EXPECT_EQUAL(aggr.result(), 300.0);
+ EXPECT_TRUE(aggr.enum_value() == Aggr::SUM);
+}
+
+TEST("require that Sum combine works as expected") {
+ using Type = Sum<double>;
+ EXPECT_EQUAL(Type::combine(3,7), 10.0);
+ EXPECT_EQUAL(Type::combine(5,4), 9.0);
}
TEST("require that MAX aggregator works as expected") {
@@ -72,6 +118,13 @@ TEST("require that MAX aggregator works as expected") {
aggr.next(30.0), EXPECT_EQUAL(aggr.result(), 30.0);
aggr.first(100.0), EXPECT_EQUAL(aggr.result(), 100.0);
aggr.next(200.0), EXPECT_EQUAL(aggr.result(), 200.0);
+ EXPECT_TRUE(aggr.enum_value() == Aggr::MAX);
+}
+
+TEST("require that Max combine works as expected") {
+ using Type = Max<double>;
+ EXPECT_EQUAL(Type::combine(3,7), 7.0);
+ EXPECT_EQUAL(Type::combine(5,4), 5.0);
}
TEST("require that MEDIAN aggregator works as expected") {
@@ -85,6 +138,7 @@ TEST("require that MEDIAN aggregator works as expected") {
aggr.next(16.0), EXPECT_EQUAL(aggr.result(), 16.0);
aggr.first(100.0), EXPECT_EQUAL(aggr.result(), 100.0);
aggr.next(200.0), EXPECT_EQUAL(aggr.result(), 150.0);
+ EXPECT_TRUE(aggr.enum_value() == Aggr::MEDIAN);
}
TEST("require that MEDIAN aggregator handles NaN values") {
@@ -108,6 +162,13 @@ TEST("require that MIN aggregator works as expected") {
aggr.next(30.0), EXPECT_EQUAL(aggr.result(), 10.0);
aggr.first(100.0), EXPECT_EQUAL(aggr.result(), 100.0);
aggr.next(200.0), EXPECT_EQUAL(aggr.result(), 100.0);
+ EXPECT_TRUE(aggr.enum_value() == Aggr::MIN);
+}
+
+TEST("require that Min combine works as expected") {
+ using Type = Min<double>;
+ EXPECT_EQUAL(Type::combine(3,7), 3.0);
+ EXPECT_EQUAL(Type::combine(5,4), 4.0);
}
template <template <typename T> typename A>
diff --git a/eval/src/tests/tensor/dense_single_reduce_function/dense_single_reduce_function_test.cpp b/eval/src/tests/tensor/dense_single_reduce_function/dense_single_reduce_function_test.cpp
index 949c5277e18..d3495befe7e 100644
--- a/eval/src/tests/tensor/dense_single_reduce_function/dense_single_reduce_function_test.cpp
+++ b/eval/src/tests/tensor/dense_single_reduce_function/dense_single_reduce_function_test.cpp
@@ -26,6 +26,7 @@ const TensorEngine &prod_engine = DefaultTensorEngine::ref();
EvalFixture::ParamRepo make_params() {
return EvalFixture::ParamRepo()
.add_dense({{"a", 2}, {"b", 3}, {"c", 4}, {"d", 5}})
+ .add_dense({{"a", 9}, {"b", 9}, {"c", 9}, {"d", 9}})
.add_cube("a", 2, "b", 1, "c", 1)
.add_cube("a", 1, "b", 2, "c", 1)
.add_cube("a", 1, "b", 1, "c", 2)
@@ -36,17 +37,35 @@ EvalFixture::ParamRepo make_params() {
}
EvalFixture::ParamRepo param_repo = make_params();
-void verify_optimized(const vespalib::string &expr, size_t dim_idx, Aggr aggr)
-{
+struct ReduceSpec {
+ size_t outer_size;
+ size_t reduce_size;
+ size_t inner_size;
+ Aggr aggr;
+};
+
+void verify_optimized_impl(const vespalib::string &expr, const std::vector<ReduceSpec> &spec_list) {
EvalFixture slow_fixture(prod_engine, expr, param_repo, false);
EvalFixture fixture(prod_engine, expr, param_repo, true);
EXPECT_EQUAL(fixture.result(), EvalFixture::ref(expr, param_repo));
EXPECT_EQUAL(fixture.result(), slow_fixture.result());
auto info = fixture.find_all<DenseSingleReduceFunction>();
- ASSERT_EQUAL(info.size(), 1u);
- EXPECT_TRUE(info[0]->result_is_mutable());
- EXPECT_EQUAL(info[0]->dim_idx(), dim_idx);
- EXPECT_EQUAL(int(info[0]->aggr()), int(aggr));
+ ASSERT_EQUAL(info.size(), spec_list.size());
+ for (size_t i = 0; i < spec_list.size(); ++i) {
+ EXPECT_TRUE(info[i]->result_is_mutable());
+ EXPECT_EQUAL(info[i]->outer_size(), spec_list[i].outer_size);
+ EXPECT_EQUAL(info[i]->reduce_size(), spec_list[i].reduce_size);
+ EXPECT_EQUAL(info[i]->inner_size(), spec_list[i].inner_size);
+ EXPECT_EQUAL(int(info[i]->aggr()), int(spec_list[i].aggr));
+ }
+}
+
+void verify_optimized(const vespalib::string &expr, const ReduceSpec &spec) {
+ verify_optimized_impl(expr, {spec});
+}
+
+void verify_optimized(const vespalib::string &expr, const ReduceSpec &spec1, const ReduceSpec &spec2) {
+ verify_optimized_impl(expr, {spec1, spec2});
}
void verify_not_optimized(const vespalib::string &expr) {
@@ -58,11 +77,6 @@ void verify_not_optimized(const vespalib::string &expr) {
EXPECT_TRUE(info.empty());
}
-TEST("require that multi-dimensional reduce is not optimized") {
- TEST_DO(verify_not_optimized("reduce(a2b3c4d5,sum,a,b)"));
- TEST_DO(verify_not_optimized("reduce(a2b3c4d5,sum,c,d)"));
-}
-
TEST("require that reduce to scalar is not optimized") {
TEST_DO(verify_not_optimized("reduce(a10,sum,a)"));
TEST_DO(verify_not_optimized("reduce(a10,sum)"));
@@ -79,45 +93,83 @@ TEST("require that mixed reduce is not optimized") {
TEST_DO(verify_not_optimized("reduce(xyz_mixed,sum,z)"));
}
-// NB: these are shadowed by the remove dimension optimization
-TEST("require that reducing self-aggregating trivial dimensions is not optimized") {
+TEST("require that reducing trivial dimensions is not optimized") {
TEST_DO(verify_not_optimized("reduce(a1b1c1,avg,c)"));
+ TEST_DO(verify_not_optimized("reduce(a1b1c1,count,c)"));
TEST_DO(verify_not_optimized("reduce(a1b1c1,prod,c)"));
TEST_DO(verify_not_optimized("reduce(a1b1c1,sum,c)"));
TEST_DO(verify_not_optimized("reduce(a1b1c1,max,c)"));
+ TEST_DO(verify_not_optimized("reduce(a1b1c1,median,c)"));
TEST_DO(verify_not_optimized("reduce(a1b1c1,min,c)"));
}
-TEST("require that reducing trivial dimension with COUNT is 'optimized'") {
- TEST_DO(verify_optimized("reduce(a1b1c1,count,a)", 0, Aggr::COUNT));
- TEST_DO(verify_optimized("reduce(a1b1c1,count,b)", 1, Aggr::COUNT));
- TEST_DO(verify_optimized("reduce(a1b1c1,count,c)", 2, Aggr::COUNT));
+TEST("require that atleast_8 dense single reduce works") {
+ TEST_DO(verify_optimized("reduce(a9b9c9d9,avg,a)", {1, 9, 729, Aggr::AVG}));
+ TEST_DO(verify_optimized("reduce(a9b9c9d9,avg,b)", {9, 9, 81, Aggr::AVG}));
+ TEST_DO(verify_optimized("reduce(a9b9c9d9,avg,c)", {81, 9, 9, Aggr::AVG}));
+ TEST_DO(verify_optimized("reduce(a9b9c9d9,avg,d)", {729, 9, 1, Aggr::AVG}));
+ TEST_DO(verify_optimized("reduce(a9b9c9d9,sum,c,d)", {81, 81, 1, Aggr::SUM}));
+}
+
+TEST("require that simple aggregators can be decomposed into multiple reduce operations") {
+ TEST_DO(verify_optimized("reduce(a2b3c4d5,sum,a,c)", {3, 4, 5, Aggr::SUM}, {1, 2, 60, Aggr::SUM}));
+ TEST_DO(verify_optimized("reduce(a2b3c4d5,min,a,c)", {3, 4, 5, Aggr::MIN}, {1, 2, 60, Aggr::MIN}));
+ TEST_DO(verify_optimized("reduce(a2b3c4d5,max,a,c)", {3, 4, 5, Aggr::MAX}, {1, 2, 60, Aggr::MAX}));
+}
+
+TEST("require that reduce dimensions can be listed in reverse order") {
+ TEST_DO(verify_optimized("reduce(a2b3c4d5,sum,c,a)", {3, 4, 5, Aggr::SUM}, {1, 2, 60, Aggr::SUM}));
+ TEST_DO(verify_optimized("reduce(a2b3c4d5,min,c,a)", {3, 4, 5, Aggr::MIN}, {1, 2, 60, Aggr::MIN}));
+ TEST_DO(verify_optimized("reduce(a2b3c4d5,max,c,a)", {3, 4, 5, Aggr::MAX}, {1, 2, 60, Aggr::MAX}));
+}
+
+TEST("require that non-simple aggregators cannot be decomposed into multiple reduce operations") {
+ TEST_DO(verify_not_optimized("reduce(a2b3c4d5,avg,a,c)"));
+ TEST_DO(verify_not_optimized("reduce(a2b3c4d5,count,a,c)"));
+ TEST_DO(verify_not_optimized("reduce(a2b3c4d5,median,a,c)"));
}
vespalib::string make_expr(const vespalib::string &arg, const vespalib::string &dim, bool float_cells, Aggr aggr) {
return make_string("reduce(%s%s,%s,%s)", arg.c_str(), float_cells ? "f" : "", AggrNames::name_of(aggr)->c_str(), dim.c_str());
}
-void verify_optimized_multi(const vespalib::string &arg, const vespalib::string &dim, size_t dim_idx) {
+void verify_optimized_multi(const vespalib::string &arg, const vespalib::string &dim, size_t outer_size, size_t reduce_size, size_t inner_size) {
for (bool float_cells: {false, true}) {
for (Aggr aggr: Aggregator::list()) {
- auto expr = make_expr(arg, dim, float_cells, aggr);
- TEST_DO(verify_optimized(expr, dim_idx, aggr));
+ if (aggr != Aggr::PROD) {
+ auto expr = make_expr(arg, dim, float_cells, aggr);
+ TEST_DO(verify_optimized(expr, {outer_size, reduce_size, inner_size, aggr}));
+ }
}
}
}
TEST("require that normal dense single reduce works") {
- TEST_DO(verify_optimized_multi("a2b3c4d5", "a", 0));
- TEST_DO(verify_optimized_multi("a2b3c4d5", "b", 1));
- TEST_DO(verify_optimized_multi("a2b3c4d5", "c", 2));
- TEST_DO(verify_optimized_multi("a2b3c4d5", "d", 3));
+ TEST_DO(verify_optimized_multi("a2b3c4d5", "a", 1, 2, 60));
+ TEST_DO(verify_optimized_multi("a2b3c4d5", "b", 2, 3, 20));
+ TEST_DO(verify_optimized_multi("a2b3c4d5", "c", 6, 4, 5));
+ TEST_DO(verify_optimized_multi("a2b3c4d5", "d", 24, 5, 1));
+}
+
+TEST("require that dimension-combined dense single reduce works") {
+ TEST_DO(verify_optimized_multi("a2b3c4d5", "a,b", 1, 6, 20));
+ TEST_DO(verify_optimized_multi("a2b3c4d5", "b,c", 2, 12, 5));
+ TEST_DO(verify_optimized_multi("a2b3c4d5", "c,d", 6, 20, 1));
}
TEST("require that minimal dense single reduce works") {
- TEST_DO(verify_optimized_multi("a2b1c1", "a", 0));
- TEST_DO(verify_optimized_multi("a1b2c1", "b", 1));
- TEST_DO(verify_optimized_multi("a1b1c2", "c", 2));
+ TEST_DO(verify_optimized_multi("a2b1c1", "a", 1, 2, 1));
+ TEST_DO(verify_optimized_multi("a1b2c1", "b", 1, 2, 1));
+ TEST_DO(verify_optimized_multi("a1b1c2", "c", 1, 2, 1));
+}
+
+TEST("require that trivial dimensions can be trivially reduced") {
+ TEST_DO(verify_optimized_multi("a2b1c1", "a,b", 1, 2, 1));
+ TEST_DO(verify_optimized_multi("a2b1c1", "a,c", 1, 2, 1));
+ TEST_DO(verify_optimized_multi("a1b2c1", "b,a", 1, 2, 1));
+ TEST_DO(verify_optimized_multi("a1b2c1", "b,c", 1, 2, 1));
+ TEST_DO(verify_optimized_multi("a1b1c2", "c,a", 1, 2, 1));
+ TEST_DO(verify_optimized_multi("a1b1c2", "c,b", 1, 2, 1));
}
TEST_MAIN() { TEST_RUN_ALL(); }
diff --git a/eval/src/tests/tensor/instruction_benchmark/instruction_benchmark.cpp b/eval/src/tests/tensor/instruction_benchmark/instruction_benchmark.cpp
index 8887f2cb6aa..816923bb87c 100644
--- a/eval/src/tests/tensor/instruction_benchmark/instruction_benchmark.cpp
+++ b/eval/src/tests/tensor/instruction_benchmark/instruction_benchmark.cpp
@@ -122,6 +122,32 @@ MyPeekSpec verbatim_peek() { return MyPeekSpec(false); }
//-----------------------------------------------------------------------------
+struct MultiOpParam {
+ std::vector<Instruction> list;
+};
+
+void my_multi_instruction_op(InterpretedFunction::State &state, uint64_t param_in) {
+ const auto &param = *(MultiOpParam*)(param_in);
+ for (const auto &item: param.list) {
+ item.perform(state);
+ }
+}
+
+void collect_op1_chain(const TensorFunction &node, const EngineOrFactory &engine, Stash &stash, std::vector<Instruction> &list) {
+ if (auto op1 = as<tensor_function::Op1>(node)) {
+ collect_op1_chain(op1->child(), engine, stash, list);
+ list.push_back(node.compile_self(engine, stash));
+ }
+}
+
+Instruction compile_op1_chain(const TensorFunction &node, const EngineOrFactory &engine, Stash &stash) {
+ auto &param = stash.create<MultiOpParam>();
+ collect_op1_chain(node, engine, stash, param.list);
+ return {my_multi_instruction_op,(uint64_t)(&param)};
+}
+
+//-----------------------------------------------------------------------------
+
struct Impl {
size_t order;
vespalib::string name;
@@ -145,7 +171,10 @@ struct Impl {
const auto &lhs_node = tensor_function::inject(lhs, 0, stash);
const auto &reduce_node = tensor_function::reduce(lhs_node, aggr, dims, stash);
const auto &node = optimize ? optimize_tensor_function(engine, reduce_node, stash) : reduce_node;
- return node.compile_self(engine, stash);
+ // since reduce might be optimized into multiple chained
+ // instructions, we need some extra magic to package these
+ // instructions into a single compound instruction.
+ return compile_op1_chain(node, engine, stash);
}
Instruction create_rename(const ValueType &lhs, const std::vector<vespalib::string> &from, const std::vector<vespalib::string> &to, Stash &stash) const {
// create a complete tensor function, but only compile the relevant instruction
diff --git a/eval/src/vespa/eval/eval/CMakeLists.txt b/eval/src/vespa/eval/eval/CMakeLists.txt
index d27de8e3d21..5cf7440237b 100644
--- a/eval/src/vespa/eval/eval/CMakeLists.txt
+++ b/eval/src/vespa/eval/eval/CMakeLists.txt
@@ -5,6 +5,7 @@ vespa_add_library(eval_eval OBJECT
array_array_map.cpp
basic_nodes.cpp
call_nodes.cpp
+ cell_type.cpp
compile_tensor_function.cpp
delete_node.cpp
double_value_builder.cpp
diff --git a/eval/src/vespa/eval/eval/aggr.cpp b/eval/src/vespa/eval/eval/aggr.cpp
index 4abd5e41f47..a73cc4314c6 100644
--- a/eval/src/vespa/eval/eval/aggr.cpp
+++ b/eval/src/vespa/eval/eval/aggr.cpp
@@ -17,6 +17,7 @@ struct Wrapper : Aggregator {
virtual void first(double value) final override { aggr = T{value}; }
virtual void next(double value) final override { aggr.sample(value); }
virtual double result() const final override { return aggr.result(); }
+ virtual Aggr enum_value() const final override { return T::enum_value(); }
};
} // namespace vespalib::eval::<unnamed>
diff --git a/eval/src/vespa/eval/eval/aggr.h b/eval/src/vespa/eval/eval/aggr.h
index f52c029eee5..e69b1071e61 100644
--- a/eval/src/vespa/eval/eval/aggr.h
+++ b/eval/src/vespa/eval/eval/aggr.h
@@ -53,6 +53,7 @@ struct Aggregator {
virtual void first(double value) = 0;
virtual void next(double value) = 0;
virtual double result() const = 0;
+ virtual Aggr enum_value() const = 0;
virtual ~Aggregator();
static Aggregator &create(Aggr aggr, Stash &stash);
static std::vector<Aggr> list();
@@ -60,11 +61,37 @@ struct Aggregator {
namespace aggr {
+// can we start by picking any value from the set to be reduced and
+// use the templated aggregator 'combine' function in arbitrary order
+// to end up with (approximately) the correct result?
+constexpr bool is_simple(Aggr aggr) {
+ return ((aggr == Aggr::PROD) ||
+ (aggr == Aggr::SUM) ||
+ (aggr == Aggr::MAX) ||
+ (aggr == Aggr::MIN));
+}
+
+// will a single value reduce to itself?
+constexpr bool is_ident(Aggr aggr) {
+ return ((aggr == Aggr::AVG) ||
+ (aggr == Aggr::PROD) ||
+ (aggr == Aggr::SUM) ||
+ (aggr == Aggr::MAX) ||
+ (aggr == Aggr::MEDIAN) ||
+ (aggr == Aggr::MIN));
+}
+
+// should we avoid doing clever stuff with this aggregator?
+constexpr bool is_complex(Aggr aggr) {
+ return (aggr == Aggr::MEDIAN);
+}
+
template <typename T> class Avg {
private:
T _sum;
size_t _cnt;
public:
+ using value_type = T;
constexpr Avg() : _sum{0}, _cnt{0} {}
constexpr Avg(T value) : _sum{value}, _cnt{1} {}
constexpr void sample(T value) {
@@ -76,56 +103,69 @@ public:
_cnt += rhs._cnt;
};
constexpr T result() const { return (_sum / _cnt); }
+ static constexpr Aggr enum_value() { return Aggr::AVG; }
};
template <typename T> class Count {
private:
size_t _cnt;
public:
+ using value_type = T;
constexpr Count() : _cnt{0} {}
constexpr Count(T) : _cnt{1} {}
constexpr void sample(T) { ++_cnt; }
constexpr void merge(const Count &rhs) { _cnt += rhs._cnt; }
constexpr T result() const { return _cnt; }
+ static constexpr Aggr enum_value() { return Aggr::COUNT; }
};
template <typename T> class Prod {
private:
T _prod;
public:
+ using value_type = T;
constexpr Prod() : _prod{1} {}
constexpr Prod(T value) : _prod{value} {}
constexpr void sample(T value) { _prod *= value; }
constexpr void merge(const Prod &rhs) { _prod *= rhs._prod; }
constexpr T result() const { return _prod; }
+ static constexpr Aggr enum_value() { return Aggr::PROD; }
+ static constexpr T combine(T a, T b) { return (a * b); }
};
template <typename T> class Sum {
private:
T _sum;
public:
+ using value_type = T;
constexpr Sum() : _sum{0} {}
constexpr Sum(T value) : _sum{value} {}
constexpr void sample(T value) { _sum += value; }
constexpr void merge(const Sum &rhs) { _sum += rhs._sum; }
constexpr T result() const { return _sum; }
+ static constexpr Aggr enum_value() { return Aggr::SUM; }
+ static constexpr T combine(T a, T b) { return (a + b); }
};
template <typename T> class Max {
private:
T _max;
public:
+ using value_type = T;
constexpr Max() : _max{-std::numeric_limits<T>::infinity()} {}
constexpr Max(T value) : _max{value} {}
constexpr void sample(T value) { _max = std::max(_max, value); }
constexpr void merge(const Max &rhs) { _max = std::max(_max, rhs._max); }
constexpr T result() const { return _max; }
+ static constexpr Aggr enum_value() { return Aggr::MAX; }
+ static constexpr T combine(T a, T b) { return std::max(a,b); }
};
template <typename T> class Median {
private:
std::vector<T> _seen;
public:
+ using value_type = T;
constexpr Median() : _seen() {}
constexpr Median(T value) : _seen({value}) {}
constexpr void sample(T value) { _seen.push_back(value); }
@@ -156,20 +196,24 @@ public:
}
return result;
}
+ static constexpr Aggr enum_value() { return Aggr::MEDIAN; }
};
template <typename T> class Min {
private:
T _min;
public:
+ using value_type = T;
constexpr Min() : _min{std::numeric_limits<T>::infinity()} {}
constexpr Min(T value) : _min{value} {}
constexpr void sample(T value) { _min = std::min(_min, value); }
constexpr void merge(const Min &rhs) { _min = std::min(_min, rhs._min); }
constexpr T result() const { return _min; }
+ static constexpr Aggr enum_value() { return Aggr::MIN; }
+ static constexpr T combine(T a, T b) { return std::min(a,b); }
};
-} // namespave vespalib::eval::aggr
+} // namespace vespalib::eval::aggr
struct TypifyAggr {
template <template<typename> typename TT> using Result = TypifyResultSimpleTemplate<TT>;
diff --git a/eval/src/vespa/eval/eval/cell_type.cpp b/eval/src/vespa/eval/eval/cell_type.cpp
new file mode 100644
index 00000000000..e5729c547b0
--- /dev/null
+++ b/eval/src/vespa/eval/eval/cell_type.cpp
@@ -0,0 +1,3 @@
+// Copyright Verizon Media. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+
+#include "cell_type.h"
diff --git a/eval/src/vespa/eval/eval/cell_type.h b/eval/src/vespa/eval/eval/cell_type.h
new file mode 100644
index 00000000000..0e878f26f47
--- /dev/null
+++ b/eval/src/vespa/eval/eval/cell_type.h
@@ -0,0 +1,39 @@
+// Copyright Verizon Media. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+
+#pragma once
+
+#include <vespa/vespalib/util/typify.h>
+#include <cstdlib>
+
+namespace vespalib::eval {
+
+enum class CellType : char { FLOAT, DOUBLE };
+
+// utility templates
+
+template <typename CT> inline bool check_cell_type(CellType type);
+template <> inline bool check_cell_type<double>(CellType type) { return (type == CellType::DOUBLE); }
+template <> inline bool check_cell_type<float>(CellType type) { return (type == CellType::FLOAT); }
+
+template <typename LCT, typename RCT> struct UnifyCellTypes{};
+template <> struct UnifyCellTypes<double, double> { using type = double; };
+template <> struct UnifyCellTypes<double, float> { using type = double; };
+template <> struct UnifyCellTypes<float, double> { using type = double; };
+template <> struct UnifyCellTypes<float, float> { using type = float; };
+
+template <typename CT> inline CellType get_cell_type();
+template <> inline CellType get_cell_type<double>() { return CellType::DOUBLE; }
+template <> inline CellType get_cell_type<float>() { return CellType::FLOAT; }
+
+struct TypifyCellType {
+ template <typename T> using Result = TypifyResultType<T>;
+ template <typename F> static decltype(auto) resolve(CellType value, F &&f) {
+ switch(value) {
+ case CellType::DOUBLE: return f(Result<double>());
+ case CellType::FLOAT: return f(Result<float>());
+ }
+ abort();
+ }
+};
+
+} // namespace
diff --git a/eval/src/vespa/eval/eval/value_type.h b/eval/src/vespa/eval/eval/value_type.h
index 38f4705edf2..ae69b5a3349 100644
--- a/eval/src/vespa/eval/eval/value_type.h
+++ b/eval/src/vespa/eval/eval/value_type.h
@@ -2,7 +2,7 @@
#pragma once
-#include <vespa/vespalib/util/typify.h>
+#include "cell_type.h"
#include <vespa/vespalib/stllike/string.h>
#include <vector>
@@ -16,7 +16,7 @@ namespace vespalib::eval {
class ValueType
{
public:
- enum class CellType : char { FLOAT, DOUBLE };
+ using CellType = vespalib::eval::CellType;
struct Dimension {
using size_type = uint32_t;
static constexpr size_type npos = -1;
@@ -110,31 +110,4 @@ public:
std::ostream &operator<<(std::ostream &os, const ValueType &type);
-// utility templates
-
-template <typename CT> inline bool check_cell_type(ValueType::CellType type);
-template <> inline bool check_cell_type<double>(ValueType::CellType type) { return (type == ValueType::CellType::DOUBLE); }
-template <> inline bool check_cell_type<float>(ValueType::CellType type) { return (type == ValueType::CellType::FLOAT); }
-
-template <typename LCT, typename RCT> struct UnifyCellTypes{};
-template <> struct UnifyCellTypes<double, double> { using type = double; };
-template <> struct UnifyCellTypes<double, float> { using type = double; };
-template <> struct UnifyCellTypes<float, double> { using type = double; };
-template <> struct UnifyCellTypes<float, float> { using type = float; };
-
-template <typename CT> inline ValueType::CellType get_cell_type();
-template <> inline ValueType::CellType get_cell_type<double>() { return ValueType::CellType::DOUBLE; }
-template <> inline ValueType::CellType get_cell_type<float>() { return ValueType::CellType::FLOAT; }
-
-struct TypifyCellType {
- template <typename T> using Result = TypifyResultType<T>;
- template <typename F> static decltype(auto) resolve(ValueType::CellType value, F &&f) {
- switch(value) {
- case ValueType::CellType::DOUBLE: return f(Result<double>());
- case ValueType::CellType::FLOAT: return f(Result<float>());
- }
- abort();
- }
-};
-
} // namespace
diff --git a/eval/src/vespa/eval/instruction/generic_peek.cpp b/eval/src/vespa/eval/instruction/generic_peek.cpp
index 651ce4df28a..5802a60d43a 100644
--- a/eval/src/vespa/eval/instruction/generic_peek.cpp
+++ b/eval/src/vespa/eval/instruction/generic_peek.cpp
@@ -46,12 +46,12 @@ struct DimSpec {
return std::get<size_t>(child_or_label);
}
vespalib::stringref get_label_name() const {
- auto label = std::get<TensorSpec::Label>(child_or_label);
+ auto & label = std::get<TensorSpec::Label>(child_or_label);
assert(label.is_mapped());
return label.name;
}
size_t get_label_index() const {
- auto label = std::get<TensorSpec::Label>(child_or_label);
+ auto & label = std::get<TensorSpec::Label>(child_or_label);
assert(label.is_indexed());
return label.index;
}
diff --git a/eval/src/vespa/eval/tensor/dense/dense_remove_dimension_optimizer.cpp b/eval/src/vespa/eval/tensor/dense/dense_remove_dimension_optimizer.cpp
index 0cecd588317..a48527e83f5 100644
--- a/eval/src/vespa/eval/tensor/dense/dense_remove_dimension_optimizer.cpp
+++ b/eval/src/vespa/eval/tensor/dense/dense_remove_dimension_optimizer.cpp
@@ -14,15 +14,6 @@ using namespace eval::tensor_function;
namespace {
-bool is_ident_aggr(Aggr aggr) {
- return ((aggr == Aggr::AVG) ||
- (aggr == Aggr::PROD) ||
- (aggr == Aggr::SUM) ||
- (aggr == Aggr::MAX) ||
- (aggr == Aggr::MEDIAN) ||
- (aggr == Aggr::MIN));
-}
-
bool is_trivial_dim_list(const ValueType &type, const std::vector<vespalib::string> &dim_list) {
size_t npos = ValueType::Dimension::npos;
for (const vespalib::string &dim: dim_list) {
@@ -43,7 +34,7 @@ DenseRemoveDimensionOptimizer::optimize(const eval::TensorFunction &expr, Stash
const TensorFunction &child = reduce->child();
if (expr.result_type().is_dense() &&
child.result_type().is_dense() &&
- is_ident_aggr(reduce->aggr()) &&
+ eval::aggr::is_ident(reduce->aggr()) &&
is_trivial_dim_list(child.result_type(), reduce->dimensions()))
{
assert(expr.result_type().cell_type() == child.result_type().cell_type());
diff --git a/eval/src/vespa/eval/tensor/dense/dense_single_reduce_function.cpp b/eval/src/vespa/eval/tensor/dense/dense_single_reduce_function.cpp
index 4ca15a3b5ac..5f688657645 100644
--- a/eval/src/vespa/eval/tensor/dense/dense_single_reduce_function.cpp
+++ b/eval/src/vespa/eval/tensor/dense/dense_single_reduce_function.cpp
@@ -4,6 +4,7 @@
#include "dense_tensor_view.h"
#include <vespa/vespalib/util/typify.h>
#include <vespa/eval/eval/value.h>
+#include <cassert>
namespace vespalib::tensor {
@@ -25,27 +26,16 @@ namespace {
struct Params {
const ValueType &result_type;
size_t outer_size;
- size_t dim_size;
+ size_t reduce_size;
size_t inner_size;
- Params(const ValueType &result_type_in, const ValueType &child_type, size_t dim_idx)
- : result_type(result_type_in), outer_size(1), dim_size(1), inner_size(1)
- {
- for (size_t i = 0; i < child_type.dimensions().size(); ++i) {
- if (i < dim_idx) {
- outer_size *= child_type.dimensions()[i].size;
- } else if (i == dim_idx) {
- dim_size *= child_type.dimensions()[i].size;
- } else {
- inner_size *= child_type.dimensions()[i].size;
- }
- }
- }
+ Params(const ValueType &result_type_in, size_t outer_size_in, size_t reduce_size_in, size_t inner_size_in)
+ : result_type(result_type_in), outer_size(outer_size_in), reduce_size(reduce_size_in), inner_size(inner_size_in) {}
};
template <typename CT, typename AGGR>
-CT reduce_cells(const CT *src, size_t dim_size, size_t stride) {
+CT reduce_cells(const CT *src, size_t reduce_size, size_t stride) {
AGGR aggr(*src);
- for (size_t i = 1; i < dim_size; ++i) {
+ for (size_t i = 1; i < reduce_size; ++i) {
src += stride;
aggr.sample(*src);
}
@@ -86,45 +76,160 @@ auto reduce_cells_atleast_8(const CT *src, size_t n, size_t stride) {
}
template <typename CT, typename AGGR, bool atleast_8, bool is_inner>
-void my_single_reduce_op(InterpretedFunction::State &state, uint64_t param) {
- const auto &params = unwrap_param<Params>(param);
- const CT *src = state.peek(0).cells().typify<CT>().cbegin();
- auto dst_cells = state.stash.create_uninitialized_array<CT>(params.outer_size * params.inner_size);
- CT *dst = dst_cells.begin();
- const size_t block_size = (params.dim_size * params.inner_size);
+void trace_reduce_impl(const Params &params, const CT *src, CT *dst) {
+ constexpr bool aggr_is_complex = is_complex(AGGR::enum_value());
+ const size_t block_size = (params.reduce_size * params.inner_size);
for (size_t outer = 0; outer < params.outer_size; ++outer) {
for (size_t inner = 0; inner < params.inner_size; ++inner) {
- if (atleast_8) {
+ if (atleast_8 && !aggr_is_complex) {
if (is_inner) {
- *dst++ = reduce_cells_atleast_8<CT, AGGR>(src + inner, params.dim_size);
+ *dst++ = reduce_cells_atleast_8<CT, AGGR>(src + inner, params.reduce_size);
} else {
- *dst++ = reduce_cells_atleast_8<CT, AGGR>(src + inner, params.dim_size, params.inner_size);
+ *dst++ = reduce_cells_atleast_8<CT, AGGR>(src + inner, params.reduce_size, params.inner_size);
}
} else {
- *dst++ = reduce_cells<CT, AGGR>(src + inner, params.dim_size, params.inner_size);
+ *dst++ = reduce_cells<CT, AGGR>(src + inner, params.reduce_size, params.inner_size);
}
}
src += block_size;
}
+}
+
+template <typename CT, typename AGGR>
+void fold_reduce_impl(const Params &params, const CT *src, CT *dst) {
+ for (size_t outer = 0; outer < params.outer_size; ++outer) {
+ auto saved_dst = dst;
+ for (size_t inner = 0; inner < params.inner_size; ++inner) {
+ *dst++ = *src++;
+ }
+ for (size_t dim = 1; dim < params.reduce_size; ++dim) {
+ dst = saved_dst;
+ for (size_t inner = 0; inner < params.inner_size; ++inner) {
+ *dst = AGGR::combine(*dst, *src++);
+ ++dst;
+ }
+ }
+ }
+}
+
+template <typename CT, typename AGGR, bool atleast_8, bool is_inner>
+void my_single_reduce_op(InterpretedFunction::State &state, uint64_t param) {
+ static_assert(std::is_same_v<CT,typename AGGR::value_type>);
+ constexpr bool aggr_is_simple = is_simple(AGGR::enum_value());
+ const auto &params = unwrap_param<Params>(param);
+ const CT *src = state.peek(0).cells().typify<CT>().cbegin();
+ auto dst_cells = state.stash.create_uninitialized_array<CT>(params.outer_size * params.inner_size);
+ CT *dst = dst_cells.begin();
+ if constexpr (aggr_is_simple && !is_inner) {
+ fold_reduce_impl<CT, AGGR>(params, src, dst);
+ } else {
+ trace_reduce_impl<CT,AGGR,atleast_8,is_inner>(params, src, dst);
+ }
state.pop_push(state.stash.create<DenseTensorView>(params.result_type, TypedCells(dst_cells)));
}
struct MyGetFun {
template <typename R1, typename R2, typename R3, typename R4> static auto invoke() {
- return my_single_reduce_op<R1, typename R2::template templ<R1>, R3::value, R4::value>;
+ using AggrType = typename R2::template templ<R1>;
+ return my_single_reduce_op<R1, AggrType, R3::value, R4::value>;
}
};
using MyTypify = TypifyValue<TypifyCellType,TypifyAggr,TypifyBool>;
+std::pair<std::vector<vespalib::string>,ValueType> sort_and_drop_trivial(const std::vector<vespalib::string> &list_in, const ValueType &type_in) {
+ std::vector<vespalib::string> dropped;
+ std::vector<vespalib::string> list_out;
+ for (const auto &dim_name: list_in) {
+ auto dim_idx = type_in.dimension_index(dim_name);
+ assert(dim_idx != ValueType::Dimension::npos);
+ const auto &dim = type_in.dimensions()[dim_idx];
+ assert(dim.is_indexed());
+ if (dim.is_trivial()) {
+ dropped.push_back(dim_name);
+ } else {
+ list_out.push_back(dim_name);
+ }
+ }
+ std::sort(list_out.begin(), list_out.end());
+ ValueType type_out = dropped.empty() ? type_in : type_in.reduce(dropped);
+ assert(!type_out.is_error());
+ return {list_out, type_out};
+}
+
+template <typename T> struct VectorLookupLoop {
+ const std::vector<T> &list;
+ size_t index;
+ VectorLookupLoop(const std::vector<T> &list_in) : list(list_in), index(0) {}
+ bool valid() const { return (index < list.size()); }
+ void next() { ++index; }
+ const T &get() const { return list[index]; }
+};
+
+DenseSingleReduceSpec extract_next(const eval::ValueType &type, eval::Aggr aggr,
+ std::vector<vespalib::string> &todo)
+{
+ size_t outer_size = 1;
+ size_t reduce_size = 1;
+ size_t inner_size = 1;
+ auto dims = type.nontrivial_indexed_dimensions();
+ std::vector<vespalib::string> do_now;
+ std::vector<vespalib::string> do_later;
+ auto a = VectorLookupLoop(dims);
+ auto b = VectorLookupLoop(todo);
+ while (a.valid() && b.valid() && (a.get().name < b.get())) {
+ outer_size *= a.get().size;
+ a.next();
+ }
+ while (a.valid() && b.valid() && (a.get().name == b.get())) {
+ reduce_size *= a.get().size;
+ do_now.push_back(b.get());
+ a.next();
+ b.next();
+ }
+ while (a.valid()) {
+ inner_size *= a.get().size;
+ a.next();
+ }
+ while (b.valid()) {
+ do_later.push_back(b.get());
+ b.next();
+ }
+ todo = do_later;
+ assert(!do_now.empty());
+ return {type.reduce(do_now), outer_size, reduce_size, inner_size, aggr};
+}
+
} // namespace vespalib::tensor::<unnamed>
-DenseSingleReduceFunction::DenseSingleReduceFunction(const ValueType &result_type,
- const TensorFunction &child,
- size_t dim_idx, Aggr aggr)
- : Op1(result_type, child),
- _dim_idx(dim_idx),
- _aggr(aggr)
+std::vector<DenseSingleReduceSpec>
+make_dense_single_reduce_list(const eval::ValueType &type, eval::Aggr aggr,
+ const std::vector<vespalib::string> &reduce_dims)
+{
+ auto res_type = type.reduce(reduce_dims);
+ if (reduce_dims.empty() || !type.is_dense() || !res_type.is_dense()) {
+ return {};
+ }
+ std::vector<DenseSingleReduceSpec> list;
+ auto [todo, curr_type] = sort_and_drop_trivial(reduce_dims, type);
+ while (!todo.empty()) {
+ list.push_back(extract_next(curr_type, aggr, todo));
+ curr_type = list.back().result_type;
+ }
+ assert(curr_type == res_type);
+ if ((list.size() > 1) && !eval::aggr::is_simple(aggr)) {
+ return {};
+ }
+ return list;
+}
+
+DenseSingleReduceFunction::DenseSingleReduceFunction(const DenseSingleReduceSpec &spec,
+ const TensorFunction &child)
+ : Op1(spec.result_type, child),
+ _outer_size(spec.outer_size),
+ _reduce_size(spec.reduce_size),
+ _inner_size(spec.inner_size),
+ _aggr(spec.aggr)
{
}
@@ -133,24 +238,25 @@ DenseSingleReduceFunction::~DenseSingleReduceFunction() = default;
InterpretedFunction::Instruction
DenseSingleReduceFunction::compile_self(eval::EngineOrFactory, Stash &stash) const
{
- auto &params = stash.create<Params>(result_type(), child().result_type(), _dim_idx);
auto op = typify_invoke<4,MyTypify,MyGetFun>(result_type().cell_type(), _aggr,
- (params.dim_size >= 8), (params.inner_size == 1));
+ (_reduce_size >= 8), (_inner_size == 1));
+ auto &params = stash.create<Params>(result_type(), _outer_size, _reduce_size, _inner_size);
return InterpretedFunction::Instruction(op, wrap_param<Params>(params));
}
const TensorFunction &
DenseSingleReduceFunction::optimize(const TensorFunction &expr, Stash &stash)
{
- auto reduce = as<Reduce>(expr);
- if (reduce && (reduce->dimensions().size() == 1) &&
- reduce->child().result_type().is_dense() &&
- expr.result_type().is_dense())
- {
- size_t dim_idx = reduce->child().result_type().dimension_index(reduce->dimensions()[0]);
- assert(dim_idx != ValueType::Dimension::npos);
- assert(expr.result_type().cell_type() == reduce->child().result_type().cell_type());
- return stash.create<DenseSingleReduceFunction>(expr.result_type(), reduce->child(), dim_idx, reduce->aggr());
+ if (auto reduce = as<Reduce>(expr)) {
+ const auto &child = reduce->child();
+ auto spec_list = make_dense_single_reduce_list(child.result_type(), reduce->aggr(), reduce->dimensions());
+ if (!spec_list.empty()) {
+ const auto *prev = &child;
+ for (const auto &spec: spec_list) {
+ prev = &stash.create<DenseSingleReduceFunction>(spec, *prev);
+ }
+ return *prev;
+ }
}
return expr;
}
diff --git a/eval/src/vespa/eval/tensor/dense/dense_single_reduce_function.h b/eval/src/vespa/eval/tensor/dense/dense_single_reduce_function.h
index 7f9313df600..f2db3155290 100644
--- a/eval/src/vespa/eval/tensor/dense/dense_single_reduce_function.h
+++ b/eval/src/vespa/eval/tensor/dense/dense_single_reduce_function.h
@@ -6,22 +6,46 @@
namespace vespalib::tensor {
+struct DenseSingleReduceSpec {
+ eval::ValueType result_type;
+ size_t outer_size;
+ size_t reduce_size;
+ size_t inner_size;
+ eval::Aggr aggr;
+};
+
+/**
+ * Decompose the specified reduce operation into a sequence of single
+ * dense reduce operations. Returns an empty list if decomposition
+ * fails.
+ **/
+std::vector<DenseSingleReduceSpec>
+make_dense_single_reduce_list(const eval::ValueType &type, eval::Aggr aggr,
+ const std::vector<vespalib::string> &reduce_dims);
+
/**
- * Tensor function reducing a single dimension of a dense
- * tensor where the result is also a dense tensor.
+ * Tensor function reducing a single dimension of a dense tensor where
+ * the result is also a dense tensor. The optimize function may create
+ * multiple tensor functions to compose a multi-stage reduce
+ * operation. Adjacent reduced dimensions will be handled is if they
+ * were a single dimension. Trivial dimensions will be trivially
+ * reduced along with any other dimension.
**/
class DenseSingleReduceFunction : public eval::tensor_function::Op1
{
private:
- size_t _dim_idx;
+ size_t _outer_size;
+ size_t _reduce_size;
+ size_t _inner_size;
eval::Aggr _aggr;
public:
- DenseSingleReduceFunction(const eval::ValueType &result_type,
- const eval::TensorFunction &child,
- size_t dim_idx, eval::Aggr aggr);
+ DenseSingleReduceFunction(const DenseSingleReduceSpec &spec,
+ const eval::TensorFunction &child);
~DenseSingleReduceFunction() override;
- size_t dim_idx() const { return _dim_idx; }
+ size_t outer_size() const { return _outer_size; }
+ size_t reduce_size() const { return _reduce_size; }
+ size_t inner_size() const { return _inner_size; }
eval::Aggr aggr() const { return _aggr; }
bool result_is_mutable() const override { return true; }
eval::InterpretedFunction::Instruction compile_self(eval::EngineOrFactory engine, Stash &stash) const override;
diff --git a/jdisc_http_service/abi-spec.json b/jdisc_http_service/abi-spec.json
index 8bf7f30964a..8b48631d4aa 100644
--- a/jdisc_http_service/abi-spec.json
+++ b/jdisc_http_service/abi-spec.json
@@ -742,6 +742,7 @@
"public com.yahoo.jdisc.http.ServerConfig$Builder filter(java.util.List)",
"public com.yahoo.jdisc.http.ServerConfig$Builder defaultFilters(com.yahoo.jdisc.http.ServerConfig$DefaultFilters$Builder)",
"public com.yahoo.jdisc.http.ServerConfig$Builder defaultFilters(java.util.List)",
+ "public com.yahoo.jdisc.http.ServerConfig$Builder strictFiltering(boolean)",
"public com.yahoo.jdisc.http.ServerConfig$Builder maxWorkerThreads(int)",
"public com.yahoo.jdisc.http.ServerConfig$Builder minWorkerThreads(int)",
"public com.yahoo.jdisc.http.ServerConfig$Builder stopTimeout(double)",
@@ -930,6 +931,7 @@
"public com.yahoo.jdisc.http.ServerConfig$Filter filter(int)",
"public java.util.List defaultFilters()",
"public com.yahoo.jdisc.http.ServerConfig$DefaultFilters defaultFilters(int)",
+ "public boolean strictFiltering()",
"public int maxWorkerThreads()",
"public int minWorkerThreads()",
"public double stopTimeout()",
diff --git a/jdisc_http_service/src/main/java/com/yahoo/jdisc/http/server/jetty/FilterResolver.java b/jdisc_http_service/src/main/java/com/yahoo/jdisc/http/server/jetty/FilterResolver.java
index b80dd216b04..1e2686aa184 100644
--- a/jdisc_http_service/src/main/java/com/yahoo/jdisc/http/server/jetty/FilterResolver.java
+++ b/jdisc_http_service/src/main/java/com/yahoo/jdisc/http/server/jetty/FilterResolver.java
@@ -2,6 +2,12 @@
package com.yahoo.jdisc.http.server.jetty;
import com.yahoo.jdisc.Metric;
+import com.yahoo.jdisc.NoopSharedResource;
+import com.yahoo.jdisc.Response;
+import com.yahoo.jdisc.handler.FastContentWriter;
+import com.yahoo.jdisc.handler.ResponseDispatch;
+import com.yahoo.jdisc.handler.ResponseHandler;
+import com.yahoo.jdisc.http.HttpRequest;
import com.yahoo.jdisc.http.filter.RequestFilter;
import com.yahoo.jdisc.http.filter.ResponseFilter;
import com.yahoo.jdisc.http.servlet.ServletRequest;
@@ -22,10 +28,12 @@ class FilterResolver {
private final FilterBindings bindings;
private final Metric metric;
+ private final boolean strictFiltering;
- FilterResolver(FilterBindings bindings, Metric metric) {
+ FilterResolver(FilterBindings bindings, Metric metric, boolean strictFiltering) {
this.bindings = bindings;
this.metric = metric;
+ this.strictFiltering = strictFiltering;
}
Optional<RequestFilter> resolveRequestFilter(HttpServletRequest servletRequest, URI jdiscUri) {
@@ -33,8 +41,13 @@ class FilterResolver {
if (maybeFilterId.isPresent()) {
metric.add(MetricDefinitions.FILTERING_REQUEST_HANDLED, 1L, createMetricContext(servletRequest, maybeFilterId.get()));
servletRequest.setAttribute(ServletRequest.JDISC_REQUEST_CHAIN, maybeFilterId.get());
- } else {
+ } else if (!strictFiltering) {
metric.add(MetricDefinitions.FILTERING_REQUEST_UNHANDLED, 1L, createMetricContext(servletRequest, null));
+ } else {
+ String syntheticFilterId = RejectingRequestFilter.SYNTHETIC_FILTER_CHAIN_ID;
+ metric.add(MetricDefinitions.FILTERING_REQUEST_HANDLED, 1L, createMetricContext(servletRequest, syntheticFilterId));
+ servletRequest.setAttribute(ServletRequest.JDISC_REQUEST_CHAIN, syntheticFilterId);
+ return Optional.of(RejectingRequestFilter.INSTANCE);
}
return maybeFilterId.map(bindings::getRequestFilter);
}
@@ -56,4 +69,20 @@ class FilterResolver {
: Map.of();
return JDiscHttpServlet.getConnector(request).createRequestMetricContext(request, extraDimensions);
}
+
+ private static class RejectingRequestFilter extends NoopSharedResource implements RequestFilter {
+
+ private static final RejectingRequestFilter INSTANCE = new RejectingRequestFilter();
+ private static final String SYNTHETIC_FILTER_CHAIN_ID = "strict-reject";
+
+ @Override
+ public void filter(HttpRequest request, ResponseHandler handler) {
+ Response response = new Response(Response.Status.FORBIDDEN);
+ response.headers().add("Content-Type", "text/plain");
+ try (FastContentWriter writer = ResponseDispatch.newInstance(response).connectFastWriter(handler)) {
+ writer.write("Request did not match any request filter chain");
+ }
+ }
+ }
+
}
diff --git a/jdisc_http_service/src/main/java/com/yahoo/jdisc/http/server/jetty/JDiscContext.java b/jdisc_http_service/src/main/java/com/yahoo/jdisc/http/server/jetty/JDiscContext.java
index 66471587bd5..b37a7352dc6 100644
--- a/jdisc_http_service/src/main/java/com/yahoo/jdisc/http/server/jetty/JDiscContext.java
+++ b/jdisc_http_service/src/main/java/com/yahoo/jdisc/http/server/jetty/JDiscContext.java
@@ -20,7 +20,7 @@ public class JDiscContext {
Metric metric,
ServerConfig serverConfig) {
- this.filterResolver = new FilterResolver(filterBindings, metric);
+ this.filterResolver = new FilterResolver(filterBindings, metric, serverConfig.strictFiltering());
this.container = container;
this.janitor = janitor;
this.metric = metric;
diff --git a/jdisc_http_service/src/main/resources/configdefinitions/jdisc.http.jdisc.http.server.def b/jdisc_http_service/src/main/resources/configdefinitions/jdisc.http.jdisc.http.server.def
index f33dc35ea0b..f75a4aaa441 100644
--- a/jdisc_http_service/src/main/resources/configdefinitions/jdisc.http.jdisc.http.server.def
+++ b/jdisc_http_service/src/main/resources/configdefinitions/jdisc.http.jdisc.http.server.def
@@ -33,6 +33,9 @@ defaultFilters[].filterId string
# The local port which the default filter should be applied to
defaultFilters[].localPort int
+# Reject all requests not handled by a request filter (chain)
+strictFiltering bool default = false
+
# Max number of threads in underlying Jetty pool
maxWorkerThreads int default = 200
diff --git a/jdisc_http_service/src/test/java/com/yahoo/jdisc/http/server/jetty/FilterTestCase.java b/jdisc_http_service/src/test/java/com/yahoo/jdisc/http/server/jetty/FilterTestCase.java
index fd929b3e037..9c5c4027ae3 100644
--- a/jdisc_http_service/src/test/java/com/yahoo/jdisc/http/server/jetty/FilterTestCase.java
+++ b/jdisc_http_service/src/test/java/com/yahoo/jdisc/http/server/jetty/FilterTestCase.java
@@ -35,6 +35,7 @@ import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
+import static org.hamcrest.CoreMatchers.containsString;
import static org.hamcrest.CoreMatchers.is;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.mockito.ArgumentMatchers.any;
@@ -495,7 +496,7 @@ public class FilterTestCase {
.build();
MetricConsumerMock metricConsumerMock = new MetricConsumerMock();
MyRequestHandler requestHandler = new MyRequestHandler();
- TestDriver testDriver = newDriver(requestHandler, filterBindings, metricConsumerMock);
+ TestDriver testDriver = newDriver(requestHandler, filterBindings, metricConsumerMock, false);
testDriver.client().get("/status.html");
assertThat(requestHandler.awaitInvocation(), is(true));
@@ -510,25 +511,47 @@ public class FilterTestCase {
assertThat(testDriver.close(), is(true));
}
+ @Test
+ public void requireThatStrictFilteringRejectsRequestsNotMatchingFilterChains() throws IOException {
+ RequestFilter filter = mock(RequestFilter.class);
+ FilterBindings filterBindings = new FilterBindings.Builder()
+ .addRequestFilter("my-request-filter", filter)
+ .addRequestFilterBinding("my-request-filter", "http://*/filtered/*")
+ .build();
+ MyRequestHandler requestHandler = new MyRequestHandler();
+ TestDriver testDriver = newDriver(requestHandler, filterBindings, new MetricConsumerMock(), true);
+
+ testDriver.client().get("/unfiltered/")
+ .expectStatusCode(is(Response.Status.FORBIDDEN))
+ .expectContent(containsString("Request did not match any request filter chain"));
+ verify(filter, never()).filter(any(), any());
+ assertThat(testDriver.close(), is(true));
+ }
+
private static TestDriver newDriver(MyRequestHandler requestHandler, FilterBindings filterBindings) {
- return newDriver(requestHandler, filterBindings, new MetricConsumerMock());
+ return newDriver(requestHandler, filterBindings, new MetricConsumerMock(), false);
}
- private static TestDriver newDriver(MyRequestHandler requestHandler, FilterBindings filterBindings, MetricConsumerMock metricConsumer) {
+ private static TestDriver newDriver(
+ MyRequestHandler requestHandler,
+ FilterBindings filterBindings,
+ MetricConsumerMock metricConsumer,
+ boolean strictFiltering) {
return TestDriver.newInstance(
JettyHttpServer.class,
requestHandler,
- newFilterModule(filterBindings, metricConsumer));
+ newFilterModule(filterBindings, metricConsumer, strictFiltering));
}
- private static com.google.inject.Module newFilterModule(FilterBindings filterBindings, MetricConsumerMock metricConsumer) {
+ private static com.google.inject.Module newFilterModule(
+ FilterBindings filterBindings, MetricConsumerMock metricConsumer, boolean strictFiltering) {
return Modules.combine(
new AbstractModule() {
@Override
protected void configure() {
bind(FilterBindings.class).toInstance(filterBindings);
- bind(ServerConfig.class).toInstance(new ServerConfig(new ServerConfig.Builder()));
+ bind(ServerConfig.class).toInstance(new ServerConfig(new ServerConfig.Builder().strictFiltering(strictFiltering)));
bind(ConnectorConfig.class).toInstance(new ConnectorConfig(new ConnectorConfig.Builder()));
bind(ServletPathsConfig.class).toInstance(new ServletPathsConfig(new ServletPathsConfig.Builder()));
}
diff --git a/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/NodeRepository.java b/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/NodeRepository.java
index 663d1d19995..05bdfd25b76 100644
--- a/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/NodeRepository.java
+++ b/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/NodeRepository.java
@@ -503,12 +503,6 @@ public class NodeRepository extends AbstractComponent {
}
}
- /** Deactivate nodes owned by application guarded by given lock */
- public void deactivate(ApplicationTransaction transaction) {
- deactivate(db.readNodes(transaction.application(), State.reserved, State.active), transaction);
- applications.remove(transaction);
- }
-
/**
* Deactivates these nodes in a transaction and returns the nodes in the new state which will hold if the
* transaction commits.
@@ -517,6 +511,19 @@ public class NodeRepository extends AbstractComponent {
return db.writeTo(State.inactive, nodes, Agent.application, Optional.empty(), transaction.nested());
}
+ /** Removes this application: Active nodes are deactivated while all non-active nodes are set dirty. */
+ public void remove(ApplicationTransaction transaction) {
+ NodeList applicationNodes = list(transaction.application());
+ NodeList activeNodes = applicationNodes.state(State.active);
+ deactivate(activeNodes.asList(), transaction);
+ db.writeTo(State.dirty,
+ applicationNodes.except(activeNodes.asSet()).asList(),
+ Agent.system,
+ Optional.of("Application is removed"),
+ transaction.nested());
+ applications.remove(transaction);
+ }
+
/** Move nodes to the dirty state */
public List<Node> setDirty(List<Node> nodes, Agent agent, String reason) {
return performOn(NodeListFilter.from(nodes), (node, lock) -> setDirty(node, agent, reason));
@@ -532,6 +539,7 @@ public class NodeRepository extends AbstractComponent {
return db.writeTo(State.dirty, node, agent, Optional.of(reason));
}
+
public List<Node> dirtyRecursively(String hostname, Agent agent, String reason) {
Node nodeToDirty = getNode(hostname).orElseThrow(() ->
new IllegalArgumentException("Could not deallocate " + hostname + ": Node not found"));
diff --git a/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/autoscale/Autoscaler.java b/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/autoscale/Autoscaler.java
index b7729577bda..1a8c4c8a6c2 100644
--- a/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/autoscale/Autoscaler.java
+++ b/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/autoscale/Autoscaler.java
@@ -20,7 +20,7 @@ import java.util.logging.Logger;
*/
public class Autoscaler {
- private final Logger log = Logger.getLogger(this.getClass().getName());
+ private static final Logger log = Logger.getLogger(Autoscaler.class.getName());
/** What cost difference factor is worth a reallocation? */
private static final double costDifferenceWorthReallocation = 0.1;
@@ -74,9 +74,8 @@ public class Autoscaler {
Optional<Double> cpuLoad = clusterTimeseries.averageLoad(Resource.cpu, cluster);
Optional<Double> memoryLoad = clusterTimeseries.averageLoad(Resource.memory, cluster);
Optional<Double> diskLoad = clusterTimeseries.averageLoad(Resource.disk, cluster);
- if (cpuLoad.isEmpty() || memoryLoad.isEmpty() || diskLoad.isEmpty()) {
- return Advice.none();
- }
+ if (cpuLoad.isEmpty() || memoryLoad.isEmpty() || diskLoad.isEmpty()) return Advice.none();
+
var target = ResourceTarget.idealLoad(cpuLoad.get(), memoryLoad.get(), diskLoad.get(), currentAllocation);
Optional<AllocatableClusterResources> bestAllocation =
diff --git a/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/maintenance/AutoscalingMaintainer.java b/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/maintenance/AutoscalingMaintainer.java
index 0b8cb4f635b..b53f56e4743 100644
--- a/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/maintenance/AutoscalingMaintainer.java
+++ b/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/maintenance/AutoscalingMaintainer.java
@@ -57,7 +57,7 @@ public class AutoscalingMaintainer extends NodeRepositoryMaintainer {
private void autoscale(ApplicationId application, List<Node> applicationNodes) {
try (MaintenanceDeployment deployment = new MaintenanceDeployment(application, deployer, metric, nodeRepository())) {
- if ( ! deployment.isValid()) return; // Another config server will consider this application
+ if ( ! deployment.isValid()) return;
nodesByCluster(applicationNodes).forEach((clusterId, clusterNodes) -> autoscale(application, clusterId, clusterNodes, deployment));
}
}
diff --git a/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/provisioning/NodeRepositoryProvisioner.java b/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/provisioning/NodeRepositoryProvisioner.java
index edf151ff2d8..ede6f4ef250 100644
--- a/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/provisioning/NodeRepositoryProvisioner.java
+++ b/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/provisioning/NodeRepositoryProvisioner.java
@@ -29,6 +29,7 @@ import com.yahoo.vespa.hosted.provision.autoscale.AllocatableClusterResources;
import com.yahoo.vespa.hosted.provision.autoscale.AllocationOptimizer;
import com.yahoo.vespa.hosted.provision.autoscale.Limits;
import com.yahoo.vespa.hosted.provision.autoscale.ResourceTarget;
+import com.yahoo.vespa.hosted.provision.node.Agent;
import com.yahoo.vespa.hosted.provision.node.Allocation;
import com.yahoo.vespa.hosted.provision.node.filter.ApplicationFilter;
import com.yahoo.vespa.hosted.provision.node.filter.NodeHostFilter;
@@ -132,7 +133,7 @@ public class NodeRepositoryProvisioner implements Provisioner {
@Override
public void remove(ApplicationTransaction transaction) {
- nodeRepository.deactivate(transaction);
+ nodeRepository.remove(transaction);
loadBalancerProvisioner.ifPresent(lbProvisioner -> lbProvisioner.deactivate(transaction));
}
diff --git a/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/provisioning/ProvisionedHost.java b/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/provisioning/ProvisionedHost.java
index 61cedbb9373..02621c79019 100644
--- a/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/provisioning/ProvisionedHost.java
+++ b/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/provisioning/ProvisionedHost.java
@@ -7,10 +7,12 @@ import com.yahoo.config.provision.Flavor;
import com.yahoo.config.provision.NodeResources;
import com.yahoo.config.provision.NodeType;
import com.yahoo.vespa.hosted.provision.Node;
+import com.yahoo.vespa.hosted.provision.node.Address;
import com.yahoo.vespa.hosted.provision.node.IP;
import com.yahoo.vespa.hosted.provision.node.OsVersion;
import com.yahoo.vespa.hosted.provision.node.Status;
+import java.util.List;
import java.util.Objects;
import java.util.Optional;
import java.util.Set;
@@ -26,25 +28,33 @@ public class ProvisionedHost {
private final String hostHostname;
private final Flavor hostFlavor;
private final Optional<ApplicationId> exclusiveTo;
- private final String nodeHostname;
+ private final List<Address> nodeAddresses;
private final NodeResources nodeResources;
private final Version osVersion;
public ProvisionedHost(String id, String hostHostname, Flavor hostFlavor, Optional<ApplicationId> exclusiveTo,
- String nodeHostname, NodeResources nodeResources, Version osVersion) {
+ List<Address> nodeAddresses, NodeResources nodeResources, Version osVersion) {
this.id = Objects.requireNonNull(id, "Host id must be set");
this.hostHostname = Objects.requireNonNull(hostHostname, "Host hostname must be set");
this.hostFlavor = Objects.requireNonNull(hostFlavor, "Host flavor must be set");
this.exclusiveTo = Objects.requireNonNull(exclusiveTo, "exclusiveTo must be set");
- this.nodeHostname = Objects.requireNonNull(nodeHostname, "Node hostname must be set");
+ this.nodeAddresses = validateNodeAddresses(nodeAddresses);
this.nodeResources = Objects.requireNonNull(nodeResources, "Node resources must be set");
this.osVersion = Objects.requireNonNull(osVersion, "OS version must be set");
}
+ private static List<Address> validateNodeAddresses(List<Address> nodeAddresses) {
+ Objects.requireNonNull(nodeAddresses, "Node addresses must be set");
+ if (nodeAddresses.isEmpty()) {
+ throw new IllegalArgumentException("There must be at least one node address");
+ }
+ return nodeAddresses;
+ }
+
/** Generate {@link Node} instance representing the provisioned physical host */
public Node generateHost() {
Node.Builder builder = Node
- .create(id, IP.Config.EMPTY, hostHostname, hostFlavor, NodeType.host)
+ .create(id, IP.Config.of(Set.of(), Set.of(), nodeAddresses), hostHostname, hostFlavor, NodeType.host)
.status(Status.initial().withOsVersion(OsVersion.EMPTY.withCurrent(Optional.of(osVersion))));
exclusiveTo.ifPresent(builder::exclusiveTo);
return builder.build();
@@ -52,7 +62,7 @@ public class ProvisionedHost {
/** Generate {@link Node} instance representing the node running on this physical host */
public Node generateNode() {
- return Node.createDockerNode(Set.of(), nodeHostname, hostHostname, nodeResources, NodeType.tenant).build();
+ return Node.createDockerNode(Set.of(), nodeHostname(), hostHostname, nodeResources, NodeType.tenant).build();
}
public String getId() {
@@ -68,7 +78,11 @@ public class ProvisionedHost {
}
public String nodeHostname() {
- return nodeHostname;
+ return nodeAddresses.get(0).hostname();
+ }
+
+ public List<Address> nodeAddresses() {
+ return nodeAddresses;
}
public NodeResources nodeResources() { return nodeResources; }
@@ -81,14 +95,14 @@ public class ProvisionedHost {
return id.equals(that.id) &&
hostHostname.equals(that.hostHostname) &&
hostFlavor.equals(that.hostFlavor) &&
- nodeHostname.equals(that.nodeHostname) &&
+ nodeAddresses.equals(that.nodeAddresses) &&
nodeResources.equals(that.nodeResources) &&
osVersion.equals(that.osVersion);
}
@Override
public int hashCode() {
- return Objects.hash(id, hostHostname, hostFlavor, nodeHostname, nodeResources, osVersion);
+ return Objects.hash(id, hostHostname, hostFlavor, nodeAddresses, nodeResources, osVersion);
}
@Override
@@ -97,7 +111,7 @@ public class ProvisionedHost {
"id='" + id + '\'' +
", hostHostname='" + hostHostname + '\'' +
", hostFlavor=" + hostFlavor +
- ", nodeHostname='" + nodeHostname + '\'' +
+ ", nodeAddresses='" + nodeAddresses + '\'' +
", nodeResources=" + nodeResources +
", osVersion=" + osVersion +
'}';
diff --git a/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/restapi/NodesV2ApiHandler.java b/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/restapi/NodesV2ApiHandler.java
index 304cebb3c01..c43629aeb09 100644
--- a/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/restapi/NodesV2ApiHandler.java
+++ b/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/restapi/NodesV2ApiHandler.java
@@ -29,6 +29,7 @@ import com.yahoo.vespa.hosted.provision.NoSuchNodeException;
import com.yahoo.vespa.hosted.provision.Node;
import com.yahoo.vespa.hosted.provision.NodeRepository;
import com.yahoo.vespa.hosted.provision.applications.Application;
+import com.yahoo.vespa.hosted.provision.node.Address;
import com.yahoo.vespa.hosted.provision.node.Agent;
import com.yahoo.vespa.hosted.provision.node.IP;
import com.yahoo.vespa.hosted.provision.node.filter.ApplicationFilter;
@@ -256,8 +257,12 @@ public class NodesV2ApiHandler extends LoggingRequestHandler {
Set<String> ipAddressPool = new HashSet<>();
inspector.field("additionalIpAddresses").traverse((ArrayTraverser) (i, item) -> ipAddressPool.add(item.asString()));
+ List<Address> addressPool = new ArrayList<>();
+ inspector.field("additionalHostnames").traverse((ArrayTraverser) (i, item) ->
+ addressPool.add(new Address(item.asString())));
+
Node.Builder builder = Node.create(inspector.field("openStackId").asString(),
- IP.Config.of(ipAddresses, ipAddressPool, List.of()),
+ IP.Config.of(ipAddresses, ipAddressPool, addressPool),
inspector.field("hostname").asString(),
flavorFromSlime(inspector),
nodeTypeFromSlime(inspector.field("type")));
diff --git a/node-repository/src/test/java/com/yahoo/vespa/hosted/provision/autoscale/AutoscalingTester.java b/node-repository/src/test/java/com/yahoo/vespa/hosted/provision/autoscale/AutoscalingTester.java
index 4d8b6d13a86..3faa4c244ee 100644
--- a/node-repository/src/test/java/com/yahoo/vespa/hosted/provision/autoscale/AutoscalingTester.java
+++ b/node-repository/src/test/java/com/yahoo/vespa/hosted/provision/autoscale/AutoscalingTester.java
@@ -20,6 +20,7 @@ import com.yahoo.vespa.hosted.provision.Node;
import com.yahoo.vespa.hosted.provision.NodeRepository;
import com.yahoo.vespa.hosted.provision.Nodelike;
import com.yahoo.vespa.hosted.provision.applications.Application;
+import com.yahoo.vespa.hosted.provision.node.Address;
import com.yahoo.vespa.hosted.provision.node.Agent;
import com.yahoo.vespa.hosted.provision.node.IP;
import com.yahoo.vespa.hosted.provision.provisioning.FatalProvisioningException;
@@ -294,7 +295,7 @@ class AutoscalingTester {
"hostname" + index,
hostFlavor,
Optional.empty(),
- "nodename" + index,
+ List.of(new Address("nodename" + index)),
resources,
osVersion));
}
diff --git a/node-repository/src/test/java/com/yahoo/vespa/hosted/provision/maintenance/DynamicProvisioningMaintainerTest.java b/node-repository/src/test/java/com/yahoo/vespa/hosted/provision/maintenance/DynamicProvisioningMaintainerTest.java
index 478376bc0cd..22eec482b02 100644
--- a/node-repository/src/test/java/com/yahoo/vespa/hosted/provision/maintenance/DynamicProvisioningMaintainerTest.java
+++ b/node-repository/src/test/java/com/yahoo/vespa/hosted/provision/maintenance/DynamicProvisioningMaintainerTest.java
@@ -20,6 +20,7 @@ import com.yahoo.vespa.flags.InMemoryFlagSource;
import com.yahoo.vespa.flags.custom.HostCapacity;
import com.yahoo.vespa.hosted.provision.Node;
import com.yahoo.vespa.hosted.provision.NodeRepository;
+import com.yahoo.vespa.hosted.provision.node.Address;
import com.yahoo.vespa.hosted.provision.node.Agent;
import com.yahoo.vespa.hosted.provision.node.Allocation;
import com.yahoo.vespa.hosted.provision.node.Generation;
@@ -338,7 +339,7 @@ public class DynamicProvisioningMaintainerTest {
"hostname" + index,
hostFlavor,
Optional.empty(),
- "nodename" + index,
+ List.of(new Address("nodename" + index)),
resources,
osVersion));
}
diff --git a/node-repository/src/test/java/com/yahoo/vespa/hosted/provision/provisioning/DynamicDockerProvisionTest.java b/node-repository/src/test/java/com/yahoo/vespa/hosted/provision/provisioning/DynamicDockerProvisionTest.java
index 4917a59879f..919d02c435c 100644
--- a/node-repository/src/test/java/com/yahoo/vespa/hosted/provision/provisioning/DynamicDockerProvisionTest.java
+++ b/node-repository/src/test/java/com/yahoo/vespa/hosted/provision/provisioning/DynamicDockerProvisionTest.java
@@ -20,6 +20,7 @@ import com.yahoo.config.provision.SystemName;
import com.yahoo.config.provision.Zone;
import com.yahoo.vespa.hosted.provision.Node;
import com.yahoo.vespa.hosted.provision.NodeList;
+import com.yahoo.vespa.hosted.provision.node.Address;
import com.yahoo.vespa.hosted.provision.node.Agent;
import com.yahoo.vespa.hosted.provision.node.IP;
import com.yahoo.vespa.hosted.provision.provisioning.HostProvisioner.HostSharing;
@@ -471,7 +472,7 @@ public class DynamicDockerProvisionTest {
throw new OutOfCapacityException("No host flavor matches " + resources);
return provisionIndexes.stream()
.map(i -> new ProvisionedHost("id-" + i, "host-" + i, hostFlavor.get(), Optional.empty(),
- "host-" + i + "-1", resources, osVersion))
+ List.of(new Address("host-" + i + "-1")), resources, osVersion))
.collect(Collectors.toList());
}
diff --git a/node-repository/src/test/java/com/yahoo/vespa/hosted/provision/provisioning/ProvisioningTest.java b/node-repository/src/test/java/com/yahoo/vespa/hosted/provision/provisioning/ProvisioningTest.java
index 2fe39780cf5..cbac5a39e09 100644
--- a/node-repository/src/test/java/com/yahoo/vespa/hosted/provision/provisioning/ProvisioningTest.java
+++ b/node-repository/src/test/java/com/yahoo/vespa/hosted/provision/provisioning/ProvisioningTest.java
@@ -103,9 +103,13 @@ public class ProvisioningTest {
assertEquals(5, tester.getNodes(application1, Node.State.inactive).size());
// delete app
+ NodeList previouslyActive = tester.getNodes(application1, Node.State.active);
+ NodeList previouslyInactive = tester.getNodes(application1, Node.State.inactive);
tester.remove(application1);
- assertEquals(tester.toHostNames(state1.allHosts), tester.toHostNames(tester.nodeRepository().getNodes(application1, Node.State.inactive)));
+ assertEquals(tester.toHostNames(previouslyActive.asList()), tester.toHostNames(tester.nodeRepository().getNodes(application1, Node.State.inactive)));
+ assertEquals(tester.toHostNames(previouslyInactive.asList()), tester.toHostNames(tester.nodeRepository().getNodes(Node.State.dirty)));
assertEquals(0, tester.getNodes(application1, Node.State.active).size());
+ assertTrue(tester.nodeRepository().applications().get(application1).isEmpty());
// other application is unaffected
assertEquals(state1App2.hostNames(), tester.toHostNames(tester.nodeRepository().getNodes(application2, Node.State.active)));
@@ -121,6 +125,7 @@ public class ProvisioningTest {
tester.activate(application2, state2App2.allHosts);
// deploy first app again
+ tester.nodeRepository().setReady(tester.nodeRepository().getNodes(Node.State.dirty), Agent.system, "recycled");
SystemState state7 = prepare(application1, 2, 2, 3, 3, defaultResources, tester);
state7.assertEquals(state1);
tester.activate(application1, state7.allHosts);
diff --git a/node-repository/src/test/java/com/yahoo/vespa/hosted/provision/provisioning/ProvisioningTester.java b/node-repository/src/test/java/com/yahoo/vespa/hosted/provision/provisioning/ProvisioningTester.java
index d39ea3786f1..f012f0a428f 100644
--- a/node-repository/src/test/java/com/yahoo/vespa/hosted/provision/provisioning/ProvisioningTester.java
+++ b/node-repository/src/test/java/com/yahoo/vespa/hosted/provision/provisioning/ProvisioningTester.java
@@ -242,8 +242,8 @@ public class ProvisioningTester {
public void deactivate(ApplicationId applicationId) {
try (var lock = nodeRepository.lock(applicationId)) {
NestedTransaction deactivateTransaction = new NestedTransaction();
- nodeRepository.deactivate(new ApplicationTransaction(new ProvisionLock(applicationId, lock),
- deactivateTransaction));
+ nodeRepository.remove(new ApplicationTransaction(new ProvisionLock(applicationId, lock),
+ deactivateTransaction));
deactivateTransaction.commit();
}
}
diff --git a/node-repository/src/test/java/com/yahoo/vespa/hosted/provision/restapi/NodesV2ApiTest.java b/node-repository/src/test/java/com/yahoo/vespa/hosted/provision/restapi/NodesV2ApiTest.java
index a98d383e219..86427fe30ae 100644
--- a/node-repository/src/test/java/com/yahoo/vespa/hosted/provision/restapi/NodesV2ApiTest.java
+++ b/node-repository/src/test/java/com/yahoo/vespa/hosted/provision/restapi/NodesV2ApiTest.java
@@ -91,8 +91,9 @@ public class NodesV2ApiTest {
// POST new nodes
assertResponse(new Request("http://localhost:8080/nodes/v2/node",
("[" + asNodeJson("host8.yahoo.com", "default", "127.0.8.1") + "," + // test with only 1 ip address
- asHostJson("host9.yahoo.com", "large-variant", "127.0.9.1", "::9:1") + "," +
- asNodeJson("parent2.yahoo.com", NodeType.host, "large-variant", Optional.of(TenantName.from("myTenant")), Optional.of(ApplicationId.from("tenant1", "app1", "instance1")), Optional.empty(), "127.0.127.1", "::127:1") + "," +
+ asHostJson("host9.yahoo.com", "large-variant", List.of("node9-1.yahoo.com"), "127.0.9.1", "::9:1") + "," +
+ asNodeJson("parent2.yahoo.com", NodeType.host, "large-variant", Optional.of(TenantName.from("myTenant")),
+ Optional.of(ApplicationId.from("tenant1", "app1", "instance1")), Optional.empty(), List.of(), "127.0.127.1", "::127:1") + "," +
asDockerNodeJson("host11.yahoo.com", "parent.host.yahoo.com", "::11") + "]").
getBytes(StandardCharsets.UTF_8),
Request.Method.POST),
@@ -322,7 +323,7 @@ public class NodesV2ApiTest {
// Attempt to POST host node with already assigned IP
tester.assertResponse(new Request("http://localhost:8080/nodes/v2/node",
- "[" + asHostJson("host200.yahoo.com", "default", "127.0.2.1") + "]",
+ "[" + asHostJson("host200.yahoo.com", "default", List.of(), "127.0.2.1") + "]",
Request.Method.POST), 400,
"{\"error-code\":\"BAD_REQUEST\",\"message\":\"Cannot assign [127.0.2.1] to host200.yahoo.com: [127.0.2.1] already assigned to host2.yahoo.com\"}");
@@ -334,7 +335,7 @@ public class NodesV2ApiTest {
// Node types running a single container can share their IP address with child node
tester.assertResponse(new Request("http://localhost:8080/nodes/v2/node",
- "[" + asNodeJson("cfghost42.yahoo.com", NodeType.confighost, "default", Optional.empty(), Optional.empty(), Optional.empty(), "127.0.42.1") + "]",
+ "[" + asNodeJson("cfghost42.yahoo.com", NodeType.confighost, "default", Optional.empty(), Optional.empty(), Optional.empty(), List.of(), "127.0.42.1") + "]",
Request.Method.POST), 200,
"{\"message\":\"Added 1 nodes to the provisioned state\"}");
tester.assertResponse(new Request("http://localhost:8080/nodes/v2/node",
@@ -350,7 +351,7 @@ public class NodesV2ApiTest {
// ... nor with child node on different host
tester.assertResponse(new Request("http://localhost:8080/nodes/v2/node",
- "[" + asNodeJson("cfghost43.yahoo.com", NodeType.confighost, "default", Optional.empty(), Optional.empty(), Optional.empty(), "127.0.43.1") + "]",
+ "[" + asNodeJson("cfghost43.yahoo.com", NodeType.confighost, "default", Optional.empty(), Optional.empty(), Optional.empty(), List.of(), "127.0.43.1") + "]",
Request.Method.POST), 200,
"{\"message\":\"Added 1 nodes to the provisioned state\"}");
tester.assertResponse(new Request("http://localhost:8080/nodes/v2/node/cfg42.yahoo.com",
@@ -392,7 +393,7 @@ public class NodesV2ApiTest {
@Test
public void fails_to_ready_node_with_hard_fail() throws Exception {
assertResponse(new Request("http://localhost:8080/nodes/v2/node",
- ("[" + asHostJson("host12.yahoo.com", "default") + "]").
+ ("[" + asHostJson("host12.yahoo.com", "default", List.of()) + "]").
getBytes(StandardCharsets.UTF_8),
Request.Method.POST),
"{\"message\":\"Added 1 nodes to the provisioned state\"}");
@@ -961,7 +962,8 @@ public class NodesV2ApiTest {
public void test_node_switch_hostname() throws Exception {
String hostname = "host42.yahoo.com";
// Add host with switch hostname
- String json = asNodeJson(hostname, NodeType.host, "default", Optional.empty(), Optional.empty(), Optional.of("switch0"), "127.0.42.1", "::42:1");
+ String json = asNodeJson(hostname, NodeType.host, "default", Optional.empty(), Optional.empty(),
+ Optional.of("switch0"), List.of(), "127.0.42.1", "::42:1");
assertResponse(new Request("http://localhost:8080/nodes/v2/node",
("[" + json + "]").getBytes(StandardCharsets.UTF_8),
Request.Method.POST),
@@ -1013,17 +1015,22 @@ public class NodesV2ApiTest {
"\"flavor\":\"" + flavor + "\"}";
}
- private static String asHostJson(String hostname, String flavor, String... ipAddress) {
- return asNodeJson(hostname, NodeType.host, flavor, Optional.empty(), Optional.empty(), Optional.empty(), ipAddress);
+ private static String asHostJson(String hostname, String flavor, List<String> additionalHostnames, String... ipAddress) {
+ return asNodeJson(hostname, NodeType.host, flavor, Optional.empty(), Optional.empty(), Optional.empty(),
+ additionalHostnames, ipAddress);
}
- private static String asNodeJson(String hostname, NodeType nodeType, String flavor, Optional<TenantName> reservedTo, Optional<ApplicationId> exclusiveTo, Optional<String> switchHostname, String... ipAddress) {
+ private static String asNodeJson(String hostname, NodeType nodeType, String flavor, Optional<TenantName> reservedTo,
+ Optional<ApplicationId> exclusiveTo, Optional<String> switchHostname,
+ List<String> additionalHostnames, String... ipAddress) {
return "{\"hostname\":\"" + hostname + "\", \"openStackId\":\"" + hostname + "\"," +
createIpAddresses(ipAddress) +
"\"flavor\":\"" + flavor + "\"" +
(reservedTo.map(tenantName -> ", \"reservedTo\":\"" + tenantName.value() + "\"").orElse("")) +
(exclusiveTo.map(appId -> ", \"exclusiveTo\":\"" + appId.serializedForm() + "\"").orElse("")) +
(switchHostname.map(s -> ", \"switchHostname\":\"" + s + "\"").orElse("")) +
+ (additionalHostnames.isEmpty() ? "" : ", \"additionalHostnames\":[\"" +
+ String.join("\",\"", additionalHostnames) + "\"]") +
", \"type\":\"" + nodeType + "\"}";
}
diff --git a/node-repository/src/test/java/com/yahoo/vespa/hosted/provision/restapi/responses/node9.json b/node-repository/src/test/java/com/yahoo/vespa/hosted/provision/restapi/responses/node9.json
index dac9fd30267..809e58bd7b6 100644
--- a/node-repository/src/test/java/com/yahoo/vespa/hosted/provision/restapi/responses/node9.json
+++ b/node-repository/src/test/java/com/yahoo/vespa/hosted/provision/restapi/responses/node9.json
@@ -25,5 +25,6 @@
"127.0.9.1",
"::9:1"
],
- "additionalIpAddresses": []
+ "additionalIpAddresses": [],
+ "additionalHostnames": ["node9-1.yahoo.com"]
}
diff --git a/vespajlib/src/main/java/com/yahoo/collections/AbstractFilteringList.java b/vespajlib/src/main/java/com/yahoo/collections/AbstractFilteringList.java
index de1040852f5..4a24cdcc7bf 100644
--- a/vespajlib/src/main/java/com/yahoo/collections/AbstractFilteringList.java
+++ b/vespajlib/src/main/java/com/yahoo/collections/AbstractFilteringList.java
@@ -10,6 +10,7 @@ import java.util.Iterator;
import java.util.List;
import java.util.Optional;
import java.util.Random;
+import java.util.Set;
import java.util.function.BiFunction;
import java.util.function.Function;
import java.util.function.Predicate;
@@ -72,6 +73,9 @@ public abstract class AbstractFilteringList<Type, ListType extends AbstractFilte
/** Returns the items in this as an immutable list. */
public final List<Type> asList() { return items; }
+ /** Returns the items in this as a set. */
+ public final Set<Type> asSet() { return new HashSet<>(items); }
+
/** Returns the items in this as an immutable list after mapping with the given function. */
public final <OtherType> List<OtherType> mapToList(Function<Type, OtherType> mapper) {
return items.stream().map(mapper).collect(toUnmodifiableList());
diff --git a/zkfacade/src/main/java/com/yahoo/vespa/curator/Curator.java b/zkfacade/src/main/java/com/yahoo/vespa/curator/Curator.java
index 127eeba71e1..5966ef77877 100644
--- a/zkfacade/src/main/java/com/yahoo/vespa/curator/Curator.java
+++ b/zkfacade/src/main/java/com/yahoo/vespa/curator/Curator.java
@@ -197,13 +197,14 @@ public class Curator implements AutoCloseable {
* A convenience method which sets some content at a path.
* If the path and any of its parents does not exists they are created.
*/
+ // TODO: Use create().orSetData() in Curator 4 and later
public void set(Path path, byte[] data) {
+ if ( ! exists(path))
+ create(path);
+
String absolutePath = path.getAbsolute();
try {
- if ( ! exists(path))
- framework().create().creatingParentsIfNeeded().forPath(absolutePath, data);
- else
- framework().setData().forPath(absolutePath, data);
+ framework().setData().forPath(absolutePath, data);
} catch (Exception e) {
throw new RuntimeException("Could not set data at " + absolutePath, e);
}