Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Add kvstore rate limiter #2557

Merged
merged 4 commits into from
Aug 26, 2021
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
6 changes: 0 additions & 6 deletions conf/nebula-storaged.conf.default
Original file line number Diff line number Diff line change
Expand Up @@ -100,9 +100,3 @@
--rocksdb_column_family_options={"write_buffer_size":"67108864","max_write_buffer_number":"4","max_bytes_for_level_base":"268435456"}
# rocksdb BlockBasedTableOptions in json, each name and value of option is string, given as "option_name":"option_value" separated by comma
--rocksdb_block_based_table_options={"block_size":"8192"}

############## Runtime global Options ##############
# Index rebuild batch number
--rebuild_index_batch_num=256
# Index rebuild processing interval, microsecond.
--rebuild_index_process_interval=3000
11 changes: 4 additions & 7 deletions conf/nebula-storaged.conf.production
Original file line number Diff line number Diff line change
Expand Up @@ -100,10 +100,7 @@
--enable_rocksdb_prefix_filtering=false

############### misc ####################
--max_handlers_per_req=1

############## Runtime global Options ##############
# Index rebuild batch number
--rebuild_index_batch_num=256
# Index rebuild processing interval, microsecond.
--rebuild_index_process_interval=3000
--snapshot_part_rate_limit=8388608
--snapshot_batch_size=1048576
--rebuild_index_part_rate_limit=4194304
--rebuild_index_batch_size=1048576
2 changes: 1 addition & 1 deletion src/kvstore/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -7,7 +7,7 @@ nebula_add_library(
NebulaStore.cpp
RocksEngineConfig.cpp
LogEncoder.cpp
SnapshotManagerImpl.cpp
NebulaSnapshotManager.cpp
plugins/elasticsearch/ESListener.cpp
)

Expand Down
12 changes: 10 additions & 2 deletions src/kvstore/LogEncoder.h
Original file line number Diff line number Diff line change
Expand Up @@ -7,6 +7,7 @@
#ifndef KVSTORE_LOGENCODER_H_
#define KVSTORE_LOGENCODER_H_

#include "common/cpp/helpers.h"
#include "kvstore/Common.h"

namespace nebula {
Expand Down Expand Up @@ -54,37 +55,44 @@ HostAddr decodeHost(LogType type, const folly::StringPiece& encoded);

int64_t getTimestamp(const folly::StringPiece& command);

class BatchHolder {
class BatchHolder : public nebula::cpp::NonCopyable, public nebula::cpp::NonMovable {
public:
BatchHolder() = default;
~BatchHolder() = default;

void put(std::string&& key, std::string&& val) {
size_ += key.size() + val.size();
auto op = std::make_tuple(
BatchLogType::OP_BATCH_PUT, std::forward<std::string>(key), std::forward<std::string>(val));
batch_.emplace_back(std::move(op));
}

void remove(std::string&& key) {
size_ += key.size();
auto op = std::make_tuple(BatchLogType::OP_BATCH_REMOVE, std::forward<std::string>(key), "");
batch_.emplace_back(std::move(op));
}

void rangeRemove(std::string&& begin, std::string&& end) {
size_ += begin.size() + end.size();
auto op = std::make_tuple(BatchLogType::OP_BATCH_REMOVE_RANGE,
std::forward<std::string>(begin),
std::forward<std::string>(end));
batch_.emplace_back(std::move(op));
}

void clear() { batch_.clear(); }
void reserve(int32_t size) { batch_.reserve(size); }

const std::vector<std::tuple<BatchLogType, std::string, std::string>>& getBatch() {
return batch_;
}

// size of the batch, in bytes
size_t size() { return size_; }

private:
std::vector<std::tuple<BatchLogType, std::string, std::string>> batch_;
size_t size_{0};
};

} // namespace kvstore
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -4,25 +4,46 @@
* attached with Common Clause Condition 1.0, found in the LICENSES directory.
*/

#include "kvstore/SnapshotManagerImpl.h"
#include "kvstore/NebulaSnapshotManager.h"

#include "common/utils/NebulaKeyUtils.h"
#include "kvstore/LogEncoder.h"
#include "kvstore/RateLimiter.h"

DEFINE_int32(snapshot_batch_size, 1024 * 1024 * 10, "batch size for snapshot");
DEFINE_uint32(snapshot_part_rate_limit,
1024 * 1024 * 2,
"max bytes of pulling snapshot for each partition in one second");
DEFINE_uint32(snapshot_batch_size, 1024 * 512, "batch size for snapshot, in bytes");

namespace nebula {
namespace kvstore {

const int32_t kReserveNum = 1024 * 4;
void SnapshotManagerImpl::accessAllRowsInSnapshot(GraphSpaceID spaceId,
PartitionID partId,
raftex::SnapshotCallback cb) {

NebulaSnapshotManager::NebulaSnapshotManager(NebulaStore* kv) : store_(kv) {
// Snapshot rate is limited to FLAGS_snapshot_worker_threads * FLAGS_snapshot_part_rate_limit.
// So by default, the total send rate is limited to 4 * 2Mb = 8Mb.
rateLimiter_.reset(
new RateLimiter(FLAGS_snapshot_part_rate_limit, FLAGS_snapshot_part_rate_limit));
}

void NebulaSnapshotManager::accessAllRowsInSnapshot(GraphSpaceID spaceId,
PartitionID partId,
raftex::SnapshotCallback cb) {
CHECK_NOTNULL(store_);
auto tables = NebulaKeyUtils::snapshotPrefix(partId);
std::vector<std::string> data;
int64_t totalSize = 0;
int64_t totalCount = 0;
LOG(INFO) << folly::format(
"Space {} Part {} start send snapshot, rate limited to {}, batch size is {}",
spaceId,
partId,
FLAGS_snapshot_part_rate_limit,
FLAGS_snapshot_batch_size);
// raft will make sure that there won't be concurrent accessAllRowsInSnapshot of a given partition
rateLimiter_->add(spaceId, partId);
SCOPE_EXIT { rateLimiter_->remove(spaceId, partId); };

for (const auto& prefix : tables) {
if (!accessTable(spaceId, partId, prefix, cb, data, totalCount, totalSize)) {
Expand All @@ -34,13 +55,13 @@ void SnapshotManagerImpl::accessAllRowsInSnapshot(GraphSpaceID spaceId,

// Promise is set in callback. Access part of the data, and try to send to
// peers. If send failed, will return false.
bool SnapshotManagerImpl::accessTable(GraphSpaceID spaceId,
PartitionID partId,
const std::string& prefix,
raftex::SnapshotCallback& cb,
std::vector<std::string>& data,
int64_t& totalCount,
int64_t& totalSize) {
bool NebulaSnapshotManager::accessTable(GraphSpaceID spaceId,
PartitionID partId,
const std::string& prefix,
raftex::SnapshotCallback& cb,
std::vector<std::string>& data,
int64_t& totalCount,
int64_t& totalSize) {
std::unique_ptr<KVIterator> iter;
auto ret = store_->prefix(spaceId, partId, prefix, &iter);
if (ret != nebula::cpp2::ErrorCode::SUCCEEDED) {
Expand All @@ -50,9 +71,10 @@ bool SnapshotManagerImpl::accessTable(GraphSpaceID spaceId,
return false;
}
data.reserve(kReserveNum);
int32_t batchSize = 0;
size_t batchSize = 0;
while (iter && iter->valid()) {
if (batchSize >= FLAGS_snapshot_batch_size) {
rateLimiter_->consume(spaceId, partId, batchSize);
if (cb(data, totalCount, totalSize, raftex::SnapshotStatus::IN_PROGRESS)) {
data.clear();
batchSize = 0;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -4,19 +4,21 @@
* attached with Common Clause Condition 1.0, found in the LICENSES directory.
*/

#ifndef KVSTORE_SNAPSHOTMANAGERIMPL_H_
#define KVSTORE_SNAPSHOTMANAGERIMPL_H_
#pragma once

#include <folly/TokenBucket.h>

#include "common/base/Base.h"
#include "kvstore/KVStore.h"
#include "kvstore/NebulaStore.h"
#include "kvstore/RateLimiter.h"
#include "kvstore/raftex/SnapshotManager.h"

namespace nebula {
namespace kvstore {

class SnapshotManagerImpl : public raftex::SnapshotManager {
class NebulaSnapshotManager : public raftex::SnapshotManager {
public:
explicit SnapshotManagerImpl(KVStore* kv) : store_(kv) {}
explicit NebulaSnapshotManager(NebulaStore* kv);

void accessAllRowsInSnapshot(GraphSpaceID spaceId,
PartitionID partId,
Expand All @@ -31,10 +33,9 @@ class SnapshotManagerImpl : public raftex::SnapshotManager {
int64_t& totalCount,
int64_t& totalSize);

KVStore* store_;
std::unique_ptr<RateLimiter> rateLimiter_;
NebulaStore* store_;
};

} // namespace kvstore
} // namespace nebula

#endif // KVSTORE_SNAPSHOTMANAGERIMPL_H_
4 changes: 2 additions & 2 deletions src/kvstore/NebulaStore.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -14,8 +14,8 @@

#include "common/fs/FileUtils.h"
#include "common/network/NetworkUtils.h"
#include "kvstore/NebulaSnapshotManager.h"
#include "kvstore/RocksEngine.h"
#include "kvstore/SnapshotManagerImpl.h"

DEFINE_string(engine_type, "rocksdb", "rocksdb, memory...");
DEFINE_int32(custom_filter_interval_secs,
Expand Down Expand Up @@ -55,7 +55,7 @@ bool NebulaStore::init() {
bgWorkers_->start(FLAGS_num_workers, "nebula-bgworkers");
storeWorker_ = std::make_shared<thread::GenericWorker>();
CHECK(storeWorker_->start());
snapshot_.reset(new SnapshotManagerImpl(this));
snapshot_.reset(new NebulaSnapshotManager(this));
raftService_ = raftex::RaftexService::createService(ioPool_, workers_, raftAddr_.port);
if (!raftService_->start()) {
LOG(ERROR) << "Start the raft service failed";
Expand Down
70 changes: 70 additions & 0 deletions src/kvstore/RateLimiter.h
Original file line number Diff line number Diff line change
@@ -0,0 +1,70 @@
/* Copyright (c) 2021 vesoft inc. All rights reserved.
*
* This source code is licensed under Apache 2.0 License,
* attached with Common Clause Condition 1.0, found in the LICENSES directory.
*/

#pragma once

#include <folly/TokenBucket.h>

#include "common/base/Base.h"
#include "common/thrift/ThriftTypes.h"
#include "common/time/WallClock.h"

namespace nebula {
namespace kvstore {

// A simple wrapper for foly::TokenBucket, it would limit the speed to rate_ * buckets_.size().
// For now, there are two major cases: snapshot (both for balance or catch up) and rebuild index.
class RateLimiter {
public:
RateLimiter(int32_t rate, int32_t burstSize)
: rate_(static_cast<double>(rate)), burstSize_(static_cast<double>(burstSize)) {}

void add(GraphSpaceID spaceId, PartitionID partId) {
std::lock_guard<std::mutex> guard(lock_);
DCHECK(buckets_.find({spaceId, partId}) == buckets_.end());
// token will be available after 1 second, to prevent speed spike at the beginning
auto now = time::WallClock::fastNowInSec();
folly::TokenBucket bucket(rate_, burstSize_, static_cast<double>(now + 1));
buckets_.emplace(std::make_pair(spaceId, partId), std::move(bucket));
}

void remove(GraphSpaceID spaceId, PartitionID partId) {
std::lock_guard<std::mutex> guard(lock_);
DCHECK(buckets_.find({spaceId, partId}) != buckets_.end());
buckets_.erase({spaceId, partId});
}

// Caller must make sure the **the parition has been add, and won't be removed during consume.**
// Snaphot and rebuild index follow this principle by design.
void consume(GraphSpaceID spaceId, PartitionID partId, size_t toConsume) {
DCHECK(buckets_.find({spaceId, partId}) != buckets_.end());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is it redundant?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah, just to prevent misuse.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Doesn't it need to be locked here?

Copy link
Contributor Author

@critical27 critical27 Aug 26, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There won't be concurrent add/remove/consume of a {spaceId, partId}. Caller must make sure part has been added, and won't remove during consume. I'll add some comments.

auto iter = buckets_.find({spaceId, partId});
if (iter != buckets_.end()) {
if (toConsume > burstSize_) {
// consumeWithBorrowAndWait do nothing when toConsume > burstSize_, we sleep 1s instead
std::this_thread::sleep_for(std::chrono::seconds(1));
} else {
// If there are enouth tokens, consume and return immediately.
// If not, cosume anyway, but sleep enough time before return.
auto now = time::WallClock::fastNowInSec();
iter->second.consumeWithBorrowAndWait(static_cast<double>(toConsume),
static_cast<double>(now));
}
} else {
LOG_EVERY_N(WARNING, 100) << folly::format(
"Rate limiter is not available for [{},{}]", spaceId, partId);
}
}

private:
std::unordered_map<std::pair<GraphSpaceID, PartitionID>, folly::TokenBucket> buckets_;
std::mutex lock_;
double rate_;
double burstSize_;
};

} // namespace kvstore
} // namespace nebula
8 changes: 5 additions & 3 deletions src/kvstore/RocksEngineConfig.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -74,7 +74,9 @@ DEFINE_string(rocksdb_stats_level, "kExceptHistogramOrTimers", "rocksdb statisti

DEFINE_int32(num_compaction_threads, 0, "Number of total compaction threads. 0 means unlimited.");

DEFINE_int32(rate_limit, 0, "write limit in bytes per sec. The unit is MB. 0 means unlimited.");
DEFINE_int32(rocksdb_rate_limit,
0,
"write limit in bytes per sec. The unit is MB. 0 means unlimited.");

DEFINE_bool(enable_rocksdb_prefix_filtering,
false,
Expand Down Expand Up @@ -248,9 +250,9 @@ rocksdb::Status initRocksdbOptions(rocksdb::Options& baseOpts,
rocksdb::NewConcurrentTaskLimiter("compaction", FLAGS_num_compaction_threads)};
baseOpts.compaction_thread_limiter = compaction_thread_limiter;
}
if (FLAGS_rate_limit > 0) {
if (FLAGS_rocksdb_rate_limit > 0) {
static std::shared_ptr<rocksdb::RateLimiter> rate_limiter{
rocksdb::NewGenericRateLimiter(FLAGS_rate_limit * 1024 * 1024)};
rocksdb::NewGenericRateLimiter(FLAGS_rocksdb_rate_limit * 1024 * 1024)};
baseOpts.rate_limiter = rate_limiter;
}

Expand Down
2 changes: 1 addition & 1 deletion src/kvstore/raftex/test/RaftexTestBase.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -343,7 +343,7 @@ std::vector<std::shared_ptr<SnapshotManager>> snapshots(
const std::vector<std::shared_ptr<RaftexService>>& services) {
std::vector<std::shared_ptr<SnapshotManager>> snapshots;
for (auto& service : services) {
std::shared_ptr<SnapshotManager> snapshot(new test::SnapshotManagerImpl(service.get()));
std::shared_ptr<SnapshotManager> snapshot(new test::NebulaSnapshotManager(service.get()));
snapshots.emplace_back(std::move(snapshot));
}
return snapshots;
Expand Down
8 changes: 4 additions & 4 deletions src/kvstore/raftex/test/TestShard.h
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,7 @@ std::string encodeRemovePeer(const HostAddr& addr);
HostAddr decodeRemovePeer(const folly::StringPiece& log);

class TestShard : public RaftPart {
friend class SnapshotManagerImpl;
friend class NebulaSnapshotManager;

public:
TestShard(size_t idx,
Expand Down Expand Up @@ -137,13 +137,13 @@ class TestShard : public RaftPart {
std::function<void(size_t idx, const char*, TermID)> becomeLeaderCB_;
};

class SnapshotManagerImpl : public SnapshotManager {
class NebulaSnapshotManager : public SnapshotManager {
public:
explicit SnapshotManagerImpl(RaftexService* service) : service_(service) {
explicit NebulaSnapshotManager(RaftexService* service) : service_(service) {
CHECK_NOTNULL(service);
}

~SnapshotManagerImpl() { LOG(INFO) << "~SnapshotManagerImpl()"; }
~NebulaSnapshotManager() { LOG(INFO) << "~NebulaSnapshotManager()"; }

void accessAllRowsInSnapshot(GraphSpaceID spaceId,
PartitionID partId,
Expand Down
15 changes: 15 additions & 0 deletions src/kvstore/test/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -185,3 +185,18 @@ nebula_add_executable(
gtest
boost_regex
)

nebula_add_test(
NAME
rate_limiter_test
SOURCES
RateLimiterTest.cpp
OBJECTS
${KVSTORE_TEST_LIBS}
LIBRARIES
${THRIFT_LIBRARIES}
${ROCKSDB_LIBRARIES}
${PROXYGEN_LIBRARIES}
wangle
gtest
)
Loading