Skip to content

Commit

Permalink
Add no-dbsize-check command flag
Browse files Browse the repository at this point in the history
  • Loading branch information
caipengbo committed Jan 26, 2024
1 parent ebb7bf6 commit 6417414
Show file tree
Hide file tree
Showing 23 changed files with 99 additions and 61 deletions.
2 changes: 1 addition & 1 deletion src/commands/cmd_hash.cc
Original file line number Diff line number Diff line change
Expand Up @@ -432,7 +432,7 @@ REDIS_REGISTER_COMMANDS(MakeCmdAttr<CommandHGet>("hget", 3, "read-only", 1, 1, 1
MakeCmdAttr<CommandHIncrByFloat>("hincrbyfloat", 4, "write", 1, 1, 1),
MakeCmdAttr<CommandHMSet>("hset", -4, "write", 1, 1, 1),
MakeCmdAttr<CommandHSetNX>("hsetnx", -4, "write", 1, 1, 1),
MakeCmdAttr<CommandHDel>("hdel", -3, "write", 1, 1, 1),
MakeCmdAttr<CommandHDel>("hdel", -3, "write no-dbsize-check", 1, 1, 1),
MakeCmdAttr<CommandHStrlen>("hstrlen", 3, "read-only", 1, 1, 1),
MakeCmdAttr<CommandHExists>("hexists", 3, "read-only", 1, 1, 1),
MakeCmdAttr<CommandHLen>("hlen", 2, "read-only", 1, 1, 1),
Expand Down
6 changes: 3 additions & 3 deletions src/commands/cmd_json.cc
Original file line number Diff line number Diff line change
Expand Up @@ -600,15 +600,15 @@ REDIS_REGISTER_COMMANDS(MakeCmdAttr<CommandJsonSet>("json.set", 4, "write", 1, 1
MakeCmdAttr<CommandJsonType>("json.type", -2, "read-only", 1, 1, 1),
MakeCmdAttr<CommandJsonArrAppend>("json.arrappend", -4, "write", 1, 1, 1),
MakeCmdAttr<CommandJsonArrInsert>("json.arrinsert", -5, "write", 1, 1, 1),
MakeCmdAttr<CommandJsonArrTrim>("json.arrtrim", 5, "write", 1, 1, 1),
MakeCmdAttr<CommandJsonClear>("json.clear", -2, "write", 1, 1, 1),
MakeCmdAttr<CommandJsonArrTrim>("json.arrtrim", 5, "write no-dbsize-check", 1, 1, 1),
MakeCmdAttr<CommandJsonClear>("json.clear", -2, "write no-dbsize-check", 1, 1, 1),
MakeCmdAttr<CommandJsonToggle>("json.toggle", -2, "write", 1, 1, 1),
MakeCmdAttr<CommandJsonArrLen>("json.arrlen", -2, "read-only", 1, 1, 1),
MakeCmdAttr<CommandJsonMerge>("json.merge", 4, "write", 1, 1, 1),
MakeCmdAttr<CommandJsonObjkeys>("json.objkeys", -2, "read-only", 1, 1, 1),
MakeCmdAttr<CommandJsonArrPop>("json.arrpop", -2, "write", 1, 1, 1),
MakeCmdAttr<CommanderJsonArrIndex>("json.arrindex", -4, "read-only", 1, 1, 1),
MakeCmdAttr<CommandJsonDel>("json.del", -2, "write", 1, 1, 1),
MakeCmdAttr<CommandJsonDel>("json.del", -2, "write no-dbsize-check", 1, 1, 1),
// JSON.FORGET is an alias for JSON.DEL, refer: https://redis.io/commands/json.forget/
MakeCmdAttr<CommandJsonDel>("json.forget", -2, "write", 1, 1, 1),
MakeCmdAttr<CommandJsonNumIncrBy>("json.numincrby", 4, "write", 1, 1, 1),
Expand Down
4 changes: 2 additions & 2 deletions src/commands/cmd_key.cc
Original file line number Diff line number Diff line change
Expand Up @@ -350,8 +350,8 @@ REDIS_REGISTER_COMMANDS(MakeCmdAttr<CommandTTL>("ttl", 2, "read-only", 1, 1, 1),
MakeCmdAttr<CommandPExpireAt>("pexpireat", 3, "write", 1, 1, 1),
MakeCmdAttr<CommandExpireTime>("expiretime", 2, "read-only", 1, 1, 1),
MakeCmdAttr<CommandPExpireTime>("pexpiretime", 2, "read-only", 1, 1, 1),
MakeCmdAttr<CommandDel>("del", -2, "write", 1, -1, 1),
MakeCmdAttr<CommandDel>("unlink", -2, "write", 1, -1, 1),
MakeCmdAttr<CommandDel>("del", -2, "write no-dbsize-check", 1, -1, 1),
MakeCmdAttr<CommandDel>("unlink", -2, "write no-dbsize-check", 1, -1, 1),
MakeCmdAttr<CommandRename>("rename", 3, "write", 1, 2, 1),
MakeCmdAttr<CommandRenameNX>("renamenx", 3, "write", 1, 2, 1), )

Expand Down
4 changes: 2 additions & 2 deletions src/commands/cmd_list.cc
Original file line number Diff line number Diff line change
Expand Up @@ -861,9 +861,9 @@ REDIS_REGISTER_COMMANDS(MakeCmdAttr<CommandBLPop>("blpop", -3, "write no-script"
MakeCmdAttr<CommandLPush>("lpush", -3, "write", 1, 1, 1),
MakeCmdAttr<CommandLPushX>("lpushx", -3, "write", 1, 1, 1),
MakeCmdAttr<CommandLRange>("lrange", 4, "read-only", 1, 1, 1),
MakeCmdAttr<CommandLRem>("lrem", 4, "write", 1, 1, 1),
MakeCmdAttr<CommandLRem>("lrem", 4, "write no-dbsize-check", 1, 1, 1),
MakeCmdAttr<CommandLSet>("lset", 4, "write", 1, 1, 1),
MakeCmdAttr<CommandLTrim>("ltrim", 4, "write", 1, 1, 1),
MakeCmdAttr<CommandLTrim>("ltrim", 4, "write no-dbsize-check", 1, 1, 1),
MakeCmdAttr<CommandLMPop>("lmpop", -4, "write", CommandLMPop::keyRangeGen),
MakeCmdAttr<CommandRPop>("rpop", -2, "write", 1, 1, 1),
MakeCmdAttr<CommandRPopLPUSH>("rpoplpush", 3, "write", 1, 2, 1),
Expand Down
20 changes: 16 additions & 4 deletions src/commands/cmd_server.cc
Original file line number Diff line number Diff line change
Expand Up @@ -594,8 +594,16 @@ class CommandDebug : public Commander {
} else if (subcommand_ == "protocol" && args.size() == 3) {
protocol_type_ = util::ToLower(args[2]);
return Status::OK();
} else if (subcommand_ == "dbsize-limit" && args.size() == 3) {
auto val = ParseInt<int32_t>(args[2], {0, 1}, 10);
if (!val) {
return {Status::RedisParseErr, "invalid debug dbsize-limit value"};
}

dbsize_limit_ = static_cast<bool>(val);
return Status::OK();
}
return {Status::RedisInvalidCmd, "Syntax error, DEBUG SLEEP <seconds>|PROTOCOL <type>"};
return {Status::RedisInvalidCmd, "Syntax error, DEBUG SLEEP <seconds>|PROTOCOL <type>|DBSIZE-LIMIT <0|1>"};
}

Status Execute(Server *srv, Connection *conn, std::string *output) override {
Expand Down Expand Up @@ -636,8 +644,11 @@ class CommandDebug : public Commander {
"Wrong protocol type name. Please use one of the following: "
"string|integer|array|set|bignum|true|false|null");
}
} else if (subcommand_ == "dbsize-limit") {
srv->storage->SetDBSizeLimit(dbsize_limit_);
*output = redis::SimpleString("OK");
} else {
return {Status::RedisInvalidCmd, "Unknown subcommand, should be DEBUG or PROTOCOL"};
return {Status::RedisInvalidCmd, "Unknown subcommand, should be DEBUG, PROTOCOL or DBSIZE-LIMIT"};
}
return Status::OK();
}
Expand All @@ -646,6 +657,7 @@ class CommandDebug : public Commander {
std::string subcommand_;
std::string protocol_type_;
uint64_t microsecond_ = 0;
bool dbsize_limit_ = false;
};

class CommandCommand : public Commander {
Expand Down Expand Up @@ -1316,8 +1328,8 @@ REDIS_REGISTER_COMMANDS(MakeCmdAttr<CommandAuth>("auth", 2, "read-only ok-loadin
MakeCmdAttr<CommandConfig>("config", -2, "read-only", 0, 0, 0, GenerateConfigFlag),
MakeCmdAttr<CommandNamespace>("namespace", -3, "read-only exclusive", 0, 0, 0),
MakeCmdAttr<CommandKeys>("keys", 2, "read-only", 0, 0, 0),
MakeCmdAttr<CommandFlushDB>("flushdb", 1, "write", 0, 0, 0),
MakeCmdAttr<CommandFlushAll>("flushall", 1, "write", 0, 0, 0),
MakeCmdAttr<CommandFlushDB>("flushdb", 1, "write no-dbsize-check", 0, 0, 0),
MakeCmdAttr<CommandFlushAll>("flushall", 1, "write no-dbsize-check", 0, 0, 0),
MakeCmdAttr<CommandDBSize>("dbsize", -1, "read-only", 0, 0, 0),
MakeCmdAttr<CommandSlowlog>("slowlog", -2, "read-only", 0, 0, 0),
MakeCmdAttr<CommandPerfLog>("perflog", -2, "read-only", 0, 0, 0),
Expand Down
2 changes: 1 addition & 1 deletion src/commands/cmd_set.cc
Original file line number Diff line number Diff line change
Expand Up @@ -438,7 +438,7 @@ class CommandSScan : public CommandSubkeyScanBase {
};

REDIS_REGISTER_COMMANDS(MakeCmdAttr<CommandSAdd>("sadd", -3, "write", 1, 1, 1),
MakeCmdAttr<CommandSRem>("srem", -3, "write", 1, 1, 1),
MakeCmdAttr<CommandSRem>("srem", -3, "write no-dbsize-check", 1, 1, 1),
MakeCmdAttr<CommandSCard>("scard", 2, "read-only", 1, 1, 1),
MakeCmdAttr<CommandSMembers>("smembers", 2, "read-only", 1, 1, 1),
MakeCmdAttr<CommandSIsMember>("sismember", 3, "read-only", 1, 1, 1),
Expand Down
2 changes: 1 addition & 1 deletion src/commands/cmd_sortedint.cc
Original file line number Diff line number Diff line change
Expand Up @@ -250,7 +250,7 @@ class CommandSortedintRevRangeByValue : public CommandSortedintRangeByValue {
};

REDIS_REGISTER_COMMANDS(MakeCmdAttr<CommandSortedintAdd>("siadd", -3, "write", 1, 1, 1),
MakeCmdAttr<CommandSortedintRem>("sirem", -3, "write", 1, 1, 1),
MakeCmdAttr<CommandSortedintRem>("sirem", -3, "write no-dbsize-check", 1, 1, 1),
MakeCmdAttr<CommandSortedintCard>("sicard", 2, "read-only", 1, 1, 1),
MakeCmdAttr<CommandSortedintExists>("siexists", -3, "read-only", 1, 1, 1),
MakeCmdAttr<CommandSortedintRange>("sirange", -4, "read-only", 1, 1, 1),
Expand Down
4 changes: 2 additions & 2 deletions src/commands/cmd_stream.cc
Original file line number Diff line number Diff line change
Expand Up @@ -1190,14 +1190,14 @@ class CommandXSetId : public Commander {
};

REDIS_REGISTER_COMMANDS(MakeCmdAttr<CommandXAdd>("xadd", -5, "write", 1, 1, 1),
MakeCmdAttr<CommandXDel>("xdel", -3, "write", 1, 1, 1),
MakeCmdAttr<CommandXDel>("xdel", -3, "write no-dbsize-check", 1, 1, 1),
MakeCmdAttr<CommandXGroup>("xgroup", -4, "write", 2, 2, 1),
MakeCmdAttr<CommandXLen>("xlen", -2, "read-only", 1, 1, 1),
MakeCmdAttr<CommandXInfo>("xinfo", -2, "read-only", 0, 0, 0),
MakeCmdAttr<CommandXRange>("xrange", -4, "read-only", 1, 1, 1),
MakeCmdAttr<CommandXRevRange>("xrevrange", -2, "read-only", 1, 1, 1),
MakeCmdAttr<CommandXRead>("xread", -4, "read-only", 0, 0, 0),
MakeCmdAttr<CommandXTrim>("xtrim", -4, "write", 1, 1, 1),
MakeCmdAttr<CommandXTrim>("xtrim", -4, "write no-dbsize-check", 1, 1, 1),
MakeCmdAttr<CommandXSetId>("xsetid", -3, "write", 1, 1, 1))

} // namespace redis
2 changes: 1 addition & 1 deletion src/commands/cmd_string.cc
Original file line number Diff line number Diff line change
Expand Up @@ -626,7 +626,7 @@ REDIS_REGISTER_COMMANDS(
MakeCmdAttr<CommandGetSet>("getset", 3, "write", 1, 1, 1),
MakeCmdAttr<CommandGetRange>("getrange", 4, "read-only", 1, 1, 1),
MakeCmdAttr<CommandSubStr>("substr", 4, "read-only", 1, 1, 1),
MakeCmdAttr<CommandGetDel>("getdel", 2, "write", 1, 1, 1),
MakeCmdAttr<CommandGetDel>("getdel", 2, "write no-dbsize-check", 1, 1, 1),
MakeCmdAttr<CommandSetRange>("setrange", 4, "write", 1, 1, 1),
MakeCmdAttr<CommandMGet>("mget", -2, "read-only", 1, -1, 1),
MakeCmdAttr<CommandAppend>("append", 3, "write", 1, 1, 1), MakeCmdAttr<CommandSet>("set", -3, "write", 1, 1, 1),
Expand Down
8 changes: 4 additions & 4 deletions src/commands/cmd_zset.cc
Original file line number Diff line number Diff line change
Expand Up @@ -1534,10 +1534,10 @@ REDIS_REGISTER_COMMANDS(MakeCmdAttr<CommandZAdd>("zadd", -4, "write", 1, 1, 1),
MakeCmdAttr<CommandZRevRangeByLex>("zrevrangebylex", -4, "read-only", 1, 1, 1),
MakeCmdAttr<CommandZRangeByScore>("zrangebyscore", -4, "read-only", 1, 1, 1),
MakeCmdAttr<CommandZRank>("zrank", -3, "read-only", 1, 1, 1),
MakeCmdAttr<CommandZRem>("zrem", -3, "write", 1, 1, 1),
MakeCmdAttr<CommandZRemRangeByRank>("zremrangebyrank", 4, "write", 1, 1, 1),
MakeCmdAttr<CommandZRemRangeByScore>("zremrangebyscore", 4, "write", 1, 1, 1),
MakeCmdAttr<CommandZRemRangeByLex>("zremrangebylex", 4, "write", 1, 1, 1),
MakeCmdAttr<CommandZRem>("zrem", -3, "write no-dbsize-check", 1, 1, 1),
MakeCmdAttr<CommandZRemRangeByRank>("zremrangebyrank", 4, "write no-dbsize-check", 1, 1, 1),
MakeCmdAttr<CommandZRemRangeByScore>("zremrangebyscore", 4, "write no-dbsize-check", 1, 1, 1),
MakeCmdAttr<CommandZRemRangeByLex>("zremrangebylex", 4, "write no-dbsize-check", 1, 1, 1),
MakeCmdAttr<CommandZRevRangeByScore>("zrevrangebyscore", -4, "read-only", 1, 1, 1),
MakeCmdAttr<CommandZRevRank>("zrevrank", -3, "read-only", 1, 1, 1),
MakeCmdAttr<CommandZScore>("zscore", 3, "read-only", 1, 1, 1),
Expand Down
27 changes: 15 additions & 12 deletions src/commands/commander.h
Original file line number Diff line number Diff line change
Expand Up @@ -52,18 +52,19 @@ class Connection;
struct CommandAttributes;

enum CommandFlags : uint64_t {
kCmdWrite = 1ULL << 0, // "write" flag
kCmdReadOnly = 1ULL << 1, // "read-only" flag
kCmdReplication = 1ULL << 2, // "replication" flag
kCmdPubSub = 1ULL << 3, // "pub-sub" flag
kCmdScript = 1ULL << 4, // "script" flag
kCmdLoading = 1ULL << 5, // "ok-loading" flag
kCmdMulti = 1ULL << 6, // "multi" flag
kCmdExclusive = 1ULL << 7, // "exclusive" flag
kCmdNoMulti = 1ULL << 8, // "no-multi" flag
kCmdNoScript = 1ULL << 9, // "no-script" flag
kCmdROScript = 1ULL << 10, // "ro-script" flag for read-only script commands
kCmdCluster = 1ULL << 11, // "cluster" flag
kCmdWrite = 1ULL << 0, // "write" flag
kCmdReadOnly = 1ULL << 1, // "read-only" flag
kCmdReplication = 1ULL << 2, // "replication" flag
kCmdPubSub = 1ULL << 3, // "pub-sub" flag
kCmdScript = 1ULL << 4, // "script" flag
kCmdLoading = 1ULL << 5, // "ok-loading" flag
kCmdMulti = 1ULL << 6, // "multi" flag
kCmdExclusive = 1ULL << 7, // "exclusive" flag
kCmdNoMulti = 1ULL << 8, // "no-multi" flag
kCmdNoScript = 1ULL << 9, // "no-script" flag
kCmdROScript = 1ULL << 10, // "ro-script" flag for read-only script commands
kCmdCluster = 1ULL << 11, // "cluster" flag
kCmdNoDBSizeCheck = 1ULL << 12, // "no-dbsize-check" flag
};

class Commander {
Expand Down Expand Up @@ -178,6 +179,8 @@ inline uint64_t ParseCommandFlags(const std::string &description, const std::str
flags |= kCmdROScript;
else if (flag == "cluster")
flags |= kCmdCluster;
else if (flag == "no-dbsize-check")
flags |= kCmdNoDBSizeCheck;
else {
std::cout << fmt::format("Encountered non-existent flag '{}' in command {} in command attribute parsing", flag,
cmd_name)
Expand Down
5 changes: 5 additions & 0 deletions src/server/redis_connection.cc
Original file line number Diff line number Diff line change
Expand Up @@ -501,6 +501,11 @@ void Connection::ExecuteCommands(std::deque<CommandTokens> *to_process_cmds) {
continue;
}

if ((cmd_flags & kCmdWrite) && !(cmd_flags & kCmdNoDBSizeCheck) && srv_->storage->ReachedDBSizeLimit()) {
Reply(redis::Error("ERR write command not allowed when reached max-db-size."));
continue;
}

if (!config->slave_serve_stale_data && srv_->IsSlave() && cmd_name != "info" && cmd_name != "slaveof" &&
srv_->GetReplicationState() != kReplConnected) {
Reply(
Expand Down
2 changes: 1 addition & 1 deletion src/storage/redis_db.cc
Original file line number Diff line number Diff line change
Expand Up @@ -200,7 +200,7 @@ rocksdb::Status Database::MDel(const std::vector<Slice> &keys, uint64_t *deleted

if (*deleted_cnt == 0) return rocksdb::Status::OK();

return storage_->Write(storage_->DefaultWriteOptions(), batch->GetWriteBatch(), /*ignore_max_db_size*/ true);
return storage_->Write(storage_->DefaultWriteOptions(), batch->GetWriteBatch());
}

rocksdb::Status Database::Exists(const std::vector<Slice> &keys, int *ret) {
Expand Down
20 changes: 7 additions & 13 deletions src/storage/storage.cc
Original file line number Diff line number Diff line change
Expand Up @@ -601,21 +601,15 @@ void Storage::MultiGet(const rocksdb::ReadOptions &options, rocksdb::ColumnFamil
}
}

rocksdb::Status Storage::Write(const rocksdb::WriteOptions &options, rocksdb::WriteBatch *updates,
bool ignore_max_db_size) {
rocksdb::Status Storage::Write(const rocksdb::WriteOptions &options, rocksdb::WriteBatch *updates) {
if (is_txn_mode_) {
// The batch won't be flushed until the transaction was committed or rollback
return rocksdb::Status::OK();
}
return writeToDB(options, updates, ignore_max_db_size);
return writeToDB(options, updates);
}

rocksdb::Status Storage::writeToDB(const rocksdb::WriteOptions &options, rocksdb::WriteBatch *updates,
bool ignore_max_db_size) {
if (!ignore_max_db_size && db_size_limit_reached_) {
return rocksdb::Status::SpaceLimit();
}

rocksdb::Status Storage::writeToDB(const rocksdb::WriteOptions &options, rocksdb::WriteBatch *updates) {
// Put replication id logdata at the end of write batch
if (replid_.length() == kReplIdLength) {
updates->PutLogData(ServerLogData(kReplIdLog, replid_).Encode());
Expand All @@ -628,7 +622,7 @@ rocksdb::Status Storage::Delete(const rocksdb::WriteOptions &options, rocksdb::C
const rocksdb::Slice &key) {
auto batch = GetWriteBatchBase();
batch->Delete(cf_handle, key);
return Write(options, batch->GetWriteBatch(), /*ignore_max_db_size*/ true);
return Write(options, batch->GetWriteBatch());
}

rocksdb::Status Storage::DeleteRange(const std::string &first_key, const std::string &last_key) {
Expand All @@ -644,7 +638,7 @@ rocksdb::Status Storage::DeleteRange(const std::string &first_key, const std::st
return s;
}

return Write(write_opts_, batch->GetWriteBatch(), /*ignore_max_db_size*/ true);
return Write(write_opts_, batch->GetWriteBatch());
}

rocksdb::Status Storage::FlushScripts(const rocksdb::WriteOptions &options, rocksdb::ColumnFamilyHandle *cf_handle) {
Expand All @@ -659,7 +653,7 @@ rocksdb::Status Storage::FlushScripts(const rocksdb::WriteOptions &options, rock
return s;
}

return Write(options, batch->GetWriteBatch(), /*ignore_max_db_size*/ true);
return Write(options, batch->GetWriteBatch());
}

Status Storage::ReplicaApplyWriteBatch(std::string &&raw_batch) {
Expand Down Expand Up @@ -818,7 +812,7 @@ Status Storage::WriteToPropagateCF(const std::string &key, const std::string &va
auto batch = GetWriteBatchBase();
auto cf = GetCFHandle(kPropagateColumnFamilyName);
batch->Put(cf, key, value);
auto s = Write(write_opts_, batch->GetWriteBatch(), /*ignore_max_db_size*/ true);
auto s = Write(write_opts_, batch->GetWriteBatch());
if (!s.ok()) {
return {Status::NotOK, s.ToString()};
}
Expand Down
10 changes: 5 additions & 5 deletions src/storage/storage.h
Original file line number Diff line number Diff line change
Expand Up @@ -147,8 +147,7 @@ class Storage {
rocksdb::Iterator *NewIterator(const rocksdb::ReadOptions &options, rocksdb::ColumnFamilyHandle *column_family);
rocksdb::Iterator *NewIterator(const rocksdb::ReadOptions &options);

[[nodiscard]] rocksdb::Status Write(const rocksdb::WriteOptions &options, rocksdb::WriteBatch *updates,
bool ignore_max_db_size = false);
[[nodiscard]] rocksdb::Status Write(const rocksdb::WriteOptions &options, rocksdb::WriteBatch *updates);
const rocksdb::WriteOptions &DefaultWriteOptions() { return write_opts_; }
rocksdb::ReadOptions DefaultScanOptions() const;
rocksdb::ReadOptions DefaultMultiGetOptions() const;
Expand All @@ -172,6 +171,8 @@ class Storage {
void PurgeOldBackups(uint32_t num_backups_to_keep, uint32_t backup_max_keep_hours);
uint64_t GetTotalSize(const std::string &ns = kDefaultNamespace);
void CheckDBSizeLimit();
bool ReachedDBSizeLimit() { return db_size_limit_reached_; }
void SetDBSizeLimit(bool limit) { db_size_limit_reached_ = limit; }
void SetIORateLimit(int64_t max_io_mb);

std::shared_lock<std::shared_mutex> ReadLockGuard();
Expand Down Expand Up @@ -244,7 +245,7 @@ class Storage {
Config *config_ = nullptr;
std::vector<rocksdb::ColumnFamilyHandle *> cf_handles_;
LockManager lock_mgr_;
bool db_size_limit_reached_ = false;
std::atomic<bool> db_size_limit_reached_{false};

DBStats db_stats_;

Expand All @@ -264,8 +265,7 @@ class Storage {

rocksdb::WriteOptions write_opts_ = rocksdb::WriteOptions();

rocksdb::Status writeToDB(const rocksdb::WriteOptions &options, rocksdb::WriteBatch *updates,
bool ignore_max_db_size = false);
rocksdb::Status writeToDB(const rocksdb::WriteOptions &options, rocksdb::WriteBatch *updates);
void recordKeyspaceStat(const rocksdb::ColumnFamilyHandle *column_family, const rocksdb::Status &s);
};

Expand Down
2 changes: 1 addition & 1 deletion src/types/redis_hash.cc
Original file line number Diff line number Diff line change
Expand Up @@ -227,7 +227,7 @@ rocksdb::Status Hash::Delete(const Slice &user_key, const std::vector<Slice> &fi
std::string bytes;
metadata.Encode(&bytes);
batch->Put(metadata_cf_handle_, ns_key, bytes);
return storage_->Write(storage_->DefaultWriteOptions(), batch->GetWriteBatch(), /*ignore_max_db_size*/ true);
return storage_->Write(storage_->DefaultWriteOptions(), batch->GetWriteBatch());
}

rocksdb::Status Hash::MSet(const Slice &user_key, const std::vector<FieldValue> &field_values, bool nx,
Expand Down
Loading

0 comments on commit 6417414

Please sign in to comment.