[t:3983] merging auto-upgrade to mainline

git-svn-id: file:///svn/toku/tokudb@41591 c7de825b-a66e-492c-adef-691d508d4ae1
This commit is contained in:
Leif Walsh 2013-04-17 00:00:17 -04:00 committed by Yoni Fogel
parent 1974ace7b5
commit 8d0368fd2f
37 changed files with 1358 additions and 527 deletions

View file

@ -755,11 +755,12 @@ translation_default(struct translation *t) { // destination into which to creat
}
static void
static enum deserialize_error_code
translation_deserialize_from_buffer(struct translation *t, // destination into which to deserialize
DISKOFF location_on_disk, //Location of translation_buffer
u_int64_t size_on_disk,
unsigned char * translation_buffer) { // buffer with serialized translation
enum deserialize_error_code e;
assert(location_on_disk!=0);
t->type = TRANSLATION_CHECKPOINTED;
{
@ -768,7 +769,11 @@ translation_deserialize_from_buffer(struct translation *t, // destination int
u_int64_t offset = size_on_disk - 4;
//printf("%s:%d read from %ld (x1764 offset=%ld) size=%ld\n", __FILE__, __LINE__, block_translation_address_on_disk, offset, block_translation_size_on_disk);
u_int32_t stored_x1764 = toku_dtoh32(*(int*)(translation_buffer + offset));
assert(x1764 == stored_x1764);
if (x1764 != stored_x1764) {
fprintf(stderr, "Translation table checksum failure: calc=0x%08x read=0x%08x\n", x1764, stored_x1764);
e = DS_XSUM_FAIL;
goto exit;
}
}
struct rbuf rt;
rt.buf = translation_buffer;
@ -789,6 +794,9 @@ PRNTF("ReadIn", i, t->block_translation[i].size, t->block_translation[i].u.disko
assert(calculate_size_on_disk(t) == (int64_t)size_on_disk);
assert(t->block_translation[RESERVED_BLOCKNUM_TRANSLATION].size == (int64_t)size_on_disk);
assert(t->block_translation[RESERVED_BLOCKNUM_TRANSLATION].u.diskoff == location_on_disk);
e = DS_OK;
exit:
return e;
}
// We just initialized a translation, inform block allocator to reserve space for each blocknum in use.
@ -817,17 +825,22 @@ blocktable_note_translation (BLOCK_ALLOCATOR allocator, struct translation *t) {
// The one read from disk is the last known checkpointed one, so we are keeping it in
// place and then setting current (which is never stored on disk) for current use.
// The translation_buffer has translation only, we create the rest of the block_table.
void
enum deserialize_error_code
toku_blocktable_create_from_buffer(BLOCK_TABLE *btp,
DISKOFF location_on_disk, //Location of translation_buffer
DISKOFF size_on_disk,
unsigned char *translation_buffer) {
BLOCK_TABLE bt = blocktable_create_internal();
translation_deserialize_from_buffer(&bt->checkpointed, location_on_disk, size_on_disk, translation_buffer);
enum deserialize_error_code e = translation_deserialize_from_buffer(&bt->checkpointed, location_on_disk, size_on_disk, translation_buffer);
if (e != DS_OK) {
goto exit;
}
blocktable_note_translation(bt->block_allocator, &bt->checkpointed);
// we just filled in checkpointed, now copy it to current.
copy_translation(&bt->current, &bt->checkpointed, TRANSLATION_CURRENT);
*btp = bt;
exit:
return e;
}

View file

@ -5,6 +5,8 @@
#ident "Copyright (c) 2007-2010 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 <brttypes.h>
#if defined(__cplusplus) || defined(__cilkplusplus)
extern "C" {
#endif
@ -21,7 +23,7 @@ struct block_translation_pair {
};
void toku_blocktable_create_new(BLOCK_TABLE *btp);
void toku_blocktable_create_from_buffer(BLOCK_TABLE *btp, DISKOFF location_on_disk, DISKOFF size_on_disk, unsigned char *translation_buffer);
enum deserialize_error_code toku_blocktable_create_from_buffer(BLOCK_TABLE *btp, DISKOFF location_on_disk, DISKOFF size_on_disk, unsigned char *translation_buffer);
void toku_blocktable_destroy(BLOCK_TABLE *btp);
void toku_brtheader_lock(struct brt_header *h);

View file

@ -251,7 +251,6 @@ struct brtnode {
int height; /* height is always >= 0. 0 for leaf, >0 for nonleaf. */
int dirty;
u_int32_t fullhash;
uint32_t optimized_for_upgrade; // version number to which this leaf has been optimized, zero if never optimized for upgrade
int n_children; //for internal nodes, if n_children==TREE_FANOUT+1 then the tree needs to be rebalanced.
// for leaf nodes, represents number of basement nodes
unsigned int totalchildkeylens;
@ -377,9 +376,6 @@ struct brt_header {
uint64_t time_of_creation; // time this tree was created
uint64_t time_of_last_modification; // last time this header was serialized to disk (read from disk, overwritten when written to disk)
uint64_t time_of_last_verification; // last time that this tree was verified
BOOL upgrade_brt_performed; // initially FALSE, set TRUE when brt has been fully updated (even though nodes may not have been)
int64_t num_blocks_to_upgrade_13; // Number of v13 blocks still not newest version.
int64_t num_blocks_to_upgrade_14; // Number of v14 blocks still not newest version.
unsigned int nodesize;
unsigned int basementnodesize;
// this field is protected by tree_lock, see comment for tree_lock
@ -411,6 +407,10 @@ struct brt_header {
uint32_t count_of_optimize_in_progress_read_from_disk; // the number of hot optimize operations in progress on this tree at the time of the last crash (this field is in-memory only)
MSN msn_at_start_of_last_completed_optimize; // all messages before this msn have been applied to leaf nodes
enum toku_compression_method compression_method;
// Current Minimum MSN to be used when upgrading pre-MSN BRT's.
// This is decremented from our currnt MIN_MSN so as not to clash
// with any existing 'normal' MSN's.
MSN highest_unused_msn_for_upgrade;
};
struct brt {
@ -479,7 +479,7 @@ toku_brt_header_init(struct brt_header *h,
int toku_serialize_brt_header_size (struct brt_header *h);
int toku_serialize_brt_header_to (int fd, struct brt_header *h);
int toku_serialize_brt_header_to_wbuf (struct wbuf *, struct brt_header *h, int64_t address_translation, int64_t size_translation);
int toku_deserialize_brtheader_from (int fd, LSN max_acceptable_lsn, struct brt_header **brth);
enum deserialize_error_code toku_deserialize_brtheader_from (int fd, LSN max_acceptable_lsn, struct brt_header **brth);
int 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);
@ -818,7 +818,6 @@ int toku_brtheader_close (CACHEFILE cachefile, int fd, void *header_v, char **er
int toku_brtheader_begin_checkpoint (LSN checkpoint_lsn, void *header_v) __attribute__((__warn_unused_result__));
int toku_brtheader_checkpoint (CACHEFILE cachefile, int fd, void *header_v) __attribute__((__warn_unused_result__));
int toku_brtheader_end_checkpoint (CACHEFILE cachefile, int fd, void *header_v) __attribute__((__warn_unused_result__));
int toku_maybe_upgrade_brt(BRT t) __attribute__((__warn_unused_result__));
int toku_db_badformat(void) __attribute__((__warn_unused_result__));
int toku_brt_remove_on_commit(TOKUTXN child, DBT* iname_dbt_p) __attribute__((__warn_unused_result__));
@ -826,10 +825,6 @@ int toku_brt_remove_now(CACHETABLE ct, DBT* iname_dbt_p) __attribute__((__warn_u
typedef enum {
BRT_UPGRADE_FOOTPRINT = 0,
BRT_UPGRADE_HEADER_13, // how many headers were upgraded from version 13
BRT_UPGRADE_NONLEAF_13,
BRT_UPGRADE_LEAF_13,
BRT_UPGRADE_OPTIMIZED_FOR_UPGRADE, // how many optimize_for_upgrade messages were sent
BRT_UPGRADE_STATUS_NUM_ROWS
} brt_upgrade_status_entry;
@ -937,7 +932,6 @@ brt_leaf_put_cmd (
BRTNODE leafnode,
BASEMENTNODE bn,
BRT_MSG cmd,
bool* made_change,
uint64_t *workdone,
OMT snapshot_txnids,
OMT live_list_reverse
@ -949,7 +943,6 @@ void toku_apply_cmd_to_leaf(
DESCRIPTOR desc,
BRTNODE node,
BRT_MSG cmd,
bool *made_change,
uint64_t *workdone,
OMT snapshot_txnids,
OMT live_list_reverse

File diff suppressed because it is too large Load diff

View file

@ -10,7 +10,7 @@
// dummymsn needed to simulate msn because messages are injected at a lower level than toku_brt_root_put_cmd()
#define MIN_DUMMYMSN ((MSN) {(uint64_t)100000000000})
#define MIN_DUMMYMSN ((MSN) {(uint64_t)1 << 62})
static MSN dummymsn;
static int testsetup_initialized = 0;

View file

@ -696,7 +696,6 @@ void toku_brtnode_clone_callback(
cloned_node->height = node->height;
cloned_node->dirty = node->dirty;
cloned_node->fullhash = node->fullhash;
cloned_node->optimized_for_upgrade = node->optimized_for_upgrade;
cloned_node->n_children = node->n_children;
cloned_node->totalchildkeylens = node->totalchildkeylens;
@ -721,6 +720,7 @@ void toku_brtnode_clone_callback(
// clear dirty bit
node->dirty = 0;
cloned_node->dirty = 0;
node->layout_version_read_from_disk = BRT_LAYOUT_VERSION;
// set new pair attr if necessary
if (node->height == 0) {
*new_attr = make_brtnode_pair_attr(node);
@ -763,6 +763,7 @@ void toku_brtnode_flush_callback (
toku_cachefile_get_workqueue_load(cachefile, &n_workitems, &n_threads);
int r = toku_serialize_brtnode_to(fd, brtnode->thisnodename, brtnode, ndd, !is_clone, h, n_workitems, n_threads, for_checkpoint);
assert_zero(r);
brtnode->layout_version_read_from_disk = BRT_LAYOUT_VERSION;
}
brt_status_update_flush_reason(brtnode, for_checkpoint);
}
@ -818,7 +819,8 @@ void toku_brtnode_pe_est_callback(
assert(brtnode_pv != NULL);
long bytes_to_free = 0;
BRTNODE node = (BRTNODE)brtnode_pv;
if (node->dirty || node->height == 0) {
if (node->dirty || node->height == 0 ||
node->layout_version_read_from_disk < BRT_FIRST_LAYOUT_VERSION_WITH_BASEMENT_NODES) {
*bytes_freed_estimate = 0;
*cost = PE_CHEAP;
goto exit;
@ -876,10 +878,15 @@ compress_internal_node_partition(BRTNODE node, int i)
// callback for partially evicting a node
int toku_brtnode_pe_callback (void *brtnode_pv, PAIR_ATTR UU(old_attr), PAIR_ATTR* new_attr, void* UU(extraargs)) {
BRTNODE node = (BRTNODE)brtnode_pv;
//
// Don't partially evict dirty nodes
if (node->dirty) {
goto exit;
}
// Don't partially evict nodes whose partitions can't be read back
// from disk individually
if (node->layout_version_read_from_disk < BRT_FIRST_LAYOUT_VERSION_WITH_BASEMENT_NODES) {
goto exit;
}
//
// partial eviction for nonleaf nodes
//
@ -1404,7 +1411,6 @@ toku_initialize_empty_brtnode (BRTNODE n, BLOCKNUM nodename, int height, int num
n->layout_version_original = layout_version;
n->layout_version_read_from_disk = layout_version;
n->height = height;
n->optimized_for_upgrade = 0;
n->totalchildkeylens = 0;
n->childkeys = 0;
n->bp = 0;
@ -1623,7 +1629,6 @@ struct setval_extra_s {
LEAFENTRY le;
OMT snapshot_txnids;
OMT live_list_reverse;
bool made_change;
uint64_t * workdone; // set by brt_leaf_apply_cmd_once()
};
@ -1659,7 +1664,6 @@ static void setval_fun (const DBT *new_val, void *svextra_v) {
svextra->workdone);
svextra->setval_r = 0;
}
svextra->made_change = TRUE;
}
// We are already past the msn filter (in brt_leaf_put_cmd(), which calls do_update()),
@ -1667,7 +1671,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, bool* made_change,
LEAFENTRY le, OMT snapshot_txnids, OMT live_list_reverse,
uint64_t * workdone) {
LEAFENTRY le_for_update;
DBT key;
@ -1710,7 +1714,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, 0, workdone};
keyp, idx, le_for_update, snapshot_txnids, live_list_reverse, workdone};
// call handlerton's brt->update_fun(), which passes setval_extra to setval_fun()
FAKE_DB(db, desc);
int r = update_fun(
@ -1721,10 +1725,6 @@ static int do_update(brt_update_func update_fun, DESCRIPTOR desc, BRTNODE leafno
setval_fun, &setval_extra
);
*made_change = setval_extra.made_change;
// TODO(leif): ensure that really bad return codes actually cause a
// crash higher up the stack somewhere
if (r == 0) { r = setval_extra.setval_r; }
return r;
}
@ -1738,7 +1738,6 @@ brt_leaf_put_cmd (
BRTNODE leafnode, // bn is within leafnode
BASEMENTNODE bn,
BRT_MSG cmd,
bool* made_change,
uint64_t *workdone,
OMT snapshot_txnids,
OMT live_list_reverse
@ -1754,7 +1753,6 @@ brt_leaf_put_cmd (
u_int32_t omt_size;
int r;
struct cmd_leafval_heaviside_extra be = {compare_fun, desc, cmd->u.id.key};
*made_change = 0;
unsigned int doing_seqinsert = bn->seqinsert;
bn->seqinsert = 0;
@ -1763,7 +1761,6 @@ brt_leaf_put_cmd (
case BRT_INSERT_NO_OVERWRITE:
case BRT_INSERT: {
u_int32_t idx;
*made_change = 1;
if (doing_seqinsert) {
idx = toku_omt_size(bn->buffer);
r = toku_omt_fetch(bn->buffer, idx-1, &storeddatav);
@ -1816,7 +1813,6 @@ brt_leaf_put_cmd (
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);
*made_change = 1;
{
// Now we must find the next leafentry.
@ -1851,9 +1847,6 @@ brt_leaf_put_cmd (
break;
}
case BRT_OPTIMIZE_FOR_UPGRADE:
*made_change = 1;
// TODO 4053: Record version of software that sent the optimize_for_upgrade message, but that goes in the
// node's optimize_for_upgrade field, not in the basement.
// fall through so that optimize_for_upgrade performs rest of the optimize logic
case BRT_COMMIT_BROADCAST_ALL:
case BRT_OPTIMIZE:
@ -1872,7 +1865,6 @@ brt_leaf_put_cmd (
//Item was deleted.
deleted = 1;
}
*made_change = 1;
}
if (deleted)
omt_size--;
@ -1899,7 +1891,6 @@ brt_leaf_put_cmd (
//Item was deleted.
deleted = 1;
}
*made_change = 1;
}
if (deleted)
omt_size--;
@ -1914,10 +1905,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, made_change, workdone);
r = do_update(update_fun, desc, leafnode, bn, cmd, idx, NULL, snapshot_txnids, live_list_reverse, workdone);
} else if (r==0) {
storeddata=storeddatav;
r = do_update(update_fun, desc, leafnode, bn, cmd, idx, storeddata, snapshot_txnids, live_list_reverse, made_change, workdone);
r = do_update(update_fun, desc, leafnode, bn, cmd, idx, storeddata, snapshot_txnids, live_list_reverse, workdone);
} // otherwise, a worse error, just return it
break;
}
@ -1929,7 +1920,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, made_change, workdone);
r = do_update(update_fun, desc, leafnode, bn, cmd, idx, storeddata, snapshot_txnids, live_list_reverse, workdone);
// TODO(leif): This early return means get_leaf_reactivity()
// and VERIFY_NODE() never get called. Is this a problem?
assert(r==0);
@ -2481,7 +2472,6 @@ brtnode_put_cmd (
// and instead defer to these functions
//
if (node->height==0) {
bool made_change = false;
uint64_t workdone = 0;
toku_apply_cmd_to_leaf(
compare_fun,
@ -2489,7 +2479,6 @@ brtnode_put_cmd (
desc,
node,
cmd,
&made_change,
&workdone,
snapshot_txnids,
live_list_reverse
@ -2512,7 +2501,6 @@ void toku_apply_cmd_to_leaf(
DESCRIPTOR desc,
BRTNODE node,
BRT_MSG cmd,
bool *made_change,
uint64_t *workdone,
OMT snapshot_txnids,
OMT live_list_reverse
@ -2556,7 +2544,6 @@ void toku_apply_cmd_to_leaf(
node,
BLB(node, childnum),
cmd,
made_change,
workdone,
snapshot_txnids,
live_list_reverse);
@ -2565,7 +2552,6 @@ void toku_apply_cmd_to_leaf(
}
}
else if (brt_msg_applies_all(cmd)) {
bool bn_made_change = false;
for (int childnum=0; childnum<node->n_children; childnum++) {
if (cmd->msn.msn > BLB(node, childnum)->max_msn_applied.msn) {
BLB(node, childnum)->max_msn_applied = cmd->msn;
@ -2575,11 +2561,9 @@ void toku_apply_cmd_to_leaf(
node,
BLB(node, childnum),
cmd,
&bn_made_change,
workdone,
snapshot_txnids,
live_list_reverse);
if (bn_made_change) *made_change = 1;
} else {
STATUS_VALUE(BRT_MSN_DISCARDS)++;
}
@ -2775,30 +2759,13 @@ toku_brt_hot_index_recovery(TOKUTXN txn, FILENUMS filenums, int do_fsync, int do
return r;
}
static int brt_optimize (BRT brt, BOOL upgrade);
// Effect: Optimize the brt.
int
toku_brt_optimize (BRT brt) {
int r = brt_optimize(brt, FALSE);
return r;
}
int
toku_brt_optimize_for_upgrade (BRT brt) {
int r = brt_optimize(brt, TRUE);
return r;
}
static int
brt_optimize (BRT brt, BOOL upgrade) {
int r = 0;
TXNID oldest = TXNID_NONE_LIVING;
if (!upgrade) {
TOKULOGGER logger = toku_cachefile_logger(brt->cf);
oldest = toku_logger_get_oldest_living_xid(logger, NULL);
}
TOKULOGGER logger = toku_cachefile_logger(brt->cf);
TXNID oldest = toku_logger_get_oldest_living_xid(logger, NULL);
XIDS root_xids = xids_get_root_xids();
XIDS message_xids;
@ -2814,16 +2781,8 @@ brt_optimize (BRT brt, BOOL upgrade) {
DBT val;
toku_init_dbt(&key);
toku_init_dbt(&val);
if (upgrade) {
// maybe there's a better place than the val dbt to put the version, but it seems harmless and is convenient
toku_fill_dbt(&val, &this_version, sizeof(this_version));
BRT_MSG_S brtcmd = { BRT_OPTIMIZE_FOR_UPGRADE, ZERO_MSN, message_xids, .u.id={&key,&val}};
r = toku_brt_root_put_cmd(brt, &brtcmd);
}
else {
BRT_MSG_S brtcmd = { BRT_OPTIMIZE, ZERO_MSN, message_xids, .u.id={&key,&val}};
r = toku_brt_root_put_cmd(brt, &brtcmd);
}
BRT_MSG_S brtcmd = { BRT_OPTIMIZE, ZERO_MSN, message_xids, .u.id={&key,&val}};
r = toku_brt_root_put_cmd(brt, &brtcmd);
xids_destroy(&message_xids);
return r;
}
@ -3305,14 +3264,13 @@ brt_init_header_partial (BRT t, TOKUTXN txn) {
t->h->cf = t->cf;
t->h->nodesize=t->nodesize;
t->h->basementnodesize=t->basementnodesize;
t->h->num_blocks_to_upgrade_13 = 0;
t->h->num_blocks_to_upgrade_14 = 0;
t->h->root_xid_that_created = txn ? txn->ancestor_txnid64 : TXNID_NONE;
t->h->compare_fun = t->compare_fun;
t->h->update_fun = t->update_fun;
t->h->in_memory_stats = ZEROSTATS;
t->h->on_disk_stats = ZEROSTATS;
t->h->checkpoint_staging_stats = ZEROSTATS;
t->h->highest_unused_msn_for_upgrade.msn = MIN_MSN.msn - 1;
BLOCKNUM root = t->h->root_blocknum;
if ((r=setup_initial_brt_root_node(t, root))!=0) { return r; }
@ -3419,7 +3377,17 @@ int toku_read_brt_header_and_store_in_cachefile (BRT brt, CACHEFILE cf, LSN max_
int r;
{
int fd = toku_cachefile_get_and_pin_fd (cf);
r = toku_deserialize_brtheader_from(fd, max_acceptable_lsn, &h);
enum deserialize_error_code e = toku_deserialize_brtheader_from(fd, max_acceptable_lsn, &h);
if (e == DS_XSUM_FAIL) {
fprintf(stderr, "Checksum failure while reading header in file %s.\n", toku_cachefile_fname_in_env(cf));
assert(false); // make absolutely sure we crash before doing anything else
} else if (e == DS_ERRNO) {
r = errno;
} else if (e == DS_OK) {
r = 0;
} else {
assert(false);
}
toku_cachefile_unpin_fd(cf);
}
if (r!=0) return r;
@ -3700,9 +3668,6 @@ brt_open(BRT t, const char *fname_in_env, int is_create, int only_create, CACHET
assert(t->h->dict_id.dictid != DICTIONARY_ID_NONE.dictid);
assert(t->h->dict_id.dictid < dict_id_serial);
r = toku_maybe_upgrade_brt(t); // possibly do some work to complete the version upgrade of brt
if (r!=0) goto died_after_read_and_pin;
// brtheader_note_brt_open must be after all functions that can fail.
r = brtheader_note_brt_open(t);
if (r!=0) goto died_after_read_and_pin;
@ -4797,8 +4762,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) } };
bool made_change;
brt_leaf_put_cmd(t->compare_fun, t->update_fun, &t->h->cmp_descriptor, leafnode, bn, &brtcmd, &made_change, &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); // 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)++;
}
@ -6857,6 +6821,7 @@ toku_brt_header_init(struct brt_header *h,
h->flags = 0;
h->root_xid_that_created = root_xid_that_created;
h->compression_method = compression_method;
h->highest_unused_msn_for_upgrade.msn = MIN_MSN.msn - 1;
}
#include <valgrind/helgrind.h>

View file

@ -115,8 +115,6 @@ int toku_brt_insert (BRT brt, DBT *k, DBT *v, TOKUTXN txn) __attribute__ ((warn
int toku_brt_optimize (BRT brt) __attribute__ ((warn_unused_result));
int toku_brt_optimize_for_upgrade (BRT brt) __attribute__ ((warn_unused_result));
// Effect: Insert a key and data pair into a brt if the oplsn is newer than the brt lsn. This function is called during recovery.
// Returns 0 if successful
int toku_brt_maybe_insert (BRT brt, DBT *k, DBT *v, TOKUTXN txn, BOOL oplsn_valid, LSN oplsn, BOOL do_logging, enum brt_msg_type type) __attribute__ ((warn_unused_result));

View file

@ -23,7 +23,7 @@ enum brt_layout_version_e {
// ALERT ALERT ALERT: version 16 never released to customers, internal and beta use only
BRT_LAYOUT_VERSION_17 = 17, // Dr. No: Add STAT64INFO_S to brt_header
BRT_LAYOUT_VERSION_18 = 18, // Dr. No: Add HOT info to brt_header
BRT_LAYOUT_VERSION_19 = 19, // Doofenshmirtz: Add compression method, msn_for_upgrade, TODO
BRT_LAYOUT_VERSION_19 = 19, // Doofenshmirtz: Add compression method, highest_unused_msn_for_upgrade
BRT_NEXT_VERSION, // the version after the current version
BRT_LAYOUT_VERSION = BRT_NEXT_VERSION-1, // A hack so I don't have to change this line.
BRT_LAYOUT_MIN_SUPPORTED_VERSION = BRT_LAYOUT_VERSION_13, // Minimum version supported

View file

@ -139,7 +139,6 @@ dump_node (int f, BLOCKNUM blocknum, struct brt_header *h) {
printf(" thisnodename=%" PRId64 "\n", n->thisnodename.b);
//printf(" log_lsn =%lld\n", n->log_lsn.lsn); // The log_lsn is a memory-only value.
printf(" height =%d\n", n->height);
printf(" optimized_for_upgrade = %u\n", n->optimized_for_upgrade);
printf(" layout_version=%d\n", n->layout_version);
printf(" layout_version_original=%d\n", n->layout_version_original);
printf(" layout_version_read_from_disk=%d\n", n->layout_version_read_from_disk);

View file

@ -102,7 +102,7 @@ typedef struct __toku_lsn { u_int64_t lsn; } LSN;
* Make the MSN be a struct instead of an integer so that we get better type checking. */
typedef struct __toku_msn { u_int64_t msn; } MSN;
#define ZERO_MSN ((MSN){0}) // dummy used for message construction, to be filled in when msg is applied to tree
#define MIN_MSN ((MSN){(u_int64_t)1000*1000*1000}) // first 1B values reserved for messages created before Dr. No (for upgrade)
#define MIN_MSN ((MSN){(u_int64_t)1 << 62}) // first 2^62 values reserved for messages created before Dr. No (for upgrade)
#define MAX_MSN ((MSN){UINT64_MAX})
/* At the brt layer, a FILENUM uniquely identifies an open file.
@ -273,6 +273,12 @@ enum reactivity {
RE_FISSIBLE
};
enum deserialize_error_code {
DS_OK = 0,
DS_XSUM_FAIL,
DS_ERRNO
};
#if defined(__cplusplus) || defined(__cilkplusplus)
};
#endif

View file

@ -12,6 +12,7 @@
#include <toku_portability.h>
#include "rbuf.h"
#include "x1764.h"
#include "mempool.h"
#if defined(__cplusplus) || defined(__cilkplusplus)
extern "C" {
@ -168,7 +169,9 @@ leafentry_disksize_13(LEAFENTRY_13 le);
int
toku_le_upgrade_13_14(LEAFENTRY_13 old_leafentry, // NULL if there was no stored data.
size_t *new_leafentry_memorysize,
LEAFENTRY *new_leafentry_p);
LEAFENTRY *new_leafentry_p,
OMT omt,
struct mempool *mp);
#if defined(__cplusplus) || defined(__cilkplusplus)

View file

@ -1,4 +1,4 @@
/* -*- mode: C; c-basic-offset: 4 -*- */
/* -*- mode: C; c-basic-offset: 4; indent-tabs-mode: nil -*- */
#ident "$Id$"
#ident "Copyright (c) 2007-2011 Tokutek Inc. All rights reserved."
@ -74,6 +74,7 @@ int
test_main (int argc __attribute__((__unused__)), const char *argv[] __attribute__((__unused__))) {
unsigned long eltsize, nodesize, repeat;
initialize_dummymsn();
if (argc != 4) {
fprintf(stderr, "Usage: %s <eltsize> <nodesize> <repeat>\n", argv[0]);
return 2;

View file

@ -301,7 +301,6 @@ test_prefetching(void) {
sn.layout_version = BRT_LAYOUT_VERSION;
sn.layout_version_original = BRT_LAYOUT_VERSION;
sn.height = 1;
sn.optimized_for_upgrade = 1234;
sn.n_children = 3;
sn.dirty = 1;

View file

@ -1,4 +1,4 @@
/* -*- mode: C; c-basic-offset: 4 -*- */
/* -*- mode: C; c-basic-offset: 4; indent-tabs-mode: nil -*- */
#ident "$Id$"
#ident "Copyright (c) 2007, 2008 Tokutek Inc. All rights reserved."
@ -243,7 +243,6 @@ test_serialize_nonleaf(void) {
sn.layout_version = BRT_LAYOUT_VERSION;
sn.layout_version_original = BRT_LAYOUT_VERSION;
sn.height = 1;
sn.optimized_for_upgrade = 1234;
sn.n_children = 2;
sn.dirty = 1;
hello_string = toku_strdup("hello");
@ -344,7 +343,6 @@ test_serialize_leaf(void) {
sn.layout_version = BRT_LAYOUT_VERSION;
sn.layout_version_original = BRT_LAYOUT_VERSION;
sn.height = 0;
sn.optimized_for_upgrade = 1234;
sn.n_children = 2;
sn.dirty = 1;
LEAFENTRY elts[3];
@ -424,6 +422,7 @@ test_serialize_leaf(void) {
int
test_main (int argc __attribute__((__unused__)), const char *argv[] __attribute__((__unused__))) {
initialize_dummymsn();
test_serialize_nonleaf();
test_serialize_leaf();

View file

@ -1,4 +1,4 @@
/* -*- mode: C; c-basic-offset: 4 -*- */
/* -*- mode: C; c-basic-offset: 4; indent-tabs-mode: nil -*- */
#ident "$Id$"
#ident "Copyright (c) 2007, 2008 Tokutek Inc. All rights reserved."
@ -67,7 +67,6 @@ test_serialize_leaf(int valsize, int nelts, double entropy) {
sn.layout_version = BRT_LAYOUT_VERSION;
sn.layout_version_original = BRT_LAYOUT_VERSION;
sn.height = 0;
sn.optimized_for_upgrade = 1234;
sn.n_children = 8;
sn.dirty = 1;
MALLOC_N(sn.n_children, sn.bp);
@ -194,7 +193,6 @@ test_serialize_nonleaf(int valsize, int nelts, double entropy) {
sn.layout_version = BRT_LAYOUT_VERSION;
sn.layout_version_original = BRT_LAYOUT_VERSION;
sn.height = 1;
sn.optimized_for_upgrade = 1234;
sn.n_children = 8;
sn.dirty = 1;
MALLOC_N(sn.n_children, sn.bp);
@ -321,6 +319,7 @@ test_main (int argc __attribute__((__unused__)), const char *argv[] __attribute_
valsize = strtol(argv[1], NULL, 0);
nelts = strtol(argv[2], NULL, 0);
initialize_dummymsn();
test_serialize_leaf(valsize, nelts, entropy);
test_serialize_nonleaf(valsize, nelts, entropy);

View file

@ -1,4 +1,4 @@
/* -*- mode: C; c-basic-offset: 4 -*- */
/* -*- mode: C; c-basic-offset: 4; indent-tabs-mode: nil -*- */
#ident "$Id$"
#ident "Copyright (c) 2007, 2008 Tokutek Inc. All rights reserved."
@ -220,7 +220,6 @@ test_serialize_leaf_check_msn(enum brtnode_verify_type bft, BOOL do_clone) {
sn.layout_version = BRT_LAYOUT_VERSION;
sn.layout_version_original = BRT_LAYOUT_VERSION;
sn.height = 0;
sn.optimized_for_upgrade = 1234;
sn.n_children = 2;
sn.dirty = 1;
MALLOC_N(sn.n_children, sn.bp);
@ -290,7 +289,6 @@ test_serialize_leaf_check_msn(enum brtnode_verify_type bft, BOOL do_clone) {
assert(dn->layout_version_original ==BRT_LAYOUT_VERSION);
assert(dn->layout_version_read_from_disk ==BRT_LAYOUT_VERSION);
assert(dn->height == 0);
assert(dn->optimized_for_upgrade == 1234);
assert(dn->n_children>=1);
assert(dn->max_msn_applied_to_node_on_disk.msn == POSTSERIALIZE_MSN_ON_DISK.msn);
{
@ -364,7 +362,6 @@ test_serialize_leaf_with_large_pivots(enum brtnode_verify_type bft, BOOL do_clon
sn.layout_version = BRT_LAYOUT_VERSION;
sn.layout_version_original = BRT_LAYOUT_VERSION;
sn.height = 0;
sn.optimized_for_upgrade = 1234;
sn.n_children = nrows;
sn.dirty = 1;
@ -508,7 +505,6 @@ test_serialize_leaf_with_many_rows(enum brtnode_verify_type bft, BOOL do_clone)
sn.layout_version = BRT_LAYOUT_VERSION;
sn.layout_version_original = BRT_LAYOUT_VERSION;
sn.height = 0;
sn.optimized_for_upgrade = 1234;
sn.n_children = 1;
sn.dirty = 1;
@ -649,7 +645,6 @@ test_serialize_leaf_with_large_rows(enum brtnode_verify_type bft, BOOL do_clone)
sn.layout_version = BRT_LAYOUT_VERSION;
sn.layout_version_original = BRT_LAYOUT_VERSION;
sn.height = 0;
sn.optimized_for_upgrade = 1234;
sn.n_children = 1;
sn.dirty = 1;
@ -798,7 +793,6 @@ test_serialize_leaf_with_empty_basement_nodes(enum brtnode_verify_type bft, BOOL
sn.layout_version = BRT_LAYOUT_VERSION;
sn.layout_version_original = BRT_LAYOUT_VERSION;
sn.height = 0;
sn.optimized_for_upgrade = 1234;
sn.n_children = 7;
sn.dirty = 1;
MALLOC_N(sn.n_children, sn.bp);
@ -879,7 +873,6 @@ test_serialize_leaf_with_empty_basement_nodes(enum brtnode_verify_type bft, BOOL
assert(dn->layout_version_original ==BRT_LAYOUT_VERSION);
assert(dn->layout_version_read_from_disk ==BRT_LAYOUT_VERSION);
assert(dn->height == 0);
assert(dn->optimized_for_upgrade == 1234);
assert(dn->n_children>0);
{
// Man, this is way too ugly. This entire test suite needs to be refactored.
@ -949,7 +942,6 @@ test_serialize_leaf_with_multiple_empty_basement_nodes(enum brtnode_verify_type
sn.layout_version = BRT_LAYOUT_VERSION;
sn.layout_version_original = BRT_LAYOUT_VERSION;
sn.height = 0;
sn.optimized_for_upgrade = 1234;
sn.n_children = 4;
sn.dirty = 1;
MALLOC_N(sn.n_children, sn.bp);
@ -1006,7 +998,6 @@ test_serialize_leaf_with_multiple_empty_basement_nodes(enum brtnode_verify_type
assert(dn->layout_version_original ==BRT_LAYOUT_VERSION);
assert(dn->layout_version_read_from_disk ==BRT_LAYOUT_VERSION);
assert(dn->height == 0);
assert(dn->optimized_for_upgrade == 1234);
assert(dn->n_children == 1);
{
const u_int32_t npartitions = dn->n_children;
@ -1069,7 +1060,6 @@ test_serialize_leaf(enum brtnode_verify_type bft, BOOL do_clone) {
sn.layout_version = BRT_LAYOUT_VERSION;
sn.layout_version_original = BRT_LAYOUT_VERSION;
sn.height = 0;
sn.optimized_for_upgrade = 1234;
sn.n_children = 2;
sn.dirty = 1;
MALLOC_N(sn.n_children, sn.bp);
@ -1135,7 +1125,6 @@ test_serialize_leaf(enum brtnode_verify_type bft, BOOL do_clone) {
assert(dn->layout_version_original ==BRT_LAYOUT_VERSION);
assert(dn->layout_version_read_from_disk ==BRT_LAYOUT_VERSION);
assert(dn->height == 0);
assert(dn->optimized_for_upgrade == 1234);
assert(dn->n_children>=1);
{
// Man, this is way too ugly. This entire test suite needs to be refactored.
@ -1211,7 +1200,6 @@ test_serialize_nonleaf(enum brtnode_verify_type bft, BOOL do_clone) {
sn.layout_version = BRT_LAYOUT_VERSION;
sn.layout_version_original = BRT_LAYOUT_VERSION;
sn.height = 1;
sn.optimized_for_upgrade = 1234;
sn.n_children = 2;
sn.dirty = 1;
hello_string = toku_strdup("hello");
@ -1282,7 +1270,6 @@ test_serialize_nonleaf(enum brtnode_verify_type bft, BOOL do_clone) {
assert(dn->layout_version_original ==BRT_LAYOUT_VERSION);
assert(dn->layout_version_read_from_disk ==BRT_LAYOUT_VERSION);
assert(dn->height == 1);
assert(dn->optimized_for_upgrade == 1234);
assert(dn->n_children==2);
assert(strcmp(kv_pair_key(dn->childkeys[0]), "hello")==0);
assert(toku_brt_pivot_key_len(dn->childkeys[0])==6);
@ -1395,6 +1382,8 @@ test_serialize_nonleaf(enum brtnode_verify_type bft, BOOL do_clone) {
int
test_main (int argc __attribute__((__unused__)), const char *argv[] __attribute__((__unused__))) {
initialize_dummymsn();
test_serialize_leaf(read_none, FALSE);
test_serialize_leaf(read_all, FALSE);
test_serialize_leaf(read_compressed, FALSE);

View file

@ -30,8 +30,6 @@ static void test_header (void) {
h->layout_version_original = 13;
h->layout_version_read_from_disk = 14;
h->build_id_original = 1234;
h->num_blocks_to_upgrade_13 = 1013;
h->num_blocks_to_upgrade_14 = 1014;
h->in_memory_stats = (STAT64INFO_S) {10, 11};
h->on_disk_stats = (STAT64INFO_S) {20, 21};
h->checkpoint_staging_stats = (STAT64INFO_S) {30, 31};
@ -51,8 +49,6 @@ static void test_header (void) {
assert(h->layout_version_original == 13);
assert(h->layout_version_read_from_disk == BRT_LAYOUT_VERSION);
assert(h->build_id_original == 1234);
assert(h->num_blocks_to_upgrade_13 == 1013);
assert(h->num_blocks_to_upgrade_14 == 1014);
assert(h->in_memory_stats.numrows == expected_stats.numrows);
assert(h->on_disk_stats.numbytes == expected_stats.numbytes);
r = toku_close_brt_nolsn(t, 0); assert(r==0);

View file

@ -1,3 +1,4 @@
/* -*- mode: C; c-basic-offset: 4; indent-tabs-mode: nil -*- */
#ident "$Id$"
#include "includes.h"
@ -89,6 +90,7 @@ test_fifo_enq (int n) {
int
test_main(int argc, const char *argv[]) {
default_parse_args(argc, argv);
initialize_dummymsn();
test_fifo_create();
test_fifo_enq(4);
test_fifo_enq(512);

View file

@ -1,4 +1,4 @@
/* -*- mode: C; c-basic-offset: 4 -*- */
/* -*- mode: C; c-basic-offset: 4; indent-tabs-mode: nil -*- */
#ident "$Id$"
#ident "Copyright (c) 2011 Tokutek Inc. All rights reserved."
@ -171,6 +171,7 @@ test_main (int argc , const char *argv[]) {
int fanout = 2;
int nperleaf = 8;
int do_verify = 1;
initialize_dummymsn();
for (int i = 1; i < argc; i++) {
const char *arg = argv[i];
if (strcmp(arg, "-v") == 0) {

View file

@ -1,4 +1,4 @@
/* -*- mode: C; c-basic-offset: 4 -*- */
/* -*- mode: C; c-basic-offset: 4; indent-tabs-mode: nil -*- */
#ident "$Id$"
#ident "Copyright (c) 2011 Tokutek Inc. All rights reserved."
@ -46,9 +46,8 @@ append_leaf(BRT brt, BRTNODE leafnode, void *key, size_t keylen, void *val, size
MSN msn = next_dummymsn();
BRT_MSG_S cmd = { BRT_INSERT, msn, xids_get_root_xids(), .u.id = { &thekey, &theval } };
bool made_change;
u_int64_t workdone=0;
toku_apply_cmd_to_leaf(brt->compare_fun, brt->update_fun, &brt->h->cmp_descriptor, leafnode, &cmd, &made_change, &workdone, NULL, NULL);
toku_apply_cmd_to_leaf(brt->compare_fun, brt->update_fun, &brt->h->cmp_descriptor, leafnode, &cmd, &workdone, NULL, NULL);
{
int r = toku_brt_lookup(brt, &thekey, lookup_checkf, &pair);
assert(r==0);
@ -56,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, &made_change, &workdone, NULL, NULL);
toku_apply_cmd_to_leaf(brt->compare_fun, brt->update_fun, &brt->h->cmp_descriptor, leafnode, &badcmd, &workdone, NULL, NULL);
// message should be rejected for duplicate msn, row should still have original val
@ -69,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, &made_change, &workdone, NULL, NULL);
toku_apply_cmd_to_leaf(brt->compare_fun, brt->update_fun, &brt->h->cmp_descriptor, leafnode, &cmd2, &workdone, NULL, NULL);
// message should be accepted, val should have new value
{
@ -81,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, &made_change, &workdone, NULL, NULL);
toku_apply_cmd_to_leaf(brt->compare_fun, brt->update_fun, &brt->h->cmp_descriptor, leafnode, &cmd3, &workdone, NULL, NULL);
// message should be rejected, val should still have value in pair2
{
@ -165,6 +164,7 @@ usage(void) {
int
test_main (int argc , const char *argv[]) {
int do_verify = 1;
initialize_dummymsn();
for (int i = 1; i < argc; i++) {
const char *arg = argv[i];
if (strcmp(arg, "-v") == 0) {

View file

@ -127,8 +127,7 @@ insert_random_message_to_leaf(BRT t, BRTNODE leafnode, BASEMENTNODE blb, LEAFENT
int64_t numbytes;
int r = apply_msg_to_leafentry(&msg, NULL, &memsize, save, NULL, NULL, NULL, NULL, NULL, &numbytes);
assert_zero(r);
bool made_change;
brt_leaf_put_cmd(t->compare_fun, t->update_fun, NULL, leafnode, blb, &msg, &made_change, NULL, NULL, NULL);
brt_leaf_put_cmd(t->compare_fun, t->update_fun, NULL, leafnode, blb, &msg, NULL, NULL, NULL);
if (msn.msn > blb->max_msn_applied.msn) {
blb->max_msn_applied = msn;
}
@ -169,12 +168,11 @@ insert_same_message_to_leaves(BRT t, BRTNODE child1, BASEMENTNODE blb1, BRTNODE
int64_t numbytes;
int r = apply_msg_to_leafentry(&msg, NULL, &memsize, save, NULL, NULL, NULL, NULL, NULL, &numbytes);
assert_zero(r);
bool made_change;
brt_leaf_put_cmd(t->compare_fun, t->update_fun, NULL, child1, blb1, &msg, &made_change, NULL, NULL, NULL);
brt_leaf_put_cmd(t->compare_fun, t->update_fun, NULL, child1, blb1, &msg, 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, &made_change, NULL, NULL, NULL);
brt_leaf_put_cmd(t->compare_fun, t->update_fun, NULL, child2, blb2, &msg, NULL, NULL, NULL);
if (msn.msn > blb2->max_msn_applied.msn) {
blb2->max_msn_applied = msn;
}
@ -586,8 +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]) {
bool made_change;
toku_apply_cmd_to_leaf(t->compare_fun, t->update_fun, &t->h->descriptor, child, parent_messages[i], &made_change, NULL, NULL, NULL);
toku_apply_cmd_to_leaf(t->compare_fun, t->update_fun, &t->h->descriptor, child, parent_messages[i], NULL, NULL, NULL);
}
}
for (i = 0; i < 8; ++i) {
@ -811,8 +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]) {
bool made_change;
toku_apply_cmd_to_leaf(t->compare_fun, t->update_fun, &t->h->descriptor, child, parent_messages[i], &made_change, NULL, NULL, NULL);
toku_apply_cmd_to_leaf(t->compare_fun, t->update_fun, &t->h->descriptor, child, parent_messages[i], NULL, NULL, NULL);
}
}
for (i = 0; i < 8; ++i) {
@ -999,9 +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]) {
bool made_change;
toku_apply_cmd_to_leaf(t->compare_fun, t->update_fun, &t->h->descriptor, child1, parent_messages[i], &made_change, NULL, NULL, NULL);
toku_apply_cmd_to_leaf(t->compare_fun, t->update_fun, &t->h->descriptor, child2, parent_messages[i], &made_change, NULL, NULL, NULL);
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);
}
}
for (i = 0; i < 8; ++i) {
@ -1124,6 +1119,7 @@ int
test_main (int argc, const char *argv[]) {
parse_args(argc, argv);
initialize_dummymsn();
int r;
CACHETABLE ct;
r = toku_brt_create_cachetable(&ct, 0, ZERO_LSN, NULL_LOGGER);

View file

@ -27,7 +27,7 @@ const ITEMLEN len_ignore = 0xFFFFFFFF;
// dummymsn needed to simulate msn because test messages are injected at a lower level than toku_brt_root_put_cmd()
#define MIN_DUMMYMSN ((MSN) {(uint64_t)1<<48})
#define MIN_DUMMYMSN ((MSN) {(uint64_t)1<<62})
static MSN dummymsn;
static int dummymsn_initialized = 0;

View file

@ -1,4 +1,4 @@
/* -*- mode: C; c-basic-offset: 4 -*- */
/* -*- mode: C; c-basic-offset: 4; indent-tabs-mode: nil -*- */
#ident "Copyright (c) 2007, 2008 Tokutek Inc. All rights reserved."
#ident "Id:"
@ -74,6 +74,7 @@ static void test_3748 (void) {
int
test_main(int argc, const char *argv[]) {
default_parse_args(argc, argv);
initialize_dummymsn();
test_3748();
return 0;

View file

@ -85,7 +85,6 @@ setup_brtnode_header(struct brtnode *node)
node->layout_version = BRT_LAYOUT_VERSION;
node->layout_version_original = BRT_LAYOUT_VERSION;
node->height = 0;
node->optimized_for_upgrade = 1324;
node->dirty = 1;
node->totalchildkeylens = 0;
}

View file

@ -0,0 +1,163 @@
/* -*- mode: C; c-basic-offset: 4 -*- */
#ident "$Id$"
#ident "Copyright (c) 2007-2012 Tokutek Inc. All rights reserved."
#include <unistd.h>
#include <stdlib.h>
#include <sys/time.h>
#include "test.h"
#include "brt-flusher.h"
#include "includes.h"
static TOKUTXN const null_txn = NULL;
static DB * const null_db = NULL;
static int
noop_getf(ITEMLEN UU(keylen), bytevec UU(key), ITEMLEN UU(vallen), bytevec UU(val), void *extra, bool UU(lock_only))
{
int *calledp = extra;
(*calledp)++;
return 0;
}
static int
get_one_value(BRT t, CACHETABLE UU(ct), void *UU(extra))
{
int r;
int called;
BRT_CURSOR cursor;
r = toku_brt_cursor(t, &cursor, null_txn, false, false);
CKERR(r);
called = 0;
r = toku_brt_cursor_first(cursor, noop_getf, &called);
CKERR(r);
assert(called == 1);
r = toku_brt_cursor_close(cursor);
CKERR(r);
return 0;
}
static int
progress(void *extra, float fraction)
{
float *stop_at = extra;
if (fraction > *stop_at) {
return 1;
} else {
return 0;
}
}
static int
do_hot_optimize(BRT t, CACHETABLE UU(ct), void *extra)
{
float *fraction = extra;
int r = toku_brt_hot_optimize(t, progress, extra);
if (*fraction < 1.0) {
CKERR2(r, 1);
} else {
CKERR(r);
}
return 0;
}
static int
insert_something(BRT t, CACHETABLE UU(ct), void *UU(extra))
{
assert(t);
return 0;
}
static int
scan_tree(BRT t, CACHETABLE UU(ct), void *UU(extra))
{
assert(t);
return 0;
}
typedef int (*tree_cb)(BRT t, CACHETABLE ct, void *extra);
static int
with_open_tree(const char *fname, tree_cb cb, void *cb_extra)
{
int r, r2;
BRT t;
CACHETABLE ct;
r = toku_brt_create_cachetable(&ct, 16*(1<<20), ZERO_LSN, NULL_LOGGER);
CKERR(r);
r = toku_open_brt(fname,
0,
&t,
4*(1<<20),
128*(1<<10),
ct,
null_txn,
toku_builtin_compare_fun,
null_db);
CKERR(r);
r2 = cb(t, ct, cb_extra);
r = toku_close_brt_nolsn(t, 0);
CKERR(r);
r = toku_cachetable_close(&ct);
CKERR(r);
return r2;
}
#define TMPBRTFMT "%s-tmpdata.brt"
static const char *origbrt_5_0 = "upgrade_test_data.brt.5.0";
static const char *origbrt_4_2 = "upgrade_test_data.brt.4.2";
static int
run_test(const char *prog, const char *origbrt) {
int r;
size_t templen = strlen(prog) + strlen(TMPBRTFMT) - 2;
char tempbrt[templen + 1];
snprintf(tempbrt, templen + 1, TMPBRTFMT, prog);
{
size_t len = 4 + strlen(origbrt) + strlen(tempbrt);
char buf[len + 1];
snprintf(buf, len + 1, "cp %s %s", origbrt, tempbrt);
r = system(buf);
CKERR(r);
}
r = with_open_tree(tempbrt, get_one_value, NULL);
CKERR(r);
r = with_open_tree(tempbrt, insert_something, NULL);
CKERR(r);
float fraction = 0.5;
r = with_open_tree(tempbrt, do_hot_optimize, &fraction);
CKERR(r);
fraction = 1.0;
r = with_open_tree(tempbrt, do_hot_optimize, &fraction);
CKERR(r);
r = with_open_tree(tempbrt, scan_tree, NULL);
CKERR(r);
r = unlink(tempbrt);
CKERR(r);
return r;
}
int
test_main(int argc __attribute__((__unused__)), const char *argv[])
{
int r;
r = run_test(argv[0], origbrt_5_0);
CKERR(r);
r = run_test(argv[0], origbrt_4_2);
CKERR(r);
return r;
}

View file

@ -1,4 +1,4 @@
/* -*- mode: C; c-basic-offset: 4 -*- */
/* -*- mode: C; c-basic-offset: 4; indent-tabs-mode: nil -*- */
#ident "$Id$"
#ident "Copyright (c) 2011 Tokutek Inc. All rights reserved."
@ -174,6 +174,7 @@ usage(void) {
int
test_main (int argc , const char *argv[]) {
initialize_dummymsn();
int height = 1;
int fanout = 2;
int nperleaf = 8;

View file

@ -1,4 +1,4 @@
/* -*- mode: C; c-basic-offset: 4 -*- */
/* -*- mode: C; c-basic-offset: 4; indent-tabs-mode: nil -*- */
#ident "$Id$"
#ident "Copyright (c) 2011 Tokutek Inc. All rights reserved."
@ -145,6 +145,7 @@ test_main (int argc , const char *argv[]) {
int fanout = 2;
int nperleaf = 8;
int do_verify = 1;
initialize_dummymsn();
for (int i = 1; i < argc; i++) {
const char *arg = argv[i];
if (strcmp(arg, "-v") == 0) {

View file

@ -1,4 +1,4 @@
/* -*- mode: C; c-basic-offset: 4 -*- */
/* -*- mode: C; c-basic-offset: 4; indent-tabs-mode: nil -*- */
#ident "$Id$"
#ident "Copyright (c) 2011 Tokutek Inc. All rights reserved."
@ -100,6 +100,7 @@ usage(void) {
int
test_main (int argc , const char *argv[]) {
int do_verify = 1;
initialize_dummymsn();
for (int i = 1; i < argc; i++) {
const char *arg = argv[i];
if (strcmp(arg, "-v") == 0) {

View file

@ -1,4 +1,4 @@
/* -*- mode: C; c-basic-offset: 4 -*- */
/* -*- mode: C; c-basic-offset: 4; indent-tabs-mode: nil -*- */
#ident "$Id$"
#ident "Copyright (c) 2011 Tokutek Inc. All rights reserved."
@ -145,6 +145,7 @@ test_main (int argc , const char *argv[]) {
int fanout = 3;
int nperleaf = 8;
int do_verify = 1;
initialize_dummymsn();
for (int i = 1; i < argc; i++) {
const char *arg = argv[i];
if (strcmp(arg, "-v") == 0) {

View file

@ -1,4 +1,4 @@
/* -*- mode: C; c-basic-offset: 4 -*- */
/* -*- mode: C; c-basic-offset: 4; indent-tabs-mode: nil -*- */
#ident "$Id$"
#ident "Copyright (c) 2011 Tokutek Inc. All rights reserved."
@ -160,6 +160,7 @@ test_main (int argc , const char *argv[]) {
int fanout = 2;
int nperleaf = 8;
int do_verify = 1;
initialize_dummymsn();
for (int i = 1; i < argc; i++) {
const char *arg = argv[i];
if (strcmp(arg, "-v") == 0) {

View file

@ -1,4 +1,4 @@
/* -*- mode: C; c-basic-offset: 4 -*- */
/* -*- mode: C; c-basic-offset: 4; indent-tabs-mode: nil -*- */
#ident "$Id$"
#ident "Copyright (c) 2011 Tokutek Inc. All rights reserved."
@ -100,6 +100,7 @@ usage(void) {
int
test_main (int argc , const char *argv[]) {
int do_verify = 1;
initialize_dummymsn();
for (int i = 1; i < argc; i++) {
const char *arg = argv[i];
if (strcmp(arg, "-v") == 0) {

View file

@ -1,4 +1,4 @@
/* -*- mode: C; c-basic-offset: 4 -*- */
/* -*- mode: C; c-basic-offset: 4; indent-tabs-mode: nil -*- */
#ident "$Id$"
#ident "Copyright (c) 2011 Tokutek Inc. All rights reserved."
@ -145,6 +145,7 @@ test_main (int argc , const char *argv[]) {
int fanout = 3;
int nperleaf = 8;
int do_verify = 1;
initialize_dummymsn();
for (int i = 1; i < argc; i++) {
const char *arg = argv[i];
if (strcmp(arg, "-v") == 0) {

View file

@ -2326,15 +2326,20 @@ leafentry_disksize_13(LEAFENTRY_13 le) {
int
toku_le_upgrade_13_14(LEAFENTRY_13 old_leafentry,
size_t *new_leafentry_memorysize,
LEAFENTRY *new_leafentry_p) {
LEAFENTRY *new_leafentry_p,
OMT omt,
struct mempool *mp) {
ULE_S ule;
int rval;
invariant(old_leafentry);
le_unpack_13(&ule, old_leafentry);
// We used to pass NULL for omt and mempool, so that we would use
// malloc instead of a mempool. However after supporting upgrade,
// we need to use mempools and the OMT.
rval = le_pack(&ule, // create packed leafentry
new_leafentry_memorysize,
new_leafentry_p,
NULL, NULL, NULL); // NULL for omt means that we use malloc instead of mempool
omt, mp, NULL);
ule_cleanup(&ule);
return rval;
}

View file

@ -54,6 +54,7 @@ enum {ROWS_PER_TRANSACTION=10000};
uint NUM_DBS=1;
uint NUM_ROWS=100000;
int CHECK_RESULTS=0;
int optimize=0;
int littlenode = 0;
enum { old_default_cachesize=1024 }; // MB
int CACHESIZE=old_default_cachesize;
@ -136,6 +137,11 @@ static void preload_dbs(DB **dbs)
nested_insert(dbs, 0, NULL, row, generated_value);
}
if (optimize) {
if (verbose) { printf("\noptimizing");fflush(stdout);}
do_hot_optimize_on_dbs(env, dbs, 1);
}
if ( CHECK_RESULTS) {
if ( verbose ) {printf("\nchecking");fflush(stdout);}
check_results_nested(&dbs[0], NUM_ROWS);
@ -288,6 +294,8 @@ static void do_args(int argc, char * const argv[]) {
CHECK_RESULTS = 1;
} else if (strcmp(argv[0], "-n")==0) {
littlenode = 1;
} else if (strcmp(argv[0], "-o")==0) {
optimize = 1;
} else {
fprintf(stderr, "Unknown arg: %s\n", argv[0]);
resultcode=1;

View file

@ -20,6 +20,7 @@ enum {ROWS_PER_TRANSACTION=10000};
int NUM_DBS=5;
int NUM_ROWS=100000;
int CHECK_RESULTS=0;
int optimize=0;
int littlenode = 0;
enum { old_default_cachesize=1024 }; // MB
int CACHESIZE=old_default_cachesize;
@ -71,6 +72,11 @@ static void preload_dbs(DB **dbs)
if ( key.flags ) { toku_free(key.data); key.data = NULL; }
if ( val.flags ) { toku_free(val.data); key.data = NULL; }
if (optimize) {
if (verbose) { printf("\noptimizing");fflush(stdout);}
do_hot_optimize_on_dbs(env, dbs, NUM_DBS);
}
if ( CHECK_RESULTS) {
if ( verbose ) {printf("\nchecking");fflush(stdout);}
check_results(env, dbs, NUM_DBS, NUM_ROWS);
@ -193,6 +199,8 @@ static void do_args(int argc, char * const argv[]) {
CHECK_RESULTS = 1;
} else if (strcmp(argv[0], "-n")==0) {
littlenode = 1;
} else if (strcmp(argv[0], "-o")==0) {
optimize = 1;
} else {
fprintf(stderr, "Unknown arg: %s\n", argv[0]);
resultcode=1;

View file

@ -85,6 +85,21 @@ pkey_for_val(int key, int i) {
}
static int __attribute__((unused))
dummy_progress(void *UU(extra), float UU(progress))
{
return 0;
}
static void __attribute__((unused))
do_hot_optimize_on_dbs(DB_ENV *UU(env), DB **dbs, int num_dbs)
{
for (int i = 0; i < num_dbs; ++i) {
int r = dbs[i]->hot_optimize(dbs[i], dummy_progress, NULL);
CKERR(r);
}
}
// don't check first n rows (expect to have been deleted)
static void UU()
check_results_after_row_n(DB_ENV *env, DB **dbs, const int num_dbs, const int num_rows, const int first_row_to_check) {

View file

@ -31,6 +31,16 @@ char *db_v4_dir = OLDDATADIR "env_preload.4.2.0.cleanshutdown";
char *db_v4_dir_node4k = OLDDATADIR "env_preload.4.2.0.node4k.cleanshutdown";
char *db_v4_dir_flat = OLDDATADIR "env_preload.4.2.0.flat.cleanshutdown";
// HACK: Newer versions of the database/brt to use with this old
// upgrade test code.
char *db_v6_dir = OLDDATADIR "env_preload.5.0.8.cleanshutdown";
char *db_v6_dir_node4k = OLDDATADIR "env_preload.5.0.8.node4k.cleanshutdown";
char *db_v6_dir_flat = OLDDATADIR "env_preload.5.0.8.flat.cleanshutdown";
char *db_v7_dir = OLDDATADIR "env_preload.5.2.7.cleanshutdown";
char *db_v7_dir_node4k = OLDDATADIR "env_preload.5.2.7.node4k.cleanshutdown";
char *db_v7_dir_flat = OLDDATADIR "env_preload.5.2.7.flat.cleanshutdown";
// should put this in test.h:
static __attribute__((__unused__)) int
@ -98,6 +108,24 @@ static void setup(void) {
else if ( SRC_VERSION == 5 ) {
src_db_dir = db_v5_dir;
}
else if (SRC_VERSION == 6) {
if (flat) {
src_db_dir = db_v6_dir_flat;
} else if (littlenode) {
src_db_dir = db_v6_dir_node4k;
} else {
src_db_dir = db_v6_dir;
}
}
else if (SRC_VERSION == 7) {
if (flat) {
src_db_dir = db_v7_dir_flat;
} else if (littlenode) {
src_db_dir = db_v7_dir_node4k;
} else {
src_db_dir = db_v7_dir;
}
}
else {
fprintf(stderr, "unsupported TokuDB version %d to upgrade\n", SRC_VERSION);
assert(0);