summaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/deploy/Deployment.java19
-rw-r--r--eval/src/vespa/eval/eval/fast_value.hpp12
-rw-r--r--eval/src/vespa/eval/eval/fast_value_index.h24
-rw-r--r--fnet/src/vespa/fnet/connection.cpp5
-rw-r--r--searchlib/src/tests/postinglistbm/stress_runner.cpp24
-rw-r--r--searchlib/src/tests/tensor/distance_calculator/distance_calculator_test.cpp13
-rw-r--r--searchlib/src/tests/transactionlog/translogclient_test.cpp1
-rw-r--r--searchlib/src/tests/transactionlogstress/translogstress.cpp1
-rw-r--r--searchlib/src/vespa/searchlib/tensor/CMakeLists.txt1
-rw-r--r--searchlib/src/vespa/searchlib/tensor/distance_calculator.h14
-rw-r--r--searchlib/src/vespa/searchlib/tensor/fast_value_view.cpp39
-rw-r--r--searchlib/src/vespa/searchlib/tensor/fast_value_view.h24
-rw-r--r--searchlib/src/vespa/searchlib/tensor/tensor_buffer_operations.cpp33
-rw-r--r--searchlib/src/vespa/searchlib/transactionlog/translogclient.cpp1
-rw-r--r--vespalib/src/tests/net/tls/capabilities/capabilities_test.cpp2
-rw-r--r--vespalib/src/tests/net/tls/openssl_impl/openssl_impl_test.cpp22
-rw-r--r--vespalib/src/vespa/vespalib/net/tls/capability_set.cpp4
-rw-r--r--vespalib/src/vespa/vespalib/net/tls/capability_set.h2
-rw-r--r--vespalib/src/vespa/vespalib/net/tls/policy_checking_certificate_verifier.cpp6
-rw-r--r--vespalib/src/vespa/vespalib/net/tls/verification_result.cpp26
-rw-r--r--vespalib/src/vespa/vespalib/net/tls/verification_result.h23
21 files changed, 197 insertions, 99 deletions
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/deploy/Deployment.java b/configserver/src/main/java/com/yahoo/vespa/config/server/deploy/Deployment.java
index df449ca017b..66a5bc5a023 100644
--- a/configserver/src/main/java/com/yahoo/vespa/config/server/deploy/Deployment.java
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/deploy/Deployment.java
@@ -1,13 +1,12 @@
// Copyright Yahoo. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
package com.yahoo.vespa.config.server.deploy;
-import com.google.common.base.Supplier;
-import com.google.common.base.Suppliers;
import com.yahoo.concurrent.UncheckedTimeoutException;
import com.yahoo.config.FileReference;
import com.yahoo.config.application.api.DeployLogger;
import com.yahoo.config.provision.ActivationContext;
import com.yahoo.config.provision.ApplicationId;
+import com.yahoo.config.provision.ApplicationLockException;
import com.yahoo.config.provision.ApplicationTransaction;
import com.yahoo.config.provision.HostFilter;
import com.yahoo.config.provision.HostSpec;
@@ -29,11 +28,14 @@ import com.yahoo.vespa.config.server.session.PrepareParams;
import com.yahoo.vespa.config.server.session.Session;
import com.yahoo.vespa.config.server.session.SessionRepository;
import com.yahoo.vespa.config.server.tenant.Tenant;
+import com.yahoo.yolean.concurrent.Memoized;
+
import java.time.Clock;
import java.time.Duration;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Supplier;
import java.util.logging.Level;
import java.util.logging.Logger;
import java.util.stream.Collectors;
@@ -113,8 +115,6 @@ public class Deployment implements com.yahoo.config.provision.Deployment {
deleteSession();
throw e;
}
-
- waitForResourcesOrTimeout(params, session, provisioner);
}
/** Activates this. If it is not already prepared, this will call prepare first. */
@@ -125,6 +125,8 @@ public class Deployment implements com.yahoo.config.provision.Deployment {
validateSessionStatus(session);
PrepareParams params = this.params.get();
+ waitForResourcesOrTimeout(params, session, provisioner);
+
ApplicationId applicationId = session.getApplicationId();
try (ActionTimer timer = applicationRepository.timerFor(applicationId, "deployment.activateMillis")) {
TimeoutBudget timeoutBudget = params.getTimeoutBudget();
@@ -263,7 +265,7 @@ public class Deployment implements com.yahoo.config.provision.Deployment {
// Use supplier because we shouldn't/can't create this before validateSessionStatus() for prepared deployments,
// memoize because we want to create this once for unprepared deployments
- return Suppliers.memoize(() -> {
+ return new Memoized<>(() -> {
TimeoutBudget timeoutBudget = new TimeoutBudget(clock, timeout);
PrepareParams.Builder params = new PrepareParams.Builder()
@@ -288,20 +290,19 @@ public class Deployment implements com.yahoo.config.provision.Deployment {
Set<HostSpec> preparedHosts = session.getAllocatedHosts().getHosts();
ActivationContext context = new ActivationContext(session.getSessionId());
- ProvisionLock lock = new ProvisionLock(session.getApplicationId(), () -> {});
- AtomicReference<TransientException> lastException = new AtomicReference<>();
+ AtomicReference<Exception> lastException = new AtomicReference<>();
while (true) {
params.getTimeoutBudget().assertNotTimedOut(
() -> "Timeout exceeded while waiting for application resources of '" + session.getApplicationId() + "'" +
Optional.ofNullable(lastException.get()).map(e -> ". Last exception: " + e.getMessage()).orElse(""));
- try {
+ try (ProvisionLock lock = provisioner.get().lock(session.getApplicationId())) {
// Call to activate to make sure that everything is ready, but do not commit the transaction
ApplicationTransaction transaction = new ApplicationTransaction(lock, new NestedTransaction());
provisioner.get().activate(preparedHosts, context, transaction);
return;
- } catch (TransientException e) {
+ } catch (ApplicationLockException | TransientException e) {
lastException.set(e);
try {
Thread.sleep(durationBetweenResourceReadyChecks.toMillis());
diff --git a/eval/src/vespa/eval/eval/fast_value.hpp b/eval/src/vespa/eval/eval/fast_value.hpp
index 2eaefa3670c..47f99d19055 100644
--- a/eval/src/vespa/eval/eval/fast_value.hpp
+++ b/eval/src/vespa/eval/eval/fast_value.hpp
@@ -1,7 +1,7 @@
// Copyright Yahoo. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
#include "value_builder_factory.h"
-#include "fast_addr_map.h"
+#include "fast_value_index.h"
#include "inline_operation.h"
#include <vespa/eval/instruction/generic_join.h>
#include <vespa/vespalib/stllike/hashtable.hpp>
@@ -12,16 +12,6 @@ namespace vespalib::eval {
//-----------------------------------------------------------------------------
-// This is the class instructions will look for when optimizing sparse
-// operations by calling inline functions directly.
-struct FastValueIndex final : Value::Index {
- FastAddrMap map;
- FastValueIndex(size_t num_mapped_dims_in, const StringIdVector &labels, size_t expected_subspaces_in)
- : map(num_mapped_dims_in, labels, expected_subspaces_in) {}
- size_t size() const override { return map.size(); }
- std::unique_ptr<View> create_view(ConstArrayRef<size_t> dims) const override;
-};
-
inline bool is_fast(const Value::Index &index) {
return (std::type_index(typeid(index)) == std::type_index(typeid(FastValueIndex)));
}
diff --git a/eval/src/vespa/eval/eval/fast_value_index.h b/eval/src/vespa/eval/eval/fast_value_index.h
new file mode 100644
index 00000000000..edf96490db6
--- /dev/null
+++ b/eval/src/vespa/eval/eval/fast_value_index.h
@@ -0,0 +1,24 @@
+// Copyright Yahoo. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+
+#pragma once
+
+#include "value.h"
+#include "fast_addr_map.h"
+
+namespace vespalib::eval {
+
+/*
+ * Tensor value index, used to map labels to dense subspace indexes.
+ *
+ * This is the class instructions will look for when optimizing sparse
+ * operations by calling inline functions directly.
+ */
+struct FastValueIndex final : Value::Index {
+ FastAddrMap map;
+ FastValueIndex(size_t num_mapped_dims_in, const StringIdVector &labels, size_t expected_subspaces_in)
+ : map(num_mapped_dims_in, labels, expected_subspaces_in) {}
+ size_t size() const override { return map.size(); }
+ std::unique_ptr<View> create_view(ConstArrayRef<size_t> dims) const override;
+};
+
+}
diff --git a/fnet/src/vespa/fnet/connection.cpp b/fnet/src/vespa/fnet/connection.cpp
index 26367c904b2..e344f2a22a6 100644
--- a/fnet/src/vespa/fnet/connection.cpp
+++ b/fnet/src/vespa/fnet/connection.cpp
@@ -475,7 +475,7 @@ FNET_Connection::FNET_Connection(FNET_TransportThread *owner,
_streamer(streamer),
_serverAdapter(serverAdapter),
_socket(owner->owner().create_server_crypto_socket(std::move(socket))),
- _resolve_handler(nullptr),
+ _resolve_handler(),
_context(),
_state(FNET_CONNECTING),
_flags(owner->owner().getConfig()),
@@ -506,7 +506,7 @@ FNET_Connection::FNET_Connection(FNET_TransportThread *owner,
_streamer(streamer),
_serverAdapter(serverAdapter),
_socket(),
- _resolve_handler(nullptr),
+ _resolve_handler(),
_context(context),
_state(FNET_CONNECTING),
_flags(owner->owner().getConfig()),
@@ -529,6 +529,7 @@ FNET_Connection::FNET_Connection(FNET_TransportThread *owner,
FNET_Connection::~FNET_Connection()
{
+ assert(!_resolve_handler);
assert(_cleanup == nullptr);
_num_connections.fetch_sub(1, std::memory_order_relaxed);
}
diff --git a/searchlib/src/tests/postinglistbm/stress_runner.cpp b/searchlib/src/tests/postinglistbm/stress_runner.cpp
index 179e4f49ef4..3e3db3701c7 100644
--- a/searchlib/src/tests/postinglistbm/stress_runner.cpp
+++ b/searchlib/src/tests/postinglistbm/stress_runner.cpp
@@ -2,7 +2,6 @@
#include "stress_runner.h"
-#include <vespa/fastos/thread.h>
#include <vespa/searchlib/test/fakedata/fake_match_loop.h>
#include <vespa/searchlib/test/fakedata/fakeposting.h>
#include <vespa/searchlib/test/fakedata/fakeword.h>
@@ -13,7 +12,7 @@
#include <condition_variable>
#include <mutex>
#include <vector>
-#include <thread>
+#include <vespa/vespalib/util/thread.h>
#include <vespa/log/log.h>
LOG_SETUP(".stress_runner");
@@ -43,7 +42,7 @@ private:
uint32_t _stride;
bool _unpack;
- FastOS_ThreadPool *_threadPool;
+ vespalib::ThreadPool _threadPool;
std::vector<StressWorkerUP> _workers;
uint32_t _workersDone;
@@ -88,7 +87,7 @@ public:
double runWorkers(const std::string &postingFormat);
};
-class StressWorker : public FastOS_Runnable {
+class StressWorker : vespalib::Runnable {
protected:
StressMaster& _master;
uint32_t _id;
@@ -102,7 +101,7 @@ public:
StressWorker(StressMaster& master, uint32_t id);
virtual ~StressWorker();
- virtual void Run(FastOS_ThreadInterface* thisThread, void* arg) override;
+ virtual void run() override;
};
class DirectStressWorker : public StressWorker {
@@ -147,7 +146,7 @@ StressMaster::StressMaster(vespalib::Rand48 &rnd,
_skipCommonPairsRate(skipCommonPairsRate),
_stride(stride),
_unpack(unpack),
- _threadPool(nullptr),
+ _threadPool(),
_workers(),
_workersDone(0),
_wordSet(wordSet),
@@ -159,17 +158,12 @@ StressMaster::StressMaster(vespalib::Rand48 &rnd,
_tasks()
{
LOG(info, "StressMaster::StressMaster()");
-
- _threadPool = new FastOS_ThreadPool(400);
}
StressMaster::~StressMaster()
{
LOG(info, "StressMaster::~StressMaster()");
-
- _threadPool->Close();
- delete _threadPool;
- _threadPool = nullptr;
+ _threadPool.join();
_workers.clear();
dropPostings();
}
@@ -329,7 +323,7 @@ StressMaster::runWorkers(const std::string &postingFormat)
}
for (auto& worker : _workers) {
- _threadPool->NewThread(worker.get());
+ _threadPool.start([obj = worker.get()](){obj->run();});
}
{
@@ -357,10 +351,8 @@ StressWorker::StressWorker(StressMaster& master, uint32_t id)
StressWorker::~StressWorker() = default;
void
-StressWorker::Run(FastOS_ThreadInterface* thisThread, void* arg)
+StressWorker::run()
{
- (void) thisThread;
- (void) arg;
LOG(debug, "StressWorker::Run(), id=%u", _id);
bool unpack = _master.getUnpack();
diff --git a/searchlib/src/tests/tensor/distance_calculator/distance_calculator_test.cpp b/searchlib/src/tests/tensor/distance_calculator/distance_calculator_test.cpp
index 5e556979254..ef4292ddbb4 100644
--- a/searchlib/src/tests/tensor/distance_calculator/distance_calculator_test.cpp
+++ b/searchlib/src/tests/tensor/distance_calculator/distance_calculator_test.cpp
@@ -18,6 +18,8 @@ using namespace vespalib::eval;
using search::AttributeVector;
+using OptSubspace = std::optional<uint32_t>;
+
std::unique_ptr<Value> make_tensor(const vespalib::string& expr) {
return SimpleValue::from_spec(TensorSpec::from_expr(expr));
}
@@ -49,6 +51,11 @@ public:
auto calc = DistanceCalculator::make_with_validation(*attr, *qt);
return calc->calc_raw_score(docid);
}
+ OptSubspace calc_closest_subspace(uint32_t docid, const vespalib::string& query_tensor) {
+ auto qt = make_tensor(query_tensor);
+ auto calc = DistanceCalculator::make_with_validation(*attr, *qt);
+ return calc->calc_closest_subspace(attr->asTensorAttribute()->get_vectors(docid));
+ }
void make_calc_throws(const vespalib::string& query_tensor) {
auto qt = make_tensor(query_tensor);
DistanceCalculator::make_with_validation(*attr, *qt);
@@ -63,9 +70,11 @@ TEST_F(DistanceCalculatorTest, calculation_over_dense_tensor_attribute)
vespalib::string qt = "tensor(y[2]):[7,10]";
EXPECT_DOUBLE_EQ(16, calc_distance(1, qt));
EXPECT_DOUBLE_EQ(max_distance, calc_distance(2, qt));
+ EXPECT_EQ(OptSubspace(0), calc_closest_subspace(1, qt));
EXPECT_DOUBLE_EQ(1.0/(1.0 + 4.0), calc_rawscore(1, qt));
EXPECT_DOUBLE_EQ(0.0, calc_rawscore(2, qt));
+ EXPECT_EQ(OptSubspace(), calc_closest_subspace(2, qt));
}
TEST_F(DistanceCalculatorTest, calculation_over_mixed_tensor_attribute)
@@ -77,8 +86,12 @@ TEST_F(DistanceCalculatorTest, calculation_over_mixed_tensor_attribute)
vespalib::string qt_2 = "tensor(y[2]):[1,10]";
EXPECT_DOUBLE_EQ(16, calc_distance(1, qt_1));
EXPECT_DOUBLE_EQ(4, calc_distance(1, qt_2));
+ EXPECT_EQ(OptSubspace(1), calc_closest_subspace(1, qt_1));
+ EXPECT_EQ(OptSubspace(0), calc_closest_subspace(1, qt_2));
EXPECT_DOUBLE_EQ(max_distance, calc_distance(2, qt_1));
EXPECT_DOUBLE_EQ(max_distance, calc_distance(3, qt_1));
+ EXPECT_EQ(OptSubspace(), calc_closest_subspace(2, qt_1));
+ EXPECT_EQ(OptSubspace(), calc_closest_subspace(3, qt_1));
EXPECT_DOUBLE_EQ(1.0/(1.0 + 4.0), calc_rawscore(1, qt_1));
EXPECT_DOUBLE_EQ(1.0/(1.0 + 2.0), calc_rawscore(1, qt_2));
diff --git a/searchlib/src/tests/transactionlog/translogclient_test.cpp b/searchlib/src/tests/transactionlog/translogclient_test.cpp
index cdf157b3fa0..af277ecbc68 100644
--- a/searchlib/src/tests/transactionlog/translogclient_test.cpp
+++ b/searchlib/src/tests/transactionlog/translogclient_test.cpp
@@ -11,7 +11,6 @@
#include <vespa/vespalib/util/destructor_callbacks.h>
#include <vespa/fnet/transport.h>
#include <vespa/fastos/file.h>
-#include <vespa/fastos/thread.h>
#include <thread>
#include <vespa/log/log.h>
diff --git a/searchlib/src/tests/transactionlogstress/translogstress.cpp b/searchlib/src/tests/transactionlogstress/translogstress.cpp
index 14a66c71c18..124eb39e84b 100644
--- a/searchlib/src/tests/transactionlogstress/translogstress.cpp
+++ b/searchlib/src/tests/transactionlogstress/translogstress.cpp
@@ -12,7 +12,6 @@
#include <sstream>
#include <thread>
#include <unistd.h>
-#include <vespa/fastos/thread.h>
#include <vespa/log/log.h>
#include <vespa/vespalib/util/time.h>
diff --git a/searchlib/src/vespa/searchlib/tensor/CMakeLists.txt b/searchlib/src/vespa/searchlib/tensor/CMakeLists.txt
index 9f96bce90c9..a64bd6af4a9 100644
--- a/searchlib/src/vespa/searchlib/tensor/CMakeLists.txt
+++ b/searchlib/src/vespa/searchlib/tensor/CMakeLists.txt
@@ -13,6 +13,7 @@ vespa_add_library(searchlib_tensor OBJECT
distance_function_factory.cpp
empty_subspace.cpp
euclidean_distance.cpp
+ fast_value_view.cpp
geo_degrees_distance.cpp
hamming_distance.cpp
hash_set_visited_tracker.cpp
diff --git a/searchlib/src/vespa/searchlib/tensor/distance_calculator.h b/searchlib/src/vespa/searchlib/tensor/distance_calculator.h
index 320f071cbbb..f501b004254 100644
--- a/searchlib/src/vespa/searchlib/tensor/distance_calculator.h
+++ b/searchlib/src/vespa/searchlib/tensor/distance_calculator.h
@@ -4,6 +4,7 @@
#include "distance_function.h"
#include "i_tensor_attribute.h"
#include "vector_bundle.h"
+#include <optional>
namespace vespalib::eval { struct Value; }
@@ -64,6 +65,19 @@ public:
return result;
}
+ std::optional<uint32_t> calc_closest_subspace(VectorBundle vectors) {
+ double best_distance = 0.0;
+ std::optional<uint32_t> closest_subspace;
+ for (uint32_t i = 0; i < vectors.subspaces(); ++i) {
+ double distance = _dist_fun->calc(_query_tensor_cells, vectors.cells(i));
+ if (!closest_subspace.has_value() || distance < best_distance) {
+ best_distance = distance;
+ closest_subspace = i;
+ }
+ }
+ return closest_subspace;
+ }
+
/**
* Create a calculator for the given attribute tensor and query tensor, if possible.
*
diff --git a/searchlib/src/vespa/searchlib/tensor/fast_value_view.cpp b/searchlib/src/vespa/searchlib/tensor/fast_value_view.cpp
new file mode 100644
index 00000000000..29cc47cb543
--- /dev/null
+++ b/searchlib/src/vespa/searchlib/tensor/fast_value_view.cpp
@@ -0,0 +1,39 @@
+// Copyright Yahoo. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+
+#include "fast_value_view.h"
+#include <vespa/vespalib/stllike/hash_map.hpp>
+
+using vespalib::ConstArrayRef;
+using vespalib::MemoryUsage;
+using vespalib::string_id;
+using vespalib::eval::FastAddrMap;
+using vespalib::eval::TypedCells;
+using vespalib::eval::Value;
+using vespalib::eval::ValueType;
+using vespalib::eval::self_memory_usage;
+
+namespace search::tensor {
+
+FastValueView::FastValueView(const ValueType& type, ConstArrayRef<string_id> labels, TypedCells cells, size_t num_mapped_dimensions, size_t num_subspaces)
+ : Value(),
+ _type(type),
+ _labels(labels.begin(), labels.end()),
+ _index(num_mapped_dimensions, _labels, num_subspaces),
+ _cells(cells)
+{
+ for (size_t i = 0; i < num_subspaces; ++i) {
+ ConstArrayRef<string_id> addr(_labels.data() + (i * num_mapped_dimensions), num_mapped_dimensions);
+ _index.map.add_mapping(FastAddrMap::hash_labels(addr));
+ }
+ assert(_index.map.size() == num_subspaces);
+}
+
+MemoryUsage
+FastValueView::get_memory_usage() const
+{
+ MemoryUsage usage = self_memory_usage<FastValueView>();
+ usage.merge(_index.map.estimate_extra_memory_usage());
+ return usage;
+}
+
+}
diff --git a/searchlib/src/vespa/searchlib/tensor/fast_value_view.h b/searchlib/src/vespa/searchlib/tensor/fast_value_view.h
new file mode 100644
index 00000000000..c3f13ac5856
--- /dev/null
+++ b/searchlib/src/vespa/searchlib/tensor/fast_value_view.h
@@ -0,0 +1,24 @@
+// Copyright Yahoo. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+
+#pragma once
+
+#include <vespa/eval/eval/fast_value_index.h>
+
+namespace search::tensor {
+
+/*
+ * Tensor view that is not self-contained. It references external cell values.
+ */
+struct FastValueView final : vespalib::eval::Value {
+ const vespalib::eval::ValueType& _type;
+ vespalib::StringIdVector _labels;
+ vespalib::eval::FastValueIndex _index;
+ vespalib::eval::TypedCells _cells;
+ FastValueView(const vespalib::eval::ValueType& type, vespalib::ConstArrayRef<vespalib::string_id> labels, vespalib::eval::TypedCells cells, size_t num_mapped_dimensions, size_t num_subspaces);
+ const vespalib::eval::ValueType& type() const override { return _type; }
+ const vespalib::eval::Value::Index& index() const override { return _index; }
+ vespalib::eval::TypedCells cells() const override { return _cells; }
+ vespalib::MemoryUsage get_memory_usage() const override;
+};
+
+}
diff --git a/searchlib/src/vespa/searchlib/tensor/tensor_buffer_operations.cpp b/searchlib/src/vespa/searchlib/tensor/tensor_buffer_operations.cpp
index fcdb9311ec6..135c62b3cfa 100644
--- a/searchlib/src/vespa/searchlib/tensor/tensor_buffer_operations.cpp
+++ b/searchlib/src/vespa/searchlib/tensor/tensor_buffer_operations.cpp
@@ -1,8 +1,7 @@
// Copyright Yahoo. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
#include "tensor_buffer_operations.h"
-#include <vespa/eval/eval/fast_value.hpp>
-#include <vespa/eval/eval/value.h>
+#include "fast_value_view.h"
#include <vespa/eval/eval/value_codec.h>
#include <vespa/eval/eval/value_type.h>
#include <vespa/eval/streamed/streamed_value_view.h>
@@ -35,36 +34,6 @@ adjust_min_alignment(size_t min_alignment)
return std::max(std::max(sizeof(uint32_t), sizeof(string_id)), min_alignment);
}
-struct FastValueView final : Value {
- const ValueType& _type;
- StringIdVector _labels;
- FastValueIndex _index;
- TypedCells _cells;
- FastValueView(const ValueType& type, ConstArrayRef<string_id> labels, TypedCells cells, size_t num_mapped_dimensions, size_t num_subspaces);
- const ValueType& type() const override { return _type; }
- const Value::Index& index() const override { return _index; }
- TypedCells cells() const override { return _cells; }
- MemoryUsage get_memory_usage() const override {
- MemoryUsage usage = self_memory_usage<FastValueView>();
- usage.merge(_index.map.estimate_extra_memory_usage());
- return usage;
- }
-};
-
-FastValueView::FastValueView(const ValueType& type, ConstArrayRef<string_id> labels, TypedCells cells, size_t num_mapped_dimensions, size_t num_subspaces)
- : Value(),
- _type(type),
- _labels(labels.begin(), labels.end()),
- _index(num_mapped_dimensions, _labels, num_subspaces),
- _cells(cells)
-{
- for (size_t i = 0; i < num_subspaces; ++i) {
- ConstArrayRef<string_id> addr(_labels.data() + (i * num_mapped_dimensions), num_mapped_dimensions);
- _index.map.add_mapping(FastAddrMap::hash_labels(addr));
- }
- assert(_index.map.size() == num_subspaces);
-}
-
}
TensorBufferOperations::TensorBufferOperations(const vespalib::eval::ValueType& tensor_type)
diff --git a/searchlib/src/vespa/searchlib/transactionlog/translogclient.cpp b/searchlib/src/vespa/searchlib/transactionlog/translogclient.cpp
index 133fabd3e5f..17f06b189c6 100644
--- a/searchlib/src/vespa/searchlib/transactionlog/translogclient.cpp
+++ b/searchlib/src/vespa/searchlib/transactionlog/translogclient.cpp
@@ -8,7 +8,6 @@
#include <vespa/fnet/transport.h>
#include <vespa/vespalib/util/threadstackexecutor.h>
#include <vespa/vespalib/util/size_literals.h>
-#include <vespa/fastos/thread.h>
#include <vespa/log/log.h>
diff --git a/vespalib/src/tests/net/tls/capabilities/capabilities_test.cpp b/vespalib/src/tests/net/tls/capabilities/capabilities_test.cpp
index 0bf04289a65..ed133f9ffe8 100644
--- a/vespalib/src/tests/net/tls/capabilities/capabilities_test.cpp
+++ b/vespalib/src/tests/net/tls/capabilities/capabilities_test.cpp
@@ -124,7 +124,7 @@ TEST("All known capability sets can be looked up by name") {
check_capability_set_mapping("vespa.telemetry", CapabilitySet::telemetry());
check_capability_set_mapping("vespa.cluster_controller_node", CapabilitySet::cluster_controller_node());
check_capability_set_mapping("vespa.logserver_node", CapabilitySet::logserver_node());
- check_capability_set_mapping("vespa.config_server", CapabilitySet::config_server());
+ check_capability_set_mapping("vespa.config_server_node", CapabilitySet::config_server_node());
}
TEST("Unknown capability set name returns nullopt") {
diff --git a/vespalib/src/tests/net/tls/openssl_impl/openssl_impl_test.cpp b/vespalib/src/tests/net/tls/openssl_impl/openssl_impl_test.cpp
index 0178443643e..068345b7254 100644
--- a/vespalib/src/tests/net/tls/openssl_impl/openssl_impl_test.cpp
+++ b/vespalib/src/tests/net/tls/openssl_impl/openssl_impl_test.cpp
@@ -735,6 +735,28 @@ TEST_F("Authz policy-derived peer capabilities are propagated to CryptoCodec", C
Capability::content_status_pages()}));
}
+TEST_F("Handshake is allowed if at least one policy matches, even if resulting capability set is empty", CertFixture) {
+ auto server_ck = f.create_ca_issued_peer_cert({}, {{"DNS:hello.world.example.com"}});
+ auto authorized = authorized_peers({policy_with({required_san_dns("stale.memes.example.com")},
+ CapabilitySet::make_empty()),
+ policy_with({required_san_dns("fresh.memes.example.com")},
+ CapabilitySet::make_with_all_capabilities())});
+ f.reset_server_with_cert_opts(server_ck, std::move(authorized));
+ auto client_ck = f.create_ca_issued_peer_cert({}, {{"DNS:stale.memes.example.com"}});
+ f.reset_client_with_cert_opts(client_ck, AuthorizedPeers::allow_all_authenticated());
+
+ ASSERT_TRUE(f.handshake());
+
+ // Note: "inversion" of client <-> server is because the capabilities are that of the _peer_.
+ auto client_caps = f.server->granted_capabilities();
+ auto server_caps = f.client->granted_capabilities();
+ // Server (from client's PoV) implicitly has all capabilities since client doesn't specify any policies
+ EXPECT_EQUAL(server_caps, CapabilitySet::make_with_all_capabilities());
+ // Client (from server's PoV) only has capabilities for the rule matching its DNS SAN entry.
+ // In this case, it is the empty set.
+ EXPECT_EQUAL(client_caps, CapabilitySet::make_empty());
+}
+
void reset_peers_with_server_authz_mode(CertFixture& f, AuthorizationMode authz_mode) {
auto ck = f.create_ca_issued_peer_cert({"hello.world.example.com"}, {});
diff --git a/vespalib/src/vespa/vespalib/net/tls/capability_set.cpp b/vespalib/src/vespa/vespalib/net/tls/capability_set.cpp
index 1b879f0c635..b17cf1ba851 100644
--- a/vespalib/src/vespa/vespalib/net/tls/capability_set.cpp
+++ b/vespalib/src/vespa/vespalib/net/tls/capability_set.cpp
@@ -32,7 +32,7 @@ std::optional<CapabilitySet> CapabilitySet::find_capability_set(const string& ca
{"vespa.telemetry", telemetry()},
{"vespa.cluster_controller_node", cluster_controller_node()},
{"vespa.logserver_node", logserver_node()},
- {"vespa.config_server", config_server()}
+ {"vespa.config_server_node", config_server_node()}
});
auto iter = name_to_cap_set.find(cap_set_name);
return (iter != name_to_cap_set.end()) ? std::optional<CapabilitySet>(iter->second) : std::nullopt;
@@ -89,7 +89,7 @@ CapabilitySet CapabilitySet::logserver_node() noexcept {
return shared_app_node_capabilities();
}
-CapabilitySet CapabilitySet::config_server() noexcept {
+CapabilitySet CapabilitySet::config_server_node() noexcept {
return CapabilitySet::of({Capability::client_filereceiver_api(),
Capability::container_management_api(),
Capability::slobrok_api(),
diff --git a/vespalib/src/vespa/vespalib/net/tls/capability_set.h b/vespalib/src/vespa/vespalib/net/tls/capability_set.h
index 8aad28a4162..0811739217e 100644
--- a/vespalib/src/vespa/vespalib/net/tls/capability_set.h
+++ b/vespalib/src/vespa/vespalib/net/tls/capability_set.h
@@ -111,7 +111,7 @@ public:
[[nodiscard]] static CapabilitySet telemetry() noexcept;
[[nodiscard]] static CapabilitySet cluster_controller_node() noexcept;
[[nodiscard]] static CapabilitySet logserver_node() noexcept;
- [[nodiscard]] static CapabilitySet config_server() noexcept;
+ [[nodiscard]] static CapabilitySet config_server_node() noexcept;
[[nodiscard]] static CapabilitySet make_with_all_capabilities() noexcept;
[[nodiscard]] static constexpr CapabilitySet make_empty() noexcept { return {}; };
diff --git a/vespalib/src/vespa/vespalib/net/tls/policy_checking_certificate_verifier.cpp b/vespalib/src/vespa/vespalib/net/tls/policy_checking_certificate_verifier.cpp
index e280434c59f..a3f9b3f52c9 100644
--- a/vespalib/src/vespa/vespalib/net/tls/policy_checking_certificate_verifier.cpp
+++ b/vespalib/src/vespa/vespalib/net/tls/policy_checking_certificate_verifier.cpp
@@ -74,13 +74,15 @@ VerificationResult PolicyConfiguredCertificateVerifier::verify(const PeerCredent
return VerificationResult::make_authorized_with_all_capabilities();
}
CapabilitySet caps;
+ bool matched_any_policy = false;
for (const auto& policy : _authorized_peers.peer_policies()) {
if (matches_all_policy_requirements(peer_creds, policy)) {
caps.add_all(policy.granted_capabilities());
+ matched_any_policy = true;
}
}
- if (!caps.empty()) {
- return VerificationResult::make_authorized_with_capabilities(std::move(caps));
+ if (matched_any_policy) {
+ return VerificationResult::make_authorized_with_capabilities(caps);
} else {
return VerificationResult::make_not_authorized();
}
diff --git a/vespalib/src/vespa/vespalib/net/tls/verification_result.cpp b/vespalib/src/vespa/vespalib/net/tls/verification_result.cpp
index f1e50d3115e..37b95c3c07a 100644
--- a/vespalib/src/vespa/vespalib/net/tls/verification_result.cpp
+++ b/vespalib/src/vespa/vespalib/net/tls/verification_result.cpp
@@ -6,14 +6,18 @@
namespace vespalib::net::tls {
-VerificationResult::VerificationResult() = default;
+VerificationResult::VerificationResult() noexcept
+ : _granted_capabilities(),
+ _authorized(false)
+{}
-VerificationResult::VerificationResult(CapabilitySet granted_capabilities)
- : _granted_capabilities(std::move(granted_capabilities))
+VerificationResult::VerificationResult(bool authorized, CapabilitySet granted_capabilities) noexcept
+ : _granted_capabilities(granted_capabilities),
+ _authorized(authorized)
{}
-VerificationResult::VerificationResult(const VerificationResult&) = default;
-VerificationResult& VerificationResult::operator=(const VerificationResult&) = default;
+VerificationResult::VerificationResult(const VerificationResult&) noexcept = default;
+VerificationResult& VerificationResult::operator=(const VerificationResult&) noexcept = default;
VerificationResult::VerificationResult(VerificationResult&&) noexcept = default;
VerificationResult& VerificationResult::operator=(VerificationResult&&) noexcept = default;
VerificationResult::~VerificationResult() = default;
@@ -29,18 +33,18 @@ void VerificationResult::print(asciistream& os) const {
}
VerificationResult
-VerificationResult::make_authorized_with_capabilities(CapabilitySet granted_capabilities) {
- return VerificationResult(std::move(granted_capabilities));
+VerificationResult::make_authorized_with_capabilities(CapabilitySet granted_capabilities) noexcept {
+ return {true, granted_capabilities};
}
VerificationResult
-VerificationResult::make_authorized_with_all_capabilities() {
- return VerificationResult(CapabilitySet::make_with_all_capabilities());
+VerificationResult::make_authorized_with_all_capabilities() noexcept {
+ return {true, CapabilitySet::make_with_all_capabilities()};
}
VerificationResult
-VerificationResult::make_not_authorized() {
- return {};
+VerificationResult::make_not_authorized() noexcept {
+ return {false, CapabilitySet::make_empty()};
}
asciistream& operator<<(asciistream& os, const VerificationResult& res) {
diff --git a/vespalib/src/vespa/vespalib/net/tls/verification_result.h b/vespalib/src/vespa/vespalib/net/tls/verification_result.h
index 92b32ad92f7..896908f7c13 100644
--- a/vespalib/src/vespa/vespalib/net/tls/verification_result.h
+++ b/vespalib/src/vespa/vespalib/net/tls/verification_result.h
@@ -16,22 +16,27 @@ namespace vespalib::net::tls {
* This result contains the union set of all capabilities granted by the matching
* authorization rules. If no rules matched, the set will be empty. The capability
* set will also be empty for a default-constructed instance.
+ *
+ * It is possible for a VerificationResult to be successful but with an empty
+ * capability set. If capabilities are enforced, this will effectively only
+ * allow mTLS handshakes to go through, allowing rudimentary health checking.
*/
class VerificationResult {
CapabilitySet _granted_capabilities;
+ bool _authorized;
- explicit VerificationResult(CapabilitySet granted_capabilities);
+ VerificationResult(bool authorized, CapabilitySet granted_capabilities) noexcept;
public:
- VerificationResult();
- VerificationResult(const VerificationResult&);
- VerificationResult& operator=(const VerificationResult&);
+ VerificationResult() noexcept; // Unauthorized by default
+ VerificationResult(const VerificationResult&) noexcept;
+ VerificationResult& operator=(const VerificationResult&) noexcept;
VerificationResult(VerificationResult&&) noexcept;
VerificationResult& operator=(VerificationResult&&) noexcept;
~VerificationResult();
- // Returns true iff at least one capability been granted.
+ // Returns true iff the peer matched at least one policy or authorization is not enforced.
[[nodiscard]] bool success() const noexcept {
- return !_granted_capabilities.empty();
+ return _authorized;
}
[[nodiscard]] const CapabilitySet& granted_capabilities() const noexcept {
@@ -40,9 +45,9 @@ public:
void print(asciistream& os) const;
- static VerificationResult make_authorized_with_capabilities(CapabilitySet granted_capabilities);
- static VerificationResult make_authorized_with_all_capabilities();
- static VerificationResult make_not_authorized();
+ static VerificationResult make_authorized_with_capabilities(CapabilitySet granted_capabilities) noexcept;
+ static VerificationResult make_authorized_with_all_capabilities() noexcept;
+ static VerificationResult make_not_authorized() noexcept;
};
asciistream& operator<<(asciistream&, const VerificationResult&);