class alignas()

in folly/concurrency/detail/ConcurrentHashMap-detail.h [840:1633]


class alignas(64) SIMDTable {
 public:
  using Node =
      concurrenthashmap::simd::NodeT<KeyType, ValueType, Allocator, Atom>;

 private:
  using HashPair = std::pair<std::size_t, std::size_t>;
  struct alignas(kRequiredVectorAlignment) Chunk {
    static constexpr unsigned kCapacity = 14;
    static constexpr unsigned kDesiredCapacity = 12;

    static constexpr MaskType kFullMask = FullMask<kCapacity>::value;

   private:
    // Non-empty tags have their top bit set.

    // tags [0,8)
    Atom<uint64_t> tags_low_;

    // tags_hi_ holds tags [8,14), hostedOverflowCount and outboundOverflowCount

    // hostedOverflowCount: the number of values in this chunk that were placed
    // because they overflowed their desired chunk.

    // outboundOverflowCount: num values that would have been placed into this
    // chunk if there had been space, including values that also overflowed
    // previous full chunks.  This value saturates; once it becomes 255 it no
    // longer increases nor decreases.

    // Note: more bits can be used for outboundOverflowCount if this
    // optimization becomes useful
    Atom<uint64_t> tags_hi_;

    std::array<aligned_storage_for_t<Atom<Node*>>, kCapacity> rawItems_;

   public:
    void clear() {
      for (size_t i = 0; i < kCapacity; i++) {
        item(i).store(nullptr, std::memory_order_relaxed);
      }
      tags_low_.store(0, std::memory_order_relaxed);
      tags_hi_.store(0, std::memory_order_relaxed);
    }

    std::size_t tag(std::size_t index) const {
      std::size_t off = index % 8;
      const Atom<uint64_t>& tag_src = off == index ? tags_low_ : tags_hi_;
      uint64_t tags = tag_src.load(std::memory_order_relaxed);
      tags >>= (off * 8);
      return tags & 0xff;
    }

    void setTag(std::size_t index, std::size_t tag) {
      std::size_t off = index % 8;
      Atom<uint64_t>& old_tags = off == index ? tags_low_ : tags_hi_;
      uint64_t new_tags = old_tags.load(std::memory_order_relaxed);
      uint64_t mask = 0xffUL << (off * 8);
      new_tags = (new_tags & ~mask) | (tag << (off * 8));
      old_tags.store(new_tags, std::memory_order_release);
    }

    void setNodeAndTag(std::size_t index, Node* node, std::size_t tag) {
      FOLLY_SAFE_DCHECK(
          index < kCapacity && (tag == 0x0 || (tag >= 0x80 && tag <= 0xff)),
          "");
      item(index).store(node, std::memory_order_release);
      setTag(index, tag);
    }

    void clearNodeAndTag(std::size_t index) {
      setNodeAndTag(index, nullptr, 0);
    }

    ////////
    // Tag filtering using SSE2 intrinsics

    SparseMaskIter tagMatchIter(std::size_t needle) const {
      FOLLY_SAFE_DCHECK(needle >= 0x80 && needle < 0x100, "");
      uint64_t low = tags_low_.load(std::memory_order_acquire);
      uint64_t hi = tags_hi_.load(std::memory_order_acquire);
      auto tagV = _mm_set_epi64x(hi, low);
      auto needleV = _mm_set1_epi8(static_cast<uint8_t>(needle));
      auto eqV = _mm_cmpeq_epi8(tagV, needleV);
      auto mask = _mm_movemask_epi8(eqV) & kFullMask;
      return SparseMaskIter{mask};
    }

    MaskType occupiedMask() const {
      uint64_t low = tags_low_.load(std::memory_order_relaxed);
      uint64_t hi = tags_hi_.load(std::memory_order_relaxed);
      auto tagV = _mm_set_epi64x(hi, low);
      return _mm_movemask_epi8(tagV) & kFullMask;
    }

    DenseMaskIter occupiedIter() const {
      // Currently only invoked when relaxed semantics are sufficient.
      return DenseMaskIter{nullptr /*unused*/, occupiedMask()};
    }

    FirstEmptyInMask firstEmpty() const {
      return FirstEmptyInMask{occupiedMask() ^ kFullMask};
    }

    Atom<Node*>* itemAddr(std::size_t i) const {
      return static_cast<Atom<Node*>*>(
          const_cast<void*>(static_cast<void const*>(&rawItems_[i])));
    }

    Atom<Node*>& item(size_t i) { return *launder(itemAddr(i)); }

    static constexpr uint64_t kOutboundOverflowIndex = 7 * 8;
    static constexpr uint64_t kSaturatedOutboundOverflowCount = 0xffUL
        << kOutboundOverflowIndex;
    static constexpr uint64_t kOutboundOverflowOperand = 0x1UL
        << kOutboundOverflowIndex;

    unsigned outboundOverflowCount() const {
      uint64_t count = tags_hi_.load(std::memory_order_relaxed);
      return count >> kOutboundOverflowIndex;
    }

    void incrOutboundOverflowCount() {
      uint64_t count = tags_hi_.load(std::memory_order_relaxed);
      if (count < kSaturatedOutboundOverflowCount) {
        tags_hi_.store(
            count + kOutboundOverflowOperand, std::memory_order_relaxed);
      }
    }

    void decrOutboundOverflowCount() {
      uint64_t count = tags_hi_.load(std::memory_order_relaxed);
      if (count < kSaturatedOutboundOverflowCount) {
        tags_hi_.store(
            count - kOutboundOverflowOperand, std::memory_order_relaxed);
      }
    }

    static constexpr uint64_t kHostedOverflowIndex = 6 * 8;
    static constexpr uint64_t kHostedOverflowOperand = 0x10UL
        << kHostedOverflowIndex;

    unsigned hostedOverflowCount() const {
      uint64_t control = tags_hi_.load(std::memory_order_relaxed);
      return (control >> 52) & 0xf;
    }

    void incrHostedOverflowCount() {
      tags_hi_.fetch_add(kHostedOverflowOperand, std::memory_order_relaxed);
    }

    void decrHostedOverflowCount() {
      tags_hi_.fetch_sub(kHostedOverflowOperand, std::memory_order_relaxed);
    }
  };

  class Chunks : public hazptr_obj_base<Chunks, Atom, HazptrTableDeleter> {
    Chunks() {}
    ~Chunks() {}

   public:
    static Chunks* create(size_t count, hazptr_obj_cohort<Atom>* cohort) {
      auto buf = Allocator().allocate(sizeof(Chunks) + sizeof(Chunk) * count);
      auto chunks = new (buf) Chunks();
      DCHECK(cohort);
      chunks->set_cohort_tag(cohort); // defined in hazptr_obj
      for (size_t i = 0; i < count; i++) {
        new (&chunks->chunks_[i]) Chunk;
        chunks->chunks_[i].clear();
      }
      return chunks;
    }

    void destroy(size_t count) {
      for (size_t i = 0; i < count; i++) {
        chunks_[i].~Chunk();
      }
      this->~Chunks();
      Allocator().deallocate(
          (uint8_t*)this, sizeof(Chunk) * count + sizeof(*this));
    }

    void reclaim_nodes(size_t count) {
      for (size_t i = 0; i < count; i++) {
        Chunk& chunk = chunks_[i];
        auto occupied = chunk.occupiedIter();
        while (occupied.hasNext()) {
          auto idx = occupied.next();
          chunk.setTag(idx, 0);
          Node* node =
              chunk.item(idx).exchange(nullptr, std::memory_order_relaxed);
          // Tags and node ptrs should be in sync at this point.
          DCHECK(node);
          node->retire();
        }
      }
    }

    Chunk* getChunk(size_t index, size_t ccount) {
      DCHECK(isPowTwo(ccount));
      return &chunks_[index & (ccount - 1)];
    }

   private:
    Chunk chunks_[0];
  };

 public:
  static constexpr float kDefaultLoadFactor =
      Chunk::kDesiredCapacity / (float)Chunk::kCapacity;

  typedef std::pair<const KeyType, ValueType> value_type;

  using InsertType = concurrenthashmap::InsertType;

  class Iterator {
   public:
    FOLLY_ALWAYS_INLINE Iterator()
        : hazptrs_(make_hazard_pointer_array<2, Atom>()) {}
    FOLLY_ALWAYS_INLINE explicit Iterator(std::nullptr_t) : hazptrs_() {}
    FOLLY_ALWAYS_INLINE ~Iterator() {}

    void setNode(
        Node* node,
        Chunks* chunks,
        size_t chunk_count,
        uint64_t chunk_idx,
        uint64_t tag_idx) {
      DCHECK(chunk_idx < chunk_count || chunk_idx == 0);
      DCHECK(isPowTwo(chunk_count));
      node_ = node;
      chunks_ = chunks;
      chunk_count_ = chunk_count;
      chunk_idx_ = chunk_idx;
      tag_idx_ = tag_idx;
    }

    const value_type& operator*() const {
      DCHECK(node_);
      return node_->getItem();
    }

    const value_type* operator->() const {
      DCHECK(node_);
      return &(node_->getItem());
    }

    const Iterator& operator++() {
      DCHECK(node_);
      ++tag_idx_;
      findNextNode();
      return *this;
    }

    void next() {
      if (node_) {
        return;
      }
      findNextNode();
    }

    bool operator==(const Iterator& o) const { return node_ == o.node_; }

    bool operator!=(const Iterator& o) const { return !(*this == o); }

    Iterator& operator=(const Iterator& o) = delete;

    Iterator& operator=(Iterator&& o) noexcept {
      if (this != &o) {
        hazptrs_ = std::move(o.hazptrs_);
        node_ = std::exchange(o.node_, nullptr);
        chunks_ = std::exchange(o.chunks_, nullptr);
        chunk_count_ = std::exchange(o.chunk_count_, 0);
        chunk_idx_ = std::exchange(o.chunk_idx_, 0);
        tag_idx_ = std::exchange(o.tag_idx_, 0);
      }
      return *this;
    }

    Iterator(const Iterator& o) = delete;

    Iterator(Iterator&& o) noexcept
        : hazptrs_(std::move(o.hazptrs_)),
          node_(std::exchange(o.node_, nullptr)),
          chunks_(std::exchange(o.chunks_, nullptr)),
          chunk_count_(std::exchange(o.chunk_count_, 0)),
          chunk_idx_(std::exchange(o.chunk_idx_, 0)),
          tag_idx_(std::exchange(o.tag_idx_, 0)) {}

    // These are accessed directly from the functions above
    hazptr_array<2, Atom> hazptrs_;

   private:
    void findNextNode() {
      do {
        if (tag_idx_ >= Chunk::kCapacity) {
          tag_idx_ = 0;
          ++chunk_idx_;
        }
        if (chunk_idx_ >= chunk_count_) {
          node_ = nullptr;
          break;
        }
        DCHECK(chunks_);
        // Note that iteration could also be implemented with tag filtering
        node_ = hazptrs_[1].protect(
            chunks_->getChunk(chunk_idx_, chunk_count_)->item(tag_idx_));
        if (node_) {
          break;
        }
        ++tag_idx_;
      } while (true);
    }

    Node* node_{nullptr};
    Chunks* chunks_{nullptr};
    size_t chunk_count_{0};
    uint64_t chunk_idx_{0};
    uint64_t tag_idx_{0};
  };

  SIMDTable(
      size_t initial_size,
      float load_factor,
      size_t max_size,
      hazptr_obj_cohort<Atom>* cohort)
      : load_factor_(load_factor),
        max_size_(max_size),
        chunks_(nullptr),
        chunk_count_(0) {
    DCHECK(cohort);
    DCHECK(
        max_size_ == 0 ||
        (isPowTwo(max_size_) &&
         (folly::popcount(max_size_ - 1) + ShardBits <= 32)));
    DCHECK(load_factor_ > 0.0);
    load_factor_ = std::min<float>(load_factor_, 1.0);
    rehash(initial_size, cohort);
  }

  ~SIMDTable() {
    auto chunks = chunks_.load(std::memory_order_relaxed);
    // To catch use-after-destruction bugs in user code.
    chunks_.store(nullptr, std::memory_order_release);
    // We can delete and not retire() here, since users must have
    // their own synchronization around destruction.
    auto count = chunk_count_.load(std::memory_order_relaxed);
    chunks->reclaim_nodes(count);
    chunks->destroy(count);
  }

  size_t size() { return size_.load(std::memory_order_acquire); }

  void clearSize() { size_.store(0, std::memory_order_release); }

  void incSize() {
    auto sz = size_.load(std::memory_order_relaxed);
    size_.store(sz + 1, std::memory_order_release);
  }

  void decSize() {
    auto sz = size_.load(std::memory_order_relaxed);
    DCHECK_GT(sz, 0);
    size_.store(sz - 1, std::memory_order_release);
  }

  bool empty() { return size() == 0; }

  template <typename MatchFunc, typename K, typename... Args>
  bool insert(
      Iterator& it,
      const K& k,
      InsertType type,
      MatchFunc match,
      hazptr_obj_cohort<Atom>* cohort,
      Args&&... args) {
    Node* node;
    Chunks* chunks;
    size_t ccount, chunk_idx, tag_idx;

    auto h = HashFn()(k);
    auto hp = splitHash(h);

    std::unique_lock<Mutex> g(m_);

    if (!prepare_insert(
            it,
            k,
            type,
            match,
            cohort,
            chunk_idx,
            tag_idx,
            node,
            chunks,
            ccount,
            hp)) {
      return false;
    }

    auto cur = (Node*)Allocator().allocate(sizeof(Node));
    new (cur) Node(cohort, std::forward<Args>(args)...);

    if (!node) {
      std::tie(chunk_idx, tag_idx) =
          findEmptyInsertLocation(chunks, ccount, hp);
      incSize();
    }

    Chunk* chunk = chunks->getChunk(chunk_idx, ccount);
    chunk->setNodeAndTag(tag_idx, cur, hp.second);
    it.setNode(cur, chunks, ccount, chunk_idx, tag_idx);
    it.hazptrs_[1].reset_protection(cur);

    g.unlock();
    // Retire not under lock
    if (node) {
      node->retire();
    }
    return true;
  }

  template <typename MatchFunc, typename K, typename... Args>
  bool insert(
      Iterator& it,
      const K& k,
      InsertType type,
      MatchFunc match,
      Node* cur,
      hazptr_obj_cohort<Atom>* cohort) {
    DCHECK(cur != nullptr);
    Node* node;
    Chunks* chunks;
    size_t ccount, chunk_idx, tag_idx;

    auto h = HashFn()(k);
    auto hp = splitHash(h);

    std::unique_lock<Mutex> g(m_);

    if (!prepare_insert(
            it,
            k,
            type,
            match,
            cohort,
            chunk_idx,
            tag_idx,
            node,
            chunks,
            ccount,
            hp)) {
      return false;
    }

    if (!node) {
      std::tie(chunk_idx, tag_idx) =
          findEmptyInsertLocation(chunks, ccount, hp);
      incSize();
    }

    Chunk* chunk = chunks->getChunk(chunk_idx, ccount);
    chunk->setNodeAndTag(tag_idx, cur, hp.second);
    it.setNode(cur, chunks, ccount, chunk_idx, tag_idx);
    it.hazptrs_[1].reset_protection(cur);

    g.unlock();
    // Retire not under lock
    if (node) {
      node->retire();
    }
    return true;
  }

  void rehash(size_t size, hazptr_obj_cohort<Atom>* cohort) {
    size_t new_chunk_count = size == 0 ? 0 : (size - 1) / Chunk::kCapacity + 1;
    rehash_internal(folly::nextPowTwo(new_chunk_count), cohort);
  }

  template <typename K>
  bool find(Iterator& res, const K& k) {
    auto& hazz = res.hazptrs_[1];
    auto h = HashFn()(k);
    auto hp = splitHash(h);
    size_t ccount;
    Chunks* chunks;
    getChunksAndCount(ccount, chunks, res.hazptrs_[0]);

    size_t step = probeDelta(hp);
    auto& chunk_idx = hp.first;
    for (size_t tries = 0; tries < ccount; ++tries) {
      Chunk* chunk = chunks->getChunk(chunk_idx, ccount);
      auto hits = chunk->tagMatchIter(hp.second);
      while (hits.hasNext()) {
        size_t tag_idx = hits.next();
        Node* node = hazz.protect(chunk->item(tag_idx));
        if (LIKELY(node && KeyEqual()(k, node->getItem().first))) {
          chunk_idx = chunk_idx & (ccount - 1);
          res.setNode(node, chunks, ccount, chunk_idx, tag_idx);
          return true;
        }
        hazz.reset_protection();
      }

      if (LIKELY(chunk->outboundOverflowCount() == 0)) {
        break;
      }
      chunk_idx += step;
    }
    return false;
  }

  template <typename K, typename MatchFunc>
  std::size_t erase(const K& key, Iterator* iter, MatchFunc match) {
    auto h = HashFn()(key);
    const HashPair hp = splitHash(h);

    std::unique_lock<Mutex> g(m_);

    size_t ccount = chunk_count_.load(std::memory_order_relaxed);
    auto chunks = chunks_.load(std::memory_order_relaxed);
    DCHECK(chunks); // Use-after-destruction by user.
    size_t chunk_idx, tag_idx;

    Node* node = find_internal(key, hp, chunks, ccount, chunk_idx, tag_idx);

    if (!node) {
      return 0;
    }

    if (!match(node->getItem().second)) {
      return 0;
    }

    Chunk* chunk = chunks->getChunk(chunk_idx, ccount);

    // Decrement any overflow counters
    if (chunk->hostedOverflowCount() != 0) {
      size_t index = hp.first;
      size_t delta = probeDelta(hp);
      bool preferredChunk = true;
      while (true) {
        Chunk* overflowChunk = chunks->getChunk(index, ccount);
        if (chunk == overflowChunk) {
          if (!preferredChunk) {
            overflowChunk->decrHostedOverflowCount();
          }
          break;
        }
        overflowChunk->decrOutboundOverflowCount();
        preferredChunk = false;
        index += delta;
      }
    }

    chunk->clearNodeAndTag(tag_idx);

    decSize();
    if (iter) {
      iter->hazptrs_[0].reset_protection(chunks);
      iter->setNode(nullptr, chunks, ccount, chunk_idx, tag_idx + 1);
      iter->next();
    }
    // Retire the node while not under the lock.
    g.unlock();
    node->retire();
    return 1;
  }

  void clear(hazptr_obj_cohort<Atom>* cohort) {
    size_t ccount;
    Chunks* chunks;
    {
      std::lock_guard<Mutex> g(m_);
      ccount = chunk_count_.load(std::memory_order_relaxed);
      auto newchunks = Chunks::create(ccount, cohort);
      chunks = chunks_.load(std::memory_order_relaxed);
      chunks_.store(newchunks, std::memory_order_release);
      clearSize();
    }
    DCHECK(chunks); // Use-after-destruction by user.
    chunks->reclaim_nodes(ccount);
    chunks->retire(HazptrTableDeleter(ccount));
  }

  void max_load_factor(float factor) {
    DCHECK(factor > 0.0);
    if (factor > 1.0) {
      throw_exception<std::invalid_argument>("load factor must be <= 1.0");
    }
    std::lock_guard<Mutex> g(m_);
    load_factor_ = factor;
    auto ccount = chunk_count_.load(std::memory_order_relaxed);
    grow_threshold_ = ccount * Chunk::kCapacity * load_factor_;
  }

  Iterator cbegin() {
    Iterator res;
    size_t ccount;
    Chunks* chunks;
    getChunksAndCount(ccount, chunks, res.hazptrs_[0]);
    res.setNode(nullptr, chunks, ccount, 0, 0);
    res.next();
    return res;
  }

  Iterator cend() { return Iterator(nullptr); }

 private:
  static HashPair splitHash(std::size_t hash) {
    std::size_t c = _mm_crc32_u64(0, hash);
    size_t tag = (c >> 24) | 0x80;
    hash += c;
    return std::make_pair(hash, tag);
  }

  static size_t probeDelta(HashPair hp) { return 2 * hp.second + 1; }

  // Must hold lock.
  template <typename K>
  Node* find_internal(
      const K& k,
      const HashPair& hp,
      Chunks* chunks,
      size_t ccount,
      size_t& chunk_idx,
      size_t& tag_idx) {
    // must be called with mutex held
    size_t step = probeDelta(hp);
    chunk_idx = hp.first;

    for (size_t tries = 0; tries < ccount; ++tries) {
      Chunk* chunk = chunks->getChunk(chunk_idx, ccount);
      auto hits = chunk->tagMatchIter(hp.second);
      while (hits.hasNext()) {
        tag_idx = hits.next();
        Node* node = chunk->item(tag_idx).load(std::memory_order_acquire);
        if (LIKELY(node && KeyEqual()(k, node->getItem().first))) {
          chunk_idx = (chunk_idx & (ccount - 1));
          return node;
        }
      }
      if (LIKELY(chunk->outboundOverflowCount() == 0)) {
        break;
      }
      chunk_idx += step;
    }
    return nullptr;
  }

  template <typename MatchFunc, typename K, typename... Args>
  bool prepare_insert(
      Iterator& it,
      const K& k,
      InsertType type,
      MatchFunc match,
      hazptr_obj_cohort<Atom>* cohort,
      size_t& chunk_idx,
      size_t& tag_idx,
      Node*& node,
      Chunks*& chunks,
      size_t& ccount,
      const HashPair& hp) {
    ccount = chunk_count_.load(std::memory_order_relaxed);
    chunks = chunks_.load(std::memory_order_relaxed);

    if (size() >= grow_threshold_ && type == InsertType::DOES_NOT_EXIST) {
      if (max_size_ && size() << 1 > max_size_) {
        // Would exceed max size.
        throw_exception<std::bad_alloc>();
      }
      rehash_internal(ccount << 1, cohort);
      ccount = chunk_count_.load(std::memory_order_relaxed);
      chunks = chunks_.load(std::memory_order_relaxed);
    }

    DCHECK(chunks); // Use-after-destruction by user.
    node = find_internal(k, hp, chunks, ccount, chunk_idx, tag_idx);

    it.hazptrs_[0].reset_protection(chunks);
    if (node) {
      it.hazptrs_[1].reset_protection(node);
      it.setNode(node, chunks, ccount, chunk_idx, tag_idx);
      if (type == InsertType::MATCH) {
        if (!match(node->getItem().second)) {
          return false;
        }
      } else if (type == InsertType::DOES_NOT_EXIST) {
        return false;
      }
    } else {
      if (type != InsertType::DOES_NOT_EXIST && type != InsertType::ANY) {
        it.hazptrs_[0].reset_protection();
        return false;
      }
      // Already checked for rehash on DOES_NOT_EXIST, now check on ANY
      if (size() >= grow_threshold_ && type == InsertType::ANY) {
        if (max_size_ && size() << 1 > max_size_) {
          // Would exceed max size.
          throw_exception<std::bad_alloc>();
        }
        rehash_internal(ccount << 1, cohort);
        ccount = chunk_count_.load(std::memory_order_relaxed);
        chunks = chunks_.load(std::memory_order_relaxed);
        DCHECK(chunks); // Use-after-destruction by user.
        it.hazptrs_[0].reset_protection(chunks);
      }
    }
    return true;
  }

  void rehash_internal(
      size_t new_chunk_count, hazptr_obj_cohort<Atom>* cohort) {
    DCHECK(isPowTwo(new_chunk_count));
    auto old_chunk_count = chunk_count_.load(std::memory_order_relaxed);
    if (old_chunk_count >= new_chunk_count) {
      return;
    }
    auto new_chunks = Chunks::create(new_chunk_count, cohort);
    auto old_chunks = chunks_.load(std::memory_order_relaxed);
    grow_threshold_ =
        to_integral(new_chunk_count * Chunk::kCapacity * load_factor_);

    for (size_t i = 0; i < old_chunk_count; i++) {
      DCHECK(old_chunks); // Use-after-destruction by user.
      Chunk* oldchunk = old_chunks->getChunk(i, old_chunk_count);
      auto occupied = oldchunk->occupiedIter();
      while (occupied.hasNext()) {
        auto idx = occupied.next();
        Node* node = oldchunk->item(idx).load(std::memory_order_relaxed);
        size_t new_chunk_idx;
        size_t new_tag_idx;
        auto h = HashFn()(node->getItem().first);
        auto hp = splitHash(h);
        std::tie(new_chunk_idx, new_tag_idx) =
            findEmptyInsertLocation(new_chunks, new_chunk_count, hp);
        Chunk* newchunk = new_chunks->getChunk(new_chunk_idx, new_chunk_count);
        newchunk->setNodeAndTag(new_tag_idx, node, hp.second);
      }
    }

    seqlock_.fetch_add(1, std::memory_order_release);
    chunk_count_.store(new_chunk_count, std::memory_order_release);
    chunks_.store(new_chunks, std::memory_order_release);
    seqlock_.fetch_add(1, std::memory_order_release);
    if (old_chunks) {
      old_chunks->retire(HazptrTableDeleter(old_chunk_count));
    }
  }

  void getChunksAndCount(
      size_t& ccount, Chunks*& chunks, hazptr_holder<Atom>& hazptr) {
    while (true) {
      auto seqlock = seqlock_.load(std::memory_order_acquire);
      ccount = chunk_count_.load(std::memory_order_acquire);
      chunks = hazptr.protect(chunks_);
      auto seqlock2 = seqlock_.load(std::memory_order_acquire);
      if (!(seqlock & 1) && (seqlock == seqlock2)) {
        break;
      }
    }
    DCHECK(chunks);
  }

  std::pair<size_t, size_t> findEmptyInsertLocation(
      Chunks* chunks, size_t ccount, const HashPair& hp) {
    size_t chunk_idx = hp.first;
    Chunk* dst_chunk = chunks->getChunk(chunk_idx, ccount);
    auto firstEmpty = dst_chunk->firstEmpty();

    if (!firstEmpty.hasIndex()) {
      size_t delta = probeDelta(hp);
      do {
        dst_chunk->incrOutboundOverflowCount();
        chunk_idx += delta;
        dst_chunk = chunks->getChunk(chunk_idx, ccount);
        firstEmpty = dst_chunk->firstEmpty();
      } while (!firstEmpty.hasIndex());
      dst_chunk->incrHostedOverflowCount();
    }
    size_t dst_tag_idx = firstEmpty.index();
    return std::make_pair(chunk_idx & (ccount - 1), dst_tag_idx);
  }

  Mutex m_;
  float load_factor_; // ceil of 1.0
  size_t grow_threshold_;
  Atom<size_t> size_{0};
  size_t const max_size_;

  // Fields needed for read-only access, on separate cacheline.
  alignas(64) Atom<Chunks*> chunks_{nullptr};
  std::atomic<uint64_t> seqlock_{0};
  Atom<size_t> chunk_count_;
};