Commit f2c4fe13 authored by Zardosht Kasheff's avatar Zardosht Kasheff Committed by Yoni Fogel

[t:4875], [t:4887], merge from tokudb.4875 to main

git-svn-id: file:///svn/toku/tokudb@43896 c7de825b-a66e-492c-adef-691d508d4ae1
parent 939721e7
......@@ -84,15 +84,15 @@ static inline void unlock_for_blocktable (BLOCK_TABLE bt);
static void
ft_set_dirty(FT h, BOOL for_checkpoint){
assert(toku_mutex_is_locked(&h->blocktable->mutex));
assert(h->type == FT_CURRENT);
ft_set_dirty(FT ft, BOOL for_checkpoint){
assert(toku_mutex_is_locked(&ft->blocktable->mutex));
assert(ft->h->type == FT_CURRENT);
if (for_checkpoint) {
assert(h->checkpoint_header->type == FT_CHECKPOINT_INPROGRESS);
h->checkpoint_header->dirty = 1;
assert(ft->checkpoint_header->type == FT_CHECKPOINT_INPROGRESS);
ft->checkpoint_header->dirty = 1;
}
else {
h->dirty = 1;
ft->h->dirty = 1;
}
}
......@@ -449,9 +449,9 @@ PRNTF("blokAllokator", 1L, size, offset, bt);
//Fills wbuf with bt
//A clean shutdown runs checkpoint start so that current and inprogress are copies.
void
toku_serialize_translation_to_wbuf_unlocked(BLOCK_TABLE bt, struct wbuf *w,
toku_serialize_translation_to_wbuf(BLOCK_TABLE bt, struct wbuf *w,
int64_t *address, int64_t *size) {
assert(toku_mutex_is_locked(&bt->mutex));
lock_for_blocktable(bt);
struct translation *t = &bt->inprogress;
BLOCKNUM b = make_blocknum(RESERVED_BLOCKNUM_TRANSLATION);
......@@ -478,6 +478,7 @@ toku_serialize_translation_to_wbuf_unlocked(BLOCK_TABLE bt, struct wbuf *w,
wbuf_int(w, checksum);
*address = t->block_translation[b.b].u.diskoff;
*size = t->block_translation[b.b].size;
unlock_for_blocktable(bt);
}
......
......@@ -52,7 +52,7 @@ void toku_blocknum_realloc_on_disk(BLOCK_TABLE bt, BLOCKNUM b, DISKOFF size, DIS
void toku_translate_blocknum_to_offset_size(BLOCK_TABLE bt, BLOCKNUM b, DISKOFF *offset, DISKOFF *size);
//Serialization
void toku_serialize_translation_to_wbuf_unlocked(BLOCK_TABLE bt, struct wbuf *w, int64_t *address, int64_t *size);
void toku_serialize_translation_to_wbuf(BLOCK_TABLE bt, struct wbuf *w, int64_t *address, int64_t *size);
void toku_block_table_swap_for_redirect(BLOCK_TABLE old_bt, BLOCK_TABLE new_bt);
......
......@@ -281,11 +281,13 @@ toku_checkpoint(CACHETABLE ct, TOKULOGGER logger,
multi_operation_checkpoint_lock();
SET_CHECKPOINT_FOOTPRINT(20);
ydb_lock();
toku_ft_open_close_lock();
SET_CHECKPOINT_FOOTPRINT(30);
STATUS_VALUE(CP_TIME_LAST_CHECKPOINT_BEGIN) = time(NULL);
r = toku_cachetable_begin_checkpoint(ct, logger);
toku_ft_open_close_unlock();
multi_operation_checkpoint_unlock();
ydb_unlock();
......
......@@ -65,7 +65,7 @@ cachetable_put_empty_node_with_dep_nodes(
void
create_new_ftnode_with_dep_nodes(
FT h,
FT ft,
FTNODE *result,
int height,
int n_children,
......@@ -76,15 +76,15 @@ create_new_ftnode_with_dep_nodes(
BLOCKNUM name;
cachetable_put_empty_node_with_dep_nodes(
h,
ft,
num_dependent_nodes,
dependent_nodes,
&name,
&fullhash,
result);
assert(h->nodesize > 0);
assert(h->basementnodesize > 0);
assert(ft->h->nodesize > 0);
assert(ft->h->basementnodesize > 0);
if (height == 0) {
assert(n_children > 0);
}
......@@ -94,9 +94,9 @@ create_new_ftnode_with_dep_nodes(
name,
height,
n_children,
h->layout_version,
h->nodesize,
h->flags);
ft->h->layout_version,
ft->h->nodesize,
ft->h->flags);
assert((*result)->nodesize > 0);
(*result)->fullhash = fullhash;
......@@ -208,10 +208,10 @@ toku_pin_ftnode_off_client_thread(
}
void
toku_unpin_ftnode_off_client_thread(FT h, FTNODE node)
toku_unpin_ftnode_off_client_thread(FT ft, FTNODE node)
{
int r = toku_cachetable_unpin(
h->cf,
ft->cf,
node->thisnodename,
node->fullhash,
(enum cachetable_dirty) node->dirty,
......@@ -221,11 +221,11 @@ toku_unpin_ftnode_off_client_thread(FT h, FTNODE node)
}
void
toku_unpin_ftnode(FT h, FTNODE node)
toku_unpin_ftnode(FT ft, FTNODE node)
{
// printf("%*sUnpin %ld\n", 8-node->height, "", node->thisnodename.b);
//VERIFY_NODE(brt,node);
toku_unpin_ftnode_off_client_thread(h, node);
toku_unpin_ftnode_off_client_thread(ft, node);
}
void
......
......@@ -807,9 +807,9 @@ ftleaf_split(
name,
0,
num_children_in_b,
h->layout_version,
h->nodesize,
h->flags);
h->h->layout_version,
h->h->nodesize,
h->h->flags);
assert(B->nodesize > 0);
B->fullhash = fullhash;
}
......@@ -1002,7 +1002,7 @@ ft_split_child(
FTNODE nodea, nodeb;
DBT splitk;
// printf("%s:%d node %" PRIu64 "->u.n.n_children=%d height=%d\n", __FILE__, __LINE__, node->thisnodename.b, node->u.n.n_children, node->height);
assert(h->nodesize>=node->nodesize); /* otherwise we might be in trouble because the nodesize shrank. */
assert(h->h->nodesize>=node->nodesize); /* otherwise we might be in trouble because the nodesize shrank. */
// for test
call_flusher_thread_callback(flt_flush_before_split);
......
This diff is collapsed.
This diff is collapsed.
......@@ -241,6 +241,8 @@ toku_ft_handle_stat64 (FT_HANDLE, TOKUTXN, struct ftstat64_s *stat) __attribute_
int toku_ft_layer_init(void (*ydb_lock_callback)(void),
void (*ydb_unlock_callback)(void))
__attribute__ ((warn_unused_result));
void toku_ft_open_close_lock(void);
void toku_ft_open_close_unlock(void);
int toku_ft_layer_destroy(void) __attribute__ ((warn_unused_result));
int toku_ft_serialize_layer_init(void) __attribute__ ((warn_unused_result));
int toku_ft_serialize_layer_destroy(void) __attribute__ ((warn_unused_result));
......@@ -259,10 +261,6 @@ void toku_ft_suppress_recovery_logs (FT_HANDLE brt, TOKUTXN txn);
int toku_ft_get_fragmentation(FT_HANDLE brt, TOKU_DB_FRAGMENTATION report) __attribute__ ((warn_unused_result));
BOOL toku_ft_is_empty_fast (FT_HANDLE brt);
// Effect: Return TRUE if there are no messages or leaf entries in the tree. If so, it's empty. If there are messages or leaf entries, we say it's not empty
// even though if we were to optimize the tree it might turn out that they are empty.
BOOL toku_ft_is_empty_fast (FT_HANDLE brt) __attribute__ ((warn_unused_result));
// Effect: Return TRUE if there are no messages or leaf entries in the tree. If so, it's empty. If there are messages or leaf entries, we say it's not empty
// even though if we were to optimize the tree it might turn out that they are empty.
......
This diff is collapsed.
......@@ -74,7 +74,7 @@ int toku_testsetup_nonleaf (FT_HANDLE brt, int height, BLOCKNUM *blocknum, int n
int toku_testsetup_root(FT_HANDLE brt, BLOCKNUM blocknum) {
assert(testsetup_initialized);
brt->ft->root_blocknum = blocknum;
brt->ft->h->root_blocknum = blocknum;
return 0;
}
......
......@@ -410,8 +410,8 @@ toku_verify_ft_with_progress (FT_HANDLE brt, int (*progress_callback)(void *extr
int r = toku_verify_ftnode(brt, ZERO_MSN, ZERO_MSN, root_node, -1, NULL, NULL, progress_callback, progress_extra, 1, verbose, keep_on_going);
if (r == 0) {
toku_ft_lock(brt->ft);
brt->ft->time_of_last_verification = time(NULL);
brt->ft->dirty = 1;
brt->ft->h->time_of_last_verification = time(NULL);
brt->ft->h->dirty = 1;
toku_ft_unlock(brt->ft);
}
return r;
......
This diff is collapsed.
......@@ -22,13 +22,19 @@ void toku_ft_destroy_treelock(FT h);
void toku_ft_grab_treelock(FT h);
void toku_ft_release_treelock(FT h);
void toku_ft_init_reflock(FT ft);
void toku_ft_destroy_reflock(FT ft);
void toku_ft_grab_reflock(FT ft);
void toku_ft_release_reflock(FT ft);
int toku_create_new_ft(FT *ftp, FT_OPTIONS options, CACHEFILE cf, TOKUTXN txn);
void toku_ft_free (FT h);
int toku_read_ft_and_store_in_cachefile (FT_HANDLE brt, CACHEFILE cf, LSN max_acceptable_lsn, FT *header, BOOL* was_open);
void toku_ft_note_ft_handle_open(FT ft, FT_HANDLE live);
int toku_ft_needed(FT h);
int toku_ft_needed_unlocked(FT h);
BOOL toku_ft_has_one_reference_unlocked(FT ft);
int toku_remove_ft (FT h, char **error_string, BOOL oplsn_valid, LSN oplsn) __attribute__ ((warn_unused_result));
FT_HANDLE toku_ft_get_some_existing_ft_handle(FT h);
......@@ -71,5 +77,8 @@ void toku_ft_update_cmp_descriptor(FT h);
void toku_ft_update_stats(STAT64INFO headerstats, STAT64INFO_S delta);
void toku_ft_decrease_stats(STAT64INFO headerstats, STAT64INFO_S delta);
void toku_ft_remove_reference(FT ft,
bool oplsn_valid, LSN oplsn,
remove_ft_ref_callback remove_ref, void *extra);
#endif
......@@ -881,8 +881,8 @@ toku_serialize_ftnode_to (int fd, BLOCKNUM blocknum, FTNODE node, FTNODE_DISK_DA
int r = toku_serialize_ftnode_to_memory(
node,
ndd,
h->basementnodesize,
h->compression_method,
h->h->basementnodesize,
h->h->compression_method,
do_rebalancing,
FALSE, // in_parallel
&n_to_write,
......@@ -1786,7 +1786,7 @@ deserialize_and_upgrade_internal_node(FTNODE node,
// of messages in the buffer.
MSN lowest;
u_int64_t amount = n_in_this_buffer;
lowest.msn = __sync_sub_and_fetch(&bfe->h->highest_unused_msn_for_upgrade.msn, amount);
lowest.msn = __sync_sub_and_fetch(&bfe->h->h->highest_unused_msn_for_upgrade.msn, amount);
if (highest_msn.msn == 0) {
highest_msn.msn = lowest.msn + n_in_this_buffer;
}
......@@ -2035,7 +2035,7 @@ deserialize_and_upgrade_leaf_node(FTNODE node,
// Whatever this is must be less than the MSNs of every message above
// it, so it's ok to take it here.
bn->max_msn_applied = bfe->h->highest_unused_msn_for_upgrade;
bn->max_msn_applied = bfe->h->h->highest_unused_msn_for_upgrade;
bn->stale_ancestor_messages_applied = false;
node->max_msn_applied_to_node_on_disk = bn->max_msn_applied;
......@@ -2625,7 +2625,7 @@ toku_serialize_rollback_log_to (int fd, BLOCKNUM blocknum, ROLLBACK_LOG_NODE log
size_t n_to_write;
char *compressed_buf;
{
int r = toku_serialize_rollback_log_to_memory(log, n_workitems, n_threads, h->compression_method, &n_to_write, &compressed_buf);
int r = toku_serialize_rollback_log_to_memory(log, n_workitems, n_threads, h->h->compression_method, &n_to_write, &compressed_buf);
if (r!=0) return r;
}
......@@ -2949,9 +2949,9 @@ toku_upgrade_subtree_estimates_to_stat64info(int fd, FT h)
FTNODE_DISK_DATA unused_ndd = NULL;
struct ftnode_fetch_extra bfe;
fill_bfe_for_min_read(&bfe, h);
e = deserialize_ftnode_from_fd(fd, h->root_blocknum, 0, &unused_node, &unused_ndd,
&bfe, &h->on_disk_stats);
h->in_memory_stats = h->on_disk_stats;
e = deserialize_ftnode_from_fd(fd, h->h->root_blocknum, 0, &unused_node, &unused_ndd,
&bfe, &h->h->on_disk_stats);
h->in_memory_stats = h->h->on_disk_stats;
if (unused_node) {
toku_ftnode_free(&unused_node);
......
......@@ -85,34 +85,34 @@ dump_descriptor(DESCRIPTOR d) {
static void
dump_header (int f, FT *header, CACHEFILE cf) {
FT h;
FT ft;
int r;
char timestr[26];
r = toku_deserialize_ft_from (f, MAX_LSN, &h);
r = toku_deserialize_ft_from (f, MAX_LSN, &ft);
assert(r==0);
h->cf = cf;
ft->cf = cf;
printf("ft:\n");
printf(" layout_version=%d\n", h->layout_version);
printf(" layout_version_original=%d\n", h->layout_version_original);
printf(" layout_version_read_from_disk=%d\n", h->layout_version_read_from_disk);
printf(" build_id=%d\n", h->build_id);
printf(" build_id_original=%d\n", h->build_id_original);
format_time(h->time_of_creation, timestr);
printf(" time_of_creation= %"PRIu64" %s\n", h->time_of_creation, timestr);
format_time(h->time_of_last_modification, timestr);
printf(" time_of_last_modification=%"PRIu64" %s\n", h->time_of_last_modification, timestr);
printf(" dirty=%d\n", h->dirty);
printf(" checkpoint_count=%" PRId64 "\n", h->checkpoint_count);
printf(" checkpoint_lsn=%" PRId64 "\n", h->checkpoint_lsn.lsn);
printf(" nodesize=%u\n", h->nodesize);
printf(" basementnodesize=%u\n", h->basementnodesize);
printf(" compression_method=%u\n", (unsigned) h->compression_method);
printf(" unnamed_root=%" PRId64 "\n", h->root_blocknum.b);
printf(" flags=%u\n", h->flags);
dump_descriptor(&h->descriptor);
printf(" estimated numrows=%" PRId64 "\n", h->in_memory_stats.numrows);
printf(" estimated numbytes=%" PRId64 "\n", h->in_memory_stats.numbytes);
*header = h;
printf(" layout_version=%d\n", ft->h->layout_version);
printf(" layout_version_original=%d\n", ft->h->layout_version_original);
printf(" layout_version_read_from_disk=%d\n", ft->layout_version_read_from_disk);
printf(" build_id=%d\n", ft->h->build_id);
printf(" build_id_original=%d\n", ft->h->build_id_original);
format_time(ft->h->time_of_creation, timestr);
printf(" time_of_creation= %"PRIu64" %s\n", ft->h->time_of_creation, timestr);
format_time(ft->h->time_of_last_modification, timestr);
printf(" time_of_last_modification=%"PRIu64" %s\n", ft->h->time_of_last_modification, timestr);
printf(" dirty=%d\n", ft->h->dirty);
printf(" checkpoint_count=%" PRId64 "\n", ft->h->checkpoint_count);
printf(" checkpoint_lsn=%" PRId64 "\n", ft->h->checkpoint_lsn.lsn);
printf(" nodesize=%u\n", ft->h->nodesize);
printf(" basementnodesize=%u\n", ft->h->basementnodesize);
printf(" compression_method=%u\n", (unsigned) ft->h->compression_method);
printf(" unnamed_root=%" PRId64 "\n", ft->h->root_blocknum.b);
printf(" flags=%u\n", ft->h->flags);
dump_descriptor(&ft->descriptor);
printf(" estimated numrows=%" PRId64 "\n", ft->in_memory_stats.numrows);
printf(" estimated numbytes=%" PRId64 "\n", ft->in_memory_stats.numbytes);
*header = ft;
}
static int
......@@ -506,14 +506,14 @@ main (int argc, const char *const argv[]) {
const char *n = argv[0];
int f = open(n, O_RDWR + O_BINARY); assert(f>=0);
FT h;
FT ft;
// create a cachefile for the header
int r = toku_create_cachetable(&ct, 1<<25, (LSN){0}, 0);
assert(r == 0);
CACHEFILE cf;
r = toku_cachetable_openfd (&cf, ct, f, n);
assert(r==0);
dump_header(f, &h, cf);
dump_header(f, &ft, cf);
if (interactive) {
while (1) {
printf("ftdump>"); fflush(stdout);
......@@ -530,25 +530,25 @@ main (int argc, const char *const argv[]) {
if (strcmp(fields[0], "help") == 0) {
interactive_help();
} else if (strcmp(fields[0], "header") == 0) {
toku_ft_free(h);
dump_header(f, &h, cf);
toku_ft_free(ft);
dump_header(f, &ft, cf);
} else if (strcmp(fields[0], "block") == 0 && nfields == 2) {
BLOCKNUM blocknum = make_blocknum(getuint64(fields[1]));
dump_block(f, blocknum, h);
dump_block(f, blocknum, ft);
} else if (strcmp(fields[0], "node") == 0 && nfields == 2) {
BLOCKNUM off = make_blocknum(getuint64(fields[1]));
dump_node(f, off, h);
dump_node(f, off, ft);
} else if (strcmp(fields[0], "dumpdata") == 0 && nfields == 2) {
dump_data = strtol(fields[1], NULL, 10);
} else if (strcmp(fields[0], "block_translation") == 0 || strcmp(fields[0], "bx") == 0) {
u_int64_t offset = 0;
if (nfields == 2)
offset = getuint64(fields[1]);
dump_block_translation(h, offset);
dump_block_translation(ft, offset);
} else if (strcmp(fields[0], "fragmentation") == 0) {
dump_fragmentation(f, h);
dump_fragmentation(f, ft);
} else if (strcmp(fields[0], "garbage") == 0) {
dump_garbage_stats(f, h);
dump_garbage_stats(f, ft);
} else if (strcmp(fields[0], "file") == 0 && nfields >= 3) {
u_int64_t offset = getuint64(fields[1]);
u_int64_t size = getuint64(fields[2]);
......@@ -565,18 +565,18 @@ main (int argc, const char *const argv[]) {
}
}
} else if (rootnode) {
dump_node(f, h->root_blocknum, h);
dump_node(f, ft->h->root_blocknum, ft);
} else {
printf("Block translation:");
toku_dump_translation_table(stdout, h->blocktable);
toku_dump_translation_table(stdout, ft->blocktable);
struct __dump_node_extra info;
info.f = f;
info.h = h;
toku_blocktable_iterate(h->blocktable, TRANSLATION_CHECKPOINTED,
info.h = ft;
toku_blocktable_iterate(ft->blocktable, TRANSLATION_CHECKPOINTED,
dump_node_wrapper, &info, TRUE, TRUE);
}
toku_ft_free(h);
toku_ft_free(ft);
return 0;
}
......@@ -507,7 +507,7 @@ int toku_ft_loader_internal_init (/* out */ FTLOADER *blp,
#define SET_TO_MY_STRDUP(lval, s) do { char *v = toku_strdup(s); if (!v) { int r = errno; toku_ft_loader_internal_destroy(bl, TRUE); return r; } lval = v; } while (0)
MY_CALLOC_N(N, bl->root_xids_that_created);
for (int i=0; i<N; i++) if (brts[i]) bl->root_xids_that_created[i]=brts[i]->ft->root_xid_that_created;
for (int i=0; i<N; i++) if (brts[i]) bl->root_xids_that_created[i]=brts[i]->ft->h->root_xid_that_created;
MY_CALLOC_N(N, bl->dbs);
for (int i=0; i<N; i++) if (brts[i]) bl->dbs[i]=dbs[i];
MY_CALLOC_N(N, bl->descriptors);
......@@ -2206,11 +2206,12 @@ static int toku_loader_write_ft_from_q (FTLOADER bl,
if (bl->root_xids_that_created)
root_xid_that_created = bl->root_xids_that_created[which_db];
struct ft h;
toku_ft_init(&h, (BLOCKNUM){0}, bl->load_lsn, root_xid_that_created, target_nodesize, target_basementnodesize, target_compression_method);
// TODO: (Zardosht/Yoni/Leif), do this code properly
struct ft ft;
toku_ft_init(&ft, (BLOCKNUM){0}, bl->load_lsn, root_xid_that_created, target_nodesize, target_basementnodesize, target_compression_method);
struct dbout out;
dbout_init(&out, &h);
dbout_init(&out, &ft);
out.fd = fd;
out.current_off = 8192; // leave 8K reserved at beginning
out.n_translations = 3; // 3 translations reserved at the beginning
......@@ -2333,7 +2334,7 @@ static int toku_loader_write_ft_from_q (FTLOADER bl,
}
if (deltas.numrows || deltas.numbytes) {
toku_ft_update_stats(&h.in_memory_stats, deltas);
toku_ft_update_stats(&ft.in_memory_stats, deltas);
}
cleanup_maxkey(&maxkey);
......@@ -2375,7 +2376,7 @@ static int toku_loader_write_ft_from_q (FTLOADER bl,
{
invariant(sts.n_subtrees==1);
out.h->root_blocknum = make_blocknum(sts.subtrees[0].block);
out.h->h->root_blocknum = make_blocknum(sts.subtrees[0].block);
toku_free(sts.subtrees); sts.subtrees = NULL;
// write the descriptor
......@@ -2766,16 +2767,15 @@ static int write_translation_table (struct dbout *out, long long *off_of_transla
static int
write_header (struct dbout *out, long long translation_location_on_disk, long long translation_size_on_disk) {
int result = 0;
out->h->checkpoint_staging_stats = out->h->in_memory_stats; // #4184
unsigned int size = toku_serialize_ft_size (out->h);
unsigned int size = toku_serialize_ft_size (out->h->h);
struct wbuf wbuf;
char *MALLOC_N(size, buf);
if (buf == NULL) {
result = errno;
} else {
wbuf_init(&wbuf, buf, size);
toku_serialize_ft_to_wbuf(&wbuf, out->h, translation_location_on_disk, translation_size_on_disk);
out->h->h->on_disk_stats = out->h->in_memory_stats;
toku_serialize_ft_to_wbuf(&wbuf, out->h->h, translation_location_on_disk, translation_size_on_disk);
if (wbuf.ndone != size)
result = EINVAL;
else
......
......@@ -38,6 +38,7 @@ typedef struct ftnode_leaf_basement_node *BASEMENTNODE;
typedef struct ftnode_nonleaf_childinfo *NONLEAF_CHILDINFO;
typedef struct sub_block *SUB_BLOCK;
typedef struct ft *FT;
typedef struct ft_header *FT_HEADER;
typedef struct ft_options *FT_OPTIONS;
struct wbuf;
struct dbuf;
......@@ -252,6 +253,7 @@ typedef int (*ft_compare_func)(DB *, const DBT *, const DBT *);
typedef void (*setval_func)(const DBT *, void *);
typedef int (*ft_update_func)(DB *, const DBT *, const DBT *, const DBT *, setval_func, void *);
typedef void (*on_redirect_callback)(FT_HANDLE, void*);
typedef void (*remove_ft_ref_callback)(FT, void*);
#define UU(x) x __attribute__((__unused__))
......
......@@ -197,7 +197,7 @@ toku_logger_open_rollback(TOKULOGGER logger, CACHETABLE cachetable, BOOL create)
//Verify it is empty
assert(!t->ft->panic);
//Must have no data blocks (rollback logs or otherwise).
toku_block_verify_no_data_blocks_except_root_unlocked(t->ft->blocktable, t->ft->root_blocknum);
toku_block_verify_no_data_blocks_except_root_unlocked(t->ft->blocktable, t->ft->h->root_blocknum);
BOOL is_empty;
is_empty = toku_ft_is_empty_fast(t);
assert(is_empty);
......@@ -216,26 +216,26 @@ toku_logger_close_rollback(TOKULOGGER logger, BOOL recovery_failed) {
if (!logger->is_panicked && cf) {
FT_HANDLE ft_to_close;
{ //Find "brt"
FT h = toku_cachefile_get_userdata(cf);
if (!h->panic && recovery_failed) {
r = toku_ft_set_panic(h, EINVAL, "Recovery failed");
FT ft = toku_cachefile_get_userdata(cf);
if (!ft->panic && recovery_failed) {
r = toku_ft_set_panic(ft, EINVAL, "Recovery failed");
assert_zero(r);
}
//Verify it is safe to close it.
if (!h->panic) { //If paniced, it is safe to close.
assert(!h->dirty); //Must not be dirty.
if (!ft->panic) { //If paniced, it is safe to close.
assert(!ft->h->dirty); //Must not be dirty.
//Must have no data blocks (rollback logs or otherwise).
toku_block_verify_no_data_blocks_except_root_unlocked(h->blocktable, h->root_blocknum);
toku_block_verify_no_data_blocks_except_root_unlocked(ft->blocktable, ft->h->root_blocknum);
}
assert(!h->dirty);
ft_to_close = toku_ft_get_some_existing_ft_handle(h);
assert(!ft->h->dirty);
ft_to_close = toku_ft_get_some_existing_ft_handle(ft);
assert(ft_to_close);
{
BOOL is_empty;
is_empty = toku_ft_is_empty_fast(ft_to_close);
assert(is_empty);
}
assert(!h->dirty); // it should not have been dirtied by the toku_ft_is_empty test.
assert(!ft->h->dirty); // it should not have been dirtied by the toku_ft_is_empty test.
}
r = toku_ft_handle_close(ft_to_close, FALSE, ZERO_LSN);
......
......@@ -330,11 +330,15 @@ test_prefetching(void) {
FT_HANDLE XMALLOC(brt);
FT XCALLOC(brt_h);
toku_ft_init(brt_h,
make_blocknum(0),
ZERO_LSN,
TXNID_NONE,
4*1024*1024,
128*1024,
TOKU_DEFAULT_COMPRESSION_METHOD);
brt->ft = brt_h;
brt_h->type = FT_CURRENT;
brt_h->panic = 0; brt_h->panic_string = 0;
brt_h->basementnodesize = 128*1024;
brt_h->compression_method = TOKU_DEFAULT_COMPRESSION_METHOD;
toku_ft_init_treelock(brt_h);
toku_blocktable_create_new(&brt_h->blocktable);
//Want to use block #20
......
......@@ -273,11 +273,15 @@ test_serialize_nonleaf(void) {
FT_HANDLE XMALLOC(brt);
FT XCALLOC(brt_h);
toku_ft_init(brt_h,
make_blocknum(0),
ZERO_LSN,
TXNID_NONE,
4*1024*1024,
128*1024,
TOKU_DEFAULT_COMPRESSION_METHOD);
brt->ft = brt_h;
brt_h->type = FT_CURRENT;
brt_h->panic = 0; brt_h->panic_string = 0;
brt_h->basementnodesize = 128*1024;
brt_h->compression_method = TOKU_DEFAULT_COMPRESSION_METHOD;
toku_ft_init_treelock(brt_h);
toku_blocktable_create_new(&brt_h->blocktable);
//Want to use block #20
......@@ -359,11 +363,15 @@ test_serialize_leaf(void) {
FT_HANDLE XMALLOC(brt);
FT XCALLOC(brt_h);
toku_ft_init(brt_h,
make_blocknum(0),
ZERO_LSN,
TXNID_NONE,
4*1024*1024,
128*1024,
TOKU_DEFAULT_COMPRESSION_METHOD);
brt->ft = brt_h;
brt_h->type = FT_CURRENT;
brt_h->panic = 0; brt_h->panic_string = 0;
brt_h->basementnodesize = 128*1024;
brt_h->compression_method = TOKU_DEFAULT_COMPRESSION_METHOD;
toku_ft_init_treelock(brt_h);
toku_blocktable_create_new(&brt_h->blocktable);
//Want to use block #20
......
......@@ -104,11 +104,15 @@ test_serialize_leaf(int valsize, int nelts, double entropy) {
FT_HANDLE XMALLOC(brt);
FT XCALLOC(brt_h);
toku_ft_init(brt_h,
make_blocknum(0),
ZERO_LSN,
TXNID_NONE,
4*1024*1024,
128*1024,
TOKU_DEFAULT_COMPRESSION_METHOD);
brt->ft = brt_h;
brt_h->type = FT_CURRENT;
brt_h->panic = 0; brt_h->panic_string = 0;
brt_h->basementnodesize = 128*1024;
brt_h->compression_method = TOKU_DEFAULT_COMPRESSION_METHOD;
brt_h->compare_fun = long_key_cmp;
toku_ft_init_treelock(brt_h);
toku_blocktable_create_new(&brt_h->blocktable);
......@@ -237,11 +241,15 @@ test_serialize_nonleaf(int valsize, int nelts, double entropy) {
FT_HANDLE XMALLOC(brt);
FT XCALLOC(brt_h);
toku_ft_init(brt_h,
make_blocknum(0),
ZERO_LSN,
TXNID_NONE,
4*1024*1024,
128*1024,
TOKU_DEFAULT_COMPRESSION_METHOD);
brt->ft = brt_h;
brt_h->type = FT_CURRENT;
brt_h->panic = 0; brt_h->panic_string = 0;
brt_h->basementnodesize = 128*1024;
brt_h->compression_method = TOKU_DEFAULT_COMPRESSION_METHOD;
brt_h->compare_fun = long_key_cmp;
toku_ft_init_treelock(brt_h);
toku_blocktable_create_new(&brt_h->blocktable);
......
......@@ -250,11 +250,15 @@ test_serialize_leaf_check_msn(enum ftnode_verify_type bft, BOOL do_clone) {
FT_HANDLE XMALLOC(brt);
FT XCALLOC(brt_h);
toku_ft_init(brt_h,
make_blocknum(0),
ZERO_LSN,
TXNID_NONE,
4*1024*1024,
128*1024,
TOKU_DEFAULT_COMPRESSION_METHOD);
brt->ft = brt_h;
brt_h->type = FT_CURRENT;
brt_h->panic = 0; brt_h->panic_string = 0;
brt_h->basementnodesize = 128*1024;
brt_h->compression_method = TOKU_DEFAULT_COMPRESSION_METHOD;
toku_ft_init_treelock(brt_h);
toku_blocktable_create_new(&brt_h->blocktable);
//Want to use block #20
......@@ -392,11 +396,15 @@ test_serialize_leaf_with_large_pivots(enum ftnode_verify_type bft, BOOL do_clone
FT_HANDLE XMALLOC(brt);
FT XCALLOC(brt_h);
toku_ft_init(brt_h,
make_blocknum(0),
ZERO_LSN,
TXNID_NONE,
4*1024*1024,
128*1024,
TOKU_DEFAULT_COMPRESSION_METHOD);
brt->ft = brt_h;
brt_h->type = FT_CURRENT;
brt_h->panic = 0; brt_h->panic_string = 0;
brt_h->basementnodesize = 128*1024;
brt_h->compression_method = TOKU_DEFAULT_COMPRESSION_METHOD;
toku_ft_init_treelock(brt_h);
toku_blocktable_create_new(&brt_h->blocktable);
//Want to use block #20
......@@ -531,11 +539,15 @@ test_serialize_leaf_with_many_rows(enum ftnode_verify_type bft, BOOL do_clone) {
FT_HANDLE XMALLOC(brt);
FT XCALLOC(brt_h);
toku_ft_init(brt_h,
make_blocknum(0),
ZERO_LSN,
TXNID_NONE,
4*1024*1024,
128*1024,
TOKU_DEFAULT_COMPRESSION_METHOD);
brt->ft = brt_h;
brt_h->type = FT_CURRENT;
brt_h->panic = 0; brt_h->panic_string = 0;
brt_h->basementnodesize = 128*1024;
brt_h->compression_method = TOKU_DEFAULT_COMPRESSION_METHOD;
toku_ft_init_treelock(brt_h);
toku_blocktable_create_new(&brt_h->blocktable);
//Want to use block #20
......@@ -675,11 +687,15 @@ test_serialize_leaf_with_large_rows(enum ftnode_verify_type bft, BOOL do_clone)
FT_HANDLE XMALLOC(brt);
FT XCALLOC(brt_h);
toku_ft_init(brt_h,
make_blocknum(0),
ZERO_LSN,
TXNID_NONE,
4*1024*1024,
128*1024,
TOKU_DEFAULT_COMPRESSION_METHOD);
brt->ft = brt_h;
brt_h->type = FT_CURRENT;
brt_h->panic = 0; brt_h->panic_string = 0;
brt_h->basementnodesize = 128*1024;
brt_h->compression_method = TOKU_DEFAULT_COMPRESSION_METHOD;
toku_ft_init_treelock(brt_h);
toku_blocktable_create_new(&brt_h->blocktable);
//Want to use block #20
......@@ -835,11 +851,15 @@ test_serialize_leaf_with_empty_basement_nodes(enum ftnode_verify_type bft, BOOL
FT_HANDLE XMALLOC(brt);
FT XCALLOC(brt_h);
toku_ft_init(brt_h,
make_blocknum(0),
ZERO_LSN,
TXNID_NONE,
4*1024*1024,
128*1024,
TOKU_DEFAULT_COMPRESSION_METHOD);
brt->ft = brt_h;
brt_h->type = FT_CURRENT;
brt_h->panic = 0; brt_h->panic_string = 0;
brt_h->basementnodesize = 128*1024;
brt_h->compression_method = TOKU_DEFAULT_COMPRESSION_METHOD;
toku_ft_init_treelock(brt_h);
toku_blocktable_create_new(&brt_h->blocktable);
//Want to use block #20
......@@ -959,11 +979,15 @@ test_serialize_leaf_with_multiple_empty_basement_nodes(enum ftnode_verify_type b
FT_HANDLE XMALLOC(brt);
FT XCALLOC(brt_h);
toku_ft_init(brt_h,
make_blocknum(0),
ZERO_LSN,
TXNID_NONE,
4*1024*1024,
128*1024,
TOKU_DEFAULT_COMPRESSION_METHOD);
brt->ft = brt_h;
brt_h->type = FT_CURRENT;
brt_h->panic = 0; brt_h->panic_string = 0;
brt_h->basementnodesize = 128*1024;
brt_h->compression_method = TOKU_DEFAULT_COMPRESSION_METHOD;
toku_ft_init_treelock(brt_h);
toku_blocktable_create_new(&brt_h->blocktable);
//Want to use block #20
......@@ -1088,11 +1112,15 @@ test_serialize_leaf(enum ftnode_verify_type bft, BOOL do_clone) {
FT_HANDLE XMALLOC(brt);
FT XCALLOC(brt_h);
toku_ft_init(brt_h,
make_blocknum(0),
ZERO_LSN,
TXNID_NONE,
4*1024*1024,
128*1024,
TOKU_DEFAULT_COMPRESSION_METHOD);
brt->ft = brt_h;
brt_h->type = FT_CURRENT;
brt_h->panic = 0; brt_h->panic_string = 0;
brt_h->basementnodesize = 128*1024;
brt_h->compression_method = TOKU_DEFAULT_COMPRESSION_METHOD;
toku_ft_init_treelock(brt_h);
toku_blocktable_create_new(&brt_h->blocktable);
//Want to use block #20
......@@ -1230,11 +1258,15 @@ test_serialize_nonleaf(enum ftnode_verify_type bft, BOOL do_clone) {
FT_HANDLE XMALLOC(brt);
FT XCALLOC(brt_h);
toku_ft_init(brt_h,
make_blocknum(0),
ZERO_LSN,
TXNID_NONE,
4*1024*1024,
128*1024,
TOKU_DEFAULT_COMPRESSION_METHOD);
brt->ft = brt_h;
brt_h->type = FT_CURRENT;
brt_h->panic = 0; brt_h->panic_string = 0;
brt_h->basementnodesize = 128*1024;
brt_h->compression_method = TOKU_DEFAULT_COMPRESSION_METHOD;
toku_ft_init_treelock(brt_h);
toku_blocktable_create_new(&brt_h->blocktable);
//Want to use block #20
......
......@@ -25,14 +25,15 @@ static void test_header (void) {
r = toku_open_ft_handle(fname, 1, &t, 1024, 256, TOKU_DEFAULT_COMPRESSION_METHOD, ct, null_txn, toku_builtin_compare_fun);
assert(r==0);
// now insert some info into the header
FT h = t->ft;
h->dirty = 1;
h->layout_version_original = 13;
h->layout_version_read_from_disk = 14;
h->build_id_original = 1234;
h->in_memory_stats = (STAT64INFO_S) {10, 11};
h->on_disk_stats = (STAT64INFO_S) {20, 21};
h->checkpoint_staging_stats = (STAT64INFO_S) {30, 31};
FT ft = t->ft;
ft->h->dirty = 1;
// cast away const because we actually want to fiddle with the header
// in this test
*((int *) &ft->h->layout_version_original) = 13;
ft->layout_version_read_from_disk = 14;
*((uint32_t *) &ft->h->build_id_original) = 1234;
ft->in_memory_stats = (STAT64INFO_S) {10, 11};
ft->h->on_disk_stats = (STAT64INFO_S) {20, 21};
r = toku_close_ft_handle_nolsn(t, 0); assert(r==0);
r = toku_cachetable_close(&ct);
assert(r==0);
......@@ -43,20 +44,17 @@ static void test_header (void) {
r = toku_open_ft_handle(fname, 0, &t, 1024, 256, TOKU_DEFAULT_COMPRESSION_METHOD, ct, null_txn, toku_builtin_compare_fun);
assert(r==0);
h = t->ft;
STAT64INFO_S expected_stats = {20, 21}; // on checkpoint, on_disk_stats copied to checkpoint_staging_stats
assert(h->layout_version == FT_LAYOUT_VERSION);
assert(h->layout_version_original == 13);
assert(h->layout_version_read_from_disk == FT_LAYOUT_VERSION);
assert(h->build_id_original == 1234);
assert(h->in_memory_stats.numrows == expected_stats.numrows);
assert(h->on_disk_stats.numbytes == expected_stats.numbytes);
ft = t->ft;
STAT64INFO_S expected_stats = {20, 21}; // on checkpoint, on_disk_stats copied to ft->checkpoint_header->on_disk_stats
assert(ft->h->layout_version == FT_LAYOUT_VERSION);
assert(ft->h->layout_version_original == 13);
assert(ft->layout_version_read_from_disk == FT_LAYOUT_VERSION);
assert(ft->h->build_id_original == 1234);
assert(ft->in_memory_stats.numrows == expected_stats.numrows);
assert(ft->h->on_disk_stats.numbytes == expected_stats.numbytes);
r = toku_close_ft_handle_nolsn(t, 0); assert(r==0);
r = toku_cachetable_close(&ct);
assert(r==0);
}
int
......
......@@ -658,7 +658,6 @@ static int remove_txn (OMTVALUE hv, u_int32_t UU(idx), void *txnv)
if (txn->txnid64==h->txnid_that_created_or_locked_when_empty) {
h->txnid_that_created_or_locked_when_empty = TXNID_NONE;
h->root_that_created_or_locked_when_empty = TXNID_NONE;
}
if (txn->txnid64==h->txnid_that_suppressed_recovery_logs) {
h->txnid_that_suppressed_recovery_logs = TXNID_NONE;
......
......@@ -500,7 +500,8 @@ toku_db_change_descriptor(DB *db, DB_TXN* txn, const DBT* descriptor, u_int32_t
goto cleanup;
}
if (!is_db_hot_index) {
r = toku_db_pre_acquire_fileops_lock(db, txn);
//TODO(zardosht): why doesn't hot_index need to do locking?
r = toku_db_pre_acquire_table_lock(db, txn);
if (r != 0) { goto cleanup; }
}
......@@ -677,9 +678,9 @@ locked_db_open(DB *db, DB_TXN *txn, const char *fname, const char *dbname, DBTYP
static int
locked_db_change_descriptor(DB *db, DB_TXN* txn, const DBT* descriptor, u_int32_t flags) {
toku_ydb_lock();
toku_multi_operation_client_lock(); //Cannot begin checkpoint
int r = toku_db_change_descriptor(db, txn, descriptor, flags);
toku_ydb_unlock();
toku_multi_operation_client_unlock(); //Can now begin checkpoint
return r;
}
......
......@@ -19,6 +19,13 @@ struct toku_list {
struct toku_list *next, *prev;
};
static inline int toku_list_num_elements_est(struct toku_list *head) {
if (head->next == head) return 0;
if (head->next == head->prev) return 1;
return 2;
}
static inline void toku_list_init(struct toku_list *head) {
head->next = head->prev = head;
}
......
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