Compare commits
14 Commits
Author | SHA1 | Date | |
---|---|---|---|
|
3e0ff19b1b | ||
|
414de699b6 | ||
|
300b2249dc | ||
|
a28f13c937 | ||
|
de97dddf61 | ||
|
05835f4071 | ||
|
51b540921d | ||
|
e8b841eb47 | ||
|
6ee0acdbb4 | ||
|
6fc2818ef8 | ||
|
5881f4ac50 | ||
|
28aa6d4e76 | ||
|
9d7de9605a | ||
|
f64f08e0bc |
20
HISTORY.md
20
HISTORY.md
@ -1,5 +1,17 @@
|
||||
# Rocksdb Change Log
|
||||
## Unreleased
|
||||
## 6.22.3 (2021-07-19)
|
||||
### Bug Fixes
|
||||
* Fixed block cache entry stat scans not to hold the DB mutex, which was a serious performance bug for tail latencies in TransactionDB and likely elsewhere.
|
||||
|
||||
## 6.22.2 (2021-07-14)
|
||||
### Bug Fixes
|
||||
* Fix continuous logging of an existing background error on every user write
|
||||
|
||||
## 6.22.1 (2021-06-25)
|
||||
### Bug Fixes
|
||||
* `GetLiveFilesMetaData()` now populates the `temperature`, `oldest_ancester_time`, and `file_creation_time` fields of its `LiveFileMetaData` results when the information is available. Previously these fields always contained zero indicating unknown.
|
||||
|
||||
## 6.22.0 (2021-06-18)
|
||||
### Behavior Changes
|
||||
* Added two additional tickers, MEMTABLE_PAYLOAD_BYTES_AT_FLUSH and MEMTABLE_GARBAGE_BYTES_AT_FLUSH. These stats can be used to estimate the ratio of "garbage" (outdated) bytes in the memtable that are discarded at flush time.
|
||||
* Added API comments clarifying safe usage of Disable/EnableManualCompaction and EventListener callbacks for compaction.
|
||||
@ -7,11 +19,15 @@
|
||||
### Bug Fixes
|
||||
* fs_posix.cc GetFreeSpace() always report disk space available to root even when running as non-root. Linux defaults often have disk mounts with 5 to 10 percent of total space reserved only for root. Out of space could result for non-root users.
|
||||
* Subcompactions are now disabled when user-defined timestamps are used, since the subcompaction boundary picking logic is currently not timestamp-aware, which could lead to incorrect results when different subcompactions process keys that only differ by timestamp.
|
||||
* Fix an issue that `DeleteFilesInRange()` may cause ongoing compaction reports corruption exception, or ASSERT for debug build. There's no actual data loss or corruption that we find.
|
||||
* Fixed confusingly duplicated output in LOG for periodic stats ("DUMPING STATS"), including "Compaction Stats" and "File Read Latency Histogram By Level".
|
||||
* Fixed performance bugs in background gathering of block cache entry statistics, that could consume a lot of CPU when there are many column families with a shared block cache.
|
||||
|
||||
### New Features
|
||||
* Marked the Ribbon filter and optimize_filters_for_memory features as production-ready, each enabling memory savings for Bloom-like filters. Use `NewRibbonFilterPolicy` in place of `NewBloomFilterPolicy` to use Ribbon filters instead of Bloom, or `ribbonfilter` in place of `bloomfilter` in configuration string.
|
||||
* Allow `DBWithTTL` to use `DeleteRange` api just like other DBs. `DeleteRangeCF()` which executes `WriteBatchInternal::DeleteRange()` has been added to the handler in `DBWithTTLImpl::Write()` to implement it.
|
||||
* Add BlockBasedTableOptions.prepopulate_block_cache. If enabled, it prepopulate warm/hot data blocks which are already in memory into block cache at the time of flush. On a flush, the data block that is in memory (in memtables) get flushed to the device. If using Direct IO, additional IO is incurred to read this data back into memory again, which is avoided by enabling this option and it also helps with Distributed FileSystem. More details in include/rocksdb/table.h.
|
||||
* Added a `cancel` field to `CompactRangeOptions`, allowing individual in-process manual range compactions to be cancelled.
|
||||
|
||||
## 6.21.0 (2021-05-21)
|
||||
### Bug Fixes
|
||||
@ -23,7 +39,6 @@
|
||||
* Handle return code by io_uring_submit_and_wait() and io_uring_wait_cqe().
|
||||
* In the IngestExternalFile() API, only try to sync the ingested file if the file is linked and the FileSystem/Env supports reopening a writable file.
|
||||
* Fixed a bug that `AdvancedColumnFamilyOptions.max_compaction_bytes` is under-calculated for manual compaction (`CompactRange()`). Manual compaction is split to multiple compactions if the compaction size exceed the `max_compaction_bytes`. The bug creates much larger compaction which size exceed the user setting. On the other hand, larger manual compaction size can increase the subcompaction parallelism, you can tune that by setting `max_compaction_bytes`.
|
||||
* Fixed confusingly duplicated output in LOG for periodic stats ("DUMPING STATS"), including "Compaction Stats" and "File Read Latency Histogram By Level".
|
||||
|
||||
### Behavior Changes
|
||||
* Due to the fix of false-postive alert of "SST file is ahead of WAL", all the CFs with no SST file (CF empty) will bypass the consistency check. We fixed a false-positive, but introduced a very rare true-negative which will be triggered in the following conditions: A CF with some delete operations in the last a few queries which will result in an empty CF (those are flushed to SST file and a compaction triggered which combines this file and all other SST files and generates an empty CF, or there is another reason to write a manifest entry for this CF after a flush that generates no SST file from an empty CF). The deletion entries are logged in a WAL and this WAL was corrupted, while the CF's log number points to the next WAL (due to the flush). Therefore, the DB can only recover to the point without these trailing deletions and cause the inconsistent DB status.
|
||||
@ -36,7 +51,6 @@
|
||||
* Add an experimental Remote Compaction feature, which allows the user to run Compaction on a different host or process. The feature is still under development, currently only works on some basic use cases. The interface will be changed without backward/forward compatibility support.
|
||||
* RocksDB would validate total entries read in flush, and compare with counter inserted into it. If flush_verify_memtable_count = true (default), flush will fail. Otherwise, only log to info logs.
|
||||
* Add `TableProperties::num_filter_entries`, which can be used with `TableProperties::filter_size` to calculate the effective bits per filter entry (unique user key or prefix) for a table file.
|
||||
* Added a `cancel` field to `CompactRangeOptions`, allowing individual in-process manual range compactions to be cancelled.
|
||||
|
||||
### Performance Improvements
|
||||
* BlockPrefetcher is used by iterators to prefetch data if they anticipate more data to be used in future. It is enabled implicitly by rocksdb. Added change to take in account read pattern if reads are sequential. This would disable prefetching for random reads in MultiGet and iterators as readahead_size is increased exponential doing large prefetches.
|
||||
|
2
Makefile
2
Makefile
@ -516,8 +516,10 @@ endif
|
||||
ifdef ASSERT_STATUS_CHECKED
|
||||
# TODO: finish fixing all tests to pass this check
|
||||
TESTS_FAILING_ASC = \
|
||||
c_test \
|
||||
db_test \
|
||||
db_test2 \
|
||||
env_test \
|
||||
range_locking_test \
|
||||
testutil_test \
|
||||
|
||||
|
35
cache/cache_entry_stats.h
vendored
35
cache/cache_entry_stats.h
vendored
@ -51,7 +51,8 @@ namespace ROCKSDB_NAMESPACE {
|
||||
template <class Stats>
|
||||
class CacheEntryStatsCollector {
|
||||
public:
|
||||
// Gathers stats and saves results into `stats`
|
||||
// Gather and save stats if saved stats are too old. (Use GetStats() to
|
||||
// read saved stats.)
|
||||
//
|
||||
// Maximum allowed age for a "hit" on saved results is determined by the
|
||||
// two interval parameters. Both set to 0 forces a re-scan. For example
|
||||
@ -61,10 +62,9 @@ class CacheEntryStatsCollector {
|
||||
// Justification: scans can vary wildly in duration, e.g. from 0.02 sec
|
||||
// to as much as 20 seconds, so we want to be able to cap the absolute
|
||||
// and relative frequency of scans.
|
||||
void GetStats(Stats *stats, int min_interval_seconds,
|
||||
int min_interval_factor) {
|
||||
void CollectStats(int min_interval_seconds, int min_interval_factor) {
|
||||
// Waits for any pending reader or writer (collector)
|
||||
std::lock_guard<std::mutex> lock(mutex_);
|
||||
std::lock_guard<std::mutex> lock(working_mutex_);
|
||||
|
||||
uint64_t max_age_micros =
|
||||
static_cast<uint64_t>(std::max(min_interval_seconds, 0)) * 1000000U;
|
||||
@ -79,19 +79,28 @@ class CacheEntryStatsCollector {
|
||||
uint64_t start_time_micros = clock_->NowMicros();
|
||||
if ((start_time_micros - last_end_time_micros_) > max_age_micros) {
|
||||
last_start_time_micros_ = start_time_micros;
|
||||
saved_stats_.BeginCollection(cache_, clock_, start_time_micros);
|
||||
working_stats_.BeginCollection(cache_, clock_, start_time_micros);
|
||||
|
||||
cache_->ApplyToAllEntries(saved_stats_.GetEntryCallback(), {});
|
||||
cache_->ApplyToAllEntries(working_stats_.GetEntryCallback(), {});
|
||||
TEST_SYNC_POINT_CALLBACK(
|
||||
"CacheEntryStatsCollector::GetStats:AfterApplyToAllEntries", nullptr);
|
||||
|
||||
uint64_t end_time_micros = clock_->NowMicros();
|
||||
last_end_time_micros_ = end_time_micros;
|
||||
saved_stats_.EndCollection(cache_, clock_, end_time_micros);
|
||||
working_stats_.EndCollection(cache_, clock_, end_time_micros);
|
||||
} else {
|
||||
saved_stats_.SkippedCollection();
|
||||
working_stats_.SkippedCollection();
|
||||
}
|
||||
// Copy to caller
|
||||
|
||||
// Save so that we don't need to wait for an outstanding collection in
|
||||
// order to make of copy of the last saved stats
|
||||
std::lock_guard<std::mutex> lock2(saved_mutex_);
|
||||
saved_stats_ = working_stats_;
|
||||
}
|
||||
|
||||
// Gets saved stats, regardless of age
|
||||
void GetStats(Stats *stats) {
|
||||
std::lock_guard<std::mutex> lock(saved_mutex_);
|
||||
*stats = saved_stats_;
|
||||
}
|
||||
|
||||
@ -129,6 +138,7 @@ class CacheEntryStatsCollector {
|
||||
Cache::Priority::HIGH);
|
||||
if (!s.ok()) {
|
||||
assert(h == nullptr);
|
||||
delete new_ptr;
|
||||
return s;
|
||||
}
|
||||
}
|
||||
@ -145,6 +155,7 @@ class CacheEntryStatsCollector {
|
||||
private:
|
||||
explicit CacheEntryStatsCollector(Cache *cache, SystemClock *clock)
|
||||
: saved_stats_(),
|
||||
working_stats_(),
|
||||
last_start_time_micros_(0),
|
||||
last_end_time_micros_(/*pessimistic*/ 10000000),
|
||||
cache_(cache),
|
||||
@ -154,10 +165,14 @@ class CacheEntryStatsCollector {
|
||||
delete static_cast<CacheEntryStatsCollector *>(value);
|
||||
}
|
||||
|
||||
std::mutex mutex_;
|
||||
std::mutex saved_mutex_;
|
||||
Stats saved_stats_;
|
||||
|
||||
std::mutex working_mutex_;
|
||||
Stats working_stats_;
|
||||
uint64_t last_start_time_micros_;
|
||||
uint64_t last_end_time_micros_;
|
||||
|
||||
Cache *const cache_;
|
||||
SystemClock *const clock_;
|
||||
};
|
||||
|
10
cache/lru_cache.cc
vendored
10
cache/lru_cache.cc
vendored
@ -553,6 +553,9 @@ Status LRUCacheShard::Insert(const Slice& key, uint32_t hash, void* value,
|
||||
e->SetSecondaryCacheCompatible(true);
|
||||
e->info_.helper = helper;
|
||||
} else {
|
||||
#ifdef __SANITIZE_THREAD__
|
||||
e->is_secondary_cache_compatible_for_tsan = false;
|
||||
#endif // __SANITIZE_THREAD__
|
||||
e->info_.deleter = deleter;
|
||||
}
|
||||
e->charge = charge;
|
||||
@ -596,15 +599,12 @@ void LRUCacheShard::Erase(const Slice& key, uint32_t hash) {
|
||||
|
||||
bool LRUCacheShard::IsReady(Cache::Handle* handle) {
|
||||
LRUHandle* e = reinterpret_cast<LRUHandle*>(handle);
|
||||
MutexLock l(&mutex_);
|
||||
bool ready = true;
|
||||
if (e->IsPending()) {
|
||||
assert(secondary_cache_);
|
||||
assert(e->sec_handle);
|
||||
if (e->sec_handle->IsReady()) {
|
||||
Promote(e);
|
||||
} else {
|
||||
ready = false;
|
||||
}
|
||||
ready = e->sec_handle->IsReady();
|
||||
}
|
||||
return ready;
|
||||
}
|
||||
|
35
cache/lru_cache.h
vendored
35
cache/lru_cache.h
vendored
@ -12,6 +12,7 @@
|
||||
#include <string>
|
||||
|
||||
#include "cache/sharded_cache.h"
|
||||
#include "port/lang.h"
|
||||
#include "port/malloc.h"
|
||||
#include "port/port.h"
|
||||
#include "rocksdb/secondary_cache.h"
|
||||
@ -80,8 +81,8 @@ struct LRUHandle {
|
||||
IN_HIGH_PRI_POOL = (1 << 2),
|
||||
// Whether this entry has had any lookups (hits).
|
||||
HAS_HIT = (1 << 3),
|
||||
// Can this be inserted into the tiered cache
|
||||
IS_TIERED_CACHE_COMPATIBLE = (1 << 4),
|
||||
// Can this be inserted into the secondary cache
|
||||
IS_SECONDARY_CACHE_COMPATIBLE = (1 << 4),
|
||||
// Is the handle still being read from a lower tier
|
||||
IS_PENDING = (1 << 5),
|
||||
// Has the item been promoted from a lower tier
|
||||
@ -90,6 +91,14 @@ struct LRUHandle {
|
||||
|
||||
uint8_t flags;
|
||||
|
||||
#ifdef __SANITIZE_THREAD__
|
||||
// TSAN can report a false data race on flags, where one thread is writing
|
||||
// to one of the mutable bits and another thread is reading this immutable
|
||||
// bit. So precisely suppress that TSAN warning, we separate out this bit
|
||||
// during TSAN runs.
|
||||
bool is_secondary_cache_compatible_for_tsan;
|
||||
#endif // __SANITIZE_THREAD__
|
||||
|
||||
// Beginning of the key (MUST BE THE LAST FIELD IN THIS STRUCT!)
|
||||
char key_data[1];
|
||||
|
||||
@ -113,7 +122,11 @@ struct LRUHandle {
|
||||
bool InHighPriPool() const { return flags & IN_HIGH_PRI_POOL; }
|
||||
bool HasHit() const { return flags & HAS_HIT; }
|
||||
bool IsSecondaryCacheCompatible() const {
|
||||
return flags & IS_TIERED_CACHE_COMPATIBLE;
|
||||
#ifdef __SANITIZE_THREAD__
|
||||
return is_secondary_cache_compatible_for_tsan;
|
||||
#else
|
||||
return flags & IS_SECONDARY_CACHE_COMPATIBLE;
|
||||
#endif // __SANITIZE_THREAD__
|
||||
}
|
||||
bool IsPending() const { return flags & IS_PENDING; }
|
||||
bool IsPromoted() const { return flags & IS_PROMOTED; }
|
||||
@ -144,12 +157,15 @@ struct LRUHandle {
|
||||
|
||||
void SetHit() { flags |= HAS_HIT; }
|
||||
|
||||
void SetSecondaryCacheCompatible(bool tiered) {
|
||||
if (tiered) {
|
||||
flags |= IS_TIERED_CACHE_COMPATIBLE;
|
||||
void SetSecondaryCacheCompatible(bool compat) {
|
||||
if (compat) {
|
||||
flags |= IS_SECONDARY_CACHE_COMPATIBLE;
|
||||
} else {
|
||||
flags &= ~IS_TIERED_CACHE_COMPATIBLE;
|
||||
flags &= ~IS_SECONDARY_CACHE_COMPATIBLE;
|
||||
}
|
||||
#ifdef __SANITIZE_THREAD__
|
||||
is_secondary_cache_compatible_for_tsan = compat;
|
||||
#endif // __SANITIZE_THREAD__
|
||||
}
|
||||
|
||||
void SetIncomplete(bool incomp) {
|
||||
@ -170,6 +186,11 @@ struct LRUHandle {
|
||||
|
||||
void Free() {
|
||||
assert(refs == 0);
|
||||
#ifdef __SANITIZE_THREAD__
|
||||
// Here we can safely assert they are the same without a data race reported
|
||||
assert(((flags & IS_SECONDARY_CACHE_COMPATIBLE) != 0) ==
|
||||
is_secondary_cache_compatible_for_tsan);
|
||||
#endif // __SANITIZE_THREAD__
|
||||
if (!IsSecondaryCacheCompatible() && info_.deleter) {
|
||||
(*info_.deleter)(key(), value);
|
||||
} else if (IsSecondaryCacheCompatible()) {
|
||||
|
@ -11,6 +11,7 @@
|
||||
|
||||
#include "cache/cache_entry_roles.h"
|
||||
#include "cache/lru_cache.h"
|
||||
#include "db/column_family.h"
|
||||
#include "db/db_test_util.h"
|
||||
#include "port/stack_trace.h"
|
||||
#include "rocksdb/table.h"
|
||||
@ -152,13 +153,15 @@ class DBBlockCacheTest : public DBTestBase {
|
||||
}
|
||||
|
||||
#ifndef ROCKSDB_LITE
|
||||
const std::array<size_t, kNumCacheEntryRoles>& GetCacheEntryRoleCountsBg() {
|
||||
const std::array<size_t, kNumCacheEntryRoles> GetCacheEntryRoleCountsBg() {
|
||||
// Verify in cache entry role stats
|
||||
ColumnFamilyHandleImpl* cfh =
|
||||
static_cast<ColumnFamilyHandleImpl*>(dbfull()->DefaultColumnFamily());
|
||||
InternalStats* internal_stats_ptr = cfh->cfd()->internal_stats();
|
||||
return internal_stats_ptr->TEST_GetCacheEntryRoleStats(/*foreground=*/false)
|
||||
.entry_counts;
|
||||
InternalStats::CacheEntryRoleStats stats;
|
||||
internal_stats_ptr->TEST_GetCacheEntryRoleStats(&stats,
|
||||
/*foreground=*/false);
|
||||
return stats.entry_counts;
|
||||
}
|
||||
#endif // ROCKSDB_LITE
|
||||
};
|
||||
@ -170,7 +173,13 @@ TEST_F(DBBlockCacheTest, IteratorBlockCacheUsage) {
|
||||
auto options = GetOptions(table_options);
|
||||
InitTable(options);
|
||||
|
||||
std::shared_ptr<Cache> cache = NewLRUCache(0, 0, false);
|
||||
LRUCacheOptions co;
|
||||
co.capacity = 0;
|
||||
co.num_shard_bits = 0;
|
||||
co.strict_capacity_limit = false;
|
||||
// Needed not to count entry stats collector
|
||||
co.metadata_charge_policy = kDontChargeCacheMetadata;
|
||||
std::shared_ptr<Cache> cache = NewLRUCache(co);
|
||||
table_options.block_cache = cache;
|
||||
options.table_factory.reset(NewBlockBasedTableFactory(table_options));
|
||||
Reopen(options);
|
||||
@ -194,7 +203,13 @@ TEST_F(DBBlockCacheTest, TestWithoutCompressedBlockCache) {
|
||||
auto options = GetOptions(table_options);
|
||||
InitTable(options);
|
||||
|
||||
std::shared_ptr<Cache> cache = NewLRUCache(0, 0, false);
|
||||
LRUCacheOptions co;
|
||||
co.capacity = 0;
|
||||
co.num_shard_bits = 0;
|
||||
co.strict_capacity_limit = false;
|
||||
// Needed not to count entry stats collector
|
||||
co.metadata_charge_policy = kDontChargeCacheMetadata;
|
||||
std::shared_ptr<Cache> cache = NewLRUCache(co);
|
||||
table_options.block_cache = cache;
|
||||
options.table_factory.reset(NewBlockBasedTableFactory(table_options));
|
||||
Reopen(options);
|
||||
@ -265,7 +280,13 @@ TEST_F(DBBlockCacheTest, TestWithCompressedBlockCache) {
|
||||
|
||||
ReadOptions read_options;
|
||||
std::shared_ptr<Cache> compressed_cache = NewLRUCache(1 << 25, 0, false);
|
||||
std::shared_ptr<Cache> cache = NewLRUCache(0, 0, false);
|
||||
LRUCacheOptions co;
|
||||
co.capacity = 0;
|
||||
co.num_shard_bits = 0;
|
||||
co.strict_capacity_limit = false;
|
||||
// Needed not to count entry stats collector
|
||||
co.metadata_charge_policy = kDontChargeCacheMetadata;
|
||||
std::shared_ptr<Cache> cache = NewLRUCache(co);
|
||||
table_options.block_cache = cache;
|
||||
table_options.no_block_cache = false;
|
||||
table_options.block_cache_compressed = compressed_cache;
|
||||
@ -944,10 +965,15 @@ TEST_F(DBBlockCacheTest, CacheCompressionDict) {
|
||||
}
|
||||
|
||||
static void ClearCache(Cache* cache) {
|
||||
auto roles = CopyCacheDeleterRoleMap();
|
||||
std::deque<std::string> keys;
|
||||
Cache::ApplyToAllEntriesOptions opts;
|
||||
auto callback = [&](const Slice& key, void* /*value*/, size_t /*charge*/,
|
||||
Cache::DeleterFn /*deleter*/) {
|
||||
Cache::DeleterFn deleter) {
|
||||
if (roles.find(deleter) == roles.end()) {
|
||||
// Keep the stats collector
|
||||
return;
|
||||
}
|
||||
keys.push_back(key.ToString());
|
||||
};
|
||||
cache->ApplyToAllEntries(callback, opts);
|
||||
@ -1126,6 +1152,9 @@ TEST_F(DBBlockCacheTest, CacheEntryRoleStats) {
|
||||
&h, Cache::Priority::HIGH));
|
||||
ASSERT_GT(cache->GetUsage(), cache->GetCapacity());
|
||||
expected = {};
|
||||
// For CacheEntryStatsCollector
|
||||
expected[static_cast<size_t>(CacheEntryRole::kMisc)] = 1;
|
||||
// For Fill-it-up
|
||||
expected[static_cast<size_t>(CacheEntryRole::kMisc)]++;
|
||||
// Still able to hit on saved stats
|
||||
EXPECT_EQ(prev_expected, GetCacheEntryRoleCountsBg());
|
||||
@ -1134,6 +1163,48 @@ TEST_F(DBBlockCacheTest, CacheEntryRoleStats) {
|
||||
EXPECT_EQ(expected, GetCacheEntryRoleCountsBg());
|
||||
|
||||
cache->Release(h);
|
||||
|
||||
// Now we test that the DB mutex is not held during scans, for the ways
|
||||
// we know how to (possibly) trigger them. Without a better good way to
|
||||
// check this, we simply inject an acquire & release of the DB mutex
|
||||
// deep in the stat collection code. If we were already holding the
|
||||
// mutex, that is UB that would at least be found by TSAN.
|
||||
int scan_count = 0;
|
||||
SyncPoint::GetInstance()->SetCallBack(
|
||||
"CacheEntryStatsCollector::GetStats:AfterApplyToAllEntries",
|
||||
[this, &scan_count](void*) {
|
||||
dbfull()->TEST_LockMutex();
|
||||
dbfull()->TEST_UnlockMutex();
|
||||
++scan_count;
|
||||
});
|
||||
SyncPoint::GetInstance()->EnableProcessing();
|
||||
|
||||
// Different things that might trigger a scan, with mock sleeps to
|
||||
// force a miss.
|
||||
env_->MockSleepForSeconds(10000);
|
||||
dbfull()->DumpStats();
|
||||
ASSERT_EQ(scan_count, 1);
|
||||
|
||||
env_->MockSleepForSeconds(10000);
|
||||
ASSERT_TRUE(
|
||||
db_->GetMapProperty(DB::Properties::kBlockCacheEntryStats, &values));
|
||||
ASSERT_EQ(scan_count, 2);
|
||||
|
||||
env_->MockSleepForSeconds(10000);
|
||||
std::string value_str;
|
||||
ASSERT_TRUE(
|
||||
db_->GetProperty(DB::Properties::kBlockCacheEntryStats, &value_str));
|
||||
ASSERT_EQ(scan_count, 3);
|
||||
|
||||
env_->MockSleepForSeconds(10000);
|
||||
ASSERT_TRUE(db_->GetProperty(DB::Properties::kCFStats, &value_str));
|
||||
// To match historical speed, querying this property no longer triggers
|
||||
// a scan, even if results are old. But periodic dump stats should keep
|
||||
// things reasonably updated.
|
||||
ASSERT_EQ(scan_count, /*unchanged*/ 3);
|
||||
|
||||
SyncPoint::GetInstance()->DisableProcessing();
|
||||
SyncPoint::GetInstance()->ClearAllCallBacks();
|
||||
}
|
||||
EXPECT_GE(iterations_tested, 1);
|
||||
}
|
||||
|
@ -3600,6 +3600,41 @@ TEST_F(DBCompactionTest, CompactFilesOverlapInL0Bug) {
|
||||
ASSERT_EQ("new_val", Get(Key(0)));
|
||||
}
|
||||
|
||||
TEST_F(DBCompactionTest, DeleteFilesInRangeConflictWithCompaction) {
|
||||
Options options = CurrentOptions();
|
||||
DestroyAndReopen(options);
|
||||
const Snapshot* snapshot = nullptr;
|
||||
const int kMaxKey = 10;
|
||||
|
||||
for (int i = 0; i < kMaxKey; i++) {
|
||||
ASSERT_OK(Put(Key(i), Key(i)));
|
||||
ASSERT_OK(Delete(Key(i)));
|
||||
if (!snapshot) {
|
||||
snapshot = db_->GetSnapshot();
|
||||
}
|
||||
}
|
||||
ASSERT_OK(Flush());
|
||||
MoveFilesToLevel(1);
|
||||
ASSERT_OK(Put(Key(kMaxKey), Key(kMaxKey)));
|
||||
ASSERT_OK(dbfull()->TEST_WaitForCompact());
|
||||
// test DeleteFilesInRange() deletes the files already picked for compaction
|
||||
SyncPoint::GetInstance()->LoadDependency(
|
||||
{{"VersionSet::LogAndApply:WriteManifestStart",
|
||||
"BackgroundCallCompaction:0"},
|
||||
{"DBImpl::BackgroundCompaction:Finish",
|
||||
"VersionSet::LogAndApply:WriteManifestDone"}});
|
||||
SyncPoint::GetInstance()->EnableProcessing();
|
||||
|
||||
// release snapshot which mark bottommost file for compaction
|
||||
db_->ReleaseSnapshot(snapshot);
|
||||
std::string begin_string = Key(0);
|
||||
std::string end_string = Key(kMaxKey + 1);
|
||||
Slice begin(begin_string);
|
||||
Slice end(end_string);
|
||||
ASSERT_OK(DeleteFilesInRange(db_, db_->DefaultColumnFamily(), &begin, &end));
|
||||
SyncPoint::GetInstance()->DisableProcessing();
|
||||
}
|
||||
|
||||
TEST_F(DBCompactionTest, CompactBottomLevelFilesWithDeletions) {
|
||||
// bottom-level files may contain deletions due to snapshots protecting the
|
||||
// deleted keys. Once the snapshot is released, we should see files with many
|
||||
|
@ -63,6 +63,7 @@
|
||||
#include "memtable/hash_linklist_rep.h"
|
||||
#include "memtable/hash_skiplist_rep.h"
|
||||
#include "monitoring/in_memory_stats_history.h"
|
||||
#include "monitoring/instrumented_mutex.h"
|
||||
#include "monitoring/iostats_context_imp.h"
|
||||
#include "monitoring/perf_context_imp.h"
|
||||
#include "monitoring/persistent_stats_history.h"
|
||||
@ -911,18 +912,31 @@ void DBImpl::DumpStats() {
|
||||
if (shutdown_initiated_) {
|
||||
return;
|
||||
}
|
||||
|
||||
TEST_SYNC_POINT("DBImpl::DumpStats:StartRunning");
|
||||
{
|
||||
InstrumentedMutexLock l(&mutex_);
|
||||
for (auto cfd : *versions_->GetColumnFamilySet()) {
|
||||
if (cfd->initialized()) {
|
||||
// Release DB mutex for gathering cache entry stats. Pass over all
|
||||
// column families for this first so that other stats are dumped
|
||||
// near-atomically.
|
||||
InstrumentedMutexUnlock u(&mutex_);
|
||||
cfd->internal_stats()->CollectCacheEntryStats(/*foreground=*/false);
|
||||
}
|
||||
}
|
||||
|
||||
const std::string* property = &DB::Properties::kDBStats;
|
||||
const DBPropertyInfo* property_info = GetPropertyInfo(*property);
|
||||
assert(property_info != nullptr);
|
||||
assert(!property_info->need_out_of_mutex);
|
||||
default_cf_internal_stats_->GetStringProperty(*property_info, *property,
|
||||
&stats);
|
||||
|
||||
property = &DB::Properties::kCFStatsNoFileHistogram;
|
||||
property_info = GetPropertyInfo(*property);
|
||||
assert(property_info != nullptr);
|
||||
assert(!property_info->need_out_of_mutex);
|
||||
for (auto cfd : *versions_->GetColumnFamilySet()) {
|
||||
if (cfd->initialized()) {
|
||||
cfd->internal_stats()->GetStringProperty(*property_info, *property,
|
||||
@ -933,6 +947,7 @@ void DBImpl::DumpStats() {
|
||||
property = &DB::Properties::kCFFileHistogram;
|
||||
property_info = GetPropertyInfo(*property);
|
||||
assert(property_info != nullptr);
|
||||
assert(!property_info->need_out_of_mutex);
|
||||
for (auto cfd : *versions_->GetColumnFamilySet()) {
|
||||
if (cfd->initialized()) {
|
||||
cfd->internal_stats()->GetStringProperty(*property_info, *property,
|
||||
@ -3231,16 +3246,21 @@ bool DBImpl::GetProperty(ColumnFamilyHandle* column_family,
|
||||
}
|
||||
return ret_value;
|
||||
} else if (property_info->handle_string) {
|
||||
InstrumentedMutexLock l(&mutex_);
|
||||
return cfd->internal_stats()->GetStringProperty(*property_info, property,
|
||||
value);
|
||||
} else if (property_info->handle_string_dbimpl) {
|
||||
std::string tmp_value;
|
||||
bool ret_value = (this->*(property_info->handle_string_dbimpl))(&tmp_value);
|
||||
if (ret_value) {
|
||||
*value = tmp_value;
|
||||
if (property_info->need_out_of_mutex) {
|
||||
return cfd->internal_stats()->GetStringProperty(*property_info, property,
|
||||
value);
|
||||
} else {
|
||||
InstrumentedMutexLock l(&mutex_);
|
||||
return cfd->internal_stats()->GetStringProperty(*property_info, property,
|
||||
value);
|
||||
}
|
||||
} else if (property_info->handle_string_dbimpl) {
|
||||
if (property_info->need_out_of_mutex) {
|
||||
return (this->*(property_info->handle_string_dbimpl))(value);
|
||||
} else {
|
||||
InstrumentedMutexLock l(&mutex_);
|
||||
return (this->*(property_info->handle_string_dbimpl))(value);
|
||||
}
|
||||
return ret_value;
|
||||
}
|
||||
// Shouldn't reach here since exactly one of handle_string and handle_int
|
||||
// should be non-nullptr.
|
||||
@ -3258,9 +3278,14 @@ bool DBImpl::GetMapProperty(ColumnFamilyHandle* column_family,
|
||||
if (property_info == nullptr) {
|
||||
return false;
|
||||
} else if (property_info->handle_map) {
|
||||
InstrumentedMutexLock l(&mutex_);
|
||||
return cfd->internal_stats()->GetMapProperty(*property_info, property,
|
||||
value);
|
||||
if (property_info->need_out_of_mutex) {
|
||||
return cfd->internal_stats()->GetMapProperty(*property_info, property,
|
||||
value);
|
||||
} else {
|
||||
InstrumentedMutexLock l(&mutex_);
|
||||
return cfd->internal_stats()->GetMapProperty(*property_info, property,
|
||||
value);
|
||||
}
|
||||
}
|
||||
// If we reach this point it means that handle_map is not provided for the
|
||||
// requested property
|
||||
@ -3709,6 +3734,8 @@ Status DBImpl::DeleteFilesInRanges(ColumnFamilyHandle* column_family,
|
||||
deleted_files.insert(level_file);
|
||||
level_file->being_compacted = true;
|
||||
}
|
||||
vstorage->ComputeCompactionScore(*cfd->ioptions(),
|
||||
*cfd->GetLatestMutableCFOptions());
|
||||
}
|
||||
}
|
||||
if (edit.GetDeletedFiles().empty()) {
|
||||
|
@ -251,6 +251,7 @@ Status DBImpl::WriteImpl(const WriteOptions& write_options,
|
||||
write_thread_.EnterAsBatchGroupLeader(&w, &write_group);
|
||||
|
||||
IOStatus io_s;
|
||||
Status pre_release_cb_status;
|
||||
if (status.ok()) {
|
||||
// Rules for when we can update the memtable concurrently
|
||||
// 1. supported by memtable
|
||||
@ -361,7 +362,7 @@ Status DBImpl::WriteImpl(const WriteOptions& write_options,
|
||||
writer->sequence, disable_memtable, writer->log_used, index++,
|
||||
pre_release_callback_cnt);
|
||||
if (!ws.ok()) {
|
||||
status = ws;
|
||||
status = pre_release_cb_status = ws;
|
||||
break;
|
||||
}
|
||||
}
|
||||
@ -414,10 +415,13 @@ Status DBImpl::WriteImpl(const WriteOptions& write_options,
|
||||
|
||||
if (!w.CallbackFailed()) {
|
||||
if (!io_s.ok()) {
|
||||
assert(pre_release_cb_status.ok());
|
||||
IOStatusCheck(io_s);
|
||||
} else {
|
||||
WriteStatusCheck(status);
|
||||
WriteStatusCheck(pre_release_cb_status);
|
||||
}
|
||||
} else {
|
||||
assert(io_s.ok() && pre_release_cb_status.ok());
|
||||
}
|
||||
|
||||
if (need_log_sync) {
|
||||
|
@ -394,7 +394,7 @@ const std::unordered_map<std::string, DBPropertyInfo>
|
||||
{DB::Properties::kDBStats,
|
||||
{false, &InternalStats::HandleDBStats, nullptr, nullptr, nullptr}},
|
||||
{DB::Properties::kBlockCacheEntryStats,
|
||||
{false, &InternalStats::HandleBlockCacheEntryStats, nullptr,
|
||||
{true, &InternalStats::HandleBlockCacheEntryStats, nullptr,
|
||||
&InternalStats::HandleBlockCacheEntryStatsMap, nullptr}},
|
||||
{DB::Properties::kSSTables,
|
||||
{false, &InternalStats::HandleSsTables, nullptr, nullptr, nullptr}},
|
||||
@ -510,7 +510,7 @@ const std::unordered_map<std::string, DBPropertyInfo>
|
||||
{false, nullptr, &InternalStats::HandleBlockCachePinnedUsage, nullptr,
|
||||
nullptr}},
|
||||
{DB::Properties::kOptionsStatistics,
|
||||
{false, nullptr, nullptr, nullptr,
|
||||
{true, nullptr, nullptr, nullptr,
|
||||
&DBImpl::GetPropertyHandleOptionsStatistics}},
|
||||
};
|
||||
|
||||
@ -526,29 +526,41 @@ InternalStats::InternalStats(int num_levels, SystemClock* clock,
|
||||
number_levels_(num_levels),
|
||||
clock_(clock),
|
||||
cfd_(cfd),
|
||||
started_at_(clock->NowMicros()) {}
|
||||
|
||||
Status InternalStats::CollectCacheEntryStats(bool foreground) {
|
||||
// Lazy initialize/reference the collector. It is pinned in cache (through
|
||||
// a shared_ptr) so that it does not get immediately ejected from a full
|
||||
// cache, which would force a re-scan on the next GetStats.
|
||||
if (!cache_entry_stats_collector_) {
|
||||
Cache* block_cache;
|
||||
bool ok = HandleBlockCacheStat(&block_cache);
|
||||
if (ok) {
|
||||
// Extract or create stats collector.
|
||||
Status s = CacheEntryStatsCollector<CacheEntryRoleStats>::GetShared(
|
||||
block_cache, clock_, &cache_entry_stats_collector_);
|
||||
if (!s.ok()) {
|
||||
// Block cache likely under pressure. Scanning could make it worse,
|
||||
// so skip.
|
||||
return s;
|
||||
}
|
||||
started_at_(clock->NowMicros()) {
|
||||
Cache* block_cache = nullptr;
|
||||
bool ok = GetBlockCacheForStats(&block_cache);
|
||||
if (ok) {
|
||||
assert(block_cache);
|
||||
// Extract or create stats collector. Could fail in rare cases.
|
||||
Status s = CacheEntryStatsCollector<CacheEntryRoleStats>::GetShared(
|
||||
block_cache, clock_, &cache_entry_stats_collector_);
|
||||
if (s.ok()) {
|
||||
assert(cache_entry_stats_collector_);
|
||||
} else {
|
||||
return Status::NotFound("block cache not configured");
|
||||
assert(!cache_entry_stats_collector_);
|
||||
}
|
||||
} else {
|
||||
assert(!block_cache);
|
||||
}
|
||||
}
|
||||
|
||||
void InternalStats::TEST_GetCacheEntryRoleStats(CacheEntryRoleStats* stats,
|
||||
bool foreground) {
|
||||
CollectCacheEntryStats(foreground);
|
||||
if (cache_entry_stats_collector_) {
|
||||
cache_entry_stats_collector_->GetStats(stats);
|
||||
}
|
||||
}
|
||||
|
||||
void InternalStats::CollectCacheEntryStats(bool foreground) {
|
||||
// This function is safe to call from any thread because
|
||||
// cache_entry_stats_collector_ field is const after constructor
|
||||
// and ->GetStats does its own synchronization, which also suffices for
|
||||
// cache_entry_stats_.
|
||||
|
||||
if (!cache_entry_stats_collector_) {
|
||||
return; // nothing to do (e.g. no block cache)
|
||||
}
|
||||
assert(cache_entry_stats_collector_);
|
||||
|
||||
// For "background" collections, strictly cap the collection time by
|
||||
// expanding effective cache TTL. For foreground, be more aggressive about
|
||||
@ -556,9 +568,8 @@ Status InternalStats::CollectCacheEntryStats(bool foreground) {
|
||||
int min_interval_seconds = foreground ? 10 : 180;
|
||||
// 1/500 = max of 0.2% of one CPU thread
|
||||
int min_interval_factor = foreground ? 10 : 500;
|
||||
cache_entry_stats_collector_->GetStats(
|
||||
&cache_entry_stats_, min_interval_seconds, min_interval_factor);
|
||||
return Status::OK();
|
||||
cache_entry_stats_collector_->CollectStats(min_interval_seconds,
|
||||
min_interval_factor);
|
||||
}
|
||||
|
||||
std::function<void(const Slice&, void*, size_t, Cache::DeleterFn)>
|
||||
@ -649,21 +660,25 @@ void InternalStats::CacheEntryRoleStats::ToMap(
|
||||
|
||||
bool InternalStats::HandleBlockCacheEntryStats(std::string* value,
|
||||
Slice /*suffix*/) {
|
||||
Status s = CollectCacheEntryStats(/*foreground*/ true);
|
||||
if (!s.ok()) {
|
||||
if (!cache_entry_stats_collector_) {
|
||||
return false;
|
||||
}
|
||||
*value = cache_entry_stats_.ToString(clock_);
|
||||
CollectCacheEntryStats(/*foreground*/ true);
|
||||
CacheEntryRoleStats stats;
|
||||
cache_entry_stats_collector_->GetStats(&stats);
|
||||
*value = stats.ToString(clock_);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool InternalStats::HandleBlockCacheEntryStatsMap(
|
||||
std::map<std::string, std::string>* values, Slice /*suffix*/) {
|
||||
Status s = CollectCacheEntryStats(/*foreground*/ true);
|
||||
if (!s.ok()) {
|
||||
if (!cache_entry_stats_collector_) {
|
||||
return false;
|
||||
}
|
||||
cache_entry_stats_.ToMap(values, clock_);
|
||||
CollectCacheEntryStats(/*foreground*/ true);
|
||||
CacheEntryRoleStats stats;
|
||||
cache_entry_stats_collector_->GetStats(&stats);
|
||||
stats.ToMap(values, clock_);
|
||||
return true;
|
||||
}
|
||||
|
||||
@ -1123,7 +1138,7 @@ bool InternalStats::HandleEstimateOldestKeyTime(uint64_t* value, DBImpl* /*db*/,
|
||||
return *value > 0 && *value < std::numeric_limits<uint64_t>::max();
|
||||
}
|
||||
|
||||
bool InternalStats::HandleBlockCacheStat(Cache** block_cache) {
|
||||
bool InternalStats::GetBlockCacheForStats(Cache** block_cache) {
|
||||
assert(block_cache != nullptr);
|
||||
auto* table_factory = cfd_->ioptions()->table_factory.get();
|
||||
assert(table_factory != nullptr);
|
||||
@ -1135,7 +1150,7 @@ bool InternalStats::HandleBlockCacheStat(Cache** block_cache) {
|
||||
bool InternalStats::HandleBlockCacheCapacity(uint64_t* value, DBImpl* /*db*/,
|
||||
Version* /*version*/) {
|
||||
Cache* block_cache;
|
||||
bool ok = HandleBlockCacheStat(&block_cache);
|
||||
bool ok = GetBlockCacheForStats(&block_cache);
|
||||
if (!ok) {
|
||||
return false;
|
||||
}
|
||||
@ -1146,7 +1161,7 @@ bool InternalStats::HandleBlockCacheCapacity(uint64_t* value, DBImpl* /*db*/,
|
||||
bool InternalStats::HandleBlockCacheUsage(uint64_t* value, DBImpl* /*db*/,
|
||||
Version* /*version*/) {
|
||||
Cache* block_cache;
|
||||
bool ok = HandleBlockCacheStat(&block_cache);
|
||||
bool ok = GetBlockCacheForStats(&block_cache);
|
||||
if (!ok) {
|
||||
return false;
|
||||
}
|
||||
@ -1157,7 +1172,7 @@ bool InternalStats::HandleBlockCacheUsage(uint64_t* value, DBImpl* /*db*/,
|
||||
bool InternalStats::HandleBlockCachePinnedUsage(uint64_t* value, DBImpl* /*db*/,
|
||||
Version* /*version*/) {
|
||||
Cache* block_cache;
|
||||
bool ok = HandleBlockCacheStat(&block_cache);
|
||||
bool ok = GetBlockCacheForStats(&block_cache);
|
||||
if (!ok) {
|
||||
return false;
|
||||
}
|
||||
@ -1504,7 +1519,8 @@ void InternalStats::DumpCFStatsNoFileHistogram(std::string* value) {
|
||||
vstorage->GetTotalBlobFileSize() / kGB);
|
||||
value->append(buf);
|
||||
|
||||
double seconds_up = (clock_->NowMicros() - started_at_ + 1) / kMicrosInSec;
|
||||
uint64_t now_micros = clock_->NowMicros();
|
||||
double seconds_up = (now_micros - started_at_ + 1) / kMicrosInSec;
|
||||
double interval_seconds_up = seconds_up - cf_stats_snapshot_.seconds_up;
|
||||
snprintf(buf, sizeof(buf), "Uptime(secs): %.1f total, %.1f interval\n",
|
||||
seconds_up, interval_seconds_up);
|
||||
@ -1619,14 +1635,20 @@ void InternalStats::DumpCFStatsNoFileHistogram(std::string* value) {
|
||||
cf_stats_snapshot_.comp_stats = compaction_stats_sum;
|
||||
cf_stats_snapshot_.stall_count = total_stall_count;
|
||||
|
||||
// Always treat CFStats context as "background"
|
||||
Status s = CollectCacheEntryStats(/*foreground=*/false);
|
||||
if (s.ok()) {
|
||||
value->append(cache_entry_stats_.ToString(clock_));
|
||||
} else {
|
||||
value->append("Block cache: ");
|
||||
value->append(s.ToString());
|
||||
value->append("\n");
|
||||
// Do not gather cache entry stats during CFStats because DB
|
||||
// mutex is held. Only dump last cached collection (rely on DB
|
||||
// periodic stats dump to update)
|
||||
if (cache_entry_stats_collector_) {
|
||||
CacheEntryRoleStats stats;
|
||||
// thread safe
|
||||
cache_entry_stats_collector_->GetStats(&stats);
|
||||
|
||||
constexpr uint64_t kDayInMicros = uint64_t{86400} * 1000000U;
|
||||
|
||||
// Skip if stats are extremely old (> 1 day, incl not yet populated)
|
||||
if (now_micros - stats.last_end_time_micros_ < kDayInMicros) {
|
||||
value->append(stats.ToString(clock_));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -392,7 +392,6 @@ class InternalStats {
|
||||
cf_stats_count_[i] = 0;
|
||||
cf_stats_value_[i] = 0;
|
||||
}
|
||||
cache_entry_stats_.Clear();
|
||||
for (auto& comp_stat : comp_stats_) {
|
||||
comp_stat.Clear();
|
||||
}
|
||||
@ -459,20 +458,20 @@ class InternalStats {
|
||||
bool GetIntPropertyOutOfMutex(const DBPropertyInfo& property_info,
|
||||
Version* version, uint64_t* value);
|
||||
|
||||
// Unless there is a recent enough collection of the stats, collect and
|
||||
// saved new cache entry stats. If `foreground`, require data to be more
|
||||
// recent to skip re-collection.
|
||||
//
|
||||
// This should only be called while NOT holding the DB mutex.
|
||||
void CollectCacheEntryStats(bool foreground);
|
||||
|
||||
const uint64_t* TEST_GetCFStatsValue() const { return cf_stats_value_; }
|
||||
|
||||
const std::vector<CompactionStats>& TEST_GetCompactionStats() const {
|
||||
return comp_stats_;
|
||||
}
|
||||
|
||||
const CacheEntryRoleStats& TEST_GetCacheEntryRoleStats(bool foreground) {
|
||||
Status s = CollectCacheEntryStats(foreground);
|
||||
if (!s.ok()) {
|
||||
assert(false);
|
||||
cache_entry_stats_.Clear();
|
||||
}
|
||||
return cache_entry_stats_;
|
||||
}
|
||||
void TEST_GetCacheEntryRoleStats(CacheEntryRoleStats* stats, bool foreground);
|
||||
|
||||
// Store a mapping from the user-facing DB::Properties string to our
|
||||
// DBPropertyInfo struct used internally for retrieving properties.
|
||||
@ -492,16 +491,18 @@ class InternalStats {
|
||||
void DumpCFStatsNoFileHistogram(std::string* value);
|
||||
void DumpCFFileHistogram(std::string* value);
|
||||
|
||||
bool HandleBlockCacheStat(Cache** block_cache);
|
||||
|
||||
Status CollectCacheEntryStats(bool foreground);
|
||||
bool GetBlockCacheForStats(Cache** block_cache);
|
||||
|
||||
// Per-DB stats
|
||||
std::atomic<uint64_t> db_stats_[kIntStatsNumMax];
|
||||
// Per-ColumnFamily stats
|
||||
uint64_t cf_stats_value_[INTERNAL_CF_STATS_ENUM_MAX];
|
||||
uint64_t cf_stats_count_[INTERNAL_CF_STATS_ENUM_MAX];
|
||||
CacheEntryRoleStats cache_entry_stats_;
|
||||
// Initialize/reference the collector in constructor so that we don't need
|
||||
// additional synchronization in InternalStats, relying on synchronization
|
||||
// in CacheEntryStatsCollector::GetStats. This collector is pinned in cache
|
||||
// (through a shared_ptr) so that it does not get immediately ejected from
|
||||
// a full cache, which would force a re-scan on the next GetStats.
|
||||
std::shared_ptr<CacheEntryStatsCollector<CacheEntryRoleStats>>
|
||||
cache_entry_stats_collector_;
|
||||
// Per-ColumnFamily/level compaction stats
|
||||
|
@ -4092,7 +4092,7 @@ Status VersionSet::ProcessManifestWrites(
|
||||
{
|
||||
FileOptions opt_file_opts = fs_->OptimizeForManifestWrite(file_options_);
|
||||
mu->Unlock();
|
||||
|
||||
TEST_SYNC_POINT("VersionSet::LogAndApply:WriteManifestStart");
|
||||
TEST_SYNC_POINT_CALLBACK("VersionSet::LogAndApply:WriteManifest", nullptr);
|
||||
if (!first_writer.edit_list.front()->IsColumnFamilyManipulation()) {
|
||||
for (int i = 0; i < static_cast<int>(versions.size()); ++i) {
|
||||
@ -5590,6 +5590,9 @@ void VersionSet::GetLiveFilesMetaData(std::vector<LiveFileMetaData>* metadata) {
|
||||
filemetadata.oldest_blob_file_number = file->oldest_blob_file_number;
|
||||
filemetadata.file_checksum = file->file_checksum;
|
||||
filemetadata.file_checksum_func_name = file->file_checksum_func_name;
|
||||
filemetadata.temperature = file->temperature;
|
||||
filemetadata.oldest_ancester_time = file->TryGetOldestAncesterTime();
|
||||
filemetadata.file_creation_time = file->TryGetFileCreationTime();
|
||||
metadata->push_back(filemetadata);
|
||||
}
|
||||
}
|
||||
|
@ -10,7 +10,6 @@
|
||||
|
||||
#include <string>
|
||||
|
||||
#include "rocksdb/customizable.h"
|
||||
#include "rocksdb/rocksdb_namespace.h"
|
||||
|
||||
namespace ROCKSDB_NAMESPACE {
|
||||
@ -21,7 +20,7 @@ class Slice;
|
||||
// used as keys in an sstable or a database. A Comparator implementation
|
||||
// must be thread-safe since rocksdb may invoke its methods concurrently
|
||||
// from multiple threads.
|
||||
class Comparator : public Customizable {
|
||||
class Comparator {
|
||||
public:
|
||||
Comparator() : timestamp_size_(0) {}
|
||||
|
||||
@ -38,11 +37,7 @@ class Comparator : public Customizable {
|
||||
|
||||
virtual ~Comparator() {}
|
||||
|
||||
static Status CreateFromString(const ConfigOptions& opts,
|
||||
const std::string& id,
|
||||
const Comparator** comp);
|
||||
static const char* Type() { return "Comparator"; }
|
||||
|
||||
// Three-way comparison. Returns value:
|
||||
// < 0 iff "a" < "b",
|
||||
// == 0 iff "a" == "b",
|
||||
|
@ -8,7 +8,6 @@
|
||||
|
||||
#pragma once
|
||||
|
||||
#include <atomic>
|
||||
#include <string>
|
||||
#include <unordered_map>
|
||||
#include <unordered_set>
|
||||
@ -270,7 +269,7 @@ class Configurable {
|
||||
protected:
|
||||
// True once the object is prepared. Once the object is prepared, only
|
||||
// mutable options can be configured.
|
||||
std::atomic<bool> prepared_;
|
||||
bool prepared_;
|
||||
|
||||
// Returns the raw pointer for the associated named option.
|
||||
// The name is typically the name of an option registered via the
|
||||
|
@ -578,7 +578,7 @@ class Statistics {
|
||||
// Resets all ticker and histogram stats
|
||||
virtual Status Reset() { return Status::NotSupported("Not implemented"); }
|
||||
|
||||
// String representation of the statistic object.
|
||||
// String representation of the statistic object. Must be thread-safe.
|
||||
virtual std::string ToString() const {
|
||||
// Do nothing by default
|
||||
return std::string("ToString(): not implemented");
|
||||
|
@ -35,6 +35,7 @@ enum class OptionType {
|
||||
kCompactionPri,
|
||||
kSliceTransform,
|
||||
kCompressionType,
|
||||
kComparator,
|
||||
kCompactionFilter,
|
||||
kCompactionFilterFactory,
|
||||
kCompactionStopStyle,
|
||||
|
@ -10,8 +10,8 @@
|
||||
#include "rocksdb/rocksdb_namespace.h"
|
||||
|
||||
#define ROCKSDB_MAJOR 6
|
||||
#define ROCKSDB_MINOR 21
|
||||
#define ROCKSDB_PATCH 0
|
||||
#define ROCKSDB_MINOR 22
|
||||
#define ROCKSDB_PATCH 3
|
||||
|
||||
// Do not use these. We made the mistake of declaring macros starting with
|
||||
// double underscore. Now we have to live with our choice. We'll deprecate these
|
||||
|
@ -58,7 +58,8 @@ public class MemoryUtilTest {
|
||||
db.getAggregatedLongProperty(UNFLUSHED_MEMTABLE_SIZE));
|
||||
assertThat(usage.get(MemoryUsageType.kTableReadersTotal)).isEqualTo(
|
||||
db.getAggregatedLongProperty(TABLE_READERS));
|
||||
assertThat(usage.get(MemoryUsageType.kCacheTotal)).isEqualTo(0);
|
||||
// TODO(peterd): disable block cache entry stats and check for 0
|
||||
assertThat(usage.get(MemoryUsageType.kCacheTotal)).isLessThan(1024);
|
||||
|
||||
db.put(key, value);
|
||||
db.flush(flushOptions);
|
||||
|
@ -51,8 +51,7 @@ class InstrumentedMutex {
|
||||
int stats_code_;
|
||||
};
|
||||
|
||||
// A wrapper class for port::Mutex that provides additional layer
|
||||
// for collecting stats and instrumentation.
|
||||
// RAII wrapper for InstrumentedMutex
|
||||
class InstrumentedMutexLock {
|
||||
public:
|
||||
explicit InstrumentedMutexLock(InstrumentedMutex* mutex) : mutex_(mutex) {
|
||||
@ -69,6 +68,22 @@ class InstrumentedMutexLock {
|
||||
void operator=(const InstrumentedMutexLock&) = delete;
|
||||
};
|
||||
|
||||
// RAII wrapper for temporary releasing InstrumentedMutex inside
|
||||
// InstrumentedMutexLock
|
||||
class InstrumentedMutexUnlock {
|
||||
public:
|
||||
explicit InstrumentedMutexUnlock(InstrumentedMutex* mutex) : mutex_(mutex) {
|
||||
mutex_->Unlock();
|
||||
}
|
||||
|
||||
~InstrumentedMutexUnlock() { mutex_->Lock(); }
|
||||
|
||||
private:
|
||||
InstrumentedMutex* const mutex_;
|
||||
InstrumentedMutexUnlock(const InstrumentedMutexUnlock&) = delete;
|
||||
void operator=(const InstrumentedMutexUnlock&) = delete;
|
||||
};
|
||||
|
||||
class InstrumentedCondVar {
|
||||
public:
|
||||
explicit InstrumentedCondVar(InstrumentedMutex* instrumented_mutex)
|
||||
|
@ -535,30 +535,22 @@ static std::unordered_map<std::string, OptionTypeInfo>
|
||||
OptionVerificationType::kNormal, OptionTypeFlags::kNone,
|
||||
{0, OptionType::kCompressionType})},
|
||||
{"comparator",
|
||||
OptionTypeInfo::AsCustomRawPtr<const Comparator>(
|
||||
offset_of(&ImmutableCFOptions::user_comparator),
|
||||
OptionVerificationType::kByName, OptionTypeFlags::kCompareLoose,
|
||||
// Serializes a Comparator
|
||||
[](const ConfigOptions& /*opts*/, const std::string&,
|
||||
const void* addr, std::string* value) {
|
||||
// it's a const pointer of const Comparator*
|
||||
const auto* ptr = static_cast<const Comparator* const*>(addr);
|
||||
|
||||
// Since the user-specified comparator will be wrapped by
|
||||
// InternalKeyComparator, we should persist the user-specified
|
||||
// one instead of InternalKeyComparator.
|
||||
if (*ptr == nullptr) {
|
||||
*value = kNullptrString;
|
||||
} else {
|
||||
const Comparator* root_comp = (*ptr)->GetRootComparator();
|
||||
if (root_comp == nullptr) {
|
||||
root_comp = (*ptr);
|
||||
}
|
||||
*value = root_comp->Name();
|
||||
}
|
||||
return Status::OK();
|
||||
},
|
||||
/* Use the default match function*/ nullptr)},
|
||||
{offset_of(&ImmutableCFOptions::user_comparator),
|
||||
OptionType::kComparator, OptionVerificationType::kByName,
|
||||
OptionTypeFlags::kCompareLoose,
|
||||
// Parses the string and sets the corresponding comparator
|
||||
[](const ConfigOptions& opts, const std::string& /*name*/,
|
||||
const std::string& value, void* addr) {
|
||||
auto old_comparator = static_cast<const Comparator**>(addr);
|
||||
const Comparator* new_comparator = *old_comparator;
|
||||
Status status =
|
||||
opts.registry->NewStaticObject(value, &new_comparator);
|
||||
if (status.ok()) {
|
||||
*old_comparator = new_comparator;
|
||||
return status;
|
||||
}
|
||||
return Status::OK();
|
||||
}}},
|
||||
{"memtable_insert_with_hint_prefix_extractor",
|
||||
{offset_of(
|
||||
&ImmutableCFOptions::memtable_insert_with_hint_prefix_extractor),
|
||||
|
@ -771,15 +771,6 @@ static int RegisterTestObjects(ObjectLibrary& library,
|
||||
guard->reset(new mock::MockTableFactory());
|
||||
return guard->get();
|
||||
});
|
||||
library.Register<const Comparator>(
|
||||
test::SimpleSuffixReverseComparator::kClassName(),
|
||||
[](const std::string& /*uri*/,
|
||||
std::unique_ptr<const Comparator>* /*guard*/,
|
||||
std::string* /* errmsg */) {
|
||||
static test::SimpleSuffixReverseComparator ssrc;
|
||||
return &ssrc;
|
||||
});
|
||||
|
||||
return static_cast<int>(library.GetFactoryCount(&num_types));
|
||||
}
|
||||
|
||||
@ -789,7 +780,6 @@ static int RegisterLocalObjects(ObjectLibrary& library,
|
||||
// Load any locally defined objects here
|
||||
return static_cast<int>(library.GetFactoryCount(&num_types));
|
||||
}
|
||||
#endif // !ROCKSDB_LITE
|
||||
|
||||
class LoadCustomizableTest : public testing::Test {
|
||||
public:
|
||||
@ -829,31 +819,7 @@ TEST_F(LoadCustomizableTest, LoadTableFactoryTest) {
|
||||
ASSERT_STREQ(factory->Name(), "MockTable");
|
||||
}
|
||||
}
|
||||
|
||||
TEST_F(LoadCustomizableTest, LoadComparatorTest) {
|
||||
const Comparator* bytewise = BytewiseComparator();
|
||||
const Comparator* reverse = ReverseBytewiseComparator();
|
||||
|
||||
const Comparator* result = nullptr;
|
||||
ASSERT_NOK(Comparator::CreateFromString(
|
||||
config_options_, test::SimpleSuffixReverseComparator::kClassName(),
|
||||
&result));
|
||||
ASSERT_OK(
|
||||
Comparator::CreateFromString(config_options_, bytewise->Name(), &result));
|
||||
ASSERT_EQ(result, bytewise);
|
||||
ASSERT_OK(
|
||||
Comparator::CreateFromString(config_options_, reverse->Name(), &result));
|
||||
ASSERT_EQ(result, reverse);
|
||||
|
||||
if (RegisterTests("Test")) {
|
||||
ASSERT_OK(Comparator::CreateFromString(
|
||||
config_options_, test::SimpleSuffixReverseComparator::kClassName(),
|
||||
&result));
|
||||
ASSERT_NE(result, nullptr);
|
||||
ASSERT_STREQ(result->Name(),
|
||||
test::SimpleSuffixReverseComparator::kClassName());
|
||||
}
|
||||
}
|
||||
#endif // !ROCKSDB_LITE
|
||||
|
||||
} // namespace ROCKSDB_NAMESPACE
|
||||
int main(int argc, char** argv) {
|
||||
|
@ -562,6 +562,23 @@ bool SerializeSingleOptionHelper(const void* opt_address,
|
||||
: kNullptrString;
|
||||
break;
|
||||
}
|
||||
case OptionType::kComparator: {
|
||||
// it's a const pointer of const Comparator*
|
||||
const auto* ptr = static_cast<const Comparator* const*>(opt_address);
|
||||
// Since the user-specified comparator will be wrapped by
|
||||
// InternalKeyComparator, we should persist the user-specified one
|
||||
// instead of InternalKeyComparator.
|
||||
if (*ptr == nullptr) {
|
||||
*value = kNullptrString;
|
||||
} else {
|
||||
const Comparator* root_comp = (*ptr)->GetRootComparator();
|
||||
if (root_comp == nullptr) {
|
||||
root_comp = (*ptr);
|
||||
}
|
||||
*value = root_comp->Name();
|
||||
}
|
||||
break;
|
||||
}
|
||||
case OptionType::kCompactionFilter: {
|
||||
// it's a const pointer of const CompactionFilter*
|
||||
const auto* ptr =
|
||||
|
17
port/lang.h
17
port/lang.h
@ -15,6 +15,8 @@
|
||||
#endif
|
||||
#endif
|
||||
|
||||
// ASAN (Address sanitizer)
|
||||
|
||||
#if defined(__clang__)
|
||||
#if defined(__has_feature)
|
||||
#if __has_feature(address_sanitizer)
|
||||
@ -39,3 +41,18 @@
|
||||
#else
|
||||
#define STATIC_AVOID_DESTRUCTION(Type, name) static Type& name = *new Type
|
||||
#endif
|
||||
|
||||
// TSAN (Thread sanitizer)
|
||||
|
||||
// For simplicity, standardize on the GCC define
|
||||
#if defined(__clang__)
|
||||
#if defined(__has_feature) && __has_feature(thread_sanitizer)
|
||||
#define __SANITIZE_THREAD__ 1
|
||||
#endif // __has_feature(thread_sanitizer)
|
||||
#endif // __clang__
|
||||
|
||||
#ifdef __SANITIZE_THREAD__
|
||||
#define TSAN_SUPPRESSION __attribute__((no_sanitize("thread")))
|
||||
#else
|
||||
#define TSAN_SUPPRESSION
|
||||
#endif // TSAN_SUPPRESSION
|
||||
|
@ -36,6 +36,8 @@ void* SaveStack(int* /*num_frames*/, int /*first_frames_to_skip*/) {
|
||||
#include <sys/sysctl.h>
|
||||
#endif
|
||||
|
||||
#include "port/lang.h"
|
||||
|
||||
namespace ROCKSDB_NAMESPACE {
|
||||
namespace port {
|
||||
|
||||
@ -163,8 +165,7 @@ static void StackTraceHandler(int sig) {
|
||||
|
||||
// Efforts to fix or suppress TSAN warnings "signal-unsafe call inside of
|
||||
// a signal" have failed, so just warn the user about them.
|
||||
#if defined(__clang__) && defined(__has_feature)
|
||||
#if __has_feature(thread_sanitizer)
|
||||
#ifdef __SANITIZE_THREAD__
|
||||
fprintf(stderr,
|
||||
"==> NOTE: any above warnings about \"signal-unsafe call\" are\n"
|
||||
"==> ignorable, as they are expected when generating a stack\n"
|
||||
@ -173,7 +174,6 @@ static void StackTraceHandler(int sig) {
|
||||
"==> in the TSAN warning can be useful for that. (The stack\n"
|
||||
"==> trace printed by the signal handler is likely obscured\n"
|
||||
"==> by TSAN output.)\n");
|
||||
#endif
|
||||
#endif
|
||||
|
||||
// re-signal to default handler (so we still get core dump if needed...)
|
||||
|
@ -2564,24 +2564,21 @@ void BlockBasedTable::MultiGet(const ReadOptions& read_options,
|
||||
s = Status::OK();
|
||||
}
|
||||
if (s.ok() && !results.back().IsEmpty()) {
|
||||
if (results.back().IsReady()) {
|
||||
// Since we have a valid handle, check the value. If its nullptr,
|
||||
// it means the cache is waiting for the final result and we're
|
||||
// supposed to call WaitAll() to wait for the result.
|
||||
if (results.back().GetValue() != nullptr) {
|
||||
// Found it in the cache. Add NULL handle to indicate there is
|
||||
// nothing to read from disk.
|
||||
if (results.back().GetCacheHandle()) {
|
||||
results.back().UpdateCachedValue();
|
||||
// Its possible the cache lookup returned a non-null handle,
|
||||
// but the lookup actually failed to produce a valid value
|
||||
if (results.back().GetValue() == nullptr) {
|
||||
block_handles.emplace_back(handle);
|
||||
total_len += block_size(handle);
|
||||
}
|
||||
}
|
||||
if (results.back().GetValue() != nullptr) {
|
||||
block_handles.emplace_back(BlockHandle::NullBlockHandle());
|
||||
}
|
||||
block_handles.emplace_back(BlockHandle::NullBlockHandle());
|
||||
} else {
|
||||
// We have to wait for the asynchronous cache lookup to finish,
|
||||
// and then we may have to read the block from disk anyway
|
||||
// We have to wait for the cache lookup to finish in the
|
||||
// background, and then we may have to read the block from disk
|
||||
// anyway
|
||||
assert(results.back().GetCacheHandle());
|
||||
wait_for_cache_results = true;
|
||||
block_handles.emplace_back(handle);
|
||||
cache_handles.emplace_back(results.back().GetCacheHandle());
|
||||
|
@ -98,8 +98,10 @@ class PlainInternalKeyComparator : public InternalKeyComparator {
|
||||
class SimpleSuffixReverseComparator : public Comparator {
|
||||
public:
|
||||
SimpleSuffixReverseComparator() {}
|
||||
static const char* kClassName() { return "SimpleSuffixReverseComparator"; }
|
||||
virtual const char* Name() const override { return kClassName(); }
|
||||
|
||||
virtual const char* Name() const override {
|
||||
return "SimpleSuffixReverseComparator";
|
||||
}
|
||||
|
||||
virtual int Compare(const Slice& a, const Slice& b) const override {
|
||||
Slice prefix_a = Slice(a.data(), 8);
|
||||
|
@ -9,6 +9,7 @@
|
||||
#include <fstream>
|
||||
|
||||
#include "monitoring/instrumented_mutex.h"
|
||||
#include "port/lang.h"
|
||||
#include "rocksdb/file_system.h"
|
||||
#include "rocksdb/options.h"
|
||||
#include "trace_replay/trace_replay.h"
|
||||
@ -157,20 +158,6 @@ class IOTracer {
|
||||
// mutex and ignore the operation if writer_is null. So its ok if
|
||||
// tracing_enabled shows non updated value.
|
||||
|
||||
#if defined(__clang__)
|
||||
#if defined(__has_feature) && __has_feature(thread_sanitizer)
|
||||
#define TSAN_SUPPRESSION __attribute__((no_sanitize("thread")))
|
||||
#endif // __has_feature(thread_sanitizer)
|
||||
#else // __clang__
|
||||
#ifdef __SANITIZE_THREAD__
|
||||
#define TSAN_SUPPRESSION __attribute__((no_sanitize("thread")))
|
||||
#endif // __SANITIZE_THREAD__
|
||||
#endif // __clang__
|
||||
|
||||
#ifndef TSAN_SUPPRESSION
|
||||
#define TSAN_SUPPRESSION
|
||||
#endif // TSAN_SUPPRESSION
|
||||
|
||||
// Start writing IO operations to the trace_writer.
|
||||
TSAN_SUPPRESSION Status
|
||||
StartIOTrace(SystemClock* clock, const TraceOptions& trace_options,
|
||||
|
@ -8,17 +8,11 @@
|
||||
// found in the LICENSE file. See the AUTHORS file for names of contributors.
|
||||
|
||||
#include "rocksdb/comparator.h"
|
||||
|
||||
#include <stdint.h>
|
||||
|
||||
#include <algorithm>
|
||||
#include <memory>
|
||||
#include <mutex>
|
||||
|
||||
#include "options/configurable_helper.h"
|
||||
#include "port/port.h"
|
||||
#include "rocksdb/slice.h"
|
||||
#include "rocksdb/utilities/object_registry.h"
|
||||
|
||||
namespace ROCKSDB_NAMESPACE {
|
||||
|
||||
@ -26,8 +20,8 @@ namespace {
|
||||
class BytewiseComparatorImpl : public Comparator {
|
||||
public:
|
||||
BytewiseComparatorImpl() { }
|
||||
static const char* kClassName() { return "leveldb.BytewiseComparator"; }
|
||||
const char* Name() const override { return kClassName(); }
|
||||
|
||||
const char* Name() const override { return "leveldb.BytewiseComparator"; }
|
||||
|
||||
int Compare(const Slice& a, const Slice& b) const override {
|
||||
return a.compare(b);
|
||||
@ -145,10 +139,9 @@ class ReverseBytewiseComparatorImpl : public BytewiseComparatorImpl {
|
||||
public:
|
||||
ReverseBytewiseComparatorImpl() { }
|
||||
|
||||
static const char* kClassName() {
|
||||
const char* Name() const override {
|
||||
return "rocksdb.ReverseBytewiseComparator";
|
||||
}
|
||||
const char* Name() const override { return kClassName(); }
|
||||
|
||||
int Compare(const Slice& a, const Slice& b) const override {
|
||||
return -a.compare(b);
|
||||
@ -227,77 +220,4 @@ const Comparator* ReverseBytewiseComparator() {
|
||||
return &rbytewise;
|
||||
}
|
||||
|
||||
#ifndef ROCKSDB_LITE
|
||||
static int RegisterBuiltinComparators(ObjectLibrary& library,
|
||||
const std::string& /*arg*/) {
|
||||
library.Register<const Comparator>(
|
||||
BytewiseComparatorImpl::kClassName(),
|
||||
[](const std::string& /*uri*/,
|
||||
std::unique_ptr<const Comparator>* /*guard */,
|
||||
std::string* /* errmsg */) { return BytewiseComparator(); });
|
||||
library.Register<const Comparator>(
|
||||
ReverseBytewiseComparatorImpl::kClassName(),
|
||||
[](const std::string& /*uri*/,
|
||||
std::unique_ptr<const Comparator>* /*guard */,
|
||||
std::string* /* errmsg */) { return ReverseBytewiseComparator(); });
|
||||
return 2;
|
||||
}
|
||||
#endif // ROCKSDB_LITE
|
||||
|
||||
Status Comparator::CreateFromString(const ConfigOptions& config_options,
|
||||
const std::string& value,
|
||||
const Comparator** result) {
|
||||
#ifndef ROCKSDB_LITE
|
||||
static std::once_flag once;
|
||||
std::call_once(once, [&]() {
|
||||
RegisterBuiltinComparators(*(ObjectLibrary::Default().get()), "");
|
||||
});
|
||||
#endif // ROCKSDB_LITE
|
||||
std::string id;
|
||||
std::unordered_map<std::string, std::string> opt_map;
|
||||
Status status =
|
||||
ConfigurableHelper::GetOptionsMap(value, *result, &id, &opt_map);
|
||||
if (!status.ok()) { // GetOptionsMap failed
|
||||
return status;
|
||||
}
|
||||
std::string curr_opts;
|
||||
#ifndef ROCKSDB_LITE
|
||||
if (*result != nullptr && (*result)->GetId() == id) {
|
||||
// Try to get the existing options, ignoring any errors
|
||||
ConfigOptions embedded = config_options;
|
||||
embedded.delimiter = ";";
|
||||
(*result)->GetOptionString(embedded, &curr_opts).PermitUncheckedError();
|
||||
}
|
||||
#endif
|
||||
if (id == BytewiseComparatorImpl::kClassName()) {
|
||||
*result = BytewiseComparator();
|
||||
} else if (id == ReverseBytewiseComparatorImpl::kClassName()) {
|
||||
*result = ReverseBytewiseComparator();
|
||||
} else if (value.empty()) {
|
||||
// No Id and no options. Clear the object
|
||||
*result = nullptr;
|
||||
return Status::OK();
|
||||
} else if (id.empty()) { // We have no Id but have options. Not good
|
||||
return Status::NotSupported("Cannot reset object ", id);
|
||||
} else {
|
||||
#ifndef ROCKSDB_LITE
|
||||
status = config_options.registry->NewStaticObject(id, result);
|
||||
#else
|
||||
status = Status::NotSupported("Cannot load object in LITE mode ", id);
|
||||
#endif // ROCKSDB_LITE
|
||||
if (!status.ok()) {
|
||||
if (config_options.ignore_unsupported_options &&
|
||||
status.IsNotSupported()) {
|
||||
return Status::OK();
|
||||
} else {
|
||||
return status;
|
||||
}
|
||||
} else if (!curr_opts.empty() || !opt_map.empty()) {
|
||||
Comparator* comparator = const_cast<Comparator*>(*result);
|
||||
status = ConfigurableHelper::ConfigureNewObject(
|
||||
config_options, comparator, id, curr_opts, opt_map);
|
||||
}
|
||||
}
|
||||
return status;
|
||||
}
|
||||
} // namespace ROCKSDB_NAMESPACE
|
||||
|
@ -4,7 +4,9 @@
|
||||
// (found in the LICENSE.Apache file in the root directory).
|
||||
|
||||
#include "rocksdb/utilities/sim_cache.h"
|
||||
|
||||
#include <cstdlib>
|
||||
|
||||
#include "db/db_test_util.h"
|
||||
#include "port/stack_trace.h"
|
||||
|
||||
@ -87,6 +89,8 @@ TEST_F(SimCacheTest, SimCache) {
|
||||
options.table_factory.reset(NewBlockBasedTableFactory(table_options));
|
||||
Reopen(options);
|
||||
RecordCacheCounters(options);
|
||||
// due to cache entry stats collector
|
||||
uint64_t base_misses = simCache->get_miss_counter();
|
||||
|
||||
std::vector<std::unique_ptr<Iterator>> iterators(kNumBlocks);
|
||||
Iterator* iter = nullptr;
|
||||
@ -99,8 +103,8 @@ TEST_F(SimCacheTest, SimCache) {
|
||||
CheckCacheCounters(options, 1, 0, 1, 0);
|
||||
iterators[i].reset(iter);
|
||||
}
|
||||
ASSERT_EQ(kNumBlocks,
|
||||
simCache->get_hit_counter() + simCache->get_miss_counter());
|
||||
ASSERT_EQ(kNumBlocks, simCache->get_hit_counter() +
|
||||
simCache->get_miss_counter() - base_misses);
|
||||
ASSERT_EQ(0, simCache->get_hit_counter());
|
||||
size_t usage = simCache->GetUsage();
|
||||
ASSERT_LT(0, usage);
|
||||
@ -137,8 +141,8 @@ TEST_F(SimCacheTest, SimCache) {
|
||||
CheckCacheCounters(options, 1, 0, 1, 0);
|
||||
}
|
||||
ASSERT_EQ(0, simCache->GetPinnedUsage());
|
||||
ASSERT_EQ(3 * kNumBlocks + 1,
|
||||
simCache->get_hit_counter() + simCache->get_miss_counter());
|
||||
ASSERT_EQ(3 * kNumBlocks + 1, simCache->get_hit_counter() +
|
||||
simCache->get_miss_counter() - base_misses);
|
||||
ASSERT_EQ(6, simCache->get_hit_counter());
|
||||
}
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user