Handle OOM errors for more types. Fix clang warnings and errors

This commit is contained in:
Roman Gershman 2022-05-18 21:53:40 +03:00
parent 30cf9541c2
commit 5e4aa0a1a8
20 changed files with 147 additions and 85 deletions

2
helio

@ -1 +1 @@
Subproject commit b7755de9b28e0185d6aba2f2424b8e9c64ae5328
Subproject commit 0ceb8599b2b3a66e471b7294bc63f00d168ab306

View File

@ -122,7 +122,8 @@ class DashTable : public detail::DashTableBase {
// false for duplicate, true if inserted.
template <typename U, typename V> std::pair<iterator, bool> Insert(U&& key, V&& value) {
return InsertInternal(std::forward<U>(key), std::forward<V>(value), DefaultEvictionPolicy{});
DefaultEvictionPolicy policy;
return InsertInternal(std::forward<U>(key), std::forward<V>(value), policy);
}
template <typename U, typename V, typename EvictionPolicy>
@ -244,7 +245,7 @@ class DashTable : public detail::DashTableBase {
private:
template <typename U, typename V, typename EvictionPolicy>
std::pair<iterator, bool> InsertInternal(U&& key, V&& value, EvictionPolicy&& policy);
std::pair<iterator, bool> InsertInternal(U&& key, V&& value, EvictionPolicy& policy);
void IncreaseDepth(unsigned new_depth);
void Split(uint32_t seg_id);
@ -566,7 +567,7 @@ template <typename _Key, typename _Value, typename Policy>
bool DashTable<_Key, _Value, Policy>::ShiftRight(bucket_iterator it) {
auto* seg = segment_[it.seg_id_];
typename Segment_t::Hash_t hash_val;
typename Segment_t::Hash_t hash_val = 0;
auto& bucket = seg->GetBucket(it.bucket_id_);
if (bucket.GetBusy() & (1 << (kBucketWidth - 1))) {
@ -675,7 +676,7 @@ void DashTable<_Key, _Value, Policy>::Reserve(size_t size) {
template <typename _Key, typename _Value, typename Policy>
template <typename U, typename V, typename EvictionPolicy>
auto DashTable<_Key, _Value, Policy>::InsertInternal(U&& key, V&& value, EvictionPolicy&& ev)
auto DashTable<_Key, _Value, Policy>::InsertInternal(U&& key, V&& value, EvictionPolicy& ev)
-> std::pair<iterator, bool> {
uint64_t key_hash = DoHash(key);
uint32_t seg_id = SegmentId(key_hash);
@ -700,7 +701,7 @@ auto DashTable<_Key, _Value, Policy>::InsertInternal(U&& key, V&& value, Evictio
// At this point we must split the segment.
// try garbage collect or evict.
if constexpr (ev.can_evict || ev.can_gc) {
if constexpr (EvictionPolicy::can_evict || EvictionPolicy::can_gc) {
// Try gc.
uint8_t bid[2];
SegmentType::FillProbeArray(key_hash, bid);
@ -716,7 +717,7 @@ auto DashTable<_Key, _Value, Policy>::InsertInternal(U&& key, V&& value, Evictio
// The difference between gc and eviction is that gc can be applied even if
// the table can grow since we throw away logically deleted items.
// For eviction to be applied we should reach the growth limit.
if constexpr (ev.can_gc) {
if constexpr (EvictionPolicy::can_gc) {
unsigned res = ev.GarbageCollect(buckets, this);
garbage_collected_ += res;
if (res)
@ -731,7 +732,7 @@ auto DashTable<_Key, _Value, Policy>::InsertInternal(U&& key, V&& value, Evictio
}
// We evict only if our policy says we can not grow
if constexpr (ev.can_evict) {
if constexpr (EvictionPolicy::can_evict) {
bool can_grow = ev.CanGrow(*this);
if (!can_grow) {
unsigned res = ev.Evict(buckets, this);

View File

@ -681,7 +681,7 @@ class DashCursor {
*/
template <unsigned NUM_SLOTS> void SlotBitmap<NUM_SLOTS>::SetSlot(unsigned index, bool probe) {
if (SINGLE) {
if constexpr (SINGLE) {
assert(((val_[0].d >> (index + 18)) & 1) == 0);
val_[0].d |= (1 << (index + 18));
val_[0].d |= (unsigned(probe) << (index + 4));
@ -698,7 +698,7 @@ template <unsigned NUM_SLOTS> void SlotBitmap<NUM_SLOTS>::SetSlot(unsigned index
template <unsigned NUM_SLOTS> void SlotBitmap<NUM_SLOTS>::ClearSlot(unsigned index) {
assert(Size() > 0);
if (SINGLE) {
if constexpr (SINGLE) {
uint32_t new_bitmap = val_[0].d & (~(1u << (index + 18))) & (~(1u << (index + 4)));
new_bitmap -= 1;
val_[0].d = new_bitmap;
@ -712,7 +712,7 @@ template <unsigned NUM_SLOTS> void SlotBitmap<NUM_SLOTS>::ClearSlot(unsigned ind
template <unsigned NUM_SLOTS> bool SlotBitmap<NUM_SLOTS>::ShiftLeft() {
constexpr uint32_t kBusyLastSlot = (kAllocMask >> 1) + 1;
bool res;
if (SINGLE) {
if constexpr (SINGLE) {
constexpr uint32_t kShlMask = kAllocMask - 1; // reset lsb
res = (val_[0].d & (kBusyLastSlot << 18)) != 0;
uint32_t l = (val_[0].d << 1) & (kShlMask << 4);
@ -745,7 +745,7 @@ template <unsigned NUM_SLOTS> void SlotBitmap<NUM_SLOTS>::Swap(unsigned slot_a,
if (slot_a > slot_b)
std::swap(slot_a, slot_b);
if (SINGLE) {
if constexpr (SINGLE) {
uint32_t a = (val_[0].d << (slot_b - slot_a)) ^ val_[0].d;
uint32_t bm = (1 << (slot_b + 4)) | (1 << (slot_b + 18));
a &= bm;

View File

@ -33,7 +33,7 @@ constexpr inline unsigned long long operator""_KB(unsigned long long x) {
*/
namespace detail {
class Page;
struct Page;
constexpr unsigned kNumFreePages = 29;

View File

@ -27,6 +27,6 @@ extern const char kExpiryOutOfRange[];
extern const char kSyntaxErrType[];
extern const char kScriptErrType[];
extern const char kIndexOutOfRange[];
extern const char kOutOfMemory[];
} // namespace dfly

View File

@ -72,6 +72,7 @@ const char kExpiryOutOfRange[] = "expiry is out of range";
const char kSyntaxErrType[] = "syntax_error";
const char kScriptErrType[] = "script_error";
const char kIndexOutOfRange[] = "index out of range";
const char kOutOfMemory[] = "Out of memory";
const char* RespExpr::TypeName(Type t) {

View File

@ -70,6 +70,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, std::string_view s) {
return !(left == s);
}
void PrintTo(const RespExpr::Vec& vec, std::ostream* os);
} // namespace facade

View File

@ -240,6 +240,9 @@ void RedisReplyBuilder::SendError(OpStatus status) {
case OpStatus::SYNTAX_ERR:
SendError(kSyntaxErr);
break;
case OpStatus::OUT_OF_MEMORY:
SendError(kOutOfMemory);
break;
default:
LOG(ERROR) << "Unsupported status " << status;
SendError("Internal error");
@ -254,6 +257,7 @@ void RedisReplyBuilder::SendLong(long num) {
void RedisReplyBuilder::SendDouble(double val) {
char buf[64];
StringBuilder sb(buf, sizeof(buf));
CHECK(dfly_conv.ToShortest(val, &sb));

View File

@ -13,7 +13,7 @@ namespace facade {
class ConnectionContext;
class Connection;
class ConnectionStats;
struct ConnectionStats;
class ServiceInterface {
public:

View File

@ -150,6 +150,7 @@ SliceEvents& SliceEvents::operator+=(const SliceEvents& o) {
#undef ADD
DbSlice::DbSlice(uint32_t index, bool caching_mode, EngineShard* owner)
: shard_id_(index), caching_mode_(caching_mode), owner_(owner) {
db_arr_.emplace_back();
@ -263,7 +264,8 @@ OpResult<pair<PrimeIterator, unsigned>> DbSlice::FindFirst(DbIndex db_index, Arg
return OpStatus::KEY_NOTFOUND;
}
auto DbSlice::AddOrFind(DbIndex db_index, string_view key) -> pair<PrimeIterator, bool> {
auto DbSlice::AddOrFind(DbIndex db_index, string_view key) noexcept(false)
-> pair<PrimeIterator, bool> {
DCHECK(IsDbValid(db_index));
auto& db = db_arr_[db_index];
@ -288,8 +290,16 @@ auto DbSlice::AddOrFind(DbIndex db_index, string_view key) -> pair<PrimeIterator
// Fast-path if change_cb_ is empty so we Find or Add using
// the insert operation: twice more efficient.
CompactObj co_key{key};
PrimeIterator it;
bool inserted;
// I try/catch just for sake of having a convenient place to set a breakpoint.
try {
tie(it, inserted) = db->prime.Insert(std::move(co_key), PrimeValue{}, evp);
} catch (bad_alloc& e) {
throw e;
}
auto [it, inserted] = db->prime.Insert(std::move(co_key), PrimeValue{}, evp);
if (inserted) { // new entry
db->stats.inline_keys += it->first.IsInline();
db->stats.obj_memory_usage += it->first.MallocUsed();
@ -440,20 +450,22 @@ uint32_t DbSlice::GetMCFlag(DbIndex db_ind, const PrimeKey& key) const {
return it.is_done() ? 0 : it->second;
}
PrimeIterator DbSlice::AddNew(DbIndex db_ind, string_view key, PrimeValue obj,
uint64_t expire_at_ms) {
auto [res, added] = AddOrFind(db_ind, key, std::move(obj), expire_at_ms);
uint64_t expire_at_ms) noexcept(false) {
auto [it, added] = AddOrFind(db_ind, key, std::move(obj), expire_at_ms);
CHECK(added);
return res;
return it;
}
pair<PrimeIterator, bool> DbSlice::AddOrFind(DbIndex db_ind, string_view key, PrimeValue obj,
uint64_t expire_at_ms) {
uint64_t expire_at_ms) noexcept(false) {
DCHECK_LT(db_ind, db_arr_.size());
DCHECK(!obj.IsRef());
auto res = AddOrFind(db_ind, key);
pair<PrimeIterator, bool> res = AddOrFind(db_ind, key);
if (!res.second) // have not inserted.
return res;
@ -476,6 +488,7 @@ pair<PrimeIterator, bool> DbSlice::AddOrFind(DbIndex db_ind, string_view key, Pr
return res;
}
size_t DbSlice::DbSize(DbIndex db_ind) const {
DCHECK_LT(db_ind, db_array_size());

View File

@ -123,13 +123,13 @@ class DbSlice {
// Return .second=true if insertion ocurred, false if we return the existing key.
// throws: bad_alloc is insertion could not happen due to out of memory.
std::pair<PrimeIterator, bool> AddOrFind(DbIndex db_ind, std::string_view key);
std::pair<PrimeIterator, bool> AddOrFind(DbIndex db_ind, std::string_view key) noexcept(false);
// Returns second=true if insertion took place, false otherwise.
// expire_at_ms equal to 0 - means no expiry.
// throws: bad_alloc is insertion could not happen due to out of memory.
std::pair<PrimeIterator, bool> AddOrFind(DbIndex db_ind, std::string_view key, PrimeValue obj,
uint64_t expire_at_ms);
uint64_t expire_at_ms) noexcept(false);
// Either adds or removes (if at == 0) expiry. Returns true if a change was made.
// Does not change expiry if at != 0 and expiry already exists.
@ -141,7 +141,8 @@ class DbSlice {
// Adds a new entry. Requires: key does not exist in this slice.
// Returns the iterator to the newly added entry.
// throws: bad_alloc is insertion could not happen due to out of memory.
PrimeIterator AddNew(DbIndex db_ind, std::string_view key, PrimeValue obj, uint64_t expire_at_ms);
PrimeIterator AddNew(DbIndex db_ind, std::string_view key, PrimeValue obj,
uint64_t expire_at_ms) noexcept(false);
// Creates a database with index `db_ind`. If such database exists does nothing.
void ActivateDb(DbIndex db_ind);

View File

@ -2,7 +2,10 @@
// See LICENSE for licensing terms.
//
// #include <mimalloc-new-delete.h>
#ifdef NDEBUG
#include <mimalloc-new-delete.h>
#endif
#include <mimalloc.h>
#include "base/init.h"
@ -15,6 +18,7 @@
#include "util/uring/uring_pool.h"
#include "util/varz.h"
DECLARE_uint32(port);
DECLARE_uint32(memcache_port);
DECLARE_uint64(maxmemory);

View File

@ -357,9 +357,7 @@ TEST_F(DflyEngineTest, LimitMemory) {
}
TEST_F(DflyEngineTest, FlushAll) {
auto fb0 = pp_->at(0)->LaunchFiber([&] {
Run({"flushall"});
});
auto fb0 = pp_->at(0)->LaunchFiber([&] { Run({"flushall"}); });
auto fb1 = pp_->at(1)->LaunchFiber([&] {
Run({"select", "2"});
@ -375,6 +373,37 @@ TEST_F(DflyEngineTest, FlushAll) {
fb1.join();
}
TEST_F(DflyEngineTest, OOM) {
shard_set->TEST_EnableHeartBeat();
max_memory_limit = 0;
size_t i = 0;
RespExpr resp;
for (; i < 10000; i += 3) {
resp = Run({"mset", StrCat("key", i), "bar", StrCat("key", i + 1), "bar", StrCat("key", i + 2),
"bar"});
if (resp != "OK")
break;
ASSERT_EQ(resp, "OK");
}
EXPECT_THAT(resp, ErrArg("Out of mem"));
for (; i < 10000; ++i) {
resp = Run({"set", StrCat("key", i), "bar"});
if (resp != "OK")
break;
}
EXPECT_THAT(resp, ErrArg("Out of mem"));
for (; i < 10000; ++i) {
resp = Run({"rpush", StrCat("list", i), "bar"});
if (resp.type == RespExpr::ERROR)
break;
ASSERT_THAT(resp, IntArg(1));
}
EXPECT_THAT(resp, ErrArg("Out of mem"));
}
// TODO: to test transactions with a single shard since then all transactions become local.
// To consider having a parameter in dragonfly engine controlling number of shards
// unconditionally from number of cpus. TO TEST BLPOP under multi for single/multi argument case.

View File

@ -300,27 +300,6 @@ void EngineShard::Heartbeat() {
}
void EngineShard::CacheStats() {
#if 0
mi_heap_t* tlh = mi_resource_.heap();
struct Sum {
size_t used = 0;
size_t comitted = 0;
} sum;
auto visit_cb = [](const mi_heap_t* heap, const mi_heap_area_t* area, void* block,
size_t block_size, void* arg) -> bool {
DCHECK(!block);
Sum* sum = (Sum*)arg;
// mimalloc mistakenly exports used in blocks instead of bytes.
sum->used += block_size * area->used;
sum->comitted += area->committed;
DVLOG(1) << "block_size " << block_size << "/" << area->block_size << ", reserved "
<< area->reserved << " comitted " << area->committed << " used: " << area->used;
return true; // continue iteration
};
#endif
// mi_heap_visit_blocks(tlh, false /* visit all blocks*/, visit_cb, &sum);
mi_stats_merge();
@ -344,6 +323,11 @@ void EngineShard::AddBlocked(Transaction* trans) {
blocking_controller_->AddWatched(trans);
}
void EngineShard::TEST_EnableHeartbeat() {
auto* pb = ProactorBase::me();
periodic_task_ = pb->AddPeriodic(1, [this] { Heartbeat(); });
}
/**
@ -361,7 +345,7 @@ void EngineShardSet::Init(uint32_t sz, bool update_db_time) {
cached_stats.resize(sz);
shard_queue_.resize(sz);
pp_->AwaitFiberOnAll([&](uint32_t index, ProactorBase* pb) {
pp_->AwaitFiberOnAll([&](uint32_t index, ProactorBase* pb) {
if (index < shard_queue_.size()) {
InitThreadLocal(pb, update_db_time);
}
@ -382,4 +366,8 @@ const vector<EngineShardSet::CachedStats>& EngineShardSet::GetCachedStats() {
return cached_stats;
}
void EngineShardSet::TEST_EnableHeartBeat() {
RunBriefInParallel([](EngineShard* shard) { shard->TEST_EnableHeartbeat(); });
}
} // namespace dfly

View File

@ -121,21 +121,6 @@ class EngineShard {
// for everyone to use for string transformations during atomic cpu sequences.
sds tmp_str1;
#if 0
size_t TEST_WatchedDbsLen() const {
return watched_dbs_.size();
}
size_t TEST_AwakenIndicesLen() const {
return awakened_indices_.size();
}
size_t TEST_AwakenTransLen() const {
return awakened_transactions_.size();
}
bool HasResultConverged(TxId notifyid) const;
#endif
// Moving average counters.
enum MovingCnt {
@ -148,6 +133,8 @@ class EngineShard {
return counter_[unsigned(type)].SumTail();
}
void TEST_EnableHeartbeat();
private:
EngineShard(util::ProactorBase* pb, bool update_db_time, mi_heap_t* heap);
@ -235,6 +222,9 @@ class EngineShardSet {
template <typename U> void RunBlockingInParallel(U&& func);
// Used in tests
void TEST_EnableHeartBeat();
private:
void InitThreadLocal(util::ProactorBase* pb, bool update_db_time);

View File

@ -299,8 +299,16 @@ OpResult<string> OpRPopLPushSingleShard(const OpArgs& op_args, string_view src,
}
quicklist* dest_ql = nullptr;
auto [dest_it, created] = db_slice.AddOrFind(op_args.db_ind, dest);
if (created) {
pair<PrimeIterator, bool> res;
try {
res = db_slice.AddOrFind(op_args.db_ind, dest);
} catch (bad_alloc&) {
return OpStatus::OUT_OF_MEMORY;
}
PrimeIterator& dest_it = res.first;
if (res.second) {
robj* obj = createQuicklistObject();
dest_ql = (quicklist*)obj->ptr;
quicklistSetOptions(dest_ql, FLAGS_list_max_listpack_size, FLAGS_list_compress_depth);
@ -366,8 +374,13 @@ OpResult<uint32_t> OpPush(const OpArgs& op_args, std::string_view key, ListDir d
return it_res.status();
it = *it_res;
} else {
tie(it, new_key) = es->db_slice().AddOrFind(op_args.db_ind, key);
try {
tie(it, new_key) = es->db_slice().AddOrFind(op_args.db_ind, key);
} catch (bad_alloc&) {
return OpStatus::OUT_OF_MEMORY;
}
}
quicklist* ql = nullptr;
if (new_key) {
@ -599,11 +612,11 @@ void ListFamily::LInsert(CmdArgList args, ConnectionContext* cntx) {
};
OpResult<int> result = cntx->transaction->ScheduleSingleHopT(std::move(cb));
if (result.status() == OpStatus::WRONG_TYPE) {
return (*cntx)->SendError(kWrongTypeErr);
if (result) {
return (*cntx)->SendLong(result.value());
}
(*cntx)->SendLong(result.value());
(*cntx)->SendError(result.status());
}
void ListFamily::LTrim(CmdArgList args, ConnectionContext* cntx) {
@ -751,11 +764,11 @@ void ListFamily::PushGeneric(ListDir dir, bool skip_notexists, CmdArgList args,
};
OpResult<uint32_t> result = cntx->transaction->ScheduleSingleHopT(std::move(cb));
if (result.status() == OpStatus::WRONG_TYPE) {
return (*cntx)->SendError(kWrongTypeErr);
if (result) {
return (*cntx)->SendLong(result.value());
}
return (*cntx)->SendLong(result.value());
return (*cntx)->SendError(result.status());
}
void ListFamily::PopGeneric(ListDir dir, CmdArgList args, ConnectionContext* cntx) {

View File

@ -789,7 +789,7 @@ void Service::EvalInternal(const EvalArgs& eval_args, Interpreter* interpreter,
// TODO: to determine whether the script is RO by scanning all "redis.p?call" calls
// and checking whether all invocations consist of RO commands.
// we can do it once during script insertion into script mgr.
cntx->conn_state.script_info.emplace();
cntx->conn_state.script_info.emplace(ConnectionState::Script{});
for (size_t i = 0; i < eval_args.keys.size(); ++i) {
cntx->conn_state.script_info->keys.insert(ArgS(eval_args.keys, i));
}

View File

@ -288,6 +288,10 @@ void StringFamily::Set(CmdArgList args, ConnectionContext* cntx) {
return builder->SendStored();
}
if (result == OpStatus::OUT_OF_MEMORY) {
return builder->SendError(kOutOfMemory);
}
CHECK_EQ(result, OpStatus::SKIPPED); // in case of NX option
return builder->SendSetSkipped();
@ -587,8 +591,12 @@ void StringFamily::MSet(CmdArgList args, ConnectionContext* cntx) {
return OpMSet(OpArgs{es, t->db_index()}, args);
};
transaction->ScheduleSingleHop(std::move(cb));
(*cntx)->SendOk();
OpStatus status = transaction->ScheduleSingleHop(std::move(cb));
if (status == OpStatus::OK) {
(*cntx)->SendOk();
} else {
(*cntx)->SendError(status);
}
}
void StringFamily::MSetNx(CmdArgList args, ConnectionContext* cntx) {
@ -745,9 +753,8 @@ OpStatus StringFamily::OpMSet(const OpArgs& op_args, ArgSlice args) {
for (size_t i = 0; i < args.size(); i += 2) {
DVLOG(1) << "MSet " << args[i] << ":" << args[i + 1];
auto res = sg.Set(params, args[i], args[i + 1]);
if (!res) {
LOG(ERROR) << "Unexpected error " << res.status();
return OpStatus::OK; // Multi-key operations must return OK.
if (!res) { // OOM for example.
return res.status();
}
}
@ -768,8 +775,11 @@ OpResult<int64_t> StringFamily::OpIncrBy(const OpArgs& op_args, std::string_view
CompactObj cobj;
cobj.SetInt(incr);
db_slice.AddNew(op_args.db_ind, key, std::move(cobj), 0);
try {
db_slice.AddNew(op_args.db_ind, key, std::move(cobj), 0);
} catch (bad_alloc&) {
return OpStatus::OUT_OF_MEMORY;
}
return incr;
}

View File

@ -26,7 +26,7 @@ DbTableStats& DbTableStats::operator+=(const DbTableStats& o) {
}
DbTable::DbTable(std::pmr::memory_resource* mr)
: prime(4, detail::PrimeTablePolicy{}, mr),
: prime(2, detail::PrimeTablePolicy{}, mr),
expire(0, detail::ExpireTablePolicy{}, mr),
mcflag(0, detail::ExpireTablePolicy{}, mr) {
}

View File

@ -338,7 +338,11 @@ bool Transaction::RunInShard(EngineShard* shard) {
cb_ = nullptr; // We can do it because only a single thread runs the callback.
local_result_ = status;
} else {
CHECK_EQ(OpStatus::OK, status);
if (status == OpStatus::OUT_OF_MEMORY) {
local_result_ = status;
} else {
CHECK_EQ(OpStatus::OK, status);
}
}
} catch (std::bad_alloc&) {
// TODO: to log at most once per sec.