]> git.apps.os.sepia.ceph.com Git - ceph.git/commitdiff
crimson/os/seastore/btree: introduce parent<->child pointers for fixed-kv-btree nodes
authorXuehan Xu <xxhdx1985126@gmail.com>
Tue, 11 Oct 2022 02:34:16 +0000 (10:34 +0800)
committerMatan Breizman <mbreizma@redhat.com>
Tue, 23 May 2023 13:24:44 +0000 (13:24 +0000)
maintain correct parent<->child pointers when modifying the btree

Signed-off-by: Xuehan Xu <xxhdx1985126@gmail.com>
(cherry picked from commit 71051f997fca8ff0fdcd55586e3150bffa2a916a)

src/crimson/os/seastore/backref/backref_tree_node.h
src/crimson/os/seastore/btree/btree_range_pin.h
src/crimson/os/seastore/btree/fixed_kv_btree.h
src/crimson/os/seastore/btree/fixed_kv_node.h
src/crimson/os/seastore/cache.cc
src/crimson/os/seastore/cached_extent.cc
src/crimson/os/seastore/cached_extent.h
src/crimson/os/seastore/lba_manager/btree/lba_btree_node.cc
src/crimson/os/seastore/lba_manager/btree/lba_btree_node.h

index 5a40675de98b52ab82f5bba0be01bc07eda241e6..bebbc0aa4b2f5ddad143613f7cd24c86133316df 100644 (file)
@@ -130,4 +130,5 @@ using BackrefLeafNodeRef = BackrefLeafNode::Ref;
 template <> struct fmt::formatter<crimson::os::seastore::backref::backref_map_val_t> : fmt::ostream_formatter {};
 template <> struct fmt::formatter<crimson::os::seastore::backref::BackrefInternalNode> : fmt::ostream_formatter {};
 template <> struct fmt::formatter<crimson::os::seastore::backref::BackrefLeafNode> : fmt::ostream_formatter {};
+template <> struct fmt::formatter<crimson::os::seastore::backref::backref_node_meta_t> : fmt::ostream_formatter {};
 #endif
index da1a67ba5d35958df9188bacbdce4eb0f81e18b5..5942e85f3175fd4fe157640b91af3bc936bafb21 100644 (file)
@@ -41,6 +41,10 @@ struct fixed_kv_node_meta_t {
       (end > other.begin);
   }
 
+  bool is_in_range(const bound_t key) const {
+    return begin <= key && end > key;
+  }
+
   std::pair<fixed_kv_node_meta_t, fixed_kv_node_meta_t> split_into(bound_t pivot) const {
     return std::make_pair(
       fixed_kv_node_meta_t{begin, pivot, depth},
@@ -116,9 +120,13 @@ struct fixed_kv_node_meta_le_t {
 template <typename T>
 class btree_pin_set_t;
 
+template <typename node_key_t>
+class FixedKVNode;
+
 template <typename node_bound_t>
 class btree_range_pin_t : public boost::intrusive::set_base_hook<> {
   friend class btree_pin_set_t<node_bound_t>;
+  friend class FixedKVNode<node_bound_t>;
   fixed_kv_node_meta_t<node_bound_t> range;
 
   btree_pin_set_t<node_bound_t> *pins = nullptr;
index e2e94f0454c44c92448f2643431ea97389a93389..4a5c22d6de02a88e65115d9a1702f99ba5cbc2d9 100644 (file)
@@ -823,6 +823,21 @@ public:
         n_fixed_kv_extent->get_bptr().c_str());
       n_fixed_kv_extent->set_modify_time(fixed_kv_extent.get_modify_time());
       n_fixed_kv_extent->pin.set_range(n_fixed_kv_extent->get_node_meta());
+
+      if (fixed_kv_extent.get_type() == internal_node_t::TYPE) {
+        if (!fixed_kv_extent.is_pending()) {
+          n_fixed_kv_extent->copy_sources.emplace(&fixed_kv_extent);
+          n_fixed_kv_extent->prior_instance = &fixed_kv_extent;
+        } else {
+          ceph_assert(fixed_kv_extent.is_mutation_pending());
+          n_fixed_kv_extent->copy_sources.emplace(
+            (typename internal_node_t::base_t*
+             )fixed_kv_extent.get_prior_instance().get());
+          n_fixed_kv_extent->children = std::move(fixed_kv_extent.children);
+          n_fixed_kv_extent->prior_instance = fixed_kv_extent.get_prior_instance();
+          n_fixed_kv_extent->adjust_ptracker_for_children();
+        }
+      }
       
       /* This is a bit underhanded.  Any relative addrs here must necessarily
        * be record relative as we are rewriting a dirty extent.  Thus, we
@@ -853,7 +868,8 @@ public:
         n_fixed_kv_extent->get_node_meta().depth,
         n_fixed_kv_extent->get_node_meta().begin,
         e->get_paddr(),
-        n_fixed_kv_extent->get_paddr()
+        n_fixed_kv_extent->get_paddr(),
+        n_fixed_kv_extent
       ).si_then([c, e] {
         c.cache.retire_extent(c.trans, e);
       });
@@ -877,17 +893,19 @@ public:
     depth_t depth,
     node_key_t laddr,
     paddr_t old_addr,
-    paddr_t new_addr)
+    paddr_t new_addr,
+    typename internal_node_t::base_ref nextent)
   {
     LOG_PREFIX(FixedKVBtree::update_internal_mapping);
     SUBTRACET(
       seastore_fixedkv_tree,
-      "updating laddr {} at depth {} from {} to {}",
+      "updating laddr {} at depth {} from {} to {}, nextent {}",
       c.trans,
       laddr,
       depth,
       old_addr,
-      new_addr);
+      new_addr,
+      *nextent);
 
     return lower_bound(
       c, laddr
@@ -970,7 +988,7 @@ public:
           parent.node
         );
         typename internal_node_t::Ref mparent = mut->cast<internal_node_t>();
-        mparent->update(piter, new_addr);
+        mparent->update(piter, new_addr, nextent.get());
 
         /* Note, iter is now invalid as we didn't udpate either the parent
          * node reference to the new mutable instance nor did we update the
@@ -1439,11 +1457,13 @@ private:
 
       parent_node->update(
         parent_iter,
-        left->get_paddr());
+        left->get_paddr(),
+        left.get());
       parent_node->insert(
         parent_iter + 1,
         pivot,
-        right->get_paddr());
+        right->get_paddr(),
+        right.get());
 
       SUBTRACET(
         seastore_fixedkv_tree,
@@ -1669,7 +1689,8 @@ private:
 
         parent_pos.node->update(
           liter,
-          replacement->get_paddr());
+          replacement->get_paddr(),
+          replacement.get());
         parent_pos.node->remove(riter);
 
         pos.node = replacement;
@@ -1692,11 +1713,13 @@ private:
 
         parent_pos.node->update(
           liter,
-          replacement_l->get_paddr());
+          replacement_l->get_paddr(),
+          replacement_l.get());
         parent_pos.node->replace(
           riter,
           pivot,
-          replacement_r->get_paddr());
+          replacement_r->get_paddr(),
+          replacement_r.get());
 
         if (donor_is_left) {
           assert(parent_pos.pos > 0);
index 75628f6fcbff7e842a7056e8c3aca5a01edf8f18..101ad4945d3f10165aed8222ce5f645d2cb041ac 100644 (file)
@@ -29,27 +29,435 @@ namespace crimson::os::seastore {
 template <typename node_key_t>
 struct FixedKVNode : CachedExtent {
   using FixedKVNodeRef = TCachedExtentRef<FixedKVNode>;
+  struct parent_tracker_t
+    : public boost::intrusive_ref_counter<
+       parent_tracker_t, boost::thread_unsafe_counter> {
+    parent_tracker_t(FixedKVNodeRef parent)
+      : parent(parent) {}
+    parent_tracker_t(FixedKVNode* parent)
+      : parent(parent) {}
+    FixedKVNodeRef parent = nullptr;
+    ~parent_tracker_t() {
+      // this is parent's tracker, reset it
+      if (parent->my_tracker == this) {
+       parent->my_tracker = nullptr;
+      }
+    }
+  };
 
+  using parent_tracker_ref = boost::intrusive_ptr<parent_tracker_t>;
   btree_range_pin_t<node_key_t> pin;
 
-  FixedKVNode(ceph::bufferptr &&ptr) : CachedExtent(std::move(ptr)), pin(this) {}
+  struct copy_source_cmp_t {
+    using is_transparent = node_key_t;
+    bool operator()(const FixedKVNodeRef &l, const FixedKVNodeRef &r) const {
+      assert(l->pin.range.end <= r->pin.range.begin
+       || r->pin.range.end <= l->pin.range.begin
+       || (l->pin.range.begin == r->pin.range.begin
+           && l->pin.range.end == r->pin.range.end));
+      return l->pin.range.begin < r->pin.range.begin;
+    }
+    bool operator()(const node_key_t &l, const FixedKVNodeRef &r) const {
+      return l < r->pin.range.begin;
+    }
+    bool operator()(const FixedKVNodeRef &l, const node_key_t &r) const {
+      return l->pin.range.begin < r;
+    }
+  };
+
+  /*
+   *
+   * Nodes of fixed-kv-btree connect to their child nodes by pointers following
+   * invariants below:
+   *
+   * 1. if nodes are stable:
+   *   a. parent points at the node's stable parent
+   *   b. prior_instance is empty
+   *   c. child pointers point at stable children. Child resolution is done
+   *      directly via this array.
+   *   c. copy_sources is empty
+   * 2. if nodes are mutation_pending:
+   *   a. parent is empty and needs to be fixed upon commit
+   *   b. prior_instance points to its stable version
+   *   c. child pointers are null except for initial_pending() children of
+   *      this transaction. Child resolution is done by first checking this
+   *      array, and then recursively resolving via the parent. We copy child
+   *      pointers from parent on commit.
+   *   c. copy_sources is empty
+   * 3. if nodes are initial_pending
+   *   a. parent points at its pending parent on this transaction (must exist)
+   *   b. prior_instance is empty or, if it's the result of rewrite, points to
+   *      its stable predecessor
+   *   c. child pointers are null except for initial_pending() children of
+   *      this transaction (live due to 3a below). Child resolution is done
+   *      by first checking this array, and then recursively resolving via
+   *      the correct copy_sources entry. We copy child pointers from copy_sources
+   *      on commit.
+   *   d. copy_sources contains the set of stable nodes at the same tree-level(only
+   *      its "prior_instance" if the node is the result of a rewrite), with which
+   *      the lba range of this node overlaps.
+   */
+  std::vector<CachedExtent*> children;
+  std::set<FixedKVNodeRef, copy_source_cmp_t> copy_sources;
+  uint16_t capacity = 0;
+  parent_tracker_t* my_tracker = nullptr;
+  parent_tracker_ref parent_tracker;
+
+  FixedKVNode(uint16_t capacity, ceph::bufferptr &&ptr)
+    : CachedExtent(std::move(ptr)),
+      pin(this),
+      children(capacity, nullptr),
+      capacity(capacity) {}
   FixedKVNode(const FixedKVNode &rhs)
-    : CachedExtent(rhs), pin(rhs.pin, this) {}
+    : CachedExtent(rhs),
+      pin(rhs.pin, this),
+      children(rhs.capacity, nullptr),
+      capacity(rhs.capacity) {}
 
   virtual fixed_kv_node_meta_t<node_key_t> get_node_meta() const = 0;
+  virtual uint16_t get_node_size() const = 0;
 
   virtual ~FixedKVNode() = default;
+  virtual node_key_t get_key_from_idx(uint16_t idx) const = 0;
+
+  template<typename iter_t>
+  void update_child_ptr(iter_t iter, ChildableCachedExtent* child) {
+    children[iter.get_offset()] = child;
+    set_child_ptracker(child);
+  }
+
+  template<typename iter_t>
+  void insert_child_ptr(iter_t iter, ChildableCachedExtent* child) {
+    auto raw_children = children.data();
+    auto offset = iter.get_offset();
+    std::memmove(
+      &raw_children[offset + 1],
+      &raw_children[offset],
+      (get_node_size() - offset) * sizeof(ChildableCachedExtent*));
+    children[offset] = child;
+    set_child_ptracker(child);
+  }
+
+  template<typename iter_t>
+  void remove_child_ptr(iter_t iter) {
+    LOG_PREFIX(FixedKVNode::remove_child_ptr);
+    auto raw_children = children.data();
+    auto offset = iter.get_offset();
+    SUBTRACE(seastore_fixedkv_tree, "trans.{}, pos {}, total size {}, extent {}",
+      this->pending_for_transaction,
+      offset,
+      get_node_size(),
+      (void*)raw_children[offset]);
+    // parent tracker of the child being removed will be
+    // reset when the child is invalidated, so no need to
+    // reset it here
+    std::memmove(
+      &raw_children[offset],
+      &raw_children[offset + 1],
+      (get_node_size() - offset - 1) * sizeof(ChildableCachedExtent*));
+  }
+
+  FixedKVNode& get_stable_for_key(node_key_t key) {
+    ceph_assert(is_pending());
+    if (is_mutation_pending()) {
+      return (FixedKVNode&)*get_prior_instance();
+    } else {
+      ceph_assert(!copy_sources.empty());
+      auto it = copy_sources.upper_bound(key);
+      it--;
+      auto &copy_source = *it;
+      ceph_assert(copy_source->get_node_meta().is_in_range(key));
+      return *copy_source;
+    }
+  }
+
+  static void push_copy_sources(
+    FixedKVNode &dest,
+    FixedKVNode &src)
+  {
+    ceph_assert(dest.is_initial_pending());
+    if (!src.is_pending()) {
+      dest.copy_sources.emplace(&src);
+    } else if (src.is_mutation_pending()) {
+      dest.copy_sources.emplace(
+       src.get_prior_instance()->template cast<FixedKVNode>());
+    } else {
+      ceph_assert(src.is_initial_pending());
+      dest.copy_sources.insert(
+       src.copy_sources.begin(),
+       src.copy_sources.end());
+    }
+  }
+
+  virtual uint16_t get_node_split_pivot() = 0;
+
+  static void move_child_ptrs(
+    FixedKVNode &dest,
+    FixedKVNode &src,
+    size_t dest_start,
+    size_t src_start,
+    size_t src_end)
+  {
+    std::memmove(
+      dest.children.data() + dest_start,
+      src.children.data() + src_start,
+      (src_end - src_start) * sizeof(ChildableCachedExtent*));
+
+    ceph_assert(src_start < src_end);
+    ceph_assert(src.children.size() >= src_end);
+    for (auto it = src.children.begin() + src_start;
+       it != src.children.begin() + src_end;
+       it++)
+    {
+      auto child = *it;
+      if (is_valid_child_ptr(child)) {
+       dest.set_child_ptracker(child);
+      }
+    }
+  }
+
+  void split_child_ptrs(
+    FixedKVNode &left,
+    FixedKVNode &right)
+  {
+    assert(!left.my_tracker);
+    assert(!right.my_tracker);
+    push_copy_sources(left, *this);
+    push_copy_sources(right, *this);
+    if (is_pending()) {
+      uint16_t pivot = get_node_split_pivot();
+      move_child_ptrs(left, *this, 0, 0, pivot);
+      move_child_ptrs(right, *this, 0, pivot, get_node_size());
+      my_tracker = nullptr;
+    }
+  }
+
+  void merge_child_ptrs(
+    FixedKVNode &left,
+    FixedKVNode &right)
+  {
+    ceph_assert(!my_tracker);
+    push_copy_sources(*this, left);
+    push_copy_sources(*this, right);
+
+    if (left.is_pending()) {
+      move_child_ptrs(*this, left, 0, 0, left.get_node_size());
+      left.my_tracker = nullptr;
+    }
+
+    if (right.is_pending()) {
+      move_child_ptrs(*this, right, left.get_node_size(), 0, right.get_node_size());
+      right.my_tracker = nullptr;
+    }
+  }
+
+  static void balance_child_ptrs(
+    FixedKVNode &left,
+    FixedKVNode &right,
+    bool prefer_left,
+    FixedKVNode &replacement_left,
+    FixedKVNode &replacement_right)
+  {
+    size_t l_size = left.get_node_size();
+    size_t r_size = right.get_node_size();
+    size_t total = l_size + r_size;
+    size_t pivot_idx = (l_size + r_size) / 2;
+    if (total % 2 && prefer_left) {
+      pivot_idx++;
+    }
+
+    assert(!replacement_left.my_tracker);
+    assert(!replacement_right.my_tracker);
+    if (pivot_idx < l_size) {
+      // deal with left
+      push_copy_sources(replacement_left, left);
+      push_copy_sources(replacement_right, left);
+      if (left.is_pending()) {
+       move_child_ptrs(replacement_left, left, 0, 0, pivot_idx);
+       move_child_ptrs(replacement_right, left, 0, pivot_idx, l_size);
+       left.my_tracker = nullptr;
+      }
+
+      // deal with right
+      push_copy_sources(replacement_right, right);
+      if (right.is_pending()) {
+       move_child_ptrs(replacement_right, right, l_size - pivot_idx, 0, r_size);
+       right.my_tracker= nullptr;
+      }
+    } else {
+      // deal with left
+      push_copy_sources(replacement_left, left);
+      if (left.is_pending()) {
+       move_child_ptrs(replacement_left, left, 0, 0, l_size);
+       left.my_tracker = nullptr;
+      }
+
+      // deal with right
+      push_copy_sources(replacement_left, right);
+      push_copy_sources(replacement_right, right);
+      if (right.is_pending()) {
+       move_child_ptrs(replacement_left, right, l_size, 0, pivot_idx - l_size);
+       move_child_ptrs(replacement_right, right, 0, pivot_idx - l_size, r_size);
+       right.my_tracker= nullptr;
+      }
+    }
+  }
+
+  void set_parent_tracker_from_prior_instance() {
+    if (pin.is_root()) {
+      return;
+    }
+    assert(is_mutation_pending());
+    auto &prior = (FixedKVNode&)(*get_prior_instance());
+    parent_tracker = prior.parent_tracker;
+    auto &parent = parent_tracker->parent;
+    assert(parent);
+    assert(parent->is_valid());
+    //TODO: can this search be avoided?
+    auto off = parent->lower_bound_offset(get_node_meta().begin);
+    assert(parent->get_key_from_idx(off) == get_node_meta().begin);
+    parent->children[off] = this;
+  }
+
+  bool is_children_empty() const {
+    for (auto it = children.begin();
+       it != children.begin() + get_node_size();
+       it++) {
+      if (is_valid_child_ptr(*it)
+         && (*it)->is_valid()) {
+       return false;
+      }
+    }
+    return true;
+  }
+
+  void set_children_from_prior_instance() {
+    assert(get_prior_instance());
+    auto &prior = (FixedKVNode&)(*get_prior_instance());
+    assert(prior.my_tracker || prior.is_children_empty());
+
+    if (prior.my_tracker) {
+      prior.my_tracker->parent.reset(this);
+      my_tracker = prior.my_tracker;
+      // All my initial pending children is pointing to the original
+      // tracker which has been dropped by the above line, so need
+      // to adjust them to point to the new tracker
+      adjust_ptracker_for_children();
+    }
+    assert(my_tracker || is_children_empty());
+  }
+
+  void adjust_ptracker_for_children() {
+    auto begin = children.begin();
+    auto end = begin + get_node_size();
+    ceph_assert(end <= children.end());
+    for (auto it = begin; it != end; it++) {
+      auto child = *it;
+      if (child) {
+       set_child_ptracker((FixedKVNode*)child);
+      }
+    }
+  }
 
   void on_delta_write(paddr_t record_block_offset) final {
     // All in-memory relative addrs are necessarily record-relative
     assert(get_prior_instance());
+    assert(pending_for_transaction);
     pin.take_pin(get_prior_instance()->template cast<FixedKVNode>()->pin);
     resolve_relative_addrs(record_block_offset);
   }
 
+  virtual uint16_t lower_bound_offset(node_key_t) const = 0;
+  virtual uint16_t upper_bound_offset(node_key_t) const = 0;
+  virtual uint16_t child_pos_for_key(node_key_t) const = 0;
+
+  virtual bool validate_stable_children() = 0;
+
+  template<typename iter_t>
+  uint16_t copy_children_from_stable_source(
+    FixedKVNode &source,
+    iter_t foreign_start_it,
+    iter_t foreign_end_it,
+    iter_t local_start_it) {
+    auto foreign_it = foreign_start_it, local_it = local_start_it;
+    while (foreign_it != foreign_end_it
+         && local_it.get_offset() < get_node_size())
+    {
+      auto &child = children[local_it.get_offset()];
+      if (foreign_it.get_key() == local_it.get_key()) {
+       // the foreign key is preserved
+       if (!child) {
+         child = source.children[foreign_it.get_offset()];
+       }
+       foreign_it++;
+       local_it++;
+      } else if (foreign_it.get_key() < local_it.get_key()) {
+       // the foreign key has been removed, because, if it hasn't,
+       // there must have been a local key before the one pointed
+       // by the current "local_it" that's equal to this foreign key
+       // and has pushed the foreign_it forward.
+       foreign_it++;
+      } else {
+       // the local key must be a newly inserted one.
+       local_it++;
+      }
+    }
+    return local_it.get_offset();
+  }
+
+  template<typename Func>
+  void copy_children_from_stable_sources(Func &&get_iter) {
+    if (!copy_sources.empty()) {
+      auto it = --copy_sources.upper_bound(get_node_meta().begin);
+      auto &cs = *it;
+      uint16_t start_pos = cs->lower_bound_offset(
+       get_node_meta().begin);
+      if (start_pos == cs->get_node_size()) {
+       it++;
+       start_pos = 0;
+      }
+      uint16_t local_next_pos = 0;
+      for (; it != copy_sources.end(); it++) {
+       auto& copy_source = *it;
+       auto end_pos = copy_source->get_node_size();
+       if (copy_source->get_node_meta().is_in_range(get_node_meta().end)) {
+         end_pos = copy_source->upper_bound_offset(get_node_meta().end);
+       }
+       auto local_start_iter = get_iter(*this, local_next_pos);
+       auto foreign_start_iter = get_iter(*copy_source, start_pos);
+       auto foreign_end_iter = get_iter(*copy_source, end_pos);
+       local_next_pos = copy_children_from_stable_source(
+         *copy_source, foreign_start_iter, foreign_end_iter, local_start_iter);
+       if (end_pos != copy_source->get_node_size()) {
+         break;
+       }
+       start_pos = 0;
+      }
+    }
+  }
+
+  void on_invalidated(Transaction &t) final {
+    parent_tracker.reset();
+  }
+
+  bool is_rewrite() {
+    return is_initial_pending() && get_prior_instance();
+  }
+
   void on_initial_write() final {
     // All in-memory relative addrs are necessarily block-relative
     resolve_relative_addrs(get_paddr());
+    ceph_assert(
+      parent_tracker
+       ? (parent_tracker->parent && parent_tracker->parent->is_valid())
+       : true);
+  }
+
+  void set_child_ptracker(FixedKVNode *child) {
+    if (!my_tracker) {
+      my_tracker = new parent_tracker_t(this);
+    }
+    child->parent_tracker.reset(my_tracker);
   }
 
   void on_clean_read() final {
@@ -81,6 +489,8 @@ struct FixedKVInternalNode
       NODE_KEY, NODE_KEY_LE,
       paddr_t, paddr_le_t> {
   using Ref = TCachedExtentRef<node_type_t>;
+  using base_t = FixedKVNode<NODE_KEY>;
+  using base_ref = typename FixedKVNode<NODE_KEY>::FixedKVNodeRef;
   using node_layout_t =
     common::FixedKVNodeLayout<
       CAPACITY,
@@ -92,17 +502,101 @@ struct FixedKVInternalNode
       paddr_le_t>;
   using internal_const_iterator_t = typename node_layout_t::const_iterator;
   using internal_iterator_t = typename node_layout_t::iterator;
-  template <typename... T>
-  FixedKVInternalNode(T&&... t) :
-    FixedKVNode<NODE_KEY>(std::forward<T>(t)...),
-    node_layout_t(this->get_bptr().c_str()) {}
+  using this_type_t = FixedKVInternalNode<
+    CAPACITY,
+    NODE_KEY,
+    NODE_KEY_LE,
+    node_size,
+    node_type_t>;
 
-  virtual ~FixedKVInternalNode() {}
+  FixedKVInternalNode(ceph::bufferptr &&ptr)
+    : FixedKVNode<NODE_KEY>(CAPACITY, std::move(ptr)),
+      node_layout_t(this->get_bptr().c_str()) {}
+  FixedKVInternalNode(const FixedKVInternalNode &rhs)
+    : FixedKVNode<NODE_KEY>(rhs),
+      node_layout_t(this->get_bptr().c_str()) {}
+
+  uint16_t get_node_split_pivot() final {
+    return this->get_split_pivot().get_offset();
+  }
+
+  void prepare_write() final {
+    if (this->is_initial_pending()) {
+      if (this->is_rewrite()) {
+       this->set_children_from_prior_instance();
+      }
+      this->copy_children_from_stable_sources(
+       [this](base_t &node, uint16_t pos) {
+         ceph_assert(node.get_type() == this->get_type());
+         auto &n = static_cast<this_type_t&>(node);
+         return n.iter_idx(pos);
+       }
+      );
+      if (this->is_rewrite()) {
+       this->reset_prior_instance();
+      } else {
+       this->adjust_ptracker_for_children();
+      }
+      assert(this->validate_stable_children());
+      this->copy_sources.clear();
+    }
+  }
+
+  bool validate_stable_children() final {
+    LOG_PREFIX(FixedKVInternalNode::validate_stable_children);
+    if (this->children.empty()) {
+      return false;
+    }
+
+    for (auto i : *this) {
+      auto child = (FixedKVNode<NODE_KEY>*)this->children[i.get_offset()];
+      if (child && child->range.begin != i.get_key()) {
+       SUBERROR(seastore_fixedkv_tree,
+         "stable child not valid: child {}, child meta{}, key {}",
+         *child,
+         child->get_node_meta(),
+         i.get_key());
+       ceph_abort();
+       return false;
+      }
+    }
+    return true;
+  }
+
+  virtual ~FixedKVInternalNode() {
+    if (!this->pin.is_root()
+       && this->is_valid()
+       && !this->is_pending()) {
+      ceph_assert(this->parent_tracker);
+      auto &parent = this->parent_tracker->parent;
+      ceph_assert(parent);
+      auto off = parent->lower_bound_offset(this->get_meta().begin);
+      assert(parent->get_key_from_idx(off) == get_node_meta().begin);
+      assert(parent->children[off] == this);
+      parent->children[off] = nullptr;
+    }
+  }
+
+  uint16_t lower_bound_offset(NODE_KEY key) const final {
+    return this->lower_bound(key).get_offset();
+  }
+
+  uint16_t upper_bound_offset(NODE_KEY key) const final {
+    return this->upper_bound(key).get_offset();
+  }
+
+  NODE_KEY get_key_from_idx(uint16_t idx) const final {
+    return this->iter_idx(idx).get_key();
+  }
 
   fixed_kv_node_meta_t<NODE_KEY> get_node_meta() const {
     return this->get_meta();
   }
 
+  uint16_t get_node_size() const final {
+    return this->get_size();
+  }
+
   typename node_layout_t::delta_buffer_t delta_buffer;
   typename node_layout_t::delta_buffer_t *maybe_get_delta_buffer() {
     return this->is_mutation_pending() 
@@ -114,9 +608,30 @@ struct FixedKVInternalNode
     return CachedExtentRef(new node_type_t(*this));
   };
 
+  void on_replace_prior(Transaction&) final {
+    ceph_assert(!this->is_rewrite());
+    this->set_children_from_prior_instance();
+    auto &prior = (this_type_t&)(*this->get_prior_instance());
+    auto copied = this->copy_children_from_stable_source(
+      prior,
+      prior.begin(),
+      prior.end(),
+      this->begin());
+    ceph_assert(copied <= get_node_size());
+    assert(this->validate_stable_children());
+    this->set_parent_tracker_from_prior_instance();
+  }
+
   void update(
     internal_const_iterator_t iter,
-    paddr_t addr) {
+    paddr_t addr,
+    FixedKVNode<NODE_KEY>* nextent) {
+    LOG_PREFIX(FixedKVInternalNode::update);
+    SUBTRACE(seastore_fixedkv_tree, "trans.{}, pos {}, {}",
+      this->pending_for_transaction,
+      iter.get_offset(),
+      *nextent);
+    this->update_child_ptr(iter, nextent);
     return this->journal_update(
       iter,
       this->maybe_generate_relative(addr),
@@ -126,7 +641,15 @@ struct FixedKVInternalNode
   void insert(
     internal_const_iterator_t iter,
     NODE_KEY pivot,
-    paddr_t addr) {
+    paddr_t addr,
+    FixedKVNode<NODE_KEY>* nextent) {
+    LOG_PREFIX(FixedKVInternalNode::insert);
+    SUBTRACE(seastore_fixedkv_tree, "trans.{}, pos {}, key {}, {}",
+      this->pending_for_transaction,
+      iter.get_offset(),
+      pivot,
+      *nextent);
+    this->insert_child_ptr(iter, nextent);
     return this->journal_insert(
       iter,
       pivot,
@@ -135,6 +658,12 @@ struct FixedKVInternalNode
   }
 
   void remove(internal_const_iterator_t iter) {
+    LOG_PREFIX(FixedKVInternalNode::remove);
+    SUBTRACE(seastore_fixedkv_tree, "trans.{}, pos {}, key {}",
+      this->pending_for_transaction,
+      iter.get_offset(),
+      iter.get_key());
+    this->remove_child_ptr(iter);
     return this->journal_remove(
       iter,
       maybe_get_delta_buffer());
@@ -143,7 +672,16 @@ struct FixedKVInternalNode
   void replace(
     internal_const_iterator_t iter,
     NODE_KEY pivot,
-    paddr_t addr) {
+    paddr_t addr,
+    FixedKVNode<NODE_KEY>* nextent) {
+    LOG_PREFIX(FixedKVInternalNode::replace);
+    SUBTRACE(seastore_fixedkv_tree, "trans.{}, pos {}, old key {}, key {}, {}",
+      this->pending_for_transaction,
+      iter.get_offset(),
+      iter.get_key(),
+      pivot,
+      *nextent);
+    this->update_child_ptr(iter, nextent);
     return this->journal_replace(
       iter,
       pivot,
@@ -157,6 +695,7 @@ struct FixedKVInternalNode
       c.trans, node_size, placement_hint_t::HOT, INIT_GENERATION);
     auto right = c.cache.template alloc_new_extent<node_type_t>(
       c.trans, node_size, placement_hint_t::HOT, INIT_GENERATION);
+    this->split_child_ptrs(*left, *right);
     auto pivot = this->split_into(*left, *right);
     left->pin.set_range(left->get_meta());
     right->pin.set_range(right->get_meta());
@@ -171,6 +710,7 @@ struct FixedKVInternalNode
     Ref &right) {
     auto replacement = c.cache.template alloc_new_extent<node_type_t>(
       c.trans, node_size, placement_hint_t::HOT, INIT_GENERATION);
+    replacement->merge_child_ptrs(*this, *right);
     replacement->merge_from(*this, *right->template cast<node_type_t>());
     replacement->pin.set_range(replacement->get_meta());
     return replacement;
@@ -194,6 +734,12 @@ struct FixedKVInternalNode
       prefer_left,
       *replacement_left,
       *replacement_right);
+    this->balance_child_ptrs(
+      *this,
+      right,
+      prefer_left,
+      *replacement_left,
+      *replacement_right);
 
     replacement_left->pin.set_range(replacement_left->get_meta());
     replacement_right->pin.set_range(replacement_right->get_meta());
@@ -251,8 +797,17 @@ struct FixedKVInternalNode
 
   std::ostream &print_detail(std::ostream &out) const
   {
-    return out << ", size=" << this->get_size()
-              << ", meta=" << this->get_meta();
+    out << ", size=" << this->get_size()
+       << ", meta=" << this->get_meta()
+       << ", parent_tracker=" << (void*)this->parent_tracker.get();
+    if (this->parent_tracker) {
+      out << ", parent=" << (void*)this->parent_tracker->parent.get();
+    }
+    out << ", my_tracker=" << (void*)this->my_tracker;
+    if (this->my_tracker) {
+      out << ", my_tracker->parent=" << (void*)this->my_tracker->parent.get();
+    }
+    return out;
   }
 
   ceph::bufferlist get_delta() {
@@ -322,17 +877,60 @@ struct FixedKVLeafNode
       VAL,
       VAL_LE>;
   using internal_const_iterator_t = typename node_layout_t::const_iterator;
-  template <typename... T>
-  FixedKVLeafNode(T&&... t) :
-    FixedKVNode<NODE_KEY>(std::forward<T>(t)...),
-    node_layout_t(this->get_bptr().c_str()) {}
+  FixedKVLeafNode(ceph::bufferptr &&ptr)
+    : FixedKVNode<NODE_KEY>(0, std::move(ptr)),
+      node_layout_t(this->get_bptr().c_str()) {}
+  FixedKVLeafNode(const FixedKVLeafNode &rhs)
+    : FixedKVNode<NODE_KEY>(rhs),
+      node_layout_t(this->get_bptr().c_str()) {}
 
-  virtual ~FixedKVLeafNode() {}
+  uint16_t get_node_split_pivot() final {
+    return this->get_split_pivot().get_offset();
+  }
+
+  bool validate_stable_children() final {
+    return true;
+  }
+
+  virtual ~FixedKVLeafNode() {
+    if (!this->pin.is_root()
+       && this->is_valid()
+       && !this->is_pending()) {
+      ceph_assert(this->parent_tracker);
+      auto &parent = this->parent_tracker->parent;
+      ceph_assert(parent);
+      auto off = parent->lower_bound_offset(this->get_meta().begin);
+      assert(parent->get_key_from_idx(off) == get_node_meta().begin);
+      assert(parent->children[off] == this);
+      parent->children[off] = nullptr;
+    }
+  }
+
+  void on_replace_prior(Transaction &t) final {
+    this->set_parent_tracker();
+    assert(this->mutate_state.empty());
+  }
+
+  uint16_t lower_bound_offset(NODE_KEY key) const final {
+    return this->lower_bound(key).get_offset();
+  }
+
+  uint16_t upper_bound_offset(NODE_KEY key) const final {
+    return this->upper_bound(key).get_offset();
+  }
+
+  NODE_KEY get_key_from_idx(uint16_t idx) const final {
+    return this->iter_idx(idx).get_key();
+  }
 
   fixed_kv_node_meta_t<NODE_KEY> get_node_meta() const {
     return this->get_meta();
   }
 
+  uint16_t get_node_size() const final {
+    return this->get_size();
+  }
+
   typename node_layout_t::delta_buffer_t delta_buffer;
   virtual typename node_layout_t::delta_buffer_t *maybe_get_delta_buffer() {
     return this->is_mutation_pending() ? &delta_buffer : nullptr;
@@ -426,8 +1024,13 @@ struct FixedKVLeafNode
 
   std::ostream &print_detail(std::ostream &out) const
   {
-    return out << ", size=" << this->get_size()
-              << ", meta=" << this->get_meta();
+    out << ", size=" << this->get_size()
+       << ", meta=" << this->get_meta()
+       << ", parent_tracker=" << (void*)this->parent_tracker.get();
+    if (this->parent_tracker) {
+      out << ", parent=" << (void*)this->parent_tracker->parent.get();
+    }
+    return out;
   }
 
   constexpr static size_t get_min_capacity() {
@@ -451,3 +1054,14 @@ struct FixedKVLeafNode
 };
 
 } // namespace crimson::os::seastore
+
+#if FMT_VERSION >= 90000
+template <>
+struct fmt::formatter<
+  crimson::os::seastore::FixedKVNode<
+    crimson::os::seastore::laddr_t>> : fmt::ostream_formatter {};
+template <>
+struct fmt::formatter<
+  crimson::os::seastore::FixedKVNode<
+    crimson::os::seastore::paddr_t>> : fmt::ostream_formatter {};
+#endif
index f0c02b08b12a22f74963c84f219c602b4400ec12..c41d29b941c6f362345e678f4c19a97862db51db 100644 (file)
@@ -1018,6 +1018,8 @@ CachedExtentRef Cache::duplicate_for_write(
   ret->prior_instance = i;
   // duplicate_for_write won't occur after ool write finished
   assert(!i->prior_poffset);
+  auto [iter, inserted] = i->mutation_pendings.insert(*ret);
+  ceph_assert(inserted);
   t.add_mutated_extent(ret);
   if (ret->get_type() == extent_types_t::ROOT) {
     t.root = ret->cast<RootBlock>();
index c84475344517d0d60cb5de8da58902d124315d7a..f1ea69d633260ee5486e626cf04ef24b72750810 100644 (file)
@@ -79,6 +79,17 @@ CachedExtent::~CachedExtent()
   }
 }
 
+CachedExtent* CachedExtent::get_transactional_view(Transaction &t) {
+  auto it = mutation_pendings.find(
+    t.get_trans_id(),
+    trans_spec_view_t::cmp_t());
+  if (it != mutation_pendings.end()) {
+    return (CachedExtent*)&(*it);
+  } else {
+    return this;
+  }
+}
+
 std::ostream &LogicalCachedExtent::print_detail(std::ostream &out) const
 {
   out << ", laddr=" << laddr;
index 4603d5a2cd8c38b446c1d428b3a056bdf055f29a..ea1a749b7f23a50a6bfe436e96610070050e3213 100644 (file)
@@ -501,6 +501,10 @@ public:
 
   void set_invalid(Transaction &t);
 
+  CachedExtentRef get_prior_instance() {
+    return prior_instance;
+  }
+
 private:
   template <typename T>
   friend class read_set_item_t;
@@ -585,6 +589,8 @@ private:
   rewrite_gen_t rewrite_generation = NULL_GENERATION;
 
 protected:
+  trans_view_set_t mutation_pendings;
+
   CachedExtent(CachedExtent &&other) = delete;
   CachedExtent(ceph::bufferptr &&ptr) : ptr(std::move(ptr)) {}
   CachedExtent(const CachedExtent &other)
@@ -605,6 +611,8 @@ protected:
   struct retired_placeholder_t{};
   CachedExtent(retired_placeholder_t) : state(extent_state_t::INVALID) {}
 
+  CachedExtent& get_transactional_view(Transaction &t);
+
   friend class Cache;
   template <typename T, typename... Args>
   static TCachedExtentRef<T> make_cached_extent_ref(
@@ -612,8 +620,8 @@ protected:
     return new T(std::forward<Args>(args)...);
   }
 
-  CachedExtentRef get_prior_instance() {
-    return prior_instance;
+  void reset_prior_instance() {
+    prior_instance.reset();
   }
 
   /// Sets last_committed_crc
index e3e69421fc641d27711f7548db7caa32b09ba79b..e3eed252ebe4303915cd06ec2c9642aa7d9527a1 100644 (file)
@@ -29,8 +29,13 @@ std::ostream& operator<<(std::ostream& out, const lba_map_val_t& v)
 
 std::ostream &LBALeafNode::print_detail(std::ostream &out) const
 {
-  return out << ", size=" << get_size()
-            << ", meta=" << get_meta();
+  out << ", size=" << get_size()
+      << ", meta=" << get_meta()
+      << ", parent_tracker=" << (void*)parent_tracker.get();
+  if (parent_tracker) {
+    return out << ", parent=" << (void*)parent_tracker->parent.get();
+  }
+  return out;
 }
 
 void LBALeafNode::resolve_relative_addrs(paddr_t base)
index f39d47735441d2ef59b073c60153aba6927292dc..571e906fa8fedaf2460dc78b0450f9fcfb536303 100644 (file)
@@ -219,6 +219,7 @@ using LBALeafNodeRef = TCachedExtentRef<LBALeafNode>;
 }
 
 #if FMT_VERSION >= 90000
+template <> struct fmt::formatter<crimson::os::seastore::lba_manager::btree::lba_node_meta_t> : fmt::ostream_formatter {};
 template <> struct fmt::formatter<crimson::os::seastore::lba_manager::btree::lba_map_val_t> : fmt::ostream_formatter {};
 template <> struct fmt::formatter<crimson::os::seastore::lba_manager::btree::LBAInternalNode> : fmt::ostream_formatter {};
 template <> struct fmt::formatter<crimson::os::seastore::lba_manager::btree::LBALeafNode> : fmt::ostream_formatter {};