Node::~Node()
{
// XXX: tolerate failure between allocate() and as_child()
- if (is_root()) {
- super->do_untrack_root(*this);
+ if (!super && !_parent_info.has_value()) {
+ // To erase a node:
+ // 1. I'm not tracking any children or cursors
+ // 2. unlink parent/super --ptr-> me
+ // 3. unlink me --ref-> parent/super
+ // 4. extent is retired
+ assert(!impl->is_extent_valid());
+
+ // TODO: maybe its possible when eagain happens internally, we should
+ // revisit to make sure tree operations can be aborted normally,
+ // without resource leak or hitting unexpected asserts.
} else {
- _parent_info->ptr->do_untrack_child(*this);
+ assert(impl->is_extent_valid());
+ if (is_root()) {
+ super->do_untrack_root(*this);
+ } else {
+ _parent_info->ptr->do_untrack_child(*this);
+ }
}
}
template void Node::as_child<false>(const search_position_t&, Ref<InternalNode>);
node_future<> Node::apply_split_to_parent(
- context_t c, Ref<Node> split_right)
+ context_t c, Ref<Node> split_right, bool update_right_index)
{
assert(!is_root());
// TODO(cross-node string dedup)
return parent_info().ptr->apply_child_split(
- c, this, split_right);
+ c, this, split_right, update_right_index);
}
node_future<Ref<tree_cursor_t>>
return parent_info().ptr->get_next_cursor(c, parent_info().position);
}
+node_future<>
+Node::try_merge_adjacent(context_t c, bool update_parent_index)
+{
+ impl->validate_non_empty();
+ assert(!is_root());
+ Ref<Node> this_ref = this;
+ if (!impl->is_size_underflow()) {
+ if (update_parent_index) {
+ return fix_parent_index(c);
+ } else {
+ parent_info().ptr->validate_child_tracked(*this);
+ return node_ertr::now();
+ }
+ }
+
+ return parent_info().ptr->get_child_peers(c, parent_info().position
+ ).safe_then([c, this_ref = std::move(this_ref), this,
+ update_parent_index] (auto lr_nodes) mutable -> node_future<> {
+ auto& [lnode, rnode] = lr_nodes;
+ Ref<Node> left_for_merge;
+ Ref<Node> right_for_merge;
+ bool is_left;
+ if (!lnode && !rnode) {
+ // XXX: this is possible before node rebalance is implemented,
+ // when its parent cannot merge with its peers and has only one child
+ // (this node).
+ } else if (!lnode) {
+ left_for_merge = std::move(this_ref);
+ right_for_merge = std::move(rnode);
+ is_left = true;
+ } else if (!rnode) {
+ left_for_merge = std::move(lnode);
+ right_for_merge = std::move(this_ref);
+ is_left = false;
+ } else { // lnode && rnode
+ if (lnode->impl->free_size() > rnode->impl->free_size()) {
+ left_for_merge = std::move(lnode);
+ right_for_merge = std::move(this_ref);
+ is_left = false;
+ } else { // lnode free size <= rnode free size
+ left_for_merge = std::move(this_ref);
+ right_for_merge = std::move(rnode);
+ is_left = true;
+ }
+ }
+
+ if (left_for_merge) {
+ assert(right_for_merge);
+ auto [merge_stage, merge_size] = left_for_merge->impl->evaluate_merge(
+ *right_for_merge->impl);
+ if (merge_size <= left_for_merge->impl->total_size()) {
+ // proceed merge
+ bool update_index_after_merge;
+ if (is_left) {
+ update_index_after_merge = false;
+ } else {
+ update_index_after_merge = update_parent_index;
+ }
+ logger().info("OTree::Node::MergeAdjacent: merge {} and {} "
+ "at merge_stage={}, merge_size={}B, update_index={} ...",
+ left_for_merge->get_name(), right_for_merge->get_name(),
+ merge_stage, merge_size, update_index_after_merge);
+ // we currently cannot generate delta depends on another extent content,
+ // so use rebuild_extent() as a workaround to rebuild the node from a
+ // fresh extent, thus no need to generate delta.
+ return left_for_merge->rebuild_extent(c
+ ).safe_then([c, merge_stage, merge_size, update_index_after_merge,
+ left_for_merge = std::move(left_for_merge),
+ right_for_merge = std::move(right_for_merge)] (auto left_mut) mutable {
+ if (left_for_merge->impl->node_type() == node_type_t::LEAF) {
+ auto& left = *static_cast<LeafNode*>(left_for_merge.get());
+ left.on_layout_change();
+ }
+ search_position_t left_last_pos = left_for_merge->impl->merge(
+ left_mut, *right_for_merge->impl, merge_stage, merge_size);
+ left_for_merge->track_merge(right_for_merge, merge_stage, left_last_pos);
+ return left_for_merge->parent_info().ptr->apply_children_merge(
+ c, std::move(left_for_merge),
+ std::move(right_for_merge), update_index_after_merge);
+ });
+ } else {
+ // size will overflow if merge
+ }
+ }
+
+ // cannot merge
+ if (update_parent_index) {
+ return fix_parent_index(c);
+ } else {
+ parent_info().ptr->validate_child_tracked(*this);
+ return node_ertr::now();
+ }
+ // XXX: rebalance
+ });
+}
+
+node_future<> Node::erase_node(context_t c, Ref<Node>&& this_ref)
+{
+ assert(this_ref.get() == this);
+ assert(!is_tracking());
+ assert(!is_root());
+ assert(this_ref->use_count() == 1);
+ return parent_info().ptr->erase_child(c, std::move(this_ref));
+}
+
+node_future<> Node::fix_parent_index(context_t c)
+{
+ assert(!is_root());
+ auto& parent = parent_info().ptr;
+ // one-way unlink
+ parent->do_untrack_child(*this);
+ // the rest of parent tracks should be correct
+ parent->validate_tracked_children();
+ return parent->fix_index(c, this);
+}
+
node_future<Ref<Node>> Node::load(
context_t c, laddr_t addr, bool expect_is_level_tail)
{
});
}
+node_future<NodeExtentMutable> Node::rebuild_extent(context_t c)
+{
+ logger().debug("OTree::Node::Rebuild: {} ...", get_name());
+ assert(!is_root());
+ // assume I'm already ref counted by caller
+
+ // note: laddr can be changed after rebuild, but we don't fix the parent
+ // mapping as it is part of the merge process.
+ return impl->rebuild_extent(c);
+}
+
+node_future<> Node::retire(context_t c, Ref<Node>&& this_ref)
+{
+ logger().debug("OTree::Node::Retire: {} ...", get_name());
+ assert(this_ref.get() == this);
+ assert(!is_tracking());
+ assert(!super);
+ // make sure the parent also has untracked this node
+ assert(!_parent_info.has_value());
+ assert(this_ref->use_count() == 1);
+
+ return impl->retire_extent(c
+ ).safe_then([this_ref = std::move(this_ref)]{ /* deallocate node */});
+}
+
+void Node::make_tail(context_t c)
+{
+ assert(!impl->is_level_tail());
+ assert(!impl->is_keys_empty());
+ logger().debug("OTree::Node::MakeTail: {} ...", get_name());
+ impl->prepare_mutate(c);
+ auto tail_pos = impl->make_tail();
+ if (impl->node_type() == node_type_t::INTERNAL) {
+ auto& node = *static_cast<InternalNode*>(this);
+ node.track_make_tail(tail_pos);
+ }
+}
+
/*
* InternalNode
*/
}
node_future<> InternalNode::apply_child_split(
- context_t c, Ref<Node> left_child, Ref<Node> right_child)
+ context_t c, Ref<Node> left_child, Ref<Node> right_child,
+ bool update_right_index)
{
auto& left_pos = left_child->parent_info().position;
if (left_pos.is_end()) {
assert(impl->is_level_tail());
assert(right_child->impl->is_level_tail());
+ assert(!update_right_index);
}
#endif
impl->prepare_mutate(c);
- auto left_key = *left_child->impl->get_pivot_index();
+ logger().debug("OTree::Internal::ApplyChildSplit: apply {}'s child "
+ "{} to split to {}, update_index={} ...",
+ get_name(), left_child->get_name(),
+ right_child->get_name(), update_right_index);
+
+ // update layout from left_pos => left_child_addr to right_child_addr
auto left_child_addr = left_child->impl->laddr();
- auto op_right_key = right_child->impl->get_pivot_index();
auto right_child_addr = right_child->impl->laddr();
- if (op_right_key.has_value()) {
- logger().debug("OTree::Internal::Insert: "
- "left_pos({}), left_child({}, {:#x}), right_child({}, {:#x}) ...",
- left_pos, left_key, left_child_addr, *op_right_key, right_child_addr);
- } else {
- logger().debug("OTree::Internal::Insert: "
- "left_pos({}), left_child({}, {:#x}), right_child(N/A, {:#x}) ...",
- left_pos, left_key, left_child_addr, right_child_addr);
- }
- // update left_pos => left_child to left_pos => right_child
impl->replace_child_addr(left_pos, right_child_addr, left_child_addr);
- replace_track(right_child, left_child);
- search_position_t insert_pos = left_pos;
- auto [insert_stage, insert_size] = impl->evaluate_insert(
- left_key, left_child_addr, insert_pos);
- auto free_size = impl->free_size();
- if (free_size >= insert_size) {
- // insert
- [[maybe_unused]] auto p_value = impl->insert(
- left_key, left_child_addr, insert_pos, insert_stage, insert_size);
- assert(impl->free_size() == free_size - insert_size);
- assert(insert_pos <= left_pos);
- assert(p_value->value == left_child_addr);
- track_insert(insert_pos, insert_stage, left_child, right_child);
- validate_tracked_children();
- return node_ertr::now();
+ // update track from left_pos => left_child to right_child
+ replace_track(right_child, left_child, update_right_index);
+
+ auto left_key = *left_child->impl->get_pivot_index();
+ Ref<InternalNode> this_ref = this;
+ return insert_or_split(
+ c, left_pos, left_key, left_child,
+ (update_right_index ? right_child : nullptr)
+ ).safe_then([this, c, this_ref = std::move(this_ref)] (auto split_right) {
+ if (split_right) {
+ // even if update_right_index could be true,
+ // we haven't fixed the right_child index of this node yet,
+ // so my parent index should be correct now.
+ return apply_split_to_parent(c, split_right, false);
+ } else {
+ return node_ertr::now();
+ }
+ }).safe_then([c, update_right_index, right_child] {
+ if (update_right_index) {
+ // right_child must be already untracked by insert_or_split()
+ return right_child->parent_info().ptr->fix_index(c, right_child);
+ } else {
+ // there is no need to call try_merge_adjacent() because
+ // the filled size of the inserted node or the split right node
+ // won't be reduced if update_right_index is false.
+ return node_ertr::now();
+ }
+ });
+}
+
+node_future<> InternalNode::erase_child(context_t c, Ref<Node>&& child_ref)
+{
+ // this is a special version of recursive merge
+ impl->validate_non_empty();
+ assert(child_ref->use_count() == 1);
+ validate_child_tracked(*child_ref);
+
+ // fix the child's previous node as the new tail,
+ // and trigger prv_child_ref->try_merge_adjacent() at the end
+ bool fix_tail = (child_ref->parent_info().position.is_end() &&
+ !impl->is_keys_empty());
+ return node_ertr::now().safe_then([c, this, fix_tail] {
+ if (fix_tail) {
+ search_position_t new_tail_pos;
+ const laddr_packed_t* new_tail_p_addr = nullptr;
+ impl->get_largest_slot(&new_tail_pos, nullptr, &new_tail_p_addr);
+ return get_or_track_child(c, new_tail_pos, new_tail_p_addr->value);
+ } else {
+ return node_ertr::make_ready_future<Ref<Node>>();
+ }
+ }).safe_then([c, this, child_ref = std::move(child_ref)]
+ (auto&& new_tail_child) mutable {
+ auto child_pos = child_ref->parent_info().position;
+ if (new_tail_child) {
+ logger().info("OTree::Internal::EraseChild: erase {}'s child {} at pos({}), "
+ "and fix new child tail {} at pos({}) ...",
+ get_name(), child_ref->get_name(), child_pos,
+ new_tail_child->get_name(),
+ new_tail_child->parent_info().position);
+ assert(!new_tail_child->impl->is_level_tail());
+ new_tail_child->make_tail(c);
+ assert(new_tail_child->impl->is_level_tail());
+ if (new_tail_child->impl->node_type() == node_type_t::LEAF) {
+ // no need to proceed merge because the filled size is not changed
+ new_tail_child.reset();
+ }
+ } else {
+ logger().info("OTree::Internal::EraseChild: erase {}'s child {} at pos({}) ...",
+ get_name(), child_ref->get_name(), child_pos);
+ }
+
+ do_untrack_child(*child_ref);
+ Ref<InternalNode> this_ref = this;
+ child_ref->_parent_info.reset();
+ return child_ref->retire(c, std::move(child_ref)
+ ).safe_then([c, this, child_pos, this_ref = std::move(this_ref)] () mutable {
+ if ((impl->is_level_tail() && impl->is_keys_empty()) ||
+ (!impl->is_level_tail() && impl->is_keys_one())) {
+ // there is only one value left
+ // fast path without mutating the extent
+ logger().debug("OTree::Internal::EraseChild: {} has one value left, erase ...",
+ get_name());
+#ifndef NDEBUG
+ if (impl->is_level_tail()) {
+ assert(child_pos.is_end());
+ } else {
+ assert(child_pos == search_position_t::begin());
+ }
+#endif
+
+ if (is_root()) {
+ // Note: if merge/split works as expected, we should never encounter the
+ // situation when the internal root has <=1 children:
+ //
+ // A newly created internal root (see Node::upgrade_root()) will have 2
+ // children after split is finished.
+ //
+ // When merge happens, children will try to merge each other, and if the
+ // root detects there is only one child left, the root will be
+ // down-graded to the only child.
+ //
+ // In order to preserve the invariant, we need to make sure the new
+ // internal root also has at least 2 children.
+ ceph_abort("trying to erase the last item from the internal root node");
+ }
+
+ // track erase
+ assert(tracked_child_nodes.empty());
+
+ // no child should be referencing this node now, this_ref is the last one.
+ assert(this_ref->use_count() == 1);
+ Ref<Node> node_ref = this_ref;
+ this_ref.reset();
+ return Node::erase_node(c, std::move(node_ref));
+ }
+
+ impl->prepare_mutate(c);
+ auto [erase_stage, next_or_last_pos] = impl->erase(child_pos);
+ if (child_pos.is_end()) {
+ // next_or_last_pos as last_pos
+ track_make_tail(next_or_last_pos);
+ } else {
+ // next_or_last_pos as next_pos
+ track_erase(child_pos, erase_stage);
+ }
+ validate_tracked_children();
+
+ if (is_root()) {
+ return try_downgrade_root(c, std::move(this_ref));
+ } else {
+ bool update_parent_index;
+ if (impl->is_level_tail()) {
+ update_parent_index = false;
+ } else {
+ // next_or_last_pos as next_pos
+ next_or_last_pos.is_end() ? update_parent_index = true
+ : update_parent_index = false;
+ }
+ return try_merge_adjacent(c, update_parent_index);
+ }
+ }).safe_then([c, new_tail_child = std::move(new_tail_child)] {
+ // finally, check if the new tail child needs to merge
+ if (new_tail_child && !new_tail_child->is_root()) {
+ assert(new_tail_child->impl->is_level_tail());
+ return new_tail_child->try_merge_adjacent(c, false);
+ } else {
+ return node_ertr::now();
+ }
+ });
+ });
+}
+
+node_future<> InternalNode::fix_index(context_t c, Ref<Node> child)
+{
+ impl->validate_non_empty();
+
+ auto& child_pos = child->parent_info().position;
+ // child must already be untracked before calling fix_index()
+ assert(tracked_child_nodes.find(child_pos) == tracked_child_nodes.end());
+ validate_child_inconsistent(*child);
+
+ impl->prepare_mutate(c);
+
+ key_view_t new_key = *child->impl->get_pivot_index();
+ logger().debug("OTree::Internal::FixIndex: fix {}'s index of child {} at pos({}), "
+ "new_key={} ...",
+ get_name(), child->get_name(), child_pos, new_key);
+
+ // erase the incorrect item
+ auto [erase_stage, next_pos] = impl->erase(child_pos);
+ track_erase(child_pos, erase_stage);
+ validate_tracked_children();
+
+ // find out whether there is a need to fix parent index recursively
+ bool update_parent_index;
+ if (impl->is_level_tail()) {
+ update_parent_index = false;
+ } else {
+ next_pos.is_end() ? update_parent_index = true
+ : update_parent_index = false;
}
- // split and insert
+
Ref<InternalNode> this_ref = this;
- return (is_root() ? upgrade_root(c) : node_ertr::now()
- ).safe_then([this, c] {
- return InternalNode::allocate(
- c, impl->field_type(), impl->is_level_tail(), impl->level());
- }).safe_then([this_ref, this, c, left_key, left_child, right_child,
- insert_pos, insert_stage=insert_stage, insert_size=insert_size](auto fresh_right) mutable {
- auto right_node = fresh_right.node;
- auto left_child_addr = left_child->impl->laddr();
- auto [split_pos, is_insert_left, p_value] = impl->split_insert(
- fresh_right.mut, *right_node->impl, left_key, left_child_addr,
- insert_pos, insert_stage, insert_size);
- assert(p_value->value == left_child_addr);
- track_split(split_pos, right_node);
- if (is_insert_left) {
- track_insert(insert_pos, insert_stage, left_child);
+ return insert_or_split(c, next_pos, new_key, child
+ ).safe_then([this, c, update_parent_index,
+ this_ref = std::move(this_ref)] (auto split_right) {
+ if (split_right) {
+ // after split, the parent index to the split_right will be incorrect
+ // if update_parent_index is true.
+ return apply_split_to_parent(c, split_right, update_parent_index);
} else {
- right_node->track_insert(insert_pos, insert_stage, left_child);
+ // no split path
+ if (is_root()) {
+ // no need to call try_downgrade_root() because the number of keys
+ // has not changed.
+ return node_ertr::now();
+ } else {
+ // for non-root, maybe need merge adjacent or fix parent,
+ // because the filled node size may be reduced.
+ return try_merge_adjacent(c, update_parent_index);
+ }
}
- validate_tracked_children();
- right_node->validate_tracked_children();
+ });
+}
- return apply_split_to_parent(c, right_node);
- // TODO (optimize)
- // try to acquire space from siblings before split... see btrfs
+node_future<> InternalNode::apply_children_merge(
+ context_t c, Ref<Node>&& left_child,
+ Ref<Node>&& right_child, bool update_index)
+{
+ auto left_pos = left_child->parent_info().position;
+ auto left_addr = left_child->impl->laddr();
+ auto& right_pos = right_child->parent_info().position;
+ auto right_addr = right_child->impl->laddr();
+ logger().debug("OTree::Internal::ApplyChildMerge: apply {}'s child "
+ "{} at pos({}), to merge with {} at pos({}), update_index={} ...",
+ get_name(), left_child->get_name(), left_pos,
+ right_child->get_name(), right_pos, update_index);
+
+#ifndef NDEBUG
+ assert(left_child->parent_info().ptr == this);
+ assert(!left_pos.is_end());
+ const laddr_packed_t* p_value_left;
+ impl->get_slot(left_pos, nullptr, &p_value_left);
+ assert(p_value_left->value == left_addr);
+
+ assert(right_child->use_count() == 1);
+ assert(right_child->parent_info().ptr == this);
+ const laddr_packed_t* p_value_right;
+ if (right_pos.is_end()) {
+ assert(right_child->impl->is_level_tail());
+ assert(left_child->impl->is_level_tail());
+ assert(impl->is_level_tail());
+ assert(!update_index);
+ p_value_right = impl->get_tail_value();
+ } else {
+ assert(!right_child->impl->is_level_tail());
+ assert(!left_child->impl->is_level_tail());
+ impl->get_slot(right_pos, nullptr, &p_value_right);
+ }
+ assert(p_value_right->value == right_addr);
+#endif
+
+ // XXX: we may jump to try_downgrade_root() without mutating this node.
+
+ // update layout from right_pos => right_addr to left_addr
+ impl->prepare_mutate(c);
+ impl->replace_child_addr(right_pos, left_addr, right_addr);
+
+ // update track from right_pos => right_child to left_child
+ do_untrack_child(*left_child);
+ replace_track(left_child, right_child, update_index);
+
+ // erase left_pos from layout
+ auto [erase_stage, next_pos] = impl->erase(left_pos);
+ track_erase<false>(left_pos, erase_stage);
+ assert(next_pos == left_child->parent_info().position);
+
+ // All good to retire the right_child.
+ // I'm already ref-counted by left_child.
+ return right_child->retire(c, std::move(right_child)
+ ).safe_then([c, this, update_index,
+ left_child = std::move(left_child)] () mutable {
+ Ref<InternalNode> this_ref = this;
+ if (update_index) {
+ return left_child->fix_parent_index(c
+ ).safe_then([c, this, this_ref = std::move(this_ref)] {
+ // I'm all good but:
+ // - my number of keys is reduced by 1
+ // - my size may underflow,
+ // but try_merge_adjacent() is already part of fix_index()
+ if (is_root()) {
+ return try_downgrade_root(c, std::move(this_ref));
+ } else {
+ return node_ertr::now();
+ }
+ });
+ } else {
+ left_child.reset();
+ validate_tracked_children();
+ // I'm all good but:
+ // - my number of keys is reduced by 1
+ // - my size may underflow
+ if (is_root()) {
+ return try_downgrade_root(c, std::move(this_ref));
+ } else {
+ return try_merge_adjacent(c, false);
+ }
+ }
+ });
+}
+
+node_future<std::pair<Ref<Node>, Ref<Node>>> InternalNode::get_child_peers(
+ context_t c, const search_position_t& pos)
+{
+ // assume I'm already ref counted by caller
+ search_position_t prev_pos;
+ const laddr_packed_t* prev_p_child_addr = nullptr;
+ search_position_t next_pos;
+ const laddr_packed_t* next_p_child_addr = nullptr;
+
+ if (pos.is_end()) {
+ assert(impl->is_level_tail());
+ if (!impl->is_keys_empty()) {
+ // got previous child only
+ impl->get_largest_slot(&prev_pos, nullptr, &prev_p_child_addr);
+ assert(prev_pos < pos);
+ assert(prev_p_child_addr != nullptr);
+ } else {
+ // no keys, so no peer children
+ }
+ } else { // !pos.is_end()
+ if (pos != search_position_t::begin()) {
+ // got previous child
+ prev_pos = pos;
+ impl->get_prev_slot(prev_pos, nullptr, &prev_p_child_addr);
+ assert(prev_pos < pos);
+ assert(prev_p_child_addr != nullptr);
+ } else {
+ // is already the first child, so no previous child
+ }
+
+ next_pos = pos;
+ impl->get_next_slot(next_pos, nullptr, &next_p_child_addr);
+ if (next_pos.is_end()) {
+ if (impl->is_level_tail()) {
+ // the next child is the tail
+ next_p_child_addr = impl->get_tail_value();
+ assert(pos < next_pos);
+ assert(next_p_child_addr != nullptr);
+ } else {
+ // next child doesn't exist
+ assert(next_p_child_addr == nullptr);
+ }
+ } else {
+ // got the next child
+ assert(pos < next_pos);
+ assert(next_p_child_addr != nullptr);
+ }
+ }
+
+ return node_ertr::now().safe_then([this, c, prev_pos, prev_p_child_addr] {
+ if (prev_p_child_addr != nullptr) {
+ return get_or_track_child(c, prev_pos, prev_p_child_addr->value);
+ } else {
+ return node_ertr::make_ready_future<Ref<Node>>();
+ }
+ }).safe_then([this, c, next_pos, next_p_child_addr] (Ref<Node> lnode) {
+ if (next_p_child_addr != nullptr) {
+ return get_or_track_child(c, next_pos, next_p_child_addr->value
+ ).safe_then([lnode] (Ref<Node> rnode) {
+ return node_ertr::make_ready_future<std::pair<Ref<Node>, Ref<Node>>>(
+ lnode, rnode);
+ });
+ } else {
+ return node_ertr::make_ready_future<std::pair<Ref<Node>, Ref<Node>>>(
+ lnode, nullptr);
+ }
});
}
);
}
+void InternalNode::track_merge(
+ Ref<Node> _right_node, match_stage_t stage, search_position_t& left_last_pos)
+{
+ assert(level() == _right_node->level());
+ assert(impl->node_type() == _right_node->impl->node_type());
+ auto& right_node = *static_cast<InternalNode*>(_right_node.get());
+ if (right_node.tracked_child_nodes.empty()) {
+ return;
+ }
+
+ match_stage_t curr_stage = STAGE_BOTTOM;
+
+ // prepare the initial left_last_pos for offset
+ while (curr_stage < stage) {
+ left_last_pos.index_by_stage(curr_stage) = 0;
+ ++curr_stage;
+ }
+ ++left_last_pos.index_by_stage(curr_stage);
+
+ // fix the tracked child nodes of right_node, stage by stage.
+ auto& right_tracked_children = right_node.tracked_child_nodes;
+ auto rit = right_tracked_children.begin();
+ while (curr_stage <= STAGE_TOP) {
+ auto right_pos_until = search_position_t::begin();
+ right_pos_until.index_by_stage(curr_stage) = INDEX_UPPER_BOUND;
+ auto rend = right_tracked_children.lower_bound(right_pos_until);
+ while (rit != rend) {
+ auto new_pos = rit->second->parent_info().position;
+ assert(new_pos == rit->first);
+ assert(rit->second->parent_info().ptr == &right_node);
+ new_pos += left_last_pos;
+ auto p_child = rit->second;
+ rit = right_tracked_children.erase(rit);
+ p_child->as_child(new_pos, this);
+ }
+ left_last_pos.index_by_stage(curr_stage) = 0;
+ ++curr_stage;
+ }
+
+ // fix the end tracked child node of right_node, if exists.
+ if (rit != right_tracked_children.end()) {
+ assert(rit->first == search_position_t::end());
+ assert(rit->second->parent_info().position == search_position_t::end());
+ assert(right_node.impl->is_level_tail());
+ assert(impl->is_level_tail());
+ auto p_child = rit->second;
+ rit = right_tracked_children.erase(rit);
+ p_child->as_child(search_position_t::end(), this);
+ }
+ assert(right_tracked_children.empty());
+
+ validate_tracked_children();
+}
+
node_future<> InternalNode::test_clone_root(
context_t c_other, RootNodeTracker& tracker_other) const
{
});
}
+node_future<> InternalNode::try_downgrade_root(
+ context_t c, Ref<Node>&& this_ref)
+{
+ assert(this_ref.get() == this);
+ assert(is_root());
+ assert(impl->is_level_tail());
+ if (!impl->is_keys_empty()) {
+ // I have more than 1 values, no need to downgrade
+ return node_ertr::now();
+ }
+
+ // proceed downgrade root to the only child
+ laddr_t child_addr = impl->get_tail_value()->value;
+ return get_or_track_child(c, search_position_t::end(), child_addr
+ ).safe_then([c, this, this_ref = std::move(this_ref)] (auto child) mutable {
+ logger().info("OTree::Internal::DownGradeRoot: downgrade {} to new root {}",
+ get_name(), child->get_name());
+ // Invariant, see InternalNode::erase_child()
+ // the new internal root should have at least 2 children.
+ assert(child->impl->is_level_tail());
+ if (child->impl->node_type() == node_type_t::INTERNAL) {
+ ceph_assert(!child->impl->is_keys_empty());
+ }
+
+ this->super->do_untrack_root(*this);
+ assert(tracked_child_nodes.size() == 1);
+ do_untrack_child(*child);
+ child->_parent_info.reset();
+ child->make_root_from(c, std::move(this->super), impl->laddr());
+ return retire(c, std::move(this_ref));
+ });
+}
+
+node_future<Ref<InternalNode>> InternalNode::insert_or_split(
+ context_t c,
+ const search_position_t& pos,
+ const key_view_t& insert_key,
+ Ref<Node> insert_child,
+ Ref<Node> outdated_child)
+{
+ // XXX: check the insert_child is unlinked from this node
+#ifndef NDEBUG
+ auto _insert_key = *insert_child->impl->get_pivot_index();
+ assert(insert_key.compare_to(_insert_key) == MatchKindCMP::EQ);
+#endif
+ auto insert_value = insert_child->impl->laddr();
+ auto insert_pos = pos;
+ logger().debug("OTree::Internal::InsertSplit: insert {} "
+ "with insert_key={}, insert_child={}, insert_pos({}), "
+ "outdated_child={} ...",
+ get_name(), insert_key, insert_child->get_name(),
+ insert_pos, (outdated_child ? "True" : "False"));
+ auto [insert_stage, insert_size] = impl->evaluate_insert(
+ insert_key, insert_value, insert_pos);
+ auto free_size = impl->free_size();
+ if (free_size >= insert_size) {
+ // proceed to insert
+ [[maybe_unused]] auto p_value = impl->insert(
+ insert_key, insert_value, insert_pos, insert_stage, insert_size);
+ assert(impl->free_size() == free_size - insert_size);
+ assert(insert_pos <= pos);
+ assert(p_value->value == insert_value);
+ track_insert(insert_pos, insert_stage, insert_child);
+
+ if (outdated_child) {
+ // untrack the inaccurate child after updated its position
+ // before validate, and before fix_index()
+ validate_child_inconsistent(*outdated_child);
+ // we will need its parent_info valid for the following fix_index()
+ do_untrack_child(*outdated_child);
+ }
+
+ validate_tracked_children();
+ return node_ertr::make_ready_future<Ref<InternalNode>>(nullptr);
+ }
+
+ // proceed to split with insert
+ // assume I'm already ref-counted by caller
+ return (is_root() ? upgrade_root(c) : node_ertr::now()
+ ).safe_then([this, c] {
+ return InternalNode::allocate(
+ c, impl->field_type(), impl->is_level_tail(), impl->level());
+ }).safe_then([this, insert_key, insert_child, insert_pos,
+ insert_stage=insert_stage, insert_size=insert_size,
+ outdated_child](auto fresh_right) mutable {
+ // I'm the left_node and need to split into the right_node
+ auto right_node = fresh_right.node;
+ logger().info("OTree::Internal::InsertSplit: proceed split {} to fresh {} "
+ "with insert_child={}, outdated_child={} ...",
+ get_name(), right_node->get_name(),
+ insert_child->get_name(),
+ (outdated_child ? outdated_child->get_name() : "N/A"));
+ auto insert_value = insert_child->impl->laddr();
+ auto [split_pos, is_insert_left, p_value] = impl->split_insert(
+ fresh_right.mut, *right_node->impl, insert_key, insert_value,
+ insert_pos, insert_stage, insert_size);
+ assert(p_value->value == insert_value);
+ track_split(split_pos, right_node);
+ if (is_insert_left) {
+ track_insert(insert_pos, insert_stage, insert_child);
+ } else {
+ right_node->track_insert(insert_pos, insert_stage, insert_child);
+ }
+
+ if (outdated_child) {
+ // untrack the inaccurate child after updated its position
+ // before validate, and before fix_index()
+ auto& _parent = outdated_child->parent_info().ptr;
+ _parent->validate_child_inconsistent(*outdated_child);
+ // we will need its parent_info valid for the following fix_index()
+ _parent->do_untrack_child(*outdated_child);
+ }
+
+ validate_tracked_children();
+ right_node->validate_tracked_children();
+ return right_node;
+ });
+}
+
node_future<Ref<Node>> InternalNode::get_or_track_child(
context_t c, const search_position_t& position, laddr_t child_addr)
{
}
void InternalNode::replace_track(
- Ref<Node> new_child, Ref<Node> old_child)
+ Ref<Node> new_child, Ref<Node> old_child, bool is_new_child_outdated)
{
assert(old_child->parent_info().ptr == this);
auto& pos = old_child->parent_info().position;
do_untrack_child(*old_child);
- new_child->as_child(pos, this);
+ if (is_new_child_outdated) {
+ // we need to keep track of the outdated child through
+ // insert and split.
+ new_child->as_child<false>(pos, this);
+ } else {
+ new_child->as_child(pos, this);
+ }
// ok, safe to release ref
old_child->_parent_info.reset();
- validate_child_tracked(*new_child);
+#ifndef NDEBUG
+ if (is_new_child_outdated) {
+ validate_child_inconsistent(*new_child);
+ } else {
+ validate_child_tracked(*new_child);
+ }
+#endif
}
void InternalNode::track_split(
}
}
+template <bool VALIDATE>
+void InternalNode::track_erase(
+ const search_position_t& erase_pos, match_stage_t erase_stage)
+{
+ auto first = tracked_child_nodes.lower_bound(erase_pos);
+ assert(first == tracked_child_nodes.end() ||
+ first->first != erase_pos);
+ auto pos_upper_bound = erase_pos;
+ pos_upper_bound.index_by_stage(erase_stage) = INDEX_UPPER_BOUND;
+ auto last = tracked_child_nodes.lower_bound(pos_upper_bound);
+ std::vector<Node*> p_nodes;
+ std::for_each(first, last, [&p_nodes](auto& kv) {
+ p_nodes.push_back(kv.second);
+ });
+ tracked_child_nodes.erase(first, last);
+ for (auto& p_node: p_nodes) {
+ auto new_pos = p_node->parent_info().position;
+ assert(new_pos.index_by_stage(erase_stage) > 0);
+ --new_pos.index_by_stage(erase_stage);
+ p_node->as_child<VALIDATE>(new_pos, this);
+ }
+}
+
+void InternalNode::track_make_tail(const search_position_t& last_pos)
+{
+ // assume I'm ref counted by the caller.
+ assert(impl->is_level_tail());
+ assert(!last_pos.is_end());
+ assert(tracked_child_nodes.find(search_position_t::end()) ==
+ tracked_child_nodes.end());
+ auto last_it = tracked_child_nodes.find(last_pos);
+ if (last_it != tracked_child_nodes.end()) {
+ assert(std::next(last_it) == tracked_child_nodes.end());
+ auto p_last_child = last_it->second;
+ tracked_child_nodes.erase(last_it);
+ p_last_child->as_child(search_position_t::end(), this);
+ } else {
+ assert(tracked_child_nodes.lower_bound(last_pos) ==
+ tracked_child_nodes.end());
+ }
+}
+
void InternalNode::validate_child(const Node& child) const
{
#ifndef NDEBUG
#endif
}
+void InternalNode::validate_child_inconsistent(const Node& child) const
+{
+#ifndef NDEBUG
+ assert(impl->level() - 1 == child.impl->level());
+ assert(this == child.parent_info().ptr);
+ auto& child_pos = child.parent_info().position;
+ // the tail value has no key to fix
+ assert(!child_pos.is_end());
+ assert(!child.impl->is_level_tail());
+
+ key_view_t current_key;
+ const laddr_packed_t* p_value;
+ impl->get_slot(child_pos, ¤t_key, &p_value);
+ key_view_t new_key = *child.impl->get_pivot_index();
+ assert(current_key.compare_to(new_key) != MatchKindCMP::EQ);
+ assert(p_value->value == child.impl->laddr());
+#endif
+}
+
node_future<InternalNode::fresh_node_t> InternalNode::allocate(
context_t c, field_type_t field_type, bool is_level_tail, level_t level)
{
node_future<Ref<tree_cursor_t>>
LeafNode::erase(context_t c, const search_position_t& pos, bool get_next)
{
- ceph_abort("not implemented");
- return node_ertr::make_ready_future<Ref<tree_cursor_t>>(
- tree_cursor_t::get_invalid());
+ assert(!pos.is_end());
+ assert(!impl->is_keys_empty());
+ Ref<LeafNode> this_ref = this;
+ logger().debug("OTree::Leaf::Erase: erase {}'s pos({}), get_next={} ...",
+ get_name(), pos, get_next);
+
+ // get the next cursor
+ return node_ertr::now().safe_then([c, &pos, get_next, this] {
+ if (get_next) {
+ return get_next_cursor(c, pos);
+ } else {
+ return node_ertr::make_ready_future<Ref<tree_cursor_t>>();
+ }
+ }).safe_then([c, &pos, this_ref = std::move(this_ref),
+ this] (Ref<tree_cursor_t> next_cursor) {
+ if (next_cursor && next_cursor->is_end()) {
+ // reset the node reference from the end cursor
+ next_cursor.reset();
+ }
+ return node_ertr::now(
+ ).safe_then([c, &pos, this_ref = std::move(this_ref), this] () mutable {
+#ifndef NDEBUG
+ assert(!impl->is_keys_empty());
+ if (impl->is_keys_one()) {
+ assert(pos == search_position_t::begin());
+ }
+#endif
+ if (!is_root() && impl->is_keys_one()) {
+ // we need to keep the root as an empty leaf node
+ // fast path without mutating the extent
+ // track_erase
+ logger().debug("OTree::Leaf::Erase: {} has one value left, erase ...",
+ get_name());
+ assert(tracked_cursors.size() == 1);
+ auto iter = tracked_cursors.begin();
+ assert(iter->first == pos);
+ iter->second->invalidate();
+ tracked_cursors.clear();
+
+ // no cursor should be referencing this node now, this_ref is the last one.
+ assert(this_ref->use_count() == 1);
+ Ref<Node> node_ref = this_ref;
+ this_ref.reset();
+ return Node::erase_node(c, std::move(node_ref));
+ }
+
+ on_layout_change();
+ impl->prepare_mutate(c);
+ auto [erase_stage, next_pos] = impl->erase(pos);
+ track_erase(pos, erase_stage);
+ validate_tracked_cursors();
+
+ if (is_root()) {
+ return node_ertr::now();
+ } else {
+ bool update_parent_index;
+ if (impl->is_level_tail()) {
+ update_parent_index = false;
+ } else {
+ next_pos.is_end() ? update_parent_index = true
+ : update_parent_index = false;
+ }
+ return try_merge_adjacent(c, update_parent_index);
+ }
+ }).safe_then([next_cursor] {
+ return next_cursor;
+ });
+ });
}
node_future<> LeafNode::extend_value(
return node_ertr::now();
}
+void LeafNode::track_merge(
+ Ref<Node> _right_node, match_stage_t stage, search_position_t& left_last_pos)
+{
+ assert(level() == _right_node->level());
+ // assert(impl->node_type() == _right_node->impl->node_type());
+ auto& right_node = *static_cast<LeafNode*>(_right_node.get());
+ if (right_node.tracked_cursors.empty()) {
+ return;
+ }
+
+ match_stage_t curr_stage = STAGE_BOTTOM;
+
+ // prepare the initial left_last_pos for offset
+ while (curr_stage < stage) {
+ left_last_pos.index_by_stage(curr_stage) = 0;
+ ++curr_stage;
+ }
+ ++left_last_pos.index_by_stage(curr_stage);
+
+ // fix the tracked child nodes of right_node, stage by stage.
+ auto& right_tracked_cursors = right_node.tracked_cursors;
+ auto rit = right_tracked_cursors.begin();
+ while (curr_stage <= STAGE_TOP) {
+ auto right_pos_until = search_position_t::begin();
+ right_pos_until.index_by_stage(curr_stage) = INDEX_UPPER_BOUND;
+ auto rend = right_tracked_cursors.lower_bound(right_pos_until);
+ while (rit != rend) {
+ auto new_pos = rit->second->get_position();
+ assert(new_pos == rit->first);
+ assert(rit->second->get_leaf_node().get() == &right_node);
+ new_pos += left_last_pos;
+ auto p_cursor = rit->second;
+ rit = right_tracked_cursors.erase(rit);
+ p_cursor->update_track<true>(this, new_pos);
+ }
+ left_last_pos.index_by_stage(curr_stage) = 0;
+ ++curr_stage;
+ }
+ assert(right_tracked_cursors.empty());
+
+ validate_tracked_cursors();
+}
+
node_future<> LeafNode::test_clone_root(
context_t c_other, RootNodeTracker& tracker_other) const
{
validate_tracked_cursors();
right_node->validate_tracked_cursors();
- return apply_split_to_parent(c, right_node
+ return apply_split_to_parent(c, right_node, false
).safe_then([ret] {
return ret;
});
}
}
+void LeafNode::track_erase(
+ const search_position_t& erase_pos, match_stage_t erase_stage)
+{
+ // erase tracking and invalidate the erased cursor
+ auto to_erase = tracked_cursors.find(erase_pos);
+ assert(to_erase != tracked_cursors.end());
+ to_erase->second->invalidate();
+ auto first = tracked_cursors.erase(to_erase);
+
+ // update cursor position
+ assert(first == tracked_cursors.lower_bound(erase_pos));
+ auto pos_upper_bound = erase_pos;
+ pos_upper_bound.index_by_stage(erase_stage) = INDEX_UPPER_BOUND;
+ auto last = tracked_cursors.lower_bound(pos_upper_bound);
+ std::vector<tree_cursor_t*> p_cursors;
+ std::for_each(first, last, [&p_cursors](auto& kv) {
+ p_cursors.push_back(kv.second);
+ });
+ tracked_cursors.erase(first, last);
+ for (auto& p_cursor : p_cursors) {
+ search_position_t new_pos = p_cursor->get_position();
+ assert(new_pos.index_by_stage(erase_stage) > 0);
+ --new_pos.index_by_stage(erase_stage);
+ p_cursor->update_track<true>(this, new_pos);
+ }
+}
+
node_future<LeafNode::fresh_node_t> LeafNode::allocate(
context_t c, field_type_t field_type, bool is_level_tail)
{