1
0
Fork 0
mirror of https://github.com/dragonflydb/dragonfly.git synced 2024-12-14 11:58:02 +00:00

Merge branch 'main' into chakaz/huge-values-migration-3

This commit is contained in:
shahar 2024-12-11 09:31:08 +02:00
commit be98628ecd
No known key found for this signature in database
GPG key ID: 2ED7F7EDC7FDCCA4
33 changed files with 524 additions and 478 deletions

2
helio

@ -1 +1 @@
Subproject commit ff9b6cd35bf082a9d48cf0904b0e8557cf31b6d2
Subproject commit 32e8c4ec830e8d4224d8a13a11c1607f357da80f

View file

@ -305,9 +305,9 @@ pair<void*, bool> DefragSet(unsigned encoding, void* ptr, float ratio) {
return DefragIntSet((intset*)ptr, ratio);
}
// StringMap supports re-allocation of it's internal nodes
case kEncodingStrMap2: {
return DefragStrMap2((StringMap*)ptr, ratio);
// Still not implemented
return {ptr, false};
}
default:

View file

@ -16,6 +16,7 @@
#include "core/detail/bitpacking.h"
#include "core/flat_set.h"
#include "core/mi_memory_resource.h"
#include "core/string_set.h"
extern "C" {
#include "redis/intset.h"
@ -575,6 +576,14 @@ TEST_F(CompactObjectTest, DefragHash) {
}
}
TEST_F(CompactObjectTest, DefragSet) {
// This is still not implemented
StringSet* s = new StringSet();
s->Add("str");
cobj_.InitRobj(OBJ_SET, kEncodingStrMap2, s);
ASSERT_FALSE(cobj_.DefragIfNeeded(0.8));
}
TEST_F(CompactObjectTest, RawInterface) {
string str(50, 'a'), tmp, owned;
cobj_.SetString(str);

View file

@ -91,6 +91,10 @@ inline bool operator==(const RespExpr& left, std::string_view s) {
return left.type == RespExpr::STRING && ToSV(left.GetBuf()) == s;
}
inline bool operator==(const RespExpr& left, int64_t val) {
return left.type == RespExpr::INT64 && left.GetInt() == val;
}
inline bool operator!=(const RespExpr& left, std::string_view s) {
return !(left == s);
}

View file

@ -11,6 +11,7 @@
namespace facade {
using namespace std;
constexpr static long kMaxBulkLen = 256 * (1ul << 20); // 256MB.
auto RedisParser::Parse(Buffer str, uint32_t* consumed, RespExpr::Vec* res) -> Result {
DCHECK(!str.empty());
@ -218,7 +219,11 @@ auto RedisParser::ParseLen(Buffer str, int64_t* res) -> ResultConsumed {
const char* s = reinterpret_cast<const char*>(str.data());
const char* pos = reinterpret_cast<const char*>(memchr(s, '\n', str.size()));
if (!pos) {
Result r = str.size() < 32 ? INPUT_PENDING : BAD_ARRAYLEN;
Result r = INPUT_PENDING;
if (str.size() >= 32) {
LOG(WARNING) << "Unexpected format " << string_view{s, str.size()};
r = BAD_ARRAYLEN;
}
return {r, 0};
}
@ -227,10 +232,16 @@ auto RedisParser::ParseLen(Buffer str, int64_t* res) -> ResultConsumed {
}
// Skip the first character and 2 last ones (\r\n).
bool success = absl::SimpleAtoi(std::string_view{s + 1, size_t(pos - 1 - s)}, res);
unsigned consumed = pos - s + 1;
string_view len_token{s + 1, size_t(pos - 1 - s)};
bool success = absl::SimpleAtoi(len_token, res);
return ResultConsumed{success ? OK : BAD_ARRAYLEN, consumed};
unsigned consumed = pos - s + 1;
if (success && *res >= -1) {
return ResultConsumed{OK, consumed};
}
LOG(WARNING) << "Failed to parse len " << len_token;
return ResultConsumed{BAD_ARRAYLEN, consumed};
}
auto RedisParser::ConsumeArrayLen(Buffer str) -> ResultConsumed {
@ -247,8 +258,8 @@ auto RedisParser::ConsumeArrayLen(Buffer str) -> ResultConsumed {
len *= 2;
}
if (len < -1 || len > max_arr_len_) {
LOG_IF(WARNING, len > max_arr_len_) << "Multibulk len is too large " << len;
if (len > max_arr_len_) {
LOG(WARNING) << "Multibulk len is too large " << len;
return {BAD_ARRAYLEN, res.second};
}
@ -310,15 +321,14 @@ auto RedisParser::ParseArg(Buffer str) -> ResultConsumed {
return res;
}
if (len < -1 || len > kMaxBulkLen)
return {BAD_ARRAYLEN, res.second};
if (len == -1) { // Resp2 NIL
cached_expr_->emplace_back(RespExpr::NIL);
cached_expr_->back().u = Buffer{};
HandleFinishArg();
} else {
DVLOG(1) << "String(" << len << ")";
LOG_IF(WARNING, len > kMaxBulkLen) << "Large bulk len: " << len;
cached_expr_->emplace_back(RespExpr::STRING);
cached_expr_->back().u = Buffer{};
bulk_len_ = len;

View file

@ -22,8 +22,6 @@ namespace facade {
*/
class RedisParser {
public:
constexpr static long kMaxBulkLen = 256 * (1ul << 20); // 256MB.
enum Result : uint8_t {
OK,
INPUT_PENDING,

View file

@ -170,7 +170,7 @@ TEST_F(RedisParserTest, Empty) {
}
TEST_F(RedisParserTest, LargeBulk) {
std::string_view prefix("*1\r\n$1024\r\n");
string_view prefix("*1\r\n$1024\r\n");
ASSERT_EQ(RedisParser::INPUT_PENDING, Parse(prefix));
ASSERT_EQ(prefix.size(), consumed_);
@ -191,6 +191,18 @@ TEST_F(RedisParserTest, LargeBulk) {
ASSERT_EQ(RedisParser::INPUT_PENDING, Parse(part1));
ASSERT_EQ(RedisParser::INPUT_PENDING, Parse(half));
ASSERT_EQ(RedisParser::OK, Parse("\r\n"));
prefix = "*1\r\n$270000000\r\n";
ASSERT_EQ(RedisParser::INPUT_PENDING, Parse(prefix));
ASSERT_EQ(prefix.size(), consumed_);
string chunk(1000000, 'a');
for (unsigned i = 0; i < 270; ++i) {
ASSERT_EQ(RedisParser::INPUT_PENDING, Parse(chunk));
ASSERT_EQ(chunk.size(), consumed_);
}
ASSERT_EQ(RedisParser::OK, Parse("\r\n"));
ASSERT_THAT(args_, ElementsAre(ArgType(RespExpr::STRING)));
EXPECT_EQ(270000000, args_[0].GetBuf().size());
}
TEST_F(RedisParserTest, NILs) {

View file

@ -79,7 +79,8 @@ typedef struct streamCG {
/* A specific consumer in a consumer group. */
typedef struct streamConsumer {
mstime_t seen_time; /* Last time this consumer was active. */
mstime_t seen_time; /* Last time this consumer tried to perform an action (attempted reading/claiming). */
mstime_t active_time; /* Last time this consumer was active (successful reading/claiming). */
sds name; /* Consumer name. This is how the consumer
will be identified in the consumer group
protocol. Case sensitive. */
@ -124,10 +125,6 @@ typedef struct {
/* Prototypes of exported APIs. */
// struct client;
/* Flags for streamLookupConsumer */
#define SLC_DEFAULT 0
#define SLC_NO_REFRESH (1<<0) /* Do not update consumer's seen-time */
/* Flags for streamCreateConsumer */
#define SCC_DEFAULT 0
#define SCC_NO_NOTIFY (1<<0) /* Do not notify key space if consumer created */
@ -149,7 +146,7 @@ void streamIteratorGetField(streamIterator *si, unsigned char **fieldptr, unsign
void streamIteratorRemoveEntry(streamIterator *si, streamID *current);
void streamIteratorStop(streamIterator *si);
streamCG *streamLookupCG(stream *s, sds groupname);
streamConsumer *streamLookupConsumer(streamCG *cg, sds name, int flags);
streamConsumer *streamLookupConsumer(streamCG *cg, sds name);
streamCG *streamCreateCG(stream *s, const char *name, size_t namelen, streamID *id, long long entries_read);
void streamEncodeID(void *buf, streamID *id);
void streamDecodeID(void *buf, streamID *id);

View file

@ -1068,9 +1068,8 @@ streamCG *streamLookupCG(stream *s, sds groupname) {
return (cg == raxNotFound) ? NULL : cg;
}
/* Lookup the consumer with the specified name in the group 'cg'. Its last
* seen time is updated unless the SLC_NO_REFRESH flag is specified. */
streamConsumer *streamLookupConsumer(streamCG *cg, sds name, int flags) {
/* Lookup the consumer with the specified name in the group 'cg' */
streamConsumer *streamLookupConsumer(streamCG *cg, sds name) {
if (cg == NULL) return NULL;
streamConsumer *consumer = raxFind(cg->consumers,(unsigned char*)name,
sdslen(name));

View file

@ -96,10 +96,9 @@ bool CommandId::IsMultiTransactional() const {
return CO::IsTransKind(name()) || CO::IsEvalKind(name());
}
uint64_t CommandId::Invoke(CmdArgList args, Transaction* tx, facade::SinkReplyBuilder* builder,
ConnectionContext* cntx) const {
uint64_t CommandId::Invoke(CmdArgList args, const CommandContext& cmd_cntx) const {
int64_t before = absl::GetCurrentTimeNanos();
handler_(args, tx, builder, cntx);
handler_(args, cmd_cntx);
int64_t after = absl::GetCurrentTimeNanos();
ServerState* ss = ServerState::tlocal(); // Might have migrated thread, read after invocation
@ -133,21 +132,6 @@ optional<facade::ErrorReply> CommandId::Validate(CmdArgList tail_args) const {
return nullopt;
}
CommandId&& CommandId::SetHandler(Handler2 f) && {
handler_ = [f = std::move(f)](CmdArgList args, Transaction* tx, facade::SinkReplyBuilder* builder,
ConnectionContext*) { f(args, tx, builder); };
return std::move(*this);
}
CommandId&& CommandId::SetHandler(Handler3 f) && {
handler_ = [f = std::move(f)](CmdArgList args, Transaction* tx, facade::SinkReplyBuilder* builder,
ConnectionContext* cntx) {
f(std::move(args), CommandContext{tx, builder, cntx});
};
return std::move(*this);
};
CommandRegistry::CommandRegistry() {
vector<string> rename_command = GetFlag(FLAGS_rename_command);

View file

@ -96,23 +96,13 @@ class CommandId : public facade::CommandId {
command_stats_ = std::make_unique<CmdCallStats[]>(thread_count);
}
using Handler =
fu2::function_base<true /*owns*/, true /*copyable*/, fu2::capacity_default,
false /* non-throwing*/, false /* strong exceptions guarantees*/,
void(CmdArgList, Transaction*, facade::SinkReplyBuilder*,
ConnectionContext*) const>;
using Handler2 =
fu2::function_base<true, true, fu2::capacity_default, false, false,
void(CmdArgList, Transaction*, facade::SinkReplyBuilder*) const>;
using Handler3 = fu2::function_base<true, true, fu2::capacity_default, false, false,
void(CmdArgList, const CommandContext&) const>;
using ArgValidator = fu2::function_base<true, true, fu2::capacity_default, false, false,
std::optional<facade::ErrorReply>(CmdArgList) const>;
// Returns the invoke time in usec.
uint64_t Invoke(CmdArgList args, Transaction*, facade::SinkReplyBuilder*,
ConnectionContext* cntx) const;
uint64_t Invoke(CmdArgList args, const CommandContext& cmd_cntx) const;
// Returns error if validation failed, otherwise nullopt
std::optional<facade::ErrorReply> Validate(CmdArgList tail_args) const;
@ -135,15 +125,11 @@ class CommandId : public facade::CommandId {
static const char* OptName(CO::CommandOpt fl);
CommandId&& SetHandler(Handler f) && {
CommandId&& SetHandler(Handler3 f) && {
handler_ = std::move(f);
return std::move(*this);
}
CommandId&& SetHandler(Handler2 f) &&;
CommandId&& SetHandler(Handler3 f) &&;
CommandId&& SetValidator(ArgValidator f) && {
validator_ = std::move(f);
return std::move(*this);
@ -169,7 +155,7 @@ class CommandId : public facade::CommandId {
private:
bool implicit_acl_;
std::unique_ptr<CmdCallStats[]> command_stats_;
Handler handler_;
Handler3 handler_;
ArgValidator validator_;
};

View file

@ -119,7 +119,6 @@ atomic_uint64_t rss_mem_peak(0);
unsigned kernel_version = 0;
size_t max_memory_limit = 0;
size_t serialization_max_chunk_size = 0;
Namespaces* namespaces = nullptr;
size_t FetchRssMemory(io::StatusData sdata) {
@ -438,7 +437,7 @@ ThreadLocalMutex::~ThreadLocalMutex() {
}
void ThreadLocalMutex::lock() {
if (serialization_max_chunk_size != 0) {
if (ServerState::tlocal()->serialization_max_chunk_size != 0) {
DCHECK_EQ(EngineShard::tlocal(), shard_);
util::fb2::NoOpLock noop_lk_;
if (locked_fiber_ != nullptr) {
@ -452,7 +451,7 @@ void ThreadLocalMutex::lock() {
}
void ThreadLocalMutex::unlock() {
if (serialization_max_chunk_size != 0) {
if (ServerState::tlocal()->serialization_max_chunk_size != 0) {
DCHECK_EQ(EngineShard::tlocal(), shard_);
flag_ = false;
cond_var_.notify_one();

View file

@ -388,8 +388,6 @@ struct BorrowedInterpreter {
bool owned_ = false;
};
extern size_t serialization_max_chunk_size;
class LocalBlockingCounter {
public:
void lock() {

View file

@ -229,14 +229,16 @@ size_t CalculateEvictionBytes() {
size_t goal_bytes = CalculateHowManyBytesToEvictOnShard(max_memory_limit, global_used_memory,
shard_memory_budget_threshold);
// TODO: Eviction due to rss usage is not working well as it causes eviction
// of to many keys untill we finally see decrease in rss. We need to improve
// this logic before we enable it.
/*
const double rss_oom_deny_ratio = ServerState::tlocal()->rss_oom_deny_ratio;
/* If rss_oom_deny_ratio is set, we should evict depending on rss memory too */
// If rss_oom_deny_ratio is set, we should evict depending on rss memory too
if (rss_oom_deny_ratio > 0.0) {
const size_t max_rss_memory = size_t(rss_oom_deny_ratio * max_memory_limit);
/* We start eviction when we have less than eviction_memory_budget_threshold * 100% of free rss
* memory */
const size_t shard_rss_memory_budget_threshold =
// We start eviction when we have less than eviction_memory_budget_threshold * 100% of free rss
memory const size_t shard_rss_memory_budget_threshold =
size_t(max_rss_memory * eviction_memory_budget_threshold) / shards_count;
// Calculate how much rss memory is used by all shards
@ -247,6 +249,8 @@ size_t CalculateEvictionBytes() {
goal_bytes, CalculateHowManyBytesToEvictOnShard(max_rss_memory, global_used_rss_memory,
shard_rss_memory_budget_threshold));
}
*/
return goal_bytes;
}

View file

@ -64,6 +64,7 @@ streamConsumer* StreamCreateConsumer(streamCG* cg, string_view name, uint64_t no
consumer->name = sdsnewlen(name.data(), name.size());
consumer->pel = raxNew();
consumer->seen_time = now_ms;
consumer->active_time = -1;
return consumer;
}

View file

@ -17,7 +17,8 @@ class CommandAggregator {
public:
using WriteCmdCallback = std::function<void(absl::Span<const string_view>)>;
CommandAggregator(string_view key, WriteCmdCallback cb) : key_(key), cb_(cb) {
CommandAggregator(string_view key, WriteCmdCallback cb, size_t max_agg_bytes)
: key_(key), cb_(cb), max_aggragation_bytes_(max_agg_bytes) {
}
~CommandAggregator() {
@ -29,7 +30,7 @@ class CommandAggregator {
agg_bytes_ += arg.size();
members_.push_back(std::move(arg));
if (commit_mode != CommitMode::kNoCommit && agg_bytes_ >= serialization_max_chunk_size) {
if (commit_mode != CommitMode::kNoCommit && agg_bytes_ >= max_aggragation_bytes_) {
CommitPending();
}
}
@ -55,18 +56,20 @@ class CommandAggregator {
vector<string> members_;
absl::InlinedVector<string_view, 5> args_;
size_t agg_bytes_ = 0;
size_t max_aggragation_bytes_;
};
} // namespace
CmdSerializer::CmdSerializer(FlushSerialized cb) : cb_(std::move(cb)) {
CmdSerializer::CmdSerializer(FlushSerialized cb, size_t max_serialization_buffer_size)
: cb_(std::move(cb)), max_serialization_buffer_size_(max_serialization_buffer_size) {
}
void CmdSerializer::SerializeEntry(string_view key, const PrimeValue& pk, const PrimeValue& pv,
uint64_t expire_ms) {
// We send RESTORE commands for small objects, or objects we don't support breaking.
bool use_restore_serialization = true;
if (serialization_max_chunk_size > 0 && pv.MallocUsed() > serialization_max_chunk_size) {
if (max_serialization_buffer_size_ > 0 && pv.MallocUsed() > max_serialization_buffer_size_) {
switch (pv.ObjType()) {
case OBJ_SET:
SerializeSet(key, pv);
@ -138,7 +141,8 @@ void CmdSerializer::SerializeExpireIfNeeded(string_view key, uint64_t expire_ms)
void CmdSerializer::SerializeSet(string_view key, const PrimeValue& pv) {
CommandAggregator aggregator(
key, [&](absl::Span<const string_view> args) { SerializeCommand("SADD", args); });
key, [&](absl::Span<const string_view> args) { SerializeCommand("SADD", args); },
max_serialization_buffer_size_);
container_utils::IterateSet(pv, [&](container_utils::ContainerEntry ce) {
aggregator.AddArg(ce.ToString());
@ -148,7 +152,8 @@ void CmdSerializer::SerializeSet(string_view key, const PrimeValue& pv) {
void CmdSerializer::SerializeZSet(string_view key, const PrimeValue& pv) {
CommandAggregator aggregator(
key, [&](absl::Span<const string_view> args) { SerializeCommand("ZADD", args); });
key, [&](absl::Span<const string_view> args) { SerializeCommand("ZADD", args); },
max_serialization_buffer_size_);
container_utils::IterateSortedSet(
pv.GetRobjWrapper(),
@ -162,7 +167,8 @@ void CmdSerializer::SerializeZSet(string_view key, const PrimeValue& pv) {
void CmdSerializer::SerializeHash(string_view key, const PrimeValue& pv) {
CommandAggregator aggregator(
key, [&](absl::Span<const string_view> args) { SerializeCommand("HSET", args); });
key, [&](absl::Span<const string_view> args) { SerializeCommand("HSET", args); },
max_serialization_buffer_size_);
container_utils::IterateMap(
pv, [&](container_utils::ContainerEntry k, container_utils::ContainerEntry v) {
@ -174,7 +180,8 @@ void CmdSerializer::SerializeHash(string_view key, const PrimeValue& pv) {
void CmdSerializer::SerializeList(string_view key, const PrimeValue& pv) {
CommandAggregator aggregator(
key, [&](absl::Span<const string_view> args) { SerializeCommand("RPUSH", args); });
key, [&](absl::Span<const string_view> args) { SerializeCommand("RPUSH", args); },
max_serialization_buffer_size_);
container_utils::IterateList(pv, [&](container_utils::ContainerEntry ce) {
aggregator.AddArg(ce.ToString());

View file

@ -21,7 +21,7 @@ class CmdSerializer {
public:
using FlushSerialized = std::function<void(std::string)>;
explicit CmdSerializer(FlushSerialized cb);
explicit CmdSerializer(FlushSerialized cb, size_t max_serialization_buffer_size);
void SerializeEntry(std::string_view key, const PrimeValue& pk, const PrimeValue& pv,
uint64_t expire_ms);
@ -39,6 +39,7 @@ class CmdSerializer {
uint64_t expire_ms);
FlushSerialized cb_;
size_t max_serialization_buffer_size_;
};
} // namespace dfly

View file

@ -10,6 +10,7 @@
#include "base/logging.h"
#include "server/cluster/cluster_defs.h"
#include "server/journal/cmd_serializer.h"
#include "server/server_state.h"
#include "util/fibers/synchronization.h"
using namespace facade;
@ -208,10 +209,10 @@ void RestoreStreamer::Run() {
do {
if (fiber_cancelled_)
return;
cursor = pt->Traverse(cursor, [&](PrimeTable::bucket_iterator it) {
cursor = pt->TraverseBuckets(cursor, [&](PrimeTable::bucket_iterator it) {
std::lock_guard guard(big_value_mu_);
if (fiber_cancelled_) // Traverse could have yieleded
if (fiber_cancelled_) // Could be cancelled any time as Traverse may preempt
return;
db_slice_->FlushChangeToEarlierCallbacks(0 /*db_id always 0 for cluster*/,
@ -321,10 +322,12 @@ void RestoreStreamer::OnDbChange(DbIndex db_index, const DbSlice::ChangeReq& req
void RestoreStreamer::WriteEntry(string_view key, const PrimeValue& pk, const PrimeValue& pv,
uint64_t expire_ms) {
CmdSerializer serializer([&](std::string s) {
Write(std::move(s));
ThrottleIfNeeded();
});
CmdSerializer serializer(
[&](std::string s) {
Write(std::move(s));
ThrottleIfNeeded();
},
ServerState::tlocal()->serialization_max_chunk_size);
serializer.SerializeEntry(key, pk, pv, expire_ms);
}

View file

@ -728,6 +728,11 @@ void SetRssOomDenyRatioOnAllThreads(double ratio) {
shard_set->pool()->AwaitBrief(cb);
}
void SetSerializationMaxChunkSize(size_t val) {
auto cb = [val](unsigned, auto*) { ServerState::tlocal()->serialization_max_chunk_size = val; };
shard_set->pool()->AwaitBrief(cb);
}
} // namespace
Service::Service(ProactorPool* pp)
@ -791,6 +796,8 @@ void Service::Init(util::AcceptServer* acceptor, std::vector<facade::Listener*>
config_registry.RegisterSetter<double>("rss_oom_deny_ratio",
[](double val) { SetRssOomDenyRatioOnAllThreads(val); });
config_registry.RegisterSetter<size_t>("serialization_max_chunk_size",
[](size_t val) { SetSerializationMaxChunkSize(val); });
config_registry.RegisterMutable("pipeline_squash");
@ -835,7 +842,6 @@ void Service::Init(util::AcceptServer* acceptor, std::vector<facade::Listener*>
[val](auto index, auto* context) { ServerState::tlocal()->acl_log.SetTotalEntries(val); });
});
serialization_max_chunk_size = GetFlag(FLAGS_serialization_max_chunk_size);
uint32_t shard_num = GetFlag(FLAGS_num_shards);
if (shard_num == 0 || shard_num > pp_.size()) {
LOG_IF(WARNING, shard_num > pp_.size())
@ -867,6 +873,7 @@ void Service::Init(util::AcceptServer* acceptor, std::vector<facade::Listener*>
Transaction::Init(shard_num);
SetRssOomDenyRatioOnAllThreads(absl::GetFlag(FLAGS_rss_oom_deny_ratio));
SetSerializationMaxChunkSize(absl::GetFlag(FLAGS_serialization_max_chunk_size));
// Requires that shard_set will be initialized before because server_family_.Init might
// load the snapshot.
@ -1049,7 +1056,11 @@ std::optional<ErrorReply> Service::VerifyCommandState(const CommandId* cid, CmdA
allowed_by_state = false;
break;
case GlobalState::TAKEN_OVER:
allowed_by_state = !cid->IsWriteOnly();
// Only PING, admin commands, and all commands via admin connections are allowed
// we prohibit even read commands, because read commands running in pipeline can take a while
// to send all data to a client which leads to fail in takeover
allowed_by_state = dfly_cntx.conn()->IsPrivileged() || (cid->opt_mask() & CO::ADMIN) ||
cid->name() == "PING";
break;
default:
break;
@ -1335,7 +1346,7 @@ bool Service::InvokeCmd(const CommandId* cid, CmdArgList tail_args, SinkReplyBui
auto last_error = builder->ConsumeLastError();
DCHECK(last_error.empty());
try {
invoke_time_usec = cid->Invoke(tail_args, tx, builder, cntx);
invoke_time_usec = cid->Invoke(tail_args, CommandContext{tx, builder, cntx});
} catch (std::exception& e) {
LOG(ERROR) << "Internal error, system probably unstable " << e.what();
return false;
@ -1638,34 +1649,31 @@ absl::flat_hash_map<std::string, unsigned> Service::UknownCmdMap() const {
return unknown_cmds_;
}
void Service::Quit(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder,
ConnectionContext* cntx) {
if (builder->GetProtocol() == Protocol::REDIS)
builder->SendOk();
void Service::Quit(CmdArgList args, const CommandContext& cmd_cntx) {
if (cmd_cntx.rb->GetProtocol() == Protocol::REDIS)
cmd_cntx.rb->SendOk();
builder->CloseConnection();
cmd_cntx.rb->CloseConnection();
DeactivateMonitoring(static_cast<ConnectionContext*>(cntx));
cntx->conn()->ShutdownSelf();
DeactivateMonitoring(cmd_cntx.conn_cntx);
cmd_cntx.conn_cntx->conn()->ShutdownSelf();
}
void Service::Multi(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder,
ConnectionContext* cntx) {
if (cntx->conn_state.exec_info.IsCollecting()) {
return builder->SendError("MULTI calls can not be nested");
void Service::Multi(CmdArgList args, const CommandContext& cmd_cntx) {
if (cmd_cntx.conn_cntx->conn_state.exec_info.IsCollecting()) {
return cmd_cntx.rb->SendError("MULTI calls can not be nested");
}
cntx->conn_state.exec_info.state = ConnectionState::ExecInfo::EXEC_COLLECT;
cmd_cntx.conn_cntx->conn_state.exec_info.state = ConnectionState::ExecInfo::EXEC_COLLECT;
// TODO: to protect against huge exec transactions.
return builder->SendOk();
return cmd_cntx.rb->SendOk();
}
void Service::Watch(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder,
ConnectionContext* cntx) {
auto& exec_info = cntx->conn_state.exec_info;
void Service::Watch(CmdArgList args, const CommandContext& cmd_cntx) {
auto& exec_info = cmd_cntx.conn_cntx->conn_state.exec_info;
// Skip if EXEC will already fail due previous WATCH.
if (exec_info.watched_dirty.load(memory_order_relaxed)) {
return builder->SendOk();
return cmd_cntx.rb->SendOk();
}
atomic_uint32_t keys_existed = 0;
@ -1673,28 +1681,27 @@ void Service::Watch(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder,
ShardId shard_id = shard->shard_id();
ShardArgs largs = t->GetShardArgs(shard_id);
for (auto k : largs) {
t->GetDbSlice(shard_id).RegisterWatchedKey(cntx->db_index(), k, &exec_info);
t->GetDbSlice(shard_id).RegisterWatchedKey(cmd_cntx.conn_cntx->db_index(), k, &exec_info);
}
auto res = GenericFamily::OpExists(t->GetOpArgs(shard), largs);
keys_existed.fetch_add(res.value_or(0), memory_order_relaxed);
return OpStatus::OK;
};
tx->ScheduleSingleHop(std::move(cb));
cmd_cntx.tx->ScheduleSingleHop(std::move(cb));
// Duplicate keys are stored to keep correct count.
exec_info.watched_existed += keys_existed.load(memory_order_relaxed);
for (string_view key : args) {
exec_info.watched_keys.emplace_back(cntx->db_index(), key);
exec_info.watched_keys.emplace_back(cmd_cntx.conn_cntx->db_index(), key);
}
return builder->SendOk();
return cmd_cntx.rb->SendOk();
}
void Service::Unwatch(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder,
ConnectionContext* cntx) {
UnwatchAllKeys(cntx->ns, &cntx->conn_state.exec_info);
return builder->SendOk();
void Service::Unwatch(CmdArgList args, const CommandContext& cmd_cntx) {
UnwatchAllKeys(cmd_cntx.conn_cntx->ns, &cmd_cntx.conn_cntx->conn_state.exec_info);
return cmd_cntx.rb->SendOk();
}
optional<CapturingReplyBuilder::Payload> Service::FlushEvalAsyncCmds(ConnectionContext* cntx,
@ -1760,41 +1767,37 @@ void Service::CallFromScript(ConnectionContext* cntx, Interpreter::CallArgs& ca)
DispatchCommand(ca.args, &replier, cntx);
}
void Service::Eval(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder,
ConnectionContext* cntx, bool read_only) {
void Service::Eval(CmdArgList args, const CommandContext& cmd_cntx, bool read_only) {
string_view body = ArgS(args, 0);
auto* rb = static_cast<RedisReplyBuilder*>(builder);
auto* rb = static_cast<RedisReplyBuilder*>(cmd_cntx.rb);
if (body.empty()) {
return rb->SendNull();
}
BorrowedInterpreter interpreter{tx, &cntx->conn_state};
BorrowedInterpreter interpreter{cmd_cntx.tx, &cmd_cntx.conn_cntx->conn_state};
auto res = server_family_.script_mgr()->Insert(body, interpreter);
if (!res)
return builder->SendError(res.error().Format(), facade::kScriptErrType);
return cmd_cntx.rb->SendError(res.error().Format(), facade::kScriptErrType);
string sha{std::move(res.value())};
CallSHA(args, sha, interpreter, builder, cntx, read_only);
CallSHA(args, sha, interpreter, cmd_cntx.rb, cmd_cntx.conn_cntx, read_only);
}
void Service::EvalRo(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder,
ConnectionContext* cntx) {
Eval(args, tx, builder, cntx, true);
void Service::EvalRo(CmdArgList args, const CommandContext& cmd_cntx) {
Eval(args, cmd_cntx, true);
}
void Service::EvalSha(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder,
ConnectionContext* cntx, bool read_only) {
void Service::EvalSha(CmdArgList args, const CommandContext& cmd_cntx, bool read_only) {
string sha = absl::AsciiStrToLower(ArgS(args, 0));
BorrowedInterpreter interpreter{cntx->transaction, &cntx->conn_state};
CallSHA(args, sha, interpreter, builder, cntx, read_only);
BorrowedInterpreter interpreter{cmd_cntx.tx, &cmd_cntx.conn_cntx->conn_state};
CallSHA(args, sha, interpreter, cmd_cntx.rb, cmd_cntx.conn_cntx, read_only);
}
void Service::EvalShaRo(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder,
ConnectionContext* cntx) {
EvalSha(args, tx, builder, cntx, true);
void Service::EvalShaRo(CmdArgList args, const CommandContext& cmd_cntx) {
EvalSha(args, cmd_cntx, true);
}
void Service::CallSHA(CmdArgList args, string_view sha, Interpreter* interpreter,
@ -2026,15 +2029,14 @@ void Service::EvalInternal(CmdArgList args, const EvalArgs& eval_args, Interpret
}
}
void Service::Discard(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder,
ConnectionContext* cntx) {
auto* rb = static_cast<RedisReplyBuilder*>(builder);
void Service::Discard(CmdArgList args, const CommandContext& cmd_cntx) {
auto* rb = static_cast<RedisReplyBuilder*>(cmd_cntx.rb);
if (!cntx->conn_state.exec_info.IsCollecting()) {
return builder->SendError("DISCARD without MULTI");
if (!cmd_cntx.conn_cntx->conn_state.exec_info.IsCollecting()) {
return cmd_cntx.rb->SendError("DISCARD without MULTI");
}
MultiCleanup(cntx);
MultiCleanup(cmd_cntx.conn_cntx);
rb->SendOk();
}
@ -2134,25 +2136,25 @@ void StartMultiExec(ConnectionContext* cntx, ConnectionState::ExecInfo* exec_inf
};
}
void Service::Exec(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder,
ConnectionContext* cntx) {
auto* rb = static_cast<RedisReplyBuilder*>(builder);
auto& exec_info = cntx->conn_state.exec_info;
void Service::Exec(CmdArgList args, const CommandContext& cmd_cntx) {
auto* rb = static_cast<RedisReplyBuilder*>(cmd_cntx.rb);
auto& exec_info = cmd_cntx.conn_cntx->conn_state.exec_info;
auto* cntx = cmd_cntx.conn_cntx;
// Clean the context no matter the outcome
absl::Cleanup exec_clear = [&cntx] { MultiCleanup(cntx); };
absl::Cleanup exec_clear = [cntx] { MultiCleanup(cntx); };
if (exec_info.state == ConnectionState::ExecInfo::EXEC_ERROR) {
return builder->SendError("-EXECABORT Transaction discarded because of previous errors");
return rb->SendError("-EXECABORT Transaction discarded because of previous errors");
}
// Check basic invariants
if (!exec_info.IsCollecting()) {
return builder->SendError("EXEC without MULTI");
return rb->SendError("EXEC without MULTI");
}
if (IsWatchingOtherDbs(cntx->db_index(), exec_info)) {
return builder->SendError("Dragonfly does not allow WATCH and EXEC on different databases");
return rb->SendError("Dragonfly does not allow WATCH and EXEC on different databases");
}
if (exec_info.watched_dirty.load(memory_order_relaxed)) {
@ -2167,7 +2169,8 @@ void Service::Exec(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder,
// We borrow a single interpreter for all the EVALs/Script load inside. Returned by MultiCleanup
if (state != ExecScriptUse::NONE) {
exec_info.preborrowed_interpreter = BorrowedInterpreter(tx, &cntx->conn_state).Release();
exec_info.preborrowed_interpreter =
BorrowedInterpreter(cmd_cntx.tx, &cntx->conn_state).Release();
}
// Determine according multi mode, not only only flag, but based on presence of global commands
@ -2183,7 +2186,7 @@ void Service::Exec(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder,
// EXEC should not run if any of the watched keys expired.
if (!exec_info.watched_keys.empty() &&
!CheckWatchedKeyExpiry(cntx, registry_.Find("EXISTS"), exec_cid_)) {
tx->UnlockMulti();
cmd_cntx.tx->UnlockMulti();
return rb->SendNull();
}
@ -2197,7 +2200,7 @@ void Service::Exec(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder,
if (!exec_info.body.empty()) {
if (GetFlag(FLAGS_track_exec_frequencies)) {
string descr = CreateExecDescriptor(exec_info.body, tx->GetUniqueShardCnt());
string descr = CreateExecDescriptor(exec_info.body, cmd_cntx.tx->GetUniqueShardCnt());
ServerState::tlocal()->exec_freq_count[descr]++;
}
@ -2209,7 +2212,7 @@ void Service::Exec(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder,
for (auto& scmd : exec_info.body) {
VLOG(2) << "TX CMD " << scmd.Cid()->name() << " " << scmd.NumArgs();
tx->MultiSwitchCmd(scmd.Cid());
cmd_cntx.tx->MultiSwitchCmd(scmd.Cid());
cntx->cid = scmd.Cid();
arg_vec.resize(scmd.NumArgs());
@ -2218,14 +2221,14 @@ void Service::Exec(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder,
CmdArgList args = absl::MakeSpan(arg_vec);
if (scmd.Cid()->IsTransactional()) {
OpStatus st = tx->InitByArgs(cntx->ns, cntx->conn_state.db_index, args);
OpStatus st = cmd_cntx.tx->InitByArgs(cntx->ns, cntx->conn_state.db_index, args);
if (st != OpStatus::OK) {
builder->SendError(st);
rb->SendError(st);
break;
}
}
bool ok = InvokeCmd(scmd.Cid(), args, builder, cntx);
bool ok = InvokeCmd(scmd.Cid(), args, rb, cmd_cntx.conn_cntx);
if (!ok || rb->GetError()) // checks for i/o error, not logical error.
break;
}
@ -2234,78 +2237,78 @@ void Service::Exec(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder,
if (scheduled) {
VLOG(2) << "Exec unlocking " << exec_info.body.size() << " commands";
tx->UnlockMulti();
cmd_cntx.tx->UnlockMulti();
}
cntx->cid = exec_cid_;
VLOG(2) << "Exec completed";
}
void Service::Publish(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder,
ConnectionContext* cntx) {
void Service::Publish(CmdArgList args, const CommandContext& cmd_cntx) {
if (cluster::IsClusterEnabled()) {
return builder->SendError("PUBLISH is not supported in cluster mode yet");
return cmd_cntx.rb->SendError("PUBLISH is not supported in cluster mode yet");
}
string_view channel = ArgS(args, 0);
string_view messages[] = {ArgS(args, 1)};
auto* cs = ServerState::tlocal()->channel_store();
builder->SendLong(cs->SendMessages(channel, messages));
cmd_cntx.rb->SendLong(cs->SendMessages(channel, messages));
}
void Service::Subscribe(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder,
ConnectionContext* cntx) {
void Service::Subscribe(CmdArgList args, const CommandContext& cmd_cntx) {
if (cluster::IsClusterEnabled()) {
return builder->SendError("SUBSCRIBE is not supported in cluster mode yet");
return cmd_cntx.rb->SendError("SUBSCRIBE is not supported in cluster mode yet");
}
cntx->ChangeSubscription(true /*add*/, true /* reply*/, std::move(args),
static_cast<RedisReplyBuilder*>(builder));
cmd_cntx.conn_cntx->ChangeSubscription(true /*add*/, true /* reply*/, std::move(args),
static_cast<RedisReplyBuilder*>(cmd_cntx.rb));
}
void Service::Unsubscribe(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder,
ConnectionContext* cntx) {
void Service::Unsubscribe(CmdArgList args, const CommandContext& cmd_cntx) {
auto* rb = static_cast<RedisReplyBuilder*>(cmd_cntx.rb);
if (cluster::IsClusterEnabled()) {
return builder->SendError("UNSUBSCRIBE is not supported in cluster mode yet");
return cmd_cntx.rb->SendError("UNSUBSCRIBE is not supported in cluster mode yet");
}
if (args.size() == 0) {
cmd_cntx.conn_cntx->UnsubscribeAll(true, rb);
} else {
cmd_cntx.conn_cntx->ChangeSubscription(false, true, args, rb);
}
}
void Service::PSubscribe(CmdArgList args, const CommandContext& cmd_cntx) {
auto* rb = static_cast<RedisReplyBuilder*>(cmd_cntx.rb);
if (cluster::IsClusterEnabled()) {
return rb->SendError("PSUBSCRIBE is not supported in cluster mode yet");
}
cmd_cntx.conn_cntx->ChangePSubscription(true, true, args, rb);
}
void Service::PUnsubscribe(CmdArgList args, const CommandContext& cmd_cntx) {
auto* rb = static_cast<RedisReplyBuilder*>(cmd_cntx.rb);
if (cluster::IsClusterEnabled()) {
return rb->SendError("PUNSUBSCRIBE is not supported in cluster mode yet");
}
if (args.size() == 0) {
cntx->UnsubscribeAll(true, static_cast<RedisReplyBuilder*>(builder));
cmd_cntx.conn_cntx->PUnsubscribeAll(true, rb);
} else {
cntx->ChangeSubscription(false, true, args, static_cast<RedisReplyBuilder*>(builder));
}
}
void Service::PSubscribe(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder,
ConnectionContext* cntx) {
if (cluster::IsClusterEnabled()) {
return builder->SendError("PSUBSCRIBE is not supported in cluster mode yet");
}
cntx->ChangePSubscription(true, true, args, static_cast<RedisReplyBuilder*>(builder));
}
void Service::PUnsubscribe(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder,
ConnectionContext* cntx) {
if (cluster::IsClusterEnabled()) {
return builder->SendError("PUNSUBSCRIBE is not supported in cluster mode yet");
}
if (args.size() == 0) {
cntx->PUnsubscribeAll(true, static_cast<RedisReplyBuilder*>(builder));
} else {
cntx->ChangePSubscription(false, true, args, static_cast<RedisReplyBuilder*>(builder));
cmd_cntx.conn_cntx->ChangePSubscription(false, true, args, rb);
}
}
// Not a real implementation. Serves as a decorator to accept some function commands
// for testing.
void Service::Function(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder,
ConnectionContext* cntx) {
void Service::Function(CmdArgList args, const CommandContext& cmd_cntx) {
string sub_cmd = absl::AsciiStrToUpper(ArgS(args, 0));
if (sub_cmd == "FLUSH") {
return builder->SendOk();
return cmd_cntx.rb->SendOk();
}
string err = UnknownSubCmd(sub_cmd, "FUNCTION");
return builder->SendError(err, kSyntaxErrType);
return cmd_cntx.rb->SendError(err, kSyntaxErrType);
}
void Service::PubsubChannels(string_view pattern, SinkReplyBuilder* builder) {
@ -2329,22 +2332,22 @@ void Service::PubsubNumSub(CmdArgList args, SinkReplyBuilder* builder) {
}
}
void Service::Monitor(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder,
ConnectionContext* cntx) {
VLOG(1) << "starting monitor on this connection: " << cntx->conn()->GetClientId();
void Service::Monitor(CmdArgList args, const CommandContext& cmd_cntx) {
VLOG(1) << "starting monitor on this connection: " << cmd_cntx.conn_cntx->conn()->GetClientId();
// we are registering the current connection for all threads so they will be aware of
// this connection, to send to it any command
builder->SendOk();
cntx->ChangeMonitor(true /* start */);
cmd_cntx.rb->SendOk();
cmd_cntx.conn_cntx->ChangeMonitor(true /* start */);
}
void Service::Pubsub(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder,
ConnectionContext* cntx) {
void Service::Pubsub(CmdArgList args, const CommandContext& cmd_cntx) {
auto* rb = static_cast<RedisReplyBuilder*>(cmd_cntx.rb);
if (cluster::IsClusterEnabled()) {
return builder->SendError("PUBSUB is not supported in cluster mode yet");
return rb->SendError("PUBSUB is not supported in cluster mode yet");
}
if (args.size() < 1) {
builder->SendError(WrongNumArgsError(cntx->cid->name()));
rb->SendError(WrongNumArgsError(cmd_cntx.conn_cntx->cid->name()));
return;
}
@ -2363,7 +2366,6 @@ void Service::Pubsub(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder
"HELP",
"\tPrints this help."};
auto* rb = static_cast<RedisReplyBuilder*>(builder);
rb->SendSimpleStrArr(help_arr);
return;
}
@ -2374,19 +2376,18 @@ void Service::Pubsub(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder
pattern = ArgS(args, 1);
}
PubsubChannels(pattern, builder);
PubsubChannels(pattern, rb);
} else if (subcmd == "NUMPAT") {
PubsubPatterns(builder);
PubsubPatterns(rb);
} else if (subcmd == "NUMSUB") {
args.remove_prefix(1);
PubsubNumSub(args, builder);
PubsubNumSub(args, rb);
} else {
builder->SendError(UnknownSubCmd(subcmd, "PUBSUB"));
rb->SendError(UnknownSubCmd(subcmd, "PUBSUB"));
}
}
void Service::Command(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder,
ConnectionContext* cntx) {
void Service::Command(CmdArgList args, const CommandContext& cmd_cntx) {
unsigned cmd_cnt = 0;
registry_.Traverse([&](string_view name, const CommandId& cd) {
if ((cd.opt_mask() & CO::HIDDEN) == 0) {
@ -2394,8 +2395,8 @@ void Service::Command(CmdArgList args, Transaction* tx, SinkReplyBuilder* builde
}
});
auto* rb = static_cast<RedisReplyBuilder*>(builder);
auto serialize_command = [&rb, this](string_view name, const CommandId& cid) {
auto* rb = static_cast<RedisReplyBuilder*>(cmd_cntx.rb);
auto serialize_command = [rb, this](string_view name, const CommandId& cid) {
rb->StartArray(7);
rb->SendSimpleString(cid.name());
rb->SendLong(cid.arity());
@ -2440,7 +2441,7 @@ void Service::Command(CmdArgList args, Transaction* tx, SinkReplyBuilder* builde
// COUNT
if (subcmd == "COUNT") {
return builder->SendLong(cmd_cnt);
return rb->SendLong(cmd_cnt);
}
bool sufficient_args = (args.size() == 2);
@ -2461,10 +2462,10 @@ void Service::Command(CmdArgList args, Transaction* tx, SinkReplyBuilder* builde
sufficient_args = (args.size() == 1);
if (subcmd == "DOCS" && sufficient_args) {
return builder->SendOk();
return rb->SendOk();
}
return builder->SendError(kSyntaxErr, kSyntaxErrType);
return rb->SendError(kSyntaxErr, kSyntaxErrType);
}
VarzValue::Map Service::GetVarzStats() {
@ -2595,12 +2596,9 @@ Service::ContextInfo Service::GetContextInfo(facade::ConnectionContext* cntx) co
.blocked = server_cntx->blocked};
}
using ServiceFunc = void (Service::*)(CmdArgList, ConnectionContext* cntx);
#define HFUNC(x) SetHandler(&Service::x)
#define MFUNC(x) \
SetHandler([this](CmdArgList sp, Transaction* tx, SinkReplyBuilder* builder, \
ConnectionContext* cntx) { this->x(std::move(sp), tx, builder, cntx); })
#define MFUNC(x) \
SetHandler([this](CmdArgList sp, const CommandContext& cntx) { this->x(std::move(sp), cntx); })
namespace acl {
constexpr uint32_t kQuit = FAST | CONNECTION;

View file

@ -125,43 +125,27 @@ class Service : public facade::ServiceInterface {
private:
using SinkReplyBuilder = facade::SinkReplyBuilder;
static void Quit(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder,
ConnectionContext* cntx);
static void Multi(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder,
ConnectionContext* cntx);
static void Quit(CmdArgList args, const CommandContext& cmd_cntx);
static void Multi(CmdArgList args, const CommandContext& cmd_cntx);
static void Watch(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder,
ConnectionContext* cntx);
static void Unwatch(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder,
ConnectionContext* cntx);
static void Watch(CmdArgList args, const CommandContext& cmd_cntx);
static void Unwatch(CmdArgList args, const CommandContext& cmd_cntx);
void Discard(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder,
ConnectionContext* cntx);
void Eval(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder, ConnectionContext* cntx,
bool read_only = false);
void EvalRo(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder, ConnectionContext* cntx);
void EvalSha(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder, ConnectionContext* cntx,
bool read_only = false);
void EvalShaRo(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder,
ConnectionContext* cntx);
void Exec(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder, ConnectionContext* cntx);
void Publish(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder,
ConnectionContext* cntx);
void Subscribe(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder,
ConnectionContext* cntx);
void Unsubscribe(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder,
ConnectionContext* cntx);
void PSubscribe(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder,
ConnectionContext* cntx);
void PUnsubscribe(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder,
ConnectionContext* cntx);
void Function(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder,
ConnectionContext* cntx);
void Monitor(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder,
ConnectionContext* cntx);
void Pubsub(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder, ConnectionContext* cntx);
void Command(CmdArgList args, Transaction* tx, SinkReplyBuilder* builder,
ConnectionContext* cntx);
void Discard(CmdArgList args, const CommandContext& cmd_cntx);
void Eval(CmdArgList args, const CommandContext& cmd_cntx, bool read_only = false);
void EvalRo(CmdArgList args, const CommandContext& cmd_cntx);
void EvalSha(CmdArgList args, const CommandContext& cmd_cntx, bool read_only = false);
void EvalShaRo(CmdArgList args, const CommandContext& cmd_cntx);
void Exec(CmdArgList args, const CommandContext& cmd_cntx);
void Publish(CmdArgList args, const CommandContext& cmd_cntx);
void Subscribe(CmdArgList args, const CommandContext& cmd_cntx);
void Unsubscribe(CmdArgList args, const CommandContext& cmd_cntx);
void PSubscribe(CmdArgList args, const CommandContext& cmd_cntx);
void PUnsubscribe(CmdArgList args, const CommandContext& cmd_cntx);
void Function(CmdArgList args, const CommandContext& cmd_cntx);
void Monitor(CmdArgList args, const CommandContext& cmd_cntx);
void Pubsub(CmdArgList args, const CommandContext& cmd_cntx);
void Command(CmdArgList args, const CommandContext& cmd_cntx);
void PubsubChannels(std::string_view pattern, SinkReplyBuilder* builder);
void PubsubPatterns(SinkReplyBuilder* builder);

View file

@ -9,7 +9,6 @@ extern "C" {
}
// Custom types: Range 30-35 is used by DF RDB types.
constexpr uint8_t RDB_TYPE_JSON_OLD = 20;
constexpr uint8_t RDB_TYPE_JSON = 30;
constexpr uint8_t RDB_TYPE_HASH_WITH_EXPIRY = 31;
constexpr uint8_t RDB_TYPE_SET_WITH_EXPIRY = 32;

View file

@ -477,6 +477,8 @@ void RdbLoaderBase::OpaqueObjLoader::operator()(const unique_ptr<LoadTrace>& ptr
CreateZSet(ptr.get());
break;
case RDB_TYPE_STREAM_LISTPACKS:
case RDB_TYPE_STREAM_LISTPACKS_2:
case RDB_TYPE_STREAM_LISTPACKS_3:
CreateStream(ptr.get());
break;
default:
@ -955,8 +957,16 @@ void RdbLoaderBase::OpaqueObjLoader::CreateStream(const LoadTrace* ltrace) {
}
s->length = ltrace->stream_trace->stream_len;
s->last_id.ms = ltrace->stream_trace->ms;
s->last_id.seq = ltrace->stream_trace->seq;
CopyStreamId(ltrace->stream_trace->last_id, &s->last_id);
CopyStreamId(ltrace->stream_trace->first_id, &s->first_id);
CopyStreamId(ltrace->stream_trace->max_deleted_entry_id, &s->max_deleted_entry_id);
s->entries_added = ltrace->stream_trace->entries_added;
if (rdb_type_ == RDB_TYPE_STREAM_LISTPACKS) {
/* Since the rax is already loaded, we can find the first entry's
* ID. */
streamGetEdgeID(s, 1, 1, &s->first_id);
}
for (const auto& cg : ltrace->stream_trace->cgroup) {
string_view cgname = ToSV(cg.name);
@ -964,7 +974,12 @@ void RdbLoaderBase::OpaqueObjLoader::CreateStream(const LoadTrace* ltrace) {
cg_id.ms = cg.ms;
cg_id.seq = cg.seq;
streamCG* cgroup = streamCreateCG(s, cgname.data(), cgname.size(), &cg_id, 0);
uint64_t entries_read = cg.entries_read;
if (rdb_type_ == RDB_TYPE_STREAM_LISTPACKS) {
entries_read = streamEstimateDistanceFromFirstEverEntry(s, &cg_id);
}
streamCG* cgroup = streamCreateCG(s, cgname.data(), cgname.size(), &cg_id, entries_read);
if (cgroup == NULL) {
LOG(ERROR) << "Duplicated consumer group name " << cgname;
ec_ = RdbError(errc::duplicate_key);
@ -995,6 +1010,7 @@ void RdbLoaderBase::OpaqueObjLoader::CreateStream(const LoadTrace* ltrace) {
return;
}
consumer->active_time = cons.active_time;
/* Create the PEL (pending entries list) about entries owned by this specific
* consumer. */
for (const auto& rawid : cons.nack_arr) {
@ -1512,7 +1528,7 @@ error_code RdbLoaderBase::ReadObj(int rdbtype, OpaqueObj* dest) {
// RDB_TYPE_JSON == 20. On newer versions > 9 we bumped up RDB_TYPE_JSON to 30
// because it overlapped with the new type RDB_TYPE_SET_LISTPACK
if (rdb_version_ < 10) {
// consider it RDB_TYPE_JSON_OLD
// consider it RDB_TYPE_JSON_OLD (20)
iores = ReadJson();
} else {
iores = ReadGeneric(rdbtype);
@ -1876,21 +1892,20 @@ auto RdbLoaderBase::ReadStreams(int rdbtype) -> io::Result<OpaqueObj> {
// so if there are still unread elements return the partial stream.
if (listpacks > n) {
pending_read_.remaining = listpacks - n;
return OpaqueObj{std::move(load_trace), RDB_TYPE_STREAM_LISTPACKS};
} else if (pending_read_.remaining > 0) {
pending_read_.remaining = 0;
return OpaqueObj{std::move(load_trace), rdbtype};
}
// Load stream metadata.
pending_read_.remaining = 0;
// Load stream metadata.
load_trace->stream_trace.reset(new StreamTrace);
/* Load total number of items inside the stream. */
SET_OR_UNEXPECT(LoadLen(nullptr), load_trace->stream_trace->stream_len);
/* Load the last entry ID. */
SET_OR_UNEXPECT(LoadLen(nullptr), load_trace->stream_trace->ms);
SET_OR_UNEXPECT(LoadLen(nullptr), load_trace->stream_trace->seq);
SET_OR_UNEXPECT(LoadLen(nullptr), load_trace->stream_trace->last_id.ms);
SET_OR_UNEXPECT(LoadLen(nullptr), load_trace->stream_trace->last_id.seq);
if (rdbtype >= RDB_TYPE_STREAM_LISTPACKS_2) {
/* Load the first entry ID. */
@ -1907,13 +1922,7 @@ auto RdbLoaderBase::ReadStreams(int rdbtype) -> io::Result<OpaqueObj> {
/* During migration the offset can be initialized to the stream's
* length. At this point, we also don't care about tombstones
* because CG offsets will be later initialized as well. */
load_trace->stream_trace->max_deleted_entry_id.ms = 0;
load_trace->stream_trace->max_deleted_entry_id.seq = 0;
load_trace->stream_trace->entries_added = load_trace->stream_trace->stream_len;
// TODO add implementation, we need to find the first entry's ID.
// The redis code is next
// streamGetEdgeID(s,1,1,&s->first_id);
}
/* Consumer groups loading */
@ -1937,24 +1946,11 @@ auto RdbLoaderBase::ReadStreams(int rdbtype) -> io::Result<OpaqueObj> {
SET_OR_UNEXPECT(LoadLen(nullptr), cgroup.ms);
SET_OR_UNEXPECT(LoadLen(nullptr), cgroup.seq);
uint64_t cg_offset;
cgroup.entries_read = 0;
if (rdbtype >= RDB_TYPE_STREAM_LISTPACKS_2) {
SET_OR_UNEXPECT(LoadLen(nullptr), cg_offset);
(void)cg_offset;
} else {
// TODO implement
// cg_offset = should be calculated like streamEstimateDistanceFromFirstEverEntry();
SET_OR_UNEXPECT(LoadLen(nullptr), cgroup.entries_read);
}
// TODO add our implementation for the next Redis logic
// streamCG* cgroup = streamCreateCG(s, cgname, sdslen(cgname), &cg_id, cg_offset);
// if (cgroup == NULL) {
// rdbReportCorruptRDB("Duplicated consumer group name %s", cgname);
// decrRefCount(o);
// sdsfree(cgname);
// return NULL;
// }
/* Load the global PEL for this consumer group, however we'll
* not yet populate the NACK structures with the message
* owner, since consumers for this group and their messages will
@ -2724,6 +2720,11 @@ std::error_code RdbLoaderBase::FromOpaque(const OpaqueObj& opaque, LoadConfig co
return visitor.ec();
}
void RdbLoaderBase::CopyStreamId(const StreamID& src, struct streamID* dest) {
dest->ms = src.ms;
dest->seq = src.seq;
}
void RdbLoader::LoadItemsBuffer(DbIndex db_ind, const ItemsBuf& ib) {
EngineShard* es = EngineShard::tlocal();
DbContext db_cntx{&namespaces->GetDefaultNamespace(), db_ind, GetCurrentTimeMs()};

View file

@ -16,6 +16,8 @@ extern "C" {
#include "server/common.h"
#include "server/journal/serializer.h"
struct streamID;
namespace dfly {
class EngineShardSet;
@ -84,15 +86,15 @@ class RdbLoaderBase {
};
struct StreamID {
uint64_t ms;
uint64_t seq;
uint64_t ms = 0;
uint64_t seq = 0;
};
struct StreamCGTrace {
RdbVariant name;
uint64_t ms;
uint64_t seq;
uint64_t entries_read;
std::vector<StreamPelTrace> pel_arr;
std::vector<StreamConsumerTrace> cons_arr;
};
@ -100,10 +102,10 @@ class RdbLoaderBase {
struct StreamTrace {
size_t lp_len;
size_t stream_len;
uint64_t ms, seq;
StreamID last_id;
StreamID first_id; /* The first non-tombstone entry, zero if empty. */
StreamID max_deleted_entry_id; /* The maximal ID that was deleted. */
uint64_t entries_added; /* All time count of elements added. */
uint64_t entries_added = 0; /* All time count of elements added. */
std::vector<StreamCGTrace> cgroup;
};
@ -192,6 +194,8 @@ class RdbLoaderBase {
std::error_code EnsureReadInternal(size_t min_to_read);
static void CopyStreamId(const StreamID& src, struct streamID* dest);
base::IoBuf* mem_buf_ = nullptr;
base::IoBuf origin_mem_buf_;
::io::Source* src_ = nullptr;

View file

@ -51,10 +51,16 @@ ABSL_FLAG(dfly::CompressionMode, compression_mode, dfly::CompressionMode::MULTI_
"set 2 for multi entry zstd compression on df snapshot and single entry on rdb snapshot,"
"set 3 for multi entry lz4 compression on df snapshot and single entry on rdb snapshot");
ABSL_FLAG(int, compression_level, 2, "The compression level to use on zstd/lz4 compression");
// TODO: to retire both flags in v1.27 (Jan 2025)
ABSL_FLAG(bool, list_rdb_encode_v2, true,
"V2 rdb encoding of list uses listpack encoding format, compatible with redis 7. V1 rdb "
"enconding of list uses ziplist encoding compatible with redis 6");
ABSL_FLAG(bool, stream_rdb_encode_v2, false,
"V2 uses format, compatible with redis 7.2 and Dragonfly v1.26+, while v1 format "
"is compatible with redis 6");
namespace dfly {
using namespace std;
@ -160,7 +166,7 @@ std::string AbslUnparseFlag(dfly::CompressionMode flag) {
dfly::CompressionMode GetDefaultCompressionMode() {
const auto flag = absl::GetFlag(FLAGS_compression_mode);
if (serialization_max_chunk_size == 0) {
if (ServerState::tlocal()->serialization_max_chunk_size == 0) {
return flag;
}
@ -209,12 +215,12 @@ uint8_t RdbObjectType(const PrimeValue& pv) {
}
break;
case OBJ_STREAM:
return RDB_TYPE_STREAM_LISTPACKS;
return absl::GetFlag(FLAGS_stream_rdb_encode_v2) ? RDB_TYPE_STREAM_LISTPACKS_3
: RDB_TYPE_STREAM_LISTPACKS;
case OBJ_MODULE:
return RDB_TYPE_MODULE_2;
case OBJ_JSON:
return RDB_TYPE_JSON; // save with RDB_TYPE_JSON, deprecate RDB_TYPE_JSON_OLD after July
// 2024.
return RDB_TYPE_JSON;
case OBJ_SBF:
return RDB_TYPE_SBF;
}
@ -657,6 +663,22 @@ error_code RdbSerializer::SaveStreamObject(const PrimeValue& pv) {
RETURN_ON_ERR(SaveLen(s->last_id.ms));
RETURN_ON_ERR(SaveLen(s->last_id.seq));
uint8_t rdb_type = RdbObjectType(pv);
// 'first_id', 'max_deleted_entry_id' and 'entries_added' are added
// in RDB_TYPE_STREAM_LISTPACKS_2
if (rdb_type >= RDB_TYPE_STREAM_LISTPACKS_2) {
/* Save the first entry ID. */
RETURN_ON_ERR(SaveLen(s->first_id.ms));
RETURN_ON_ERR(SaveLen(s->first_id.seq));
/* Save the maximal tombstone ID. */
RETURN_ON_ERR(SaveLen(s->max_deleted_entry_id.ms));
RETURN_ON_ERR(SaveLen(s->max_deleted_entry_id.seq));
/* Save the offset. */
RETURN_ON_ERR(SaveLen(s->entries_added));
}
/* The consumer groups and their clients are part of the stream
* type, so serialize every consumer group. */
@ -678,16 +700,20 @@ error_code RdbSerializer::SaveStreamObject(const PrimeValue& pv) {
RETURN_ON_ERR(SaveString((uint8_t*)ri.key, ri.key_len));
/* Last ID. */
RETURN_ON_ERR(SaveLen(s->last_id.ms));
RETURN_ON_ERR(SaveLen(cg->last_id.ms));
RETURN_ON_ERR(SaveLen(s->last_id.seq));
RETURN_ON_ERR(SaveLen(cg->last_id.seq));
if (rdb_type >= RDB_TYPE_STREAM_LISTPACKS_2) {
/* Save the group's logical reads counter. */
RETURN_ON_ERR(SaveLen(cg->entries_read));
}
/* Save the global PEL. */
RETURN_ON_ERR(SaveStreamPEL(cg->pel, true));
/* Save the consumers of this group. */
RETURN_ON_ERR(SaveStreamConsumers(cg));
RETURN_ON_ERR(SaveStreamConsumers(rdb_type >= RDB_TYPE_STREAM_LISTPACKS_3, cg));
}
}
@ -818,7 +844,7 @@ error_code RdbSerializer::SaveStreamPEL(rax* pel, bool nacks) {
return error_code{};
}
error_code RdbSerializer::SaveStreamConsumers(streamCG* cg) {
error_code RdbSerializer::SaveStreamConsumers(bool save_active, streamCG* cg) {
/* Number of consumers in this consumer group. */
RETURN_ON_ERR(SaveLen(raxSize(cg->consumers)));
@ -836,10 +862,15 @@ error_code RdbSerializer::SaveStreamConsumers(streamCG* cg) {
/* Consumer name. */
RETURN_ON_ERR(SaveString(ri.key, ri.key_len));
/* Last seen time. */
/* seen time. */
absl::little_endian::Store64(buf, consumer->seen_time);
RETURN_ON_ERR(WriteRaw(buf));
if (save_active) {
/* Active time. */
absl::little_endian::Store64(buf, consumer->active_time);
RETURN_ON_ERR(WriteRaw(buf));
}
/* Consumer PEL, without the ACKs (see last parameter of the function
* passed with value of 0), at loading time we'll lookup the ID
* in the consumer group global PEL and will put a reference in the

View file

@ -255,7 +255,7 @@ class RdbSerializer : public SerializerBase {
std::error_code SaveBinaryDouble(double val);
std::error_code SaveListPackAsZiplist(uint8_t* lp);
std::error_code SaveStreamPEL(rax* pel, bool nacks);
std::error_code SaveStreamConsumers(streamCG* cg);
std::error_code SaveStreamConsumers(bool save_active, streamCG* cg);
std::error_code SavePlainNodeAsZiplist(const quicklistNode* node);
// Might preempt

View file

@ -36,6 +36,8 @@ ABSL_DECLARE_FLAG(dfly::CompressionMode, compression_mode);
namespace dfly {
static const auto kMatchNil = ArgType(RespExpr::NIL);
class RdbTest : public BaseFamilyTest {
protected:
void SetUp();
@ -136,13 +138,20 @@ TEST_F(RdbTest, Stream) {
auto resp = Run({"type", "key:10"});
EXPECT_EQ(resp, "stream");
resp = Run({"xinfo", "groups", "key:0"});
EXPECT_THAT(resp, ArrLen(2));
EXPECT_THAT(resp.GetVec()[0],
RespElementsAre("name", "g1", "consumers", 0, "pending", 0, "last-delivered-id",
"1655444851524-3", "entries-read", 128, "lag", 0));
EXPECT_THAT(resp.GetVec()[1],
RespElementsAre("name", "g2", "consumers", 1, "pending", 0, "last-delivered-id",
"1655444851523-1", "entries-read", kMatchNil, "lag", kMatchNil));
resp = Run({"xinfo", "groups", "key:1"}); // test dereferences array of size 1
EXPECT_THAT(resp, RespArray(ElementsAre("name", "g2", "consumers", IntArg(0), "pending",
IntArg(0), "last-delivered-id", "1655444851523-1",
"entries-read", IntArg(0), "lag", IntArg(0))));
EXPECT_THAT(resp, RespElementsAre("name", "g2", "consumers", IntArg(0), "pending", IntArg(0),
"last-delivered-id", "1655444851523-1", "entries-read",
kMatchNil, "lag", kMatchNil));
resp = Run({"xinfo", "groups", "key:2"});
EXPECT_THAT(resp, ArrLen(0));
@ -629,14 +638,30 @@ TEST_F(RdbTest, LoadHugeList) {
// Tests loading a huge stream, where the stream is loaded in multiple partial
// reads.
TEST_F(RdbTest, LoadHugeStream) {
TEST_current_time_ms = 1000;
// Add a huge stream (test:0) with 2000 entries, and 4 1k elements per entry
// (note must be more than 512*4kb elements to test partial reads).
for (int i = 0; i != 2000; i++) {
// We add 2000 entries to the stream to ensure that the stream, because populate strream
// adds only a single entry at a time, with multiple elements in it.
for (unsigned i = 0; i < 2000; i++) {
Run({"debug", "populate", "1", "test", "2000", "rand", "type", "stream", "elements", "4"});
}
ASSERT_EQ(2000, CheckedInt({"xlen", "test:0"}));
Run({"XGROUP", "CREATE", "test:0", "grp1", "0"});
Run({"XGROUP", "CREATE", "test:0", "grp2", "0"});
Run({"XREADGROUP", "GROUP", "grp1", "Alice", "COUNT", "1", "STREAMS", "test:0", ">"});
Run({"XREADGROUP", "GROUP", "grp2", "Alice", "COUNT", "1", "STREAMS", "test:0", ">"});
RespExpr resp = Run({"save", "df"});
auto resp = Run({"xinfo", "stream", "test:0"});
EXPECT_THAT(
resp, RespElementsAre("length", 2000, "radix-tree-keys", 2000, "radix-tree-nodes", 2010,
"last-generated-id", "1000-1999", "max-deleted-entry-id", "0-0",
"entries-added", 2000, "recorded-first-entry-id", "1000-0", "groups", 2,
"first-entry", ArrLen(2), "last-entry", ArrLen(2)));
resp = Run({"save", "df"});
ASSERT_EQ(resp, "OK");
auto save_info = service_->server_family().GetLastSaveInfo();
@ -644,18 +669,29 @@ TEST_F(RdbTest, LoadHugeStream) {
ASSERT_EQ(resp, "OK");
ASSERT_EQ(2000, CheckedInt({"xlen", "test:0"}));
resp = Run({"xinfo", "stream", "test:0"});
EXPECT_THAT(
resp, RespElementsAre("length", 2000, "radix-tree-keys", 2000, "radix-tree-nodes", 2010,
"last-generated-id", "1000-1999", "max-deleted-entry-id", "0-0",
"entries-added", 2000, "recorded-first-entry-id", "1000-0", "groups", 2,
"first-entry", ArrLen(2), "last-entry", ArrLen(2)));
resp = Run({"xinfo", "groups", "test:0"});
EXPECT_THAT(resp, RespElementsAre(RespElementsAre("name", "grp1", "consumers", 1, "pending", 1,
"last-delivered-id", "1000-0", "entries-read",
1, "lag", 1999),
_));
}
TEST_F(RdbTest, LoadStream2) {
auto ec = LoadRdb("RDB_TYPE_STREAM_LISTPACKS_2.rdb");
ASSERT_FALSE(ec) << ec.message();
auto res = Run({"XINFO", "STREAM", "mystream"});
ASSERT_THAT(
res.GetVec(),
ElementsAre("length", IntArg(2), "radix-tree-keys", IntArg(1), "radix-tree-nodes", IntArg(2),
"last-generated-id", "1732613360686-0", "max-deleted-entry-id", "0-0",
"entries-added", IntArg(0), "recorded-first-entry-id", "0-0", "groups", IntArg(1),
"first-entry", ArgType(RespExpr::ARRAY), "last-entry", ArgType(RespExpr::ARRAY)));
ASSERT_THAT(res.GetVec(),
ElementsAre("length", 2, "radix-tree-keys", 1, "radix-tree-nodes", 2,
"last-generated-id", "1732613360686-0", "max-deleted-entry-id", "0-0",
"entries-added", 2, "recorded-first-entry-id", "1732613352350-0",
"groups", 1, "first-entry", RespElementsAre("1732613352350-0", _),
"last-entry", RespElementsAre("1732613360686-0", _)));
}
TEST_F(RdbTest, LoadStream3) {
@ -664,10 +700,10 @@ TEST_F(RdbTest, LoadStream3) {
auto res = Run({"XINFO", "STREAM", "mystream"});
ASSERT_THAT(
res.GetVec(),
ElementsAre("length", IntArg(2), "radix-tree-keys", IntArg(1), "radix-tree-nodes", IntArg(2),
"last-generated-id", "1732614679549-0", "max-deleted-entry-id", "0-0",
"entries-added", IntArg(0), "recorded-first-entry-id", "0-0", "groups", IntArg(1),
"first-entry", ArgType(RespExpr::ARRAY), "last-entry", ArgType(RespExpr::ARRAY)));
ElementsAre("length", 2, "radix-tree-keys", 1, "radix-tree-nodes", 2, "last-generated-id",
"1732614679549-0", "max-deleted-entry-id", "0-0", "entries-added", 2,
"recorded-first-entry-id", "1732614676541-0", "groups", 1, "first-entry",
ArgType(RespExpr::ARRAY), "last-entry", ArgType(RespExpr::ARRAY)));
}
TEST_F(RdbTest, SnapshotTooBig) {

View file

@ -877,6 +877,7 @@ void ServerFamily::Init(util::AcceptServer* acceptor, std::vector<facade::Listen
absl::GetFlag(FLAGS_s3_ec2_metadata), absl::GetFlag(FLAGS_s3_sign_payload));
#else
LOG(ERROR) << "Compiled without AWS support";
exit(1);
#endif
} else if (IsGCSPath(flag_dir)) {
auto gcs = std::make_shared<detail::GcsSnapshotStorage>();

View file

@ -298,6 +298,7 @@ class ServerState { // public struct - to allow initialization.
// Exec descriptor frequency count for this thread.
absl::flat_hash_map<std::string, unsigned> exec_freq_count;
double rss_oom_deny_ratio;
size_t serialization_max_chunk_size;
private:
int64_t live_transactions_ = 0;

View file

@ -78,7 +78,7 @@ void SliceSnapshot::Start(bool stream_journal, const Cancellation* cll, Snapshot
journal_cb_id_ = journal->RegisterOnChange(std::move(journal_cb));
}
const auto flush_threshold = serialization_max_chunk_size;
const auto flush_threshold = ServerState::tlocal()->serialization_max_chunk_size;
std::function<void(size_t, RdbSerializer::FlushState)> flush_fun;
if (flush_threshold != 0 && allow_flush == SnapshotFlush::kAllow) {
flush_fun = [this, flush_threshold](size_t bytes_serialized,

View file

@ -46,7 +46,7 @@ struct ParsedStreamId {
// Whether to lookup messages after the last ID in the stream. Used for XREAD
// when using ID '$'.
bool last_id = false;
bool resolve_last_id = false;
};
struct RangeId {
@ -82,7 +82,8 @@ struct NACKInfo {
struct ConsumerInfo {
string name;
size_t seen_time;
mstime_t seen_time;
mstime_t active_time;
size_t pel_count;
vector<NACKInfo> pending;
size_t idle;
@ -361,7 +362,7 @@ int StreamAppendItem(stream* s, CmdArgList fields, uint64_t now_ms, streamID* ad
}
/* Avoid overflow when trying to add an element to the stream (listpack
* can only host up to 32bit length sttrings, and also a total listpack size
* can only host up to 32bit length strings, and also a total listpack size
* can't be bigger than 32bit length. */
size_t totelelen = 0;
for (size_t i = 0; i < fields.size(); i++) {
@ -769,6 +770,7 @@ OpResult<RecordVec> OpRange(const OpArgs& op_args, string_view key, const RangeO
LOG(DFATAL) << "Internal error";
return OpStatus::SKIPPED; // ("NACK half-created. Should not be possible.");
}
opts.consumer->active_time = now_ms;
}
if (opts.count == result.size())
break;
@ -985,6 +987,7 @@ void GetConsumers(stream* s, streamCG* cg, long long count, GroupInfo* ginfo) {
consumer_info.name = consumer->name;
consumer_info.seen_time = consumer->seen_time;
consumer_info.active_time = consumer->active_time;
consumer_info.pel_count = raxSize(consumer->pel);
/* Consumer PEL */
@ -1106,6 +1109,7 @@ OpResult<vector<ConsumerInfo>> OpConsumers(const DbContext& db_cntx, EngineShard
consumer_info.name = consumer->name;
consumer_info.pel_count = raxSize(consumer->pel);
consumer_info.idle = idle;
consumer_info.active_time = consumer->active_time;
result.push_back(std::move(consumer_info));
}
raxStop(&ri);
@ -1181,6 +1185,18 @@ OpResult<FindGroupResult> FindGroup(const OpArgs& op_args, string_view key, stri
return FindGroupResult{s, cg, std::move(res_it->post_updater), res_it->it};
}
// Try to get the consumer. If not found, create a new one.
streamConsumer* FindOrAddConsumer(string_view name, streamCG* cg, uint64_t now_ms) {
// Try to get the consumer. If not found, create a new one.
auto cname = WrapSds(name);
streamConsumer* consumer = streamLookupConsumer(cg, cname);
if (consumer)
consumer->seen_time = now_ms;
else // TODO: notify xgroup-createconsumer event once we support stream events.
consumer = StreamCreateConsumer(cg, name, now_ms, SCC_DEFAULT);
return consumer;
}
constexpr uint8_t kClaimForce = 1 << 0;
constexpr uint8_t kClaimJustID = 1 << 1;
constexpr uint8_t kClaimLastID = 1 << 2;
@ -1240,7 +1256,6 @@ OpResult<ClaimInfo> OpClaim(const OpArgs& op_args, string_view key, const ClaimO
auto cgr_res = FindGroup(op_args, key, opts.group);
RETURN_ON_BAD_STATUS(cgr_res);
streamConsumer* consumer = nullptr;
uint64_t now_ms = op_args.db_cntx.time_now_ms;
ClaimInfo result;
result.justid = (opts.flags & kClaimJustID);
@ -1254,6 +1269,8 @@ OpResult<ClaimInfo> OpClaim(const OpArgs& op_args, string_view key, const ClaimO
StreamMemTracker tracker;
streamConsumer* consumer = FindOrAddConsumer(opts.consumer, cgr_res->cg, now_ms);
for (streamID id : ids) {
std::array<uint8_t, sizeof(streamID)> buf;
StreamEncodeID(buf.begin(), &id);
@ -1288,13 +1305,6 @@ OpResult<ClaimInfo> OpClaim(const OpArgs& op_args, string_view key, const ClaimO
}
}
// Try to get the consumer. If not found, create a new one.
auto cname = WrapSds(opts.consumer);
if ((consumer = streamLookupConsumer(cgr_res->cg, cname, SLC_NO_REFRESH)) == nullptr) {
consumer = StreamCreateConsumer(cgr_res->cg, opts.consumer, now_ms,
SCC_NO_NOTIFY | SCC_NO_DIRTIFY);
}
// If the entry belongs to the same consumer, we don't have to
// do anything. Else remove the entry from the old consumer.
if (nack->consumer != consumer) {
@ -1320,9 +1330,11 @@ OpResult<ClaimInfo> OpClaim(const OpArgs& op_args, string_view key, const ClaimO
raxInsert(consumer->pel, buf.begin(), sizeof(buf), nack, nullptr);
nack->consumer = consumer;
}
consumer->active_time = now_ms;
/* Send the reply for this entry. */
AppendClaimResultItem(result, cgr_res->s, id);
// TODO: propagate this change with streamPropagateXCLAIM
}
}
tracker.UpdateStreamSize(cgr_res->it->second);
@ -1382,8 +1394,7 @@ OpResult<uint32_t> OpDelConsumer(const OpArgs& op_args, string_view key, string_
StreamMemTracker mem_tracker;
long long pending = 0;
streamConsumer* consumer =
streamLookupConsumer(cgroup_res->cg, WrapSds(consumer_name), SLC_NO_REFRESH);
streamConsumer* consumer = streamLookupConsumer(cgroup_res->cg, WrapSds(consumer_name));
if (consumer) {
pending = raxSize(consumer->pel);
streamDelConsumer(cgroup_res->cg, consumer);
@ -1571,13 +1582,8 @@ OpResult<ClaimInfo> OpAutoClaim(const OpArgs& op_args, string_view key, const Cl
uint64_t now_ms = op_args.db_cntx.time_now_ms;
int count = opts.count;
auto cname = WrapSds(opts.consumer);
streamConsumer* consumer = streamLookupConsumer(group, cname, SLC_DEFAULT);
if (consumer == nullptr) {
consumer = StreamCreateConsumer(group, opts.consumer, now_ms, SCC_DEFAULT);
// TODO: notify xgroup-createconsumer event once we support stream events.
}
consumer->seen_time = now_ms;
streamConsumer* consumer = FindOrAddConsumer(opts.consumer, group, now_ms);
while (attempts-- && count && raxNext(&ri)) {
streamNACK* nack = (streamNACK*)ri.data;
@ -1621,7 +1627,7 @@ OpResult<ClaimInfo> OpAutoClaim(const OpArgs& op_args, string_view key, const Cl
raxInsert(consumer->pel, ri.key, ri.key_len, nack, nullptr);
nack->consumer = consumer;
}
consumer->active_time = now_ms;
AppendClaimResultItem(result, stream, id);
count--;
// TODO: propagate xclaim to replica
@ -1760,7 +1766,7 @@ OpResult<PendingResult> OpPending(const OpArgs& op_args, string_view key, const
streamConsumer* consumer = nullptr;
if (!opts.consumer_name.empty()) {
consumer = streamLookupConsumer(cgroup_res->cg, WrapSds(opts.consumer_name), SLC_NO_REFRESH);
consumer = streamLookupConsumer(cgroup_res->cg, WrapSds(opts.consumer_name));
}
PendingResult result;
@ -2114,8 +2120,12 @@ std::optional<ReadOpts> ParseReadArgsOrReply(CmdArgList args, bool read_group,
size_t pair_count = args.size() - opts.streams_arg;
if ((pair_count % 2) != 0) {
const auto m = "Unbalanced list of streams: for each stream key an ID must be specified";
builder->SendError(m, kSyntaxErr);
const char* cmd_name = read_group ? "xreadgroup" : "xread";
const char* symbol = read_group ? ">" : "$";
const auto msg = absl::StrCat("Unbalanced '", cmd_name,
"' list of streams: for each stream key an ID or '", symbol,
"' must be specified");
builder->SendError(msg, kSyntaxErr);
return std::nullopt;
}
streams_count = pair_count / 2;
@ -2149,7 +2159,7 @@ std::optional<ReadOpts> ParseReadArgsOrReply(CmdArgList args, bool read_group,
}
id.val.ms = 0;
id.val.seq = 0;
id.last_id = true;
id.resolve_last_id = true;
sitem.id = id;
auto [_, is_inserted] = opts.stream_ids.emplace(key, sitem);
if (!is_inserted) {
@ -2325,12 +2335,8 @@ void XReadBlock(ReadOpts* opts, Transaction* tx, SinkReplyBuilder* builder,
// Update consumer
if (sitem.group) {
auto cname = WrapSds(opts->consumer_name);
range_opts.consumer = streamLookupConsumer(sitem.group, cname, SLC_NO_REFRESH);
if (!range_opts.consumer) {
range_opts.consumer = StreamCreateConsumer(
sitem.group, opts->consumer_name, GetCurrentTimeMs(), SCC_NO_NOTIFY | SCC_NO_DIRTIFY);
}
range_opts.consumer =
FindOrAddConsumer(opts->consumer_name, sitem.group, GetCurrentTimeMs());
}
range_opts.noack = opts->noack;
@ -2898,7 +2904,7 @@ void StreamFamily::XInfo(CmdArgList args, const CommandContext& cmd_cntx) {
rb->SendBulkString("consumers");
rb->StartArray(ginfo.consumer_info_vec.size());
for (const auto& consumer_info : ginfo.consumer_info_vec) {
rb->StartCollection(4, RedisReplyBuilder::MAP);
rb->StartCollection(5, RedisReplyBuilder::MAP);
rb->SendBulkString("name");
rb->SendBulkString(consumer_info.name);
@ -2906,6 +2912,9 @@ void StreamFamily::XInfo(CmdArgList args, const CommandContext& cmd_cntx) {
rb->SendBulkString("seen-time");
rb->SendLong(consumer_info.seen_time);
rb->SendBulkString("active-time");
rb->SendLong(consumer_info.active_time);
rb->SendBulkString("pel-count");
rb->SendLong(consumer_info.pel_count);
@ -2957,14 +2966,20 @@ void StreamFamily::XInfo(CmdArgList args, const CommandContext& cmd_cntx) {
OpResult<vector<ConsumerInfo>> result = shard_set->Await(sid, std::move(cb));
if (result) {
rb->StartArray(result->size());
int64_t now_ms = GetCurrentTimeMs();
for (const auto& consumer_info : *result) {
rb->StartCollection(3, RedisReplyBuilder::MAP);
int64_t active = consumer_info.active_time;
int64_t inactive = active != -1 ? now_ms - active : -1;
rb->StartCollection(4, RedisReplyBuilder::MAP);
rb->SendBulkString("name");
rb->SendBulkString(consumer_info.name);
rb->SendBulkString("pending");
rb->SendLong(consumer_info.pel_count);
rb->SendBulkString("idle");
rb->SendLong(consumer_info.idle);
rb->SendBulkString("inactive");
rb->SendLong(inactive);
}
return;
}
@ -3095,26 +3110,11 @@ variant<bool, facade::ErrorReply> HasEntries2(const OpArgs& op_args, string_view
return facade::ErrorReply{
NoGroupOrKey(skey, opts->group_name, " in XREADGROUP with GROUP option")};
auto cname = WrapSds(opts->consumer_name);
consumer = streamLookupConsumer(group, cname, SLC_NO_REFRESH);
if (!consumer) {
consumer = StreamCreateConsumer(group, opts->consumer_name, op_args.db_cntx.time_now_ms,
SCC_NO_NOTIFY | SCC_NO_DIRTIFY);
}
consumer = FindOrAddConsumer(opts->consumer_name, group, op_args.db_cntx.time_now_ms);
requested_sitem.group = group;
requested_sitem.consumer = consumer;
}
// Resolve $ to the last ID in the stream.
if (requested_sitem.id.last_id && !opts->read_group) {
requested_sitem.id.val = last_id;
streamIncrID(&requested_sitem.id.val); // include id's strictly greater
requested_sitem.id.last_id = false;
return false;
}
if (opts->read_group) {
// If '>' is not provided, consumer PEL is used. So don't need to block.
if (requested_sitem.id.val.ms != UINT64_MAX || requested_sitem.id.val.seq != UINT64_MAX) {
opts->serve_history = true;
@ -3126,6 +3126,14 @@ variant<bool, facade::ErrorReply> HasEntries2(const OpArgs& op_args, string_view
requested_sitem.id.val = requested_sitem.group->last_id;
streamIncrID(&requested_sitem.id.val);
}
} else {
// Resolve $ to the last ID in the stream.
if (requested_sitem.id.resolve_last_id) {
requested_sitem.id.val = last_id;
streamIncrID(&requested_sitem.id.val); // include id's strictly greater
requested_sitem.id.resolve_last_id = false;
return false;
}
}
return streamCompareID(&last_id, &requested_sitem.id.val) >= 0;

View file

@ -4,6 +4,7 @@
#include "server/stream_family.h"
#include "base/flags.h"
#include "base/gtest.h"
#include "base/logging.h"
#include "facade/facade_test.h"
@ -14,6 +15,8 @@ using namespace testing;
using namespace std;
using namespace util;
ABSL_DECLARE_FLAG(bool, stream_rdb_encode_v2);
namespace dfly {
const auto kMatchNil = ArgType(RespExpr::NIL);
@ -404,8 +407,8 @@ TEST_F(StreamFamilyTest, XReadInvalidArgs) {
EXPECT_THAT(resp, ErrArg("syntax error"));
// Unbalanced list of streams.
resp = Run({"xread", "count", "invalid", "streams", "s1", "s2", "s3", "0", "0"});
EXPECT_THAT(resp, ErrArg("syntax error"));
resp = Run({"xread", "count", "invalid", "streams", "s1", "s2", "0", "0"});
EXPECT_THAT(resp, ErrArg("value is not an integer"));
// Wrong type.
Run({"set", "foo", "v"});
@ -442,7 +445,12 @@ TEST_F(StreamFamilyTest, XReadGroupInvalidArgs) {
// Unbalanced list of streams.
resp = Run({"xreadgroup", "group", "group", "alice", "streams", "s1", "s2", "s3", "0", "0"});
EXPECT_THAT(resp, ErrArg("syntax error"));
EXPECT_THAT(resp, ErrArg("Unbalanced 'xreadgroup' list of streams: for each stream key an ID or "
"'>' must be specified"));
resp = Run({"XREAD", "COUNT", "1", "STREAMS", "mystream"});
ASSERT_THAT(resp, ErrArg("Unbalanced 'xread' list of streams: for each stream key an ID or '$' "
"must be specified"));
}
TEST_F(StreamFamilyTest, XReadGroupEmpty) {
@ -857,8 +865,8 @@ TEST_F(StreamFamilyTest, XInfoConsumers) {
Run({"xgroup", "createconsumer", "mystream", "mygroup", "second-consumer"});
resp = Run({"xinfo", "consumers", "mystream", "mygroup"});
EXPECT_THAT(resp, ArrLen(2));
EXPECT_THAT(resp.GetVec()[0], ArrLen(6));
EXPECT_THAT(resp.GetVec()[1], ArrLen(6));
EXPECT_THAT(resp.GetVec()[0], ArrLen(8));
EXPECT_THAT(resp.GetVec()[1], ArrLen(8));
EXPECT_THAT(resp.GetVec()[0].GetVec()[1], "first-consumer");
EXPECT_THAT(resp.GetVec()[1].GetVec()[1], "second-consumer");
@ -1016,7 +1024,7 @@ TEST_F(StreamFamilyTest, XInfoStream) {
"consumers", ArrLen(1)));
EXPECT_THAT(resp.GetVec()[17].GetVec()[0].GetVec()[13].GetVec()[0].GetVec(),
ElementsAre("name", "first-consumer", "seen-time", ArgType(RespExpr::INT64),
"pel-count", IntArg(0), "pending", ArrLen(0)));
"active-time", IntArg(-1), "pel-count", IntArg(0), "pending", ArrLen(0)));
// full with count less than number of messages in stream
resp = Run({"xinfo", "stream", "mystream", "full", "count", "5"});
@ -1040,9 +1048,9 @@ TEST_F(StreamFamilyTest, XInfoStream) {
EXPECT_THAT(resp.GetVec()[17].GetVec()[0].GetVec()[9], IntArg(11)); // pel-count
EXPECT_THAT(resp.GetVec()[17].GetVec()[0].GetVec()[11], ArrLen(11)); // pending list
// consumer
EXPECT_THAT(resp.GetVec()[17].GetVec()[0].GetVec()[13].GetVec()[0].GetVec()[5],
IntArg(11)); // pel-count
EXPECT_THAT(resp.GetVec()[17].GetVec()[0].GetVec()[13].GetVec()[0].GetVec()[7],
IntArg(11)); // pel-count
EXPECT_THAT(resp.GetVec()[17].GetVec()[0].GetVec()[13].GetVec()[0].GetVec()[9],
ArrLen(11)); // pending list
// delete message
@ -1069,9 +1077,10 @@ TEST_F(StreamFamilyTest, XInfoStream) {
ElementsAre("name", "mygroup", "last-delivered-id", "11-1", "entries-read",
IntArg(11), "lag", IntArg(0), "pel-count", IntArg(11), "pending",
ArrLen(11), "consumers", ArrLen(1)));
EXPECT_THAT(resp.GetVec()[17].GetVec()[0].GetVec()[13].GetVec()[0].GetVec(),
ElementsAre("name", "first-consumer", "seen-time", ArgType(RespExpr::INT64),
"pel-count", IntArg(11), "pending", ArrLen(11)));
EXPECT_THAT(
resp.GetVec()[17].GetVec()[0].GetVec()[13].GetVec()[0].GetVec(),
ElementsAre("name", "first-consumer", "seen-time", ArgType(RespExpr::INT64), "active-time",
ArgType(RespExpr::INT64), "pel-count", IntArg(11), "pending", ArrLen(11)));
}
TEST_F(StreamFamilyTest, AutoClaimPelItemsFromAnotherConsumer) {
@ -1190,4 +1199,51 @@ TEST_F(StreamFamilyTest, XsetIdSmallerMaxDeleted) {
ASSERT_THAT(resp, ErrArg("smaller"));
}
TEST_F(StreamFamilyTest, SeenActiveTime) {
TEST_current_time_ms = 1000;
Run({"XGROUP", "CREATE", "mystream", "mygroup", "$", "MKSTREAM"});
Run({"XREADGROUP", "GROUP", "mygroup", "Alice", "COUNT", "1", "STREAMS", "mystream", ">"});
AdvanceTime(100);
auto resp = Run({"xinfo", "consumers", "mystream", "mygroup"});
EXPECT_THAT(resp, RespElementsAre("name", "Alice", "pending", IntArg(0), "idle", IntArg(100),
"inactive", IntArg(-1)));
Run({"XADD", "mystream", "*", "f", "v"});
Run({"XREADGROUP", "GROUP", "mygroup", "Alice", "COUNT", "1", "STREAMS", "mystream", ">"});
AdvanceTime(50);
resp = Run({"xinfo", "consumers", "mystream", "mygroup"});
EXPECT_THAT(resp, RespElementsAre("name", "Alice", "pending", IntArg(1), "idle", IntArg(50),
"inactive", IntArg(50)));
AdvanceTime(100);
resp = Run({"XREADGROUP", "GROUP", "mygroup", "Alice", "COUNT", "1", "STREAMS", "mystream", ">"});
EXPECT_THAT(resp, ArgType(RespExpr::NIL_ARRAY));
resp = Run({"xinfo", "consumers", "mystream", "mygroup"});
// Idle is 0 because XREADGROUP just run, but inactive continues clocking because nothing was
// read.
EXPECT_THAT(resp, RespElementsAre("name", "Alice", "pending", IntArg(1), "idle", IntArg(0),
"inactive", IntArg(150)));
// Serialize/deserialize.
resp = Run({"XINFO", "STREAM", "mystream", "FULL"});
auto groups = resp.GetVec()[17];
auto consumers = groups.GetVec()[0].GetVec()[13].GetVec()[0];
EXPECT_THAT(consumers, RespElementsAre("name", "Alice", "seen-time", IntArg(1250), "active-time",
IntArg(1100), "pel-count", IntArg(1), "pending", _));
absl::SetFlag(&FLAGS_stream_rdb_encode_v2, true);
resp = Run({"DUMP", "mystream"});
Run({"del", "mystream"});
resp = Run({"RESTORE", "mystream", "0", resp.GetString()});
EXPECT_EQ(resp, "OK");
resp = Run({"XINFO", "STREAM", "mystream", "FULL"});
groups = resp.GetVec()[17];
consumers = groups.GetVec()[0].GetVec()[13].GetVec()[0];
EXPECT_THAT(consumers, RespElementsAre("name", "Alice", "seen-time", IntArg(1250), "active-time",
IntArg(1100), "pel-count", IntArg(1), "pending", _));
}
} // namespace dfly

View file

@ -1563,9 +1563,6 @@ OpResult<KeyIndex> DetermineKeys(const CommandId* cid, CmdArgList args) {
string_view arg = ArgS(args, i);
if (absl::EqualsIgnoreCase(arg, "STREAMS")) {
size_t left = args.size() - i - 1;
if (left < 2 || left % 2 != 0)
return OpStatus::SYNTAX_ERR;
return KeyIndex(i + 1, i + 1 + (left / 2));
}
}

View file

@ -6,31 +6,6 @@ from . import dfly_args
from .instance import DflyInstance, DflyInstanceFactory
async def calculate_estimated_connection_memory(
async_client: aioredis.Redis, df_server: DflyInstance
):
memory_info = await async_client.info("memory")
already_used_rss_memory = memory_info["used_memory_rss"]
connections_number = 100
connections = []
for _ in range(connections_number):
conn = aioredis.Redis(port=df_server.port)
await conn.ping()
connections.append(conn)
await asyncio.sleep(1) # Wait RSS update
memory_info = await async_client.info("memory")
estimated_connections_memory = memory_info["used_memory_rss"] - already_used_rss_memory
# Close test connection
for conn in connections:
await conn.close()
return estimated_connections_memory // connections_number
@pytest.mark.opt_only
@pytest.mark.parametrize(
"type, keys, val_size, elements",
@ -188,106 +163,49 @@ async def test_eval_with_oom(df_factory: DflyInstanceFactory):
assert rss_before_eval * 1.01 > info["used_memory_rss"]
@pytest.mark.skip("rss eviction disabled")
@pytest.mark.asyncio
@dfly_args(
{
"proactor_threads": 1,
"cache_mode": "true",
"maxmemory": "256mb",
"rss_oom_deny_ratio": 0.5,
"max_eviction_per_heartbeat": 1000,
"maxmemory": "5gb",
"rss_oom_deny_ratio": 0.8,
"max_eviction_per_heartbeat": 100,
}
)
async def test_cache_eviction_with_rss_deny_oom(
async_client: aioredis.Redis,
df_server: DflyInstance,
):
"""
Test to verify that cache eviction is triggered even if used memory is small but rss memory is above limit
"""
max_memory = 256 * 1024 * 1024 # 256 MB
rss_max_memory = int(max_memory * 0.5) # 50% of max memory
max_memory = 5 * 1024 * 1024 * 1024 # 5G
rss_max_memory = int(max_memory * 0.8)
data_fill_size = int(0.55 * rss_max_memory) # 55% of rss_max_memory
rss_increase_size = int(0.55 * rss_max_memory) # 55% of max rss_max_memory
data_fill_size = int(0.9 * rss_max_memory) # 95% of rss_max_memory
key_size = 1024 * 5 # 5 kb
num_keys = data_fill_size // key_size
await asyncio.sleep(1) # Wait for RSS update
estimated_connection_memory = await calculate_estimated_connection_memory(
async_client, df_server
)
num_connections = rss_increase_size // estimated_connection_memory
logging.info(
f"Estimated connection memory: {estimated_connection_memory}. Number of connections: {num_connections}."
)
# Fill data to 55% of rss max memory
await async_client.execute_command("DEBUG", "POPULATE", num_keys, "key", key_size)
await asyncio.sleep(1) # Wait for RSS heartbeat update
# First test that eviction is not triggered without connection creation
stats_info = await async_client.info("stats")
assert stats_info["evicted_keys"] == 0, "No eviction should start yet."
val_size = 1024 * 5 # 5 kb
num_keys = data_fill_size // val_size
await async_client.execute_command("DEBUG", "POPULATE", num_keys, "key", val_size)
# Test that used memory is less than 90% of max memory
memory_info = await async_client.info("memory")
assert (
memory_info["used_memory"] < max_memory * 0.9
), "Used memory should be less than 90% of max memory."
assert (
memory_info["used_memory_rss"] < rss_max_memory * 0.9
memory_info["used_memory_rss"] > rss_max_memory * 0.9
), "RSS memory should be less than 90% of rss max memory (max_memory * rss_oom_deny_ratio)."
# Disable heartbeat eviction
await async_client.execute_command("CONFIG SET enable_heartbeat_eviction false")
# Increase RSS memory by 55% of rss max memory
# We can simulate RSS increase by creating new connections
connections = []
for _ in range(num_connections):
conn = aioredis.Redis(port=df_server.port)
await conn.ping()
connections.append(conn)
await asyncio.sleep(1)
# Check that RSS memory is above rss limit
memory_info = await async_client.info("memory")
assert (
memory_info["used_memory_rss"] >= rss_max_memory * 0.9
), "RSS memory should exceed 90% of the maximum RSS memory limit (max_memory * rss_oom_deny_ratio)."
# Enable heartbeat eviction
await async_client.execute_command("CONFIG SET enable_heartbeat_eviction true")
await asyncio.sleep(1) # Wait for RSS heartbeat update
await async_client.execute_command("MEMORY DECOMMIT")
await asyncio.sleep(1) # Wait for RSS update
# Get RSS memory after creating new connections
memory_info = await async_client.info("memory")
stats_info = await async_client.info("stats")
logging.info(f'Evicted keys number: {stats_info["evicted_keys"]}. Total keys: {num_keys}.')
assert (
memory_info["used_memory"] < data_fill_size
), "Used memory should be less than initial fill size due to eviction."
assert (
memory_info["used_memory_rss"] < rss_max_memory * 0.9
), "RSS memory should be less than 90% of rss max memory (max_memory * rss_oom_deny_ratio) after eviction."
# Check that eviction has occurred
assert (
stats_info["evicted_keys"] > 0
), "Eviction should have occurred due to rss memory pressure."
for conn in connections:
await conn.close()
while memory_info["used_memory_rss"] > rss_max_memory * 0.9:
await asyncio.sleep(1)
memory_info = await async_client.info("memory")
logging.info(
f'Current rss: {memory_info["used_memory_rss"]}. rss eviction threshold: {rss_max_memory * 0.9}.'
)
stats_info = await async_client.info("stats")
logging.info(f'Current evicted: {stats_info["evicted_keys"]}. Total keys: {num_keys}.')