diff --git a/src/crimson/os/seastore/btree/fixed_kv_btree.h b/src/crimson/os/seastore/btree/fixed_kv_btree.h
index 9e5726c1293..5000b2e94de 100644
--- a/src/crimson/os/seastore/btree/fixed_kv_btree.h
+++ b/src/crimson/os/seastore/btree/fixed_kv_btree.h
@@ -505,6 +505,7 @@ public:
       Transaction::get_extent_ret ret;
 
       if constexpr (std::is_base_of_v<typename internal_node_t::base_t, child_node_t>) {
+        assert(i->get_val() != P_ADDR_ZERO);
         ret = c.trans.get_extent(
           i->get_val().maybe_relative_to(node->get_paddr()),
           &child_node);
@@ -515,6 +516,9 @@ public:
         ret = c.trans.get_extent(
           i->get_val().pladdr.get_paddr().maybe_relative_to(node->get_paddr()),
           &child_node);
+        if (i->get_val().pladdr.get_paddr() == P_ADDR_ZERO) {
+          assert(ret == Transaction::get_extent_ret::ABSENT);
+        }
       }
       if (ret == Transaction::get_extent_ret::PRESENT) {
         if (child_node->is_stable_written()) {
@@ -533,7 +537,7 @@ public:
           }
         } else if (child_node->is_pending()) {
           if (child_node->is_mutation_pending()) {
-            auto &prior = (child_node_t &)*child_node->prior_instance;
+            auto &prior = (child_node_t &)*child_node->get_prior_instance();
             assert(prior.is_valid());
             assert(prior.is_parent_valid());
             if (node->is_mutation_pending()) {
@@ -565,18 +569,8 @@ public:
           auto pos = n.lower_bound_offset(i->get_key());
           assert(pos < n.get_node_size());
           child = n.children[pos];
-          if (is_valid_child_ptr(child)) {
-            auto c = (child_node_t*)child;
-            assert(c->has_parent_tracker());
-            assert(c->get_parent_node().get() == &n);
-          }
         } else {
           child = node->children[i->get_offset()];
-          if (is_valid_child_ptr(child)) {
-            auto c = (child_node_t*)child;
-            assert(c->has_parent_tracker());
-            assert(c->get_parent_node().get() == node.get());
-          }
         }
 
         if (!is_valid_child_ptr(child)) {
@@ -592,6 +586,23 @@ public:
                 : true);
             }
           }
+          if (child == get_reserved_ptr()) {
+            if constexpr(
+              !std::is_base_of_v<typename internal_node_t::base_t,
+                                 child_node_t>) {
+              assert(i->get_val().pladdr.is_paddr());
+              assert(i->get_val().pladdr.get_paddr() == P_ADDR_ZERO);
+            } else {
+              ceph_abort();
+            }
+          }
+        } else {
+          auto c = (child_node_t*)child;
+          assert(c->has_parent_tracker());
+          assert(c->get_parent_node().get() == node.get()
+            || (node->is_pending() && c->is_stable()
+                && c->get_parent_node().get() == &node->get_stable_for_key(
+                  i->get_key())));
         }
       } else {
         ceph_abort("impossible");
@@ -1034,46 +1045,7 @@ public:
         fixed_kv_extent.get_user_hint(),
         // get target rewrite generation
         fixed_kv_extent.get_rewrite_generation());
-      fixed_kv_extent.get_bptr().copy_out(
-        0,
-        fixed_kv_extent.get_length(),
-        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->range = n_fixed_kv_extent->get_node_meta();
-      n_fixed_kv_extent->set_last_committed_crc(fixed_kv_extent.get_last_committed_crc());
-
-      if (fixed_kv_extent.get_type() == internal_node_t::TYPE ||
-          leaf_node_t::do_has_children) {
-        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
-       * are using resolve_relative_addrs with a (likely negative) block
-       * relative offset to correct them to block-relative offsets adjusted
-       * for our new transaction location.
-       *
-       * Upon commit, these now block relative addresses will be interpretted
-       * against the real final address.
-       */
-      if (!n_fixed_kv_extent->get_paddr().is_absolute()) {
-	// backend_type_t::SEGMENTED
-	assert(n_fixed_kv_extent->get_paddr().is_record_relative());
-	n_fixed_kv_extent->resolve_relative_addrs(
-	  make_record_relative_paddr(0).block_relative_to(
-	    n_fixed_kv_extent->get_paddr()));
-      } // else: backend_type_t::RANDOM_BLOCK
+      n_fixed_kv_extent->rewrite(fixed_kv_extent, 0);
       
       SUBTRACET(
         seastore_fixedkv_tree,
diff --git a/src/crimson/os/seastore/btree/fixed_kv_node.h b/src/crimson/os/seastore/btree/fixed_kv_node.h
index 14a1703abbc..79495cb35d1 100644
--- a/src/crimson/os/seastore/btree/fixed_kv_node.h
+++ b/src/crimson/os/seastore/btree/fixed_kv_node.h
@@ -59,7 +59,7 @@ struct FixedKVNode : ChildableCachedExtent {
    * 	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
+   * 	d. 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
@@ -67,7 +67,7 @@ struct FixedKVNode : ChildableCachedExtent {
    * 	   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
+   * 	d. 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
@@ -80,6 +80,8 @@ struct FixedKVNode : ChildableCachedExtent {
    * 	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.
+   * 4. EXIST_CLEAN and EXIST_MUTATION_PENDING belong to 3 above (except that they
+   * 	cannot be rewritten) because their parents must be mutated upon remapping.
    */
   std::vector<ChildableCachedExtent*> children;
   std::set<FixedKVNodeRef, copy_source_cmp_t> copy_sources;
@@ -157,6 +159,43 @@ struct FixedKVNode : ChildableCachedExtent {
       (get_node_size() - offset - 1) * sizeof(ChildableCachedExtent*));
   }
 
+  virtual bool have_children() const = 0;
+
+  void on_rewrite(CachedExtent &extent, extent_len_t off) final {
+    assert(get_type() == extent.get_type());
+    assert(off == 0);
+    auto &foreign_extent = (FixedKVNode&)extent;
+    range = get_node_meta();
+
+    if (have_children()) {
+      if (!foreign_extent.is_pending()) {
+	copy_sources.emplace(&foreign_extent);
+      } else {
+	ceph_assert(foreign_extent.is_mutation_pending());
+	copy_sources.emplace(
+	  foreign_extent.get_prior_instance()->template cast<FixedKVNode>());
+	children = std::move(foreign_extent.children);
+	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
+     * are using resolve_relative_addrs with a (likely negative) block
+     * relative offset to correct them to block-relative offsets adjusted
+     * for our new transaction location.
+     *
+     * Upon commit, these now block relative addresses will be interpretted
+     * against the real final address.
+     */
+    if (!get_paddr().is_absolute()) {
+      // backend_type_t::SEGMENTED
+      assert(get_paddr().is_record_relative());
+      resolve_relative_addrs(
+	make_record_relative_paddr(0).block_relative_to(get_paddr()));
+    } // else: backend_type_t::RANDOM_BLOCK
+  }
+
   FixedKVNode& get_stable_for_key(node_key_t key) const {
     ceph_assert(is_pending());
     if (is_mutation_pending()) {
@@ -242,7 +281,7 @@ struct FixedKVNode : ChildableCachedExtent {
       return c.cache.template get_extent_viewable_by_trans<T>(c.trans, (T*)child);
     } else if (is_pending()) {
       auto &sparent = get_stable_for_key(key);
-      auto spos = sparent.child_pos_for_key(key);
+      auto spos = sparent.lower_bound_offset(key);
       auto child = sparent.children[spos];
       if (is_valid_child_ptr(child)) {
 	return c.cache.template get_extent_viewable_by_trans<T>(c.trans, (T*)child);
@@ -415,7 +454,6 @@ struct FixedKVNode : ChildableCachedExtent {
 
   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;
 
@@ -488,10 +526,6 @@ struct FixedKVNode : ChildableCachedExtent {
     reset_parent_tracker();
   }
 
-  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());
@@ -564,6 +598,10 @@ struct FixedKVInternalNode
     : FixedKVNode<NODE_KEY>(rhs),
       node_layout_t(this->get_bptr().c_str()) {}
 
+  bool have_children() const final {
+    return true;
+  }
+
   bool is_leaf_and_has_children() const final {
     return false;
   }
@@ -648,13 +686,6 @@ struct FixedKVInternalNode
     return this->upper_bound(key).get_offset();
   }
 
-  uint16_t child_pos_for_key(NODE_KEY key) const final {
-    auto it = this->upper_bound(key);
-    assert(it != this->begin());
-    --it;
-    return it.get_offset();
-  }
-
   NODE_KEY get_key_from_idx(uint16_t idx) const final {
     return this->iter_idx(idx).get_key();
   }
@@ -690,7 +721,7 @@ struct FixedKVInternalNode
     return CachedExtentRef(new node_type_t(*this));
   };
 
-  void on_replace_prior(Transaction&) final {
+  void on_replace_prior() final {
     ceph_assert(!this->is_rewrite());
     this->set_children_from_prior_instance();
     auto &prior = (this_type_t&)(*this->get_prior_instance());
@@ -977,6 +1008,10 @@ struct FixedKVLeafNode
 
   static constexpr bool do_has_children = has_children;
 
+  bool have_children() const final {
+    return do_has_children;
+  }
+
   bool is_leaf_and_has_children() const final {
     return has_children;
   }
@@ -1014,7 +1049,7 @@ struct FixedKVLeafNode
     } else if (this->is_pending()) {
       auto key = this->iter_idx(pos).get_key();
       auto &sparent = this->get_stable_for_key(key);
-      auto spos = sparent.child_pos_for_key(key);
+      auto spos = sparent.lower_bound_offset(key);
       auto child = sparent.children[spos];
       if (is_valid_child_ptr(child)) {
 	ceph_assert(child->is_logical());
@@ -1078,7 +1113,7 @@ struct FixedKVLeafNode
       true);
   }
 
-  void on_replace_prior(Transaction&) final {
+  void on_replace_prior() final {
     ceph_assert(!this->is_rewrite());
     if constexpr (has_children) {
       this->set_children_from_prior_instance();
@@ -1104,10 +1139,6 @@ struct FixedKVLeafNode
     return this->upper_bound(key).get_offset();
   }
 
-  uint16_t child_pos_for_key(NODE_KEY key) const final {
-    return lower_bound_offset(key);
-  }
-
   NODE_KEY get_key_from_idx(uint16_t idx) const final {
     return this->iter_idx(idx).get_key();
   }
diff --git a/src/crimson/os/seastore/cache.cc b/src/crimson/os/seastore/cache.cc
index 2d08a4ddf45..f85bafb4c74 100644
--- a/src/crimson/os/seastore/cache.cc
+++ b/src/crimson/os/seastore/cache.cc
@@ -829,7 +829,7 @@ void Cache::commit_replace_extent(
     add_to_dirty(next);
   }
 
-  next->on_replace_prior(t);
+  next->on_replace_prior();
   invalidate_extent(t, *prev);
 }
 
@@ -1566,6 +1566,7 @@ void Cache::complete_commit(
     i->on_initial_write();
 
     i->state = CachedExtent::extent_state_t::CLEAN;
+    i->prior_instance.reset();
     DEBUGT("add extent as fresh, inline={} -- {}",
 	   t, is_inline, *i);
     const auto t_src = t.get_src();
diff --git a/src/crimson/os/seastore/cached_extent.cc b/src/crimson/os/seastore/cached_extent.cc
index 37884227186..cdad6dfb1b0 100644
--- a/src/crimson/os/seastore/cached_extent.cc
+++ b/src/crimson/os/seastore/cached_extent.cc
@@ -137,7 +137,7 @@ LogicalCachedExtent::~LogicalCachedExtent() {
   }
 }
 
-void LogicalCachedExtent::on_replace_prior(Transaction &t) {
+void LogicalCachedExtent::on_replace_prior() {
   assert(is_mutation_pending());
   take_prior_parent_tracker();
   assert(get_parent_node());
diff --git a/src/crimson/os/seastore/cached_extent.h b/src/crimson/os/seastore/cached_extent.h
index 942be723231..4778117c8a6 100644
--- a/src/crimson/os/seastore/cached_extent.h
+++ b/src/crimson/os/seastore/cached_extent.h
@@ -30,15 +30,6 @@ class SegmentedAllocator;
 class TransactionManager;
 class ExtentPlacementManager;
 
-template <
-  typename node_key_t,
-  typename node_val_t,
-  typename internal_node_t,
-  typename leaf_node_t,
-  typename pin_t,
-  size_t node_size,
-  bool leaf_has_children>
-class FixedKVBtree;
 template <typename, typename>
 class BtreeNodeMapping;
 
@@ -192,15 +183,6 @@ class CachedExtent
   friend class onode::DummyNodeExtent;
   friend class onode::TestReplayExtent;
 
-  template <
-    typename node_key_t,
-    typename node_val_t,
-    typename internal_node_t,
-    typename leaf_node_t,
-    typename pin_t,
-    size_t node_size,
-    bool leaf_has_children>
-  friend class FixedKVBtree;
   uint32_t last_committed_crc = 0;
 
   // Points at current version while in state MUTATION_PENDING
@@ -299,7 +281,7 @@ public:
    * with the states of Cache and can't wait till transaction
    * completes.
    */
-  virtual void on_replace_prior(Transaction &t) {}
+  virtual void on_replace_prior() {}
 
   /**
    * on_invalidated
@@ -325,6 +307,31 @@ public:
     return true;
   }
 
+  void rewrite(CachedExtent &e, extent_len_t o) {
+    assert(is_initial_pending());
+    if (!e.is_pending()) {
+      prior_instance = &e;
+    } else {
+      assert(e.is_mutation_pending());
+      prior_instance = e.get_prior_instance();
+    }
+    e.get_bptr().copy_out(
+      o,
+      get_length(),
+      get_bptr().c_str());
+    set_modify_time(e.get_modify_time());
+    set_last_committed_crc(e.get_last_committed_crc());
+    on_rewrite(e, o);
+  }
+
+  /**
+   * on_rewrite
+   *
+   * Called when this extent is rewriting another one.
+   *
+   */
+  virtual void on_rewrite(CachedExtent &, extent_len_t) = 0;
+
   friend std::ostream &operator<<(std::ostream &, extent_state_t);
   virtual std::ostream &print_detail(std::ostream &out) const { return out; }
   std::ostream &print(std::ostream &out) const {
@@ -419,6 +426,10 @@ public:
     return is_mutable() || state == extent_state_t::EXIST_CLEAN;
   }
 
+  bool is_rewrite() {
+    return is_initial_pending() && get_prior_instance();
+  }
+
   /// Returns true if extent is stable, written and shared among transactions
   bool is_stable_written() const {
     return state == extent_state_t::CLEAN_PENDING ||
@@ -442,7 +453,8 @@ public:
 
   /// Returns true if extent has a pending delta
   bool is_mutation_pending() const {
-    return state == extent_state_t::MUTATION_PENDING;
+    return state == extent_state_t::MUTATION_PENDING
+      || state == extent_state_t::EXIST_MUTATION_PENDING;
   }
 
   /// Returns true if extent is a fresh extent
@@ -797,6 +809,10 @@ protected:
    */
   virtual void update_in_extent_chksum_field(uint32_t) {}
 
+  void set_prior_instance(CachedExtentRef p) {
+    prior_instance = p;
+  }
+
   /// Sets last_committed_crc
   void set_last_committed_crc(uint32_t crc) {
     last_committed_crc = crc;
@@ -1157,6 +1173,8 @@ public:
     return false;
   }
 
+  void on_rewrite(CachedExtent&, extent_len_t) final {}
+
   std::ostream &print_detail(std::ostream &out) const final {
     return out << ", RetiredExtentPlaceholder";
   }
@@ -1242,6 +1260,12 @@ public:
     : ChildableCachedExtent(std::forward<T>(t)...)
   {}
 
+  void on_rewrite(CachedExtent &extent, extent_len_t off) final {
+    assert(get_type() == extent.get_type());
+    auto &lextent = (LogicalCachedExtent&)extent;
+    set_laddr(lextent.get_laddr() + off);
+  }
+
   bool has_laddr() const {
     return laddr != L_ADDR_NULL;
   }
@@ -1286,7 +1310,7 @@ public:
   virtual ~LogicalCachedExtent();
 
 protected:
-  void on_replace_prior(Transaction &t) final;
+  void on_replace_prior() final;
 
   virtual void apply_delta(const ceph::bufferlist &bl) = 0;
 
diff --git a/src/crimson/os/seastore/lba_manager/btree/lba_btree_node.h b/src/crimson/os/seastore/lba_manager/btree/lba_btree_node.h
index 3b760ff06a2..c5da860e24f 100644
--- a/src/crimson/os/seastore/lba_manager/btree/lba_btree_node.h
+++ b/src/crimson/os/seastore/lba_manager/btree/lba_btree_node.h
@@ -171,6 +171,8 @@ struct LBALeafNode
 
     for (auto i : *this) {
       auto child = (LogicalCachedExtent*)this->children[i.get_offset()];
+      // Children may not be marked as stable yet,
+      // the specific order is undefined in the transaction prepare record phase.
       if (is_valid_child_ptr(child) && child->get_laddr() != i.get_key()) {
 	SUBERROR(seastore_fixedkv_tree,
 	  "stable child not valid: child {}, key {}",
@@ -194,7 +196,11 @@ struct LBALeafNode
 	iter.get_offset(),
 	*nextent);
       // child-ptr may already be correct, see LBAManager::update_mappings()
-      this->update_child_ptr(iter, nextent);
+      if (!nextent->has_parent_tracker()) {
+	this->update_child_ptr(iter, nextent);
+      }
+      assert(nextent->has_parent_tracker()
+	&& nextent->get_parent_node<LBALeafNode>().get() == this);
     }
     if (val.pladdr.is_paddr()) {
       val.pladdr = maybe_generate_relative(val.pladdr.get_paddr());
diff --git a/src/crimson/os/seastore/root_block.cc b/src/crimson/os/seastore/root_block.cc
index 5b8e0ca5d1f..dec6e12ea4e 100644
--- a/src/crimson/os/seastore/root_block.cc
+++ b/src/crimson/os/seastore/root_block.cc
@@ -7,7 +7,7 @@
 
 namespace crimson::os::seastore {
 
-void RootBlock::on_replace_prior(Transaction &t) {
+void RootBlock::on_replace_prior() {
   if (!lba_root_node) {
     auto &prior = static_cast<RootBlock&>(*get_prior_instance());
     if (prior.lba_root_node) {
diff --git a/src/crimson/os/seastore/root_block.h b/src/crimson/os/seastore/root_block.h
index 0e45519ce45..a9a7cd19822 100644
--- a/src/crimson/os/seastore/root_block.h
+++ b/src/crimson/os/seastore/root_block.h
@@ -50,6 +50,8 @@ struct RootBlock : CachedExtent {
       backref_root_node(nullptr)
   {}
 
+  void on_rewrite(CachedExtent&, extent_len_t) final {}
+
   CachedExtentRef duplicate_for_write(Transaction&) final {
     return CachedExtentRef(new RootBlock(*this));
   };
@@ -59,7 +61,7 @@ struct RootBlock : CachedExtent {
     return extent_types_t::ROOT;
   }
 
-  void on_replace_prior(Transaction &t) final;
+  void on_replace_prior() final;
 
   /// dumps root as delta
   ceph::bufferlist get_delta() final {
diff --git a/src/crimson/os/seastore/transaction_manager.cc b/src/crimson/os/seastore/transaction_manager.cc
index 6f8fb7b8436..763aae38dd4 100644
--- a/src/crimson/os/seastore/transaction_manager.cc
+++ b/src/crimson/os/seastore/transaction_manager.cc
@@ -494,12 +494,7 @@ TransactionManager::rewrite_logical_extent(
       lextent->get_user_hint(),
       // get target rewrite generation
       lextent->get_rewrite_generation())->cast<LogicalCachedExtent>();
-    lextent->get_bptr().copy_out(
-      0,
-      lextent->get_length(),
-      nlextent->get_bptr().c_str());
-    nlextent->set_laddr(lextent->get_laddr());
-    nlextent->set_modify_time(lextent->get_modify_time());
+    nlextent->rewrite(*lextent, 0);
 
     DEBUGT("rewriting logical extent -- {} to {}", t, *lextent, *nlextent);
 
@@ -540,13 +535,7 @@ TransactionManager::rewrite_logical_extent(
         bool first_extent = (off == 0);
         ceph_assert(left >= nextent->get_length());
         auto nlextent = nextent->template cast<LogicalCachedExtent>();
-        lextent->get_bptr().copy_out(
-          off,
-          nlextent->get_length(),
-          nlextent->get_bptr().c_str());
-        nlextent->set_laddr(lextent->get_laddr() + off);
-        nlextent->set_modify_time(lextent->get_modify_time());
-        nlextent->set_last_committed_crc(lextent->get_last_committed_crc());
+        nlextent->rewrite(*lextent, off);
         DEBUGT("rewriting logical extent -- {} to {}", t, *lextent, *nlextent);
 
         /* This update_mapping is, strictly speaking, unnecessary for delayed_alloc
diff --git a/src/test/crimson/seastore/test_block.h b/src/test/crimson/seastore/test_block.h
index 7073b8a1904..76891076f33 100644
--- a/src/test/crimson/seastore/test_block.h
+++ b/src/test/crimson/seastore/test_block.h
@@ -111,6 +111,8 @@ struct TestBlockPhysical : crimson::os::seastore::CachedExtent{
 
   std::vector<test_block_delta_t> delta = {};
 
+  void on_rewrite(CachedExtent&, extent_len_t) final {}
+
   TestBlockPhysical(ceph::bufferptr &&ptr)
     : CachedExtent(std::move(ptr)) {}
   TestBlockPhysical(const TestBlockPhysical &other)