Commit 340045d0 authored by Zardosht Kasheff's avatar Zardosht Kasheff Committed by Yoni Fogel

[t:4762], merge to main

git-svn-id: file:///svn/toku/tokudb@42336 c7de825b-a66e-492c-adef-691d508d4ae1
parent 80ca6d25
......@@ -918,6 +918,7 @@ brt_leaf_apply_cmd_once (
LEAFENTRY le,
OMT snapshot_xids,
OMT live_list_reverse,
OMT live_root_txns,
uint64_t *workdonep
);
......@@ -932,7 +933,8 @@ brt_leaf_put_cmd (
BRT_MSG cmd,
uint64_t *workdone,
OMT snapshot_txnids,
OMT live_list_reverse
OMT live_list_reverse,
OMT live_root_txns
);
void toku_apply_cmd_to_leaf(
......@@ -943,7 +945,8 @@ void toku_apply_cmd_to_leaf(
BRT_MSG cmd,
uint64_t *workdone,
OMT snapshot_txnids,
OMT live_list_reverse
OMT live_list_reverse,
OMT live_root_txns
);
// FIXME needs toku prefix
......@@ -955,7 +958,8 @@ void brtnode_put_cmd (
BRT_MSG cmd,
bool is_fresh,
OMT snapshot_txnids,
OMT live_list_reverse
OMT live_list_reverse,
OMT live_root_txns
);
void toku_reset_root_xid_that_created(BRT brt, TXNID new_root_xid_that_created);
......
......@@ -147,6 +147,7 @@ int toku_testsetup_insert_to_leaf (BRT brt, BLOCKNUM blocknum, char *key, int ke
&cmd,
true,
NULL,
NULL,
NULL
);
......
......@@ -1563,6 +1563,7 @@ brt_leaf_apply_cmd_once (
LEAFENTRY le,
OMT snapshot_xids,
OMT live_list_reverse,
OMT live_root_txns,
uint64_t *workdone
)
// Effect: Apply cmd to leafentry (msn is ignored)
......@@ -1585,7 +1586,7 @@ brt_leaf_apply_cmd_once (
// 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.
{
int r = apply_msg_to_leafentry(cmd, le, &newsize, &new_le, bn->buffer, &bn->buffer_mempool, &maybe_free, snapshot_xids, live_list_reverse, &numbytes_delta);
int r = apply_msg_to_leafentry(cmd, le, &newsize, &new_le, bn->buffer, &bn->buffer_mempool, &maybe_free, snapshot_xids, live_list_reverse, live_root_txns, &numbytes_delta);
invariant(r==0);
}
......@@ -1658,6 +1659,7 @@ struct setval_extra_s {
LEAFENTRY le;
OMT snapshot_txnids;
OMT live_list_reverse;
OMT live_root_txns;
uint64_t * workdone; // set by brt_leaf_apply_cmd_once()
};
......@@ -1689,7 +1691,7 @@ static void setval_fun (const DBT *new_val, void *svextra_v) {
}
brt_leaf_apply_cmd_once(svextra->leafnode, svextra->bn, &msg,
svextra->idx, svextra->le,
svextra->snapshot_txnids, svextra->live_list_reverse,
svextra->snapshot_txnids, svextra->live_list_reverse, svextra->live_root_txns,
svextra->workdone);
svextra->setval_r = 0;
}
......@@ -1700,7 +1702,7 @@ static void setval_fun (const DBT *new_val, void *svextra_v) {
// 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(brt_update_func update_fun, DESCRIPTOR desc, BRTNODE leafnode, BASEMENTNODE bn, BRT_MSG cmd, int idx,
LEAFENTRY le, OMT snapshot_txnids, OMT live_list_reverse,
LEAFENTRY le, OMT snapshot_txnids, OMT live_list_reverse, OMT live_root_txns,
uint64_t * workdone) {
LEAFENTRY le_for_update;
DBT key;
......@@ -1743,7 +1745,7 @@ static int do_update(brt_update_func update_fun, DESCRIPTOR desc, BRTNODE leafno
}
struct setval_extra_s setval_extra = {setval_tag, FALSE, 0, leafnode, bn, cmd->msn, cmd->xids,
keyp, idx, le_for_update, snapshot_txnids, live_list_reverse, workdone};
keyp, idx, le_for_update, snapshot_txnids, live_list_reverse, live_root_txns, workdone};
// call handlerton's brt->update_fun(), which passes setval_extra to setval_fun()
FAKE_DB(db, desc);
int r = update_fun(
......@@ -1769,7 +1771,8 @@ brt_leaf_put_cmd (
BRT_MSG cmd,
uint64_t *workdone,
OMT snapshot_txnids,
OMT live_list_reverse
OMT live_list_reverse,
OMT live_root_txns
)
// Effect:
// Put a cmd into a leaf.
......@@ -1809,7 +1812,7 @@ brt_leaf_put_cmd (
assert(r==0);
storeddata=storeddatav;
}
brt_leaf_apply_cmd_once(leafnode, bn, cmd, idx, storeddata, snapshot_txnids, live_list_reverse, workdone);
brt_leaf_apply_cmd_once(leafnode, bn, cmd, idx, storeddata, snapshot_txnids, live_list_reverse, live_root_txns, workdone);
// if the insertion point is within a window of the right edge of
// the leaf then it is sequential
......@@ -1841,7 +1844,7 @@ brt_leaf_put_cmd (
while (1) {
u_int32_t num_leafentries_before = toku_omt_size(bn->buffer);
brt_leaf_apply_cmd_once(leafnode, bn, cmd, idx, storeddata, snapshot_txnids, live_list_reverse, workdone);
brt_leaf_apply_cmd_once(leafnode, bn, cmd, idx, storeddata, snapshot_txnids, live_list_reverse, live_root_txns, workdone);
{
// Now we must find the next leafentry.
......@@ -1887,7 +1890,7 @@ brt_leaf_put_cmd (
storeddata=storeddatav;
int deleted = 0;
if (!le_is_clean(storeddata)) { //If already clean, nothing to do.
brt_leaf_apply_cmd_once(leafnode, bn, cmd, idx, storeddata, snapshot_txnids, live_list_reverse, workdone);
brt_leaf_apply_cmd_once(leafnode, bn, cmd, idx, storeddata, snapshot_txnids, live_list_reverse, live_root_txns, workdone);
u_int32_t new_omt_size = toku_omt_size(bn->buffer);
if (new_omt_size != omt_size) {
assert(new_omt_size+1 == omt_size);
......@@ -1913,7 +1916,7 @@ brt_leaf_put_cmd (
storeddata=storeddatav;
int deleted = 0;
if (le_has_xids(storeddata, cmd->xids)) {
brt_leaf_apply_cmd_once(leafnode, bn, cmd, idx, storeddata, snapshot_txnids, live_list_reverse, workdone);
brt_leaf_apply_cmd_once(leafnode, bn, cmd, idx, storeddata, snapshot_txnids, live_list_reverse, live_root_txns, workdone);
u_int32_t new_omt_size = toku_omt_size(bn->buffer);
if (new_omt_size != omt_size) {
assert(new_omt_size+1 == omt_size);
......@@ -1934,10 +1937,10 @@ brt_leaf_put_cmd (
r = toku_omt_find_zero(bn->buffer, toku_cmd_leafval_heaviside, &be,
&storeddatav, &idx);
if (r==DB_NOTFOUND) {
r = do_update(update_fun, desc, leafnode, bn, cmd, idx, NULL, snapshot_txnids, live_list_reverse, workdone);
r = do_update(update_fun, desc, leafnode, bn, cmd, idx, NULL, snapshot_txnids, live_list_reverse, live_root_txns, workdone);
} else if (r==0) {
storeddata=storeddatav;
r = do_update(update_fun, desc, leafnode, bn, cmd, idx, storeddata, snapshot_txnids, live_list_reverse, workdone);
r = do_update(update_fun, desc, leafnode, bn, cmd, idx, storeddata, snapshot_txnids, live_list_reverse, live_root_txns, workdone);
} // otherwise, a worse error, just return it
break;
}
......@@ -1949,7 +1952,7 @@ brt_leaf_put_cmd (
r = toku_omt_fetch(bn->buffer, idx, &storeddatav);
assert(r==0);
storeddata=storeddatav;
r = do_update(update_fun, desc, leafnode, bn, cmd, idx, storeddata, snapshot_txnids, live_list_reverse, workdone);
r = do_update(update_fun, desc, leafnode, bn, cmd, idx, storeddata, snapshot_txnids, live_list_reverse, live_root_txns, workdone);
// TODO(leif): This early return means get_leaf_reactivity()
// and VERIFY_NODE() never get called. Is this a problem?
assert(r==0);
......@@ -2286,6 +2289,7 @@ brt_basement_node_gc_once(BASEMENTNODE bn,
LEAFENTRY leaf_entry,
OMT snapshot_xids,
OMT live_list_reverse,
OMT live_root_txns,
STAT64INFO_S * delta)
{
assert(leaf_entry);
......@@ -2313,7 +2317,8 @@ brt_basement_node_gc_once(BASEMENTNODE bn,
&bn->buffer_mempool,
&maybe_free,
snapshot_xids,
live_list_reverse);
live_list_reverse,
live_root_txns);
// These will represent the number of bytes and rows changed as
// part of the garbage collection.
......@@ -2355,6 +2360,7 @@ static void
basement_node_gc_all_les(BASEMENTNODE bn,
OMT snapshot_xids,
OMT live_list_reverse,
OMT live_root_txns,
STAT64INFO_S * delta)
{
int r = 0;
......@@ -2366,7 +2372,7 @@ basement_node_gc_all_les(BASEMENTNODE bn,
r = toku_omt_fetch(bn->buffer, index, &storedatav);
assert(r == 0);
leaf_entry = storedatav;
brt_basement_node_gc_once(bn, index, leaf_entry, snapshot_xids, live_list_reverse, delta);
brt_basement_node_gc_once(bn, index, leaf_entry, snapshot_xids, live_list_reverse, live_root_txns, delta);
// Check if the leaf entry was deleted or not.
if (num_leafentries_before == toku_omt_size(bn->buffer)) {
++index;
......@@ -2378,7 +2384,8 @@ basement_node_gc_all_les(BASEMENTNODE bn,
static void
brt_leaf_gc_all_les(BRTNODE node,
OMT snapshot_xids,
OMT live_list_reverse)
OMT live_list_reverse,
OMT live_root_txns)
{
toku_assert_entire_node_in_memory(node);
assert(node->height == 0);
......@@ -2389,7 +2396,7 @@ brt_leaf_gc_all_les(BRTNODE node,
STAT64INFO_S delta;
delta.numrows = 0;
delta.numbytes = 0;
basement_node_gc_all_les(bn, snapshot_xids, live_list_reverse, &delta);
basement_node_gc_all_les(bn, snapshot_xids, live_list_reverse, live_root_txns, &delta);
// Update the header stats, but only if the leaf node is
// dirty.
if (node->dirty) {
......@@ -2424,32 +2431,39 @@ toku_bnc_flush_to_child(
&brtcmd,
is_fresh,
NULL, // pass NULL omts (snapshot_txnids and live_list_reverse)
NULL // we're going to handle GC ourselves next
NULL, // we're going to handle GC ourselves next
NULL
);
}));
// Run garbage collection, if we are a leaf entry.
TOKULOGGER logger = toku_cachefile_logger(cf);
if (child->height == 0 && logger) {
int r;
OMT snapshot_txnids = NULL;
OMT live_list_reverse = NULL;
OMT live_root_txns = NULL;
{
toku_pthread_mutex_lock(&logger->txn_list_lock);
int r = toku_omt_clone_noptr(&snapshot_txnids,
r = toku_omt_clone_noptr(&snapshot_txnids,
logger->snapshot_txnids);
assert_zero(r);
r = toku_omt_clone_pool(&live_list_reverse,
logger->live_list_reverse,
sizeof(XID_PAIR_S));
assert_zero(r);
r = toku_omt_clone_noptr(&live_root_txns,
logger->live_root_txns);
assert_zero(r);
// take advantage of surrounding mutex, update stats.
size_t buffsize = bnc->n_bytes_in_buffer;
STATUS_VALUE(BRT_MSG_BYTES_OUT) += buffsize;
// may be misleading if there's a broadcast message in there
STATUS_VALUE(BRT_MSG_BYTES_CURR) -= buffsize;
toku_pthread_mutex_unlock(&logger->txn_list_lock);
}
// Perform the garbage collection.
brt_leaf_gc_all_les(child, snapshot_txnids, live_list_reverse);
brt_leaf_gc_all_les(child, snapshot_txnids, live_list_reverse, live_root_txns);
// Free the OMT's we used for garbage collecting.
toku_omt_destroy(&snapshot_txnids);
......@@ -2485,7 +2499,8 @@ brtnode_put_cmd (
BRT_MSG cmd,
bool is_fresh,
OMT snapshot_txnids,
OMT live_list_reverse
OMT live_list_reverse,
OMT live_root_txns
)
// Effect: Push CMD into the subtree rooted at NODE.
// If NODE is a leaf, then
......@@ -2510,7 +2525,8 @@ brtnode_put_cmd (
cmd,
&workdone,
snapshot_txnids,
live_list_reverse
live_list_reverse,
live_root_txns
);
} else {
brt_nonleaf_put_cmd(compare_fun, desc, node, cmd, is_fresh);
......@@ -2532,7 +2548,8 @@ void toku_apply_cmd_to_leaf(
BRT_MSG cmd,
uint64_t *workdone,
OMT snapshot_txnids,
OMT live_list_reverse
OMT live_list_reverse,
OMT live_root_txns
)
{
VERIFY_NODE(t, node);
......@@ -2575,7 +2592,8 @@ void toku_apply_cmd_to_leaf(
cmd,
workdone,
snapshot_txnids,
live_list_reverse);
live_list_reverse,
live_root_txns);
} else {
STATUS_VALUE(BRT_MSN_DISCARDS)++;
}
......@@ -2592,7 +2610,8 @@ void toku_apply_cmd_to_leaf(
cmd,
workdone,
snapshot_txnids,
live_list_reverse);
live_list_reverse,
live_root_txns);
} else {
STATUS_VALUE(BRT_MSN_DISCARDS)++;
}
......@@ -2613,6 +2632,7 @@ static void push_something_at_root (BRT brt, BRTNODE *nodep, BRT_MSG cmd)
TOKULOGGER logger = toku_cachefile_logger(brt->cf);
OMT snapshot_txnids = logger ? logger->snapshot_txnids : NULL;
OMT live_list_reverse = logger ? logger->live_list_reverse : NULL;
OMT live_root_txns = logger ? logger->live_root_txns : NULL;
MSN cmd_msn = cmd->msn;
invariant(cmd_msn.msn > node->max_msn_applied_to_node_on_disk.msn);
brtnode_put_cmd(
......@@ -2623,7 +2643,8 @@ static void push_something_at_root (BRT brt, BRTNODE *nodep, BRT_MSG cmd)
cmd,
true,
snapshot_txnids,
live_list_reverse
live_list_reverse,
live_root_txns
);
//
// assumption is that brtnode_put_cmd will
......@@ -4478,7 +4499,7 @@ does_txn_read_entry(TXNID id, TOKUTXN context) {
if (id < oldest_live_in_snapshot || id == context->ancestor_txnid64) {
rval = TOKUDB_ACCEPT;
}
else if (id > context->snapshot_txnid64 || toku_is_txn_in_live_root_txn_list(context, id)) {
else if (id > context->snapshot_txnid64 || toku_is_txn_in_live_root_txn_list(context->live_root_txn_list, id)) {
rval = 0;
}
else {
......@@ -4791,7 +4812,7 @@ do_brt_leaf_put_cmd(BRT t, BRTNODE leafnode, BASEMENTNODE bn, BRTNODE ancestor,
toku_fill_dbt(&hk, key, keylen);
DBT hv;
BRT_MSG_S brtcmd = { type, msn, xids, .u.id = { &hk, toku_fill_dbt(&hv, val, vallen) } };
brt_leaf_put_cmd(t->compare_fun, t->update_fun, &t->h->cmp_descriptor, leafnode, bn, &brtcmd, &BP_WORKDONE(ancestor, childnum), NULL, NULL); // pass NULL omts (snapshot_txnids and live_list_reverse) to prevent GC from running on message application for a query
brt_leaf_put_cmd(t->compare_fun, t->update_fun, &t->h->cmp_descriptor, leafnode, bn, &brtcmd, &BP_WORKDONE(ancestor, childnum), NULL, NULL, NULL); // pass NULL omts (snapshot_txnids and live_list_reverse) to prevent GC from running on message application for a query
} else {
STATUS_VALUE(BRT_MSN_DISCARDS)++;
}
......
......@@ -2740,7 +2740,7 @@ static void add_pair_to_leafnode (struct leaf_buf *lbuf, unsigned char *key, int
DBT theval = { .data = val, .size = vallen };
BRT_MSG_S cmd = { BRT_INSERT, ZERO_MSN, lbuf->xids, .u.id = { &thekey, &theval } };
uint64_t workdone=0;
brt_leaf_apply_cmd_once(leafnode, BLB(leafnode,0), &cmd, idx, NULL, NULL, NULL, &workdone);
brt_leaf_apply_cmd_once(leafnode, BLB(leafnode,0), &cmd, idx, NULL, NULL, NULL, NULL, &workdone);
}
static int write_literal(struct dbout *out, void*data, size_t len) {
......
......@@ -158,9 +158,9 @@ toku_find_xid_by_xid (OMTVALUE v, void *xidv) {
int
toku_find_pair_by_xid (OMTVALUE v, void *xidv) {
XID_PAIR pair = v;
TXNID *xidfind = xidv;
if (pair->xid1<*xidfind) return -1;
if (pair->xid1>*xidfind) return +1;
TXNID xidfind = (TXNID)xidv;
if (pair->xid1<xidfind) return -1;
if (pair->xid1>xidfind) return +1;
return 0;
}
......@@ -176,17 +176,17 @@ static int
live_list_reverse_note_txn_end_iter(OMTVALUE live_xidv, u_int32_t UU(index), void*txnv) {
TOKUTXN txn = txnv;
TXNID xid = txn->txnid64; // xid of txn that is closing
TXNID *live_xid = live_xidv; // xid on closing txn's live list
TXNID live_xid = (TXNID)live_xidv; // xid on closing txn's live list
OMTVALUE pairv;
XID_PAIR pair;
uint32_t idx;
int r;
OMT reverse = txn->logger->live_list_reverse;
r = toku_omt_find_zero(reverse, toku_find_pair_by_xid, live_xid, &pairv, &idx);
r = toku_omt_find_zero(reverse, toku_find_pair_by_xid, (void *)live_xid, &pairv, &idx);
invariant(r==0);
pair = pairv;
invariant(pair->xid1 == *live_xid); //sanity check
invariant(pair->xid1 == live_xid); //sanity check
if (pair->xid2 == xid) {
//There is a record that needs to be either deleted or updated
TXNID olderxid;
......@@ -200,7 +200,7 @@ live_list_reverse_note_txn_end_iter(OMTVALUE live_xidv, u_int32_t UU(index), voi
//There is an older txn
olderxid = (TXNID) olderv;
invariant(olderxid < xid);
if (olderxid >= *live_xid) {
if (olderxid >= live_xid) {
//older txn is new enough, we need to update.
pair->xid2 = olderxid;
should_delete = FALSE;
......@@ -296,12 +296,13 @@ void toku_rollback_txn_close (TOKUTXN txn) {
}
if (txn->parent==NULL) {
OMTVALUE txnagain;
OMTVALUE v;
u_int32_t idx;
//Remove txn from list of live root txns
r = toku_omt_find_zero(logger->live_root_txns, find_xid, txn, &txnagain, &idx);
r = toku_omt_find_zero(logger->live_root_txns, toku_find_xid_by_xid, (OMTVALUE)txn->txnid64, &v, &idx);
assert(r==0);
assert(txn==txnagain);
TXNID xid = (TXNID) v;
invariant(xid == txn->txnid64);
r = toku_omt_delete_at(logger->live_root_txns, idx);
assert(r==0);
}
......@@ -328,11 +329,6 @@ void toku_rollback_txn_close (TOKUTXN txn) {
{
//Free memory used for live root txns local list
invariant(toku_omt_size(txn->live_root_txn_list) > 0);
OMTVALUE v;
//store a single array of txnids
r = toku_omt_fetch(txn->live_root_txn_list, 0, &v);
invariant(r==0);
toku_free(v);
toku_omt_destroy(&txn->live_root_txn_list);
}
}
......
......@@ -39,7 +39,7 @@ append_leaf(BRTNODE leafnode, void *key, size_t keylen, void *val, size_t vallen
// apply an insert to the leaf node
BRT_MSG_S cmd = { BRT_INSERT, msn, xids_get_root_xids(), .u.id = { &thekey, &theval } };
brt_leaf_apply_cmd_once(leafnode, BLB(leafnode,0), &cmd, idx, NULL, NULL, NULL, NULL);
brt_leaf_apply_cmd_once(leafnode, BLB(leafnode,0), &cmd, idx, NULL, NULL, NULL, NULL, NULL);
leafnode->max_msn_applied_to_node_on_disk = msn;
......
......@@ -47,7 +47,7 @@ append_leaf(BRT brt, BRTNODE leafnode, void *key, size_t keylen, void *val, size
BRT_MSG_S cmd = { BRT_INSERT, msn, xids_get_root_xids(), .u.id = { &thekey, &theval } };
u_int64_t workdone=0;
toku_apply_cmd_to_leaf(brt->compare_fun, brt->update_fun, &brt->h->cmp_descriptor, leafnode, &cmd, &workdone, NULL, NULL);
toku_apply_cmd_to_leaf(brt->compare_fun, brt->update_fun, &brt->h->cmp_descriptor, leafnode, &cmd, &workdone, NULL, NULL, NULL);
{
int r = toku_brt_lookup(brt, &thekey, lookup_checkf, &pair);
assert(r==0);
......@@ -55,7 +55,7 @@ append_leaf(BRT brt, BRTNODE leafnode, void *key, size_t keylen, void *val, size
}
BRT_MSG_S badcmd = { BRT_INSERT, msn, xids_get_root_xids(), .u.id = { &thekey, &badval } };
toku_apply_cmd_to_leaf(brt->compare_fun, brt->update_fun, &brt->h->cmp_descriptor, leafnode, &badcmd, &workdone, NULL, NULL);
toku_apply_cmd_to_leaf(brt->compare_fun, brt->update_fun, &brt->h->cmp_descriptor, leafnode, &badcmd, &workdone, NULL, NULL, NULL);
// message should be rejected for duplicate msn, row should still have original val
......@@ -68,7 +68,7 @@ append_leaf(BRT brt, BRTNODE leafnode, void *key, size_t keylen, void *val, size
// now verify that message with proper msn gets through
msn = next_dummymsn();
BRT_MSG_S cmd2 = { BRT_INSERT, msn, xids_get_root_xids(), .u.id = { &thekey, &val2 } };
toku_apply_cmd_to_leaf(brt->compare_fun, brt->update_fun, &brt->h->cmp_descriptor, leafnode, &cmd2, &workdone, NULL, NULL);
toku_apply_cmd_to_leaf(brt->compare_fun, brt->update_fun, &brt->h->cmp_descriptor, leafnode, &cmd2, &workdone, NULL, NULL, NULL);
// message should be accepted, val should have new value
{
......@@ -80,7 +80,7 @@ append_leaf(BRT brt, BRTNODE leafnode, void *key, size_t keylen, void *val, size
// now verify that message with lesser (older) msn is rejected
msn.msn = msn.msn - 10;
BRT_MSG_S cmd3 = { BRT_INSERT, msn, xids_get_root_xids(), .u.id = { &thekey, &badval } };
toku_apply_cmd_to_leaf(brt->compare_fun, brt->update_fun, &brt->h->cmp_descriptor, leafnode, &cmd3, &workdone, NULL, NULL);
toku_apply_cmd_to_leaf(brt->compare_fun, brt->update_fun, &brt->h->cmp_descriptor, leafnode, &cmd3, &workdone, NULL, NULL, NULL);
// message should be rejected, val should still have value in pair2
{
......
......@@ -125,9 +125,9 @@ insert_random_message_to_leaf(BRT t, BRTNODE leafnode, BASEMENTNODE blb, LEAFENT
msg.u.id.val = valdbt;
size_t memsize;
int64_t numbytes;
int r = apply_msg_to_leafentry(&msg, NULL, &memsize, save, NULL, NULL, NULL, NULL, NULL, &numbytes);
int r = apply_msg_to_leafentry(&msg, NULL, &memsize, save, NULL, NULL, NULL, NULL, NULL, NULL, &numbytes);
assert_zero(r);
brt_leaf_put_cmd(t->compare_fun, t->update_fun, NULL, leafnode, blb, &msg, NULL, NULL, NULL);
brt_leaf_put_cmd(t->compare_fun, t->update_fun, NULL, leafnode, blb, &msg, NULL, NULL, NULL, NULL);
if (msn.msn > blb->max_msn_applied.msn) {
blb->max_msn_applied = msn;
}
......@@ -166,13 +166,13 @@ insert_same_message_to_leaves(BRT t, BRTNODE child1, BASEMENTNODE blb1, BRTNODE
msg.u.id.val = valdbt;
size_t memsize;
int64_t numbytes;
int r = apply_msg_to_leafentry(&msg, NULL, &memsize, save, NULL, NULL, NULL, NULL, NULL, &numbytes);
int r = apply_msg_to_leafentry(&msg, NULL, &memsize, save, NULL, NULL, NULL, NULL, NULL, NULL, &numbytes);
assert_zero(r);
brt_leaf_put_cmd(t->compare_fun, t->update_fun, NULL, child1, blb1, &msg, NULL, NULL, NULL);
brt_leaf_put_cmd(t->compare_fun, t->update_fun, NULL, child1, blb1, &msg, NULL, NULL, NULL, NULL);
if (msn.msn > blb1->max_msn_applied.msn) {
blb1->max_msn_applied = msn;
}
brt_leaf_put_cmd(t->compare_fun, t->update_fun, NULL, child2, blb2, &msg, NULL, NULL, NULL);
brt_leaf_put_cmd(t->compare_fun, t->update_fun, NULL, child2, blb2, &msg, NULL, NULL, NULL, NULL);
if (msn.msn > blb2->max_msn_applied.msn) {
blb2->max_msn_applied = msn;
}
......@@ -584,7 +584,7 @@ flush_to_leaf(BRT t, bool make_leaf_up_to_date, bool use_flush) {
if (make_leaf_up_to_date) {
for (i = 0; i < num_parent_messages; ++i) {
if (!parent_messages_is_fresh[i]) {
toku_apply_cmd_to_leaf(t->compare_fun, t->update_fun, &t->h->descriptor, child, parent_messages[i], NULL, NULL, NULL);
toku_apply_cmd_to_leaf(t->compare_fun, t->update_fun, &t->h->descriptor, child, parent_messages[i], NULL, NULL, NULL, NULL);
}
}
for (i = 0; i < 8; ++i) {
......@@ -808,7 +808,7 @@ flush_to_leaf_with_keyrange(BRT t, bool make_leaf_up_to_date) {
for (i = 0; i < num_parent_messages; ++i) {
if (dummy_cmp(NULL, parent_messages[i]->u.id.key, &childkeys[7]) <= 0 &&
!parent_messages_is_fresh[i]) {
toku_apply_cmd_to_leaf(t->compare_fun, t->update_fun, &t->h->descriptor, child, parent_messages[i], NULL, NULL, NULL);
toku_apply_cmd_to_leaf(t->compare_fun, t->update_fun, &t->h->descriptor, child, parent_messages[i], NULL, NULL, NULL, NULL);
}
}
for (i = 0; i < 8; ++i) {
......@@ -995,8 +995,8 @@ compare_apply_and_flush(BRT t, bool make_leaf_up_to_date) {
if (make_leaf_up_to_date) {
for (i = 0; i < num_parent_messages; ++i) {
if (!parent_messages_is_fresh[i]) {
toku_apply_cmd_to_leaf(t->compare_fun, t->update_fun, &t->h->descriptor, child1, parent_messages[i], NULL, NULL, NULL);
toku_apply_cmd_to_leaf(t->compare_fun, t->update_fun, &t->h->descriptor, child2, parent_messages[i], NULL, NULL, NULL);
toku_apply_cmd_to_leaf(t->compare_fun, t->update_fun, &t->h->descriptor, child1, parent_messages[i], NULL, NULL, NULL, NULL);
toku_apply_cmd_to_leaf(t->compare_fun, t->update_fun, &t->h->descriptor, child2, parent_messages[i], NULL, NULL, NULL, NULL);
}
}
for (i = 0; i < 8; ++i) {
......
......@@ -399,7 +399,7 @@ test_le_apply(ULE ule_initial, BRT_MSG msg, ULE ule_expected) {
&result_memsize,
&le_result,
NULL,
NULL, NULL, NULL, NULL, &ignoreme);
NULL, NULL, NULL, NULL, NULL, &ignoreme);
CKERR(r);
if (le_result)
......
......@@ -42,7 +42,7 @@ append_leaf(BRTNODE leafnode, void *key, size_t keylen, void *val, size_t vallen
// apply an insert to the leaf node
BRT_MSG_S cmd = { BRT_INSERT, msn, xids_get_root_xids(), .u.id = { &thekey, &theval } };
brt_leaf_apply_cmd_once(leafnode, BLB(leafnode, 0), &cmd, idx, NULL, NULL, NULL, NULL);
brt_leaf_apply_cmd_once(leafnode, BLB(leafnode, 0), &cmd, idx, NULL, NULL, NULL, NULL, NULL);
// Create bad tree (don't do following):
// leafnode->max_msn_applied_to_node = msn;
......
......@@ -30,7 +30,7 @@ append_leaf(BRTNODE leafnode, void *key, size_t keylen, void *val, size_t vallen
// apply an insert to the leaf node
MSN msn = next_dummymsn();
BRT_MSG_S cmd = { BRT_INSERT, msn, xids_get_root_xids(), .u.id = { &thekey, &theval } };
brt_leaf_apply_cmd_once(leafnode, BLB(leafnode, 0), &cmd, idx, NULL, NULL, NULL, NULL);
brt_leaf_apply_cmd_once(leafnode, BLB(leafnode, 0), &cmd, idx, NULL, NULL, NULL, NULL, NULL);
// dont forget to dirty the node
leafnode->dirty = 1;
......
......@@ -31,7 +31,7 @@ append_leaf(BRTNODE leafnode, void *key, size_t keylen, void *val, size_t vallen
// apply an insert to the leaf node
MSN msn = next_dummymsn();
BRT_MSG_S cmd = { BRT_INSERT, msn, xids_get_root_xids(), .u.id = { &thekey, &theval } };
brt_leaf_apply_cmd_once(leafnode, BLB(leafnode, 0), &cmd, idx, NULL, NULL, NULL, NULL);
brt_leaf_apply_cmd_once(leafnode, BLB(leafnode, 0), &cmd, idx, NULL, NULL, NULL, NULL, NULL);
// dont forget to dirty the node
leafnode->dirty = 1;
......
......@@ -30,7 +30,7 @@ append_leaf(BRTNODE leafnode, void *key, size_t keylen, void *val, size_t vallen
// apply an insert to the leaf node
MSN msn = next_dummymsn();
BRT_MSG_S cmd = { BRT_INSERT, msn, xids_get_root_xids(), .u.id = { &thekey, &theval } };
brt_leaf_apply_cmd_once(leafnode, BLB(leafnode, 0), &cmd, idx, NULL, NULL, NULL, NULL);
brt_leaf_apply_cmd_once(leafnode, BLB(leafnode, 0), &cmd, idx, NULL, NULL, NULL, NULL, NULL);
// dont forget to dirty the node
leafnode->dirty = 1;
......
......@@ -31,7 +31,7 @@ append_leaf(BRTNODE leafnode, void *key, size_t keylen, void *val, size_t vallen
// apply an insert to the leaf node
MSN msn = next_dummymsn();
BRT_MSG_S cmd = { BRT_INSERT, msn, xids_get_root_xids(), .u.id = { &thekey, &theval } };
brt_leaf_apply_cmd_once(leafnode, BLB(leafnode,0), &cmd, idx, NULL, NULL, NULL, NULL);
brt_leaf_apply_cmd_once(leafnode, BLB(leafnode,0), &cmd, idx, NULL, NULL, NULL, NULL, NULL);
// dont forget to dirty the node
leafnode->dirty = 1;
......
......@@ -31,7 +31,7 @@ append_leaf(BRTNODE leafnode, void *key, size_t keylen, void *val, size_t vallen
// apply an insert to the leaf node
MSN msn = next_dummymsn();
BRT_MSG_S cmd = { BRT_INSERT, msn, xids_get_root_xids(), .u.id = { &thekey, &theval } };
brt_leaf_apply_cmd_once(leafnode, BLB(leafnode, 0), &cmd, idx, NULL, NULL, NULL, NULL);
brt_leaf_apply_cmd_once(leafnode, BLB(leafnode, 0), &cmd, idx, NULL, NULL, NULL, NULL, NULL);
// dont forget to dirty the node
leafnode->dirty = 1;
......
......@@ -30,7 +30,7 @@ append_leaf(BRTNODE leafnode, void *key, size_t keylen, void *val, size_t vallen
// apply an insert to the leaf node
MSN msn = next_dummymsn();
BRT_MSG_S cmd = { BRT_INSERT, msn, xids_get_root_xids(), .u.id = { &thekey, &theval } };
brt_leaf_apply_cmd_once(leafnode, BLB(leafnode, 0), &cmd, idx, NULL, NULL, NULL, NULL);
brt_leaf_apply_cmd_once(leafnode, BLB(leafnode, 0), &cmd, idx, NULL, NULL, NULL, NULL, NULL);
// dont forget to dirty the node
leafnode->dirty = 1;
......
......@@ -97,33 +97,14 @@ void toku_txn_set_container_db_txn (TOKUTXN tokutxn, DB_TXN*container) {
tokutxn->container_db_txn = container;
}
static int
fill_xids (OMTVALUE xev, u_int32_t idx, void *varray) {
TOKUTXN txn = xev;
TXNID *xids = varray;
xids[idx] = txn->txnid64;
return 0;
}
// Create list of root transactions that were live when this txn began.
static int
setup_live_root_txn_list(TOKUTXN txn) {
int r;
OMT global = txn->logger->live_root_txns;
uint32_t num = toku_omt_size(global);
// global list must have at least one live root txn, this current one
invariant(num > 0);
TXNID *XMALLOC_N(num, xids);
OMTVALUE *XMALLOC_N(num, xidsp);
uint32_t i;
for (i = 0; i < num; i++) {
xidsp[i] = &xids[i];
}
r = toku_omt_iterate(global, fill_xids, xids);
assert_zero(r);
r = toku_omt_create_steal_sorted_array(&txn->live_root_txn_list, &xidsp, num, num);
int r = toku_omt_clone_noptr(
&txn->live_root_txn_list,
global
);
return r;
}
......@@ -145,17 +126,17 @@ static int
live_list_reverse_note_txn_start_iter(OMTVALUE live_xidv, u_int32_t UU(index), void*txnv) {
TOKUTXN txn = txnv;
TXNID xid = txn->txnid64; // xid of new txn that is being started
TXNID *live_xid = live_xidv; // xid on the new txn's live list
TXNID live_xid = (TXNID)live_xidv; // xid on the new txn's live list
OMTVALUE pairv;
XID_PAIR pair;
uint32_t idx;
int r;
OMT reverse = txn->logger->live_list_reverse;
r = toku_omt_find_zero(reverse, toku_find_pair_by_xid, live_xid, &pairv, &idx);
r = toku_omt_find_zero(reverse, toku_find_pair_by_xid, (void *)live_xid, &pairv, &idx);
if (r==0) {
pair = pairv;
invariant(pair->xid1 == *live_xid); //sanity check
invariant(pair->xid1 == live_xid); //sanity check
invariant(pair->xid2 < xid); //Must be older
pair->xid2 = txn->txnid64;
}
......@@ -163,7 +144,7 @@ live_list_reverse_note_txn_start_iter(OMTVALUE live_xidv, u_int32_t UU(index), v
invariant(r==DB_NOTFOUND);
//Make new entry
XMALLOC(pair);
pair->xid1 = *live_xid;
pair->xid1 = live_xid;
pair->xid2 = txn->txnid64;
r = toku_omt_insert_at(reverse, pair, idx);
assert_zero(r);
......@@ -314,7 +295,7 @@ toku_txn_start_txn(TOKUTXN txn) {
// add ancestor information, and maintain global live root txn list
if (parent == NULL) {
//Add txn to list (omt) of live root txns
r = toku_omt_insert_at(logger->live_root_txns, txn, toku_omt_size(logger->live_root_txns)); //We know it is the newest one.
r = toku_omt_insert_at(logger->live_root_txns, (OMTVALUE) txn->txnid64, toku_omt_size(logger->live_root_txns)); //We know it is the newest one.
if (r!=0) goto died;
txn->ancestor_txnid64 = txn->txnid64;
}
......@@ -660,29 +641,18 @@ TXNID toku_get_oldest_in_live_root_txn_list(TOKUTXN txn) {
int r;
r = toku_omt_fetch(omt, 0, &v);
assert_zero(r);
TXNID *xidp = v;
return *xidp;
}
//Heaviside function to find a TXNID* by TXNID* (used to find the index)
static int
find_xidp (OMTVALUE v, void *xidv) {
TXNID xid = *(TXNID *)v;
TXNID xidfind = *(TXNID *)xidv;
if (xid < xidfind) return -1;
if (xid > xidfind) return +1;
return 0;
TXNID xid = (TXNID)v;
return xid;
}
BOOL toku_is_txn_in_live_root_txn_list(TOKUTXN txn, TXNID xid) {
OMT omt = txn->live_root_txn_list;
BOOL toku_is_txn_in_live_root_txn_list(OMT live_root_txn_list, TXNID xid) {
OMTVALUE txnidpv;
uint32_t index;
BOOL retval = FALSE;
int r = toku_omt_find_zero(omt, find_xidp, &xid, &txnidpv, &index);
int r = toku_omt_find_zero(live_root_txn_list, toku_find_xid_by_xid, (void *)xid, &txnidpv, &index);
if (r==0) {
TXNID *txnidp = txnidpv;
invariant(*txnidp == xid);
TXNID txnid = (TXNID)txnidpv;
invariant(txnid == xid);
retval = TRUE;
}
else {
......@@ -738,7 +708,7 @@ verify_snapshot_system(TOKULOGGER logger) {
OMTVALUE v;
r = toku_omt_fetch(snapshot_txn->live_root_txn_list, j, &v);
assert_zero(r);
live_root_txn_list[j] = *(TXNID*)v;
live_root_txn_list[j] = (TXNID)v;
}
}
for (j = 0; j < num_live_root_txn_list; j++) {
......@@ -773,7 +743,7 @@ verify_snapshot_system(TOKULOGGER logger) {
if (txn->snapshot_type != TXN_SNAPSHOT_NONE) {
BOOL expect = txn->snapshot_txnid64 >= pair->xid1 &&
txn->snapshot_txnid64 <= pair->xid2;
BOOL found = toku_is_txn_in_live_root_txn_list(txn, pair->xid1);
BOOL found = toku_is_txn_in_live_root_txn_list(txn->live_root_txn_list, pair->xid1);
invariant((expect==FALSE) == (found==FALSE));
}
}
......
......@@ -106,7 +106,7 @@ typedef struct {
void toku_txn_get_status(TOKULOGGER logger, TXN_STATUS s);
BOOL toku_is_txn_in_live_root_txn_list(TOKUTXN txn, TXNID xid);
BOOL toku_is_txn_in_live_root_txn_list(OMT live_root_txn_list, TXNID xid);
TXNID toku_get_oldest_in_live_root_txn_list(TOKUTXN txn);
......
......@@ -194,7 +194,7 @@ toku_get_youngest_live_list_txnid_for(TXNID xc, OMT live_list_reverse) {
uint32_t idx;
TXNID rval;
int r;
r = toku_omt_find_zero(live_list_reverse, toku_find_pair_by_xid, &xc, &pairv, &idx);
r = toku_omt_find_zero(live_list_reverse, toku_find_pair_by_xid, (void *)xc, &pairv, &idx);
if (r==0) {
pair = pairv;
invariant(pair->xid1 == xc); //sanity check
......@@ -229,7 +229,7 @@ xid_reads_committed_xid(TXNID tl1, TXNID xc, OMT live_list_reverse) {
}
static void
garbage_collection(ULE ule, OMT snapshot_xids, OMT live_list_reverse) {
garbage_collection(ULE ule, OMT snapshot_xids, OMT live_list_reverse, OMT live_root_txns) {
if (ule->num_cuxrs == 1) goto done;
// will fail if too many num_cuxrs
BOOL necessary_static[MAX_TRANSACTION_RECORDS];
......@@ -250,6 +250,12 @@ garbage_collection(ULE ule, OMT snapshot_xids, OMT live_list_reverse) {
TXNID tl1;
TXNID xc = ule->uxrs[curr_committed_entry].xid;
BOOL is_xc_live = toku_is_txn_in_live_root_txn_list(live_root_txns, xc);
if (is_xc_live) {
curr_committed_entry--;
continue;
}
tl1 = toku_get_youngest_live_list_txnid_for(xc, live_list_reverse);
//
// If we find that the committed transaction is in the live list,
......@@ -261,8 +267,10 @@ garbage_collection(ULE ule, OMT snapshot_xids, OMT live_list_reverse) {
// This issue was found while testing flusher threads, and was fixed for #3979
//
if (tl1 == xc) {
curr_committed_entry--;
continue;
// if tl1 == xc, that means xc should be live and show up in
// live_root_txns, which we check above. So, if we get
// here, something is wrong.
assert(false);
}
if (tl1 == TXNID_NONE) {
// set tl1 to youngest live transaction older than ule->uxrs[curr_committed_entry]->xid
......@@ -338,6 +346,7 @@ apply_msg_to_leafentry(BRT_MSG msg, // message to apply to leafentry
void **maybe_free,
OMT snapshot_xids,
OMT live_list_reverse,
OMT live_root_txns,
int64_t * numbytes_delta_p) { // change in total size of key and val, not including any overhead
ULE_S ule;
int rval;
......@@ -351,8 +360,8 @@ apply_msg_to_leafentry(BRT_MSG msg, // message to apply to leafentry
oldnumbytes = ule_get_innermost_numbytes(&ule);
}
msg_modify_ule(&ule, msg); // modify unpacked leafentry
if (snapshot_xids && live_list_reverse) {
garbage_collection(&ule, snapshot_xids, live_list_reverse);
if (snapshot_xids && live_list_reverse && live_root_txns) {
garbage_collection(&ule, snapshot_xids, live_list_reverse, live_root_txns);
}
rval = le_pack(&ule, // create packed leafentry
new_leafentry_memorysize,
......@@ -394,13 +403,15 @@ garbage_collect_leafentry(LEAFENTRY old_leaf_entry,
struct mempool *mp,
void **maybe_free,
OMT snapshot_xids,
OMT live_list_reverse) {
OMT live_list_reverse,
OMT live_root_txns) {
int r = 0;
ULE_S ule;
le_unpack(&ule, old_leaf_entry);
assert(snapshot_xids);
assert(live_list_reverse);
garbage_collection(&ule, snapshot_xids, live_list_reverse);
assert(live_root_txns);
garbage_collection(&ule, snapshot_xids, live_list_reverse, live_root_txns);
r = le_pack(&ule,
new_leaf_entry_memory_size,
new_leaf_entry,
......
......@@ -61,6 +61,7 @@ int apply_msg_to_leafentry(BRT_MSG msg,
void **maybe_free,
OMT snapshot_xids,
OMT live_list_reverse,
OMT live_root_txns,
int64_t * numbytes_delta_p);
int garbage_collect_leafentry(LEAFENTRY old_leaf_entry,
......@@ -70,7 +71,8 @@ int garbage_collect_leafentry(LEAFENTRY old_leaf_entry,
struct mempool *mp,
void **maybe_free,
OMT snapshot_xids,
OMT live_list_reverse);
OMT live_list_reverse,
OMT live_root_txns);
TXNID toku_get_youngest_live_list_txnid_for(TXNID xc, OMT live_list_reverse);
......
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