Commit 7ea6dd41 authored by Zardosht Kasheff's avatar Zardosht Kasheff Committed by Yoni Fogel

refs #4606, merge to main

git-svn-id: file:///svn/toku/tokudb@46797 c7de825b-a66e-492c-adef-691d508d4ae1
parent 8cc19d27
......@@ -162,7 +162,7 @@ if (CMAKE_CXX_COMPILER_ID MATCHES Intel)
set(CMAKE_CXX_FLAGS "-Wcheck ${CMAKE_CXX_FLAGS}")
else()
set(CMAKE_C_FLAGS "-std=c99 ${CMAKE_C_FLAGS}")
set(CMAKE_CXX_FLAGS "-std=c++11 ${CMAKE_CXX_FLAGS}")
set(CMAKE_CXX_FLAGS "-std=c++0x ${CMAKE_CXX_FLAGS}")
## set gcc warnings
set(CMAKE_C_FLAGS "-Wextra ${CMAKE_C_FLAGS}")
set(CMAKE_CXX_FLAGS "-Wextra ${CMAKE_CXX_FLAGS}")
......
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
......@@ -226,7 +226,7 @@ toku_checkpoint_destroy(void) {
// Take a checkpoint of all currently open dictionaries
int
toku_checkpoint(CACHETABLE ct, TOKULOGGER logger,
toku_checkpoint(CHECKPOINTER cp, TOKULOGGER logger,
void (*callback_f)(void*), void * extra,
void (*callback2_f)(void*), void * extra2,
checkpoint_caller_t caller_id) {
......@@ -270,7 +270,7 @@ toku_checkpoint(CACHETABLE ct, TOKULOGGER logger,
SET_CHECKPOINT_FOOTPRINT(30);
STATUS_VALUE(CP_TIME_LAST_CHECKPOINT_BEGIN) = time(NULL);
r = toku_cachetable_begin_checkpoint(ct, logger);
r = toku_cachetable_begin_checkpoint(cp, logger);
toku_ft_open_close_unlock();
multi_operation_checkpoint_unlock();
......@@ -279,7 +279,7 @@ toku_checkpoint(CACHETABLE ct, TOKULOGGER logger,
if (r==0) {
if (callback_f)
callback_f(extra); // callback is called with checkpoint_safe_lock still held
r = toku_cachetable_end_checkpoint(ct, logger, callback2_f, extra2);
r = toku_cachetable_end_checkpoint(cp, logger, callback2_f, extra2);
}
SET_CHECKPOINT_FOOTPRINT(50);
if (r==0 && logger) {
......
......@@ -71,7 +71,7 @@ typedef enum {SCHEDULED_CHECKPOINT = 0, // "normal" checkpoint taken on check
// Callbacks are called during checkpoint procedure while checkpoint_safe lock is still held.
// Callbacks are primarily intended for use in testing.
// caller_id identifies why the checkpoint is being taken.
int toku_checkpoint(CACHETABLE ct, TOKULOGGER logger,
int toku_checkpoint(CHECKPOINTER cp, TOKULOGGER logger,
void (*callback_f)(void*), void * extra,
void (*callback2_f)(void*), void * extra2,
checkpoint_caller_t caller_id);
......
......@@ -75,7 +75,7 @@ void toku_compress (enum toku_compression_method a,
assert(1 <= *destLen);
*destLen = 1;
} else {
qlz_state_compress *XMALLOC(qsc);
qlz_state_compress *XCALLOC(qsc);
size_t actual_destlen = qlz_compress(source, (char*)(dest+1), sourceLen, qsc);
assert(actual_destlen +1 <= *destLen);
*destLen = actual_destlen+1; // add one for the rfc1950-style header byte.
......@@ -132,7 +132,7 @@ void toku_decompress (Bytef *dest, uLongf destLen,
}
case TOKU_QUICKLZ_METHOD:
if (sourceLen>1) {
qlz_state_decompress *XMALLOC(qsd);
qlz_state_decompress *XCALLOC(qsd);
uLongf actual_destlen = qlz_decompress((char*)source+1, dest, qsd);
assert(actual_destlen == destLen);
toku_free(qsd);
......
......@@ -9,6 +9,7 @@
#include <fttypes.h>
#include <ft-flusher.h>
#include <ft-internal.h>
#include "ft.h"
static void
ftnode_get_key_and_fullhash(
......@@ -57,7 +58,8 @@ cachetable_put_empty_node_with_dep_nodes(
dependent_fullhash,
dependent_dirty_bits,
name,
fullhash);
fullhash,
toku_node_save_ct_pair);
assert_zero(r);
*result = new_node;
......@@ -131,10 +133,44 @@ toku_pin_ftnode(
bool apply_ancestor_messages, // this bool is probably temporary, for #3972, once we know how range query estimates work, will revisit this
FTNODE *node_p,
bool* msgs_applied)
{
toku_cachetable_begin_batched_pin(brt->ft->cf);
int r = toku_pin_ftnode_batched(
brt,
blocknum,
fullhash,
unlockers,
ancestors,
bounds,
bfe,
may_modify_node,
apply_ancestor_messages,
false,
node_p,
msgs_applied
);
toku_cachetable_end_batched_pin(brt->ft->cf);
return r;
}
int
toku_pin_ftnode_batched(
FT_HANDLE brt,
BLOCKNUM blocknum,
uint32_t fullhash,
UNLOCKERS unlockers,
ANCESTORS ancestors,
const PIVOT_BOUNDS bounds,
FTNODE_FETCH_EXTRA bfe,
bool may_modify_node,
bool apply_ancestor_messages, // this bool is probably temporary, for #3972, once we know how range query estimates work, will revisit this
bool end_batch_on_success,
FTNODE *node_p,
bool* msgs_applied)
{
void *node_v;
*msgs_applied = false;
int r = toku_cachetable_get_and_pin_nonblocking(
int r = toku_cachetable_get_and_pin_nonblocking_batched(
brt->ft->cf,
blocknum,
fullhash,
......@@ -149,6 +185,9 @@ toku_pin_ftnode(
unlockers);
if (r==0) {
FTNODE node = (FTNODE) node_v;
if (end_batch_on_success) {
toku_cachetable_end_batched_pin(brt->ft->cf);
}
if (apply_ancestor_messages && node->height == 0) {
toku_apply_ancestors_messages_to_node(brt, node, ancestors, bounds, msgs_applied);
}
......@@ -174,6 +213,31 @@ toku_pin_ftnode_off_client_thread(
uint32_t num_dependent_nodes,
FTNODE* dependent_nodes,
FTNODE *node_p)
{
toku_cachetable_begin_batched_pin(h->cf);
toku_pin_ftnode_off_client_thread_batched(
h,
blocknum,
fullhash,
bfe,
may_modify_node,
num_dependent_nodes,
dependent_nodes,
node_p
);
toku_cachetable_end_batched_pin(h->cf);
}
void
toku_pin_ftnode_off_client_thread_batched(
FT h,
BLOCKNUM blocknum,
uint32_t fullhash,
FTNODE_FETCH_EXTRA bfe,
bool may_modify_node,
uint32_t num_dependent_nodes,
FTNODE* dependent_nodes,
FTNODE *node_p)
{
void *node_v;
CACHEFILE dependent_cf[num_dependent_nodes];
......@@ -187,7 +251,7 @@ toku_pin_ftnode_off_client_thread(
dependent_dirty_bits[i] = (enum cachetable_dirty) dependent_nodes[i]->dirty;
}
int r = toku_cachetable_get_and_pin_with_dep_pairs(
int r = toku_cachetable_get_and_pin_with_dep_pairs_batched(
h->cf,
blocknum,
fullhash,
......@@ -232,8 +296,7 @@ toku_unpin_ftnode_off_client_thread(FT ft, FTNODE node)
{
int r = toku_cachetable_unpin(
ft->cf,
node->thisnodename,
node->fullhash,
node->ct_pair,
(enum cachetable_dirty) node->dirty,
make_ftnode_pair_attr(node)
);
......@@ -253,8 +316,7 @@ toku_unpin_ftnode_read_only(FT_HANDLE brt, FTNODE node)
{
int r = toku_cachetable_unpin(
brt->ft->cf,
node->thisnodename,
node->fullhash,
node->ct_pair,
(enum cachetable_dirty) node->dirty,
make_invalid_pair_attr()
);
......
......@@ -58,7 +58,7 @@ toku_create_new_ftnode (
* a partial fetch is not required and there is no contention for the node)
* or it returns TOKUDB_TRY_AGAIN after unlocking its ancestors (using
* unlockers and ancestors) and bringing the necessary pieces of the node
* into memory.
* into memory.
*/
int
toku_pin_ftnode(
......@@ -75,6 +75,26 @@ toku_pin_ftnode(
bool* msgs_applied
);
/**
* Batched version of toku_pin_ftnode, see cachetable batched API for more
* details.
*/
int
toku_pin_ftnode_batched(
FT_HANDLE brt,
BLOCKNUM blocknum,
uint32_t fullhash,
UNLOCKERS unlockers,
ANCESTORS ancestors,
const PIVOT_BOUNDS pbounds,
FTNODE_FETCH_EXTRA bfe,
bool may_modify_node,
bool apply_ancestor_messages, // this bool is probably temporary, for #3972, once we know how range query estimates work, will revisit this
bool end_batch_on_success,
FTNODE *node_p,
bool* msgs_applied
);
/**
* Unfortunately, this function is poorly named
* as over time, client threads have also started
......@@ -100,6 +120,22 @@ toku_pin_ftnode_off_client_thread(
*/
int toku_maybe_pin_ftnode_clean(FT ft, BLOCKNUM blocknum, uint32_t fullhash, FTNODE *nodep, bool may_modify_node);
/**
* Batched version of toku_pin_ftnode_off_client_thread, see cachetable
* batched API for more details.
*/
void
toku_pin_ftnode_off_client_thread_batched(
FT h,
BLOCKNUM blocknum,
uint32_t fullhash,
FTNODE_FETCH_EXTRA bfe,
bool may_modify_node,
uint32_t num_dependent_nodes,
FTNODE* dependent_nodes,
FTNODE *node_p
);
/**
* Effect: Unpin a brt node. Used for
* nodes that were pinned off client thread.
......
......@@ -1415,14 +1415,13 @@ ft_merge_child(
// now we possibly flush the children
//
if (did_merge) {
BLOCKNUM bn = childb->thisnodename;
// for test
call_flusher_thread_callback(flt_flush_before_unpin_remove);
// merge_remove_key_callback will free the blocknum
int rrb = toku_cachetable_unpin_and_remove(
h->cf,
bn,
childb->ct_pair,
merge_remove_key_callback,
h
);
......
......@@ -252,6 +252,7 @@ struct ftnode {
// for internal nodes, the ith partition corresponds to the ith message buffer
// for leaf nodes, the ith partition corresponds to the ith basement node
struct ftnode_partition *bp;
PAIR ct_pair;
};
// ftnode partition macros
......@@ -642,7 +643,7 @@ void toku_evict_bn_from_memory(FTNODE node, int childnum, FT h);
void toku_ft_status_update_pivot_fetch_reason(struct ftnode_fetch_extra *bfe);
extern void toku_ftnode_clone_callback(void* value_data, void** cloned_value_data, PAIR_ATTR* new_attr, bool for_checkpoint, void* write_extraargs);
extern void toku_ftnode_flush_callback (CACHEFILE cachefile, int fd, BLOCKNUM nodename, void *ftnode_v, void** UU(disk_data), void *extraargs, PAIR_ATTR size, PAIR_ATTR* new_size, bool write_me, bool keep_me, bool for_checkpoint, bool is_clone);
extern int toku_ftnode_fetch_callback (CACHEFILE cachefile, int fd, BLOCKNUM nodename, uint32_t fullhash, void **ftnode_pv, void** UU(disk_data), PAIR_ATTR *sizep, int*dirty, void*extraargs);
extern int toku_ftnode_fetch_callback (CACHEFILE cachefile, PAIR p, int fd, BLOCKNUM nodename, uint32_t fullhash, void **ftnode_pv, void** UU(disk_data), PAIR_ATTR *sizep, int*dirty, void*extraargs);
extern void toku_ftnode_pe_est_callback(void* ftnode_pv, void* disk_data, long* bytes_freed_estimate, enum partial_eviction_cost *cost, void* write_extraargs);
extern int toku_ftnode_pe_callback (void *ftnode_pv, PAIR_ATTR old_attr, PAIR_ATTR* new_attr, void *extraargs);
extern bool toku_ftnode_pf_req_callback(void* ftnode_pv, void* read_extraargs);
......
This diff is collapsed.
......@@ -364,8 +364,7 @@ done:
{
int r = toku_cachetable_unpin(
brt->ft->cf,
node->thisnodename,
toku_cachetable_hash(brt->ft->cf, node->thisnodename),
node->ct_pair,
CACHETABLE_CLEAN,
make_ftnode_pair_attr(node)
);
......
......@@ -364,6 +364,12 @@ ft_note_unpin_by_checkpoint (CACHEFILE UU(cachefile), void *header_v)
// End of Functions that are callbacks to the cachefile
/////////////////////////////////////////////////////////////////////////
void toku_node_save_ct_pair(void *value_data, PAIR p) {
FTNODE CAST_FROM_VOIDP(node, value_data);
node->ct_pair = p;
}
static int setup_initial_ft_root_node (FT ft, BLOCKNUM blocknum) {
FTNODE XMALLOC(node);
toku_initialize_empty_ftnode(node, blocknum, 0, 1, ft->h->layout_version, ft->h->nodesize, ft->h->flags);
......@@ -373,7 +379,8 @@ static int setup_initial_ft_root_node (FT ft, BLOCKNUM blocknum) {
node->fullhash = fullhash;
int r = toku_cachetable_put(ft->cf, blocknum, fullhash,
node, make_ftnode_pair_attr(node),
get_write_callbacks_for_node(ft));
get_write_callbacks_for_node(ft),
toku_node_save_ct_pair);
if (r != 0)
toku_free(node);
else
......
......@@ -108,6 +108,7 @@ void toku_ft_set_basementnodesize(FT ft, unsigned int basementnodesize);
void toku_ft_get_basementnodesize(FT ft, unsigned int *basementnodesize);
void toku_ft_set_compression_method(FT ft, enum toku_compression_method method);
void toku_ft_get_compression_method(FT ft, enum toku_compression_method *methodp);
void toku_node_save_ct_pair(void *value_data, PAIR p);
// mark the ft as a blackhole. any message injections will be a no op.
void toku_ft_set_blackhole(FT_HANDLE ft_handle);
......
......@@ -137,6 +137,8 @@ struct logged_btt_pair {
typedef struct cachetable *CACHETABLE;
typedef struct cachefile *CACHEFILE;
typedef struct ctpair *PAIR;
typedef class checkpointer *CHECKPOINTER;
/* tree command types */
enum ft_msg_type {
......
......@@ -179,7 +179,8 @@ upgrade_log(const char *env_dir, const char *log_dir, LSN last_lsn, TXNID last_x
assert(r==0);
}
{ //Checkpoint
r = toku_checkpoint(ct, logger, NULL, NULL, NULL, NULL, UPGRADE_CHECKPOINT); //fsyncs log dir
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
r = toku_checkpoint(cp, logger, NULL, NULL, NULL, NULL, UPGRADE_CHECKPOINT); //fsyncs log dir
assert(r == 0);
}
{ //Close cachetable and logger
......
......@@ -44,11 +44,11 @@ void memarena_clear (MEMARENA ma) {
static size_t
round_to_page (size_t size) {
const size_t PAGE_SIZE = 4096;
const size_t result = PAGE_SIZE+((size-1)&~(PAGE_SIZE-1));
assert(0==(result&(PAGE_SIZE-1))); // make sure it's aligned
const size_t _PAGE_SIZE = 4096;
const size_t result = _PAGE_SIZE+((size-1)&~(_PAGE_SIZE-1));
assert(0==(result&(_PAGE_SIZE-1))); // make sure it's aligned
assert(result>=size); // make sure it's not too small
assert(result<size+PAGE_SIZE); // make sure we didn't grow by more than a page.
assert(result<size+_PAGE_SIZE); // make sure we didn't grow by more than a page.
return result;
}
......
......@@ -40,46 +40,46 @@ minicron_do (void *pv)
struct minicron *CAST_FROM_VOIDP(p, pv);
toku_mutex_lock(&p->mutex);
while (1) {
if (p->do_shutdown) {
toku_mutex_unlock(&p->mutex);
return 0;
}
if (p->period_in_seconds==0) {
// if we aren't supposed to do it then just do an untimed wait.
toku_cond_wait(&p->condvar, &p->mutex);
} else {
// Recompute the wakeup time every time (instead of once per call to f) in case the period changges.
toku_timespec_t wakeup_at = p->time_of_last_call_to_f;
wakeup_at.tv_sec += p->period_in_seconds;
toku_timespec_t now;
toku_gettime(&now);
//printf("wakeup at %.6f (after %d seconds) now=%.6f\n", wakeup_at.tv_sec + wakeup_at.tv_nsec*1e-9, p->period_in_seconds, now.tv_sec + now.tv_nsec*1e-9);
int r = toku_cond_timedwait(&p->condvar, &p->mutex, &wakeup_at);
if (r!=0 && r!=ETIMEDOUT) fprintf(stderr, "%s:%d r=%d (%s)", __FILE__, __LINE__, r, strerror(r));
assert(r==0 || r==ETIMEDOUT);
}
// Now we woke up, and we should figure out what to do
if (p->do_shutdown) {
toku_mutex_unlock(&p->mutex);
return 0;
}
if (p->period_in_seconds >0) {
// maybe do a checkpoint
toku_timespec_t now;
toku_gettime(&now);
toku_timespec_t time_to_call = p->time_of_last_call_to_f;
time_to_call.tv_sec += p->period_in_seconds;
int compare = timespec_compare(&time_to_call, &now);
//printf("compare(%.6f, %.6f)=%d\n", time_to_call.tv_sec + time_to_call.tv_nsec*1e-9, now.tv_sec+now.tv_nsec*1e-9, compare);
if (compare <= 0) {
toku_mutex_unlock(&p->mutex);
int r = p->f(p->arg);
assert(r==0);
toku_mutex_lock(&p->mutex);
toku_gettime(&p->time_of_last_call_to_f); // the period is measured between calls to f.
}
}
if (p->do_shutdown) {
toku_mutex_unlock(&p->mutex);
return 0;
}
if (p->period_in_seconds==0) {
// if we aren't supposed to do it then just do an untimed wait.
toku_cond_wait(&p->condvar, &p->mutex);
} else {
// Recompute the wakeup time every time (instead of once per call to f) in case the period changges.
toku_timespec_t wakeup_at = p->time_of_last_call_to_f;
wakeup_at.tv_sec += p->period_in_seconds;
toku_timespec_t now;
toku_gettime(&now);
//printf("wakeup at %.6f (after %d seconds) now=%.6f\n", wakeup_at.tv_sec + wakeup_at.tv_nsec*1e-9, p->period_in_seconds, now.tv_sec + now.tv_nsec*1e-9);
int r = toku_cond_timedwait(&p->condvar, &p->mutex, &wakeup_at);
if (r!=0 && r!=ETIMEDOUT) fprintf(stderr, "%s:%d r=%d (%s)", __FILE__, __LINE__, r, strerror(r));
assert(r==0 || r==ETIMEDOUT);
}
// Now we woke up, and we should figure out what to do
if (p->do_shutdown) {
toku_mutex_unlock(&p->mutex);
return 0;
}
if (p->period_in_seconds >0) {
// maybe do a checkpoint
toku_timespec_t now;
toku_gettime(&now);
toku_timespec_t time_to_call = p->time_of_last_call_to_f;
time_to_call.tv_sec += p->period_in_seconds;
int compare = timespec_compare(&time_to_call, &now);
//printf("compare(%.6f, %.6f)=%d\n", time_to_call.tv_sec + time_to_call.tv_nsec*1e-9, now.tv_sec+now.tv_nsec*1e-9, compare);
if (compare <= 0) {
toku_mutex_unlock(&p->mutex);
int r = p->f(p->arg);
assert(r==0);
toku_mutex_lock(&p->mutex);
toku_gettime(&p->time_of_last_call_to_f); // the period is measured between calls to f.
}
}
}
}
......
......@@ -96,6 +96,7 @@ struct recover_env {
keep_cachetable_callback_t keep_cachetable_callback; // after recovery, store the cachetable into the environment.
CACHETABLE ct;
TOKULOGGER logger;
CHECKPOINTER cp;
ft_compare_func bt_compare;
ft_update_func update_function;
generate_row_for_put_func generate_row_for_put;
......@@ -226,7 +227,7 @@ static int recover_env_init (RECOVER_ENV renv,
renv->generate_row_for_del = generate_row_for_del;
file_map_init(&renv->fmap);
renv->goforward = false;
renv->cp = toku_cachetable_get_checkpointer(renv->ct);
if (tokudb_recovery_trace)
fprintf(stderr, "%s:%d\n", __FUNCTION__, __LINE__);
return r;
......@@ -1510,7 +1511,7 @@ static int do_recovery(RECOVER_ENV renv, const char *env_dir, const char *log_di
// checkpoint
tnow = time(NULL);
fprintf(stderr, "%.24s Tokudb recovery making a checkpoint\n", ctime(&tnow));
r = toku_checkpoint(renv->ct, renv->logger, NULL, NULL, NULL, NULL, RECOVERY_CHECKPOINT);
r = toku_checkpoint(renv->cp, renv->logger, NULL, NULL, NULL, NULL, RECOVERY_CHECKPOINT);
assert(r == 0);
tnow = time(NULL);
fprintf(stderr, "%.24s Tokudb recovery done\n", ctime(&tnow));
......
......@@ -53,7 +53,7 @@ void toku_rollback_flush_callback (CACHEFILE cachefile, int fd, BLOCKNUM logname
}
}
int toku_rollback_fetch_callback (CACHEFILE cachefile, int fd, BLOCKNUM logname, uint32_t fullhash,
int toku_rollback_fetch_callback (CACHEFILE cachefile, PAIR p, int fd, BLOCKNUM logname, uint32_t fullhash,
void **rollback_pv, void** UU(disk_data), PAIR_ATTR *sizep, int * UU(dirtyp), void *extraargs) {
int r;
FT CAST_FROM_VOIDP(h, extraargs);
......@@ -62,6 +62,7 @@ int toku_rollback_fetch_callback (CACHEFILE cachefile, int fd, BLOCKNUM logname,
ROLLBACK_LOG_NODE *result = (ROLLBACK_LOG_NODE*)rollback_pv;
r = toku_deserialize_rollback_log_from(fd, logname, fullhash, result, h);
if (r==0) {
(*result)->ct_pair = p;
*sizep = rollback_memory_size(*result);
}
return r;
......
......@@ -12,7 +12,7 @@
#include "fttypes.h"
void toku_rollback_flush_callback(CACHEFILE cachefile, int fd, BLOCKNUM logname, void *rollback_v, void** UU(disk_data), void *extraargs, PAIR_ATTR size, PAIR_ATTR* new_size, bool write_me, bool keep_me, bool for_checkpoint, bool UU(is_clone));
int toku_rollback_fetch_callback(CACHEFILE cachefile, int fd, BLOCKNUM logname, uint32_t fullhash, void **rollback_pv, void** UU(disk_data), PAIR_ATTR *sizep, int * UU(dirtyp), void *extraargs);
int toku_rollback_fetch_callback(CACHEFILE cachefile, PAIR p, int fd, BLOCKNUM logname, uint32_t fullhash, void **rollback_pv, void** UU(disk_data), PAIR_ATTR *sizep, int * UU(dirtyp), void *extraargs);
void toku_rollback_pe_est_callback(
void* rollback_v,
void* UU(disk_data),
......
......@@ -22,7 +22,7 @@ void toku_rollback_log_unpin_and_remove(TOKUTXN txn, ROLLBACK_LOG_NODE log) {
int r;
CACHEFILE cf = txn->logger->rollback_cachefile;
FT CAST_FROM_VOIDP(h, toku_cachefile_get_userdata(cf));
r = toku_cachetable_unpin_and_remove (cf, log->blocknum, rollback_unpin_remove_callback, h);
r = toku_cachetable_unpin_and_remove (cf, log->ct_pair, rollback_unpin_remove_callback, h);
assert(r == 0);
}
......@@ -62,6 +62,11 @@ rollback_memory_size(ROLLBACK_LOG_NODE log) {
return make_rollback_pair_attr(size);
}
static void toku_rollback_node_save_ct_pair(void *value_data, PAIR p) {
ROLLBACK_LOG_NODE CAST_FROM_VOIDP(log, value_data);
log->ct_pair = p;
}
// create and pin a new rollback log node. chain it to the other rollback nodes
// by providing a previous blocknum/ hash and assigning the new rollback log
// node the next sequence number
......@@ -91,7 +96,8 @@ static void rollback_log_create (TOKUTXN txn, BLOCKNUM previous, uint32_t previo
*result = log;
r = toku_cachetable_put(cf, log->blocknum, log->hash,
log, rollback_memory_size(log),
get_write_callbacks_for_rollback_log(h));
get_write_callbacks_for_rollback_log(h),
toku_rollback_node_save_ct_pair);
assert(r == 0);
txn->roll_info.current_rollback = log->blocknum;
txn->roll_info.current_rollback_hash = log->hash;
......@@ -100,7 +106,7 @@ static void rollback_log_create (TOKUTXN txn, BLOCKNUM previous, uint32_t previo
void toku_rollback_log_unpin(TOKUTXN txn, ROLLBACK_LOG_NODE log) {
int r;
CACHEFILE cf = txn->logger->rollback_cachefile;
r = toku_cachetable_unpin(cf, log->blocknum, log->hash,
r = toku_cachetable_unpin(cf, log->ct_pair,
(enum cachetable_dirty)log->dirty, rollback_memory_size(log));
assert(r == 0);
}
......
......@@ -78,6 +78,7 @@ struct rollback_log_node {
struct roll_entry *newest_logentry;
MEMARENA rollentry_arena;
size_t rollentry_resident_bytecount; // How many bytes for the rollentries that are stored in main memory.
PAIR ct_pair;
};
#endif // TOKU_ROLLBACK_H
set_property(DIRECTORY APPEND PROPERTY COMPILE_DEFINITIONS _GNU_SOURCE DONT_DEPRECATE_ERRNO)
if(BUILD_TESTING)
if(BUILD_TESTING OR BUILD_FT_TESTS)
## get a list of the sources in this directory
file(GLOB srcs RELATIVE "${CMAKE_CURRENT_SOURCE_DIR}" *.cc)
set(tests)
......@@ -131,4 +131,4 @@ if(BUILD_TESTING)
ft/benchmark-test_256
)
set_tests_properties(${phenomenally_long_tests} PROPERTIES TIMEOUT 14400)
endif(BUILD_TESTING)
endif(BUILD_TESTING OR BUILD_FT_TESTS)
......@@ -13,7 +13,7 @@ static void
unlock_test_fun (void *v) {
assert(v == NULL);
// CT lock is held
int r = toku_cachetable_unpin_ct_prelocked_no_flush(f1, make_blocknum(2), 2, CACHETABLE_CLEAN, make_pair_attr(8));
int r = toku_test_cachetable_unpin_ct_prelocked_no_flush(f1, make_blocknum(2), 2, CACHETABLE_CLEAN, make_pair_attr(8));
assert(r==0);
}
......@@ -34,11 +34,11 @@ run_test (void) {
long s2;
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, def_write_callback(NULL), def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8)); assert(r==0);
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8)); assert(r==0);
for (int i = 0; i < 20; i++) {
r = toku_cachetable_get_and_pin(f1, make_blocknum(2), 2, &v2, &s2, def_write_callback(NULL), def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
r = toku_cachetable_unpin(f1, make_blocknum(2), 2, CACHETABLE_CLEAN, make_pair_attr(8)); assert(r==0);
r = toku_test_cachetable_unpin(f1, make_blocknum(2), 2, CACHETABLE_CLEAN, make_pair_attr(8)); assert(r==0);
}
//
......@@ -50,9 +50,10 @@ run_test (void) {
// pin 1 and 2
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v2, &s2, def_write_callback(NULL), def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
r = toku_cachetable_begin_checkpoint(ct, NULL);
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
r = toku_cachetable_begin_checkpoint(cp, NULL);
// mark nodes as pending a checkpoint, so that get_and_pin_nonblocking on block 1 will return TOKUDB_TRY_AGAIN
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8)); assert(r==0);
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8)); assert(r==0);
r = toku_cachetable_get_and_pin(f1, make_blocknum(2), 2, &v2, &s2, def_write_callback(NULL), def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
// now we try to pin 1, and it should get evicted out from under us
......@@ -78,7 +79,7 @@ run_test (void) {
assert(r==TOKUDB_TRY_AGAIN);
r = toku_cachetable_end_checkpoint(
ct,
cp,
NULL,
NULL,
NULL
......
/* -*- mode: C++; c-basic-offset: 4; indent-tabs-mode: nil -*- */
// vim: ft=cpp:expandtab:ts=8:sw=4:softtabstop=4:
#ident "$Id$"
#ident "Copyright (c) 2007-2012 Tokutek Inc. All rights reserved."
#include "includes.h"
#include "test.h"
CACHETABLE ct;
bool checkpoint_began;
static void
flush (CACHEFILE f __attribute__((__unused__)),
int UU(fd),
CACHEKEY k __attribute__((__unused__)),
void *v __attribute__((__unused__)),
void** UU(dd),
void *e __attribute__((__unused__)),
PAIR_ATTR s __attribute__((__unused__)),
PAIR_ATTR* new_size __attribute__((__unused__)),
bool w __attribute__((__unused__)),
bool keep __attribute__((__unused__)),
bool c __attribute__((__unused__)),
bool UU(is_clone)
) {
/* Do nothing */
if (verbose) { printf("FLUSH: %d\n", (int)k.b); }
}
static int
fetch (CACHEFILE f __attribute__((__unused__)),
int UU(fd),
CACHEKEY k __attribute__((__unused__)),
uint32_t fullhash __attribute__((__unused__)),
void **value,
void** UU(dd),
PAIR_ATTR *sizep,
int *dirtyp,
void *extraargs
) {
*dirtyp = 0;
*value = extraargs;
*sizep = make_pair_attr(8);
return 0;
}
static void
pe_est_callback(
void* UU(ftnode_pv),
void* UU(dd),
long* bytes_freed_estimate,
enum partial_eviction_cost *cost,
void* UU(write_extraargs)
)
{
*bytes_freed_estimate = 1;
*cost = PE_CHEAP;
}
static int
pe_callback (
void *ftnode_pv __attribute__((__unused__)),
PAIR_ATTR bytes_to_free,
PAIR_ATTR* bytes_freed,
void* extraargs __attribute__((__unused__))
)
{
//
// The purpose of this test is to verify the fix for #4302.
// The problem with #4302 was as follows. During
// toku_cachetable_put_with_dep_pairs, there is a region
// where we assert that no checkpoint begins. In that region,
// we were calling maybe_flush_some, which releases the
// cachetable lock and calls pe_callback here. Beginning a
// checkpoint in this time frame causes an assert to fail.
// So, before the fix for #4302, an assert would fail when calling
// begin_checkpoint here. If at some point in the future, this call here
// causes a deadlock, then we need to find another way to ensure that
// a checkpoint that begins during an eviction caused by
// toku_cachetable_put_with_dep_pairs does not cause a crash.
//
if (!checkpoint_began) {
int r = toku_cachetable_begin_checkpoint(ct, NULL); assert(r == 0);
checkpoint_began = true;
}
*bytes_freed = make_pair_attr(bytes_to_free.size - 1);
return 0;
}
static void
test_get_key_and_fullhash(
CACHEKEY* cachekey,
uint32_t* fullhash,
void* UU(extra))
{
CACHEKEY name;
name.b = 2;
*cachekey = name;
*fullhash = 2;
}
static void
cachetable_test (void) {
const int test_limit = 12;
int r;
r = toku_create_cachetable(&ct, test_limit, ZERO_LSN, NULL_LOGGER); assert(r == 0);
char fname1[] = __SRCFILE__ "test1.dat";
unlink(fname1);
CACHEFILE f1;
r = toku_cachetable_openf(&f1, ct, fname1, O_RDWR|O_CREAT, S_IRWXU|S_IRWXG|S_IRWXO); assert(r == 0);
void* v1;
long s1;
uint64_t val1 = 0;
uint64_t val2 = 0;
CACHETABLE_WRITE_CALLBACK wc = def_write_callback(NULL);
wc.flush_callback = flush;
wc.pe_est_callback = pe_est_callback;
wc.pe_callback = pe_callback;
r = toku_cachetable_get_and_pin(
f1,
make_blocknum(1),
1,
&v1,
&s1,
wc,
fetch,
def_pf_req_callback, def_pf_callback,
true,
&val1
);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
CACHEKEY key;
uint32_t fullhash;
checkpoint_began = false;
r = toku_cachetable_put_with_dep_pairs(
f1,
test_get_key_and_fullhash,
&val2,
make_pair_attr(8),
wc,
NULL,
0, // number of dependent pairs that we may need to checkpoint
NULL, // array of cachefiles of dependent pairs
NULL, // array of cachekeys of dependent pairs
NULL, //array of fullhashes of dependent pairs
NULL, // array stating dirty/cleanness of dependent pairs
&key,
&fullhash
);
r = toku_cachetable_unpin(f1, make_blocknum(2), 2, CACHETABLE_CLEAN, make_pair_attr(8));
// end the checkpoint began in pe_callback
assert(checkpoint_began);
r = toku_cachetable_end_checkpoint(
ct,
NULL,
NULL,
NULL
);
toku_cachetable_verify(ct);
r = toku_cachefile_close(&f1, 0, false, ZERO_LSN); assert(r == 0);
r = toku_cachetable_close(&ct); lazy_assert_zero(r);
}
int
test_main(int argc, const char *argv[]) {
default_parse_args(argc, argv);
cachetable_test();
return 0;
}
......@@ -53,7 +53,7 @@ cachetable_test (void) {
assert_zero(r);
// sleep 3 seconds
usleep(3*1024*1024);
r = toku_cachetable_unpin_and_remove(f1, make_blocknum(1), NULL, NULL);
r = toku_test_cachetable_unpin_and_remove(f1, make_blocknum(1), NULL, NULL);
assert_zero(r);
void *ret;
......
......@@ -32,7 +32,8 @@ static void *put_same_key(void *arg) {
toku_cachetable_hash(f1,make_blocknum(1)),
NULL,
make_pair_attr(4),
def_write_callback(NULL)
def_write_callback(NULL),
put_callback_nop
);
assert(r==0);
return arg;
......@@ -74,7 +75,7 @@ cachetable_test (void) {
assert_zero(r);
// sleep 3 seconds
usleep(3*1024*1024);
r = toku_cachetable_unpin_and_remove(f1, make_blocknum(1), test_remove_key, NULL);
r = toku_test_cachetable_unpin_and_remove(f1, make_blocknum(1), test_remove_key, NULL);
assert_zero(r);
void *ret;
......@@ -83,7 +84,7 @@ cachetable_test (void) {
r = toku_pthread_join(put_tid, &ret);
assert_zero(r);
r = toku_cachetable_unpin(f1, make_blocknum(1), toku_cachetable_hash(f1, make_blocknum(1)), CACHETABLE_CLEAN, make_pair_attr(2));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), toku_cachetable_hash(f1, make_blocknum(1)), CACHETABLE_CLEAN, make_pair_attr(2));
toku_cachetable_verify(ct);
r = toku_cachefile_close(&f1, 0, false, ZERO_LSN); assert(r == 0);
......
......@@ -57,19 +57,20 @@ cachetable_test (void) {
CACHETABLE_WRITE_CALLBACK wc = def_write_callback(NULL);
wc.flush_callback = flush;
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, pf_req_callback, pf_callback, true, NULL);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8));
flush_called = false;
pf_req_called = false;
pf_called = false;
r = toku_cachetable_begin_checkpoint(ct, NULL);
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
r = toku_cachetable_begin_checkpoint(cp, NULL);
assert_zero(r);
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, pf_req_callback, pf_callback, true, NULL);
assert_zero(r);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8));
assert_zero(r);
r = toku_cachetable_end_checkpoint(
ct,
cp,
NULL,
NULL,
NULL
......
......@@ -13,6 +13,8 @@ bool dirty_flush_called;
bool check_pe_callback;
bool pe_callback_called;
CACHETABLE ct;
static int
pe_callback (
void *ftnode_pv __attribute__((__unused__)),
......@@ -63,9 +65,11 @@ static void *f2_pin(void *arg) {
check_pe_callback = true;
r = toku_cachetable_get_and_pin(f2, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
assert(r == 0);
ct->ev.signal_eviction_thread();
usleep(1*1024*1024);
assert(pe_callback_called);
pe_callback_called = false;
r = toku_cachetable_unpin(f2, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_test_cachetable_unpin(f2, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
check_pe_callback = false;
assert(!pe_callback_called);
assert(r == 0);
......@@ -80,8 +84,9 @@ cachetable_test (void) {
check_flush = false;
dirty_flush_called = false;
CACHETABLE ct;
r = toku_create_cachetable(&ct, test_limit, ZERO_LSN, NULL_LOGGER); assert(r == 0);
evictor_test_helpers::disable_ev_thread(&ct->ev); // disable eviction thread
char fname1[] = __SRCFILE__ "test1.dat";
unlink(fname1);
char fname2[] = __SRCFILE__ "test2.dat";
......@@ -101,7 +106,7 @@ cachetable_test (void) {
for (int i = 0; i < 20; i++) {
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
assert(r == 0);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8));
assert(r == 0);
}
......
......@@ -44,12 +44,12 @@ cachetable_test (void) {
CACHETABLE_WRITE_CALLBACK wc = def_write_callback(NULL);
wc.flush_callback = flush;
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8));
r = toku_cachetable_get_and_pin(f1, make_blocknum(2), 2, &v2, &s2, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
r = toku_cachetable_unpin(f1, make_blocknum(2), 2, CACHETABLE_CLEAN, make_pair_attr(8));
//r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, 8);
r = toku_test_cachetable_unpin(f1, make_blocknum(2), 2, CACHETABLE_CLEAN, make_pair_attr(8));
//r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, 8);
r = toku_cachefile_close(&f1, 0, false, ZERO_LSN); assert(r == 0);
......
......@@ -59,24 +59,6 @@ flush (
sleep_random();
}
static int
fetch (
CACHEFILE UU(thiscf),
int UU(fd),
CACHEKEY UU(key),
uint32_t UU(fullhash),
void **UU(value),
void **UU(dd),
PAIR_ATTR *UU(sizep),
int *UU(dirtyp),
void *UU(extraargs)
)
{
assert(0); // should not be called
return 0;
}
static void*
do_update (void *UU(ignore))
{
......@@ -89,7 +71,7 @@ do_update (void *UU(ignore))
long size;
CACHETABLE_WRITE_CALLBACK wc = def_write_callback(NULL);
wc.flush_callback = flush;
int r = toku_cachetable_get_and_pin(cf, key, hi, &vv, &size, wc, fetch, def_pf_req_callback, def_pf_callback, true, 0);
int r = toku_cachetable_get_and_pin(cf, key, hi, &vv, &size, wc, fetch_die, def_pf_req_callback, def_pf_callback, true, 0);
//printf("g");
assert(r==0);
assert(size==sizeof(int));
......@@ -97,7 +79,7 @@ do_update (void *UU(ignore))
assert(*v==42);
*v = 43;
//printf("[%d]43\n", i);
r = toku_cachetable_unpin(cf, key, hi, CACHETABLE_DIRTY, make_pair_attr(item_size));
r = toku_test_cachetable_unpin(cf, key, hi, CACHETABLE_DIRTY, make_pair_attr(item_size));
sleep_random();
}
return 0;
......@@ -106,7 +88,8 @@ do_update (void *UU(ignore))
static void*
do_checkpoint (void *UU(v))
{
int r = toku_checkpoint(ct, NULL, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT);
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
int r = toku_checkpoint(cp, NULL, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT);
assert(r == 0);
return 0;
}
......@@ -140,10 +123,10 @@ static void checkpoint_pending(void) {
values[i] = 42;
CACHETABLE_WRITE_CALLBACK wc = def_write_callback(NULL);
wc.flush_callback = flush;
r = toku_cachetable_put(cf, key, hi, &values[i], make_pair_attr(sizeof(int)), wc);
r = toku_cachetable_put(cf, key, hi, &values[i], make_pair_attr(sizeof(int)), wc, put_callback_nop);
assert(r == 0);
r = toku_cachetable_unpin(cf, key, hi, CACHETABLE_DIRTY, make_pair_attr(item_size));
r = toku_test_cachetable_unpin(cf, key, hi, CACHETABLE_DIRTY, make_pair_attr(item_size));
assert(r == 0);
}
......@@ -162,15 +145,15 @@ static void checkpoint_pending(void) {
// after the checkpoint, all of the items should be 43
//printf("E43\n");
n_flush = n_write_me = n_keep_me = n_fetch = 0; expect_value = 43;
r = toku_checkpoint(ct, NULL, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT);
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
r = toku_checkpoint(cp, NULL, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT);
assert(r == 0);
assert(n_flush == N && n_write_me == N && n_keep_me == N);
// a subsequent checkpoint should cause no flushes, or writes since all of the items are clean
n_flush = n_write_me = n_keep_me = n_fetch = 0;
r = toku_checkpoint(ct, NULL, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT);
r = toku_checkpoint(cp, NULL, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT);
assert(r == 0);
assert(n_flush == 0 && n_write_me == 0 && n_keep_me == 0);
......
......@@ -41,6 +41,7 @@ flush (CACHEFILE f __attribute__((__unused__)),
static int
fetch (CACHEFILE f __attribute__((__unused__)),
PAIR UU(p),
int UU(fd),
CACHEKEY k __attribute__((__unused__)),
uint32_t fullhash __attribute__((__unused__)),
......@@ -87,17 +88,17 @@ cachetable_test (void) {
// Then we will begin a checkpoint, which should theoretically mark both as pending, but
// flush will be called only for v1, because v1 is dirty
//
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
r = toku_cachetable_begin_checkpoint(cp, NULL); assert(r == 0);
r = toku_cachetable_begin_checkpoint(ct, NULL); assert(r == 0);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8));
r = toku_cachetable_unpin(f1, make_blocknum(2), 2, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(2), 2, CACHETABLE_CLEAN, make_pair_attr(8));
check_me = true;
flush_called = false;
r = toku_cachetable_end_checkpoint(
ct,
cp,
NULL,
NULL,
NULL
......
......@@ -41,6 +41,7 @@ flush (CACHEFILE f __attribute__((__unused__)),
static int
fetch (CACHEFILE f __attribute__((__unused__)),
PAIR UU(p),
int UU(fd),
CACHEKEY k __attribute__((__unused__)),
uint32_t fullhash __attribute__((__unused__)),
......@@ -89,14 +90,14 @@ cachetable_test (void) {
// Then we will begin a checkpoint, which should theoretically mark both as pending, but
// flush will be called only for v1, because v1 is dirty
//
r = toku_cachetable_begin_checkpoint(ct, NULL); assert(r == 0);
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
r = toku_cachetable_begin_checkpoint(cp, NULL); assert(r == 0);
check_me = true;
flush_called = false;
r = toku_cachetable_end_checkpoint(
ct,
cp,
NULL,
NULL,
NULL
......
......@@ -14,12 +14,12 @@ static const int item_size = 1;
static int n_flush, n_write_me, n_keep_me, n_fetch;
static void flush(
CACHEFILE cf,
CACHEFILE UU(cf),
int UU(fd),
CACHEKEY key,
void *value,
CACHEKEY UU(key),
void *UU(value),
void** UU(dd),
void *extraargs,
void *UU(extraargs),
PAIR_ATTR size,
PAIR_ATTR* UU(new_size),
bool write_me,
......@@ -28,7 +28,7 @@ static void flush(
bool UU(is_clone)
)
{
cf = cf; key = key; value = value; extraargs = extraargs;
//cf = cf; key = key; value = value; extraargs = extraargs;
// assert(key == make_blocknum((long)value));
assert(size.size == item_size);
n_flush++;
......@@ -78,10 +78,10 @@ static void cachetable_checkpoint_test(int n, enum cachetable_dirty dirty) {
uint32_t hi = toku_cachetable_hash(f1, key);
CACHETABLE_WRITE_CALLBACK wc = def_write_callback(NULL);
wc.flush_callback = flush;
r = toku_cachetable_put(f1, key, hi, (void *)(long)i, make_pair_attr(1), wc);
r = toku_cachetable_put(f1, key, hi, (void *)(long)i, make_pair_attr(1), wc, put_callback_nop);
assert(r == 0);
r = toku_cachetable_unpin(f1, key, hi, dirty, make_pair_attr(item_size));
r = toku_test_cachetable_unpin(f1, key, hi, dirty, make_pair_attr(item_size));
assert(r == 0);
void *v;
......@@ -99,8 +99,8 @@ static void cachetable_checkpoint_test(int n, enum cachetable_dirty dirty) {
// the checkpoint should cause n writes, but since n <= the cachetable size,
// all items should be kept in the cachetable
n_flush = n_write_me = n_keep_me = n_fetch = 0;
r = toku_checkpoint(ct, NULL, checkpoint_callback, &callback_was_called, checkpoint_callback2, &callback2_was_called, CLIENT_CHECKPOINT);
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
r = toku_checkpoint(cp, NULL, checkpoint_callback, &callback_was_called, checkpoint_callback2, &callback2_was_called, CLIENT_CHECKPOINT);
assert(r == 0);
assert(callback_was_called != 0);
assert(callback2_was_called != 0);
......@@ -114,7 +114,7 @@ static void cachetable_checkpoint_test(int n, enum cachetable_dirty dirty) {
r = toku_cachetable_maybe_get_and_pin(f1, key, hi, &v);
if (r != 0)
continue;
r = toku_cachetable_unpin(f1, key, hi, CACHETABLE_CLEAN, make_pair_attr(item_size));
r = toku_test_cachetable_unpin(f1, key, hi, CACHETABLE_CLEAN, make_pair_attr(item_size));
assert(r == 0);
int its_dirty;
......@@ -132,7 +132,7 @@ static void cachetable_checkpoint_test(int n, enum cachetable_dirty dirty) {
n_flush = n_write_me = n_keep_me = n_fetch = 0;
r = toku_checkpoint(ct, NULL, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT);
r = toku_checkpoint(cp, NULL, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT);
assert(r == 0);
assert(n_flush == 0 && n_write_me == 0 && n_keep_me == 0);
......
/* -*- mode: C++; c-basic-offset: 4; indent-tabs-mode: nil -*- */
// vim: ft=cpp:expandtab:ts=8:sw=4:softtabstop=4:
#ident "$Id: cachetable-checkpointer_test.cc 45903 2012-07-19 13:06:39Z leifwalsh $"
#ident "Copyright (c) 2007-2012 Tokutek Inc. All rights reserved."
#include "includes.h"
#include "test.h"
#include "cachetable-internal.h"
//
// Wrapper for the checkpointer and necessary
// data to run the tests.
//
struct checkpointer_test {
checkpointer m_cp;
pair_list m_pl;
void test_begin_checkpoint();
void test_pending_bits();
void test_update_cachefiles();
void test_end_checkpoint();
};
//
// Dummy callbacks for checkpointing
//
static int dummy_log_fassociate(CACHEFILE UU(cf), void* UU(p))
{ return 0; }
static int dummy_log_rollback(CACHEFILE UU(cf), void* UU(p))
{ return 0; }
static int dummy_close_usr(CACHEFILE UU(cf), int UU(i), void* UU(p), char** UU(c), bool UU(b), LSN UU(lsn))
{ return 0; }
static int dummy_chckpnt_usr(CACHEFILE UU(cf), int UU(i), void* UU(p))
{ return 0; }
static int dummy_begin(LSN UU(lsn), void* UU(p))
{ return 0; }
static int dummy_end(CACHEFILE UU(cf), int UU(i), void* UU(p))
{ return 0; }
static int dummy_note_pin(CACHEFILE UU(cf), void* UU(p))
{ return 0; }
static int dummy_note_unpin(CACHEFILE UU(cf), void* UU(p))
{ return 0; }
//
// Helper function to set dummy functions in given cachefile.
//
static void
create_dummy_functions(CACHEFILE cf)
{
void *ud = NULL;
toku_cachefile_set_userdata (cf,
ud,
&dummy_log_fassociate,
&dummy_log_rollback,
&dummy_close_usr,
&dummy_chckpnt_usr,
&dummy_begin,
&dummy_end,
&dummy_note_pin,
&dummy_note_unpin);
}
//------------------------------------------------------------------------------
// test_begin_checkpoint() -
//
// Description:
//
void checkpointer_test::test_begin_checkpoint() {
int r = 0;
cachefile_list cfl;
cfl.init();
cachetable ctbl;
ctbl.list.init();
m_cp.init(&ctbl, NULL, &cfl);
// 1. Call checkpoint with NO cachefiles.
r = m_cp.begin_checkpoint();
if (r) { assert(!"CHECKPOINTER: Checkpoint with no cachefiles failed!\n"); }
// 2. Call checkpoint with ONE cachefile.
//cachefile cf;
struct cachefile cf;
cf.next = NULL;
cf.for_checkpoint = false;
m_cp.m_cf_list->m_head = &cf;
create_dummy_functions(&cf);
r = m_cp.begin_checkpoint();
if (r) { assert(!"CHECKPOINTER: Checkpoint with one cachefile failed!\n"); }
assert(m_cp.m_checkpoint_num_files == 1);
assert(cf.for_checkpoint == true);
// 3. Call checkpoint with MANY cachefiles.
const uint32_t count = 3;
struct cachefile cfs[count];
m_cp.m_cf_list->m_head = &cfs[0];
for (uint32_t i = 0; i < count; ++i) {
cfs[i].for_checkpoint = false;
create_dummy_functions(&cfs[i]);
if (i == count - 1) {
cfs[i].next = NULL;
} else {
cfs[i].next = &cfs[i + 1];
}
}
r = m_cp.begin_checkpoint();
if (r) { assert(!"CHECKPOINTER: Multiple checkpoint failed!\n"); }
assert(m_cp.m_checkpoint_num_files == count);
for (uint32_t i = 0; i < count; ++i) {
assert(cfs[i].for_checkpoint == true);
}
ctbl.list.destroy();
m_cp.destroy();
}
//------------------------------------------------------------------------------
// test_pending_bits() -
//
// Description:
//
void checkpointer_test::test_pending_bits() {
cachefile_list cfl;
cfl.init();
cachetable ctbl;
ctbl.list.init();
m_cp.m_ct = &ctbl;
//
// 1. Empty hash chain.
//
m_cp.turn_on_pending_bits();
//
// 2. One entry in pair chain
//
struct cachefile cf;
memset(&cf, 0, sizeof(cf));
cf.next = NULL;
cf.for_checkpoint = true;
m_cp.m_cf_list->m_head = &cf;
create_dummy_functions(&cf);
CACHEKEY k;
k.b = 0;
uint32_t hash = toku_cachetable_hash(&cf, k);
ctpair p;
CACHETABLE_WRITE_CALLBACK cb;
pair_attr_s attr;
attr.size = 0;
attr.nonleaf_size = 0;
attr.leaf_size = 0;
attr.rollback_size = 0;
attr.cache_pressure_size = 0;
attr.is_valid = true;
pair_init(&p,
&cf,
k,
NULL,
attr,
CACHETABLE_CLEAN,
hash,
cb,
NULL,
&ctbl.list);
m_cp.m_ct->list.put(&p);
m_cp.turn_on_pending_bits();
assert(p.checkpoint_pending);
m_cp.m_ct->list.evict(&p);
//
// 3. Many hash chain entries.
//
const uint32_t count = 3;
ctpair pairs[count];
for (uint32_t i = 0; i < count; ++i) {
CACHEKEY key;
key.b = i;
uint32_t full_hash = toku_cachetable_hash(&cf, key);
pair_init(&(pairs[i]),
&cf,
key,
NULL,
attr,
CACHETABLE_CLEAN,
full_hash,
cb,
NULL,
&ctbl.list);
m_cp.m_ct->list.put(&pairs[i]);
}
m_cp.turn_on_pending_bits();
for (uint32_t i = 0; i < count; ++i) {
assert(pairs[i].checkpoint_pending);
}
for (uint32_t i = 0; i < count; ++i) {
CACHEKEY key;
key.b = i;
uint32_t full_hash = toku_cachetable_hash(&cf, key);
PAIR pp = m_cp.m_ct->list.find_pair(&cf, key, full_hash);
assert(pp);
m_cp.m_ct->list.evict(pp);
}
int r = ctbl.list.destroy();
assert_zero(r);
}
//------------------------------------------------------------------------------
// test_update_cachefiles() -
//
// Description:
//
void checkpointer_test::test_update_cachefiles() {
}
//------------------------------------------------------------------------------
// test_end_checkpoint() -
//
// Description:
//
void checkpointer_test::test_end_checkpoint() {
/************
-add data
-call begin checkpoint
-add data
-call end checkpoint
-verify that 2nd added data NOT checkpointed
-verify that 1st added data WAS checkpointed
*************/
}
//------------------------------------------------------------------------------
// test_main() -
//
// Description:
//
int
test_main(int argc, const char *argv[]) {
int r = 0;
default_parse_args(argc, argv);
checkpointer_test cp_test;
// Run the tests.
cp_test.test_begin_checkpoint();
cp_test.test_pending_bits();
cp_test.test_update_cachefiles();
return r;
}
......@@ -51,7 +51,7 @@ cleaner_callback(
assert(!cleaner_called);
assert(flush_called);
cleaner_called = true;
int r = toku_cachetable_unpin(f1, blocknum, fullhash, CACHETABLE_CLEAN, make_pair_attr(8));
int r = toku_test_cachetable_unpin(f1, blocknum, fullhash, CACHETABLE_CLEAN, make_pair_attr(8));
assert_zero(r);
return 0;
}
......@@ -77,15 +77,16 @@ cachetable_test (void) {
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
PAIR_ATTR attr = make_pair_attr(8);
attr.cache_pressure_size = 8;
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, attr);
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, attr);
cleaner_called = false;
r = toku_cachetable_begin_checkpoint(ct, NULL);
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
r = toku_cachetable_begin_checkpoint(cp, NULL);
assert_zero(r);
toku_cleaner_thread(ct);
toku_cleaner_thread_for_test(ct);
assert(cleaner_called);
r = toku_cachetable_end_checkpoint(
ct,
cp,
NULL,
NULL,
NULL
......
......@@ -51,7 +51,7 @@ cleaner_callback(
assert(!cleaner_called);
assert(flush_called);
cleaner_called = true;
int r = toku_cachetable_unpin(f1, blocknum, fullhash, CACHETABLE_CLEAN, make_pair_attr(8));
int r = toku_test_cachetable_unpin(f1, blocknum, fullhash, CACHETABLE_CLEAN, make_pair_attr(8));
assert_zero(r);
return 0;
}
......@@ -77,15 +77,16 @@ cachetable_test (void) {
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
PAIR_ATTR attr = make_pair_attr(8);
attr.cache_pressure_size = 8;
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, attr);
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, attr);
cleaner_called = false;
r = toku_cachetable_begin_checkpoint(ct, NULL);
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
r = toku_cachetable_begin_checkpoint(cp, NULL);
assert_zero(r);
toku_cleaner_thread(ct);
toku_cleaner_thread_for_test(ct);
assert(!cleaner_called);
r = toku_cachetable_end_checkpoint(
ct,
cp,
NULL,
NULL,
NULL
......
......@@ -90,7 +90,7 @@ run_test (void) {
true,
&expect);
assert_zero(r);
r = toku_cachetable_unpin(f1, make_blocknum(i+1), i+1, CACHETABLE_DIRTY, attrs[i]);
r = toku_test_cachetable_unpin(f1, make_blocknum(i+1), i+1, CACHETABLE_DIRTY, attrs[i]);
assert_zero(r);
expect.size += attrs[i].size;
expect.nonleaf_size += attrs[i].nonleaf_size;
......@@ -114,7 +114,7 @@ run_test (void) {
def_pf_callback,
true,
&expect);
toku_cachetable_unpin(f1, make_blocknum(n_pairs + 1), n_pairs + 1, CACHETABLE_CLEAN,
toku_test_cachetable_unpin(f1, make_blocknum(n_pairs + 1), n_pairs + 1, CACHETABLE_CLEAN,
make_pair_attr(test_limit - expect.size + 20));
usleep(2*1024*1024);
......
......@@ -23,10 +23,10 @@ cachetable_test (void) {
r = toku_cachetable_openf(&f1, ct, fname1, O_RDWR|O_CREAT, S_IRWXU|S_IRWXG|S_IRWXO); assert(r == 0);
usleep(4000000);
r = toku_cachetable_begin_checkpoint(ct, NULL); assert(r == 0);
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
r = toku_cachetable_begin_checkpoint(cp, NULL); assert(r == 0);
r = toku_cachetable_end_checkpoint(
ct,
cp,
NULL,
NULL,
NULL
......
......@@ -57,7 +57,7 @@ run_test (void) {
usleep(4000000);
for (int i = 0; i < 8; ++i) {
r = toku_cachetable_unpin(f1, make_blocknum(i+1), i+1, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(i+1), i+1, CACHETABLE_CLEAN, make_pair_attr(8));
assert_zero(r);
}
......
......@@ -53,7 +53,7 @@ run_test (void) {
// set cachepressure_size to 0
PAIR_ATTR attr = make_pair_attr(8);
attr.cache_pressure_size = 0;
r = toku_cachetable_unpin(f1, make_blocknum(i+1), i+1, CACHETABLE_CLEAN, attr);
r = toku_test_cachetable_unpin(f1, make_blocknum(i+1), i+1, CACHETABLE_CLEAN, attr);
assert_zero(r);
}
......
......@@ -25,7 +25,7 @@ my_cleaner_callback(
assert(fullhash == 100);
PAIR_ATTR attr = make_pair_attr(8);
attr.cache_pressure_size = 100;
int r = toku_cachetable_unpin(f1, make_blocknum(100), 100, CACHETABLE_CLEAN, attr);
int r = toku_test_cachetable_unpin(f1, make_blocknum(100), 100, CACHETABLE_CLEAN, attr);
my_cleaner_callback_called = true;
return r;
}
......@@ -58,7 +58,7 @@ run_test (void) {
NULL);
PAIR_ATTR attr = make_pair_attr(8);
attr.cache_pressure_size = 100;
r = toku_cachetable_unpin(f1, make_blocknum(100), 100, CACHETABLE_CLEAN, attr);
r = toku_test_cachetable_unpin(f1, make_blocknum(100), 100, CACHETABLE_CLEAN, attr);
for (int i = 0; i < 4; ++i) {
r = toku_cachetable_get_and_pin(f1, make_blocknum(i+1), i+1, &vs[i], &ss[i],
......@@ -72,7 +72,7 @@ run_test (void) {
// set cachepressure_size to 0
attr = make_pair_attr(8);
attr.cache_pressure_size = 0;
r = toku_cachetable_unpin(f1, make_blocknum(i+1), i+1, CACHETABLE_CLEAN, attr);
r = toku_test_cachetable_unpin(f1, make_blocknum(i+1), i+1, CACHETABLE_CLEAN, attr);
assert_zero(r);
}
......
......@@ -20,18 +20,18 @@ cachetable_test (void) {
// test that putting something too big in the cachetable works fine
CACHETABLE_WRITE_CALLBACK wc = def_write_callback(NULL);
r = toku_cachetable_put(f1, make_blocknum(num_entries+1), num_entries+1, NULL, make_pair_attr(test_limit*2), wc);
r = toku_cachetable_put(f1, make_blocknum(num_entries+1), num_entries+1, NULL, make_pair_attr(test_limit*2), wc, put_callback_nop);
assert(r==0);
r = toku_cachetable_unpin(f1, make_blocknum(num_entries+1), num_entries+1, CACHETABLE_DIRTY, make_pair_attr(test_limit*2));
r = toku_test_cachetable_unpin(f1, make_blocknum(num_entries+1), num_entries+1, CACHETABLE_DIRTY, make_pair_attr(test_limit*2));
assert(r==0);
for (int64_t i = 0; i < num_entries; i++) {
r = toku_cachetable_put(f1, make_blocknum(i), i, NULL, make_pair_attr(1), wc);
r = toku_cachetable_put(f1, make_blocknum(i), i, NULL, make_pair_attr(1), wc, put_callback_nop);
assert(toku_cachefile_count_pinned(f1, 0) == (i+1));
}
for (int64_t i = 0; i < num_entries; i++) {
r = toku_cachetable_unpin(f1, make_blocknum(i), i, CACHETABLE_DIRTY, make_pair_attr(1));
r = toku_test_cachetable_unpin(f1, make_blocknum(i), i, CACHETABLE_DIRTY, make_pair_attr(1));
}
......
......@@ -36,6 +36,7 @@ flush (CACHEFILE f __attribute__((__unused__)),
static int
fetch (CACHEFILE f __attribute__((__unused__)),
PAIR UU(p),
int UU(fd),
CACHEKEY k __attribute__((__unused__)),
uint32_t fullhash __attribute__((__unused__)),
......@@ -72,26 +73,31 @@ cachetable_test (void) {
wc.flush_callback = flush;
for (int i = 0; i < 100000; i++) {
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, fetch, def_pf_req_callback, def_pf_callback, true, NULL);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(1));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(1));
}
for (int i = 0; i < 8; i++) {
r = toku_cachetable_get_and_pin(f1, make_blocknum(2), 2, &v2, &s2, wc, fetch, def_pf_req_callback, def_pf_callback, true, NULL);
r = toku_cachetable_unpin(f1, make_blocknum(2), 2, CACHETABLE_CLEAN, make_pair_attr(1));
r = toku_test_cachetable_unpin(f1, make_blocknum(2), 2, CACHETABLE_CLEAN, make_pair_attr(1));
}
for (int i = 0; i < 4; i++) {
r = toku_cachetable_get_and_pin(f1, make_blocknum(3), 3, &v2, &s2, wc, fetch, def_pf_req_callback, def_pf_callback, true, NULL);
r = toku_cachetable_unpin(f1, make_blocknum(3), 3, CACHETABLE_CLEAN, make_pair_attr(1));
r = toku_test_cachetable_unpin(f1, make_blocknum(3), 3, CACHETABLE_CLEAN, make_pair_attr(1));
}
for (int i = 0; i < 2; i++) {
r = toku_cachetable_get_and_pin(f1, make_blocknum(4), 4, &v2, &s2, wc, fetch, def_pf_req_callback, def_pf_callback, true, NULL);
r = toku_cachetable_unpin(f1, make_blocknum(4), 4, CACHETABLE_CLEAN, make_pair_attr(1));
r = toku_test_cachetable_unpin(f1, make_blocknum(4), 4, CACHETABLE_CLEAN, make_pair_attr(1));
}
flush_may_occur = true;
expected_flushed_key = 4;
r = toku_cachetable_put(f1, make_blocknum(5), 5, NULL, make_pair_attr(4), wc);
r = toku_cachetable_put(f1, make_blocknum(5), 5, NULL, make_pair_attr(4), wc, put_callback_nop);
ct->ev.signal_eviction_thread();
usleep(1*1024*1024);
flush_may_occur = true;
expected_flushed_key = 5;
r = toku_cachetable_unpin(f1, make_blocknum(5), 5, CACHETABLE_CLEAN, make_pair_attr(4));
r = toku_test_cachetable_unpin(f1, make_blocknum(5), 5, CACHETABLE_CLEAN, make_pair_attr(4));
ct->ev.signal_eviction_thread();
usleep(1*1024*1024);
check_flush = false;
r = toku_cachefile_close(&f1, 0, false, ZERO_LSN); assert(r == 0 );
......
......@@ -33,6 +33,7 @@ flush (CACHEFILE f __attribute__((__unused__)),
static int
fetch (CACHEFILE f __attribute__((__unused__)),
PAIR UU(p),
int UU(fd),
CACHEKEY k __attribute__((__unused__)),
uint32_t fullhash __attribute__((__unused__)),
......@@ -113,37 +114,41 @@ cachetable_test (void) {
wc.flush_callback = flush;
wc.pe_callback = pe_callback;
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, fetch, def_pf_req_callback, def_pf_callback, true, NULL);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(4));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(4));
}
for (int i = 0; i < 8; i++) {
CACHETABLE_WRITE_CALLBACK wc = def_write_callback(NULL);
wc.flush_callback = flush;
wc.pe_callback = pe_callback;
r = toku_cachetable_get_and_pin(f1, make_blocknum(2), 2, &v2, &s2, wc, fetch, def_pf_req_callback, def_pf_callback, true, NULL);
r = toku_cachetable_unpin(f1, make_blocknum(2), 2, CACHETABLE_CLEAN, make_pair_attr(4));
r = toku_test_cachetable_unpin(f1, make_blocknum(2), 2, CACHETABLE_CLEAN, make_pair_attr(4));
}
for (int i = 0; i < 4; i++) {
CACHETABLE_WRITE_CALLBACK wc = def_write_callback(NULL);
wc.flush_callback = flush;
wc.pe_callback = pe_callback;
r = toku_cachetable_get_and_pin(f1, make_blocknum(3), 3, &v2, &s2, wc, fetch, def_pf_req_callback, def_pf_callback, true, NULL);
r = toku_cachetable_unpin(f1, make_blocknum(3), 3, CACHETABLE_CLEAN, make_pair_attr(4));
r = toku_test_cachetable_unpin(f1, make_blocknum(3), 3, CACHETABLE_CLEAN, make_pair_attr(4));
}
for (int i = 0; i < 2; i++) {
CACHETABLE_WRITE_CALLBACK wc = def_write_callback(NULL);
wc.flush_callback = flush;
wc.pe_callback = pe_callback;
r = toku_cachetable_get_and_pin(f1, make_blocknum(4), 4, &v2, &s2, wc, fetch, def_pf_req_callback, def_pf_callback, true, NULL);
r = toku_cachetable_unpin(f1, make_blocknum(4), 4, CACHETABLE_CLEAN, make_pair_attr(4));
r = toku_test_cachetable_unpin(f1, make_blocknum(4), 4, CACHETABLE_CLEAN, make_pair_attr(4));
}
flush_may_occur = false;
expected_bytes_to_free = 4;
CACHETABLE_WRITE_CALLBACK wc = def_write_callback(NULL);
wc.flush_callback = other_flush;
wc.pe_callback = other_pe_callback;
r = toku_cachetable_put(f1, make_blocknum(5), 5, NULL, make_pair_attr(4), wc);
r = toku_cachetable_put(f1, make_blocknum(5), 5, NULL, make_pair_attr(4), wc, put_callback_nop);
ct->ev.signal_eviction_thread();
usleep(1*1024*1024);
flush_may_occur = true;
r = toku_cachetable_unpin(f1, make_blocknum(5), 5, CACHETABLE_CLEAN, make_pair_attr(4));
r = toku_test_cachetable_unpin(f1, make_blocknum(5), 5, CACHETABLE_CLEAN, make_pair_attr(4));
ct->ev.signal_eviction_thread();
usleep(1*1024*1024);
assert(expected_bytes_to_free == 0);
......
......@@ -33,6 +33,7 @@ flush (CACHEFILE f __attribute__((__unused__)),
static int
fetch (CACHEFILE f __attribute__((__unused__)),
PAIR UU(p),
int UU(fd),
CACHEKEY k __attribute__((__unused__)),
uint32_t fullhash __attribute__((__unused__)),
......@@ -88,6 +89,7 @@ pe_callback (
)
{
*bytes_freed = make_pair_attr(bytes_to_free.size-1);
usleep(1*1024*1024);
if (verbose) printf("calling pe_callback\n");
expected_bytes_to_free--;
int* CAST_FROM_VOIDP(foo, ftnode_pv);
......@@ -114,6 +116,8 @@ cachetable_test (void) {
int r;
CACHETABLE ct;
r = toku_create_cachetable(&ct, test_limit, ZERO_LSN, NULL_LOGGER); assert(r == 0);
evictor_test_helpers::set_hysteresis_limits(&ct->ev, test_limit, 100*test_limit);
evictor_test_helpers::disable_ev_thread(&ct->ev);
char fname1[] = __SRCFILE__ "test1.dat";
unlink(fname1);
CACHEFILE f1;
......@@ -129,7 +133,7 @@ cachetable_test (void) {
wc.pe_est_callback = pe_est_callback;
wc.pe_callback = pe_callback;
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, fetch, def_pf_req_callback, def_pf_callback, true, NULL);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(4));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(4));
}
for (int i = 0; i < 8; i++) {
CACHETABLE_WRITE_CALLBACK wc = def_write_callback(NULL);
......@@ -137,7 +141,7 @@ cachetable_test (void) {
wc.pe_est_callback = pe_est_callback;
wc.pe_callback = pe_callback;
r = toku_cachetable_get_and_pin(f1, make_blocknum(2), 2, &v2, &s2, wc, fetch, def_pf_req_callback, def_pf_callback, true, NULL);
r = toku_cachetable_unpin(f1, make_blocknum(2), 2, CACHETABLE_CLEAN, make_pair_attr(4));
r = toku_test_cachetable_unpin(f1, make_blocknum(2), 2, CACHETABLE_CLEAN, make_pair_attr(4));
}
for (int i = 0; i < 4; i++) {
CACHETABLE_WRITE_CALLBACK wc = def_write_callback(NULL);
......@@ -145,7 +149,7 @@ cachetable_test (void) {
wc.pe_est_callback = pe_est_callback;
wc.pe_callback = pe_callback;
r = toku_cachetable_get_and_pin(f1, make_blocknum(3), 3, &v2, &s2, wc, fetch, def_pf_req_callback, def_pf_callback, true, NULL);
r = toku_cachetable_unpin(f1, make_blocknum(3), 3, CACHETABLE_CLEAN, make_pair_attr(4));
r = toku_test_cachetable_unpin(f1, make_blocknum(3), 3, CACHETABLE_CLEAN, make_pair_attr(4));
}
for (int i = 0; i < 2; i++) {
CACHETABLE_WRITE_CALLBACK wc = def_write_callback(NULL);
......@@ -153,7 +157,7 @@ cachetable_test (void) {
wc.pe_est_callback = pe_est_callback;
wc.pe_callback = pe_callback;
r = toku_cachetable_get_and_pin(f1, make_blocknum(4), 4, &v2, &s2, wc, fetch, def_pf_req_callback, def_pf_callback, true, NULL);
r = toku_cachetable_unpin(f1, make_blocknum(4), 4, CACHETABLE_CLEAN, make_pair_attr(4));
r = toku_test_cachetable_unpin(f1, make_blocknum(4), 4, CACHETABLE_CLEAN, make_pair_attr(4));
}
flush_may_occur = false;
expected_bytes_to_free = 4;
......@@ -161,9 +165,10 @@ cachetable_test (void) {
wc.flush_callback = other_flush;
wc.pe_est_callback = pe_est_callback;
wc.pe_callback = other_pe_callback;
r = toku_cachetable_put(f1, make_blocknum(5), 5, NULL, make_pair_attr(4), wc);
r = toku_cachetable_put(f1, make_blocknum(5), 5, NULL, make_pair_attr(4), wc, put_callback_nop);
flush_may_occur = true;
r = toku_cachetable_unpin(f1, make_blocknum(5), 5, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(5), 5, CACHETABLE_CLEAN, make_pair_attr(8));
ct->ev.signal_eviction_thread();
// we are testing that having a wildly different estimate than
// what actually gets freed is ok
......@@ -171,7 +176,7 @@ cachetable_test (void) {
// whereas in reality, only 1 byte will be freed
// we measure that only 1 byte gets freed (which leaves cachetable
// oversubscrubed)
usleep(2*1024*1024);
usleep(3*1024*1024);
assert(expected_bytes_to_free == 3);
......
......@@ -46,6 +46,7 @@ flush (CACHEFILE f __attribute__((__unused__)),
static int
fetch (CACHEFILE f __attribute__((__unused__)),
PAIR UU(p),
int UU(fd),
CACHEKEY k __attribute__((__unused__)),
uint32_t fullhash __attribute__((__unused__)),
......@@ -111,26 +112,26 @@ cachetable_test (void) {
wc.pe_callback = pe_callback;
for (int i = 0; i < 100000; i++) {
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, fetch, def_pf_req_callback, def_pf_callback, true, NULL);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(1));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(1));
}
for (int i = 0; i < 8; i++) {
r = toku_cachetable_get_and_pin(f1, make_blocknum(2), 2, &v2, &s2, wc, fetch, def_pf_req_callback, def_pf_callback, true, NULL);
r = toku_cachetable_unpin(f1, make_blocknum(2), 2, CACHETABLE_CLEAN, make_pair_attr(1));
r = toku_test_cachetable_unpin(f1, make_blocknum(2), 2, CACHETABLE_CLEAN, make_pair_attr(1));
}
for (int i = 0; i < 4; i++) {
r = toku_cachetable_get_and_pin(f1, make_blocknum(3), 3, &v2, &s2, wc, fetch, def_pf_req_callback, def_pf_callback, true, NULL);
r = toku_cachetable_unpin(f1, make_blocknum(3), 3, CACHETABLE_CLEAN, make_pair_attr(1));
r = toku_test_cachetable_unpin(f1, make_blocknum(3), 3, CACHETABLE_CLEAN, make_pair_attr(1));
}
for (int i = 0; i < 2; i++) {
r = toku_cachetable_get_and_pin(f1, make_blocknum(4), 4, &v2, &s2, wc, fetch, def_pf_req_callback, def_pf_callback, true, NULL);
r = toku_cachetable_unpin(f1, make_blocknum(4), 4, CACHETABLE_CLEAN, make_pair_attr(1));
r = toku_test_cachetable_unpin(f1, make_blocknum(4), 4, CACHETABLE_CLEAN, make_pair_attr(1));
}
flush_may_occur = true;
expected_flushed_key = 4;
r = toku_cachetable_put(f1, make_blocknum(5), 5, NULL, make_pair_attr(4), wc);
r = toku_cachetable_put(f1, make_blocknum(5), 5, NULL, make_pair_attr(4), wc, put_callback_nop);
flush_may_occur = true;
expected_flushed_key = 5;
r = toku_cachetable_unpin(f1, make_blocknum(5), 5, CACHETABLE_CLEAN, make_pair_attr(4));
r = toku_test_cachetable_unpin(f1, make_blocknum(5), 5, CACHETABLE_CLEAN, make_pair_attr(4));
check_flush = false;
r = toku_cachefile_close(&f1, 0, false, ZERO_LSN); assert(r == 0 );
......
......@@ -41,8 +41,9 @@ flush (
}
static void *run_end_checkpoint(void *arg) {
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
int r = toku_cachetable_end_checkpoint(
ct,
cp,
NULL,
NULL,
NULL
......@@ -72,9 +73,10 @@ cachetable_test (void) {
wc.clone_callback = clone_callback;
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
assert_zero(r);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8));
assert_zero(r);
r = toku_cachetable_begin_checkpoint(ct, NULL);
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
r = toku_cachetable_begin_checkpoint(cp, NULL);
clone_flush_started = false;
......@@ -88,7 +90,7 @@ cachetable_test (void) {
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
assert_zero(r);
assert(clone_flush_started && !clone_flush_completed);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
assert_zero(r);
void *ret;
......
......@@ -67,15 +67,16 @@ cachetable_test (void) {
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
assert_zero(r);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8));
assert_zero(r);
flush_completed = false;
r = toku_cachetable_begin_checkpoint(ct, NULL); assert_zero(r);
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
r = toku_cachetable_begin_checkpoint(cp, NULL); assert_zero(r);
assert_zero(r);
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
assert_zero(r);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
assert_zero(r);
pf_called = false;
......@@ -85,12 +86,12 @@ cachetable_test (void) {
toku_cachetable_pf_pinned_pair(v1, true_pf_callback, NULL, f1, make_blocknum(1), 1);
assert(pf_called);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
assert_zero(r);
assert(pf_called);
r = toku_cachetable_end_checkpoint(
ct,
cp,
NULL,
NULL,
NULL
......
......@@ -71,26 +71,27 @@ cachetable_test (void) {
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
assert_zero(r);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8));
assert_zero(r);
flush_completed = false;
r = toku_cachetable_begin_checkpoint(ct, NULL); assert_zero(r);
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
r = toku_cachetable_begin_checkpoint(cp, NULL); assert_zero(r);
assert_zero(r);
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
assert_zero(r);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
assert_zero(r);
pf_called = false;
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, true_pf_req_callback, true_pf_callback, true, NULL);
assert_zero(r);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
assert_zero(r);
assert(pf_called);
r = toku_cachetable_end_checkpoint(
ct,
cp,
NULL,
NULL,
NULL
......
......@@ -51,13 +51,14 @@ cachetable_test (enum cachetable_dirty dirty, bool cloneable) {
wc.flush_callback = flush;
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, dirty, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, dirty, make_pair_attr(8));
// test that having a pin that passes false for may_modify_value does not stall behind checkpoint
r = toku_cachetable_begin_checkpoint(ct, NULL); assert_zero(r);
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
r = toku_cachetable_begin_checkpoint(cp, NULL); assert_zero(r);
r = toku_cachetable_get_and_pin_nonblocking(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, false, NULL, NULL);
assert(r == 0);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
assert(r == 0);
r = toku_cachetable_get_and_pin_nonblocking(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL, NULL);
......@@ -66,11 +67,11 @@ cachetable_test (enum cachetable_dirty dirty, bool cloneable) {
}
else {
assert(r == 0);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
}
r = toku_cachetable_end_checkpoint(
ct,
cp,
NULL,
NULL,
NULL
......
......@@ -65,21 +65,22 @@ cachetable_test (void) {
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), toku_cachetable_hash(f1, make_blocknum(1)), &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
assert_zero(r);
r = toku_cachetable_unpin(f1, make_blocknum(1), toku_cachetable_hash(f1, make_blocknum(1)), CACHETABLE_DIRTY, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), toku_cachetable_hash(f1, make_blocknum(1)), CACHETABLE_DIRTY, make_pair_attr(8));
assert_zero(r);
flush_completed = false;
evict_called = false;
r = toku_cachetable_begin_checkpoint(ct, NULL); assert_zero(r);
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
r = toku_cachetable_begin_checkpoint(cp, NULL); assert_zero(r);
assert_zero(r);
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), toku_cachetable_hash(f1, make_blocknum(1)), &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
assert_zero(r);
r = toku_cachetable_unpin_and_remove(f1, make_blocknum(1), NULL, NULL);
r = toku_test_cachetable_unpin_and_remove(f1, make_blocknum(1), NULL, NULL);
assert_zero(r);
r = toku_cachetable_end_checkpoint(
ct,
cp,
NULL,
NULL,
NULL
......
......@@ -23,7 +23,7 @@ cachetable_count_pinned_test (int n) {
uint32_t hi;
hi = toku_cachetable_hash(f1, make_blocknum(i));
CACHETABLE_WRITE_CALLBACK wc = def_write_callback(NULL);
r = toku_cachetable_put(f1, make_blocknum(i), hi, (void *)(long)i, make_pair_attr(1), wc);
r = toku_cachetable_put(f1, make_blocknum(i), hi, (void *)(long)i, make_pair_attr(1), wc, put_callback_nop);
assert(r == 0);
assert(toku_cachefile_count_pinned(f1, 0) == i);
......@@ -32,14 +32,14 @@ cachetable_count_pinned_test (int n) {
assert(r == -1);
assert(toku_cachefile_count_pinned(f1, 0) == i);
//r = toku_cachetable_unpin(f1, make_blocknum(i), hi, CACHETABLE_CLEAN, 1);
//r = toku_test_cachetable_unpin(f1, make_blocknum(i), hi, CACHETABLE_CLEAN, 1);
//assert(r == 0);
assert(toku_cachefile_count_pinned(f1, 0) == i);
}
for (i=n; i>0; i--) {
uint32_t hi;
hi = toku_cachetable_hash(f1, make_blocknum(i));
r = toku_cachetable_unpin(f1, make_blocknum(i), hi, CACHETABLE_CLEAN, make_pair_attr(1));
r = toku_test_cachetable_unpin(f1, make_blocknum(i), hi, CACHETABLE_CLEAN, make_pair_attr(1));
assert(r == 0);
if (i-1) assert(toku_cachetable_assert_all_unpinned(ct));
assert(toku_cachefile_count_pinned(f1, 0) == i-1);
......
......@@ -30,7 +30,7 @@ cachetable_debug_test (int n) {
uint32_t hi;
hi = toku_cachetable_hash(f1, make_blocknum(i));
CACHETABLE_WRITE_CALLBACK wc = def_write_callback(NULL);
r = toku_cachetable_put(f1, make_blocknum(i), hi, (void *)(long)i, make_pair_attr(item_size), wc);
r = toku_cachetable_put(f1, make_blocknum(i), hi, (void *)(long)i, make_pair_attr(item_size), wc, put_callback_nop);
assert(r == 0);
void *v; int dirty; long long pinned; long pair_size;
......@@ -41,7 +41,7 @@ cachetable_debug_test (int n) {
assert(pinned == 1);
assert(pair_size == item_size);
r = toku_cachetable_unpin(f1, make_blocknum(i), hi, CACHETABLE_CLEAN, make_pair_attr(1));
r = toku_test_cachetable_unpin(f1, make_blocknum(i), hi, CACHETABLE_CLEAN, make_pair_attr(1));
assert(r == 0);
toku_cachetable_get_state(ct, &num_entries, &hash_size, &size_current, &size_limit);
......
......@@ -33,6 +33,7 @@ static int fetch_calls = 0;
static int
fetch (CACHEFILE f __attribute__((__unused__)),
PAIR UU(p),
int UU(fd),
CACHEKEY k __attribute__((__unused__)),
uint32_t fullhash __attribute__((__unused__)),
......@@ -104,7 +105,7 @@ static void cachetable_eviction_full_test (void) {
0
);
assert(r==0);
r = toku_cachetable_unpin(f1, key, fullhash, CACHETABLE_DIRTY, make_pair_attr(1));
r = toku_test_cachetable_unpin(f1, key, fullhash, CACHETABLE_DIRTY, make_pair_attr(1));
assert(r == 0);
}
expect_full_flush = true;
......@@ -125,7 +126,7 @@ static void cachetable_eviction_full_test (void) {
0
);
assert(r==0);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(1));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(1));
assert(r == 0);
toku_cachetable_verify(ct);
......
......@@ -32,6 +32,7 @@ static int fetch_calls = 0;
static int
fetch (CACHEFILE f __attribute__((__unused__)),
PAIR UU(p),
int UU(fd),
CACHEKEY k __attribute__((__unused__)),
uint32_t fullhash __attribute__((__unused__)),
......@@ -117,7 +118,7 @@ static void cachetable_eviction_full_test (void) {
0
);
assert(r==0);
r = toku_cachetable_unpin(f1, key, fullhash, CACHETABLE_DIRTY, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, key, fullhash, CACHETABLE_DIRTY, make_pair_attr(8));
assert(r == 0);
}
expect_full_flush = true;
......@@ -139,7 +140,7 @@ static void cachetable_eviction_full_test (void) {
0
);
assert(r==0);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(1));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(1));
assert(r == 0);
toku_cachetable_verify(ct);
......
......@@ -24,7 +24,7 @@ flush (CACHEFILE f __attribute__((__unused__)),
bool UU(is_clone)
) {
if (do_sleep) {
sleep(2);
sleep(3);
}
}
......@@ -39,6 +39,7 @@ static void cachetable_predef_fetch_maybegetandpin_test (void) {
int r;
CACHETABLE ct;
r = toku_create_cachetable(&ct, test_limit, ZERO_LSN, NULL_LOGGER); assert(r == 0);
evictor_test_helpers::disable_ev_thread(&ct->ev);
char fname1[] = __SRCFILE__ "test1.dat";
unlink(fname1);
CACHEFILE f1;
......@@ -66,7 +67,7 @@ static void cachetable_predef_fetch_maybegetandpin_test (void) {
0
);
assert(r==0);
r = toku_cachetable_unpin(f1, key, fullhash, CACHETABLE_DIRTY, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, key, fullhash, CACHETABLE_DIRTY, make_pair_attr(8));
}
struct timeval tstart;
......@@ -91,13 +92,14 @@ static void cachetable_predef_fetch_maybegetandpin_test (void) {
0
);
assert(r==0);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
ct->ev.signal_eviction_thread();
usleep(1*1024*1024);
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
toku_cachetable_verify(ct);
void *v = 0;
long size = 0;
// now verify that the block we are trying to evict may be pinned
// now verify that the block we are trying to evict is gone
wc = def_write_callback(NULL);
wc.flush_callback = flush;
r = toku_cachetable_get_and_pin_nonblocking(f1, key, fullhash, &v, &size, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL, NULL);
......@@ -113,7 +115,7 @@ static void cachetable_predef_fetch_maybegetandpin_test (void) {
if (verbose)printf("time %" PRIu64 " \n", tdelta_usec(&tend, &tstart));
toku_cachetable_verify(ct);
r = toku_cachetable_unpin(f1, key, fullhash, CACHETABLE_CLEAN, make_pair_attr(1));
r = toku_test_cachetable_unpin(f1, key, fullhash, CACHETABLE_CLEAN, make_pair_attr(1));
assert(r == 0);
toku_cachetable_verify(ct);
......
......@@ -29,7 +29,7 @@ pe_callback (
void* extraargs __attribute__((__unused__))
)
{
sleep(2);
sleep(3);
*bytes_freed = make_pair_attr(bytes_to_free.size-7);
return 0;
}
......@@ -45,6 +45,7 @@ static void cachetable_prefetch_maybegetandpin_test (void) {
int r;
CACHETABLE ct;
r = toku_create_cachetable(&ct, test_limit, ZERO_LSN, NULL_LOGGER); assert(r == 0);
evictor_test_helpers::disable_ev_thread(&ct->ev);
char fname1[] = __SRCFILE__ "test1.dat";
unlink(fname1);
CACHEFILE f1;
......@@ -73,7 +74,7 @@ static void cachetable_prefetch_maybegetandpin_test (void) {
0
);
assert(r==0);
r = toku_cachetable_unpin(f1, key, fullhash, CACHETABLE_DIRTY, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, key, fullhash, CACHETABLE_DIRTY, make_pair_attr(8));
}
struct timeval tstart;
......@@ -96,7 +97,10 @@ static void cachetable_prefetch_maybegetandpin_test (void) {
0
);
assert(r==0);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
ct->ev.signal_eviction_thread();
usleep(1*1024*1024);
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
toku_cachetable_verify(ct);
......@@ -140,7 +144,7 @@ static void cachetable_prefetch_maybegetandpin_test (void) {
if (verbose) printf("time %" PRIu64 " \n", tdelta_usec(&tend, &tstart));
toku_cachetable_verify(ct);
r = toku_cachetable_unpin(f1, key, fullhash, CACHETABLE_CLEAN, make_pair_attr(1));
r = toku_test_cachetable_unpin(f1, key, fullhash, CACHETABLE_CLEAN, make_pair_attr(1));
assert(r == 0);
toku_cachetable_verify(ct);
......
/* -*- mode: C++; c-basic-offset: 4; indent-tabs-mode: nil -*- */
// vim: ft=cpp:expandtab:ts=8:sw=4:softtabstop=4:
#ident "$Id: cachetable-simple-verify.cc 45903 2012-07-19 13:06:39Z leifwalsh $"
#ident "Copyright (c) 2007-2012 Tokutek Inc. All rights reserved."
#include "includes.h"
#include "test.h"
#include "cachetable-internal.h"
class evictor_unit_test {
public:
evictor m_ev;
pair_list m_pl;
KIBBUTZ m_kb;
void init();
void destroy();
void run_test();
void verify_ev_init(long limit);
void verify_ev_destroy();
void verify_ev_counts();
void verify_ev_m_size_reserved();
void verify_ev_handling_cache_pressure();
// function to disable the eviction thread from waking up every second
void disable_ev_thread();
};
// initialize this class to run tests
void evictor_unit_test::init() {
m_pl.init();
m_kb = toku_kibbutz_create(1);
}
// destroy class after tests have run
void evictor_unit_test::destroy() {
m_pl.destroy();
toku_kibbutz_destroy(m_kb);
}
// test that verifies evictor.init properly worked
void evictor_unit_test::verify_ev_init(long limit) {
assert(m_ev.m_kibbutz == m_kb);
assert(m_ev.m_pl == &m_pl);
assert(m_ev.m_low_size_watermark == limit);
assert(m_ev.m_num_sleepers == 0);
assert(m_ev.m_run_thread == true);
assert(m_ev.m_size_current == 0);
assert(m_ev.m_size_leaf== 0);
assert(m_ev.m_size_nonleaf== 0);
assert(m_ev.m_size_rollback== 0);
assert(m_ev.m_size_cachepressure == 0);
assert(m_ev.m_size_evicting == 0);
// this comes from definition of unreservable_memory in cachetable.cc
assert(m_ev.m_size_reserved == (limit/4));
}
// test that verifies evictor.destroy properly worked
void evictor_unit_test::verify_ev_destroy() {
assert(m_ev.m_num_sleepers == 0);
assert(m_ev.m_run_thread == false);
}
void evictor_unit_test::disable_ev_thread() {
toku_mutex_lock(&m_ev.m_ev_thread_lock);
m_ev.m_period_in_seconds = 0;
// signal eviction thread so that it wakes up
// and then sleeps indefinitely
m_ev.signal_eviction_thread();
toku_mutex_unlock(&m_ev.m_ev_thread_lock);
// sleep for one second to ensure eviction thread picks up new period
usleep(1*1024*1024);
}
// test that verifies that counts, such as m_size_current
// are accurately maintained
void evictor_unit_test::verify_ev_counts() {
long limit = 10;
long expected_m_size_reserved = limit/4;
m_ev.init(limit, &m_pl, m_kb, 0);
this->verify_ev_init(limit);
m_ev.add_to_size_current(1);
assert(m_ev.m_size_current == 1);
assert(m_ev.m_size_reserved == expected_m_size_reserved);
assert(m_ev.m_size_leaf == 0);
assert(m_ev.m_size_nonleaf == 0);
assert(m_ev.m_size_rollback == 0);
assert(m_ev.m_size_cachepressure == 0);
assert(m_ev.m_size_evicting == 0);
m_ev.add_to_size_current(3);
assert(m_ev.m_size_current == 4);
m_ev.remove_from_size_current(4);
assert(m_ev.m_size_current == 0);
assert(m_ev.m_size_reserved == expected_m_size_reserved);
PAIR_ATTR attr = {
.size = 1,
.nonleaf_size = 2,
.leaf_size = 3,
.rollback_size = 4,
.cache_pressure_size = 5,
.is_valid = true
};
m_ev.add_pair_attr(attr);
assert(m_ev.m_size_current == 1);
assert(m_ev.m_size_nonleaf == 2);
assert(m_ev.m_size_leaf == 3);
assert(m_ev.m_size_rollback == 4);
assert(m_ev.m_size_cachepressure == 5);
m_ev.remove_pair_attr(attr);
assert(m_ev.m_size_current == 0);
assert(m_ev.m_size_leaf == 0);
assert(m_ev.m_size_nonleaf == 0);
assert(m_ev.m_size_rollback == 0);
assert(m_ev.m_size_cachepressure == 0);
PAIR_ATTR other_attr = {
.size = 2,
.nonleaf_size = 3,
.leaf_size = 4,
.rollback_size = 5,
.cache_pressure_size = 6,
.is_valid = true
};
m_ev.change_pair_attr(attr, other_attr);
assert(m_ev.m_size_current == 1);
assert(m_ev.m_size_leaf == 1);
assert(m_ev.m_size_nonleaf == 1);
assert(m_ev.m_size_rollback == 1);
assert(m_ev.m_size_cachepressure == 1);
m_ev.destroy();
this->verify_ev_destroy();
}
// test to verify the functionality surrounding m_size_reserved
void evictor_unit_test::verify_ev_m_size_reserved() {
long limit = 400;
long expected_m_size_reserved = 100; //limit/4
m_ev.init(limit, &m_pl, m_kb, 0);
this->verify_ev_init(limit);
assert(m_ev.m_size_reserved == expected_m_size_reserved);
m_ev.m_num_eviction_thread_runs = 0;
m_ev.reserve_memory(0.5);
assert(m_ev.m_size_reserved == 100+150); //100 original, 150 from last call
assert(m_ev.m_size_current == 150);
assert(m_ev.m_size_evicting == 0);
usleep(1*1024*1024); // sleep to give eviction thread a chance to wake up
assert(m_ev.m_num_eviction_thread_runs == 1);
m_ev.destroy();
this->verify_ev_destroy();
}
// test to verify functionality of handling cache pressure,
// ensures that wait_for_cache_pressure_to_subside works correctly,
// that decrease_m_size_evicting works correctly, and the logic for when to wake
// threads up works correctly
void evictor_unit_test::verify_ev_handling_cache_pressure() {
long limit = 400;
m_ev.init(limit, &m_pl, m_kb, 0);
this->verify_ev_init(limit);
m_ev.m_low_size_watermark = 400;
m_ev.m_low_size_hysteresis = 400;
m_ev.m_high_size_hysteresis = 500;
m_ev.m_high_size_watermark = 500;
m_ev.m_size_current = 500;
m_ev.m_num_eviction_thread_runs = 0;
// test that waiting for cache pressure wakes eviction thread
assert(m_ev.m_num_sleepers == 0);
m_ev.wait_for_cache_pressure_to_subside();
assert(m_ev.m_num_eviction_thread_runs == 1);
assert(m_ev.m_num_sleepers == 0);
m_ev.m_num_eviction_thread_runs = 0;
m_ev.m_size_evicting = 101;
m_ev.decrease_size_evicting(101);
usleep(1*1024*1024);
// should not have been signaled because we have no sleepers
assert(m_ev.m_num_eviction_thread_runs == 0);
m_ev.m_num_eviction_thread_runs = 0;
m_ev.m_size_evicting = 101;
m_ev.m_num_sleepers = 1;
m_ev.decrease_size_evicting(2);
usleep(1*1024*1024);
// should have been signaled because we have sleepers
assert(m_ev.m_num_eviction_thread_runs == 1);
assert(m_ev.m_num_sleepers == 1); // make sure fake sleeper did not go away
m_ev.m_num_eviction_thread_runs = 0;
m_ev.m_size_evicting = 102;
m_ev.m_num_sleepers = 1;
m_ev.decrease_size_evicting(1);
usleep(1*1024*1024);
// should not have been signaled because we did not go to less than 100
assert(m_ev.m_num_eviction_thread_runs == 0);
assert(m_ev.m_num_sleepers == 1); // make sure fake sleeper did not go away
m_ev.m_size_evicting = 0;
m_ev.m_num_sleepers = 0;
m_ev.destroy();
this->verify_ev_destroy();
}
void evictor_unit_test::run_test() {
this->verify_ev_counts();
this->verify_ev_m_size_reserved();
this->verify_ev_handling_cache_pressure();
return;
//this->disable_ev_thread();
//usleep(9*1024*1024);
}
int
test_main(int argc, const char *argv[]) {
default_parse_args(argc, argv);
evictor_unit_test ev_test;
ev_test.init();
ev_test.run_test();
ev_test.destroy();
return 0;
}
/* -*- mode: C++; c-basic-offset: 4; indent-tabs-mode: nil -*- */
// vim: ft=cpp:expandtab:ts=8:sw=4:softtabstop=4:
#ident "$Id: cachetable-simple-verify.cc 45903 2012-07-19 13:06:39Z leifwalsh $"
#ident "Copyright (c) 2007-2012 Tokutek Inc. All rights reserved."
#include "includes.h"
#include "test.h"
bool pf_called;
enum pin_evictor_test_type {
pin_in_memory,
pin_fetch,
pin_partial_fetch
};
static bool pf_req_callback(void* UU(ftnode_pv), void* UU(read_extraargs)) {
return true;
}
static int pf_callback(void* UU(ftnode_pv), void* UU(disk_data), void* UU(read_extraargs), int UU(fd), PAIR_ATTR* sizep) {
*sizep = make_pair_attr(8);
return 0;
}
static void
cachetable_test (enum pin_evictor_test_type test_type, bool nonblocking) {
const int test_limit = 7;
int r;
CACHETABLE ct;
r = toku_create_cachetable(&ct, test_limit, ZERO_LSN, NULL_LOGGER); assert(r == 0);
evictor_test_helpers::set_hysteresis_limits(&ct->ev, test_limit, test_limit);
evictor_test_helpers::disable_ev_thread(&ct->ev);
char fname1[] = __SRCFILE__ "test1.dat";
unlink(fname1);
CACHEFILE f1;
r = toku_cachetable_openf(&f1, ct, fname1, O_RDWR|O_CREAT, S_IRWXU|S_IRWXG|S_IRWXO); assert(r == 0);
void* v1;
long s1;
CACHETABLE_WRITE_CALLBACK wc = def_write_callback(NULL);
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
// at this point, we should have 8 bytes of data in a cachetable that supports 7
// adding data via get_and_pin or get_and_pin_nonblocking should induce eviction
uint64_t old_num_ev_runs = 0;
uint64_t new_num_ev_runs = 0;
if (test_type == pin_in_memory) {
old_num_ev_runs = evictor_test_helpers::get_num_eviction_runs(&ct->ev);
if (nonblocking) {
r = toku_cachetable_get_and_pin_nonblocking(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL, NULL);
assert_zero(r);
}
else {
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
assert_zero(r);
}
new_num_ev_runs = evictor_test_helpers::get_num_eviction_runs(&ct->ev);
assert(new_num_ev_runs == old_num_ev_runs);
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
assert_zero(r);
}
else if (test_type == pin_fetch) {
old_num_ev_runs = evictor_test_helpers::get_num_eviction_runs(&ct->ev);
if (nonblocking) {
r = toku_cachetable_get_and_pin_nonblocking(f1, make_blocknum(2), 2, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL, NULL);
assert(r == TOKUDB_TRY_AGAIN);
new_num_ev_runs = evictor_test_helpers::get_num_eviction_runs(&ct->ev);
assert(new_num_ev_runs > old_num_ev_runs);
}
else {
r = toku_cachetable_get_and_pin(f1, make_blocknum(2), 2, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
assert_zero(r);
new_num_ev_runs = evictor_test_helpers::get_num_eviction_runs(&ct->ev);
assert(new_num_ev_runs > old_num_ev_runs);
r = toku_test_cachetable_unpin(f1, make_blocknum(2), 2, CACHETABLE_CLEAN, make_pair_attr(8));
assert_zero(r);
}
}
else if (test_type == pin_partial_fetch) {
old_num_ev_runs = evictor_test_helpers::get_num_eviction_runs(&ct->ev);
if (nonblocking) {
r = toku_cachetable_get_and_pin_nonblocking(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, pf_req_callback, pf_callback, true, NULL, NULL);
assert(r == TOKUDB_TRY_AGAIN);
new_num_ev_runs = evictor_test_helpers::get_num_eviction_runs(&ct->ev);
assert(new_num_ev_runs > old_num_ev_runs);
}
else {
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, pf_req_callback, pf_callback, true, NULL);
assert_zero(r);
new_num_ev_runs = evictor_test_helpers::get_num_eviction_runs(&ct->ev);
assert(new_num_ev_runs > old_num_ev_runs);
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
assert_zero(r);
}
}
else {
assert(false);
}
toku_cachetable_verify(ct);
r = toku_cachefile_close(&f1, 0, false, ZERO_LSN); assert(r == 0);
r = toku_cachetable_close(&ct); lazy_assert_zero(r);
}
int
test_main(int argc, const char *argv[]) {
default_parse_args(argc, argv);
cachetable_test(pin_in_memory, true);
cachetable_test(pin_fetch, true);
cachetable_test(pin_partial_fetch, true);
cachetable_test(pin_in_memory, false);
cachetable_test(pin_fetch, false);
cachetable_test(pin_partial_fetch, false);
return 0;
}
......@@ -15,7 +15,7 @@ cleaner_callback(
void* UU(extraargs)
)
{
int r = toku_cachetable_unpin(f1,blocknum, fullhash,CACHETABLE_CLEAN,make_pair_attr(8));
int r = toku_test_cachetable_unpin(f1,blocknum, fullhash,CACHETABLE_CLEAN,make_pair_attr(8));
assert(r==0);
return 0;
}
......@@ -41,7 +41,7 @@ cachetable_test (void) {
CACHETABLE_WRITE_CALLBACK wc = def_write_callback(NULL);
wc.cleaner_callback = cleaner_callback;
r = toku_cachetable_get_and_pin(f1, make_blocknum(i), i, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
r = toku_cachetable_unpin(f1, make_blocknum(i), i, CACHETABLE_DIRTY, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(i), i, CACHETABLE_DIRTY, make_pair_attr(8));
}
r = toku_cachefile_flush(f1);
assert(r == 0);
......
......@@ -28,14 +28,14 @@ test_cachetable_def_flush (int n) {
for (i=0; i<n; i++) {
uint32_t hi;
hi = toku_cachetable_hash(f1, make_blocknum(i));
r = toku_cachetable_put(f1, make_blocknum(i), hi, (void *)(long)i, make_pair_attr(1), wc);
r = toku_cachetable_put(f1, make_blocknum(i), hi, (void *)(long)i, make_pair_attr(1), wc, put_callback_nop);
assert(r == 0);
r = toku_cachetable_unpin(f1, make_blocknum(i), hi, CACHETABLE_CLEAN, make_pair_attr(1));
r = toku_test_cachetable_unpin(f1, make_blocknum(i), hi, CACHETABLE_CLEAN, make_pair_attr(1));
assert(r == 0);
hi = toku_cachetable_hash(f2, make_blocknum(i));
r = toku_cachetable_put(f2, make_blocknum(i), hi, (void *)(long)i, make_pair_attr(1), wc);
r = toku_cachetable_put(f2, make_blocknum(i), hi, (void *)(long)i, make_pair_attr(1), wc, put_callback_nop);
assert(r == 0);
r = toku_cachetable_unpin(f2, make_blocknum(i), hi, CACHETABLE_CLEAN, make_pair_attr(1));
r = toku_test_cachetable_unpin(f2, make_blocknum(i), hi, CACHETABLE_CLEAN, make_pair_attr(1));
assert(r == 0);
}
toku_cachetable_verify(ct);
......@@ -47,12 +47,12 @@ test_cachetable_def_flush (int n) {
hi = toku_cachetable_hash(f1, make_blocknum(i));
r = toku_cachetable_maybe_get_and_pin(f1, make_blocknum(i), hi, &v);
assert(r == 0 && v == (void *)(long)i);
r = toku_cachetable_unpin(f1, make_blocknum(i), hi, CACHETABLE_CLEAN, make_pair_attr(1));
r = toku_test_cachetable_unpin(f1, make_blocknum(i), hi, CACHETABLE_CLEAN, make_pair_attr(1));
assert(r == 0);
hi = toku_cachetable_hash(f2, make_blocknum(i));
r = toku_cachetable_maybe_get_and_pin(f2, make_blocknum(i), hi, &v);
assert(r == 0 && v == (void *)(long)i);
r = toku_cachetable_unpin(f2, make_blocknum(i), hi, CACHETABLE_CLEAN, make_pair_attr(1));
r = toku_test_cachetable_unpin(f2, make_blocknum(i), hi, CACHETABLE_CLEAN, make_pair_attr(1));
assert(r == 0);
}
......@@ -70,7 +70,7 @@ test_cachetable_def_flush (int n) {
hi = toku_cachetable_hash(f2, make_blocknum(i));
r = toku_cachetable_maybe_get_and_pin(f2, make_blocknum(i), hi, &v);
assert(r == 0);
r = toku_cachetable_unpin(f2, make_blocknum(i), hi, CACHETABLE_CLEAN, make_pair_attr(1));
r = toku_test_cachetable_unpin(f2, make_blocknum(i), hi, CACHETABLE_CLEAN, make_pair_attr(1));
assert(r == 0);
}
......
......@@ -26,18 +26,18 @@ flush (CACHEFILE cf __attribute__((__unused__)),
static int
fetch (
CACHEFILE cf,
CACHEFILE UU(cf),
PAIR UU(p),
int UU(fd),
CACHEKEY key,
uint32_t hash,
uint32_t UU(hash),
void **vptr,
void** UU(dd),
PAIR_ATTR *sizep,
int *dirtyp,
void *extra
void *UU(extra)
)
{
cf = cf; hash = hash; extra = extra;
*sizep = make_pair_attr((long) key.b);
*vptr = toku_malloc(sizep->size);
*dirtyp = 0;
......@@ -68,7 +68,7 @@ cachetable_getandpin_test (int n) {
assert(r == 0);
assert(size == i);
r = toku_cachetable_unpin(f1, make_blocknum(i), hi, CACHETABLE_CLEAN, make_pair_attr(i));
r = toku_test_cachetable_unpin(f1, make_blocknum(i), hi, CACHETABLE_CLEAN, make_pair_attr(i));
assert(r == 0);
}
toku_cachetable_verify(ct);
......
......@@ -18,7 +18,7 @@ static void kibbutz_work(void *fe_v)
// note that we make the size 16 to induce an eviction
// once evictions are moved to their own thread, we need
// to modify this test
int r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(16));
int r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(16));
assert(r==0);
remove_background_job_from_cf(f1);
}
......
......@@ -15,6 +15,7 @@ bool pf_req_called;
static int
fetch (CACHEFILE f __attribute__((__unused__)),
PAIR UU(p),
int UU(fd),
CACHEKEY k __attribute__((__unused__)),
uint32_t fullhash __attribute__((__unused__)),
......@@ -32,6 +33,7 @@ fetch (CACHEFILE f __attribute__((__unused__)),
static int
err_fetch (CACHEFILE f __attribute__((__unused__)),
PAIR UU(p),
int UU(fd),
CACHEKEY k __attribute__((__unused__)),
uint32_t fullhash __attribute__((__unused__)),
......@@ -108,12 +110,12 @@ cachetable_test (void) {
assert(r == 0);
// make sure that prefetch should not happen, because we have already pinned node
assert(!doing_prefetch);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
//
// now get and pin node again, and make sure that partial fetch and fetch are not called
//
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, err_fetch, pf_req_callback, err_pf_callback, true, NULL);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
//
// now make sure that if we say a partial fetch is required, that we get a partial fetch
// and that read_extraargs properly passed down
......@@ -122,7 +124,7 @@ cachetable_test (void) {
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, err_fetch, true_pf_req_callback, true_pf_callback, true, &fetch_val);
assert(pf_req_called);
assert(s1 == sizeof(fetch_val)+1);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
// close and reopen cachefile so we can do some simple prefetch tests
r = toku_cachefile_close(&f1, 0, false, ZERO_LSN); assert(r == 0);
......@@ -149,7 +151,7 @@ cachetable_test (void) {
//
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, err_fetch, pf_req_callback, err_pf_callback, true, NULL);
assert(&fetch_val == v1);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
//
// now verify a prefetch that requires a partial fetch works, and that we can then pin the node
......@@ -168,7 +170,7 @@ cachetable_test (void) {
assert(doing_prefetch);
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, err_fetch, pf_req_callback, err_pf_callback, true, NULL);
assert(&fetch_val == v1);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
toku_cachetable_verify(ct);
......
......@@ -70,6 +70,7 @@ flush (CACHEFILE f __attribute__((__unused__)),
static int
fetch (CACHEFILE f __attribute__((__unused__)),
PAIR UU(p),
int UU(fd),
CACHEKEY k,
uint32_t fullhash __attribute__((__unused__)),
......@@ -187,7 +188,7 @@ static void *move_numbers(void *arg) {
usleep(10);
(*first_val)++;
(*second_val)--;
r = toku_cachetable_unpin(f1, less_key, less_fullhash, less_dirty, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, less_key, less_fullhash, less_dirty, make_pair_attr(8));
int third = 0;
int num_possible_values = (NUM_ELEMENTS-1) - greater;
......@@ -219,9 +220,9 @@ static void *move_numbers(void *arg) {
usleep(10);
(*second_val)++;
(*third_val)--;
r = toku_cachetable_unpin(f1, third_key, third_fullhash, third_dirty, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, third_key, third_fullhash, third_dirty, make_pair_attr(8));
}
r = toku_cachetable_unpin(f1, greater_key, greater_fullhash, greater_dirty, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, greater_key, greater_fullhash, greater_dirty, make_pair_attr(8));
}
return arg;
}
......@@ -247,7 +248,7 @@ static void *read_random_numbers(void *arg) {
NULL
);
if (r1 == 0) {
r1 = toku_cachetable_unpin(f1, make_blocknum(rand_key1), rand_key1, CACHETABLE_CLEAN, make_pair_attr(8));
r1 = toku_test_cachetable_unpin(f1, make_blocknum(rand_key1), rand_key1, CACHETABLE_CLEAN, make_pair_attr(8));
assert(r1 == 0);
}
}
......@@ -269,10 +270,10 @@ static void *checkpoints(void *arg) {
// now run a checkpoint
//
int r;
r = toku_cachetable_begin_checkpoint(ct, NULL); assert(r == 0);
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
r = toku_cachetable_begin_checkpoint(cp, NULL); assert(r == 0);
r = toku_cachetable_end_checkpoint(
ct,
cp,
NULL,
NULL,
NULL
......
......@@ -27,17 +27,18 @@ run_test (void) {
long s2;
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, def_write_callback(NULL), def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8)); assert(r==0);
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8)); assert(r==0);
for (int i = 0; i < 20; i++) {
r = toku_cachetable_get_and_pin(f1, make_blocknum(2), 2, &v2, &s2, def_write_callback(NULL), def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
r = toku_cachetable_unpin(f1, make_blocknum(2), 2, CACHETABLE_CLEAN, make_pair_attr(8)); assert(r==0);
r = toku_test_cachetable_unpin(f1, make_blocknum(2), 2, CACHETABLE_CLEAN, make_pair_attr(8)); assert(r==0);
}
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v2, &s2, def_write_callback(NULL), def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
r = toku_cachetable_begin_checkpoint(ct, NULL);
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
r = toku_cachetable_begin_checkpoint(cp, NULL);
// mark nodes as pending a checkpoint, so that get_and_pin_nonblocking on block 1 will return TOKUDB_TRY_AGAIN
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8)); assert(r==0);
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8)); assert(r==0);
r = toku_cachetable_get_and_pin_nonblocking(
f1,
......@@ -54,10 +55,10 @@ run_test (void) {
NULL
);
assert(r==0);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8)); assert(r==0);
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8)); assert(r==0);
r = toku_cachetable_end_checkpoint(
ct,
cp,
NULL,
NULL,
NULL
......
......@@ -17,12 +17,12 @@ const int item_size = 1;
int n_flush, n_write_me, n_keep_me, n_fetch;
static void flush(
CACHEFILE cf,
CACHEFILE UU(cf),
int UU(fd),
CACHEKEY key,
void *value,
CACHEKEY UU(key),
void *UU(value),
void** UU(dd),
void *extraargs,
void *UU(extraargs),
PAIR_ATTR size,
PAIR_ATTR* UU(new_size),
bool write_me,
......@@ -31,7 +31,6 @@ static void flush(
bool UU(is_clone)
)
{
cf = cf; key = key; value = value; extraargs = extraargs;
// assert(key == make_blocknum((long)value));
assert(size.size == item_size);
n_flush++;
......@@ -40,18 +39,18 @@ static void flush(
}
static int fetch(
CACHEFILE cf,
CACHEFILE UU(cf),
PAIR UU(p),
int UU(fd),
CACHEKEY key,
uint32_t fullhash,
void **value,
CACHEKEY UU(key),
uint32_t UU(fullhash),
void **UU(value),
void** UU(dd),
PAIR_ATTR *sizep,
PAIR_ATTR *UU(sizep),
int *dirtyp,
void *extraargs
void *UU(extraargs)
)
{
cf = cf; key = key; fullhash = fullhash; value = value; sizep = sizep; extraargs = extraargs;
n_fetch++;
sleep(10);
*value = 0;
......@@ -93,10 +92,10 @@ static void cachetable_prefetch_checkpoint_test(int n, enum cachetable_dirty dir
for (i=0; i<n; i++) {
CACHEKEY key = make_blocknum(i);
uint32_t hi = toku_cachetable_hash(f1, key);
r = toku_cachetable_put(f1, key, hi, (void *)(long)i, make_pair_attr(1), wc);
r = toku_cachetable_put(f1, key, hi, (void *)(long)i, make_pair_attr(1), wc, put_callback_nop);
assert(r == 0);
r = toku_cachetable_unpin(f1, key, hi, dirty, make_pair_attr(item_size));
r = toku_test_cachetable_unpin(f1, key, hi, dirty, make_pair_attr(item_size));
assert(r == 0);
void *v;
......@@ -114,8 +113,8 @@ static void cachetable_prefetch_checkpoint_test(int n, enum cachetable_dirty dir
// the checkpoint should cause n writes, but since n <= the cachetable size,
// all items should be kept in the cachetable
n_flush = n_write_me = n_keep_me = n_fetch = 0;
r = toku_checkpoint(ct, NULL, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT);
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
r = toku_checkpoint(cp, NULL, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT);
assert(r == 0);
assert(n_flush == n && n_write_me == n && n_keep_me == n);
......@@ -127,7 +126,7 @@ static void cachetable_prefetch_checkpoint_test(int n, enum cachetable_dirty dir
r = toku_cachetable_maybe_get_and_pin(f1, key, hi, &v);
if (r != 0)
continue;
r = toku_cachetable_unpin(f1, key, hi, CACHETABLE_CLEAN, make_pair_attr(item_size));
r = toku_test_cachetable_unpin(f1, key, hi, CACHETABLE_CLEAN, make_pair_attr(item_size));
assert(r == 0);
int its_dirty;
......@@ -144,7 +143,7 @@ static void cachetable_prefetch_checkpoint_test(int n, enum cachetable_dirty dir
// a subsequent checkpoint should cause no flushes, or writes since all of the items are clean
n_flush = n_write_me = n_keep_me = n_fetch = 0;
r = toku_checkpoint(ct, NULL, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT);
r = toku_checkpoint(cp, NULL, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT);
assert(r == 0);
assert(n_flush == 0 && n_write_me == 0 && n_keep_me == 0);
......
......@@ -29,6 +29,7 @@ static int fetch_calls = 0;
static int
fetch (CACHEFILE f __attribute__((__unused__)),
PAIR UU(p),
int UU(fd),
CACHEKEY k __attribute__((__unused__)),
uint32_t fullhash __attribute__((__unused__)),
......
......@@ -30,6 +30,7 @@ static int fetch_calls = 0;
static int
fetch (CACHEFILE f __attribute__((__unused__)),
PAIR UU(p),
int UU(fd),
CACHEKEY k __attribute__((__unused__)),
uint32_t fullhash __attribute__((__unused__)),
......@@ -88,7 +89,7 @@ static void cachetable_prefetch_full_test (bool partial_fetch) {
0
);
assert(r==0);
r = toku_cachetable_unpin(f1, key, fullhash, CACHETABLE_CLEAN, make_pair_attr(1));
r = toku_test_cachetable_unpin(f1, key, fullhash, CACHETABLE_CLEAN, make_pair_attr(1));
}
r = toku_cachefile_prefetch(f1, key, fullhash, wc, fetch, def_pf_req_callback, def_pf_callback, 0, NULL);
......
......@@ -8,6 +8,7 @@
#include "includes.h"
#include "test.h"
#include "cachetable-internal.h"
static int flush_calls = 0;
static int flush_evict_calls = 0;
......@@ -28,10 +29,11 @@ flush (CACHEFILE f __attribute__((__unused__)),
bool UU(is_clone)
) {
assert(w == false);
sleep(1);
flush_calls++;
if (keep == false) {
flush_evict_calls++;
if (verbose) printf("%s:%d flush %" PRId64 "\n", __FUNCTION__, __LINE__, k.b);
if (verbose) printf("%s:%d flush %" PRId64 "\n", __FUNCTION__, __LINE__, k.b);
evicted_keys |= 1 << k.b;
}
}
......@@ -40,6 +42,7 @@ static int fetch_calls = 0;
static int
fetch (CACHEFILE f __attribute__((__unused__)),
PAIR UU(p),
int UU(fd),
CACHEKEY k,
uint32_t fullhash __attribute__((__unused__)),
......@@ -65,6 +68,8 @@ static void cachetable_prefetch_flowcontrol_test (int cachetable_size_limit) {
int r;
CACHETABLE ct;
r = toku_create_cachetable(&ct, cachetable_size_limit, ZERO_LSN, NULL_LOGGER); assert(r == 0);
evictor_test_helpers::set_hysteresis_limits(&ct->ev, cachetable_size_limit, cachetable_size_limit);
evictor_test_helpers::disable_ev_thread(&ct->ev);
char fname1[] = __SRCFILE__ "test1.dat";
unlink(fname1);
CACHEFILE f1;
......@@ -75,40 +80,33 @@ static void cachetable_prefetch_flowcontrol_test (int cachetable_size_limit) {
wc.flush_callback = flush;
// prefetch keys 0 .. N-1. they should all fit in the cachetable
for (i=0; i<cachetable_size_limit; i++) {
for (i=0; i<cachetable_size_limit+1; i++) {
CACHEKEY key = make_blocknum(i);
uint32_t fullhash = toku_cachetable_hash(f1, key);
r = toku_cachefile_prefetch(f1, key, fullhash, wc, fetch, def_pf_req_callback, def_pf_callback, 0, NULL);
bool doing_prefetch = false;
r = toku_cachefile_prefetch(f1, key, fullhash, wc, fetch, def_pf_req_callback, def_pf_callback, 0, &doing_prefetch);
assert(doing_prefetch);
toku_cachetable_verify(ct);
}
// wait for all of the blocks to be fetched
sleep(10);
sleep(3);
// prefetch keys N .. 2*N-1. 0 .. N-1 should be evicted.
for (i=i; i<2*cachetable_size_limit; i++) {
for (i=i+1; i<2*cachetable_size_limit; i++) {
CACHEKEY key = make_blocknum(i);
uint32_t fullhash = toku_cachetable_hash(f1, key);
r = toku_cachefile_prefetch(f1, key, fullhash, wc, fetch, def_pf_req_callback, def_pf_callback, 0, NULL);
bool doing_prefetch = false;
r = toku_cachefile_prefetch(f1, key, fullhash, wc, fetch, def_pf_req_callback, def_pf_callback, 0, &doing_prefetch);
assert(!doing_prefetch);
toku_cachetable_verify(ct);
// sleep(1);
}
// wait for everything to finish
sleep(10);
#if 0 //If we flush using reader thread.
assert(flush_evict_calls == cachetable_size_limit);
assert(evicted_keys == (1 << cachetable_size_limit)-1);
#else
assert(flush_evict_calls == 0);
assert(evicted_keys == 0);
#endif
char *error_string;
r = toku_cachefile_close(&f1, &error_string, false, ZERO_LSN); assert(r == 0);
if (verbose) printf("%s:%d 0x%x 0x%x\n", __FUNCTION__, __LINE__,
evicted_keys, (1 << (2*cachetable_size_limit))-1);
assert(evicted_keys == (1 << (2*cachetable_size_limit))-1);
evicted_keys, (1 << (2*cachetable_size_limit))-1);
r = toku_cachetable_close(&ct); assert(r == 0 && ct == 0);
}
......
......@@ -29,6 +29,7 @@ flush (CACHEFILE f __attribute__((__unused__)),
static int
fetch (CACHEFILE f __attribute__((__unused__)),
PAIR UU(p),
int UU(fd),
CACHEKEY k __attribute__((__unused__)),
uint32_t fullhash __attribute__((__unused__)),
......@@ -105,7 +106,7 @@ static void cachetable_prefetch_maybegetandpin_test (bool do_partial_fetch) {
0
);
assert(r==0);
r = toku_cachetable_unpin(f1, key, fullhash, CACHETABLE_CLEAN, make_pair_attr(1));
r = toku_test_cachetable_unpin(f1, key, fullhash, CACHETABLE_CLEAN, make_pair_attr(1));
}
struct timeval tstart;
......@@ -132,7 +133,7 @@ static void cachetable_prefetch_maybegetandpin_test (bool do_partial_fetch) {
toku_cachetable_verify(ct);
r = toku_cachetable_unpin(f1, key, fullhash, CACHETABLE_CLEAN, make_pair_attr(1));
r = toku_test_cachetable_unpin(f1, key, fullhash, CACHETABLE_CLEAN, make_pair_attr(1));
assert(r == 0);
toku_cachetable_verify(ct);
......
......@@ -9,6 +9,7 @@
static int
fetch (CACHEFILE f __attribute__((__unused__)),
PAIR UU(p),
int UU(fd),
CACHEKEY k __attribute__((__unused__)),
uint32_t fullhash __attribute__((__unused__)),
......@@ -58,7 +59,7 @@ static void cachetable_prefetch_maybegetandpin_test (void) {
assert(i>1);
toku_cachetable_verify(ct);
r = toku_cachetable_unpin(f1, key, fullhash, CACHETABLE_CLEAN, make_pair_attr(1));
r = toku_test_cachetable_unpin(f1, key, fullhash, CACHETABLE_CLEAN, make_pair_attr(1));
assert(r == 0);
toku_cachetable_verify(ct);
......
......@@ -12,6 +12,7 @@ static int fetch_calls = 0;
static int
fetch (CACHEFILE f __attribute__((__unused__)),
PAIR UU(p),
int UU(fd),
CACHEKEY k __attribute__((__unused__)),
uint32_t fullhash __attribute__((__unused__)),
......@@ -68,7 +69,7 @@ static void cachetable_prefetch_maybegetandpin_test (void) {
// there should only be 1 fetch callback
assert(fetch_calls == 1);
r = toku_cachetable_unpin(f1, key, fullhash, CACHETABLE_CLEAN, make_pair_attr(1));
r = toku_test_cachetable_unpin(f1, key, fullhash, CACHETABLE_CLEAN, make_pair_attr(1));
assert(r == 0);
toku_cachetable_verify(ct);
......
......@@ -72,6 +72,7 @@ flush (CACHEFILE f __attribute__((__unused__)),
static int
fetch (CACHEFILE f __attribute__((__unused__)),
PAIR UU(p),
int UU(fd),
CACHEKEY k,
uint32_t fullhash __attribute__((__unused__)),
......@@ -159,13 +160,13 @@ static void move_number_to_child(
usleep(10);
(*parent_val)++;
(*child_val)--;
r = toku_cachetable_unpin(f1, parent_key, parent_fullhash, CACHETABLE_DIRTY, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, parent_key, parent_fullhash, CACHETABLE_DIRTY, make_pair_attr(8));
assert_zero(r);
if (child < NUM_INTERNAL) {
move_number_to_child(child, child_val, CACHETABLE_DIRTY);
}
else {
r = toku_cachetable_unpin(f1, child_key, child_fullhash, CACHETABLE_DIRTY, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, child_key, child_fullhash, CACHETABLE_DIRTY, make_pair_attr(8));
assert_zero(r);
}
}
......@@ -306,7 +307,7 @@ static void merge_and_split_child(
// lets get rid of other_child_val with a merge
*child_val += *other_child_val;
*other_child_val = INT64_MAX;
toku_cachetable_unpin_and_remove(f1, other_child_key, remove_data, NULL);
toku_test_cachetable_unpin_and_remove(f1, other_child_key, remove_data, NULL);
dirties[1] = CACHETABLE_DIRTY;
child_dirty = CACHETABLE_DIRTY;
......@@ -327,22 +328,23 @@ static void merge_and_split_child(
hashes,
dirties,
&new_key,
&new_fullhash
&new_fullhash,
put_callback_nop
);
assert(new_key.b == other_child);
assert(new_fullhash == other_child_fullhash);
*data_val = 5000;
*child_val -= 5000;
r = toku_cachetable_unpin(f1, parent_key, parent_fullhash, CACHETABLE_DIRTY, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, parent_key, parent_fullhash, CACHETABLE_DIRTY, make_pair_attr(8));
assert_zero(r);
r = toku_cachetable_unpin(f1, other_child_key, other_child_fullhash, CACHETABLE_DIRTY, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, other_child_key, other_child_fullhash, CACHETABLE_DIRTY, make_pair_attr(8));
assert_zero(r);
if (child < NUM_INTERNAL) {
merge_and_split_child(child, child_val, CACHETABLE_DIRTY);
}
else {
r = toku_cachetable_unpin(f1, child_key, child_fullhash, CACHETABLE_DIRTY, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, child_key, child_fullhash, CACHETABLE_DIRTY, make_pair_attr(8));
assert_zero(r);
}
}
......@@ -397,10 +399,10 @@ static void *checkpoints(void *arg) {
// now run a checkpoint
//
int r;
r = toku_cachetable_begin_checkpoint(ct, NULL); assert(r == 0);
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
r = toku_cachetable_begin_checkpoint(cp, NULL); assert(r == 0);
r = toku_cachetable_end_checkpoint(
ct,
cp,
NULL,
NULL,
NULL
......
......@@ -22,7 +22,7 @@ cachetable_put_test (int n) {
uint32_t hi;
hi = toku_cachetable_hash(f1, make_blocknum(i));
CACHETABLE_WRITE_CALLBACK wc = def_write_callback(NULL);
r = toku_cachetable_put(f1, make_blocknum(i), hi, (void *)(long)i, make_pair_attr(1), wc);
r = toku_cachetable_put(f1, make_blocknum(i), hi, (void *)(long)i, make_pair_attr(1), wc, put_callback_nop);
assert(r == 0);
assert(toku_cachefile_count_pinned(f1, 0) == i);
......@@ -31,24 +31,20 @@ cachetable_put_test (int n) {
assert(r == -1);
assert(toku_cachefile_count_pinned(f1, 0) == i);
//r = toku_cachetable_unpin(f1, make_blocknum(i), hi, CACHETABLE_CLEAN, 1);
//r = toku_test_cachetable_unpin(f1, make_blocknum(i), hi, CACHETABLE_CLEAN, 1);
//assert(r == 0);
assert(toku_cachefile_count_pinned(f1, 0) == i);
}
for (i=n; i>0; i--) {
uint32_t hi;
hi = toku_cachetable_hash(f1, make_blocknum(i));
r = toku_cachetable_unpin(f1, make_blocknum(i), hi, CACHETABLE_CLEAN, make_pair_attr(1));
r = toku_test_cachetable_unpin(f1, make_blocknum(i), hi, CACHETABLE_CLEAN, make_pair_attr(1));
assert(r == 0);
assert(toku_cachefile_count_pinned(f1, 0) == i-1);
}
assert(toku_cachefile_count_pinned(f1, 1) == 0);
toku_cachetable_verify(ct);
CACHEKEY k = make_blocknum(n+1);
r = toku_cachetable_unpin(f1, k, toku_cachetable_hash(f1, k), CACHETABLE_CLEAN, make_pair_attr(1));
assert(r != 0);
r = toku_cachefile_close(&f1, 0, false, ZERO_LSN); assert(r == 0);
r = toku_cachetable_close(&ct); assert(r == 0 && ct == 0);
}
......
/* -*- mode: C++; c-basic-offset: 4; indent-tabs-mode: nil -*- */
// vim: ft=cpp:expandtab:ts=8:sw=4:softtabstop=4:
#ident "$Id$"
#ident "Copyright (c) 2007, 2008 Tokutek Inc. All rights reserved."
#include "includes.h"
#include "test.h"
// this mutex is used by some of the tests to serialize access to some
// global data, especially between the test thread and the cachetable
// writeback threads
toku_mutex_t test_mutex;
static inline void test_mutex_init(void) {
toku_mutex_init(&test_mutex, 0);
}
static inline void test_mutex_destroy(void) {
toku_mutex_destroy(&test_mutex);
}
static inline void test_mutex_lock(void) {
toku_mutex_lock(&test_mutex);
}
static inline void test_mutex_unlock(void) {
toku_mutex_unlock(&test_mutex);
}
static void maybe_flush(CACHETABLE t) {
toku_cachetable_maybe_flush_some(t);
}
enum { KEYLIMIT = 4, TRIALLIMIT=256000 };
static CACHEKEY keys[KEYLIMIT];
static void* vals[KEYLIMIT];
static int n_keys=0;
static void r_flush (CACHEFILE f __attribute__((__unused__)),
int UU(fd),
CACHEKEY k,
void *value,
void** UU(dd),
void *extra __attribute__((__unused__)),
PAIR_ATTR size __attribute__((__unused__)),
PAIR_ATTR* new_size __attribute__((__unused__)),
bool write_me __attribute__((__unused__)),
bool keep_me,
bool for_checkpoint __attribute__((__unused__)),
bool UU(is_clone)
) {
int i;
//printf("Flush\n");
if (keep_me) return;
test_mutex_lock();
for (i=0; i<n_keys; i++) {
if (keys[i].b==k.b) {
assert(vals[i]==value);
if (!keep_me) {
if (verbose) printf("%s: %d/%d %" PRIx64 "\n", __FUNCTION__, i, n_keys, k.b);
keys[i]=keys[n_keys-1];
vals[i]=vals[n_keys-1];
n_keys--;
test_mutex_unlock();
return;
}
}
}
fprintf(stderr, "Whoops\n");
abort();
test_mutex_unlock();
}
static int r_fetch (CACHEFILE f __attribute__((__unused__)),
int UU(fd),
CACHEKEY key __attribute__((__unused__)),
uint32_t fullhash __attribute__((__unused__)),
void**value __attribute__((__unused__)),
void** UU(dd),
PAIR_ATTR *sizep __attribute__((__unused__)),
int *dirtyp __attribute__((__unused__)),
void*extraargs __attribute__((__unused__))) {
// fprintf(stderr, "Whoops, this should never be called");
return -42;
}
static void test_rename (void) {
CACHETABLE t;
CACHEFILE f;
int i;
int r;
test_mutex_init();
const char fname[] = __SRCFILE__ "rename.dat";
r=toku_create_cachetable(&t, KEYLIMIT, ZERO_LSN, NULL_LOGGER); assert(r==0);
unlink(fname);
r = toku_cachetable_openf(&f, t, fname, O_RDWR|O_CREAT, S_IRWXU|S_IRWXG|S_IRWXO);
assert(r==0);
for (i=0; i<TRIALLIMIT; i++) {
int ra = random()%3;
if (ra<=1) {
// Insert something
CACHEKEY nkey = make_blocknum(random());
long nval = random();
if (verbose) printf("n_keys=%d Insert %08" PRIx64 "\n", n_keys, nkey.b);
uint32_t hnkey = toku_cachetable_hash(f, nkey);
CACHETABLE_WRITE_CALLBACK wc = def_write_callback(NULL);
wc.flush_callback = r_flush;
r = toku_cachetable_put(f, nkey, hnkey,
(void*)nval, make_pair_attr(1),
wc);
assert(r==0);
test_mutex_lock();
while (n_keys >= KEYLIMIT) {
test_mutex_unlock();
toku_pthread_yield(); maybe_flush(t);
test_mutex_lock();
}
assert(n_keys<KEYLIMIT);
keys[n_keys] = nkey;
vals[n_keys] = (void*)nval;
n_keys++;
test_mutex_unlock();
r = toku_cachetable_unpin(f, nkey, hnkey, CACHETABLE_DIRTY, make_pair_attr(1));
assert(r==0);
} else if (ra==2 && n_keys>0) {
// Rename something
int objnum = random()%n_keys;
CACHEKEY nkey = make_blocknum(random());
test_mutex_lock();
CACHEKEY okey = keys[objnum];
test_mutex_unlock();
void *current_value;
long current_size;
if (verbose) printf("Rename %" PRIx64 " to %" PRIx64 "\n", okey.b, nkey.b);
CACHETABLE_WRITE_CALLBACK wc = def_write_callback(NULL);
wc.flush_callback = r_flush;
r = toku_cachetable_get_and_pin(f, okey, toku_cachetable_hash(f, okey), &current_value, &current_size, wc, r_fetch, def_pf_req_callback, def_pf_callback, true, 0);
if (r == -42) continue;
assert(r==0);
r = toku_cachetable_rename(f, okey, nkey);
assert(r==0);
test_mutex_lock();
// assert(objnum < n_keys && keys[objnum] == okey);
// get_and_pin may reorganize the keys[], so we need to find it again
int j;
for (j=0; j < n_keys; j++)
if (keys[j].b == okey.b)
break;
assert(j < n_keys);
keys[j]=nkey;
test_mutex_unlock();
r = toku_cachetable_unpin(f, nkey, toku_cachetable_hash(f, nkey), CACHETABLE_DIRTY, make_pair_attr(1));
}
}
// test rename fails if old key does not exist in the cachetable
CACHEKEY okey, nkey;
while (1) {
okey = make_blocknum(random());
void *v;
r = toku_cachetable_maybe_get_and_pin(f, okey, toku_cachetable_hash(f, okey), &v);
if (r != 0)
break;
r = toku_cachetable_unpin(f, okey, toku_cachetable_hash(f, okey), CACHETABLE_CLEAN, make_pair_attr(1));
assert(r == 0);
}
nkey = make_blocknum(random());
r = toku_cachetable_rename(f, okey, nkey);
assert(r != 0);
r = toku_cachefile_close(&f, 0, false, ZERO_LSN);
assert(r == 0);
r = toku_cachetable_close(&t);
assert(r == 0);
test_mutex_destroy();
assert(n_keys == 0);
}
int
test_main (int argc, const char *argv[]) {
// parse args
default_parse_args(argc, argv);
toku_os_initialize_settings(verbose);
// run tests
int i;
for (i=0; i<1; i++)
test_rename();
return 0;
}
......@@ -34,6 +34,7 @@ static void f_flush (CACHEFILE f,
}
static int f_fetch (CACHEFILE f,
PAIR UU(p),
int UU(fd),
CACHEKEY key,
uint32_t fullhash __attribute__((__unused__)),
......@@ -74,8 +75,8 @@ static void writeit (void) {
for (j=0; j<BLOCKSIZE; j++) ((char*)buf)[j]=(char)((i+j)%256);
CACHETABLE_WRITE_CALLBACK wc = def_write_callback(NULL);
wc.flush_callback = f_flush;
r = toku_cachetable_put(f, key, fullhash, buf, make_pair_attr(BLOCKSIZE), wc); assert(r==0);
r = toku_cachetable_unpin(f, key, fullhash, CACHETABLE_CLEAN, make_pair_attr(BLOCKSIZE)); assert(r==0);
r = toku_cachetable_put(f, key, fullhash, buf, make_pair_attr(BLOCKSIZE), wc, put_callback_nop); assert(r==0);
r = toku_test_cachetable_unpin(f, key, fullhash, CACHETABLE_CLEAN, make_pair_attr(BLOCKSIZE)); assert(r==0);
}
gettimeofday(&end, 0);
double diff = toku_tdiff(&end, &start);
......@@ -98,7 +99,7 @@ static void readit (void) {
CACHETABLE_WRITE_CALLBACK wc = def_write_callback(NULL);
wc.flush_callback = f_flush;
r=toku_cachetable_get_and_pin(f, key, fullhash, &block, &current_size, wc, f_fetch, def_pf_req_callback, def_pf_callback, true, 0); assert(r==0);
r=toku_cachetable_unpin(f, key, fullhash, CACHETABLE_CLEAN, make_pair_attr(BLOCKSIZE)); assert(r==0);
r=toku_test_cachetable_unpin(f, key, fullhash, CACHETABLE_CLEAN, make_pair_attr(BLOCKSIZE)); assert(r==0);
}
r = toku_cachefile_close(&f, 0, false, ZERO_LSN); assert(r == 0);
r = toku_cachetable_close(&t); assert(r == 0);
......
......@@ -75,7 +75,7 @@ test_clean (enum cachetable_dirty dirty, bool cloneable) {
wc.clone_callback = cloneable ? clone_callback : NULL;
wc.flush_callback = flush;
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, dirty, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, dirty, make_pair_attr(8));
check_flush = true;
clone_called = false;
......@@ -83,7 +83,8 @@ test_clean (enum cachetable_dirty dirty, bool cloneable) {
flush_called = false;
// begin checkpoint, since pair is clean, we should not
// have the clone called
r = toku_cachetable_begin_checkpoint(ct, NULL);
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
r = toku_cachetable_begin_checkpoint(cp, NULL);
assert_zero(r);
struct timeval tstart;
struct timeval tend;
......@@ -91,7 +92,7 @@ test_clean (enum cachetable_dirty dirty, bool cloneable) {
// test that having a pin that passes false for may_modify_value does not stall behind checkpoint
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, false, NULL);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
gettimeofday(&tend, NULL);
assert(tdelta_usec(&tend, &tstart) <= 2000000);
assert(!clone_called);
......@@ -118,7 +119,7 @@ test_clean (enum cachetable_dirty dirty, bool cloneable) {
}
// at this point, there should be no more dirty writes
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
gettimeofday(&tend, NULL);
if (cloneable || !dirty ) {
assert(tdelta_usec(&tend, &tstart) <= 2000000);
......@@ -128,7 +129,7 @@ test_clean (enum cachetable_dirty dirty, bool cloneable) {
}
r = toku_cachetable_end_checkpoint(
ct,
cp,
NULL,
NULL,
NULL
......
......@@ -63,19 +63,20 @@ test_clean (enum cachetable_dirty dirty, bool cloneable) {
wc.clone_callback = cloneable ? clone_callback : NULL;
wc.flush_callback = flush;
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8));
// begin checkpoint, since pair is clean, we should not
// have the clone called
r = toku_cachetable_begin_checkpoint(ct, NULL);
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
r = toku_cachetable_begin_checkpoint(cp, NULL);
assert_zero(r);
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
// at this point, there should be no more dirty writes
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, dirty, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, dirty, make_pair_attr(8));
usleep(2*1024*1024);
r = toku_cachetable_end_checkpoint(
ct,
cp,
NULL,
NULL,
NULL
......
......@@ -27,14 +27,14 @@ cachetable_test (void) {
r = toku_cachetable_maybe_get_and_pin(f1, make_blocknum(1), 1, &v1);
assert(r==-1);
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
// maybe_get_and_pin_clean should succeed, maybe_get_and_pin should fail
r = toku_cachetable_maybe_get_and_pin(f1, make_blocknum(1), 1, &v1);
assert(r==-1);
r = toku_cachetable_maybe_get_and_pin_clean(f1, make_blocknum(1), 1, &v1);
assert(r == 0);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8));
// maybe_get_and_pin_clean should succeed, maybe_get_and_pin should fail
r = toku_cachetable_maybe_get_and_pin(f1, make_blocknum(1), 1, &v1);
assert(r==0);
......@@ -43,20 +43,21 @@ cachetable_test (void) {
assert(r==-1);
r = toku_cachetable_maybe_get_and_pin_clean(f1, make_blocknum(1), 1, &v1);
assert(r==-1);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8));
// sanity check, this should still succeed, because the PAIR is dirty
r = toku_cachetable_maybe_get_and_pin(f1, make_blocknum(1), 1, &v1);
assert(r==0);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8));
r = toku_cachetable_begin_checkpoint(ct, NULL); assert(r == 0);
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8));
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
r = toku_cachetable_begin_checkpoint(cp, NULL); assert(r == 0);
// now these should fail, because the node should be pending a checkpoint
r = toku_cachetable_maybe_get_and_pin(f1, make_blocknum(1), 1, &v1);
assert(r==-1);
r = toku_cachetable_maybe_get_and_pin(f1, make_blocknum(1), 1, &v1);
assert(r==-1);
r = toku_cachetable_end_checkpoint(
ct,
cp,
NULL,
NULL,
NULL
......
......@@ -50,6 +50,7 @@ flush (CACHEFILE f __attribute__((__unused__)),
static int
fetch (CACHEFILE f __attribute__((__unused__)),
PAIR UU(p),
int UU(fd),
CACHEKEY k __attribute__((__unused__)),
uint32_t fullhash __attribute__((__unused__)),
......@@ -102,11 +103,12 @@ cachetable_test (bool write_first, bool write_second, bool start_checkpoint) {
enum cachetable_dirty cd[2];
cd[0] = write_first ? CACHETABLE_DIRTY : CACHETABLE_CLEAN;
cd[1] = write_second ? CACHETABLE_DIRTY : CACHETABLE_CLEAN;
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
if (start_checkpoint) {
//
// should mark the v1 and v2 as pending
//
r = toku_cachetable_begin_checkpoint(ct, NULL); assert(r==0);
r = toku_cachetable_begin_checkpoint(cp, NULL); assert(r==0);
}
//
// This call should cause a flush for both
......@@ -139,13 +141,13 @@ cachetable_test (bool write_first, bool write_second, bool start_checkpoint) {
assert(!v2_written);
}
check_me = false;
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_cachetable_unpin(f1, make_blocknum(2), 2, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_cachetable_unpin(f1, make_blocknum(3), 3, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(2), 2, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(3), 3, CACHETABLE_CLEAN, make_pair_attr(8));
if (start_checkpoint) {
r = toku_cachetable_end_checkpoint(
ct,
cp,
NULL,
NULL,
NULL
......
......@@ -50,7 +50,7 @@ static void kibbutz_work(void *fe_v)
CACHEFILE CAST_FROM_VOIDP(f1, fe_v);
sleep(2);
foo = true;
int r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
int r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
assert(r==0);
remove_background_job_from_cf(f1);
}
......@@ -88,13 +88,13 @@ run_test (void) {
assert(r==TOKUDB_TRY_AGAIN);
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
assert(foo);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8)); assert(r==0);
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8)); assert(r==0);
// now make sure we get TOKUDB_TRY_AGAIN when a partial fetch is involved
// first make sure value is there
r = toku_cachetable_get_and_pin_nonblocking(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL, NULL);
assert(r==0);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8)); assert(r==0);
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8)); assert(r==0);
// now make sure that we get TOKUDB_TRY_AGAIN for the partial fetch
r = toku_cachetable_get_and_pin_nonblocking(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, true_def_pf_req_callback, true_def_pf_callback, true, NULL, NULL);
assert(r==TOKUDB_TRY_AGAIN);
......@@ -105,13 +105,14 @@ run_test (void) {
//
r = toku_cachetable_get_and_pin_nonblocking(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL, NULL);
assert(r==0);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8)); assert(r==0);
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8)); assert(r==0);
// this should mark the PAIR as pending
r = toku_cachetable_begin_checkpoint(ct, NULL); assert(r == 0);
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
r = toku_cachetable_begin_checkpoint(cp, NULL); assert(r == 0);
r = toku_cachetable_get_and_pin_nonblocking(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL, NULL);
assert(r==TOKUDB_TRY_AGAIN);
r = toku_cachetable_end_checkpoint(
ct,
cp,
NULL,
NULL,
NULL
......
......@@ -43,7 +43,7 @@ static void kibbutz_work(void *fe_v)
CACHEFILE CAST_FROM_VOIDP(f1, fe_v);
sleep(2);
foo = true;
int r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
int r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
assert(r==0);
remove_background_job_from_cf(f1);
}
......@@ -70,15 +70,16 @@ run_test (void) {
cachefile_kibbutz_enq(f1, kibbutz_work, f1);
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
assert(foo);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
//now let's do a simple checkpoint test
// first dirty the PAIR
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8));
// now this should mark the pair for checkpoint
r = toku_cachetable_begin_checkpoint(ct, NULL);
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
r = toku_cachetable_begin_checkpoint(cp, NULL);
//
// now we pin the pair again, and verify in flush callback that the pair is being checkpointed
......@@ -87,12 +88,12 @@ run_test (void) {
flush_called = false;
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
assert(flush_called);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
check_me = false;
r = toku_cachetable_end_checkpoint(
ct,
cp,
NULL,
NULL,
NULL
......
......@@ -50,6 +50,7 @@ flush (CACHEFILE f __attribute__((__unused__)),
static int
fetch (CACHEFILE f __attribute__((__unused__)),
PAIR UU(p),
int UU(fd),
CACHEKEY k __attribute__((__unused__)),
uint32_t fullhash __attribute__((__unused__)),
......@@ -107,11 +108,12 @@ cachetable_test (bool write_first, bool write_second, bool start_checkpoint) {
enum cachetable_dirty cd[2];
cd[0] = write_first ? CACHETABLE_DIRTY : CACHETABLE_CLEAN;
cd[1] = write_second ? CACHETABLE_DIRTY : CACHETABLE_CLEAN;
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
if (start_checkpoint) {
//
// should mark the v1 and v2 as pending
//
r = toku_cachetable_begin_checkpoint(ct, NULL); assert(r==0);
r = toku_cachetable_begin_checkpoint(cp, NULL); assert(r==0);
}
//
// This call should cause a flush for both
......@@ -135,7 +137,8 @@ cachetable_test (bool write_first, bool write_second, bool start_checkpoint) {
dependent_fullhash,
cd,
&put_key,
&put_fullhash
&put_fullhash,
put_callback_nop
);
assert(r == 0);
assert(put_key.b == 3);
......@@ -147,13 +150,13 @@ cachetable_test (bool write_first, bool write_second, bool start_checkpoint) {
}
check_me = false;
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_cachetable_unpin(f1, make_blocknum(2), 2, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_cachetable_unpin(f1, make_blocknum(3), 3, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(2), 2, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(3), 3, CACHETABLE_CLEAN, make_pair_attr(8));
if (start_checkpoint) {
r = toku_cachetable_end_checkpoint(
ct,
cp,
NULL,
NULL,
NULL
......
......@@ -40,10 +40,11 @@ cachetable_test (void) {
//long s2;
CACHETABLE_WRITE_CALLBACK wc = def_write_callback(NULL);
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
r = toku_cachetable_begin_checkpoint(ct, NULL); assert(r == 0);
r = toku_cachetable_unpin_and_remove(f1, make_blocknum(1), remove_key_expect_checkpoint, NULL);
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
r = toku_cachetable_begin_checkpoint(cp, NULL); assert(r == 0);
r = toku_test_cachetable_unpin_and_remove(f1, make_blocknum(1), remove_key_expect_checkpoint, NULL);
r = toku_cachetable_end_checkpoint(
ct,
cp,
NULL,
NULL,
NULL
......@@ -51,7 +52,7 @@ cachetable_test (void) {
assert(r==0);
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
r = toku_cachetable_unpin_and_remove(f1, make_blocknum(1), remove_key_expect_no_checkpoint, NULL);
r = toku_test_cachetable_unpin_and_remove(f1, make_blocknum(1), remove_key_expect_no_checkpoint, NULL);
toku_cachetable_verify(ct);
......
......@@ -22,7 +22,7 @@ cachetable_test (void) {
//long s2;
CACHETABLE_WRITE_CALLBACK wc = def_write_callback(NULL);
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8));
toku_cachetable_verify(ct);
r = toku_cachefile_close(&f1, 0, false, ZERO_LSN); assert(r == 0);
r = toku_cachetable_close(&ct); lazy_assert_zero(r);
......
This diff is collapsed.
This diff is collapsed.
......@@ -9,12 +9,13 @@ CACHETABLE ct;
//
// This test exposed a bug (#3970) caught only by Valgrind.
// freed memory was being accessed by toku_cachetable_unpin_and_remove
// freed memory was being accessed by toku_test_cachetable_unpin_and_remove
//
static void *run_end_chkpt(void *arg) {
assert(arg == NULL);
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
int r = toku_cachetable_end_checkpoint(
ct,
cp,
NULL,
NULL,
NULL
......@@ -40,7 +41,7 @@ run_test (void) {
long s1;
//long s2;
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), toku_cachetable_hash(f1, make_blocknum(1)), &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
toku_cachetable_unpin(
toku_test_cachetable_unpin(
f1,
make_blocknum(1),
toku_cachetable_hash(f1, make_blocknum(1)),
......@@ -49,7 +50,8 @@ run_test (void) {
);
// now this should mark the pair for checkpoint
r = toku_cachetable_begin_checkpoint(ct, NULL);
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
r = toku_cachetable_begin_checkpoint(cp, NULL);
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), toku_cachetable_hash(f1, make_blocknum(1)), &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
toku_pthread_t mytid;
......@@ -58,7 +60,7 @@ run_test (void) {
// give checkpoint thread a chance to start waiting on lock
sleep(1);
r = toku_cachetable_unpin_and_remove(f1, make_blocknum(1), NULL, NULL);
r = toku_test_cachetable_unpin_and_remove(f1, make_blocknum(1), NULL, NULL);
assert(r==0);
void* ret;
......
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
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