rocksdb/memtable/write_buffer_manager.cc

204 lines
6.2 KiB
C++
Raw Normal View History

// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
// This source code is licensed under both the GPLv2 (found in the
// COPYING file in the root directory) and Apache 2.0 License
// (found in the LICENSE.Apache file in the root directory).
//
// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
// Use of this source code is governed by a BSD-style license that can be
// found in the LICENSE file. See the AUTHORS file for names of contributors.
#include "rocksdb/write_buffer_manager.h"
Stall writes in WriteBufferManager when memory_usage exceeds buffer_size (#7898) Summary: When WriteBufferManager is shared across DBs and column families to maintain memory usage under a limit, OOMs have been observed when flush cannot finish but writes continuously insert to memtables. In order to avoid OOMs, when memory usage goes beyond buffer_limit_ and DBs tries to write, this change will stall incoming writers until flush is completed and memory_usage drops. Design: Stall condition: When total memory usage exceeds WriteBufferManager::buffer_size_ (memory_usage() >= buffer_size_) WriterBufferManager::ShouldStall() returns true. DBImpl first block incoming/future writers by calling write_thread_.BeginWriteStall() (which adds dummy stall object to the writer's queue). Then DB is blocked on a state State::Blocked (current write doesn't go through). WBStallInterface object maintained by every DB instance is added to the queue of WriteBufferManager. If multiple DBs tries to write during this stall, they will also be blocked when check WriteBufferManager::ShouldStall() returns true. End Stall condition: When flush is finished and memory usage goes down, stall will end only if memory waiting to be flushed is less than buffer_size/2. This lower limit will give time for flush to complete and avoid continous stalling if memory usage remains close to buffer_size. WriterBufferManager::EndWriteStall() is called, which removes all instances from its queue and signal them to continue. Their state is changed to State::Running and they are unblocked. DBImpl then signal all incoming writers of that DB to continue by calling write_thread_.EndWriteStall() (which removes dummy stall object from the queue). DB instance creates WBMStallInterface which is an interface to block and signal DBs during stall. When DB needs to be blocked or signalled by WriteBufferManager, state_for_wbm_ state is changed accordingly (RUNNING or BLOCKED). Pull Request resolved: https://github.com/facebook/rocksdb/pull/7898 Test Plan: Added a new test db/db_write_buffer_manager_test.cc Reviewed By: anand1976 Differential Revision: D26093227 Pulled By: akankshamahajan15 fbshipit-source-id: 2bbd982a3fb7033f6de6153aa92a221249861aae
2021-04-21 20:53:05 +00:00
Use deleters to label cache entries and collect stats (#8297) Summary: This change gathers and publishes statistics about the kinds of items in block cache. This is especially important for profiling relative usage of cache by index vs. filter vs. data blocks. It works by iterating over the cache during periodic stats dump (InternalStats, stats_dump_period_sec) or on demand when DB::Get(Map)Property(kBlockCacheEntryStats), except that for efficiency and sharing among column families, saved data from the last scan is used when the data is not considered too old. The new information can be seen in info LOG, for example: Block cache LRUCache@0x7fca62229330 capacity: 95.37 MB collections: 8 last_copies: 0 last_secs: 0.00178 secs_since: 0 Block cache entry stats(count,size,portion): DataBlock(7092,28.24 MB,29.6136%) FilterBlock(215,867.90 KB,0.888728%) FilterMetaBlock(2,5.31 KB,0.00544%) IndexBlock(217,180.11 KB,0.184432%) WriteBuffer(1,256.00 KB,0.262144%) Misc(1,0.00 KB,0%) And also through DB::GetProperty and GetMapProperty (here using ldb just for demonstration): $ ./ldb --db=/dev/shm/dbbench/ get_property rocksdb.block-cache-entry-stats rocksdb.block-cache-entry-stats.bytes.data-block: 0 rocksdb.block-cache-entry-stats.bytes.deprecated-filter-block: 0 rocksdb.block-cache-entry-stats.bytes.filter-block: 0 rocksdb.block-cache-entry-stats.bytes.filter-meta-block: 0 rocksdb.block-cache-entry-stats.bytes.index-block: 178992 rocksdb.block-cache-entry-stats.bytes.misc: 0 rocksdb.block-cache-entry-stats.bytes.other-block: 0 rocksdb.block-cache-entry-stats.bytes.write-buffer: 0 rocksdb.block-cache-entry-stats.capacity: 8388608 rocksdb.block-cache-entry-stats.count.data-block: 0 rocksdb.block-cache-entry-stats.count.deprecated-filter-block: 0 rocksdb.block-cache-entry-stats.count.filter-block: 0 rocksdb.block-cache-entry-stats.count.filter-meta-block: 0 rocksdb.block-cache-entry-stats.count.index-block: 215 rocksdb.block-cache-entry-stats.count.misc: 1 rocksdb.block-cache-entry-stats.count.other-block: 0 rocksdb.block-cache-entry-stats.count.write-buffer: 0 rocksdb.block-cache-entry-stats.id: LRUCache@0x7f3636661290 rocksdb.block-cache-entry-stats.percent.data-block: 0.000000 rocksdb.block-cache-entry-stats.percent.deprecated-filter-block: 0.000000 rocksdb.block-cache-entry-stats.percent.filter-block: 0.000000 rocksdb.block-cache-entry-stats.percent.filter-meta-block: 0.000000 rocksdb.block-cache-entry-stats.percent.index-block: 2.133751 rocksdb.block-cache-entry-stats.percent.misc: 0.000000 rocksdb.block-cache-entry-stats.percent.other-block: 0.000000 rocksdb.block-cache-entry-stats.percent.write-buffer: 0.000000 rocksdb.block-cache-entry-stats.secs_for_last_collection: 0.000052 rocksdb.block-cache-entry-stats.secs_since_last_collection: 0 Solution detail - We need some way to flag what kind of blocks each entry belongs to, preferably without changing the Cache API. One of the complications is that Cache is a general interface that could have other users that don't adhere to whichever convention we decide on for keys and values. Or we would pay for an extra field in the Handle that would only be used for this purpose. This change uses a back-door approach, the deleter, to indicate the "role" of a Cache entry (in addition to the value type, implicitly). This has the added benefit of ensuring proper code origin whenever we recognize a particular role for a cache entry; if the entry came from some other part of the code, it will use an unrecognized deleter, which we simply attribute to the "Misc" role. An internal API makes for simple instantiation and automatic registration of Cache deleters for a given value type and "role". Another internal API, CacheEntryStatsCollector, solves the problem of caching the results of a scan and sharing them, to ensure scans are neither excessive nor redundant so as not to harm Cache performance. Because code is added to BlocklikeTraits, it is pulled out of block_based_table_reader.cc into its own file. This is a reformulation of https://github.com/facebook/rocksdb/issues/8276, without the type checking option (could still be added), and with actual stat gathering. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8297 Test Plan: manual testing with db_bench, and a couple of basic unit tests Reviewed By: ltamasi Differential Revision: D28488721 Pulled By: pdillinger fbshipit-source-id: 472f524a9691b5afb107934be2d41d84f2b129fb
2021-05-19 23:45:51 +00:00
#include "cache/cache_entry_roles.h"
Refactor WriteBufferManager::CacheRep into CacheReservationManager (#8506) Summary: Context: To help cap various memory usage by a single limit of the block cache capacity, we charge the memory usage through inserting/releasing dummy entries in the block cache. CacheReservationManager is such a class (non thread-safe) responsible for inserting/removing dummy entries to reserve cache space for memory used by the class user. - Refactored the inner private class CacheRep of WriteBufferManager into public CacheReservationManager class for reusability such as for https://github.com/facebook/rocksdb/pull/8428 - Encapsulated implementation details of cache key generation and dummy entries insertion/release in cache reservation as discussed in https://github.com/facebook/rocksdb/pull/8506#discussion_r666550838 - Consolidated increase/decrease cache reservation into one API - UpdateCacheReservation. - Adjusted the previous dummy entry release algorithm in decreasing cache reservation to be loop-releasing dummy entries to stay symmetric to dummy entry insertion algorithm - Made the previous dummy entry release algorithm in delayed decrease mode more aggressive for better decreasing cache reservation when memory used is less likely to increase back. Previously, the algorithms only release 1 dummy entries when new_mem_used < 3/4 * cache_allocated_size_ and cache_allocated_size_ - kSizeDummyEntry > new_mem_used. Now, the algorithms loop-releases as many dummy entries as possible when new_mem_used < 3/4 * cache_allocated_size_. - Updated WriteBufferManager's test cases to adapt to changes on the release algorithm mentioned above and left comment for some test cases for clarity - Replaced the previous cache key prefix generation (utilizing object address related to the cache client) with one that utilizes Cache->NewID() to prevent cache-key collision among dummy entry clients sharing the same cache. The specific collision we are preventing happens when the object address is reused for a new cache-key prefix while the old cache-key using that same object address in its prefix still exists in the cache. This could happen due to that, under LRU cache policy, there is a possible delay in releasing a cache entry after the cache client object owning that cache entry get deallocated. In this case, the object address related to the cache client object can get reused for other client object to generate a new cache-key prefix. This prefix generation can be made obsolete after Peter's unification of all the code generating cache key, mentioned in https://github.com/facebook/rocksdb/pull/8506#discussion_r667265255 Pull Request resolved: https://github.com/facebook/rocksdb/pull/8506 Test Plan: - Passing the added unit tests cache_reservation_manager_test.cc - Passing existing and adjusted write_buffer_manager_test.cc Reviewed By: ajkr Differential Revision: D29644135 Pulled By: hx235 fbshipit-source-id: 0fc93fbfe4a40bb41be85c314f8f2bafa8b741f7
2021-08-24 19:42:31 +00:00
#include "cache/cache_reservation_manager.h"
Stall writes in WriteBufferManager when memory_usage exceeds buffer_size (#7898) Summary: When WriteBufferManager is shared across DBs and column families to maintain memory usage under a limit, OOMs have been observed when flush cannot finish but writes continuously insert to memtables. In order to avoid OOMs, when memory usage goes beyond buffer_limit_ and DBs tries to write, this change will stall incoming writers until flush is completed and memory_usage drops. Design: Stall condition: When total memory usage exceeds WriteBufferManager::buffer_size_ (memory_usage() >= buffer_size_) WriterBufferManager::ShouldStall() returns true. DBImpl first block incoming/future writers by calling write_thread_.BeginWriteStall() (which adds dummy stall object to the writer's queue). Then DB is blocked on a state State::Blocked (current write doesn't go through). WBStallInterface object maintained by every DB instance is added to the queue of WriteBufferManager. If multiple DBs tries to write during this stall, they will also be blocked when check WriteBufferManager::ShouldStall() returns true. End Stall condition: When flush is finished and memory usage goes down, stall will end only if memory waiting to be flushed is less than buffer_size/2. This lower limit will give time for flush to complete and avoid continous stalling if memory usage remains close to buffer_size. WriterBufferManager::EndWriteStall() is called, which removes all instances from its queue and signal them to continue. Their state is changed to State::Running and they are unblocked. DBImpl then signal all incoming writers of that DB to continue by calling write_thread_.EndWriteStall() (which removes dummy stall object from the queue). DB instance creates WBMStallInterface which is an interface to block and signal DBs during stall. When DB needs to be blocked or signalled by WriteBufferManager, state_for_wbm_ state is changed accordingly (RUNNING or BLOCKED). Pull Request resolved: https://github.com/facebook/rocksdb/pull/7898 Test Plan: Added a new test db/db_write_buffer_manager_test.cc Reviewed By: anand1976 Differential Revision: D26093227 Pulled By: akankshamahajan15 fbshipit-source-id: 2bbd982a3fb7033f6de6153aa92a221249861aae
2021-04-21 20:53:05 +00:00
#include "db/db_impl/db_impl.h"
Refactor WriteBufferManager::CacheRep into CacheReservationManager (#8506) Summary: Context: To help cap various memory usage by a single limit of the block cache capacity, we charge the memory usage through inserting/releasing dummy entries in the block cache. CacheReservationManager is such a class (non thread-safe) responsible for inserting/removing dummy entries to reserve cache space for memory used by the class user. - Refactored the inner private class CacheRep of WriteBufferManager into public CacheReservationManager class for reusability such as for https://github.com/facebook/rocksdb/pull/8428 - Encapsulated implementation details of cache key generation and dummy entries insertion/release in cache reservation as discussed in https://github.com/facebook/rocksdb/pull/8506#discussion_r666550838 - Consolidated increase/decrease cache reservation into one API - UpdateCacheReservation. - Adjusted the previous dummy entry release algorithm in decreasing cache reservation to be loop-releasing dummy entries to stay symmetric to dummy entry insertion algorithm - Made the previous dummy entry release algorithm in delayed decrease mode more aggressive for better decreasing cache reservation when memory used is less likely to increase back. Previously, the algorithms only release 1 dummy entries when new_mem_used < 3/4 * cache_allocated_size_ and cache_allocated_size_ - kSizeDummyEntry > new_mem_used. Now, the algorithms loop-releases as many dummy entries as possible when new_mem_used < 3/4 * cache_allocated_size_. - Updated WriteBufferManager's test cases to adapt to changes on the release algorithm mentioned above and left comment for some test cases for clarity - Replaced the previous cache key prefix generation (utilizing object address related to the cache client) with one that utilizes Cache->NewID() to prevent cache-key collision among dummy entry clients sharing the same cache. The specific collision we are preventing happens when the object address is reused for a new cache-key prefix while the old cache-key using that same object address in its prefix still exists in the cache. This could happen due to that, under LRU cache policy, there is a possible delay in releasing a cache entry after the cache client object owning that cache entry get deallocated. In this case, the object address related to the cache client object can get reused for other client object to generate a new cache-key prefix. This prefix generation can be made obsolete after Peter's unification of all the code generating cache key, mentioned in https://github.com/facebook/rocksdb/pull/8506#discussion_r667265255 Pull Request resolved: https://github.com/facebook/rocksdb/pull/8506 Test Plan: - Passing the added unit tests cache_reservation_manager_test.cc - Passing existing and adjusted write_buffer_manager_test.cc Reviewed By: ajkr Differential Revision: D29644135 Pulled By: hx235 fbshipit-source-id: 0fc93fbfe4a40bb41be85c314f8f2bafa8b741f7
2021-08-24 19:42:31 +00:00
#include "rocksdb/status.h"
#include "util/coding.h"
namespace ROCKSDB_NAMESPACE {
WriteBufferManager::WriteBufferManager(size_t _buffer_size,
Stall writes in WriteBufferManager when memory_usage exceeds buffer_size (#7898) Summary: When WriteBufferManager is shared across DBs and column families to maintain memory usage under a limit, OOMs have been observed when flush cannot finish but writes continuously insert to memtables. In order to avoid OOMs, when memory usage goes beyond buffer_limit_ and DBs tries to write, this change will stall incoming writers until flush is completed and memory_usage drops. Design: Stall condition: When total memory usage exceeds WriteBufferManager::buffer_size_ (memory_usage() >= buffer_size_) WriterBufferManager::ShouldStall() returns true. DBImpl first block incoming/future writers by calling write_thread_.BeginWriteStall() (which adds dummy stall object to the writer's queue). Then DB is blocked on a state State::Blocked (current write doesn't go through). WBStallInterface object maintained by every DB instance is added to the queue of WriteBufferManager. If multiple DBs tries to write during this stall, they will also be blocked when check WriteBufferManager::ShouldStall() returns true. End Stall condition: When flush is finished and memory usage goes down, stall will end only if memory waiting to be flushed is less than buffer_size/2. This lower limit will give time for flush to complete and avoid continous stalling if memory usage remains close to buffer_size. WriterBufferManager::EndWriteStall() is called, which removes all instances from its queue and signal them to continue. Their state is changed to State::Running and they are unblocked. DBImpl then signal all incoming writers of that DB to continue by calling write_thread_.EndWriteStall() (which removes dummy stall object from the queue). DB instance creates WBMStallInterface which is an interface to block and signal DBs during stall. When DB needs to be blocked or signalled by WriteBufferManager, state_for_wbm_ state is changed accordingly (RUNNING or BLOCKED). Pull Request resolved: https://github.com/facebook/rocksdb/pull/7898 Test Plan: Added a new test db/db_write_buffer_manager_test.cc Reviewed By: anand1976 Differential Revision: D26093227 Pulled By: akankshamahajan15 fbshipit-source-id: 2bbd982a3fb7033f6de6153aa92a221249861aae
2021-04-21 20:53:05 +00:00
std::shared_ptr<Cache> cache,
bool allow_stall)
: buffer_size_(_buffer_size),
mutable_limit_(buffer_size_ * 7 / 8),
memory_used_(0),
memory_active_(0),
cache_res_mgr_(nullptr),
Stall writes in WriteBufferManager when memory_usage exceeds buffer_size (#7898) Summary: When WriteBufferManager is shared across DBs and column families to maintain memory usage under a limit, OOMs have been observed when flush cannot finish but writes continuously insert to memtables. In order to avoid OOMs, when memory usage goes beyond buffer_limit_ and DBs tries to write, this change will stall incoming writers until flush is completed and memory_usage drops. Design: Stall condition: When total memory usage exceeds WriteBufferManager::buffer_size_ (memory_usage() >= buffer_size_) WriterBufferManager::ShouldStall() returns true. DBImpl first block incoming/future writers by calling write_thread_.BeginWriteStall() (which adds dummy stall object to the writer's queue). Then DB is blocked on a state State::Blocked (current write doesn't go through). WBStallInterface object maintained by every DB instance is added to the queue of WriteBufferManager. If multiple DBs tries to write during this stall, they will also be blocked when check WriteBufferManager::ShouldStall() returns true. End Stall condition: When flush is finished and memory usage goes down, stall will end only if memory waiting to be flushed is less than buffer_size/2. This lower limit will give time for flush to complete and avoid continous stalling if memory usage remains close to buffer_size. WriterBufferManager::EndWriteStall() is called, which removes all instances from its queue and signal them to continue. Their state is changed to State::Running and they are unblocked. DBImpl then signal all incoming writers of that DB to continue by calling write_thread_.EndWriteStall() (which removes dummy stall object from the queue). DB instance creates WBMStallInterface which is an interface to block and signal DBs during stall. When DB needs to be blocked or signalled by WriteBufferManager, state_for_wbm_ state is changed accordingly (RUNNING or BLOCKED). Pull Request resolved: https://github.com/facebook/rocksdb/pull/7898 Test Plan: Added a new test db/db_write_buffer_manager_test.cc Reviewed By: anand1976 Differential Revision: D26093227 Pulled By: akankshamahajan15 fbshipit-source-id: 2bbd982a3fb7033f6de6153aa92a221249861aae
2021-04-21 20:53:05 +00:00
allow_stall_(allow_stall),
stall_active_(false) {
#ifndef ROCKSDB_LITE
if (cache) {
Refactor WriteBufferManager::CacheRep into CacheReservationManager (#8506) Summary: Context: To help cap various memory usage by a single limit of the block cache capacity, we charge the memory usage through inserting/releasing dummy entries in the block cache. CacheReservationManager is such a class (non thread-safe) responsible for inserting/removing dummy entries to reserve cache space for memory used by the class user. - Refactored the inner private class CacheRep of WriteBufferManager into public CacheReservationManager class for reusability such as for https://github.com/facebook/rocksdb/pull/8428 - Encapsulated implementation details of cache key generation and dummy entries insertion/release in cache reservation as discussed in https://github.com/facebook/rocksdb/pull/8506#discussion_r666550838 - Consolidated increase/decrease cache reservation into one API - UpdateCacheReservation. - Adjusted the previous dummy entry release algorithm in decreasing cache reservation to be loop-releasing dummy entries to stay symmetric to dummy entry insertion algorithm - Made the previous dummy entry release algorithm in delayed decrease mode more aggressive for better decreasing cache reservation when memory used is less likely to increase back. Previously, the algorithms only release 1 dummy entries when new_mem_used < 3/4 * cache_allocated_size_ and cache_allocated_size_ - kSizeDummyEntry > new_mem_used. Now, the algorithms loop-releases as many dummy entries as possible when new_mem_used < 3/4 * cache_allocated_size_. - Updated WriteBufferManager's test cases to adapt to changes on the release algorithm mentioned above and left comment for some test cases for clarity - Replaced the previous cache key prefix generation (utilizing object address related to the cache client) with one that utilizes Cache->NewID() to prevent cache-key collision among dummy entry clients sharing the same cache. The specific collision we are preventing happens when the object address is reused for a new cache-key prefix while the old cache-key using that same object address in its prefix still exists in the cache. This could happen due to that, under LRU cache policy, there is a possible delay in releasing a cache entry after the cache client object owning that cache entry get deallocated. In this case, the object address related to the cache client object can get reused for other client object to generate a new cache-key prefix. This prefix generation can be made obsolete after Peter's unification of all the code generating cache key, mentioned in https://github.com/facebook/rocksdb/pull/8506#discussion_r667265255 Pull Request resolved: https://github.com/facebook/rocksdb/pull/8506 Test Plan: - Passing the added unit tests cache_reservation_manager_test.cc - Passing existing and adjusted write_buffer_manager_test.cc Reviewed By: ajkr Differential Revision: D29644135 Pulled By: hx235 fbshipit-source-id: 0fc93fbfe4a40bb41be85c314f8f2bafa8b741f7
2021-08-24 19:42:31 +00:00
// Memtable's memory usage tends to fluctuate frequently
// therefore we set delayed_decrease = true to save some dummy entry
// insertion on memory increase right after memory decrease
cache_res_mgr_.reset(
Refactor WriteBufferManager::CacheRep into CacheReservationManager (#8506) Summary: Context: To help cap various memory usage by a single limit of the block cache capacity, we charge the memory usage through inserting/releasing dummy entries in the block cache. CacheReservationManager is such a class (non thread-safe) responsible for inserting/removing dummy entries to reserve cache space for memory used by the class user. - Refactored the inner private class CacheRep of WriteBufferManager into public CacheReservationManager class for reusability such as for https://github.com/facebook/rocksdb/pull/8428 - Encapsulated implementation details of cache key generation and dummy entries insertion/release in cache reservation as discussed in https://github.com/facebook/rocksdb/pull/8506#discussion_r666550838 - Consolidated increase/decrease cache reservation into one API - UpdateCacheReservation. - Adjusted the previous dummy entry release algorithm in decreasing cache reservation to be loop-releasing dummy entries to stay symmetric to dummy entry insertion algorithm - Made the previous dummy entry release algorithm in delayed decrease mode more aggressive for better decreasing cache reservation when memory used is less likely to increase back. Previously, the algorithms only release 1 dummy entries when new_mem_used < 3/4 * cache_allocated_size_ and cache_allocated_size_ - kSizeDummyEntry > new_mem_used. Now, the algorithms loop-releases as many dummy entries as possible when new_mem_used < 3/4 * cache_allocated_size_. - Updated WriteBufferManager's test cases to adapt to changes on the release algorithm mentioned above and left comment for some test cases for clarity - Replaced the previous cache key prefix generation (utilizing object address related to the cache client) with one that utilizes Cache->NewID() to prevent cache-key collision among dummy entry clients sharing the same cache. The specific collision we are preventing happens when the object address is reused for a new cache-key prefix while the old cache-key using that same object address in its prefix still exists in the cache. This could happen due to that, under LRU cache policy, there is a possible delay in releasing a cache entry after the cache client object owning that cache entry get deallocated. In this case, the object address related to the cache client object can get reused for other client object to generate a new cache-key prefix. This prefix generation can be made obsolete after Peter's unification of all the code generating cache key, mentioned in https://github.com/facebook/rocksdb/pull/8506#discussion_r667265255 Pull Request resolved: https://github.com/facebook/rocksdb/pull/8506 Test Plan: - Passing the added unit tests cache_reservation_manager_test.cc - Passing existing and adjusted write_buffer_manager_test.cc Reviewed By: ajkr Differential Revision: D29644135 Pulled By: hx235 fbshipit-source-id: 0fc93fbfe4a40bb41be85c314f8f2bafa8b741f7
2021-08-24 19:42:31 +00:00
new CacheReservationManager(cache, true /* delayed_decrease */));
}
#else
(void)cache;
#endif // ROCKSDB_LITE
}
Fix race in WriteBufferManager (#9009) Summary: EndWriteStall has a data race: `queue_.empty()` is checked outside of the mutex, so once we enter the critical section another thread may already have cleared the list, and accessing the `front()` is undefined behavior (and causes interesting crashes under high concurrency). This PR fixes the bug, and also rewrites the logic to make it easier to reason about it. It also fixes another subtle bug: if some writers are stalled and `SetBufferSize(0)` is called, which disables the WBM, the writer are not unblocked because of an early `enabled()` check in `EndWriteStall()`. It doesn't significantly change the locking behavior, as before writers won't lock unless entering a stall condition, and `FreeMem` almost always locks if stalling is allowed, but that is inevitable with the current design. Liveness is guaranteed by the fact that if some writes are blocked, eventually all writes will be blocked due to `stall_active_`, and eventually all memory is freed. While at it, do a couple of optimizations: - In `WBMStallInterface::Signal()` signal the CV only after releasing the lock. Signaling under the lock is a common pitfall, as it causes the woken-up thread to immediately go back to sleep because the mutex is still locked by the awaker. - Move all allocations and deallocations outside of the lock. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9009 Test Plan: ``` USE_CLANG=1 make -j64 all check ``` Reviewed By: akankshamahajan15 Differential Revision: D31550668 Pulled By: ot fbshipit-source-id: 5125387c3dc7ecaaa2b8bbc736e58c4156698580
2021-10-12 07:14:41 +00:00
WriteBufferManager::~WriteBufferManager() {
#ifndef NDEBUG
std::unique_lock<std::mutex> lock(mu_);
assert(queue_.empty());
#endif
}
Refactor WriteBufferManager::CacheRep into CacheReservationManager (#8506) Summary: Context: To help cap various memory usage by a single limit of the block cache capacity, we charge the memory usage through inserting/releasing dummy entries in the block cache. CacheReservationManager is such a class (non thread-safe) responsible for inserting/removing dummy entries to reserve cache space for memory used by the class user. - Refactored the inner private class CacheRep of WriteBufferManager into public CacheReservationManager class for reusability such as for https://github.com/facebook/rocksdb/pull/8428 - Encapsulated implementation details of cache key generation and dummy entries insertion/release in cache reservation as discussed in https://github.com/facebook/rocksdb/pull/8506#discussion_r666550838 - Consolidated increase/decrease cache reservation into one API - UpdateCacheReservation. - Adjusted the previous dummy entry release algorithm in decreasing cache reservation to be loop-releasing dummy entries to stay symmetric to dummy entry insertion algorithm - Made the previous dummy entry release algorithm in delayed decrease mode more aggressive for better decreasing cache reservation when memory used is less likely to increase back. Previously, the algorithms only release 1 dummy entries when new_mem_used < 3/4 * cache_allocated_size_ and cache_allocated_size_ - kSizeDummyEntry > new_mem_used. Now, the algorithms loop-releases as many dummy entries as possible when new_mem_used < 3/4 * cache_allocated_size_. - Updated WriteBufferManager's test cases to adapt to changes on the release algorithm mentioned above and left comment for some test cases for clarity - Replaced the previous cache key prefix generation (utilizing object address related to the cache client) with one that utilizes Cache->NewID() to prevent cache-key collision among dummy entry clients sharing the same cache. The specific collision we are preventing happens when the object address is reused for a new cache-key prefix while the old cache-key using that same object address in its prefix still exists in the cache. This could happen due to that, under LRU cache policy, there is a possible delay in releasing a cache entry after the cache client object owning that cache entry get deallocated. In this case, the object address related to the cache client object can get reused for other client object to generate a new cache-key prefix. This prefix generation can be made obsolete after Peter's unification of all the code generating cache key, mentioned in https://github.com/facebook/rocksdb/pull/8506#discussion_r667265255 Pull Request resolved: https://github.com/facebook/rocksdb/pull/8506 Test Plan: - Passing the added unit tests cache_reservation_manager_test.cc - Passing existing and adjusted write_buffer_manager_test.cc Reviewed By: ajkr Differential Revision: D29644135 Pulled By: hx235 fbshipit-source-id: 0fc93fbfe4a40bb41be85c314f8f2bafa8b741f7
2021-08-24 19:42:31 +00:00
std::size_t WriteBufferManager::dummy_entries_in_cache_usage() const {
if (cache_res_mgr_ != nullptr) {
return cache_res_mgr_->GetTotalReservedCacheSize();
Refactor WriteBufferManager::CacheRep into CacheReservationManager (#8506) Summary: Context: To help cap various memory usage by a single limit of the block cache capacity, we charge the memory usage through inserting/releasing dummy entries in the block cache. CacheReservationManager is such a class (non thread-safe) responsible for inserting/removing dummy entries to reserve cache space for memory used by the class user. - Refactored the inner private class CacheRep of WriteBufferManager into public CacheReservationManager class for reusability such as for https://github.com/facebook/rocksdb/pull/8428 - Encapsulated implementation details of cache key generation and dummy entries insertion/release in cache reservation as discussed in https://github.com/facebook/rocksdb/pull/8506#discussion_r666550838 - Consolidated increase/decrease cache reservation into one API - UpdateCacheReservation. - Adjusted the previous dummy entry release algorithm in decreasing cache reservation to be loop-releasing dummy entries to stay symmetric to dummy entry insertion algorithm - Made the previous dummy entry release algorithm in delayed decrease mode more aggressive for better decreasing cache reservation when memory used is less likely to increase back. Previously, the algorithms only release 1 dummy entries when new_mem_used < 3/4 * cache_allocated_size_ and cache_allocated_size_ - kSizeDummyEntry > new_mem_used. Now, the algorithms loop-releases as many dummy entries as possible when new_mem_used < 3/4 * cache_allocated_size_. - Updated WriteBufferManager's test cases to adapt to changes on the release algorithm mentioned above and left comment for some test cases for clarity - Replaced the previous cache key prefix generation (utilizing object address related to the cache client) with one that utilizes Cache->NewID() to prevent cache-key collision among dummy entry clients sharing the same cache. The specific collision we are preventing happens when the object address is reused for a new cache-key prefix while the old cache-key using that same object address in its prefix still exists in the cache. This could happen due to that, under LRU cache policy, there is a possible delay in releasing a cache entry after the cache client object owning that cache entry get deallocated. In this case, the object address related to the cache client object can get reused for other client object to generate a new cache-key prefix. This prefix generation can be made obsolete after Peter's unification of all the code generating cache key, mentioned in https://github.com/facebook/rocksdb/pull/8506#discussion_r667265255 Pull Request resolved: https://github.com/facebook/rocksdb/pull/8506 Test Plan: - Passing the added unit tests cache_reservation_manager_test.cc - Passing existing and adjusted write_buffer_manager_test.cc Reviewed By: ajkr Differential Revision: D29644135 Pulled By: hx235 fbshipit-source-id: 0fc93fbfe4a40bb41be85c314f8f2bafa8b741f7
2021-08-24 19:42:31 +00:00
} else {
return 0;
}
}
Stall writes in WriteBufferManager when memory_usage exceeds buffer_size (#7898) Summary: When WriteBufferManager is shared across DBs and column families to maintain memory usage under a limit, OOMs have been observed when flush cannot finish but writes continuously insert to memtables. In order to avoid OOMs, when memory usage goes beyond buffer_limit_ and DBs tries to write, this change will stall incoming writers until flush is completed and memory_usage drops. Design: Stall condition: When total memory usage exceeds WriteBufferManager::buffer_size_ (memory_usage() >= buffer_size_) WriterBufferManager::ShouldStall() returns true. DBImpl first block incoming/future writers by calling write_thread_.BeginWriteStall() (which adds dummy stall object to the writer's queue). Then DB is blocked on a state State::Blocked (current write doesn't go through). WBStallInterface object maintained by every DB instance is added to the queue of WriteBufferManager. If multiple DBs tries to write during this stall, they will also be blocked when check WriteBufferManager::ShouldStall() returns true. End Stall condition: When flush is finished and memory usage goes down, stall will end only if memory waiting to be flushed is less than buffer_size/2. This lower limit will give time for flush to complete and avoid continous stalling if memory usage remains close to buffer_size. WriterBufferManager::EndWriteStall() is called, which removes all instances from its queue and signal them to continue. Their state is changed to State::Running and they are unblocked. DBImpl then signal all incoming writers of that DB to continue by calling write_thread_.EndWriteStall() (which removes dummy stall object from the queue). DB instance creates WBMStallInterface which is an interface to block and signal DBs during stall. When DB needs to be blocked or signalled by WriteBufferManager, state_for_wbm_ state is changed accordingly (RUNNING or BLOCKED). Pull Request resolved: https://github.com/facebook/rocksdb/pull/7898 Test Plan: Added a new test db/db_write_buffer_manager_test.cc Reviewed By: anand1976 Differential Revision: D26093227 Pulled By: akankshamahajan15 fbshipit-source-id: 2bbd982a3fb7033f6de6153aa92a221249861aae
2021-04-21 20:53:05 +00:00
void WriteBufferManager::ReserveMem(size_t mem) {
if (cache_res_mgr_ != nullptr) {
Stall writes in WriteBufferManager when memory_usage exceeds buffer_size (#7898) Summary: When WriteBufferManager is shared across DBs and column families to maintain memory usage under a limit, OOMs have been observed when flush cannot finish but writes continuously insert to memtables. In order to avoid OOMs, when memory usage goes beyond buffer_limit_ and DBs tries to write, this change will stall incoming writers until flush is completed and memory_usage drops. Design: Stall condition: When total memory usage exceeds WriteBufferManager::buffer_size_ (memory_usage() >= buffer_size_) WriterBufferManager::ShouldStall() returns true. DBImpl first block incoming/future writers by calling write_thread_.BeginWriteStall() (which adds dummy stall object to the writer's queue). Then DB is blocked on a state State::Blocked (current write doesn't go through). WBStallInterface object maintained by every DB instance is added to the queue of WriteBufferManager. If multiple DBs tries to write during this stall, they will also be blocked when check WriteBufferManager::ShouldStall() returns true. End Stall condition: When flush is finished and memory usage goes down, stall will end only if memory waiting to be flushed is less than buffer_size/2. This lower limit will give time for flush to complete and avoid continous stalling if memory usage remains close to buffer_size. WriterBufferManager::EndWriteStall() is called, which removes all instances from its queue and signal them to continue. Their state is changed to State::Running and they are unblocked. DBImpl then signal all incoming writers of that DB to continue by calling write_thread_.EndWriteStall() (which removes dummy stall object from the queue). DB instance creates WBMStallInterface which is an interface to block and signal DBs during stall. When DB needs to be blocked or signalled by WriteBufferManager, state_for_wbm_ state is changed accordingly (RUNNING or BLOCKED). Pull Request resolved: https://github.com/facebook/rocksdb/pull/7898 Test Plan: Added a new test db/db_write_buffer_manager_test.cc Reviewed By: anand1976 Differential Revision: D26093227 Pulled By: akankshamahajan15 fbshipit-source-id: 2bbd982a3fb7033f6de6153aa92a221249861aae
2021-04-21 20:53:05 +00:00
ReserveMemWithCache(mem);
} else if (enabled()) {
memory_used_.fetch_add(mem, std::memory_order_relaxed);
}
if (enabled()) {
memory_active_.fetch_add(mem, std::memory_order_relaxed);
}
}
// Should only be called from write thread
void WriteBufferManager::ReserveMemWithCache(size_t mem) {
#ifndef ROCKSDB_LITE
assert(cache_res_mgr_ != nullptr);
// Use a mutex to protect various data structures. Can be optimized to a
// lock-free solution if it ends up with a performance bottleneck.
std::lock_guard<std::mutex> lock(cache_res_mgr_mu_);
size_t new_mem_used = memory_used_.load(std::memory_order_relaxed) + mem;
memory_used_.store(new_mem_used, std::memory_order_relaxed);
Refactor WriteBufferManager::CacheRep into CacheReservationManager (#8506) Summary: Context: To help cap various memory usage by a single limit of the block cache capacity, we charge the memory usage through inserting/releasing dummy entries in the block cache. CacheReservationManager is such a class (non thread-safe) responsible for inserting/removing dummy entries to reserve cache space for memory used by the class user. - Refactored the inner private class CacheRep of WriteBufferManager into public CacheReservationManager class for reusability such as for https://github.com/facebook/rocksdb/pull/8428 - Encapsulated implementation details of cache key generation and dummy entries insertion/release in cache reservation as discussed in https://github.com/facebook/rocksdb/pull/8506#discussion_r666550838 - Consolidated increase/decrease cache reservation into one API - UpdateCacheReservation. - Adjusted the previous dummy entry release algorithm in decreasing cache reservation to be loop-releasing dummy entries to stay symmetric to dummy entry insertion algorithm - Made the previous dummy entry release algorithm in delayed decrease mode more aggressive for better decreasing cache reservation when memory used is less likely to increase back. Previously, the algorithms only release 1 dummy entries when new_mem_used < 3/4 * cache_allocated_size_ and cache_allocated_size_ - kSizeDummyEntry > new_mem_used. Now, the algorithms loop-releases as many dummy entries as possible when new_mem_used < 3/4 * cache_allocated_size_. - Updated WriteBufferManager's test cases to adapt to changes on the release algorithm mentioned above and left comment for some test cases for clarity - Replaced the previous cache key prefix generation (utilizing object address related to the cache client) with one that utilizes Cache->NewID() to prevent cache-key collision among dummy entry clients sharing the same cache. The specific collision we are preventing happens when the object address is reused for a new cache-key prefix while the old cache-key using that same object address in its prefix still exists in the cache. This could happen due to that, under LRU cache policy, there is a possible delay in releasing a cache entry after the cache client object owning that cache entry get deallocated. In this case, the object address related to the cache client object can get reused for other client object to generate a new cache-key prefix. This prefix generation can be made obsolete after Peter's unification of all the code generating cache key, mentioned in https://github.com/facebook/rocksdb/pull/8506#discussion_r667265255 Pull Request resolved: https://github.com/facebook/rocksdb/pull/8506 Test Plan: - Passing the added unit tests cache_reservation_manager_test.cc - Passing existing and adjusted write_buffer_manager_test.cc Reviewed By: ajkr Differential Revision: D29644135 Pulled By: hx235 fbshipit-source-id: 0fc93fbfe4a40bb41be85c314f8f2bafa8b741f7
2021-08-24 19:42:31 +00:00
Status s =
cache_res_mgr_->UpdateCacheReservation<CacheEntryRole::kWriteBuffer>(
Refactor WriteBufferManager::CacheRep into CacheReservationManager (#8506) Summary: Context: To help cap various memory usage by a single limit of the block cache capacity, we charge the memory usage through inserting/releasing dummy entries in the block cache. CacheReservationManager is such a class (non thread-safe) responsible for inserting/removing dummy entries to reserve cache space for memory used by the class user. - Refactored the inner private class CacheRep of WriteBufferManager into public CacheReservationManager class for reusability such as for https://github.com/facebook/rocksdb/pull/8428 - Encapsulated implementation details of cache key generation and dummy entries insertion/release in cache reservation as discussed in https://github.com/facebook/rocksdb/pull/8506#discussion_r666550838 - Consolidated increase/decrease cache reservation into one API - UpdateCacheReservation. - Adjusted the previous dummy entry release algorithm in decreasing cache reservation to be loop-releasing dummy entries to stay symmetric to dummy entry insertion algorithm - Made the previous dummy entry release algorithm in delayed decrease mode more aggressive for better decreasing cache reservation when memory used is less likely to increase back. Previously, the algorithms only release 1 dummy entries when new_mem_used < 3/4 * cache_allocated_size_ and cache_allocated_size_ - kSizeDummyEntry > new_mem_used. Now, the algorithms loop-releases as many dummy entries as possible when new_mem_used < 3/4 * cache_allocated_size_. - Updated WriteBufferManager's test cases to adapt to changes on the release algorithm mentioned above and left comment for some test cases for clarity - Replaced the previous cache key prefix generation (utilizing object address related to the cache client) with one that utilizes Cache->NewID() to prevent cache-key collision among dummy entry clients sharing the same cache. The specific collision we are preventing happens when the object address is reused for a new cache-key prefix while the old cache-key using that same object address in its prefix still exists in the cache. This could happen due to that, under LRU cache policy, there is a possible delay in releasing a cache entry after the cache client object owning that cache entry get deallocated. In this case, the object address related to the cache client object can get reused for other client object to generate a new cache-key prefix. This prefix generation can be made obsolete after Peter's unification of all the code generating cache key, mentioned in https://github.com/facebook/rocksdb/pull/8506#discussion_r667265255 Pull Request resolved: https://github.com/facebook/rocksdb/pull/8506 Test Plan: - Passing the added unit tests cache_reservation_manager_test.cc - Passing existing and adjusted write_buffer_manager_test.cc Reviewed By: ajkr Differential Revision: D29644135 Pulled By: hx235 fbshipit-source-id: 0fc93fbfe4a40bb41be85c314f8f2bafa8b741f7
2021-08-24 19:42:31 +00:00
new_mem_used);
// We absorb the error since WriteBufferManager is not able to handle
// this failure properly. Ideallly we should prevent this allocation
// from happening if this cache reservation fails.
// [TODO] We'll need to improve it in the future and figure out what to do on
// error
s.PermitUncheckedError();
#else
(void)mem;
#endif // ROCKSDB_LITE
}
Stall writes in WriteBufferManager when memory_usage exceeds buffer_size (#7898) Summary: When WriteBufferManager is shared across DBs and column families to maintain memory usage under a limit, OOMs have been observed when flush cannot finish but writes continuously insert to memtables. In order to avoid OOMs, when memory usage goes beyond buffer_limit_ and DBs tries to write, this change will stall incoming writers until flush is completed and memory_usage drops. Design: Stall condition: When total memory usage exceeds WriteBufferManager::buffer_size_ (memory_usage() >= buffer_size_) WriterBufferManager::ShouldStall() returns true. DBImpl first block incoming/future writers by calling write_thread_.BeginWriteStall() (which adds dummy stall object to the writer's queue). Then DB is blocked on a state State::Blocked (current write doesn't go through). WBStallInterface object maintained by every DB instance is added to the queue of WriteBufferManager. If multiple DBs tries to write during this stall, they will also be blocked when check WriteBufferManager::ShouldStall() returns true. End Stall condition: When flush is finished and memory usage goes down, stall will end only if memory waiting to be flushed is less than buffer_size/2. This lower limit will give time for flush to complete and avoid continous stalling if memory usage remains close to buffer_size. WriterBufferManager::EndWriteStall() is called, which removes all instances from its queue and signal them to continue. Their state is changed to State::Running and they are unblocked. DBImpl then signal all incoming writers of that DB to continue by calling write_thread_.EndWriteStall() (which removes dummy stall object from the queue). DB instance creates WBMStallInterface which is an interface to block and signal DBs during stall. When DB needs to be blocked or signalled by WriteBufferManager, state_for_wbm_ state is changed accordingly (RUNNING or BLOCKED). Pull Request resolved: https://github.com/facebook/rocksdb/pull/7898 Test Plan: Added a new test db/db_write_buffer_manager_test.cc Reviewed By: anand1976 Differential Revision: D26093227 Pulled By: akankshamahajan15 fbshipit-source-id: 2bbd982a3fb7033f6de6153aa92a221249861aae
2021-04-21 20:53:05 +00:00
void WriteBufferManager::ScheduleFreeMem(size_t mem) {
if (enabled()) {
memory_active_.fetch_sub(mem, std::memory_order_relaxed);
}
}
void WriteBufferManager::FreeMem(size_t mem) {
if (cache_res_mgr_ != nullptr) {
Stall writes in WriteBufferManager when memory_usage exceeds buffer_size (#7898) Summary: When WriteBufferManager is shared across DBs and column families to maintain memory usage under a limit, OOMs have been observed when flush cannot finish but writes continuously insert to memtables. In order to avoid OOMs, when memory usage goes beyond buffer_limit_ and DBs tries to write, this change will stall incoming writers until flush is completed and memory_usage drops. Design: Stall condition: When total memory usage exceeds WriteBufferManager::buffer_size_ (memory_usage() >= buffer_size_) WriterBufferManager::ShouldStall() returns true. DBImpl first block incoming/future writers by calling write_thread_.BeginWriteStall() (which adds dummy stall object to the writer's queue). Then DB is blocked on a state State::Blocked (current write doesn't go through). WBStallInterface object maintained by every DB instance is added to the queue of WriteBufferManager. If multiple DBs tries to write during this stall, they will also be blocked when check WriteBufferManager::ShouldStall() returns true. End Stall condition: When flush is finished and memory usage goes down, stall will end only if memory waiting to be flushed is less than buffer_size/2. This lower limit will give time for flush to complete and avoid continous stalling if memory usage remains close to buffer_size. WriterBufferManager::EndWriteStall() is called, which removes all instances from its queue and signal them to continue. Their state is changed to State::Running and they are unblocked. DBImpl then signal all incoming writers of that DB to continue by calling write_thread_.EndWriteStall() (which removes dummy stall object from the queue). DB instance creates WBMStallInterface which is an interface to block and signal DBs during stall. When DB needs to be blocked or signalled by WriteBufferManager, state_for_wbm_ state is changed accordingly (RUNNING or BLOCKED). Pull Request resolved: https://github.com/facebook/rocksdb/pull/7898 Test Plan: Added a new test db/db_write_buffer_manager_test.cc Reviewed By: anand1976 Differential Revision: D26093227 Pulled By: akankshamahajan15 fbshipit-source-id: 2bbd982a3fb7033f6de6153aa92a221249861aae
2021-04-21 20:53:05 +00:00
FreeMemWithCache(mem);
} else if (enabled()) {
memory_used_.fetch_sub(mem, std::memory_order_relaxed);
}
// Check if stall is active and can be ended.
Fix race in WriteBufferManager (#9009) Summary: EndWriteStall has a data race: `queue_.empty()` is checked outside of the mutex, so once we enter the critical section another thread may already have cleared the list, and accessing the `front()` is undefined behavior (and causes interesting crashes under high concurrency). This PR fixes the bug, and also rewrites the logic to make it easier to reason about it. It also fixes another subtle bug: if some writers are stalled and `SetBufferSize(0)` is called, which disables the WBM, the writer are not unblocked because of an early `enabled()` check in `EndWriteStall()`. It doesn't significantly change the locking behavior, as before writers won't lock unless entering a stall condition, and `FreeMem` almost always locks if stalling is allowed, but that is inevitable with the current design. Liveness is guaranteed by the fact that if some writes are blocked, eventually all writes will be blocked due to `stall_active_`, and eventually all memory is freed. While at it, do a couple of optimizations: - In `WBMStallInterface::Signal()` signal the CV only after releasing the lock. Signaling under the lock is a common pitfall, as it causes the woken-up thread to immediately go back to sleep because the mutex is still locked by the awaker. - Move all allocations and deallocations outside of the lock. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9009 Test Plan: ``` USE_CLANG=1 make -j64 all check ``` Reviewed By: akankshamahajan15 Differential Revision: D31550668 Pulled By: ot fbshipit-source-id: 5125387c3dc7ecaaa2b8bbc736e58c4156698580
2021-10-12 07:14:41 +00:00
MaybeEndWriteStall();
Stall writes in WriteBufferManager when memory_usage exceeds buffer_size (#7898) Summary: When WriteBufferManager is shared across DBs and column families to maintain memory usage under a limit, OOMs have been observed when flush cannot finish but writes continuously insert to memtables. In order to avoid OOMs, when memory usage goes beyond buffer_limit_ and DBs tries to write, this change will stall incoming writers until flush is completed and memory_usage drops. Design: Stall condition: When total memory usage exceeds WriteBufferManager::buffer_size_ (memory_usage() >= buffer_size_) WriterBufferManager::ShouldStall() returns true. DBImpl first block incoming/future writers by calling write_thread_.BeginWriteStall() (which adds dummy stall object to the writer's queue). Then DB is blocked on a state State::Blocked (current write doesn't go through). WBStallInterface object maintained by every DB instance is added to the queue of WriteBufferManager. If multiple DBs tries to write during this stall, they will also be blocked when check WriteBufferManager::ShouldStall() returns true. End Stall condition: When flush is finished and memory usage goes down, stall will end only if memory waiting to be flushed is less than buffer_size/2. This lower limit will give time for flush to complete and avoid continous stalling if memory usage remains close to buffer_size. WriterBufferManager::EndWriteStall() is called, which removes all instances from its queue and signal them to continue. Their state is changed to State::Running and they are unblocked. DBImpl then signal all incoming writers of that DB to continue by calling write_thread_.EndWriteStall() (which removes dummy stall object from the queue). DB instance creates WBMStallInterface which is an interface to block and signal DBs during stall. When DB needs to be blocked or signalled by WriteBufferManager, state_for_wbm_ state is changed accordingly (RUNNING or BLOCKED). Pull Request resolved: https://github.com/facebook/rocksdb/pull/7898 Test Plan: Added a new test db/db_write_buffer_manager_test.cc Reviewed By: anand1976 Differential Revision: D26093227 Pulled By: akankshamahajan15 fbshipit-source-id: 2bbd982a3fb7033f6de6153aa92a221249861aae
2021-04-21 20:53:05 +00:00
}
void WriteBufferManager::FreeMemWithCache(size_t mem) {
#ifndef ROCKSDB_LITE
assert(cache_res_mgr_ != nullptr);
// Use a mutex to protect various data structures. Can be optimized to a
// lock-free solution if it ends up with a performance bottleneck.
std::lock_guard<std::mutex> lock(cache_res_mgr_mu_);
size_t new_mem_used = memory_used_.load(std::memory_order_relaxed) - mem;
memory_used_.store(new_mem_used, std::memory_order_relaxed);
Refactor WriteBufferManager::CacheRep into CacheReservationManager (#8506) Summary: Context: To help cap various memory usage by a single limit of the block cache capacity, we charge the memory usage through inserting/releasing dummy entries in the block cache. CacheReservationManager is such a class (non thread-safe) responsible for inserting/removing dummy entries to reserve cache space for memory used by the class user. - Refactored the inner private class CacheRep of WriteBufferManager into public CacheReservationManager class for reusability such as for https://github.com/facebook/rocksdb/pull/8428 - Encapsulated implementation details of cache key generation and dummy entries insertion/release in cache reservation as discussed in https://github.com/facebook/rocksdb/pull/8506#discussion_r666550838 - Consolidated increase/decrease cache reservation into one API - UpdateCacheReservation. - Adjusted the previous dummy entry release algorithm in decreasing cache reservation to be loop-releasing dummy entries to stay symmetric to dummy entry insertion algorithm - Made the previous dummy entry release algorithm in delayed decrease mode more aggressive for better decreasing cache reservation when memory used is less likely to increase back. Previously, the algorithms only release 1 dummy entries when new_mem_used < 3/4 * cache_allocated_size_ and cache_allocated_size_ - kSizeDummyEntry > new_mem_used. Now, the algorithms loop-releases as many dummy entries as possible when new_mem_used < 3/4 * cache_allocated_size_. - Updated WriteBufferManager's test cases to adapt to changes on the release algorithm mentioned above and left comment for some test cases for clarity - Replaced the previous cache key prefix generation (utilizing object address related to the cache client) with one that utilizes Cache->NewID() to prevent cache-key collision among dummy entry clients sharing the same cache. The specific collision we are preventing happens when the object address is reused for a new cache-key prefix while the old cache-key using that same object address in its prefix still exists in the cache. This could happen due to that, under LRU cache policy, there is a possible delay in releasing a cache entry after the cache client object owning that cache entry get deallocated. In this case, the object address related to the cache client object can get reused for other client object to generate a new cache-key prefix. This prefix generation can be made obsolete after Peter's unification of all the code generating cache key, mentioned in https://github.com/facebook/rocksdb/pull/8506#discussion_r667265255 Pull Request resolved: https://github.com/facebook/rocksdb/pull/8506 Test Plan: - Passing the added unit tests cache_reservation_manager_test.cc - Passing existing and adjusted write_buffer_manager_test.cc Reviewed By: ajkr Differential Revision: D29644135 Pulled By: hx235 fbshipit-source-id: 0fc93fbfe4a40bb41be85c314f8f2bafa8b741f7
2021-08-24 19:42:31 +00:00
Status s =
cache_res_mgr_->UpdateCacheReservation<CacheEntryRole::kWriteBuffer>(
Refactor WriteBufferManager::CacheRep into CacheReservationManager (#8506) Summary: Context: To help cap various memory usage by a single limit of the block cache capacity, we charge the memory usage through inserting/releasing dummy entries in the block cache. CacheReservationManager is such a class (non thread-safe) responsible for inserting/removing dummy entries to reserve cache space for memory used by the class user. - Refactored the inner private class CacheRep of WriteBufferManager into public CacheReservationManager class for reusability such as for https://github.com/facebook/rocksdb/pull/8428 - Encapsulated implementation details of cache key generation and dummy entries insertion/release in cache reservation as discussed in https://github.com/facebook/rocksdb/pull/8506#discussion_r666550838 - Consolidated increase/decrease cache reservation into one API - UpdateCacheReservation. - Adjusted the previous dummy entry release algorithm in decreasing cache reservation to be loop-releasing dummy entries to stay symmetric to dummy entry insertion algorithm - Made the previous dummy entry release algorithm in delayed decrease mode more aggressive for better decreasing cache reservation when memory used is less likely to increase back. Previously, the algorithms only release 1 dummy entries when new_mem_used < 3/4 * cache_allocated_size_ and cache_allocated_size_ - kSizeDummyEntry > new_mem_used. Now, the algorithms loop-releases as many dummy entries as possible when new_mem_used < 3/4 * cache_allocated_size_. - Updated WriteBufferManager's test cases to adapt to changes on the release algorithm mentioned above and left comment for some test cases for clarity - Replaced the previous cache key prefix generation (utilizing object address related to the cache client) with one that utilizes Cache->NewID() to prevent cache-key collision among dummy entry clients sharing the same cache. The specific collision we are preventing happens when the object address is reused for a new cache-key prefix while the old cache-key using that same object address in its prefix still exists in the cache. This could happen due to that, under LRU cache policy, there is a possible delay in releasing a cache entry after the cache client object owning that cache entry get deallocated. In this case, the object address related to the cache client object can get reused for other client object to generate a new cache-key prefix. This prefix generation can be made obsolete after Peter's unification of all the code generating cache key, mentioned in https://github.com/facebook/rocksdb/pull/8506#discussion_r667265255 Pull Request resolved: https://github.com/facebook/rocksdb/pull/8506 Test Plan: - Passing the added unit tests cache_reservation_manager_test.cc - Passing existing and adjusted write_buffer_manager_test.cc Reviewed By: ajkr Differential Revision: D29644135 Pulled By: hx235 fbshipit-source-id: 0fc93fbfe4a40bb41be85c314f8f2bafa8b741f7
2021-08-24 19:42:31 +00:00
new_mem_used);
// We absorb the error since WriteBufferManager is not able to handle
// this failure properly.
// [TODO] We'll need to improve it in the future and figure out what to do on
// error
s.PermitUncheckedError();
#else
(void)mem;
#endif // ROCKSDB_LITE
}
Stall writes in WriteBufferManager when memory_usage exceeds buffer_size (#7898) Summary: When WriteBufferManager is shared across DBs and column families to maintain memory usage under a limit, OOMs have been observed when flush cannot finish but writes continuously insert to memtables. In order to avoid OOMs, when memory usage goes beyond buffer_limit_ and DBs tries to write, this change will stall incoming writers until flush is completed and memory_usage drops. Design: Stall condition: When total memory usage exceeds WriteBufferManager::buffer_size_ (memory_usage() >= buffer_size_) WriterBufferManager::ShouldStall() returns true. DBImpl first block incoming/future writers by calling write_thread_.BeginWriteStall() (which adds dummy stall object to the writer's queue). Then DB is blocked on a state State::Blocked (current write doesn't go through). WBStallInterface object maintained by every DB instance is added to the queue of WriteBufferManager. If multiple DBs tries to write during this stall, they will also be blocked when check WriteBufferManager::ShouldStall() returns true. End Stall condition: When flush is finished and memory usage goes down, stall will end only if memory waiting to be flushed is less than buffer_size/2. This lower limit will give time for flush to complete and avoid continous stalling if memory usage remains close to buffer_size. WriterBufferManager::EndWriteStall() is called, which removes all instances from its queue and signal them to continue. Their state is changed to State::Running and they are unblocked. DBImpl then signal all incoming writers of that DB to continue by calling write_thread_.EndWriteStall() (which removes dummy stall object from the queue). DB instance creates WBMStallInterface which is an interface to block and signal DBs during stall. When DB needs to be blocked or signalled by WriteBufferManager, state_for_wbm_ state is changed accordingly (RUNNING or BLOCKED). Pull Request resolved: https://github.com/facebook/rocksdb/pull/7898 Test Plan: Added a new test db/db_write_buffer_manager_test.cc Reviewed By: anand1976 Differential Revision: D26093227 Pulled By: akankshamahajan15 fbshipit-source-id: 2bbd982a3fb7033f6de6153aa92a221249861aae
2021-04-21 20:53:05 +00:00
void WriteBufferManager::BeginWriteStall(StallInterface* wbm_stall) {
assert(wbm_stall != nullptr);
Fix race in WriteBufferManager (#9009) Summary: EndWriteStall has a data race: `queue_.empty()` is checked outside of the mutex, so once we enter the critical section another thread may already have cleared the list, and accessing the `front()` is undefined behavior (and causes interesting crashes under high concurrency). This PR fixes the bug, and also rewrites the logic to make it easier to reason about it. It also fixes another subtle bug: if some writers are stalled and `SetBufferSize(0)` is called, which disables the WBM, the writer are not unblocked because of an early `enabled()` check in `EndWriteStall()`. It doesn't significantly change the locking behavior, as before writers won't lock unless entering a stall condition, and `FreeMem` almost always locks if stalling is allowed, but that is inevitable with the current design. Liveness is guaranteed by the fact that if some writes are blocked, eventually all writes will be blocked due to `stall_active_`, and eventually all memory is freed. While at it, do a couple of optimizations: - In `WBMStallInterface::Signal()` signal the CV only after releasing the lock. Signaling under the lock is a common pitfall, as it causes the woken-up thread to immediately go back to sleep because the mutex is still locked by the awaker. - Move all allocations and deallocations outside of the lock. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9009 Test Plan: ``` USE_CLANG=1 make -j64 all check ``` Reviewed By: akankshamahajan15 Differential Revision: D31550668 Pulled By: ot fbshipit-source-id: 5125387c3dc7ecaaa2b8bbc736e58c4156698580
2021-10-12 07:14:41 +00:00
assert(allow_stall_);
// Allocate outside of the lock.
std::list<StallInterface*> new_node = {wbm_stall};
{
Stall writes in WriteBufferManager when memory_usage exceeds buffer_size (#7898) Summary: When WriteBufferManager is shared across DBs and column families to maintain memory usage under a limit, OOMs have been observed when flush cannot finish but writes continuously insert to memtables. In order to avoid OOMs, when memory usage goes beyond buffer_limit_ and DBs tries to write, this change will stall incoming writers until flush is completed and memory_usage drops. Design: Stall condition: When total memory usage exceeds WriteBufferManager::buffer_size_ (memory_usage() >= buffer_size_) WriterBufferManager::ShouldStall() returns true. DBImpl first block incoming/future writers by calling write_thread_.BeginWriteStall() (which adds dummy stall object to the writer's queue). Then DB is blocked on a state State::Blocked (current write doesn't go through). WBStallInterface object maintained by every DB instance is added to the queue of WriteBufferManager. If multiple DBs tries to write during this stall, they will also be blocked when check WriteBufferManager::ShouldStall() returns true. End Stall condition: When flush is finished and memory usage goes down, stall will end only if memory waiting to be flushed is less than buffer_size/2. This lower limit will give time for flush to complete and avoid continous stalling if memory usage remains close to buffer_size. WriterBufferManager::EndWriteStall() is called, which removes all instances from its queue and signal them to continue. Their state is changed to State::Running and they are unblocked. DBImpl then signal all incoming writers of that DB to continue by calling write_thread_.EndWriteStall() (which removes dummy stall object from the queue). DB instance creates WBMStallInterface which is an interface to block and signal DBs during stall. When DB needs to be blocked or signalled by WriteBufferManager, state_for_wbm_ state is changed accordingly (RUNNING or BLOCKED). Pull Request resolved: https://github.com/facebook/rocksdb/pull/7898 Test Plan: Added a new test db/db_write_buffer_manager_test.cc Reviewed By: anand1976 Differential Revision: D26093227 Pulled By: akankshamahajan15 fbshipit-source-id: 2bbd982a3fb7033f6de6153aa92a221249861aae
2021-04-21 20:53:05 +00:00
std::unique_lock<std::mutex> lock(mu_);
Fix race in WriteBufferManager (#9009) Summary: EndWriteStall has a data race: `queue_.empty()` is checked outside of the mutex, so once we enter the critical section another thread may already have cleared the list, and accessing the `front()` is undefined behavior (and causes interesting crashes under high concurrency). This PR fixes the bug, and also rewrites the logic to make it easier to reason about it. It also fixes another subtle bug: if some writers are stalled and `SetBufferSize(0)` is called, which disables the WBM, the writer are not unblocked because of an early `enabled()` check in `EndWriteStall()`. It doesn't significantly change the locking behavior, as before writers won't lock unless entering a stall condition, and `FreeMem` almost always locks if stalling is allowed, but that is inevitable with the current design. Liveness is guaranteed by the fact that if some writes are blocked, eventually all writes will be blocked due to `stall_active_`, and eventually all memory is freed. While at it, do a couple of optimizations: - In `WBMStallInterface::Signal()` signal the CV only after releasing the lock. Signaling under the lock is a common pitfall, as it causes the woken-up thread to immediately go back to sleep because the mutex is still locked by the awaker. - Move all allocations and deallocations outside of the lock. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9009 Test Plan: ``` USE_CLANG=1 make -j64 all check ``` Reviewed By: akankshamahajan15 Differential Revision: D31550668 Pulled By: ot fbshipit-source-id: 5125387c3dc7ecaaa2b8bbc736e58c4156698580
2021-10-12 07:14:41 +00:00
// Verify if the stall conditions are stil active.
if (ShouldStall()) {
stall_active_.store(true, std::memory_order_relaxed);
queue_.splice(queue_.end(), std::move(new_node));
}
Stall writes in WriteBufferManager when memory_usage exceeds buffer_size (#7898) Summary: When WriteBufferManager is shared across DBs and column families to maintain memory usage under a limit, OOMs have been observed when flush cannot finish but writes continuously insert to memtables. In order to avoid OOMs, when memory usage goes beyond buffer_limit_ and DBs tries to write, this change will stall incoming writers until flush is completed and memory_usage drops. Design: Stall condition: When total memory usage exceeds WriteBufferManager::buffer_size_ (memory_usage() >= buffer_size_) WriterBufferManager::ShouldStall() returns true. DBImpl first block incoming/future writers by calling write_thread_.BeginWriteStall() (which adds dummy stall object to the writer's queue). Then DB is blocked on a state State::Blocked (current write doesn't go through). WBStallInterface object maintained by every DB instance is added to the queue of WriteBufferManager. If multiple DBs tries to write during this stall, they will also be blocked when check WriteBufferManager::ShouldStall() returns true. End Stall condition: When flush is finished and memory usage goes down, stall will end only if memory waiting to be flushed is less than buffer_size/2. This lower limit will give time for flush to complete and avoid continous stalling if memory usage remains close to buffer_size. WriterBufferManager::EndWriteStall() is called, which removes all instances from its queue and signal them to continue. Their state is changed to State::Running and they are unblocked. DBImpl then signal all incoming writers of that DB to continue by calling write_thread_.EndWriteStall() (which removes dummy stall object from the queue). DB instance creates WBMStallInterface which is an interface to block and signal DBs during stall. When DB needs to be blocked or signalled by WriteBufferManager, state_for_wbm_ state is changed accordingly (RUNNING or BLOCKED). Pull Request resolved: https://github.com/facebook/rocksdb/pull/7898 Test Plan: Added a new test db/db_write_buffer_manager_test.cc Reviewed By: anand1976 Differential Revision: D26093227 Pulled By: akankshamahajan15 fbshipit-source-id: 2bbd982a3fb7033f6de6153aa92a221249861aae
2021-04-21 20:53:05 +00:00
}
Fix race in WriteBufferManager (#9009) Summary: EndWriteStall has a data race: `queue_.empty()` is checked outside of the mutex, so once we enter the critical section another thread may already have cleared the list, and accessing the `front()` is undefined behavior (and causes interesting crashes under high concurrency). This PR fixes the bug, and also rewrites the logic to make it easier to reason about it. It also fixes another subtle bug: if some writers are stalled and `SetBufferSize(0)` is called, which disables the WBM, the writer are not unblocked because of an early `enabled()` check in `EndWriteStall()`. It doesn't significantly change the locking behavior, as before writers won't lock unless entering a stall condition, and `FreeMem` almost always locks if stalling is allowed, but that is inevitable with the current design. Liveness is guaranteed by the fact that if some writes are blocked, eventually all writes will be blocked due to `stall_active_`, and eventually all memory is freed. While at it, do a couple of optimizations: - In `WBMStallInterface::Signal()` signal the CV only after releasing the lock. Signaling under the lock is a common pitfall, as it causes the woken-up thread to immediately go back to sleep because the mutex is still locked by the awaker. - Move all allocations and deallocations outside of the lock. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9009 Test Plan: ``` USE_CLANG=1 make -j64 all check ``` Reviewed By: akankshamahajan15 Differential Revision: D31550668 Pulled By: ot fbshipit-source-id: 5125387c3dc7ecaaa2b8bbc736e58c4156698580
2021-10-12 07:14:41 +00:00
// If the node was not consumed, the stall has ended already and we can signal
// the caller.
if (!new_node.empty()) {
new_node.front()->Signal();
Stall writes in WriteBufferManager when memory_usage exceeds buffer_size (#7898) Summary: When WriteBufferManager is shared across DBs and column families to maintain memory usage under a limit, OOMs have been observed when flush cannot finish but writes continuously insert to memtables. In order to avoid OOMs, when memory usage goes beyond buffer_limit_ and DBs tries to write, this change will stall incoming writers until flush is completed and memory_usage drops. Design: Stall condition: When total memory usage exceeds WriteBufferManager::buffer_size_ (memory_usage() >= buffer_size_) WriterBufferManager::ShouldStall() returns true. DBImpl first block incoming/future writers by calling write_thread_.BeginWriteStall() (which adds dummy stall object to the writer's queue). Then DB is blocked on a state State::Blocked (current write doesn't go through). WBStallInterface object maintained by every DB instance is added to the queue of WriteBufferManager. If multiple DBs tries to write during this stall, they will also be blocked when check WriteBufferManager::ShouldStall() returns true. End Stall condition: When flush is finished and memory usage goes down, stall will end only if memory waiting to be flushed is less than buffer_size/2. This lower limit will give time for flush to complete and avoid continous stalling if memory usage remains close to buffer_size. WriterBufferManager::EndWriteStall() is called, which removes all instances from its queue and signal them to continue. Their state is changed to State::Running and they are unblocked. DBImpl then signal all incoming writers of that DB to continue by calling write_thread_.EndWriteStall() (which removes dummy stall object from the queue). DB instance creates WBMStallInterface which is an interface to block and signal DBs during stall. When DB needs to be blocked or signalled by WriteBufferManager, state_for_wbm_ state is changed accordingly (RUNNING or BLOCKED). Pull Request resolved: https://github.com/facebook/rocksdb/pull/7898 Test Plan: Added a new test db/db_write_buffer_manager_test.cc Reviewed By: anand1976 Differential Revision: D26093227 Pulled By: akankshamahajan15 fbshipit-source-id: 2bbd982a3fb7033f6de6153aa92a221249861aae
2021-04-21 20:53:05 +00:00
}
}
Fix race in WriteBufferManager (#9009) Summary: EndWriteStall has a data race: `queue_.empty()` is checked outside of the mutex, so once we enter the critical section another thread may already have cleared the list, and accessing the `front()` is undefined behavior (and causes interesting crashes under high concurrency). This PR fixes the bug, and also rewrites the logic to make it easier to reason about it. It also fixes another subtle bug: if some writers are stalled and `SetBufferSize(0)` is called, which disables the WBM, the writer are not unblocked because of an early `enabled()` check in `EndWriteStall()`. It doesn't significantly change the locking behavior, as before writers won't lock unless entering a stall condition, and `FreeMem` almost always locks if stalling is allowed, but that is inevitable with the current design. Liveness is guaranteed by the fact that if some writes are blocked, eventually all writes will be blocked due to `stall_active_`, and eventually all memory is freed. While at it, do a couple of optimizations: - In `WBMStallInterface::Signal()` signal the CV only after releasing the lock. Signaling under the lock is a common pitfall, as it causes the woken-up thread to immediately go back to sleep because the mutex is still locked by the awaker. - Move all allocations and deallocations outside of the lock. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9009 Test Plan: ``` USE_CLANG=1 make -j64 all check ``` Reviewed By: akankshamahajan15 Differential Revision: D31550668 Pulled By: ot fbshipit-source-id: 5125387c3dc7ecaaa2b8bbc736e58c4156698580
2021-10-12 07:14:41 +00:00
// Called when memory is freed in FreeMem or the buffer size has changed.
void WriteBufferManager::MaybeEndWriteStall() {
// Cannot early-exit on !enabled() because SetBufferSize(0) needs to unblock
// the writers.
if (!allow_stall_) {
return;
}
Stall writes in WriteBufferManager when memory_usage exceeds buffer_size (#7898) Summary: When WriteBufferManager is shared across DBs and column families to maintain memory usage under a limit, OOMs have been observed when flush cannot finish but writes continuously insert to memtables. In order to avoid OOMs, when memory usage goes beyond buffer_limit_ and DBs tries to write, this change will stall incoming writers until flush is completed and memory_usage drops. Design: Stall condition: When total memory usage exceeds WriteBufferManager::buffer_size_ (memory_usage() >= buffer_size_) WriterBufferManager::ShouldStall() returns true. DBImpl first block incoming/future writers by calling write_thread_.BeginWriteStall() (which adds dummy stall object to the writer's queue). Then DB is blocked on a state State::Blocked (current write doesn't go through). WBStallInterface object maintained by every DB instance is added to the queue of WriteBufferManager. If multiple DBs tries to write during this stall, they will also be blocked when check WriteBufferManager::ShouldStall() returns true. End Stall condition: When flush is finished and memory usage goes down, stall will end only if memory waiting to be flushed is less than buffer_size/2. This lower limit will give time for flush to complete and avoid continous stalling if memory usage remains close to buffer_size. WriterBufferManager::EndWriteStall() is called, which removes all instances from its queue and signal them to continue. Their state is changed to State::Running and they are unblocked. DBImpl then signal all incoming writers of that DB to continue by calling write_thread_.EndWriteStall() (which removes dummy stall object from the queue). DB instance creates WBMStallInterface which is an interface to block and signal DBs during stall. When DB needs to be blocked or signalled by WriteBufferManager, state_for_wbm_ state is changed accordingly (RUNNING or BLOCKED). Pull Request resolved: https://github.com/facebook/rocksdb/pull/7898 Test Plan: Added a new test db/db_write_buffer_manager_test.cc Reviewed By: anand1976 Differential Revision: D26093227 Pulled By: akankshamahajan15 fbshipit-source-id: 2bbd982a3fb7033f6de6153aa92a221249861aae
2021-04-21 20:53:05 +00:00
Fix race in WriteBufferManager (#9009) Summary: EndWriteStall has a data race: `queue_.empty()` is checked outside of the mutex, so once we enter the critical section another thread may already have cleared the list, and accessing the `front()` is undefined behavior (and causes interesting crashes under high concurrency). This PR fixes the bug, and also rewrites the logic to make it easier to reason about it. It also fixes another subtle bug: if some writers are stalled and `SetBufferSize(0)` is called, which disables the WBM, the writer are not unblocked because of an early `enabled()` check in `EndWriteStall()`. It doesn't significantly change the locking behavior, as before writers won't lock unless entering a stall condition, and `FreeMem` almost always locks if stalling is allowed, but that is inevitable with the current design. Liveness is guaranteed by the fact that if some writes are blocked, eventually all writes will be blocked due to `stall_active_`, and eventually all memory is freed. While at it, do a couple of optimizations: - In `WBMStallInterface::Signal()` signal the CV only after releasing the lock. Signaling under the lock is a common pitfall, as it causes the woken-up thread to immediately go back to sleep because the mutex is still locked by the awaker. - Move all allocations and deallocations outside of the lock. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9009 Test Plan: ``` USE_CLANG=1 make -j64 all check ``` Reviewed By: akankshamahajan15 Differential Revision: D31550668 Pulled By: ot fbshipit-source-id: 5125387c3dc7ecaaa2b8bbc736e58c4156698580
2021-10-12 07:14:41 +00:00
if (IsStallThresholdExceeded()) {
return; // Stall conditions have not resolved.
}
// Perform all deallocations outside of the lock.
std::list<StallInterface*> cleanup;
std::unique_lock<std::mutex> lock(mu_);
if (!stall_active_.load(std::memory_order_relaxed)) {
return; // Nothing to do.
}
// Unblock new writers.
stall_active_.store(false, std::memory_order_relaxed);
// Unblock the writers in the queue.
for (StallInterface* wbm_stall : queue_) {
wbm_stall->Signal();
Stall writes in WriteBufferManager when memory_usage exceeds buffer_size (#7898) Summary: When WriteBufferManager is shared across DBs and column families to maintain memory usage under a limit, OOMs have been observed when flush cannot finish but writes continuously insert to memtables. In order to avoid OOMs, when memory usage goes beyond buffer_limit_ and DBs tries to write, this change will stall incoming writers until flush is completed and memory_usage drops. Design: Stall condition: When total memory usage exceeds WriteBufferManager::buffer_size_ (memory_usage() >= buffer_size_) WriterBufferManager::ShouldStall() returns true. DBImpl first block incoming/future writers by calling write_thread_.BeginWriteStall() (which adds dummy stall object to the writer's queue). Then DB is blocked on a state State::Blocked (current write doesn't go through). WBStallInterface object maintained by every DB instance is added to the queue of WriteBufferManager. If multiple DBs tries to write during this stall, they will also be blocked when check WriteBufferManager::ShouldStall() returns true. End Stall condition: When flush is finished and memory usage goes down, stall will end only if memory waiting to be flushed is less than buffer_size/2. This lower limit will give time for flush to complete and avoid continous stalling if memory usage remains close to buffer_size. WriterBufferManager::EndWriteStall() is called, which removes all instances from its queue and signal them to continue. Their state is changed to State::Running and they are unblocked. DBImpl then signal all incoming writers of that DB to continue by calling write_thread_.EndWriteStall() (which removes dummy stall object from the queue). DB instance creates WBMStallInterface which is an interface to block and signal DBs during stall. When DB needs to be blocked or signalled by WriteBufferManager, state_for_wbm_ state is changed accordingly (RUNNING or BLOCKED). Pull Request resolved: https://github.com/facebook/rocksdb/pull/7898 Test Plan: Added a new test db/db_write_buffer_manager_test.cc Reviewed By: anand1976 Differential Revision: D26093227 Pulled By: akankshamahajan15 fbshipit-source-id: 2bbd982a3fb7033f6de6153aa92a221249861aae
2021-04-21 20:53:05 +00:00
}
Fix race in WriteBufferManager (#9009) Summary: EndWriteStall has a data race: `queue_.empty()` is checked outside of the mutex, so once we enter the critical section another thread may already have cleared the list, and accessing the `front()` is undefined behavior (and causes interesting crashes under high concurrency). This PR fixes the bug, and also rewrites the logic to make it easier to reason about it. It also fixes another subtle bug: if some writers are stalled and `SetBufferSize(0)` is called, which disables the WBM, the writer are not unblocked because of an early `enabled()` check in `EndWriteStall()`. It doesn't significantly change the locking behavior, as before writers won't lock unless entering a stall condition, and `FreeMem` almost always locks if stalling is allowed, but that is inevitable with the current design. Liveness is guaranteed by the fact that if some writes are blocked, eventually all writes will be blocked due to `stall_active_`, and eventually all memory is freed. While at it, do a couple of optimizations: - In `WBMStallInterface::Signal()` signal the CV only after releasing the lock. Signaling under the lock is a common pitfall, as it causes the woken-up thread to immediately go back to sleep because the mutex is still locked by the awaker. - Move all allocations and deallocations outside of the lock. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9009 Test Plan: ``` USE_CLANG=1 make -j64 all check ``` Reviewed By: akankshamahajan15 Differential Revision: D31550668 Pulled By: ot fbshipit-source-id: 5125387c3dc7ecaaa2b8bbc736e58c4156698580
2021-10-12 07:14:41 +00:00
cleanup = std::move(queue_);
Stall writes in WriteBufferManager when memory_usage exceeds buffer_size (#7898) Summary: When WriteBufferManager is shared across DBs and column families to maintain memory usage under a limit, OOMs have been observed when flush cannot finish but writes continuously insert to memtables. In order to avoid OOMs, when memory usage goes beyond buffer_limit_ and DBs tries to write, this change will stall incoming writers until flush is completed and memory_usage drops. Design: Stall condition: When total memory usage exceeds WriteBufferManager::buffer_size_ (memory_usage() >= buffer_size_) WriterBufferManager::ShouldStall() returns true. DBImpl first block incoming/future writers by calling write_thread_.BeginWriteStall() (which adds dummy stall object to the writer's queue). Then DB is blocked on a state State::Blocked (current write doesn't go through). WBStallInterface object maintained by every DB instance is added to the queue of WriteBufferManager. If multiple DBs tries to write during this stall, they will also be blocked when check WriteBufferManager::ShouldStall() returns true. End Stall condition: When flush is finished and memory usage goes down, stall will end only if memory waiting to be flushed is less than buffer_size/2. This lower limit will give time for flush to complete and avoid continous stalling if memory usage remains close to buffer_size. WriterBufferManager::EndWriteStall() is called, which removes all instances from its queue and signal them to continue. Their state is changed to State::Running and they are unblocked. DBImpl then signal all incoming writers of that DB to continue by calling write_thread_.EndWriteStall() (which removes dummy stall object from the queue). DB instance creates WBMStallInterface which is an interface to block and signal DBs during stall. When DB needs to be blocked or signalled by WriteBufferManager, state_for_wbm_ state is changed accordingly (RUNNING or BLOCKED). Pull Request resolved: https://github.com/facebook/rocksdb/pull/7898 Test Plan: Added a new test db/db_write_buffer_manager_test.cc Reviewed By: anand1976 Differential Revision: D26093227 Pulled By: akankshamahajan15 fbshipit-source-id: 2bbd982a3fb7033f6de6153aa92a221249861aae
2021-04-21 20:53:05 +00:00
}
void WriteBufferManager::RemoveDBFromQueue(StallInterface* wbm_stall) {
assert(wbm_stall != nullptr);
Fix race in WriteBufferManager (#9009) Summary: EndWriteStall has a data race: `queue_.empty()` is checked outside of the mutex, so once we enter the critical section another thread may already have cleared the list, and accessing the `front()` is undefined behavior (and causes interesting crashes under high concurrency). This PR fixes the bug, and also rewrites the logic to make it easier to reason about it. It also fixes another subtle bug: if some writers are stalled and `SetBufferSize(0)` is called, which disables the WBM, the writer are not unblocked because of an early `enabled()` check in `EndWriteStall()`. It doesn't significantly change the locking behavior, as before writers won't lock unless entering a stall condition, and `FreeMem` almost always locks if stalling is allowed, but that is inevitable with the current design. Liveness is guaranteed by the fact that if some writes are blocked, eventually all writes will be blocked due to `stall_active_`, and eventually all memory is freed. While at it, do a couple of optimizations: - In `WBMStallInterface::Signal()` signal the CV only after releasing the lock. Signaling under the lock is a common pitfall, as it causes the woken-up thread to immediately go back to sleep because the mutex is still locked by the awaker. - Move all allocations and deallocations outside of the lock. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9009 Test Plan: ``` USE_CLANG=1 make -j64 all check ``` Reviewed By: akankshamahajan15 Differential Revision: D31550668 Pulled By: ot fbshipit-source-id: 5125387c3dc7ecaaa2b8bbc736e58c4156698580
2021-10-12 07:14:41 +00:00
// Deallocate the removed nodes outside of the lock.
std::list<StallInterface*> cleanup;
Stall writes in WriteBufferManager when memory_usage exceeds buffer_size (#7898) Summary: When WriteBufferManager is shared across DBs and column families to maintain memory usage under a limit, OOMs have been observed when flush cannot finish but writes continuously insert to memtables. In order to avoid OOMs, when memory usage goes beyond buffer_limit_ and DBs tries to write, this change will stall incoming writers until flush is completed and memory_usage drops. Design: Stall condition: When total memory usage exceeds WriteBufferManager::buffer_size_ (memory_usage() >= buffer_size_) WriterBufferManager::ShouldStall() returns true. DBImpl first block incoming/future writers by calling write_thread_.BeginWriteStall() (which adds dummy stall object to the writer's queue). Then DB is blocked on a state State::Blocked (current write doesn't go through). WBStallInterface object maintained by every DB instance is added to the queue of WriteBufferManager. If multiple DBs tries to write during this stall, they will also be blocked when check WriteBufferManager::ShouldStall() returns true. End Stall condition: When flush is finished and memory usage goes down, stall will end only if memory waiting to be flushed is less than buffer_size/2. This lower limit will give time for flush to complete and avoid continous stalling if memory usage remains close to buffer_size. WriterBufferManager::EndWriteStall() is called, which removes all instances from its queue and signal them to continue. Their state is changed to State::Running and they are unblocked. DBImpl then signal all incoming writers of that DB to continue by calling write_thread_.EndWriteStall() (which removes dummy stall object from the queue). DB instance creates WBMStallInterface which is an interface to block and signal DBs during stall. When DB needs to be blocked or signalled by WriteBufferManager, state_for_wbm_ state is changed accordingly (RUNNING or BLOCKED). Pull Request resolved: https://github.com/facebook/rocksdb/pull/7898 Test Plan: Added a new test db/db_write_buffer_manager_test.cc Reviewed By: anand1976 Differential Revision: D26093227 Pulled By: akankshamahajan15 fbshipit-source-id: 2bbd982a3fb7033f6de6153aa92a221249861aae
2021-04-21 20:53:05 +00:00
if (enabled() && allow_stall_) {
std::unique_lock<std::mutex> lock(mu_);
Fix race in WriteBufferManager (#9009) Summary: EndWriteStall has a data race: `queue_.empty()` is checked outside of the mutex, so once we enter the critical section another thread may already have cleared the list, and accessing the `front()` is undefined behavior (and causes interesting crashes under high concurrency). This PR fixes the bug, and also rewrites the logic to make it easier to reason about it. It also fixes another subtle bug: if some writers are stalled and `SetBufferSize(0)` is called, which disables the WBM, the writer are not unblocked because of an early `enabled()` check in `EndWriteStall()`. It doesn't significantly change the locking behavior, as before writers won't lock unless entering a stall condition, and `FreeMem` almost always locks if stalling is allowed, but that is inevitable with the current design. Liveness is guaranteed by the fact that if some writes are blocked, eventually all writes will be blocked due to `stall_active_`, and eventually all memory is freed. While at it, do a couple of optimizations: - In `WBMStallInterface::Signal()` signal the CV only after releasing the lock. Signaling under the lock is a common pitfall, as it causes the woken-up thread to immediately go back to sleep because the mutex is still locked by the awaker. - Move all allocations and deallocations outside of the lock. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9009 Test Plan: ``` USE_CLANG=1 make -j64 all check ``` Reviewed By: akankshamahajan15 Differential Revision: D31550668 Pulled By: ot fbshipit-source-id: 5125387c3dc7ecaaa2b8bbc736e58c4156698580
2021-10-12 07:14:41 +00:00
for (auto it = queue_.begin(); it != queue_.end();) {
auto next = std::next(it);
if (*it == wbm_stall) {
cleanup.splice(cleanup.end(), queue_, std::move(it));
}
it = next;
}
Stall writes in WriteBufferManager when memory_usage exceeds buffer_size (#7898) Summary: When WriteBufferManager is shared across DBs and column families to maintain memory usage under a limit, OOMs have been observed when flush cannot finish but writes continuously insert to memtables. In order to avoid OOMs, when memory usage goes beyond buffer_limit_ and DBs tries to write, this change will stall incoming writers until flush is completed and memory_usage drops. Design: Stall condition: When total memory usage exceeds WriteBufferManager::buffer_size_ (memory_usage() >= buffer_size_) WriterBufferManager::ShouldStall() returns true. DBImpl first block incoming/future writers by calling write_thread_.BeginWriteStall() (which adds dummy stall object to the writer's queue). Then DB is blocked on a state State::Blocked (current write doesn't go through). WBStallInterface object maintained by every DB instance is added to the queue of WriteBufferManager. If multiple DBs tries to write during this stall, they will also be blocked when check WriteBufferManager::ShouldStall() returns true. End Stall condition: When flush is finished and memory usage goes down, stall will end only if memory waiting to be flushed is less than buffer_size/2. This lower limit will give time for flush to complete and avoid continous stalling if memory usage remains close to buffer_size. WriterBufferManager::EndWriteStall() is called, which removes all instances from its queue and signal them to continue. Their state is changed to State::Running and they are unblocked. DBImpl then signal all incoming writers of that DB to continue by calling write_thread_.EndWriteStall() (which removes dummy stall object from the queue). DB instance creates WBMStallInterface which is an interface to block and signal DBs during stall. When DB needs to be blocked or signalled by WriteBufferManager, state_for_wbm_ state is changed accordingly (RUNNING or BLOCKED). Pull Request resolved: https://github.com/facebook/rocksdb/pull/7898 Test Plan: Added a new test db/db_write_buffer_manager_test.cc Reviewed By: anand1976 Differential Revision: D26093227 Pulled By: akankshamahajan15 fbshipit-source-id: 2bbd982a3fb7033f6de6153aa92a221249861aae
2021-04-21 20:53:05 +00:00
}
Fix race in WriteBufferManager (#9009) Summary: EndWriteStall has a data race: `queue_.empty()` is checked outside of the mutex, so once we enter the critical section another thread may already have cleared the list, and accessing the `front()` is undefined behavior (and causes interesting crashes under high concurrency). This PR fixes the bug, and also rewrites the logic to make it easier to reason about it. It also fixes another subtle bug: if some writers are stalled and `SetBufferSize(0)` is called, which disables the WBM, the writer are not unblocked because of an early `enabled()` check in `EndWriteStall()`. It doesn't significantly change the locking behavior, as before writers won't lock unless entering a stall condition, and `FreeMem` almost always locks if stalling is allowed, but that is inevitable with the current design. Liveness is guaranteed by the fact that if some writes are blocked, eventually all writes will be blocked due to `stall_active_`, and eventually all memory is freed. While at it, do a couple of optimizations: - In `WBMStallInterface::Signal()` signal the CV only after releasing the lock. Signaling under the lock is a common pitfall, as it causes the woken-up thread to immediately go back to sleep because the mutex is still locked by the awaker. - Move all allocations and deallocations outside of the lock. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9009 Test Plan: ``` USE_CLANG=1 make -j64 all check ``` Reviewed By: akankshamahajan15 Differential Revision: D31550668 Pulled By: ot fbshipit-source-id: 5125387c3dc7ecaaa2b8bbc736e58c4156698580
2021-10-12 07:14:41 +00:00
wbm_stall->Signal();
Stall writes in WriteBufferManager when memory_usage exceeds buffer_size (#7898) Summary: When WriteBufferManager is shared across DBs and column families to maintain memory usage under a limit, OOMs have been observed when flush cannot finish but writes continuously insert to memtables. In order to avoid OOMs, when memory usage goes beyond buffer_limit_ and DBs tries to write, this change will stall incoming writers until flush is completed and memory_usage drops. Design: Stall condition: When total memory usage exceeds WriteBufferManager::buffer_size_ (memory_usage() >= buffer_size_) WriterBufferManager::ShouldStall() returns true. DBImpl first block incoming/future writers by calling write_thread_.BeginWriteStall() (which adds dummy stall object to the writer's queue). Then DB is blocked on a state State::Blocked (current write doesn't go through). WBStallInterface object maintained by every DB instance is added to the queue of WriteBufferManager. If multiple DBs tries to write during this stall, they will also be blocked when check WriteBufferManager::ShouldStall() returns true. End Stall condition: When flush is finished and memory usage goes down, stall will end only if memory waiting to be flushed is less than buffer_size/2. This lower limit will give time for flush to complete and avoid continous stalling if memory usage remains close to buffer_size. WriterBufferManager::EndWriteStall() is called, which removes all instances from its queue and signal them to continue. Their state is changed to State::Running and they are unblocked. DBImpl then signal all incoming writers of that DB to continue by calling write_thread_.EndWriteStall() (which removes dummy stall object from the queue). DB instance creates WBMStallInterface which is an interface to block and signal DBs during stall. When DB needs to be blocked or signalled by WriteBufferManager, state_for_wbm_ state is changed accordingly (RUNNING or BLOCKED). Pull Request resolved: https://github.com/facebook/rocksdb/pull/7898 Test Plan: Added a new test db/db_write_buffer_manager_test.cc Reviewed By: anand1976 Differential Revision: D26093227 Pulled By: akankshamahajan15 fbshipit-source-id: 2bbd982a3fb7033f6de6153aa92a221249861aae
2021-04-21 20:53:05 +00:00
}
} // namespace ROCKSDB_NAMESPACE