} else {
TRACET("{} got {}, {}",
c.trans, offset, iter.get_key(), iter.get_val());
- auto e = iter.get_pin();
+ auto e = iter.get_pin(c);
return get_mapping_ret(
interruptible::ready_future_marker{},
std::move(e));
TRACET("{}~{} got {}, {}, repeat ...",
c.trans, offset, end, pos.get_key(), pos.get_val());
ceph_assert((pos.get_key().add_offset(pos.get_val().len)) > offset);
- ret.push_back(pos.get_pin());
+ ret.push_back(pos.get_pin(c));
return BackrefBtree::iterate_repeat_ret_inner(
interruptible::ready_future_marker{},
seastar::stop_iteration::no);
state.ret = iter;
});
});
- }).si_then([](auto &&state) {
- return state.ret->get_pin();
+ }).si_then([c](auto &&state) {
+ return state.ret->get_pin(c);
});
}
class BtreeBackrefPin : public BtreeNodePin<paddr_t, laddr_t> {
extent_types_t type;
public:
- BtreeBackrefPin() = default;
+ BtreeBackrefPin(op_context_t<paddr_t> ctx)
+ : BtreeNodePin(ctx) {}
BtreeBackrefPin(
+ op_context_t<paddr_t> ctx,
CachedExtentRef parent,
uint16_t pos,
backref_map_val_t &val,
backref_node_meta_t &&meta)
: BtreeNodePin(
+ ctx,
parent,
pos,
val.laddr,
namespace crimson::os::seastore {
template <typename key_t, typename val_t>
-void BtreeNodePin<key_t, val_t>::link_extent(LogicalCachedExtent *ref) {
- assert(ref->is_valid());
- // it's only when reading logical extents from disk that we need to
- // link them to lba leaves
- if (!ref->is_pending() && !ref->is_exist_clean()) {
- assert(parent);
- assert(pos != std::numeric_limits<uint16_t>::max());
- if (parent->is_initial_pending()) {
- auto &p = ((FixedKVNode<key_t>&)*parent).get_stable_for_key(
- pin.range.begin);
- p.link_child(ref, pos);
- } else if (parent->is_mutation_pending()) {
- auto &p = (FixedKVNode<key_t>&)*parent->get_prior_instance();
- p.link_child(ref, pos);
- } else {
- assert(!parent->is_pending() && parent->is_valid());
- auto &p = (FixedKVNode<key_t>&)*parent;
- p.link_child(ref, pos);
- }
- pos = std::numeric_limits<uint16_t>::max();
+get_child_ret_t<LogicalCachedExtent>
+BtreeNodePin<key_t, val_t>::get_logical_extent(
+ Transaction &t)
+{
+ assert(parent);
+ assert(parent->is_valid());
+ assert(pos != std::numeric_limits<uint16_t>::max());
+ auto &p = (FixedKVNode<key_t>&)*parent;
+ auto v = p.get_logical_child(ctx, pos);
+ if (!v.has_child()) {
+ this->child_pos = v.get_child_pos();
}
- pin.set_extent(ref);
+ return v;
}
-template void BtreeNodePin<laddr_t, paddr_t>::link_extent(LogicalCachedExtent*);
-template void BtreeNodePin<paddr_t, laddr_t>::link_extent(LogicalCachedExtent*);
+template class BtreeNodePin<laddr_t, paddr_t>;
+template class BtreeNodePin<paddr_t, laddr_t>;
+
} // namespace crimson::os::seastore
#include "crimson/common/log.h"
+#include "crimson/os/seastore/cache.h"
#include "crimson/os/seastore/cached_extent.h"
#include "crimson/os/seastore/seastore_types.h"
namespace crimson::os::seastore {
+template <typename node_key_t>
+struct op_context_t {
+ Cache &cache;
+ Transaction &trans;
+ btree_pin_set_t<node_key_t> *pins = nullptr;
+};
+
constexpr uint16_t MAX_FIXEDKVBTREE_DEPTH = 8;
template <typename T>
template <typename key_t, typename val_t>
class BtreeNodePin : public PhysicalNodePin<key_t, val_t> {
+ op_context_t<key_t> ctx;
/**
* parent
*
public:
using val_type = val_t;
- BtreeNodePin() = default;
+ BtreeNodePin(op_context_t<key_t> ctx) : ctx(ctx) {}
BtreeNodePin(
+ op_context_t<key_t> ctx,
CachedExtentRef parent,
uint16_t pos,
val_t &value,
extent_len_t len,
fixed_kv_node_meta_t<key_t> &&meta)
- : parent(parent), value(value), len(len), pos(pos) {
+ : ctx(ctx),
+ parent(parent),
+ value(value),
+ len(len),
+ pos(pos)
+ {
pin.set_range(std::move(meta));
+ if (!parent->is_pending()) {
+ this->child_pos = {parent, pos};
+ }
}
CachedExtentRef get_parent() const final {
parent = pin;
}
- void link_extent(LogicalCachedExtent *ref) final;
+ void link_extent(LogicalCachedExtent *ref) final {
+ pin.set_extent(ref);
+ pos = std::numeric_limits<uint16_t>::max();
+ }
+
+ uint16_t get_pos() const final {
+ return pos;
+ }
extent_len_t get_length() const final {
ceph_assert(pin.range.end > pin.range.begin);
PhysicalNodePinRef<key_t, val_t> duplicate() const final {
auto ret = std::unique_ptr<BtreeNodePin<key_t, val_t>>(
- new BtreeNodePin<key_t, val_t>);
+ new BtreeNodePin<key_t, val_t>(ctx));
ret->pin.set_range(pin.range);
ret->value = value;
ret->parent = parent;
ret->len = len;
+ ret->pos = pos;
return ret;
}
bool has_been_invalidated() const final {
return parent->has_been_invalidated();
}
+
+ get_child_ret_t<LogicalCachedExtent> get_logical_extent(Transaction&) final;
};
}
template <typename T>
phy_tree_root_t& get_phy_tree_root(root_t& r);
-template <typename node_key_t>
-struct op_context_t {
- Cache &cache;
- Transaction &trans;
- btree_pin_set_t<node_key_t> *pins = nullptr;
-};
-
using get_phy_tree_root_node_ret =
std::pair<bool,
::crimson::interruptible::interruptible_future<
template <typename T>
void unlink_phy_tree_root_node(RootBlockRef &root_block);
-
template <typename T>
Transaction::tree_stats_t& get_tree_stats(Transaction &t);
return leaf.pos == 0;
}
- PhysicalNodePinRef<node_key_t, typename pin_t::val_type> get_pin() const {
+ PhysicalNodePinRef<node_key_t, typename pin_t::val_type>
+ get_pin(op_context_t<node_key_t> ctx) const {
assert(!is_end());
auto val = get_val();
auto key = get_key();
return std::make_unique<pin_t>(
+ ctx,
leaf.node,
leaf.pos,
val,
return leaf.node;
}
+ uint16_t get_leaf_pos() {
+ return leaf.pos;
+ }
private:
iterator() noexcept {}
iterator(depth_t depth) noexcept : internal(depth - 1) {}
F &f,
mapped_space_visitor_t *visitor
) {
- LOG_PREFIX(FixedKVBtree::lookup_internal_level);
assert(depth > 1);
auto &parent_entry = iter.get_internal(depth + 1);
auto parent = parent_entry.node;
return seastar::now();
};
- auto child_pos = parent->get_child(c.trans, node_iter);
- auto &child = child_pos.child;
- if (child) {
- SUBTRACET(seastore_fixedkv_tree,
- "got child on {}, pos: {}, res: {}",
- c.trans,
- *parent_entry.node,
- parent_entry.pos,
- *child);
-
- ceph_assert(child->is_valid());
- if (!child->is_pending_in_trans(c.trans.get_trans_id())) {
- c.trans.add_to_read_set(child);
- if (!child->is_mutation_pending()) {
- c.cache.touch_extent(*child);
- }
- }
- return child->wait_io().then(
- [child, on_found=std::move(on_found), node_iter]() mutable {
+ auto v = parent->template get_child<internal_node_t>(c, node_iter);
+ if (v.has_child()) {
+ return v.get_child_fut().then(
+ [on_found=std::move(on_found), node_iter, c,
+ parent_entry](auto child) mutable {
+ LOG_PREFIX(FixedKVBtree::lookup_internal_level);
+ SUBTRACET(seastore_fixedkv_tree,
+ "got child on {}, pos: {}, res: {}",
+ c.trans,
+ *parent_entry.node,
+ parent_entry.pos,
+ *child);
auto &cnode = (typename internal_node_t::base_t &)*child;
assert(cnode.get_node_meta().begin == node_iter.get_key());
assert(cnode.get_node_meta().end > node_iter.get_key());
});
}
+ auto child_pos = v.get_child_pos();
auto next_iter = node_iter + 1;
auto begin = node_iter->get_key();
auto end = next_iter == parent->end()
begin,
end,
std::make_optional<node_position_t<internal_node_t>>(
- child_pos.stable_parent->template cast<internal_node_t>(),
- child_pos.pos)
+ child_pos.template get_parent<internal_node_t>(),
+ child_pos.get_pos())
).si_then([on_found=std::move(on_found)](InternalNodeRef node) {
return on_found(node);
});
F &f,
mapped_space_visitor_t *visitor
) {
- LOG_PREFIX(FixedKVBtree::lookup_leaf);
auto &parent_entry = iter.get_internal(2);
auto parent = parent_entry.node;
assert(parent);
return seastar::now();
};
- auto child_pos = parent->get_child(c.trans, node_iter);
- auto &child = child_pos.child;
- if (child) {
- SUBTRACET(seastore_fixedkv_tree,
- "got child on {}, pos: {}, res: {}",
- c.trans,
- *parent_entry.node,
- parent_entry.pos,
- *child);
-
- ceph_assert(child->is_valid());
- if (!child->is_pending_in_trans(c.trans.get_trans_id())) {
- c.trans.add_to_read_set(child);
- if (!child->is_mutation_pending()) {
- c.cache.touch_extent(*child);
- }
- }
- return child->wait_io().then(
- [child, on_found=std::move(on_found), node_iter]() mutable {
+ auto v = parent->template get_child<leaf_node_t>(c, node_iter);
+ if (v.has_child()) {
+ return v.get_child_fut().then(
+ [on_found=std::move(on_found), node_iter, c,
+ parent_entry](auto child) mutable {
+ LOG_PREFIX(FixedKVBtree::lookup_leaf);
+ SUBTRACET(seastore_fixedkv_tree,
+ "got child on {}, pos: {}, res: {}",
+ c.trans,
+ *parent_entry.node,
+ parent_entry.pos,
+ *child);
auto &cnode = (typename internal_node_t::base_t &)*child;
assert(cnode.get_node_meta().begin == node_iter.get_key());
assert(cnode.get_node_meta().end > node_iter.get_key());
});
}
+ auto child_pos = v.get_child_pos();
auto next_iter = node_iter + 1;
auto begin = node_iter->get_key();
auto end = next_iter == parent->end()
begin,
end,
std::make_optional<node_position_t<leaf_node_t>>(
- child_pos.stable_parent->template cast<leaf_node_t>(),
- child_pos.pos)
+ child_pos.template get_parent<leaf_node_t>(),
+ child_pos.get_pos())
).si_then([on_found=std::move(on_found)](LeafNodeRef node) {
return on_found(node);
});
return seastar::now();
};
- auto child_pos = parent_pos.node->get_child(c.trans, donor_iter);
- auto &child = child_pos.child;
- if (child) {
- SUBTRACET(seastore_fixedkv_tree,
- "got child on {}, pos: {}, res: {}",
- c.trans,
- *parent_pos.node,
- donor_iter.get_offset(),
- *child);
-
- ceph_assert(child->is_valid());
- if (!child->is_pending_in_trans(c.trans.get_trans_id())) {
- c.trans.add_to_read_set(child);
- if (!child->is_mutation_pending()) {
- c.cache.touch_extent(*child);
- }
- }
- return child->wait_io().then(
- [child, do_merge=std::move(do_merge), &pos,
- donor_iter, donor_is_left]() mutable {
+ auto v = parent_pos.node->template get_child<NodeType>(c, donor_iter);
+ if (v.has_child()) {
+ return v.get_child_fut().then(
+ [do_merge=std::move(do_merge), &pos,
+ donor_iter, donor_is_left, c, parent_pos](auto child) mutable {
+ LOG_PREFIX(FixedKVBtree::merge_level);
+ SUBTRACET(seastore_fixedkv_tree,
+ "got child on {}, pos: {}, res: {}",
+ c.trans,
+ *parent_pos.node,
+ donor_iter.get_offset(),
+ *child);
auto &node = (typename internal_node_t::base_t&)*child;
assert(donor_is_left ?
node.get_node_meta().end == pos.node->get_node_meta().begin :
});
}
+ auto child_pos = v.get_child_pos();
return get_node<NodeType>(
c,
depth,
begin,
end,
std::make_optional<node_position_t<NodeType>>(
- child_pos.stable_parent->template cast<NodeType>(),
- child_pos.pos)
+ child_pos.template get_parent<NodeType>(),
+ child_pos.get_pos())
).si_then([do_merge=std::move(do_merge)](typename NodeType::Ref donor) {
return do_merge(donor);
});
}
}
- struct child_pos_t {
- FixedKVNodeRef stable_parent;
- uint16_t pos = std::numeric_limits<uint16_t>::max();
- CachedExtentRef child;
- child_pos_t(CachedExtentRef child) : child(child) {}
- child_pos_t(FixedKVNodeRef stable_parent, uint16_t pos)
- : stable_parent(stable_parent), pos(pos) {}
- };
-
void link_child(ChildableCachedExtent* child, uint16_t pos) {
assert(pos < get_node_size());
assert(child);
set_child_ptracker(child);
}
- template <typename iter_t>
- child_pos_t get_child(Transaction &t, iter_t iter) {
+ virtual get_child_ret_t<LogicalCachedExtent>
+ get_logical_child(op_context_t<node_key_t> c, uint16_t pos) = 0;
+
+ template <typename T, typename iter_t>
+ get_child_ret_t<T> get_child(op_context_t<node_key_t> c, iter_t iter) {
auto pos = iter.get_offset();
assert(children.capacity());
auto child = children[pos];
if (is_valid_child_ptr(child)) {
- return child_pos_t(child->get_transactional_view(t));
+ ceph_assert(child->get_type() == T::TYPE);
+ return c.cache.template get_extent_viewable_by_trans<T>(c.trans, (T*)child);
} else if (is_pending()) {
auto key = iter.get_key();
auto &sparent = get_stable_for_key(key);
auto spos = sparent.child_pos_for_key(key);
auto child = sparent.children[spos];
if (is_valid_child_ptr(child)) {
- return child_pos_t(child->get_transactional_view(t));
+ ceph_assert(child->get_type() == T::TYPE);
+ return c.cache.template get_extent_viewable_by_trans<T>(c.trans, (T*)child);
} else {
return child_pos_t(&sparent, spos);
}
}
}
+ get_child_ret_t<LogicalCachedExtent>
+ get_logical_child(op_context_t<NODE_KEY>, uint16_t pos) final {
+ ceph_abort("impossible");
+ return get_child_ret_t<LogicalCachedExtent>(child_pos_t(nullptr, 0));
+ }
+
bool validate_stable_children() final {
LOG_PREFIX(FixedKVInternalNode::validate_stable_children);
if (this->children.empty()) {
return this->get_split_pivot().get_offset();
}
+ get_child_ret_t<LogicalCachedExtent>
+ get_logical_child(op_context_t<NODE_KEY> c, uint16_t pos) final {
+ auto child = this->children[pos];
+ if (is_valid_child_ptr(child)) {
+ ceph_assert(child->is_logical());
+ return c.cache.template get_extent_viewable_by_trans<
+ LogicalCachedExtent>(c.trans, (LogicalCachedExtent*)child);
+ } 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 child = sparent.children[spos];
+ if (is_valid_child_ptr(child)) {
+ ceph_assert(child->is_logical());
+ return c.cache.template get_extent_viewable_by_trans<
+ LogicalCachedExtent>(c.trans, (LogicalCachedExtent*)child);
+ } else {
+ return child_pos_t(&sparent, spos);
+ }
+ } else {
+ return child_pos_t(this, pos);
+ }
+ }
+
bool validate_stable_children() override {
return true;
}
[](T &){}, [](T &) {});
}
+
/**
* get_extent_if_cached
*
return print_detail_l(out);
}
+void child_pos_t::link_child(ChildableCachedExtent *c) {
+ get_parent<FixedKVNode<laddr_t>>()->link_child(c, pos);
+}
+
void CachedExtent::set_invalid(Transaction &t) {
state = extent_state_t::INVALID;
if (trans_view_hook.is_linked()) {
#include "include/buffer.h"
#include "crimson/common/errorator.h"
+#include "crimson/common/interruptible_future.h"
#include "crimson/os/seastore/seastore_types.h"
namespace crimson::os::seastore {
uint64_t bytes = 0;
};
+class ChildableCachedExtent;
class LogicalCachedExtent;
+class child_pos_t {
+public:
+ child_pos_t(CachedExtentRef stable_parent, uint16_t pos)
+ : stable_parent(stable_parent), pos(pos) {}
+
+ template <typename parent_t>
+ TCachedExtentRef<parent_t> get_parent() {
+ ceph_assert(stable_parent);
+ return stable_parent->template cast<parent_t>();
+ }
+ uint16_t get_pos() {
+ return pos;
+ }
+ void link_child(ChildableCachedExtent *c);
+private:
+ CachedExtentRef stable_parent;
+ uint16_t pos = std::numeric_limits<uint16_t>::max();
+};
+
+template <typename T>
+struct get_child_ret_t {
+ std::variant<child_pos_t, seastar::future<TCachedExtentRef<T>>> ret;
+ get_child_ret_t(child_pos_t pos)
+ : ret(std::move(pos)) {}
+ get_child_ret_t(seastar::future<TCachedExtentRef<T>> child)
+ : ret(std::move(child)) {}
+
+ bool has_child() const {
+ return ret.index() == 1;
+ }
+
+ child_pos_t &get_child_pos() {
+ ceph_assert(ret.index() == 0);
+ return std::get<0>(ret);
+ }
+
+ seastar::future<TCachedExtentRef<T>> &get_child_fut() {
+ ceph_assert(ret.index() == 1);
+ return std::get<1>(ret);
+ }
+};
+
template <typename key_t, typename>
class PhysicalNodePin;
virtual PhysicalNodePinRef<key_t, val_t> duplicate() const = 0;
virtual bool has_been_invalidated() const = 0;
virtual CachedExtentRef get_parent() const = 0;
+ virtual uint16_t get_pos() const = 0;
+
+ virtual get_child_ret_t<LogicalCachedExtent>
+ get_logical_extent(Transaction &t) = 0;
+
+ void link_child(ChildableCachedExtent *c) {
+ ceph_assert(child_pos);
+ child_pos->link_child(c);
+ }
virtual ~PhysicalNodePin() {}
+protected:
+ std::optional<child_pos_t> child_pos = std::nullopt;
};
using LBAPin = PhysicalNodePin<laddr_t, paddr_t>;
TRACET("{}~{} got {}, {}, repeat ...",
c.trans, offset, length, pos.get_key(), pos.get_val());
ceph_assert((pos.get_key() + pos.get_val().len) > offset);
- ret.push_back(pos.get_pin());
+ ret.push_back(pos.get_pin(c));
return typename LBABtree::iterate_repeat_ret_inner(
interruptible::ready_future_marker{},
seastar::stop_iteration::no);
} else {
TRACET("{} got {}, {}",
c.trans, offset, iter.get_key(), iter.get_val());
- auto e = iter.get_pin();
+ auto e = iter.get_pin(c);
return get_mapping_ret(
interruptible::ready_future_marker{},
std::move(e));
state.ret = iter;
});
});
- }).si_then([](auto &&state) {
- return state.ret->get_pin();
+ }).si_then([c](auto &&state) {
+ return state.ret->get_pin(c);
});
}
iter.get_val().paddr == logn->get_paddr()) {
assert(!iter.get_leaf_node()->is_pending());
iter.get_leaf_node()->link_child(logn.get(), iter.get_leaf_pos());
- logn->set_pin(iter.get_pin());
+ logn->set_pin(iter.get_pin(c));
ceph_assert(iter.get_val().len == e->get_length());
if (c.pins) {
c.pins->add_pin(
class BtreeLBAPin : public BtreeNodePin<laddr_t, paddr_t> {
public:
- BtreeLBAPin() = default;
+ BtreeLBAPin(op_context_t<laddr_t> ctx)
+ : BtreeNodePin(ctx) {}
BtreeLBAPin(
+ op_context_t<laddr_t> c,
CachedExtentRef parent,
uint16_t pos,
lba_map_val_t &val,
lba_node_meta_t &&meta)
: BtreeNodePin(
+ c,
parent,
pos,
val.paddr,
using context_t = ObjectDataHandler::context_t;
using get_iertr = ObjectDataHandler::write_iertr;
-auto read_pin(
- context_t ctx,
- LBAPinRef pin) {
- return ctx.tm.pin_to_extent<ObjectDataBlock>(
- ctx.t,
- std::move(pin)
- ).handle_error_interruptible(
- get_iertr::pass_further{},
- crimson::ct_error::assert_all{ "read_pin: invalid error" }
- );
-}
-
/**
* extent_to_write_t
*
std::nullopt,
std::nullopt);
} else {
- return read_pin(ctx, pin->duplicate()
+ return ctx.tm.read_pin<ObjectDataBlock>(
+ ctx.t, pin->duplicate()
).si_then([prepend_len](auto left_extent) {
return get_iertr::make_ready_future<operate_ret_bare>(
std::nullopt,
left_to_write_extent,
std::nullopt);
} else {
- return read_pin(ctx, pin->duplicate()
+ return ctx.tm.read_pin<ObjectDataBlock>(
+ ctx.t, pin->duplicate()
).si_then([prepend_offset=extent_len, prepend_len,
left_to_write_extent=std::move(left_to_write_extent)]
(auto left_extent) mutable {
std::nullopt);
} else {
auto append_offset = overwrite_plan.data_end - right_pin_begin;
- return read_pin(ctx, pin->duplicate()
+ return ctx.tm.read_pin<ObjectDataBlock>(
+ ctx.t, pin->duplicate()
).si_then([append_offset, append_len](auto right_extent) {
return get_iertr::make_ready_future<operate_ret_bare>(
std::nullopt,
std::nullopt);
} else {
auto append_offset = overwrite_plan.data_end - right_pin_begin;
- return read_pin(ctx, pin->duplicate()
+ return ctx.tm.read_pin<ObjectDataBlock>(
+ ctx.t, pin->duplicate()
).si_then([append_offset, append_len,
right_to_write_extent=std::move(right_to_write_extent)]
(auto right_extent) mutable {
} else {
/* First pin overlaps the boundary and has data, read in extent
* and rewrite portion prior to size */
- return read_pin(
- ctx,
+ return ctx.tm.read_pin<ObjectDataBlock>(
+ ctx.t,
pin.duplicate()
).si_then([ctx, size, pin_offset, &pin, &object_data, &to_write](
auto extent) {
current = end;
return seastar::now();
} else {
- return ctx.tm.pin_to_extent<ObjectDataBlock>(
+ return ctx.tm.read_pin<ObjectDataBlock>(
ctx.t,
std::move(pin)
).si_then([&ret, ¤t, end](auto extent) {
// Only extent split can happen during the lookup
ceph_assert(pin_seg_paddr >= paddr &&
pin_seg_paddr.add_offset(pin_len) <= paddr.add_offset(len));
- return pin_to_extent_by_type(t, std::move(pin), type
+ return read_pin_by_type(t, std::move(pin), type
).si_then([&list](auto ret) {
list.emplace_back(std::move(ret));
return seastar::now();
t, offset, length);
}
- /**
- * pin_to_extent
- *
- * Get extent mapped at pin.
- */
- using pin_to_extent_iertr = base_iertr;
- template <typename T>
- using pin_to_extent_ret = pin_to_extent_iertr::future<
- TCachedExtentRef<T>>;
- template <typename T>
- pin_to_extent_ret<T> pin_to_extent(
- Transaction &t,
- LBAPinRef pin) {
- LOG_PREFIX(TransactionManager::pin_to_extent);
- SUBTRACET(seastore_tm, "getting extent {}", t, *pin);
- static_assert(is_logical_type(T::TYPE));
- using ret = pin_to_extent_ret<T>;
- auto &pref = *pin;
- return cache->get_absent_extent<T>(
- t,
- pref.get_val(),
- pref.get_length(),
- [this, pin=std::move(pin)](T &extent) mutable {
- assert(!extent.has_pin());
- assert(!extent.has_been_invalidated());
- assert(!pin->has_been_invalidated());
- assert(pin->get_parent());
- extent.set_pin(std::move(pin));
- lba_manager->add_pin(extent.get_pin());
- }
- ).si_then([FNAME, &t](auto ref) mutable -> ret {
- SUBTRACET(seastore_tm, "got extent -- {}", t, *ref);
- return pin_to_extent_ret<T>(
- interruptible::ready_future_marker{},
- std::move(ref));
- });
- }
-
- /**
- * pin_to_extent_by_type
- *
- * Get extent mapped at pin.
- */
- using pin_to_extent_by_type_ret = pin_to_extent_iertr::future<
- LogicalCachedExtentRef>;
- pin_to_extent_by_type_ret pin_to_extent_by_type(
- Transaction &t,
- LBAPinRef pin,
- extent_types_t type) {
- LOG_PREFIX(TransactionManager::pin_to_extent_by_type);
- SUBTRACET(seastore_tm, "getting extent {} type {}", t, *pin, type);
- assert(is_logical_type(type));
- auto &pref = *pin;
- return cache->get_absent_extent_by_type(
- t,
- type,
- pref.get_val(),
- pref.get_key(),
- pref.get_length(),
- [this, pin=std::move(pin)](CachedExtent &extent) mutable {
- auto &lextent = static_cast<LogicalCachedExtent&>(extent);
- assert(!lextent.has_pin());
- assert(!lextent.has_been_invalidated());
- assert(!pin->has_been_invalidated());
- lextent.set_pin(std::move(pin));
- lba_manager->add_pin(lextent.get_pin());
- }
- ).si_then([FNAME, &t](auto ref) {
- SUBTRACET(seastore_tm, "got extent -- {}", t, *ref);
- return pin_to_extent_by_type_ret(
- interruptible::ready_future_marker{},
- std::move(ref->template cast<LogicalCachedExtent>()));
- });
- }
-
/**
* read_extent
*
SUBTRACET(seastore_tm, "{}~{}", t, offset, length);
return get_pin(
t, offset
- ).si_then([this, FNAME, &t, offset, length] (auto pin) {
+ ).si_then([this, FNAME, &t, offset, length] (auto pin)
+ -> read_extent_ret<T> {
if (length != pin->get_length() || !pin->get_val().is_real()) {
SUBERRORT(seastore_tm,
"offset {} len {} got wrong pin {}",
t, offset, length, *pin);
ceph_assert(0 == "Should be impossible");
}
- return this->pin_to_extent<T>(t, std::move(pin));
+ return this->read_pin<T>(t, std::move(pin));
});
}
SUBTRACET(seastore_tm, "{}", t, offset);
return get_pin(
t, offset
- ).si_then([this, FNAME, &t, offset] (auto pin) {
+ ).si_then([this, FNAME, &t, offset] (auto pin)
+ -> read_extent_ret<T> {
if (!pin->get_val().is_real()) {
SUBERRORT(seastore_tm,
"offset {} got wrong pin {}",
t, offset, *pin);
ceph_assert(0 == "Should be impossible");
}
- return this->pin_to_extent<T>(t, std::move(pin));
+ return this->read_pin<T>(t, std::move(pin));
});
}
+ template <typename T>
+ base_iertr::future<TCachedExtentRef<T>> read_pin(
+ Transaction &t,
+ LBAMappingRef pin)
+ {
+ auto v = pin->get_logical_extent(t);
+ if (v.has_child()) {
+ return v.get_child_fut().then([](auto extent) {
+ return extent->template cast<T>();
+ });
+ } else {
+ return pin_to_extent<T>(t, std::move(pin));
+ }
+ }
+
+ base_iertr::future<LogicalCachedExtentRef> read_pin_by_type(
+ Transaction &t,
+ LBAMappingRef pin,
+ extent_types_t type)
+ {
+ auto v = pin->get_logical_extent(t);
+ if (v.has_child()) {
+ return std::move(v.get_child_fut());
+ } else {
+ return pin_to_extent_by_type(t, std::move(pin), type);
+ }
+ }
+
/// Obtain mutable copy of extent
LogicalCachedExtentRef get_mutable_extent(Transaction &t, LogicalCachedExtentRef ref) {
LOG_PREFIX(TransactionManager::get_mutable_extent);
ExtentPlacementManager::dispatch_result_t dispatch_result,
std::optional<journal_seq_t> seq_to_trim = std::nullopt);
+ /**
+ * pin_to_extent
+ *
+ * Get extent mapped at pin.
+ */
+ using pin_to_extent_iertr = base_iertr;
+ template <typename T>
+ using pin_to_extent_ret = pin_to_extent_iertr::future<
+ TCachedExtentRef<T>>;
+ template <typename T>
+ pin_to_extent_ret<T> pin_to_extent(
+ Transaction &t,
+ LBAMappingRef pin) {
+ LOG_PREFIX(TransactionManager::pin_to_extent);
+ SUBTRACET(seastore_tm, "getting extent {}", t, *pin);
+ static_assert(is_logical_type(T::TYPE));
+ using ret = pin_to_extent_ret<T>;
+ auto &pref = *pin;
+ return cache->get_absent_extent<T>(
+ t,
+ pref.get_val(),
+ pref.get_length(),
+ [pin=std::move(pin)]
+ (T &extent) mutable {
+ assert(!extent.has_laddr());
+ assert(!extent.has_been_invalidated());
+ assert(!pin->has_been_invalidated());
+ assert(pin->get_parent());
+ pin->link_child(&extent);
+ extent.set_laddr(pin->get_key());
+ }
+ ).si_then([FNAME, &t](auto ref) mutable -> ret {
+ SUBTRACET(seastore_tm, "got extent -- {}", t, *ref);
+ return pin_to_extent_ret<T>(
+ interruptible::ready_future_marker{},
+ std::move(ref));
+ });
+ }
+
+ /**
+ * pin_to_extent_by_type
+ *
+ * Get extent mapped at pin.
+ */
+ using pin_to_extent_by_type_ret = pin_to_extent_iertr::future<
+ LogicalCachedExtentRef>;
+ pin_to_extent_by_type_ret pin_to_extent_by_type(
+ Transaction &t,
+ LBAMappingRef pin,
+ extent_types_t type)
+ {
+ LOG_PREFIX(TransactionManager::pin_to_extent_by_type);
+ SUBTRACET(seastore_tm, "getting extent {} type {}", t, *pin, type);
+ assert(is_logical_type(type));
+ auto &pref = *pin;
+ return cache->get_absent_extent_by_type(
+ t,
+ type,
+ pref.get_val(),
+ pref.get_key(),
+ pref.get_length(),
+ [pin=std::move(pin)](CachedExtent &extent) mutable {
+ auto &lextent = static_cast<LogicalCachedExtent&>(extent);
+ assert(!lextent.has_laddr());
+ assert(!lextent.has_been_invalidated());
+ assert(!pin->has_been_invalidated());
+ assert(pin->get_parent());
+ assert(!pin->get_parent()->is_pending());
+ pin->link_child(&lextent);
+ lextent.set_pin(std::move(pin));
+ lba_manager->add_pin(lextent.get_pin());
+ }
+ ).si_then([FNAME, &t](auto ref) {
+ SUBTRACET(seastore_tm, "got extent -- {}", t, *ref);
+ return pin_to_extent_by_type_ret(
+ interruptible::ready_future_marker{},
+ std::move(ref->template cast<LogicalCachedExtent>()));
+ });
+ }
+
+
public:
// Testing interfaces
auto get_epm() {