summaryrefslogtreecommitdiff
path: root/storage/tokudb/ft-index/ft/ft-ops.cc
diff options
context:
space:
mode:
Diffstat (limited to 'storage/tokudb/ft-index/ft/ft-ops.cc')
-rw-r--r--storage/tokudb/ft-index/ft/ft-ops.cc408
1 files changed, 366 insertions, 42 deletions
diff --git a/storage/tokudb/ft-index/ft/ft-ops.cc b/storage/tokudb/ft-index/ft/ft-ops.cc
index 64b6b498c9a..f9701ec34b1 100644
--- a/storage/tokudb/ft-index/ft/ft-ops.cc
+++ b/storage/tokudb/ft-index/ft/ft-ops.cc
@@ -367,6 +367,9 @@ status_init(void)
STATUS_INIT(FT_PRO_NUM_DIDNT_WANT_PROMOTE, PROMOTION_STOPPED_AFTER_LOCKING_CHILD, PARCOUNT, "promotion: stopped anyway, after locking the child", TOKU_ENGINE_STATUS|TOKU_GLOBAL_STATUS);
STATUS_INIT(FT_BASEMENT_DESERIALIZE_FIXED_KEYSIZE, BASEMENT_DESERIALIZATION_FIXED_KEY, PARCOUNT, "basement nodes deserialized with fixed-keysize", TOKU_ENGINE_STATUS|TOKU_GLOBAL_STATUS);
STATUS_INIT(FT_BASEMENT_DESERIALIZE_VARIABLE_KEYSIZE, BASEMENT_DESERIALIZATION_VARIABLE_KEY, PARCOUNT, "basement nodes deserialized with variable-keysize", TOKU_ENGINE_STATUS|TOKU_GLOBAL_STATUS);
+ STATUS_INIT(FT_PRO_RIGHTMOST_LEAF_SHORTCUT_SUCCESS, nullptr, PARCOUNT, "promotion: succeeded in using the rightmost leaf shortcut", TOKU_ENGINE_STATUS);
+ STATUS_INIT(FT_PRO_RIGHTMOST_LEAF_SHORTCUT_FAIL_POS, nullptr, PARCOUNT, "promotion: tried the rightmost leaf shorcut but failed (out-of-bounds)", TOKU_ENGINE_STATUS);
+ STATUS_INIT(FT_PRO_RIGHTMOST_LEAF_SHORTCUT_FAIL_REACTIVE,nullptr, PARCOUNT, "promotion: tried the rightmost leaf shorcut but failed (child reactive)", TOKU_ENGINE_STATUS);
ft_status.initialized = true;
}
@@ -890,6 +893,11 @@ void toku_ftnode_clone_callback(
for (int i = 0; i < node->n_children-1; i++) {
toku_clone_dbt(&cloned_node->childkeys[i], node->childkeys[i]);
}
+ if (node->height > 0) {
+ // need to move messages here so that we don't serialize stale
+ // messages to the fresh tree - ft verify code complains otherwise.
+ toku_move_ftnode_messages_to_stale(ft, node);
+ }
// clone partition
ftnode_clone_partitions(node, cloned_node);
@@ -932,11 +940,14 @@ void toku_ftnode_flush_callback(
int height = ftnode->height;
if (write_me) {
toku_assert_entire_node_in_memory(ftnode);
- if (height == 0) {
+ if (height > 0 && !is_clone) {
+ // cloned nodes already had their stale messages moved, see toku_ftnode_clone_callback()
+ toku_move_ftnode_messages_to_stale(h, ftnode);
+ } else if (height == 0) {
ft_leaf_run_gc(h, ftnode);
- }
- if (height == 0 && !is_clone) {
- ftnode_update_disk_stats(ftnode, h, for_checkpoint);
+ if (!is_clone) {
+ ftnode_update_disk_stats(ftnode, h, for_checkpoint);
+ }
}
int r = toku_serialize_ftnode_to(fd, ftnode->thisnodename, ftnode, ndd, !is_clone, h, for_checkpoint);
assert_zero(r);
@@ -1079,9 +1090,10 @@ exit:
return;
}
+static void ft_bnc_move_messages_to_stale(FT ft, NONLEAF_CHILDINFO bnc);
+
// replace the child buffer with a compressed version of itself.
-// @return the old child buffer
-static NONLEAF_CHILDINFO
+static void
compress_internal_node_partition(FTNODE node, int i, enum toku_compression_method compression_method)
{
// if we should evict, compress the
@@ -1092,11 +1104,9 @@ compress_internal_node_partition(FTNODE node, int i, enum toku_compression_metho
sub_block_init(sb);
toku_create_compressed_partition_from_available(node, i, compression_method, sb);
- // now set the state to compressed and return the old, available partition
- NONLEAF_CHILDINFO bnc = BNC(node, i);
+ // now set the state to compressed
set_BSB(node, i, sb);
BP_STATE(node,i) = PT_COMPRESSED;
- return bnc;
}
void toku_evict_bn_from_memory(FTNODE node, int childnum, FT h) {
@@ -1149,18 +1159,27 @@ int toku_ftnode_pe_callback(void *ftnode_pv, PAIR_ATTR old_attr, void *write_ext
for (int i = 0; i < node->n_children; i++) {
if (BP_STATE(node,i) == PT_AVAIL) {
if (BP_SHOULD_EVICT(node,i)) {
- NONLEAF_CHILDINFO bnc;
- if (ft_compress_buffers_before_eviction) {
- // When partially evicting, always compress with quicklz
- bnc = compress_internal_node_partition(
+ NONLEAF_CHILDINFO bnc = BNC(node, i);
+ if (ft_compress_buffers_before_eviction &&
+ // We may not serialize and compress a partition in memory if its
+ // in memory layout version is different than what's on disk (and
+ // therefore requires upgrade).
+ //
+ // Auto-upgrade code assumes that if a node's layout version read
+ // from disk is not current, it MUST require upgrade. Breaking
+ // this rule would cause upgrade code to upgrade this partition
+ // again after we serialize it as the current version, which is bad.
+ node->layout_version == node->layout_version_read_from_disk) {
+ ft_bnc_move_messages_to_stale(ft, bnc);
+ compress_internal_node_partition(
node,
i,
+ // Always compress with quicklz
TOKU_QUICKLZ_METHOD
);
} else {
// We're not compressing buffers before eviction. Simply
// detach the buffer and set the child's state to on-disk.
- bnc = BNC(node, i);
set_BNULL(node, i);
BP_STATE(node, i) = PT_ON_DISK;
}
@@ -1626,12 +1645,10 @@ ft_init_new_root(FT ft, FTNODE oldroot, FTNODE *newrootp)
BLOCKNUM old_blocknum = oldroot->thisnodename;
uint32_t old_fullhash = oldroot->fullhash;
- PAIR old_pair = oldroot->ct_pair;
int new_height = oldroot->height+1;
uint32_t new_fullhash;
BLOCKNUM new_blocknum;
- PAIR new_pair = NULL;
cachetable_put_empty_node_with_dep_nodes(
ft,
@@ -1641,7 +1658,6 @@ ft_init_new_root(FT ft, FTNODE oldroot, FTNODE *newrootp)
&new_fullhash,
&newroot
);
- new_pair = newroot->ct_pair;
assert(newroot);
assert(new_height > 0);
@@ -1653,22 +1669,18 @@ ft_init_new_root(FT ft, FTNODE oldroot, FTNODE *newrootp)
ft->h->layout_version,
ft->h->flags
);
+ newroot->fullhash = new_fullhash;
MSN msna = oldroot->max_msn_applied_to_node_on_disk;
newroot->max_msn_applied_to_node_on_disk = msna;
BP_STATE(newroot,0) = PT_AVAIL;
newroot->dirty = 1;
- // now do the "switcheroo"
- BP_BLOCKNUM(newroot,0) = new_blocknum;
- newroot->thisnodename = old_blocknum;
- newroot->fullhash = old_fullhash;
- newroot->ct_pair = old_pair;
-
- oldroot->thisnodename = new_blocknum;
- oldroot->fullhash = new_fullhash;
- oldroot->ct_pair = new_pair;
-
- toku_cachetable_swap_pair_values(old_pair, new_pair);
+ // Set the first child to have the new blocknum,
+ // and then swap newroot with oldroot. The new root
+ // will inherit the hash/blocknum/pair from oldroot,
+ // keeping the root blocknum constant.
+ BP_BLOCKNUM(newroot, 0) = new_blocknum;
+ toku_ftnode_swap_pair_values(newroot, oldroot);
toku_ft_split_child(
ft,
@@ -2757,6 +2769,16 @@ static void inject_message_in_locked_node(
// verify that msn of latest message was captured in root node
paranoid_invariant(msg->msn.msn == node->max_msn_applied_to_node_on_disk.msn);
+ if (node->thisnodename.b == ft->rightmost_blocknum.b) {
+ if (ft->seqinsert_score < FT_SEQINSERT_SCORE_THRESHOLD) {
+ // we promoted to the rightmost leaf node and the seqinsert score has not yet saturated.
+ toku_sync_fetch_and_add(&ft->seqinsert_score, 1);
+ }
+ } else if (ft->seqinsert_score != 0) {
+ // we promoted to something other than the rightmost leaf node and the score should reset
+ ft->seqinsert_score = 0;
+ }
+
// if we call toku_ft_flush_some_child, then that function unpins the root
// otherwise, we unpin ourselves
if (node->height > 0 && toku_ft_nonleaf_is_gorged(node, ft->h->nodesize)) {
@@ -2913,6 +2935,21 @@ static inline bool should_inject_in_node(seqinsert_loc loc, int height, int dept
return (height == 0 || (loc == NEITHER_EXTREME && (height <= 1 || depth >= 2)));
}
+static void ft_set_or_verify_rightmost_blocknum(FT ft, BLOCKNUM b)
+// Given: 'b', the _definitive_ and constant rightmost blocknum of 'ft'
+{
+ if (ft->rightmost_blocknum.b == RESERVED_BLOCKNUM_NULL) {
+ toku_ft_lock(ft);
+ if (ft->rightmost_blocknum.b == RESERVED_BLOCKNUM_NULL) {
+ ft->rightmost_blocknum = b;
+ }
+ toku_ft_unlock(ft);
+ }
+ // The rightmost blocknum only transitions from RESERVED_BLOCKNUM_NULL to non-null.
+ // If it's already set, verify that the stored value is consistent with 'b'
+ invariant(ft->rightmost_blocknum.b == b.b);
+}
+
static void push_something_in_subtree(
FT ft,
FTNODE subtree_root,
@@ -2960,6 +2997,14 @@ static void push_something_in_subtree(
default:
STATUS_INC(FT_PRO_NUM_INJECT_DEPTH_GT3, 1); break;
}
+ // If the target node is a non-root leaf node on the right extreme,
+ // set the rightmost blocknum. We know there are no messages above us
+ // because promotion would not chose to inject directly into this leaf
+ // otherwise. We explicitly skip the root node because then we don't have
+ // to worry about changing the rightmost blocknum when the root splits.
+ if (subtree_root->height == 0 && loc == RIGHT_EXTREME && subtree_root->thisnodename.b != ft->h->root_blocknum.b) {
+ ft_set_or_verify_rightmost_blocknum(ft, subtree_root->thisnodename);
+ }
inject_message_in_locked_node(ft, subtree_root, target_childnum, msg, flow_deltas, gc_info);
} else {
int r;
@@ -3230,7 +3275,260 @@ void toku_ft_root_put_msg(
}
}
-// Effect: Insert the key-val pair into ft.
+static int ft_compare_keys(FT ft, const DBT *a, const DBT *b)
+// Effect: Compare two keys using the given fractal tree's comparator/descriptor
+{
+ FAKE_DB(db, &ft->cmp_descriptor);
+ return ft->compare_fun(&db, a, b);
+}
+
+static LEAFENTRY bn_get_le_and_key(BASEMENTNODE bn, int idx, DBT *key)
+// Effect: Gets the i'th leafentry from the given basement node and
+// fill its key in *key
+// Requires: The i'th leafentry exists.
+{
+ LEAFENTRY le;
+ uint32_t le_len;
+ void *le_key;
+ int r = bn->data_buffer.fetch_klpair(idx, &le, &le_len, &le_key);
+ invariant_zero(r);
+ toku_fill_dbt(key, le_key, le_len);
+ return le;
+}
+
+static LEAFENTRY ft_leaf_leftmost_le_and_key(FTNODE leaf, DBT *leftmost_key)
+// Effect: If a leftmost key exists in the given leaf, toku_fill_dbt()
+// the key into *leftmost_key
+// Requires: Leaf is fully in memory and pinned for read or write.
+// Return: leafentry if it exists, nullptr otherwise
+{
+ for (int i = 0; i < leaf->n_children; i++) {
+ BASEMENTNODE bn = BLB(leaf, i);
+ if (bn->data_buffer.num_klpairs() > 0) {
+ // Get the first (leftmost) leafentry and its key
+ return bn_get_le_and_key(bn, 0, leftmost_key);
+ }
+ }
+ return nullptr;
+}
+
+static LEAFENTRY ft_leaf_rightmost_le_and_key(FTNODE leaf, DBT *rightmost_key)
+// Effect: If a rightmost key exists in the given leaf, toku_fill_dbt()
+// the key into *rightmost_key
+// Requires: Leaf is fully in memory and pinned for read or write.
+// Return: leafentry if it exists, nullptr otherwise
+{
+ for (int i = leaf->n_children - 1; i >= 0; i--) {
+ BASEMENTNODE bn = BLB(leaf, i);
+ size_t num_les = bn->data_buffer.num_klpairs();
+ if (num_les > 0) {
+ // Get the last (rightmost) leafentry and its key
+ return bn_get_le_and_key(bn, num_les - 1, rightmost_key);
+ }
+ }
+ return nullptr;
+}
+
+static int ft_leaf_get_relative_key_pos(FT ft, FTNODE leaf, const DBT *key, bool *nondeleted_key_found, int *target_childnum)
+// Effect: Determines what the relative position of the given key is with
+// respect to a leaf node, and if it exists.
+// Requires: Leaf is fully in memory and pinned for read or write.
+// Requires: target_childnum is non-null
+// Return: < 0 if key is less than the leftmost key in the leaf OR the relative position is unknown, for any reason.
+// 0 if key is in the bounds [leftmost_key, rightmost_key] for this leaf or the leaf is empty
+// > 0 if key is greater than the rightmost key in the leaf
+// *nondeleted_key_found is set (if non-null) if the target key was found and is not deleted, unmodified otherwise
+// *target_childnum is set to the child that (does or would) contain the key, if calculated, unmodified otherwise
+{
+ DBT rightmost_key;
+ LEAFENTRY rightmost_le = ft_leaf_rightmost_le_and_key(leaf, &rightmost_key);
+ if (rightmost_le == nullptr) {
+ // If we can't get a rightmost key then the leaf is empty.
+ // In such a case, we don't have any information about what keys would be in this leaf.
+ // We have to assume the leaf node that would contain this key is to the left.
+ return -1;
+ }
+ // We have a rightmost leafentry, so it must exist in some child node
+ invariant(leaf->n_children > 0);
+
+ int relative_pos = 0;
+ int c = ft_compare_keys(ft, key, &rightmost_key);
+ if (c > 0) {
+ relative_pos = 1;
+ *target_childnum = leaf->n_children - 1;
+ } else if (c == 0) {
+ if (nondeleted_key_found != nullptr && !le_latest_is_del(rightmost_le)) {
+ *nondeleted_key_found = true;
+ }
+ relative_pos = 0;
+ *target_childnum = leaf->n_children - 1;
+ } else {
+ // The key is less than the rightmost. It may still be in bounds if it's >= the leftmost.
+ DBT leftmost_key;
+ LEAFENTRY leftmost_le = ft_leaf_leftmost_le_and_key(leaf, &leftmost_key);
+ invariant_notnull(leftmost_le); // Must exist because a rightmost exists
+ c = ft_compare_keys(ft, key, &leftmost_key);
+ if (c > 0) {
+ if (nondeleted_key_found != nullptr) {
+ // The caller wants to know if a nondeleted key can be found.
+ LEAFENTRY target_le;
+ int childnum = toku_ftnode_which_child(leaf, key, &ft->cmp_descriptor, ft->compare_fun);
+ BASEMENTNODE bn = BLB(leaf, childnum);
+ struct msg_leafval_heaviside_extra extra = { ft->compare_fun, &ft->cmp_descriptor, key };
+ int r = bn->data_buffer.find_zero<decltype(extra), toku_msg_leafval_heaviside>(
+ extra,
+ &target_le,
+ nullptr, nullptr, nullptr
+ );
+ *target_childnum = childnum;
+ if (r == 0 && !le_latest_is_del(leftmost_le)) {
+ *nondeleted_key_found = true;
+ }
+ }
+ relative_pos = 0;
+ } else if (c == 0) {
+ if (nondeleted_key_found != nullptr && !le_latest_is_del(leftmost_le)) {
+ *nondeleted_key_found = true;
+ }
+ relative_pos = 0;
+ *target_childnum = 0;
+ } else {
+ relative_pos = -1;
+ }
+ }
+
+ return relative_pos;
+}
+
+static void ft_insert_directly_into_leaf(FT ft, FTNODE leaf, int target_childnum, DBT *key, DBT *val,
+ XIDS message_xids, enum ft_msg_type type, txn_gc_info *gc_info);
+static int getf_nothing(ITEMLEN, bytevec, ITEMLEN, bytevec, void *, bool);
+
+static int ft_maybe_insert_into_rightmost_leaf(FT ft, DBT *key, DBT *val, XIDS message_xids, enum ft_msg_type type,
+ txn_gc_info *gc_info, bool unique)
+// Effect: Pins the rightmost leaf node and attempts to do an insert.
+// There are three reasons why we may not succeed.
+// - The rightmost leaf is too full and needs a split.
+// - The key to insert is not within the provable bounds of this leaf node.
+// - The key is within bounds, but it already exists.
+// Return: 0 if this function did insert, DB_KEYEXIST if a unique key constraint exists and
+// some nondeleted leafentry with the same key exists
+// < 0 if this function did not insert, for a reason other than DB_KEYEXIST.
+// Note: Treat this function as a possible, but not necessary, optimization for insert.
+// Rationale: We want O(1) insertions down the rightmost path of the tree.
+{
+ int r = -1;
+
+ uint32_t rightmost_fullhash;
+ BLOCKNUM rightmost_blocknum = ft->rightmost_blocknum;
+ FTNODE rightmost_leaf = nullptr;
+
+ // Don't do the optimization if our heurstic suggests that
+ // insertion pattern is not sequential.
+ if (ft->seqinsert_score < FT_SEQINSERT_SCORE_THRESHOLD) {
+ goto cleanup;
+ }
+
+ // We know the seqinsert score is high enough that we should
+ // attemp to directly insert into the right most leaf. Because
+ // the score is non-zero, the rightmost blocknum must have been
+ // set. See inject_message_in_locked_node(), which only increases
+ // the score if the target node blocknum == rightmost_blocknum
+ invariant(rightmost_blocknum.b != RESERVED_BLOCKNUM_NULL);
+
+ // Pin the rightmost leaf with a write lock.
+ rightmost_fullhash = toku_cachetable_hash(ft->cf, rightmost_blocknum);
+ struct ftnode_fetch_extra bfe;
+ fill_bfe_for_full_read(&bfe, ft);
+ toku_pin_ftnode(ft, rightmost_blocknum, rightmost_fullhash, &bfe, PL_WRITE_CHEAP, &rightmost_leaf, true);
+
+ // The rightmost blocknum never chances once it is initialized to something
+ // other than null. Verify that the pinned node has the correct blocknum.
+ invariant(rightmost_leaf->thisnodename.b == rightmost_blocknum.b);
+
+ // If the rightmost leaf is reactive, bail out out and let the normal promotion pass
+ // take care of it. This also ensures that if any of our ancestors are reactive,
+ // they'll be taken care of too.
+ if (get_leaf_reactivity(rightmost_leaf, ft->h->nodesize) != RE_STABLE) {
+ STATUS_INC(FT_PRO_RIGHTMOST_LEAF_SHORTCUT_FAIL_REACTIVE, 1);
+ goto cleanup;
+ }
+
+ // The groundwork has been laid for an insertion directly into the rightmost
+ // leaf node. We know that it is pinned for write, fully in memory, has
+ // no messages above it, and is not reactive.
+ //
+ // Now, two more things must be true for this insertion to actually happen:
+ // 1. The key to insert is within the bounds of this leafnode, or to the right.
+ // 2. If there is a uniqueness constraint, it passes.
+ bool nondeleted_key_found;
+ int relative_pos;
+ int target_childnum;
+
+ nondeleted_key_found = false;
+ target_childnum = -1;
+ relative_pos = ft_leaf_get_relative_key_pos(ft, rightmost_leaf, key,
+ unique ? &nondeleted_key_found : nullptr,
+ &target_childnum);
+ if (relative_pos >= 0) {
+ STATUS_INC(FT_PRO_RIGHTMOST_LEAF_SHORTCUT_SUCCESS, 1);
+ if (unique && nondeleted_key_found) {
+ r = DB_KEYEXIST;
+ } else {
+ ft_insert_directly_into_leaf(ft, rightmost_leaf, target_childnum,
+ key, val, message_xids, type, gc_info);
+ r = 0;
+ }
+ } else {
+ STATUS_INC(FT_PRO_RIGHTMOST_LEAF_SHORTCUT_FAIL_POS, 1);
+ r = -1;
+ }
+
+cleanup:
+ // If we did the insert, the rightmost leaf was unpinned for us.
+ if (r != 0 && rightmost_leaf != nullptr) {
+ toku_unpin_ftnode(ft, rightmost_leaf);
+ }
+
+ return r;
+}
+
+static void ft_txn_log_insert(FT ft, DBT *key, DBT *val, TOKUTXN txn, bool do_logging, enum ft_msg_type type);
+
+int toku_ft_insert_unique(FT_HANDLE ft_h, DBT *key, DBT *val, TOKUTXN txn, bool do_logging) {
+// Effect: Insert a unique key-val pair into the fractal tree.
+// Return: 0 on success, DB_KEYEXIST if the overwrite constraint failed
+ XIDS message_xids = txn != nullptr ? toku_txn_get_xids(txn) : xids_get_root_xids();
+
+ TXN_MANAGER txn_manager = toku_ft_get_txn_manager(ft_h);
+ txn_manager_state txn_state_for_gc(txn_manager);
+
+ TXNID oldest_referenced_xid_estimate = toku_ft_get_oldest_referenced_xid_estimate(ft_h);
+ txn_gc_info gc_info(&txn_state_for_gc,
+ oldest_referenced_xid_estimate,
+ // no messages above us, we can implicitly promote uxrs based on this xid
+ oldest_referenced_xid_estimate,
+ true);
+ int r = ft_maybe_insert_into_rightmost_leaf(ft_h->ft, key, val, message_xids, FT_INSERT, &gc_info, true);
+ if (r != 0 && r != DB_KEYEXIST) {
+ // Default to a regular unique check + insert algorithm if we couldn't
+ // do it based on the rightmost leaf alone.
+ int lookup_r = toku_ft_lookup(ft_h, key, getf_nothing, nullptr);
+ if (lookup_r == DB_NOTFOUND) {
+ toku_ft_send_insert(ft_h, key, val, message_xids, FT_INSERT, &gc_info);
+ r = 0;
+ } else {
+ r = DB_KEYEXIST;
+ }
+ }
+
+ if (r == 0) {
+ ft_txn_log_insert(ft_h->ft, key, val, txn, do_logging, FT_INSERT);
+ }
+ return r;
+}
+
+// Effect: Insert the key-val pair into an ft.
void toku_ft_insert (FT_HANDLE ft_handle, DBT *key, DBT *val, TOKUTXN txn) {
toku_ft_maybe_insert(ft_handle, key, val, txn, false, ZERO_LSN, true, FT_INSERT);
}
@@ -3356,32 +3654,38 @@ TXNID toku_ft_get_oldest_referenced_xid_estimate(FT_HANDLE ft_h) {
return txn_manager != nullptr ? toku_txn_manager_get_oldest_referenced_xid_estimate(txn_manager) : TXNID_NONE;
}
-void toku_ft_maybe_insert (FT_HANDLE ft_h, DBT *key, DBT *val, TOKUTXN txn, bool oplsn_valid, LSN oplsn, bool do_logging, enum ft_msg_type type) {
- paranoid_invariant(type==FT_INSERT || type==FT_INSERT_NO_OVERWRITE);
- XIDS message_xids = xids_get_root_xids(); //By default use committed messages
+static void ft_txn_log_insert(FT ft, DBT *key, DBT *val, TOKUTXN txn, bool do_logging, enum ft_msg_type type) {
+ paranoid_invariant(type == FT_INSERT || type == FT_INSERT_NO_OVERWRITE);
+
+ //By default use committed messages
TXNID_PAIR xid = toku_txn_get_txnid(txn);
if (txn) {
BYTESTRING keybs = {key->size, (char *) key->data};
- toku_logger_save_rollback_cmdinsert(txn, toku_cachefile_filenum(ft_h->ft->cf), &keybs);
- toku_txn_maybe_note_ft(txn, ft_h->ft);
- message_xids = toku_txn_get_xids(txn);
+ toku_logger_save_rollback_cmdinsert(txn, toku_cachefile_filenum(ft->cf), &keybs);
+ toku_txn_maybe_note_ft(txn, ft);
}
TOKULOGGER logger = toku_txn_logger(txn);
if (do_logging && logger) {
BYTESTRING keybs = {.len=key->size, .data=(char *) key->data};
BYTESTRING valbs = {.len=val->size, .data=(char *) val->data};
if (type == FT_INSERT) {
- toku_log_enq_insert(logger, (LSN*)0, 0, txn, toku_cachefile_filenum(ft_h->ft->cf), xid, keybs, valbs);
+ toku_log_enq_insert(logger, (LSN*)0, 0, txn, toku_cachefile_filenum(ft->cf), xid, keybs, valbs);
}
else {
- toku_log_enq_insert_no_overwrite(logger, (LSN*)0, 0, txn, toku_cachefile_filenum(ft_h->ft->cf), xid, keybs, valbs);
+ toku_log_enq_insert_no_overwrite(logger, (LSN*)0, 0, txn, toku_cachefile_filenum(ft->cf), xid, keybs, valbs);
}
}
+}
+
+void toku_ft_maybe_insert (FT_HANDLE ft_h, DBT *key, DBT *val, TOKUTXN txn, bool oplsn_valid, LSN oplsn, bool do_logging, enum ft_msg_type type) {
+ ft_txn_log_insert(ft_h->ft, key, val, txn, do_logging, type);
LSN treelsn;
if (oplsn_valid && oplsn.lsn <= (treelsn = toku_ft_checkpoint_lsn(ft_h->ft)).lsn) {
// do nothing
} else {
+ XIDS message_xids = txn ? toku_txn_get_xids(txn) : xids_get_root_xids();
+
TXN_MANAGER txn_manager = toku_ft_get_txn_manager(ft_h);
txn_manager_state txn_state_for_gc(txn_manager);
@@ -3391,10 +3695,26 @@ void toku_ft_maybe_insert (FT_HANDLE ft_h, DBT *key, DBT *val, TOKUTXN txn, bool
// no messages above us, we can implicitly promote uxrs based on this xid
oldest_referenced_xid_estimate,
txn != nullptr ? !txn->for_recovery : false);
- toku_ft_send_insert(ft_h, key, val, message_xids, type, &gc_info);
+ int r = ft_maybe_insert_into_rightmost_leaf(ft_h->ft, key, val, message_xids, FT_INSERT, &gc_info, false);
+ if (r != 0) {
+ toku_ft_send_insert(ft_h, key, val, message_xids, type, &gc_info);
+ }
}
}
+static void ft_insert_directly_into_leaf(FT ft, FTNODE leaf, int target_childnum, DBT *key, DBT *val,
+ XIDS message_xids, enum ft_msg_type type, txn_gc_info *gc_info)
+// Effect: Insert directly into a leaf node a fractal tree. Does not do any logging.
+// Requires: Leaf is fully in memory and pinned for write.
+// Requires: If this insertion were to happen through the root node, the promotion
+// algorithm would have selected the given leaf node as the point of injection.
+// That means this function relies on the current implementation of promotion.
+{
+ FT_MSG_S ftcmd = { type, ZERO_MSN, message_xids, .u = { .id = { key, val } } };
+ size_t flow_deltas[] = { 0, 0 };
+ inject_message_in_locked_node(ft, leaf, target_childnum, &ftcmd, flow_deltas, gc_info);
+}
+
static void
ft_send_update_msg(FT_HANDLE ft_h, FT_MSG_S *msg, TOKUTXN txn) {
msg->xids = (txn
@@ -4894,6 +5214,13 @@ int copy_to_stale(const int32_t &offset, const uint32_t UU(idx), struct copy_to_
return 0;
}
+static void ft_bnc_move_messages_to_stale(FT ft, NONLEAF_CHILDINFO bnc) {
+ struct copy_to_stale_extra cts_extra = { .ft = ft, .bnc = bnc };
+ int r = bnc->fresh_message_tree.iterate_over_marked<struct copy_to_stale_extra, copy_to_stale>(&cts_extra);
+ invariant_zero(r);
+ bnc->fresh_message_tree.delete_all_marked();
+}
+
__attribute__((nonnull))
void
toku_move_ftnode_messages_to_stale(FT ft, FTNODE node) {
@@ -4906,10 +5233,7 @@ toku_move_ftnode_messages_to_stale(FT ft, FTNODE node) {
// We can't delete things out of the fresh tree inside the above
// procedures because we're still looking at the fresh tree. Instead
// we have to move messages after we're done looking at it.
- struct copy_to_stale_extra cts_extra = { .ft = ft, .bnc = bnc };
- int r = bnc->fresh_message_tree.iterate_over_marked<struct copy_to_stale_extra, copy_to_stale>(&cts_extra);
- invariant_zero(r);
- bnc->fresh_message_tree.delete_all_marked();
+ ft_bnc_move_messages_to_stale(ft, bnc);
}
}