hash_linklist_rep.cc 32 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867868869870871872873874875876877878879880881882883884885886887888889890891892893894895896897898899900901902903904905906907908909910911912913914915916917918919920921922923924
  1. // Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
  2. // This source code is licensed under both the GPLv2 (found in the
  3. // COPYING file in the root directory) and Apache 2.0 License
  4. // (found in the LICENSE.Apache file in the root directory).
  5. //
  6. #include <algorithm>
  7. #include <atomic>
  8. #include "db/memtable.h"
  9. #include "memory/arena.h"
  10. #include "memtable/skiplist.h"
  11. #include "monitoring/histogram.h"
  12. #include "port/port.h"
  13. #include "rocksdb/memtablerep.h"
  14. #include "rocksdb/slice.h"
  15. #include "rocksdb/slice_transform.h"
  16. #include "rocksdb/utilities/options_type.h"
  17. #include "util/hash.h"
  18. namespace ROCKSDB_NAMESPACE {
  19. namespace {
  20. using Key = const char*;
  21. using MemtableSkipList = SkipList<Key, const MemTableRep::KeyComparator&>;
  22. using Pointer = std::atomic<void*>;
  23. // A data structure used as the header of a link list of a hash bucket.
  24. struct BucketHeader {
  25. Pointer next;
  26. std::atomic<uint32_t> num_entries;
  27. explicit BucketHeader(void* n, uint32_t count)
  28. : next(n), num_entries(count) {}
  29. bool IsSkipListBucket() {
  30. return next.load(std::memory_order_relaxed) == this;
  31. }
  32. uint32_t GetNumEntries() const {
  33. return num_entries.load(std::memory_order_relaxed);
  34. }
  35. // REQUIRES: called from single-threaded Insert()
  36. void IncNumEntries() {
  37. // Only one thread can do write at one time. No need to do atomic
  38. // incremental. Update it with relaxed load and store.
  39. num_entries.store(GetNumEntries() + 1, std::memory_order_relaxed);
  40. }
  41. };
  42. // A data structure used as the header of a skip list of a hash bucket.
  43. struct SkipListBucketHeader {
  44. BucketHeader Counting_header;
  45. MemtableSkipList skip_list;
  46. explicit SkipListBucketHeader(const MemTableRep::KeyComparator& cmp,
  47. Allocator* allocator, uint32_t count)
  48. : Counting_header(this, // Pointing to itself to indicate header type.
  49. count),
  50. skip_list(cmp, allocator) {}
  51. };
  52. struct Node {
  53. // Accessors/mutators for links. Wrapped in methods so we can
  54. // add the appropriate barriers as necessary.
  55. Node* Next() {
  56. // Use an 'acquire load' so that we observe a fully initialized
  57. // version of the returned Node.
  58. return next_.load(std::memory_order_acquire);
  59. }
  60. void SetNext(Node* x) {
  61. // Use a 'release store' so that anybody who reads through this
  62. // pointer observes a fully initialized version of the inserted node.
  63. next_.store(x, std::memory_order_release);
  64. }
  65. // No-barrier variants that can be safely used in a few locations.
  66. Node* NoBarrier_Next() { return next_.load(std::memory_order_relaxed); }
  67. void NoBarrier_SetNext(Node* x) { next_.store(x, std::memory_order_relaxed); }
  68. // Needed for placement new below which is fine
  69. Node() = default;
  70. private:
  71. std::atomic<Node*> next_;
  72. // Prohibit copying due to the below
  73. Node(const Node&) = delete;
  74. Node& operator=(const Node&) = delete;
  75. public:
  76. char key[1];
  77. };
  78. // Memory structure of the mem table:
  79. // It is a hash table, each bucket points to one entry, a linked list or a
  80. // skip list. In order to track total number of records in a bucket to determine
  81. // whether should switch to skip list, a header is added just to indicate
  82. // number of entries in the bucket.
  83. //
  84. //
  85. // +-----> NULL Case 1. Empty bucket
  86. // |
  87. // |
  88. // | +---> +-------+
  89. // | | | Next +--> NULL
  90. // | | +-------+
  91. // +-----+ | | | | Case 2. One Entry in bucket.
  92. // | +-+ | | Data | next pointer points to
  93. // +-----+ | | | NULL. All other cases
  94. // | | | | | next pointer is not NULL.
  95. // +-----+ | +-------+
  96. // | +---+
  97. // +-----+ +-> +-------+ +> +-------+ +-> +-------+
  98. // | | | | Next +--+ | Next +--+ | Next +-->NULL
  99. // +-----+ | +-------+ +-------+ +-------+
  100. // | +-----+ | Count | | | | |
  101. // +-----+ +-------+ | Data | | Data |
  102. // | | | | | |
  103. // +-----+ Case 3. | | | |
  104. // | | A header +-------+ +-------+
  105. // +-----+ points to
  106. // | | a linked list. Count indicates total number
  107. // +-----+ of rows in this bucket.
  108. // | |
  109. // +-----+ +-> +-------+ <--+
  110. // | | | | Next +----+
  111. // +-----+ | +-------+ Case 4. A header points to a skip
  112. // | +----+ | Count | list and next pointer points to
  113. // +-----+ +-------+ itself, to distinguish case 3 or 4.
  114. // | | | | Count still is kept to indicates total
  115. // +-----+ | Skip +--> of entries in the bucket for debugging
  116. // | | | List | Data purpose.
  117. // | | | +-->
  118. // +-----+ | |
  119. // | | +-------+
  120. // +-----+
  121. //
  122. // We don't have data race when changing cases because:
  123. // (1) When changing from case 2->3, we create a new bucket header, put the
  124. // single node there first without changing the original node, and do a
  125. // release store when changing the bucket pointer. In that case, a reader
  126. // who sees a stale value of the bucket pointer will read this node, while
  127. // a reader sees the correct value because of the release store.
  128. // (2) When changing case 3->4, a new header is created with skip list points
  129. // to the data, before doing an acquire store to change the bucket pointer.
  130. // The old header and nodes are never changed, so any reader sees any
  131. // of those existing pointers will guarantee to be able to iterate to the
  132. // end of the linked list.
  133. // (3) Header's next pointer in case 3 might change, but they are never equal
  134. // to itself, so no matter a reader sees any stale or newer value, it will
  135. // be able to correctly distinguish case 3 and 4.
  136. //
  137. // The reason that we use case 2 is we want to make the format to be efficient
  138. // when the utilization of buckets is relatively low. If we use case 3 for
  139. // single entry bucket, we will need to waste 12 bytes for every entry,
  140. // which can be significant decrease of memory utilization.
  141. class HashLinkListRep : public MemTableRep {
  142. public:
  143. HashLinkListRep(const MemTableRep::KeyComparator& compare,
  144. Allocator* allocator, const SliceTransform* transform,
  145. size_t bucket_size, uint32_t threshold_use_skiplist,
  146. size_t huge_page_tlb_size, Logger* logger,
  147. int bucket_entries_logging_threshold,
  148. bool if_log_bucket_dist_when_flash);
  149. KeyHandle Allocate(const size_t len, char** buf) override;
  150. void Insert(KeyHandle handle) override;
  151. bool Contains(const char* key) const override;
  152. size_t ApproximateMemoryUsage() override;
  153. void Get(const LookupKey& k, void* callback_args,
  154. bool (*callback_func)(void* arg, const char* entry)) override;
  155. ~HashLinkListRep() override;
  156. MemTableRep::Iterator* GetIterator(Arena* arena = nullptr) override;
  157. MemTableRep::Iterator* GetDynamicPrefixIterator(
  158. Arena* arena = nullptr) override;
  159. private:
  160. friend class DynamicIterator;
  161. size_t bucket_size_;
  162. // Maps slices (which are transformed user keys) to buckets of keys sharing
  163. // the same transform.
  164. Pointer* buckets_;
  165. const uint32_t threshold_use_skiplist_;
  166. // The user-supplied transform whose domain is the user keys.
  167. const SliceTransform* transform_;
  168. const MemTableRep::KeyComparator& compare_;
  169. Logger* logger_;
  170. int bucket_entries_logging_threshold_;
  171. bool if_log_bucket_dist_when_flash_;
  172. bool LinkListContains(Node* head, const Slice& key) const;
  173. bool IsEmptyBucket(Pointer& bucket_pointer) const {
  174. return bucket_pointer.load(std::memory_order_acquire) == nullptr;
  175. }
  176. // Precondition: GetLinkListFirstNode() must have been called first and return
  177. // null so that it must be a skip list bucket
  178. SkipListBucketHeader* GetSkipListBucketHeader(Pointer& bucket_pointer) const;
  179. // Returning nullptr indicates it is a skip list bucket.
  180. Node* GetLinkListFirstNode(Pointer& bucket_pointer) const;
  181. Slice GetPrefix(const Slice& internal_key) const {
  182. return transform_->Transform(ExtractUserKey(internal_key));
  183. }
  184. size_t GetHash(const Slice& slice) const {
  185. return GetSliceRangedNPHash(slice, bucket_size_);
  186. }
  187. Pointer& GetBucket(size_t i) const { return buckets_[i]; }
  188. Pointer& GetBucket(const Slice& slice) const {
  189. return GetBucket(GetHash(slice));
  190. }
  191. bool Equal(const Slice& a, const Key& b) const {
  192. return (compare_(b, a) == 0);
  193. }
  194. bool Equal(const Key& a, const Key& b) const { return (compare_(a, b) == 0); }
  195. bool KeyIsAfterNode(const Slice& internal_key, const Node* n) const {
  196. // nullptr n is considered infinite
  197. return (n != nullptr) && (compare_(n->key, internal_key) < 0);
  198. }
  199. bool KeyIsAfterNode(const Key& key, const Node* n) const {
  200. // nullptr n is considered infinite
  201. return (n != nullptr) && (compare_(n->key, key) < 0);
  202. }
  203. bool KeyIsAfterOrAtNode(const Slice& internal_key, const Node* n) const {
  204. // nullptr n is considered infinite
  205. return (n != nullptr) && (compare_(n->key, internal_key) <= 0);
  206. }
  207. bool KeyIsAfterOrAtNode(const Key& key, const Node* n) const {
  208. // nullptr n is considered infinite
  209. return (n != nullptr) && (compare_(n->key, key) <= 0);
  210. }
  211. Node* FindGreaterOrEqualInBucket(Node* head, const Slice& key) const;
  212. Node* FindLessOrEqualInBucket(Node* head, const Slice& key) const;
  213. class FullListIterator : public MemTableRep::Iterator {
  214. public:
  215. explicit FullListIterator(MemtableSkipList* list, Allocator* allocator)
  216. : iter_(list), full_list_(list), allocator_(allocator) {}
  217. ~FullListIterator() override = default;
  218. // Returns true iff the iterator is positioned at a valid node.
  219. bool Valid() const override { return iter_.Valid(); }
  220. // Returns the key at the current position.
  221. // REQUIRES: Valid()
  222. const char* key() const override {
  223. assert(Valid());
  224. return iter_.key();
  225. }
  226. // Advances to the next position.
  227. // REQUIRES: Valid()
  228. void Next() override {
  229. assert(Valid());
  230. iter_.Next();
  231. }
  232. // Advances to the previous position.
  233. // REQUIRES: Valid()
  234. void Prev() override {
  235. assert(Valid());
  236. iter_.Prev();
  237. }
  238. // Advance to the first entry with a key >= target
  239. void Seek(const Slice& internal_key, const char* memtable_key) override {
  240. const char* encoded_key = (memtable_key != nullptr)
  241. ? memtable_key
  242. : EncodeKey(&tmp_, internal_key);
  243. iter_.Seek(encoded_key);
  244. }
  245. // Retreat to the last entry with a key <= target
  246. void SeekForPrev(const Slice& internal_key,
  247. const char* memtable_key) override {
  248. const char* encoded_key = (memtable_key != nullptr)
  249. ? memtable_key
  250. : EncodeKey(&tmp_, internal_key);
  251. iter_.SeekForPrev(encoded_key);
  252. }
  253. // Position at the first entry in collection.
  254. // Final state of iterator is Valid() iff collection is not empty.
  255. void SeekToFirst() override { iter_.SeekToFirst(); }
  256. // Position at the last entry in collection.
  257. // Final state of iterator is Valid() iff collection is not empty.
  258. void SeekToLast() override { iter_.SeekToLast(); }
  259. private:
  260. MemtableSkipList::Iterator iter_;
  261. // To destruct with the iterator.
  262. std::unique_ptr<MemtableSkipList> full_list_;
  263. std::unique_ptr<Allocator> allocator_;
  264. std::string tmp_; // For passing to EncodeKey
  265. };
  266. class LinkListIterator : public MemTableRep::Iterator {
  267. public:
  268. explicit LinkListIterator(const HashLinkListRep* const hash_link_list_rep,
  269. Node* head)
  270. : hash_link_list_rep_(hash_link_list_rep),
  271. head_(head),
  272. node_(nullptr) {}
  273. ~LinkListIterator() override = default;
  274. // Returns true iff the iterator is positioned at a valid node.
  275. bool Valid() const override { return node_ != nullptr; }
  276. // Returns the key at the current position.
  277. // REQUIRES: Valid()
  278. const char* key() const override {
  279. assert(Valid());
  280. return node_->key;
  281. }
  282. // Advances to the next position.
  283. // REQUIRES: Valid()
  284. void Next() override {
  285. assert(Valid());
  286. node_ = node_->Next();
  287. }
  288. // Advances to the previous position.
  289. // REQUIRES: Valid()
  290. void Prev() override {
  291. // Prefix iterator does not support total order.
  292. // We simply set the iterator to invalid state
  293. Reset(nullptr);
  294. }
  295. // Advance to the first entry with a key >= target
  296. void Seek(const Slice& internal_key,
  297. const char* /*memtable_key*/) override {
  298. node_ =
  299. hash_link_list_rep_->FindGreaterOrEqualInBucket(head_, internal_key);
  300. }
  301. // Retreat to the last entry with a key <= target
  302. void SeekForPrev(const Slice& /*internal_key*/,
  303. const char* /*memtable_key*/) override {
  304. // Since we do not support Prev()
  305. // We simply do not support SeekForPrev
  306. Reset(nullptr);
  307. }
  308. // Position at the first entry in collection.
  309. // Final state of iterator is Valid() iff collection is not empty.
  310. void SeekToFirst() override {
  311. // Prefix iterator does not support total order.
  312. // We simply set the iterator to invalid state
  313. Reset(nullptr);
  314. }
  315. // Position at the last entry in collection.
  316. // Final state of iterator is Valid() iff collection is not empty.
  317. void SeekToLast() override {
  318. // Prefix iterator does not support total order.
  319. // We simply set the iterator to invalid state
  320. Reset(nullptr);
  321. }
  322. protected:
  323. void Reset(Node* head) {
  324. head_ = head;
  325. node_ = nullptr;
  326. }
  327. private:
  328. friend class HashLinkListRep;
  329. const HashLinkListRep* const hash_link_list_rep_;
  330. Node* head_;
  331. Node* node_;
  332. virtual void SeekToHead() { node_ = head_; }
  333. };
  334. class DynamicIterator : public HashLinkListRep::LinkListIterator {
  335. public:
  336. explicit DynamicIterator(HashLinkListRep& memtable_rep)
  337. : HashLinkListRep::LinkListIterator(&memtable_rep, nullptr),
  338. memtable_rep_(memtable_rep) {}
  339. // Advance to the first entry with a key >= target
  340. void Seek(const Slice& k, const char* memtable_key) override {
  341. auto transformed = memtable_rep_.GetPrefix(k);
  342. Pointer& bucket = memtable_rep_.GetBucket(transformed);
  343. if (memtable_rep_.IsEmptyBucket(bucket)) {
  344. skip_list_iter_.reset();
  345. Reset(nullptr);
  346. } else {
  347. Node* first_linked_list_node =
  348. memtable_rep_.GetLinkListFirstNode(bucket);
  349. if (first_linked_list_node != nullptr) {
  350. // The bucket is organized as a linked list
  351. skip_list_iter_.reset();
  352. Reset(first_linked_list_node);
  353. HashLinkListRep::LinkListIterator::Seek(k, memtable_key);
  354. } else {
  355. SkipListBucketHeader* skip_list_header =
  356. memtable_rep_.GetSkipListBucketHeader(bucket);
  357. assert(skip_list_header != nullptr);
  358. // The bucket is organized as a skip list
  359. if (!skip_list_iter_) {
  360. skip_list_iter_.reset(
  361. new MemtableSkipList::Iterator(&skip_list_header->skip_list));
  362. } else {
  363. skip_list_iter_->SetList(&skip_list_header->skip_list);
  364. }
  365. if (memtable_key != nullptr) {
  366. skip_list_iter_->Seek(memtable_key);
  367. } else {
  368. IterKey encoded_key;
  369. encoded_key.EncodeLengthPrefixedKey(k);
  370. skip_list_iter_->Seek(encoded_key.GetUserKey().data());
  371. }
  372. }
  373. }
  374. }
  375. bool Valid() const override {
  376. if (skip_list_iter_) {
  377. return skip_list_iter_->Valid();
  378. }
  379. return HashLinkListRep::LinkListIterator::Valid();
  380. }
  381. const char* key() const override {
  382. if (skip_list_iter_) {
  383. return skip_list_iter_->key();
  384. }
  385. return HashLinkListRep::LinkListIterator::key();
  386. }
  387. void Next() override {
  388. if (skip_list_iter_) {
  389. skip_list_iter_->Next();
  390. } else {
  391. HashLinkListRep::LinkListIterator::Next();
  392. }
  393. }
  394. private:
  395. // the underlying memtable
  396. const HashLinkListRep& memtable_rep_;
  397. std::unique_ptr<MemtableSkipList::Iterator> skip_list_iter_;
  398. };
  399. class EmptyIterator : public MemTableRep::Iterator {
  400. // This is used when there wasn't a bucket. It is cheaper than
  401. // instantiating an empty bucket over which to iterate.
  402. public:
  403. EmptyIterator() = default;
  404. bool Valid() const override { return false; }
  405. const char* key() const override {
  406. assert(false);
  407. return nullptr;
  408. }
  409. void Next() override {}
  410. void Prev() override {}
  411. void Seek(const Slice& /*user_key*/,
  412. const char* /*memtable_key*/) override {}
  413. void SeekForPrev(const Slice& /*user_key*/,
  414. const char* /*memtable_key*/) override {}
  415. void SeekToFirst() override {}
  416. void SeekToLast() override {}
  417. private:
  418. };
  419. };
  420. HashLinkListRep::HashLinkListRep(
  421. const MemTableRep::KeyComparator& compare, Allocator* allocator,
  422. const SliceTransform* transform, size_t bucket_size,
  423. uint32_t threshold_use_skiplist, size_t huge_page_tlb_size, Logger* logger,
  424. int bucket_entries_logging_threshold, bool if_log_bucket_dist_when_flash)
  425. : MemTableRep(allocator),
  426. bucket_size_(bucket_size),
  427. // Threshold to use skip list doesn't make sense if less than 3, so we
  428. // force it to be minimum of 3 to simplify implementation.
  429. threshold_use_skiplist_(std::max(threshold_use_skiplist, 3U)),
  430. transform_(transform),
  431. compare_(compare),
  432. logger_(logger),
  433. bucket_entries_logging_threshold_(bucket_entries_logging_threshold),
  434. if_log_bucket_dist_when_flash_(if_log_bucket_dist_when_flash) {
  435. char* mem = allocator_->AllocateAligned(sizeof(Pointer) * bucket_size,
  436. huge_page_tlb_size, logger);
  437. buckets_ = new (mem) Pointer[bucket_size];
  438. for (size_t i = 0; i < bucket_size_; ++i) {
  439. buckets_[i].store(nullptr, std::memory_order_relaxed);
  440. }
  441. }
  442. HashLinkListRep::~HashLinkListRep() = default;
  443. KeyHandle HashLinkListRep::Allocate(const size_t len, char** buf) {
  444. char* mem = allocator_->AllocateAligned(sizeof(Node) + len);
  445. Node* x = new (mem) Node();
  446. *buf = x->key;
  447. return static_cast<void*>(x);
  448. }
  449. SkipListBucketHeader* HashLinkListRep::GetSkipListBucketHeader(
  450. Pointer& bucket_pointer) const {
  451. Pointer* first_next_pointer =
  452. static_cast<Pointer*>(bucket_pointer.load(std::memory_order_acquire));
  453. assert(first_next_pointer != nullptr);
  454. assert(first_next_pointer->load(std::memory_order_relaxed) != nullptr);
  455. // Counting header
  456. BucketHeader* header = reinterpret_cast<BucketHeader*>(first_next_pointer);
  457. assert(header->IsSkipListBucket());
  458. assert(header->GetNumEntries() > threshold_use_skiplist_);
  459. auto* skip_list_bucket_header =
  460. reinterpret_cast<SkipListBucketHeader*>(header);
  461. assert(skip_list_bucket_header->Counting_header.next.load(
  462. std::memory_order_relaxed) == header);
  463. return skip_list_bucket_header;
  464. }
  465. Node* HashLinkListRep::GetLinkListFirstNode(Pointer& bucket_pointer) const {
  466. Pointer* first_next_pointer =
  467. static_cast<Pointer*>(bucket_pointer.load(std::memory_order_acquire));
  468. assert(first_next_pointer != nullptr);
  469. if (first_next_pointer->load(std::memory_order_relaxed) == nullptr) {
  470. // Single entry bucket
  471. return reinterpret_cast<Node*>(first_next_pointer);
  472. }
  473. // It is possible that after we fetch first_next_pointer it is modified
  474. // and the next is not null anymore. In this case, the bucket should have been
  475. // modified to a counting header, so we should reload the first_next_pointer
  476. // to make sure we see the update.
  477. first_next_pointer =
  478. static_cast<Pointer*>(bucket_pointer.load(std::memory_order_acquire));
  479. // Counting header
  480. BucketHeader* header = reinterpret_cast<BucketHeader*>(first_next_pointer);
  481. if (!header->IsSkipListBucket()) {
  482. assert(header->GetNumEntries() <= threshold_use_skiplist_);
  483. return reinterpret_cast<Node*>(
  484. header->next.load(std::memory_order_acquire));
  485. }
  486. assert(header->GetNumEntries() > threshold_use_skiplist_);
  487. return nullptr;
  488. }
  489. void HashLinkListRep::Insert(KeyHandle handle) {
  490. Node* x = static_cast<Node*>(handle);
  491. assert(!Contains(x->key));
  492. Slice internal_key = GetLengthPrefixedSlice(x->key);
  493. auto transformed = GetPrefix(internal_key);
  494. auto& bucket = buckets_[GetHash(transformed)];
  495. Pointer* first_next_pointer =
  496. static_cast<Pointer*>(bucket.load(std::memory_order_relaxed));
  497. if (first_next_pointer == nullptr) {
  498. // Case 1. empty bucket
  499. // NoBarrier_SetNext() suffices since we will add a barrier when
  500. // we publish a pointer to "x" in prev[i].
  501. x->NoBarrier_SetNext(nullptr);
  502. bucket.store(x, std::memory_order_release);
  503. return;
  504. }
  505. BucketHeader* header = nullptr;
  506. if (first_next_pointer->load(std::memory_order_relaxed) == nullptr) {
  507. // Case 2. only one entry in the bucket
  508. // Need to convert to a Counting bucket and turn to case 4.
  509. Node* first = reinterpret_cast<Node*>(first_next_pointer);
  510. // Need to add a bucket header.
  511. // We have to first convert it to a bucket with header before inserting
  512. // the new node. Otherwise, we might need to change next pointer of first.
  513. // In that case, a reader might sees the next pointer is NULL and wrongly
  514. // think the node is a bucket header.
  515. auto* mem = allocator_->AllocateAligned(sizeof(BucketHeader));
  516. header = new (mem) BucketHeader(first, 1);
  517. bucket.store(header, std::memory_order_release);
  518. } else {
  519. header = reinterpret_cast<BucketHeader*>(first_next_pointer);
  520. if (header->IsSkipListBucket()) {
  521. // Case 4. Bucket is already a skip list
  522. assert(header->GetNumEntries() > threshold_use_skiplist_);
  523. auto* skip_list_bucket_header =
  524. reinterpret_cast<SkipListBucketHeader*>(header);
  525. // Only one thread can execute Insert() at one time. No need to do atomic
  526. // incremental.
  527. skip_list_bucket_header->Counting_header.IncNumEntries();
  528. skip_list_bucket_header->skip_list.Insert(x->key);
  529. return;
  530. }
  531. }
  532. if (bucket_entries_logging_threshold_ > 0 &&
  533. header->GetNumEntries() ==
  534. static_cast<uint32_t>(bucket_entries_logging_threshold_)) {
  535. Info(logger_,
  536. "HashLinkedList bucket %" ROCKSDB_PRIszt
  537. " has more than %d "
  538. "entries. Key to insert: %s",
  539. GetHash(transformed), header->GetNumEntries(),
  540. GetLengthPrefixedSlice(x->key).ToString(true).c_str());
  541. }
  542. if (header->GetNumEntries() == threshold_use_skiplist_) {
  543. // Case 3. number of entries reaches the threshold so need to convert to
  544. // skip list.
  545. LinkListIterator bucket_iter(
  546. this, reinterpret_cast<Node*>(
  547. first_next_pointer->load(std::memory_order_relaxed)));
  548. auto mem = allocator_->AllocateAligned(sizeof(SkipListBucketHeader));
  549. SkipListBucketHeader* new_skip_list_header = new (mem)
  550. SkipListBucketHeader(compare_, allocator_, header->GetNumEntries() + 1);
  551. auto& skip_list = new_skip_list_header->skip_list;
  552. // Add all current entries to the skip list
  553. for (bucket_iter.SeekToHead(); bucket_iter.Valid(); bucket_iter.Next()) {
  554. skip_list.Insert(bucket_iter.key());
  555. }
  556. // insert the new entry
  557. skip_list.Insert(x->key);
  558. // Set the bucket
  559. bucket.store(new_skip_list_header, std::memory_order_release);
  560. } else {
  561. // Case 5. Need to insert to the sorted linked list without changing the
  562. // header.
  563. Node* first =
  564. reinterpret_cast<Node*>(header->next.load(std::memory_order_relaxed));
  565. assert(first != nullptr);
  566. // Advance counter unless the bucket needs to be advanced to skip list.
  567. // In that case, we need to make sure the previous count never exceeds
  568. // threshold_use_skiplist_ to avoid readers to cast to wrong format.
  569. header->IncNumEntries();
  570. Node* cur = first;
  571. Node* prev = nullptr;
  572. while (true) {
  573. if (cur == nullptr) {
  574. break;
  575. }
  576. Node* next = cur->Next();
  577. // Make sure the lists are sorted.
  578. // If x points to head_ or next points nullptr, it is trivially satisfied.
  579. assert((cur == first) || (next == nullptr) ||
  580. KeyIsAfterNode(next->key, cur));
  581. if (KeyIsAfterNode(internal_key, cur)) {
  582. // Keep searching in this list
  583. prev = cur;
  584. cur = next;
  585. } else {
  586. break;
  587. }
  588. }
  589. // Our data structure does not allow duplicate insertion
  590. assert(cur == nullptr || !Equal(x->key, cur->key));
  591. // NoBarrier_SetNext() suffices since we will add a barrier when
  592. // we publish a pointer to "x" in prev[i].
  593. x->NoBarrier_SetNext(cur);
  594. if (prev) {
  595. prev->SetNext(x);
  596. } else {
  597. header->next.store(static_cast<void*>(x), std::memory_order_release);
  598. }
  599. }
  600. }
  601. bool HashLinkListRep::Contains(const char* key) const {
  602. Slice internal_key = GetLengthPrefixedSlice(key);
  603. auto transformed = GetPrefix(internal_key);
  604. Pointer& bucket = GetBucket(transformed);
  605. if (IsEmptyBucket(bucket)) {
  606. return false;
  607. }
  608. Node* linked_list_node = GetLinkListFirstNode(bucket);
  609. if (linked_list_node != nullptr) {
  610. return LinkListContains(linked_list_node, internal_key);
  611. }
  612. SkipListBucketHeader* skip_list_header = GetSkipListBucketHeader(bucket);
  613. if (skip_list_header != nullptr) {
  614. return skip_list_header->skip_list.Contains(key);
  615. }
  616. return false;
  617. }
  618. size_t HashLinkListRep::ApproximateMemoryUsage() {
  619. // Memory is always allocated from the allocator.
  620. return 0;
  621. }
  622. void HashLinkListRep::Get(const LookupKey& k, void* callback_args,
  623. bool (*callback_func)(void* arg, const char* entry)) {
  624. auto transformed = transform_->Transform(k.user_key());
  625. Pointer& bucket = GetBucket(transformed);
  626. if (IsEmptyBucket(bucket)) {
  627. return;
  628. }
  629. auto* link_list_head = GetLinkListFirstNode(bucket);
  630. if (link_list_head != nullptr) {
  631. LinkListIterator iter(this, link_list_head);
  632. for (iter.Seek(k.internal_key(), nullptr);
  633. iter.Valid() && callback_func(callback_args, iter.key());
  634. iter.Next()) {
  635. }
  636. } else {
  637. auto* skip_list_header = GetSkipListBucketHeader(bucket);
  638. if (skip_list_header != nullptr) {
  639. // Is a skip list
  640. MemtableSkipList::Iterator iter(&skip_list_header->skip_list);
  641. for (iter.Seek(k.memtable_key().data());
  642. iter.Valid() && callback_func(callback_args, iter.key());
  643. iter.Next()) {
  644. }
  645. }
  646. }
  647. }
  648. MemTableRep::Iterator* HashLinkListRep::GetIterator(Arena* alloc_arena) {
  649. // allocate a new arena of similar size to the one currently in use
  650. Arena* new_arena = new Arena(allocator_->BlockSize());
  651. auto list = new MemtableSkipList(compare_, new_arena);
  652. HistogramImpl keys_per_bucket_hist;
  653. for (size_t i = 0; i < bucket_size_; ++i) {
  654. int count = 0;
  655. Pointer& bucket = GetBucket(i);
  656. if (!IsEmptyBucket(bucket)) {
  657. auto* link_list_head = GetLinkListFirstNode(bucket);
  658. if (link_list_head != nullptr) {
  659. LinkListIterator itr(this, link_list_head);
  660. for (itr.SeekToHead(); itr.Valid(); itr.Next()) {
  661. list->Insert(itr.key());
  662. count++;
  663. }
  664. } else {
  665. auto* skip_list_header = GetSkipListBucketHeader(bucket);
  666. assert(skip_list_header != nullptr);
  667. // Is a skip list
  668. MemtableSkipList::Iterator itr(&skip_list_header->skip_list);
  669. for (itr.SeekToFirst(); itr.Valid(); itr.Next()) {
  670. list->Insert(itr.key());
  671. count++;
  672. }
  673. }
  674. }
  675. if (if_log_bucket_dist_when_flash_) {
  676. keys_per_bucket_hist.Add(count);
  677. }
  678. }
  679. if (if_log_bucket_dist_when_flash_ && logger_ != nullptr) {
  680. Info(logger_, "hashLinkedList Entry distribution among buckets: %s",
  681. keys_per_bucket_hist.ToString().c_str());
  682. }
  683. if (alloc_arena == nullptr) {
  684. return new FullListIterator(list, new_arena);
  685. } else {
  686. auto mem = alloc_arena->AllocateAligned(sizeof(FullListIterator));
  687. return new (mem) FullListIterator(list, new_arena);
  688. }
  689. }
  690. MemTableRep::Iterator* HashLinkListRep::GetDynamicPrefixIterator(
  691. Arena* alloc_arena) {
  692. if (alloc_arena == nullptr) {
  693. return new DynamicIterator(*this);
  694. } else {
  695. auto mem = alloc_arena->AllocateAligned(sizeof(DynamicIterator));
  696. return new (mem) DynamicIterator(*this);
  697. }
  698. }
  699. bool HashLinkListRep::LinkListContains(Node* head,
  700. const Slice& user_key) const {
  701. Node* x = FindGreaterOrEqualInBucket(head, user_key);
  702. return (x != nullptr && Equal(user_key, x->key));
  703. }
  704. Node* HashLinkListRep::FindGreaterOrEqualInBucket(Node* head,
  705. const Slice& key) const {
  706. Node* x = head;
  707. while (true) {
  708. if (x == nullptr) {
  709. return x;
  710. }
  711. Node* next = x->Next();
  712. // Make sure the lists are sorted.
  713. // If x points to head_ or next points nullptr, it is trivially satisfied.
  714. assert((x == head) || (next == nullptr) || KeyIsAfterNode(next->key, x));
  715. if (KeyIsAfterNode(key, x)) {
  716. // Keep searching in this list
  717. x = next;
  718. } else {
  719. break;
  720. }
  721. }
  722. return x;
  723. }
  724. struct HashLinkListRepOptions {
  725. static const char* kName() { return "HashLinkListRepFactoryOptions"; }
  726. size_t bucket_count;
  727. uint32_t threshold_use_skiplist;
  728. size_t huge_page_tlb_size;
  729. int bucket_entries_logging_threshold;
  730. bool if_log_bucket_dist_when_flash;
  731. };
  732. static std::unordered_map<std::string, OptionTypeInfo> hash_linklist_info = {
  733. {"bucket_count",
  734. {offsetof(struct HashLinkListRepOptions, bucket_count), OptionType::kSizeT,
  735. OptionVerificationType::kNormal, OptionTypeFlags::kNone}},
  736. {"threshold",
  737. {offsetof(struct HashLinkListRepOptions, threshold_use_skiplist),
  738. OptionType::kUInt32T, OptionVerificationType::kNormal,
  739. OptionTypeFlags::kNone}},
  740. {"huge_page_size",
  741. {offsetof(struct HashLinkListRepOptions, huge_page_tlb_size),
  742. OptionType::kSizeT, OptionVerificationType::kNormal,
  743. OptionTypeFlags::kNone}},
  744. {"logging_threshold",
  745. {offsetof(struct HashLinkListRepOptions, bucket_entries_logging_threshold),
  746. OptionType::kInt, OptionVerificationType::kNormal,
  747. OptionTypeFlags::kNone}},
  748. {"log_when_flash",
  749. {offsetof(struct HashLinkListRepOptions, if_log_bucket_dist_when_flash),
  750. OptionType::kBoolean, OptionVerificationType::kNormal,
  751. OptionTypeFlags::kNone}},
  752. };
  753. class HashLinkListRepFactory : public MemTableRepFactory {
  754. public:
  755. explicit HashLinkListRepFactory(size_t bucket_count,
  756. uint32_t threshold_use_skiplist,
  757. size_t huge_page_tlb_size,
  758. int bucket_entries_logging_threshold,
  759. bool if_log_bucket_dist_when_flash) {
  760. options_.bucket_count = bucket_count;
  761. options_.threshold_use_skiplist = threshold_use_skiplist;
  762. options_.huge_page_tlb_size = huge_page_tlb_size;
  763. options_.bucket_entries_logging_threshold =
  764. bucket_entries_logging_threshold;
  765. options_.if_log_bucket_dist_when_flash = if_log_bucket_dist_when_flash;
  766. RegisterOptions(&options_, &hash_linklist_info);
  767. }
  768. using MemTableRepFactory::CreateMemTableRep;
  769. MemTableRep* CreateMemTableRep(const MemTableRep::KeyComparator& compare,
  770. Allocator* allocator,
  771. const SliceTransform* transform,
  772. Logger* logger) override;
  773. static const char* kClassName() { return "HashLinkListRepFactory"; }
  774. static const char* kNickName() { return "hash_linkedlist"; }
  775. const char* Name() const override { return kClassName(); }
  776. const char* NickName() const override { return kNickName(); }
  777. private:
  778. HashLinkListRepOptions options_;
  779. };
  780. } // namespace
  781. MemTableRep* HashLinkListRepFactory::CreateMemTableRep(
  782. const MemTableRep::KeyComparator& compare, Allocator* allocator,
  783. const SliceTransform* transform, Logger* logger) {
  784. return new HashLinkListRep(
  785. compare, allocator, transform, options_.bucket_count,
  786. options_.threshold_use_skiplist, options_.huge_page_tlb_size, logger,
  787. options_.bucket_entries_logging_threshold,
  788. options_.if_log_bucket_dist_when_flash);
  789. }
  790. MemTableRepFactory* NewHashLinkListRepFactory(
  791. size_t bucket_count, size_t huge_page_tlb_size,
  792. int bucket_entries_logging_threshold, bool if_log_bucket_dist_when_flash,
  793. uint32_t threshold_use_skiplist) {
  794. return new HashLinkListRepFactory(
  795. bucket_count, threshold_use_skiplist, huge_page_tlb_size,
  796. bucket_entries_logging_threshold, if_log_bucket_dist_when_flash);
  797. }
  798. } // namespace ROCKSDB_NAMESPACE