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 ©_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 {
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,
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()
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),
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,
}
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());
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,
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());
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;
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());
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() {
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;
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() {
};
} // 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