aboutsummaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
authorHenning Baldersheim <balder@yahoo-inc.com>2024-01-30 16:06:18 +0100
committerGitHub <noreply@github.com>2024-01-30 16:06:18 +0100
commit343f5555420577915cde15b76476e3e46699d20a (patch)
tree7e76cadd80d84e770e01617f8845d326691421dd
parent60816fd636024deb0263681d64dc89cb5594ccc0 (diff)
parent47a3ffe8aaeef1240db829164ffd63958a10019f (diff)
Merge pull request #30105 from vespa-engine/balder/use_per_document_throttled_delete_bucketv8.296.15
GC leftovers from use_per_document_throttled_delete_bucket
-rw-r--r--config-model/src/main/java/com/yahoo/vespa/model/admin/metricsproxy/MetricsProxyContainerCluster.java2
-rw-r--r--config-model/src/main/java/com/yahoo/vespa/model/content/storagecluster/FileStorProducer.java1
-rw-r--r--configdefinitions/src/vespa/stor-filestor.def29
-rw-r--r--storage/src/tests/persistence/filestorage/filestormanagertest.cpp86
-rw-r--r--storage/src/vespa/storage/persistence/asynchandler.cpp23
-rw-r--r--storage/src/vespa/storage/persistence/asynchandler.h1
-rw-r--r--storage/src/vespa/storage/persistence/filestorage/filestormanager.cpp4
-rw-r--r--storage/src/vespa/storage/persistence/persistencehandler.cpp21
-rw-r--r--storage/src/vespa/storage/persistence/persistencehandler.h3
9 files changed, 43 insertions, 127 deletions
diff --git a/config-model/src/main/java/com/yahoo/vespa/model/admin/metricsproxy/MetricsProxyContainerCluster.java b/config-model/src/main/java/com/yahoo/vespa/model/admin/metricsproxy/MetricsProxyContainerCluster.java
index cbeac3cc849..0d85696d503 100644
--- a/config-model/src/main/java/com/yahoo/vespa/model/admin/metricsproxy/MetricsProxyContainerCluster.java
+++ b/config-model/src/main/java/com/yahoo/vespa/model/admin/metricsproxy/MetricsProxyContainerCluster.java
@@ -203,7 +203,7 @@ public class MetricsProxyContainerCluster extends ContainerCluster<MetricsProxyC
private Optional<String> getSystemName() {
Monitoring monitoring = getMonitoringService();
- return monitoring != null && ! monitoring.getClustername().equals("") ?
+ return monitoring != null && !monitoring.getClustername().isEmpty() ?
Optional.of(monitoring.getClustername()) : Optional.empty();
}
diff --git a/config-model/src/main/java/com/yahoo/vespa/model/content/storagecluster/FileStorProducer.java b/config-model/src/main/java/com/yahoo/vespa/model/content/storagecluster/FileStorProducer.java
index 56ca23523b6..4f283d6d9c3 100644
--- a/config-model/src/main/java/com/yahoo/vespa/model/content/storagecluster/FileStorProducer.java
+++ b/config-model/src/main/java/com/yahoo/vespa/model/content/storagecluster/FileStorProducer.java
@@ -74,7 +74,6 @@ public class FileStorProducer implements StorFilestorConfig.Producer {
builder.num_response_threads(responseNumThreads);
builder.response_sequencer_type(responseSequencerType);
builder.use_async_message_handling_on_schedule(useAsyncMessageHandlingOnSchedule);
- builder.use_per_document_throttled_delete_bucket(true);
var throttleBuilder = new StorFilestorConfig.Async_operation_throttler.Builder();
builder.async_operation_throttler(throttleBuilder);
}
diff --git a/configdefinitions/src/vespa/stor-filestor.def b/configdefinitions/src/vespa/stor-filestor.def
index 950797f8bc2..13e304823ff 100644
--- a/configdefinitions/src/vespa/stor-filestor.def
+++ b/configdefinitions/src/vespa/stor-filestor.def
@@ -93,32 +93,3 @@ async_operation_throttler.resize_rate double default=3.0
## level, i.e. per ApplyBucketDiff message, regardless of how many document operations
## are contained within.
async_operation_throttler.throttle_individual_merge_feed_ops bool default=true
-
-## Specify throttling used for async persistence operations. This throttling takes place
-## before operations are dispatched to Proton and serves as a limiter for how many
-## operations may be in flight in Proton's internal queues.
-##
-## - UNLIMITED is, as it says on the tin, unlimited. Offers no actual throttling, but
-## has near zero overhead and never blocks.
-## - DYNAMIC uses DynamicThrottlePolicy under the hood and will block if the window
-## is full (if a blocking throttler API call is invoked).
-##
-## TODO deprecate in favor of the async_operation_throttler struct instead.
-async_operation_throttler_type enum { UNLIMITED, DYNAMIC } default=DYNAMIC
-
-## Specifies the extent the throttling window is increased by when the async throttle
-## policy has decided that more concurrent operations are desirable. Also affects the
-## _minimum_ size of the throttling window; its size is implicitly set to max(this config
-## value, number of threads).
-##
-## Only applies if async_operation_throttler_type == DYNAMIC.
-## DEPRECATED! use the async_operation_throttler struct instead
-async_operation_dynamic_throttling_window_increment int default=20 restart
-
-## If set, DeleteBucket operations are internally expanded to an individually persistence-
-## throttled remove per document stored in the bucket. This makes the cost model of
-## executing a DeleteBucket symmetrical with feeding the documents to the bucket in the
-## first place.
-##
-## This is a live config.
-use_per_document_throttled_delete_bucket bool default=true
diff --git a/storage/src/tests/persistence/filestorage/filestormanagertest.cpp b/storage/src/tests/persistence/filestorage/filestormanagertest.cpp
index 4911ad88692..4846c90397a 100644
--- a/storage/src/tests/persistence/filestorage/filestormanagertest.cpp
+++ b/storage/src/tests/persistence/filestorage/filestormanagertest.cpp
@@ -5,6 +5,7 @@
#include <tests/common/teststorageapp.h>
#include <tests/persistence/filestorage/forwardingmessagesender.h>
#include <vespa/config/common/exceptions.h>
+#include <memory>
#include <vespa/config/helper/configgetter.hpp>
#include <vespa/document/fieldset/fieldsets.h>
#include <vespa/document/repo/documenttyperepo.h>
@@ -92,16 +93,16 @@ struct FileStorTestBase : Test {
std::unique_ptr<vdstestlib::DirConfig> config;
std::unique_ptr<vdstestlib::DirConfig> config2;
std::unique_ptr<vdstestlib::DirConfig> smallConfig;
- const uint32_t _waitTime;
+ const int32_t _waitTime;
const document::DocumentType* _testdoctype1;
- FileStorTestBase() : _node(), _waitTime(LONG_WAITTIME) {}
+ FileStorTestBase();
~FileStorTestBase() override;
void SetUp() override;
void TearDown() override;
- void createBucket(document::BucketId bid) {
+ void createBucket(document::BucketId bid) const {
_node->getPersistenceProvider().createBucket(makeSpiBucket(bid));
StorBucketDatabase::WrappedEntry entry(
@@ -110,13 +111,13 @@ struct FileStorTestBase : Test {
entry.write();
}
- document::Document::UP createDocument(const std::string& content, const std::string& id) {
+ document::Document::UP createDocument(const std::string& content, const std::string& id) const {
return _node->getTestDocMan().createDocument(content, id);
}
std::shared_ptr<api::PutCommand> make_put_command(StorageMessage::Priority pri = 20,
const std::string& docid = "id:foo:testdoctype1::bar",
- Timestamp timestamp = 100) {
+ Timestamp timestamp = 100) const {
Document::SP doc(createDocument("my content", docid));
auto bucket = make_bucket_for_doc(doc->getId());
auto cmd = std::make_shared<api::PutCommand>(bucket, std::move(doc), timestamp);
@@ -124,7 +125,7 @@ struct FileStorTestBase : Test {
return cmd;
}
- std::shared_ptr<api::GetCommand> make_get_command(StorageMessage::Priority pri,
+ static std::shared_ptr<api::GetCommand> make_get_command(StorageMessage::Priority pri,
const std::string& docid = "id:foo:testdoctype1::bar") {
document::DocumentId did(docid);
auto bucket = make_bucket_for_doc(did);
@@ -139,12 +140,11 @@ struct FileStorTestBase : Test {
auto clusterStateBundle = _node->getStateUpdater().getClusterStateBundle();
const auto &clusterState = *clusterStateBundle->getBaselineClusterState();
uint16_t distributor(
- _node->getDistribution()->getIdealDistributorNode(
- clusterState, bucket));
+ _node->getDistribution()->getIdealDistributorNode(clusterState, bucket));
return distributor == distributorIndex;
}
- document::BucketId getFirstBucketNotOwnedByDistributor(uint16_t distributor) {
+ document::BucketId getFirstBucketNotOwnedByDistributor(uint16_t distributor) const {
for (int i = 0; i < 1000; ++i) {
if (!ownsBucket(distributor, document::BucketId(16, i))) {
return document::BucketId(16, i);
@@ -153,28 +153,25 @@ struct FileStorTestBase : Test {
return document::BucketId(0);
}
- spi::dummy::DummyPersistence& getDummyPersistence() {
+ spi::dummy::DummyPersistence& getDummyPersistence() const {
return dynamic_cast<spi::dummy::DummyPersistence&>(_node->getPersistenceProvider());
}
- void setClusterState(const std::string& state) {
- _node->getStateUpdater().setClusterState(
- lib::ClusterState::CSP(
- new lib::ClusterState(state)));
+ void setClusterState(const std::string& state) const {
+ _node->getStateUpdater().setClusterState(lib::ClusterState::CSP(new lib::ClusterState(state)));
}
void setupDisks() {
std::string rootOfRoot = "filestormanagertest";
- config.reset(new vdstestlib::DirConfig(getStandardConfig(true, rootOfRoot)));
+ config = std::make_unique<vdstestlib::DirConfig>(getStandardConfig(true, rootOfRoot));
- config2.reset(new vdstestlib::DirConfig(*config));
+ config2 = std::make_unique<vdstestlib::DirConfig>(*config);
config2->getConfig("stor-server").set("root_folder", rootOfRoot + "-vdsroot.2");
config2->getConfig("stor-devices").set("root_folder", rootOfRoot + "-vdsroot.2");
config2->getConfig("stor-server").set("node_index", "1");
- smallConfig.reset(new vdstestlib::DirConfig(*config));
- vdstestlib::DirConfig::Config& c(
- smallConfig->getConfig("stor-filestor", true));
+ smallConfig = std::make_unique<vdstestlib::DirConfig>(*config);
+ vdstestlib::DirConfig::Config& c(smallConfig->getConfig("stor-filestor", true));
c.set("initial_index_read", "128");
c.set("use_direct_io", "false");
c.set("maximum_gap_to_read_through", "64");
@@ -202,11 +199,16 @@ struct FileStorTestBase : Test {
std::shared_ptr<api::StorageMessage> cmd,
const Metric& metric);
- auto& thread_metrics_of(FileStorManager& manager) {
+ static auto& thread_metrics_of(FileStorManager& manager) {
return manager.get_metrics().threads[0];
}
};
+FileStorTestBase::FileStorTestBase()
+ : _node(),
+ _waitTime(LONG_WAITTIME),
+ _testdoctype1(nullptr)
+{}
FileStorTestBase::~FileStorTestBase() = default;
std::unique_ptr<DiskThread>
@@ -243,7 +245,8 @@ struct FileStorHandlerComponents {
FileStorMetrics metrics;
std::unique_ptr<FileStorHandler> filestorHandler;
- FileStorHandlerComponents(FileStorTestBase& test, uint32_t threadsPerDisk = 1)
+ explicit FileStorHandlerComponents(FileStorTestBase& test) : FileStorHandlerComponents(test, 1) {}
+ FileStorHandlerComponents(FileStorTestBase& test, uint32_t threadsPerDisk)
: top(),
dummyManager(new DummyStorageLink),
messageSender(*dummyManager),
@@ -269,7 +272,7 @@ struct PersistenceHandlerComponents : public FileStorHandlerComponents {
BucketOwnershipNotifier bucketOwnershipNotifier;
std::unique_ptr<PersistenceHandler> persistenceHandler;
- PersistenceHandlerComponents(FileStorTestBase& test)
+ explicit PersistenceHandlerComponents(FileStorTestBase& test)
: FileStorHandlerComponents(test),
executor(test._node->executor()),
component(test._node->getComponentRegister(), "test"),
@@ -311,7 +314,6 @@ FileStorTestBase::TearDown()
}
struct FileStorManagerTest : public FileStorTestBase {
- void do_test_delete_bucket(bool use_throttled_delete);
};
TEST_F(FileStorManagerTest, header_only_put) {
@@ -767,8 +769,8 @@ TEST_F(FileStorManagerTest, priority) {
document::BucketIdFactory factory;
// Create buckets in separate, initial pass to avoid races with puts
- for (uint32_t i=0; i<documents.size(); ++i) {
- document::BucketId bucket(16, factory.getBucketId(documents[i]->getId()).getRawId());
+ for (const auto & document : documents) {
+ document::BucketId bucket(16, factory.getBucketId(document->getId()).getRawId());
_node->getPersistenceProvider().createBucket(makeSpiBucket(bucket));
}
@@ -980,9 +982,9 @@ TEST_F(FileStorManagerTest, split_single_group) {
}
// Test that the documents are all still there
- for (uint32_t i=0; i<documents.size(); ++i) {
+ for (const auto & document : documents) {
document::BucketId bucket(17, state ? 0x10001 : 0x00001);
- auto cmd = std::make_shared<api::GetCommand>(makeDocumentBucket(bucket), documents[i]->getId(), document::AllFields::NAME);
+ auto cmd = std::make_shared<api::GetCommand>(makeDocumentBucket(bucket), document->getId(), document::AllFields::NAME);
cmd->setAddress(_storage3);
filestorHandler.schedule(cmd);
filestorHandler.flush(true);
@@ -1159,8 +1161,8 @@ TEST_F(FileStorManagerTest, join) {
// Perform a join, check that other files are gone
{
auto cmd = std::make_shared<api::JoinBucketsCommand>(makeDocumentBucket(document::BucketId(16, 1)));
- cmd->getSourceBuckets().emplace_back(document::BucketId(17, 0x00001));
- cmd->getSourceBuckets().emplace_back(document::BucketId(17, 0x10001));
+ cmd->getSourceBuckets().emplace_back(17, 0x00001);
+ cmd->getSourceBuckets().emplace_back(17, 0x10001);
filestorHandler.schedule(cmd);
filestorHandler.flush(true);
ASSERT_EQ(1, top.getNumReplies());
@@ -1371,12 +1373,11 @@ TEST_F(FileStorManagerTest, remove_location) {
}
}
-void FileStorManagerTest::do_test_delete_bucket(bool use_throttled_delete) {
+TEST_F(FileStorManagerTest, delete_bucket) {
TestFileStorComponents c(*this);
auto config_uri = config::ConfigUri(config->getConfigId());
StorFilestorConfigBuilder my_config(*config_from<StorFilestorConfig>(config_uri));
- my_config.usePerDocumentThrottledDeleteBucket = use_throttled_delete;
c.manager->on_configure(my_config);
auto& top = c.top;
@@ -1421,23 +1422,12 @@ void FileStorManagerTest::do_test_delete_bucket(bool use_throttled_delete) {
StorBucketDatabase::WrappedEntry entry(_node->getStorageBucketDatabase().get(bid, "foo"));
EXPECT_FALSE(entry.exists());
}
- if (use_throttled_delete) {
- auto& metrics = thread_metrics_of(*c.manager)->remove_by_gid;
- EXPECT_EQ(metrics.failed.getValue(), 0);
- EXPECT_EQ(metrics.count.getValue(), 1);
- // We can't reliably test the actual latency here without wiring mock clock bumping into
- // the async remove by GID execution, but we can at least test that we updated the metric.
- EXPECT_EQ(metrics.latency.getCount(), 1);
- }
-}
-
-// TODO remove once throttled behavior is the default
-TEST_F(FileStorManagerTest, delete_bucket_legacy) {
- do_test_delete_bucket(false);
-}
-
-TEST_F(FileStorManagerTest, delete_bucket_throttled) {
- do_test_delete_bucket(true);
+ auto& metrics = thread_metrics_of(*c.manager)->remove_by_gid;
+ EXPECT_EQ(metrics.failed.getValue(), 0);
+ EXPECT_EQ(metrics.count.getValue(), 1);
+ // We can't reliably test the actual latency here without wiring mock clock bumping into
+ // the async remove by GID execution, but we can at least test that we updated the metric.
+ EXPECT_EQ(metrics.latency.getCount(), 1);
}
TEST_F(FileStorManagerTest, delete_bucket_rejects_outdated_bucket_info) {
diff --git a/storage/src/vespa/storage/persistence/asynchandler.cpp b/storage/src/vespa/storage/persistence/asynchandler.cpp
index a69f9e55afb..725cf2c7511 100644
--- a/storage/src/vespa/storage/persistence/asynchandler.cpp
+++ b/storage/src/vespa/storage/persistence/asynchandler.cpp
@@ -225,29 +225,6 @@ AsyncHandler::on_delete_bucket_complete(const document::Bucket& bucket) const {
}
}
-MessageTracker::UP
-AsyncHandler::handleDeleteBucket(api::DeleteBucketCommand& cmd, MessageTracker::UP tracker) const
-{
- tracker->setMetric(_env._metrics.deleteBuckets);
- LOG(debug, "DeletingBucket(%s)", cmd.getBucketId().toString().c_str());
- if (_env._fileStorHandler.isMerging(cmd.getBucket())) {
- _env._fileStorHandler.clearMergeStatus(cmd.getBucket(),
- api::ReturnCode(api::ReturnCode::ABORTED, "Bucket was deleted during the merge"));
- }
- spi::Bucket bucket(cmd.getBucket());
- if (!checkProviderBucketInfoMatches(bucket, cmd.getBucketInfo())) {
- return tracker;
- }
-
- auto task = makeResultTask([this, tracker = std::move(tracker), bucket = cmd.getBucket()]([[maybe_unused]] spi::Result::UP ignored) {
- // TODO Even if an non OK response can not be handled sanely we might probably log a message, or increment a metric
- on_delete_bucket_complete(bucket);
- tracker->sendReply();
- });
- _spi.deleteBucketAsync(bucket, std::make_unique<ResultTaskOperationDone>(_sequencedExecutor, cmd.getBucketId(), std::move(task)));
- return tracker;
-}
-
namespace {
class GatherBucketMetadata : public BucketProcessor::EntryProcessor {
diff --git a/storage/src/vespa/storage/persistence/asynchandler.h b/storage/src/vespa/storage/persistence/asynchandler.h
index 1433176036b..c78dfe6282d 100644
--- a/storage/src/vespa/storage/persistence/asynchandler.h
+++ b/storage/src/vespa/storage/persistence/asynchandler.h
@@ -31,7 +31,6 @@ public:
MessageTrackerUP handleUpdate(api::UpdateCommand& cmd, MessageTrackerUP tracker) const;
MessageTrackerUP handleRunTask(RunTaskCommand & cmd, MessageTrackerUP tracker) const;
MessageTrackerUP handleSetBucketState(api::SetBucketStateCommand& cmd, MessageTrackerUP tracker) const;
- MessageTrackerUP handleDeleteBucket(api::DeleteBucketCommand& cmd, MessageTrackerUP tracker) const;
MessageTrackerUP handle_delete_bucket_throttling(api::DeleteBucketCommand& cmd, MessageTrackerUP tracker) const;
MessageTrackerUP handleCreateBucket(api::CreateBucketCommand& cmd, MessageTrackerUP tracker) const;
MessageTrackerUP handleRemoveLocation(api::RemoveLocationCommand& cmd, MessageTrackerUP tracker) const;
diff --git a/storage/src/vespa/storage/persistence/filestorage/filestormanager.cpp b/storage/src/vespa/storage/persistence/filestorage/filestormanager.cpp
index 61c7da6e286..90703050009 100644
--- a/storage/src/vespa/storage/persistence/filestorage/filestormanager.cpp
+++ b/storage/src/vespa/storage/persistence/filestorage/filestormanager.cpp
@@ -212,8 +212,7 @@ FileStorManager::on_configure(const StorFilestorConfig& config)
_use_async_message_handling_on_schedule = config.useAsyncMessageHandlingOnSchedule;
_host_info_reporter.set_noise_level(config.resourceUsageReporterNoiseLevel);
- const bool use_dynamic_throttling = ((config.asyncOperationThrottlerType == StorFilestorConfig::AsyncOperationThrottlerType::DYNAMIC) ||
- (config.asyncOperationThrottler.type == StorFilestorConfig::AsyncOperationThrottler::Type::DYNAMIC));
+ const bool use_dynamic_throttling = (config.asyncOperationThrottler.type == StorFilestorConfig::AsyncOperationThrottler::Type::DYNAMIC);
const bool throttle_merge_feed_ops = config.asyncOperationThrottler.throttleIndividualMergeFeedOps;
if (!liveUpdate) {
@@ -248,7 +247,6 @@ FileStorManager::on_configure(const StorFilestorConfig& config)
std::lock_guard guard(_lock);
for (auto& ph : _persistenceHandlers) {
ph->set_throttle_merge_feed_ops(throttle_merge_feed_ops);
- ph->set_use_per_document_throttled_delete_bucket(config.usePerDocumentThrottledDeleteBucket);
}
}
}
diff --git a/storage/src/vespa/storage/persistence/persistencehandler.cpp b/storage/src/vespa/storage/persistence/persistencehandler.cpp
index 78ad6eac0e2..29d39845f5a 100644
--- a/storage/src/vespa/storage/persistence/persistencehandler.cpp
+++ b/storage/src/vespa/storage/persistence/persistencehandler.cpp
@@ -24,8 +24,7 @@ PersistenceHandler::PersistenceHandler(vespalib::ISequencedTaskExecutor & sequen
cfg.commonMergeChainOptimalizationMinimumSize),
_asyncHandler(_env, provider, bucketOwnershipNotifier, sequencedExecutor, component.getBucketIdFactory()),
_splitJoinHandler(_env, provider, bucketOwnershipNotifier, cfg.enableMultibitSplitOptimalization),
- _simpleHandler(_env, provider, component.getBucketIdFactory()),
- _use_per_op_throttled_delete_bucket(false)
+ _simpleHandler(_env, provider, component.getBucketIdFactory())
{
}
@@ -69,11 +68,7 @@ PersistenceHandler::handleCommandSplitByType(api::StorageCommand& msg, MessageTr
case api::MessageType::CREATEBUCKET_ID:
return _asyncHandler.handleCreateBucket(static_cast<api::CreateBucketCommand&>(msg), std::move(tracker));
case api::MessageType::DELETEBUCKET_ID:
- if (use_per_op_throttled_delete_bucket()) {
- return _asyncHandler.handle_delete_bucket_throttling(static_cast<api::DeleteBucketCommand&>(msg), std::move(tracker));
- } else {
- return _asyncHandler.handleDeleteBucket(static_cast<api::DeleteBucketCommand&>(msg), std::move(tracker));
- }
+ return _asyncHandler.handle_delete_bucket_throttling(static_cast<api::DeleteBucketCommand&>(msg), std::move(tracker));
case api::MessageType::JOINBUCKETS_ID:
return _splitJoinHandler.handleJoinBuckets(static_cast<api::JoinBucketsCommand&>(msg), std::move(tracker));
case api::MessageType::SPLITBUCKET_ID:
@@ -114,7 +109,7 @@ PersistenceHandler::handleCommandSplitByType(api::StorageCommand& msg, MessageTr
default:
break;
}
- return MessageTracker::UP();
+ return {};
}
MessageTracker::UP
@@ -186,14 +181,4 @@ PersistenceHandler::set_throttle_merge_feed_ops(bool throttle) noexcept
_mergeHandler.set_throttle_merge_feed_ops(throttle);
}
-bool
-PersistenceHandler::use_per_op_throttled_delete_bucket() const noexcept {
- return _use_per_op_throttled_delete_bucket.load(std::memory_order_relaxed);
-}
-
-void
-PersistenceHandler::set_use_per_document_throttled_delete_bucket(bool throttle) noexcept {
- _use_per_op_throttled_delete_bucket.store(throttle, std::memory_order_relaxed);
-}
-
}
diff --git a/storage/src/vespa/storage/persistence/persistencehandler.h b/storage/src/vespa/storage/persistence/persistencehandler.h
index 9639b772a28..595815d2bb3 100644
--- a/storage/src/vespa/storage/persistence/persistencehandler.h
+++ b/storage/src/vespa/storage/persistence/persistencehandler.h
@@ -38,14 +38,12 @@ public:
const SimpleMessageHandler & simpleMessageHandler() const { return _simpleHandler; }
void set_throttle_merge_feed_ops(bool throttle) noexcept;
- void set_use_per_document_throttled_delete_bucket(bool throttle) noexcept;
private:
// Message handling functions
MessageTracker::UP handleCommandSplitByType(api::StorageCommand&, MessageTracker::UP tracker) const;
MessageTracker::UP handleReply(api::StorageReply&, MessageTracker::UP) const;
MessageTracker::UP processMessage(api::StorageMessage& msg, MessageTracker::UP tracker) const;
- [[nodiscard]] bool use_per_op_throttled_delete_bucket() const noexcept;
const framework::Clock & _clock;
PersistenceUtil _env;
@@ -54,7 +52,6 @@ private:
AsyncHandler _asyncHandler;
SplitJoinHandler _splitJoinHandler;
SimpleMessageHandler _simpleHandler;
- std::atomic<bool> _use_per_op_throttled_delete_bucket;
};
} // storage