Commit ac575d01 authored by John Esmet's avatar John Esmet

FT-260 Move ftnode related structures and declarations to ft/node.h and ft/node.cc

parent 79a9a40d
......@@ -53,6 +53,7 @@ set(FT_SOURCES
logger
log_upgrade
msg_buffer
node
quicklz
recover
rollback
......
......@@ -89,12 +89,13 @@ PATENT RIGHTS GRANT:
#ident "Copyright (c) 2007-2013 Tokutek Inc. All rights reserved."
#ident "The technology is licensed by the Massachusetts Institute of Technology, Rutgers State University of New Jersey, and the Research Foundation of State University of New York at Stony Brook under United States of America Serial No. 11/760379 and to the patents and/or patent applications resulting from it."
#include <ft-cachetable-wrappers.h>
#include "ft/fttypes.h"
#include "ft/ft-cachetable-wrappers.h"
#include "ft/ft-flusher.h"
#include "ft/ft-internal.h"
#include "ft/ft.h"
#include "ft/node.h"
#include <fttypes.h>
#include <ft-flusher.h>
#include <ft-internal.h>
#include <ft.h>
#include <util/context.h>
static void
......
......@@ -92,8 +92,9 @@ PATENT RIGHTS GRANT:
#ident "Copyright (c) 2007-2013 Tokutek Inc. All rights reserved."
#ident "The technology is licensed by the Massachusetts Institute of Technology, Rutgers State University of New Jersey, and the Research Foundation of State University of New York at Stony Brook under United States of America Serial No. 11/760379 and to the patents and/or patent applications resulting from it."
#include "ft/ft-internal.h"
#include "ft/cachetable.h"
#include "ft/ft-internal.h"
#include "ft/node.h"
/**
* Put an empty node (that is, no fields filled) into the cachetable.
......
......@@ -89,15 +89,16 @@ PATENT RIGHTS GRANT:
#ident "Copyright (c) 2007-2013 Tokutek Inc. All rights reserved."
#ident "The technology is licensed by the Massachusetts Institute of Technology, Rutgers State University of New Jersey, and the Research Foundation of State University of New York at Stony Brook under United States of America Serial No. 11/760379 and to the patents and/or patent applications resulting from it."
#include <ft-internal.h>
#include <ft-flusher.h>
#include <ft-flusher-internal.h>
#include <ft-cachetable-wrappers.h>
#include <ft.h>
#include <toku_assert.h>
#include <portability/toku_atomic.h>
#include <util/status.h>
#include <util/context.h>
#include "ft/ft.h"
#include "ft/ft-cachetable-wrappers.h"
#include "ft/ft-internal.h"
#include "ft/ft-flusher.h"
#include "ft/ft-flusher-internal.h"
#include "ft/node.h"
#include "portability/toku_assert.h"
#include "portability/toku_atomic.h"
#include "util/status.h"
#include "util/context.h"
/* Status is intended for display to humans to help understand system behavior.
* It does not need to be perfectly thread-safe.
......@@ -307,7 +308,7 @@ static bool
recurse_if_child_is_gorged(FTNODE child, void* extra)
{
struct flush_status_update_extra *fste = (flush_status_update_extra *)extra;
return toku_ft_nonleaf_is_gorged(child, fste->nodesize);
return toku_ftnode_nonleaf_is_gorged(child, fste->nodesize);
}
int
......@@ -497,7 +498,7 @@ ct_maybe_merge_child(struct flusher_advice *fa,
struct ftnode_fetch_extra bfe;
fill_bfe_for_full_read(&bfe, h);
toku_pin_ftnode(h, root, fullhash, &bfe, PL_WRITE_EXPENSIVE, &root_node, true);
toku_assert_entire_node_in_memory(root_node);
toku_ftnode_assert_fully_in_memory(root_node);
}
(void) toku_sync_fetch_and_add(&STATUS_VALUE(FT_FLUSHER_CLEANER_NUM_LEAF_MERGES_STARTED), 1);
......@@ -545,13 +546,12 @@ ct_flusher_advice_init(struct flusher_advice *fa, struct flush_status_update_ext
// a leaf node that is not entirely in memory. If so, then
// we cannot be sure if the node is reactive.
//
static bool may_node_be_reactive(FT ft, FTNODE node)
static bool ft_ftnode_may_be_reactive(FT ft, FTNODE node)
{
if (node->height == 0) {
return true;
}
else {
return (get_nonleaf_reactivity(node, ft->h->fanout) != RE_STABLE);
} else {
return toku_ftnode_get_nonleaf_reactivity(node, ft->h->fanout) != RE_STABLE;
}
}
......@@ -576,9 +576,9 @@ handle_split_of_child(
paranoid_invariant(node->height>0);
paranoid_invariant(0 <= childnum);
paranoid_invariant(childnum < node->n_children);
toku_assert_entire_node_in_memory(node);
toku_assert_entire_node_in_memory(childa);
toku_assert_entire_node_in_memory(childb);
toku_ftnode_assert_fully_in_memory(node);
toku_ftnode_assert_fully_in_memory(childa);
toku_ftnode_assert_fully_in_memory(childb);
NONLEAF_CHILDINFO old_bnc = BNC(node, childnum);
paranoid_invariant(toku_bnc_nbytesinbuf(old_bnc)==0);
int cnum;
......@@ -653,9 +653,9 @@ handle_split_of_child(
)
/* Keep pushing to the children, but not if the children would require a pushdown */
toku_assert_entire_node_in_memory(node);
toku_assert_entire_node_in_memory(childa);
toku_assert_entire_node_in_memory(childb);
toku_ftnode_assert_fully_in_memory(node);
toku_ftnode_assert_fully_in_memory(childa);
toku_ftnode_assert_fully_in_memory(childb);
VERIFY_NODE(t, node);
VERIFY_NODE(t, childa);
......@@ -680,7 +680,7 @@ ftleaf_disk_size(FTNODE node)
// Effect: get the disk size of a leafentry
{
paranoid_invariant(node->height == 0);
toku_assert_entire_node_in_memory(node);
toku_ftnode_assert_fully_in_memory(node);
uint64_t retval = 0;
for (int i = 0; i < node->n_children; i++) {
retval += BLB_DATA(node, i)->get_disk_size();
......@@ -771,8 +771,8 @@ move_leafentries(
static void ftnode_finalize_split(FTNODE node, FTNODE B, MSN max_msn_applied_to_node) {
// Effect: Finalizes a split by updating some bits and dirtying both nodes
toku_assert_entire_node_in_memory(node);
toku_assert_entire_node_in_memory(B);
toku_ftnode_assert_fully_in_memory(node);
toku_ftnode_assert_fully_in_memory(B);
verify_all_in_mempool(node);
verify_all_in_mempool(B);
......@@ -851,7 +851,7 @@ ftleaf_split(
paranoid_invariant(node->height==0);
toku_assert_entire_node_in_memory(node);
toku_ftnode_assert_fully_in_memory(node);
verify_all_in_mempool(node);
MSN max_msn_applied_to_node = node->max_msn_applied_to_node_on_disk;
......@@ -996,7 +996,7 @@ ft_nonleaf_split(
{
//VERIFY_NODE(t,node);
STATUS_VALUE(FT_FLUSHER_SPLIT_NONLEAF)++;
toku_assert_entire_node_in_memory(node);
toku_ftnode_assert_fully_in_memory(node);
int old_n_children = node->n_children;
int n_children_in_a = old_n_children/2;
int n_children_in_b = old_n_children-n_children_in_a;
......@@ -1112,7 +1112,7 @@ ft_split_child(
}
static void bring_node_fully_into_memory(FTNODE node, FT ft) {
if (!is_entire_node_in_memory(node)) {
if (!toku_ftnode_fully_in_memory(node)) {
struct ftnode_fetch_extra bfe;
fill_bfe_for_full_read(&bfe, ft);
toku_cachetable_pf_pinned_pair(
......@@ -1136,12 +1136,12 @@ flush_this_child(
// Effect: Push everything in the CHILDNUMth buffer of node down into the child.
{
update_flush_status(child, 0);
toku_assert_entire_node_in_memory(node);
toku_ftnode_assert_fully_in_memory(node);
if (fa->should_destroy_basement_nodes(fa)) {
maybe_destroy_child_blbs(node, child, h);
}
bring_node_fully_into_memory(child, h);
toku_assert_entire_node_in_memory(child);
toku_ftnode_assert_fully_in_memory(child);
paranoid_invariant(node->height>0);
paranoid_invariant(child->thisnodename.b!=0);
// VERIFY_NODE does not work off client thread as of now
......@@ -1163,8 +1163,8 @@ static void
merge_leaf_nodes(FTNODE a, FTNODE b)
{
STATUS_VALUE(FT_FLUSHER_MERGE_LEAF)++;
toku_assert_entire_node_in_memory(a);
toku_assert_entire_node_in_memory(b);
toku_ftnode_assert_fully_in_memory(a);
toku_ftnode_assert_fully_in_memory(b);
paranoid_invariant(a->height == 0);
paranoid_invariant(b->height == 0);
paranoid_invariant(a->n_children > 0);
......@@ -1268,7 +1268,7 @@ maybe_merge_pinned_leaf_nodes(
{
unsigned int sizea = toku_serialize_ftnode_size(a);
unsigned int sizeb = toku_serialize_ftnode_size(b);
uint32_t num_leafentries = get_leaf_num_entries(a) + get_leaf_num_entries(b);
uint32_t num_leafentries = toku_ftnode_leaf_num_entries(a) + toku_ftnode_leaf_num_entries(b);
if (num_leafentries > 1 && (sizea + sizeb)*4 > (nodesize*3)) {
// the combined size is more than 3/4 of a node, so don't merge them.
*did_merge = false;
......@@ -1301,8 +1301,8 @@ maybe_merge_pinned_nonleaf_nodes(
bool *did_rebalance,
DBT *splitk)
{
toku_assert_entire_node_in_memory(a);
toku_assert_entire_node_in_memory(b);
toku_ftnode_assert_fully_in_memory(a);
toku_ftnode_assert_fully_in_memory(b);
paranoid_invariant(parent_splitk->data);
int old_n_children = a->n_children;
int new_n_children = old_n_children + b->n_children;
......@@ -1366,9 +1366,9 @@ maybe_merge_pinned_nodes(
{
MSN msn_max;
paranoid_invariant(a->height == b->height);
toku_assert_entire_node_in_memory(parent);
toku_assert_entire_node_in_memory(a);
toku_assert_entire_node_in_memory(b);
toku_ftnode_assert_fully_in_memory(parent);
toku_ftnode_assert_fully_in_memory(a);
toku_ftnode_assert_fully_in_memory(b);
parent->dirty = 1; // just to make sure
{
MSN msna = a->max_msn_applied_to_node_on_disk;
......@@ -1413,7 +1413,7 @@ ft_merge_child(
// this function should not be called
// if the child is not mergable
paranoid_invariant(node->n_children > 1);
toku_assert_entire_node_in_memory(node);
toku_ftnode_assert_fully_in_memory(node);
int childnuma,childnumb;
if (childnum_to_merge > 0) {
......@@ -1577,7 +1577,7 @@ void toku_ft_flush_some_child(FT ft, FTNODE parent, struct flusher_advice *fa)
int dirtied = 0;
NONLEAF_CHILDINFO bnc = NULL;
paranoid_invariant(parent->height>0);
toku_assert_entire_node_in_memory(parent);
toku_ftnode_assert_fully_in_memory(parent);
TXNID parent_oldest_referenced_xid_known = parent->oldest_referenced_xid_known;
// pick the child we want to flush to
......@@ -1608,7 +1608,7 @@ void toku_ft_flush_some_child(FT ft, FTNODE parent, struct flusher_advice *fa)
// Let's do a quick check to see if the child may be reactive
// If the child cannot be reactive, then we can safely unlock
// the parent before finishing reading in the entire child node.
bool may_child_be_reactive = may_node_be_reactive(ft, child);
bool may_child_be_reactive = ft_ftnode_may_be_reactive(ft, child);
paranoid_invariant(child->thisnodename.b!=0);
......@@ -1649,7 +1649,7 @@ void toku_ft_flush_some_child(FT ft, FTNODE parent, struct flusher_advice *fa)
// we wont be splitting/merging child
// and we have already replaced the bnc
// for the root with a fresh one
enum reactivity child_re = get_node_reactivity(ft, child);
enum reactivity child_re = toku_ftnode_get_reactivity(ft, child);
if (parent && child_re == RE_STABLE) {
toku_unpin_ftnode(ft, parent);
parent = NULL;
......@@ -1679,7 +1679,7 @@ void toku_ft_flush_some_child(FT ft, FTNODE parent, struct flusher_advice *fa)
// let's get the reactivity of the child again,
// it is possible that the flush got rid of some values
// and now the parent is no longer reactive
child_re = get_node_reactivity(ft, child);
child_re = toku_ftnode_get_reactivity(ft, child);
// if the parent has been unpinned above, then
// this is our only option, even if the child is not stable
// if the child is not stable, we'll handle it the next
......@@ -1724,6 +1724,79 @@ void toku_ft_flush_some_child(FT ft, FTNODE parent, struct flusher_advice *fa)
}
}
void toku_bnc_flush_to_child(FT ft, NONLEAF_CHILDINFO bnc, FTNODE child, TXNID parent_oldest_referenced_xid_known) {
paranoid_invariant(bnc);
TOKULOGGER logger = toku_cachefile_logger(ft->cf);
TXN_MANAGER txn_manager = logger != nullptr ? toku_logger_get_txn_manager(logger) : nullptr;
TXNID oldest_referenced_xid_for_simple_gc = TXNID_NONE;
txn_manager_state txn_state_for_gc(txn_manager);
bool do_garbage_collection = child->height == 0 && txn_manager != nullptr;
if (do_garbage_collection) {
txn_state_for_gc.init();
oldest_referenced_xid_for_simple_gc = toku_txn_manager_get_oldest_referenced_xid_estimate(txn_manager);
}
txn_gc_info gc_info(&txn_state_for_gc,
oldest_referenced_xid_for_simple_gc,
child->oldest_referenced_xid_known,
true);
struct flush_msg_fn {
FT ft;
FTNODE child;
NONLEAF_CHILDINFO bnc;
txn_gc_info *gc_info;
STAT64INFO_S stats_delta;
size_t remaining_memsize = bnc->msg_buffer.buffer_size_in_use();
flush_msg_fn(FT t, FTNODE n, NONLEAF_CHILDINFO nl, txn_gc_info *g) :
ft(t), child(n), bnc(nl), gc_info(g), remaining_memsize(bnc->msg_buffer.buffer_size_in_use()) {
stats_delta = { 0, 0 };
}
int operator()(FT_MSG msg, bool is_fresh) {
size_t flow_deltas[] = { 0, 0 };
size_t memsize_in_buffer = message_buffer::msg_memsize_in_buffer(msg);
if (remaining_memsize <= bnc->flow[0]) {
// this message is in the current checkpoint's worth of
// the end of the message buffer
flow_deltas[0] = memsize_in_buffer;
} else if (remaining_memsize <= bnc->flow[0] + bnc->flow[1]) {
// this message is in the last checkpoint's worth of the
// end of the message buffer
flow_deltas[1] = memsize_in_buffer;
}
toku_ftnode_put_msg(
ft->compare_fun,
ft->update_fun,
&ft->cmp_descriptor,
child,
-1,
msg,
is_fresh,
gc_info,
flow_deltas,
&stats_delta
);
remaining_memsize -= memsize_in_buffer;
return 0;
}
} flush_fn(ft, child, bnc, &gc_info);
bnc->msg_buffer.iterate(flush_fn);
child->oldest_referenced_xid_known = parent_oldest_referenced_xid_known;
invariant(flush_fn.remaining_memsize == 0);
if (flush_fn.stats_delta.numbytes || flush_fn.stats_delta.numrows) {
toku_ft_update_stats(&ft->in_memory_stats, flush_fn.stats_delta);
}
if (do_garbage_collection) {
size_t buffsize = bnc->msg_buffer.buffer_size_in_use();
// may be misleading if there's a broadcast message in there
toku_ft_status_note_msg_bytes_out(buffsize);
}
}
static void
update_cleaner_status(
FTNODE node,
......@@ -1912,7 +1985,7 @@ static void flush_node_fun(void *fe_v)
// If so, call toku_ft_flush_some_child on the node (because this flush intends to
// pass a meaningful oldest referenced xid for simple garbage collection), and it is the
// responsibility of the flush to unlock the node. otherwise, we unlock it here.
if (fe->node->height > 0 && toku_ft_nonleaf_is_gorged(fe->node, fe->h->h->nodesize)) {
if (fe->node->height > 0 && toku_ftnode_nonleaf_is_gorged(fe->node, fe->h->h->nodesize)) {
toku_ft_flush_some_child(fe->h, fe->node, &fa);
}
else {
......@@ -1984,7 +2057,7 @@ void toku_ft_flush_node_on_background_thread(FT h, FTNODE parent)
//
// successfully locked child
//
bool may_child_be_reactive = may_node_be_reactive(h, child);
bool may_child_be_reactive = ft_ftnode_may_be_reactive(h, child);
if (!may_child_be_reactive) {
// We're going to unpin the parent, so before we do, we must
// check to see if we need to blow away the basement nodes to
......
......@@ -151,11 +151,14 @@ toku_flusher_thread_set_callback(
* Puts a workitem on the flusher thread queue, scheduling the node to be
* flushed by toku_ft_flush_some_child.
*/
void
toku_ft_flush_node_on_background_thread(
FT ft,
FTNODE parent
);
void toku_ft_flush_node_on_background_thread(FT ft, FTNODE parent);
enum split_mode {
SPLIT_EVENLY,
SPLIT_LEFT_HEAVY,
SPLIT_RIGHT_HEAVY
};
// Given pinned node and pinned child, split child into two
// and update node with information about its new child.
......@@ -166,6 +169,7 @@ void toku_ft_split_child(
FTNODE child,
enum split_mode split_mode
);
// Given pinned node, merge childnum with a neighbor and update node with
// information about the change
void toku_ft_merge_child(
......@@ -217,8 +221,6 @@ ft_nonleaf_split(
FTNODE* dependent_nodes
);
/************************************************************************
* HOT optimize, should perhaps be factored out to its own header file *
************************************************************************
......
......@@ -89,14 +89,15 @@ PATENT RIGHTS GRANT:
#ident "Copyright (c) 2007-2013 Tokutek Inc. All rights reserved."
#ident "The technology is licensed by the Massachusetts Institute of Technology, Rutgers State University of New Jersey, and the Research Foundation of State University of New York at Stony Brook under United States of America Serial No. 11/760379 and to the patents and/or patent applications resulting from it."
#include <ft-flusher.h>
#include <ft-flusher-internal.h>
#include <ft-cachetable-wrappers.h>
#include <ft-internal.h>
#include <ft.h>
#include <portability/toku_atomic.h>
#include <util/status.h>
#include <util/context.h>
#include "ft/ft.h"
#include "ft/ft-cachetable-wrappers.h"
#include "ft/ft-flusher.h"
#include "ft/ft-flusher-internal.h"
#include "ft/ft-internal.h"
#include "ft/node.h"
#include "portability/toku_atomic.h"
#include "util/context.h"
#include "util/status.h"
// Member Descirption:
// 1. highest_pivot_key - this is the key that corresponds to the
......@@ -339,7 +340,7 @@ toku_ft_hot_optimize(FT_HANDLE ft_handle, DBT* left, DBT* right,
PL_WRITE_EXPENSIVE,
&root,
true);
toku_assert_entire_node_in_memory(root);
toku_ftnode_assert_fully_in_memory(root);
}
// Prepare HOT diagnostics.
......
......@@ -129,348 +129,6 @@ enum { FT_DEFAULT_BASEMENT_NODE_SIZE = 128 * 1024 };
// happen into the rightmost leaf node due to promotion.
enum { FT_SEQINSERT_SCORE_THRESHOLD = 100 };
//
// Field in ftnode_fetch_extra that tells the
// partial fetch callback what piece of the node
// is needed by the ydb
//
enum ftnode_fetch_type {
ftnode_fetch_none=1, // no partitions needed.
ftnode_fetch_subset, // some subset of partitions needed
ftnode_fetch_prefetch, // this is part of a prefetch call
ftnode_fetch_all, // every partition is needed
ftnode_fetch_keymatch, // one child is needed if it holds both keys
};
enum split_mode {
SPLIT_EVENLY,
SPLIT_LEFT_HEAVY,
SPLIT_RIGHT_HEAVY
};
enum reactivity {
RE_STABLE,
RE_FUSIBLE,
RE_FISSIBLE
};
static bool is_valid_ftnode_fetch_type(enum ftnode_fetch_type type) UU();
static bool is_valid_ftnode_fetch_type(enum ftnode_fetch_type type) {
switch (type) {
case ftnode_fetch_none:
case ftnode_fetch_subset:
case ftnode_fetch_prefetch:
case ftnode_fetch_all:
case ftnode_fetch_keymatch:
return true;
default:
return false;
}
}
//
// An extra parameter passed to cachetable functions
// That is used in all types of fetch callbacks.
// The contents help the partial fetch and fetch
// callbacks retrieve the pieces of a node necessary
// for the ensuing operation (flush, query, ...)
//
struct ftnode_fetch_extra {
enum ftnode_fetch_type type;
// needed for reading a node off disk
FT h;
// used in the case where type == ftnode_fetch_subset
// parameters needed to find out which child needs to be decompressed (so it can be read)
ft_search *search;
DBT range_lock_left_key, range_lock_right_key;
bool left_is_neg_infty, right_is_pos_infty;
// states if we should try to aggressively fetch basement nodes
// that are not specifically needed for current query,
// but may be needed for other cursor operations user is doing
// For example, if we have not disabled prefetching,
// and the user is doing a dictionary wide scan, then
// even though a query may only want one basement node,
// we fetch all basement nodes in a leaf node.
bool disable_prefetching;
// this value will be set during the fetch_callback call by toku_ftnode_fetch_callback or toku_ftnode_pf_req_callback
// thi callbacks need to evaluate this anyway, so we cache it here so the search code does not reevaluate it
int child_to_read;
// when we read internal nodes, we want to read all the data off disk in one I/O
// then we'll treat it as normal and only decompress the needed partitions etc.
bool read_all_partitions;
// Accounting: How many bytes were read, and how much time did we spend doing I/O?
uint64_t bytes_read;
tokutime_t io_time;
tokutime_t decompress_time;
tokutime_t deserialize_time;
};
typedef struct ftnode_fetch_extra *FTNODE_FETCH_EXTRA;
struct toku_msg_buffer_key_msn_heaviside_extra {
DESCRIPTOR desc;
ft_compare_func cmp;
message_buffer *msg_buffer;
const DBT *key;
MSN msn;
};
// comparison function for inserting messages into a
// ftnode_nonleaf_childinfo's message_tree
int
toku_msg_buffer_key_msn_heaviside(const int32_t &v, const struct toku_msg_buffer_key_msn_heaviside_extra &extra);
struct toku_msg_buffer_key_msn_cmp_extra {
DESCRIPTOR desc;
ft_compare_func cmp;
message_buffer *msg_buffer;
};
// same thing for qsort_r
int
toku_msg_buffer_key_msn_cmp(const struct toku_msg_buffer_key_msn_cmp_extra &extrap, const int &a, const int &b);
typedef toku::omt<int32_t> off_omt_t;
typedef toku::omt<int32_t, int32_t, true> marked_off_omt_t;
// data of an available partition of a nonleaf ftnode
struct ftnode_nonleaf_childinfo {
message_buffer msg_buffer;
off_omt_t broadcast_list;
marked_off_omt_t fresh_message_tree;
off_omt_t stale_message_tree;
uint64_t flow[2]; // current and last checkpoint
};
unsigned int toku_bnc_nbytesinbuf(NONLEAF_CHILDINFO bnc);
int toku_bnc_n_entries(NONLEAF_CHILDINFO bnc);
long toku_bnc_memory_size(NONLEAF_CHILDINFO bnc);
long toku_bnc_memory_used(NONLEAF_CHILDINFO bnc);
void toku_bnc_insert_msg(NONLEAF_CHILDINFO bnc, const void *key, ITEMLEN keylen, const void *data, ITEMLEN datalen, enum ft_msg_type type, MSN msn, XIDS xids, bool is_fresh, DESCRIPTOR desc, ft_compare_func cmp);
void toku_bnc_empty(NONLEAF_CHILDINFO bnc);
void toku_bnc_flush_to_child(FT h, NONLEAF_CHILDINFO bnc, FTNODE child, TXNID parent_oldest_referenced_xid_known);
bool toku_bnc_should_promote(FT ft, NONLEAF_CHILDINFO bnc) __attribute__((const, nonnull));
bool toku_ft_nonleaf_is_gorged(FTNODE node, uint32_t nodesize);
enum reactivity get_nonleaf_reactivity(FTNODE node, unsigned int fanout);
enum reactivity get_node_reactivity(FT ft, FTNODE node);
uint32_t get_leaf_num_entries(FTNODE node);
// data of an available partition of a leaf ftnode
struct ftnode_leaf_basement_node {
bn_data data_buffer;
unsigned int seqinsert; // number of sequential inserts to this leaf
MSN max_msn_applied; // max message sequence number applied
bool stale_ancestor_messages_applied;
STAT64INFO_S stat64_delta; // change in stat64 counters since basement was last written to disk
};
enum pt_state { // declare this to be packed so that when used below it will only take 1 byte.
PT_INVALID = 0,
PT_ON_DISK = 1,
PT_COMPRESSED = 2,
PT_AVAIL = 3};
enum ftnode_child_tag {
BCT_INVALID = 0,
BCT_NULL,
BCT_SUBBLOCK,
BCT_LEAF,
BCT_NONLEAF
};
typedef struct ftnode_child_pointer {
union {
struct sub_block *subblock;
struct ftnode_nonleaf_childinfo *nonleaf;
struct ftnode_leaf_basement_node *leaf;
} u;
enum ftnode_child_tag tag;
} FTNODE_CHILD_POINTER;
struct ftnode_disk_data {
//
// stores the offset to the beginning of the partition on disk from the ftnode, and the length, needed to read a partition off of disk
// the value is only meaningful if the node is clean. If the node is dirty, then the value is meaningless
// The START is the distance from the end of the compressed node_info data, to the beginning of the compressed partition
// The SIZE is the size of the compressed partition.
// Rationale: We cannot store the size from the beginning of the node since we don't know how big the header will be.
// However, later when we are doing aligned writes, we won't be able to store the size from the end since we want things to align.
uint32_t start;
uint32_t size;
};
#define BP_START(node_dd,i) ((node_dd)[i].start)
#define BP_SIZE(node_dd,i) ((node_dd)[i].size)
// a ftnode partition, associated with a child of a node
struct ftnode_partition {
// the following three variables are used for nonleaf nodes
// for leaf nodes, they are meaningless
BLOCKNUM blocknum; // blocknum of child
// How many bytes worth of work was performed by messages in each buffer.
uint64_t workdone;
//
// pointer to the partition. Depending on the state, they may be different things
// if state == PT_INVALID, then the node was just initialized and ptr == NULL
// if state == PT_ON_DISK, then ptr == NULL
// if state == PT_COMPRESSED, then ptr points to a struct sub_block*
// if state == PT_AVAIL, then ptr is:
// a struct ftnode_nonleaf_childinfo for internal nodes,
// a struct ftnode_leaf_basement_node for leaf nodes
//
struct ftnode_child_pointer ptr;
//
// at any time, the partitions may be in one of the following three states (stored in pt_state):
// PT_INVALID - means that the partition was just initialized
// PT_ON_DISK - means that the partition is not in memory and needs to be read from disk. To use, must read off disk and decompress
// PT_COMPRESSED - means that the partition is compressed in memory. To use, must decompress
// PT_AVAIL - means the partition is decompressed and in memory
//
enum pt_state state; // make this an enum to make debugging easier.
// clock count used to for pe_callback to determine if a node should be evicted or not
// for now, saturating the count at 1
uint8_t clock_count;
};
struct ftnode {
MSN max_msn_applied_to_node_on_disk; // max_msn_applied that will be written to disk
unsigned int flags;
BLOCKNUM thisnodename; // Which block number is this node?
int layout_version; // What version of the data structure?
int layout_version_original; // different (<) from layout_version if upgraded from a previous version (useful for debugging)
int layout_version_read_from_disk; // transient, not serialized to disk, (useful for debugging)
uint32_t build_id; // build_id (svn rev number) of software that wrote this node to disk
int height; /* height is always >= 0. 0 for leaf, >0 for nonleaf. */
int dirty;
uint32_t fullhash;
int n_children; //for internal nodes, if n_children==fanout+1 then the tree needs to be rebalanced.
// for leaf nodes, represents number of basement nodes
unsigned int totalchildkeylens;
DBT *childkeys; /* Pivot keys. Child 0's keys are <= childkeys[0]. Child 1's keys are <= childkeys[1].
Child 1's keys are > childkeys[0]. */
// What's the oldest referenced xid that this node knows about? The real oldest
// referenced xid might be younger, but this is our best estimate. We use it
// as a heuristic to transition provisional mvcc entries from provisional to
// committed (from implicity committed to really committed).
//
// A better heuristic would be the oldest live txnid, but we use this since it
// still works well most of the time, and its readily available on the inject
// code path.
TXNID oldest_referenced_xid_known;
// array of size n_children, consisting of ftnode partitions
// each one is associated with a child
// for internal nodes, the ith partition corresponds to the ith message buffer
// for leaf nodes, the ith partition corresponds to the ith basement node
struct ftnode_partition *bp;
PAIR ct_pair;
};
// ftnode partition macros
// BP stands for ftnode_partition
#define BP_BLOCKNUM(node,i) ((node)->bp[i].blocknum)
#define BP_STATE(node,i) ((node)->bp[i].state)
#define BP_WORKDONE(node, i)((node)->bp[i].workdone)
//
// macros for managing a node's clock
// Should be managed by ft-ops.c, NOT by serialize/deserialize
//
//
// BP_TOUCH_CLOCK uses a compare and swap because multiple threads
// that have a read lock on an internal node may try to touch the clock
// simultaneously
//
#define BP_TOUCH_CLOCK(node, i) ((node)->bp[i].clock_count = 1)
#define BP_SWEEP_CLOCK(node, i) ((node)->bp[i].clock_count = 0)
#define BP_SHOULD_EVICT(node, i) ((node)->bp[i].clock_count == 0)
// not crazy about having these two here, one is for the case where we create new
// nodes, such as in splits and creating new roots, and the other is for when
// we are deserializing a node and not all bp's are touched
#define BP_INIT_TOUCHED_CLOCK(node, i) ((node)->bp[i].clock_count = 1)
#define BP_INIT_UNTOUCHED_CLOCK(node, i) ((node)->bp[i].clock_count = 0)
// internal node macros
static inline void set_BNULL(FTNODE node, int i) {
paranoid_invariant(i >= 0);
paranoid_invariant(i < node->n_children);
node->bp[i].ptr.tag = BCT_NULL;
}
static inline bool is_BNULL (FTNODE node, int i) {
paranoid_invariant(i >= 0);
paranoid_invariant(i < node->n_children);
return node->bp[i].ptr.tag == BCT_NULL;
}
static inline NONLEAF_CHILDINFO BNC(FTNODE node, int i) {
paranoid_invariant(i >= 0);
paranoid_invariant(i < node->n_children);
FTNODE_CHILD_POINTER p = node->bp[i].ptr;
paranoid_invariant(p.tag==BCT_NONLEAF);
return p.u.nonleaf;
}
static inline void set_BNC(FTNODE node, int i, NONLEAF_CHILDINFO nl) {
paranoid_invariant(i >= 0);
paranoid_invariant(i < node->n_children);
FTNODE_CHILD_POINTER *p = &node->bp[i].ptr;
p->tag = BCT_NONLEAF;
p->u.nonleaf = nl;
}
static inline BASEMENTNODE BLB(FTNODE node, int i) {
paranoid_invariant(i >= 0);
// The optimizer really doesn't like it when we compare
// i to n_children as signed integers. So we assert that
// n_children is in fact positive before doing a comparison
// on the values forcibly cast to unsigned ints.
paranoid_invariant(node->n_children > 0);
paranoid_invariant((unsigned) i < (unsigned) node->n_children);
FTNODE_CHILD_POINTER p = node->bp[i].ptr;
paranoid_invariant(p.tag==BCT_LEAF);
return p.u.leaf;
}
static inline void set_BLB(FTNODE node, int i, BASEMENTNODE bn) {
paranoid_invariant(i >= 0);
paranoid_invariant(i < node->n_children);
FTNODE_CHILD_POINTER *p = &node->bp[i].ptr;
p->tag = BCT_LEAF;
p->u.leaf = bn;
}
static inline SUB_BLOCK BSB(FTNODE node, int i) {
paranoid_invariant(i >= 0);
paranoid_invariant(i < node->n_children);
FTNODE_CHILD_POINTER p = node->bp[i].ptr;
paranoid_invariant(p.tag==BCT_SUBBLOCK);
return p.u.subblock;
}
static inline void set_BSB(FTNODE node, int i, SUB_BLOCK sb) {
paranoid_invariant(i >= 0);
paranoid_invariant(i < node->n_children);
FTNODE_CHILD_POINTER *p = &node->bp[i].ptr;
p->tag = BCT_SUBBLOCK;
p->u.subblock = sb;
}
// ftnode leaf basementnode macros,
#define BLB_MAX_MSN_APPLIED(node,i) (BLB(node,i)->max_msn_applied)
#define BLB_MAX_DSN_APPLIED(node,i) (BLB(node,i)->max_dsn_applied)
#define BLB_DATA(node,i) (&(BLB(node,i)->data_buffer))
#define BLB_NBYTESINDATA(node,i) (BLB_DATA(node,i)->get_disk_size())
#define BLB_SEQINSERT(node,i) (BLB(node,i)->seqinsert)
/* pivot flags (must fit in 8 bits) */
enum {
FT_PIVOT_TRUNC = 4,
FT_PIVOT_FRONT_COMPRESS = 8,
};
uint32_t compute_child_fullhash (CACHEFILE cf, FTNODE node, int childnum);
// The ft_header is not managed by the cachetable. Instead, it hangs off the cachefile as userdata.
......@@ -638,18 +296,25 @@ struct ft_handle {
struct ft_options options;
};
// TODO: Move to cachetable header
PAIR_ATTR make_ftnode_pair_attr(FTNODE node);
PAIR_ATTR make_invalid_pair_attr(void);
// Cachetable callbacks for ftnodes.
void toku_ftnode_clone_callback(void* value_data, void** cloned_value_data, long* clone_size, PAIR_ATTR* new_attr, bool for_checkpoint, void* write_extraargs);
void toku_ftnode_checkpoint_complete_callback(void *value_data);
void toku_ftnode_flush_callback (CACHEFILE cachefile, int fd, BLOCKNUM nodename, void *ftnode_v, void** UU(disk_data), void *extraargs, PAIR_ATTR size, PAIR_ATTR* new_size, bool write_me, bool keep_me, bool for_checkpoint, bool is_clone);
int toku_ftnode_fetch_callback (CACHEFILE cachefile, PAIR p, int fd, BLOCKNUM nodename, uint32_t fullhash, void **ftnode_pv, void** UU(disk_data), PAIR_ATTR *sizep, int*dirty, void*extraargs);
void toku_ftnode_pe_est_callback(void* ftnode_pv, void* disk_data, long* bytes_freed_estimate, enum partial_eviction_cost *cost, void* write_extraargs);
int toku_ftnode_pe_callback(void *ftnode_pv, PAIR_ATTR old_attr, void *extraargs,
void (*finalize)(PAIR_ATTR new_attr, void *extra), void *finalize_extra);
bool toku_ftnode_pf_req_callback(void* ftnode_pv, void* read_extraargs);
int toku_ftnode_pf_callback(void* ftnode_pv, void* UU(disk_data), void* read_extraargs, int fd, PAIR_ATTR* sizep);
int toku_ftnode_cleaner_callback( void *ftnode_pv, BLOCKNUM blocknum, uint32_t fullhash, void *extraargs);
/* serialization code */
void
toku_create_compressed_partition_from_available(
FTNODE node,
int childnum,
void toku_create_compressed_partition_from_available(FTNODE node, int childnum,
enum toku_compression_method compression_method,
SUB_BLOCK sb
);
void rebalance_ftnode_leaf(FTNODE node, unsigned int basementnodesize);
SUB_BLOCK sb);
int toku_serialize_ftnode_to_memory (FTNODE node,
FTNODE_DISK_DATA* ndd,
unsigned int basementnodesize,
......@@ -727,19 +392,6 @@ void toku_serialize_ft_to_wbuf (
int toku_deserialize_ft_from (int fd, LSN max_acceptable_lsn, FT *ft);
void toku_serialize_descriptor_contents_to_fd(int fd, const DESCRIPTOR desc, DISKOFF offset);
void toku_serialize_descriptor_contents_to_wbuf(struct wbuf *wb, const DESCRIPTOR desc);
BASEMENTNODE toku_create_empty_bn(void);
BASEMENTNODE toku_create_empty_bn_no_buffer(void); // create a basement node with a null buffer.
NONLEAF_CHILDINFO toku_clone_nl(NONLEAF_CHILDINFO orig_childinfo);
BASEMENTNODE toku_clone_bn(BASEMENTNODE orig_bn);
NONLEAF_CHILDINFO toku_create_empty_nl(void);
// FIXME needs toku prefix
void destroy_basement_node (BASEMENTNODE bn);
// FIXME needs toku prefix
void destroy_nonleaf_childinfo (NONLEAF_CHILDINFO nl);
void toku_destroy_ftnode_internals(FTNODE node);
void toku_ftnode_free (FTNODE *node);
bool is_entire_node_in_memory(FTNODE node);
void toku_assert_entire_node_in_memory(FTNODE node);
// append a child node to a parent node
void toku_ft_nonleaf_append_child(FTNODE node, FTNODE child, const DBT *pivotkey);
......@@ -760,31 +412,10 @@ void toku_ft_status_update_pivot_fetch_reason(struct ftnode_fetch_extra *bfe);
void toku_ft_status_update_flush_reason(FTNODE node, uint64_t uncompressed_bytes_flushed, uint64_t bytes_written, tokutime_t write_time, bool for_checkpoint);
void toku_ft_status_update_serialize_times(FTNODE node, tokutime_t serialize_time, tokutime_t compress_time);
void toku_ft_status_update_deserialize_times(FTNODE node, tokutime_t deserialize_time, tokutime_t decompress_time);
void toku_ftnode_clone_callback(void* value_data, void** cloned_value_data, long* clone_size, PAIR_ATTR* new_attr, bool for_checkpoint, void* write_extraargs);
void toku_ftnode_checkpoint_complete_callback(void *value_data);
void toku_ftnode_flush_callback (CACHEFILE cachefile, int fd, BLOCKNUM nodename, void *ftnode_v, void** UU(disk_data), void *extraargs, PAIR_ATTR size, PAIR_ATTR* new_size, bool write_me, bool keep_me, bool for_checkpoint, bool is_clone);
int toku_ftnode_fetch_callback (CACHEFILE cachefile, PAIR p, int fd, BLOCKNUM nodename, uint32_t fullhash, void **ftnode_pv, void** UU(disk_data), PAIR_ATTR *sizep, int*dirty, void*extraargs);
void toku_ftnode_pe_est_callback(void* ftnode_pv, void* disk_data, long* bytes_freed_estimate, enum partial_eviction_cost *cost, void* write_extraargs);
int toku_ftnode_pe_callback(void *ftnode_pv, PAIR_ATTR old_attr, void *extraargs,
void (*finalize)(PAIR_ATTR new_attr, void *extra), void *finalize_extra);
bool toku_ftnode_pf_req_callback(void* ftnode_pv, void* read_extraargs);
int toku_ftnode_pf_callback(void* ftnode_pv, void* UU(disk_data), void* read_extraargs, int fd, PAIR_ATTR* sizep);
int toku_ftnode_cleaner_callback( void *ftnode_pv, BLOCKNUM blocknum, uint32_t fullhash, void *extraargs);
void toku_evict_bn_from_memory(FTNODE node, int childnum, FT h);
BASEMENTNODE toku_detach_bn(FTNODE node, int childnum);
static inline CACHETABLE_WRITE_CALLBACK get_write_callbacks_for_node(FT h) {
CACHETABLE_WRITE_CALLBACK wc;
wc.flush_callback = toku_ftnode_flush_callback;
wc.pe_est_callback = toku_ftnode_pe_est_callback;
wc.pe_callback = toku_ftnode_pe_callback;
wc.cleaner_callback = toku_ftnode_cleaner_callback;
wc.clone_callback = toku_ftnode_clone_callback;
wc.checkpoint_complete_callback = toku_ftnode_checkpoint_complete_callback;
wc.write_extraargs = h;
return wc;
}
void toku_ft_status_note_msn_discard(void);
void toku_ft_status_note_update(bool broadcast);
void toku_ft_status_note_msg_bytes_out(size_t buffsize);
void toku_ft_status_note_ftnode(int height, bool created); // created = false means destroyed
//
// Helper function to fill a ftnode_fetch_extra with data
......@@ -792,22 +423,7 @@ static inline CACHETABLE_WRITE_CALLBACK get_write_callbacks_for_node(FT h) {
// necessary. Used in cases where the entire node
// is required, such as for flushes.
//
static inline void fill_bfe_for_full_read(struct ftnode_fetch_extra *bfe, FT h) {
bfe->type = ftnode_fetch_all;
bfe->h = h;
bfe->search = NULL;
toku_init_dbt(&bfe->range_lock_left_key);
toku_init_dbt(&bfe->range_lock_right_key);
bfe->left_is_neg_infty = false;
bfe->right_is_pos_infty = false;
bfe->child_to_read = -1;
bfe->disable_prefetching = false;
bfe->read_all_partitions = false;
bfe->bytes_read = 0;
bfe->io_time = 0;
bfe->deserialize_time = 0;
bfe->decompress_time = 0;
}
void fill_bfe_for_full_read(struct ftnode_fetch_extra *bfe, FT ft);
//
// Helper function to fill a ftnode_fetch_extra with data
......@@ -816,79 +432,19 @@ static inline void fill_bfe_for_full_read(struct ftnode_fetch_extra *bfe, FT h)
// is known in advance, e.g. for keysrange when the left and right key
// are in the same basement node.
//
static inline void fill_bfe_for_keymatch(
struct ftnode_fetch_extra *bfe,
FT h,
const DBT *left,
const DBT *right,
bool disable_prefetching,
bool read_all_partitions
)
{
paranoid_invariant(h->h->type == FT_CURRENT);
bfe->type = ftnode_fetch_keymatch;
bfe->h = h;
bfe->search = nullptr;
toku_init_dbt(&bfe->range_lock_left_key);
toku_init_dbt(&bfe->range_lock_right_key);
if (left) {
toku_copyref_dbt(&bfe->range_lock_left_key, *left);
}
if (right) {
toku_copyref_dbt(&bfe->range_lock_right_key, *right);
}
bfe->left_is_neg_infty = left == nullptr;
bfe->right_is_pos_infty = right == nullptr;
bfe->child_to_read = -1;
bfe->disable_prefetching = disable_prefetching;
bfe->read_all_partitions = read_all_partitions;
bfe->bytes_read = 0;
bfe->io_time = 0;
bfe->deserialize_time = 0;
bfe->decompress_time = 0;
}
void fill_bfe_for_keymatch(struct ftnode_fetch_extra *bfe, FT ft,
const DBT *left, const DBT *right,
bool disable_prefetching, bool read_all_partitions);
//
// Helper function to fill a ftnode_fetch_extra with data
// that will tell the fetch callback that some subset of the node
// necessary. Used in cases where some of the node is required
// such as for a point query.
//
static inline void fill_bfe_for_subset_read(
struct ftnode_fetch_extra *bfe,
FT h,
ft_search *search,
const DBT *left,
const DBT *right,
bool left_is_neg_infty,
bool right_is_pos_infty,
bool disable_prefetching,
bool read_all_partitions
)
{
paranoid_invariant(h->h->type == FT_CURRENT);
bfe->type = ftnode_fetch_subset;
bfe->h = h;
bfe->search = search;
toku_init_dbt(&bfe->range_lock_left_key);
toku_init_dbt(&bfe->range_lock_right_key);
if (left) {
toku_copyref_dbt(&bfe->range_lock_left_key, *left);
}
if (right) {
toku_copyref_dbt(&bfe->range_lock_right_key, *right);
}
bfe->left_is_neg_infty = left_is_neg_infty;
bfe->right_is_pos_infty = right_is_pos_infty;
bfe->child_to_read = -1;
bfe->disable_prefetching = disable_prefetching;
bfe->read_all_partitions = read_all_partitions;
bfe->bytes_read = 0;
bfe->io_time = 0;
bfe->deserialize_time = 0;
bfe->decompress_time = 0;
}
void fill_bfe_for_subset_read(struct ftnode_fetch_extra *bfe, FT ft, ft_search *search,
const DBT *left, const DBT *right,
bool left_is_neg_infty, bool right_is_pos_infty,
bool disable_prefetching, bool read_all_partitions);
//
// Helper function to fill a ftnode_fetch_extra with data
......@@ -896,41 +452,11 @@ static inline void fill_bfe_for_subset_read(
// necessary, only the pivots and/or subtree estimates.
// Currently used for stat64.
//
static inline void fill_bfe_for_min_read(struct ftnode_fetch_extra *bfe, FT h) {
paranoid_invariant(h->h->type == FT_CURRENT);
bfe->type = ftnode_fetch_none;
bfe->h = h;
bfe->search = NULL;
toku_init_dbt(&bfe->range_lock_left_key);
toku_init_dbt(&bfe->range_lock_right_key);
bfe->left_is_neg_infty = false;
bfe->right_is_pos_infty = false;
bfe->child_to_read = -1;
bfe->disable_prefetching = false;
bfe->read_all_partitions = false;
bfe->bytes_read = 0;
bfe->io_time = 0;
bfe->deserialize_time = 0;
bfe->decompress_time = 0;
}
static inline void destroy_bfe_for_prefetch(struct ftnode_fetch_extra *bfe) {
paranoid_invariant(bfe->type == ftnode_fetch_prefetch);
toku_destroy_dbt(&bfe->range_lock_left_key);
toku_destroy_dbt(&bfe->range_lock_right_key);
}
// this is in a strange place because it needs the cursor struct to be defined
void fill_bfe_for_prefetch(struct ftnode_fetch_extra *bfe,
FT h,
struct ft_cursor *c);
void fill_bfe_for_min_read(struct ftnode_fetch_extra *bfe, FT ft);
struct ancestors {
FTNODE node; // This is the root node if next is NULL.
int childnum; // which buffer holds messages destined to the node whose ancestors this list represents.
struct ancestors *next; // Parent of this node (so next->node.(next->childnum) refers to this node).
};
typedef struct ancestors *ANCESTORS;
void fill_bfe_for_prefetch(struct ftnode_fetch_extra *bfe, FT ft, struct ft_cursor *cursor);
void destroy_bfe_for_prefetch(struct ftnode_fetch_extra *bfe);
struct pivot_bounds {
const DBT * const lower_bound_exclusive;
......@@ -938,21 +464,9 @@ struct pivot_bounds {
};
typedef struct pivot_bounds const * const PIVOT_BOUNDS;
__attribute__((nonnull))
void toku_move_ftnode_messages_to_stale(FT ft, FTNODE node);
void toku_apply_ancestors_messages_to_node (FT_HANDLE t, FTNODE node, ANCESTORS ancestors, struct pivot_bounds const * const bounds, bool* msgs_applied, int child_to_read);
__attribute__((nonnull))
bool toku_ft_leaf_needs_ancestors_messages(FT ft, FTNODE node, ANCESTORS ancestors, struct pivot_bounds const * const bounds, MSN *const max_msn_in_path, int child_to_read);
__attribute__((nonnull))
void toku_ft_bn_update_max_msn(FTNODE node, MSN max_msn_applied, int child_to_read);
int
toku_ft_search_which_child(
DESCRIPTOR desc,
ft_compare_func cmp,
FTNODE node,
ft_search *search
);
const DBT *prepivotkey (FTNODE node, int childnum, const DBT * const lower_bound_exclusive);
const DBT *postpivotkey (FTNODE node, int childnum, const DBT * const upper_bound_inclusive);
struct pivot_bounds next_pivot_keys (FTNODE node, int childnum, struct pivot_bounds const * const old_pb);
bool
toku_bfe_wants_child_available (struct ftnode_fetch_extra* bfe, int childnum);
......@@ -967,29 +481,6 @@ toku_bfe_rightmost_child_wanted(struct ftnode_fetch_extra *bfe, FTNODE node);
// put the ftnode into the cache table
void toku_create_new_ftnode (FT_HANDLE t, FTNODE *result, int height, int n_children);
// Effect: Fill in N as an empty ftnode.
void toku_initialize_empty_ftnode (FTNODE n, BLOCKNUM nodename, int height, int num_children,
int layout_version, unsigned int flags);
int toku_ftnode_which_child(FTNODE node, const DBT *k,
DESCRIPTOR desc, ft_compare_func cmp)
__attribute__((__warn_unused_result__));
/**
* Finds the next child for HOT to flush to, given that everything up to
* and including k has been flattened.
*
* If k falls between pivots in node, then we return the childnum where k
* lies.
*
* If k is equal to some pivot, then we return the next (to the right)
* childnum.
*/
int toku_ftnode_hot_next_child(FTNODE node,
const DBT *k,
DESCRIPTOR desc,
ft_compare_func cmp);
/* Stuff for testing */
// toku_testsetup_initialize() must be called before any other test_setup_xxx() functions are called.
void toku_testsetup_initialize(void);
......@@ -1174,57 +665,6 @@ typedef struct {
void toku_ft_get_status(FT_STATUS);
void
toku_ft_bn_apply_msg_once(
BASEMENTNODE bn,
const FT_MSG msg,
uint32_t idx,
uint32_t le_keylen,
LEAFENTRY le,
txn_gc_info *gc_info,
uint64_t *workdonep,
STAT64INFO stats_to_update
);
void
toku_ft_bn_apply_msg(
ft_compare_func compare_fun,
ft_update_func update_fun,
DESCRIPTOR desc,
BASEMENTNODE bn,
FT_MSG msg,
txn_gc_info *gc_info,
uint64_t *workdone,
STAT64INFO stats_to_update
);
void
toku_ft_leaf_apply_msg(
ft_compare_func compare_fun,
ft_update_func update_fun,
DESCRIPTOR desc,
FTNODE node,
int target_childnum,
FT_MSG msg,
txn_gc_info *gc_info,
uint64_t *workdone,
STAT64INFO stats_to_update
);
void
toku_ft_node_put_msg(
ft_compare_func compare_fun,
ft_update_func update_fun,
DESCRIPTOR desc,
FTNODE node,
int target_childnum,
FT_MSG msg,
bool is_fresh,
txn_gc_info *gc_info,
size_t flow_deltas[],
STAT64INFO stats_to_update
);
void toku_flusher_thread_set_callback(void (*callback_f)(int, void*), void* extra);
int toku_upgrade_subtree_estimates_to_stat64info(int fd, FT h) __attribute__((nonnull));
......@@ -1245,6 +685,5 @@ typedef int (*FT_GET_CALLBACK_FUNCTION)(ITEMLEN keylen, bytevec key, ITEMLEN val
typedef bool (*FT_CHECK_INTERRUPT_CALLBACK)(void *extra);
struct ft_search;
struct ft_cursor;
int toku_ft_search(FT_HANDLE ft_handle, ft_search *search, FT_GET_CALLBACK_FUNCTION getf, void *getf_v, struct ft_cursor *ftcursor, bool can_bulk_fetch);
......@@ -89,8 +89,8 @@ PATENT RIGHTS GRANT:
#ident "Copyright (c) 2007-2013 Tokutek Inc. All rights reserved."
#ident "The technology is licensed by the Massachusetts Institute of Technology, Rutgers State University of New Jersey, and the Research Foundation of State University of New York at Stony Brook under United States of America Serial No. 11/760379 and to the patents and/or patent applications resulting from it."
#include <ft-internal.h>
#include <db.h>
#include "ft/node.h"
#include "ft-internal.h"
/*
* ft-node-deserialize.c -
......
......@@ -206,6 +206,7 @@ basement nodes, bulk fetch, and partial fetch:
#include "ft-cachetable-wrappers.h"
#include "ft-flusher.h"
#include "ft-internal.h"
#include "node.h"
#include "ft_layout_version.h"
#include "log-internal.h"
#include "sub_block.h"
......@@ -431,99 +432,6 @@ void toku_note_deserialized_basement_node(bool fixed_key_size) {
}
}
bool is_entire_node_in_memory(FTNODE node) {
for (int i = 0; i < node->n_children; i++) {
if(BP_STATE(node,i) != PT_AVAIL) {
return false;
}
}
return true;
}
void
toku_assert_entire_node_in_memory(FTNODE UU() node) {
paranoid_invariant(is_entire_node_in_memory(node));
}
uint32_t
get_leaf_num_entries(FTNODE node) {
uint32_t result = 0;
int i;
toku_assert_entire_node_in_memory(node);
for ( i = 0; i < node->n_children; i++) {
result += BLB_DATA(node, i)->num_klpairs();
}
return result;
}
static enum reactivity
get_leaf_reactivity (FTNODE node, uint32_t nodesize) {
enum reactivity re = RE_STABLE;
toku_assert_entire_node_in_memory(node);
paranoid_invariant(node->height==0);
unsigned int size = toku_serialize_ftnode_size(node);
if (size > nodesize && get_leaf_num_entries(node) > 1) {
re = RE_FISSIBLE;
}
else if ((size*4) < nodesize && !BLB_SEQINSERT(node, node->n_children-1)) {
re = RE_FUSIBLE;
}
return re;
}
enum reactivity
get_nonleaf_reactivity(FTNODE node, unsigned int fanout) {
paranoid_invariant(node->height>0);
int n_children = node->n_children;
if (n_children > (int) fanout) return RE_FISSIBLE;
if (n_children*4 < (int) fanout) return RE_FUSIBLE;
return RE_STABLE;
}
enum reactivity
get_node_reactivity(FT ft, FTNODE node) {
toku_assert_entire_node_in_memory(node);
if (node->height==0)
return get_leaf_reactivity(node, ft->h->nodesize);
else
return get_nonleaf_reactivity(node, ft->h->fanout);
}
unsigned int
toku_bnc_nbytesinbuf(NONLEAF_CHILDINFO bnc)
{
return bnc->msg_buffer.buffer_size_in_use();
}
// return true if the size of the buffers plus the amount of work done is large enough. (But return false if there is nothing to be flushed (the buffers empty)).
bool
toku_ft_nonleaf_is_gorged (FTNODE node, uint32_t nodesize) {
uint64_t size = toku_serialize_ftnode_size(node);
bool buffers_are_empty = true;
toku_assert_entire_node_in_memory(node);
//
// the nonleaf node is gorged if the following holds true:
// - the buffers are non-empty
// - the total workdone by the buffers PLUS the size of the buffers
// is greater than nodesize (which as of Maxwell should be
// 4MB)
//
paranoid_invariant(node->height > 0);
for (int child = 0; child < node->n_children; ++child) {
size += BP_WORKDONE(node, child);
}
for (int child = 0; child < node->n_children; ++child) {
if (toku_bnc_nbytesinbuf(BNC(node, child)) > 0) {
buffers_are_empty = false;
break;
}
}
return ((size > nodesize)
&&
(!buffers_are_empty));
}
static void ft_verify_flags(FT UU(ft), FTNODE UU(node)) {
paranoid_invariant(ft->h->flags == node->flags);
}
......@@ -536,13 +444,7 @@ uint32_t compute_child_fullhash (CACHEFILE cf, FTNODE node, int childnum) {
return toku_cachetable_hash(cf, BP_BLOCKNUM(node, childnum));
}
int
toku_bnc_n_entries(NONLEAF_CHILDINFO bnc)
{
return bnc->msg_buffer.num_entries();
}
static const DBT *prepivotkey (FTNODE node, int childnum, const DBT * const lower_bound_exclusive) {
const DBT *prepivotkey (FTNODE node, int childnum, const DBT * const lower_bound_exclusive) {
if (childnum==0)
return lower_bound_exclusive;
else {
......@@ -550,53 +452,26 @@ static const DBT *prepivotkey (FTNODE node, int childnum, const DBT * const lowe
}
}
static const DBT *postpivotkey (FTNODE node, int childnum, const DBT * const upper_bound_inclusive) {
const DBT *postpivotkey (FTNODE node, int childnum, const DBT * const upper_bound_inclusive) {
if (childnum+1 == node->n_children)
return upper_bound_inclusive;
else {
return &node->childkeys[childnum];
}
}
static struct pivot_bounds next_pivot_keys (FTNODE node, int childnum, struct pivot_bounds const * const old_pb) {
struct pivot_bounds next_pivot_keys (FTNODE node, int childnum, struct pivot_bounds const * const old_pb) {
struct pivot_bounds pb = {.lower_bound_exclusive = prepivotkey(node, childnum, old_pb->lower_bound_exclusive),
.upper_bound_inclusive = postpivotkey(node, childnum, old_pb->upper_bound_inclusive)};
return pb;
}
// how much memory does this child buffer consume?
long
toku_bnc_memory_size(NONLEAF_CHILDINFO bnc)
{
return (sizeof(*bnc) +
bnc->msg_buffer.memory_footprint() +
bnc->fresh_message_tree.memory_size() +
bnc->stale_message_tree.memory_size() +
bnc->broadcast_list.memory_size());
}
// how much memory in this child buffer holds useful data?
// originally created solely for use by test program(s).
long
toku_bnc_memory_used(NONLEAF_CHILDINFO bnc)
{
return (sizeof(*bnc) +
bnc->msg_buffer.memory_size_in_use() +
bnc->fresh_message_tree.memory_size() +
bnc->stale_message_tree.memory_size() +
bnc->broadcast_list.memory_size());
}
static long
get_avail_internal_node_partition_size(FTNODE node, int i)
{
static long get_avail_internal_node_partition_size(FTNODE node, int i) {
paranoid_invariant(node->height > 0);
return toku_bnc_memory_size(BNC(node, i));
}
static long
ftnode_cachepressure_size(FTNODE node)
{
static long ftnode_cachepressure_size(FTNODE node) {
long retval = 0;
bool totally_empty = true;
if (node->height == 0) {
......@@ -802,45 +677,6 @@ void toku_ft_status_update_flush_reason(FTNODE node,
}
}
static void ftnode_update_disk_stats(
FTNODE ftnode,
FT ft,
bool for_checkpoint
)
{
STAT64INFO_S deltas = ZEROSTATS;
// capture deltas before rebalancing basements for serialization
deltas = toku_get_and_clear_basement_stats(ftnode);
// locking not necessary here with respect to checkpointing
// in Clayface (because of the pending lock and cachetable lock
// in toku_cachetable_begin_checkpoint)
// essentially, if we are dealing with a for_checkpoint
// parameter in a function that is called by the flush_callback,
// then the cachetable needs to ensure that this is called in a safe
// manner that does not interfere with the beginning
// of a checkpoint, which it does with the cachetable lock
// and pending lock
toku_ft_update_stats(&ft->h->on_disk_stats, deltas);
if (for_checkpoint) {
toku_ft_update_stats(&ft->checkpoint_header->on_disk_stats, deltas);
}
}
static void ftnode_clone_partitions(FTNODE node, FTNODE cloned_node) {
for (int i = 0; i < node->n_children; i++) {
BP_BLOCKNUM(cloned_node,i) = BP_BLOCKNUM(node,i);
paranoid_invariant(BP_STATE(node,i) == PT_AVAIL);
BP_STATE(cloned_node,i) = PT_AVAIL;
BP_WORKDONE(cloned_node, i) = BP_WORKDONE(node, i);
if (node->height == 0) {
set_BLB(cloned_node, i, toku_clone_bn(BLB(node,i)));
}
else {
set_BNC(cloned_node, i, toku_clone_nl(BNC(node,i)));
}
}
}
void toku_ftnode_checkpoint_complete_callback(void *value_data) {
FTNODE node = static_cast<FTNODE>(value_data);
if (node->height > 0) {
......@@ -864,14 +700,14 @@ void toku_ftnode_clone_callback(
)
{
FTNODE node = static_cast<FTNODE>(value_data);
toku_assert_entire_node_in_memory(node);
toku_ftnode_assert_fully_in_memory(node);
FT ft = static_cast<FT>(write_extraargs);
FTNODE XCALLOC(cloned_node);
if (node->height == 0) {
// set header stats, must be done before rebalancing
ftnode_update_disk_stats(node, ft, for_checkpoint);
toku_ftnode_update_disk_stats(node, ft, for_checkpoint);
// rebalance the leaf node
rebalance_ftnode_leaf(node, ft->h->basementnodesize);
toku_ftnode_leaf_rebalance(node, ft->h->basementnodesize);
}
cloned_node->oldest_referenced_xid_known = node->oldest_referenced_xid_known;
......@@ -900,7 +736,7 @@ void toku_ftnode_clone_callback(
toku_move_ftnode_messages_to_stale(ft, node);
}
// clone partition
ftnode_clone_partitions(node, cloned_node);
toku_ftnode_clone_partitions(node, cloned_node);
// clear dirty bit
node->dirty = 0;
......@@ -917,8 +753,6 @@ void toku_ftnode_clone_callback(
*cloned_value_data = cloned_node;
}
static void ft_leaf_run_gc(FT ft, FTNODE node);
void toku_ftnode_flush_callback(
CACHEFILE UU(cachefile),
int fd,
......@@ -940,14 +774,14 @@ void toku_ftnode_flush_callback(
assert(ftnode->thisnodename.b==nodename.b);
int height = ftnode->height;
if (write_me) {
toku_assert_entire_node_in_memory(ftnode);
toku_ftnode_assert_fully_in_memory(ftnode);
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);
toku_ftnode_leaf_run_gc(h, ftnode);
if (!is_clone) {
ftnode_update_disk_stats(ftnode, h, for_checkpoint);
toku_ftnode_update_disk_stats(ftnode, h, for_checkpoint);
}
}
int r = toku_serialize_ftnode_to(fd, ftnode->thisnodename, ftnode, ndd, !is_clone, h, for_checkpoint);
......@@ -1091,12 +925,8 @@ void toku_ftnode_pe_est_callback(
return;
}
static void ft_bnc_move_messages_to_stale(FT ft, NONLEAF_CHILDINFO bnc);
// replace the child buffer with a compressed version of itself.
static void
compress_internal_node_partition(FTNODE node, int i, enum toku_compression_method compression_method)
{
static void compress_internal_node_partition(FTNODE node, int i, enum toku_compression_method compression_method) {
// if we should evict, compress the
// message buffer into a sub_block
assert(BP_STATE(node, i) == PT_AVAIL);
......@@ -1110,24 +940,6 @@ compress_internal_node_partition(FTNODE node, int i, enum toku_compression_metho
BP_STATE(node,i) = PT_COMPRESSED;
}
void toku_evict_bn_from_memory(FTNODE node, int childnum, FT h) {
// free the basement node
assert(!node->dirty);
BASEMENTNODE bn = BLB(node, childnum);
toku_ft_decrease_stats(&h->in_memory_stats, bn->stat64_delta);
destroy_basement_node(bn);
set_BNULL(node, childnum);
BP_STATE(node, childnum) = PT_ON_DISK;
}
BASEMENTNODE toku_detach_bn(FTNODE node, int childnum) {
assert(BP_STATE(node, childnum) == PT_AVAIL);
BASEMENTNODE bn = BLB(node, childnum);
set_BNULL(node, childnum);
BP_STATE(node, childnum) = PT_ON_DISK;
return bn;
}
// callback for partially evicting a node
int toku_ftnode_pe_callback(void *ftnode_pv, PAIR_ATTR old_attr, void *write_extraargs,
void (*finalize)(PAIR_ATTR new_attr, void *extra), void *finalize_extra) {
......@@ -1171,7 +983,7 @@ int toku_ftnode_pe_callback(void *ftnode_pv, PAIR_ATTR old_attr, void *write_ext
// 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);
toku_ft_bnc_move_messages_to_stale(ft, bnc);
compress_internal_node_partition(
node,
i,
......@@ -1466,6 +1278,34 @@ void toku_ft_status_update_deserialize_times(FTNODE node, tokutime_t deserialize
}
}
void toku_ft_status_note_msn_discard(void) {
STATUS_INC(FT_MSN_DISCARDS, 1);
}
void toku_ft_status_note_update(bool broadcast) {
if (broadcast) {
STATUS_INC(FT_UPDATES_BROADCAST, 1);
} else {
STATUS_INC(FT_UPDATES, 1);
}
}
void toku_ft_status_note_msg_bytes_out(size_t buffsize) {
STATUS_INC(FT_MSG_BYTES_OUT, buffsize);
STATUS_INC(FT_MSG_BYTES_CURR, -buffsize);
}
void toku_ft_status_note_ftnode(int height, bool created) {
if (created) {
if (height == 0) {
STATUS_INC(FT_CREATE_LEAF, 1);
} else {
STATUS_INC(FT_CREATE_NONLEAF, 1);
}
} else {
// created = false means destroyed
}
}
// callback for partially reading a node
// could have just used toku_ftnode_fetch_callback, but wanted to separate the two cases to separate functions
int toku_ftnode_pf_callback(void* ftnode_pv, void* disk_data, void* read_extraargs, int fd, PAIR_ATTR* sizep) {
......@@ -1522,118 +1362,131 @@ int toku_ftnode_pf_callback(void* ftnode_pv, void* disk_data, void* read_extraar
return 0;
}
struct msg_leafval_heaviside_extra {
ft_compare_func compare_fun;
DESCRIPTOR desc;
DBT const * const key;
};
//TODO: #1125 optimize
static int
toku_msg_leafval_heaviside(DBT const &kdbt, const struct msg_leafval_heaviside_extra &be) {
int toku_msg_leafval_heaviside(DBT const &kdbt, const struct toku_msg_leafval_heaviside_extra &be) {
FAKE_DB(db, be.desc);
DBT const * const key = be.key;
DBT const *const key = be.key;
return be.compare_fun(&db, &kdbt, key);
}
static int
ft_compare_pivot(DESCRIPTOR desc, ft_compare_func cmp, const DBT *key, const DBT *pivot)
{
int r;
FAKE_DB(db, desc);
r = cmp(&db, key, pivot);
return r;
void fill_bfe_for_full_read(struct ftnode_fetch_extra *bfe, FT h) {
bfe->type = ftnode_fetch_all;
bfe->h = h;
bfe->search = nullptr;
toku_init_dbt(&bfe->range_lock_left_key);
toku_init_dbt(&bfe->range_lock_right_key);
bfe->left_is_neg_infty = false;
bfe->right_is_pos_infty = false;
bfe->child_to_read = -1;
bfe->disable_prefetching = false;
bfe->read_all_partitions = false;
bfe->bytes_read = 0;
bfe->io_time = 0;
bfe->deserialize_time = 0;
bfe->decompress_time = 0;
}
// destroys the internals of the ftnode, but it does not free the values
// that are stored
// this is common functionality for toku_ftnode_free and rebalance_ftnode_leaf
// MUST NOT do anything besides free the structures that have been allocated
void toku_destroy_ftnode_internals(FTNODE node)
{
for (int i=0; i<node->n_children-1; i++) {
toku_destroy_dbt(&node->childkeys[i]);
void fill_bfe_for_keymatch(struct ftnode_fetch_extra *bfe, FT h,
const DBT *left, const DBT *right,
bool disable_prefetching, bool read_all_partitions) {
paranoid_invariant(h->h->type == FT_CURRENT);
bfe->type = ftnode_fetch_keymatch;
bfe->h = h;
bfe->search = nullptr;
toku_init_dbt(&bfe->range_lock_left_key);
toku_init_dbt(&bfe->range_lock_right_key);
if (left) {
toku_copyref_dbt(&bfe->range_lock_left_key, *left);
}
toku_free(node->childkeys);
node->childkeys = NULL;
for (int i=0; i < node->n_children; i++) {
if (BP_STATE(node,i) == PT_AVAIL) {
if (node->height > 0) {
destroy_nonleaf_childinfo(BNC(node,i));
} else {
destroy_basement_node(BLB(node, i));
}
} else if (BP_STATE(node,i) == PT_COMPRESSED) {
SUB_BLOCK sb = BSB(node,i);
toku_free(sb->compressed_ptr);
toku_free(sb);
} else {
paranoid_invariant(is_BNULL(node, i));
}
set_BNULL(node, i);
if (right) {
toku_copyref_dbt(&bfe->range_lock_right_key, *right);
}
toku_free(node->bp);
node->bp = NULL;
bfe->left_is_neg_infty = left == nullptr;
bfe->right_is_pos_infty = right == nullptr;
bfe->child_to_read = -1;
bfe->disable_prefetching = disable_prefetching;
bfe->read_all_partitions = read_all_partitions;
bfe->bytes_read = 0;
bfe->io_time = 0;
bfe->deserialize_time = 0;
bfe->decompress_time = 0;
}
/* Frees a node, including all the stuff in the hash table. */
void toku_ftnode_free(FTNODE *nodep) {
FTNODE node = *nodep;
if (node->height == 0) {
STATUS_INC(FT_DESTROY_LEAF, 1);
} else {
STATUS_INC(FT_DESTROY_NONLEAF, 1);
void fill_bfe_for_subset_read(struct ftnode_fetch_extra *bfe, FT h, ft_search *search,
const DBT *left, const DBT *right,
bool left_is_neg_infty, bool right_is_pos_infty,
bool disable_prefetching, bool read_all_partitions) {
paranoid_invariant(h->h->type == FT_CURRENT);
bfe->type = ftnode_fetch_subset;
bfe->h = h;
bfe->search = search;
toku_init_dbt(&bfe->range_lock_left_key);
toku_init_dbt(&bfe->range_lock_right_key);
if (left) {
toku_copyref_dbt(&bfe->range_lock_left_key, *left);
}
toku_destroy_ftnode_internals(node);
toku_free(node);
*nodep = nullptr;
if (right) {
toku_copyref_dbt(&bfe->range_lock_right_key, *right);
}
bfe->left_is_neg_infty = left_is_neg_infty;
bfe->right_is_pos_infty = right_is_pos_infty;
bfe->child_to_read = -1;
bfe->disable_prefetching = disable_prefetching;
bfe->read_all_partitions = read_all_partitions;
bfe->bytes_read = 0;
bfe->io_time = 0;
bfe->deserialize_time = 0;
bfe->decompress_time = 0;
}
void
toku_initialize_empty_ftnode (FTNODE n, BLOCKNUM nodename, int height, int num_children, int layout_version, unsigned int flags)
// Effect: Fill in N as an empty ftnode.
{
paranoid_invariant(layout_version != 0);
paranoid_invariant(height >= 0);
if (height == 0) {
STATUS_INC(FT_CREATE_LEAF, 1);
} else {
STATUS_INC(FT_CREATE_NONLEAF, 1);
}
void fill_bfe_for_min_read(struct ftnode_fetch_extra *bfe, FT ft) {
paranoid_invariant(ft->h->type == FT_CURRENT);
bfe->type = ftnode_fetch_none;
bfe->h = ft;
bfe->search = nullptr;
toku_init_dbt(&bfe->range_lock_left_key);
toku_init_dbt(&bfe->range_lock_right_key);
bfe->left_is_neg_infty = false;
bfe->right_is_pos_infty = false;
bfe->child_to_read = -1;
bfe->disable_prefetching = false;
bfe->read_all_partitions = false;
bfe->bytes_read = 0;
bfe->io_time = 0;
bfe->deserialize_time = 0;
bfe->decompress_time = 0;
}
n->max_msn_applied_to_node_on_disk = ZERO_MSN; // correct value for root node, harmless for others
n->flags = flags;
n->thisnodename = nodename;
n->layout_version = layout_version;
n->layout_version_original = layout_version;
n->layout_version_read_from_disk = layout_version;
n->height = height;
n->totalchildkeylens = 0;
n->childkeys = 0;
n->bp = 0;
n->n_children = num_children;
n->oldest_referenced_xid_known = TXNID_NONE;
if (num_children > 0) {
XMALLOC_N(num_children-1, n->childkeys);
XMALLOC_N(num_children, n->bp);
for (int i = 0; i < num_children; i++) {
BP_BLOCKNUM(n,i).b=0;
BP_STATE(n,i) = PT_INVALID;
BP_WORKDONE(n,i) = 0;
BP_INIT_TOUCHED_CLOCK(n, i);
set_BNULL(n,i);
if (height > 0) {
set_BNC(n, i, toku_create_empty_nl());
} else {
set_BLB(n, i, toku_create_empty_bn());
}
void fill_bfe_for_prefetch(struct ftnode_fetch_extra *bfe, FT ft, struct ft_cursor *cursor) {
paranoid_invariant(ft->h->type == FT_CURRENT);
bfe->type = ftnode_fetch_prefetch;
bfe->h = ft;
bfe->search = nullptr;
toku_init_dbt(&bfe->range_lock_left_key);
toku_init_dbt(&bfe->range_lock_right_key);
const DBT *left = &cursor->range_lock_left_key;
if (left->data) {
toku_clone_dbt(&bfe->range_lock_left_key, *left);
}
const DBT *right = &cursor->range_lock_right_key;
if (right->data) {
toku_clone_dbt(&bfe->range_lock_right_key, *right);
}
n->dirty = 1; // special case exception, it's okay to mark as dirty because the basements are empty
bfe->left_is_neg_infty = cursor->left_is_neg_infty;
bfe->right_is_pos_infty = cursor->right_is_pos_infty;
bfe->child_to_read = -1;
bfe->disable_prefetching = cursor->disable_prefetching;
bfe->read_all_partitions = false;
bfe->bytes_read = 0;
bfe->io_time = 0;
bfe->deserialize_time = 0;
bfe->decompress_time = 0;
}
void destroy_bfe_for_prefetch(struct ftnode_fetch_extra *bfe) {
paranoid_invariant(bfe->type == ftnode_fetch_prefetch);
toku_destroy_dbt(&bfe->range_lock_left_key);
toku_destroy_dbt(&bfe->range_lock_right_key);
}
static void
......@@ -1703,1031 +1556,17 @@ ft_init_new_root(FT ft, FTNODE oldroot, FTNODE *newrootp)
&bfe,
PL_WRITE_EXPENSIVE, // may_modify_node
newrootp,
true
);
}
static void
init_childinfo(FTNODE node, int childnum, FTNODE child) {
BP_BLOCKNUM(node,childnum) = child->thisnodename;
BP_STATE(node,childnum) = PT_AVAIL;
BP_WORKDONE(node, childnum) = 0;
set_BNC(node, childnum, toku_create_empty_nl());
}
static void
init_childkey(FTNODE node, int childnum, const DBT *pivotkey) {
toku_clone_dbt(&node->childkeys[childnum], *pivotkey);
node->totalchildkeylens += pivotkey->size;
}
// Used only by test programs: append a child node to a parent node
void
toku_ft_nonleaf_append_child(FTNODE node, FTNODE child, const DBT *pivotkey) {
int childnum = node->n_children;
node->n_children++;
XREALLOC_N(node->n_children, node->bp);
init_childinfo(node, childnum, child);
XREALLOC_N(node->n_children-1, node->childkeys);
if (pivotkey) {
invariant(childnum > 0);
init_childkey(node, childnum-1, pivotkey);
}
node->dirty = 1;
}
void
toku_ft_bn_apply_msg_once (
BASEMENTNODE bn,
const FT_MSG msg,
uint32_t idx,
uint32_t le_keylen,
LEAFENTRY le,
txn_gc_info *gc_info,
uint64_t *workdone,
STAT64INFO stats_to_update
)
// Effect: Apply msg to leafentry (msn is ignored)
// Calculate work done by message on leafentry and add it to caller's workdone counter.
// idx is the location where it goes
// le is old leafentry
{
size_t newsize=0, oldsize=0, workdone_this_le=0;
LEAFENTRY new_le=0;
int64_t numbytes_delta = 0; // how many bytes of user data (not including overhead) were added or deleted from this row
int64_t numrows_delta = 0; // will be +1 or -1 or 0 (if row was added or deleted or not)
uint32_t key_storage_size = ft_msg_get_keylen(msg) + sizeof(uint32_t);
if (le) {
oldsize = leafentry_memsize(le) + key_storage_size;
}
// toku_le_apply_msg() may call bn_data::mempool_malloc_and_update_dmt() to allocate more space.
// That means le is guaranteed to not cause a sigsegv but it may point to a mempool that is
// no longer in use. We'll have to release the old mempool later.
toku_le_apply_msg(
msg,
le,
&bn->data_buffer,
idx,
le_keylen,
gc_info,
&new_le,
&numbytes_delta
);
// at this point, we cannot trust cmd->u.id.key to be valid.
// The dmt may have realloced its mempool and freed the one containing key.
newsize = new_le ? (leafentry_memsize(new_le) + + key_storage_size) : 0;
if (le && new_le) {
workdone_this_le = (oldsize > newsize ? oldsize : newsize); // work done is max of le size before and after message application
} else { // we did not just replace a row, so ...
if (le) {
// ... we just deleted a row ...
workdone_this_le = oldsize;
numrows_delta = -1;
}
if (new_le) {
// ... or we just added a row
workdone_this_le = newsize;
numrows_delta = 1;
}
}
if (workdone) { // test programs may call with NULL
*workdone += workdone_this_le;
}
// now update stat64 statistics
bn->stat64_delta.numrows += numrows_delta;
bn->stat64_delta.numbytes += numbytes_delta;
// the only reason stats_to_update may be null is for tests
if (stats_to_update) {
stats_to_update->numrows += numrows_delta;
stats_to_update->numbytes += numbytes_delta;
}
}
static const uint32_t setval_tag = 0xee0ccb99; // this was gotten by doing "cat /dev/random|head -c4|od -x" to get a random number. We want to make sure that the user actually passes us the setval_extra_s that we passed in.
struct setval_extra_s {
uint32_t tag;
bool did_set_val;
int setval_r; // any error code that setval_fun wants to return goes here.
// need arguments for toku_ft_bn_apply_msg_once
BASEMENTNODE bn;
MSN msn; // captured from original message, not currently used
XIDS xids;
const DBT *key;
uint32_t idx;
uint32_t le_keylen;
LEAFENTRY le;
txn_gc_info *gc_info;
uint64_t * workdone; // set by toku_ft_bn_apply_msg_once()
STAT64INFO stats_to_update;
};
/*
* If new_val == NULL, we send a delete message instead of an insert.
* This happens here instead of in do_delete() for consistency.
* setval_fun() is called from handlerton, passing in svextra_v
* from setval_extra_s input arg to ft->update_fun().
*/
static void setval_fun (const DBT *new_val, void *svextra_v) {
struct setval_extra_s *CAST_FROM_VOIDP(svextra, svextra_v);
paranoid_invariant(svextra->tag==setval_tag);
paranoid_invariant(!svextra->did_set_val);
svextra->did_set_val = true;
{
// can't leave scope until toku_ft_bn_apply_msg_once if
// this is a delete
DBT val;
FT_MSG_S msg = { FT_NONE, svextra->msn, svextra->xids,
.u = { .id = {svextra->key, NULL} } };
if (new_val) {
msg.type = FT_INSERT;
msg.u.id.val = new_val;
} else {
msg.type = FT_DELETE_ANY;
toku_init_dbt(&val);
msg.u.id.val = &val;
}
toku_ft_bn_apply_msg_once(svextra->bn, &msg,
svextra->idx, svextra->le_keylen, svextra->le,
svextra->gc_info,
svextra->workdone, svextra->stats_to_update);
svextra->setval_r = 0;
}
}
// We are already past the msn filter (in toku_ft_bn_apply_msg(), which calls do_update()),
// so capturing the msn in the setval_extra_s is not strictly required. The alternative
// would be to put a dummy msn in the messages created by setval_fun(), but preserving
// the original msn seems cleaner and it preserves accountability at a lower layer.
static int do_update(ft_update_func update_fun, DESCRIPTOR desc, BASEMENTNODE bn, FT_MSG msg, uint32_t idx,
LEAFENTRY le,
void* keydata,
uint32_t keylen,
txn_gc_info *gc_info,
uint64_t * workdone,
STAT64INFO stats_to_update) {
LEAFENTRY le_for_update;
DBT key;
const DBT *keyp;
const DBT *update_function_extra;
DBT vdbt;
const DBT *vdbtp;
// the location of data depends whether this is a regular or
// broadcast update
if (msg->type == FT_UPDATE) {
// key is passed in with command (should be same as from le)
// update function extra is passed in with command
STATUS_INC(FT_UPDATES, 1);
keyp = msg->u.id.key;
update_function_extra = msg->u.id.val;
} else if (msg->type == FT_UPDATE_BROADCAST_ALL) {
// key is not passed in with broadcast, it comes from le
// update function extra is passed in with command
paranoid_invariant(le); // for broadcast updates, we just hit all leafentries
// so this cannot be null
paranoid_invariant(keydata);
paranoid_invariant(keylen);
paranoid_invariant(msg->u.id.key->size == 0);
STATUS_INC(FT_UPDATES_BROADCAST, 1);
keyp = toku_fill_dbt(&key, keydata, keylen);
update_function_extra = msg->u.id.val;
} else {
abort();
}
if (le && !le_latest_is_del(le)) {
// if the latest val exists, use it, and we'll use the leafentry later
uint32_t vallen;
void *valp = le_latest_val_and_len(le, &vallen);
vdbtp = toku_fill_dbt(&vdbt, valp, vallen);
} else {
// otherwise, the val and leafentry are both going to be null
vdbtp = NULL;
}
le_for_update = le;
struct setval_extra_s setval_extra = {setval_tag, false, 0, bn, msg->msn, msg->xids,
keyp, idx, keylen, le_for_update, gc_info,
workdone, stats_to_update};
// call handlerton's ft->update_fun(), which passes setval_extra to setval_fun()
FAKE_DB(db, desc);
int r = update_fun(
&db,
keyp,
vdbtp,
update_function_extra,
setval_fun, &setval_extra
);
if (r == 0) { r = setval_extra.setval_r; }
return r;
}
// Should be renamed as something like "apply_msg_to_basement()."
void
toku_ft_bn_apply_msg (
ft_compare_func compare_fun,
ft_update_func update_fun,
DESCRIPTOR desc,
BASEMENTNODE bn,
FT_MSG msg,
txn_gc_info *gc_info,
uint64_t *workdone,
STAT64INFO stats_to_update
)
// Effect:
// Put a msg into a leaf.
// Calculate work done by message on leafnode and add it to caller's workdone counter.
// The leaf could end up "too big" or "too small". The caller must fix that up.
{
LEAFENTRY storeddata;
void* key = NULL;
uint32_t keylen = 0;
uint32_t num_klpairs;
int r;
struct msg_leafval_heaviside_extra be = {compare_fun, desc, msg->u.id.key};
unsigned int doing_seqinsert = bn->seqinsert;
bn->seqinsert = 0;
switch (msg->type) {
case FT_INSERT_NO_OVERWRITE:
case FT_INSERT: {
uint32_t idx;
if (doing_seqinsert) {
idx = bn->data_buffer.num_klpairs();
DBT kdbt;
r = bn->data_buffer.fetch_key_and_len(idx-1, &kdbt.size, &kdbt.data);
if (r != 0) goto fz;
int cmp = toku_msg_leafval_heaviside(kdbt, be);
if (cmp >= 0) goto fz;
r = DB_NOTFOUND;
} else {
fz:
r = bn->data_buffer.find_zero<decltype(be), toku_msg_leafval_heaviside>(
be,
&storeddata,
&key,
&keylen,
&idx
);
}
if (r==DB_NOTFOUND) {
storeddata = 0;
} else {
assert_zero(r);
}
toku_ft_bn_apply_msg_once(bn, msg, idx, keylen, storeddata, gc_info, workdone, stats_to_update);
// if the insertion point is within a window of the right edge of
// the leaf then it is sequential
// window = min(32, number of leaf entries/16)
{
uint32_t s = bn->data_buffer.num_klpairs();
uint32_t w = s / 16;
if (w == 0) w = 1;
if (w > 32) w = 32;
// within the window?
if (s - idx <= w)
bn->seqinsert = doing_seqinsert + 1;
}
break;
}
case FT_DELETE_ANY:
case FT_ABORT_ANY:
case FT_COMMIT_ANY: {
uint32_t idx;
// Apply to all the matches
r = bn->data_buffer.find_zero<decltype(be), toku_msg_leafval_heaviside>(
be,
&storeddata,
&key,
&keylen,
&idx
);
if (r == DB_NOTFOUND) break;
assert_zero(r);
toku_ft_bn_apply_msg_once(bn, msg, idx, keylen, storeddata, gc_info, workdone, stats_to_update);
break;
}
case FT_OPTIMIZE_FOR_UPGRADE:
// fall through so that optimize_for_upgrade performs rest of the optimize logic
case FT_COMMIT_BROADCAST_ALL:
case FT_OPTIMIZE:
// Apply to all leafentries
num_klpairs = bn->data_buffer.num_klpairs();
for (uint32_t idx = 0; idx < num_klpairs; ) {
DBT curr_keydbt;
void* curr_keyp = NULL;
uint32_t curr_keylen = 0;
r = bn->data_buffer.fetch_klpair(idx, &storeddata, &curr_keylen, &curr_keyp);
assert_zero(r);
toku_fill_dbt(&curr_keydbt, curr_keyp, curr_keylen);
// because this is a broadcast message, we need
// to fill the key in the message that we pass into toku_ft_bn_apply_msg_once
msg->u.id.key = &curr_keydbt;
int deleted = 0;
if (!le_is_clean(storeddata)) { //If already clean, nothing to do.
toku_ft_bn_apply_msg_once(bn, msg, idx, curr_keylen, storeddata, gc_info, workdone, stats_to_update);
// at this point, we cannot trust msg->u.id.key to be valid.
uint32_t new_dmt_size = bn->data_buffer.num_klpairs();
if (new_dmt_size != num_klpairs) {
paranoid_invariant(new_dmt_size + 1 == num_klpairs);
//Item was deleted.
deleted = 1;
}
}
if (deleted)
num_klpairs--;
else
idx++;
}
paranoid_invariant(bn->data_buffer.num_klpairs() == num_klpairs);
break;
case FT_COMMIT_BROADCAST_TXN:
case FT_ABORT_BROADCAST_TXN:
// Apply to all leafentries if txn is represented
num_klpairs = bn->data_buffer.num_klpairs();
for (uint32_t idx = 0; idx < num_klpairs; ) {
DBT curr_keydbt;
void* curr_keyp = NULL;
uint32_t curr_keylen = 0;
r = bn->data_buffer.fetch_klpair(idx, &storeddata, &curr_keylen, &curr_keyp);
assert_zero(r);
toku_fill_dbt(&curr_keydbt, curr_keyp, curr_keylen);
// because this is a broadcast message, we need
// to fill the key in the message that we pass into toku_ft_bn_apply_msg_once
msg->u.id.key = &curr_keydbt;
int deleted = 0;
if (le_has_xids(storeddata, msg->xids)) {
toku_ft_bn_apply_msg_once(bn, msg, idx, curr_keylen, storeddata, gc_info, workdone, stats_to_update);
uint32_t new_dmt_size = bn->data_buffer.num_klpairs();
if (new_dmt_size != num_klpairs) {
paranoid_invariant(new_dmt_size + 1 == num_klpairs);
//Item was deleted.
deleted = 1;
}
}
if (deleted)
num_klpairs--;
else
idx++;
}
paranoid_invariant(bn->data_buffer.num_klpairs() == num_klpairs);
break;
case FT_UPDATE: {
uint32_t idx;
r = bn->data_buffer.find_zero<decltype(be), toku_msg_leafval_heaviside>(
be,
&storeddata,
&key,
&keylen,
&idx
);
if (r==DB_NOTFOUND) {
{
//Point to msg's copy of the key so we don't worry about le being freed
//TODO: 46 MAYBE Get rid of this when le_apply message memory is better handled
key = msg->u.id.key->data;
keylen = msg->u.id.key->size;
}
r = do_update(update_fun, desc, bn, msg, idx, NULL, NULL, 0, gc_info, workdone, stats_to_update);
} else if (r==0) {
r = do_update(update_fun, desc, bn, msg, idx, storeddata, key, keylen, gc_info, workdone, stats_to_update);
} // otherwise, a worse error, just return it
break;
}
case FT_UPDATE_BROADCAST_ALL: {
// apply to all leafentries.
uint32_t idx = 0;
uint32_t num_leafentries_before;
while (idx < (num_leafentries_before = bn->data_buffer.num_klpairs())) {
void* curr_key = nullptr;
uint32_t curr_keylen = 0;
r = bn->data_buffer.fetch_klpair(idx, &storeddata, &curr_keylen, &curr_key);
assert_zero(r);
//TODO: 46 replace this with something better than cloning key
// TODO: (Zardosht) This may be unnecessary now, due to how the key
// is handled in the bndata. Investigate and determine
char clone_mem[curr_keylen]; // only lasts one loop, alloca would overflow (end of function)
memcpy((void*)clone_mem, curr_key, curr_keylen);
curr_key = (void*)clone_mem;
// This is broken below. Have a compilation error checked
// in as a reminder
r = do_update(update_fun, desc, bn, msg, idx, storeddata, curr_key, curr_keylen, gc_info, workdone, stats_to_update);
assert_zero(r);
if (num_leafentries_before == bn->data_buffer.num_klpairs()) {
// we didn't delete something, so increment the index.
idx++;
}
}
break;
}
case FT_NONE: break; // don't do anything
}
return;
}
static inline int
key_msn_cmp(const DBT *a, const DBT *b, const MSN amsn, const MSN bmsn,
DESCRIPTOR descriptor, ft_compare_func key_cmp)
{
FAKE_DB(db, descriptor);
int r = key_cmp(&db, a, b);
if (r == 0) {
if (amsn.msn > bmsn.msn) {
r = +1;
} else if (amsn.msn < bmsn.msn) {
r = -1;
} else {
r = 0;
}
}
return r;
}
int
toku_msg_buffer_key_msn_heaviside(const int32_t &offset, const struct toku_msg_buffer_key_msn_heaviside_extra &extra)
{
MSN query_msn;
DBT query_key;
extra.msg_buffer->get_message_key_msn(offset, &query_key, &query_msn);
return key_msn_cmp(&query_key, extra.key, query_msn, extra.msn,
extra.desc, extra.cmp);
}
int
toku_msg_buffer_key_msn_cmp(const struct toku_msg_buffer_key_msn_cmp_extra &extra, const int32_t &ao, const int32_t &bo)
{
MSN amsn, bmsn;
DBT akey, bkey;
extra.msg_buffer->get_message_key_msn(ao, &akey, &amsn);
extra.msg_buffer->get_message_key_msn(bo, &bkey, &bmsn);
return key_msn_cmp(&akey, &bkey, amsn, bmsn,
extra.desc, extra.cmp);
}
// Effect: Enqueue the message represented by the parameters into the
// bnc's buffer, and put it in either the fresh or stale message tree,
// or the broadcast list.
static void bnc_insert_msg(NONLEAF_CHILDINFO bnc, FT_MSG msg, bool is_fresh, DESCRIPTOR desc, ft_compare_func cmp) {
int r = 0;
int32_t offset;
bnc->msg_buffer.enqueue(msg, is_fresh, &offset);
enum ft_msg_type type = ft_msg_get_type(msg);
if (ft_msg_type_applies_once(type)) {
DBT key;
toku_fill_dbt(&key, ft_msg_get_key(msg), ft_msg_get_keylen(msg));
struct toku_msg_buffer_key_msn_heaviside_extra extra = { .desc = desc, .cmp = cmp, .msg_buffer = &bnc->msg_buffer, .key = &key, .msn = msg->msn };
if (is_fresh) {
r = bnc->fresh_message_tree.insert<struct toku_msg_buffer_key_msn_heaviside_extra, toku_msg_buffer_key_msn_heaviside>(offset, extra, nullptr);
assert_zero(r);
} else {
r = bnc->stale_message_tree.insert<struct toku_msg_buffer_key_msn_heaviside_extra, toku_msg_buffer_key_msn_heaviside>(offset, extra, nullptr);
assert_zero(r);
}
} else {
invariant(ft_msg_type_applies_all(type) || ft_msg_type_does_nothing(type));
const uint32_t idx = bnc->broadcast_list.size();
r = bnc->broadcast_list.insert_at(offset, idx);
assert_zero(r);
}
}
// This is only exported for tests.
void toku_bnc_insert_msg(NONLEAF_CHILDINFO bnc, const void *key, ITEMLEN keylen, const void *data, ITEMLEN datalen, enum ft_msg_type type, MSN msn, XIDS xids, bool is_fresh, DESCRIPTOR desc, ft_compare_func cmp)
{
DBT k, v;
FT_MSG_S msg = {
type, msn, xids, .u = { .id = { toku_fill_dbt(&k, key, keylen), toku_fill_dbt(&v, data, datalen) } }
};
bnc_insert_msg(bnc, &msg, is_fresh, desc, cmp);
}
// append a msg to a nonleaf node's child buffer
static void ft_append_msg_to_child_buffer(ft_compare_func compare_fun, DESCRIPTOR desc, FTNODE node,
int childnum, FT_MSG msg, bool is_fresh) {
paranoid_invariant(BP_STATE(node,childnum) == PT_AVAIL);
bnc_insert_msg(BNC(node, childnum), msg, is_fresh, desc, compare_fun);
node->dirty = 1;
}
// This is only exported for tests.
void toku_ft_append_to_child_buffer(ft_compare_func compare_fun, DESCRIPTOR desc, FTNODE node, int childnum, enum ft_msg_type type, MSN msn, XIDS xids, bool is_fresh, const DBT *key, const DBT *val) {
FT_MSG_S msg = {
type, msn, xids, .u = { .id = { key, val } }
};
ft_append_msg_to_child_buffer(compare_fun, desc, node, childnum, &msg, is_fresh);
}
static void ft_nonleaf_msg_once_to_child(ft_compare_func compare_fun, DESCRIPTOR desc, FTNODE node, int target_childnum, FT_MSG msg, bool is_fresh, size_t flow_deltas[])
// Previously we had passive aggressive promotion, but that causes a lot of I/O a the checkpoint. So now we are just putting it in the buffer here.
// Also we don't worry about the node getting overfull here. It's the caller's problem.
{
unsigned int childnum = (target_childnum >= 0
? target_childnum
: toku_ftnode_which_child(node, msg->u.id.key, desc, compare_fun));
ft_append_msg_to_child_buffer(compare_fun, desc, node, childnum, msg, is_fresh);
NONLEAF_CHILDINFO bnc = BNC(node, childnum);
bnc->flow[0] += flow_deltas[0];
bnc->flow[1] += flow_deltas[1];
}
/* Find the leftmost child that may contain the key.
* If the key exists it will be in the child whose number
* is the return value of this function.
*/
int toku_ftnode_which_child(FTNODE node, const DBT *k,
DESCRIPTOR desc, ft_compare_func cmp) {
// a funny case of no pivots
if (node->n_children <= 1) return 0;
// check the last key to optimize seq insertions
int n = node->n_children-1;
int c = ft_compare_pivot(desc, cmp, k, &node->childkeys[n-1]);
if (c > 0) return n;
// binary search the pivots
int lo = 0;
int hi = n-1; // skip the last one, we checked it above
int mi;
while (lo < hi) {
mi = (lo + hi) / 2;
c = ft_compare_pivot(desc, cmp, k, &node->childkeys[mi]);
if (c > 0) {
lo = mi+1;
continue;
}
if (c < 0) {
hi = mi;
continue;
}
return mi;
}
return lo;
}
// Used for HOT.
int
toku_ftnode_hot_next_child(FTNODE node,
const DBT *k,
DESCRIPTOR desc,
ft_compare_func cmp) {
int low = 0;
int hi = node->n_children - 1;
int mi;
while (low < hi) {
mi = (low + hi) / 2;
int r = ft_compare_pivot(desc, cmp, k, &node->childkeys[mi]);
if (r > 0) {
low = mi + 1;
} else if (r < 0) {
hi = mi;
} else {
// if they were exactly equal, then we want the sub-tree under
// the next pivot.
return mi + 1;
}
}
invariant(low == hi);
return low;
}
// TODO Use this function to clean up other places where bits of messages are passed around
// such as toku_bnc_insert_msg() and the call stack above it.
static uint64_t
ft_msg_size(FT_MSG msg) {
size_t keyval_size = msg->u.id.key->size + msg->u.id.val->size;
size_t xids_size = xids_get_serialize_size(msg->xids);
return keyval_size + KEY_VALUE_OVERHEAD + FT_MSG_OVERHEAD + xids_size;
}
static void
ft_nonleaf_msg_all(ft_compare_func compare_fun, DESCRIPTOR desc, FTNODE node, FT_MSG msg, bool is_fresh, size_t flow_deltas[])
// Effect: Put the message into a nonleaf node. We put it into all children, possibly causing the children to become reactive.
// We don't do the splitting and merging. That's up to the caller after doing all the puts it wants to do.
// The re_array[i] gets set to the reactivity of any modified child i. (And there may be several such children.)
{
for (int i = 0; i < node->n_children; i++) {
ft_nonleaf_msg_once_to_child(compare_fun, desc, node, i, msg, is_fresh, flow_deltas);
}
}
static bool
ft_msg_applies_once(FT_MSG msg)
{
return ft_msg_type_applies_once(msg->type);
}
static bool
ft_msg_applies_all(FT_MSG msg)
{
return ft_msg_type_applies_all(msg->type);
}
static bool
ft_msg_does_nothing(FT_MSG msg)
{
return ft_msg_type_does_nothing(msg->type);
}
static void
ft_nonleaf_put_msg(ft_compare_func compare_fun, DESCRIPTOR desc, FTNODE node, int target_childnum, FT_MSG msg, bool is_fresh, size_t flow_deltas[])
// Effect: Put the message into a nonleaf node. We may put it into a child, possibly causing the child to become reactive.
// We don't do the splitting and merging. That's up to the caller after doing all the puts it wants to do.
// The re_array[i] gets set to the reactivity of any modified child i. (And there may be several such children.)
//
{
//
// see comments in toku_ft_leaf_apply_msg
// to understand why we handle setting
// node->max_msn_applied_to_node_on_disk here,
// and don't do it in toku_ft_node_put_msg
//
MSN msg_msn = msg->msn;
invariant(msg_msn.msn > node->max_msn_applied_to_node_on_disk.msn);
node->max_msn_applied_to_node_on_disk = msg_msn;
if (ft_msg_applies_once(msg)) {
ft_nonleaf_msg_once_to_child(compare_fun, desc, node, target_childnum, msg, is_fresh, flow_deltas);
} else if (ft_msg_applies_all(msg)) {
ft_nonleaf_msg_all(compare_fun, desc, node, msg, is_fresh, flow_deltas);
} else {
paranoid_invariant(ft_msg_does_nothing(msg));
}
}
// Garbage collect one leaf entry.
static void
ft_basement_node_gc_once(BASEMENTNODE bn,
uint32_t index,
void* keyp,
uint32_t keylen,
LEAFENTRY leaf_entry,
txn_gc_info *gc_info,
STAT64INFO_S * delta)
{
paranoid_invariant(leaf_entry);
// Don't run garbage collection on non-mvcc leaf entries.
if (leaf_entry->type != LE_MVCC) {
goto exit;
}
// Don't run garbage collection if this leafentry decides it's not worth it.
if (!toku_le_worth_running_garbage_collection(leaf_entry, gc_info)) {
goto exit;
}
LEAFENTRY new_leaf_entry;
new_leaf_entry = NULL;
// The mempool doesn't free itself. When it allocates new memory,
// this pointer will be set to the older memory that must now be
// freed.
void * maybe_free;
maybe_free = NULL;
// These will represent the number of bytes and rows changed as
// part of the garbage collection.
int64_t numbytes_delta;
int64_t numrows_delta;
toku_le_garbage_collect(leaf_entry,
&bn->data_buffer,
index,
keyp,
keylen,
gc_info,
&new_leaf_entry,
&numbytes_delta);
numrows_delta = 0;
if (new_leaf_entry) {
numrows_delta = 0;
} else {
numrows_delta = -1;
}
// If we created a new mempool buffer we must free the
// old/original buffer.
if (maybe_free) {
toku_free(maybe_free);
}
// Update stats.
bn->stat64_delta.numrows += numrows_delta;
bn->stat64_delta.numbytes += numbytes_delta;
delta->numrows += numrows_delta;
delta->numbytes += numbytes_delta;
exit:
return;
}
// Garbage collect all leaf entries for a given basement node.
static void
basement_node_gc_all_les(BASEMENTNODE bn,
txn_gc_info *gc_info,
STAT64INFO_S * delta)
{
int r = 0;
uint32_t index = 0;
uint32_t num_leafentries_before;
while (index < (num_leafentries_before = bn->data_buffer.num_klpairs())) {
void* keyp = NULL;
uint32_t keylen = 0;
LEAFENTRY leaf_entry;
r = bn->data_buffer.fetch_klpair(index, &leaf_entry, &keylen, &keyp);
assert_zero(r);
ft_basement_node_gc_once(
bn,
index,
keyp,
keylen,
leaf_entry,
gc_info,
delta
);
// Check if the leaf entry was deleted or not.
if (num_leafentries_before == bn->data_buffer.num_klpairs()) {
++index;
}
}
}
// Garbage collect all leaf entires in all basement nodes.
static void
ft_leaf_gc_all_les(FT ft, FTNODE node, txn_gc_info *gc_info)
{
toku_assert_entire_node_in_memory(node);
paranoid_invariant_zero(node->height);
// Loop through each leaf entry, garbage collecting as we go.
for (int i = 0; i < node->n_children; ++i) {
// Perform the garbage collection.
BASEMENTNODE bn = BLB(node, i);
STAT64INFO_S delta;
delta.numrows = 0;
delta.numbytes = 0;
basement_node_gc_all_les(bn, gc_info, &delta);
toku_ft_update_stats(&ft->in_memory_stats, delta);
}
}
static void
ft_leaf_run_gc(FT ft, FTNODE node) {
TOKULOGGER logger = toku_cachefile_logger(ft->cf);
if (logger) {
TXN_MANAGER txn_manager = toku_logger_get_txn_manager(logger);
txn_manager_state txn_state_for_gc(txn_manager);
txn_state_for_gc.init();
TXNID oldest_referenced_xid_for_simple_gc = toku_txn_manager_get_oldest_referenced_xid_estimate(txn_manager);
// Perform full garbage collection.
//
// - txn_state_for_gc
// a fresh snapshot of the transaction system.
// - oldest_referenced_xid_for_simple_gc
// the oldest xid in any live list as of right now - suitible for simple gc
// - node->oldest_referenced_xid_known
// the last known oldest referenced xid for this node and any unapplied messages.
// it is a lower bound on the actual oldest referenced xid - but becasue there
// may be abort messages above us, we need to be careful to only use this value
// for implicit promotion (as opposed to the oldest referenced xid for simple gc)
//
// The node has its own oldest referenced xid because it must be careful not to implicitly promote
// provisional entries for transactions that are no longer live, but may have abort messages
// somewhere above us in the tree.
txn_gc_info gc_info(&txn_state_for_gc,
oldest_referenced_xid_for_simple_gc,
node->oldest_referenced_xid_known,
true);
ft_leaf_gc_all_les(ft, node, &gc_info);
}
}
void toku_bnc_flush_to_child(
FT ft,
NONLEAF_CHILDINFO bnc,
FTNODE child,
TXNID parent_oldest_referenced_xid_known
)
{
paranoid_invariant(bnc);
TOKULOGGER logger = toku_cachefile_logger(ft->cf);
TXN_MANAGER txn_manager = logger != nullptr ? toku_logger_get_txn_manager(logger) : nullptr;
TXNID oldest_referenced_xid_for_simple_gc = TXNID_NONE;
txn_manager_state txn_state_for_gc(txn_manager);
bool do_garbage_collection = child->height == 0 && txn_manager != nullptr;
if (do_garbage_collection) {
txn_state_for_gc.init();
oldest_referenced_xid_for_simple_gc = toku_txn_manager_get_oldest_referenced_xid_estimate(txn_manager);
}
txn_gc_info gc_info(&txn_state_for_gc,
oldest_referenced_xid_for_simple_gc,
child->oldest_referenced_xid_known,
true);
struct flush_msg_fn {
FT ft;
FTNODE child;
NONLEAF_CHILDINFO bnc;
txn_gc_info *gc_info;
STAT64INFO_S stats_delta;
size_t remaining_memsize = bnc->msg_buffer.buffer_size_in_use();
flush_msg_fn(FT t, FTNODE n, NONLEAF_CHILDINFO nl, txn_gc_info *g) :
ft(t), child(n), bnc(nl), gc_info(g), remaining_memsize(bnc->msg_buffer.buffer_size_in_use()) {
stats_delta = { 0, 0 };
}
int operator()(FT_MSG msg, bool is_fresh) {
size_t flow_deltas[] = { 0, 0 };
size_t memsize_in_buffer = message_buffer::msg_memsize_in_buffer(msg);
if (remaining_memsize <= bnc->flow[0]) {
// this message is in the current checkpoint's worth of
// the end of the message buffer
flow_deltas[0] = memsize_in_buffer;
} else if (remaining_memsize <= bnc->flow[0] + bnc->flow[1]) {
// this message is in the last checkpoint's worth of the
// end of the message buffer
flow_deltas[1] = memsize_in_buffer;
}
toku_ft_node_put_msg(
ft->compare_fun,
ft->update_fun,
&ft->cmp_descriptor,
child,
-1,
msg,
is_fresh,
gc_info,
flow_deltas,
&stats_delta
);
remaining_memsize -= memsize_in_buffer;
return 0;
}
} flush_fn(ft, child, bnc, &gc_info);
bnc->msg_buffer.iterate(flush_fn);
child->oldest_referenced_xid_known = parent_oldest_referenced_xid_known;
invariant(flush_fn.remaining_memsize == 0);
if (flush_fn.stats_delta.numbytes || flush_fn.stats_delta.numrows) {
toku_ft_update_stats(&ft->in_memory_stats, flush_fn.stats_delta);
}
if (do_garbage_collection) {
size_t buffsize = bnc->msg_buffer.buffer_size_in_use();
STATUS_INC(FT_MSG_BYTES_OUT, buffsize);
// may be misleading if there's a broadcast message in there
STATUS_INC(FT_MSG_BYTES_CURR, -buffsize);
}
}
bool toku_bnc_should_promote(FT ft, NONLEAF_CHILDINFO bnc) {
static const double factor = 0.125;
const uint64_t flow_threshold = ft->h->nodesize * factor;
return bnc->flow[0] >= flow_threshold || bnc->flow[1] >= flow_threshold;
}
void
toku_ft_node_put_msg (
ft_compare_func compare_fun,
ft_update_func update_fun,
DESCRIPTOR desc,
FTNODE node,
int target_childnum,
FT_MSG msg,
bool is_fresh,
txn_gc_info *gc_info,
size_t flow_deltas[],
STAT64INFO stats_to_update
)
// Effect: Push message into the subtree rooted at NODE.
// If NODE is a leaf, then
// put message into leaf, applying it to the leafentries
// If NODE is a nonleaf, then push the message into the message buffer(s) of the relevent child(ren).
// The node may become overfull. That's not our problem.
{
toku_assert_entire_node_in_memory(node);
//
// see comments in toku_ft_leaf_apply_msg
// to understand why we don't handle setting
// node->max_msn_applied_to_node_on_disk here,
// and instead defer to these functions
//
if (node->height==0) {
toku_ft_leaf_apply_msg(compare_fun, update_fun, desc, node, target_childnum, msg, gc_info, nullptr, stats_to_update);
} else {
ft_nonleaf_put_msg(compare_fun, desc, node, target_childnum, msg, is_fresh, flow_deltas);
}
}
static const struct pivot_bounds infinite_bounds = {.lower_bound_exclusive=NULL,
.upper_bound_inclusive=NULL};
// Effect: applies the message to the leaf if the appropriate basement node is in memory.
// This function is called during message injection and/or flushing, so the entire
// node MUST be in memory.
void toku_ft_leaf_apply_msg(
ft_compare_func compare_fun,
ft_update_func update_fun,
DESCRIPTOR desc,
FTNODE node,
int target_childnum, // which child to inject to, or -1 if unknown
FT_MSG msg,
txn_gc_info *gc_info,
uint64_t *workdone,
STAT64INFO stats_to_update
)
{
VERIFY_NODE(t, node);
toku_assert_entire_node_in_memory(node);
//
// Because toku_ft_leaf_apply_msg is called with the intent of permanently
// applying a message to a leaf node (meaning the message is permanently applied
// and will be purged from the system after this call, as opposed to
// toku_apply_ancestors_messages_to_node, which applies a message
// for a query, but the message may still reside in the system and
// be reapplied later), we mark the node as dirty and
// take the opportunity to update node->max_msn_applied_to_node_on_disk.
//
node->dirty = 1;
//
// we cannot blindly update node->max_msn_applied_to_node_on_disk,
// we must check to see if the msn is greater that the one already stored,
// because the message may have already been applied earlier (via
// toku_apply_ancestors_messages_to_node) to answer a query
//
// This is why we handle node->max_msn_applied_to_node_on_disk both here
// and in ft_nonleaf_put_msg, as opposed to in one location, toku_ft_node_put_msg.
//
MSN msg_msn = msg->msn;
if (msg_msn.msn > node->max_msn_applied_to_node_on_disk.msn) {
node->max_msn_applied_to_node_on_disk = msg_msn;
}
if (ft_msg_applies_once(msg)) {
unsigned int childnum = (target_childnum >= 0
? target_childnum
: toku_ftnode_which_child(node, msg->u.id.key, desc, compare_fun));
BASEMENTNODE bn = BLB(node, childnum);
if (msg->msn.msn > bn->max_msn_applied.msn) {
bn->max_msn_applied = msg->msn;
toku_ft_bn_apply_msg(compare_fun,
update_fun,
desc,
bn,
msg,
gc_info,
workdone,
stats_to_update);
} else {
STATUS_INC(FT_MSN_DISCARDS, 1);
}
}
else if (ft_msg_applies_all(msg)) {
for (int childnum=0; childnum<node->n_children; childnum++) {
if (msg->msn.msn > BLB(node, childnum)->max_msn_applied.msn) {
BLB(node, childnum)->max_msn_applied = msg->msn;
toku_ft_bn_apply_msg(compare_fun,
update_fun,
desc,
BLB(node, childnum),
msg,
gc_info,
workdone,
stats_to_update);
} else {
STATUS_INC(FT_MSN_DISCARDS, 1);
}
}
}
else if (!ft_msg_does_nothing(msg)) {
abort();
}
VERIFY_NODE(t, node);
true
);
}
// TODO Use this function to clean up other places where bits of messages are passed around
// such as toku_bnc_insert_msg() and the call stack above it.
static uint64_t
ft_msg_size(FT_MSG msg) {
size_t keyval_size = msg->u.id.key->size + msg->u.id.val->size;
size_t xids_size = xids_get_serialize_size(msg->xids);
return keyval_size + KEY_VALUE_OVERHEAD + FT_MSG_OVERHEAD + xids_size;
}
static void inject_message_in_locked_node(
......@@ -2744,7 +1583,7 @@ static void inject_message_in_locked_node(
// check in frwlock. Should be possible with TOKU_PTHREAD_DEBUG, nop
// otherwise.
invariant(toku_ctpair_is_write_locked(node->ct_pair));
toku_assert_entire_node_in_memory(node);
toku_ftnode_assert_fully_in_memory(node);
// Take the newer of the two oldest referenced xid values from the node and gc_info.
// The gc_info usually has a newer value, because we got it at the top of this call
......@@ -2762,7 +1601,7 @@ static void inject_message_in_locked_node(
msg->msn.msn = toku_sync_add_and_fetch(&ft->h->max_msn_in_ft.msn, 1);
paranoid_invariant(msg->msn.msn > node->max_msn_applied_to_node_on_disk.msn);
STAT64INFO_S stats_delta = {0,0};
toku_ft_node_put_msg(
toku_ftnode_put_msg(
ft->compare_fun,
ft->update_fun,
&ft->cmp_descriptor,
......@@ -2778,20 +1617,19 @@ static void inject_message_in_locked_node(
toku_ft_update_stats(&ft->in_memory_stats, stats_delta);
}
//
// assumption is that toku_ft_node_put_msg will
// assumption is that toku_ftnode_put_msg will
// mark the node as dirty.
// enforcing invariant here.
//
paranoid_invariant(node->dirty != 0);
// TODO: Why not at height 0?
// update some status variables
if (node->height != 0) {
uint64_t msgsize = ft_msg_size(msg);
STATUS_INC(FT_MSG_BYTES_IN, msgsize);
STATUS_INC(FT_MSG_BYTES_CURR, msgsize);
STATUS_INC(FT_MSG_NUM, 1);
if (ft_msg_applies_all(msg)) {
if (ft_msg_type_applies_all(msg->type)) {
STATUS_INC(FT_MSG_NUM_BROADCAST, 1);
}
}
......@@ -2811,7 +1649,7 @@ static void inject_message_in_locked_node(
// 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)) {
if (node->height > 0 && toku_ftnode_nonleaf_is_gorged(node, ft->h->nodesize)) {
toku_ft_flush_node_on_background_thread(ft, node);
}
else {
......@@ -2836,7 +1674,7 @@ static bool process_maybe_reactive_child(FT ft, FTNODE parent, FTNODE child, int
// true if relocking is needed
// false otherwise
{
enum reactivity re = get_node_reactivity(ft, child);
enum reactivity re = toku_ftnode_get_reactivity(ft, child);
enum reactivity newre;
BLOCKNUM child_blocknum;
uint32_t child_fullhash;
......@@ -2870,7 +1708,7 @@ static bool process_maybe_reactive_child(FT ft, FTNODE parent, FTNODE child, int
child_blocknum = BP_BLOCKNUM(newparent, childnum);
child_fullhash = compute_child_fullhash(ft->cf, newparent, childnum);
toku_pin_ftnode_with_dep_nodes(ft, child_blocknum, child_fullhash, &bfe, PL_WRITE_CHEAP, 1, &newparent, &newchild, true);
newre = get_node_reactivity(ft, newchild);
newre = toku_ftnode_get_reactivity(ft, newchild);
if (newre == RE_FISSIBLE) {
enum split_mode split_mode;
if (newparent->height == 1 && (loc & LEFT_EXTREME) && childnum == 0) {
......@@ -2916,7 +1754,7 @@ static bool process_maybe_reactive_child(FT ft, FTNODE parent, FTNODE child, int
child_blocknum = BP_BLOCKNUM(newparent, childnum);
child_fullhash = compute_child_fullhash(ft->cf, newparent, childnum);
toku_pin_ftnode_with_dep_nodes(ft, child_blocknum, child_fullhash, &bfe, PL_READ, 1, &newparent, &newchild, true);
newre = get_node_reactivity(ft, newchild);
newre = toku_ftnode_get_reactivity(ft, newchild);
if (newre == RE_FUSIBLE && newparent->n_children >= 2) {
toku_unpin_ftnode_read_only(ft, newchild);
toku_ft_merge_child(ft, newparent, childnum);
......@@ -2949,7 +1787,7 @@ static void inject_message_at_this_blocknum(FT ft, CACHEKEY cachekey, uint32_t f
struct ftnode_fetch_extra bfe;
fill_bfe_for_full_read(&bfe, ft);
toku_pin_ftnode(ft, cachekey, fullhash, &bfe, PL_WRITE_CHEAP, &node, true);
toku_assert_entire_node_in_memory(node);
toku_ftnode_assert_fully_in_memory(node);
paranoid_invariant(node->fullhash==fullhash);
ft_verify_flags(ft, node);
inject_message_in_locked_node(ft, node, -1, msg, flow_deltas, gc_info);
......@@ -2980,6 +1818,12 @@ static void ft_set_or_verify_rightmost_blocknum(FT ft, BLOCKNUM b)
invariant(ft->rightmost_blocknum.b == b.b);
}
bool toku_bnc_should_promote(FT ft, NONLEAF_CHILDINFO bnc) {
static const double factor = 0.125;
const uint64_t flow_threshold = ft->h->nodesize * factor;
return bnc->flow[0] >= flow_threshold || bnc->flow[1] >= flow_threshold;
}
static void push_something_in_subtree(
FT ft,
FTNODE subtree_root,
......@@ -3013,7 +1857,7 @@ static void push_something_in_subtree(
// When the birdie is still saying we should promote, we use get_and_pin so that we wait to get the node.
// If the birdie doesn't say to promote, we try maybe_get_and_pin. If we get the node cheaply, and it's dirty, we promote anyway.
{
toku_assert_entire_node_in_memory(subtree_root);
toku_ftnode_assert_fully_in_memory(subtree_root);
if (should_inject_in_node(loc, subtree_root->height, depth)) {
switch (depth) {
case 0:
......@@ -3042,7 +1886,7 @@ static void push_something_in_subtree(
NONLEAF_CHILDINFO bnc;
// toku_ft_root_put_msg should not have called us otherwise.
paranoid_invariant(ft_msg_applies_once(msg));
paranoid_invariant(ft_msg_type_applies_once(msg->type));
childnum = (target_childnum >= 0 ? target_childnum
: toku_ftnode_which_child(subtree_root, msg->u.id.key, &ft->cmp_descriptor, ft->compare_fun));
......@@ -3107,7 +1951,7 @@ static void push_something_in_subtree(
STATUS_INC(FT_PRO_NUM_STOP_LOCK_CHILD, 1);
goto relock_and_push_here;
}
if (is_entire_node_in_memory(child)) {
if (toku_ftnode_fully_in_memory(child)) {
// toku_pin_ftnode... touches the clock but toku_maybe_pin_ftnode... doesn't.
// This prevents partial eviction.
for (int i = 0; i < child->n_children; ++i) {
......@@ -3229,7 +2073,7 @@ void toku_ft_root_put_msg(
change_lock_type:
// get the root node
toku_pin_ftnode(ft, root_key, fullhash, &bfe, lock_type, &node, true);
toku_assert_entire_node_in_memory(node);
toku_ftnode_assert_fully_in_memory(node);
paranoid_invariant(node->fullhash==fullhash);
ft_verify_flags(ft, node);
......@@ -3238,7 +2082,7 @@ void toku_ft_root_put_msg(
// injection thread to change lock type back and forth, when only one
// of them needs to in order to handle the split. That's not great,
// but root splits are incredibly rare.
enum reactivity re = get_node_reactivity(ft, node);
enum reactivity re = toku_ftnode_get_reactivity(ft, node);
switch (re) {
case RE_STABLE:
case RE_FUSIBLE: // cannot merge anything at the root
......@@ -3281,7 +2125,7 @@ void toku_ft_root_put_msg(
// anyway.
// Now, either inject here or promote. We decide based on a heuristic:
if (node->height == 0 || !ft_msg_applies_once(msg)) {
if (node->height == 0 || !ft_msg_type_applies_once(msg->type)) {
// If the root's a leaf or we're injecting a broadcast, drop the read lock and inject here.
toku_unpin_ftnode_read_only(ft, node);
STATUS_INC(FT_PRO_NUM_ROOT_H0_INJECT, 1);
......@@ -3404,7 +2248,7 @@ static int ft_leaf_get_relative_key_pos(FT ft, FTNODE leaf, const DBT *key, bool
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 };
struct toku_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,
......@@ -3479,7 +2323,7 @@ static int ft_maybe_insert_into_rightmost_leaf(FT ft, DBT *key, DBT *val, XIDS m
// 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) {
if (toku_ftnode_get_leaf_reactivity(rightmost_leaf, ft->h->nodesize) != RE_STABLE) {
STATUS_INC(FT_PRO_RIGHTMOST_LEAF_SHORTCUT_FAIL_REACTIVE, 1);
goto cleanup;
}
......@@ -4464,580 +3308,7 @@ void toku_ft_handle_create(FT_HANDLE *ft_handle_ptr) {
*ft_handle_ptr = ft_handle;
}
struct store_msg_buffer_offset_extra {
int32_t *offsets;
int i;
};
int store_msg_buffer_offset(const int32_t &offset, const uint32_t UU(idx), struct store_msg_buffer_offset_extra *const extra) __attribute__((nonnull(3)));
int store_msg_buffer_offset(const int32_t &offset, const uint32_t UU(idx), struct store_msg_buffer_offset_extra *const extra)
{
extra->offsets[extra->i] = offset;
extra->i++;
return 0;
}
/**
* Given pointers to offsets within a message buffer where we can find messages,
* figure out the MSN of each message, and compare those MSNs. Returns 1,
* 0, or -1 if a is larger than, equal to, or smaller than b.
*/
int msg_buffer_offset_msn_cmp(message_buffer &msg_buffer, const int32_t &ao, const int32_t &bo);
int msg_buffer_offset_msn_cmp(message_buffer &msg_buffer, const int32_t &ao, const int32_t &bo)
{
MSN amsn, bmsn;
msg_buffer.get_message_key_msn(ao, nullptr, &amsn);
msg_buffer.get_message_key_msn(bo, nullptr, &bmsn);
if (amsn.msn > bmsn.msn) {
return +1;
}
if (amsn.msn < bmsn.msn) {
return -1;
}
return 0;
}
/**
* Given a message buffer and and offset, apply the message with toku_ft_bn_apply_msg, or discard it,
* based on its MSN and the MSN of the basement node.
*/
static void
do_bn_apply_msg(FT_HANDLE ft_handle, BASEMENTNODE bn, message_buffer *msg_buffer, int32_t offset,
txn_gc_info *gc_info, uint64_t *workdone, STAT64INFO stats_to_update) {
DBT k, v;
FT_MSG_S msg = msg_buffer->get_message(offset, &k, &v);
// The messages are being iterated over in (key,msn) order or just in
// msn order, so all the messages for one key, from one buffer, are in
// ascending msn order. So it's ok that we don't update the basement
// node's msn until the end.
if (msg.msn.msn > bn->max_msn_applied.msn) {
toku_ft_bn_apply_msg(
ft_handle->ft->compare_fun,
ft_handle->ft->update_fun,
&ft_handle->ft->cmp_descriptor,
bn,
&msg,
gc_info,
workdone,
stats_to_update
);
} else {
STATUS_INC(FT_MSN_DISCARDS, 1);
}
// We must always mark message as stale since it has been marked
// (using omt::iterate_and_mark_range)
// It is possible to call do_bn_apply_msg even when it won't apply the message because
// the node containing it could have been evicted and brought back in.
msg_buffer->set_freshness(offset, false);
}
struct iterate_do_bn_apply_msg_extra {
FT_HANDLE t;
BASEMENTNODE bn;
NONLEAF_CHILDINFO bnc;
txn_gc_info *gc_info;
uint64_t *workdone;
STAT64INFO stats_to_update;
};
int iterate_do_bn_apply_msg(const int32_t &offset, const uint32_t UU(idx), struct iterate_do_bn_apply_msg_extra *const e) __attribute__((nonnull(3)));
int iterate_do_bn_apply_msg(const int32_t &offset, const uint32_t UU(idx), struct iterate_do_bn_apply_msg_extra *const e)
{
do_bn_apply_msg(e->t, e->bn, &e->bnc->msg_buffer, offset, e->gc_info, e->workdone, e->stats_to_update);
return 0;
}
/**
* Given the bounds of the basement node to which we will apply messages,
* find the indexes within message_tree which contain the range of
* relevant messages.
*
* The message tree contains offsets into the buffer, where messages are
* found. The pivot_bounds are the lower bound exclusive and upper bound
* inclusive, because they come from pivot keys in the tree. We want OMT
* indices, which must have the lower bound be inclusive and the upper
* bound exclusive. We will get these by telling omt::find to look
* for something strictly bigger than each of our pivot bounds.
*
* Outputs the OMT indices in lbi (lower bound inclusive) and ube (upper
* bound exclusive).
*/
template<typename find_bounds_omt_t>
static void
find_bounds_within_message_tree(
DESCRIPTOR desc, /// used for cmp
ft_compare_func cmp, /// used to compare keys
const find_bounds_omt_t &message_tree, /// tree holding message buffer offsets, in which we want to look for indices
message_buffer *msg_buffer, /// message buffer in which messages are found
struct pivot_bounds const * const bounds, /// key bounds within the basement node we're applying messages to
uint32_t *lbi, /// (output) "lower bound inclusive" (index into message_tree)
uint32_t *ube /// (output) "upper bound exclusive" (index into message_tree)
)
{
int r = 0;
if (bounds->lower_bound_exclusive) {
// By setting msn to MAX_MSN and by using direction of +1, we will
// get the first message greater than (in (key, msn) order) any
// message (with any msn) with the key lower_bound_exclusive.
// This will be a message we want to try applying, so it is the
// "lower bound inclusive" within the message_tree.
struct toku_msg_buffer_key_msn_heaviside_extra lbi_extra;
ZERO_STRUCT(lbi_extra);
lbi_extra.desc = desc;
lbi_extra.cmp = cmp;
lbi_extra.msg_buffer = msg_buffer;
lbi_extra.key = bounds->lower_bound_exclusive;
lbi_extra.msn = MAX_MSN;
int32_t found_lb;
r = message_tree.template find<struct toku_msg_buffer_key_msn_heaviside_extra, toku_msg_buffer_key_msn_heaviside>(lbi_extra, +1, &found_lb, lbi);
if (r == DB_NOTFOUND) {
// There is no relevant data (the lower bound is bigger than
// any message in this tree), so we have no range and we're
// done.
*lbi = 0;
*ube = 0;
return;
}
if (bounds->upper_bound_inclusive) {
// Check if what we found for lbi is greater than the upper
// bound inclusive that we have. If so, there are no relevant
// messages between these bounds.
const DBT *ubi = bounds->upper_bound_inclusive;
const int32_t offset = found_lb;
DBT found_lbidbt;
msg_buffer->get_message_key_msn(offset, &found_lbidbt, nullptr);
FAKE_DB(db, desc);
int c = cmp(&db, &found_lbidbt, ubi);
// These DBTs really are both inclusive bounds, so we need
// strict inequality in order to determine that there's
// nothing between them. If they're equal, then we actually
// need to apply the message pointed to by lbi, and also
// anything with the same key but a bigger msn.
if (c > 0) {
*lbi = 0;
*ube = 0;
return;
}
}
} else {
// No lower bound given, it's negative infinity, so we start at
// the first message in the OMT.
*lbi = 0;
}
if (bounds->upper_bound_inclusive) {
// Again, we use an msn of MAX_MSN and a direction of +1 to get
// the first thing bigger than the upper_bound_inclusive key.
// This is therefore the smallest thing we don't want to apply,
// and omt::iterate_on_range will not examine it.
struct toku_msg_buffer_key_msn_heaviside_extra ube_extra;
ZERO_STRUCT(ube_extra);
ube_extra.desc = desc;
ube_extra.cmp = cmp;
ube_extra.msg_buffer = msg_buffer;
ube_extra.key = bounds->upper_bound_inclusive;
ube_extra.msn = MAX_MSN;
r = message_tree.template find<struct toku_msg_buffer_key_msn_heaviside_extra, toku_msg_buffer_key_msn_heaviside>(ube_extra, +1, nullptr, ube);
if (r == DB_NOTFOUND) {
// Couldn't find anything in the buffer bigger than our key,
// so we need to look at everything up to the end of
// message_tree.
*ube = message_tree.size();
}
} else {
// No upper bound given, it's positive infinity, so we need to go
// through the end of the OMT.
*ube = message_tree.size();
}
}
/**
* For each message in the ancestor's buffer (determined by childnum) that
* is key-wise between lower_bound_exclusive and upper_bound_inclusive,
* apply the message to the basement node. We treat the bounds as minus
* or plus infinity respectively if they are NULL. Do not mark the node
* as dirty (preserve previous state of 'dirty' bit).
*/
static void
bnc_apply_messages_to_basement_node(
FT_HANDLE t, // used for comparison function
BASEMENTNODE bn, // where to apply messages
FTNODE ancestor, // the ancestor node where we can find messages to apply
int childnum, // which child buffer of ancestor contains messages we want
struct pivot_bounds const * const bounds, // contains pivot key bounds of this basement node
txn_gc_info *gc_info,
bool* msgs_applied
)
{
int r;
NONLEAF_CHILDINFO bnc = BNC(ancestor, childnum);
// Determine the offsets in the message trees between which we need to
// apply messages from this buffer
STAT64INFO_S stats_delta = {0,0};
uint64_t workdone_this_ancestor = 0;
uint32_t stale_lbi, stale_ube;
if (!bn->stale_ancestor_messages_applied) {
find_bounds_within_message_tree(&t->ft->cmp_descriptor, t->ft->compare_fun, bnc->stale_message_tree, &bnc->msg_buffer, bounds, &stale_lbi, &stale_ube);
} else {
stale_lbi = 0;
stale_ube = 0;
}
uint32_t fresh_lbi, fresh_ube;
find_bounds_within_message_tree(&t->ft->cmp_descriptor, t->ft->compare_fun, bnc->fresh_message_tree, &bnc->msg_buffer, bounds, &fresh_lbi, &fresh_ube);
// We now know where all the messages we must apply are, so one of the
// following 4 cases will do the application, depending on which of
// the lists contains relevant messages:
//
// 1. broadcast messages and anything else, or a mix of fresh and stale
// 2. only fresh messages
// 3. only stale messages
if (bnc->broadcast_list.size() > 0 ||
(stale_lbi != stale_ube && fresh_lbi != fresh_ube)) {
// We have messages in multiple trees, so we grab all
// the relevant messages' offsets and sort them by MSN, then apply
// them in MSN order.
const int buffer_size = ((stale_ube - stale_lbi) + (fresh_ube - fresh_lbi) + bnc->broadcast_list.size());
toku::scoped_malloc offsets_buf(buffer_size * sizeof(int32_t));
int32_t *offsets = reinterpret_cast<int32_t *>(offsets_buf.get());
struct store_msg_buffer_offset_extra sfo_extra = { .offsets = offsets, .i = 0 };
// Populate offsets array with offsets to stale messages
r = bnc->stale_message_tree.iterate_on_range<struct store_msg_buffer_offset_extra, store_msg_buffer_offset>(stale_lbi, stale_ube, &sfo_extra);
assert_zero(r);
// Then store fresh offsets, and mark them to be moved to stale later.
r = bnc->fresh_message_tree.iterate_and_mark_range<struct store_msg_buffer_offset_extra, store_msg_buffer_offset>(fresh_lbi, fresh_ube, &sfo_extra);
assert_zero(r);
// Store offsets of all broadcast messages.
r = bnc->broadcast_list.iterate<struct store_msg_buffer_offset_extra, store_msg_buffer_offset>(&sfo_extra);
assert_zero(r);
invariant(sfo_extra.i == buffer_size);
// Sort by MSN.
r = toku::sort<int32_t, message_buffer, msg_buffer_offset_msn_cmp>::mergesort_r(offsets, buffer_size, bnc->msg_buffer);
assert_zero(r);
// Apply the messages in MSN order.
for (int i = 0; i < buffer_size; ++i) {
*msgs_applied = true;
do_bn_apply_msg(t, bn, &bnc->msg_buffer, offsets[i], gc_info, &workdone_this_ancestor, &stats_delta);
}
} else if (stale_lbi == stale_ube) {
// No stale messages to apply, we just apply fresh messages, and mark them to be moved to stale later.
struct iterate_do_bn_apply_msg_extra iter_extra = { .t = t, .bn = bn, .bnc = bnc, .gc_info = gc_info, .workdone = &workdone_this_ancestor, .stats_to_update = &stats_delta };
if (fresh_ube - fresh_lbi > 0) *msgs_applied = true;
r = bnc->fresh_message_tree.iterate_and_mark_range<struct iterate_do_bn_apply_msg_extra, iterate_do_bn_apply_msg>(fresh_lbi, fresh_ube, &iter_extra);
assert_zero(r);
} else {
invariant(fresh_lbi == fresh_ube);
// No fresh messages to apply, we just apply stale messages.
if (stale_ube - stale_lbi > 0) *msgs_applied = true;
struct iterate_do_bn_apply_msg_extra iter_extra = { .t = t, .bn = bn, .bnc = bnc, .gc_info = gc_info, .workdone = &workdone_this_ancestor, .stats_to_update = &stats_delta };
r = bnc->stale_message_tree.iterate_on_range<struct iterate_do_bn_apply_msg_extra, iterate_do_bn_apply_msg>(stale_lbi, stale_ube, &iter_extra);
assert_zero(r);
}
//
// update stats
//
if (workdone_this_ancestor > 0) {
(void) toku_sync_fetch_and_add(&BP_WORKDONE(ancestor, childnum), workdone_this_ancestor);
}
if (stats_delta.numbytes || stats_delta.numrows) {
toku_ft_update_stats(&t->ft->in_memory_stats, stats_delta);
}
}
static void
apply_ancestors_messages_to_bn(
FT_HANDLE t,
FTNODE node,
int childnum,
ANCESTORS ancestors,
struct pivot_bounds const * const bounds,
txn_gc_info *gc_info,
bool* msgs_applied
)
{
BASEMENTNODE curr_bn = BLB(node, childnum);
struct pivot_bounds curr_bounds = next_pivot_keys(node, childnum, bounds);
for (ANCESTORS curr_ancestors = ancestors; curr_ancestors; curr_ancestors = curr_ancestors->next) {
if (curr_ancestors->node->max_msn_applied_to_node_on_disk.msn > curr_bn->max_msn_applied.msn) {
paranoid_invariant(BP_STATE(curr_ancestors->node, curr_ancestors->childnum) == PT_AVAIL);
bnc_apply_messages_to_basement_node(
t,
curr_bn,
curr_ancestors->node,
curr_ancestors->childnum,
&curr_bounds,
gc_info,
msgs_applied
);
// We don't want to check this ancestor node again if the
// next time we query it, the msn hasn't changed.
curr_bn->max_msn_applied = curr_ancestors->node->max_msn_applied_to_node_on_disk;
}
}
// At this point, we know all the stale messages above this
// basement node have been applied, and any new messages will be
// fresh, so we don't need to look at stale messages for this
// basement node, unless it gets evicted (and this field becomes
// false when it's read in again).
curr_bn->stale_ancestor_messages_applied = true;
}
void
toku_apply_ancestors_messages_to_node (
FT_HANDLE t,
FTNODE node,
ANCESTORS ancestors,
struct pivot_bounds const * const bounds,
bool* msgs_applied,
int child_to_read
)
// Effect:
// Bring a leaf node up-to-date according to all the messages in the ancestors.
// If the leaf node is already up-to-date then do nothing.
// If the leaf node is not already up-to-date, then record the work done
// for that leaf in each ancestor.
// Requires:
// This is being called when pinning a leaf node for the query path.
// The entire root-to-leaf path is pinned and appears in the ancestors list.
{
VERIFY_NODE(t, node);
paranoid_invariant(node->height == 0);
TXN_MANAGER txn_manager = toku_ft_get_txn_manager(t);
txn_manager_state txn_state_for_gc(txn_manager);
TXNID oldest_referenced_xid_for_simple_gc = toku_ft_get_oldest_referenced_xid_estimate(t);
txn_gc_info gc_info(&txn_state_for_gc,
oldest_referenced_xid_for_simple_gc,
node->oldest_referenced_xid_known,
true);
if (!node->dirty && child_to_read >= 0) {
paranoid_invariant(BP_STATE(node, child_to_read) == PT_AVAIL);
apply_ancestors_messages_to_bn(
t,
node,
child_to_read,
ancestors,
bounds,
&gc_info,
msgs_applied
);
}
else {
// know we are a leaf node
// An important invariant:
// We MUST bring every available basement node for a dirty node up to date.
// flushing on the cleaner thread depends on this. This invariant
// allows the cleaner thread to just pick an internal node and flush it
// as opposed to being forced to start from the root.
for (int i = 0; i < node->n_children; i++) {
if (BP_STATE(node, i) != PT_AVAIL) { continue; }
apply_ancestors_messages_to_bn(
t,
node,
i,
ancestors,
bounds,
&gc_info,
msgs_applied
);
}
}
VERIFY_NODE(t, node);
}
static bool bn_needs_ancestors_messages(
FT ft,
FTNODE node,
int childnum,
struct pivot_bounds const * const bounds,
ANCESTORS ancestors,
MSN* max_msn_applied
)
{
BASEMENTNODE bn = BLB(node, childnum);
struct pivot_bounds curr_bounds = next_pivot_keys(node, childnum, bounds);
bool needs_ancestors_messages = false;
for (ANCESTORS curr_ancestors = ancestors; curr_ancestors; curr_ancestors = curr_ancestors->next) {
if (curr_ancestors->node->max_msn_applied_to_node_on_disk.msn > bn->max_msn_applied.msn) {
paranoid_invariant(BP_STATE(curr_ancestors->node, curr_ancestors->childnum) == PT_AVAIL);
NONLEAF_CHILDINFO bnc = BNC(curr_ancestors->node, curr_ancestors->childnum);
if (bnc->broadcast_list.size() > 0) {
needs_ancestors_messages = true;
goto cleanup;
}
if (!bn->stale_ancestor_messages_applied) {
uint32_t stale_lbi, stale_ube;
find_bounds_within_message_tree(&ft->cmp_descriptor,
ft->compare_fun,
bnc->stale_message_tree,
&bnc->msg_buffer,
&curr_bounds,
&stale_lbi,
&stale_ube);
if (stale_lbi < stale_ube) {
needs_ancestors_messages = true;
goto cleanup;
}
}
uint32_t fresh_lbi, fresh_ube;
find_bounds_within_message_tree(&ft->cmp_descriptor,
ft->compare_fun,
bnc->fresh_message_tree,
&bnc->msg_buffer,
&curr_bounds,
&fresh_lbi,
&fresh_ube);
if (fresh_lbi < fresh_ube) {
needs_ancestors_messages = true;
goto cleanup;
}
if (curr_ancestors->node->max_msn_applied_to_node_on_disk.msn > max_msn_applied->msn) {
max_msn_applied->msn = curr_ancestors->node->max_msn_applied_to_node_on_disk.msn;
}
}
}
cleanup:
return needs_ancestors_messages;
}
bool toku_ft_leaf_needs_ancestors_messages(
FT ft,
FTNODE node,
ANCESTORS ancestors,
struct pivot_bounds const * const bounds,
MSN *const max_msn_in_path,
int child_to_read
)
// Effect: Determine whether there are messages in a node's ancestors
// which must be applied to it. These messages are in the correct
// keyrange for any available basement nodes, and are in nodes with the
// correct max_msn_applied_to_node_on_disk.
// Notes:
// This is an approximate query.
// Output:
// max_msn_in_path: max of "max_msn_applied_to_node_on_disk" over
// ancestors. This is used later to update basement nodes'
// max_msn_applied values in case we don't do the full algorithm.
// Returns:
// true if there may be some such messages
// false only if there are definitely no such messages
// Rationale:
// When we pin a node with a read lock, we want to quickly determine if
// we should exchange it for a write lock in preparation for applying
// messages. If there are no messages, we don't need the write lock.
{
paranoid_invariant(node->height == 0);
bool needs_ancestors_messages = false;
// child_to_read may be -1 in test cases
if (!node->dirty && child_to_read >= 0) {
paranoid_invariant(BP_STATE(node, child_to_read) == PT_AVAIL);
needs_ancestors_messages = bn_needs_ancestors_messages(
ft,
node,
child_to_read,
bounds,
ancestors,
max_msn_in_path
);
}
else {
for (int i = 0; i < node->n_children; ++i) {
if (BP_STATE(node, i) != PT_AVAIL) { continue; }
needs_ancestors_messages = bn_needs_ancestors_messages(
ft,
node,
i,
bounds,
ancestors,
max_msn_in_path
);
if (needs_ancestors_messages) {
goto cleanup;
}
}
}
cleanup:
return needs_ancestors_messages;
}
void toku_ft_bn_update_max_msn(FTNODE node, MSN max_msn_applied, int child_to_read) {
invariant(node->height == 0);
if (!node->dirty && child_to_read >= 0) {
paranoid_invariant(BP_STATE(node, child_to_read) == PT_AVAIL);
BASEMENTNODE bn = BLB(node, child_to_read);
if (max_msn_applied.msn > bn->max_msn_applied.msn) {
// see comment below
(void) toku_sync_val_compare_and_swap(&bn->max_msn_applied.msn, bn->max_msn_applied.msn, max_msn_applied.msn);
}
}
else {
for (int i = 0; i < node->n_children; ++i) {
if (BP_STATE(node, i) != PT_AVAIL) { continue; }
BASEMENTNODE bn = BLB(node, i);
if (max_msn_applied.msn > bn->max_msn_applied.msn) {
// This function runs in a shared access context, so to silence tools
// like DRD, we use a CAS and ignore the result.
// Any threads trying to update these basement nodes should be
// updating them to the same thing (since they all have a read lock on
// the same root-to-leaf path) so this is safe.
(void) toku_sync_val_compare_and_swap(&bn->max_msn_applied.msn, bn->max_msn_applied.msn, max_msn_applied.msn);
}
}
}
}
struct copy_to_stale_extra {
FT ft;
NONLEAF_CHILDINFO bnc;
};
int copy_to_stale(const int32_t &offset, const uint32_t UU(idx), struct copy_to_stale_extra *const extra) __attribute__((nonnull(3)));
int copy_to_stale(const int32_t &offset, const uint32_t UU(idx), struct copy_to_stale_extra *const extra)
{
MSN msn;
DBT key;
extra->bnc->msg_buffer.get_message_key_msn(offset, &key, &msn);
struct toku_msg_buffer_key_msn_heaviside_extra heaviside_extra = { .desc = &extra->ft->cmp_descriptor, .cmp = extra->ft->compare_fun, .msg_buffer = &extra->bnc->msg_buffer, .key = &key, .msn = msn };
int r = extra->bnc->stale_message_tree.insert<struct toku_msg_buffer_key_msn_heaviside_extra, toku_msg_buffer_key_msn_heaviside>(offset, heaviside_extra, nullptr);
invariant_zero(r);
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) {
invariant(node->height > 0);
for (int i = 0; i < node->n_children; ++i) {
if (BP_STATE(node, i) != PT_AVAIL) {
continue;
}
NONLEAF_CHILDINFO bnc = BNC(node, i);
// 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.
ft_bnc_move_messages_to_stale(ft, bnc);
}
}
/******************************* search ***************************************/
// Return true if this key is within the search bound. If there is no search bound then the tree search continues.
static bool search_continue(ft_search *search, void *key, uint32_t key_len) {
......@@ -5208,32 +3479,16 @@ ftnode_pf_callback_and_free_bfe(void *ftnode_pv, void* disk_data, void *read_ext
return r;
}
void fill_bfe_for_prefetch(struct ftnode_fetch_extra *bfe,
FT h,
struct ft_cursor *c) {
paranoid_invariant(h->h->type == FT_CURRENT);
bfe->type = ftnode_fetch_prefetch;
bfe->h = h;
bfe->search = NULL;
toku_init_dbt(&bfe->range_lock_left_key);
toku_init_dbt(&bfe->range_lock_right_key);
const DBT *left = &c->range_lock_left_key;
if (left->data) {
toku_clone_dbt(&bfe->range_lock_left_key, *left);
}
const DBT *right = &c->range_lock_right_key;
if (right->data) {
toku_clone_dbt(&bfe->range_lock_right_key, *right);
}
bfe->left_is_neg_infty = c->left_is_neg_infty;
bfe->right_is_pos_infty = c->right_is_pos_infty;
bfe->child_to_read = -1;
bfe->disable_prefetching = c->disable_prefetching;
bfe->read_all_partitions = false;
bfe->bytes_read = 0;
bfe->io_time = 0;
bfe->deserialize_time = 0;
bfe->decompress_time = 0;
CACHETABLE_WRITE_CALLBACK get_write_callbacks_for_node(FT ft) {
CACHETABLE_WRITE_CALLBACK wc;
wc.flush_callback = toku_ftnode_flush_callback;
wc.pe_est_callback = toku_ftnode_pe_est_callback;
wc.pe_callback = toku_ftnode_pe_callback;
wc.cleaner_callback = toku_ftnode_cleaner_callback;
wc.clone_callback = toku_ftnode_clone_callback;
wc.checkpoint_complete_callback = toku_ftnode_checkpoint_complete_callback;
wc.write_extraargs = ft;
return wc;
}
static void
......@@ -5276,6 +3531,7 @@ struct unlock_ftnode_extra {
FTNODE node;
bool msgs_applied;
};
// When this is called, the cachetable lock is held
static void
unlock_ftnode_fun (void *v) {
......@@ -5576,6 +3832,11 @@ ft_search_node(
return r;
}
static const struct pivot_bounds infinite_bounds = {
.lower_bound_exclusive = nullptr,
.upper_bound_inclusive = nullptr,
};
int toku_ft_search(FT_HANDLE ft_handle, ft_search *search, FT_GET_CALLBACK_FUNCTION getf, void *getf_v, FT_CURSOR ftcursor, bool can_bulk_fetch)
// Effect: Perform a search. Associate cursor with a leaf if possible.
// All searches are performed through this function.
......
......@@ -93,6 +93,7 @@ PATENT RIGHTS GRANT:
#include "ft-flusher.h"
#include "ft-internal.h"
#include "ft.h"
#include "node.h"
#include "fttypes.h"
#include "ule.h"
......@@ -222,7 +223,7 @@ int toku_testsetup_insert_to_leaf (FT_HANDLE ft_handle, BLOCKNUM blocknum, const
static size_t zero_flow_deltas[] = { 0, 0 };
txn_gc_info gc_info(nullptr, TXNID_NONE, TXNID_NONE, true);
toku_ft_node_put_msg (
toku_ftnode_put_msg(
ft_handle->ft->compare_fun,
ft_handle->ft->update_fun,
&ft_handle->ft->cmp_descriptor,
......
......@@ -100,6 +100,7 @@ PATENT RIGHTS GRANT:
#include "ft-cachetable-wrappers.h"
#include "ft-internal.h"
#include "ft.h"
#include "node.h"
static int
compare_pairs (FT_HANDLE ft_handle, const DBT *a, const DBT *b) {
......@@ -399,7 +400,7 @@ toku_verify_ftnode_internal(FT_HANDLE ft_handle,
BLOCKNUM blocknum = node->thisnodename;
//printf("%s:%d pin %p\n", __FILE__, __LINE__, node_v);
toku_assert_entire_node_in_memory(node);
toku_ftnode_assert_fully_in_memory(node);
this_msn = node->max_msn_applied_to_node_on_disk;
if (height >= 0) {
......@@ -501,7 +502,7 @@ toku_verify_ftnode (FT_HANDLE ft_handle,
MSN this_msn;
//printf("%s:%d pin %p\n", __FILE__, __LINE__, node_v);
toku_assert_entire_node_in_memory(node);
toku_ftnode_assert_fully_in_memory(node);
this_msn = node->max_msn_applied_to_node_on_disk;
int result = 0;
......
......@@ -89,12 +89,12 @@ PATENT RIGHTS GRANT:
#ident "Copyright (c) 2007-2013 Tokutek Inc. All rights reserved."
#ident "The technology is licensed by the Massachusetts Institute of Technology, Rutgers State University of New Jersey, and the Research Foundation of State University of New York at Stony Brook under United States of America Serial No. 11/760379 and to the patents and/or patent applications resulting from it."
#include "ft.h"
#include "ft-internal.h"
#include "ft-cachetable-wrappers.h"
#include "log-internal.h"
#include <ft/log_header.h>
#include "ft/ft.h"
#include "ft/ft-cachetable-wrappers.h"
#include "ft/ft-internal.h"
#include "ft/log-internal.h"
#include "ft/log_header.h"
#include "ft/node.h"
#include <memory.h>
#include <toku_assert.h>
......
......@@ -89,17 +89,18 @@ PATENT RIGHTS GRANT:
#ident "Copyright (c) 2007-2013 Tokutek Inc. All rights reserved."
#ident "The technology is licensed by the Massachusetts Institute of Technology, Rutgers State University of New Jersey, and the Research Foundation of State University of New York at Stony Brook under United States of America Serial No. 11/760379 and to the patents and/or patent applications resulting from it."
#include "ft-internal.h"
#include "log-internal.h"
#include <compress.h>
#include <portability/toku_atomic.h>
#include <util/sort.h>
#include <util/threadpool.h>
#include "ft.h"
#include <util/status.h>
#include <util/scoped_malloc.h>
#include "ft/cachetable.h"
#include "ft/compress.h"
#include "ft/ft.h"
#include "ft/ft-internal.h"
#include "ft/node.h"
#include "ft/log-internal.h"
#include "ft/rollback.h"
#include "portability/toku_atomic.h"
#include "util/sort.h"
#include "util/threadpool.h"
#include "util/status.h"
#include "util/scoped_malloc.h"
static FT_UPGRADE_STATUS_S ft_upgrade_status;
......@@ -532,7 +533,7 @@ toku_serialize_ftnode_size (FTNODE node) {
// As of now, this seems to be called if and only if the entire node is supposed
// to be in memory, so we will assert it.
//
toku_assert_entire_node_in_memory(node);
toku_ftnode_assert_fully_in_memory(node);
result += serialize_node_header_size(node);
result += serialize_ftnode_info_size(node);
for (int i = 0; i < node->n_children; i++) {
......@@ -541,208 +542,6 @@ toku_serialize_ftnode_size (FTNODE node) {
return result;
}
struct array_info {
uint32_t offset;
LEAFENTRY* le_array;
uint32_t* key_sizes_array;
const void** key_ptr_array;
};
static int
array_item(const void* key, const uint32_t keylen, const LEAFENTRY &le, const uint32_t idx, struct array_info *const ai) {
ai->le_array[idx+ai->offset] = le;
ai->key_sizes_array[idx+ai->offset] = keylen;
ai->key_ptr_array[idx+ai->offset] = key;
return 0;
}
// There must still be at least one child
// Requires that all messages in buffers above have been applied.
// Because all messages above have been applied, setting msn of all new basements
// to max msn of existing basements is correct. (There cannot be any messages in
// buffers above that still need to be applied.)
void
rebalance_ftnode_leaf(FTNODE node, unsigned int basementnodesize)
{
assert(node->height == 0);
assert(node->dirty);
uint32_t num_orig_basements = node->n_children;
// Count number of leaf entries in this leaf (num_le).
uint32_t num_le = 0;
for (uint32_t i = 0; i < num_orig_basements; i++) {
num_le += BLB_DATA(node, i)->num_klpairs();
}
uint32_t num_alloc = num_le ? num_le : 1; // simplify logic below by always having at least one entry per array
// Create an array of OMTVALUE's that store all the pointers to all the data.
// Each element in leafpointers is a pointer to a leaf.
toku::scoped_malloc leafpointers_buf(sizeof(LEAFENTRY) * num_alloc);
LEAFENTRY *leafpointers = reinterpret_cast<LEAFENTRY *>(leafpointers_buf.get());
leafpointers[0] = NULL;
toku::scoped_malloc key_pointers_buf(sizeof(void *) * num_alloc);
const void **key_pointers = reinterpret_cast<const void **>(key_pointers_buf.get());
key_pointers[0] = NULL;
toku::scoped_malloc key_sizes_buf(sizeof(uint32_t) * num_alloc);
uint32_t *key_sizes = reinterpret_cast<uint32_t *>(key_sizes_buf.get());
// Capture pointers to old mempools' buffers (so they can be destroyed)
toku::scoped_malloc old_bns_buf(sizeof(BASEMENTNODE) * num_orig_basements);
BASEMENTNODE *old_bns = reinterpret_cast<BASEMENTNODE *>(old_bns_buf.get());
old_bns[0] = NULL;
uint32_t curr_le = 0;
for (uint32_t i = 0; i < num_orig_basements; i++) {
bn_data* bd = BLB_DATA(node, i);
struct array_info ai {.offset = curr_le, .le_array = leafpointers, .key_sizes_array = key_sizes, .key_ptr_array = key_pointers };
bd->iterate<array_info, array_item>(&ai);
curr_le += bd->num_klpairs();
}
// Create an array that will store indexes of new pivots.
// Each element in new_pivots is the index of a pivot key.
// (Allocating num_le of them is overkill, but num_le is an upper bound.)
toku::scoped_malloc new_pivots_buf(sizeof(uint32_t) * num_alloc);
uint32_t *new_pivots = reinterpret_cast<uint32_t *>(new_pivots_buf.get());
new_pivots[0] = 0;
// Each element in le_sizes is the size of the leafentry pointed to by leafpointers.
toku::scoped_malloc le_sizes_buf(sizeof(size_t) * num_alloc);
size_t *le_sizes = reinterpret_cast<size_t *>(le_sizes_buf.get());
le_sizes[0] = 0;
// Create an array that will store the size of each basement.
// This is the sum of the leaf sizes of all the leaves in that basement.
// We don't know how many basements there will be, so we use num_le as the upper bound.
// Sum of all le sizes in a single basement
toku::scoped_calloc bn_le_sizes_buf(sizeof(size_t) * num_alloc);
size_t *bn_le_sizes = reinterpret_cast<size_t *>(bn_le_sizes_buf.get());
// Sum of all key sizes in a single basement
toku::scoped_calloc bn_key_sizes_buf(sizeof(size_t) * num_alloc);
size_t *bn_key_sizes = reinterpret_cast<size_t *>(bn_key_sizes_buf.get());
// TODO 4050: All these arrays should be combined into a single array of some bn_info struct (pivot, msize, num_les).
// Each entry is the number of leafentries in this basement. (Again, num_le is overkill upper baound.)
toku::scoped_malloc num_les_this_bn_buf(sizeof(uint32_t) * num_alloc);
uint32_t *num_les_this_bn = reinterpret_cast<uint32_t *>(num_les_this_bn_buf.get());
num_les_this_bn[0] = 0;
// Figure out the new pivots.
// We need the index of each pivot, and for each basement we need
// the number of leaves and the sum of the sizes of the leaves (memory requirement for basement).
uint32_t curr_pivot = 0;
uint32_t num_le_in_curr_bn = 0;
uint32_t bn_size_so_far = 0;
for (uint32_t i = 0; i < num_le; i++) {
uint32_t curr_le_size = leafentry_disksize((LEAFENTRY) leafpointers[i]);
le_sizes[i] = curr_le_size;
if ((bn_size_so_far + curr_le_size + sizeof(uint32_t) + key_sizes[i] > basementnodesize) && (num_le_in_curr_bn != 0)) {
// cap off the current basement node to end with the element before i
new_pivots[curr_pivot] = i-1;
curr_pivot++;
num_le_in_curr_bn = 0;
bn_size_so_far = 0;
}
num_le_in_curr_bn++;
num_les_this_bn[curr_pivot] = num_le_in_curr_bn;
bn_le_sizes[curr_pivot] += curr_le_size;
bn_key_sizes[curr_pivot] += sizeof(uint32_t) + key_sizes[i]; // uint32_t le_offset
bn_size_so_far += curr_le_size + sizeof(uint32_t) + key_sizes[i];
}
// curr_pivot is now the total number of pivot keys in the leaf node
int num_pivots = curr_pivot;
int num_children = num_pivots + 1;
// now we need to fill in the new basement nodes and pivots
// TODO: (Zardosht) this is an ugly thing right now
// Need to figure out how to properly deal with seqinsert.
// I am not happy with how this is being
// handled with basement nodes
uint32_t tmp_seqinsert = BLB_SEQINSERT(node, num_orig_basements - 1);
// choose the max msn applied to any basement as the max msn applied to all new basements
MSN max_msn = ZERO_MSN;
for (uint32_t i = 0; i < num_orig_basements; i++) {
MSN curr_msn = BLB_MAX_MSN_APPLIED(node,i);
max_msn = (curr_msn.msn > max_msn.msn) ? curr_msn : max_msn;
}
// remove the basement node in the node, we've saved a copy
for (uint32_t i = 0; i < num_orig_basements; i++) {
// save a reference to the old basement nodes
// we will need them to ensure that the memory
// stays intact
old_bns[i] = toku_detach_bn(node, i);
}
// Now destroy the old basements, but do not destroy leaves
toku_destroy_ftnode_internals(node);
// now reallocate pieces and start filling them in
invariant(num_children > 0);
node->totalchildkeylens = 0;
XCALLOC_N(num_pivots, node->childkeys); // allocate pointers to pivot structs
node->n_children = num_children;
XCALLOC_N(num_children, node->bp); // allocate pointers to basements (bp)
for (int i = 0; i < num_children; i++) {
set_BLB(node, i, toku_create_empty_bn()); // allocate empty basements and set bp pointers
}
// now we start to fill in the data
// first the pivots
for (int i = 0; i < num_pivots; i++) {
uint32_t keylen = key_sizes[new_pivots[i]];
const void *key = key_pointers[new_pivots[i]];
toku_memdup_dbt(&node->childkeys[i], key, keylen);
node->totalchildkeylens += keylen;
}
uint32_t baseindex_this_bn = 0;
// now the basement nodes
for (int i = 0; i < num_children; i++) {
// put back seqinsert
BLB_SEQINSERT(node, i) = tmp_seqinsert;
// create start (inclusive) and end (exclusive) boundaries for data of basement node
uint32_t curr_start = (i==0) ? 0 : new_pivots[i-1]+1; // index of first leaf in basement
uint32_t curr_end = (i==num_pivots) ? num_le : new_pivots[i]+1; // index of first leaf in next basement
uint32_t num_in_bn = curr_end - curr_start; // number of leaves in this basement
// create indexes for new basement
invariant(baseindex_this_bn == curr_start);
uint32_t num_les_to_copy = num_les_this_bn[i];
invariant(num_les_to_copy == num_in_bn);
bn_data* bd = BLB_DATA(node, i);
bd->set_contents_as_clone_of_sorted_array(
num_les_to_copy,
&key_pointers[baseindex_this_bn],
&key_sizes[baseindex_this_bn],
&leafpointers[baseindex_this_bn],
&le_sizes[baseindex_this_bn],
bn_key_sizes[i], // Total key sizes
bn_le_sizes[i] // total le sizes
);
BP_STATE(node,i) = PT_AVAIL;
BP_TOUCH_CLOCK(node,i);
BLB_MAX_MSN_APPLIED(node,i) = max_msn;
baseindex_this_bn += num_les_to_copy; // set to index of next bn
}
node->max_msn_applied_to_node_on_disk = max_msn;
// destroy buffers of old mempools
for (uint32_t i = 0; i < num_orig_basements; i++) {
destroy_basement_node(old_bns[i]);
}
} // end of rebalance_ftnode_leaf()
struct serialize_times {
tokutime_t serialize_time;
tokutime_t compress_time;
......@@ -907,10 +706,10 @@ int toku_serialize_ftnode_to_memory(FTNODE node,
// The resulting buffer is guaranteed to be 512-byte aligned and the total length is a multiple of 512 (so we pad with zeros at the end if needed).
// 512-byte padding is for O_DIRECT to work.
{
toku_assert_entire_node_in_memory(node);
toku_ftnode_assert_fully_in_memory(node);
if (do_rebalancing && node->height == 0) {
rebalance_ftnode_leaf(node, basementnodesize);
toku_ftnode_leaf_rebalance(node, basementnodesize);
}
const int npartitions = node->n_children;
......
......@@ -96,6 +96,7 @@ PATENT RIGHTS GRANT:
#include "fttypes.h"
#include "ft-internal.h"
#include "node.h"
#include "ft_layout_version.h"
#include "block_table.h"
#include "rbuf.h"
......
......@@ -111,6 +111,7 @@ PATENT RIGHTS GRANT:
#include "leafentry.h"
#include "log-internal.h"
#include "ft.h"
#include "node.h"
static size_t (*os_fwrite_fun)(const void *,size_t,size_t,FILE*)=NULL;
void ft_loader_set_os_fwrite (size_t (*fwrite_fun)(const void*,size_t,size_t,FILE*)) {
......
......@@ -129,23 +129,19 @@ struct message_buffer::buffer_entry *message_buffer::get_buffer_entry(int32_t of
}
void message_buffer::enqueue(FT_MSG msg, bool is_fresh, int32_t *offset) {
ITEMLEN keylen = ft_msg_get_keylen(msg);
ITEMLEN datalen = ft_msg_get_vallen(msg);
XIDS xids = ft_msg_get_xids(msg);
int need_space_here = sizeof(struct buffer_entry)
+ keylen + datalen
+ xids_get_size(xids)
- sizeof(XIDS_S); //Prevent double counting
int need_space_here = msg_memsize_in_buffer(msg);
int need_space_total = _memory_used + need_space_here;
if (_memory == nullptr || need_space_total > _memory_size) {
// resize the buffer to the next power of 2 greater than the needed space
int next_2 = next_power_of_two(need_space_total);
resize(next_2);
}
ITEMLEN keylen = ft_msg_get_keylen(msg);
ITEMLEN datalen = ft_msg_get_vallen(msg);
struct buffer_entry *entry = get_buffer_entry(_memory_used);
entry->type = (unsigned char) ft_msg_get_type(msg);
entry->msn = msg->msn;
xids_cpy(&entry->xids_s, xids);
xids_cpy(&entry->xids_s, ft_msg_get_xids(msg));
entry->is_fresh = is_fresh;
unsigned char *e_key = xids_get_end_of_array(&entry->xids_s);
entry->keylen = keylen;
......@@ -217,8 +213,8 @@ bool message_buffer::equals(message_buffer *other) const {
}
size_t message_buffer::msg_memsize_in_buffer(FT_MSG msg) {
return sizeof(struct buffer_entry)
+ msg->u.id.key->size + msg->u.id.val->size
+ xids_get_size(msg->xids)
- sizeof(XIDS_S);
const uint32_t keylen = ft_msg_get_keylen(msg);
const uint32_t datalen = ft_msg_get_vallen(msg);
const size_t xidslen = xids_get_size(msg->xids);
return sizeof(struct buffer_entry) + keylen + datalen + xidslen - sizeof(XIDS_S);
}
/* -*- mode: C++; c-basic-offset: 4; indent-tabs-mode: nil -*- */
// vim: ft=cpp:expandtab:ts=8:sw=4:softtabstop=4:
/*
COPYING CONDITIONS NOTICE:
This program is free software; you can redistribute it and/or modify
it under the terms of version 2 of the GNU General Public License as
published by the Free Software Foundation, and provided that the
following conditions are met:
* Redistributions of source code must retain this COPYING
CONDITIONS NOTICE, the COPYRIGHT NOTICE (below), the
DISCLAIMER (below), the UNIVERSITY PATENT NOTICE (below), the
PATENT MARKING NOTICE (below), and the PATENT RIGHTS
GRANT (below).
* Redistributions in binary form must reproduce this COPYING
CONDITIONS NOTICE, the COPYRIGHT NOTICE (below), the
DISCLAIMER (below), the UNIVERSITY PATENT NOTICE (below), the
PATENT MARKING NOTICE (below), and the PATENT RIGHTS
GRANT (below) in the documentation and/or other materials
provided with the distribution.
You should have received a copy of the GNU General Public License
along with this program; if not, write to the Free Software
Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA
02110-1301, USA.
COPYRIGHT NOTICE:
TokuDB, Tokutek Fractal Tree Indexing Library.
Copyright (C) 2007-2013 Tokutek, Inc.
DISCLAIMER:
This program is distributed in the hope that it will be useful, but
WITHOUT ANY WARRANTY; without even the implied warranty of
MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
General Public License for more details.
UNIVERSITY PATENT NOTICE:
The technology is licensed by the Massachusetts Institute of
Technology, Rutgers State University of New Jersey, and the Research
Foundation of State University of New York at Stony Brook under
United States of America Serial No. 11/760379 and to the patents
and/or patent applications resulting from it.
PATENT MARKING NOTICE:
This software is covered by US Patent No. 8,185,551.
This software is covered by US Patent No. 8,489,638.
PATENT RIGHTS GRANT:
"THIS IMPLEMENTATION" means the copyrightable works distributed by
Tokutek as part of the Fractal Tree project.
"PATENT CLAIMS" means the claims of patents that are owned or
licensable by Tokutek, both currently or in the future; and that in
the absence of this license would be infringed by THIS
IMPLEMENTATION or by using or running THIS IMPLEMENTATION.
"PATENT CHALLENGE" shall mean a challenge to the validity,
patentability, enforceability and/or non-infringement of any of the
PATENT CLAIMS or otherwise opposing any of the PATENT CLAIMS.
Tokutek hereby grants to you, for the term and geographical scope of
the PATENT CLAIMS, a non-exclusive, no-charge, royalty-free,
irrevocable (except as stated in this section) patent license to
make, have made, use, offer to sell, sell, import, transfer, and
otherwise run, modify, and propagate the contents of THIS
IMPLEMENTATION, where such license applies only to the PATENT
CLAIMS. This grant does not include claims that would be infringed
only as a consequence of further modifications of THIS
IMPLEMENTATION. If you or your agent or licensee institute or order
or agree to the institution of patent litigation against any entity
(including a cross-claim or counterclaim in a lawsuit) alleging that
THIS IMPLEMENTATION constitutes direct or contributory patent
infringement, or inducement of patent infringement, then any rights
granted to you under this License shall terminate as of the date
such litigation is filed. If you or your agent or exclusive
licensee institute or order or agree to the institution of a PATENT
CHALLENGE, then Tokutek may terminate any rights granted to you
under this License.
*/
#ident "Copyright (c) 2007-2013 Tokutek Inc. All rights reserved."
#ident "The technology is licensed by the Massachusetts Institute of Technology, Rutgers State University of New Jersey, and the Research Foundation of State University of New York at Stony Brook under United States of America Serial No. 11/760379 and to the patents and/or patent applications resulting from it."
#include "ft/ft.h"
#include "ft/ft-internal.h"
#include "ft/node.h"
#include "util/scoped_malloc.h"
#include "util/sort.h"
// Effect: Fill in N as an empty ftnode.
// TODO: Rename toku_ftnode_create
void toku_initialize_empty_ftnode(FTNODE n, BLOCKNUM nodename, int height, int num_children, int layout_version, unsigned int flags) {
paranoid_invariant(layout_version != 0);
paranoid_invariant(height >= 0);
n->max_msn_applied_to_node_on_disk = ZERO_MSN; // correct value for root node, harmless for others
n->flags = flags;
n->thisnodename = nodename;
n->layout_version = layout_version;
n->layout_version_original = layout_version;
n->layout_version_read_from_disk = layout_version;
n->height = height;
n->totalchildkeylens = 0;
n->childkeys = 0;
n->bp = 0;
n->n_children = num_children;
n->oldest_referenced_xid_known = TXNID_NONE;
if (num_children > 0) {
XMALLOC_N(num_children-1, n->childkeys);
XMALLOC_N(num_children, n->bp);
for (int i = 0; i < num_children; i++) {
BP_BLOCKNUM(n,i).b=0;
BP_STATE(n,i) = PT_INVALID;
BP_WORKDONE(n,i) = 0;
BP_INIT_TOUCHED_CLOCK(n, i);
set_BNULL(n,i);
if (height > 0) {
set_BNC(n, i, toku_create_empty_nl());
} else {
set_BLB(n, i, toku_create_empty_bn());
}
}
}
n->dirty = 1; // special case exception, it's okay to mark as dirty because the basements are empty
toku_ft_status_note_ftnode(height, true);
}
// destroys the internals of the ftnode, but it does not free the values
// that are stored
// this is common functionality for toku_ftnode_free and rebalance_ftnode_leaf
// MUST NOT do anything besides free the structures that have been allocated
void toku_destroy_ftnode_internals(FTNODE node) {
for (int i=0; i<node->n_children-1; i++) {
toku_destroy_dbt(&node->childkeys[i]);
}
toku_free(node->childkeys);
node->childkeys = NULL;
for (int i=0; i < node->n_children; i++) {
if (BP_STATE(node,i) == PT_AVAIL) {
if (node->height > 0) {
destroy_nonleaf_childinfo(BNC(node,i));
} else {
destroy_basement_node(BLB(node, i));
}
} else if (BP_STATE(node,i) == PT_COMPRESSED) {
SUB_BLOCK sb = BSB(node,i);
toku_free(sb->compressed_ptr);
toku_free(sb);
} else {
paranoid_invariant(is_BNULL(node, i));
}
set_BNULL(node, i);
}
toku_free(node->bp);
node->bp = NULL;
}
/* Frees a node, including all the stuff in the hash table. */
void toku_ftnode_free(FTNODE *nodep) {
FTNODE node = *nodep;
toku_ft_status_note_ftnode(node->height, false);
toku_destroy_ftnode_internals(node);
toku_free(node);
*nodep = nullptr;
}
void toku_ftnode_update_disk_stats(FTNODE ftnode, FT ft, bool for_checkpoint) {
STAT64INFO_S deltas = ZEROSTATS;
// capture deltas before rebalancing basements for serialization
deltas = toku_get_and_clear_basement_stats(ftnode);
// locking not necessary here with respect to checkpointing
// in Clayface (because of the pending lock and cachetable lock
// in toku_cachetable_begin_checkpoint)
// essentially, if we are dealing with a for_checkpoint
// parameter in a function that is called by the flush_callback,
// then the cachetable needs to ensure that this is called in a safe
// manner that does not interfere with the beginning
// of a checkpoint, which it does with the cachetable lock
// and pending lock
toku_ft_update_stats(&ft->h->on_disk_stats, deltas);
if (for_checkpoint) {
toku_ft_update_stats(&ft->checkpoint_header->on_disk_stats, deltas);
}
}
void toku_ftnode_clone_partitions(FTNODE node, FTNODE cloned_node) {
for (int i = 0; i < node->n_children; i++) {
BP_BLOCKNUM(cloned_node,i) = BP_BLOCKNUM(node,i);
paranoid_invariant(BP_STATE(node,i) == PT_AVAIL);
BP_STATE(cloned_node,i) = PT_AVAIL;
BP_WORKDONE(cloned_node, i) = BP_WORKDONE(node, i);
if (node->height == 0) {
set_BLB(cloned_node, i, toku_clone_bn(BLB(node,i)));
} else {
set_BNC(cloned_node, i, toku_clone_nl(BNC(node,i)));
}
}
}
void toku_evict_bn_from_memory(FTNODE node, int childnum, FT h) {
// free the basement node
assert(!node->dirty);
BASEMENTNODE bn = BLB(node, childnum);
toku_ft_decrease_stats(&h->in_memory_stats, bn->stat64_delta);
destroy_basement_node(bn);
set_BNULL(node, childnum);
BP_STATE(node, childnum) = PT_ON_DISK;
}
BASEMENTNODE toku_detach_bn(FTNODE node, int childnum) {
assert(BP_STATE(node, childnum) == PT_AVAIL);
BASEMENTNODE bn = BLB(node, childnum);
set_BNULL(node, childnum);
BP_STATE(node, childnum) = PT_ON_DISK;
return bn;
}
//
// Orthopush
//
struct store_msg_buffer_offset_extra {
int32_t *offsets;
int i;
};
int store_msg_buffer_offset(const int32_t &offset, const uint32_t UU(idx), struct store_msg_buffer_offset_extra *const extra) __attribute__((nonnull(3)));
int store_msg_buffer_offset(const int32_t &offset, const uint32_t UU(idx), struct store_msg_buffer_offset_extra *const extra)
{
extra->offsets[extra->i] = offset;
extra->i++;
return 0;
}
/**
* Given pointers to offsets within a message buffer where we can find messages,
* figure out the MSN of each message, and compare those MSNs. Returns 1,
* 0, or -1 if a is larger than, equal to, or smaller than b.
*/
int msg_buffer_offset_msn_cmp(message_buffer &msg_buffer, const int32_t &ao, const int32_t &bo);
int msg_buffer_offset_msn_cmp(message_buffer &msg_buffer, const int32_t &ao, const int32_t &bo)
{
MSN amsn, bmsn;
msg_buffer.get_message_key_msn(ao, nullptr, &amsn);
msg_buffer.get_message_key_msn(bo, nullptr, &bmsn);
if (amsn.msn > bmsn.msn) {
return +1;
}
if (amsn.msn < bmsn.msn) {
return -1;
}
return 0;
}
/**
* Given a message buffer and and offset, apply the message with toku_ft_bn_apply_msg, or discard it,
* based on its MSN and the MSN of the basement node.
*/
static void
do_bn_apply_msg(FT_HANDLE ft_handle, BASEMENTNODE bn, message_buffer *msg_buffer, int32_t offset,
txn_gc_info *gc_info, uint64_t *workdone, STAT64INFO stats_to_update) {
DBT k, v;
FT_MSG_S msg = msg_buffer->get_message(offset, &k, &v);
// The messages are being iterated over in (key,msn) order or just in
// msn order, so all the messages for one key, from one buffer, are in
// ascending msn order. So it's ok that we don't update the basement
// node's msn until the end.
if (msg.msn.msn > bn->max_msn_applied.msn) {
toku_ft_bn_apply_msg(
ft_handle->ft->compare_fun,
ft_handle->ft->update_fun,
&ft_handle->ft->cmp_descriptor,
bn,
&msg,
gc_info,
workdone,
stats_to_update
);
} else {
toku_ft_status_note_msn_discard();
}
// We must always mark message as stale since it has been marked
// (using omt::iterate_and_mark_range)
// It is possible to call do_bn_apply_msg even when it won't apply the message because
// the node containing it could have been evicted and brought back in.
msg_buffer->set_freshness(offset, false);
}
struct iterate_do_bn_apply_msg_extra {
FT_HANDLE t;
BASEMENTNODE bn;
NONLEAF_CHILDINFO bnc;
txn_gc_info *gc_info;
uint64_t *workdone;
STAT64INFO stats_to_update;
};
int iterate_do_bn_apply_msg(const int32_t &offset, const uint32_t UU(idx), struct iterate_do_bn_apply_msg_extra *const e) __attribute__((nonnull(3)));
int iterate_do_bn_apply_msg(const int32_t &offset, const uint32_t UU(idx), struct iterate_do_bn_apply_msg_extra *const e)
{
do_bn_apply_msg(e->t, e->bn, &e->bnc->msg_buffer, offset, e->gc_info, e->workdone, e->stats_to_update);
return 0;
}
/**
* Given the bounds of the basement node to which we will apply messages,
* find the indexes within message_tree which contain the range of
* relevant messages.
*
* The message tree contains offsets into the buffer, where messages are
* found. The pivot_bounds are the lower bound exclusive and upper bound
* inclusive, because they come from pivot keys in the tree. We want OMT
* indices, which must have the lower bound be inclusive and the upper
* bound exclusive. We will get these by telling omt::find to look
* for something strictly bigger than each of our pivot bounds.
*
* Outputs the OMT indices in lbi (lower bound inclusive) and ube (upper
* bound exclusive).
*/
template<typename find_bounds_omt_t>
static void
find_bounds_within_message_tree(
DESCRIPTOR desc, /// used for cmp
ft_compare_func cmp, /// used to compare keys
const find_bounds_omt_t &message_tree, /// tree holding message buffer offsets, in which we want to look for indices
message_buffer *msg_buffer, /// message buffer in which messages are found
struct pivot_bounds const * const bounds, /// key bounds within the basement node we're applying messages to
uint32_t *lbi, /// (output) "lower bound inclusive" (index into message_tree)
uint32_t *ube /// (output) "upper bound exclusive" (index into message_tree)
)
{
int r = 0;
if (bounds->lower_bound_exclusive) {
// By setting msn to MAX_MSN and by using direction of +1, we will
// get the first message greater than (in (key, msn) order) any
// message (with any msn) with the key lower_bound_exclusive.
// This will be a message we want to try applying, so it is the
// "lower bound inclusive" within the message_tree.
struct toku_msg_buffer_key_msn_heaviside_extra lbi_extra;
ZERO_STRUCT(lbi_extra);
lbi_extra.desc = desc;
lbi_extra.cmp = cmp;
lbi_extra.msg_buffer = msg_buffer;
lbi_extra.key = bounds->lower_bound_exclusive;
lbi_extra.msn = MAX_MSN;
int32_t found_lb;
r = message_tree.template find<struct toku_msg_buffer_key_msn_heaviside_extra, toku_msg_buffer_key_msn_heaviside>(lbi_extra, +1, &found_lb, lbi);
if (r == DB_NOTFOUND) {
// There is no relevant data (the lower bound is bigger than
// any message in this tree), so we have no range and we're
// done.
*lbi = 0;
*ube = 0;
return;
}
if (bounds->upper_bound_inclusive) {
// Check if what we found for lbi is greater than the upper
// bound inclusive that we have. If so, there are no relevant
// messages between these bounds.
const DBT *ubi = bounds->upper_bound_inclusive;
const int32_t offset = found_lb;
DBT found_lbidbt;
msg_buffer->get_message_key_msn(offset, &found_lbidbt, nullptr);
FAKE_DB(db, desc);
int c = cmp(&db, &found_lbidbt, ubi);
// These DBTs really are both inclusive bounds, so we need
// strict inequality in order to determine that there's
// nothing between them. If they're equal, then we actually
// need to apply the message pointed to by lbi, and also
// anything with the same key but a bigger msn.
if (c > 0) {
*lbi = 0;
*ube = 0;
return;
}
}
} else {
// No lower bound given, it's negative infinity, so we start at
// the first message in the OMT.
*lbi = 0;
}
if (bounds->upper_bound_inclusive) {
// Again, we use an msn of MAX_MSN and a direction of +1 to get
// the first thing bigger than the upper_bound_inclusive key.
// This is therefore the smallest thing we don't want to apply,
// and omt::iterate_on_range will not examine it.
struct toku_msg_buffer_key_msn_heaviside_extra ube_extra;
ZERO_STRUCT(ube_extra);
ube_extra.desc = desc;
ube_extra.cmp = cmp;
ube_extra.msg_buffer = msg_buffer;
ube_extra.key = bounds->upper_bound_inclusive;
ube_extra.msn = MAX_MSN;
r = message_tree.template find<struct toku_msg_buffer_key_msn_heaviside_extra, toku_msg_buffer_key_msn_heaviside>(ube_extra, +1, nullptr, ube);
if (r == DB_NOTFOUND) {
// Couldn't find anything in the buffer bigger than our key,
// so we need to look at everything up to the end of
// message_tree.
*ube = message_tree.size();
}
} else {
// No upper bound given, it's positive infinity, so we need to go
// through the end of the OMT.
*ube = message_tree.size();
}
}
/**
* For each message in the ancestor's buffer (determined by childnum) that
* is key-wise between lower_bound_exclusive and upper_bound_inclusive,
* apply the message to the basement node. We treat the bounds as minus
* or plus infinity respectively if they are NULL. Do not mark the node
* as dirty (preserve previous state of 'dirty' bit).
*/
static void
bnc_apply_messages_to_basement_node(
FT_HANDLE t, // used for comparison function
BASEMENTNODE bn, // where to apply messages
FTNODE ancestor, // the ancestor node where we can find messages to apply
int childnum, // which child buffer of ancestor contains messages we want
struct pivot_bounds const * const bounds, // contains pivot key bounds of this basement node
txn_gc_info *gc_info,
bool* msgs_applied
)
{
int r;
NONLEAF_CHILDINFO bnc = BNC(ancestor, childnum);
// Determine the offsets in the message trees between which we need to
// apply messages from this buffer
STAT64INFO_S stats_delta = {0,0};
uint64_t workdone_this_ancestor = 0;
uint32_t stale_lbi, stale_ube;
if (!bn->stale_ancestor_messages_applied) {
find_bounds_within_message_tree(&t->ft->cmp_descriptor, t->ft->compare_fun, bnc->stale_message_tree, &bnc->msg_buffer, bounds, &stale_lbi, &stale_ube);
} else {
stale_lbi = 0;
stale_ube = 0;
}
uint32_t fresh_lbi, fresh_ube;
find_bounds_within_message_tree(&t->ft->cmp_descriptor, t->ft->compare_fun, bnc->fresh_message_tree, &bnc->msg_buffer, bounds, &fresh_lbi, &fresh_ube);
// We now know where all the messages we must apply are, so one of the
// following 4 cases will do the application, depending on which of
// the lists contains relevant messages:
//
// 1. broadcast messages and anything else, or a mix of fresh and stale
// 2. only fresh messages
// 3. only stale messages
if (bnc->broadcast_list.size() > 0 ||
(stale_lbi != stale_ube && fresh_lbi != fresh_ube)) {
// We have messages in multiple trees, so we grab all
// the relevant messages' offsets and sort them by MSN, then apply
// them in MSN order.
const int buffer_size = ((stale_ube - stale_lbi) + (fresh_ube - fresh_lbi) + bnc->broadcast_list.size());
toku::scoped_malloc offsets_buf(buffer_size * sizeof(int32_t));
int32_t *offsets = reinterpret_cast<int32_t *>(offsets_buf.get());
struct store_msg_buffer_offset_extra sfo_extra = { .offsets = offsets, .i = 0 };
// Populate offsets array with offsets to stale messages
r = bnc->stale_message_tree.iterate_on_range<struct store_msg_buffer_offset_extra, store_msg_buffer_offset>(stale_lbi, stale_ube, &sfo_extra);
assert_zero(r);
// Then store fresh offsets, and mark them to be moved to stale later.
r = bnc->fresh_message_tree.iterate_and_mark_range<struct store_msg_buffer_offset_extra, store_msg_buffer_offset>(fresh_lbi, fresh_ube, &sfo_extra);
assert_zero(r);
// Store offsets of all broadcast messages.
r = bnc->broadcast_list.iterate<struct store_msg_buffer_offset_extra, store_msg_buffer_offset>(&sfo_extra);
assert_zero(r);
invariant(sfo_extra.i == buffer_size);
// Sort by MSN.
r = toku::sort<int32_t, message_buffer, msg_buffer_offset_msn_cmp>::mergesort_r(offsets, buffer_size, bnc->msg_buffer);
assert_zero(r);
// Apply the messages in MSN order.
for (int i = 0; i < buffer_size; ++i) {
*msgs_applied = true;
do_bn_apply_msg(t, bn, &bnc->msg_buffer, offsets[i], gc_info, &workdone_this_ancestor, &stats_delta);
}
} else if (stale_lbi == stale_ube) {
// No stale messages to apply, we just apply fresh messages, and mark them to be moved to stale later.
struct iterate_do_bn_apply_msg_extra iter_extra = { .t = t, .bn = bn, .bnc = bnc, .gc_info = gc_info, .workdone = &workdone_this_ancestor, .stats_to_update = &stats_delta };
if (fresh_ube - fresh_lbi > 0) *msgs_applied = true;
r = bnc->fresh_message_tree.iterate_and_mark_range<struct iterate_do_bn_apply_msg_extra, iterate_do_bn_apply_msg>(fresh_lbi, fresh_ube, &iter_extra);
assert_zero(r);
} else {
invariant(fresh_lbi == fresh_ube);
// No fresh messages to apply, we just apply stale messages.
if (stale_ube - stale_lbi > 0) *msgs_applied = true;
struct iterate_do_bn_apply_msg_extra iter_extra = { .t = t, .bn = bn, .bnc = bnc, .gc_info = gc_info, .workdone = &workdone_this_ancestor, .stats_to_update = &stats_delta };
r = bnc->stale_message_tree.iterate_on_range<struct iterate_do_bn_apply_msg_extra, iterate_do_bn_apply_msg>(stale_lbi, stale_ube, &iter_extra);
assert_zero(r);
}
//
// update stats
//
if (workdone_this_ancestor > 0) {
(void) toku_sync_fetch_and_add(&BP_WORKDONE(ancestor, childnum), workdone_this_ancestor);
}
if (stats_delta.numbytes || stats_delta.numrows) {
toku_ft_update_stats(&t->ft->in_memory_stats, stats_delta);
}
}
static void
apply_ancestors_messages_to_bn(
FT_HANDLE t,
FTNODE node,
int childnum,
ANCESTORS ancestors,
struct pivot_bounds const * const bounds,
txn_gc_info *gc_info,
bool* msgs_applied
)
{
BASEMENTNODE curr_bn = BLB(node, childnum);
struct pivot_bounds curr_bounds = next_pivot_keys(node, childnum, bounds);
for (ANCESTORS curr_ancestors = ancestors; curr_ancestors; curr_ancestors = curr_ancestors->next) {
if (curr_ancestors->node->max_msn_applied_to_node_on_disk.msn > curr_bn->max_msn_applied.msn) {
paranoid_invariant(BP_STATE(curr_ancestors->node, curr_ancestors->childnum) == PT_AVAIL);
bnc_apply_messages_to_basement_node(
t,
curr_bn,
curr_ancestors->node,
curr_ancestors->childnum,
&curr_bounds,
gc_info,
msgs_applied
);
// We don't want to check this ancestor node again if the
// next time we query it, the msn hasn't changed.
curr_bn->max_msn_applied = curr_ancestors->node->max_msn_applied_to_node_on_disk;
}
}
// At this point, we know all the stale messages above this
// basement node have been applied, and any new messages will be
// fresh, so we don't need to look at stale messages for this
// basement node, unless it gets evicted (and this field becomes
// false when it's read in again).
curr_bn->stale_ancestor_messages_applied = true;
}
void
toku_apply_ancestors_messages_to_node (
FT_HANDLE t,
FTNODE node,
ANCESTORS ancestors,
struct pivot_bounds const * const bounds,
bool* msgs_applied,
int child_to_read
)
// Effect:
// Bring a leaf node up-to-date according to all the messages in the ancestors.
// If the leaf node is already up-to-date then do nothing.
// If the leaf node is not already up-to-date, then record the work done
// for that leaf in each ancestor.
// Requires:
// This is being called when pinning a leaf node for the query path.
// The entire root-to-leaf path is pinned and appears in the ancestors list.
{
VERIFY_NODE(t, node);
paranoid_invariant(node->height == 0);
TXN_MANAGER txn_manager = toku_ft_get_txn_manager(t);
txn_manager_state txn_state_for_gc(txn_manager);
TXNID oldest_referenced_xid_for_simple_gc = toku_ft_get_oldest_referenced_xid_estimate(t);
txn_gc_info gc_info(&txn_state_for_gc,
oldest_referenced_xid_for_simple_gc,
node->oldest_referenced_xid_known,
true);
if (!node->dirty && child_to_read >= 0) {
paranoid_invariant(BP_STATE(node, child_to_read) == PT_AVAIL);
apply_ancestors_messages_to_bn(
t,
node,
child_to_read,
ancestors,
bounds,
&gc_info,
msgs_applied
);
}
else {
// know we are a leaf node
// An important invariant:
// We MUST bring every available basement node for a dirty node up to date.
// flushing on the cleaner thread depends on this. This invariant
// allows the cleaner thread to just pick an internal node and flush it
// as opposed to being forced to start from the root.
for (int i = 0; i < node->n_children; i++) {
if (BP_STATE(node, i) != PT_AVAIL) { continue; }
apply_ancestors_messages_to_bn(
t,
node,
i,
ancestors,
bounds,
&gc_info,
msgs_applied
);
}
}
VERIFY_NODE(t, node);
}
static bool bn_needs_ancestors_messages(
FT ft,
FTNODE node,
int childnum,
struct pivot_bounds const * const bounds,
ANCESTORS ancestors,
MSN* max_msn_applied
)
{
BASEMENTNODE bn = BLB(node, childnum);
struct pivot_bounds curr_bounds = next_pivot_keys(node, childnum, bounds);
bool needs_ancestors_messages = false;
for (ANCESTORS curr_ancestors = ancestors; curr_ancestors; curr_ancestors = curr_ancestors->next) {
if (curr_ancestors->node->max_msn_applied_to_node_on_disk.msn > bn->max_msn_applied.msn) {
paranoid_invariant(BP_STATE(curr_ancestors->node, curr_ancestors->childnum) == PT_AVAIL);
NONLEAF_CHILDINFO bnc = BNC(curr_ancestors->node, curr_ancestors->childnum);
if (bnc->broadcast_list.size() > 0) {
needs_ancestors_messages = true;
goto cleanup;
}
if (!bn->stale_ancestor_messages_applied) {
uint32_t stale_lbi, stale_ube;
find_bounds_within_message_tree(&ft->cmp_descriptor,
ft->compare_fun,
bnc->stale_message_tree,
&bnc->msg_buffer,
&curr_bounds,
&stale_lbi,
&stale_ube);
if (stale_lbi < stale_ube) {
needs_ancestors_messages = true;
goto cleanup;
}
}
uint32_t fresh_lbi, fresh_ube;
find_bounds_within_message_tree(&ft->cmp_descriptor,
ft->compare_fun,
bnc->fresh_message_tree,
&bnc->msg_buffer,
&curr_bounds,
&fresh_lbi,
&fresh_ube);
if (fresh_lbi < fresh_ube) {
needs_ancestors_messages = true;
goto cleanup;
}
if (curr_ancestors->node->max_msn_applied_to_node_on_disk.msn > max_msn_applied->msn) {
max_msn_applied->msn = curr_ancestors->node->max_msn_applied_to_node_on_disk.msn;
}
}
}
cleanup:
return needs_ancestors_messages;
}
bool toku_ft_leaf_needs_ancestors_messages(
FT ft,
FTNODE node,
ANCESTORS ancestors,
struct pivot_bounds const * const bounds,
MSN *const max_msn_in_path,
int child_to_read
)
// Effect: Determine whether there are messages in a node's ancestors
// which must be applied to it. These messages are in the correct
// keyrange for any available basement nodes, and are in nodes with the
// correct max_msn_applied_to_node_on_disk.
// Notes:
// This is an approximate query.
// Output:
// max_msn_in_path: max of "max_msn_applied_to_node_on_disk" over
// ancestors. This is used later to update basement nodes'
// max_msn_applied values in case we don't do the full algorithm.
// Returns:
// true if there may be some such messages
// false only if there are definitely no such messages
// Rationale:
// When we pin a node with a read lock, we want to quickly determine if
// we should exchange it for a write lock in preparation for applying
// messages. If there are no messages, we don't need the write lock.
{
paranoid_invariant(node->height == 0);
bool needs_ancestors_messages = false;
// child_to_read may be -1 in test cases
if (!node->dirty && child_to_read >= 0) {
paranoid_invariant(BP_STATE(node, child_to_read) == PT_AVAIL);
needs_ancestors_messages = bn_needs_ancestors_messages(
ft,
node,
child_to_read,
bounds,
ancestors,
max_msn_in_path
);
}
else {
for (int i = 0; i < node->n_children; ++i) {
if (BP_STATE(node, i) != PT_AVAIL) { continue; }
needs_ancestors_messages = bn_needs_ancestors_messages(
ft,
node,
i,
bounds,
ancestors,
max_msn_in_path
);
if (needs_ancestors_messages) {
goto cleanup;
}
}
}
cleanup:
return needs_ancestors_messages;
}
void toku_ft_bn_update_max_msn(FTNODE node, MSN max_msn_applied, int child_to_read) {
invariant(node->height == 0);
if (!node->dirty && child_to_read >= 0) {
paranoid_invariant(BP_STATE(node, child_to_read) == PT_AVAIL);
BASEMENTNODE bn = BLB(node, child_to_read);
if (max_msn_applied.msn > bn->max_msn_applied.msn) {
// see comment below
(void) toku_sync_val_compare_and_swap(&bn->max_msn_applied.msn, bn->max_msn_applied.msn, max_msn_applied.msn);
}
}
else {
for (int i = 0; i < node->n_children; ++i) {
if (BP_STATE(node, i) != PT_AVAIL) { continue; }
BASEMENTNODE bn = BLB(node, i);
if (max_msn_applied.msn > bn->max_msn_applied.msn) {
// This function runs in a shared access context, so to silence tools
// like DRD, we use a CAS and ignore the result.
// Any threads trying to update these basement nodes should be
// updating them to the same thing (since they all have a read lock on
// the same root-to-leaf path) so this is safe.
(void) toku_sync_val_compare_and_swap(&bn->max_msn_applied.msn, bn->max_msn_applied.msn, max_msn_applied.msn);
}
}
}
}
struct copy_to_stale_extra {
FT ft;
NONLEAF_CHILDINFO bnc;
};
int copy_to_stale(const int32_t &offset, const uint32_t UU(idx), struct copy_to_stale_extra *const extra) __attribute__((nonnull(3)));
int copy_to_stale(const int32_t &offset, const uint32_t UU(idx), struct copy_to_stale_extra *const extra)
{
MSN msn;
DBT key;
extra->bnc->msg_buffer.get_message_key_msn(offset, &key, &msn);
struct toku_msg_buffer_key_msn_heaviside_extra heaviside_extra = { .desc = &extra->ft->cmp_descriptor, .cmp = extra->ft->compare_fun, .msg_buffer = &extra->bnc->msg_buffer, .key = &key, .msn = msn };
int r = extra->bnc->stale_message_tree.insert<struct toku_msg_buffer_key_msn_heaviside_extra, toku_msg_buffer_key_msn_heaviside>(offset, heaviside_extra, nullptr);
invariant_zero(r);
return 0;
}
void toku_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();
}
void toku_move_ftnode_messages_to_stale(FT ft, FTNODE node) {
invariant(node->height > 0);
for (int i = 0; i < node->n_children; ++i) {
if (BP_STATE(node, i) != PT_AVAIL) {
continue;
}
NONLEAF_CHILDINFO bnc = BNC(node, i);
// 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.
toku_ft_bnc_move_messages_to_stale(ft, bnc);
}
}
//
// Balance // Availibility // Size
struct rebalance_array_info {
uint32_t offset;
LEAFENTRY *le_array;
uint32_t *key_sizes_array;
const void **key_ptr_array;
static int fn(const void* key, const uint32_t keylen, const LEAFENTRY &le,
const uint32_t idx, struct rebalance_array_info *const ai) {
ai->le_array[idx+ai->offset] = le;
ai->key_sizes_array[idx+ai->offset] = keylen;
ai->key_ptr_array[idx+ai->offset] = key;
return 0;
}
};
// There must still be at least one child
// Requires that all messages in buffers above have been applied.
// Because all messages above have been applied, setting msn of all new basements
// to max msn of existing basements is correct. (There cannot be any messages in
// buffers above that still need to be applied.)
void toku_ftnode_leaf_rebalance(FTNODE node, unsigned int basementnodesize) {
assert(node->height == 0);
assert(node->dirty);
uint32_t num_orig_basements = node->n_children;
// Count number of leaf entries in this leaf (num_le).
uint32_t num_le = 0;
for (uint32_t i = 0; i < num_orig_basements; i++) {
num_le += BLB_DATA(node, i)->num_klpairs();
}
uint32_t num_alloc = num_le ? num_le : 1; // simplify logic below by always having at least one entry per array
// Create an array of OMTVALUE's that store all the pointers to all the data.
// Each element in leafpointers is a pointer to a leaf.
toku::scoped_malloc leafpointers_buf(sizeof(LEAFENTRY) * num_alloc);
LEAFENTRY *leafpointers = reinterpret_cast<LEAFENTRY *>(leafpointers_buf.get());
leafpointers[0] = NULL;
toku::scoped_malloc key_pointers_buf(sizeof(void *) * num_alloc);
const void **key_pointers = reinterpret_cast<const void **>(key_pointers_buf.get());
key_pointers[0] = NULL;
toku::scoped_malloc key_sizes_buf(sizeof(uint32_t) * num_alloc);
uint32_t *key_sizes = reinterpret_cast<uint32_t *>(key_sizes_buf.get());
// Capture pointers to old mempools' buffers (so they can be destroyed)
toku::scoped_malloc old_bns_buf(sizeof(BASEMENTNODE) * num_orig_basements);
BASEMENTNODE *old_bns = reinterpret_cast<BASEMENTNODE *>(old_bns_buf.get());
old_bns[0] = NULL;
uint32_t curr_le = 0;
for (uint32_t i = 0; i < num_orig_basements; i++) {
bn_data* bd = BLB_DATA(node, i);
struct rebalance_array_info ai {.offset = curr_le, .le_array = leafpointers, .key_sizes_array = key_sizes, .key_ptr_array = key_pointers };
bd->iterate<rebalance_array_info, rebalance_array_info::fn>(&ai);
curr_le += bd->num_klpairs();
}
// Create an array that will store indexes of new pivots.
// Each element in new_pivots is the index of a pivot key.
// (Allocating num_le of them is overkill, but num_le is an upper bound.)
toku::scoped_malloc new_pivots_buf(sizeof(uint32_t) * num_alloc);
uint32_t *new_pivots = reinterpret_cast<uint32_t *>(new_pivots_buf.get());
new_pivots[0] = 0;
// Each element in le_sizes is the size of the leafentry pointed to by leafpointers.
toku::scoped_malloc le_sizes_buf(sizeof(size_t) * num_alloc);
size_t *le_sizes = reinterpret_cast<size_t *>(le_sizes_buf.get());
le_sizes[0] = 0;
// Create an array that will store the size of each basement.
// This is the sum of the leaf sizes of all the leaves in that basement.
// We don't know how many basements there will be, so we use num_le as the upper bound.
// Sum of all le sizes in a single basement
toku::scoped_calloc bn_le_sizes_buf(sizeof(size_t) * num_alloc);
size_t *bn_le_sizes = reinterpret_cast<size_t *>(bn_le_sizes_buf.get());
// Sum of all key sizes in a single basement
toku::scoped_calloc bn_key_sizes_buf(sizeof(size_t) * num_alloc);
size_t *bn_key_sizes = reinterpret_cast<size_t *>(bn_key_sizes_buf.get());
// TODO 4050: All these arrays should be combined into a single array of some bn_info struct (pivot, msize, num_les).
// Each entry is the number of leafentries in this basement. (Again, num_le is overkill upper baound.)
toku::scoped_malloc num_les_this_bn_buf(sizeof(uint32_t) * num_alloc);
uint32_t *num_les_this_bn = reinterpret_cast<uint32_t *>(num_les_this_bn_buf.get());
num_les_this_bn[0] = 0;
// Figure out the new pivots.
// We need the index of each pivot, and for each basement we need
// the number of leaves and the sum of the sizes of the leaves (memory requirement for basement).
uint32_t curr_pivot = 0;
uint32_t num_le_in_curr_bn = 0;
uint32_t bn_size_so_far = 0;
for (uint32_t i = 0; i < num_le; i++) {
uint32_t curr_le_size = leafentry_disksize((LEAFENTRY) leafpointers[i]);
le_sizes[i] = curr_le_size;
if ((bn_size_so_far + curr_le_size + sizeof(uint32_t) + key_sizes[i] > basementnodesize) && (num_le_in_curr_bn != 0)) {
// cap off the current basement node to end with the element before i
new_pivots[curr_pivot] = i-1;
curr_pivot++;
num_le_in_curr_bn = 0;
bn_size_so_far = 0;
}
num_le_in_curr_bn++;
num_les_this_bn[curr_pivot] = num_le_in_curr_bn;
bn_le_sizes[curr_pivot] += curr_le_size;
bn_key_sizes[curr_pivot] += sizeof(uint32_t) + key_sizes[i]; // uint32_t le_offset
bn_size_so_far += curr_le_size + sizeof(uint32_t) + key_sizes[i];
}
// curr_pivot is now the total number of pivot keys in the leaf node
int num_pivots = curr_pivot;
int num_children = num_pivots + 1;
// now we need to fill in the new basement nodes and pivots
// TODO: (Zardosht) this is an ugly thing right now
// Need to figure out how to properly deal with seqinsert.
// I am not happy with how this is being
// handled with basement nodes
uint32_t tmp_seqinsert = BLB_SEQINSERT(node, num_orig_basements - 1);
// choose the max msn applied to any basement as the max msn applied to all new basements
MSN max_msn = ZERO_MSN;
for (uint32_t i = 0; i < num_orig_basements; i++) {
MSN curr_msn = BLB_MAX_MSN_APPLIED(node,i);
max_msn = (curr_msn.msn > max_msn.msn) ? curr_msn : max_msn;
}
// remove the basement node in the node, we've saved a copy
for (uint32_t i = 0; i < num_orig_basements; i++) {
// save a reference to the old basement nodes
// we will need them to ensure that the memory
// stays intact
old_bns[i] = toku_detach_bn(node, i);
}
// Now destroy the old basements, but do not destroy leaves
toku_destroy_ftnode_internals(node);
// now reallocate pieces and start filling them in
invariant(num_children > 0);
node->totalchildkeylens = 0;
XCALLOC_N(num_pivots, node->childkeys); // allocate pointers to pivot structs
node->n_children = num_children;
XCALLOC_N(num_children, node->bp); // allocate pointers to basements (bp)
for (int i = 0; i < num_children; i++) {
set_BLB(node, i, toku_create_empty_bn()); // allocate empty basements and set bp pointers
}
// now we start to fill in the data
// first the pivots
for (int i = 0; i < num_pivots; i++) {
uint32_t keylen = key_sizes[new_pivots[i]];
const void *key = key_pointers[new_pivots[i]];
toku_memdup_dbt(&node->childkeys[i], key, keylen);
node->totalchildkeylens += keylen;
}
uint32_t baseindex_this_bn = 0;
// now the basement nodes
for (int i = 0; i < num_children; i++) {
// put back seqinsert
BLB_SEQINSERT(node, i) = tmp_seqinsert;
// create start (inclusive) and end (exclusive) boundaries for data of basement node
uint32_t curr_start = (i==0) ? 0 : new_pivots[i-1]+1; // index of first leaf in basement
uint32_t curr_end = (i==num_pivots) ? num_le : new_pivots[i]+1; // index of first leaf in next basement
uint32_t num_in_bn = curr_end - curr_start; // number of leaves in this basement
// create indexes for new basement
invariant(baseindex_this_bn == curr_start);
uint32_t num_les_to_copy = num_les_this_bn[i];
invariant(num_les_to_copy == num_in_bn);
bn_data* bd = BLB_DATA(node, i);
bd->set_contents_as_clone_of_sorted_array(
num_les_to_copy,
&key_pointers[baseindex_this_bn],
&key_sizes[baseindex_this_bn],
&leafpointers[baseindex_this_bn],
&le_sizes[baseindex_this_bn],
bn_key_sizes[i], // Total key sizes
bn_le_sizes[i] // total le sizes
);
BP_STATE(node,i) = PT_AVAIL;
BP_TOUCH_CLOCK(node,i);
BLB_MAX_MSN_APPLIED(node,i) = max_msn;
baseindex_this_bn += num_les_to_copy; // set to index of next bn
}
node->max_msn_applied_to_node_on_disk = max_msn;
// destroy buffers of old mempools
for (uint32_t i = 0; i < num_orig_basements; i++) {
destroy_basement_node(old_bns[i]);
}
}
bool toku_ftnode_fully_in_memory(FTNODE node) {
for (int i = 0; i < node->n_children; i++) {
if (BP_STATE(node,i) != PT_AVAIL) {
return false;
}
}
return true;
}
void toku_ftnode_assert_fully_in_memory(FTNODE UU(node)) {
paranoid_invariant(toku_ftnode_fully_in_memory(node));
}
uint32_t toku_ftnode_leaf_num_entries(FTNODE node) {
toku_ftnode_assert_fully_in_memory(node);
uint32_t num_entries = 0;
for (int i = 0; i < node->n_children; i++) {
num_entries += BLB_DATA(node, i)->num_klpairs();
}
return num_entries;
}
enum reactivity toku_ftnode_get_leaf_reactivity(FTNODE node, uint32_t nodesize) {
enum reactivity re = RE_STABLE;
toku_ftnode_assert_fully_in_memory(node);
paranoid_invariant(node->height==0);
unsigned int size = toku_serialize_ftnode_size(node);
if (size > nodesize && toku_ftnode_leaf_num_entries(node) > 1) {
re = RE_FISSIBLE;
} else if ((size*4) < nodesize && !BLB_SEQINSERT(node, node->n_children-1)) {
re = RE_FUSIBLE;
}
return re;
}
enum reactivity toku_ftnode_get_nonleaf_reactivity(FTNODE node, unsigned int fanout) {
paranoid_invariant(node->height > 0);
int n_children = node->n_children;
if (n_children > (int) fanout) {
return RE_FISSIBLE;
}
if (n_children * 4 < (int) fanout) {
return RE_FUSIBLE;
}
return RE_STABLE;
}
enum reactivity toku_ftnode_get_reactivity(FT ft, FTNODE node) {
toku_ftnode_assert_fully_in_memory(node);
if (node->height == 0) {
return toku_ftnode_get_leaf_reactivity(node, ft->h->nodesize);
} else {
return toku_ftnode_get_nonleaf_reactivity(node, ft->h->fanout);
}
}
unsigned int toku_bnc_nbytesinbuf(NONLEAF_CHILDINFO bnc) {
return bnc->msg_buffer.buffer_size_in_use();
}
// Return true if the size of the buffers plus the amount of work done is large enough.
// Return false if there is nothing to be flushed (the buffers empty).
bool toku_ftnode_nonleaf_is_gorged(FTNODE node, uint32_t nodesize) {
uint64_t size = toku_serialize_ftnode_size(node);
bool buffers_are_empty = true;
toku_ftnode_assert_fully_in_memory(node);
//
// the nonleaf node is gorged if the following holds true:
// - the buffers are non-empty
// - the total workdone by the buffers PLUS the size of the buffers
// is greater than nodesize (which as of Maxwell should be
// 4MB)
//
paranoid_invariant(node->height > 0);
for (int child = 0; child < node->n_children; ++child) {
size += BP_WORKDONE(node, child);
}
for (int child = 0; child < node->n_children; ++child) {
if (toku_bnc_nbytesinbuf(BNC(node, child)) > 0) {
buffers_are_empty = false;
break;
}
}
return ((size > nodesize)
&&
(!buffers_are_empty));
}
int toku_bnc_n_entries(NONLEAF_CHILDINFO bnc) {
return bnc->msg_buffer.num_entries();
}
// how much memory does this child buffer consume?
long toku_bnc_memory_size(NONLEAF_CHILDINFO bnc) {
return (sizeof(*bnc) +
bnc->msg_buffer.memory_footprint() +
bnc->fresh_message_tree.memory_size() +
bnc->stale_message_tree.memory_size() +
bnc->broadcast_list.memory_size());
}
// how much memory in this child buffer holds useful data?
// originally created solely for use by test program(s).
long toku_bnc_memory_used(NONLEAF_CHILDINFO bnc) {
return (sizeof(*bnc) +
bnc->msg_buffer.memory_size_in_use() +
bnc->fresh_message_tree.memory_size() +
bnc->stale_message_tree.memory_size() +
bnc->broadcast_list.memory_size());
}
//
// Garbage collection
// Message injection
// Message application
//
static void
init_childinfo(FTNODE node, int childnum, FTNODE child) {
BP_BLOCKNUM(node,childnum) = child->thisnodename;
BP_STATE(node,childnum) = PT_AVAIL;
BP_WORKDONE(node, childnum) = 0;
set_BNC(node, childnum, toku_create_empty_nl());
}
static void
init_childkey(FTNODE node, int childnum, const DBT *pivotkey) {
toku_clone_dbt(&node->childkeys[childnum], *pivotkey);
node->totalchildkeylens += pivotkey->size;
}
// Used only by test programs: append a child node to a parent node
void
toku_ft_nonleaf_append_child(FTNODE node, FTNODE child, const DBT *pivotkey) {
int childnum = node->n_children;
node->n_children++;
XREALLOC_N(node->n_children, node->bp);
init_childinfo(node, childnum, child);
XREALLOC_N(node->n_children-1, node->childkeys);
if (pivotkey) {
invariant(childnum > 0);
init_childkey(node, childnum-1, pivotkey);
}
node->dirty = 1;
}
void
toku_ft_bn_apply_msg_once (
BASEMENTNODE bn,
const FT_MSG msg,
uint32_t idx,
uint32_t le_keylen,
LEAFENTRY le,
txn_gc_info *gc_info,
uint64_t *workdone,
STAT64INFO stats_to_update
)
// Effect: Apply msg to leafentry (msn is ignored)
// Calculate work done by message on leafentry and add it to caller's workdone counter.
// idx is the location where it goes
// le is old leafentry
{
size_t newsize=0, oldsize=0, workdone_this_le=0;
LEAFENTRY new_le=0;
int64_t numbytes_delta = 0; // how many bytes of user data (not including overhead) were added or deleted from this row
int64_t numrows_delta = 0; // will be +1 or -1 or 0 (if row was added or deleted or not)
uint32_t key_storage_size = ft_msg_get_keylen(msg) + sizeof(uint32_t);
if (le) {
oldsize = leafentry_memsize(le) + key_storage_size;
}
// toku_le_apply_msg() may call bn_data::mempool_malloc_and_update_dmt() to allocate more space.
// That means le is guaranteed to not cause a sigsegv but it may point to a mempool that is
// no longer in use. We'll have to release the old mempool later.
toku_le_apply_msg(
msg,
le,
&bn->data_buffer,
idx,
le_keylen,
gc_info,
&new_le,
&numbytes_delta
);
// at this point, we cannot trust cmd->u.id.key to be valid.
// The dmt may have realloced its mempool and freed the one containing key.
newsize = new_le ? (leafentry_memsize(new_le) + + key_storage_size) : 0;
if (le && new_le) {
workdone_this_le = (oldsize > newsize ? oldsize : newsize); // work done is max of le size before and after message application
} else { // we did not just replace a row, so ...
if (le) {
// ... we just deleted a row ...
workdone_this_le = oldsize;
numrows_delta = -1;
}
if (new_le) {
// ... or we just added a row
workdone_this_le = newsize;
numrows_delta = 1;
}
}
if (workdone) { // test programs may call with NULL
*workdone += workdone_this_le;
}
// now update stat64 statistics
bn->stat64_delta.numrows += numrows_delta;
bn->stat64_delta.numbytes += numbytes_delta;
// the only reason stats_to_update may be null is for tests
if (stats_to_update) {
stats_to_update->numrows += numrows_delta;
stats_to_update->numbytes += numbytes_delta;
}
}
static const uint32_t setval_tag = 0xee0ccb99; // this was gotten by doing "cat /dev/random|head -c4|od -x" to get a random number. We want to make sure that the user actually passes us the setval_extra_s that we passed in.
struct setval_extra_s {
uint32_t tag;
bool did_set_val;
int setval_r; // any error code that setval_fun wants to return goes here.
// need arguments for toku_ft_bn_apply_msg_once
BASEMENTNODE bn;
MSN msn; // captured from original message, not currently used
XIDS xids;
const DBT *key;
uint32_t idx;
uint32_t le_keylen;
LEAFENTRY le;
txn_gc_info *gc_info;
uint64_t * workdone; // set by toku_ft_bn_apply_msg_once()
STAT64INFO stats_to_update;
};
/*
* If new_val == NULL, we send a delete message instead of an insert.
* This happens here instead of in do_delete() for consistency.
* setval_fun() is called from handlerton, passing in svextra_v
* from setval_extra_s input arg to ft->update_fun().
*/
static void setval_fun (const DBT *new_val, void *svextra_v) {
struct setval_extra_s *CAST_FROM_VOIDP(svextra, svextra_v);
paranoid_invariant(svextra->tag==setval_tag);
paranoid_invariant(!svextra->did_set_val);
svextra->did_set_val = true;
{
// can't leave scope until toku_ft_bn_apply_msg_once if
// this is a delete
DBT val;
FT_MSG_S msg = { FT_NONE, svextra->msn, svextra->xids,
.u = { .id = {svextra->key, NULL} } };
if (new_val) {
msg.type = FT_INSERT;
msg.u.id.val = new_val;
} else {
msg.type = FT_DELETE_ANY;
toku_init_dbt(&val);
msg.u.id.val = &val;
}
toku_ft_bn_apply_msg_once(svextra->bn, &msg,
svextra->idx, svextra->le_keylen, svextra->le,
svextra->gc_info,
svextra->workdone, svextra->stats_to_update);
svextra->setval_r = 0;
}
}
// We are already past the msn filter (in toku_ft_bn_apply_msg(), which calls do_update()),
// so capturing the msn in the setval_extra_s is not strictly required. The alternative
// would be to put a dummy msn in the messages created by setval_fun(), but preserving
// the original msn seems cleaner and it preserves accountability at a lower layer.
static int do_update(ft_update_func update_fun, DESCRIPTOR desc, BASEMENTNODE bn, FT_MSG msg, uint32_t idx,
LEAFENTRY le,
void* keydata,
uint32_t keylen,
txn_gc_info *gc_info,
uint64_t * workdone,
STAT64INFO stats_to_update) {
LEAFENTRY le_for_update;
DBT key;
const DBT *keyp;
const DBT *update_function_extra;
DBT vdbt;
const DBT *vdbtp;
// the location of data depends whether this is a regular or
// broadcast update
if (msg->type == FT_UPDATE) {
// key is passed in with command (should be same as from le)
// update function extra is passed in with command
keyp = msg->u.id.key;
update_function_extra = msg->u.id.val;
} else {
invariant(msg->type == FT_UPDATE_BROADCAST_ALL);
// key is not passed in with broadcast, it comes from le
// update function extra is passed in with command
paranoid_invariant(le); // for broadcast updates, we just hit all leafentries
// so this cannot be null
paranoid_invariant(keydata);
paranoid_invariant(keylen);
paranoid_invariant(msg->u.id.key->size == 0);
keyp = toku_fill_dbt(&key, keydata, keylen);
update_function_extra = msg->u.id.val;
}
toku_ft_status_note_update(msg->type == FT_UPDATE_BROADCAST_ALL);
if (le && !le_latest_is_del(le)) {
// if the latest val exists, use it, and we'll use the leafentry later
uint32_t vallen;
void *valp = le_latest_val_and_len(le, &vallen);
vdbtp = toku_fill_dbt(&vdbt, valp, vallen);
} else {
// otherwise, the val and leafentry are both going to be null
vdbtp = NULL;
}
le_for_update = le;
struct setval_extra_s setval_extra = {setval_tag, false, 0, bn, msg->msn, msg->xids,
keyp, idx, keylen, le_for_update, gc_info,
workdone, stats_to_update};
// call handlerton's ft->update_fun(), which passes setval_extra to setval_fun()
FAKE_DB(db, desc);
int r = update_fun(
&db,
keyp,
vdbtp,
update_function_extra,
setval_fun, &setval_extra
);
if (r == 0) { r = setval_extra.setval_r; }
return r;
}
// Should be renamed as something like "apply_msg_to_basement()."
void
toku_ft_bn_apply_msg (
ft_compare_func compare_fun,
ft_update_func update_fun,
DESCRIPTOR desc,
BASEMENTNODE bn,
FT_MSG msg,
txn_gc_info *gc_info,
uint64_t *workdone,
STAT64INFO stats_to_update
)
// Effect:
// Put a msg into a leaf.
// Calculate work done by message on leafnode and add it to caller's workdone counter.
// The leaf could end up "too big" or "too small". The caller must fix that up.
{
LEAFENTRY storeddata;
void* key = NULL;
uint32_t keylen = 0;
uint32_t num_klpairs;
int r;
struct toku_msg_leafval_heaviside_extra be = {compare_fun, desc, msg->u.id.key};
unsigned int doing_seqinsert = bn->seqinsert;
bn->seqinsert = 0;
switch (msg->type) {
case FT_INSERT_NO_OVERWRITE:
case FT_INSERT: {
uint32_t idx;
if (doing_seqinsert) {
idx = bn->data_buffer.num_klpairs();
DBT kdbt;
r = bn->data_buffer.fetch_key_and_len(idx-1, &kdbt.size, &kdbt.data);
if (r != 0) goto fz;
int cmp = toku_msg_leafval_heaviside(kdbt, be);
if (cmp >= 0) goto fz;
r = DB_NOTFOUND;
} else {
fz:
r = bn->data_buffer.find_zero<decltype(be), toku_msg_leafval_heaviside>(
be,
&storeddata,
&key,
&keylen,
&idx
);
}
if (r==DB_NOTFOUND) {
storeddata = 0;
} else {
assert_zero(r);
}
toku_ft_bn_apply_msg_once(bn, msg, idx, keylen, storeddata, gc_info, workdone, stats_to_update);
// if the insertion point is within a window of the right edge of
// the leaf then it is sequential
// window = min(32, number of leaf entries/16)
{
uint32_t s = bn->data_buffer.num_klpairs();
uint32_t w = s / 16;
if (w == 0) w = 1;
if (w > 32) w = 32;
// within the window?
if (s - idx <= w)
bn->seqinsert = doing_seqinsert + 1;
}
break;
}
case FT_DELETE_ANY:
case FT_ABORT_ANY:
case FT_COMMIT_ANY: {
uint32_t idx;
// Apply to all the matches
r = bn->data_buffer.find_zero<decltype(be), toku_msg_leafval_heaviside>(
be,
&storeddata,
&key,
&keylen,
&idx
);
if (r == DB_NOTFOUND) break;
assert_zero(r);
toku_ft_bn_apply_msg_once(bn, msg, idx, keylen, storeddata, gc_info, workdone, stats_to_update);
break;
}
case FT_OPTIMIZE_FOR_UPGRADE:
// fall through so that optimize_for_upgrade performs rest of the optimize logic
case FT_COMMIT_BROADCAST_ALL:
case FT_OPTIMIZE:
// Apply to all leafentries
num_klpairs = bn->data_buffer.num_klpairs();
for (uint32_t idx = 0; idx < num_klpairs; ) {
void* curr_keyp = NULL;
uint32_t curr_keylen = 0;
r = bn->data_buffer.fetch_klpair(idx, &storeddata, &curr_keylen, &curr_keyp);
assert_zero(r);
int deleted = 0;
if (!le_is_clean(storeddata)) { //If already clean, nothing to do.
// message application code needs a key in order to determine how much
// work was done by this message. since this is a broadcast message,
// we have to create a new message whose key is the current le's key.
DBT curr_keydbt;
FT_MSG_S curr_msg = *msg;
curr_msg.u.id.key = toku_fill_dbt(&curr_keydbt, curr_keyp, curr_keylen);
toku_ft_bn_apply_msg_once(bn, &curr_msg, idx, curr_keylen, storeddata, gc_info, workdone, stats_to_update);
// at this point, we cannot trust msg->u.id.key to be valid.
uint32_t new_dmt_size = bn->data_buffer.num_klpairs();
if (new_dmt_size != num_klpairs) {
paranoid_invariant(new_dmt_size + 1 == num_klpairs);
//Item was deleted.
deleted = 1;
}
}
if (deleted)
num_klpairs--;
else
idx++;
}
paranoid_invariant(bn->data_buffer.num_klpairs() == num_klpairs);
break;
case FT_COMMIT_BROADCAST_TXN:
case FT_ABORT_BROADCAST_TXN:
// Apply to all leafentries if txn is represented
num_klpairs = bn->data_buffer.num_klpairs();
for (uint32_t idx = 0; idx < num_klpairs; ) {
void* curr_keyp = NULL;
uint32_t curr_keylen = 0;
r = bn->data_buffer.fetch_klpair(idx, &storeddata, &curr_keylen, &curr_keyp);
assert_zero(r);
int deleted = 0;
if (le_has_xids(storeddata, msg->xids)) {
// message application code needs a key in order to determine how much
// work was done by this message. since this is a broadcast message,
// we have to create a new message whose key is the current le's key.
DBT curr_keydbt;
FT_MSG_S curr_msg = *msg;
curr_msg.u.id.key = toku_fill_dbt(&curr_keydbt, curr_keyp, curr_keylen);
toku_ft_bn_apply_msg_once(bn, &curr_msg, idx, curr_keylen, storeddata, gc_info, workdone, stats_to_update);
uint32_t new_dmt_size = bn->data_buffer.num_klpairs();
if (new_dmt_size != num_klpairs) {
paranoid_invariant(new_dmt_size + 1 == num_klpairs);
//Item was deleted.
deleted = 1;
}
}
if (deleted)
num_klpairs--;
else
idx++;
}
paranoid_invariant(bn->data_buffer.num_klpairs() == num_klpairs);
break;
case FT_UPDATE: {
uint32_t idx;
r = bn->data_buffer.find_zero<decltype(be), toku_msg_leafval_heaviside>(
be,
&storeddata,
&key,
&keylen,
&idx
);
if (r==DB_NOTFOUND) {
{
//Point to msg's copy of the key so we don't worry about le being freed
//TODO: 46 MAYBE Get rid of this when le_apply message memory is better handled
key = msg->u.id.key->data;
keylen = msg->u.id.key->size;
}
r = do_update(update_fun, desc, bn, msg, idx, NULL, NULL, 0, gc_info, workdone, stats_to_update);
} else if (r==0) {
r = do_update(update_fun, desc, bn, msg, idx, storeddata, key, keylen, gc_info, workdone, stats_to_update);
} // otherwise, a worse error, just return it
break;
}
case FT_UPDATE_BROADCAST_ALL: {
// apply to all leafentries.
uint32_t idx = 0;
uint32_t num_leafentries_before;
while (idx < (num_leafentries_before = bn->data_buffer.num_klpairs())) {
void* curr_key = nullptr;
uint32_t curr_keylen = 0;
r = bn->data_buffer.fetch_klpair(idx, &storeddata, &curr_keylen, &curr_key);
assert_zero(r);
//TODO: 46 replace this with something better than cloning key
// TODO: (Zardosht) This may be unnecessary now, due to how the key
// is handled in the bndata. Investigate and determine
char clone_mem[curr_keylen]; // only lasts one loop, alloca would overflow (end of function)
memcpy((void*)clone_mem, curr_key, curr_keylen);
curr_key = (void*)clone_mem;
// This is broken below. Have a compilation error checked
// in as a reminder
r = do_update(update_fun, desc, bn, msg, idx, storeddata, curr_key, curr_keylen, gc_info, workdone, stats_to_update);
assert_zero(r);
if (num_leafentries_before == bn->data_buffer.num_klpairs()) {
// we didn't delete something, so increment the index.
idx++;
}
}
break;
}
case FT_NONE: break; // don't do anything
}
return;
}
static inline int
key_msn_cmp(const DBT *a, const DBT *b, const MSN amsn, const MSN bmsn,
DESCRIPTOR descriptor, ft_compare_func key_cmp)
{
FAKE_DB(db, descriptor);
int r = key_cmp(&db, a, b);
if (r == 0) {
if (amsn.msn > bmsn.msn) {
r = +1;
} else if (amsn.msn < bmsn.msn) {
r = -1;
} else {
r = 0;
}
}
return r;
}
int toku_msg_buffer_key_msn_heaviside(const int32_t &offset, const struct toku_msg_buffer_key_msn_heaviside_extra &extra) {
MSN query_msn;
DBT query_key;
extra.msg_buffer->get_message_key_msn(offset, &query_key, &query_msn);
return key_msn_cmp(&query_key, extra.key, query_msn, extra.msn,
extra.desc, extra.cmp);
}
int
toku_msg_buffer_key_msn_cmp(const struct toku_msg_buffer_key_msn_cmp_extra &extra, const int32_t &ao, const int32_t &bo)
{
MSN amsn, bmsn;
DBT akey, bkey;
extra.msg_buffer->get_message_key_msn(ao, &akey, &amsn);
extra.msg_buffer->get_message_key_msn(bo, &bkey, &bmsn);
return key_msn_cmp(&akey, &bkey, amsn, bmsn,
extra.desc, extra.cmp);
}
// Effect: Enqueue the message represented by the parameters into the
// bnc's buffer, and put it in either the fresh or stale message tree,
// or the broadcast list.
static void bnc_insert_msg(NONLEAF_CHILDINFO bnc, FT_MSG msg, bool is_fresh, DESCRIPTOR desc, ft_compare_func cmp) {
int r = 0;
int32_t offset;
bnc->msg_buffer.enqueue(msg, is_fresh, &offset);
enum ft_msg_type type = ft_msg_get_type(msg);
if (ft_msg_type_applies_once(type)) {
DBT key;
toku_fill_dbt(&key, ft_msg_get_key(msg), ft_msg_get_keylen(msg));
struct toku_msg_buffer_key_msn_heaviside_extra extra = { .desc = desc, .cmp = cmp, .msg_buffer = &bnc->msg_buffer, .key = &key, .msn = msg->msn };
if (is_fresh) {
r = bnc->fresh_message_tree.insert<struct toku_msg_buffer_key_msn_heaviside_extra, toku_msg_buffer_key_msn_heaviside>(offset, extra, nullptr);
assert_zero(r);
} else {
r = bnc->stale_message_tree.insert<struct toku_msg_buffer_key_msn_heaviside_extra, toku_msg_buffer_key_msn_heaviside>(offset, extra, nullptr);
assert_zero(r);
}
} else {
invariant(ft_msg_type_applies_all(type) || ft_msg_type_does_nothing(type));
const uint32_t idx = bnc->broadcast_list.size();
r = bnc->broadcast_list.insert_at(offset, idx);
assert_zero(r);
}
}
// This is only exported for tests.
void toku_bnc_insert_msg(NONLEAF_CHILDINFO bnc, const void *key, ITEMLEN keylen, const void *data, ITEMLEN datalen, enum ft_msg_type type, MSN msn, XIDS xids, bool is_fresh, DESCRIPTOR desc, ft_compare_func cmp)
{
DBT k, v;
FT_MSG_S msg = {
type, msn, xids, .u = { .id = { toku_fill_dbt(&k, key, keylen), toku_fill_dbt(&v, data, datalen) } }
};
bnc_insert_msg(bnc, &msg, is_fresh, desc, cmp);
}
// append a msg to a nonleaf node's child buffer
static void ft_append_msg_to_child_buffer(ft_compare_func compare_fun, DESCRIPTOR desc, FTNODE node,
int childnum, FT_MSG msg, bool is_fresh) {
paranoid_invariant(BP_STATE(node,childnum) == PT_AVAIL);
bnc_insert_msg(BNC(node, childnum), msg, is_fresh, desc, compare_fun);
node->dirty = 1;
}
// This is only exported for tests.
void toku_ft_append_to_child_buffer(ft_compare_func compare_fun, DESCRIPTOR desc, FTNODE node, int childnum, enum ft_msg_type type, MSN msn, XIDS xids, bool is_fresh, const DBT *key, const DBT *val) {
FT_MSG_S msg = {
type, msn, xids, .u = { .id = { key, val } }
};
ft_append_msg_to_child_buffer(compare_fun, desc, node, childnum, &msg, is_fresh);
}
static void ft_nonleaf_msg_once_to_child(ft_compare_func compare_fun, DESCRIPTOR desc, FTNODE node, int target_childnum, FT_MSG msg, bool is_fresh, size_t flow_deltas[])
// Previously we had passive aggressive promotion, but that causes a lot of I/O a the checkpoint. So now we are just putting it in the buffer here.
// Also we don't worry about the node getting overfull here. It's the caller's problem.
{
unsigned int childnum = (target_childnum >= 0
? target_childnum
: toku_ftnode_which_child(node, msg->u.id.key, desc, compare_fun));
ft_append_msg_to_child_buffer(compare_fun, desc, node, childnum, msg, is_fresh);
NONLEAF_CHILDINFO bnc = BNC(node, childnum);
bnc->flow[0] += flow_deltas[0];
bnc->flow[1] += flow_deltas[1];
}
static int ft_compare_pivot(DESCRIPTOR desc, ft_compare_func cmp, const DBT *key, const DBT *pivot) {
FAKE_DB(db, desc);
int r = cmp(&db, key, pivot);
return r;
}
/* Find the leftmost child that may contain the key.
* If the key exists it will be in the child whose number
* is the return value of this function.
*/
int toku_ftnode_which_child(FTNODE node, const DBT *k,
DESCRIPTOR desc, ft_compare_func cmp) {
// a funny case of no pivots
if (node->n_children <= 1) return 0;
// check the last key to optimize seq insertions
int n = node->n_children-1;
int c = ft_compare_pivot(desc, cmp, k, &node->childkeys[n-1]);
if (c > 0) return n;
// binary search the pivots
int lo = 0;
int hi = n-1; // skip the last one, we checked it above
int mi;
while (lo < hi) {
mi = (lo + hi) / 2;
c = ft_compare_pivot(desc, cmp, k, &node->childkeys[mi]);
if (c > 0) {
lo = mi+1;
continue;
}
if (c < 0) {
hi = mi;
continue;
}
return mi;
}
return lo;
}
// Used for HOT.
int
toku_ftnode_hot_next_child(FTNODE node,
const DBT *k,
DESCRIPTOR desc,
ft_compare_func cmp) {
int low = 0;
int hi = node->n_children - 1;
int mi;
while (low < hi) {
mi = (low + hi) / 2;
int r = ft_compare_pivot(desc, cmp, k, &node->childkeys[mi]);
if (r > 0) {
low = mi + 1;
} else if (r < 0) {
hi = mi;
} else {
// if they were exactly equal, then we want the sub-tree under
// the next pivot.
return mi + 1;
}
}
invariant(low == hi);
return low;
}
static void
ft_nonleaf_msg_all(ft_compare_func compare_fun, DESCRIPTOR desc, FTNODE node, FT_MSG msg, bool is_fresh, size_t flow_deltas[])
// Effect: Put the message into a nonleaf node. We put it into all children, possibly causing the children to become reactive.
// We don't do the splitting and merging. That's up to the caller after doing all the puts it wants to do.
// The re_array[i] gets set to the reactivity of any modified child i. (And there may be several such children.)
{
for (int i = 0; i < node->n_children; i++) {
ft_nonleaf_msg_once_to_child(compare_fun, desc, node, i, msg, is_fresh, flow_deltas);
}
}
static void
ft_nonleaf_put_msg(ft_compare_func compare_fun, DESCRIPTOR desc, FTNODE node, int target_childnum, FT_MSG msg, bool is_fresh, size_t flow_deltas[])
// Effect: Put the message into a nonleaf node. We may put it into a child, possibly causing the child to become reactive.
// We don't do the splitting and merging. That's up to the caller after doing all the puts it wants to do.
// The re_array[i] gets set to the reactivity of any modified child i. (And there may be several such children.)
//
{
//
// see comments in toku_ft_leaf_apply_msg
// to understand why we handle setting
// node->max_msn_applied_to_node_on_disk here,
// and don't do it in toku_ftnode_put_msg
//
MSN msg_msn = msg->msn;
invariant(msg_msn.msn > node->max_msn_applied_to_node_on_disk.msn);
node->max_msn_applied_to_node_on_disk = msg_msn;
if (ft_msg_type_applies_once(msg->type)) {
ft_nonleaf_msg_once_to_child(compare_fun, desc, node, target_childnum, msg, is_fresh, flow_deltas);
} else if (ft_msg_type_applies_all(msg->type)) {
ft_nonleaf_msg_all(compare_fun, desc, node, msg, is_fresh, flow_deltas);
} else {
paranoid_invariant(ft_msg_type_does_nothing(msg->type));
}
}
// Garbage collect one leaf entry.
static void
ft_basement_node_gc_once(BASEMENTNODE bn,
uint32_t index,
void* keyp,
uint32_t keylen,
LEAFENTRY leaf_entry,
txn_gc_info *gc_info,
STAT64INFO_S * delta)
{
paranoid_invariant(leaf_entry);
// Don't run garbage collection on non-mvcc leaf entries.
if (leaf_entry->type != LE_MVCC) {
goto exit;
}
// Don't run garbage collection if this leafentry decides it's not worth it.
if (!toku_le_worth_running_garbage_collection(leaf_entry, gc_info)) {
goto exit;
}
LEAFENTRY new_leaf_entry;
new_leaf_entry = NULL;
// The mempool doesn't free itself. When it allocates new memory,
// this pointer will be set to the older memory that must now be
// freed.
void * maybe_free;
maybe_free = NULL;
// These will represent the number of bytes and rows changed as
// part of the garbage collection.
int64_t numbytes_delta;
int64_t numrows_delta;
toku_le_garbage_collect(leaf_entry,
&bn->data_buffer,
index,
keyp,
keylen,
gc_info,
&new_leaf_entry,
&numbytes_delta);
numrows_delta = 0;
if (new_leaf_entry) {
numrows_delta = 0;
} else {
numrows_delta = -1;
}
// If we created a new mempool buffer we must free the
// old/original buffer.
if (maybe_free) {
toku_free(maybe_free);
}
// Update stats.
bn->stat64_delta.numrows += numrows_delta;
bn->stat64_delta.numbytes += numbytes_delta;
delta->numrows += numrows_delta;
delta->numbytes += numbytes_delta;
exit:
return;
}
// Garbage collect all leaf entries for a given basement node.
static void
basement_node_gc_all_les(BASEMENTNODE bn,
txn_gc_info *gc_info,
STAT64INFO_S * delta)
{
int r = 0;
uint32_t index = 0;
uint32_t num_leafentries_before;
while (index < (num_leafentries_before = bn->data_buffer.num_klpairs())) {
void* keyp = NULL;
uint32_t keylen = 0;
LEAFENTRY leaf_entry;
r = bn->data_buffer.fetch_klpair(index, &leaf_entry, &keylen, &keyp);
assert_zero(r);
ft_basement_node_gc_once(
bn,
index,
keyp,
keylen,
leaf_entry,
gc_info,
delta
);
// Check if the leaf entry was deleted or not.
if (num_leafentries_before == bn->data_buffer.num_klpairs()) {
++index;
}
}
}
// Garbage collect all leaf entires in all basement nodes.
static void
ft_leaf_gc_all_les(FT ft, FTNODE node, txn_gc_info *gc_info)
{
toku_ftnode_assert_fully_in_memory(node);
paranoid_invariant_zero(node->height);
// Loop through each leaf entry, garbage collecting as we go.
for (int i = 0; i < node->n_children; ++i) {
// Perform the garbage collection.
BASEMENTNODE bn = BLB(node, i);
STAT64INFO_S delta;
delta.numrows = 0;
delta.numbytes = 0;
basement_node_gc_all_les(bn, gc_info, &delta);
toku_ft_update_stats(&ft->in_memory_stats, delta);
}
}
void toku_ftnode_leaf_run_gc(FT ft, FTNODE node) {
TOKULOGGER logger = toku_cachefile_logger(ft->cf);
if (logger) {
TXN_MANAGER txn_manager = toku_logger_get_txn_manager(logger);
txn_manager_state txn_state_for_gc(txn_manager);
txn_state_for_gc.init();
TXNID oldest_referenced_xid_for_simple_gc = toku_txn_manager_get_oldest_referenced_xid_estimate(txn_manager);
// Perform full garbage collection.
//
// - txn_state_for_gc
// a fresh snapshot of the transaction system.
// - oldest_referenced_xid_for_simple_gc
// the oldest xid in any live list as of right now - suitible for simple gc
// - node->oldest_referenced_xid_known
// the last known oldest referenced xid for this node and any unapplied messages.
// it is a lower bound on the actual oldest referenced xid - but becasue there
// may be abort messages above us, we need to be careful to only use this value
// for implicit promotion (as opposed to the oldest referenced xid for simple gc)
//
// The node has its own oldest referenced xid because it must be careful not to implicitly promote
// provisional entries for transactions that are no longer live, but may have abort messages
// somewhere above us in the tree.
txn_gc_info gc_info(&txn_state_for_gc,
oldest_referenced_xid_for_simple_gc,
node->oldest_referenced_xid_known,
true);
ft_leaf_gc_all_les(ft, node, &gc_info);
}
}
void
toku_ftnode_put_msg (
ft_compare_func compare_fun,
ft_update_func update_fun,
DESCRIPTOR desc,
FTNODE node,
int target_childnum,
FT_MSG msg,
bool is_fresh,
txn_gc_info *gc_info,
size_t flow_deltas[],
STAT64INFO stats_to_update
)
// Effect: Push message into the subtree rooted at NODE.
// If NODE is a leaf, then
// put message into leaf, applying it to the leafentries
// If NODE is a nonleaf, then push the message into the message buffer(s) of the relevent child(ren).
// The node may become overfull. That's not our problem.
{
toku_ftnode_assert_fully_in_memory(node);
//
// see comments in toku_ft_leaf_apply_msg
// to understand why we don't handle setting
// node->max_msn_applied_to_node_on_disk here,
// and instead defer to these functions
//
if (node->height==0) {
toku_ft_leaf_apply_msg(compare_fun, update_fun, desc, node, target_childnum, msg, gc_info, nullptr, stats_to_update);
} else {
ft_nonleaf_put_msg(compare_fun, desc, node, target_childnum, msg, is_fresh, flow_deltas);
}
}
// Effect: applies the message to the leaf if the appropriate basement node is in memory.
// This function is called during message injection and/or flushing, so the entire
// node MUST be in memory.
void toku_ft_leaf_apply_msg(
ft_compare_func compare_fun,
ft_update_func update_fun,
DESCRIPTOR desc,
FTNODE node,
int target_childnum, // which child to inject to, or -1 if unknown
FT_MSG msg,
txn_gc_info *gc_info,
uint64_t *workdone,
STAT64INFO stats_to_update
)
{
VERIFY_NODE(t, node);
toku_ftnode_assert_fully_in_memory(node);
//
// Because toku_ft_leaf_apply_msg is called with the intent of permanently
// applying a message to a leaf node (meaning the message is permanently applied
// and will be purged from the system after this call, as opposed to
// toku_apply_ancestors_messages_to_node, which applies a message
// for a query, but the message may still reside in the system and
// be reapplied later), we mark the node as dirty and
// take the opportunity to update node->max_msn_applied_to_node_on_disk.
//
node->dirty = 1;
//
// we cannot blindly update node->max_msn_applied_to_node_on_disk,
// we must check to see if the msn is greater that the one already stored,
// because the message may have already been applied earlier (via
// toku_apply_ancestors_messages_to_node) to answer a query
//
// This is why we handle node->max_msn_applied_to_node_on_disk both here
// and in ft_nonleaf_put_msg, as opposed to in one location, toku_ftnode_put_msg.
//
MSN msg_msn = msg->msn;
if (msg_msn.msn > node->max_msn_applied_to_node_on_disk.msn) {
node->max_msn_applied_to_node_on_disk = msg_msn;
}
if (ft_msg_type_applies_once(msg->type)) {
unsigned int childnum = (target_childnum >= 0
? target_childnum
: toku_ftnode_which_child(node, msg->u.id.key, desc, compare_fun));
BASEMENTNODE bn = BLB(node, childnum);
if (msg->msn.msn > bn->max_msn_applied.msn) {
bn->max_msn_applied = msg->msn;
toku_ft_bn_apply_msg(compare_fun,
update_fun,
desc,
bn,
msg,
gc_info,
workdone,
stats_to_update);
} else {
toku_ft_status_note_msn_discard();
}
}
else if (ft_msg_type_applies_all(msg->type)) {
for (int childnum=0; childnum<node->n_children; childnum++) {
if (msg->msn.msn > BLB(node, childnum)->max_msn_applied.msn) {
BLB(node, childnum)->max_msn_applied = msg->msn;
toku_ft_bn_apply_msg(compare_fun,
update_fun,
desc,
BLB(node, childnum),
msg,
gc_info,
workdone,
stats_to_update);
} else {
toku_ft_status_note_msn_discard();
}
}
}
else if (!ft_msg_type_does_nothing(msg->type)) {
invariant(ft_msg_type_does_nothing(msg->type));
}
VERIFY_NODE(t, node);
}
/* -*- mode: C++; c-basic-offset: 4; indent-tabs-mode: nil -*- */
// vim: ft=cpp:expandtab:ts=8:sw=4:softtabstop=4:
/*
COPYING CONDITIONS NOTICE:
This program is free software; you can redistribute it and/or modify
it under the terms of version 2 of the GNU General Public License as
published by the Free Software Foundation, and provided that the
following conditions are met:
* Redistributions of source code must retain this COPYING
CONDITIONS NOTICE, the COPYRIGHT NOTICE (below), the
DISCLAIMER (below), the UNIVERSITY PATENT NOTICE (below), the
PATENT MARKING NOTICE (below), and the PATENT RIGHTS
GRANT (below).
* Redistributions in binary form must reproduce this COPYING
CONDITIONS NOTICE, the COPYRIGHT NOTICE (below), the
DISCLAIMER (below), the UNIVERSITY PATENT NOTICE (below), the
PATENT MARKING NOTICE (below), and the PATENT RIGHTS
GRANT (below) in the documentation and/or other materials
provided with the distribution.
You should have received a copy of the GNU General Public License
along with this program; if not, write to the Free Software
Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA
02110-1301, USA.
COPYRIGHT NOTICE:
TokuDB, Tokutek Fractal Tree Indexing Library.
Copyright (C) 2007-2013 Tokutek, Inc.
DISCLAIMER:
This program is distributed in the hope that it will be useful, but
WITHOUT ANY WARRANTY; without even the implied warranty of
MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
General Public License for more details.
UNIVERSITY PATENT NOTICE:
The technology is licensed by the Massachusetts Institute of
Technology, Rutgers State University of New Jersey, and the Research
Foundation of State University of New York at Stony Brook under
United States of America Serial No. 11/760379 and to the patents
and/or patent applications resulting from it.
PATENT MARKING NOTICE:
This software is covered by US Patent No. 8,185,551.
This software is covered by US Patent No. 8,489,638.
PATENT RIGHTS GRANT:
"THIS IMPLEMENTATION" means the copyrightable works distributed by
Tokutek as part of the Fractal Tree project.
"PATENT CLAIMS" means the claims of patents that are owned or
licensable by Tokutek, both currently or in the future; and that in
the absence of this license would be infringed by THIS
IMPLEMENTATION or by using or running THIS IMPLEMENTATION.
"PATENT CHALLENGE" shall mean a challenge to the validity,
patentability, enforceability and/or non-infringement of any of the
PATENT CLAIMS or otherwise opposing any of the PATENT CLAIMS.
Tokutek hereby grants to you, for the term and geographical scope of
the PATENT CLAIMS, a non-exclusive, no-charge, royalty-free,
irrevocable (except as stated in this section) patent license to
make, have made, use, offer to sell, sell, import, transfer, and
otherwise run, modify, and propagate the contents of THIS
IMPLEMENTATION, where such license applies only to the PATENT
CLAIMS. This grant does not include claims that would be infringed
only as a consequence of further modifications of THIS
IMPLEMENTATION. If you or your agent or licensee institute or order
or agree to the institution of patent litigation against any entity
(including a cross-claim or counterclaim in a lawsuit) alleging that
THIS IMPLEMENTATION constitutes direct or contributory patent
infringement, or inducement of patent infringement, then any rights
granted to you under this License shall terminate as of the date
such litigation is filed. If you or your agent or exclusive
licensee institute or order or agree to the institution of a PATENT
CHALLENGE, then Tokutek may terminate any rights granted to you
under this License.
*/
#pragma once
#include "ft/cachetable.h"
#include "ft/bndata.h"
#include "ft/fttypes.h"
#include "ft/msg_buffer.h"
struct ftnode {
MSN max_msn_applied_to_node_on_disk; // max_msn_applied that will be written to disk
unsigned int flags;
BLOCKNUM thisnodename; // Which block number is this node?
int layout_version; // What version of the data structure?
int layout_version_original; // different (<) from layout_version if upgraded from a previous version (useful for debugging)
int layout_version_read_from_disk; // transient, not serialized to disk, (useful for debugging)
uint32_t build_id; // build_id (svn rev number) of software that wrote this node to disk
int height; /* height is always >= 0. 0 for leaf, >0 for nonleaf. */
int dirty;
uint32_t fullhash;
int n_children; //for internal nodes, if n_children==fanout+1 then the tree needs to be rebalanced.
// for leaf nodes, represents number of basement nodes
unsigned int totalchildkeylens;
DBT *childkeys; /* Pivot keys. Child 0's keys are <= childkeys[0]. Child 1's keys are <= childkeys[1].
Child 1's keys are > childkeys[0]. */
// What's the oldest referenced xid that this node knows about? The real oldest
// referenced xid might be younger, but this is our best estimate. We use it
// as a heuristic to transition provisional mvcc entries from provisional to
// committed (from implicity committed to really committed).
//
// A better heuristic would be the oldest live txnid, but we use this since it
// still works well most of the time, and its readily available on the inject
// code path.
TXNID oldest_referenced_xid_known;
// array of size n_children, consisting of ftnode partitions
// each one is associated with a child
// for internal nodes, the ith partition corresponds to the ith message buffer
// for leaf nodes, the ith partition corresponds to the ith basement node
struct ftnode_partition *bp;
struct ctpair *ct_pair;
};
// data of an available partition of a leaf ftnode
struct ftnode_leaf_basement_node {
bn_data data_buffer;
unsigned int seqinsert; // number of sequential inserts to this leaf
MSN max_msn_applied; // max message sequence number applied
bool stale_ancestor_messages_applied;
STAT64INFO_S stat64_delta; // change in stat64 counters since basement was last written to disk
};
enum pt_state { // declare this to be packed so that when used below it will only take 1 byte.
PT_INVALID = 0,
PT_ON_DISK = 1,
PT_COMPRESSED = 2,
PT_AVAIL = 3};
enum ftnode_child_tag {
BCT_INVALID = 0,
BCT_NULL,
BCT_SUBBLOCK,
BCT_LEAF,
BCT_NONLEAF
};
typedef toku::omt<int32_t> off_omt_t;
typedef toku::omt<int32_t, int32_t, true> marked_off_omt_t;
// data of an available partition of a nonleaf ftnode
struct ftnode_nonleaf_childinfo {
message_buffer msg_buffer;
off_omt_t broadcast_list;
marked_off_omt_t fresh_message_tree;
off_omt_t stale_message_tree;
uint64_t flow[2]; // current and last checkpoint
};
typedef struct ftnode_child_pointer {
union {
struct sub_block *subblock;
struct ftnode_nonleaf_childinfo *nonleaf;
struct ftnode_leaf_basement_node *leaf;
} u;
enum ftnode_child_tag tag;
} FTNODE_CHILD_POINTER;
struct ftnode_disk_data {
//
// stores the offset to the beginning of the partition on disk from the ftnode, and the length, needed to read a partition off of disk
// the value is only meaningful if the node is clean. If the node is dirty, then the value is meaningless
// The START is the distance from the end of the compressed node_info data, to the beginning of the compressed partition
// The SIZE is the size of the compressed partition.
// Rationale: We cannot store the size from the beginning of the node since we don't know how big the header will be.
// However, later when we are doing aligned writes, we won't be able to store the size from the end since we want things to align.
uint32_t start;
uint32_t size;
};
#define BP_START(node_dd,i) ((node_dd)[i].start)
#define BP_SIZE(node_dd,i) ((node_dd)[i].size)
// a ftnode partition, associated with a child of a node
struct ftnode_partition {
// the following three variables are used for nonleaf nodes
// for leaf nodes, they are meaningless
BLOCKNUM blocknum; // blocknum of child
// How many bytes worth of work was performed by messages in each buffer.
uint64_t workdone;
//
// pointer to the partition. Depending on the state, they may be different things
// if state == PT_INVALID, then the node was just initialized and ptr == NULL
// if state == PT_ON_DISK, then ptr == NULL
// if state == PT_COMPRESSED, then ptr points to a struct sub_block*
// if state == PT_AVAIL, then ptr is:
// a struct ftnode_nonleaf_childinfo for internal nodes,
// a struct ftnode_leaf_basement_node for leaf nodes
//
struct ftnode_child_pointer ptr;
//
// at any time, the partitions may be in one of the following three states (stored in pt_state):
// PT_INVALID - means that the partition was just initialized
// PT_ON_DISK - means that the partition is not in memory and needs to be read from disk. To use, must read off disk and decompress
// PT_COMPRESSED - means that the partition is compressed in memory. To use, must decompress
// PT_AVAIL - means the partition is decompressed and in memory
//
enum pt_state state; // make this an enum to make debugging easier.
// clock count used to for pe_callback to determine if a node should be evicted or not
// for now, saturating the count at 1
uint8_t clock_count;
};
//
// TODO: Fix all these names
// Organize declarations
// Fix widespread parameter ordering inconsistencies
//
BASEMENTNODE toku_create_empty_bn(void);
BASEMENTNODE toku_create_empty_bn_no_buffer(void); // create a basement node with a null buffer.
NONLEAF_CHILDINFO toku_clone_nl(NONLEAF_CHILDINFO orig_childinfo);
BASEMENTNODE toku_clone_bn(BASEMENTNODE orig_bn);
NONLEAF_CHILDINFO toku_create_empty_nl(void);
void destroy_basement_node (BASEMENTNODE bn);
void destroy_nonleaf_childinfo (NONLEAF_CHILDINFO nl);
void toku_destroy_ftnode_internals(FTNODE node);
void toku_ftnode_free (FTNODE *node);
bool toku_ftnode_fully_in_memory(FTNODE node);
void toku_ftnode_assert_fully_in_memory(FTNODE node);
void toku_evict_bn_from_memory(FTNODE node, int childnum, FT h);
BASEMENTNODE toku_detach_bn(FTNODE node, int childnum);
void toku_ftnode_update_disk_stats(FTNODE ftnode, FT ft, bool for_checkpoint);
void toku_ftnode_clone_partitions(FTNODE node, FTNODE cloned_node);
void toku_initialize_empty_ftnode(FTNODE node, BLOCKNUM nodename, int height, int num_children,
int layout_version, unsigned int flags);
int toku_ftnode_which_child(FTNODE node, const DBT *k,
DESCRIPTOR desc, ft_compare_func cmp);
//
// Field in ftnode_fetch_extra that tells the
// partial fetch callback what piece of the node
// is needed by the ydb
//
enum ftnode_fetch_type {
ftnode_fetch_none=1, // no partitions needed.
ftnode_fetch_subset, // some subset of partitions needed
ftnode_fetch_prefetch, // this is part of a prefetch call
ftnode_fetch_all, // every partition is needed
ftnode_fetch_keymatch, // one child is needed if it holds both keys
};
static bool is_valid_ftnode_fetch_type(enum ftnode_fetch_type type) UU();
static bool is_valid_ftnode_fetch_type(enum ftnode_fetch_type type) {
switch (type) {
case ftnode_fetch_none:
case ftnode_fetch_subset:
case ftnode_fetch_prefetch:
case ftnode_fetch_all:
case ftnode_fetch_keymatch:
return true;
default:
return false;
}
}
//
// An extra parameter passed to cachetable functions
// That is used in all types of fetch callbacks.
// The contents help the partial fetch and fetch
// callbacks retrieve the pieces of a node necessary
// for the ensuing operation (flush, query, ...)
//
struct ft_search;
struct ftnode_fetch_extra {
enum ftnode_fetch_type type;
// needed for reading a node off disk
FT h;
// used in the case where type == ftnode_fetch_subset
// parameters needed to find out which child needs to be decompressed (so it can be read)
ft_search *search;
DBT range_lock_left_key, range_lock_right_key;
bool left_is_neg_infty, right_is_pos_infty;
// states if we should try to aggressively fetch basement nodes
// that are not specifically needed for current query,
// but may be needed for other cursor operations user is doing
// For example, if we have not disabled prefetching,
// and the user is doing a dictionary wide scan, then
// even though a query may only want one basement node,
// we fetch all basement nodes in a leaf node.
bool disable_prefetching;
// this value will be set during the fetch_callback call by toku_ftnode_fetch_callback or toku_ftnode_pf_req_callback
// thi callbacks need to evaluate this anyway, so we cache it here so the search code does not reevaluate it
int child_to_read;
// when we read internal nodes, we want to read all the data off disk in one I/O
// then we'll treat it as normal and only decompress the needed partitions etc.
bool read_all_partitions;
// Accounting: How many bytes were read, and how much time did we spend doing I/O?
uint64_t bytes_read;
tokutime_t io_time;
tokutime_t decompress_time;
tokutime_t deserialize_time;
};
typedef struct ftnode_fetch_extra *FTNODE_FETCH_EXTRA;
//
// TODO: put the heaviside functions into their respective 'struct .*extra;' namespaces
//
struct toku_msg_buffer_key_msn_heaviside_extra {
DESCRIPTOR desc;
ft_compare_func cmp;
message_buffer *msg_buffer;
const DBT *key;
MSN msn;
};
int toku_msg_buffer_key_msn_heaviside(const int32_t &v, const struct toku_msg_buffer_key_msn_heaviside_extra &extra);
struct toku_msg_buffer_key_msn_cmp_extra {
DESCRIPTOR desc;
ft_compare_func cmp;
message_buffer *msg_buffer;
};
int toku_msg_buffer_key_msn_cmp(const struct toku_msg_buffer_key_msn_cmp_extra &extrap, const int &a, const int &b);
struct toku_msg_leafval_heaviside_extra {
ft_compare_func compare_fun;
DESCRIPTOR desc;
DBT const * const key;
};
int toku_msg_leafval_heaviside(DBT const &kdbt, const struct toku_msg_leafval_heaviside_extra &be);
unsigned int toku_bnc_nbytesinbuf(NONLEAF_CHILDINFO bnc);
int toku_bnc_n_entries(NONLEAF_CHILDINFO bnc);
long toku_bnc_memory_size(NONLEAF_CHILDINFO bnc);
long toku_bnc_memory_used(NONLEAF_CHILDINFO bnc);
void toku_bnc_insert_msg(NONLEAF_CHILDINFO bnc, const void *key, ITEMLEN keylen, const void *data, ITEMLEN datalen, enum ft_msg_type type, MSN msn, XIDS xids, bool is_fresh, DESCRIPTOR desc, ft_compare_func cmp);
void toku_bnc_empty(NONLEAF_CHILDINFO bnc);
void toku_bnc_flush_to_child(FT h, NONLEAF_CHILDINFO bnc, FTNODE child, TXNID parent_oldest_referenced_xid_known);
bool toku_bnc_should_promote(FT ft, NONLEAF_CHILDINFO bnc) __attribute__((const, nonnull));
bool toku_ftnode_nonleaf_is_gorged(FTNODE node, uint32_t nodesize);
uint32_t toku_ftnode_leaf_num_entries(FTNODE node);
void toku_ftnode_leaf_rebalance(FTNODE node, unsigned int basementnodesize);
void toku_ftnode_leaf_run_gc(FT ft, FTNODE node);
enum reactivity {
RE_STABLE,
RE_FUSIBLE,
RE_FISSIBLE
};
enum reactivity toku_ftnode_get_reactivity(FT ft, FTNODE node);
enum reactivity toku_ftnode_get_nonleaf_reactivity(FTNODE node, unsigned int fanout);
enum reactivity toku_ftnode_get_leaf_reactivity(FTNODE node, uint32_t nodesize);
/**
* Finds the next child for HOT to flush to, given that everything up to
* and including k has been flattened.
*
* If k falls between pivots in node, then we return the childnum where k
* lies.
*
* If k is equal to some pivot, then we return the next (to the right)
* childnum.
*/
int toku_ftnode_hot_next_child(FTNODE node, const DBT *k,
DESCRIPTOR desc, ft_compare_func cmp);
void toku_ftnode_put_msg(ft_compare_func compare_fun, ft_update_func update_fun,
DESCRIPTOR desc, FTNODE node, int target_childnum,
FT_MSG msg, bool is_fresh, txn_gc_info *gc_info,
size_t flow_deltas[], STAT64INFO stats_to_update);
void toku_ft_bn_apply_msg_once(BASEMENTNODE bn, const FT_MSG msg, uint32_t idx,
uint32_t le_keylen, LEAFENTRY le, txn_gc_info *gc_info,
uint64_t *workdonep, STAT64INFO stats_to_update);
void toku_ft_bn_apply_msg(ft_compare_func compare_fun, ft_update_func update_fun,
DESCRIPTOR desc, BASEMENTNODE bn, FT_MSG msg, txn_gc_info *gc_info,
uint64_t *workdone, STAT64INFO stats_to_update);
void toku_ft_leaf_apply_msg(ft_compare_func compare_fun, ft_update_func update_fun,
DESCRIPTOR desc, FTNODE node, int target_childnum,
FT_MSG msg, txn_gc_info *gc_info,
uint64_t *workdone, STAT64INFO stats_to_update);
CACHETABLE_WRITE_CALLBACK get_write_callbacks_for_node(FT ft);
//
// Message management for orthopush
//
struct ancestors {
// This is the root node if next is NULL (since the root has no ancestors)
FTNODE node;
// Which buffer holds messages destined to the node whose ancestors this list represents.
int childnum;
struct ancestors *next;
};
typedef struct ancestors *ANCESTORS;
void toku_ft_bnc_move_messages_to_stale(FT ft, NONLEAF_CHILDINFO bnc);
void toku_move_ftnode_messages_to_stale(FT ft, FTNODE node);
// TODO: Should ft_handle just be FT?
void toku_apply_ancestors_messages_to_node(FT_HANDLE t, FTNODE node, ANCESTORS ancestors,
struct pivot_bounds const *const bounds,
bool *msgs_applied, int child_to_read);
bool toku_ft_leaf_needs_ancestors_messages(FT ft, FTNODE node, ANCESTORS ancestors,
struct pivot_bounds const *const bounds,
MSN *const max_msn_in_path, int child_to_read);
void toku_ft_bn_update_max_msn(FTNODE node, MSN max_msn_applied, int child_to_read);
struct ft_search;
int toku_ft_search_which_child(DESCRIPTOR desc, ft_compare_func cmp, FTNODE node, ft_search *search);
//
// internal node inline functions
// TODO: Turn the macros into real functions
//
static inline void set_BNULL(FTNODE node, int i) {
paranoid_invariant(i >= 0);
paranoid_invariant(i < node->n_children);
node->bp[i].ptr.tag = BCT_NULL;
}
static inline bool is_BNULL (FTNODE node, int i) {
paranoid_invariant(i >= 0);
paranoid_invariant(i < node->n_children);
return node->bp[i].ptr.tag == BCT_NULL;
}
static inline NONLEAF_CHILDINFO BNC(FTNODE node, int i) {
paranoid_invariant(i >= 0);
paranoid_invariant(i < node->n_children);
FTNODE_CHILD_POINTER p = node->bp[i].ptr;
paranoid_invariant(p.tag==BCT_NONLEAF);
return p.u.nonleaf;
}
static inline void set_BNC(FTNODE node, int i, NONLEAF_CHILDINFO nl) {
paranoid_invariant(i >= 0);
paranoid_invariant(i < node->n_children);
FTNODE_CHILD_POINTER *p = &node->bp[i].ptr;
p->tag = BCT_NONLEAF;
p->u.nonleaf = nl;
}
static inline BASEMENTNODE BLB(FTNODE node, int i) {
paranoid_invariant(i >= 0);
// The optimizer really doesn't like it when we compare
// i to n_children as signed integers. So we assert that
// n_children is in fact positive before doing a comparison
// on the values forcibly cast to unsigned ints.
paranoid_invariant(node->n_children > 0);
paranoid_invariant((unsigned) i < (unsigned) node->n_children);
FTNODE_CHILD_POINTER p = node->bp[i].ptr;
paranoid_invariant(p.tag==BCT_LEAF);
return p.u.leaf;
}
static inline void set_BLB(FTNODE node, int i, BASEMENTNODE bn) {
paranoid_invariant(i >= 0);
paranoid_invariant(i < node->n_children);
FTNODE_CHILD_POINTER *p = &node->bp[i].ptr;
p->tag = BCT_LEAF;
p->u.leaf = bn;
}
static inline SUB_BLOCK BSB(FTNODE node, int i) {
paranoid_invariant(i >= 0);
paranoid_invariant(i < node->n_children);
FTNODE_CHILD_POINTER p = node->bp[i].ptr;
paranoid_invariant(p.tag==BCT_SUBBLOCK);
return p.u.subblock;
}
static inline void set_BSB(FTNODE node, int i, SUB_BLOCK sb) {
paranoid_invariant(i >= 0);
paranoid_invariant(i < node->n_children);
FTNODE_CHILD_POINTER *p = &node->bp[i].ptr;
p->tag = BCT_SUBBLOCK;
p->u.subblock = sb;
}
// ftnode partition macros
// BP stands for ftnode_partition
#define BP_BLOCKNUM(node,i) ((node)->bp[i].blocknum)
#define BP_STATE(node,i) ((node)->bp[i].state)
#define BP_WORKDONE(node, i)((node)->bp[i].workdone)
//
// macros for managing a node's clock
// Should be managed by ft-ops.c, NOT by serialize/deserialize
//
//
// BP_TOUCH_CLOCK uses a compare and swap because multiple threads
// that have a read lock on an internal node may try to touch the clock
// simultaneously
//
#define BP_TOUCH_CLOCK(node, i) ((node)->bp[i].clock_count = 1)
#define BP_SWEEP_CLOCK(node, i) ((node)->bp[i].clock_count = 0)
#define BP_SHOULD_EVICT(node, i) ((node)->bp[i].clock_count == 0)
// not crazy about having these two here, one is for the case where we create new
// nodes, such as in splits and creating new roots, and the other is for when
// we are deserializing a node and not all bp's are touched
#define BP_INIT_TOUCHED_CLOCK(node, i) ((node)->bp[i].clock_count = 1)
#define BP_INIT_UNTOUCHED_CLOCK(node, i) ((node)->bp[i].clock_count = 0)
// ftnode leaf basementnode macros,
#define BLB_MAX_MSN_APPLIED(node,i) (BLB(node,i)->max_msn_applied)
#define BLB_MAX_DSN_APPLIED(node,i) (BLB(node,i)->max_dsn_applied)
#define BLB_DATA(node,i) (&(BLB(node,i)->data_buffer))
#define BLB_NBYTESINDATA(node,i) (BLB_DATA(node,i)->get_disk_size())
#define BLB_SEQINSERT(node,i) (BLB(node,i)->seqinsert)
......@@ -245,7 +245,7 @@ doit (void) {
// what we say and flushes the child we pick
FTNODE node = NULL;
toku_pin_node_with_min_bfe(&node, node_internal, t);
toku_assert_entire_node_in_memory(node);
toku_ftnode_assert_fully_in_memory(node);
assert(node->n_children == 2);
assert(!node->dirty);
assert(toku_bnc_n_entries(node->bp[0].ptr.u.nonleaf) > 0);
......@@ -268,7 +268,7 @@ doit (void) {
assert(num_flushes_called == 1);
toku_pin_node_with_min_bfe(&node, node_internal, t);
toku_assert_entire_node_in_memory(node);
toku_ftnode_assert_fully_in_memory(node);
assert(node->dirty);
assert(node->n_children == 2);
// child 0 should have empty buffer because it flushed
......@@ -287,7 +287,7 @@ doit (void) {
toku_pin_node_with_min_bfe(&node, node_internal, t);
assert(node->dirty);
toku_assert_entire_node_in_memory(node);
toku_ftnode_assert_fully_in_memory(node);
assert(node->n_children == 2);
// both buffers should be empty now
assert(toku_bnc_n_entries(node->bp[0].ptr.u.nonleaf) == 0);
......@@ -305,7 +305,7 @@ doit (void) {
toku_pin_node_with_min_bfe(&node, node_internal, t);
assert(node->dirty); // nothing was flushed, but since we were trying to flush to a leaf, both become dirty
toku_assert_entire_node_in_memory(node);
toku_ftnode_assert_fully_in_memory(node);
assert(node->n_children == 2);
// both buffers should be empty now
assert(toku_bnc_n_entries(node->bp[0].ptr.u.nonleaf) == 0);
......@@ -326,7 +326,7 @@ doit (void) {
// use a for loop so to get us down both paths
for (int i = 0; i < 2; i++) {
toku_pin_node_with_min_bfe(&node, node_root, t);
toku_assert_entire_node_in_memory(node); // entire root is in memory
toku_ftnode_assert_fully_in_memory(node); // entire root is in memory
curr_child_to_flush = i;
num_flushes_called = 0;
toku_ft_flush_some_child(t->ft, node, &fa);
......@@ -376,7 +376,7 @@ doit (void) {
//now let's do the same test as above
toku_pin_node_with_min_bfe(&node, node_root, t);
toku_assert_entire_node_in_memory(node); // entire root is in memory
toku_ftnode_assert_fully_in_memory(node); // entire root is in memory
curr_child_to_flush = 0;
num_flushes_called = 0;
toku_ft_flush_some_child(t->ft, node, &fa);
......
......@@ -102,6 +102,7 @@ PATENT RIGHTS GRANT:
#include <portability/toku_path.h>
#include "ft.h"
#include "node.h"
#include "block_table.h"
#include "log-internal.h"
#include "logger.h"
......
......@@ -180,7 +180,7 @@ static void test_split_merge(void) {
toku_pin_ftnode(ft, rightmost_blocknum_before_merge,
toku_cachetable_hash(ft->cf, rightmost_blocknum_before_merge),
&bfe, PL_WRITE_EXPENSIVE, &rightmost_leaf, true);
invariant(get_node_reactivity(ft, rightmost_leaf) == RE_FUSIBLE);
invariant(toku_ftnode_get_reactivity(ft, rightmost_leaf) == RE_FUSIBLE);
toku_unpin_ftnode(ft, rightmost_leaf);
// - merge the rightmost child now that it's fusible
......
......@@ -95,6 +95,7 @@ PATENT RIGHTS GRANT:
#include "ft.h"
#include "fttypes.h"
#include "ft-internal.h"
#include "ft/node.h"
#include <ctype.h>
#include <stdint.h>
#include <stdio.h>
......
Markdown is supported
0%
or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment