libcuckoo  0.3.1
cuckoohash_map.hh
Go to the documentation of this file.
1 
3 #ifndef _CUCKOOHASH_MAP_HH
4 #define _CUCKOOHASH_MAP_HH
5 
6 #include <algorithm>
7 #include <array>
8 #include <atomic>
9 #include <bitset>
10 #include <cassert>
11 #include <cstdint>
12 #include <cstdlib>
13 #include <functional>
14 #include <iostream>
15 #include <iterator>
16 #include <limits>
17 #include <list>
18 #include <memory>
19 #include <mutex>
20 #include <stdexcept>
21 #include <string>
22 #include <thread>
23 #include <type_traits>
24 #include <utility>
25 #include <vector>
26 
27 #include "cuckoohash_config.hh"
28 #include "cuckoohash_util.hh"
29 #include "bucket_container.hh"
30 
31 namespace libcuckoo {
32 
45 template <class Key, class T, class Hash = std::hash<Key>,
46  class KeyEqual = std::equal_to<Key>,
47  class Allocator = std::allocator<std::pair<const Key, T>>,
48  std::size_t SLOT_PER_BUCKET = DEFAULT_SLOT_PER_BUCKET>
50 private:
51  // Type of the partial key
52  using partial_t = uint8_t;
53 
54  // The type of the buckets container
55  using buckets_t =
57 
58 public:
62  using key_type = typename buckets_t::key_type;
63  using mapped_type = typename buckets_t::mapped_type;
69  using value_type = typename buckets_t::value_type;
70  using size_type = typename buckets_t::size_type;
71  using difference_type = std::ptrdiff_t;
72  using hasher = Hash;
73  using key_equal = KeyEqual;
74  using allocator_type = typename buckets_t::allocator_type;
75  using reference = typename buckets_t::reference;
76  using const_reference = typename buckets_t::const_reference;
77  using pointer = typename buckets_t::pointer;
78  using const_pointer = typename buckets_t::const_pointer;
79  class locked_table;
80 
89  static constexpr uint16_t slot_per_bucket() { return SLOT_PER_BUCKET; }
90 
104  cuckoohash_map(size_type n = DEFAULT_SIZE, const Hash &hf = Hash(),
105  const KeyEqual &equal = KeyEqual(),
106  const Allocator &alloc = Allocator())
107  : hash_fn_(hf), eq_fn_(equal),
108  buckets_(reserve_calc(n), alloc),
109  old_buckets_(0, alloc),
110  all_locks_(get_allocator()),
111  num_remaining_lazy_rehash_locks_(0),
112  minimum_load_factor_(DEFAULT_MINIMUM_LOAD_FACTOR),
113  maximum_hashpower_(NO_MAXIMUM_HASHPOWER),
114  max_num_worker_threads_(0) {
115  all_locks_.emplace_back(get_allocator());
116  all_locks_.back().resize(std::min(bucket_count(), size_type(kMaxNumLocks)));
117  }
118 
131  template <typename InputIt>
132  cuckoohash_map(InputIt first, InputIt last,
133  size_type n = DEFAULT_SIZE, const Hash &hf = Hash(),
134  const KeyEqual &equal = KeyEqual(),
135  const Allocator &alloc = Allocator())
136  : cuckoohash_map(n, hf, equal, alloc) {
137  for (; first != last; ++first) {
138  insert(first->first, first->second);
139  }
140  }
141 
148  cuckoohash_map(const cuckoohash_map &other) = default;
149 
157  cuckoohash_map(const cuckoohash_map &other, const Allocator &alloc)
158  : hash_fn_(other.hash_fn_), eq_fn_(other.eq_fn_),
159  buckets_(other.buckets_, alloc),
160  old_buckets_(other.old_buckets_, alloc),
161  all_locks_(alloc),
162  num_remaining_lazy_rehash_locks_(
163  other.num_remaining_lazy_rehash_locks_),
164  minimum_load_factor_(other.minimum_load_factor_),
165  maximum_hashpower_(other.maximum_hashpower_),
166  max_num_worker_threads_(other.max_num_worker_threads_) {
167  if (other.get_allocator() == alloc) {
168  all_locks_ = other.all_locks_;
169  } else {
170  add_locks_from_other(other);
171  }
172  }
173 
180  cuckoohash_map(cuckoohash_map &&other) = default;
181 
189  cuckoohash_map(cuckoohash_map &&other, const Allocator &alloc)
190  : hash_fn_(std::move(other.hash_fn_)), eq_fn_(std::move(other.eq_fn_)),
191  buckets_(std::move(other.buckets_), alloc),
192  old_buckets_(std::move(other.old_buckets_), alloc),
193  all_locks_(alloc),
194  num_remaining_lazy_rehash_locks_(
195  other.num_remaining_lazy_rehash_locks_),
196  minimum_load_factor_(other.minimum_load_factor_),
197  maximum_hashpower_(other.maximum_hashpower_),
198  max_num_worker_threads_(other.max_num_worker_threads_) {
199  if (other.get_allocator() == alloc) {
200  all_locks_ = std::move(other.all_locks_);
201  } else {
202  add_locks_from_other(other);
203  }
204  }
205 
215  cuckoohash_map(std::initializer_list<value_type> init,
216  size_type n = DEFAULT_SIZE, const Hash &hf = Hash(),
217  const KeyEqual &equal = KeyEqual(),
218  const Allocator &alloc = Allocator())
219  : cuckoohash_map(init.begin(), init.end(), n, hf, equal, alloc) {}
220 
226  void swap(cuckoohash_map &other) noexcept {
227  std::swap(hash_fn_, other.hash_fn_);
228  std::swap(eq_fn_, other.eq_fn_);
229  buckets_.swap(other.buckets_);
230  all_locks_.swap(other.all_locks_);
231  other.minimum_load_factor_.store(
232  minimum_load_factor_.exchange(other.minimum_load_factor(),
233  std::memory_order_release),
234  std::memory_order_release);
235  other.maximum_hashpower_.store(
236  maximum_hashpower_.exchange(other.maximum_hashpower(),
237  std::memory_order_release),
238  std::memory_order_release);
239  }
240 
248  cuckoohash_map &operator=(const cuckoohash_map &other) = default;
249 
257  cuckoohash_map &operator=(cuckoohash_map &&other) = default;
258 
265  cuckoohash_map &operator=(std::initializer_list<value_type> ilist) {
266  clear();
267  for (const auto &item : ilist) {
268  insert(item.first, item.second);
269  }
270  return *this;
271  }
272 
291  hasher hash_function() const { return hash_fn_; }
292 
298  key_equal key_eq() const { return eq_fn_; }
299 
305  allocator_type get_allocator() const { return buckets_.get_allocator(); }
306 
313  size_type hashpower() const { return buckets_.hashpower(); }
314 
320  size_type bucket_count() const { return buckets_.size(); }
321 
327  bool empty() const { return size() == 0; }
328 
334  size_type size() const {
335  if (all_locks_.size() == 0) {
336  return 0;
337  }
338  counter_type s = 0;
339  for (spinlock &lock : get_current_locks()) {
340  s += lock.elem_counter();
341  }
342  assert(s >= 0);
343  return static_cast<size_type>(s);
344  }
345 
351  size_type capacity() const { return bucket_count() * slot_per_bucket(); }
352 
359  double load_factor() const {
360  return static_cast<double>(size()) / static_cast<double>(capacity());
361  }
362 
373  void minimum_load_factor(const double mlf) {
374  if (mlf < 0.0) {
375  throw std::invalid_argument("load factor " + std::to_string(mlf) +
376  " cannot be "
377  "less than 0");
378  } else if (mlf > 1.0) {
379  throw std::invalid_argument("load factor " + std::to_string(mlf) +
380  " cannot be "
381  "greater than 1");
382  }
383  minimum_load_factor_.store(mlf, std::memory_order_release);
384  }
385 
391  double minimum_load_factor() const {
392  return minimum_load_factor_.load(std::memory_order_acquire);
393  }
394 
404  void maximum_hashpower(size_type mhp) {
405  if (hashpower() > mhp) {
406  throw std::invalid_argument("maximum hashpower " + std::to_string(mhp) +
407  " is less than current hashpower");
408  }
409  maximum_hashpower_.store(mhp, std::memory_order_release);
410  }
411 
417  size_type maximum_hashpower() const {
418  return maximum_hashpower_.load(std::memory_order_acquire);
419  }
420 
421 
433  void max_num_worker_threads(size_type extra_threads) {
434  max_num_worker_threads_.store(extra_threads, std::memory_order_release);
435  }
436 
440  size_type max_num_worker_threads() const {
441  return max_num_worker_threads_.load(std::memory_order_acquire);
442  }
443 
465  template <typename K, typename F> bool find_fn(const K &key, F fn) const {
466  const hash_value hv = hashed_key(key);
467  const auto b = snapshot_and_lock_two<normal_mode>(hv);
468  const table_position pos = cuckoo_find(key, hv.partial, b.i1, b.i2);
469  if (pos.status == ok) {
470  fn(buckets_[pos.index].mapped(pos.slot));
471  return true;
472  } else {
473  return false;
474  }
475  }
476 
488  template <typename K, typename F> bool update_fn(const K &key, F fn) {
489  const hash_value hv = hashed_key(key);
490  const auto b = snapshot_and_lock_two<normal_mode>(hv);
491  const table_position pos = cuckoo_find(key, hv.partial, b.i1, b.i2);
492  if (pos.status == ok) {
493  fn(buckets_[pos.index].mapped(pos.slot));
494  return true;
495  } else {
496  return false;
497  }
498  }
499 
512  template <typename K, typename F> bool erase_fn(const K &key, F fn) {
513  const hash_value hv = hashed_key(key);
514  const auto b = snapshot_and_lock_two<normal_mode>(hv);
515  const table_position pos = cuckoo_find(key, hv.partial, b.i1, b.i2);
516  if (pos.status == ok) {
517  if (fn(buckets_[pos.index].mapped(pos.slot))) {
518  del_from_bucket(pos.index, pos.slot);
519  }
520  return true;
521  } else {
522  return false;
523  }
524  }
525 
554  template <typename K, typename F, typename... Args>
555  bool uprase_fn(K &&key, F fn, Args &&... val) {
556  hash_value hv = hashed_key(key);
557  auto b = snapshot_and_lock_two<normal_mode>(hv);
558  table_position pos = cuckoo_insert_loop<normal_mode>(hv, b, key);
559  UpsertContext upsert_context;
560  if (pos.status == ok) {
561  add_to_bucket(pos.index, pos.slot, hv.partial, std::forward<K>(key),
562  std::forward<Args>(val)...);
563  upsert_context = UpsertContext::NEWLY_INSERTED;
564  } else {
565  upsert_context = UpsertContext::ALREADY_EXISTED;
566  }
567  using CanInvokeWithUpsertContextT =
568  typename internal::CanInvokeWithUpsertContext<F, mapped_type>::type;
569  if (internal::InvokeUpraseFn(fn, buckets_[pos.index].mapped(pos.slot),
570  upsert_context,
571  CanInvokeWithUpsertContextT{})) {
572  del_from_bucket(pos.index, pos.slot);
573  }
574  return pos.status == ok;
575  }
576 
584  template <typename K, typename F, typename... Args>
585  bool upsert(K &&key, F fn, Args &&... val) {
586  constexpr bool kCanInvokeWithUpsertContext =
588  return uprase_fn(
589  std::forward<K>(key),
591  fn},
592  std::forward<Args>(val)...);
593  }
594 
600  template <typename K> bool find(const K &key, mapped_type &val) const {
601  return find_fn(key, [&val](const mapped_type &v) mutable { val = v; });
602  }
603 
612  template <typename K> mapped_type find(const K &key) const {
613  const hash_value hv = hashed_key(key);
614  const auto b = snapshot_and_lock_two<normal_mode>(hv);
615  const table_position pos = cuckoo_find(key, hv.partial, b.i1, b.i2);
616  if (pos.status == ok) {
617  return buckets_[pos.index].mapped(pos.slot);
618  } else {
619  throw std::out_of_range("key not found in table");
620  }
621  }
622 
627  template <typename K> bool contains(const K &key) const {
628  return find_fn(key, [](const mapped_type &) {});
629  }
630 
637  template <typename K, typename V> bool update(const K &key, V &&val) {
638  return update_fn(key, [&val](mapped_type &v) { v = std::forward<V>(val); });
639  }
640 
645  template <typename K, typename... Args> bool insert(K &&key, Args &&... val) {
646  return upsert(std::forward<K>(key), [](mapped_type &) {},
647  std::forward<Args>(val)...);
648  }
649 
656  template <typename K, typename V> bool insert_or_assign(K &&key, V &&val) {
657  return upsert(std::forward<K>(key), [&val](mapped_type &m) { m = val; },
658  std::forward<V>(val));
659  }
660 
665  template <typename K> bool erase(const K &key) {
666  return erase_fn(key, [](mapped_type &) { return true; });
667  }
668 
678  bool rehash(size_type n) { return cuckoo_rehash<normal_mode>(n); }
679 
689  bool reserve(size_type n) { return cuckoo_reserve<normal_mode>(n); }
690 
694  void clear() {
695  auto all_locks_manager = lock_all(normal_mode());
696  cuckoo_clear();
697  }
698 
705  locked_table lock_table() { return locked_table(*this); }
706 
709 private:
710  // Constructor helpers
711 
712  void add_locks_from_other(const cuckoohash_map &other) {
713  locks_t &other_locks = other.get_current_locks();
714  all_locks_.emplace_back(get_allocator());
715  all_locks_.back().resize(other_locks.size());
716  std::copy(other_locks.begin(), other_locks.end(),
717  get_current_locks().begin());
718  }
719 
720  // Hashing types and functions
721 
722  // true if the key is small and simple, which means using partial keys for
723  // lookup would probably slow us down
724  static constexpr bool is_simple() {
725  return std::is_standard_layout<key_type>::value &&
726  std::is_trivial<key_type>::value &&
727  sizeof(key_type) <= 8;
728  }
729 
730  // Whether or not the data is nothrow-move-constructible.
731  static constexpr bool is_data_nothrow_move_constructible() {
732  return std::is_nothrow_move_constructible<key_type>::value &&
733  std::is_nothrow_move_constructible<mapped_type>::value;
734  }
735 
736  // Contains a hash and partial for a given key. The partial key is used for
737  // partial-key cuckoohashing, and for finding the alternate bucket of that a
738  // key hashes to.
739  struct hash_value {
740  size_type hash;
741  partial_t partial;
742  };
743 
744  template <typename K> hash_value hashed_key(const K &key) const {
745  const size_type hash = hash_function()(key);
746  return {hash, partial_key(hash)};
747  }
748 
749  template <typename K> size_type hashed_key_only_hash(const K &key) const {
750  return hash_function()(key);
751  }
752 
753  // hashsize returns the number of buckets corresponding to a given
754  // hashpower.
755  static inline size_type hashsize(const size_type hp) {
756  return size_type(1) << hp;
757  }
758 
759  // hashmask returns the bitmask for the buckets array corresponding to a
760  // given hashpower.
761  static inline size_type hashmask(const size_type hp) {
762  return hashsize(hp) - 1;
763  }
764 
765  // The partial key must only depend on the hash value. It cannot change with
766  // the hashpower, because, in order for `cuckoo_fast_double` to work
767  // properly, the alt_index must only grow by one bit at the top each time we
768  // expand the table.
769  static partial_t partial_key(const size_type hash) {
770  const uint64_t hash_64bit = hash;
771  const uint32_t hash_32bit = (static_cast<uint32_t>(hash_64bit) ^
772  static_cast<uint32_t>(hash_64bit >> 32));
773  const uint16_t hash_16bit = (static_cast<uint16_t>(hash_32bit) ^
774  static_cast<uint16_t>(hash_32bit >> 16));
775  const uint8_t hash_8bit = (static_cast<uint8_t>(hash_16bit) ^
776  static_cast<uint8_t>(hash_16bit >> 8));
777  return hash_8bit;
778  }
779 
780  // index_hash returns the first possible bucket that the given hashed key
781  // could be.
782  static inline size_type index_hash(const size_type hp, const size_type hv) {
783  return hv & hashmask(hp);
784  }
785 
786  // alt_index returns the other possible bucket that the given hashed key
787  // could be. It takes the first possible bucket as a parameter. Note that
788  // this function will return the first possible bucket if index is the
789  // second possible bucket, so alt_index(ti, partial, alt_index(ti, partial,
790  // index_hash(ti, hv))) == index_hash(ti, hv).
791  static inline size_type alt_index(const size_type hp, const partial_t partial,
792  const size_type index) {
793  // ensure tag is nonzero for the multiply. 0xc6a4a7935bd1e995 is the
794  // hash constant from 64-bit MurmurHash2
795  const size_type nonzero_tag = static_cast<size_type>(partial) + 1;
796  return (index ^ (nonzero_tag * 0xc6a4a7935bd1e995)) & hashmask(hp);
797  }
798 
799  // Locking types
800 
801  // Counter type
802  using counter_type = int64_t;
803 
804  // A fast, lightweight spinlock
805  //
806  // Per-spinlock, we also maintain some metadata about the contents of the
807  // table. Storing data per-spinlock avoids false sharing issues when multiple
808  // threads need to update this metadata. We store the following information:
809  //
810  // - elem_counter: A counter indicating how many elements in the table are
811  // under this lock. One can compute the size of the table by summing the
812  // elem_counter over all locks.
813  //
814  // - is_migrated: When resizing with cuckoo_fast_double, we do not
815  // immediately rehash elements from the old buckets array to the new one.
816  // Instead, we'll mark all of the locks as not migrated. So anybody trying to
817  // acquire the lock must also migrate the corresponding buckets if
818  // !is_migrated.
820  class LIBCUCKOO_ALIGNAS(64) spinlock {
821  public:
822  spinlock() : elem_counter_(0), is_migrated_(true) { lock_.clear(); }
823 
824  spinlock(const spinlock &other) noexcept
825  : elem_counter_(other.elem_counter()),
826  is_migrated_(other.is_migrated()) {
827  lock_.clear();
828  }
829 
830  spinlock &operator=(const spinlock &other) noexcept {
831  elem_counter() = other.elem_counter();
832  is_migrated() = other.is_migrated();
833  return *this;
834  }
835 
836  void lock() noexcept {
837  while (lock_.test_and_set(std::memory_order_acq_rel))
838  ;
839  }
840 
841  void unlock() noexcept { lock_.clear(std::memory_order_release); }
842 
843  bool try_lock() noexcept {
844  return !lock_.test_and_set(std::memory_order_acq_rel);
845  }
846 
847  counter_type &elem_counter() noexcept { return elem_counter_; }
848  counter_type elem_counter() const noexcept { return elem_counter_; }
849 
850  bool &is_migrated() noexcept { return is_migrated_; }
851  bool is_migrated() const noexcept { return is_migrated_; }
852 
853  private:
854  std::atomic_flag lock_;
855  counter_type elem_counter_;
856  bool is_migrated_;
857  };
858 
859  template <typename U>
860  using rebind_alloc =
861  typename std::allocator_traits<allocator_type>::template rebind_alloc<U>;
862 
863  using locks_t = std::vector<spinlock, rebind_alloc<spinlock>>;
864  using all_locks_t = std::list<locks_t, rebind_alloc<locks_t>>;
865 
866  // Classes for managing locked buckets. By storing and moving around sets of
867  // locked buckets in these classes, we can ensure that they are unlocked
868  // properly.
869 
870  struct LockDeleter {
871  void operator()(spinlock *l) const { l->unlock(); }
872  };
873 
874  using LockManager = std::unique_ptr<spinlock, LockDeleter>;
875 
876  // Each of the locking methods can operate in two modes: locked_table_mode
877  // and normal_mode. When we're in locked_table_mode, we assume the caller has
878  // already taken all locks on the buckets. We also require that all data is
879  // rehashed immediately, so that the caller never has to look through any
880  // locks. In normal_mode, we actually do take locks, and can rehash lazily.
881  using locked_table_mode = std::integral_constant<bool, true>;
882  using normal_mode = std::integral_constant<bool, false>;
883 
884  class TwoBuckets {
885  public:
886  TwoBuckets() {}
887  TwoBuckets(size_type i1_, size_type i2_, locked_table_mode)
888  : i1(i1_), i2(i2_) {}
889  TwoBuckets(locks_t &locks, size_type i1_, size_type i2_, normal_mode)
890  : i1(i1_), i2(i2_), first_manager_(&locks[lock_ind(i1)]),
891  second_manager_((lock_ind(i1) != lock_ind(i2)) ? &locks[lock_ind(i2)]
892  : nullptr) {}
893 
894  void unlock() {
895  first_manager_.reset();
896  second_manager_.reset();
897  }
898 
899  size_type i1, i2;
900 
901  private:
902  LockManager first_manager_, second_manager_;
903  };
904 
905  struct AllUnlocker {
906  void operator()(cuckoohash_map *map) const {
907  for (auto it = first_locked; it != map->all_locks_.end(); ++it) {
908  locks_t &locks = *it;
909  for (spinlock &lock : locks) {
910  lock.unlock();
911  }
912  }
913  }
914 
915  typename all_locks_t::iterator first_locked;
916  };
917 
918  using AllLocksManager = std::unique_ptr<cuckoohash_map, AllUnlocker>;
919 
920  // This exception is thrown whenever we try to lock a bucket, but the
921  // hashpower is not what was expected
922  class hashpower_changed {};
923 
924  // After taking a lock on the table for the given bucket, this function will
925  // check the hashpower to make sure it is the same as what it was before the
926  // lock was taken. If it isn't unlock the bucket and throw a
927  // hashpower_changed exception.
928  inline void check_hashpower(size_type hp, spinlock &lock) const {
929  if (hashpower() != hp) {
930  lock.unlock();
931  LIBCUCKOO_DBG("%s", "hashpower changed\n");
932  throw hashpower_changed();
933  }
934  }
935 
936  // If necessary, rehashes the buckets corresponding to the given lock index,
937  // and sets the is_migrated flag to true. We should only ever do migrations
938  // if the data is nothrow move constructible, so this function is noexcept.
939  //
940  // This only works if our current locks array is at the maximum size, because
941  // otherwise, rehashing could require taking other locks. Assumes the lock at
942  // the given index is taken.
943  //
944  // If IS_LAZY is true, we assume the lock is being rehashed in a lazy
945  // (on-demand) fashion, so we additionally decrement the number of locks we
946  // need to lazy_rehash. This may trigger false sharing with other
947  // lazy-rehashing threads, but the hope is that the fraction of such
948  // operations is low-enough to not significantly impact overall performance.
949  static constexpr bool kIsLazy = true;
950  static constexpr bool kIsNotLazy = false;
951 
952  template <bool IS_LAZY>
953  void rehash_lock(size_t l) const noexcept {
954  locks_t &locks = get_current_locks();
955  spinlock &lock = locks[l];
956  if (lock.is_migrated()) return;
957 
958  assert(is_data_nothrow_move_constructible());
959  assert(locks.size() == kMaxNumLocks);
960  assert(old_buckets_.hashpower() + 1 == buckets_.hashpower());
961  assert(old_buckets_.size() >= kMaxNumLocks);
962  // Iterate through all buckets in old_buckets that are controlled by this
963  // lock, and move them into the current buckets array.
964  for (size_type bucket_ind = l; bucket_ind < old_buckets_.size();
965  bucket_ind += kMaxNumLocks) {
966  move_bucket(old_buckets_, buckets_, bucket_ind);
967  }
968  lock.is_migrated() = true;
969 
970  if (IS_LAZY) {
971  decrement_num_remaining_lazy_rehash_locks();
972  }
973  }
974 
975  // locks the given bucket index.
976  //
977  // throws hashpower_changed if it changed after taking the lock.
978  LockManager lock_one(size_type, size_type, locked_table_mode) const {
979  return LockManager();
980  }
981 
982  LockManager lock_one(size_type hp, size_type i, normal_mode) const {
983  locks_t &locks = get_current_locks();
984  const size_type l = lock_ind(i);
985  spinlock &lock = locks[l];
986  lock.lock();
987  check_hashpower(hp, lock);
988  rehash_lock<kIsLazy>(l);
989  return LockManager(&lock);
990  }
991 
992  // locks the two bucket indexes, always locking the earlier index first to
993  // avoid deadlock. If the two indexes are the same, it just locks one.
994  //
995  // throws hashpower_changed if it changed after taking the lock.
996  TwoBuckets lock_two(size_type, size_type i1, size_type i2,
997  locked_table_mode) const {
998  return TwoBuckets(i1, i2, locked_table_mode());
999  }
1000 
1001  TwoBuckets lock_two(size_type hp, size_type i1, size_type i2,
1002  normal_mode) const {
1003  size_type l1 = lock_ind(i1);
1004  size_type l2 = lock_ind(i2);
1005  if (l2 < l1) {
1006  std::swap(l1, l2);
1007  }
1008  locks_t &locks = get_current_locks();
1009  locks[l1].lock();
1010  check_hashpower(hp, locks[l1]);
1011  if (l2 != l1) {
1012  locks[l2].lock();
1013  }
1014  rehash_lock<kIsLazy>(l1);
1015  rehash_lock<kIsLazy>(l2);
1016  return TwoBuckets(locks, i1, i2, normal_mode());
1017  }
1018 
1019  // lock_three locks the three bucket indexes in numerical order, returning
1020  // the containers as a two (i1 and i2) and a one (i3). The one will not be
1021  // active if i3 shares a lock index with i1 or i2.
1022  //
1023  // throws hashpower_changed if it changed after taking the lock.
1024  std::pair<TwoBuckets, LockManager> lock_three(size_type, size_type i1,
1025  size_type i2, size_type,
1026  locked_table_mode) const {
1027  return std::make_pair(TwoBuckets(i1, i2, locked_table_mode()),
1028  LockManager());
1029  }
1030 
1031  std::pair<TwoBuckets, LockManager> lock_three(size_type hp, size_type i1,
1032  size_type i2, size_type i3,
1033  normal_mode) const {
1034  std::array<size_type, 3> l{{lock_ind(i1), lock_ind(i2), lock_ind(i3)}};
1035  // Lock in order.
1036  if (l[2] < l[1])
1037  std::swap(l[2], l[1]);
1038  if (l[2] < l[0])
1039  std::swap(l[2], l[0]);
1040  if (l[1] < l[0])
1041  std::swap(l[1], l[0]);
1042  locks_t &locks = get_current_locks();
1043  locks[l[0]].lock();
1044  check_hashpower(hp, locks[l[0]]);
1045  if (l[1] != l[0]) {
1046  locks[l[1]].lock();
1047  }
1048  if (l[2] != l[1]) {
1049  locks[l[2]].lock();
1050  }
1051  rehash_lock<kIsLazy>(l[0]);
1052  rehash_lock<kIsLazy>(l[1]);
1053  rehash_lock<kIsLazy>(l[2]);
1054  return std::make_pair(TwoBuckets(locks, i1, i2, normal_mode()),
1055  LockManager((lock_ind(i3) == lock_ind(i1) ||
1056  lock_ind(i3) == lock_ind(i2))
1057  ? nullptr
1058  : &locks[lock_ind(i3)]));
1059  }
1060 
1061  // snapshot_and_lock_two loads locks the buckets associated with the given
1062  // hash value, making sure the hashpower doesn't change before the locks are
1063  // taken. Thus it ensures that the buckets and locks corresponding to the
1064  // hash value will stay correct as long as the locks are held. It returns
1065  // the bucket indices associated with the hash value and the current
1066  // hashpower.
1067  template <typename TABLE_MODE>
1068  TwoBuckets snapshot_and_lock_two(const hash_value &hv) const {
1069  while (true) {
1070  // Keep the current hashpower and locks we're using to compute the buckets
1071  const size_type hp = hashpower();
1072  const size_type i1 = index_hash(hp, hv.hash);
1073  const size_type i2 = alt_index(hp, hv.partial, i1);
1074  try {
1075  return lock_two(hp, i1, i2, TABLE_MODE());
1076  } catch (hashpower_changed &) {
1077  // The hashpower changed while taking the locks. Try again.
1078  continue;
1079  }
1080  }
1081  }
1082 
1083  // lock_all takes all the locks, and returns a deleter object that releases
1084  // the locks upon destruction. It does NOT perform any hashpower checks, or
1085  // rehash any un-migrated buckets.
1086  //
1087  // Note that after taking all the locks, it is okay to resize the buckets_
1088  // container, since no other threads should be accessing the buckets.
1089  AllLocksManager lock_all(locked_table_mode) {
1090  return AllLocksManager();
1091  }
1092 
1093  AllLocksManager lock_all(normal_mode) {
1094  // all_locks_ should never decrease in size, so if it is non-empty now, it
1095  // will remain non-empty
1096  assert(!all_locks_.empty());
1097  const auto first_locked = std::prev(all_locks_.end());
1098  auto current_locks = first_locked;
1099  while (current_locks != all_locks_.end()) {
1100  locks_t &locks = *current_locks;
1101  for (spinlock &lock : locks) {
1102  lock.lock();
1103  }
1104  ++current_locks;
1105  }
1106  // Once we have taken all the locks of the "current" container, nobody
1107  // else can do locking operations on the table.
1108  return AllLocksManager(this, AllUnlocker{first_locked});
1109  }
1110 
1111  // lock_ind converts an index into buckets to an index into locks.
1112  static inline size_type lock_ind(const size_type bucket_ind) {
1113  return bucket_ind & (kMaxNumLocks - 1);
1114  }
1115 
1116  // Data storage types and functions
1117 
1118  // The type of the bucket
1119  using bucket = typename buckets_t::bucket;
1120 
1121  // Status codes for internal functions
1122 
1123  enum cuckoo_status {
1124  ok,
1125  failure,
1126  failure_key_not_found,
1127  failure_key_duplicated,
1128  failure_table_full,
1129  failure_under_expansion,
1130  };
1131 
1132  // A composite type for functions that need to return a table position, and
1133  // a status code.
1134  struct table_position {
1135  size_type index;
1136  size_type slot;
1137  cuckoo_status status;
1138  };
1139 
1140  // Searching types and functions
1141 
1142  // cuckoo_find searches the table for the given key, returning the position
1143  // of the element found, or a failure status code if the key wasn't found.
1144  // It expects the locks to be taken and released outside the function.
1145  template <typename K>
1146  table_position cuckoo_find(const K &key, const partial_t partial,
1147  const size_type i1, const size_type i2) const {
1148  int slot = try_read_from_bucket(buckets_[i1], partial, key);
1149  if (slot != -1) {
1150  return table_position{i1, static_cast<size_type>(slot), ok};
1151  }
1152  slot = try_read_from_bucket(buckets_[i2], partial, key);
1153  if (slot != -1) {
1154  return table_position{i2, static_cast<size_type>(slot), ok};
1155  }
1156  return table_position{0, 0, failure_key_not_found};
1157  }
1158 
1159  // try_read_from_bucket will search the bucket for the given key and return
1160  // the index of the slot if found, or -1 if not found.
1161  template <typename K>
1162  int try_read_from_bucket(const bucket &b, const partial_t partial,
1163  const K &key) const {
1164  // Silence a warning from MSVC about partial being unused if is_simple.
1165  (void)partial;
1166  for (int i = 0; i < static_cast<int>(slot_per_bucket()); ++i) {
1167  if (!b.occupied(i) || (!is_simple() && partial != b.partial(i))) {
1168  continue;
1169  } else if (key_eq()(b.key(i), key)) {
1170  return i;
1171  }
1172  }
1173  return -1;
1174  }
1175 
1176  // Insertion types and function
1177 
1191  template <typename TABLE_MODE, typename K>
1192  table_position cuckoo_insert_loop(hash_value hv, TwoBuckets &b, K &key) {
1193  table_position pos;
1194  while (true) {
1195  const size_type hp = hashpower();
1196  pos = cuckoo_insert<TABLE_MODE>(hv, b, key);
1197  switch (pos.status) {
1198  case ok:
1199  case failure_key_duplicated:
1200  return pos;
1201  case failure_table_full:
1202  // Expand the table and try again, re-grabbing the locks
1203  cuckoo_fast_double<TABLE_MODE, automatic_resize>(hp);
1204  b = snapshot_and_lock_two<TABLE_MODE>(hv);
1205  break;
1206  case failure_under_expansion:
1207  // The table was under expansion while we were cuckooing. Re-grab the
1208  // locks and try again.
1209  b = snapshot_and_lock_two<TABLE_MODE>(hv);
1210  break;
1211  default:
1212  assert(false);
1213  }
1214  }
1215  }
1216 
1217  // cuckoo_insert tries to find an empty slot in either of the buckets to
1218  // insert the given key into, performing cuckoo hashing if necessary. It
1219  // expects the locks to be taken outside the function. Before inserting, it
1220  // checks that the key isn't already in the table. cuckoo hashing presents
1221  // multiple concurrency issues, which are explained in the function. The
1222  // following return states are possible:
1223  //
1224  // ok -- Found an empty slot, locks will be held on both buckets after the
1225  // function ends, and the position of the empty slot is returned
1226  //
1227  // failure_key_duplicated -- Found a duplicate key, locks will be held, and
1228  // the position of the duplicate key will be returned
1229  //
1230  // failure_under_expansion -- Failed due to a concurrent expansion
1231  // operation. Locks are released. No meaningful position is returned.
1232  //
1233  // failure_table_full -- Failed to find an empty slot for the table. Locks
1234  // are released. No meaningful position is returned.
1235  template <typename TABLE_MODE, typename K>
1236  table_position cuckoo_insert(const hash_value hv, TwoBuckets &b, K &key) {
1237  int res1, res2;
1238  bucket &b1 = buckets_[b.i1];
1239  if (!try_find_insert_bucket(b1, res1, hv.partial, key)) {
1240  return table_position{b.i1, static_cast<size_type>(res1),
1241  failure_key_duplicated};
1242  }
1243  bucket &b2 = buckets_[b.i2];
1244  if (!try_find_insert_bucket(b2, res2, hv.partial, key)) {
1245  return table_position{b.i2, static_cast<size_type>(res2),
1246  failure_key_duplicated};
1247  }
1248  if (res1 != -1) {
1249  return table_position{b.i1, static_cast<size_type>(res1), ok};
1250  }
1251  if (res2 != -1) {
1252  return table_position{b.i2, static_cast<size_type>(res2), ok};
1253  }
1254 
1255  // We are unlucky, so let's perform cuckoo hashing.
1256  size_type insert_bucket = 0;
1257  size_type insert_slot = 0;
1258  cuckoo_status st = run_cuckoo<TABLE_MODE>(b, insert_bucket, insert_slot);
1259  if (st == failure_under_expansion) {
1260  // The run_cuckoo operation operated on an old version of the table,
1261  // so we have to try again. We signal to the calling insert method
1262  // to try again by returning failure_under_expansion.
1263  return table_position{0, 0, failure_under_expansion};
1264  } else if (st == ok) {
1265  assert(TABLE_MODE() == locked_table_mode() ||
1266  !get_current_locks()[lock_ind(b.i1)].try_lock());
1267  assert(TABLE_MODE() == locked_table_mode() ||
1268  !get_current_locks()[lock_ind(b.i2)].try_lock());
1269  assert(!buckets_[insert_bucket].occupied(insert_slot));
1270  assert(insert_bucket == index_hash(hashpower(), hv.hash) ||
1271  insert_bucket == alt_index(hashpower(), hv.partial,
1272  index_hash(hashpower(), hv.hash)));
1273  // Since we unlocked the buckets during run_cuckoo, another insert
1274  // could have inserted the same key into either b.i1 or
1275  // b.i2, so we check for that before doing the insert.
1276  table_position pos = cuckoo_find(key, hv.partial, b.i1, b.i2);
1277  if (pos.status == ok) {
1278  pos.status = failure_key_duplicated;
1279  return pos;
1280  }
1281  return table_position{insert_bucket, insert_slot, ok};
1282  }
1283  assert(st == failure);
1284  LIBCUCKOO_DBG("hash table is full (hashpower = %zu, hash_items = %zu,"
1285  "load factor = %.2f), need to increase hashpower\n",
1286  hashpower(), size(), load_factor());
1287  return table_position{0, 0, failure_table_full};
1288  }
1289 
1290  // add_to_bucket will insert the given key-value pair into the slot. The key
1291  // and value will be move-constructed into the table, so they are not valid
1292  // for use afterwards.
1293  template <typename K, typename... Args>
1294  void add_to_bucket(const size_type bucket_ind, const size_type slot,
1295  const partial_t partial, K &&key, Args &&... val) {
1296  buckets_.setKV(bucket_ind, slot, partial, std::forward<K>(key),
1297  std::forward<Args>(val)...);
1298  ++get_current_locks()[lock_ind(bucket_ind)].elem_counter();
1299  }
1300 
1301  // try_find_insert_bucket will search the bucket for the given key, and for
1302  // an empty slot. If the key is found, we store the slot of the key in
1303  // `slot` and return false. If we find an empty slot, we store its position
1304  // in `slot` and return true. If no duplicate key is found and no empty slot
1305  // is found, we store -1 in `slot` and return true.
1306  template <typename K>
1307  bool try_find_insert_bucket(const bucket &b, int &slot,
1308  const partial_t partial, const K &key) const {
1309  // Silence a warning from MSVC about partial being unused if is_simple.
1310  (void)partial;
1311  slot = -1;
1312  for (int i = 0; i < static_cast<int>(slot_per_bucket()); ++i) {
1313  if (b.occupied(i)) {
1314  if (!is_simple() && partial != b.partial(i)) {
1315  continue;
1316  }
1317  if (key_eq()(b.key(i), key)) {
1318  slot = i;
1319  return false;
1320  }
1321  } else {
1322  slot = i;
1323  }
1324  }
1325  return true;
1326  }
1327 
1328  // CuckooRecord holds one position in a cuckoo path. Since cuckoopath
1329  // elements only define a sequence of alternate hashings for different hash
1330  // values, we only need to keep track of the hash values being moved, rather
1331  // than the keys themselves.
1332  typedef struct {
1333  size_type bucket;
1334  size_type slot;
1335  hash_value hv;
1336  } CuckooRecord;
1337 
1338  // The maximum number of items in a cuckoo BFS path. It determines the
1339  // maximum number of slots we search when cuckooing.
1340  static constexpr uint8_t MAX_BFS_PATH_LEN = 5;
1341 
1342  // An array of CuckooRecords
1343  using CuckooRecords = std::array<CuckooRecord, MAX_BFS_PATH_LEN>;
1344 
1345  // run_cuckoo performs cuckoo hashing on the table in an attempt to free up
1346  // a slot on either of the insert buckets, which are assumed to be locked
1347  // before the start. On success, the bucket and slot that was freed up is
1348  // stored in insert_bucket and insert_slot. In order to perform the search
1349  // and the swaps, it has to release the locks, which can lead to certain
1350  // concurrency issues, the details of which are explained in the function.
1351  // If run_cuckoo returns ok (success), then `b` will be active, otherwise it
1352  // will not.
1353  template <typename TABLE_MODE>
1354  cuckoo_status run_cuckoo(TwoBuckets &b, size_type &insert_bucket,
1355  size_type &insert_slot) {
1356  // We must unlock the buckets here, so that cuckoopath_search and
1357  // cuckoopath_move can lock buckets as desired without deadlock.
1358  // cuckoopath_move has to move something out of one of the original
1359  // buckets as its last operation, and it will lock both buckets and
1360  // leave them locked after finishing. This way, we know that if
1361  // cuckoopath_move succeeds, then the buckets needed for insertion are
1362  // still locked. If cuckoopath_move fails, the buckets are unlocked and
1363  // we try again. This unlocking does present two problems. The first is
1364  // that another insert on the same key runs and, finding that the key
1365  // isn't in the table, inserts the key into the table. Then we insert
1366  // the key into the table, causing a duplication. To check for this, we
1367  // search the buckets for the key we are trying to insert before doing
1368  // so (this is done in cuckoo_insert, and requires that both buckets are
1369  // locked). Another problem is that an expansion runs and changes the
1370  // hashpower, meaning the buckets may not be valid anymore. In this
1371  // case, the cuckoopath functions will have thrown a hashpower_changed
1372  // exception, which we catch and handle here.
1373  size_type hp = hashpower();
1374  b.unlock();
1375  CuckooRecords cuckoo_path;
1376  bool done = false;
1377  try {
1378  while (!done) {
1379  const int depth =
1380  cuckoopath_search<TABLE_MODE>(hp, cuckoo_path, b.i1, b.i2);
1381  if (depth < 0) {
1382  break;
1383  }
1384 
1385  if (cuckoopath_move<TABLE_MODE>(hp, cuckoo_path, depth, b)) {
1386  insert_bucket = cuckoo_path[0].bucket;
1387  insert_slot = cuckoo_path[0].slot;
1388  assert(insert_bucket == b.i1 || insert_bucket == b.i2);
1389  assert(TABLE_MODE() == locked_table_mode() ||
1390  !get_current_locks()[lock_ind(b.i1)].try_lock());
1391  assert(TABLE_MODE() == locked_table_mode() ||
1392  !get_current_locks()[lock_ind(b.i2)].try_lock());
1393  assert(!buckets_[insert_bucket].occupied(insert_slot));
1394  done = true;
1395  break;
1396  }
1397  }
1398  } catch (hashpower_changed &) {
1399  // The hashpower changed while we were trying to cuckoo, which means
1400  // we want to retry. b.i1 and b.i2 should not be locked
1401  // in this case.
1402  return failure_under_expansion;
1403  }
1404  return done ? ok : failure;
1405  }
1406 
1407  // cuckoopath_search finds a cuckoo path from one of the starting buckets to
1408  // an empty slot in another bucket. It returns the depth of the discovered
1409  // cuckoo path on success, and -1 on failure. Since it doesn't take locks on
1410  // the buckets it searches, the data can change between this function and
1411  // cuckoopath_move. Thus cuckoopath_move checks that the data matches the
1412  // cuckoo path before changing it.
1413  //
1414  // throws hashpower_changed if it changed during the search.
1415  template <typename TABLE_MODE>
1416  int cuckoopath_search(const size_type hp, CuckooRecords &cuckoo_path,
1417  const size_type i1, const size_type i2) {
1418  b_slot x = slot_search<TABLE_MODE>(hp, i1, i2);
1419  if (x.depth == -1) {
1420  return -1;
1421  }
1422  // Fill in the cuckoo path slots from the end to the beginning.
1423  for (int i = x.depth; i >= 0; i--) {
1424  cuckoo_path[i].slot = x.pathcode % slot_per_bucket();
1425  x.pathcode /= slot_per_bucket();
1426  }
1427  // Fill in the cuckoo_path buckets and keys from the beginning to the
1428  // end, using the final pathcode to figure out which bucket the path
1429  // starts on. Since data could have been modified between slot_search
1430  // and the computation of the cuckoo path, this could be an invalid
1431  // cuckoo_path.
1432  CuckooRecord &first = cuckoo_path[0];
1433  if (x.pathcode == 0) {
1434  first.bucket = i1;
1435  } else {
1436  assert(x.pathcode == 1);
1437  first.bucket = i2;
1438  }
1439  {
1440  const auto lock_manager = lock_one(hp, first.bucket, TABLE_MODE());
1441  const bucket &b = buckets_[first.bucket];
1442  if (!b.occupied(first.slot)) {
1443  // We can terminate here
1444  return 0;
1445  }
1446  first.hv = hashed_key(b.key(first.slot));
1447  }
1448  for (int i = 1; i <= x.depth; ++i) {
1449  CuckooRecord &curr = cuckoo_path[i];
1450  const CuckooRecord &prev = cuckoo_path[i - 1];
1451  assert(prev.bucket == index_hash(hp, prev.hv.hash) ||
1452  prev.bucket ==
1453  alt_index(hp, prev.hv.partial, index_hash(hp, prev.hv.hash)));
1454  // We get the bucket that this slot is on by computing the alternate
1455  // index of the previous bucket
1456  curr.bucket = alt_index(hp, prev.hv.partial, prev.bucket);
1457  const auto lock_manager = lock_one(hp, curr.bucket, TABLE_MODE());
1458  const bucket &b = buckets_[curr.bucket];
1459  if (!b.occupied(curr.slot)) {
1460  // We can terminate here
1461  return i;
1462  }
1463  curr.hv = hashed_key(b.key(curr.slot));
1464  }
1465  return x.depth;
1466  }
1467 
1468  // cuckoopath_move moves keys along the given cuckoo path in order to make
1469  // an empty slot in one of the buckets in cuckoo_insert. Before the start of
1470  // this function, the two insert-locked buckets were unlocked in run_cuckoo.
1471  // At the end of the function, if the function returns true (success), then
1472  // both insert-locked buckets remain locked. If the function is
1473  // unsuccessful, then both insert-locked buckets will be unlocked.
1474  //
1475  // throws hashpower_changed if it changed during the move.
1476  template <typename TABLE_MODE>
1477  bool cuckoopath_move(const size_type hp, CuckooRecords &cuckoo_path,
1478  size_type depth, TwoBuckets &b) {
1479  if (depth == 0) {
1480  // There is a chance that depth == 0, when try_add_to_bucket sees
1481  // both buckets as full and cuckoopath_search finds one empty. In
1482  // this case, we lock both buckets. If the slot that
1483  // cuckoopath_search found empty isn't empty anymore, we unlock them
1484  // and return false. Otherwise, the bucket is empty and insertable,
1485  // so we hold the locks and return true.
1486  const size_type bucket_i = cuckoo_path[0].bucket;
1487  assert(bucket_i == b.i1 || bucket_i == b.i2);
1488  b = lock_two(hp, b.i1, b.i2, TABLE_MODE());
1489  if (!buckets_[bucket_i].occupied(cuckoo_path[0].slot)) {
1490  return true;
1491  } else {
1492  b.unlock();
1493  return false;
1494  }
1495  }
1496 
1497  while (depth > 0) {
1498  CuckooRecord &from = cuckoo_path[depth - 1];
1499  CuckooRecord &to = cuckoo_path[depth];
1500  const size_type fs = from.slot;
1501  const size_type ts = to.slot;
1502  TwoBuckets twob;
1503  LockManager extra_manager;
1504  if (depth == 1) {
1505  // Even though we are only swapping out of one of the original
1506  // buckets, we have to lock both of them along with the slot we
1507  // are swapping to, since at the end of this function, they both
1508  // must be locked. We store tb inside the extrab container so it
1509  // is unlocked at the end of the loop.
1510  std::tie(twob, extra_manager) =
1511  lock_three(hp, b.i1, b.i2, to.bucket, TABLE_MODE());
1512  } else {
1513  twob = lock_two(hp, from.bucket, to.bucket, TABLE_MODE());
1514  }
1515 
1516  bucket &fb = buckets_[from.bucket];
1517  bucket &tb = buckets_[to.bucket];
1518 
1519  // We plan to kick out fs, but let's check if it is still there;
1520  // there's a small chance we've gotten scooped by a later cuckoo. If
1521  // that happened, just... try again. Also the slot we are filling in
1522  // may have already been filled in by another thread, or the slot we
1523  // are moving from may be empty, both of which invalidate the swap.
1524  // We only need to check that the hash value is the same, because,
1525  // even if the keys are different and have the same hash value, then
1526  // the cuckoopath is still valid.
1527  if (tb.occupied(ts) || !fb.occupied(fs) ||
1528  hashed_key_only_hash(fb.key(fs)) != from.hv.hash) {
1529  return false;
1530  }
1531 
1532  buckets_.setKV(to.bucket, ts, fb.partial(fs), fb.movable_key(fs),
1533  std::move(fb.mapped(fs)));
1534  buckets_.eraseKV(from.bucket, fs);
1535  if (depth == 1) {
1536  // Hold onto the locks contained in twob
1537  b = std::move(twob);
1538  }
1539  depth--;
1540  }
1541  return true;
1542  }
1543 
1544  // A constexpr version of pow that we can use for various compile-time
1545  // constants and checks.
1546  static constexpr size_type const_pow(size_type a, size_type b) {
1547  return (b == 0) ? 1 : a * const_pow(a, b - 1);
1548  }
1549 
1550  // b_slot holds the information for a BFS path through the table.
1551  struct b_slot {
1552  // The bucket of the last item in the path.
1553  size_type bucket;
1554  // a compressed representation of the slots for each of the buckets in
1555  // the path. pathcode is sort of like a base-slot_per_bucket number, and
1556  // we need to hold at most MAX_BFS_PATH_LEN slots. Thus we need the
1557  // maximum pathcode to be at least slot_per_bucket()^(MAX_BFS_PATH_LEN).
1558  uint16_t pathcode;
1559  static_assert(const_pow(slot_per_bucket(), MAX_BFS_PATH_LEN) <
1560  std::numeric_limits<decltype(pathcode)>::max(),
1561  "pathcode may not be large enough to encode a cuckoo "
1562  "path");
1563  // The 0-indexed position in the cuckoo path this slot occupies. It must
1564  // be less than MAX_BFS_PATH_LEN, and also able to hold negative values.
1565  int8_t depth;
1566  static_assert(MAX_BFS_PATH_LEN - 1 <=
1567  std::numeric_limits<decltype(depth)>::max(),
1568  "The depth type must able to hold a value of"
1569  " MAX_BFS_PATH_LEN - 1");
1570  static_assert(-1 >= std::numeric_limits<decltype(depth)>::min(),
1571  "The depth type must be able to hold a value of -1");
1572  b_slot() {}
1573  b_slot(const size_type b, const uint16_t p, const decltype(depth) d)
1574  : bucket(b), pathcode(p), depth(d) {
1575  assert(d < MAX_BFS_PATH_LEN);
1576  }
1577  };
1578 
1579  // b_queue is the queue used to store b_slots for BFS cuckoo hashing.
1580  class b_queue {
1581  public:
1582  b_queue() noexcept : first_(0), last_(0) {}
1583 
1584  void enqueue(b_slot x) {
1585  assert(!full());
1586  slots_[last_++] = x;
1587  }
1588 
1589  b_slot dequeue() {
1590  assert(!empty());
1591  assert(first_ < last_);
1592  b_slot &x = slots_[first_++];
1593  return x;
1594  }
1595 
1596  bool empty() const { return first_ == last_; }
1597 
1598  bool full() const { return last_ == MAX_CUCKOO_COUNT; }
1599 
1600  private:
1601  // The size of the BFS queue. It holds just enough elements to fulfill a
1602  // MAX_BFS_PATH_LEN search for two starting buckets, with no circular
1603  // wrapping-around. For one bucket, this is the geometric sum
1604  // sum_{k=0}^{MAX_BFS_PATH_LEN-1} slot_per_bucket()^k
1605  // = (1 - slot_per_bucket()^MAX_BFS_PATH_LEN) / (1 - slot_per_bucket())
1606  //
1607  // Note that if slot_per_bucket() == 1, then this simply equals
1608  // MAX_BFS_PATH_LEN.
1609  static_assert(slot_per_bucket() > 0,
1610  "SLOT_PER_BUCKET must be greater than 0.");
1611  static constexpr size_type MAX_CUCKOO_COUNT =
1612  2 * ((slot_per_bucket() == 1)
1613  ? MAX_BFS_PATH_LEN
1614  : (const_pow(slot_per_bucket(), MAX_BFS_PATH_LEN) - 1) /
1615  (slot_per_bucket() - 1));
1616  // An array of b_slots. Since we allocate just enough space to complete a
1617  // full search, we should never exceed the end of the array.
1618  b_slot slots_[MAX_CUCKOO_COUNT];
1619  // The index of the head of the queue in the array
1620  size_type first_;
1621  // One past the index of the last_ item of the queue in the array.
1622  size_type last_;
1623  };
1624 
1625  // slot_search searches for a cuckoo path using breadth-first search. It
1626  // starts with the i1 and i2 buckets, and, until it finds a bucket with an
1627  // empty slot, adds each slot of the bucket in the b_slot. If the queue runs
1628  // out of space, it fails.
1629  //
1630  // throws hashpower_changed if it changed during the search
1631  template <typename TABLE_MODE>
1632  b_slot slot_search(const size_type hp, const size_type i1,
1633  const size_type i2) {
1634  b_queue q;
1635  // The initial pathcode informs cuckoopath_search which bucket the path
1636  // starts on
1637  q.enqueue(b_slot(i1, 0, 0));
1638  q.enqueue(b_slot(i2, 1, 0));
1639  while (!q.empty()) {
1640  b_slot x = q.dequeue();
1641  auto lock_manager = lock_one(hp, x.bucket, TABLE_MODE());
1642  bucket &b = buckets_[x.bucket];
1643  // Picks a (sort-of) random slot to start from
1644  size_type starting_slot = x.pathcode % slot_per_bucket();
1645  for (size_type i = 0; i < slot_per_bucket(); ++i) {
1646  uint16_t slot = (starting_slot + i) % slot_per_bucket();
1647  if (!b.occupied(slot)) {
1648  // We can terminate the search here
1649  x.pathcode = x.pathcode * slot_per_bucket() + slot;
1650  return x;
1651  }
1652 
1653  // If x has less than the maximum number of path components,
1654  // create a new b_slot item, that represents the bucket we would
1655  // have come from if we kicked out the item at this slot.
1656  const partial_t partial = b.partial(slot);
1657  if (x.depth < MAX_BFS_PATH_LEN - 1) {
1658  assert(!q.full());
1659  b_slot y(alt_index(hp, partial, x.bucket),
1660  x.pathcode * slot_per_bucket() + slot, x.depth + 1);
1661  q.enqueue(y);
1662  }
1663  }
1664  }
1665  // We didn't find a short-enough cuckoo path, so the search terminated.
1666  // Return a failure value.
1667  return b_slot(0, 0, -1);
1668  }
1669 
1670  // cuckoo_fast_double will double the size of the table by taking advantage
1671  // of the properties of index_hash and alt_index. If the key's move
1672  // constructor is not noexcept, we use cuckoo_expand_simple, since that
1673  // provides a strong exception guarantee.
1674  template <typename TABLE_MODE, typename AUTO_RESIZE>
1675  cuckoo_status cuckoo_fast_double(size_type current_hp) {
1676  if (!is_data_nothrow_move_constructible()) {
1677  LIBCUCKOO_DBG("%s", "cannot run cuckoo_fast_double because key-value"
1678  " pair is not nothrow move constructible");
1679  return cuckoo_expand_simple<TABLE_MODE, AUTO_RESIZE>(current_hp + 1);
1680  }
1681  const size_type new_hp = current_hp + 1;
1682  auto all_locks_manager = lock_all(TABLE_MODE());
1683  cuckoo_status st = check_resize_validity<AUTO_RESIZE>(current_hp, new_hp);
1684  if (st != ok) {
1685  return st;
1686  }
1687 
1688  // Finish rehashing any un-rehashed buckets, so that we can move out any
1689  // remaining data in old_buckets_. We should be running cuckoo_fast_double
1690  // only after trying to cuckoo for a while, which should mean we've tried
1691  // going through most of the table and thus done a lot of rehashing
1692  // already. So this shouldn't be too expensive.
1693  //
1694  // We restrict ourselves to the current thread because we want to avoid
1695  // possibly spawning extra threads in this function, unless the
1696  // circumstances are predictable (i.e. data is nothrow move constructible,
1697  // we're in locked_table mode and must keep the buckets_ container
1698  // up-to-date, etc).
1699  //
1700  // If we have fewer than kNumLocks buckets, there shouldn't be any buckets
1701  // left to rehash, so this should be a no-op.
1702  {
1703  locks_t &current_locks = get_current_locks();
1704  for (size_t i = 0; i < current_locks.size(); ++i) {
1705  rehash_lock<kIsNotLazy>(i);
1706  }
1707  num_remaining_lazy_rehash_locks(0);
1708  }
1709 
1710  // Resize the locks array if necessary. This is done before we update the
1711  // hashpower so that other threads don't grab the new hashpower and the old
1712  // locks.
1713  maybe_resize_locks(size_type(1) << new_hp);
1714  locks_t &current_locks = get_current_locks();
1715 
1716  // Move the current buckets into old_buckets_, and create a new empty
1717  // buckets container, which will become the new current one. The
1718  // old_buckets_ data will be destroyed when move-assigning to buckets_.
1719  old_buckets_.swap(buckets_);
1720  buckets_ = buckets_t(new_hp, get_allocator());
1721 
1722  // If we have less than kMaxNumLocks buckets, we do a full rehash in the
1723  // current thread. On-demand rehashing wouldn't be very easy with less than
1724  // kMaxNumLocks buckets, because it would require taking extra lower-index
1725  // locks to do the rehashing. Because kMaxNumLocks is relatively small,
1726  // this should not be very expensive. We have already set all locks to
1727  // migrated at the start of the function, so we shouldn't have to touch
1728  // them again.
1729  //
1730  // Otherwise, if we're in locked_table_mode, the expectation is that we can
1731  // access the latest data in buckets_ without taking any locks. So we must
1732  // rehash the data immediately. This would not be much different from
1733  // lazy-rehashing in locked_table_mode anyways, because it would still be
1734  // going on in one thread.
1735  if (old_buckets_.size() < kMaxNumLocks) {
1736  for (size_type i = 0; i < old_buckets_.size(); ++i) {
1737  move_bucket(old_buckets_, buckets_, i);
1738  }
1739  // This will also delete the old_buckets_ data.
1740  num_remaining_lazy_rehash_locks(0);
1741  } else {
1742  // Mark all current locks as un-migrated, so that we rehash the data
1743  // on-demand when the locks are taken.
1744  for (spinlock &lock : current_locks) {
1745  lock.is_migrated() = false;
1746  }
1747  num_remaining_lazy_rehash_locks(current_locks.size());
1748  if (std::is_same<TABLE_MODE, locked_table_mode>::value) {
1749  rehash_with_workers();
1750  }
1751  }
1752  return ok;
1753  }
1754 
1755  void move_bucket(buckets_t &old_buckets, buckets_t &new_buckets,
1756  size_type old_bucket_ind) const noexcept {
1757  const size_t old_hp = old_buckets.hashpower();
1758  const size_t new_hp = new_buckets.hashpower();
1759 
1760  // By doubling the table size, the index_hash and alt_index of each key got
1761  // one bit added to the top, at position old_hp, which means anything we
1762  // have to move will either be at the same bucket position, or exactly
1763  // hashsize(old_hp) later than the current bucket.
1764  bucket &old_bucket = old_buckets_[old_bucket_ind];
1765  const size_type new_bucket_ind = old_bucket_ind + hashsize(old_hp);
1766  size_type new_bucket_slot = 0;
1767 
1768  // For each occupied slot, either move it into its same position in the
1769  // new buckets container, or to the first available spot in the new
1770  // bucket in the new buckets container.
1771  for (size_type old_bucket_slot = 0; old_bucket_slot < slot_per_bucket();
1772  ++old_bucket_slot) {
1773  if (!old_bucket.occupied(old_bucket_slot)) {
1774  continue;
1775  }
1776  const hash_value hv = hashed_key(old_bucket.key(old_bucket_slot));
1777  const size_type old_ihash = index_hash(old_hp, hv.hash);
1778  const size_type old_ahash = alt_index(old_hp, hv.partial, old_ihash);
1779  const size_type new_ihash = index_hash(new_hp, hv.hash);
1780  const size_type new_ahash = alt_index(new_hp, hv.partial, new_ihash);
1781  size_type dst_bucket_ind, dst_bucket_slot;
1782  if ((old_bucket_ind == old_ihash && new_ihash == new_bucket_ind) ||
1783  (old_bucket_ind == old_ahash && new_ahash == new_bucket_ind)) {
1784  // We're moving the key to the new bucket
1785  dst_bucket_ind = new_bucket_ind;
1786  dst_bucket_slot = new_bucket_slot++;
1787  } else {
1788  // We're moving the key to the old bucket
1789  assert((old_bucket_ind == old_ihash && new_ihash == old_ihash) ||
1790  (old_bucket_ind == old_ahash && new_ahash == old_ahash));
1791  dst_bucket_ind = old_bucket_ind;
1792  dst_bucket_slot = old_bucket_slot;
1793  }
1794  new_buckets.setKV(dst_bucket_ind, dst_bucket_slot++,
1795  old_bucket.partial(old_bucket_slot),
1796  old_bucket.movable_key(old_bucket_slot),
1797  std::move(old_bucket.mapped(old_bucket_slot)));
1798  }
1799  }
1800 
1801  // Checks whether the resize is okay to proceed. Returns a status code, or
1802  // throws an exception, depending on the error type.
1803  using automatic_resize = std::integral_constant<bool, true>;
1804  using manual_resize = std::integral_constant<bool, false>;
1805 
1806  template <typename AUTO_RESIZE>
1807  cuckoo_status check_resize_validity(const size_type orig_hp,
1808  const size_type new_hp) {
1809  const size_type mhp = maximum_hashpower();
1810  if (mhp != NO_MAXIMUM_HASHPOWER && new_hp > mhp) {
1811  throw maximum_hashpower_exceeded(new_hp);
1812  }
1813  if (AUTO_RESIZE::value && load_factor() < minimum_load_factor()) {
1814  throw load_factor_too_low(minimum_load_factor());
1815  }
1816  if (hashpower() != orig_hp) {
1817  // Most likely another expansion ran before this one could grab the
1818  // locks
1819  LIBCUCKOO_DBG("%s", "another expansion is on-going\n");
1820  return failure_under_expansion;
1821  }
1822  return ok;
1823  }
1824 
1825  // When we expand the contanier, we may need to expand the locks array, if
1826  // the current locks array is smaller than the maximum size and also smaller
1827  // than the number of buckets in the upcoming buckets container. In this
1828  // case, we grow the locks array to the smaller of the maximum lock array
1829  // size and the bucket count. This is done by allocating an entirely new lock
1830  // container, taking all the locks, copying over the counters, and then
1831  // finally adding it to the end of `all_locks_`, thereby designating it the
1832  // "current" locks container. It is the responsibility of the caller to
1833  // unlock all locks taken, including the new locks, whenever it is done with
1834  // them, so that old threads can resume and potentially re-start.
1835  void maybe_resize_locks(size_type new_bucket_count) {
1836  locks_t &current_locks = get_current_locks();
1837  if (!(current_locks.size() < kMaxNumLocks &&
1838  current_locks.size() < new_bucket_count)) {
1839  return;
1840  }
1841 
1842  locks_t new_locks(get_allocator());
1843  new_locks.resize(std::min(size_type(kMaxNumLocks), new_bucket_count));
1844  assert(new_locks.size() > current_locks.size());
1845  std::copy(current_locks.begin(), current_locks.end(), new_locks.begin());
1846  for (spinlock &lock : new_locks) {
1847  lock.lock();
1848  }
1849  all_locks_.emplace_back(std::move(new_locks));
1850  }
1851 
1852  // cuckoo_expand_simple will resize the table to at least the given
1853  // new_hashpower. When we're shrinking the table, if the current table
1854  // contains more elements than can be held by new_hashpower, the resulting
1855  // hashpower will be greater than `new_hp`. It needs to take all the bucket
1856  // locks, since no other operations can change the table during expansion.
1857  // Throws maximum_hashpower_exceeded if we're expanding beyond the
1858  // maximum hashpower, and we have an actual limit.
1859  template <typename TABLE_MODE, typename AUTO_RESIZE>
1860  cuckoo_status cuckoo_expand_simple(size_type new_hp) {
1861  auto all_locks_manager = lock_all(TABLE_MODE());
1862  const size_type hp = hashpower();
1863  cuckoo_status st = check_resize_validity<AUTO_RESIZE>(hp, new_hp);
1864  if (st != ok) {
1865  return st;
1866  }
1867 
1868  // Finish rehashing any data into buckets_.
1869  rehash_with_workers();
1870 
1871  // Creates a new hash table with hashpower new_hp and adds all the elements
1872  // from buckets_ and old_buckets_. Allow this map to spawn extra threads if
1873  // it needs to resize during the resize.
1874  cuckoohash_map new_map(hashsize(new_hp) * slot_per_bucket(),
1876  new_map.max_num_worker_threads(max_num_worker_threads());
1877 
1878  parallel_exec(
1879  0, hashsize(hp),
1880  [this, &new_map]
1881  (size_type i, size_type end, std::exception_ptr &eptr) {
1882  try {
1883  for (; i < end; ++i) {
1884  auto &bucket = buckets_[i];
1885  for (size_type j = 0; j < slot_per_bucket(); ++j) {
1886  if (bucket.occupied(j)) {
1887  new_map.insert(bucket.movable_key(j),
1888  std::move(bucket.mapped(j)));
1889  }
1890  }
1891  }
1892  } catch (...) {
1893  eptr = std::current_exception();
1894  }
1895  });
1896 
1897  // Finish rehashing any data in new_map.
1898  new_map.rehash_with_workers();
1899 
1900  // Swap the buckets_ container with new_map's. This is okay, because we
1901  // have all the locks, so nobody else should be reading from the buckets
1902  // array. Then the old buckets will be deleted when new_map is deleted.
1903  maybe_resize_locks(new_map.bucket_count());
1904  buckets_.swap(new_map.buckets_);
1905 
1906  return ok;
1907  }
1908 
1909  // Executes the function over the given range, splitting the work between the
1910  // current thread and any available worker threads.
1911  //
1912  // In the noexcept version, the functor must implement operator()(size_type
1913  // start, size_type end).
1914  //
1915  // In the non-noexcept version, the functor will receive an additional
1916  // std::exception_ptr& argument.
1917 
1918  template <typename F>
1919  void parallel_exec_noexcept(size_type start, size_type end, F func) {
1920  const size_type num_extra_threads = max_num_worker_threads();
1921  const size_type num_workers = 1 + num_extra_threads;
1922  size_type work_per_thread = (end - start) / num_workers;
1923  std::vector<std::thread, rebind_alloc<std::thread>> threads(
1924  get_allocator());
1925  threads.reserve(num_extra_threads);
1926  for (size_type i = 0; i < num_extra_threads; ++i) {
1927  threads.emplace_back(func, start, start + work_per_thread);
1928  start += work_per_thread;
1929  }
1930  func(start, end);
1931  for (std::thread &t : threads) {
1932  t.join();
1933  }
1934  }
1935 
1936  template <typename F>
1937  void parallel_exec(size_type start, size_type end, F func) {
1938  const size_type num_extra_threads = max_num_worker_threads();
1939  const size_type num_workers = 1 + num_extra_threads;
1940  size_type work_per_thread = (end - start) / num_workers;
1941  std::vector<std::thread, rebind_alloc<std::thread>> threads(
1942  get_allocator());
1943  threads.reserve(num_extra_threads);
1944 
1945  std::vector<std::exception_ptr, rebind_alloc<std::exception_ptr>> eptrs(
1946  num_workers, nullptr, get_allocator());
1947  for (size_type i = 0; i < num_extra_threads; ++i) {
1948  threads.emplace_back(func, start, start + work_per_thread,
1949  std::ref(eptrs[i]));
1950  start += work_per_thread;
1951  }
1952  func(start, end, std::ref(eptrs.back()));
1953  for (std::thread &t : threads) {
1954  t.join();
1955  }
1956  for (std::exception_ptr &eptr : eptrs) {
1957  if (eptr) std::rethrow_exception(eptr);
1958  }
1959  }
1960 
1961  // Does a batch resize of the remaining data in old_buckets_. Assumes all the
1962  // locks have already been taken.
1963  void rehash_with_workers() noexcept {
1964  locks_t &current_locks = get_current_locks();
1965  parallel_exec_noexcept(
1966  0, current_locks.size(),
1967  [this](size_type start, size_type end) {
1968  for (size_type i = start; i < end; ++i) {
1969  rehash_lock<kIsNotLazy>(i);
1970  }
1971  });
1972  num_remaining_lazy_rehash_locks(0);
1973  }
1974 
1975  // Deletion functions
1976 
1977  // Removes an item from a bucket, decrementing the associated counter as
1978  // well.
1979  void del_from_bucket(const size_type bucket_ind, const size_type slot) {
1980  buckets_.eraseKV(bucket_ind, slot);
1981  --get_current_locks()[lock_ind(bucket_ind)].elem_counter();
1982  }
1983 
1984  // Empties the table, calling the destructors of all the elements it removes
1985  // from the table. It assumes the locks are taken as necessary.
1986  void cuckoo_clear() {
1987  buckets_.clear();
1988  // This will also clear out any data in old_buckets and delete it, if we
1989  // haven't already.
1990  num_remaining_lazy_rehash_locks(0);
1991  for (spinlock &lock : get_current_locks()) {
1992  lock.elem_counter() = 0;
1993  lock.is_migrated() = true;
1994  }
1995  }
1996 
1997  // Rehashing functions
1998 
1999  template <typename TABLE_MODE> bool cuckoo_rehash(size_type n) {
2000  const size_type hp = hashpower();
2001  if (n == hp) {
2002  return false;
2003  }
2004  return cuckoo_expand_simple<TABLE_MODE, manual_resize>(n) == ok;
2005  }
2006 
2007  template <typename TABLE_MODE> bool cuckoo_reserve(size_type n) {
2008  const size_type hp = hashpower();
2009  const size_type new_hp = reserve_calc(n);
2010  if (new_hp == hp) {
2011  return false;
2012  }
2013  return cuckoo_expand_simple<TABLE_MODE, manual_resize>(new_hp) == ok;
2014  }
2015 
2016  // Miscellaneous functions
2017 
2018  // reserve_calc takes in a parameter specifying a certain number of slots
2019  // for a table and returns the smallest hashpower that will hold n elements.
2020  static size_type reserve_calc(const size_type n) {
2021  const size_type buckets = (n + slot_per_bucket() - 1) / slot_per_bucket();
2022  size_type blog2;
2023  for (blog2 = 0; (size_type(1) << blog2) < buckets; ++blog2)
2024  ;
2025  assert(n <= buckets * slot_per_bucket() && buckets <= hashsize(blog2));
2026  return blog2;
2027  }
2028 
2029  // This class is a friend for unit testing
2030  friend class UnitTestInternalAccess;
2031 
2032  static constexpr size_type kMaxNumLocks = 1UL << 16;
2033 
2034  locks_t &get_current_locks() const { return all_locks_.back(); }
2035 
2036  // Get/set/decrement num remaining lazy rehash locks. If we reach 0 remaining
2037  // lazy locks, we can deallocate the memory in old_buckets_.
2038  size_type num_remaining_lazy_rehash_locks() const {
2039  return num_remaining_lazy_rehash_locks_.load(
2040  std::memory_order_acquire);
2041  }
2042 
2043  void num_remaining_lazy_rehash_locks(size_type n) const {
2044  num_remaining_lazy_rehash_locks_.store(
2045  n, std::memory_order_release);
2046  if (n == 0) {
2047  old_buckets_.clear_and_deallocate();
2048  }
2049  }
2050 
2051  void decrement_num_remaining_lazy_rehash_locks() const {
2052  size_type old_num_remaining = num_remaining_lazy_rehash_locks_.fetch_sub(
2053  1, std::memory_order_acq_rel);
2054  assert(old_num_remaining >= 1);
2055  if (old_num_remaining == 1) {
2056  old_buckets_.clear_and_deallocate();
2057  }
2058  }
2059 
2060  // Member variables
2061 
2062  // The hash function
2063  hasher hash_fn_;
2064 
2065  // The equality function
2066  key_equal eq_fn_;
2067 
2068  // container of buckets. The size or memory location of the buckets cannot be
2069  // changed unless all the locks are taken on the table. Thus, it is only safe
2070  // to access the buckets_ container when you have at least one lock held.
2071  //
2072  // Marked mutable so that const methods can rehash into this container when
2073  // necessary.
2074  mutable buckets_t buckets_;
2075 
2076  // An old container of buckets, containing data that may not have been
2077  // rehashed into the current one. If valid, this will always have a hashpower
2078  // exactly one less than the one in buckets_.
2079  //
2080  // Marked mutable so that const methods can rehash into this container when
2081  // necessary.
2082  mutable buckets_t old_buckets_;
2083 
2084  // A linked list of all lock containers. We never discard lock containers,
2085  // since there is currently no mechanism for detecting when all threads are
2086  // done looking at the memory. The back lock container in this list is
2087  // designated the "current" one, and is used by all operations taking locks.
2088  // This container can be modified if either it is empty (which should only
2089  // occur during construction), or if the modifying thread has taken all the
2090  // locks on the existing "current" container. In the latter case, a
2091  // modification must take place before a modification to the hashpower, so
2092  // that other threads can detect the change and adjust appropriately. Marked
2093  // mutable so that const methods can access and take locks.
2094  mutable all_locks_t all_locks_;
2095 
2096  // A small wrapper around std::atomic to make it copyable for constructors.
2097  template <typename AtomicT>
2098  class CopyableAtomic : public std::atomic<AtomicT> {
2099  public:
2100  using std::atomic<AtomicT>::atomic;
2101 
2102  CopyableAtomic(const CopyableAtomic& other) noexcept
2103  : CopyableAtomic(other.load(std::memory_order_acquire)) {}
2104 
2105  CopyableAtomic& operator=(const CopyableAtomic& other) noexcept {
2106  this->store(other.load(std::memory_order_acquire),
2107  std::memory_order_release);
2108  return *this;
2109  }
2110  };
2111 
2112  // We keep track of the number of remaining locks in the latest locks array,
2113  // that remain to be rehashed. Once this reaches 0, we can free the memory of
2114  // the old buckets. It should only be accessed or modified when
2115  // lazy-rehashing a lock, so not in the common case.
2116  //
2117  // Marked mutable so that we can modify this during rehashing.
2118  mutable CopyableAtomic<size_t> num_remaining_lazy_rehash_locks_;
2119 
2120  // Stores the minimum load factor allowed for automatic expansions. Whenever
2121  // an automatic expansion is triggered (during an insertion where cuckoo
2122  // hashing fails, for example), we check the load factor against this
2123  // double, and throw an exception if it's lower than this value. It can be
2124  // used to signal when the hash function is bad or the input adversarial.
2125  CopyableAtomic<double> minimum_load_factor_;
2126 
2127  // stores the maximum hashpower allowed for any expansions. If set to
2128  // NO_MAXIMUM_HASHPOWER, this limit will be disregarded.
2129  CopyableAtomic<size_type> maximum_hashpower_;
2130 
2131  // Maximum number of extra threads to spawn when doing any large batch
2132  // operations.
2133  CopyableAtomic<size_type> max_num_worker_threads_;
2134 
2135 public:
2148  public:
2152  using key_type = typename cuckoohash_map::key_type;
2153  using mapped_type = typename cuckoohash_map::mapped_type;
2154  using value_type = typename cuckoohash_map::value_type;
2155  using size_type = typename cuckoohash_map::size_type;
2156  using difference_type = typename cuckoohash_map::difference_type;
2157  using hasher = typename cuckoohash_map::hasher;
2158  using key_equal = typename cuckoohash_map::key_equal;
2159  using allocator_type = typename cuckoohash_map::allocator_type;
2160  using reference = typename cuckoohash_map::reference;
2161  using const_reference = typename cuckoohash_map::const_reference;
2162  using pointer = typename cuckoohash_map::pointer;
2163  using const_pointer = typename cuckoohash_map::const_pointer;
2164 
2171  public:
2172  using difference_type = typename locked_table::difference_type;
2173  using value_type = typename locked_table::value_type;
2174  using pointer = typename locked_table::const_pointer;
2175  using reference = typename locked_table::const_reference;
2176  using iterator_category = std::bidirectional_iterator_tag;
2177 
2178  const_iterator() {}
2179 
2180  // Return true if the iterators are from the same locked table and
2181  // location, false otherwise.
2182  bool operator==(const const_iterator &it) const {
2183  return buckets_ == it.buckets_ && index_ == it.index_ &&
2184  slot_ == it.slot_;
2185  }
2186 
2187  bool operator!=(const const_iterator &it) const {
2188  return !(operator==(it));
2189  }
2190 
2191  reference operator*() const { return (*buckets_)[index_].kvpair(slot_); }
2192 
2193  pointer operator->() const { return std::addressof(operator*()); }
2194 
2195  // Advance the iterator to the next item in the table, or to the end
2196  // of the table. Returns the iterator at its new position.
2197  const_iterator &operator++() {
2198  // Move forward until we get to a slot that is occupied, or we
2199  // get to the end
2200  ++slot_;
2201  for (; index_ < buckets_->size(); ++index_) {
2202  for (; slot_ < slot_per_bucket(); ++slot_) {
2203  if ((*buckets_)[index_].occupied(slot_)) {
2204  return *this;
2205  }
2206  }
2207  slot_ = 0;
2208  }
2209  assert(std::make_pair(index_, slot_) == end_pos(*buckets_));
2210  return *this;
2211  }
2212 
2213  // Advance the iterator to the next item in the table, or to the end
2214  // of the table. Returns the iterator at its old position.
2215  const_iterator operator++(int) {
2216  const_iterator old(*this);
2217  ++(*this);
2218  return old;
2219  }
2220 
2221  // Move the iterator back to the previous item in the table. Returns
2222  // the iterator at its new position.
2223  const_iterator &operator--() {
2224  // Move backward until we get to the beginning. Behavior is
2225  // undefined if we are iterating at the first element, so we can
2226  // assume we'll reach an element. This means we'll never reach
2227  // index_ == 0 and slot_ == 0.
2228  if (slot_ == 0) {
2229  --index_;
2230  slot_ = slot_per_bucket() - 1;
2231  } else {
2232  --slot_;
2233  }
2234  while (!(*buckets_)[index_].occupied(slot_)) {
2235  if (slot_ == 0) {
2236  --index_;
2237  slot_ = slot_per_bucket() - 1;
2238  } else {
2239  --slot_;
2240  }
2241  }
2242  return *this;
2243  }
2244 
2249  const_iterator old(*this);
2250  --(*this);
2251  return old;
2252  }
2253 
2254  protected:
2255  // The buckets owned by the locked table being iterated over. Even
2256  // though const_iterator cannot modify the buckets, we don't mark
2257  // them const so that the mutable iterator can derive from this
2258  // class. Also, since iterators should be default constructible,
2259  // copyable, and movable, we have to make this a raw pointer type.
2260  buckets_t *buckets_;
2261 
2262  // The bucket index of the item being pointed to. For implementation
2263  // convenience, we let it take on negative values.
2264  size_type index_;
2265 
2266  // The slot in the bucket of the item being pointed to. For
2267  // implementation convenience, we let it take on negative values.
2268  size_type slot_;
2269 
2270  // Returns the position signifying the end of the table
2271  static std::pair<size_type, size_type> end_pos(const buckets_t &buckets) {
2272  return std::make_pair(buckets.size(), 0);
2273  }
2274 
2275  // The private constructor is used by locked_table to create
2276  // iterators from scratch. If the given index_-slot_ pair is at the
2277  // end of the table, or the given spot is occupied, stay. Otherwise,
2278  // step forward to the next data item, or to the end of the table.
2279  const_iterator(buckets_t &buckets, size_type index,
2280  size_type slot) noexcept
2281  : buckets_(std::addressof(buckets)), index_(index), slot_(slot) {
2282  if (std::make_pair(index_, slot_) != end_pos(*buckets_) &&
2283  !(*buckets_)[index_].occupied(slot_)) {
2284  operator++();
2285  }
2286  }
2287 
2288  friend class locked_table;
2289  };
2290 
2296  class iterator : public const_iterator {
2297  public:
2298  using pointer = typename cuckoohash_map::pointer;
2299  using reference = typename cuckoohash_map::reference;
2300 
2301  iterator() {}
2302 
2303  bool operator==(const iterator &it) const {
2304  return const_iterator::operator==(it);
2305  }
2306 
2307  bool operator!=(const iterator &it) const {
2308  return const_iterator::operator!=(it);
2309  }
2310 
2311  reference operator*() {
2312  return (*const_iterator::buckets_)[const_iterator::index_].kvpair(
2313  const_iterator::slot_);
2314  }
2315 
2316  pointer operator->() { return std::addressof(operator*()); }
2317 
2318  iterator &operator++() {
2319  const_iterator::operator++();
2320  return *this;
2321  }
2322 
2323  iterator operator++(int) {
2324  iterator old(*this);
2325  const_iterator::operator++();
2326  return old;
2327  }
2328 
2329  iterator &operator--() {
2330  const_iterator::operator--();
2331  return *this;
2332  }
2333 
2334  iterator operator--(int) {
2335  iterator old(*this);
2336  const_iterator::operator--();
2337  return old;
2338  }
2339 
2340  private:
2341  iterator(buckets_t &buckets, size_type index, size_type slot) noexcept
2342  : const_iterator(buckets, index, slot) {}
2343 
2344  friend class locked_table;
2345  };
2346 
2352  static constexpr size_type slot_per_bucket() {
2354  }
2355 
2361  locked_table() = delete;
2362  locked_table(const locked_table &) = delete;
2363  locked_table &operator=(const locked_table &) = delete;
2364 
2365  locked_table(locked_table &&lt) noexcept
2366  : map_(std::move(lt.map_)),
2367  all_locks_manager_(std::move(lt.all_locks_manager_)) {}
2368 
2369  locked_table &operator=(locked_table &&lt) noexcept {
2370  unlock();
2371  map_ = std::move(lt.map_);
2372  all_locks_manager_ = std::move(lt.all_locks_manager_);
2373  return *this;
2374  }
2375 
2381  void unlock() { all_locks_manager_.reset(); }
2382 
2399  bool is_active() const { return static_cast<bool>(all_locks_manager_); }
2400 
2401  hasher hash_function() const { return map_.get().hash_function(); }
2402 
2403  key_equal key_eq() const { return map_.get().key_eq(); }
2404 
2405  allocator_type get_allocator() const { return map_.get().get_allocator(); }
2406 
2407  size_type hashpower() const { return map_.get().hashpower(); }
2408 
2409  size_type bucket_count() const { return map_.get().bucket_count(); }
2410 
2411  bool empty() const { return map_.get().empty(); }
2412 
2413  size_type size() const { return map_.get().size(); }
2414 
2415  size_type capacity() const { return map_.get().capacity(); }
2416 
2417  double load_factor() const { return map_.get().load_factor(); }
2418 
2419  void minimum_load_factor(const double mlf) {
2420  map_.get().minimum_load_factor(mlf);
2421  }
2422 
2423  double minimum_load_factor() const {
2424  return map_.get().minimum_load_factor();
2425  }
2426 
2427  void maximum_hashpower(size_type mhp) { map_.get().maximum_hashpower(mhp); }
2428 
2429  size_type maximum_hashpower() const {
2430  return map_.get().maximum_hashpower();
2431  }
2432 
2433  void max_num_worker_threads(size_type extra_threads) {
2434  map_.get().max_num_worker_threads(extra_threads);
2435  }
2436 
2437  size_type max_num_worker_threads() const {
2438  return map_.get().max_num_worker_threads();
2439  }
2440 
2453  iterator begin() { return iterator(map_.get().buckets_, 0, 0); }
2454 
2455  const_iterator begin() const {
2456  return const_iterator(map_.get().buckets_, 0, 0);
2457  }
2458 
2459  const_iterator cbegin() const { return begin(); }
2460 
2468  const auto end_pos = const_iterator::end_pos(map_.get().buckets_);
2469  return iterator(map_.get().buckets_,
2470  static_cast<size_type>(end_pos.first),
2471  static_cast<size_type>(end_pos.second));
2472  }
2473 
2474  const_iterator end() const {
2475  const auto end_pos = const_iterator::end_pos(map_.get().buckets_);
2476  return const_iterator(map_.get().buckets_,
2477  static_cast<size_type>(end_pos.first),
2478  static_cast<size_type>(end_pos.second));
2479  }
2480 
2481  const_iterator cend() const { return end(); }
2482 
2488  void clear() { map_.get().cuckoo_clear(); }
2489 
2495  template <typename K, typename... Args>
2496  std::pair<iterator, bool> insert(K &&key, Args &&... val) {
2497  hash_value hv = map_.get().hashed_key(key);
2498  auto b = map_.get().template snapshot_and_lock_two<locked_table_mode>(hv);
2499  table_position pos =
2500  map_.get().template cuckoo_insert_loop<locked_table_mode>(hv, b, key);
2501  if (pos.status == ok) {
2502  map_.get().add_to_bucket(pos.index, pos.slot, hv.partial,
2503  std::forward<K>(key),
2504  std::forward<Args>(val)...);
2505  } else {
2506  assert(pos.status == failure_key_duplicated);
2507  }
2508  return std::make_pair(iterator(map_.get().buckets_, pos.index, pos.slot),
2509  pos.status == ok);
2510  }
2511 
2512  iterator erase(const_iterator pos) {
2513  map_.get().del_from_bucket(pos.index_, pos.slot_);
2514  return iterator(map_.get().buckets_, pos.index_, pos.slot_);
2515  }
2516 
2517  iterator erase(iterator pos) {
2518  map_.get().del_from_bucket(pos.index_, pos.slot_);
2519  return iterator(map_.get().buckets_, pos.index_, pos.slot_);
2520  }
2521 
2522  template <typename K> size_type erase(const K &key) {
2523  const hash_value hv = map_.get().hashed_key(key);
2524  const auto b =
2525  map_.get().template snapshot_and_lock_two<locked_table_mode>(hv);
2526  const table_position pos =
2527  map_.get().cuckoo_find(key, hv.partial, b.i1, b.i2);
2528  if (pos.status == ok) {
2529  map_.get().del_from_bucket(pos.index, pos.slot);
2530  return 1;
2531  } else {
2532  return 0;
2533  }
2534  }
2535 
2541  template <typename K> iterator find(const K &key) {
2542  const hash_value hv = map_.get().hashed_key(key);
2543  const auto b =
2544  map_.get().template snapshot_and_lock_two<locked_table_mode>(hv);
2545  const table_position pos =
2546  map_.get().cuckoo_find(key, hv.partial, b.i1, b.i2);
2547  if (pos.status == ok) {
2548  return iterator(map_.get().buckets_, pos.index, pos.slot);
2549  } else {
2550  return end();
2551  }
2552  }
2553 
2554  template <typename K> const_iterator find(const K &key) const {
2555  const hash_value hv = map_.get().hashed_key(key);
2556  const auto b =
2557  map_.get().template snapshot_and_lock_two<locked_table_mode>(hv);
2558  const table_position pos =
2559  map_.get().cuckoo_find(key, hv.partial, b.i1, b.i2);
2560  if (pos.status == ok) {
2561  return const_iterator(map_.get().buckets_, pos.index, pos.slot);
2562  } else {
2563  return end();
2564  }
2565  }
2566 
2567  template <typename K> mapped_type &at(const K &key) {
2568  auto it = find(key);
2569  if (it == end()) {
2570  throw std::out_of_range("key not found in table");
2571  } else {
2572  return it->second;
2573  }
2574  }
2575 
2576  template <typename K> const mapped_type &at(const K &key) const {
2577  auto it = find(key);
2578  if (it == end()) {
2579  throw std::out_of_range("key not found in table");
2580  } else {
2581  return it->second;
2582  }
2583  }
2584 
2590  template <typename K> T &operator[](K &&key) {
2591  auto result = insert(std::forward<K>(key));
2592  return result.first->second;
2593  }
2594 
2595  template <typename K> size_type count(const K &key) const {
2596  const hash_value hv = map_.get().hashed_key(key);
2597  const auto b =
2598  map_.get().template snapshot_and_lock_two<locked_table_mode>(hv);
2599  return map_.get().cuckoo_find(key, hv.partial, b.i1, b.i2).status == ok
2600  ? 1
2601  : 0;
2602  }
2603 
2604  template <typename K>
2605  std::pair<iterator, iterator> equal_range(const K &key) {
2606  auto it = find(key);
2607  if (it == end()) {
2608  return std::make_pair(it, it);
2609  } else {
2610  auto start_it = it++;
2611  return std::make_pair(start_it, it);
2612  }
2613  }
2614 
2615  template <typename K>
2616  std::pair<const_iterator, const_iterator> equal_range(const K &key) const {
2617  auto it = find(key);
2618  if (it == end()) {
2619  return std::make_pair(it, it);
2620  } else {
2621  auto start_it = it++;
2622  return std::make_pair(start_it, it);
2623  }
2624  }
2625 
2635  void rehash(size_type n) {
2636  map_.get().template cuckoo_rehash<locked_table_mode>(n);
2637  }
2638 
2643  void reserve(size_type n) {
2644  map_.get().template cuckoo_reserve<locked_table_mode>(n);
2645  }
2646 
2652  bool operator==(const locked_table &lt) const {
2653  if (size() != lt.size()) {
2654  return false;
2655  }
2656  for (const auto &elem : lt) {
2657  auto it = find(elem.first);
2658  if (it == end() || it->second != elem.second) {
2659  return false;
2660  }
2661  }
2662  return true;
2663  }
2664 
2665  bool operator!=(const locked_table &lt) const {
2666  if (size() != lt.size()) {
2667  return true;
2668  }
2669  for (const auto &elem : lt) {
2670  auto it = find(elem.first);
2671  if (it == end() || it->second != elem.second) {
2672  return true;
2673  }
2674  }
2675  return false;
2676  }
2677 
2680  private:
2681  // The constructor locks the entire table. We keep this constructor private
2682  // (but expose it to the cuckoohash_map class), since we don't want users
2683  // calling it. We also complete any remaining rehashing in the table, so
2684  // that everything is in map.buckets_.
2685  locked_table(cuckoohash_map &map) noexcept
2686  : map_(map),
2687  all_locks_manager_(map.lock_all(normal_mode())) {
2688  map.rehash_with_workers();
2689  }
2690 
2691  // Dispatchers for methods on cuckoohash_map
2692 
2693  buckets_t &buckets() { return map_.get().buckets_; }
2694 
2695  const buckets_t &buckets() const { return map_.get().buckets_; }
2696 
2697  void maybe_resize_locks(size_type new_bucket_count) {
2698  map_.get().maybe_resize_locks(new_bucket_count);
2699  }
2700 
2701  locks_t &get_current_locks() { return map_.get().get_current_locks(); }
2702 
2703  // A reference to the map owned by the table
2704  std::reference_wrapper<cuckoohash_map> map_;
2705  // A manager for all the locks we took on the table.
2706  AllLocksManager all_locks_manager_;
2707 
2708  friend class cuckoohash_map;
2709 
2710  friend std::ostream &operator<<(std::ostream &os, const locked_table &lt) {
2711  os << lt.buckets();
2712  size_type size = lt.size();
2713  os.write(reinterpret_cast<const char *>(&size), sizeof(size_type));
2714  double mlf = lt.minimum_load_factor();
2715  size_type mhp = lt.maximum_hashpower();
2716  os.write(reinterpret_cast<const char *>(&mlf), sizeof(double));
2717  os.write(reinterpret_cast<const char *>(&mhp), sizeof(size_type));
2718  return os;
2719  }
2720 
2721  friend std::istream &operator>>(std::istream &is, locked_table &lt) {
2722  is >> lt.buckets();
2723 
2724  // Re-size the locks, and set the size to the stored size
2725  lt.maybe_resize_locks(lt.bucket_count());
2726  for (auto &lock : lt.get_current_locks()) {
2727  lock.elem_counter() = 0;
2728  }
2729  size_type size;
2730  is.read(reinterpret_cast<char *>(&size), sizeof(size_type));
2731  if (size > 0) {
2732  lt.get_current_locks()[0].elem_counter() = size;
2733  }
2734 
2735  double mlf;
2736  size_type mhp;
2737  is.read(reinterpret_cast<char *>(&mlf), sizeof(double));
2738  is.read(reinterpret_cast<char *>(&mhp), sizeof(size_type));
2739  lt.minimum_load_factor(mlf);
2740  lt.maximum_hashpower(mhp);
2741  return is;
2742  }
2743  };
2744 };
2745 
2753 template <class Key, class T, class Hash, class KeyEqual, class Allocator,
2754  std::size_t SLOT_PER_BUCKET>
2755 void swap(
2758  &rhs) noexcept {
2759  lhs.swap(rhs);
2760 }
2761 
2762 } // namespace libcuckoo
2763 
2764 #endif // _CUCKOOHASH_MAP_HH
libcuckoo::cuckoohash_map::insert
bool insert(K &&key, Args &&... val)
Definition: cuckoohash_map.hh:645
libcuckoo::cuckoohash_map::value_type
typename buckets_t::value_type value_type
Definition: cuckoohash_map.hh:69
libcuckoo::cuckoohash_map::size
size_type size() const
Definition: cuckoohash_map.hh:334
libcuckoo::cuckoohash_map::find
mapped_type find(const K &key) const
Definition: cuckoohash_map.hh:612
libcuckoo::cuckoohash_map::key_eq
key_equal key_eq() const
Definition: cuckoohash_map.hh:298
libcuckoo::cuckoohash_map::capacity
size_type capacity() const
Definition: cuckoohash_map.hh:351
LIBCUCKOO_SQUELCH_PADDING_WARNING
#define LIBCUCKOO_SQUELCH_PADDING_WARNING
Definition: cuckoohash_util.hh:48
libcuckoo::NO_MAXIMUM_HASHPOWER
constexpr size_t NO_MAXIMUM_HASHPOWER
Definition: cuckoohash_config.hh:27
libcuckoo::cuckoohash_map::locked_table::const_iterator::operator--
const_iterator operator--(int)
Definition: cuckoohash_map.hh:2248
libcuckoo::cuckoohash_map::update_fn
bool update_fn(const K &key, F fn)
Definition: cuckoohash_map.hh:488
libcuckoo::cuckoohash_map::locked_table::rehash
void rehash(size_type n)
Definition: cuckoohash_map.hh:2635
LIBCUCKOO_DBG
#define LIBCUCKOO_DBG(fmt,...)
When LIBCUCKOO_DEBUG is 0, LIBCUCKOO_DBG does nothing.
Definition: cuckoohash_util.hh:26
libcuckoo::cuckoohash_map::bucket_count
size_type bucket_count() const
Definition: cuckoohash_map.hh:320
libcuckoo::internal::UpsertToUpraseFn
Definition: cuckoohash_util.hh:188
libcuckoo::cuckoohash_map::find
bool find(const K &key, mapped_type &val) const
Definition: cuckoohash_map.hh:600
libcuckoo::cuckoohash_map::empty
bool empty() const
Definition: cuckoohash_map.hh:327
libcuckoo::cuckoohash_map::hash_function
hasher hash_function() const
Definition: cuckoohash_map.hh:291
libcuckoo::cuckoohash_map
Definition: cuckoohash_map.hh:49
libcuckoo::cuckoohash_map::max_num_worker_threads
size_type max_num_worker_threads() const
Definition: cuckoohash_map.hh:440
libcuckoo::cuckoohash_map::locked_table::begin
iterator begin()
Definition: cuckoohash_map.hh:2453
libcuckoo::cuckoohash_map::find_fn
bool find_fn(const K &key, F fn) const
Definition: cuckoohash_map.hh:465
libcuckoo::cuckoohash_map::lock_table
locked_table lock_table()
Definition: cuckoohash_map.hh:705
libcuckoo::cuckoohash_map::hashpower
size_type hashpower() const
Definition: cuckoohash_map.hh:313
libcuckoo::cuckoohash_map::cuckoohash_map
cuckoohash_map(std::initializer_list< value_type > init, size_type n=DEFAULT_SIZE, const Hash &hf=Hash(), const KeyEqual &equal=KeyEqual(), const Allocator &alloc=Allocator())
Definition: cuckoohash_map.hh:215
libcuckoo::swap
void swap(cuckoohash_map< Key, T, Hash, KeyEqual, Allocator, SLOT_PER_BUCKET > &lhs, cuckoohash_map< Key, T, Hash, KeyEqual, Allocator, SLOT_PER_BUCKET > &rhs) noexcept
Definition: cuckoohash_map.hh:2755
libcuckoo::cuckoohash_map::rehash
bool rehash(size_type n)
Definition: cuckoohash_map.hh:678
libcuckoo::cuckoohash_map::locked_table::const_iterator
Definition: cuckoohash_map.hh:2170
libcuckoo::cuckoohash_map::minimum_load_factor
void minimum_load_factor(const double mlf)
Definition: cuckoohash_map.hh:373
libcuckoo::cuckoohash_map::cuckoohash_map
cuckoohash_map(const cuckoohash_map &other, const Allocator &alloc)
Definition: cuckoohash_map.hh:157
libcuckoo::cuckoohash_map::operator=
cuckoohash_map & operator=(const cuckoohash_map &other)=default
libcuckoo::cuckoohash_map::locked_table::operator[]
T & operator[](K &&key)
Definition: cuckoohash_map.hh:2590
libcuckoo::cuckoohash_map::maximum_hashpower
void maximum_hashpower(size_type mhp)
Definition: cuckoohash_map.hh:404
libcuckoo::cuckoohash_map::update
bool update(const K &key, V &&val)
Definition: cuckoohash_map.hh:637
libcuckoo::cuckoohash_map::slot_per_bucket
static constexpr uint16_t slot_per_bucket()
Definition: cuckoohash_map.hh:89
libcuckoo::DEFAULT_MINIMUM_LOAD_FACTOR
constexpr double DEFAULT_MINIMUM_LOAD_FACTOR
Definition: cuckoohash_config.hh:22
libcuckoo::cuckoohash_map::upsert
bool upsert(K &&key, F fn, Args &&... val)
Definition: cuckoohash_map.hh:585
libcuckoo::internal::CanInvokeWithUpsertContext
Definition: cuckoohash_util.hh:151
libcuckoo::cuckoohash_map::swap
void swap(cuckoohash_map &other) noexcept
Definition: cuckoohash_map.hh:226
libcuckoo::cuckoohash_map::maximum_hashpower
size_type maximum_hashpower() const
Definition: cuckoohash_map.hh:417
LIBCUCKOO_ALIGNAS
#define LIBCUCKOO_ALIGNAS(x)
Definition: cuckoohash_util.hh:38
libcuckoo::cuckoohash_map::minimum_load_factor
double minimum_load_factor() const
Definition: cuckoohash_map.hh:391
libcuckoo::cuckoohash_map::locked_table::insert
std::pair< iterator, bool > insert(K &&key, Args &&... val)
Definition: cuckoohash_map.hh:2496
libcuckoo::cuckoohash_map::erase
bool erase(const K &key)
Definition: cuckoohash_map.hh:665
libcuckoo::UpsertContext
UpsertContext
Definition: cuckoohash_util.hh:136
libcuckoo::cuckoohash_map::cuckoohash_map
cuckoohash_map(InputIt first, InputIt last, size_type n=DEFAULT_SIZE, const Hash &hf=Hash(), const KeyEqual &equal=KeyEqual(), const Allocator &alloc=Allocator())
Definition: cuckoohash_map.hh:132
libcuckoo::cuckoohash_map::locked_table::end
iterator end()
Definition: cuckoohash_map.hh:2467
libcuckoo::cuckoohash_map::locked_table::unlock
void unlock()
Definition: cuckoohash_map.hh:2381
libcuckoo::cuckoohash_map::locked_table
Definition: cuckoohash_map.hh:2147
cuckoohash_config.hh
libcuckoo::cuckoohash_map::locked_table::reserve
void reserve(size_type n)
Definition: cuckoohash_map.hh:2643
libcuckoo::cuckoohash_map::max_num_worker_threads
void max_num_worker_threads(size_type extra_threads)
Definition: cuckoohash_map.hh:433
libcuckoo::bucket_container
Definition: bucket_container.hh:31
libcuckoo::cuckoohash_map::locked_table::is_active
bool is_active() const
Definition: cuckoohash_map.hh:2399
libcuckoo::cuckoohash_map::erase_fn
bool erase_fn(const K &key, F fn)
Definition: cuckoohash_map.hh:512
cuckoohash_util.hh
libcuckoo::cuckoohash_map::operator=
cuckoohash_map & operator=(std::initializer_list< value_type > ilist)
Definition: cuckoohash_map.hh:265
libcuckoo::cuckoohash_map::clear
void clear()
Definition: cuckoohash_map.hh:694
libcuckoo::cuckoohash_map::insert_or_assign
bool insert_or_assign(K &&key, V &&val)
Definition: cuckoohash_map.hh:656
libcuckoo::cuckoohash_map::cuckoohash_map
cuckoohash_map(cuckoohash_map &&other, const Allocator &alloc)
Definition: cuckoohash_map.hh:189
libcuckoo::cuckoohash_map::get_allocator
allocator_type get_allocator() const
Definition: cuckoohash_map.hh:305
libcuckoo::cuckoohash_map::uprase_fn
bool uprase_fn(K &&key, F fn, Args &&... val)
Definition: cuckoohash_map.hh:555
libcuckoo::cuckoohash_map::reserve
bool reserve(size_type n)
Definition: cuckoohash_map.hh:689
libcuckoo::cuckoohash_map::load_factor
double load_factor() const
Definition: cuckoohash_map.hh:359
libcuckoo::cuckoohash_map::locked_table::iterator
Definition: cuckoohash_map.hh:2296
libcuckoo::cuckoohash_map::contains
bool contains(const K &key) const
Definition: cuckoohash_map.hh:627
libcuckoo::cuckoohash_map::cuckoohash_map
cuckoohash_map(size_type n=DEFAULT_SIZE, const Hash &hf=Hash(), const KeyEqual &equal=KeyEqual(), const Allocator &alloc=Allocator())
Definition: cuckoohash_map.hh:104