summaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
authorHenning Baldersheim <balder@yahoo-inc.com>2022-12-19 16:06:16 +0000
committerHenning Baldersheim <balder@yahoo-inc.com>2022-12-20 05:10:52 +0000
commita00e560b0d8267e9b376e5c0e6a2139a7be63281 (patch)
treeba207915b8cac9fb92493cc103821f6cce9db31b
parent05b58ac83b06b00ae97ecafad101e44d4dd76aee (diff)
Remove stacksize from the thread pools and thread executors.
-rw-r--r--config/src/tests/file_acquirer/file_acquirer_test.cpp2
-rw-r--r--config/src/tests/frtconnectionpool/frtconnectionpool.cpp2
-rw-r--r--config/src/vespa/config/subscription/sourcespec.cpp2
-rw-r--r--documentapi/src/vespa/documentapi/messagebus/policies/asyncinitializationpolicy.cpp10
-rw-r--r--documentapi/src/vespa/documentapi/messagebus/policies/mirror_with_all.cpp2
-rw-r--r--eval/src/tests/eval/compile_cache/compile_cache_test.cpp6
-rw-r--r--fastos/src/tests/thread_joinwait_test.cpp5
-rw-r--r--fastos/src/tests/thread_stats_test.cpp2
-rw-r--r--fastos/src/tests/threadtest.cpp12
-rw-r--r--fastos/src/vespa/fastos/thread.cpp4
-rw-r--r--fastos/src/vespa/fastos/thread.h11
-rw-r--r--fnet/src/examples/ping/pingclient.cpp2
-rw-r--r--fnet/src/examples/timeout/timeout.cpp2
-rw-r--r--fnet/src/tests/connect/connect_test.cpp6
-rw-r--r--fnet/src/tests/connection_spread/connection_spread_test.cpp2
-rw-r--r--fnet/src/tests/frt/detach_supervisor/detach_supervisor_test.cpp2
-rw-r--r--fnet/src/tests/frt/parallel_rpc/parallel_rpc_test.cpp2
-rw-r--r--fnet/src/tests/sync_execute/sync_execute.cpp2
-rw-r--r--fnet/src/vespa/fnet/frt/supervisor.cpp2
-rw-r--r--fnet/src/vespa/fnet/transport.cpp2
-rw-r--r--logd/src/tests/watcher/watcher_test.cpp2
-rw-r--r--messagebus/src/vespa/messagebus/messenger.cpp2
-rw-r--r--messagebus/src/vespa/messagebus/network/rpcnetwork.cpp2
-rw-r--r--messagebus/src/vespa/messagebus/testlib/slobrok.cpp4
-rw-r--r--metrics/src/tests/metricmanagertest.cpp14
-rw-r--r--metrics/src/tests/snapshottest.cpp2
-rw-r--r--metrics/src/tests/stresstest.cpp2
-rw-r--r--persistence/src/vespa/persistence/dummyimpl/dummy_bucket_executor.cpp2
-rw-r--r--searchcore/src/apps/proton/proton.cpp2
-rw-r--r--searchcore/src/apps/tests/persistenceconformance_test.cpp5
-rw-r--r--searchcore/src/apps/vespa-feed-bm/vespa_feed_bm.cpp2
-rw-r--r--searchcore/src/apps/vespa-redistribute-bm/vespa_redistribute_bm.cpp12
-rw-r--r--searchcore/src/apps/vespa-transactionlog-inspect/vespa-transactionlog-inspect.cpp2
-rw-r--r--searchcore/src/tests/proton/attribute/attribute_initializer/attribute_initializer_test.cpp2
-rw-r--r--searchcore/src/tests/proton/attribute/attribute_manager/attribute_manager_test.cpp4
-rw-r--r--searchcore/src/tests/proton/attribute/attributeflush_test.cpp6
-rw-r--r--searchcore/src/tests/proton/common/timer/timer_test.cpp4
-rw-r--r--searchcore/src/tests/proton/docsummary/docsummary.cpp6
-rw-r--r--searchcore/src/tests/proton/documentdb/buckethandler/buckethandler_test.cpp2
-rw-r--r--searchcore/src/tests/proton/documentdb/clusterstatehandler/clusterstatehandler_test.cpp2
-rw-r--r--searchcore/src/tests/proton/documentdb/configurer/configurer_test.cpp2
-rw-r--r--searchcore/src/tests/proton/documentdb/document_subdbs/document_subdbs_test.cpp2
-rw-r--r--searchcore/src/tests/proton/documentdb/documentbucketmover/documentbucketmover_test.cpp2
-rw-r--r--searchcore/src/tests/proton/documentdb/documentdb_test.cpp4
-rw-r--r--searchcore/src/tests/proton/documentdb/job_tracked_maintenance_job/job_tracked_maintenance_job_test.cpp2
-rw-r--r--searchcore/src/tests/proton/documentdb/lid_space_compaction/lid_space_jobtest.cpp2
-rw-r--r--searchcore/src/tests/proton/documentdb/maintenancecontroller/maintenancecontroller_test.cpp2
-rw-r--r--searchcore/src/tests/proton/documentmetastore/documentmetastore_test.cpp2
-rw-r--r--searchcore/src/tests/proton/feed_and_search/feed_and_search.cpp2
-rw-r--r--searchcore/src/tests/proton/index/indexcollection_test.cpp3
-rw-r--r--searchcore/src/tests/proton/initializer/task_runner_test.cpp9
-rw-r--r--searchcore/src/tests/proton/metrics/job_tracked_flush/job_tracked_flush_test.cpp2
-rw-r--r--searchcore/src/tests/proton/proton_configurer/proton_configurer_test.cpp3
-rw-r--r--searchcore/src/vespa/searchcore/bmcluster/bm_node.cpp4
-rw-r--r--searchcore/src/vespa/searchcore/bmcluster/bm_node_stats_reporter.cpp2
-rw-r--r--searchcore/src/vespa/searchcore/proton/flushengine/flushengine.cpp5
-rw-r--r--searchcore/src/vespa/searchcore/proton/initializer/task_runner.cpp3
-rw-r--r--searchcore/src/vespa/searchcore/proton/matchengine/matchengine.cpp2
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/documentdb.cpp4
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/documentsubdbcollection.cpp2
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/executorthreadingservice.cpp13
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/executorthreadingservice.h3
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/initialize_threads_calculator.cpp5
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/proton.cpp9
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/shared_threading_service.cpp5
-rw-r--r--searchcore/src/vespa/searchcore/proton/summaryengine/summaryengine.cpp2
-rw-r--r--searchcore/src/vespa/searchcore/proton/test/transport_helper.cpp4
-rw-r--r--searchlib/src/apps/docstore/benchmarkdatastore.cpp4
-rw-r--r--searchlib/src/apps/docstore/documentstoreinspect.cpp2
-rw-r--r--searchlib/src/apps/docstore/verifylogdatastore.cpp2
-rw-r--r--searchlib/src/apps/tests/biglogtest.cpp3
-rw-r--r--searchlib/src/apps/tests/memoryindexstress_test.cpp7
-rw-r--r--searchlib/src/tests/attribute/benchmark/attributebenchmark.cpp2
-rw-r--r--searchlib/src/tests/attribute/tensorattribute/tensorattribute_test.cpp2
-rw-r--r--searchlib/src/tests/diskindex/fusion/fusion_test.cpp4
-rw-r--r--searchlib/src/tests/docstore/document_store_visitor/document_store_visitor_test.cpp2
-rw-r--r--searchlib/src/tests/docstore/file_chunk/file_chunk_test.cpp2
-rw-r--r--searchlib/src/tests/docstore/logdatastore/logdatastore_test.cpp20
-rw-r--r--searchlib/src/tests/docstore/store_by_bucket/store_by_bucket_test.cpp2
-rw-r--r--searchlib/src/tests/memoryindex/memory_index/memory_index_test.cpp2
-rw-r--r--searchlib/src/tests/postinglistbm/stress_runner.cpp2
-rw-r--r--searchlib/src/tests/tensor/hnsw_index/stress_hnsw_mt.cpp5
-rw-r--r--searchlib/src/tests/transactionlog/translogclient_test.cpp2
-rw-r--r--searchlib/src/tests/transactionlogstress/translogstress.cpp2
-rw-r--r--searchlib/src/vespa/searchlib/transactionlog/domain.cpp7
-rw-r--r--searchlib/src/vespa/searchlib/transactionlog/translogclient.cpp2
-rw-r--r--searchlib/src/vespa/searchlib/transactionlog/translogserver.cpp4
-rw-r--r--slobrok/src/tests/mirrorapi/mirrorapi.cpp2
-rw-r--r--storage/src/tests/persistence/filestorage/filestormanagertest.cpp2
-rw-r--r--storage/src/tests/storageserver/statereportertest.cpp2
-rw-r--r--storage/src/vespa/storage/distributor/distributor_stripe_pool.cpp4
-rw-r--r--storage/src/vespa/storage/distributor/distributor_stripe_pool.h6
-rw-r--r--storage/src/vespa/storage/frameworkimpl/status/statuswebserver.cpp5
-rw-r--r--storage/src/vespa/storage/storageserver/rpc/shared_rpc_resources.cpp2
-rw-r--r--storage/src/vespa/storageframework/defaultimplementation/thread/threadpoolimpl.cpp5
-rw-r--r--storage/src/vespa/storageframework/defaultimplementation/thread/threadpoolimpl.h9
-rw-r--r--vespalib/src/tests/btree/btree-stress/btree_stress_test.cpp8
-rw-r--r--vespalib/src/tests/clock/clock_benchmark.cpp2
-rw-r--r--vespalib/src/tests/coro/lazy/lazy_test.cpp15
-rw-r--r--vespalib/src/tests/datastore/sharded_hash_map/sharded_hash_map_test.cpp4
-rw-r--r--vespalib/src/tests/executor/blocking_executor_stress.cpp2
-rw-r--r--vespalib/src/tests/executor/blockingthreadstackexecutor_test.cpp8
-rw-r--r--vespalib/src/tests/executor/stress_test.cpp2
-rw-r--r--vespalib/src/tests/executor/threadstackexecutor_test.cpp10
-rw-r--r--vespalib/src/tests/util/generationhandler_stress/generation_handler_stress_test.cpp4
-rw-r--r--vespalib/src/tests/util/rcuvector/rcuvector_test.cpp4
-rw-r--r--vespalib/src/vespa/vespalib/net/async_resolver.cpp2
-rw-r--r--vespalib/src/vespa/vespalib/testkit/test_hook.cpp2
-rw-r--r--vespalib/src/vespa/vespalib/util/adaptive_sequenced_executor.cpp2
-rw-r--r--vespalib/src/vespa/vespalib/util/blockingthreadstackexecutor.cpp8
-rw-r--r--vespalib/src/vespa/vespalib/util/blockingthreadstackexecutor.h9
-rw-r--r--vespalib/src/vespa/vespalib/util/sequencedtaskexecutor.cpp5
-rw-r--r--vespalib/src/vespa/vespalib/util/shutdownguard.cpp2
-rw-r--r--vespalib/src/vespa/vespalib/util/thread.cpp2
-rw-r--r--vespalib/src/vespa/vespalib/util/threadstackexecutor.cpp18
-rw-r--r--vespalib/src/vespa/vespalib/util/threadstackexecutor.h14
-rw-r--r--vespalib/src/vespa/vespalib/util/threadstackexecutorbase.cpp6
-rw-r--r--vespalib/src/vespa/vespalib/util/threadstackexecutorbase.h4
-rw-r--r--vespalog/src/test/threads/testthreads.cpp2
-rw-r--r--vespamalloc/src/tests/allocfree/allocfree.cpp2
-rw-r--r--vespamalloc/src/tests/allocfree/linklist.cpp2
-rw-r--r--vespamalloc/src/tests/test.cpp2
122 files changed, 247 insertions, 268 deletions
diff --git a/config/src/tests/file_acquirer/file_acquirer_test.cpp b/config/src/tests/file_acquirer/file_acquirer_test.cpp
index 7daafc37fcf..8449a33a782 100644
--- a/config/src/tests/file_acquirer/file_acquirer_test.cpp
+++ b/config/src/tests/file_acquirer/file_acquirer_test.cpp
@@ -26,7 +26,7 @@ struct ServerFixture : FRT_Invokable {
ServerFixture()
: server(),
- threadPool(64_Ki),
+ threadPool(),
transport(),
orb(server.supervisor())
{
diff --git a/config/src/tests/frtconnectionpool/frtconnectionpool.cpp b/config/src/tests/frtconnectionpool/frtconnectionpool.cpp
index 46e1f698091..7cd385deb8e 100644
--- a/config/src/tests/frtconnectionpool/frtconnectionpool.cpp
+++ b/config/src/tests/frtconnectionpool/frtconnectionpool.cpp
@@ -34,7 +34,7 @@ public:
Test::Test()
: vespalib::TestApp(),
- _threadPool(64_Ki),
+ _threadPool(),
_transport()
{
_transport.Start(&_threadPool);
diff --git a/config/src/vespa/config/subscription/sourcespec.cpp b/config/src/vespa/config/subscription/sourcespec.cpp
index 928e2994b02..c05f639f9ba 100644
--- a/config/src/vespa/config/subscription/sourcespec.cpp
+++ b/config/src/vespa/config/subscription/sourcespec.cpp
@@ -127,7 +127,7 @@ ServerSpec::createSourceFactory(const TimingValues & timingValues) const
{
const auto vespaVersion = VespaVersion::getCurrentVersion();
return std::make_unique<FRTSourceFactory>(
- std::make_unique<FRTConnectionPoolWithTransport>(std::make_unique<FastOS_ThreadPool>(64_Ki),
+ std::make_unique<FRTConnectionPoolWithTransport>(std::make_unique<FastOS_ThreadPool>(),
std::make_unique<FNET_Transport>(),
*this, timingValues),
timingValues, _traceLevel, vespaVersion, _compressionType);
diff --git a/documentapi/src/vespa/documentapi/messagebus/policies/asyncinitializationpolicy.cpp b/documentapi/src/vespa/documentapi/messagebus/policies/asyncinitializationpolicy.cpp
index 01a300badab..df035ce0831 100644
--- a/documentapi/src/vespa/documentapi/messagebus/policies/asyncinitializationpolicy.cpp
+++ b/documentapi/src/vespa/documentapi/messagebus/policies/asyncinitializationpolicy.cpp
@@ -13,6 +13,9 @@
#include <vespa/documentapi/messagebus/documentprotocol.h>
#include <vespa/vespalib/text/stringtokenizer.h>
+namespace {
+ VESPA_THREAD_STACK_TAG(async_init_policy);
+}
namespace documentapi {
std::map<string, string>
@@ -34,9 +37,8 @@ AsyncInitializationPolicy::parse(string parameters) {
return retVal;
}
-AsyncInitializationPolicy::AsyncInitializationPolicy(
- const std::map<string, string>&)
- : _executor(new vespalib::ThreadStackExecutor(1, 1024)),
+AsyncInitializationPolicy::AsyncInitializationPolicy(const std::map<string, string>&)
+ : _executor(std::make_unique<vespalib::ThreadStackExecutor>(1, async_init_policy)),
_state(State::NOT_STARTED),
_syncInit(true)
{
@@ -85,7 +87,7 @@ AsyncInitializationPolicy::select(mbus::RoutingContext& context)
// entirely done with accessing the state of this policy (including
// the mutex). After setting _state == RUNNING, only the task
// is allowed to mutate _state.
- _executor->execute(vespalib::Executor::Task::UP(new Task(*this)));
+ _executor->execute(std::make_unique<Task>(*this));
_state = State::RUNNING;
}
diff --git a/documentapi/src/vespa/documentapi/messagebus/policies/mirror_with_all.cpp b/documentapi/src/vespa/documentapi/messagebus/policies/mirror_with_all.cpp
index 9c67b3b3ef7..0841c2ed32b 100644
--- a/documentapi/src/vespa/documentapi/messagebus/policies/mirror_with_all.cpp
+++ b/documentapi/src/vespa/documentapi/messagebus/policies/mirror_with_all.cpp
@@ -10,7 +10,7 @@
namespace documentapi {
MirrorAndStuff::MirrorAndStuff(const slobrok::ConfiguratorFactory & config)
- : _threadPool(std::make_unique<FastOS_ThreadPool>(64_Ki)),
+ : _threadPool(std::make_unique<FastOS_ThreadPool>()),
_transport(std::make_unique<FNET_Transport>()),
_orb(std::make_unique<FRT_Supervisor>(_transport.get())),
_mirror(std::make_unique<slobrok::api::MirrorAPI>(*_orb, config))
diff --git a/eval/src/tests/eval/compile_cache/compile_cache_test.cpp b/eval/src/tests/eval/compile_cache/compile_cache_test.cpp
index 2688dc7b4f5..6cf95cc526d 100644
--- a/eval/src/tests/eval/compile_cache/compile_cache_test.cpp
+++ b/eval/src/tests/eval/compile_cache/compile_cache_test.cpp
@@ -166,7 +166,7 @@ TEST("require that cache usage works") {
}
TEST("require that async cache usage works") {
- auto executor = std::make_shared<ThreadStackExecutor>(8, 256_Ki);
+ auto executor = std::make_shared<ThreadStackExecutor>(8);
auto binding = CompileCache::bind(executor);
CompileCache::Token::UP token_a = CompileCache::compile(*Function::parse("x+y"), PassParams::SEPARATE);
EXPECT_EQUAL(5.0, token_a->get().get_function<2>()(2.0, 3.0));
@@ -298,7 +298,7 @@ TEST_F("compile sequentially, then run all conformance tests", test::EvalSpec())
TEST_F("compile concurrently (8 threads), then run all conformance tests", test::EvalSpec()) {
f1.add_all_cases();
- auto executor = std::make_shared<ThreadStackExecutor>(8, 256_Ki);
+ auto executor = std::make_shared<ThreadStackExecutor>(8);
auto binding = CompileCache::bind(executor);
while (executor->num_idle_workers() < 8) {
std::this_thread::sleep_for(1ms);
@@ -333,7 +333,7 @@ TEST_MT_FF("require that deadlock is avoided with blocking executor", 8, std::sh
size_t loop = 16;
if (thread_id == 0) {
auto t0 = steady_clock::now();
- f1 = std::make_shared<BlockingThreadStackExecutor>(2, 256_Ki, 3);
+ f1 = std::make_shared<BlockingThreadStackExecutor>(2, 3);
auto binding = CompileCache::bind(f1);
TEST_BARRIER(); // #1
for (size_t i = 0; i < num_threads; ++i) {
diff --git a/fastos/src/tests/thread_joinwait_test.cpp b/fastos/src/tests/thread_joinwait_test.cpp
index fc96b49f926..a26501fef01 100644
--- a/fastos/src/tests/thread_joinwait_test.cpp
+++ b/fastos/src/tests/thread_joinwait_test.cpp
@@ -17,7 +17,7 @@ class Thread_JoinWait_Test : public ThreadTestBase
sprintf(testName, "Single Thread Join Wait Multiple Test %d", variant);
TestHeader(testName);
- FastOS_ThreadPool pool(128*1024);
+ FastOS_ThreadPool pool;
const int testThreads=5;
int lastThreadNum = testThreads-1;
@@ -35,8 +35,7 @@ class Thread_JoinWait_Test : public ThreadTestBase
{
jobs[i].code = WAIT_FOR_THREAD_TO_FINISH;
jobs[i].mutex = &jobMutex;
- jobs[i].ownThread = pool.NewThread(this,
- static_cast<void *>(&jobs[i]));
+ jobs[i].ownThread = pool.NewThread(this, static_cast<void *>(&jobs[i]));
rc = (jobs[i].ownThread != nullptr);
Progress(rc, "Creating Thread %d", i+1);
diff --git a/fastos/src/tests/thread_stats_test.cpp b/fastos/src/tests/thread_stats_test.cpp
index 6559513f7ff..40c1199135c 100644
--- a/fastos/src/tests/thread_stats_test.cpp
+++ b/fastos/src/tests/thread_stats_test.cpp
@@ -14,7 +14,7 @@ class Thread_Stats_Test : public ThreadTestBase
TestHeader("Thread Statistics Test");
- FastOS_ThreadPool pool(128*1024);
+ FastOS_ThreadPool pool;
Job job[2];
inactiveThreads = pool.GetNumInactiveThreads();
diff --git a/fastos/src/tests/threadtest.cpp b/fastos/src/tests/threadtest.cpp
index 968779019cb..34723e4edce 100644
--- a/fastos/src/tests/threadtest.cpp
+++ b/fastos/src/tests/threadtest.cpp
@@ -19,7 +19,7 @@ class ThreadTest : public ThreadTestBase
{
TestHeader("Too Many Threads Test");
- FastOS_ThreadPool *pool = new FastOS_ThreadPool(128*1024, MAX_THREADS);
+ FastOS_ThreadPool *pool = new FastOS_ThreadPool(MAX_THREADS);
if (Progress(pool != nullptr, "Allocating ThreadPool")) {
int i;
@@ -62,7 +62,7 @@ class ThreadTest : public ThreadTestBase
{
TestHeader("Create Single Thread And Join Test");
- FastOS_ThreadPool *pool = new FastOS_ThreadPool(128*1024);
+ FastOS_ThreadPool *pool = new FastOS_ThreadPool;
if (Progress(pool != nullptr, "Allocating ThreadPool")) {
Job job;
@@ -93,7 +93,7 @@ class ThreadTest : public ThreadTestBase
if (!silent)
TestHeader("Thread Create Performance");
- FastOS_ThreadPool *pool = new FastOS_ThreadPool(128 * 1024);
+ FastOS_ThreadPool *pool = new FastOS_ThreadPool;
if (!silent)
Progress(pool != nullptr, "Allocating ThreadPool");
@@ -166,7 +166,7 @@ class ThreadTest : public ThreadTestBase
{
TestHeader("Close Pool Test");
- FastOS_ThreadPool pool(128*1024);
+ FastOS_ThreadPool pool;
const int closePoolThreads=9;
Job jobs[closePoolThreads];
@@ -188,7 +188,7 @@ class ThreadTest : public ThreadTestBase
void BreakFlagTest () {
TestHeader("BreakFlag Test");
- FastOS_ThreadPool pool(128*1024);
+ FastOS_ThreadPool pool;
const int breakFlagThreads=4;
@@ -212,7 +212,7 @@ class ThreadTest : public ThreadTestBase
TestHeader ("Thread Id Test");
- FastOS_ThreadPool pool(128*1024);
+ FastOS_ThreadPool pool;
Job jobs[numThreads];
std::mutex slowStartMutex;
diff --git a/fastos/src/vespa/fastos/thread.cpp b/fastos/src/vespa/fastos/thread.cpp
index 3850c14fb26..9a9c3321cac 100644
--- a/fastos/src/vespa/fastos/thread.cpp
+++ b/fastos/src/vespa/fastos/thread.cpp
@@ -14,7 +14,9 @@
// FastOS_ThreadPool
// ----------------------------------------------------------------------
-FastOS_ThreadPool::FastOS_ThreadPool(int , int maxThreads)
+FastOS_ThreadPool::FastOS_ThreadPool() : FastOS_ThreadPool(0) {}
+
+FastOS_ThreadPool::FastOS_ThreadPool(int maxThreads)
: _startedThreadsCount(0),
_closeFlagMutex(),
_closeCalledFlag(false),
diff --git a/fastos/src/vespa/fastos/thread.h b/fastos/src/vespa/fastos/thread.h
index d42987640b4..2fb717403f2 100644
--- a/fastos/src/vespa/fastos/thread.h
+++ b/fastos/src/vespa/fastos/thread.h
@@ -131,14 +131,9 @@ private:
public:
FastOS_ThreadPool(const FastOS_ThreadPool&) = delete;
FastOS_ThreadPool& operator=(const FastOS_ThreadPool&) = delete;
- /**
- * Create a threadpool that can hold a maximum of [maxThreads] threads.
- * @param stackSize The stack size for threads in this pool should
- * be this many bytes.
- * @param maxThreads Maximum number of threads in threadpool.
- * (0 == no limit).
- */
- FastOS_ThreadPool(int stackSize, int maxThreads=0);
+ FastOS_ThreadPool(int maxThreads);
+ /// Unlimited threads
+ FastOS_ThreadPool();
/**
* Destructor. Closes pool if necessary.
diff --git a/fnet/src/examples/ping/pingclient.cpp b/fnet/src/examples/ping/pingclient.cpp
index af3c9f68848..9b32e40ac83 100644
--- a/fnet/src/examples/ping/pingclient.cpp
+++ b/fnet/src/examples/ping/pingclient.cpp
@@ -28,7 +28,7 @@ PingClient::main(int argc, char **argv)
}
FNET_PacketQueue queue;
- FastOS_ThreadPool pool(65000);
+ FastOS_ThreadPool pool;
PingPacketFactory factory;
FNET_SimplePacketStreamer streamer(&factory);
FNET_Transport transport;
diff --git a/fnet/src/examples/timeout/timeout.cpp b/fnet/src/examples/timeout/timeout.cpp
index 70e9a1498c7..41de852d48c 100644
--- a/fnet/src/examples/timeout/timeout.cpp
+++ b/fnet/src/examples/timeout/timeout.cpp
@@ -54,7 +54,7 @@ MyApp::main(int, char **)
ms_double ms;
clock::time_point t;
FNET_PacketQueue queue;
- FastOS_ThreadPool pool(65000);
+ FastOS_ThreadPool pool;
FNET_Transport transport;
Timeout timeout(transport.GetScheduler(), &queue);
transport.Start(&pool);
diff --git a/fnet/src/tests/connect/connect_test.cpp b/fnet/src/tests/connect/connect_test.cpp
index 1a96da8cd5f..681c3b7676a 100644
--- a/fnet/src/tests/connect/connect_test.cpp
+++ b/fnet/src/tests/connect/connect_test.cpp
@@ -92,19 +92,19 @@ struct TransportFixture : FNET_IPacketHandler, FNET_IConnectionCleanupHandler {
FNET_Transport transport;
Gate conn_lost;
Gate conn_deleted;
- TransportFixture() : streamer(nullptr), pool(128_Ki), transport(),
+ TransportFixture() : streamer(nullptr), pool(), transport(),
conn_lost(), conn_deleted()
{
transport.Start(&pool);
}
TransportFixture(AsyncResolver::HostResolver::SP host_resolver)
- : streamer(nullptr), pool(128_Ki), transport(fnet::TransportConfig().resolver(make_resolver(std::move(host_resolver)))),
+ : streamer(nullptr), pool(), transport(fnet::TransportConfig().resolver(make_resolver(std::move(host_resolver)))),
conn_lost(), conn_deleted()
{
transport.Start(&pool);
}
TransportFixture(CryptoEngine::SP crypto)
- : streamer(nullptr), pool(128_Ki), transport(fnet::TransportConfig().crypto(std::move(crypto))),
+ : streamer(nullptr), pool(), transport(fnet::TransportConfig().crypto(std::move(crypto))),
conn_lost(), conn_deleted()
{
transport.Start(&pool);
diff --git a/fnet/src/tests/connection_spread/connection_spread_test.cpp b/fnet/src/tests/connection_spread/connection_spread_test.cpp
index 0d9c0cc6e18..d65e4fb70fe 100644
--- a/fnet/src/tests/connection_spread/connection_spread_test.cpp
+++ b/fnet/src/tests/connection_spread/connection_spread_test.cpp
@@ -31,7 +31,7 @@ struct Fixture {
FastOS_ThreadPool thread_pool;
FNET_Transport client;
FNET_Transport server;
- Fixture() : streamer(), adapter(), thread_pool(128_Ki), client(8), server(8)
+ Fixture() : streamer(), adapter(), thread_pool(), client(8), server(8)
{
ASSERT_TRUE(client.Start(&thread_pool));
ASSERT_TRUE(server.Start(&thread_pool));
diff --git a/fnet/src/tests/frt/detach_supervisor/detach_supervisor_test.cpp b/fnet/src/tests/frt/detach_supervisor/detach_supervisor_test.cpp
index 3207094e56d..716c433ff61 100644
--- a/fnet/src/tests/frt/detach_supervisor/detach_supervisor_test.cpp
+++ b/fnet/src/tests/frt/detach_supervisor/detach_supervisor_test.cpp
@@ -21,7 +21,7 @@ CryptoEngine::SP null_crypto = std::make_shared<NullCryptoEngine>();
struct BasicFixture {
FastOS_ThreadPool thread_pool;
FNET_Transport transport;
- BasicFixture() : thread_pool(128_Ki), transport(fnet::TransportConfig(4).crypto(null_crypto)) {
+ BasicFixture() : thread_pool(), transport(fnet::TransportConfig(4).crypto(null_crypto)) {
ASSERT_TRUE(transport.Start(&thread_pool));
}
~BasicFixture() {
diff --git a/fnet/src/tests/frt/parallel_rpc/parallel_rpc_test.cpp b/fnet/src/tests/frt/parallel_rpc/parallel_rpc_test.cpp
index 74f1dea0c9a..c6008820bbc 100644
--- a/fnet/src/tests/frt/parallel_rpc/parallel_rpc_test.cpp
+++ b/fnet/src/tests/frt/parallel_rpc/parallel_rpc_test.cpp
@@ -19,7 +19,7 @@ struct Rpc : FRT_Invokable {
FNET_Transport transport;
FRT_Supervisor orb;
Rpc(CryptoEngine::SP crypto, size_t num_threads, bool drop_empty)
- : thread_pool(128_Ki), transport(fnet::TransportConfig(num_threads).crypto(std::move(crypto)).drop_empty_buffers(drop_empty)), orb(&transport) {}
+ : thread_pool(), transport(fnet::TransportConfig(num_threads).crypto(std::move(crypto)).drop_empty_buffers(drop_empty)), orb(&transport) {}
void start() {
ASSERT_TRUE(transport.Start(&thread_pool));
}
diff --git a/fnet/src/tests/sync_execute/sync_execute.cpp b/fnet/src/tests/sync_execute/sync_execute.cpp
index fac247d8ebd..b8fa21cf147 100644
--- a/fnet/src/tests/sync_execute/sync_execute.cpp
+++ b/fnet/src/tests/sync_execute/sync_execute.cpp
@@ -18,7 +18,7 @@ TEST("sync execute") {
DoIt exe2;
DoIt exe3;
DoIt exe4;
- FastOS_ThreadPool pool(128_Ki);
+ FastOS_ThreadPool pool;
FNET_Transport transport;
ASSERT_TRUE(transport.execute(&exe1));
ASSERT_TRUE(transport.Start(&pool));
diff --git a/fnet/src/vespa/fnet/frt/supervisor.cpp b/fnet/src/vespa/fnet/frt/supervisor.cpp
index 1681321b239..6ba9ff8ad77 100644
--- a/fnet/src/vespa/fnet/frt/supervisor.cpp
+++ b/fnet/src/vespa/fnet/frt/supervisor.cpp
@@ -291,7 +291,7 @@ FRT_Supervisor::SchedulerPtr::SchedulerPtr(FNET_TransportThread *transport_threa
namespace fnet::frt {
StandaloneFRT::StandaloneFRT(const TransportConfig &config)
- : _threadPool(std::make_unique<FastOS_ThreadPool>(1024*128)),
+ : _threadPool(std::make_unique<FastOS_ThreadPool>()),
_transport(std::make_unique<FNET_Transport>(config)),
_supervisor(std::make_unique<FRT_Supervisor>(_transport.get()))
{
diff --git a/fnet/src/vespa/fnet/transport.cpp b/fnet/src/vespa/fnet/transport.cpp
index 1130b6d3e5e..ae864ea0821 100644
--- a/fnet/src/vespa/fnet/transport.cpp
+++ b/fnet/src/vespa/fnet/transport.cpp
@@ -134,7 +134,7 @@ FNET_Transport::FNET_Transport(const fnet::TransportConfig &cfg)
: _async_resolver(cfg.resolver()),
_crypto_engine(cfg.crypto()),
_time_tools(cfg.time_tools()),
- _work_pool(std::make_unique<vespalib::ThreadStackExecutor>(1, 128_Ki, fnet_work_pool, 1024)),
+ _work_pool(std::make_unique<vespalib::ThreadStackExecutor>(1, fnet_work_pool, 1024)),
_threads(),
_config(cfg.config())
{
diff --git a/logd/src/tests/watcher/watcher_test.cpp b/logd/src/tests/watcher/watcher_test.cpp
index b9199315c76..55ba720c0cb 100644
--- a/logd/src/tests/watcher/watcher_test.cpp
+++ b/logd/src/tests/watcher/watcher_test.cpp
@@ -129,7 +129,7 @@ public:
};
WatcherTest::WatcherTest()
- : _executor(1, 256_Ki)
+ : _executor(1)
{
remove_files();
setenv("VESPA_LOG_TARGET", "file:vespa.log", true);
diff --git a/messagebus/src/vespa/messagebus/messenger.cpp b/messagebus/src/vespa/messagebus/messenger.cpp
index 1423876e95b..056be51609f 100644
--- a/messagebus/src/vespa/messagebus/messenger.cpp
+++ b/messagebus/src/vespa/messagebus/messenger.cpp
@@ -157,7 +157,7 @@ namespace mbus {
Messenger::Messenger()
: _lock(),
- _pool(128000),
+ _pool(),
_children(),
_queue(),
_closed(false)
diff --git a/messagebus/src/vespa/messagebus/network/rpcnetwork.cpp b/messagebus/src/vespa/messagebus/network/rpcnetwork.cpp
index 6716ee5fe24..bc3ba205b69 100644
--- a/messagebus/src/vespa/messagebus/network/rpcnetwork.cpp
+++ b/messagebus/src/vespa/messagebus/network/rpcnetwork.cpp
@@ -126,7 +126,7 @@ RPCNetwork::SendContext::handleVersion(const vespalib::Version *version)
RPCNetwork::RPCNetwork(const RPCNetworkParams &params) :
_owner(nullptr),
_ident(params.getIdentity()),
- _threadPool(std::make_unique<FastOS_ThreadPool>(128_Ki, 0)),
+ _threadPool(std::make_unique<FastOS_ThreadPool>()),
_transport(std::make_unique<FNET_Transport>(toFNETConfig(params))),
_orb(std::make_unique<FRT_Supervisor>(_transport.get())),
_scheduler(*_transport->GetScheduler()),
diff --git a/messagebus/src/vespa/messagebus/testlib/slobrok.cpp b/messagebus/src/vespa/messagebus/testlib/slobrok.cpp
index 3260a99678d..bcf6f829605 100644
--- a/messagebus/src/vespa/messagebus/testlib/slobrok.cpp
+++ b/messagebus/src/vespa/messagebus/testlib/slobrok.cpp
@@ -67,7 +67,7 @@ Slobrok::init()
}
Slobrok::Slobrok()
- : _pool(128000, 0),
+ : _pool(),
_env(),
_port(0),
_thread()
@@ -76,7 +76,7 @@ Slobrok::Slobrok()
}
Slobrok::Slobrok(int p)
- : _pool(128000, 0),
+ : _pool(),
_env(),
_port(p),
_thread()
diff --git a/metrics/src/tests/metricmanagertest.cpp b/metrics/src/tests/metricmanagertest.cpp
index 6b69569f499..928ff4270cb 100644
--- a/metrics/src/tests/metricmanagertest.cpp
+++ b/metrics/src/tests/metricmanagertest.cpp
@@ -152,7 +152,7 @@ namespace {
std::pair<std::string, std::string>
getMatchedMetrics(const vespalib::string& config)
{
- FastOS_ThreadPool pool(256_Ki);
+ FastOS_ThreadPool pool;
TestMetricSet mySet;
MetricManager mm;
mm.registerMetric(mm.getMetricLock(), mySet.set);
@@ -475,7 +475,7 @@ std::string dumpAllSnapshots(const MetricManager& mm,
TEST_F(MetricManagerTest, test_snapshots)
{
- FastOS_ThreadPool pool(256_Ki);
+ FastOS_ThreadPool pool;
FakeTimer* timer = new FakeTimer(1000);
std::unique_ptr<MetricManager::Timer> timerImpl(timer);
TestMetricSet mySet;
@@ -575,7 +575,7 @@ TEST_F(MetricManagerTest, test_snapshots)
TEST_F(MetricManagerTest, test_xml_output)
{
- FastOS_ThreadPool pool(256_Ki);
+ FastOS_ThreadPool pool;
FakeTimer* timer = new FakeTimer(1000);
std::unique_ptr<MetricManager::Timer> timerImpl(timer);
MetricManager mm(std::move(timerImpl));
@@ -653,7 +653,7 @@ TEST_F(MetricManagerTest, test_xml_output)
TEST_F(MetricManagerTest, test_json_output)
{
- FastOS_ThreadPool pool(256_Ki);
+ FastOS_ThreadPool pool;
FakeTimer* timer = new FakeTimer(1000);
std::unique_ptr<MetricManager::Timer> timerImpl(timer);
MetricManager mm(std::move(timerImpl));
@@ -752,7 +752,7 @@ struct MetricSnapshotTestFixture
MetricSnapshotTestFixture(MetricManagerTest& callerTest, MetricSet& metricSet)
: test(callerTest),
- pool(DEFAULT_THREAD_STACK_SIZE),
+ pool(),
timer(new FakeTimer(1000)),
manager(std::unique_ptr<MetricManager::Timer>(timer)),
mset(metricSet)
@@ -981,7 +981,7 @@ TEST_F(MetricManagerTest, json_output_can_have_multiple_sets_with_same_name)
TEST_F(MetricManagerTest, test_text_output)
{
- FastOS_ThreadPool pool(256_Ki);
+ FastOS_ThreadPool pool;
FakeTimer* timer = new FakeTimer(1000);
std::unique_ptr<MetricManager::Timer> timerImpl(timer);
MetricManager mm(std::move(timerImpl));
@@ -1080,7 +1080,7 @@ TEST_F(MetricManagerTest, test_update_hooks)
{
std::mutex output_mutex;
std::ostringstream output;
- FastOS_ThreadPool pool(256_Ki);
+ FastOS_ThreadPool pool;
FakeTimer* timer = new FakeTimer(1000);
std::unique_ptr<MetricManager::Timer> timerImpl(timer);
// Add a metric set just so one exist
diff --git a/metrics/src/tests/snapshottest.cpp b/metrics/src/tests/snapshottest.cpp
index 46b3b217c26..22eb3587eff 100644
--- a/metrics/src/tests/snapshottest.cpp
+++ b/metrics/src/tests/snapshottest.cpp
@@ -176,7 +176,7 @@ TEST_F(SnapshotTest, test_snapshot_two_days)
TestMetricSet set("test");
FakeTimer* timer;
- FastOS_ThreadPool threadPool(256_Ki);
+ FastOS_ThreadPool threadPool;
MetricManager mm(
std::unique_ptr<MetricManager::Timer>(timer = new FakeTimer));
{
diff --git a/metrics/src/tests/stresstest.cpp b/metrics/src/tests/stresstest.cpp
index 70b9a62c983..e942d47b9de 100644
--- a/metrics/src/tests/stresstest.cpp
+++ b/metrics/src/tests/stresstest.cpp
@@ -114,7 +114,7 @@ TEST(StressTest, test_stress)
OuterMetricSet metrics;
LOG(info, "Starting load givers");
- FastOS_ThreadPool threadPool(256_Ki);
+ FastOS_ThreadPool threadPool;
std::vector<Hammer::UP> hammers;
for (uint32_t i=0; i<10; ++i) {
hammers.push_back(std::make_unique<Hammer>(metrics, threadPool));
diff --git a/persistence/src/vespa/persistence/dummyimpl/dummy_bucket_executor.cpp b/persistence/src/vespa/persistence/dummyimpl/dummy_bucket_executor.cpp
index 953cfcf733f..0237723d9f3 100644
--- a/persistence/src/vespa/persistence/dummyimpl/dummy_bucket_executor.cpp
+++ b/persistence/src/vespa/persistence/dummyimpl/dummy_bucket_executor.cpp
@@ -11,7 +11,7 @@ using vespalib::makeSharedLambdaCallback;
namespace storage::spi::dummy {
DummyBucketExecutor::DummyBucketExecutor(size_t numExecutors)
- : _executor(std::make_unique<vespalib::ThreadStackExecutor>(numExecutors, 0x10000)),
+ : _executor(std::make_unique<vespalib::ThreadStackExecutor>(numExecutors)),
_lock(),
_cond(),
_inFlight(),
diff --git a/searchcore/src/apps/proton/proton.cpp b/searchcore/src/apps/proton/proton.cpp
index 60b64dd9ad8..8468487995a 100644
--- a/searchcore/src/apps/proton/proton.cpp
+++ b/searchcore/src/apps/proton/proton.cpp
@@ -283,7 +283,7 @@ App::main(int argc, char **argv)
try {
setupSignals();
setup_fadvise();
- FastOS_ThreadPool threadPool(128_Ki);
+ FastOS_ThreadPool threadPool;
Transport transport(buildTransportConfig(), threadPool);
startAndRun(threadPool, transport.transport(), argc, argv);
} catch (const vespalib::InvalidCommandLineArgumentsException &e) {
diff --git a/searchcore/src/apps/tests/persistenceconformance_test.cpp b/searchcore/src/apps/tests/persistenceconformance_test.cpp
index 52e2a314fd2..fad09101d95 100644
--- a/searchcore/src/apps/tests/persistenceconformance_test.cpp
+++ b/searchcore/src/apps/tests/persistenceconformance_test.cpp
@@ -34,7 +34,6 @@
#include <vespa/document/config/documenttypes_config_fwd.h>
#include <vespa/document/repo/documenttyperepo.h>
#include <vespa/document/test/make_bucket_space.h>
-#include <vespa/vespalib/util/size_literals.h>
#include <filesystem>
#include <vespa/log/log.h>
@@ -203,7 +202,7 @@ public:
_shared_service, _tls, _metricsWireService,
_fileHeaderContext, std::make_shared<search::attribute::Interlock>(),
_config_stores.getConfigStore(docType.toString()),
- std::make_shared<vespalib::ThreadStackExecutor>(16, 128_Ki), HwInfo());
+ std::make_shared<vespalib::ThreadStackExecutor>(16), HwInfo());
}
};
@@ -214,7 +213,7 @@ DocumentDBFactory::DocumentDBFactory(const vespalib::string &baseDir, int tlsLis
_tlsSpec(vespalib::make_string("tcp/localhost:%d", tlsListenPort)),
_queryLimiter(),
_metricsWireService(),
- _summaryExecutor(8, 128_Ki),
+ _summaryExecutor(8),
_shared_service(_summaryExecutor, _summaryExecutor),
_tls(_shared_service.transport(), "tls", tlsListenPort, baseDir, _fileHeaderContext)
{}
diff --git a/searchcore/src/apps/vespa-feed-bm/vespa_feed_bm.cpp b/searchcore/src/apps/vespa-feed-bm/vespa_feed_bm.cpp
index 95c63f56bac..8777f5c2410 100644
--- a/searchcore/src/apps/vespa-feed-bm/vespa_feed_bm.cpp
+++ b/searchcore/src/apps/vespa-feed-bm/vespa_feed_bm.cpp
@@ -158,7 +158,7 @@ void
Benchmark::run()
{
_cluster->start(_feed);
- vespalib::ThreadStackExecutor executor(_params.get_client_threads(), 128_Ki);
+ vespalib::ThreadStackExecutor executor(_params.get_client_threads());
BmFeeder feeder(_repo, *_cluster->get_feed_handler(), executor);
auto put_feed = _feed.make_feed(executor, _params, [this](BmRange range, BucketSelector bucket_selector) { return _feed.make_put_feed(range, bucket_selector); }, _feed.num_buckets(), "put");
auto update_feed = _feed.make_feed(executor, _params, [this](BmRange range, BucketSelector bucket_selector) { return _feed.make_update_feed(range, bucket_selector); }, _feed.num_buckets(), "update");
diff --git a/searchcore/src/apps/vespa-redistribute-bm/vespa_redistribute_bm.cpp b/searchcore/src/apps/vespa-redistribute-bm/vespa_redistribute_bm.cpp
index 7d190a81087..4a408edcdd9 100644
--- a/searchcore/src/apps/vespa-redistribute-bm/vespa_redistribute_bm.cpp
+++ b/searchcore/src/apps/vespa-redistribute-bm/vespa_redistribute_bm.cpp
@@ -24,7 +24,6 @@
#include <vespa/searchlib/index/dummyfileheadercontext.h>
#include <vespa/vespalib/objects/nbostream.h>
#include <vespa/vespalib/util/lambdatask.h>
-#include <vespa/vespalib/util/size_literals.h>
#include <vespa/vespalib/util/threadstackexecutor.h>
#include <getopt.h>
#include <filesystem>
@@ -209,16 +208,15 @@ public:
};
ReFeed::ReFeed(const BMParams& params, std::shared_ptr<const DocumentTypeRepo> repo, IBmFeedHandler& feed_handler, int64_t& time_bias, const std::vector<vespalib::nbostream>& feed, const vespalib::string& op_name)
- : _top_executor(1, 128_Ki),
- _executor(params.get_client_threads(), 128_Ki),
+ : _top_executor(1),
+ _executor(params.get_client_threads()),
_feeder(repo, feed_handler, _executor),
_op_name(op_name),
_params(params),
_time_bias(time_bias),
_feed(feed)
{
- _top_executor.execute(makeLambdaTask([this]()
- { run(); }));
+ _top_executor.execute(makeLambdaTask([this]() { run(); }));
}
ReFeed::~ReFeed()
@@ -355,7 +353,7 @@ Benchmark::adjust_cluster_state_after_first_redistribution()
void
Benchmark::make_feed()
{
- vespalib::ThreadStackExecutor executor(_params.get_client_threads(), 128_Ki);
+ vespalib::ThreadStackExecutor executor(_params.get_client_threads());
_put_feed = _feed.make_feed(executor, _params, [this](BmRange range, BucketSelector bucket_selector) { return _feed.make_put_feed(range, bucket_selector); }, _feed.num_buckets(), "put");
if (_params.get_refeed_mode() == ReFeedMode::UPDATE) {
_update_feed = _feed.make_feed(executor, _params, [this](BmRange range, BucketSelector bucket_selector) { return _feed.make_update_feed(range, bucket_selector); }, _feed.num_buckets(), "update");
@@ -365,7 +363,7 @@ Benchmark::make_feed()
void
Benchmark::feed()
{
- vespalib::ThreadStackExecutor executor(_params.get_client_threads(), 128_Ki);
+ vespalib::ThreadStackExecutor executor(_params.get_client_threads());
BmFeeder feeder(_repo, *_cluster->get_feed_handler(), executor);
BmNodeStatsReporter reporter(*_cluster, false);
reporter.start(500ms);
diff --git a/searchcore/src/apps/vespa-transactionlog-inspect/vespa-transactionlog-inspect.cpp b/searchcore/src/apps/vespa-transactionlog-inspect/vespa-transactionlog-inspect.cpp
index 711191125da..749ca2fab80 100644
--- a/searchcore/src/apps/vespa-transactionlog-inspect/vespa-transactionlog-inspect.cpp
+++ b/searchcore/src/apps/vespa-transactionlog-inspect/vespa-transactionlog-inspect.cpp
@@ -387,7 +387,7 @@ public:
BaseUtility(const BaseOptions &bopts)
: _bopts(bopts),
_fileHeader(),
- _threadPool(64_Ki),
+ _threadPool(),
_transport(),
_server(_transport, _bopts.tlsName, _bopts.listenPort, _bopts.tlsDir, _fileHeader),
_client(_transport, vespalib::make_string("tcp/localhost:%d", _bopts.listenPort))
diff --git a/searchcore/src/tests/proton/attribute/attribute_initializer/attribute_initializer_test.cpp b/searchcore/src/tests/proton/attribute/attribute_initializer/attribute_initializer_test.cpp
index 3016815a89c..106bdf5cc98 100644
--- a/searchcore/src/tests/proton/attribute/attribute_initializer/attribute_initializer_test.cpp
+++ b/searchcore/src/tests/proton/attribute/attribute_initializer/attribute_initializer_test.cpp
@@ -102,7 +102,7 @@ Fixture::Fixture()
: _dirHandler(test_dir),
_diskLayout(AttributeDiskLayout::create(test_dir)),
_factory(),
- _executor(1, 0x10000)
+ _executor(1)
{
}
diff --git a/searchcore/src/tests/proton/attribute/attribute_manager/attribute_manager_test.cpp b/searchcore/src/tests/proton/attribute/attribute_manager/attribute_manager_test.cpp
index a66785cb567..8f409b3e31e 100644
--- a/searchcore/src/tests/proton/attribute/attribute_manager/attribute_manager_test.cpp
+++ b/searchcore/src/tests/proton/attribute/attribute_manager/attribute_manager_test.cpp
@@ -253,7 +253,7 @@ ParallelAttributeManager::ParallelAttributeManager(search::SerialNum configSeria
alloc_strategy(),
fastAccessAttributesOnly(false),
mgr(std::make_shared<AttributeManager::SP>()),
- masterExecutor(1, 128_Ki),
+ masterExecutor(1),
master(masterExecutor),
initializer(std::make_shared<AttributeManagerInitializer>(configSerialNum, documentMetaStoreInitTask,
documentMetaStore, *baseAttrMgr, attrCfg,
@@ -261,7 +261,7 @@ ParallelAttributeManager::ParallelAttributeManager(search::SerialNum configSeria
fastAccessAttributesOnly, master, mgr))
{
documentMetaStore->setCommittedDocIdLimit(docIdLimit);
- vespalib::ThreadStackExecutor executor(3, 128_Ki);
+ vespalib::ThreadStackExecutor executor(3);
initializer::TaskRunner taskRunner(executor);
taskRunner.runTask(initializer);
}
diff --git a/searchcore/src/tests/proton/attribute/attributeflush_test.cpp b/searchcore/src/tests/proton/attribute/attributeflush_test.cpp
index 6d581f6e6f9..7249476a907 100644
--- a/searchcore/src/tests/proton/attribute/attributeflush_test.cpp
+++ b/searchcore/src/tests/proton/attribute/attributeflush_test.cpp
@@ -81,7 +81,7 @@ public:
GateSP gate;
FlushHandler()
- : _executor(1, 64_Ki),
+ : _executor(1),
gate()
{ }
~FlushHandler();
@@ -486,7 +486,7 @@ Test::requireThatOnlyOneFlusherCanRunAtTheSameTime()
av->commit(CommitParam(9));
IFlushTarget::SP ft = am.getFlushable("a8");
(static_cast<FlushableAttribute *>(ft.get()))->setCleanUpAfterFlush(false);
- vespalib::ThreadStackExecutor exec(16, 64000);
+ vespalib::ThreadStackExecutor exec(16);
for (size_t i = 10; i < 100; ++i) {
av->commit(CommitParam(i));
@@ -586,7 +586,7 @@ Test::requireThatShrinkWorks()
EXPECT_EQUAL(1000u, av->getNumDocs());
EXPECT_EQUAL(100u, av->getCommittedDocIdLimit());
EXPECT_EQUAL(createSerialNum - 1, ft->getFlushedSerialNum());
- vespalib::ThreadStackExecutor exec(1, 128_Ki);
+ vespalib::ThreadStackExecutor exec(1);
vespalib::Executor::Task::UP task = ft->initFlush(53, std::make_shared<search::FlushToken>());
exec.execute(std::move(task));
exec.sync();
diff --git a/searchcore/src/tests/proton/common/timer/timer_test.cpp b/searchcore/src/tests/proton/common/timer/timer_test.cpp
index 18699554266..ac82767cd7c 100644
--- a/searchcore/src/tests/proton/common/timer/timer_test.cpp
+++ b/searchcore/src/tests/proton/common/timer/timer_test.cpp
@@ -52,9 +52,9 @@ public:
std::unique_ptr<ScheduledT> timer;
ScheduledExecutorTest()
- : threadPool(64_Ki),
+ : threadPool(),
transport(),
- executor(1, 64_Ki)
+ executor(1)
{
transport.Start(&threadPool);
timer = make_scheduled_executor<ScheduledT>(transport, executor);
diff --git a/searchcore/src/tests/proton/docsummary/docsummary.cpp b/searchcore/src/tests/proton/docsummary/docsummary.cpp
index 9709530f2c2..9bd3d6dbec5 100644
--- a/searchcore/src/tests/proton/docsummary/docsummary.cpp
+++ b/searchcore/src/tests/proton/docsummary/docsummary.cpp
@@ -158,7 +158,7 @@ BuildContext::BuildContext(AddFieldsType add_fields)
: DocBuilder(add_fields),
_dmk("summary"),
_fixed_repo(get_repo(), get_document_type()),
- _summaryExecutor(4, 128_Ki),
+ _summaryExecutor(4),
_noTlSyncer(),
_str(_summaryExecutor, "summary",
LogDocumentStore::Config(
@@ -242,7 +242,7 @@ public:
DBContext(std::shared_ptr<const DocumentTypeRepo> repo, const char *docTypeName)
: _dmk(docTypeName),
_fileHeaderContext(),
- _summaryExecutor(8, 128_Ki),
+ _summaryExecutor(8),
_shared_service(_summaryExecutor, _summaryExecutor),
_tls(_shared_service.transport(), "tmp", 9013, ".", _fileHeaderContext),
_made_dir(std::filesystem::create_directory(std::filesystem::path("tmpdb"))),
@@ -272,7 +272,7 @@ public:
_shared_service, _tls, _dummy, _fileHeaderContext,
std::make_shared<search::attribute::Interlock>(),
std::make_unique<MemoryConfigStore>(),
- std::make_shared<vespalib::ThreadStackExecutor>(16, 128_Ki), _hwInfo),
+ std::make_shared<vespalib::ThreadStackExecutor>(16), _hwInfo),
_ddb->start();
_ddb->waitForOnlineState();
_aw = std::make_unique<AttributeWriter>(_ddb->getReadySubDB()->getAttributeManager());
diff --git a/searchcore/src/tests/proton/documentdb/buckethandler/buckethandler_test.cpp b/searchcore/src/tests/proton/documentdb/buckethandler/buckethandler_test.cpp
index cadfa8cd72f..dbc0b890cba 100644
--- a/searchcore/src/tests/proton/documentdb/buckethandler/buckethandler_test.cpp
+++ b/searchcore/src/tests/proton/documentdb/buckethandler/buckethandler_test.cpp
@@ -103,7 +103,7 @@ struct Fixture
_ready(_bucketDB, SubDbType::READY),
_removed(_bucketDB, SubDbType::REMOVED),
_notReady(_bucketDB, SubDbType::NOTREADY),
- _exec(1, 64000),
+ _exec(1),
_handler(_exec),
_changedHandler(),
_calc(new BucketStateCalculator()),
diff --git a/searchcore/src/tests/proton/documentdb/clusterstatehandler/clusterstatehandler_test.cpp b/searchcore/src/tests/proton/documentdb/clusterstatehandler/clusterstatehandler_test.cpp
index 8d012afdb1f..881852eb7a3 100644
--- a/searchcore/src/tests/proton/documentdb/clusterstatehandler/clusterstatehandler_test.cpp
+++ b/searchcore/src/tests/proton/documentdb/clusterstatehandler/clusterstatehandler_test.cpp
@@ -43,7 +43,7 @@ struct Fixture
test::GenericResultHandler _genericHandler;
test::BucketIdListResultHandler _bucketListHandler;
Fixture()
- : _exec(1, 64000),
+ : _exec(1),
_stateHandler(_exec),
_changedHandler(),
_genericHandler(),
diff --git a/searchcore/src/tests/proton/documentdb/configurer/configurer_test.cpp b/searchcore/src/tests/proton/documentdb/configurer/configurer_test.cpp
index 600e4981c0f..0bcbf1269a1 100644
--- a/searchcore/src/tests/proton/documentdb/configurer/configurer_test.cpp
+++ b/searchcore/src/tests/proton/documentdb/configurer/configurer_test.cpp
@@ -169,7 +169,7 @@ Fixture::Fixture()
_queryLimiter(),
_constantValueFactory(),
_constantValueRepo(_constantValueFactory),
- _summaryExecutor(8, 128_Ki),
+ _summaryExecutor(8),
_pendingLidsForCommit(std::make_shared<PendingLidTracker>()),
_sessionMgr(100),
_views(),
diff --git a/searchcore/src/tests/proton/documentdb/document_subdbs/document_subdbs_test.cpp b/searchcore/src/tests/proton/documentdb/document_subdbs/document_subdbs_test.cpp
index ac2e9279227..13cc4030262 100644
--- a/searchcore/src/tests/proton/documentdb/document_subdbs/document_subdbs_test.cpp
+++ b/searchcore/src/tests/proton/documentdb/document_subdbs/document_subdbs_test.cpp
@@ -359,7 +359,7 @@ struct FixtureBase
void init() {
DocumentSubDbInitializer::SP task =
_subDb.createInitializer(*_snapshot->_cfg, Traits::configSerial(), IndexConfig());
- vespalib::ThreadStackExecutor executor(1, 1_Mi);
+ vespalib::ThreadStackExecutor executor(1);
initializer::TaskRunner taskRunner(executor);
taskRunner.runTask(task);
runInMasterAndSync([&]() { _subDb.initViews(*_snapshot->_cfg); });
diff --git a/searchcore/src/tests/proton/documentdb/documentbucketmover/documentbucketmover_test.cpp b/searchcore/src/tests/proton/documentdb/documentbucketmover/documentbucketmover_test.cpp
index 44075de4179..f5a91e9838e 100644
--- a/searchcore/src/tests/proton/documentdb/documentbucketmover/documentbucketmover_test.cpp
+++ b/searchcore/src/tests/proton/documentdb/documentbucketmover/documentbucketmover_test.cpp
@@ -123,7 +123,7 @@ ControllerFixtureBase::ControllerFixtureBase(const BlockableMaintenanceJobConfig
_bucketCreateNotifier(),
_diskMemUsageNotifier(),
_refCount(),
- _singleExecutor(1, 0x10000),
+ _singleExecutor(1),
_master(_singleExecutor),
_bucketExecutor(4),
_moveHandler(*_bucketDB, storeMoveDoneContexts),
diff --git a/searchcore/src/tests/proton/documentdb/documentdb_test.cpp b/searchcore/src/tests/proton/documentdb/documentdb_test.cpp
index cdf874a6601..47cbde152ef 100644
--- a/searchcore/src/tests/proton/documentdb/documentdb_test.cpp
+++ b/searchcore/src/tests/proton/documentdb/documentdb_test.cpp
@@ -128,7 +128,7 @@ Fixture::Fixture(bool file_config)
: FixtureBase(file_config),
_dummy(),
_myDBOwner(),
- _summaryExecutor(8, 128_Ki),
+ _summaryExecutor(8),
_shared_service(_summaryExecutor, _summaryExecutor),
_hwInfo(),
_db(),
@@ -155,7 +155,7 @@ Fixture::Fixture(bool file_config)
_fileHeaderContext,
std::make_shared<search::attribute::Interlock>(),
make_config_store(),
- std::make_shared<vespalib::ThreadStackExecutor>(16, 128_Ki), _hwInfo);
+ std::make_shared<vespalib::ThreadStackExecutor>(16), _hwInfo);
_db->start();
_db->waitForOnlineState();
}
diff --git a/searchcore/src/tests/proton/documentdb/job_tracked_maintenance_job/job_tracked_maintenance_job_test.cpp b/searchcore/src/tests/proton/documentdb/job_tracked_maintenance_job/job_tracked_maintenance_job_test.cpp
index 0cf00a79650..3b08cac3049 100644
--- a/searchcore/src/tests/proton/documentdb/job_tracked_maintenance_job/job_tracked_maintenance_job_test.cpp
+++ b/searchcore/src/tests/proton/documentdb/job_tracked_maintenance_job/job_tracked_maintenance_job_test.cpp
@@ -70,7 +70,7 @@ struct Fixture
_runRetval(false),
_runGates(getGateVector(numRuns)),
_runIdx(0),
- _exec(1, 64000)
+ _exec(1)
{
}
void runJob() {
diff --git a/searchcore/src/tests/proton/documentdb/lid_space_compaction/lid_space_jobtest.cpp b/searchcore/src/tests/proton/documentdb/lid_space_compaction/lid_space_jobtest.cpp
index b941161cc35..150cc7f2d95 100644
--- a/searchcore/src/tests/proton/documentdb/lid_space_compaction/lid_space_jobtest.cpp
+++ b/searchcore/src/tests/proton/documentdb/lid_space_compaction/lid_space_jobtest.cpp
@@ -53,7 +53,7 @@ JobTestBase::init(uint32_t allowedLidBloat,
BlockableMaintenanceJobConfig blockableCfg(resourceLimitFactor, maxOutstandingMoveOps);
_job.reset();
- _singleExecutor = std::make_unique<vespalib::ThreadStackExecutor>(1, 0x10000);
+ _singleExecutor = std::make_unique<vespalib::ThreadStackExecutor>(1);
_master = std::make_unique<proton::SyncableExecutorThreadService> (*_singleExecutor);
_bucketExecutor = std::make_unique<storage::spi::dummy::DummyBucketExecutor>(4);
_job = lidspace::CompactionJob::create(compactCfg, RetainGuard(_refCount), _handler, _storer, *_master, *_bucketExecutor,
diff --git a/searchcore/src/tests/proton/documentdb/maintenancecontroller/maintenancecontroller_test.cpp b/searchcore/src/tests/proton/documentdb/maintenancecontroller/maintenancecontroller_test.cpp
index debad7eaf26..95f03558231 100644
--- a/searchcore/src/tests/proton/documentdb/maintenancecontroller/maintenancecontroller_test.cpp
+++ b/searchcore/src/tests/proton/documentdb/maintenancecontroller/maintenancecontroller_test.cpp
@@ -680,7 +680,7 @@ MyFeedHandler::appendOperation(const FeedOperation &op, DoneCallback)
}
MyExecutor::MyExecutor()
- : vespalib::ThreadStackExecutorBase(128_Ki, -1, my_executor_init),
+ : vespalib::ThreadStackExecutorBase(-1, my_executor_init),
_threadId()
{
start(1);
diff --git a/searchcore/src/tests/proton/documentmetastore/documentmetastore_test.cpp b/searchcore/src/tests/proton/documentmetastore/documentmetastore_test.cpp
index 4668b8c65ab..04dfdb7f7ff 100644
--- a/searchcore/src/tests/proton/documentmetastore/documentmetastore_test.cpp
+++ b/searchcore/src/tests/proton/documentmetastore/documentmetastore_test.cpp
@@ -1807,7 +1807,7 @@ TEST(DocumentMetaStoreTest, shrink_via_flush_target_works)
EXPECT_TRUE(ft->getApproxMemoryGain().getBefore() >
ft->getApproxMemoryGain().getAfter());
- vespalib::ThreadStackExecutor exec(1, 128_Ki);
+ vespalib::ThreadStackExecutor exec(1);
vespalib::Executor::Task::UP task = ft->initFlush(11, std::make_shared<search::FlushToken>());
exec.execute(std::move(task));
exec.sync();
diff --git a/searchcore/src/tests/proton/feed_and_search/feed_and_search.cpp b/searchcore/src/tests/proton/feed_and_search/feed_and_search.cpp
index fef4e80a355..47800db91e9 100644
--- a/searchcore/src/tests/proton/feed_and_search/feed_and_search.cpp
+++ b/searchcore/src/tests/proton/feed_and_search/feed_and_search.cpp
@@ -160,7 +160,7 @@ VESPA_THREAD_STACK_TAG(write_executor)
// searches, dumps the index to disk, and performs the searches
// again.
void Test::requireThatMemoryIndexCanBeDumpedAndSearched() {
- vespalib::ThreadStackExecutor sharedExecutor(2, 0x10000);
+ vespalib::ThreadStackExecutor sharedExecutor(2);
auto indexFieldInverter = vespalib::SequencedTaskExecutor::create(invert_executor, 2);
auto indexFieldWriter = vespalib::SequencedTaskExecutor::create(write_executor, 2);
DocBuilder doc_builder([](auto& header) { header.addField(field_name, DataType::T_STRING); });
diff --git a/searchcore/src/tests/proton/index/indexcollection_test.cpp b/searchcore/src/tests/proton/index/indexcollection_test.cpp
index c2707a9eecb..f4608124b26 100644
--- a/searchcore/src/tests/proton/index/indexcollection_test.cpp
+++ b/searchcore/src/tests/proton/index/indexcollection_test.cpp
@@ -5,7 +5,6 @@
#include <vespa/searchlib/query/tree/simplequery.h>
#include <vespa/searchcorespi/index/warmupindexcollection.h>
#include <vespa/vespalib/gtest/gtest.h>
-#include <vespa/vespalib/util/size_literals.h>
#include <vespa/vespalib/util/threadstackexecutor.h>
#include <vespa/vespalib/util/testclock.h>
#include <vespa/log/log.h>
@@ -97,7 +96,7 @@ public:
_source1(std::make_shared<MockIndexSearchable>(FieldLengthInfo(3, 5))),
_source2(std::make_shared<MockIndexSearchable>(FieldLengthInfo(7, 11))),
_fusion_source(std::make_shared<FakeIndexSearchable>()),
- _executor(1, 128_Ki),
+ _executor(1),
_warmup(std::make_shared<FakeIndexSearchable>())
{}
~IndexCollectionTest() = default;
diff --git a/searchcore/src/tests/proton/initializer/task_runner_test.cpp b/searchcore/src/tests/proton/initializer/task_runner_test.cpp
index adbc3feead2..e4f4bb1b92b 100644
--- a/searchcore/src/tests/proton/initializer/task_runner_test.cpp
+++ b/searchcore/src/tests/proton/initializer/task_runner_test.cpp
@@ -105,7 +105,7 @@ TestJob::TestJob(TestLog::UP log, InitializerTask::SP root)
: _log(std::move(log)),
_root(std::move(root))
{ }
-TestJob::~TestJob() {}
+TestJob::~TestJob() = default;
struct Fixture
@@ -114,14 +114,15 @@ struct Fixture
TaskRunner _taskRunner;
Fixture(uint32_t numThreads = 1)
- : _executor(numThreads, 128_Ki),
+ : _executor(numThreads),
_taskRunner(_executor)
- {
- }
+ { }
+ ~Fixture();
void run(const InitializerTask::SP &task) { _taskRunner.runTask(task); }
};
+Fixture::~Fixture() = default;
TEST_F("1 thread, 2 dependees, 1 depender", Fixture(1))
{
diff --git a/searchcore/src/tests/proton/metrics/job_tracked_flush/job_tracked_flush_test.cpp b/searchcore/src/tests/proton/metrics/job_tracked_flush/job_tracked_flush_test.cpp
index 1c1ece92e2e..340108c3d69 100644
--- a/searchcore/src/tests/proton/metrics/job_tracked_flush/job_tracked_flush_test.cpp
+++ b/searchcore/src/tests/proton/metrics/job_tracked_flush/job_tracked_flush_test.cpp
@@ -71,7 +71,7 @@ struct Fixture
_trackedFlush(_tracker, _target),
_task(),
_taskGate(),
- _exec(1, 64000)
+ _exec(1)
{
}
void initFlush(SerialNum currentSerial) {
diff --git a/searchcore/src/tests/proton/proton_configurer/proton_configurer_test.cpp b/searchcore/src/tests/proton/proton_configurer/proton_configurer_test.cpp
index e929b699355..51008e4cbee 100644
--- a/searchcore/src/tests/proton/proton_configurer/proton_configurer_test.cpp
+++ b/searchcore/src/tests/proton/proton_configurer/proton_configurer_test.cpp
@@ -20,7 +20,6 @@
#include <vespa/searchcore/proton/server/i_proton_disk_layout.h>
#include <vespa/searchcore/proton/server/threading_service_config.h>
#include <vespa/searchsummary/config/config-juniperrc.h>
-#include <vespa/vespalib/util/size_literals.h>
#include <vespa/vespalib/util/threadstackexecutor.h>
#include <vespa/vespalib/test/insertion_operators.h>
#include <vespa/config/subscription/configuri.h>
@@ -271,7 +270,7 @@ struct MyProtonConfigurerOwner : public IProtonConfigurerOwner,
MyProtonConfigurerOwner()
: IProtonConfigurerOwner(),
MyLog(),
- _executor(1, 128_Ki),
+ _executor(1),
_dbs()
{
}
diff --git a/searchcore/src/vespa/searchcore/bmcluster/bm_node.cpp b/searchcore/src/vespa/searchcore/bmcluster/bm_node.cpp
index 5e392b12f67..aebfe23aa6f 100644
--- a/searchcore/src/vespa/searchcore/bmcluster/bm_node.cpp
+++ b/searchcore/src/vespa/searchcore/bmcluster/bm_node.cpp
@@ -501,7 +501,7 @@ MyBmNode::MyBmNode(const vespalib::string& base_dir, int base_port, uint32_t nod
_query_limiter(),
_metrics_wire_service(),
_config_stores(),
- _summary_executor(8, 128_Ki),
+ _summary_executor(8),
_shared_service(_summary_executor, _summary_executor),
_tls(_shared_service.transport(), "tls", _tls_listen_port, _base_dir, _file_header_context),
_document_db_owner(),
@@ -579,7 +579,7 @@ MyBmNode::create_document_db(const BmClusterParams& params)
_metrics_wire_service, _file_header_context,
std::make_shared<search::attribute::Interlock>(),
_config_stores.getConfigStore(_doc_type_name.toString()),
- std::make_shared<vespalib::ThreadStackExecutor>(16, 128_Ki), HwInfo());
+ std::make_shared<vespalib::ThreadStackExecutor>(16), HwInfo());
_document_db->start();
_document_db->waitForOnlineState();
}
diff --git a/searchcore/src/vespa/searchcore/bmcluster/bm_node_stats_reporter.cpp b/searchcore/src/vespa/searchcore/bmcluster/bm_node_stats_reporter.cpp
index da3964176e6..d6e495ae2b6 100644
--- a/searchcore/src/vespa/searchcore/bmcluster/bm_node_stats_reporter.cpp
+++ b/searchcore/src/vespa/searchcore/bmcluster/bm_node_stats_reporter.cpp
@@ -34,7 +34,7 @@ bool steady_buckets_stats(const std::optional<BmBucketsStats> buckets)
BmNodeStatsReporter::BmNodeStatsReporter(BmCluster &cluster, bool report_merge_stats)
: _cluster(cluster),
- _executor(1, 128_Ki),
+ _executor(1),
_mutex(),
_cond(),
_change_time(),
diff --git a/searchcore/src/vespa/searchcore/proton/flushengine/flushengine.cpp b/searchcore/src/vespa/searchcore/proton/flushengine/flushengine.cpp
index 4968cb6791b..4b83b7d5af9 100644
--- a/searchcore/src/vespa/searchcore/proton/flushengine/flushengine.cpp
+++ b/searchcore/src/vespa/searchcore/proton/flushengine/flushengine.cpp
@@ -10,7 +10,6 @@
#include <vespa/searchcore/proton/common/eventlogger.h>
#include <vespa/searchlib/common/flush_token.h>
#include <vespa/vespalib/util/cpu_usage.h>
-#include <vespa/vespalib/util/size_literals.h>
#include <thread>
#include <vespa/log/log.h>
@@ -87,10 +86,10 @@ FlushEngine::FlushEngine(std::shared_ptr<flushengine::ITlsStatsFactory> tlsStats
_maxConcurrent(numThreads),
_idleInterval(idleInterval),
_taskId(0),
- _threadPool(128_Ki),
+ _threadPool(),
_strategy(std::move(strategy)),
_priorityStrategy(),
- _executor(numThreads, 128_Ki, CpuUsage::wrap(flush_engine_executor, CpuUsage::Category::COMPACT)),
+ _executor(numThreads, CpuUsage::wrap(flush_engine_executor, CpuUsage::Category::COMPACT)),
_lock(),
_cond(),
_handlers(),
diff --git a/searchcore/src/vespa/searchcore/proton/initializer/task_runner.cpp b/searchcore/src/vespa/searchcore/proton/initializer/task_runner.cpp
index 75c89f9adb8..46700e4d710 100644
--- a/searchcore/src/vespa/searchcore/proton/initializer/task_runner.cpp
+++ b/searchcore/src/vespa/searchcore/proton/initializer/task_runner.cpp
@@ -2,7 +2,6 @@
#include "task_runner.h"
#include <vespa/vespalib/util/lambdatask.h>
-#include <vespa/vespalib/util/size_literals.h>
#include <vespa/vespalib/util/threadstackexecutor.h>
#include <future>
@@ -94,7 +93,7 @@ TaskRunner::internalRunTasks(const TaskList &taskList, Context::SP context)
void
TaskRunner::runTask(InitializerTask::SP task)
{
- vespalib::ThreadStackExecutor executor(1, 128_Ki, task_runner);
+ vespalib::ThreadStackExecutor executor(1, task_runner);
std::promise<void> promise;
auto future = promise.get_future();
runTask(task, executor, makeLambdaTask([&]() { promise.set_value(); }));
diff --git a/searchcore/src/vespa/searchcore/proton/matchengine/matchengine.cpp b/searchcore/src/vespa/searchcore/proton/matchengine/matchengine.cpp
index 107f85f12d1..9db7eef840d 100644
--- a/searchcore/src/vespa/searchcore/proton/matchengine/matchengine.cpp
+++ b/searchcore/src/vespa/searchcore/proton/matchengine/matchengine.cpp
@@ -51,7 +51,7 @@ MatchEngine::MatchEngine(size_t numThreads, size_t threadsPerSearch, uint32_t di
_closed(false),
_forward_issues(true),
_handlers(),
- _executor(std::max(size_t(1), numThreads / threadsPerSearch), 256_Ki,
+ _executor(std::max(size_t(1), numThreads / threadsPerSearch),
CpuUsage::wrap(match_engine_executor, CpuUsage::Category::READ)),
_threadBundlePool(std::max(size_t(1), threadsPerSearch),
CpuUsage::wrap(match_engine_thread_bundle, CpuUsage::Category::READ)),
diff --git a/searchcore/src/vespa/searchcore/proton/server/documentdb.cpp b/searchcore/src/vespa/searchcore/proton/server/documentdb.cpp
index 6b62852aaae..b97722493ab 100644
--- a/searchcore/src/vespa/searchcore/proton/server/documentdb.cpp
+++ b/searchcore/src/vespa/searchcore/proton/server/documentdb.cpp
@@ -37,7 +37,6 @@
#include <vespa/searchlib/engine/searchreply.h>
#include <vespa/vespalib/util/destructor_callbacks.h>
#include <vespa/vespalib/util/exceptions.h>
-#include <vespa/vespalib/util/size_literals.h>
#include <vespa/log/log.h>
#include <vespa/searchcorespi/index/warmupconfig.h>
@@ -72,7 +71,6 @@ using searchcorespi::IFlushTarget;
namespace proton {
namespace {
-constexpr uint32_t indexing_thread_stack_size = 128_Ki;
index::IndexConfig
makeIndexConfig(const ProtonConfig::Index & cfg) {
@@ -191,7 +189,7 @@ DocumentDB::DocumentDB(const vespalib::string &baseDir,
// Only one thread per executor, or performDropFeedView() will fail.
_writeServiceConfig(configSnapshot->get_threading_service_config()),
_writeService(shared_service.shared(), shared_service.transport(), shared_service.clock(), shared_service.field_writer(),
- &shared_service.invokeService(), _writeServiceConfig, indexing_thread_stack_size),
+ &shared_service.invokeService(), _writeServiceConfig),
_initializeThreads(std::move(initializeThreads)),
_initConfigSnapshot(),
_initConfigSerialNum(0u),
diff --git a/searchcore/src/vespa/searchcore/proton/server/documentsubdbcollection.cpp b/searchcore/src/vespa/searchcore/proton/server/documentsubdbcollection.cpp
index 103234f7529..855a40780cb 100644
--- a/searchcore/src/vespa/searchcore/proton/server/documentsubdbcollection.cpp
+++ b/searchcore/src/vespa/searchcore/proton/server/documentsubdbcollection.cpp
@@ -82,7 +82,7 @@ DocumentSubDBCollection::DocumentSubDBCollection(
DocumentSubDBCollection::~DocumentSubDBCollection()
{
size_t numThreads = std::min(_subDBs.size(), static_cast<size_t>(_hwInfo.cpu().cores()));
- vespalib::ThreadStackExecutor closePool(numThreads, 0x20000);
+ vespalib::ThreadStackExecutor closePool(numThreads);
while (!_subDBs.empty()) {
closePool.execute(makeLambdaTask([subDB=_subDBs.back()]() { delete subDB; }));
_subDBs.pop_back();
diff --git a/searchcore/src/vespa/searchcore/proton/server/executorthreadingservice.cpp b/searchcore/src/vespa/searchcore/proton/server/executorthreadingservice.cpp
index 8c73067056d..f218193c02f 100644
--- a/searchcore/src/vespa/searchcore/proton/server/executorthreadingservice.cpp
+++ b/searchcore/src/vespa/searchcore/proton/server/executorthreadingservice.cpp
@@ -21,12 +21,12 @@ namespace proton {
namespace {
std::unique_ptr<SyncableThreadExecutor>
-createExecutorWithOneThread(uint32_t stackSize, uint32_t taskLimit, OptimizeFor optimize,
+createExecutorWithOneThread(uint32_t taskLimit, OptimizeFor optimize,
vespalib::Runnable::init_fun_t init_function) {
if (optimize == OptimizeFor::THROUGHPUT) {
return std::make_unique<SingleExecutor>(std::move(init_function), taskLimit);
} else {
- return std::make_unique<BlockingThreadStackExecutor>(1, stackSize, taskLimit, std::move(init_function));
+ return std::make_unique<BlockingThreadStackExecutor>(1, taskLimit, std::move(init_function));
}
}
@@ -48,17 +48,16 @@ ExecutorThreadingService::ExecutorThreadingService(vespalib::Executor & sharedEx
const vespalib::Clock & clock,
vespalib::ISequencedTaskExecutor& field_writer,
vespalib::InvokeService * invokerService,
- const ThreadingServiceConfig & cfg,
- uint32_t stackSize)
+ const ThreadingServiceConfig & cfg)
: _sharedExecutor(sharedExecutor),
_transport(transport),
_clock(clock),
- _masterExecutor(1, stackSize, CpuUsage::wrap(master_executor, CpuUsage::Category::WRITE)),
+ _masterExecutor(1, CpuUsage::wrap(master_executor, CpuUsage::Category::WRITE)),
_master_task_limit(cfg.master_task_limit()),
- _indexExecutor(createExecutorWithOneThread(stackSize, cfg.defaultTaskLimit(), cfg.optimize(),
+ _indexExecutor(createExecutorWithOneThread(cfg.defaultTaskLimit(), cfg.optimize(),
CpuUsage::wrap(index_executor, CpuUsage::Category::WRITE))),
- _summaryExecutor(createExecutorWithOneThread(stackSize, cfg.defaultTaskLimit(), cfg.optimize(),
+ _summaryExecutor(createExecutorWithOneThread(cfg.defaultTaskLimit(), cfg.optimize(),
CpuUsage::wrap(summary_executor, CpuUsage::Category::WRITE))),
_masterService(_masterExecutor),
_indexService(*_indexExecutor),
diff --git a/searchcore/src/vespa/searchcore/proton/server/executorthreadingservice.h b/searchcore/src/vespa/searchcore/proton/server/executorthreadingservice.h
index 7c8056b816c..9da4348c619 100644
--- a/searchcore/src/vespa/searchcore/proton/server/executorthreadingservice.h
+++ b/searchcore/src/vespa/searchcore/proton/server/executorthreadingservice.h
@@ -49,8 +49,7 @@ public:
const vespalib::Clock & clock,
vespalib::ISequencedTaskExecutor& field_writer,
vespalib::InvokeService * invokeService,
- const ThreadingServiceConfig& cfg,
- uint32_t stackSize = 128 * 1024);
+ const ThreadingServiceConfig& cfg);
~ExecutorThreadingService() override;
void blocking_master_execute(vespalib::Executor::Task::UP task) override;
diff --git a/searchcore/src/vespa/searchcore/proton/server/initialize_threads_calculator.cpp b/searchcore/src/vespa/searchcore/proton/server/initialize_threads_calculator.cpp
index 727e4bb1e58..17ddabebb65 100644
--- a/searchcore/src/vespa/searchcore/proton/server/initialize_threads_calculator.cpp
+++ b/searchcore/src/vespa/searchcore/proton/server/initialize_threads_calculator.cpp
@@ -2,11 +2,11 @@
#include "initialize_threads_calculator.h"
#include <vespa/vespalib/util/cpu_usage.h>
-#include <vespa/vespalib/util/size_literals.h>
#include <vespa/vespalib/util/threadstackexecutor.h>
#include <fstream>
using vespalib::CpuUsage;
+using vespalib::ThreadStackExecutor;
using CpuCategory = vespalib::CpuUsage::Category;
namespace {
@@ -52,8 +52,7 @@ InitializeThreadsCalculator::InitializeThreadsCalculator(const vespalib::string&
}
write(_path.c_str(), _num_threads);
if (_num_threads > 0) {
- _threads = std::make_shared<vespalib::ThreadStackExecutor>(_num_threads, 128_Ki,
- CpuUsage::wrap(proton_initialize_executor, CpuCategory::SETUP));
+ _threads = std::make_shared<ThreadStackExecutor>(_num_threads, CpuUsage::wrap(proton_initialize_executor, CpuCategory::SETUP));
}
}
diff --git a/searchcore/src/vespa/searchcore/proton/server/proton.cpp b/searchcore/src/vespa/searchcore/proton/server/proton.cpp
index 94b14780df5..a562e414128 100644
--- a/searchcore/src/vespa/searchcore/proton/server/proton.cpp
+++ b/searchcore/src/vespa/searchcore/proton/server/proton.cpp
@@ -148,7 +148,8 @@ struct MetricsUpdateHook : metrics::UpdateHook
const vespalib::string CUSTOM_COMPONENT_API_PATH = "/state/v1/custom/component";
-VESPA_THREAD_STACK_TAG(proton_close_executor)
+VESPA_THREAD_STACK_TAG(proton_close_executor);
+VESPA_THREAD_STACK_TAG(proton_executor);
void ensureWritableDir(const vespalib::string &dirName) {
auto filename = dirName + "/tmp.filesystem.probe";
@@ -251,7 +252,7 @@ Proton::Proton(FastOS_ThreadPool & threadPool, FNET_Transport & transport, const
_stateServer(),
// This executor can only have 1 thread as it is used for
// serializing startup.
- _executor(1, 128_Ki),
+ _executor(1, CpuUsage::wrap(proton_executor, CpuCategory::SETUP)),
_protonDiskLayout(),
_protonConfigurer(_executor, *this, _protonDiskLayout),
_protonConfigFetcher(_transport, configUri, _protonConfigurer, subscribeTimeout),
@@ -492,7 +493,7 @@ Proton::~Proton()
}
}
- vespalib::ThreadStackExecutor closePool(std::min(_documentDBMap.size(), numCores), 0x20000,
+ vespalib::ThreadStackExecutor closePool(std::min(_documentDBMap.size(), numCores),
CpuUsage::wrap(proton_close_executor, CpuCategory::SETUP));
closeDocumentDBs(closePool);
}
@@ -632,7 +633,7 @@ Proton::addDocumentDB(const document::DocumentType &docType,
// If configured value for initialize threads was 0, or we
// are performing a reconfig after startup has completed, then use
// 1 thread per document type.
- initializeThreads = std::make_shared<vespalib::ThreadStackExecutor>(1, 128_Ki);
+ initializeThreads = std::make_shared<vespalib::ThreadStackExecutor>(1);
}
auto ret = DocumentDB::create(config.basedir + "/documents",
documentDBConfig,
diff --git a/searchcore/src/vespa/searchcore/proton/server/shared_threading_service.cpp b/searchcore/src/vespa/searchcore/proton/server/shared_threading_service.cpp
index 5f522bfaffc..a0344c57000 100644
--- a/searchcore/src/vespa/searchcore/proton/server/shared_threading_service.cpp
+++ b/searchcore/src/vespa/searchcore/proton/server/shared_threading_service.cpp
@@ -5,7 +5,6 @@
#include <vespa/vespalib/util/threadstackexecutor.h>
#include <vespa/vespalib/util/cpu_usage.h>
#include <vespa/vespalib/util/sequencedtaskexecutor.h>
-#include <vespa/vespalib/util/size_literals.h>
#include <vespa/vespalib/util/nice.h>
using vespalib::CpuUsage;
@@ -21,10 +20,10 @@ SharedThreadingService::SharedThreadingService(const SharedThreadingServiceConfi
FNET_Transport& transport,
storage::spi::BucketExecutor& bucket_executor)
: _transport(transport),
- _warmup(std::make_unique<vespalib::ThreadStackExecutor>(cfg.warmup_threads(), 128_Ki,
+ _warmup(std::make_unique<vespalib::ThreadStackExecutor>(cfg.warmup_threads(),
CpuUsage::wrap(proton_warmup_executor, CpuUsage::Category::COMPACT),
cfg.shared_task_limit())),
- _shared(std::make_shared<vespalib::BlockingThreadStackExecutor>(cfg.shared_threads(), 128_Ki,
+ _shared(std::make_shared<vespalib::BlockingThreadStackExecutor>(cfg.shared_threads(),
cfg.shared_task_limit(), vespalib::be_nice(proton_shared_executor, cfg.feeding_niceness()))),
_field_writer(),
_invokeService(std::max(vespalib::adjustTimeoutByDetectedHz(1ms),
diff --git a/searchcore/src/vespa/searchcore/proton/summaryengine/summaryengine.cpp b/searchcore/src/vespa/searchcore/proton/summaryengine/summaryengine.cpp
index 0bf5b351fd0..b60b87b4ed4 100644
--- a/searchcore/src/vespa/searchcore/proton/summaryengine/summaryengine.cpp
+++ b/searchcore/src/vespa/searchcore/proton/summaryengine/summaryengine.cpp
@@ -63,7 +63,7 @@ SummaryEngine::SummaryEngine(size_t numThreads, bool async)
_closed(false),
_forward_issues(true),
_handlers(),
- _executor(numThreads, 128_Ki, CpuUsage::wrap(summary_engine_executor, CpuUsage::Category::READ)),
+ _executor(numThreads, CpuUsage::wrap(summary_engine_executor, CpuUsage::Category::READ)),
_metrics(std::make_unique<DocsumMetrics>())
{ }
diff --git a/searchcore/src/vespa/searchcore/proton/test/transport_helper.cpp b/searchcore/src/vespa/searchcore/proton/test/transport_helper.cpp
index 80e2622fa3b..d0bc4dfd4d8 100644
--- a/searchcore/src/vespa/searchcore/proton/test/transport_helper.cpp
+++ b/searchcore/src/vespa/searchcore/proton/test/transport_helper.cpp
@@ -12,7 +12,7 @@
namespace proton {
Transport::Transport()
- : _threadPool(std::make_unique<FastOS_ThreadPool>(64_Ki)),
+ : _threadPool(std::make_unique<FastOS_ThreadPool>()),
_transport(std::make_unique<FNET_Transport>()),
_clock(std::make_unique<vespalib::TestClock>())
{
@@ -36,7 +36,7 @@ VESPA_THREAD_STACK_TAG(proton_transport_and_executor_field_writer)
TransportAndExecutor::TransportAndExecutor(size_t num_threads)
: Transport(),
- _sharedExecutor(std::make_unique<vespalib::ThreadStackExecutor>(num_threads, 64_Ki)),
+ _sharedExecutor(std::make_unique<vespalib::ThreadStackExecutor>(num_threads)),
_field_writer(vespalib::SequencedTaskExecutor::create(proton_transport_and_executor_field_writer, num_threads))
{}
diff --git a/searchlib/src/apps/docstore/benchmarkdatastore.cpp b/searchlib/src/apps/docstore/benchmarkdatastore.cpp
index 9c2a40d6e98..cf2e7f7356d 100644
--- a/searchlib/src/apps/docstore/benchmarkdatastore.cpp
+++ b/searchlib/src/apps/docstore/benchmarkdatastore.cpp
@@ -97,12 +97,12 @@ BenchmarkDataStoreApp::benchmark(const vespalib::string & dir, size_t numReads,
tuning._randRead.setWantMemoryMap();
}
search::index::DummyFileHeaderContext fileHeaderContext;
- vespalib::ThreadStackExecutor executor(1, 128_Ki);
+ vespalib::ThreadStackExecutor executor(1);
transactionlog::NoSyncProxy noTlSyncer;
LogDataStore store(executor, dir, config, growStrategy, tuning,
fileHeaderContext,
noTlSyncer, NULL, true);
- vespalib::ThreadStackExecutor bmPool(numThreads, 128_Ki);
+ vespalib::ThreadStackExecutor bmPool(numThreads);
LOG(info, "Start read benchmark with %lu threads doing %lu reads in chunks of %lu reads. Totally %lu objects", numThreads, numReads, perChunk, numThreads * numReads * perChunk);
for (size_t i(0); i < numThreads; i++) {
bmPool.execute(vespalib::makeLambdaTask([&]() { read(numReads, perChunk, static_cast<const IDataStore *>(&store)); }));
diff --git a/searchlib/src/apps/docstore/documentstoreinspect.cpp b/searchlib/src/apps/docstore/documentstoreinspect.cpp
index 34afa0216ed..f19a128535f 100644
--- a/searchlib/src/apps/docstore/documentstoreinspect.cpp
+++ b/searchlib/src/apps/docstore/documentstoreinspect.cpp
@@ -108,7 +108,7 @@ DocumentStoreInspectApp::verify(const vespalib::string & dir)
GrowStrategy growStrategy;
TuneFileSummary tuning;
search::index::DummyFileHeaderContext fileHeaderContext;
- vespalib::ThreadStackExecutor executor(1, 128_Ki);
+ vespalib::ThreadStackExecutor executor(1);
transactionlog::NoSyncProxy noTlSyncer;
LogDataStore store(executor, dir, config, growStrategy, tuning,
diff --git a/searchlib/src/apps/docstore/verifylogdatastore.cpp b/searchlib/src/apps/docstore/verifylogdatastore.cpp
index 3b673c172c9..f803f7aee9f 100644
--- a/searchlib/src/apps/docstore/verifylogdatastore.cpp
+++ b/searchlib/src/apps/docstore/verifylogdatastore.cpp
@@ -51,7 +51,7 @@ VerifyLogDataStoreApp::verify(const vespalib::string & dir)
GrowStrategy growStrategy;
TuneFileSummary tuning;
search::index::DummyFileHeaderContext fileHeaderContext;
- vespalib::ThreadStackExecutor executor(1, 128_Ki);
+ vespalib::ThreadStackExecutor executor(1);
transactionlog::NoSyncProxy noTlSyncer;
try {
diff --git a/searchlib/src/apps/tests/biglogtest.cpp b/searchlib/src/apps/tests/biglogtest.cpp
index bd8991edc4b..154e94d0c66 100644
--- a/searchlib/src/apps/tests/biglogtest.cpp
+++ b/searchlib/src/apps/tests/biglogtest.cpp
@@ -4,7 +4,6 @@
#include <vespa/searchlib/docstore/logdatastore.h>
#include <vespa/searchlib/index/dummyfileheadercontext.h>
#include <vespa/searchlib/transactionlog/nosyncproxy.h>
-#include <vespa/vespalib/util/exceptions.h>
#include <vespa/vespalib/util/size_literals.h>
#include <vespa/vespalib/util/threadstackexecutor.h>
#include <vespa/vespalib/data/databuffer.h>
@@ -136,7 +135,7 @@ factory<LogDataStore>::factory(std::string dir)
: DioTune(),
_fileHeaderContext(),
_config(),
- _executor(1, 128_Ki),
+ _executor(1),
_noTlSyncer(),
_datastore(_executor, dir, _config, GrowStrategy(), tuning, _fileHeaderContext, _noTlSyncer, NULL)
{}
diff --git a/searchlib/src/apps/tests/memoryindexstress_test.cpp b/searchlib/src/apps/tests/memoryindexstress_test.cpp
index 526b4cad9a6..fb743ad93ac 100644
--- a/searchlib/src/apps/tests/memoryindexstress_test.cpp
+++ b/searchlib/src/apps/tests/memoryindexstress_test.cpp
@@ -8,7 +8,6 @@
#include <vespa/searchlib/memoryindex/memory_index.h>
#include <vespa/searchlib/query/tree/simplequery.h>
#include <vespa/searchlib/queryeval/fake_requestcontext.h>
-#include <vespa/searchlib/queryeval/fake_search.h>
#include <vespa/searchlib/queryeval/fake_searchable.h>
#include <vespa/searchlib/queryeval/searchiterator.h>
#include <vespa/searchlib/queryeval/blueprint.h>
@@ -265,13 +264,13 @@ VESPA_THREAD_STACK_TAG(push_executor)
Fixture::Fixture(uint32_t readThreads)
: schema(makeSchema()),
repo(makeDocTypeRepoConfig()),
- _executor(1, 128_Ki),
+ _executor(1),
_invertThreads(vespalib::SequencedTaskExecutor::create(invert_executor, 2)),
_pushThreads(vespalib::SequencedTaskExecutor::create(push_executor, 2)),
index(schema, MockFieldLengthInspector(), *_invertThreads, *_pushThreads),
_readThreads(readThreads),
- _writer(1, 128_Ki),
- _readers(readThreads, 128_Ki),
+ _writer(1),
+ _readers(readThreads),
_rnd(),
_keyLimit(1000000),
_readSeed(50),
diff --git a/searchlib/src/tests/attribute/benchmark/attributebenchmark.cpp b/searchlib/src/tests/attribute/benchmark/attributebenchmark.cpp
index cb6c66915f1..39e5d713d08 100644
--- a/searchlib/src/tests/attribute/benchmark/attributebenchmark.cpp
+++ b/searchlib/src/tests/attribute/benchmark/attributebenchmark.cpp
@@ -569,7 +569,7 @@ AttributeBenchmark::main(int argc, char **argv)
dc._attribute = vespalib::string(argv[optind]);
- _threadPool = new FastOS_ThreadPool(256000);
+ _threadPool = new FastOS_ThreadPool();
std::cout << "<attribute-benchmark>" << std::endl;
init(dc);
diff --git a/searchlib/src/tests/attribute/tensorattribute/tensorattribute_test.cpp b/searchlib/src/tests/attribute/tensorattribute/tensorattribute_test.cpp
index b145192eb3a..584335803a8 100644
--- a/searchlib/src/tests/attribute/tensorattribute/tensorattribute_test.cpp
+++ b/searchlib/src/tests/attribute/tensorattribute/tensorattribute_test.cpp
@@ -582,7 +582,7 @@ Fixture::Fixture(const vespalib::string &typeSpec, FixtureTraits traits)
_index_factory(),
_tensorAttr(),
_attr(),
- _executor(1, 0x10000),
+ _executor(1),
_denseTensors(false),
_traits(traits),
_mmap_allocator_base_dir("mmap-file-allocator-factory-dir")
diff --git a/searchlib/src/tests/diskindex/fusion/fusion_test.cpp b/searchlib/src/tests/diskindex/fusion/fusion_test.cpp
index 5e0111cbe8c..b3d8e46cb3a 100644
--- a/searchlib/src/tests/diskindex/fusion/fusion_test.cpp
+++ b/searchlib/src/tests/diskindex/fusion/fusion_test.cpp
@@ -390,7 +390,7 @@ FusionTest::requireThatFusionIsWorking(const vespalib::string &prefix, bool dire
ASSERT_TRUE(FileKit::hasStamp(tsName));
ASSERT_TRUE(FileKit::removeStamp(tsName));
ASSERT_FALSE(FileKit::hasStamp(tsName));
- vespalib::ThreadStackExecutor executor(4, 0x10000);
+ vespalib::ThreadStackExecutor executor(4);
do {
DiskIndex dw2(prefix + "dump2");
@@ -499,7 +499,7 @@ FusionTest::make_simple_index(const vespalib::string &dump_dir, const IFieldLeng
bool
FusionTest::try_merge_simple_indexes(const vespalib::string &dump_dir, const std::vector<vespalib::string> &sources, std::shared_ptr<IFlushToken> flush_token)
{
- vespalib::ThreadStackExecutor executor(4, 0x10000);
+ vespalib::ThreadStackExecutor executor(4);
TuneFileIndexing tuneFileIndexing;
DummyFileHeaderContext fileHeaderContext;
SelectorArray selector(20, 0);
diff --git a/searchlib/src/tests/docstore/document_store_visitor/document_store_visitor_test.cpp b/searchlib/src/tests/docstore/document_store_visitor/document_store_visitor_test.cpp
index 1df5f9a1409..efe858c016a 100644
--- a/searchlib/src/tests/docstore/document_store_visitor/document_store_visitor_test.cpp
+++ b/searchlib/src/tests/docstore/document_store_visitor/document_store_visitor_test.cpp
@@ -240,7 +240,7 @@ Fixture::Fixture()
_storeConfig(DocumentStore::Config(CompressionConfig::NONE, 0, 0),
LogDataStore::Config().setMaxFileSize(50000).setMaxBucketSpread(3.0)
.setFileConfig(WriteableFileChunk::Config(CompressionConfig(), 16_Ki))),
- _executor(1, 128_Ki),
+ _executor(1),
_fileHeaderContext(),
_tlSyncer(),
_store(),
diff --git a/searchlib/src/tests/docstore/file_chunk/file_chunk_test.cpp b/searchlib/src/tests/docstore/file_chunk/file_chunk_test.cpp
index b295291d7c4..6e69e5092bc 100644
--- a/searchlib/src/tests/docstore/file_chunk/file_chunk_test.cpp
+++ b/searchlib/src/tests/docstore/file_chunk/file_chunk_test.cpp
@@ -74,7 +74,7 @@ struct FixtureBase {
explicit FixtureBase(const vespalib::string &baseName, bool dirCleanup = true)
: dir(baseName),
- executor(1, 0x10000),
+ executor(1),
serialNum(1),
tuneFile(),
fileHeaderCtx(),
diff --git a/searchlib/src/tests/docstore/logdatastore/logdatastore_test.cpp b/searchlib/src/tests/docstore/logdatastore/logdatastore_test.cpp
index 9538db391df..ad0bf60fbe3 100644
--- a/searchlib/src/tests/docstore/logdatastore/logdatastore_test.cpp
+++ b/searchlib/src/tests/docstore/logdatastore/logdatastore_test.cpp
@@ -212,7 +212,7 @@ TEST("test that DirectIOPadding works accordng to spec") {
void verifyGrowing(const LogDataStore::Config & config, uint32_t minFiles, uint32_t maxFiles) {
DirectoryHandler tmpDir("growing");
- vespalib::ThreadStackExecutor executor(4, 128_Ki);
+ vespalib::ThreadStackExecutor executor(4);
DummyFileHeaderContext fileHeaderContext;
MyTlSyncer tlSyncer;
{
@@ -283,7 +283,7 @@ void fetchAndTest(IDataStore & datastore, uint32_t lid, const void *a, size_t sz
TEST("testTruncatedIdxFile"){
LogDataStore::Config config;
DummyFileHeaderContext fileHeaderContext;
- vespalib::ThreadStackExecutor executor(1, 128_Ki);
+ vespalib::ThreadStackExecutor executor(1);
MyTlSyncer tlSyncer;
{
// Files comes from the 'growing test'.
@@ -311,7 +311,7 @@ TEST("testTruncatedIdxFile"){
TEST("testThatEmptyIdxFilesAndDanglingDatFilesAreRemoved") {
LogDataStore::Config config;
DummyFileHeaderContext fileHeaderContext;
- vespalib::ThreadStackExecutor executor(1, 128_Ki);
+ vespalib::ThreadStackExecutor executor(1);
MyTlSyncer tlSyncer;
LogDataStore datastore(executor, "dangling-test", config,
GrowStrategy(), TuneFileSummary(),
@@ -324,7 +324,7 @@ TEST("testThatEmptyIdxFilesAndDanglingDatFilesAreRemoved") {
TEST("testThatIncompleteCompactedFilesAreRemoved") {
LogDataStore::Config config;
DummyFileHeaderContext fileHeaderContext;
- vespalib::ThreadStackExecutor executor(1, 128_Ki);
+ vespalib::ThreadStackExecutor executor(1);
MyTlSyncer tlSyncer;
LogDataStore datastore(executor, "incompletecompact-test", config,
GrowStrategy(), TuneFileSummary(),
@@ -344,7 +344,7 @@ public:
_myDir("visitcache"),
_config(),
_fileHeaderContext(),
- _executor(1, 128_Ki),
+ _executor(1),
_tlSyncer(),
_datastore(_executor, _myDir.getDir(), _config, GrowStrategy(),
TuneFileSummary(), _fileHeaderContext, _tlSyncer, nullptr)
@@ -533,7 +533,7 @@ VisitCacheStore::VisitCacheStore(UpdateStrategy strategy) :
LogDataStore::Config().setMaxFileSize(50000).setMaxBucketSpread(3.0)
.setFileConfig(WriteableFileChunk::Config(CompressionConfig(), 16_Ki))),
_fileHeaderContext(),
- _executor(1, 128_Ki),
+ _executor(1),
_tlSyncer(),
_datastore(std::make_unique<LogDocumentStore>(_executor, _myDir.getDir(), _config, GrowStrategy(),
TuneFileSummary(), _fileHeaderContext, _tlSyncer, nullptr)),
@@ -675,7 +675,7 @@ TEST("testWriteRead") {
{
std::filesystem::create_directory(std::filesystem::path("empty"));
DummyFileHeaderContext fileHeaderContext;
- vespalib::ThreadStackExecutor executor(1, 128_Ki);
+ vespalib::ThreadStackExecutor executor(1);
MyTlSyncer tlSyncer;
LogDataStore datastore(executor, "empty", config, GrowStrategy(),
TuneFileSummary(), fileHeaderContext, tlSyncer, nullptr);
@@ -711,7 +711,7 @@ TEST("testWriteRead") {
}
{
DummyFileHeaderContext fileHeaderContext;
- vespalib::ThreadStackExecutor executor(1, 128_Ki);
+ vespalib::ThreadStackExecutor executor(1);
MyTlSyncer tlSyncer;
LogDataStore datastore(executor, "empty", config,
GrowStrategy(), TuneFileSummary(),
@@ -762,7 +762,7 @@ TEST("requireThatFlushTimeIsAvailableAfterFlush") {
vespalib::system_time before(vespalib::system_clock::now());
DummyFileHeaderContext fileHeaderContext;
LogDataStore::Config config;
- vespalib::ThreadStackExecutor executor(1, 128_Ki);
+ vespalib::ThreadStackExecutor executor(1);
MyTlSyncer tlSyncer;
LogDataStore store(executor, testDir.getDir(), config, GrowStrategy(),
TuneFileSummary(), fileHeaderContext, tlSyncer, nullptr);
@@ -849,7 +849,7 @@ struct Fixture {
Fixture(const vespalib::string &dirName = "tmp",
bool dirCleanup = true,
size_t maxFileSize = 4_Ki * 2)
- : executor(1, 0x20000),
+ : executor(1),
dir(dirName),
serialNum(0),
fileHeaderCtx(),
diff --git a/searchlib/src/tests/docstore/store_by_bucket/store_by_bucket_test.cpp b/searchlib/src/tests/docstore/store_by_bucket/store_by_bucket_test.cpp
index da900ddecee..b849143427c 100644
--- a/searchlib/src/tests/docstore/store_by_bucket/store_by_bucket_test.cpp
+++ b/searchlib/src/tests/docstore/store_by_bucket/store_by_bucket_test.cpp
@@ -68,7 +68,7 @@ TEST("require that StoreByBucket gives bucket by bucket and ordered within")
{
std::mutex backing_lock;
vespalib::MemoryDataStore backing(vespalib::alloc::Alloc::alloc(256), &backing_lock);
- vespalib::ThreadStackExecutor executor(8, 128_Ki);
+ vespalib::ThreadStackExecutor executor(8);
StoreByBucket sbb(backing, executor, CompressionConfig::LZ4);
for (size_t i(1); i <=500; i++) {
add(sbb, i);
diff --git a/searchlib/src/tests/memoryindex/memory_index/memory_index_test.cpp b/searchlib/src/tests/memoryindex/memory_index/memory_index_test.cpp
index c1261a60278..69063f38aeb 100644
--- a/searchlib/src/tests/memoryindex/memory_index/memory_index_test.cpp
+++ b/searchlib/src/tests/memoryindex/memory_index/memory_index_test.cpp
@@ -156,7 +156,7 @@ VESPA_THREAD_STACK_TAG(invert_executor)
VESPA_THREAD_STACK_TAG(push_executor)
Index::Index(const MySetup &setup)
- : _executor(1, 128_Ki),
+ : _executor(1),
_invertThreads(SequencedTaskExecutor::create(invert_executor, 2)),
_pushThreads(SequencedTaskExecutor::create(push_executor, 2)),
index(setup.make_all_index_schema(), setup, *_invertThreads, *_pushThreads),
diff --git a/searchlib/src/tests/postinglistbm/stress_runner.cpp b/searchlib/src/tests/postinglistbm/stress_runner.cpp
index 09272a60e75..179e4f49ef4 100644
--- a/searchlib/src/tests/postinglistbm/stress_runner.cpp
+++ b/searchlib/src/tests/postinglistbm/stress_runner.cpp
@@ -160,7 +160,7 @@ StressMaster::StressMaster(vespalib::Rand48 &rnd,
{
LOG(info, "StressMaster::StressMaster()");
- _threadPool = new FastOS_ThreadPool(128_Ki, 400);
+ _threadPool = new FastOS_ThreadPool(400);
}
StressMaster::~StressMaster()
diff --git a/searchlib/src/tests/tensor/hnsw_index/stress_hnsw_mt.cpp b/searchlib/src/tests/tensor/hnsw_index/stress_hnsw_mt.cpp
index d8bfb1de9a9..ecf310798af 100644
--- a/searchlib/src/tests/tensor/hnsw_index/stress_hnsw_mt.cpp
+++ b/searchlib/src/tests/tensor/hnsw_index/stress_hnsw_mt.cpp
@@ -18,7 +18,6 @@
#include <vespa/searchlib/tensor/random_level_generator.h>
#include <vespa/searchlib/tensor/vector_bundle.h>
#include <vespa/vespalib/data/input.h>
-#include <vespa/vespalib/data/memory_input.h>
#include <vespa/vespalib/data/slime/slime.h>
#include <vespa/vespalib/gtest/gtest.h>
#include <vespa/vespalib/util/blockingthreadstackexecutor.h>
@@ -254,8 +253,8 @@ public:
vectors(),
gen_handler(),
index(),
- multi_prepare_workers(10, 128_Ki, 50),
- write_thread(1, 128_Ki, 500)
+ multi_prepare_workers(10, 50),
+ write_thread(1, 500)
{
loaded_vectors.load();
}
diff --git a/searchlib/src/tests/transactionlog/translogclient_test.cpp b/searchlib/src/tests/transactionlog/translogclient_test.cpp
index a922795d570..1eae8489c75 100644
--- a/searchlib/src/tests/transactionlog/translogclient_test.cpp
+++ b/searchlib/src/tests/transactionlog/translogclient_test.cpp
@@ -484,7 +484,7 @@ struct TLS {
TransLogServer tls;
TLS(const vespalib::string &name, int listenPort, const vespalib::string &baseDir,
const common::FileHeaderContext &fileHeaderContext, const DomainConfig & cfg, size_t maxThreads = 4)
- : threadPool(64_Ki),
+ : threadPool(),
transport(),
tls(transport, name, listenPort, baseDir, fileHeaderContext, cfg, maxThreads)
{
diff --git a/searchlib/src/tests/transactionlogstress/translogstress.cpp b/searchlib/src/tests/transactionlogstress/translogstress.cpp
index 9988f3e171a..eb457f312e6 100644
--- a/searchlib/src/tests/transactionlogstress/translogstress.cpp
+++ b/searchlib/src/tests/transactionlogstress/translogstress.cpp
@@ -698,7 +698,7 @@ TransLogStress::main(int argc, char **argv)
}
// start transaction log server
- FastOS_ThreadPool threadPool(256_Ki);
+ FastOS_ThreadPool threadPool;
FNET_Transport transport;
DummyFileHeaderContext fileHeaderContext;
TransLogServer tls(transport, "server", 17897, ".", fileHeaderContext, DomainConfig().setPartSizeLimit(_cfg.domainPartSize));
diff --git a/searchlib/src/vespa/searchlib/transactionlog/domain.cpp b/searchlib/src/vespa/searchlib/transactionlog/domain.cpp
index a0ee40e6674..49fa7041533 100644
--- a/searchlib/src/vespa/searchlib/transactionlog/domain.cpp
+++ b/searchlib/src/vespa/searchlib/transactionlog/domain.cpp
@@ -9,6 +9,7 @@
#include <vespa/vespalib/util/destructor_callbacks.h>
#include <vespa/vespalib/util/size_literals.h>
#include <vespa/vespalib/util/retain_guard.h>
+#include <vespa/vespalib/util/cpu_usage.h>
#include <vespa/fastos/file.h>
#include <algorithm>
#include <thread>
@@ -23,9 +24,12 @@ LOG_SETUP(".transactionlog.domain");
using vespalib::string;
using vespalib::make_string_short::fmt;
using vespalib::makeLambdaTask;
+using vespalib::CpuUsage;
using std::runtime_error;
using std::make_shared;
+using CpuCategory = vespalib::CpuUsage::Category;
+
namespace search::transactionlog {
namespace {
@@ -34,6 +38,7 @@ createCommitChunk(const DomainConfig &cfg) {
return std::make_unique<CommitChunk>(cfg.getChunkSizeLimit(), cfg.getChunkSizeLimit()/256);
}
+VESPA_THREAD_STACK_TAG(tls_domain_commit);
}
Domain::Domain(const string &domainName, const string & baseDir, vespalib::Executor & executor,
@@ -41,7 +46,7 @@ Domain::Domain(const string &domainName, const string & baseDir, vespalib::Execu
: _config(cfg),
_currentChunk(createCommitChunk(cfg)),
_lastSerial(0),
- _singleCommitter(std::make_unique<vespalib::ThreadStackExecutor>(1, 128_Ki)),
+ _singleCommitter(std::make_unique<vespalib::ThreadStackExecutor>(1, CpuUsage::wrap(tls_domain_commit, CpuCategory::WRITE))),
_executor(executor),
_sessionId(1),
_name(domainName),
diff --git a/searchlib/src/vespa/searchlib/transactionlog/translogclient.cpp b/searchlib/src/vespa/searchlib/transactionlog/translogclient.cpp
index b4534486e85..133fabd3e5f 100644
--- a/searchlib/src/vespa/searchlib/transactionlog/translogclient.cpp
+++ b/searchlib/src/vespa/searchlib/transactionlog/translogclient.cpp
@@ -47,7 +47,7 @@ struct RpcTask : public vespalib::Executor::Task {
}
TransLogClient::TransLogClient(FNET_Transport & transport, const vespalib::string & rpcTarget) :
- _executor(std::make_unique<vespalib::ThreadStackExecutor>(1, 128_Ki, translogclient_rpc_callback)),
+ _executor(std::make_unique<vespalib::ThreadStackExecutor>(1, translogclient_rpc_callback)),
_rpcTarget(rpcTarget),
_sessions(),
_supervisor(std::make_unique<FRT_Supervisor>(&transport)),
diff --git a/searchlib/src/vespa/searchlib/transactionlog/translogserver.cpp b/searchlib/src/vespa/searchlib/transactionlog/translogserver.cpp
index a7aa2384bbc..c6ae9c00e49 100644
--- a/searchlib/src/vespa/searchlib/transactionlog/translogserver.cpp
+++ b/searchlib/src/vespa/searchlib/transactionlog/translogserver.cpp
@@ -98,8 +98,8 @@ TransLogServer::TransLogServer(FNET_Transport & transport, const vespalib::strin
_name(name),
_baseDir(baseDir),
_domainConfig(cfg),
- _executor(maxThreads, 128_Ki, CpuUsage::wrap(tls_executor, CpuUsage::Category::WRITE)),
- _threadPool(std::make_unique<FastOS_ThreadPool>(120_Ki)),
+ _executor(maxThreads, CpuUsage::wrap(tls_executor, CpuUsage::Category::WRITE)),
+ _threadPool(std::make_unique<FastOS_ThreadPool>()),
_supervisor(std::make_unique<FRT_Supervisor>(&transport)),
_domains(),
_reqQ(),
diff --git a/slobrok/src/tests/mirrorapi/mirrorapi.cpp b/slobrok/src/tests/mirrorapi/mirrorapi.cpp
index 1d830ed928b..85ef56aeb1b 100644
--- a/slobrok/src/tests/mirrorapi/mirrorapi.cpp
+++ b/slobrok/src/tests/mirrorapi/mirrorapi.cpp
@@ -138,7 +138,7 @@ Test::Main()
cloud::config::SlobroksConfig::Slobrok slobrok;
slobrok.connectionspec = "tcp/localhost:18501";
specBuilder.slobrok.push_back(slobrok);
- FastOS_ThreadPool threadPool(0x10000);
+ FastOS_ThreadPool threadPool;
FNET_Transport transport;
FRT_Supervisor supervisor(&transport);
MirrorAPI mirror(supervisor, slobrok::ConfiguratorFactory(config::ConfigUri::createFromInstance(specBuilder)));
diff --git a/storage/src/tests/persistence/filestorage/filestormanagertest.cpp b/storage/src/tests/persistence/filestorage/filestormanagertest.cpp
index a5f5075a4d8..50ad7b54382 100644
--- a/storage/src/tests/persistence/filestorage/filestormanagertest.cpp
+++ b/storage/src/tests/persistence/filestorage/filestormanagertest.cpp
@@ -606,7 +606,7 @@ TEST_F(FileStorManagerTest, handler_paused_multi_thread) {
Document::SP doc(createDocument(content, "id:footype:testdoctype1:n=1234:bar").release());
- FastOS_ThreadPool pool(512_Ki);
+ FastOS_ThreadPool pool;
MessagePusherThread pushthread(filestorHandler, doc);
pushthread.start(pool);
diff --git a/storage/src/tests/storageserver/statereportertest.cpp b/storage/src/tests/storageserver/statereportertest.cpp
index 09df5063989..d7c5bdbf6ea 100644
--- a/storage/src/tests/storageserver/statereportertest.cpp
+++ b/storage/src/tests/storageserver/statereportertest.cpp
@@ -61,7 +61,7 @@ struct MetricClock : public metrics::MetricManager::Timer
}
StateReporterTest::StateReporterTest()
- : _threadPool(256_Ki),
+ : _threadPool(),
_clock(nullptr),
_top(),
_stateReporter()
diff --git a/storage/src/vespa/storage/distributor/distributor_stripe_pool.cpp b/storage/src/vespa/storage/distributor/distributor_stripe_pool.cpp
index 30d13e4eb1a..26ca8963783 100644
--- a/storage/src/vespa/storage/distributor/distributor_stripe_pool.cpp
+++ b/storage/src/vespa/storage/distributor/distributor_stripe_pool.cpp
@@ -8,7 +8,7 @@
namespace storage::distributor {
DistributorStripePool::DistributorStripePool(bool test_mode, PrivateCtorTag)
- : _thread_pool(512_Ki),
+ : _thread_pool(std::make_unique<FastOS_ThreadPool>()),
_n_stripe_bits(0),
_stripes(),
_threads(),
@@ -119,7 +119,7 @@ void DistributorStripePool::start(const std::vector<TickableStripe*>& stripes) {
}
std::unique_lock lock(_mutex); // Ensure _threads is visible to all started threads
for (auto& s : _stripes) {
- _threads.emplace_back(_thread_pool.NewThread(s.get()));
+ _threads.emplace_back(_thread_pool->NewThread(s.get()));
}
}
diff --git a/storage/src/vespa/storage/distributor/distributor_stripe_pool.h b/storage/src/vespa/storage/distributor/distributor_stripe_pool.h
index 605ca34a2b2..00f5f57edf9 100644
--- a/storage/src/vespa/storage/distributor/distributor_stripe_pool.h
+++ b/storage/src/vespa/storage/distributor/distributor_stripe_pool.h
@@ -1,13 +1,15 @@
// Copyright Yahoo. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
#pragma once
-#include <vespa/fastos/thread.h>
#include <vespa/vespalib/util/time.h>
#include <atomic>
#include <condition_variable>
#include <mutex>
#include <vector>
+class FastOS_ThreadInterface;
+class FastOS_ThreadPool;
+
namespace storage::distributor {
class DistributorStripeThread;
@@ -37,7 +39,7 @@ class DistributorStripePool {
using StripeVector = std::vector<std::unique_ptr<DistributorStripeThread>>;
using NativeThreadVector = std::vector<FastOS_ThreadInterface*>;
- FastOS_ThreadPool _thread_pool;
+ std::unique_ptr<FastOS_ThreadPool> _thread_pool;
uint8_t _n_stripe_bits;
StripeVector _stripes;
NativeThreadVector _threads;
diff --git a/storage/src/vespa/storage/frameworkimpl/status/statuswebserver.cpp b/storage/src/vespa/storage/frameworkimpl/status/statuswebserver.cpp
index b2bce8a1241..40738ccb60f 100644
--- a/storage/src/vespa/storage/frameworkimpl/status/statuswebserver.cpp
+++ b/storage/src/vespa/storage/frameworkimpl/status/statuswebserver.cpp
@@ -15,6 +15,9 @@
#include <vespa/log/log.h>
LOG_SETUP(".status");
+namespace {
+ VESPA_THREAD_STACK_TAG(status_web_server);
+}
namespace storage {
StatusWebServer::StatusWebServer(
@@ -81,7 +84,7 @@ void StatusWebServer::configure(std::unique_ptr<vespa::config::content::core::St
StatusWebServer::WebServer::WebServer(StatusWebServer& status, uint16_t port)
: _status(status),
_server(vespalib::Portal::create(vespalib::CryptoEngine::get_default(), port)),
- _executor(1, 256_Ki),
+ _executor(1, status_web_server),
_root(_server->bind("/", *this))
{
}
diff --git a/storage/src/vespa/storage/storageserver/rpc/shared_rpc_resources.cpp b/storage/src/vespa/storage/storageserver/rpc/shared_rpc_resources.cpp
index c98617c84bf..d3226d91f24 100644
--- a/storage/src/vespa/storage/storageserver/rpc/shared_rpc_resources.cpp
+++ b/storage/src/vespa/storage/storageserver/rpc/shared_rpc_resources.cpp
@@ -64,7 +64,7 @@ SharedRpcResources::SharedRpcResources(const config::ConfigUri& config_uri,
int rpc_server_port,
size_t rpc_thread_pool_size,
size_t rpc_events_before_wakeup)
- : _thread_pool(std::make_unique<FastOS_ThreadPool>(1024*60)),
+ : _thread_pool(std::make_unique<FastOS_ThreadPool>()),
_transport(std::make_unique<FNET_Transport>(fnet::TransportConfig(rpc_thread_pool_size).
events_before_wakeup(rpc_events_before_wakeup))),
_orb(std::make_unique<FRT_Supervisor>(_transport.get())),
diff --git a/storage/src/vespa/storageframework/defaultimplementation/thread/threadpoolimpl.cpp b/storage/src/vespa/storageframework/defaultimplementation/thread/threadpoolimpl.cpp
index 6ac4e214e5d..95959d06b54 100644
--- a/storage/src/vespa/storageframework/defaultimplementation/thread/threadpoolimpl.cpp
+++ b/storage/src/vespa/storageframework/defaultimplementation/thread/threadpoolimpl.cpp
@@ -3,7 +3,6 @@
#include "threadpoolimpl.h"
#include "threadimpl.h"
#include <vespa/vespalib/util/exceptions.h>
-#include <vespa/vespalib/util/size_literals.h>
#include <cassert>
#include <thread>
@@ -16,7 +15,7 @@ using vespalib::IllegalStateException;
namespace storage::framework::defaultimplementation {
ThreadPoolImpl::ThreadPoolImpl(Clock& clock)
- : _backendThreadPool(512_Ki),
+ : _backendThreadPool(std::make_unique<FastOS_ThreadPool>()),
_clock(clock),
_stopping(false)
{ }
@@ -45,7 +44,7 @@ ThreadPoolImpl::~ThreadPoolImpl()
}
std::this_thread::sleep_for(10ms);
}
- _backendThreadPool.Close();
+ _backendThreadPool->Close();
}
Thread::UP
diff --git a/storage/src/vespa/storageframework/defaultimplementation/thread/threadpoolimpl.h b/storage/src/vespa/storageframework/defaultimplementation/thread/threadpoolimpl.h
index dcdc0981fe4..c351eb2ddd0 100644
--- a/storage/src/vespa/storageframework/defaultimplementation/thread/threadpoolimpl.h
+++ b/storage/src/vespa/storageframework/defaultimplementation/thread/threadpoolimpl.h
@@ -3,7 +3,8 @@
#pragma once
#include <vespa/storageframework/generic/thread/threadpool.h>
-#include <vespa/fastos/thread.h>
+
+class FastOS_ThreadPool;
namespace storage::framework::defaultimplementation {
@@ -11,7 +12,7 @@ class ThreadImpl;
struct ThreadPoolImpl final : public ThreadPool
{
- FastOS_ThreadPool _backendThreadPool;
+ std::unique_ptr<FastOS_ThreadPool> _backendThreadPool;
std::vector<ThreadImpl*> _threads;
mutable std::mutex _threadVectorLock;
Clock & _clock;
@@ -25,10 +26,8 @@ public:
vespalib::duration maxProcessTime, int ticksBeforeWait,
std::optional<vespalib::CpuUsage::Category> cpu_category) override;
void visitThreads(ThreadVisitor&) const override;
-
- void registerThread(ThreadImpl&);
void unregisterThread(ThreadImpl&);
- FastOS_ThreadPool& getThreadPool() { return _backendThreadPool; }
+ FastOS_ThreadPool& getThreadPool() { return *_backendThreadPool; }
Clock& getClock() { return _clock; }
};
diff --git a/vespalib/src/tests/btree/btree-stress/btree_stress_test.cpp b/vespalib/src/tests/btree/btree-stress/btree_stress_test.cpp
index cf809e0d730..2993bae90c4 100644
--- a/vespalib/src/tests/btree/btree-stress/btree_stress_test.cpp
+++ b/vespalib/src/tests/btree/btree-stress/btree_stress_test.cpp
@@ -1,15 +1,11 @@
// Copyright Yahoo. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
#include <vespa/vespalib/btree/btree.h>
-#include <vespa/vespalib/btree/btreebuilder.h>
#include <vespa/vespalib/btree/btreenodeallocator.h>
-#include <vespa/vespalib/btree/btreeroot.h>
-#include <vespa/vespalib/btree/btreestore.h>
#include <vespa/vespalib/gtest/gtest.h>
#include <vespa/vespalib/util/lambdatask.h>
#include <vespa/vespalib/util/rand48.h>
-#include <vespa/vespalib/util/size_literals.h>
#include <vespa/vespalib/util/threadstackexecutor.h>
#include <vespa/vespalib/btree/btreenodeallocator.hpp>
@@ -235,8 +231,8 @@ Fixture<Params>::Fixture()
_generationHandler(),
_tree(),
_writeItr(_tree.begin()),
- _writer(1, 128_Ki),
- _readers(4, 128_Ki),
+ _writer(1),
+ _readers(4),
_rnd(),
_keyLimit(1000000),
_readSeed(50),
diff --git a/vespalib/src/tests/clock/clock_benchmark.cpp b/vespalib/src/tests/clock/clock_benchmark.cpp
index 249add4bc1a..a21ad2b05ef 100644
--- a/vespalib/src/tests/clock/clock_benchmark.cpp
+++ b/vespalib/src/tests/clock/clock_benchmark.cpp
@@ -134,7 +134,7 @@ main(int , char *argv[])
uint64_t frequency = atoll(argv[1]);
uint32_t numThreads = atoi(argv[2]);
uint64_t samples = atoll(argv[3]);
- FastOS_ThreadPool pool(0x10000);
+ FastOS_ThreadPool pool;
NSValue nsValue;
NSVolatile nsVolatile;
NSAtomic nsAtomic;
diff --git a/vespalib/src/tests/coro/lazy/lazy_test.cpp b/vespalib/src/tests/coro/lazy/lazy_test.cpp
index 29aac4440fc..f6767873957 100644
--- a/vespalib/src/tests/coro/lazy/lazy_test.cpp
+++ b/vespalib/src/tests/coro/lazy/lazy_test.cpp
@@ -3,7 +3,6 @@
#include <vespa/vespalib/coro/lazy.h>
#include <vespa/vespalib/coro/completion.h>
#include <vespa/vespalib/coro/schedule.h>
-#include <vespa/vespalib/util/size_literals.h>
#include <vespa/vespalib/util/require.h>
#include <vespa/vespalib/util/gate.h>
#include <vespa/vespalib/util/threadstackexecutor.h>
@@ -102,7 +101,7 @@ TEST(LazyTest, extract_rvalue_from_lazy_in_sync_wait) {
}
TEST(LazyTest, calculate_result_in_another_thread) {
- vespalib::ThreadStackExecutor executor(1, 128_Ki);
+ vespalib::ThreadStackExecutor executor(1);
auto result = sync_wait(try_schedule_on(executor, make_lazy(7)));
EXPECT_EQ(result.first, true);
EXPECT_EQ(result.second, 7);
@@ -111,13 +110,13 @@ TEST(LazyTest, calculate_result_in_another_thread) {
}
TEST(LazyTest, exceptions_are_propagated) {
- vespalib::ThreadStackExecutor executor(1, 128_Ki);
+ vespalib::ThreadStackExecutor executor(1);
auto lazy = try_schedule_on(executor, forward_value(will_throw()));
EXPECT_THROW(sync_wait(std::move(lazy)), vespalib::RequireFailedException);
}
TEST(LazyTest, not_able_to_switch_thread_if_executor_is_shut_down) {
- vespalib::ThreadStackExecutor executor(1, 128_Ki);
+ vespalib::ThreadStackExecutor executor(1);
executor.shutdown();
auto result = sync_wait(try_schedule_on(executor, make_lazy(7)));
EXPECT_EQ(result.first, false);
@@ -129,7 +128,7 @@ TEST(LazyTest, not_able_to_switch_thread_if_executor_is_shut_down) {
TEST(LazyTest, async_wait_with_lambda) {
Gate gate;
Received<int> result;
- vespalib::ThreadStackExecutor executor(1, 128_Ki);
+ vespalib::ThreadStackExecutor executor(1);
auto lazy = schedule_on(executor, make_lazy(7));
async_wait(std::move(lazy), [&](auto res)
{
@@ -143,7 +142,7 @@ TEST(LazyTest, async_wait_with_lambda) {
TEST(LazyTest, async_wait_with_error) {
Gate gate;
Received<int> result;
- vespalib::ThreadStackExecutor executor(1, 128_Ki);
+ vespalib::ThreadStackExecutor executor(1);
auto lazy = schedule_on(executor, will_throw());
async_wait(std::move(lazy), [&](auto res)
{
@@ -157,7 +156,7 @@ TEST(LazyTest, async_wait_with_error) {
TEST(LazyTest, async_wait_with_move_only_result) {
Gate gate;
Received<std::unique_ptr<int>> result;
- vespalib::ThreadStackExecutor executor(1, 128_Ki);
+ vespalib::ThreadStackExecutor executor(1);
auto lazy = schedule_on(executor, move_only_int());
async_wait(std::move(lazy), [&](auto res)
{
@@ -178,7 +177,7 @@ struct Refs {
TEST(LazyTest, async_wait_with_move_only_result_and_move_only_lambda) {
Gate gate;
Received<std::unique_ptr<int>> result;
- vespalib::ThreadStackExecutor executor(1, 128_Ki);
+ vespalib::ThreadStackExecutor executor(1);
auto lazy = schedule_on(executor, move_only_int());
async_wait(std::move(lazy), [refs = std::make_unique<Refs>(gate,result)](auto res)
{
diff --git a/vespalib/src/tests/datastore/sharded_hash_map/sharded_hash_map_test.cpp b/vespalib/src/tests/datastore/sharded_hash_map/sharded_hash_map_test.cpp
index 4c3fe1756c5..eec3a6e6188 100644
--- a/vespalib/src/tests/datastore/sharded_hash_map/sharded_hash_map_test.cpp
+++ b/vespalib/src/tests/datastore/sharded_hash_map/sharded_hash_map_test.cpp
@@ -135,8 +135,8 @@ DataStoreShardedHashTest::DataStoreShardedHashTest()
_allocator({}),
_store(_allocator.get_data_store()),
_hash_map(std::make_unique<MyCompare>(_store)),
- _writer(1, 128_Ki),
- _readers(4, 128_Ki),
+ _writer(1),
+ _readers(4),
_rnd(),
_keyLimit(1000000),
_read_seed(50),
diff --git a/vespalib/src/tests/executor/blocking_executor_stress.cpp b/vespalib/src/tests/executor/blocking_executor_stress.cpp
index 7292e374559..870c8cee920 100644
--- a/vespalib/src/tests/executor/blocking_executor_stress.cpp
+++ b/vespalib/src/tests/executor/blocking_executor_stress.cpp
@@ -33,7 +33,7 @@ struct MyTask : Executor::Task {
}
};
-TEST_MT_F("stress test block thread stack executor", 8, BlockingThreadStackExecutor(4, 128000, 1000))
+TEST_MT_F("stress test block thread stack executor", 8, BlockingThreadStackExecutor(4, 1000))
{
size_t loop_cnt = 100;
for (size_t i = 0; i < loop_cnt; ++i) {
diff --git a/vespalib/src/tests/executor/blockingthreadstackexecutor_test.cpp b/vespalib/src/tests/executor/blockingthreadstackexecutor_test.cpp
index d811ded9e95..8342257f3e7 100644
--- a/vespalib/src/tests/executor/blockingthreadstackexecutor_test.cpp
+++ b/vespalib/src/tests/executor/blockingthreadstackexecutor_test.cpp
@@ -48,7 +48,7 @@ struct Fixture
Gate blockedExecuteGate;
Fixture(uint32_t taskLimit, uint32_t tasksToWaitFor)
- : executor(1, 128000, taskLimit),
+ : executor(1, taskLimit),
workersEntryGate(),
workersExitLatch(tasksToWaitFor),
blockedExecuteGate()
@@ -123,14 +123,14 @@ vespalib::string get_worker_stack_trace(BlockingThreadStackExecutor &executor) {
VESPA_THREAD_STACK_TAG(my_stack_tag);
-TEST_F("require that executor has appropriate default thread stack tag", BlockingThreadStackExecutor(1, 128_Ki, 10)) {
+TEST_F("require that executor has appropriate default thread stack tag", BlockingThreadStackExecutor(1, 10)) {
vespalib::string trace = get_worker_stack_trace(f1);
if (!EXPECT_TRUE(trace.find("unnamed_blocking_executor") != vespalib::string::npos)) {
fprintf(stderr, "%s\n", trace.c_str());
}
}
-TEST_F("require that executor thread stack tag can be set", BlockingThreadStackExecutor(1, 128_Ki, 10, my_stack_tag)) {
+TEST_F("require that executor thread stack tag can be set", BlockingThreadStackExecutor(1, 10, my_stack_tag)) {
vespalib::string trace = get_worker_stack_trace(f1);
if (!EXPECT_TRUE(trace.find("my_stack_tag") != vespalib::string::npos)) {
fprintf(stderr, "%s\n", trace.c_str());
@@ -140,7 +140,7 @@ TEST_F("require that executor thread stack tag can be set", BlockingThreadStackE
TEST_F("require that tasks posted from internal worker thread will not block executor", TimeBomb(60)) {
size_t cnt = 0;
Gate fork_done;
- BlockingThreadStackExecutor executor(1, 128_Ki, 10);
+ BlockingThreadStackExecutor executor(1, 10);
struct IncTask : Executor::Task {
size_t &cnt;
IncTask(size_t &cnt_in) : cnt(cnt_in) {}
diff --git a/vespalib/src/tests/executor/stress_test.cpp b/vespalib/src/tests/executor/stress_test.cpp
index 869cc05fa4b..923b7eea3a5 100644
--- a/vespalib/src/tests/executor/stress_test.cpp
+++ b/vespalib/src/tests/executor/stress_test.cpp
@@ -105,7 +105,7 @@ Test::Main()
fprintf(stderr, "calibrating task size...\n");
uint32_t taskSize = calibrate(ms_per_task);
fprintf(stderr, "calibrated task size: %u\n", taskSize);
- ThreadStackExecutor executor(threads, 128000, 5000 + threads);
+ ThreadStackExecutor executor(threads, 5000 + threads);
{
Gate gate;
CountDownLatch latch(threads);
diff --git a/vespalib/src/tests/executor/threadstackexecutor_test.cpp b/vespalib/src/tests/executor/threadstackexecutor_test.cpp
index 688d98ff032..7d77e224fe5 100644
--- a/vespalib/src/tests/executor/threadstackexecutor_test.cpp
+++ b/vespalib/src/tests/executor/threadstackexecutor_test.cpp
@@ -39,7 +39,7 @@ struct MyState {
CountDownLatch latch; // to wait for workers
ThreadStackExecutor executor;
bool checked;
- MyState() : gate(), latch(10), executor(NUM_THREADS, 128000, 20), checked(false)
+ MyState() : gate(), latch(10), executor(NUM_THREADS, 20), checked(false)
{
MyTask::resetStats();
}
@@ -132,7 +132,7 @@ struct WaitState {
std::vector<Gate> block_task;
std::vector<Gate> wait_done;
WaitState(size_t num_threads)
- : executor(num_threads / 2, 128000), block_task(num_threads - 2), wait_done(num_threads - 1)
+ : executor(num_threads / 2), block_task(num_threads - 2), wait_done(num_threads - 1)
{
for (auto &gate: block_task) {
auto result = executor.execute(std::make_unique<WaitTask>(gate));
@@ -175,14 +175,14 @@ vespalib::string get_worker_stack_trace(ThreadStackExecutor &executor) {
VESPA_THREAD_STACK_TAG(my_stack_tag);
-TEST_F("require that executor has appropriate default thread stack tag", ThreadStackExecutor(1, 128_Ki)) {
+TEST_F("require that executor has appropriate default thread stack tag", ThreadStackExecutor(1)) {
vespalib::string trace = get_worker_stack_trace(f1);
if (!EXPECT_TRUE(trace.find("unnamed_nonblocking_executor") != vespalib::string::npos)) {
fprintf(stderr, "%s\n", trace.c_str());
}
}
-TEST_F("require that executor thread stack tag can be set", ThreadStackExecutor(1, 128_Ki, my_stack_tag)) {
+TEST_F("require that executor thread stack tag can be set", ThreadStackExecutor(1, my_stack_tag)) {
vespalib::string trace = get_worker_stack_trace(f1);
if (!EXPECT_TRUE(trace.find("my_stack_tag") != vespalib::string::npos)) {
fprintf(stderr, "%s\n", trace.c_str());
@@ -215,7 +215,7 @@ TEST("require that stats can be accumulated") {
}
TEST("Test that utilization is computed") {
- ThreadStackExecutor executor(1, 128_Ki);
+ ThreadStackExecutor executor(1);
std::this_thread::sleep_for(1s);
auto stats = executor.getStats();
EXPECT_GREATER(0.50, stats.getUtil());
diff --git a/vespalib/src/tests/util/generationhandler_stress/generation_handler_stress_test.cpp b/vespalib/src/tests/util/generationhandler_stress/generation_handler_stress_test.cpp
index fd2769fd8b1..1cc54da7f2e 100644
--- a/vespalib/src/tests/util/generationhandler_stress/generation_handler_stress_test.cpp
+++ b/vespalib/src/tests/util/generationhandler_stress/generation_handler_stress_test.cpp
@@ -97,7 +97,7 @@ Fixture::Fixture()
: ::testing::Test(),
_generationHandler(),
_readThreads(1),
- _writer(1, 128_Ki),
+ _writer(1),
_readers(),
_doneWriteWork(0),
_doneReadWork(0),
@@ -131,7 +131,7 @@ Fixture::set_read_threads(uint32_t read_threads)
_readers->shutdown();
}
_readThreads = read_threads;
- _readers = std::make_unique<ThreadStackExecutor>(read_threads, 128_Ki);
+ _readers = std::make_unique<ThreadStackExecutor>(read_threads);
}
void
diff --git a/vespalib/src/tests/util/rcuvector/rcuvector_test.cpp b/vespalib/src/tests/util/rcuvector/rcuvector_test.cpp
index 5d6ec3050da..b842d009ce8 100644
--- a/vespalib/src/tests/util/rcuvector/rcuvector_test.cpp
+++ b/vespalib/src/tests/util/rcuvector/rcuvector_test.cpp
@@ -420,8 +420,8 @@ StressFixture::StressFixture()
stop_read(false),
read_area(1000),
generation_handler(),
- writer(1, 128_Ki),
- readers(4, 128_Ki)
+ writer(1),
+ readers(4)
{
arr.ensure_size(read_area, AtomicIntWrapper(0));
}
diff --git a/vespalib/src/vespa/vespalib/net/async_resolver.cpp b/vespalib/src/vespa/vespalib/net/async_resolver.cpp
index 7eab9d7c13c..bc0a2cc8085 100644
--- a/vespalib/src/vespa/vespalib/net/async_resolver.cpp
+++ b/vespalib/src/vespa/vespalib/net/async_resolver.cpp
@@ -151,7 +151,7 @@ AsyncResolver::SP AsyncResolver::_shared_resolver(nullptr);
AsyncResolver::AsyncResolver(HostResolver::SP resolver, size_t num_threads)
: _resolver(std::move(resolver)),
- _executor(std::make_unique<ThreadStackExecutor>(num_threads, 128_Ki, async_resolver_executor_thread))
+ _executor(std::make_unique<ThreadStackExecutor>(num_threads, async_resolver_executor_thread))
{
}
diff --git a/vespalib/src/vespa/vespalib/testkit/test_hook.cpp b/vespalib/src/vespa/vespalib/testkit/test_hook.cpp
index 109f70bd2ad..4e33897d869 100644
--- a/vespalib/src/vespa/vespalib/testkit/test_hook.cpp
+++ b/vespalib/src/vespa/vespalib/testkit/test_hook.cpp
@@ -19,7 +19,7 @@ struct FastOSTestThreadRunner : FastOS_Runnable {
struct FastOSTestThreadFactory : TestThreadFactory {
FastOS_ThreadPool threadPool;
- FastOSTestThreadFactory() : threadPool(256_Ki) {}
+ FastOSTestThreadFactory() : threadPool() {}
void createThread(TestThreadEntry &entry) override {
threadPool.NewThread(new FastOSTestThreadRunner(entry), 0);
}
diff --git a/vespalib/src/vespa/vespalib/util/adaptive_sequenced_executor.cpp b/vespalib/src/vespa/vespalib/util/adaptive_sequenced_executor.cpp
index e7f43de8f92..14a235f7257 100644
--- a/vespalib/src/vespa/vespalib/util/adaptive_sequenced_executor.cpp
+++ b/vespalib/src/vespa/vespalib/util/adaptive_sequenced_executor.cpp
@@ -54,7 +54,7 @@ AdaptiveSequencedExecutor::Self::~Self()
AdaptiveSequencedExecutor::ThreadTools::ThreadTools(AdaptiveSequencedExecutor &parent_in)
: parent(parent_in),
- pool(std::make_unique<FastOS_ThreadPool>(STACK_SIZE)),
+ pool(std::make_unique<FastOS_ThreadPool>()),
allow_worker_exit()
{
}
diff --git a/vespalib/src/vespa/vespalib/util/blockingthreadstackexecutor.cpp b/vespalib/src/vespa/vespalib/util/blockingthreadstackexecutor.cpp
index a80af57d900..f40390bbf83 100644
--- a/vespalib/src/vespa/vespalib/util/blockingthreadstackexecutor.cpp
+++ b/vespalib/src/vespa/vespalib/util/blockingthreadstackexecutor.cpp
@@ -21,15 +21,15 @@ BlockingThreadStackExecutor::wakeup(unique_lock &, std::condition_variable & con
cond.notify_all();
}
-BlockingThreadStackExecutor::BlockingThreadStackExecutor(uint32_t threads, uint32_t stackSize, uint32_t taskLimit)
- : ThreadStackExecutorBase(stackSize, taskLimit, unnamed_blocking_executor)
+BlockingThreadStackExecutor::BlockingThreadStackExecutor(uint32_t threads, uint32_t taskLimit)
+ : ThreadStackExecutorBase(taskLimit, unnamed_blocking_executor)
{
start(threads);
}
-BlockingThreadStackExecutor::BlockingThreadStackExecutor(uint32_t threads, uint32_t stackSize, uint32_t taskLimit,
+BlockingThreadStackExecutor::BlockingThreadStackExecutor(uint32_t threads, uint32_t taskLimit,
init_fun_t init_function)
- : ThreadStackExecutorBase(stackSize, taskLimit, std::move(init_function))
+ : ThreadStackExecutorBase(taskLimit, std::move(init_function))
{
start(threads);
}
diff --git a/vespalib/src/vespa/vespalib/util/blockingthreadstackexecutor.h b/vespalib/src/vespa/vespalib/util/blockingthreadstackexecutor.h
index 7a095f8c70a..7d2791aa068 100644
--- a/vespalib/src/vespa/vespalib/util/blockingthreadstackexecutor.h
+++ b/vespalib/src/vespa/vespalib/util/blockingthreadstackexecutor.h
@@ -11,26 +11,23 @@ namespace vespalib {
**/
class BlockingThreadStackExecutor : public ThreadStackExecutorBase
{
-private:
+public:
bool acceptNewTask(unique_lock & guard, std::condition_variable & cond) override;
void wakeup(unique_lock & guard, std::condition_variable &) override;
-public:
/**
* Create a new blocking thread stack executor. The task limit specifies
* the maximum number of tasks that are currently handled by this
* executor. Trying to execute more tasks will block.
*
* @param threads number of worker threads (concurrent tasks)
- * @param stackSize stack size per worker thread
* @param taskLimit upper limit on accepted tasks
**/
- BlockingThreadStackExecutor(uint32_t threads, uint32_t stackSize, uint32_t taskLimit);
+ BlockingThreadStackExecutor(uint32_t threads, uint32_t taskLimit);
// same as above, but enables you to specify a custom function
// used to wrap the main loop of all worker threads
- BlockingThreadStackExecutor(uint32_t threads, uint32_t stackSize, uint32_t taskLimit,
- init_fun_t init_function);
+ BlockingThreadStackExecutor(uint32_t threads, uint32_t taskLimit, init_fun_t init_function);
~BlockingThreadStackExecutor() override;
};
diff --git a/vespalib/src/vespa/vespalib/util/sequencedtaskexecutor.cpp b/vespalib/src/vespa/vespalib/util/sequencedtaskexecutor.cpp
index e12d2065d9f..7538e2acb50 100644
--- a/vespalib/src/vespa/vespalib/util/sequencedtaskexecutor.cpp
+++ b/vespalib/src/vespa/vespalib/util/sequencedtaskexecutor.cpp
@@ -14,7 +14,6 @@ namespace vespalib {
namespace {
-constexpr uint32_t stackSize = 128_Ki;
constexpr uint8_t MAGIC = 255;
constexpr uint32_t NUM_PERFECT_PER_EXECUTOR = 8;
constexpr uint16_t INVALID_KEY = 0x8000;
@@ -77,9 +76,9 @@ SequencedTaskExecutor::create(Runnable::init_fun_t func, uint32_t threads, uint3
executors.push_back(std::make_unique<SingleExecutor>(func, taskLimit, is_task_limit_hard, watermark, 100ms));
} else {
if (is_task_limit_hard) {
- executors.push_back(std::make_unique<BlockingThreadStackExecutor>(1, stackSize, taskLimit, func));
+ executors.push_back(std::make_unique<BlockingThreadStackExecutor>(1, taskLimit, func));
} else {
- executors.push_back(std::make_unique<ThreadStackExecutor>(1, stackSize, func));
+ executors.push_back(std::make_unique<ThreadStackExecutor>(1, func));
}
}
}
diff --git a/vespalib/src/vespa/vespalib/util/shutdownguard.cpp b/vespalib/src/vespa/vespalib/util/shutdownguard.cpp
index 12e58898c06..e3e56dc78cb 100644
--- a/vespalib/src/vespa/vespalib/util/shutdownguard.cpp
+++ b/vespalib/src/vespa/vespalib/util/shutdownguard.cpp
@@ -24,7 +24,7 @@ void ShutdownGuard::Run(FastOS_ThreadInterface *, void *)
ShutdownGuard::ShutdownGuard(duration millis) :
FastOS_Runnable(),
- _pool(STACK_SIZE, 1),
+ _pool(1),
_dieAtTime(steady_clock::now() + millis)
{
_pool.NewThread(this);
diff --git a/vespalib/src/vespa/vespalib/util/thread.cpp b/vespalib/src/vespa/vespalib/util/thread.cpp
index ffa9f385967..82ba441420d 100644
--- a/vespalib/src/vespa/vespalib/util/thread.cpp
+++ b/vespalib/src/vespa/vespalib/util/thread.cpp
@@ -32,7 +32,7 @@ Thread::Proxy::~Proxy() = default;
Thread::Thread(Runnable &runnable, init_fun_t init_fun_in)
: _proxy(*this, runnable, std::move(init_fun_in)),
- _pool(STACK_SIZE, 1),
+ _pool(1),
_lock(),
_cond(),
_stopped(false),
diff --git a/vespalib/src/vespa/vespalib/util/threadstackexecutor.cpp b/vespalib/src/vespa/vespalib/util/threadstackexecutor.cpp
index a975db04a2e..9ca6c643d75 100644
--- a/vespalib/src/vespa/vespalib/util/threadstackexecutor.cpp
+++ b/vespalib/src/vespa/vespalib/util/threadstackexecutor.cpp
@@ -17,16 +17,22 @@ ThreadStackExecutor::wakeup(unique_lock &, std::condition_variable &)
{
}
-ThreadStackExecutor::ThreadStackExecutor(uint32_t threads, uint32_t stackSize,
- uint32_t taskLimit)
- : ThreadStackExecutorBase(stackSize, taskLimit, unnamed_nonblocking_executor)
+ThreadStackExecutor::ThreadStackExecutor(uint32_t threads)
+ : ThreadStackExecutor(threads, unnamed_nonblocking_executor)
+{ }
+
+ThreadStackExecutor::ThreadStackExecutor(uint32_t threads, uint32_t taskLimit)
+ : ThreadStackExecutorBase(taskLimit, unnamed_nonblocking_executor)
{
start(threads);
}
-ThreadStackExecutor::ThreadStackExecutor(uint32_t threads, uint32_t stackSize,
- init_fun_t init_function, uint32_t taskLimit)
- : ThreadStackExecutorBase(stackSize, taskLimit, std::move(init_function))
+ThreadStackExecutor::ThreadStackExecutor(uint32_t threads, init_fun_t init_function)
+ : ThreadStackExecutor(threads, std::move(init_function), 0xffffffff)
+{ }
+
+ThreadStackExecutor::ThreadStackExecutor(uint32_t threads, init_fun_t init_function, uint32_t taskLimit)
+ : ThreadStackExecutorBase(taskLimit, std::move(init_function))
{
start(threads);
}
diff --git a/vespalib/src/vespa/vespalib/util/threadstackexecutor.h b/vespalib/src/vespa/vespalib/util/threadstackexecutor.h
index 687ecfbac0b..9011799ece4 100644
--- a/vespalib/src/vespa/vespalib/util/threadstackexecutor.h
+++ b/vespalib/src/vespa/vespalib/util/threadstackexecutor.h
@@ -15,7 +15,6 @@ public:
bool acceptNewTask(unique_lock &, std::condition_variable &) override;
void wakeup(unique_lock &, std::condition_variable &) override;
-public:
/**
* Create a new thread stack executor. The task limit specifies
* the maximum number of tasks that are currently handled by this
@@ -23,21 +22,16 @@ public:
* greater than 0.
*
* @param threads number of worker threads (concurrent tasks)
- * @param stackSize stack size per worker thread
* @param taskLimit upper limit on accepted tasks
**/
- ThreadStackExecutor(uint32_t threads, uint32_t stackSize,
- uint32_t taskLimit = 0xffffffff);
+ ThreadStackExecutor(uint32_t threads, uint32_t taskLimit);
+ ThreadStackExecutor(uint32_t threads);
// same as above, but enables you to specify a custom function
// used to wrap the main loop of all worker threads
- ThreadStackExecutor(uint32_t threads, uint32_t stackSize,
- init_fun_t init_function,
- uint32_t taskLimit = 0xffffffff);
+ ThreadStackExecutor(uint32_t threads, init_fun_t init_function, uint32_t taskLimit);
+ ThreadStackExecutor(uint32_t threads, init_fun_t init_function);
- /**
- * Will invoke cleanup.
- **/
~ThreadStackExecutor() override;
};
diff --git a/vespalib/src/vespa/vespalib/util/threadstackexecutorbase.cpp b/vespalib/src/vespa/vespalib/util/threadstackexecutorbase.cpp
index 133350f3d56..8b6427d9391 100644
--- a/vespalib/src/vespa/vespalib/util/threadstackexecutorbase.cpp
+++ b/vespalib/src/vespa/vespalib/util/threadstackexecutorbase.cpp
@@ -152,12 +152,10 @@ ThreadStackExecutorBase::run()
//-----------------------------------------------------------------------------
-ThreadStackExecutorBase::ThreadStackExecutorBase(uint32_t stackSize,
- uint32_t taskLimit,
- init_fun_t init_fun)
+ThreadStackExecutorBase::ThreadStackExecutorBase(uint32_t taskLimit, init_fun_t init_fun)
: SyncableThreadExecutor(),
Runnable(),
- _pool(std::make_unique<FastOS_ThreadPool>(stackSize)),
+ _pool(std::make_unique<FastOS_ThreadPool>()),
_lock(),
_cond(),
_stats(),
diff --git a/vespalib/src/vespa/vespalib/util/threadstackexecutorbase.h b/vespalib/src/vespa/vespalib/util/threadstackexecutorbase.h
index c3552cfe579..501fde92f4c 100644
--- a/vespalib/src/vespa/vespalib/util/threadstackexecutorbase.h
+++ b/vespalib/src/vespa/vespalib/util/threadstackexecutorbase.h
@@ -150,13 +150,11 @@ protected:
* executor. Both the number of threads and the task limit must be
* greater than 0.
*
- * @param stackSize stack size per worker thread
* @param taskLimit upper limit on accepted tasks
* @param init_fun custom function used to wrap the main loop of
* each worker thread.
**/
- ThreadStackExecutorBase(uint32_t stackSize, uint32_t taskLimit,
- init_fun_t init_fun);
+ ThreadStackExecutorBase(uint32_t taskLimit, init_fun_t init_fun);
/**
* This will start the theads. This is to avoid starting tasks in
diff --git a/vespalog/src/test/threads/testthreads.cpp b/vespalog/src/test/threads/testthreads.cpp
index d1767dd70ff..8fab6bcd638 100644
--- a/vespalog/src/test/threads/testthreads.cpp
+++ b/vespalog/src/test/threads/testthreads.cpp
@@ -93,7 +93,7 @@ int
ThreadTester::Main()
{
std::cerr << "Testing that logging is threadsafe. 5 sec test.\n";
- FastOS_ThreadPool pool(128 * 1024);
+ FastOS_ThreadPool pool;
const int numWriters = 30;
const int numLoggers = 10;
diff --git a/vespamalloc/src/tests/allocfree/allocfree.cpp b/vespamalloc/src/tests/allocfree/allocfree.cpp
index 32f881a4b96..ea6f2105d27 100644
--- a/vespamalloc/src/tests/allocfree/allocfree.cpp
+++ b/vespamalloc/src/tests/allocfree/allocfree.cpp
@@ -73,7 +73,7 @@ int Test::Main() {
}
TEST_INIT("allocfree_test");
- FastOS_ThreadPool pool(128000);
+ FastOS_ThreadPool pool;
std::map<int, std::shared_ptr<FreeWorker> > freeWorkers;
std::map<int, std::shared_ptr<MallocWorker> > mallocWorkers;
diff --git a/vespamalloc/src/tests/allocfree/linklist.cpp b/vespamalloc/src/tests/allocfree/linklist.cpp
index 1bffb76b2ee..f3f23d726fd 100644
--- a/vespamalloc/src/tests/allocfree/linklist.cpp
+++ b/vespamalloc/src/tests/allocfree/linklist.cpp
@@ -124,7 +124,7 @@ int Test::Main() {
ASSERT_EQUAL(1024ul, sizeof(List));
- FastOS_ThreadPool pool(128000);
+ FastOS_ThreadPool pool;
List::AtomicHeadPtr sharedList(List::HeadPtr(nullptr, 1));
fprintf(stderr, "Start populating list\n");
for (size_t i=0; i < NumBlocks; i++) {
diff --git a/vespamalloc/src/tests/test.cpp b/vespamalloc/src/tests/test.cpp
index e19e119bb33..f413412dff0 100644
--- a/vespamalloc/src/tests/test.cpp
+++ b/vespamalloc/src/tests/test.cpp
@@ -32,7 +32,7 @@ private:
int main(int, char *[])
{
- FastOS_ThreadPool threadPool(512*1024);
+ FastOS_ThreadPool threadPool;
printf("Main stack(%p)\n", &threadPool);
Thread context;