提交 8c265c08 编写于 作者: S sdong

HashLinkList to log distribution of number of entries aross buckets

Summary: Add two parameters of hash linked list to log distribution of number of entries across all buckets, and a sample row when there are too many entries in one single bucket.

Test Plan: Turn it on in plain_table_db_test and see the logs.

Reviewers: haobo, ljin

Reviewed By: ljin

Subscribers: leveldb, nkg-, dhruba, yhchiang

Differential Revision: https://reviews.facebook.net/D19095
上级 4bff7a8a
......@@ -4,6 +4,7 @@
### Public API changes
* Replaced ColumnFamilyOptions::table_properties_collectors with ColumnFamilyOptions::table_properties_collector_factories
* Add two paramters to NewHashLinkListRepFactory() for logging on too many entries in a hash bucket when flushing.
### New Features
* Hash index for block-based table will be materialized and reconstructed more efficiently. Previously hash index is constructed by scanning the whole table during every table open.
......
......@@ -62,6 +62,7 @@ class PlainTableDBTest {
Options CurrentOptions() {
Options options;
options.table_factory.reset(NewPlainTableFactory(16, 2, 0.8, 3));
options.memtable_factory.reset(NewHashLinkListRepFactory(4, 0, 3, true));
options.prefix_extractor.reset(NewFixedPrefixTransform(8));
options.allow_mmap_reads = true;
return options;
......
......@@ -243,8 +243,14 @@ extern MemTableRepFactory* NewHashSkipListRepFactory(
// huge pages for it to be allocated, like:
// sysctl -w vm.nr_hugepages=20
// See linux doc Documentation/vm/hugetlbpage.txt
// @bucket_entries_logging_threshold: if number of entries in one bucket
// exceeds this number, log about it.
// @if_log_bucket_dist_when_flash: if true, log distribution of number of
// entries when flushing.
extern MemTableRepFactory* NewHashLinkListRepFactory(
size_t bucket_count = 50000, size_t huge_page_tlb_size = 0);
size_t bucket_count = 50000, size_t huge_page_tlb_size = 0,
int bucket_entries_logging_threshold = 4096,
bool if_log_bucket_dist_when_flash = true);
// This factory creates a cuckoo-hashing based mem-table representation.
// Cuckoo-hash is a closed-hash strategy, in which all key/value pairs
......
......@@ -13,6 +13,7 @@
#include "rocksdb/slice_transform.h"
#include "port/port.h"
#include "port/atomic_pointer.h"
#include "util/histogram.h"
#include "util/murmurhash.h"
#include "db/memtable.h"
#include "db/skiplist.h"
......@@ -54,7 +55,9 @@ class HashLinkListRep : public MemTableRep {
public:
HashLinkListRep(const MemTableRep::KeyComparator& compare, Arena* arena,
const SliceTransform* transform, size_t bucket_size,
size_t huge_page_tlb_size, Logger* logger);
size_t huge_page_tlb_size, Logger* logger,
int bucket_entries_logging_threshold,
bool if_log_bucket_dist_when_flash);
virtual KeyHandle Allocate(const size_t len, char** buf) override;
......@@ -92,6 +95,10 @@ class HashLinkListRep : public MemTableRep {
const MemTableRep::KeyComparator& compare_;
Logger* logger_;
int bucket_entries_logging_threshold_;
bool if_log_bucket_dist_when_flash_;
bool BucketContains(Node* head, const Slice& key) const;
Slice GetPrefix(const Slice& internal_key) const {
......@@ -308,11 +315,16 @@ class HashLinkListRep : public MemTableRep {
HashLinkListRep::HashLinkListRep(const MemTableRep::KeyComparator& compare,
Arena* arena, const SliceTransform* transform,
size_t bucket_size, size_t huge_page_tlb_size,
Logger* logger)
Logger* logger,
int bucket_entries_logging_threshold,
bool if_log_bucket_dist_when_flash)
: MemTableRep(arena),
bucket_size_(bucket_size),
transform_(transform),
compare_(compare) {
compare_(compare),
logger_(logger),
bucket_entries_logging_threshold_(bucket_entries_logging_threshold),
if_log_bucket_dist_when_flash_(if_log_bucket_dist_when_flash) {
char* mem = arena_->AllocateAligned(sizeof(port::AtomicPointer) * bucket_size,
huge_page_tlb_size, logger);
......@@ -416,15 +428,36 @@ MemTableRep::Iterator* HashLinkListRep::GetIterator(Arena* alloc_arena) {
// allocate a new arena of similar size to the one currently in use
Arena* new_arena = new Arena(arena_->BlockSize());
auto list = new FullList(compare_, new_arena);
HistogramImpl keys_per_bucket_hist;
for (size_t i = 0; i < bucket_size_; ++i) {
int count = 0;
bool num_entries_printed = false;
auto bucket = GetBucket(i);
if (bucket != nullptr) {
Iterator itr(this, bucket);
for (itr.SeekToHead(); itr.Valid(); itr.Next()) {
list->Insert(itr.key());
if (logger_ != nullptr &&
++count >= bucket_entries_logging_threshold_ &&
!num_entries_printed) {
num_entries_printed = true;
Info(logger_, "HashLinkedList bucket %zu has more than %d "
"entries. %dth key: %s",
i, count, count,
GetLengthPrefixedSlice(itr.key()).ToString(true).c_str());
}
}
}
if (if_log_bucket_dist_when_flash_) {
keys_per_bucket_hist.Add(count);
}
}
if (if_log_bucket_dist_when_flash_ && logger_ != nullptr) {
Info(logger_, "hashLinkedList Entry distribution among buckets: %s",
keys_per_bucket_hist.ToString().c_str());
}
if (alloc_arena == nullptr) {
return new FullListIterator(list, new_arena);
} else {
......@@ -482,13 +515,17 @@ Node* HashLinkListRep::FindGreaterOrEqualInBucket(Node* head,
MemTableRep* HashLinkListRepFactory::CreateMemTableRep(
const MemTableRep::KeyComparator& compare, Arena* arena,
const SliceTransform* transform, Logger* logger) {
return new HashLinkListRep(compare, arena, transform, bucket_count_,
huge_page_tlb_size_, logger);
return new HashLinkListRep(
compare, arena, transform, bucket_count_, huge_page_tlb_size_, logger,
bucket_entries_logging_threshold_, if_log_bucket_dist_when_flash_);
}
MemTableRepFactory* NewHashLinkListRepFactory(size_t bucket_count,
size_t huge_page_tlb_size) {
return new HashLinkListRepFactory(bucket_count, huge_page_tlb_size);
MemTableRepFactory* NewHashLinkListRepFactory(
size_t bucket_count, size_t huge_page_tlb_size,
int bucket_entries_logging_threshold, bool if_log_bucket_dist_when_flash) {
return new HashLinkListRepFactory(bucket_count, huge_page_tlb_size,
bucket_entries_logging_threshold,
if_log_bucket_dist_when_flash);
}
} // namespace rocksdb
......
......@@ -16,8 +16,13 @@ namespace rocksdb {
class HashLinkListRepFactory : public MemTableRepFactory {
public:
explicit HashLinkListRepFactory(size_t bucket_count,
size_t huge_page_tlb_size)
: bucket_count_(bucket_count), huge_page_tlb_size_(huge_page_tlb_size) {}
size_t huge_page_tlb_size,
int bucket_entries_logging_threshold,
bool if_log_bucket_dist_when_flash)
: bucket_count_(bucket_count),
huge_page_tlb_size_(huge_page_tlb_size),
bucket_entries_logging_threshold_(bucket_entries_logging_threshold),
if_log_bucket_dist_when_flash_(if_log_bucket_dist_when_flash) {}
virtual ~HashLinkListRepFactory() {}
......@@ -32,6 +37,8 @@ class HashLinkListRepFactory : public MemTableRepFactory {
private:
const size_t bucket_count_;
const size_t huge_page_tlb_size_;
int bucket_entries_logging_threshold_;
bool if_log_bucket_dist_when_flash_;
};
}
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册