#include "super.h"
#include "tree_types.h"
-namespace crimson::os::seastore::onode {
-
/**
- * in-memory subtree management:
+ * Tree example (2 levels):
*
- * resource management (bottom-up):
- * USER --> Ref<tree_cursor_t>
- * tree_cursor_t --> Ref<LeafNode>
- * Node (child) --> Ref<InternalNode> (see parent_info_t)
- * Node (root) --> Super::URef
- * Super --> Btree
+ * Root node keys: [ 3 7 ]
+ * values: [p1 p2 p3]
+ * / | \
+ * ------- | -------
+ * | | |
+ * V V V
+ * Leaf node keys: [ 1 2 3] [ 4 5 7] [ 9 11 12]
+ * values: [v1 v2 v3] [v4 v5 v6] [v7 v8 v9]
*
- * tracked lookup (top-down):
- * Btree --> Super*
- * Super --> Node* (root)
- * InternalNode --> Node* (children)
- * LeafNode --> tree_cursor_t*
+ * Tree structure properties:
+ * - As illustrated above, the parent key is strictly equal to its left child's
+ * largest key;
+ * - If a tree is indexing multiple seastore transactions, each transaction
+ * will be mapped to a Super which points to a distinct root node. So the
+ * transactions are isolated at tree level. However, tree nodes from
+ * different transactions can reference the same seastore CachedExtent before
+ * modification;
+ * - The resources of the transactional tree are tracked by tree_cursor_ts held
+ * by users. As long as any cursor is alive, the according tree hierarchy is
+ * alive and keeps tracked. See the reversed resource management sections
+ * below;
*/
+namespace crimson::os::seastore::onode {
+
class LeafNode;
class InternalNode;
+/**
+ * tree_cursor_t
+ *
+ * A cursor points to a position (LeafNode and search_position_t) of the tree
+ * where it can find the according key and value pair. The position is updated
+ * by LeafNode insert/split/delete/merge internally and is kept valid. It also
+ * caches the key-value information for a specific node layout version.
+ *
+ * Exposes public interfaces for Btree::Cursor.
+ */
using layout_version_t = uint32_t;
class tree_cursor_t final
: public boost::intrusive_ref_counter<
tree_cursor_t& operator=(const tree_cursor_t&) = delete;
tree_cursor_t& operator=(tree_cursor_t&&) = delete;
+ /**
+ * is_end
+ *
+ * Represents one-past-the-last of all the sorted key-value
+ * pairs in the tree. An end cursor won't contain valid key-value
+ * information.
+ */
bool is_end() const { return position.is_end(); }
+
+ /// Returns the key view in tree if it is not an end cursor.
const key_view_t& get_key_view() const;
+
+ /// Returns the value pointer in tree if it is not an end cursor.
const onode_t* get_p_value() const;
private:
void ensure_kv() const;
private:
+ /**
+ * Reversed resource management (tree_cursor_t)
+ *
+ * tree_cursor_t holds a reference to the LeafNode, so the LeafNode will be
+ * alive as long as any of it's cursors is still referenced by user.
+ */
Ref<LeafNode> leaf_node;
search_position_t position;
+
// cached information
mutable std::optional<key_view_t> key_view;
mutable const onode_t* p_value;
friend class Node; // get_position(), get_leaf_node()
};
+/**
+ * Node
+ *
+ * An abstracted class for both InternalNode and LeafNode.
+ *
+ * Exposes public interfaces for Btree.
+ */
class Node
: public boost::intrusive_ref_counter<
Node, boost::thread_unsafe_counter> {
crimson::ct_error::erange>;
template <class ValueT=void>
using node_future = node_ertr::future<ValueT>;
+
struct search_result_t {
bool is_end() const { return p_cursor->is_end(); }
Ref<tree_cursor_t> p_cursor;
Node& operator=(const Node&) = delete;
Node& operator=(Node&&) = delete;
+ /**
+ * level
+ *
+ * A positive value denotes the level (or height) of this node in tree.
+ * 0 means LeafNode, positive means InternalNode.
+ */
level_t level() const;
+
+ /**
+ * lookup_smallest
+ *
+ * Returns a cursor pointing to the smallest key in the sub-tree formed by
+ * this node.
+ *
+ * Returns an end cursor if it is an empty root node.
+ */
virtual node_future<Ref<tree_cursor_t>> lookup_smallest(context_t) = 0;
+
+ /**
+ * lookup_largest
+ *
+ * Returns a cursor pointing to the largest key in the sub-tree formed by
+ * this node.
+ *
+ * Returns an end cursor if it is an empty root node.
+ */
virtual node_future<Ref<tree_cursor_t>> lookup_largest(context_t) = 0;
+
+ /**
+ * lower_bound
+ *
+ * Returns a cursor pointing to the first element in the range [first, last)
+ * of the sub-tree which does not compare less than the input key. The
+ * result also denotes whether the pointed key is equal to the input key.
+ *
+ * Returns an end cursor with MatchKindBS::NE if:
+ * - It is an empty root node;
+ * - Or the input key is larger than all the keys in the sub-tree;
+ */
node_future<search_result_t> lower_bound(context_t c, const key_hobj_t& key);
+
+ /**
+ * insert
+ *
+ * Try to insert a key-value pair into the sub-tree formed by this node.
+ *
+ * Returns a boolean denoting whether the insertion is successful:
+ * - If true, the returned cursor points to the inserted element in tree;
+ * - If false, the returned cursor points to the conflicting element in tree;
+ */
node_future<std::pair<Ref<tree_cursor_t>, bool>> insert(
context_t, const key_hobj_t&, const onode_t&);
+
+ /// Recursively collects the statistics of the sub-tree formed by this node
node_future<tree_stats_t> get_tree_stats(context_t);
+
+ /// Returns an ostream containing a dump of all the elements in the node.
std::ostream& dump(std::ostream&) const;
- std::ostream& dump_brief(std::ostream&) const;
- void test_make_destructable(context_t, NodeExtentMutable&, Super::URef&&);
- virtual node_future<> test_clone_root(context_t, RootNodeTracker&) const = 0;
+ /// Returns an ostream containing an one-line summary of this node.
+ std::ostream& dump_brief(std::ostream&) const;
+ /// Initializes the tree by allocating an empty root node.
static node_future<> mkfs(context_t, RootNodeTracker&);
+
+ /// Loads the tree root. The tree must be initialized.
static node_future<Ref<Node>> load_root(context_t, RootNodeTracker&);
+ // Only for unit test purposes.
+ void test_make_destructable(context_t, NodeExtentMutable&, Super::URef&&);
+ virtual node_future<> test_clone_root(context_t, RootNodeTracker&) const = 0;
+
protected:
virtual node_future<> test_clone_non_root(context_t, Ref<InternalNode>) const {
ceph_abort("impossible path");
node_future<> insert_parent(context_t, Ref<Node> right_node);
private:
+ /**
+ * Reversed resource management (Node)
+ *
+ * Root Node holds a reference to its parent Super class, so its parent
+ * will be alive as long as this root node is alive.
+ *
+ * None-root Node holds a reference to its parent Node, so its parent will
+ * be alive as long as any of it's children is alive.
+ */
// as root
Super::URef super;
// as child/non-root
return node.dump_brief(os);
}
+/**
+ * InternalNode
+ *
+ * A concrete implementation of Node class that represents an internal tree
+ * node. Its level is always positive and its values are logical block
+ * addresses to its child nodes. An internal node cannot be empty.
+ */
class InternalNode final : public Node {
public:
// public to Node
static node_future<fresh_node_t> allocate(context_t, field_type_t, bool, level_t);
private:
+ /**
+ * Reversed resource management (InternalNode)
+ *
+ * InteralNode keeps track of its child nodes which are still alive in
+ * memory, and their positions will be updated throughout
+ * insert/split/delete/merge operations of this node.
+ */
// XXX: leverage intrusive data structure to control memory overhead
- // track the current living child nodes by position (can be end)
std::map<search_position_t, Node*> tracked_child_nodes;
InternalNodeImpl* impl;
};
+/**
+ * LeafNode
+ *
+ * A concrete implementation of Node class that represents a leaf tree node.
+ * Its level is always 0. A leaf node can only be empty if it is root.
+ */
class LeafNode final : public Node {
public:
// public to tree_cursor_t
static node_future<fresh_node_t> allocate(context_t, field_type_t, bool);
private:
+ /**
+ * Reversed resource management (LeafNode)
+ *
+ * LeafNode keeps track of the referencing cursors which are still alive in
+ * memory, and their positions will be updated throughout
+ * insert/split/delete/merge operations of this node.
+ */
// XXX: leverage intrusive data structure to control memory overhead
- // track the current living cursors by position (cannot be end)
std::map<search_position_t, tree_cursor_t*> tracked_cursors;
LeafNodeImpl* impl;
layout_version_t layout_version = 0;
namespace crimson::os::seastore::onode {
+/**
+ * DeltaRecorder
+ *
+ * An abstracted class to encapsulate different implementations to apply delta
+ * to a specific node layout.
+ */
class DeltaRecorder {
public:
virtual ~DeltaRecorder() {
namespace crimson::os::seastore::onode {
+/**
+ * DeltaRecorderT
+ *
+ * Responsible to encode and decode delta, and apply delta for a specific node
+ * layout.
+ */
template <typename FieldType, node_type_t NODE_TYPE>
class DeltaRecorderT final: public DeltaRecorder {
public:
/**
* NodeExtentAccessorT
*
- * This component is responsible to mutate the underlying NodeExtent, record
- * mutation parameters as delta and apply the recorded modifications to the
- * NodeExtent. It contains static information about node and field type in
- * order to call methods of NodeLayoutReplayable.
+ * This component is responsible to reference and mutate the underlying
+ * NodeExtent, record mutation parameters when needed, and apply the recorded
+ * modifications for a specific node layout.
*/
template <typename FieldType, node_type_t NODE_TYPE>
class NodeExtentAccessorT {
#include "node_extent_mutable.h"
#include "node_types.h"
+/**
+ * node_extent_manager.h
+ *
+ * Contains general interfaces for different backends (Dummy and Seastore).
+ */
+
namespace crimson::os::seastore::onode {
using crimson::os::seastore::LogicalCachedExtent;
}
/**
- * abstracted:
+ * Abstracted interfaces to implement:
* - CacheExtent::duplicate_for_write() -> CachedExtentRef
* - CacheExtent::get_type() -> extent_types_t
* - CacheExtent::get_delta() -> ceph::bufferlist
#include "crimson/common/log.h"
#include "crimson/os/seastore/onode_manager/staged-fltree/node_extent_manager.h"
+/**
+ * dummy.h
+ *
+ * Dummy backend implementations for test purposes.
+ */
+
namespace crimson::os::seastore::onode {
class DummySuper final: public Super {
#include "crimson/os/seastore/onode_manager/staged-fltree/node_extent_manager.h"
#include "crimson/os/seastore/onode_manager/staged-fltree/node_delta_recorder.h"
+/**
+ * seastore.h
+ *
+ * Seastore backend implementations.
+ */
+
namespace crimson::os::seastore::onode {
class SeastoreSuper final: public Super {
class NodeExtent;
-// the wrapper of NodeExtent which is mutable and safe to be mutated
+/**
+ * NodeExtentMutable
+ *
+ * A thin wrapper of NodeExtent to make sure that only the newly allocated
+ * or the duplicated NodeExtent is mutable, and the memory modifications are
+ * safe within the extent range.
+ */
class NodeExtentMutable {
public:
void copy_in_absolute(void* dst, const void* src, extent_len_t len) {
struct key_view_t;
class NodeExtentMutable;
+/**
+ * NodeImpl
+ *
+ * Hides type specific node layout implementations for Node.
+ */
class NodeImpl {
public:
using alloc_ertr = crimson::errorator<
NodeImpl() = default;
};
+/**
+ * InternalNodeImpl
+ *
+ * Hides type specific node layout implementations for InternalNode.
+ */
class InternalNodeImpl : public NodeImpl {
public:
struct internal_marker_t {};
InternalNodeImpl() = default;
};
+/**
+ * LeafNodeImpl
+ *
+ * Hides type specific node layout implementations for LeafNode.
+ */
class LeafNodeImpl : public NodeImpl {
public:
struct leaf_marker_t {};
template <> struct node_marker_type<node_type_t::LEAF> {
using type = LeafNodeImpl::leaf_marker_t; };
+/**
+ * NodeLayoutT
+ *
+ * Contains templated and concrete implementations for both InternalNodeImpl
+ * and LeafNodeImpl under a specific node layout.
+ */
template <typename FieldType, node_type_t NODE_TYPE>
class NodeLayoutT final : public InternalNodeImpl, public LeafNodeImpl {
public:
using alloc_ertr = NodeExtentManager::tm_ertr;
static alloc_ertr::future<typename parent_t::fresh_impl_t> allocate(
context_t c, bool is_level_tail, level_t level) {
- // NOTE:
- // *option1: all types of node have the same length;
- // option2: length is defined by node/field types;
- // option3: length is totally flexible;
+ // NOTE: Currently, all the node types have the same size for simplicity.
+ // But depending on the requirement, we may need to make node size
+ // configurable by field_type_t and node_type_t, or totally flexible.
return c.nm.alloc_extent(c.t, node_stage_t::EXTENT_SIZE
).safe_then([is_level_tail, level](auto extent) {
assert(extent->is_initial_pending());
namespace crimson::os::seastore::onode {
+/**
+ * NodeLayoutReplayableT
+ *
+ * Contains templated logics to modify the layout of a NodeExtend which are
+ * also replayable. Used by NodeExtentAccessorT at runtime and by
+ * DeltaRecorderT during replay.
+ */
template <typename FieldType, node_type_t NODE_TYPE>
struct NodeLayoutReplayableT {
using node_stage_t = node_extent_t<FieldType, NODE_TYPE>;
assert(NODE_TYPE == node_type_t::INTERNAL);
mut.copy_in_absolute(p_addr, new_addr);
}
-
};
}
class NodeExtentMutable;
-/*
- * internal/leaf node N0, N1
+/**
+ * item_iterator_t
*
- * (_index)
- * p_items_start
- * | item_range ------------+
- * | | +----key---------+
- * | | | |
- * V V V V
- * | |sub |oid char|ns char|colli-| |
- * |...|items|array & |array &|-sion |...|
- * | |... |len |len |offset| |
- * ^ |
- * | |
- * +---- back_offset -----+
+ * The STAGE_STRING implementation for node N0/N1, implements staged contract
+ * as an iterative container to resolve crush hash conflicts.
+ *
+ * The layout of the contaner to index ns, oid strings storing n items:
+ *
+ * # <--------- container range ---------> #
+ * #<~># items [i+1, n) #
+ * # # items [0, i) #<~>#
+ * # # <------ item i -------------> # #
+ * # # <--- item_range ---> | # #
+ * # # | # #
+ * # # next-stage | ns-oid | back_ # #
+ * # # contaner | strings | offset # #
+ * #...# range | | #...#
+ * ^ ^ |
+ * | | |
+ * | +---------------------------+
+ * + p_items_start
*/
template <node_type_t NODE_TYPE>
class item_iterator_t {
return toMatchKindCMP(l.gen, r.gen);
}
+/**
+ * string_key_view_t
+ *
+ * The layout to store char array as an oid or an ns string which may be
+ * compressed.
+ *
+ * If compressed, the physical block only stores an unsigned int of
+ * string_size_t, with value 0 denoting Type::MIN, and value max() denoting
+ * Type::MAX.
+ *
+ * If not compressed (Type::STR), the physical block stores the char array and
+ * a valid string_size_t value.
+ */
struct string_key_view_t {
enum class Type {MIN, STR, MAX};
// presumably the maximum string length is 2KiB
string_size_t length;
};
+/**
+ * string_view_masked_t
+ *
+ * A common class to hide the underlying string implementation regardless of a
+ * string_key_view_t (maybe compressed), a string/string_view, or a compressed
+ * string. And leverage this consistant class to do compare, print, convert and
+ * append operations.
+ */
class string_view_masked_t {
public:
using Type = string_key_view_t::Type;
string_view_masked_t{r.oid});
}
+/**
+ * key_hobj_t
+ *
+ * A specialized implementation of a full_key_t storing a ghobject_t passed
+ * from user.
+ */
class key_hobj_t {
public:
explicit key_hobj_t(const ghobject_t& ghobj) : ghobj{ghobj} {}
-
/*
- * common interface as full_key_t
+ * common interfaces as a full_key_t
*/
shard_t shard() const {
return ghobj.shard_id;
return key.dump(os);
}
+/**
+ * key_view_t
+ *
+ * A specialized implementation of a full_key_t pointing to the locations
+ * storing the full key in a tree node.
+ */
class key_view_t {
public:
- /*
- * common interface as full_key_t
+ /**
+ * common interfaces as a full_key_t
*/
shard_t shard() const {
return shard_pool_packed().shard;
return !operator==(o);
}
- /*
+ /**
* key_view_t specific interfaces
*/
-
bool has_shard_pool() const {
return p_shard_pool != nullptr;
}
class NodeExtentMutable;
+/**
+ * node_extent_t
+ *
+ * The top indexing stage implementation for node N0/N1/N2/N3, implements
+ * staged contract as an indexable container, and provides access to node
+ * header.
+ *
+ * The specific field layout are defined by FieldType which are
+ * node_fields_0_t, node_fields_1_t, node_fields_2_t, internal_fields_3_t and
+ * leaf_fields_3_t. Diagrams see node_stage_layout.h.
+ */
template <typename FieldType, node_type_t _NODE_TYPE>
class node_extent_t {
public:
return {offset_start, offset_end};
}
-// internal/leaf node N0, N1; leaf node N3
+/**
+ * _node_fields_013_t (node_fields_0_t, node_fields_1_t, leaf_fields_3_t
+ *
+ * The STAGE_LEFT layout implementation for node N0/N1, or the STAGE_RIGHT
+ * layout implementation for leaf node N3.
+ *
+ * The node layout storing n slots:
+ *
+ * # <----------------------------- node range --------------------------------------> #
+ * # #<~># free space #
+ * # <----- left part -----------------------------> # <~# <----- right slots -------> #
+ * # # <---- left slots -------------> #~> # #
+ * # # slots [2, n) |<~># #<~>| right slots [2, n) #
+ * # # <- slot 0 -> | <- slot 1 -> | # # | <-- s1 --> | <-- s0 --> #
+ * # # | | # # | | #
+ * # | num_ # | right | | right | # # | next-stage | next-stage #
+ * # header | keys # key | offset | key | offset | # # | container | container #
+ * # | # 0 | 0 | 1 | 1 |...#...#...| or onode 1 | or onode 0 #
+ * | | ^ ^
+ * | | | |
+ * | +----------------+ |
+ * +--------------------------------------------+
+ */
template <typename SlotType>
struct _node_fields_013_t {
// TODO: decide by NODE_BLOCK_SIZE, sizeof(SlotType), sizeof(laddr_t)
using node_fields_0_t = _node_fields_013_t<slot_0_t>;
using node_fields_1_t = _node_fields_013_t<slot_1_t>;
-// internal/leaf node N2
+/**
+ * node_fields_2_t
+ *
+ * The STAGE_STRING layout implementation for node N2.
+ *
+ * The node layout storing n slots:
+ *
+ * # <--------------------------------- node range ----------------------------------------> #
+ * # #<~># free space #
+ * # <------- left part ---------------> # <~# <--------- right slots ---------------------> #
+ * # # <---- offsets ----> #~> #<~>| slots [2, n) #
+ * # # offsets [2, n) |<~># # | <----- slot 1 ----> | <----- slot 0 ----> #
+ * # # | # # | | #
+ * # | num_ # offset | offset | # # | next-stage | ns-oid | next-stage | ns-oid #
+ * # header | keys # 0 | 1 |...#...#...| container1 | 1 | container0 | 0 #
+ * | | ^ ^
+ * | | | |
+ * | +----------------+ |
+ * +-----------------------------------------------+
+ */
struct node_fields_2_t {
// TODO: decide by NODE_BLOCK_SIZE, sizeof(node_off_t), sizeof(laddr_t)
// and the minimal size of variable_key.
node_offset_t offsets[];
} __attribute__((packed));
+/**
+ * internal_fields_3_t
+ *
+ * The STAGE_RIGHT layout implementation for N2.
+ *
+ * The node layout storing 3 children:
+ *
+ * # <---------------- node range ---------------------------> #
+ * # # <-- keys ---> # <---- laddrs -----------> #
+ * # free space: # |<~># |<~>#
+ * # # | # | #
+ * # | num_ # key | key | # laddr | laddr | laddr | #
+ * # header | keys # 0 | 1 |...# 0 | 1 | 2 |...#
+ */
// TODO: decide by NODE_BLOCK_SIZE, sizeof(snap_gen_t), sizeof(laddr_t)
static constexpr unsigned MAX_NUM_KEYS_I3 = 170u;
template <unsigned MAX_NUM_KEYS>
using leaf_fields_3_t = _node_fields_013_t<slot_3_t>;
-/*
- * block layout of a variable-sized item (right-side)
- *
- * for internal node type 0, 1:
- * previous off (block boundary) -----------------------------+
- * current off --+ |
- * | |
- * V V
- * <==== | sub |fix|sub |fix|oid char|ns char|colli-|
- * (next-item) |...addr|key|addr|key|array & |array &|-sion |(prv-item)...
- * <==== | 1 |1 |0 |0 |len |len |offset|
- * ^ |
- * | |
- * +------------ next collision ----------+
- * see item_iterator_t<node_type_t::INTERNAL>
- *
- * for internal node type 2:
- * previous off (block boundary) ----------------------+
- * current off --+ |
- * | |
- * V V
- * <==== | sub |fix|sub |fix|oid char|ns char|
- * (next-item) |...addr|key|addr|key|array & |array &|(prv-item)...
- * <==== | 1 |1 |0 |0 |len |len |
- * see sub_items_t<node_type_t::INTERNAL>
- *
- * for leaf node type 0, 1:
- * previous off (block boundary) ----------------------------------------+
- * current off --+ |
- * | |
- * V V
- * <==== | fix|o- |fix| off|off|num |oid char|ns char|colli-|
- * (next-item) |...key|node|key|...set|set|sub |array & |array &|-sion |(prv-item)
- * <==== | 1 |0 |0 | 1 |0 |keys|len |len |offset|
- * ^ |
- * | |
- * +------------ next collision ----------------------+
- * see item_iterator_t<node_type_t::LEAF>
- *
- * for leaf node type 2:
- * previous off (block boundary) ---------------------------------+
- * current off --+ |
- * | |
- * V V
- * <==== | fix|o- |fix| off|off|num |oid char|ns char|
- * (next-item) |...key|node|key|...set|set|sub |array & |array &|(prv-item)
- * <==== | 1 |0 |0 | 1 |0 |keys|len |len |
- * see sub_items_t<node_type_t::LEAF>
- */
-
}
enum class TrimType { BEFORE, AFTER, AT };
+/**
+ * staged
+ *
+ * Implements recursive logic that modifies or reads the node layout
+ * (N0/N1/N2/N3 * LEAF/INTERNAL) with the multi-stage design. The specific
+ * stage implementation is flexible. So the implementations for different
+ * stages can be assembled independently, as long as they follow the
+ * definitions of container interfaces.
+ *
+ * Multi-stage is designed to index different portions of onode keys
+ * stage-by-stage. There are at most 3 stages for a node:
+ * - STAGE_LEFT: index shard-pool-crush for N0, or index crush for N1 node;
+ * - STAGE_STRING: index ns-oid for N0/N1/N2 nodes;
+ * - STAGE_RIGHT: index snap-gen for N0/N1/N2/N3 nodes;
+ *
+ * The intention is to consolidate the high-level indexing implementations at
+ * the level of stage, so we don't need to write them repeatedly for every
+ * stage and for every node type.
+ */
template <typename Params>
struct staged {
static_assert(Params::STAGE >= STAGE_BOTTOM);
}
};
-/*
- * staged infrastructure
+/**
+ * Configurations for struct staged
+ *
+ * staged_params_* assembles different container_t implementations (defined by
+ * stated::_iterator_t) by STAGE, and constructs the final multi-stage
+ * implementations for different node layouts defined by
+ * node_extent_t<FieldType, NODE_TYPE>.
+ *
+ * The specialized implementations for different layouts are accessible through
+ * the helper type node_to_stage_t<node_extent_t<FieldType, NODE_TYPE>>.
+ *
+ * Specifically, the settings of 8 layouts are:
+ *
+ * The layout (N0, LEAF/INTERNAL) has 3 stages:
+ * - STAGE_LEFT: node_extent_t<node_fields_0_t, LEAF/INTERNAL>
+ * - STAGE_STRING: item_iterator_t<LEAF/INTERNAL>
+ * - STAGE_RIGHT: sub_items_t<LEAF/INTERNAL>
+ *
+ * The layout (N1, LEAF/INTERNAL) has 3 stages:
+ * - STAGE_LEFT: node_extent_t<node_fields_1_t, LEAF/INTERNAL>
+ * - STAGE_STRING: item_iterator_t<LEAF/INTERNAL>
+ * - STAGE_RIGHT: sub_items_t<LEAF/INTERNAL>
+ *
+ * The layout (N2, LEAF/INTERNAL) has 2 stages:
+ * - STAGE_STRING: node_extent_t<node_fields_2_t, LEAF/INTERNAL>
+ * - STAGE_RIGHT: sub_items_t<LEAF/INTERNAL>
+ *
+ * The layout (N3, LEAF) has 1 stage:
+ * - STAGE_RIGHT: node_extent_t<leaf_fields_3_t, LEAF>
+ *
+ * The layout (N3, INTERNAL) has 1 stage:
+ * - STAGE_RIGHT: node_extent_t<internal_fields_3_t, INTERNAL>
*/
template <node_type_t _NODE_TYPE>
laddr_packed_t value;
} __attribute__((packed));
-/*
- * internal node N0, N1, N2
+/**
+ * internal_sub_items_t
*
- * p_first_item +
- * (num_items) |
- * V
- * | fix|sub |fix|sub |
- * |...key|addr|key|addr|
- * | 1 |1 |0 |0 |
+ * The STAGE_RIGHT implementation for internal node N0/N1/N2, implements staged
+ * contract as an indexable container to index snap-gen to child node
+ * addresses.
+ *
+ * The layout of the contaner storing n sub-items:
+ *
+ * # <--------- container range -----------> #
+ * #<~># sub-items [2, n) #
+ * # # <- sub-item 1 -> # <- sub-item 0 -> #
+ * #...# snap-gen | laddr # snap-gen | laddr #
+ * ^
+ * |
+ * p_first_item +
*/
class internal_sub_items_t {
public:
char* p_append;
};
-/*
- * leaf node N0, N1, N2
+/**
+ * leaf_sub_items_t
+ *
+ * The STAGE_RIGHT implementation for leaf node N0/N1/N2, implements staged
+ * contract as an indexable container to index snap-gen to onode_t.
+ *
+ * The layout of the contaner storing n sub-items:
*
- * p_num_keys -----------------+
- * p_offsets --------------+ |
- * p_items_end -----+ | |
- * | | |
- * V V V
- * | fix|o- |fix| off|off|num |
- * |...key|node|key|...set|set|sub |
- * | 1 |0 |0 | 1 |0 |keys|
- * ^ | |
- * | | |
- * +--------+ <=====+
+ * # <------------------------ container range -------------------------------> #
+ * # <---------- sub-items ----------------> # <--- offsets ---------# #
+ * #<~># sub-items [2, n) #<~>| offsets [2, n) # #
+ * # # <- sub-item 1 -> # <- sub-item 0 -> # | # #
+ * #...# snap-gen | onode # snap-gen | onode #...| offset1 | offset0 # num_keys #
+ * ^ ^ ^
+ * | | |
+ * p_items_end + p_offsets + |
+ * p_num_keys +
*/
class leaf_sub_items_t {
public:
class Node;
class Super;
+
+/**
+ * RootNodeTracker
+ *
+ * An abstracted tracker to get the root node by Transaction.
+ */
class RootNodeTracker {
public:
virtual ~RootNodeTracker() = default;
friend class Super;
};
+/**
+ * Super
+ *
+ * The parent of root node. It contains the relationship between a Transaction
+ * and a root node address.
+ */
class Super {
public:
using URef = std::unique_ptr<Super>;
Node* tracked_root_node = nullptr;
};
+/**
+ * RootNodeTrackerIsolated
+ *
+ * A concrete RootNodeTracker implementation which provides root node isolation
+ * between Transactions for Seastore backend.
+ */
class RootNodeTrackerIsolated final : public RootNodeTracker {
public:
~RootNodeTrackerIsolated() override { assert(is_clean()); }
std::map<Transaction*, Super*> tracked_supers;
};
+/**
+ * RootNodeTrackerShared
+ *
+ * A concrete RootNodeTracker implementation which has no isolation between
+ * Transactions for Dummy backend.
+ */
class RootNodeTrackerShared final : public RootNodeTracker {
public:
~RootNodeTrackerShared() override { assert(is_clean()); }
#include "fwd.h"
#include "tree_types.h"
+/**
+ * tree.h
+ *
+ * An example implementation to expose tree interfaces to users. The current
+ * interface design is based on:
+ * - ceph::os::Transaction::create/touch/remove()
+ * - ceph::ObjectStore::collection_list()
+ * - ceph::BlueStore::get_onode()
+ * - db->get_iterator(PREFIIX_OBJ) by ceph::BlueStore::fsck()
+ *
+ * TODO: Redesign the interfaces based on real onode manager requirements.
+ */
+
namespace crimson::os::seastore::onode {
-/*
- * btree interfaces
- * requirements are based on:
- * ceph::os::Transaction::create/touch/remove()
- * ceph::ObjectStore::collection_list()
- * ceph::BlueStore::get_onode()
- * db->get_iterator(PREFIIX_OBJ) by ceph::BlueStore::fsck()
- */
class Node;
class Btree {
public:
namespace crimson::os::seastore::onode {
-// TODO: replace
+// TODO: Redesign according to real requirement from onode manager
struct onode_t {
// onode should be smaller than a node
uint16_t size; // address up to 64 KiB sized node
#include "tree.h"
/**
- * tree_utils
+ * tree_utils.h
*
* Contains shared logic for unit tests and perf tool.
*/