diff --git a/cmake_modules/TokuSetupCompiler.cmake b/cmake_modules/TokuSetupCompiler.cmake index 1032de832d6..c88c5ee7ed8 100644 --- a/cmake_modules/TokuSetupCompiler.cmake +++ b/cmake_modules/TokuSetupCompiler.cmake @@ -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}") diff --git a/ft/cachetable-internal.h b/ft/cachetable-internal.h new file mode 100644 index 00000000000..603d0234dae --- /dev/null +++ b/ft/cachetable-internal.h @@ -0,0 +1,486 @@ +/* -*- mode: C++; c-basic-offset: 4; indent-tabs-mode: nil -*- */ +// vim: ft=cpp:expandtab:ts=8:sw=4:softtabstop=4: + +#ifndef TokuDB_cachetable_internal_h +#define TokuDB_cachetable_internal_h + +#ident "$Id: cachetable.h 46050 2012-07-24 02:26:17Z zardosht $" +#ident "Copyright (c) 2007-2012 Tokutek Inc. All rights reserved." +#ident "The technology is licensed by the Massachusetts Institute of Technology, Rutgers State University of New Jersey, and the Research Foundation of State University of New York at Stony Brook under United States of America Serial No. 11/760379 and to the patents and/or patent applications resulting from it." + +#include "nonblocking_mutex.h" +#include "kibbutz.h" +#include "background_job_manager.h" + +////////////////////////////////////////////////////////////////////////////// +// +// This file contains the classes and structs that make up the cachetable. +// The structs are: +// - cachefile +// - ctpair +// - pair_list +// - cachefile_list +// - checkpointer +// - evictor +// - cleaner +// +// The rest of this comment assumes familiarity with the locks used in these +// classes/structs and what the locks protect. Nevertheless, here is +// a list of the locks that we have: +// - pair_list->list_lock +// - pair_list->pending_lock_expensive +// - pair_list->pending_lock_cheap +// - cachefile_list->lock +// - PAIR->mutex +// - PAIR->value_nb_mutex +// - PAIR->disk_nb_mutex +// +// Here are rules for how the locks interact: +// - To grab any of the pair_list's locks, or the cachefile_list's lock, +// the cachetable must be in existence +// - To grab the PAIR mutex, we must know the PAIR will not dissappear: +// - the PAIR must be pinned (value_nb_mutex or disk_nb_mutex is held) +// - OR, the pair_list's list lock is held +// - As a result, to get rid of a PAIR from the pair_list, we must hold +// both the pair_list's list_lock and the PAIR's mutex +// - To grab PAIR->value_nb_mutex, we must hold the PAIR's mutex +// - To grab PAIR->disk_nb_mutex, we must hold the PAIR's mutex +// and hold PAIR->value_nb_mutex +// +// Now let's talk about ordering. Here is an order from outer to inner (top locks must be grabbed first) +// - pair_list->pending_lock_expensive +// - pair_list->list_lock +// - cachefile_list->lock +// - PAIR->mutex +// - pair_list->pending_lock_cheap <-- after grabbing this lock, +// NO other locks +// should be grabbed. +// - when grabbing PAIR->value_nb_mutex or PAIR->disk_nb_mutex, +// if the acquisition will not block, then it does not matter if any other locks held, +// BUT if the acquisition will block, then NO other locks may be held besides +// PAIR->mutex. +// +// HERE ARE TWO EXAMPLES: +// To pin a PAIR on a client thread, the following must be done: +// - first grab the list lock and find the PAIR +// - with the list lock grabbed, grab PAIR->mutex +// - with PAIR->mutex held: +// - release list lock +// - pin PAIR +// - with PAIR pinned, grab pending_lock_cheap, +// - copy and clear PAIR->checkpoint_pending, +// - resolve checkpointing if necessary +// - return to user. +// The list lock may be held while pinning the PAIR if +// the PAIR has no contention. Otherwise, we may have +// get a deadlock with another thread that has the PAIR pinned, +// tries to pin some other PAIR, and in doing so, grabs the list lock. +// +// To unpin a PAIR on a client thread: +// - because the PAIR is pinned, we don't need the pair_list's list_lock +// - so, simply acquire PAIR->mutex +// - unpin the PAIR +// - return +// +////////////////////////////////////////////////////////////////////////////// +class evictor; +class pair_list; + +/////////////////////////////////////////////////////////////////////////////// +// +// Maps to a file on disk. +// +struct cachefile { + CACHEFILE next; + bool for_checkpoint; //True if part of the in-progress checkpoint + + // If set and the cachefile closes, the file will be removed. + // Clients must not operate on the cachefile after setting this, + // nor attempt to open any cachefile with the same fname (dname) + // until this cachefile has been fully closed and unlinked. + bool unlink_on_close; + int fd; /* Bug: If a file is opened read-only, then it is stuck in read-only. If it is opened read-write, then subsequent writers can write to it too. */ + CACHETABLE cachetable; + struct fileid fileid; + FILENUM filenum; + char *fname_in_env; /* Used for logging */ + + void *userdata; + int (*log_fassociate_during_checkpoint)(CACHEFILE cf, void *userdata); // When starting a checkpoint we must log all open files. + int (*log_suppress_rollback_during_checkpoint)(CACHEFILE cf, void *userdata); // When starting a checkpoint we must log which files need rollbacks suppressed + int (*close_userdata)(CACHEFILE cf, int fd, void *userdata, char **error_string, bool lsnvalid, LSN); // when closing the last reference to a cachefile, first call this function. + int (*begin_checkpoint_userdata)(LSN lsn_of_checkpoint, void *userdata); // before checkpointing cachefiles call this function. + int (*checkpoint_userdata)(CACHEFILE cf, int fd, void *userdata); // when checkpointing a cachefile, call this function. + int (*end_checkpoint_userdata)(CACHEFILE cf, int fd, void *userdata); // after checkpointing cachefiles call this function. + int (*note_pin_by_checkpoint)(CACHEFILE cf, void *userdata); // add a reference to the userdata to prevent it from being removed from memory + int (*note_unpin_by_checkpoint)(CACHEFILE cf, void *userdata); // add a reference to the userdata to prevent it from being removed from memory + BACKGROUND_JOB_MANAGER bjm; +}; + + +/////////////////////////////////////////////////////////////////////////////// +// +// The pair represents the data stored in the cachetable. +// +struct ctpair { + // these fields are essentially constants. They do not change. + CACHEFILE cachefile; + CACHEKEY key; + uint32_t fullhash; + CACHETABLE_FLUSH_CALLBACK flush_callback; + CACHETABLE_PARTIAL_EVICTION_EST_CALLBACK pe_est_callback; + CACHETABLE_PARTIAL_EVICTION_CALLBACK pe_callback; + CACHETABLE_CLEANER_CALLBACK cleaner_callback; + CACHETABLE_CLONE_CALLBACK clone_callback; + void *write_extraargs; + + // access to these fields are protected by disk_nb_mutex + void* cloned_value_data; // cloned copy of value_data used for checkpointing + long cloned_value_size; // size of cloned_value_data, used for accounting of size_current + void* disk_data; // data used to fetch/flush value_data to and from disk. + + // access to these fields are protected by value_nb_mutex + void* value_data; // data used by client threads, FTNODEs and ROLLBACK_LOG_NODEs + PAIR_ATTR attr; + enum cachetable_dirty dirty; + + // protected by PAIR->mutex + uint32_t count; // clock count + + // locks + struct nb_mutex value_nb_mutex; // single writer, protects value_data + struct nb_mutex disk_nb_mutex; // single writer, protects disk_data, is used for writing cloned nodes for checkpoint + toku_mutex_t mutex; + + // Access to checkpoint_pending is protected by two mechanisms, + // the value_nb_mutex and the pair_list's pending locks (expensive and cheap). + // checkpoint_pending may be true of false. + // Here are the rules for reading/modifying this bit. + // - To transition this field from false to true during begin_checkpoint, + // we must be holding both of the pair_list's pending locks. + // - To transition this field from true to false during end_checkpoint, + // we must be holding the value_nb_mutex. + // - For a non-checkpoint thread to read the value, we must hold both the + // value_nb_mutex and one of the pair_list's pending locks + // - For the checkpoint thread to read the value, we must + // hold the value_nb_mutex + // + bool checkpoint_pending; // If this is on, then we have got to resolve checkpointing modifying it. + + // these are variables that are only used to transfer information to background threads + // we cache them here to avoid a malloc. In the future, we should investigate if this + // is necessary, as having these fields here is not technically necessary + long size_evicting_estimate; + evictor* ev; + pair_list* list; + + // A PAIR is stored in a pair_list (which happens to be PAIR->list). + // These variables are protected by the list lock in the pair_list + PAIR clock_next,clock_prev; // In clock. + PAIR hash_chain; + PAIR pending_next; + PAIR pending_prev; +}; + +// +// This initializes the fields and members of the pair. +// +void pair_init(PAIR p, + CACHEFILE cachefile, + CACHEKEY key, + void *value, + PAIR_ATTR attr, + enum cachetable_dirty dirty, + uint32_t fullhash, + CACHETABLE_WRITE_CALLBACK write_callback, + evictor *ev, + pair_list *list); + + +/////////////////////////////////////////////////////////////////////////////// +// +// The pair list maintains the set of PAIR's that make up +// the cachetable. +// +class pair_list { +public: + // + // the following fields are protected by the list lock + // + uint32_t m_n_in_table; // number of pairs in the hash table + uint32_t m_table_size; // number of buckets in the hash table + PAIR *m_table; // hash table + // + // The following fields are the heads of various linked lists. + // They also protected by the list lock, but their + // usage is not as straightforward. For each of them, + // only ONE thread is allowed iterate over them with + // a read lock on the list lock. All other threads + // that want to modify elements in the lists or iterate over + // the lists must hold the write list lock. Here is the + // association between what threads may hold a read lock + // on the list lock while iterating: + // - clock_head -> eviction thread (evictor) + // - cleaner_head -> cleaner thread (cleaner) + // - pending_head -> checkpoint thread (checkpointer) + // + PAIR m_clock_head; // of clock . head is the next thing to be up for decrement. + PAIR m_cleaner_head; // for cleaner thread. head is the next thing to look at for possible cleaning. + PAIR m_pending_head; // list of pairs marked with checkpoint_pending + + // this field is public so we are still POD + + // usage of this lock is described above + toku_pthread_rwlock_t m_list_lock; + // + // these locks are the "pending locks" referenced + // in comments about PAIR->checkpoint_pending. There + // are two of them, but both serve the same purpose, which + // is to protect the transition of a PAIR's checkpoint pending + // value from false to true during begin_checkpoint. + // We use two locks, because threads that want to read the + // checkpoint_pending value may hold a lock for varying periods of time. + // Threads running eviction may need to protect checkpoint_pending + // while writing a node to disk, which is an expensive operation, + // so it uses pending_lock_expensive. Client threads that + // want to pin PAIRs will want to protect checkpoint_pending + // just long enough to read the value and wipe it out. This is + // a cheap operation, and as a result, uses pending_lock_cheap. + // + // By having two locks, and making begin_checkpoint first + // grab pending_lock_expensive and then pending_lock_cheap, + // we ensure that threads that want to pin nodes can grab + // only pending_lock_cheap, and never block behind threads + // holding pending_lock_expensive and writing a node out to disk + // + toku_pthread_rwlock_t m_pending_lock_expensive; + toku_pthread_rwlock_t m_pending_lock_cheap; + void init(); + int destroy(); + void evict(PAIR pair); + void put(PAIR pair); + PAIR find_pair(CACHEFILE file, CACHEKEY key, uint32_t hash); + void pending_pairs_remove (PAIR p); + void verify(); + void get_state(int *num_entries, int *hash_size); + void read_list_lock(); + void read_list_unlock(); + void write_list_lock(); + void write_list_unlock(); + void read_pending_exp_lock(); + void read_pending_exp_unlock(); + void write_pending_exp_lock(); + void write_pending_exp_unlock(); + void read_pending_cheap_lock(); + void read_pending_cheap_unlock(); + void write_pending_cheap_lock(); + void write_pending_cheap_unlock(); + +private: + void pair_remove (PAIR p); + void rehash (uint32_t newtable_size); + void add_to_clock (PAIR p); + PAIR remove_from_hash_chain (PAIR remove_me, PAIR list); +}; + +/////////////////////////////////////////////////////////////////////////////// +// +// Wrapper for the head of our cachefile list. +// +class cachefile_list { +public: + void init(); + void destroy(); + void read_lock(); + void read_unlock(); + void write_lock(); + void write_unlock(); + // access to these fields are protected by the lock + CACHEFILE m_head; + FILENUM m_next_filenum_to_use; + toku_pthread_rwlock_t m_lock; // this field is publoc so we are still POD +}; + + +/////////////////////////////////////////////////////////////////////////////// +// +// The checkpointer handles starting and finishing checkpoints of the +// cachetable's data. +// +class checkpointer { +public: + void init(CACHETABLE _ct, TOKULOGGER _logger, cachefile_list *files); + void destroy(); + int set_checkpoint_period(uint32_t new_period); + uint32_t get_checkpoint_period(); + int shutdown(); + bool has_been_shutdown(); + int begin_checkpoint(); + void add_background_job(); + void remove_background_job(); + int end_checkpoint(void (*testcallback_f)(void*), void* testextra); + TOKULOGGER get_logger(); + // used during begin_checkpoint + void increment_num_txns(); +private: + uint32_t m_checkpoint_num_txns; // how many transactions are in the checkpoint + TOKULOGGER m_logger; + LSN m_lsn_of_checkpoint_in_progress; + uint32_t m_checkpoint_num_files; // how many cachefiles are in the checkpoint + struct minicron m_checkpointer_cron; // the periodic checkpointing thread + cachefile_list *m_cf_list; + // TEMP? + CACHETABLE m_ct; + + // variable used by the checkpoint thread to know + // when all work induced by cloning on client threads is done + BACKGROUND_JOB_MANAGER m_checkpoint_clones_bjm; + // private methods for begin_checkpoint + void update_cachefiles(); + void log_begin_checkpoint(); + void turn_on_pending_bits(); + // private methods for end_checkpoint + void fill_checkpoint_cfs(CACHEFILE* checkpoint_cfs); + void checkpoint_pending_pairs(); + void checkpoint_userdata(CACHEFILE* checkpoint_cfs); + void log_end_checkpoint(); + void end_checkpoint_userdata(CACHEFILE* checkpoint_cfs); + int remove_cachefiles(CACHEFILE* checkpoint_cfs); + + // Unit test struct needs access to private members. + friend struct checkpointer_test; +}; + +// +// This is how often we want the eviction thread +// to run, in seconds. +// +const int EVICTION_PERIOD = 1; + +/////////////////////////////////////////////////////////////////////////////// +// +// The evictor handles the removal of pairs from the pair list/cachetable. +// +class evictor { +public: + void init(long _size_limit, pair_list* _pl, KIBBUTZ _kibbutz, uint32_t eviction_period); + void destroy(); + void add_pair_attr(PAIR_ATTR attr); + void remove_pair_attr(PAIR_ATTR attr); + void change_pair_attr(PAIR_ATTR old_attr, PAIR_ATTR new_attr); + void add_to_size_current(long size); + void remove_from_size_current(long size); + uint64_t reserve_memory(double fraction); + void release_reserved_memory(uint64_t reserved_memory); + void run_eviction_thread(); + void do_partial_eviction(PAIR p); + void evict_pair(PAIR p, bool checkpoint_pending); + void wait_for_cache_pressure_to_subside(); + void signal_eviction_thread(); + bool should_client_thread_sleep(); + bool should_client_wake_eviction_thread(); + // function needed for testing + void get_state(long *size_current_ptr, long *size_limit_ptr); + void fill_engine_status(); +private: + void run_eviction(); + bool run_eviction_on_pair(PAIR p); + void try_evict_pair(PAIR p); + void decrease_size_evicting(long size_evicting_estimate); + bool should_sleeping_clients_wakeup(); + bool eviction_needed(); + + pair_list* m_pl; + int64_t m_size_current; // the sum of the sizes of the pairs in the cachetable + // changes to these two values are protected + // by ev_thread_lock + int64_t m_size_reserved; // How much memory is reserved (e.g., by the loader) + int64_t m_size_evicting; // the sum of the sizes of the pairs being written + + // these are constants + int64_t m_low_size_watermark; // target max size of cachetable that eviction thread aims for + int64_t m_low_size_hysteresis; // if cachetable grows to this size, client threads wake up eviction thread upon adding data + int64_t m_high_size_watermark; // if cachetable grows to this size, client threads sleep upon adding data + int64_t m_high_size_hysteresis; // if > cachetable size, then sleeping client threads may wake up + + // mutex that protects fields listed immedietly below + toku_mutex_t m_ev_thread_lock; + // the eviction thread + toku_pthread_t m_ev_thread; + // condition variable that controls the sleeping period + // of the eviction thread + toku_cond_t m_ev_thread_cond; + // number of client threads that are currently sleeping + // due to an over-subscribed cachetable + uint32_t m_num_sleepers; + // states if the eviction thread should run. set to true + // in init, set to false during destroy + bool m_run_thread; + // bool that states if the eviction thread is currently running + bool m_ev_thread_is_running; + // period which the eviction thread sleeps + uint32_t m_period_in_seconds; + // condition variable on which client threads wait on when sleeping + // due to an over-subscribed cachetable + toku_cond_t m_flow_control_cond; + + // variables for engine status + int64_t m_size_nonleaf; + int64_t m_size_leaf; + int64_t m_size_rollback; + int64_t m_size_cachepressure; + + KIBBUTZ m_kibbutz; + + // this variable is ONLY used for testing purposes + uint64_t m_num_eviction_thread_runs; + friend class evictor_test_helpers; + friend class evictor_unit_test; +}; + +/////////////////////////////////////////////////////////////////////////////// +// +// Iterates over the clean head in the pair list, calling the cleaner +// callback on each node in that list. +// +class cleaner { +public: + void init(uint32_t cleaner_iterations, pair_list* _pl, CACHETABLE _ct); + void destroy(void); + uint32_t get_iterations(void); + void set_iterations(uint32_t new_iterations); + uint32_t get_period(void); + uint32_t get_period_unlocked(void); + void set_period(uint32_t new_period); + int run_cleaner(void); + +private: + pair_list* m_pl; + CACHETABLE m_ct; + struct minicron m_cleaner_cron; // the periodic cleaner thread + uint32_t m_cleaner_iterations; // how many times to run the cleaner per + // cleaner period (minicron has a + // minimum period of 1s so if you want + // more frequent cleaner runs you must + // use this) +}; + +/////////////////////////////////////////////////////////////////////////////// +// +// The cachetable is as close to an ENV as we get. +// +struct cachetable { + pair_list list; + cleaner cl; + evictor ev; + checkpointer cp; + cachefile_list cf_list; + + KIBBUTZ client_kibbutz; // pool of worker threads and jobs to do asynchronously for the client. + KIBBUTZ ct_kibbutz; // pool of worker threads and jobs to do asynchronously for the cachetable + KIBBUTZ checkpointing_kibbutz; // small pool for checkpointing cloned pairs + + char *env_dir; +}; + +#endif // End of header guardian. diff --git a/ft/cachetable.cc b/ft/cachetable.cc index 3a1fe8fcd34..8d4dacc4c18 100644 --- a/ft/cachetable.cc +++ b/ft/cachetable.cc @@ -13,14 +13,12 @@ #include "memory.h" #include "cachetable.h" #include "rwlock.h" -#include "nonblocking_mutex.h" #include #include "checkpoint.h" -#include "minicron.h" #include "log-internal.h" -#include "kibbutz.h" -#include "background_job_manager.h" -#include "partitioned_counter.h" +#include +#include +#include "cachetable-internal.h" /////////////////////////////////////////////////////////////////////////////////// // Engine status @@ -31,40 +29,12 @@ // These should be in the cachetable object, but we make them file-wide so that gdb can get them easily. // They were left here after engine status cleanup (#2949, rather than moved into the status struct) // so they are still easily available to the debugger and to save lots of typing. - -// If we had constructors and destructors, this would be cleaner. For now, we initialize with setup_cachetable_statistics(). -static PARTITIONED_COUNTER cachetable_miss; -static PARTITIONED_COUNTER cachetable_misstime; // time spent waiting for disk read -static PARTITIONED_COUNTER cachetable_puts; // how many times has a newly created node been put into the cachetable? -static PARTITIONED_COUNTER cachetable_prefetches; // how many times has a block been prefetched into the cachetable? -static PARTITIONED_COUNTER cachetable_evictions; -static PARTITIONED_COUNTER cleaner_executions; // number of times the cleaner thread's loop has executed - -static bool cachetables_inited = false; - -void toku_cachetables_init(void) { - assert(!cachetables_inited); - cachetables_inited = true; - cachetable_miss = create_partitioned_counter(); - cachetable_misstime = create_partitioned_counter(); - cachetable_puts = create_partitioned_counter(); - cachetable_prefetches = create_partitioned_counter(); - cachetable_evictions = create_partitioned_counter(); - cleaner_executions = create_partitioned_counter(); -} - -void toku_cachetables_destroy(void) { -#define DESTROY(x) destroy_partitioned_counter(x); x=NULL; - assert(cachetables_inited); - cachetables_inited = false; - DESTROY(cachetable_miss); - DESTROY(cachetable_misstime); - DESTROY(cachetable_puts); - DESTROY(cachetable_prefetches); - DESTROY(cachetable_evictions); - DESTROY(cleaner_executions); -#undef DESTROY -} +static uint64_t cachetable_miss; +static uint64_t cachetable_misstime; // time spent waiting for disk read +static uint64_t cachetable_puts; // how many times has a newly created node been put into the cachetable? +static uint64_t cachetable_prefetches; // how many times has a block been prefetched into the cachetable? +static uint64_t cachetable_evictions; +static uint64_t cleaner_executions; // number of times the cleaner thread's loop has executed static CACHETABLE_STATUS_S ct_status; @@ -102,40 +72,7 @@ status_init(void) { #define STATUS_VALUE(x) ct_status.status[x].value.num -typedef struct ctpair *PAIR; -struct ctpair { - CACHEFILE cachefile; - CACHEKEY key; - void* value_data; // data used by client threads, FTNODEs and ROLLBACK_LOG_NODEs - void* cloned_value_data; // cloned copy of value_data used for checkpointing - long cloned_value_size; // size of cloned_value_data, used for accounting of ct->size_current - void* disk_data; // data used to fetch/flush value_data to and from disk. - PAIR_ATTR attr; - enum cachetable_dirty dirty; - - uint32_t fullhash; - - CACHETABLE_FLUSH_CALLBACK flush_callback; - CACHETABLE_PARTIAL_EVICTION_EST_CALLBACK pe_est_callback; - CACHETABLE_PARTIAL_EVICTION_CALLBACK pe_callback; - CACHETABLE_CLEANER_CALLBACK cleaner_callback; - CACHETABLE_CLONE_CALLBACK clone_callback; - long size_evicting_estimate; - void *write_extraargs; - - PAIR clock_next,clock_prev; // In clock. - PAIR hash_chain; - uint32_t count; // clock count - - bool checkpoint_pending; // If this is on, then we have got to write the pair out to disk before modifying it. - PAIR pending_next; - PAIR pending_prev; - - struct nb_mutex value_nb_mutex; // single writer, protects value_data - struct nb_mutex disk_nb_mutex; // single writer, protects disk_data, is used for writing cloned nodes for checkpoint - struct toku_list next_for_cachefile; // link in the cachefile list -}; static void * const zero_value = 0; static PAIR_ATTR const zero_attr = { @@ -147,145 +84,39 @@ static PAIR_ATTR const zero_attr = { .is_valid = true }; -static void maybe_flush_some (CACHETABLE ct, long size); static inline void ctpair_destroy(PAIR p) { + toku_mutex_destroy(&p->mutex); nb_mutex_destroy(&p->value_nb_mutex); nb_mutex_destroy(&p->disk_nb_mutex); toku_free(p); } -// The cachetable is as close to an ENV as we get. -// cachetable_mutex -struct cachetable { - uint32_t n_in_table; // number of pairs in the hash table - uint32_t table_size; // number of buckets in the hash table - PAIR *table; // hash table - PAIR clock_head; // of clock . head is the next thing to be up for decrement. - PAIR cleaner_head; // for cleaner thread. head is the next thing to look at for possible cleaning. - CACHEFILE cachefiles; // list of cachefiles that use this cachetable - CACHEFILE cachefiles_in_checkpoint; //list of cachefiles included in checkpoint in progress - int64_t size_reserved; // How much memory is reserved (e.g., by the loader) - int64_t size_current; // the sum of the sizes of the pairs in the cachetable - int64_t size_limit; // the limit to the sum of the pair sizes - int64_t size_evicting; // the sum of the sizes of the pairs being written - TOKULOGGER logger; - toku_mutex_t mutex; // coarse lock that protects the cachetable, the cachefiles, and the pairs - - KIBBUTZ client_kibbutz; // pool of worker threads and jobs to do asynchronously for the client. - KIBBUTZ ct_kibbutz; // pool of worker threads and jobs to do asynchronously for the cachetable - KIBBUTZ checkpointing_kibbutz; // small pool for checkpointing cloned pairs - - LSN lsn_of_checkpoint_in_progress; - uint32_t checkpoint_num_files; // how many cachefiles are in the checkpoint - uint32_t checkpoint_num_txns; // how many transactions are in the checkpoint - PAIR pending_head; // list of pairs marked with checkpoint_pending - struct rwlock pending_lock; // multiple writer threads, single checkpoint thread, - // see comments in toku_cachetable_begin_checkpoint to understand - // purpose of the pending_lock - struct minicron checkpointer; // the periodic checkpointing thread - struct minicron cleaner; // the periodic cleaner thread - uint32_t cleaner_iterations; // how many times to run the cleaner per - // cleaner period (minicron has a - // minimum period of 1s so if you want - // more frequent cleaner runs you must - // use this) - char *env_dir; - - // variables for engine status - int64_t size_nonleaf; - int64_t size_leaf; - int64_t size_rollback; - int64_t size_cachepressure; - - // variable used by the checkpoint thread to know - // when all work induced by cloning on client threads is done - BACKGROUND_JOB_MANAGER checkpoint_clones_bjm; - - // temporary, for handling flow control - toku_cond_t flow_control_cond; -}; +static inline void pair_lock(PAIR p) { + toku_mutex_lock(&p->mutex); +} +static inline void pair_unlock(PAIR p) { + toku_mutex_unlock(&p->mutex); +} void toku_cachetable_get_status(CACHETABLE ct, CACHETABLE_STATUS statp) { if (!ct_status.initialized) { status_init(); } - STATUS_VALUE(CT_MISS) = read_partitioned_counter(cachetable_miss); - STATUS_VALUE(CT_MISSTIME) = read_partitioned_counter(cachetable_misstime); - STATUS_VALUE(CT_PUTS) = read_partitioned_counter(cachetable_puts); - STATUS_VALUE(CT_PREFETCHES) = read_partitioned_counter(cachetable_prefetches); - STATUS_VALUE(CT_SIZE_CURRENT) = ct->size_current; - STATUS_VALUE(CT_SIZE_LIMIT) = ct->size_limit; - STATUS_VALUE(CT_SIZE_WRITING) = ct->size_evicting; - STATUS_VALUE(CT_SIZE_NONLEAF) = ct->size_nonleaf; - STATUS_VALUE(CT_SIZE_LEAF) = ct->size_leaf; - STATUS_VALUE(CT_SIZE_ROLLBACK) = ct->size_rollback; - STATUS_VALUE(CT_SIZE_CACHEPRESSURE) = ct->size_cachepressure; - STATUS_VALUE(CT_EVICTIONS) = read_partitioned_counter(cachetable_evictions); - STATUS_VALUE(CT_CLEANER_EXECUTIONS) = read_partitioned_counter(cleaner_executions); + STATUS_VALUE(CT_MISS) = cachetable_miss; + STATUS_VALUE(CT_MISSTIME) = cachetable_misstime; + STATUS_VALUE(CT_PUTS) = cachetable_puts; + STATUS_VALUE(CT_PREFETCHES) = cachetable_prefetches; + STATUS_VALUE(CT_EVICTIONS) = cachetable_evictions; + STATUS_VALUE(CT_CLEANER_EXECUTIONS) = cleaner_executions; STATUS_VALUE(CT_CLEANER_PERIOD) = toku_get_cleaner_period_unlocked(ct); STATUS_VALUE(CT_CLEANER_ITERATIONS) = toku_get_cleaner_iterations_unlocked(ct); + ct->ev.fill_engine_status(); *statp = ct_status; } - -// Lock the cachetable. Used for a variety of purposes. -static inline void cachetable_lock(CACHETABLE ct __attribute__((unused))) { - toku_mutex_lock(&ct->mutex); -} - -// Unlock the cachetable -static inline void cachetable_unlock(CACHETABLE ct __attribute__((unused))) { - toku_mutex_unlock(&ct->mutex); -} - -// Wait for cache table space to become available -// size_current is number of bytes currently occupied by data (referred to by pairs) -// size_evicting is number of bytes queued up to be evicted -static inline void cachetable_wait_write(CACHETABLE ct) { - // if we're writing more than half the data in the cachetable - while (2*ct->size_evicting > ct->size_current) { - toku_cond_wait(&ct->flow_control_cond, &ct->mutex); - } -} - -static inline void cachetable_wakeup_write(CACHETABLE ct) { - if (8*ct->size_evicting <= ct->size_current) { - toku_cond_broadcast(&ct->flow_control_cond); - } -} - -struct cachefile { - CACHEFILE next; - CACHEFILE next_in_checkpoint; - struct toku_list pairs_for_cachefile; // list of pairs for this cachefile - bool for_checkpoint; //True if part of the in-progress checkpoint - - // If set and the cachefile closes, the file will be removed. - // Clients must not operate on the cachefile after setting this, - // nor attempt to open any cachefile with the same fname (dname) - // until this cachefile has been fully closed and unlinked. - bool unlink_on_close; - int fd; /* Bug: If a file is opened read-only, then it is stuck in read-only. If it is opened read-write, then subsequent writers can write to it too. */ - CACHETABLE cachetable; - struct fileid fileid; - FILENUM filenum; - char *fname_in_env; /* Used for logging */ - - void *userdata; - int (*log_fassociate_during_checkpoint)(CACHEFILE cf, void *userdata); // When starting a checkpoint we must log all open files. - int (*log_suppress_rollback_during_checkpoint)(CACHEFILE cf, void *userdata); // When starting a checkpoint we must log which files need rollbacks suppressed - int (*close_userdata)(CACHEFILE cf, int fd, void *userdata, char **error_string, bool lsnvalid, LSN); // when closing the last reference to a cachefile, first call this function. - int (*begin_checkpoint_userdata)(LSN lsn_of_checkpoint, void *userdata); // before checkpointing cachefiles call this function. - int (*checkpoint_userdata)(CACHEFILE cf, int fd, void *userdata); // when checkpointing a cachefile, call this function. - int (*end_checkpoint_userdata)(CACHEFILE cf, int fd, void *userdata); // after checkpointing cachefiles call this function. - int (*note_pin_by_checkpoint)(CACHEFILE cf, void *userdata); // add a reference to the userdata to prevent it from being removed from memory - int (*note_unpin_by_checkpoint)(CACHEFILE cf, void *userdata); // add a reference to the userdata to prevent it from being removed from memory - BACKGROUND_JOB_MANAGER bjm; -}; - // FIXME global with no toku prefix void remove_background_job_from_cf(CACHEFILE cf) { @@ -305,14 +136,14 @@ void cachefile_kibbutz_enq (CACHEFILE cf, void (*f)(void*), void *extra) } static int -checkpoint_thread (void *cachetable_v) +checkpoint_thread (void *checkpointer_v) // Effect: If checkpoint_period>0 thn periodically run a checkpoint. // If someone changes the checkpoint_period (calling toku_set_checkpoint_period), then the checkpoint will run sooner or later. // If someone sets the checkpoint_shutdown boolean , then this thread exits. // This thread notices those changes by waiting on a condition variable. { - CACHETABLE CAST_FROM_VOIDP(ct, cachetable_v); - int r = toku_checkpoint(ct, ct->logger, NULL, NULL, NULL, NULL, SCHEDULED_CHECKPOINT); + CHECKPOINTER CAST_FROM_VOIDP(cp, checkpointer_v); + int r = toku_checkpoint(cp, cp->get_logger(), NULL, NULL, NULL, NULL, SCHEDULED_CHECKPOINT); if (r) { fprintf(stderr, "%s:%d Got error %d while doing checkpoint\n", __FILE__, __LINE__, r); abort(); // Don't quite know what to do with these errors. @@ -321,45 +152,41 @@ checkpoint_thread (void *cachetable_v) } int toku_set_checkpoint_period (CACHETABLE ct, uint32_t new_period) { - return toku_minicron_change_period(&ct->checkpointer, new_period); + return ct->cp.set_checkpoint_period(new_period); } uint32_t toku_get_checkpoint_period (CACHETABLE ct) { - return toku_minicron_get_period(&ct->checkpointer); + return ct->cp.get_checkpoint_period(); } uint32_t toku_get_checkpoint_period_unlocked (CACHETABLE ct) { - return toku_minicron_get_period_unlocked(&ct->checkpointer); + return ct->cp.get_checkpoint_period(); } int toku_set_cleaner_period (CACHETABLE ct, uint32_t new_period) { - return toku_minicron_change_period(&ct->cleaner, new_period); + ct->cl.set_period(new_period); + return 0; } uint32_t toku_get_cleaner_period (CACHETABLE ct) { - return toku_minicron_get_period(&ct->cleaner); + return ct->cl.get_period(); } uint32_t toku_get_cleaner_period_unlocked (CACHETABLE ct) { - return toku_minicron_get_period_unlocked(&ct->cleaner); + return ct->cl.get_period_unlocked(); } int toku_set_cleaner_iterations (CACHETABLE ct, uint32_t new_iterations) { - cachetable_lock(ct); - ct->cleaner_iterations = new_iterations; - cachetable_unlock(ct); + ct->cl.set_iterations(new_iterations); return 0; } uint32_t toku_get_cleaner_iterations (CACHETABLE ct) { - cachetable_lock(ct); - uint32_t retval = toku_get_cleaner_iterations_unlocked(ct); - cachetable_unlock(ct); - return retval; + return ct->cl.get_iterations(); } uint32_t toku_get_cleaner_iterations_unlocked (CACHETABLE ct) { - return ct->cleaner_iterations; + return ct->cl.get_iterations(); } // reserve 25% as "unreservable". The loader cannot have it. @@ -372,52 +199,38 @@ int toku_create_cachetable(CACHETABLE *result, long size_limit, LSN UU(initial_l CACHETABLE MALLOC(ct); if (ct == 0) return ENOMEM; memset(ct, 0, sizeof(*ct)); - HELGRIND_VALGRIND_HG_DISABLE_CHECKING(&ct->size_nonleaf, sizeof ct->size_nonleaf); // modified only when the cachetable lock is held, but read by engine status - HELGRIND_VALGRIND_HG_DISABLE_CHECKING(&ct->size_current, sizeof ct->size_current); - HELGRIND_VALGRIND_HG_DISABLE_CHECKING(&ct->size_evicting, sizeof ct->size_evicting); - HELGRIND_VALGRIND_HG_DISABLE_CHECKING(&ct->size_leaf, sizeof ct->size_leaf); - HELGRIND_VALGRIND_HG_DISABLE_CHECKING(&ct->size_rollback, sizeof ct->size_rollback); - HELGRIND_VALGRIND_HG_DISABLE_CHECKING(&ct->size_cachepressure, sizeof ct->size_cachepressure); - ct->table_size = 4; - rwlock_init(&ct->pending_lock); - XCALLOC_N(ct->table_size, ct->table); - ct->size_limit = size_limit; - ct->size_reserved = unreservable_memory(size_limit); - ct->logger = logger; - toku_mutex_init(&ct->mutex, NULL); + + ct->list.init(); + ct->cf_list.init(); int num_processors = toku_os_get_number_active_processors(); ct->client_kibbutz = toku_kibbutz_create(num_processors); ct->ct_kibbutz = toku_kibbutz_create(2*num_processors); int checkpointing_nworkers = (num_processors/4) ? num_processors/4 : 1; ct->checkpointing_kibbutz = toku_kibbutz_create(checkpointing_nworkers); - - toku_minicron_setup(&ct->checkpointer, 0, checkpoint_thread, ct); // default is no checkpointing - toku_minicron_setup(&ct->cleaner, 0, toku_cleaner_thread, ct); // default is no cleaner, for now - ct->cleaner_iterations = 1; // default is one iteration + // must be done after creating ct_kibbutz + ct->ev.init(size_limit, &ct->list, ct->ct_kibbutz, EVICTION_PERIOD); + ct->cp.init(ct, logger, &ct->cf_list); + ct->cl.init(1, &ct->list, ct); // by default, start with one iteration ct->env_dir = toku_xstrdup("."); - bjm_init(&ct->checkpoint_clones_bjm); - toku_cond_init(&ct->flow_control_cond, NULL); *result = ct; return 0; } +// Returns a pointer to the checkpoint contained within +// the given cachetable. +CHECKPOINTER toku_cachetable_get_checkpointer(CACHETABLE ct) { + return &ct->cp; +} + uint64_t toku_cachetable_reserve_memory(CACHETABLE ct, double fraction) { - cachetable_lock(ct); - cachetable_wait_write(ct); - uint64_t reserved_memory = fraction*(ct->size_limit-ct->size_reserved); - ct->size_reserved += reserved_memory; - maybe_flush_some(ct, reserved_memory); - ct->size_current += reserved_memory; - cachetable_unlock(ct); + uint64_t reserved_memory = 0; + reserved_memory = ct->ev.reserve_memory(fraction); return reserved_memory; } void toku_cachetable_release_reserved_memory(CACHETABLE ct, uint64_t reserved_memory) { - cachetable_lock(ct); - ct->size_current -= reserved_memory; - ct->size_reserved -= reserved_memory; - cachetable_unlock(ct); + ct->ev.release_reserved_memory(reserved_memory); } void @@ -434,11 +247,11 @@ toku_cachetable_set_env_dir(CACHETABLE ct, const char *env_dir) { // Once the close has finished, there must not be a cachefile with that name // in the cachetable. int toku_cachefile_of_iname_in_env (CACHETABLE ct, const char *iname_in_env, CACHEFILE *cf) { - cachetable_lock(ct); + ct->cf_list.read_lock(); CACHEFILE extant; int r; r = ENOENT; - for (extant = ct->cachefiles; extant; extant=extant->next) { + for (extant = ct->cf_list.m_head; extant; extant = extant->next) { if (extant->fname_in_env && !strcmp(extant->fname_in_env, iname_in_env)) { *cf = extant; @@ -446,7 +259,7 @@ int toku_cachefile_of_iname_in_env (CACHETABLE ct, const char *iname_in_env, CAC break; } } - cachetable_unlock(ct); + ct->cf_list.read_unlock(); return r; } @@ -454,23 +267,21 @@ int toku_cachefile_of_iname_in_env (CACHETABLE ct, const char *iname_in_env, CAC // This function can only be called if the brt is still open, so file must // still be open int toku_cachefile_of_filenum (CACHETABLE ct, FILENUM filenum, CACHEFILE *cf) { - cachetable_lock(ct); + ct->cf_list.read_lock(); CACHEFILE extant; int r = ENOENT; *cf = NULL; - for (extant = ct->cachefiles; extant; extant=extant->next) { + for (extant = ct->cf_list.m_head; extant; extant = extant->next) { if (extant->filenum.fileid==filenum.fileid) { *cf = extant; r = 0; break; } } - cachetable_unlock(ct); + ct->cf_list.read_unlock(); return r; } -static FILENUM next_filenum_to_use={0}; - static void cachefile_init_filenum(CACHEFILE cf, int fd, const char *fname_in_env, struct fileid fileid) { cf->fd = fd; cf->fileid = fileid; @@ -490,17 +301,19 @@ toku_cachetable_reserve_filenum(CACHETABLE ct) { CACHEFILE extant; FILENUM filenum; invariant(ct); - cachetable_lock(ct); + // TODO: (Zardosht) make this function a method on the cf_list + // taking a write lock because we are modifying next_filenum_to_use + ct->cf_list.write_lock(); try_again: - for (extant = ct->cachefiles; extant; extant=extant->next) { - if (next_filenum_to_use.fileid==extant->filenum.fileid) { - next_filenum_to_use.fileid++; + for (extant = ct->cf_list.m_head; extant; extant = extant->next) { + if (ct->cf_list.m_next_filenum_to_use.fileid==extant->filenum.fileid) { + ct->cf_list.m_next_filenum_to_use.fileid++; goto try_again; } } - filenum = next_filenum_to_use; - next_filenum_to_use.fileid++; - cachetable_unlock(ct); + filenum = ct->cf_list.m_next_filenum_to_use; + ct->cf_list.m_next_filenum_to_use.fileid++; + ct->cf_list.write_unlock(); return filenum; } @@ -517,8 +330,8 @@ int toku_cachetable_openfd_with_filenum (CACHEFILE *cfptr, CACHETABLE ct, int fd r=get_error_errno(); close(fd); // no change for t:2444 return r; } - cachetable_lock(ct); - for (extant = ct->cachefiles; extant; extant=extant->next) { + ct->cf_list.write_lock(); + for (extant = ct->cf_list.m_head; extant; extant = extant->next) { if (memcmp(&extant->fileid, &fileid, sizeof(fileid))==0) { // Clients must serialize cachefile open, close, and unlink // So, during open, we should never see a closing cachefile @@ -536,7 +349,7 @@ int toku_cachetable_openfd_with_filenum (CACHEFILE *cfptr, CACHETABLE ct, int fd } // assert that the filenum is not in use - for (extant = ct->cachefiles; extant; extant=extant->next) { + for (extant = ct->cf_list.m_head; extant; extant = extant->next) { invariant(extant->filenum.fileid != filenum.fileid); } @@ -547,16 +360,15 @@ int toku_cachetable_openfd_with_filenum (CACHEFILE *cfptr, CACHETABLE ct, int fd newcf->cachetable = ct; newcf->filenum = filenum; cachefile_init_filenum(newcf, fd, fname_in_env, fileid); - newcf->next = ct->cachefiles; - ct->cachefiles = newcf; + newcf->next = ct->cf_list.m_head; + ct->cf_list.m_head = newcf; bjm_init(&newcf->bjm); - toku_list_init(&newcf->pairs_for_cachefile); *cfptr = newcf; r = 0; } exit: - cachetable_unlock(ct); + ct->cf_list.write_unlock(); return r; } @@ -626,9 +438,12 @@ static CACHEFILE remove_cf_from_list_locked (CACHEFILE cf, CACHEFILE list) { static void remove_cf_from_cachefiles_list (CACHEFILE cf) { CACHETABLE ct = cf->cachetable; - ct->cachefiles = remove_cf_from_list_locked(cf, ct->cachefiles); + ct->cf_list.write_lock(); + ct->cf_list.m_head = remove_cf_from_list_locked(cf, ct->cf_list.m_head); + ct->cf_list.write_unlock(); } +// TODO: (Zardosht) review locking of this function carefully in code review int toku_cachefile_close(CACHEFILE *cfp, char **error_string, bool oplsn_valid, LSN oplsn) { int r, close_error = 0; @@ -637,19 +452,13 @@ toku_cachefile_close(CACHEFILE *cfp, char **error_string, bool oplsn_valid, LSN bjm_wait_for_jobs_to_finish(cf->bjm); - // Hold the cachetable lock while we check some invariants and - // flush the cachefile. - cachetable_lock(ct); - // Clients should never attempt to close a cachefile that is being // checkpointed. We notify clients this is happening in the // note_pin_by_checkpoint callback. - assert(!cf->next_in_checkpoint); assert(!cf->for_checkpoint); // Flush the cachefile and remove all of its pairs from the cachetable cachetable_flush_cachefile(ct, cf); - assert(toku_list_empty(&cf->pairs_for_cachefile)); // Call the close userdata callback to notify the client this cachefile // and its underlying file are going to be closed @@ -661,9 +470,6 @@ toku_cachefile_close(CACHEFILE *cfp, char **error_string, bool oplsn_valid, LSN bjm_destroy(cf->bjm); cf->bjm = NULL; - // Don't hold the cachetable lock during fsync/close/unlink, etc - cachetable_unlock(ct); - // fsync and close the fd. r = toku_file_fsync_without_accounting(cf->fd); assert(r == 0); @@ -696,9 +502,7 @@ toku_cachefile_close(CACHEFILE *cfp, char **error_string, bool oplsn_valid, LSN int toku_cachefile_flush (CACHEFILE cf) { bjm_wait_for_jobs_to_finish(cf->bjm); CACHETABLE ct = cf->cachetable; - cachetable_lock(ct); cachetable_flush_cachefile(ct, cf); - cachetable_unlock(ct); return 0; } @@ -726,159 +530,24 @@ uint32_t toku_cachetable_hash (CACHEFILE cachefile, BLOCKNUM key) return final(cachefile->filenum.fileid, (uint32_t)(key.b>>32), (uint32_t)key.b); } -// has ct locked on entry -// This function MUST NOT release and reacquire the cachetable lock -// Its callers (toku_cachetable_put_with_dep_pairs) depend on this behavior. -static void cachetable_rehash (CACHETABLE ct, uint32_t newtable_size) { - // printf("rehash %p %d %d %d\n", t, primeindexdelta, ct->n_in_table, ct->table_size); - - assert(newtable_size>=4 && ((newtable_size & (newtable_size-1))==0)); - PAIR *XCALLOC_N(newtable_size, newtable); - uint32_t i; - //printf("%s:%d newtable_size=%d\n", __FILE__, __LINE__, newtable_size); - assert(newtable!=0); - uint32_t oldtable_size = ct->table_size; - ct->table_size=newtable_size; - for (i=0; itable[i])!=0) { - unsigned int h = p->fullhash&(newtable_size-1); - ct->table[i] = p->hash_chain; - p->hash_chain = newtable[h]; - newtable[h] = p; - } - } - toku_free(ct->table); - // printf("Freed\n"); - ct->table=newtable; - //printf("Done growing or shrinking\n"); -} - #define CLOCK_SATURATION 15 #define CLOCK_INITIAL_COUNT 3 -static void pair_remove (CACHETABLE ct, PAIR p) { - if (p->clock_prev == p) { - assert(ct->clock_head == p); - assert(p->clock_next == p); - assert(ct->cleaner_head == p); - ct->clock_head = NULL; - ct->cleaner_head = NULL; - } - else { - if (p == ct->clock_head) { - ct->clock_head = ct->clock_head->clock_next; - } - if (p == ct->cleaner_head) { - ct->cleaner_head = ct->cleaner_head->clock_next; - } - p->clock_prev->clock_next = p->clock_next; - p->clock_next->clock_prev = p->clock_prev; - - } -} - -static void pair_add_to_clock (CACHETABLE ct, PAIR p) { - // requires that p is not currently in the table. - // inserts p into the clock list at the tail. - - p->count = CLOCK_INITIAL_COUNT; - //assert either both head and tail are set or they are both NULL - // tail and head exist - if (ct->clock_head) { - assert(ct->cleaner_head); - // insert right before the head - p->clock_next = ct->clock_head; - p->clock_prev = ct->clock_head->clock_prev; - - p->clock_prev->clock_next = p; - p->clock_next->clock_prev = p; - - } - // this is the first element in the list - else { - ct->clock_head = p; - p->clock_next = p->clock_prev = ct->clock_head; - ct->cleaner_head = p; - } -} - +// Requires pair's mutex to be held static void pair_touch (PAIR p) { p->count = (p->count < CLOCK_SATURATION) ? p->count+1 : CLOCK_SATURATION; } -static PAIR remove_from_hash_chain (PAIR remove_me, PAIR list) { - if (remove_me==list) return list->hash_chain; - list->hash_chain = remove_from_hash_chain(remove_me, list->hash_chain); - return list; -} - -//Remove a pair from the list of pairs that were marked with the -//pending bit for the in-progress checkpoint. -//Requires: cachetable lock is held during duration. -static void -pending_pairs_remove (CACHETABLE ct, PAIR p) { - if (p->pending_next) { - p->pending_next->pending_prev = p->pending_prev; - } - if (p->pending_prev) { - p->pending_prev->pending_next = p->pending_next; - } - else if (ct->pending_head==p) { - ct->pending_head = p->pending_next; - } - p->pending_prev = p->pending_next = NULL; -} - -static void -cachetable_remove_pair_attr (CACHETABLE ct, PAIR_ATTR attr) { - assert(attr.is_valid); - ct->size_current -= attr.size; - ct->size_nonleaf -= attr.nonleaf_size; - ct->size_leaf -= attr.leaf_size; - ct->size_rollback -= attr.rollback_size; - ct->size_cachepressure -= attr.cache_pressure_size; - assert(ct->size_current >= 0); -} - -static void -cachetable_add_pair_attr(CACHETABLE ct, PAIR_ATTR attr) { - assert(attr.is_valid); - ct->size_current += attr.size; - ct->size_nonleaf += attr.nonleaf_size; - ct->size_leaf += attr.leaf_size; - ct->size_rollback += attr.rollback_size; - ct->size_cachepressure += attr.cache_pressure_size; -} - -static void -cachetable_change_pair_attr(CACHETABLE ct, PAIR_ATTR old_attr, PAIR_ATTR new_attr) { - cachetable_add_pair_attr(ct, new_attr); - cachetable_remove_pair_attr(ct, old_attr); -} - // Remove a pair from the cachetable // Effects: the pair is removed from the LRU list and from the cachetable's hash table. // The size of the objects in the cachetable is adjusted by the size of the pair being // removed. -static void cachetable_remove_pair (CACHETABLE ct, PAIR p) { - pair_remove(ct, p); - pending_pairs_remove(ct, p); - toku_list_remove(&p->next_for_cachefile); - - assert(ct->n_in_table>0); - ct->n_in_table--; - // Remove it from the hash chain. - { - unsigned int h = p->fullhash&(ct->table_size-1); - ct->table[h] = remove_from_hash_chain (p, ct->table[h]); - } - cachetable_remove_pair_attr(ct, p->attr); +static void cachetable_remove_pair (pair_list* list, evictor* ev, PAIR p) { + list->evict(p); + ev->remove_pair_attr(p->attr); } -static void cachetable_free_pair(CACHETABLE ct, PAIR p) { - // helgrind +static void cachetable_free_pair(PAIR p) { CACHETABLE_FLUSH_CALLBACK flush_callback = p->flush_callback; CACHEKEY key = p->key; void *value = p->value_data; @@ -886,8 +555,7 @@ static void cachetable_free_pair(CACHETABLE ct, PAIR p) { void *write_extraargs = p->write_extraargs; PAIR_ATTR old_attr = p->attr; - increment_partitioned_counter(cachetable_evictions, 1); - cachetable_unlock(ct); + cachetable_evictions++; PAIR_ATTR new_attr = p->attr; // Note that flush_callback is called with write_me false, so the only purpose of this // call is to tell the brt layer to evict the node (keep_me is false). @@ -897,8 +565,6 @@ static void cachetable_free_pair(CACHETABLE ct, PAIR p) { // pass in NULL and -1, dummy values flush_callback(NULL, -1, key, value, &disk_data, write_extraargs, old_attr, &new_attr, false, false, true, false); - cachetable_lock(ct); - ctpair_destroy(p); } @@ -908,7 +574,17 @@ static void cachetable_free_pair(CACHETABLE ct, PAIR p) { // The sole purpose of this function is to remove the node, so the write_me // argument to the flush callback is false, and the flush callback won't do // anything except destroy the node. -static void cachetable_maybe_remove_and_free_pair (CACHETABLE ct, PAIR p) { +// +// on input, pair_list's write lock is held and PAIR's mutex is held +// on exit, only the pair_list's write lock is still held +// +static void cachetable_maybe_remove_and_free_pair ( + pair_list* pl, + evictor* ev, + PAIR p + ) +{ + // this ensures that a clone running in the background first completes if (nb_mutex_users(&p->value_nb_mutex) == 0) { // assumption is that if we are about to remove the pair // that no one has grabbed the disk_nb_mutex, @@ -916,8 +592,12 @@ static void cachetable_maybe_remove_and_free_pair (CACHETABLE ct, PAIR p) { // no one is writing a cloned value out. assert(nb_mutex_users(&p->disk_nb_mutex) == 0); assert(p->cloned_value_data == NULL); - cachetable_remove_pair(ct, p); - cachetable_free_pair(ct, p); + cachetable_remove_pair(pl, ev, p); + pair_unlock(p); + cachetable_free_pair(p); + } + else { + pair_unlock(p); } } @@ -925,8 +605,11 @@ static void cachetable_maybe_remove_and_free_pair (CACHETABLE ct, PAIR p) { // responsibility of this function is to only write a locked PAIR to disk // and NOTHING else. We do not manipulate the state of the PAIR // of the cachetable here (with the exception of ct->size_current for clones) +// +// No pair_list lock should be held, and the PAIR mutex should not be held +// static void cachetable_only_write_locked_data( - CACHETABLE ct, + evictor* ev, PAIR p, bool for_checkpoint, PAIR_ATTR* new_attr, @@ -939,11 +622,19 @@ static void cachetable_only_write_locked_data( void *value = is_clone ? p->cloned_value_data : p->value_data; void *disk_data = p->disk_data; void *write_extraargs = p->write_extraargs; - PAIR_ATTR old_attr = p->attr; + PAIR_ATTR old_attr; + // we do this for drd. If we are a cloned pair and only + // have the disk_nb_mutex, it is a race to access p->attr. + // Luckily, old_attr here is only used for some test applications, + // so inaccurate non-size fields are ok. + if (is_clone) { + old_attr = make_pair_attr(p->cloned_value_size); + } + else { + old_attr = p->attr; + } bool dowrite = true; - - cachetable_unlock(ct); - + // write callback flush_callback( cachefile, @@ -960,10 +651,9 @@ static void cachetable_only_write_locked_data( is_clone //is_clone ); p->disk_data = disk_data; - cachetable_lock(ct); if (is_clone) { p->cloned_value_data = NULL; - ct->size_current -= p->cloned_value_size; + ev->remove_from_size_current(p->cloned_value_size); p->cloned_value_size = 0; } } @@ -975,265 +665,128 @@ static void cachetable_only_write_locked_data( // evictor threads that evict dirty PAIRS, and by the checkpoint thread // that needs to write out a dirty node for checkpoint. // -static void cachetable_write_locked_pair(CACHETABLE ct, PAIR p) { +// Requires on entry for p->mutex to NOT be held, otherwise +// calling cachetable_only_write_locked_data will be very expensive +// +static void cachetable_write_locked_pair( + evictor* ev, + PAIR p, + bool for_checkpoint + ) +{ PAIR_ATTR old_attr = p->attr; PAIR_ATTR new_attr = p->attr; - rwlock_read_lock(&ct->pending_lock, &ct->mutex); - bool for_checkpoint = p->checkpoint_pending; - p->checkpoint_pending = false; // grabbing the disk_nb_mutex here ensures that // after this point, no one is writing out a cloned value // if we grab the disk_nb_mutex inside the if clause, // then we may try to evict a PAIR that is in the process // of having its clone be written out - nb_mutex_lock(&p->disk_nb_mutex, &ct->mutex); + pair_lock(p); + nb_mutex_lock(&p->disk_nb_mutex, &p->mutex); + pair_unlock(p); // make sure that assumption about cloned_value_data is true // if we have grabbed the disk_nb_mutex, then that means that // there should be no cloned value data assert(p->cloned_value_data == NULL); if (p->dirty) { - cachetable_only_write_locked_data(ct, p, for_checkpoint, &new_attr, false); + cachetable_only_write_locked_data(ev, p, for_checkpoint, &new_attr, false); // // now let's update variables // if (new_attr.is_valid) { p->attr = new_attr; - cachetable_change_pair_attr(ct, old_attr, new_attr); + ev->change_pair_attr(old_attr, new_attr); } } - nb_mutex_unlock(&p->disk_nb_mutex); // the pair is no longer dirty once written p->dirty = CACHETABLE_CLEAN; - - assert(!p->checkpoint_pending); - rwlock_read_unlock(&ct->pending_lock); -} - -// complete the write of a pair by reseting the writing flag, and -// maybe removing the pair from the cachetable if there are no -// references to it - -static void cachetable_complete_write_pair (CACHETABLE ct, PAIR p) { - nb_mutex_unlock(&p->value_nb_mutex); - cachetable_maybe_remove_and_free_pair(ct, p); -} - -// Write a pair to storage -// Effects: an exclusive lock on the pair is obtained, the write callback is called, -// the pair dirty state is adjusted, and the write is completed. The keep_me -// boolean is true, so the pair is not yet evicted from the cachetable. -// Requires: This thread must hold the write lock for the pair. -static void cachetable_evict_pair(CACHETABLE ct, PAIR p) { - long old_size = p->attr.size; - // this function may change p->attr.size, so we saved - // the estimate we must have put into ct->evicting_size above - cachetable_write_locked_pair(ct, p); - - // maybe wakeup any stalled writers when the pending writes fall below - // 1/8 of the size of the cachetable - ct->size_evicting -= old_size; - assert(ct->size_evicting >= 0); - cachetable_wakeup_write(ct); - cachetable_complete_write_pair(ct, p); + pair_lock(p); + nb_mutex_unlock(&p->disk_nb_mutex); + pair_unlock(p); } // Worker thread function to writes and evicts a pair from memory to its cachefile static void cachetable_evicter(void* extra) { PAIR p = (PAIR)extra; + pair_list* pl = p->list; CACHEFILE cf = p->cachefile; - CACHETABLE ct = cf->cachetable; - cachetable_lock(ct); - cachetable_evict_pair(ct, p); - cachetable_unlock(ct); + pl->read_pending_exp_lock(); + bool for_checkpoint = p->checkpoint_pending; + p->checkpoint_pending = false; + // per the contract of evictor::evict_pair, + // the pair's mutex, p->mutex, must be held on entry + pair_lock(p); + p->ev->evict_pair(p, for_checkpoint); + pl->read_pending_exp_unlock(); bjm_remove_background_job(cf->bjm); } -// CT lock held on entry -// background job has been added for p->cachefile on entry -// responsibility of this function to make sure that background job is removed -static void try_evict_pair(CACHETABLE ct, PAIR p) { - CACHEFILE cf = p->cachefile; - // evictions without a write or unpinned pair's that are clean - // can be run in the current thread - - // must check for before we grab the write lock because we may - // be trying to evict something this thread is trying to read - if (!nb_mutex_users(&p->value_nb_mutex)) { - nb_mutex_lock(&p->value_nb_mutex, &ct->mutex); - - assert(ct->size_evicting >= 0); - ct->size_evicting += p->attr.size; - assert(ct->size_evicting >= 0); - - // if the PAIR is dirty, the running eviction requires writing the - // PAIR out. if the disk_nb_mutex is grabbed, then running - // eviction requires waiting for the disk_nb_mutex to become available, - // which may be expensive. Hence, if either is true, we - // do the eviction on a writer thread - if (!p->dirty && (nb_mutex_writers(&p->disk_nb_mutex) == 0)) { - cachetable_evict_pair(ct, p); - bjm_remove_background_job(cf->bjm); - } - else { - toku_kibbutz_enq(ct->ct_kibbutz, cachetable_evicter, p); - } - } -} - -static void do_partial_eviction(CACHETABLE ct, PAIR p) { - PAIR_ATTR new_attr; - PAIR_ATTR old_attr = p->attr; - - cachetable_unlock(ct); - p->pe_callback(p->value_data, old_attr, &new_attr, p->write_extraargs); - cachetable_lock(ct); - - cachetable_change_pair_attr(ct, old_attr, new_attr); - p->attr = new_attr; - - assert(ct->size_evicting >= p->size_evicting_estimate); - ct->size_evicting -= p->size_evicting_estimate; - cachetable_wakeup_write(ct); - nb_mutex_unlock(&p->value_nb_mutex); -} - static void cachetable_partial_eviction(void* extra) { PAIR p = (PAIR)extra; CACHEFILE cf = p->cachefile; - CACHETABLE ct = cf->cachetable; - cachetable_lock(ct); - do_partial_eviction(ct,p); - cachetable_unlock(ct); + p->ev->do_partial_eviction(p); bjm_remove_background_job(cf->bjm); } -// cachetable lock held on entry -// run eviction on PAIR, may be partial eviction or full eviction -static bool run_eviction_on_pair(PAIR curr_in_clock, CACHETABLE ct) { - bool ret_val = false; - // function meant to be called on PAIR that is not being accessed right now - assert(nb_mutex_users(&curr_in_clock->value_nb_mutex) == 0); - assert(nb_mutex_users(&curr_in_clock->disk_nb_mutex) == 0); - CACHEFILE cf = curr_in_clock->cachefile; - int r = bjm_add_background_job(cf->bjm); - if (r) { - goto exit; - } - ret_val = true; - if (curr_in_clock->count > 0) { - curr_in_clock->count--; - // call the partial eviction callback - nb_mutex_lock(&curr_in_clock->value_nb_mutex, &ct->mutex); - - void *value = curr_in_clock->value_data; - void* disk_data = curr_in_clock->disk_data; - void *write_extraargs = curr_in_clock->write_extraargs; - enum partial_eviction_cost cost; - long bytes_freed_estimate = 0; - curr_in_clock->pe_est_callback( - value, - disk_data, - &bytes_freed_estimate, - &cost, - write_extraargs - ); - if (cost == PE_CHEAP) { - curr_in_clock->size_evicting_estimate = 0; - do_partial_eviction(ct, curr_in_clock); - bjm_remove_background_job(cf->bjm); - } - else if (cost == PE_EXPENSIVE) { - // only bother running an expensive partial eviction - // if it is expected to free space - if (bytes_freed_estimate > 0) { - curr_in_clock->size_evicting_estimate = bytes_freed_estimate; - ct->size_evicting += bytes_freed_estimate; - toku_kibbutz_enq(ct->ct_kibbutz, cachetable_partial_eviction, curr_in_clock); - } - else { - nb_mutex_unlock(&curr_in_clock->value_nb_mutex); - bjm_remove_background_job(cf->bjm); - } - } - else { - assert(false); - } - } - else { - // responsibility of try_evict_pair to eventually remove background job - try_evict_pair(ct, curr_in_clock); - } -exit: - return ret_val; -} - -static void maybe_flush_some (CACHETABLE ct, long size) { - if (size + ct->size_current <= ct->size_limit + ct->size_evicting) return; - - // - // These variables will help us detect if everything in the clock is currently being accessed. - // We must detect this case otherwise we will end up in an infinite loop below. - // - CACHEKEY curr_cachekey; - curr_cachekey.b = INT64_MAX; // create initial value so compiler does not complain - FILENUM curr_filenum; - curr_filenum.fileid = UINT32_MAX; // create initial value so compiler does not complain - bool set_val = false; - - while ((ct->clock_head) && (size + ct->size_current > ct->size_limit + ct->size_evicting)) { - PAIR curr_in_clock = ct->clock_head; - if (set_val && - curr_in_clock->key.b == curr_cachekey.b && - curr_in_clock->cachefile->filenum.fileid == curr_filenum.fileid) - { - // we have identified a cycle where everything in the clock is in use - // do not return an error - // just let memory be overfull - goto exit; - } - if (nb_mutex_users(&curr_in_clock->value_nb_mutex) || nb_mutex_users(&curr_in_clock->disk_nb_mutex)) { - if (!set_val) { - set_val = true; - curr_cachekey = ct->clock_head->key; - curr_filenum = ct->clock_head->cachefile->filenum; - } - } - else { - bool eviction_run = run_eviction_on_pair(curr_in_clock, ct); - if (eviction_run) { - set_val = false; - } - else if (!set_val) { - set_val = true; - curr_cachekey = ct->clock_head->key; - curr_filenum = ct->clock_head->cachefile->filenum; - } - } - // at this point, either curr_in_clock is still in the list because it has not been fully evicted, - // and we need to move ct->clock_head over. Otherwise, curr_in_clock has been fully evicted - // and we do NOT need to move ct->clock_head, as the removal of curr_in_clock - // modified ct->clock_head - if (ct->clock_head && (ct->clock_head == curr_in_clock)) { - ct->clock_head = ct->clock_head->clock_next; - } - } - - if ((4 * ct->n_in_table < ct->table_size) && ct->table_size > 4) { - cachetable_rehash(ct, ct->table_size/2); - } -exit: - return; -} - void toku_cachetable_maybe_flush_some(CACHETABLE ct) { - cachetable_lock(ct); - maybe_flush_some(ct, 0); - cachetable_unlock(ct); + // TODO: Maybe move this... + ct->ev.signal_eviction_thread(); +} + +// Initializes a pair's members. +// +void pair_init(PAIR p, + CACHEFILE cachefile, + CACHEKEY key, + void *value, + PAIR_ATTR attr, + enum cachetable_dirty dirty, + uint32_t fullhash, + CACHETABLE_WRITE_CALLBACK write_callback, + evictor *ev, + pair_list *list) +{ + p->cachefile = cachefile; + p->key = key; + p->value_data = value; + p->cloned_value_data = NULL; + p->cloned_value_size = 0; + p->disk_data = NULL; + p->attr = attr; + p->dirty = dirty; + p->fullhash = fullhash; + + p->flush_callback = write_callback.flush_callback; + p->pe_callback = write_callback.pe_callback; + p->pe_est_callback = write_callback.pe_est_callback; + p->cleaner_callback = write_callback.cleaner_callback; + p->clone_callback = write_callback.clone_callback; + p->write_extraargs = write_callback.write_extraargs; + + p->count = 0; // Is zero the correct init value? + p->checkpoint_pending = false; + + toku_mutex_init(&p->mutex, NULL); + nb_mutex_init(&p->value_nb_mutex); + nb_mutex_init(&p->disk_nb_mutex); + + p->size_evicting_estimate = 0; // Is zero the correct init value? + + p->ev = ev; + p->list = list; + + p->clock_next = p->clock_prev = NULL; + p->pending_next = p->pending_prev = NULL; + p->hash_chain = NULL; } // has ct locked on entry // This function MUST NOT release and reacquire the cachetable lock // Its callers (toku_cachetable_put_with_dep_pairs) depend on this behavior. +// +// Requires pair list's write lock to be held on entry. +// On exit, get pair with mutex held +// static PAIR cachetable_insert_at(CACHETABLE ct, CACHEFILE cachefile, CACHEKEY key, void *value, uint32_t fullhash, @@ -1243,72 +796,58 @@ static PAIR cachetable_insert_at(CACHETABLE ct, PAIR MALLOC(p); assert(p); memset(p, 0, sizeof *p); - p->cachefile = cachefile; - p->key = key; - p->value_data = value; - p->cloned_value_data = NULL; - p->cloned_value_size = 0; - p->disk_data = NULL; - p->fullhash = fullhash; - p->dirty = dirty; - p->attr = attr; - p->flush_callback = write_callback.flush_callback; - p->pe_callback = write_callback.pe_callback; - p->pe_est_callback = write_callback.pe_est_callback; - p->cleaner_callback = write_callback.cleaner_callback; - p->clone_callback = write_callback.clone_callback; - p->write_extraargs = write_callback.write_extraargs; - p->fullhash = fullhash; - p->clock_next = p->clock_prev = 0; - nb_mutex_init(&p->value_nb_mutex); - nb_mutex_init(&p->disk_nb_mutex); - pair_add_to_clock(ct, p); - toku_list_push(&cachefile->pairs_for_cachefile, &p->next_for_cachefile); - uint32_t h = fullhash & (ct->table_size-1); - p->hash_chain = ct->table[h]; - ct->table[h] = p; - ct->n_in_table++; - cachetable_add_pair_attr(ct, attr); - if (ct->n_in_table > ct->table_size) { - cachetable_rehash(ct, ct->table_size*2); - } + pair_init(p, + cachefile, + key, + value, + attr, + dirty, + fullhash, + write_callback, + &ct->ev, + &ct->list); + + ct->list.put(p); + ct->ev.add_pair_attr(attr); return p; } // has ct locked on entry // This function MUST NOT release and reacquire the cachetable lock // Its callers (toku_cachetable_put_with_dep_pairs) depend on this behavior. +// +// Requires pair list's write lock to be held on entry +// static int cachetable_put_internal( CACHEFILE cachefile, CACHEKEY key, uint32_t fullhash, void*value, PAIR_ATTR attr, - CACHETABLE_WRITE_CALLBACK write_callback + CACHETABLE_WRITE_CALLBACK write_callback, + CACHETABLE_PUT_CALLBACK put_callback ) { CACHETABLE ct = cachefile->cachetable; { - PAIR p; - for (p=ct->table[fullhash&(cachefile->cachetable->table_size-1)]; p; p=p->hash_chain) { - if (p->key.b==key.b && p->cachefile==cachefile) { - // Ideally, we would like to just assert(false) here - // and not return an error, but as of Dr. Noga, - // cachetable-test2 depends on this behavior. - // To replace the following with an assert(false) - // we need to change the behavior of cachetable-test2 - // - // Semantically, these two asserts are not strictly right. After all, when are two functions eq? - // In practice, the functions better be the same. - assert(p->flush_callback == write_callback.flush_callback); - assert(p->pe_callback == write_callback.pe_callback); - assert(p->cleaner_callback == write_callback.cleaner_callback); - return -1; /* Already present, don't grab lock. */ - } + PAIR p = ct->list.find_pair(cachefile, key, fullhash); + if (p != NULL) { + // Ideally, we would like to just assert(false) here + // and not return an error, but as of Dr. Noga, + // cachetable-test2 depends on this behavior. + // To replace the following with an assert(false) + // we need to change the behavior of cachetable-test2 + // + // Semantically, these two asserts are not strictly right. After all, when are two functions eq? + // In practice, the functions better be the same. + assert(p->flush_callback == write_callback.flush_callback); + assert(p->pe_callback == write_callback.pe_callback); + assert(p->cleaner_callback == write_callback.cleaner_callback); + return -1; /* Already present, don't grab lock. */ } } // flushing could change the table size, but wont' change the fullhash - increment_partitioned_counter(cachetable_puts, 1); + cachetable_puts++; PAIR p = cachetable_insert_at( ct, cachefile, @@ -1319,31 +858,22 @@ static int cachetable_put_internal( write_callback, CACHETABLE_DIRTY ); - assert(p); - nb_mutex_lock(&p->value_nb_mutex, &ct->mutex); + invariant_notnull(p); + pair_lock(p); + nb_mutex_lock(&p->value_nb_mutex, &p->mutex); + pair_unlock(p); //note_hash_count(count); + invariant_notnull(put_callback); + put_callback(value, p); return 0; } -// ct is locked on entry -// gets pair if exists, and that is all. -static int cachetable_get_pair (CACHEFILE cachefile, CACHEKEY key, uint32_t fullhash, PAIR* pv) { - CACHETABLE ct = cachefile->cachetable; - PAIR p; - int r = -1; - for (p=ct->table[fullhash&(ct->table_size-1)]; p; p=p->hash_chain) { - if (p->key.b==key.b && p->cachefile==cachefile) { - *pv = p; - r = 0; - break; - } - } - return r; -} - -// ct locked on entry +// Pair mutex (p->mutex) is may or may not be held on entry, +// Holding the pair mutex on entry is not important +// for performance or corrrectness +// Pair is pinned on entry static void -clone_pair(CACHETABLE ct, PAIR p) { +clone_pair(evictor* ev, PAIR p) { PAIR_ATTR old_attr = p->attr; PAIR_ATTR new_attr; @@ -1351,7 +881,6 @@ clone_pair(CACHETABLE ct, PAIR p) { // not sure if we have to release // and regrab the cachetable lock, // but doing it for now - cachetable_unlock(ct); p->clone_callback( p->value_data, &p->cloned_value_data, @@ -1359,7 +888,6 @@ clone_pair(CACHETABLE ct, PAIR p) { true, p->write_extraargs ); - cachetable_lock(ct); // now we need to do the same actions we would do // if the PAIR had been written to disk @@ -1368,34 +896,33 @@ clone_pair(CACHETABLE ct, PAIR p) { // it doesn't matter whether we clear // the pending bit before the clone // or after the clone - p->checkpoint_pending = false; p->dirty = CACHETABLE_CLEAN; if (new_attr.is_valid) { p->attr = new_attr; - cachetable_change_pair_attr(ct, old_attr, new_attr); + ev->change_pair_attr(old_attr, new_attr); } p->cloned_value_size = p->attr.size; - ct->size_current += p->cloned_value_size; + ev->add_to_size_current(p->cloned_value_size); } static void checkpoint_cloned_pair(void* extra) { PAIR p = (PAIR)extra; CACHETABLE ct = p->cachefile->cachetable; - cachetable_lock(ct); PAIR_ATTR new_attr; // note that pending lock is not needed here because // we KNOW we are in the middle of a checkpoint // and that a begin_checkpoint cannot happen cachetable_only_write_locked_data( - ct, + p->ev, p, true, //for_checkpoint &new_attr, true //is_clone ); + pair_lock(p); nb_mutex_unlock(&p->disk_nb_mutex); - bjm_remove_background_job(ct->checkpoint_clones_bjm); - cachetable_unlock(ct); + pair_unlock(p); + ct->cp.remove_background_job(); } static void @@ -1412,68 +939,57 @@ checkpoint_cloned_pair_on_writer_thread(CACHETABLE ct, PAIR p) { // - If the PAIR is not cloneable, write the PAIR to disk for checkpoint // on the current thread // +// On entry, pair's mutex is NOT held +// static void -write_locked_pair_for_checkpoint(CACHETABLE ct, PAIR p) +write_locked_pair_for_checkpoint(CACHETABLE ct, PAIR p, bool checkpoint_pending) { - if (p->dirty && p->checkpoint_pending) { + if (p->dirty && checkpoint_pending) { if (p->clone_callback) { - nb_mutex_lock(&p->disk_nb_mutex, &ct->mutex); + pair_lock(p); + nb_mutex_lock(&p->disk_nb_mutex, &p->mutex); + pair_unlock(p); assert(!p->cloned_value_data); - clone_pair(ct, p); + clone_pair(&ct->ev, p); assert(p->cloned_value_data); // place it on the background thread and continue // responsibility of writer thread to release disk_nb_mutex - int r = bjm_add_background_job(ct->checkpoint_clones_bjm); - assert_zero(r); + ct->cp.add_background_job(); checkpoint_cloned_pair_on_writer_thread(ct, p); - // possibly run eviction because act of cloning adds - // to ct->size_current, we don't do it in - // write_pair_for_checkpoint_thread, because that clones at most - // one node at any time, where as this may be called from many - // threads simultaneously - maybe_flush_some(ct, 0); } else { // The pair is not cloneable, just write the pair to disk - - // we already have p->value_nb_mutex and we just do the write in our own thread. - cachetable_write_locked_pair(ct, p); // keeps the PAIR's write lock + // we already have p->value_nb_mutex and we just do the write in our own thread. + cachetable_write_locked_pair(&ct->ev, p, true); // keeps the PAIR's write lock } } - else { - // - // we may clear the pending bit here because we have - // both the cachetable lock and the PAIR lock. - // The rule, as mentioned in toku_cachetable_begin_checkpoint, - // is that to clear the bit, we must have both the PAIR lock - // and the pending lock - // - p->checkpoint_pending = false; - } } -// On entry: hold the ct lock -// On exit: the node is written out +// On entry and exit: hold the pair's mutex (p->mutex) // Method: take write lock // maybe write out the node // Else release write lock +// static void -write_pair_for_checkpoint_thread (CACHETABLE ct, PAIR p) +write_pair_for_checkpoint_thread (evictor* ev, PAIR p) { - nb_mutex_lock(&p->value_nb_mutex, &ct->mutex); // grab an exclusive lock on the pair + nb_mutex_lock(&p->value_nb_mutex, &p->mutex); // grab an exclusive lock on the pair if (p->dirty && p->checkpoint_pending) { if (p->clone_callback) { - nb_mutex_lock(&p->disk_nb_mutex, &ct->mutex); + nb_mutex_lock(&p->disk_nb_mutex, &p->mutex); assert(!p->cloned_value_data); - clone_pair(ct, p); + clone_pair(ev, p); assert(p->cloned_value_data); } else { // The pair is not cloneable, just write the pair to disk // we already have p->value_nb_mutex and we just do the write in our own thread. // this will grab and release disk_nb_mutex - cachetable_write_locked_pair(ct, p); // keeps the PAIR's write lock + pair_unlock(p); + cachetable_write_locked_pair(ev, p, true); // keeps the PAIR's write lock + pair_lock(p); } + p->checkpoint_pending = false; // now release value_nb_mutex, before we write the PAIR out // so that the PAIR is available to client threads @@ -1483,13 +999,15 @@ write_pair_for_checkpoint_thread (CACHETABLE ct, PAIR p) // we KNOW we are in the middle of a checkpoint // and that a begin_checkpoint cannot happen PAIR_ATTR attr; + pair_unlock(p); cachetable_only_write_locked_data( - ct, + ev, p, true, //for_checkpoint &attr, true //is_clone ); + pair_lock(p); nb_mutex_unlock(&p->disk_nb_mutex); } } @@ -1516,37 +1034,48 @@ write_pair_for_checkpoint_thread (CACHETABLE ct, PAIR p) static void checkpoint_dependent_pairs( CACHETABLE ct, uint32_t num_dependent_pairs, // number of dependent pairs that we may need to checkpoint - CACHEFILE* dependent_cfs, // array of cachefiles of dependent pairs - CACHEKEY* dependent_keys, // array of cachekeys of dependent pairs - uint32_t* dependent_fullhash, //array of fullhashes of dependent pairs + PAIR* dependent_pairs, + bool* checkpoint_pending, enum cachetable_dirty* dependent_dirty // array stating dirty/cleanness of dependent pairs ) { for (uint32_t i =0; i < num_dependent_pairs; i++) { - PAIR curr_dep_pair = NULL; - int r = cachetable_get_pair( - dependent_cfs[i], - dependent_keys[i], - dependent_fullhash[i], - &curr_dep_pair - ); - // if we are passing in a dependent pair that we - // claim is locked, then it better be here - assert(r == 0); - assert(curr_dep_pair != NULL); - // pair had better be locked, as we are assuming - // to own the write lock - assert(nb_mutex_writers(&curr_dep_pair->value_nb_mutex)); + PAIR curr_dep_pair = dependent_pairs[i]; // we need to update the dirtyness of the dependent pair, // because the client may have dirtied it while holding its lock, // and if the pair is pending a checkpoint, it needs to be written out if (dependent_dirty[i]) curr_dep_pair->dirty = CACHETABLE_DIRTY; - if (curr_dep_pair->checkpoint_pending) { - write_locked_pair_for_checkpoint(ct, curr_dep_pair); + if (checkpoint_pending[i]) { + write_locked_pair_for_checkpoint(ct, curr_dep_pair, checkpoint_pending[i]); } } } +// +// must be holding a lock on the pair_list's list_lock on entry +// +static void get_pairs( + pair_list* pl, + uint32_t num_pairs, // number of dependent pairs that we may need to checkpoint + CACHEFILE* cfs, // array of cachefiles of dependent pairs + CACHEKEY* keys, // array of cachekeys of dependent pairs + uint32_t* fullhash, //array of fullhashes of dependent pairs + PAIR* out_pairs + ) +{ + for (uint32_t i =0; i < num_pairs; i++) { + out_pairs[i] = pl->find_pair( + cfs[i], + keys[i], + fullhash[i] + ); + assert(out_pairs[i] != NULL); + // pair had better be locked, as we are assuming + // to own the write lock + assert(nb_mutex_writers(&out_pairs[i]->value_nb_mutex)); + } +} + int toku_cachetable_put_with_dep_pairs( CACHEFILE cachefile, CACHETABLE_GET_KEY_AND_FULLHASH get_key_and_fullhash, @@ -1560,85 +1089,89 @@ int toku_cachetable_put_with_dep_pairs( uint32_t* dependent_fullhash, //array of fullhashes of dependent pairs enum cachetable_dirty* dependent_dirty, // array stating dirty/cleanness of dependent pairs CACHEKEY* key, - uint32_t* fullhash + uint32_t* fullhash, + CACHETABLE_PUT_CALLBACK put_callback ) { // // need to get the key and filehash // CACHETABLE ct = cachefile->cachetable; - cachetable_lock(ct); - // - // The reason we call cachetable_wait_write outside - // of cachetable_put_internal is that we want the operations - // get_key_and_fullhash and cachetable_put_internal - // to be atomic and NOT release the cachetable lock. - // If the cachetable lock is released within cachetable_put_internal, - // we may end up with a checkpoint beginning that has - // called get_key_and_fullhash (which causes a blocknum - // to be allocated) but without the PAIR being in the cachetable - // and checkpointed. The checkpoint would have a leaked blocknum. - // So, we call cachetable_wait_write outside, and ensure that - // cachetable_put_internal does not release the cachetable lock - // - cachetable_wait_write(ct); - // - // we call maybe_flush_some outside of cachetable_put_internal - // because maybe_flush_some may release the cachetable lock - // and we require what comes below to not do so. - // we require get_key_and_fullhash and cachetable_put_internal - // to not release the cachetable lock, and we require the critical - // region described below to not begin a checkpoint. The cachetable lock - // is used to ensure that a checkpoint is not begun during - // cachetable_put_internal - // - maybe_flush_some(ct, attr.size); + if (ct->ev.should_client_thread_sleep()) { + ct->ev.wait_for_cache_pressure_to_subside(); + } + if (ct->ev.should_client_wake_eviction_thread()) { + ct->ev.signal_eviction_thread(); + } int rval; { + ct->list.write_list_lock(); get_key_and_fullhash(key, fullhash, get_key_and_fullhash_extra); rval = cachetable_put_internal( - cachefile, - *key, - *fullhash, - value, - attr, - write_callback - ); + cachefile, + *key, + *fullhash, + value, + attr, + write_callback, + put_callback + ); + PAIR dependent_pairs[num_dependent_pairs]; + get_pairs( + &ct->list, + num_dependent_pairs, + dependent_cfs, + dependent_keys, + dependent_fullhash, + dependent_pairs + ); + bool checkpoint_pending[num_dependent_pairs]; + ct->list.write_pending_cheap_lock(); + for (uint32_t i = 0; i < num_dependent_pairs; i++) { + checkpoint_pending[i] = dependent_pairs[i]->checkpoint_pending; + dependent_pairs[i]->checkpoint_pending = false; + } + ct->list.write_pending_cheap_unlock(); + ct->list.write_list_unlock(); // // now that we have inserted the row, let's checkpoint the // dependent nodes, if they need checkpointing // checkpoint_dependent_pairs( - ct, - num_dependent_pairs, - dependent_cfs, - dependent_keys, - dependent_fullhash, - dependent_dirty - ); + ct, + num_dependent_pairs, + dependent_pairs, + checkpoint_pending, + dependent_dirty + ); } - cachetable_unlock(ct); return rval; } int toku_cachetable_put(CACHEFILE cachefile, CACHEKEY key, uint32_t fullhash, void*value, PAIR_ATTR attr, - CACHETABLE_WRITE_CALLBACK write_callback + CACHETABLE_WRITE_CALLBACK write_callback, + CACHETABLE_PUT_CALLBACK put_callback ) { CACHETABLE ct = cachefile->cachetable; - cachetable_lock(ct); - cachetable_wait_write(ct); - maybe_flush_some(ct, attr.size); + if (ct->ev.should_client_thread_sleep()) { + ct->ev.wait_for_cache_pressure_to_subside(); + } + if (ct->ev.should_client_wake_eviction_thread()) { + ct->ev.signal_eviction_thread(); + } + ct->list.write_list_lock(); int r = cachetable_put_internal( cachefile, key, fullhash, value, attr, - write_callback + write_callback, + put_callback ); - cachetable_unlock(ct); + ct->list.write_list_unlock(); return r; } @@ -1653,6 +1186,8 @@ static uint64_t get_tnow(void) { // On exit, cachetable lock is still held, but PAIR lock // is either released. // +// No locks are held on entry (besides the nb_mutex of the PAIR) +// static void do_partial_fetch( CACHETABLE ct, @@ -1669,17 +1204,19 @@ do_partial_fetch( // so we do a sanity check here. assert(!p->dirty); - nb_mutex_lock(&p->disk_nb_mutex, &ct->mutex); - cachetable_unlock(ct); + pair_lock(p); + nb_mutex_lock(&p->disk_nb_mutex, &p->mutex); + pair_unlock(p); int r = pf_callback(p->value_data, p->disk_data, read_extraargs, cachefile->fd, &new_attr); lazy_assert_zero(r); - cachetable_lock(ct); p->attr = new_attr; - cachetable_change_pair_attr(ct, old_attr, new_attr); + ct->ev.change_pair_attr(old_attr, new_attr); + pair_lock(p); nb_mutex_unlock(&p->disk_nb_mutex); if (!keep_pair_locked) { nb_mutex_unlock(&p->value_nb_mutex); } + pair_unlock(p); } void toku_cachetable_pf_pinned_pair( @@ -1694,18 +1231,23 @@ void toku_cachetable_pf_pinned_pair( PAIR_ATTR attr; PAIR p = NULL; CACHETABLE ct = cf->cachetable; - cachetable_lock(ct); - int r = cachetable_get_pair(cf, key, fullhash, &p); - assert_zero(r); + ct->list.read_list_lock(); + p = ct->list.find_pair(cf, key, fullhash); + assert(p != NULL); assert(p->value_data == value); assert(nb_mutex_writers(&p->value_nb_mutex)); - nb_mutex_lock(&p->disk_nb_mutex, &ct->mutex); + ct->list.read_list_unlock(); + + pair_lock(p); + nb_mutex_lock(&p->disk_nb_mutex, &p->mutex); + pair_unlock(p); + int fd = cf->fd; - cachetable_unlock(ct); pf_callback(value, p->disk_data, read_extraargs, fd, &attr); - cachetable_lock(ct); + + pair_lock(p); nb_mutex_unlock(&p->disk_nb_mutex); - cachetable_unlock(ct); + pair_unlock(p); } @@ -1750,6 +1292,7 @@ int toku_cachetable_get_and_pin ( } // Read a pair from a cachefile into memory using the pair's fetch callback +// on entry, pair mutex (p->mutex) is NOT held, but pair is pinned static void cachetable_fetch_pair( CACHETABLE ct, CACHEFILE cf, @@ -1770,41 +1313,48 @@ static void cachetable_fetch_pair( // FIXME this should be enum cachetable_dirty, right? int dirty = 0; - nb_mutex_lock(&p->disk_nb_mutex, &ct->mutex); - cachetable_unlock(ct); + pair_lock(p); + nb_mutex_lock(&p->disk_nb_mutex, &p->mutex); + pair_unlock(p); int r; - r = fetch_callback(cf, cf->fd, key, fullhash, &toku_value, &disk_data, &attr, &dirty, read_extraargs); - if (dirty) + r = fetch_callback(cf, p, cf->fd, key, fullhash, &toku_value, &disk_data, &attr, &dirty, read_extraargs); + if (dirty) { p->dirty = CACHETABLE_DIRTY; - - cachetable_lock(ct); - // ft-ops.c asserts that get_and_pin succeeds, - // so we might as well just assert it here as opposed - // to trying to support an INVALID state + } assert(r == 0); p->value_data = toku_value; p->disk_data = disk_data; p->attr = attr; - cachetable_add_pair_attr(ct, attr); + ct->ev.add_pair_attr(attr); + pair_lock(p); nb_mutex_unlock(&p->disk_nb_mutex); if (!keep_pair_locked) { nb_mutex_unlock(&p->value_nb_mutex); } - if (0) printf("%s:%d %" PRId64 " complete\n", __FUNCTION__, __LINE__, key.b); + pair_unlock(p); } -static bool resolve_checkpointing_fast(PAIR p) { - return !(p->checkpoint_pending && (p->dirty == CACHETABLE_DIRTY) && !p->clone_callback); +static bool get_checkpoint_pending(PAIR p, pair_list* pl) { + bool checkpoint_pending = false; + pl->read_pending_cheap_lock(); + checkpoint_pending = p->checkpoint_pending; + p->checkpoint_pending = false; + pl->read_pending_cheap_unlock(); + return checkpoint_pending; +} + +static bool resolve_checkpointing_fast(PAIR p, bool checkpoint_pending) { + return !(checkpoint_pending && (p->dirty == CACHETABLE_DIRTY) && !p->clone_callback); } static void checkpoint_pair_and_dependent_pairs( CACHETABLE ct, PAIR p, + bool p_is_pending_checkpoint, uint32_t num_dependent_pairs, // number of dependent pairs that we may need to checkpoint - CACHEFILE* dependent_cfs, // array of cachefiles of dependent pairs - CACHEKEY* dependent_keys, // array of cachekeys of dependent pairs - uint32_t* dependent_fullhash, //array of fullhashes of dependent pairs + PAIR* dependent_pairs, + bool* dependent_pairs_pending_checkpoint, enum cachetable_dirty* dependent_dirty // array stating dirty/cleanness of dependent pairs ) { @@ -1831,28 +1381,118 @@ static void checkpoint_pair_and_dependent_pairs( // for checkpoint, even though writing a pair releases // the cachetable lock. // - if (p->checkpoint_pending) { - write_locked_pair_for_checkpoint(ct, p); - } + write_locked_pair_for_checkpoint(ct, p, p_is_pending_checkpoint); checkpoint_dependent_pairs( ct, num_dependent_pairs, - dependent_cfs, - dependent_keys, - dependent_fullhash, + dependent_pairs, + dependent_pairs_pending_checkpoint, dependent_dirty ); } -int toku_cachetable_get_and_pin_with_dep_pairs ( - CACHEFILE cachefile, - CACHEKEY key, - uint32_t fullhash, - void**value, + +// on input, the pair's mutex is held, +// on output, the pair's mutex is not held. +// if true, we must try again, and pair is not pinned +// if false, we succeeded, the pair is pinned +// NOTE: On entry and exit, the read list lock is held. +static bool try_pin_pair( + PAIR p, + CACHETABLE ct, + CACHEFILE cachefile, + bool may_modify_value, + uint32_t num_dependent_pairs, + PAIR* dependent_pairs, + enum cachetable_dirty* dependent_dirty, + CACHETABLE_PARTIAL_FETCH_REQUIRED_CALLBACK pf_req_callback, + CACHETABLE_PARTIAL_FETCH_CALLBACK pf_callback, + void* read_extraargs + ) +{ + bool dep_checkpoint_pending[num_dependent_pairs]; + bool try_again = true; + bool reacquire_lock = false; + if (nb_mutex_writers(&p->value_nb_mutex)) { + reacquire_lock = true; + ct->list.read_list_unlock(); + } + nb_mutex_lock(&p->value_nb_mutex, &p->mutex); + pair_touch(p); + pair_unlock(p); + if (reacquire_lock) { + ct->list.read_list_lock(); + } + + if (may_modify_value) { + ct->list.read_pending_cheap_lock(); + bool p_checkpoint_pending = p->checkpoint_pending; + p->checkpoint_pending = false; + for (uint32_t i = 0; i < num_dependent_pairs; i++) { + dep_checkpoint_pending[i] = dependent_pairs[i]->checkpoint_pending; + dependent_pairs[i]->checkpoint_pending = false; + } + ct->list.read_pending_cheap_unlock(); + checkpoint_pair_and_dependent_pairs( + ct, + p, + p_checkpoint_pending, + num_dependent_pairs, + dependent_pairs, + dep_checkpoint_pending, + dependent_dirty + ); + } + + bool partial_fetch_required = pf_req_callback(p->value_data,read_extraargs); + // shortcutting a path to getting the user the data + // helps scalability for in-memory workloads + if (!partial_fetch_required) { + try_again = false; + goto exit; + } + if (ct->ev.should_client_thread_sleep()) { + pair_lock(p); + nb_mutex_unlock(&p->value_nb_mutex); + pair_unlock(p); + try_again = true; + goto exit; + } + if (ct->ev.should_client_wake_eviction_thread()) { + ct->ev.signal_eviction_thread(); + } + // + // Just because the PAIR exists does necessarily mean the all the data the caller requires + // is in memory. A partial fetch may be required, which is evaluated above + // if the variable is true, a partial fetch is required so we must grab the PAIR's write lock + // and then call a callback to retrieve what we need + // + if (partial_fetch_required) { + // As of Dr. No, only clean PAIRs may have pieces missing, + // so we do a sanity check here. + assert(!p->dirty); + + // This may be slow, better release and re-grab the + // read list lock. + ct->list.read_list_unlock(); + do_partial_fetch(ct, cachefile, p, pf_callback, read_extraargs, true); + ct->list.read_list_lock(); + } + + try_again = false; +exit: + return try_again; +} + +int toku_cachetable_get_and_pin_with_dep_pairs_batched ( + CACHEFILE cachefile, + CACHEKEY key, + uint32_t fullhash, + void**value, long *sizep, CACHETABLE_WRITE_CALLBACK write_callback, - CACHETABLE_FETCH_CALLBACK fetch_callback, + CACHETABLE_FETCH_CALLBACK fetch_callback, CACHETABLE_PARTIAL_FETCH_REQUIRED_CALLBACK pf_req_callback, CACHETABLE_PARTIAL_FETCH_CALLBACK pf_callback, bool may_modify_value, @@ -1862,80 +1502,149 @@ int toku_cachetable_get_and_pin_with_dep_pairs ( CACHEKEY* dependent_keys, // array of cachekeys of dependent pairs uint32_t* dependent_fullhash, //array of fullhashes of dependent pairs enum cachetable_dirty* dependent_dirty // array stating dirty/cleanness of dependent pairs - ) + ) +// See cachetable.h { CACHETABLE ct = cachefile->cachetable; - PAIR p; - cachetable_lock(ct); - - cachetable_wait_write(ct); - for (p=ct->table[fullhash&(ct->table_size-1)]; p; p=p->hash_chain) { - if (p->key.b==key.b && p->cachefile==cachefile) { - // still have the cachetable lock - nb_mutex_lock(&p->value_nb_mutex, &ct->mutex); - pair_touch(p); - if (may_modify_value) { - checkpoint_pair_and_dependent_pairs( - ct, - p, - num_dependent_pairs, - dependent_cfs, - dependent_keys, - dependent_fullhash, - dependent_dirty - ); - } - cachetable_unlock(ct); - - bool partial_fetch_required = pf_req_callback(p->value_data,read_extraargs); - // shortcutting a path to getting the user the data - // helps scalability for in-memory workloads - if (!partial_fetch_required) { - *value = p->value_data; - if (sizep) *sizep = p->attr.size; - return 0; - } - cachetable_lock(ct); - // - // Just because the PAIR exists does necessarily mean the all the data the caller requires - // is in memory. A partial fetch may be required, which is evaluated above - // if the variable is true, a partial fetch is required so we must grab the PAIR's write lock - // and then call a callback to retrieve what we need - // - if (partial_fetch_required) { - // As of Dr. No, only clean PAIRs may have pieces missing, - // so we do a sanity check here. - assert(!p->dirty); + bool wait = false; + PAIR dependent_pairs[num_dependent_pairs]; + bool dep_checkpoint_pending[num_dependent_pairs]; - do_partial_fetch(ct, cachefile, p, pf_callback, read_extraargs, true); - } + // + // If in the process of pinning the node we add data to the cachetable via a partial fetch + // or a full fetch, we may need to first sleep because there is too much data in the + // cachetable. In those cases, we set the bool wait to true and goto try_again, so that + // we can do our sleep and then restart the function. + // +beginning: + if (wait) { + // We shouldn't be holding the read list lock while + // waiting for the evictor to remove pairs. + ct->list.read_list_unlock(); + ct->ev.wait_for_cache_pressure_to_subside(); + ct->list.read_list_lock(); + } + + get_pairs( + &ct->list, + num_dependent_pairs, + dependent_cfs, + dependent_keys, + dependent_fullhash, + dependent_pairs + ); + + PAIR p = ct->list.find_pair(cachefile, key, fullhash); + if (p) { + pair_lock(p); + // on entry, holds p->mutex, + // on exit, does not hold p->mutex + bool try_again = try_pin_pair( + p, + ct, + cachefile, + may_modify_value, + num_dependent_pairs, + dependent_pairs, + dependent_dirty, + pf_req_callback, + pf_callback, + read_extraargs + ); + if (try_again) { + wait = true; + goto beginning; + } + else { goto got_value; } } - // Note. hashit(t,key) may have changed as a result of flushing. But fullhash won't have changed. - // The pair was not found, we must retrieve it from disk - { - // insert a PAIR into the cachetable + else { + if (ct->ev.should_client_thread_sleep()) { + wait = true; + goto beginning; + } + if (ct->ev.should_client_wake_eviction_thread()) { + ct->ev.signal_eviction_thread(); + } + // Since we missed the pair, we need the write list + // lock. So, we have to release the read list lock + // first. + ct->list.read_list_unlock(); + ct->list.write_list_lock(); + p = ct->list.find_pair(cachefile, key, fullhash); + if (p != NULL) { + pair_lock(p); + ct->list.write_list_unlock(); + + // Don't forget to grab the read list lock + // BEFORE calling try_pin_pair. + ct->list.read_list_lock(); + + // on entry, holds p->mutex, + // on exit, does not hold p->mutex + bool try_again = try_pin_pair( + p, + ct, + cachefile, + may_modify_value, + num_dependent_pairs, + dependent_pairs, + dependent_dirty, + pf_req_callback, + pf_callback, + read_extraargs + ); + if (try_again) { + wait = true; + goto beginning; + } + else { + goto got_value; + } + } + assert(p == NULL); + + // Insert a PAIR into the cachetable + // NOTE: At this point we still have the write list lock held. p = cachetable_insert_at( - ct, - cachefile, - key, - zero_value, - fullhash, - zero_attr, + ct, + cachefile, + key, + zero_value, + fullhash, + zero_attr, write_callback, CACHETABLE_CLEAN ); assert(p); - nb_mutex_lock(&p->value_nb_mutex, &ct->mutex); + + // Pin the pair. + pair_lock(p); + nb_mutex_lock(&p->value_nb_mutex, &p->mutex); + pair_unlock(p); + if (may_modify_value) { - checkpoint_pair_and_dependent_pairs( + ct->list.read_pending_cheap_lock(); + assert(!p->checkpoint_pending); + for (uint32_t i = 0; i < num_dependent_pairs; i++) { + dep_checkpoint_pending[i] = dependent_pairs[i]->checkpoint_pending; + dependent_pairs[i]->checkpoint_pending = false; + } + ct->list.read_pending_cheap_unlock(); + } + + // We should release the lock before we perform + // these expensive operations. + // TODO: Determine if we can move this above the may_modify_value block, but after the pin. + ct->list.write_list_unlock(); + + if (may_modify_value) { + checkpoint_dependent_pairs( ct, - p, num_dependent_pairs, - dependent_cfs, - dependent_keys, - dependent_fullhash, + dependent_pairs, + dep_checkpoint_pending, dependent_dirty ); } @@ -1945,18 +1654,65 @@ int toku_cachetable_get_and_pin_with_dep_pairs ( // The pair being fetched will be marked as pending if a checkpoint happens during the // fetch because begin_checkpoint will mark as pending any pair that is locked even if it is clean. cachetable_fetch_pair(ct, cachefile, p, fetch_callback, read_extraargs, true); - increment_partitioned_counter(cachetable_miss, 1); - increment_partitioned_counter(cachetable_misstime, get_tnow() - t0); + cachetable_miss++; + cachetable_misstime += get_tnow() - t0; + + // We need to be holding the read list lock when we exit. + // We grab it here because we released it earlier to + // grab the write list lock because the checkpointing and + // fetching are expensive/slow. + ct->list.read_list_lock(); goto got_value; } got_value: *value = p->value_data; if (sizep) *sizep = p->attr.size; - maybe_flush_some(ct, 0); - cachetable_unlock(ct); return 0; } +int toku_cachetable_get_and_pin_with_dep_pairs ( + CACHEFILE cachefile, + CACHEKEY key, + uint32_t fullhash, + void**value, + long *sizep, + CACHETABLE_WRITE_CALLBACK write_callback, + CACHETABLE_FETCH_CALLBACK fetch_callback, + CACHETABLE_PARTIAL_FETCH_REQUIRED_CALLBACK pf_req_callback, + CACHETABLE_PARTIAL_FETCH_CALLBACK pf_callback, + bool may_modify_value, + void* read_extraargs, // parameter for fetch_callback, pf_req_callback, and pf_callback + uint32_t num_dependent_pairs, // number of dependent pairs that we may need to checkpoint + CACHEFILE* dependent_cfs, // array of cachefiles of dependent pairs + CACHEKEY* dependent_keys, // array of cachekeys of dependent pairs + uint32_t* dependent_fullhash, //array of fullhashes of dependent pairs + enum cachetable_dirty* dependent_dirty // array stating dirty/cleanness of dependent pairs + ) +// See cachetable.h +{ + toku_cachetable_begin_batched_pin(cachefile); + int r = toku_cachetable_get_and_pin_with_dep_pairs_batched( + cachefile, + key, + fullhash, + value, + sizep, + write_callback, + fetch_callback, + pf_req_callback, + pf_callback, + may_modify_value, + read_extraargs, + num_dependent_pairs, + dependent_cfs, + dependent_keys, + dependent_fullhash, + dependent_dirty + ); + toku_cachetable_end_batched_pin(cachefile); + return r; +} + // Lookup a key in the cachetable. If it is found and it is not being written, then // acquire a read lock on the pair, update the LRU list, and return sucess. // @@ -1969,25 +1725,40 @@ got_value: // Similarly, if the checkpoint is actually pending, we don't want to block on it. int toku_cachetable_maybe_get_and_pin (CACHEFILE cachefile, CACHEKEY key, uint32_t fullhash, void**value) { CACHETABLE ct = cachefile->cachetable; - PAIR p; int r = -1; - cachetable_lock(ct); - for (p=ct->table[fullhash&(ct->table_size-1)]; p; p=p->hash_chain) { - if (p->key.b==key.b && p->cachefile==cachefile) { - if (!p->checkpoint_pending && //If checkpoint pending, we would need to first write it, which would make it clean - p->dirty && - nb_mutex_users(&p->value_nb_mutex) == 0 - ) { - // because nb_mutex_users is 0, this is fast - nb_mutex_lock(&p->value_nb_mutex, &ct->mutex); + ct->list.read_list_lock(); + PAIR p = ct->list.find_pair(cachefile, key, fullhash); + if (p) { + pair_lock(p); + ct->list.read_list_unlock(); + if (p->dirty && + nb_mutex_users(&p->value_nb_mutex) == 0 + ) + { + // because nb_mutex_users is 0, this is fast + nb_mutex_lock(&p->value_nb_mutex, &p->mutex); + ct->list.read_pending_cheap_lock(); + // + // if pending a checkpoint, then we don't want to return + // the value to the user, because we are responsible for + // handling the checkpointing, which we do not want to do, + // because it is expensive + // + if (p->checkpoint_pending) { + nb_mutex_unlock(&p->value_nb_mutex); + r = -1; + } + else { *value = p->value_data; - pair_touch(p); r = 0; } - break; + ct->list.read_pending_cheap_unlock(); } + pair_unlock(p); + } + else { + ct->list.read_list_unlock(); } - cachetable_unlock(ct); return r; } @@ -1996,64 +1767,104 @@ int toku_cachetable_maybe_get_and_pin (CACHEFILE cachefile, CACHEKEY key, uint32 //All other conditions remain the same. int toku_cachetable_maybe_get_and_pin_clean (CACHEFILE cachefile, CACHEKEY key, uint32_t fullhash, void**value) { CACHETABLE ct = cachefile->cachetable; - PAIR p; int r = -1; - cachetable_lock(ct); - for (p=ct->table[fullhash&(ct->table_size-1)]; p; p=p->hash_chain) { - if (p->key.b==key.b && p->cachefile==cachefile) { - if (!p->checkpoint_pending && //If checkpoint pending, we would need to first write it, which would make it clean (if the pin would be used for writes. If would be used for read-only we could return it, but that would increase complexity) - nb_mutex_users(&p->value_nb_mutex) == 0 - ) { - // because nb_mutex_users is 0, this is fast - nb_mutex_lock(&p->value_nb_mutex, &ct->mutex); + ct->list.read_list_lock(); + PAIR p = ct->list.find_pair(cachefile, key, fullhash); + if (p) { + pair_lock(p); + ct->list.read_list_unlock(); + if (nb_mutex_users(&p->value_nb_mutex) == 0) { + // because nb_mutex_users is 0, this is fast + nb_mutex_lock(&p->value_nb_mutex, &p->mutex); + ct->list.read_pending_cheap_lock(); + // + // if pending a checkpoint, then we don't want to return + // the value to the user, because we are responsible for + // handling the checkpointing, which we do not want to do, + // because it is expensive + // + if (p->checkpoint_pending) { + nb_mutex_unlock(&p->value_nb_mutex); + r = -1; + } + else { *value = p->value_data; r = 0; } - break; + ct->list.read_pending_cheap_unlock(); } + pair_unlock(p); + } + else { + ct->list.read_list_unlock(); } - cachetable_unlock(ct); return r; } +// +// internal function to unpin a PAIR. +// As of Clayface, this is may be called in two ways: +// - with have_ct_lock true and flush false +// - with have_ct_lock false and flush true +// The first is for when this is run during run_unlockers in +// toku_cachetable_get_and_pin_nonblocking, the second is during +// normal operations. Only during normal operations do we want to possibly +// induce evictions. +// static int -cachetable_unpin_internal(CACHEFILE cachefile, CACHEKEY key, uint32_t fullhash, enum cachetable_dirty dirty, PAIR_ATTR attr, bool have_ct_lock, bool flush) -// size==0 means that the size didn't change. +cachetable_unpin_internal( + CACHEFILE cachefile, + PAIR p, + enum cachetable_dirty dirty, + PAIR_ATTR attr, + bool have_ct_lock, + bool flush + ) { + invariant_notnull(p); + CACHETABLE ct = cachefile->cachetable; - //printf("%s:%d is dirty now=%d\n", __FILE__, __LINE__, dirty); - int r = -1; - //assert(fullhash == toku_cachetable_hash(cachefile, key)); - if (!have_ct_lock) cachetable_lock(ct); - for (PAIR p=ct->table[fullhash&(ct->table_size-1)]; p; p=p->hash_chain) { - if (p->key.b==key.b && p->cachefile==cachefile) { - assert(nb_mutex_writers(&p->value_nb_mutex)>0); - nb_mutex_unlock(&p->value_nb_mutex); - if (dirty) p->dirty = CACHETABLE_DIRTY; - if (attr.is_valid) { - PAIR_ATTR old_attr = p->attr; - PAIR_ATTR new_attr = attr; - cachetable_change_pair_attr(ct, old_attr, new_attr); - p->attr = attr; - } - if (flush) { - maybe_flush_some(ct, 0); - } - r = 0; // we found one - break; + bool added_data_to_cachetable = false; + + PAIR_ATTR old_attr = p->attr; + PAIR_ATTR new_attr = attr; + pair_lock(p); + assert(nb_mutex_writers(&p->value_nb_mutex)>0); + if (dirty) { + p->dirty = CACHETABLE_DIRTY; + } + if (attr.is_valid) { + p->attr = attr; + } + nb_mutex_unlock(&p->value_nb_mutex); + pair_unlock(p); + + if (attr.is_valid) { + if (new_attr.size > old_attr.size) { + added_data_to_cachetable = true; + } + ct->ev.change_pair_attr(old_attr, new_attr); + } + + // see comments above this function to understand this code + if (flush && added_data_to_cachetable && !have_ct_lock) { + if (ct->ev.should_client_thread_sleep()) { + ct->ev.wait_for_cache_pressure_to_subside(); + } + if (ct->ev.should_client_wake_eviction_thread()) { + ct->ev.signal_eviction_thread(); } } - if (!have_ct_lock) cachetable_unlock(ct); - return r; + return 0; } -int toku_cachetable_unpin(CACHEFILE cachefile, CACHEKEY key, uint32_t fullhash, enum cachetable_dirty dirty, PAIR_ATTR attr) { +int toku_cachetable_unpin(CACHEFILE cachefile, PAIR p, enum cachetable_dirty dirty, PAIR_ATTR attr) { // By default we don't have the lock - return cachetable_unpin_internal(cachefile, key, fullhash, dirty, attr, false, true); + return cachetable_unpin_internal(cachefile, p, dirty, attr, false, true); } -int toku_cachetable_unpin_ct_prelocked_no_flush(CACHEFILE cachefile, CACHEKEY key, uint32_t fullhash, enum cachetable_dirty dirty, PAIR_ATTR attr) { +int toku_cachetable_unpin_ct_prelocked_no_flush(CACHEFILE cachefile, PAIR p, enum cachetable_dirty dirty, PAIR_ATTR attr) { // We hold the cachetable mutex. - return cachetable_unpin_internal(cachefile, key, fullhash, dirty, attr, true, false); + return cachetable_unpin_internal(cachefile, p, dirty, attr, true, false); } static void @@ -2066,115 +1877,245 @@ run_unlockers (UNLOCKERS unlockers) { } } -int toku_cachetable_get_and_pin_nonblocking ( - CACHEFILE cf, - CACHEKEY key, - uint32_t fullhash, - void**value, +// on entry, pair mutex is held +// on exit, is not held +// on entry and exit, the list read lock is held +static void +pin_and_release_pair( + PAIR p, + CACHETABLE ct, + bool may_modify_value, + UNLOCKERS unlockers + ) +{ + run_unlockers(unlockers); // The contract says the unlockers are run with the read or write list lock being held. + + ct->list.read_list_unlock(); + + // Now wait for the I/O to occur. + nb_mutex_lock(&p->value_nb_mutex, &p->mutex); + if (may_modify_value) { + bool checkpoint_pending = get_checkpoint_pending(p, &ct->list); + pair_unlock(p); + + // We hold the read list lock throughout this call. + // This is O.K. because in production, this function + // should always put the write on a background thread. + write_locked_pair_for_checkpoint(ct, p, checkpoint_pending); + pair_lock(p); + } + nb_mutex_unlock(&p->value_nb_mutex); + pair_unlock(p); + + ct->list.read_list_lock(); +} + +void toku_cachetable_begin_batched_pin(CACHEFILE cf) +// See cachetable.h. +{ + cf->cachetable->list.read_list_lock(); +} + +void toku_cachetable_end_batched_pin(CACHEFILE cf) +// See cachetable.h. +{ + cf->cachetable->list.read_list_unlock(); +} + +int toku_cachetable_get_and_pin_nonblocking_batched( + CACHEFILE cf, + CACHEKEY key, + uint32_t fullhash, + void**value, long* UU(sizep), CACHETABLE_WRITE_CALLBACK write_callback, - CACHETABLE_FETCH_CALLBACK fetch_callback, + CACHETABLE_FETCH_CALLBACK fetch_callback, CACHETABLE_PARTIAL_FETCH_REQUIRED_CALLBACK pf_req_callback, CACHETABLE_PARTIAL_FETCH_CALLBACK pf_callback, bool may_modify_value, void *read_extraargs, UNLOCKERS unlockers ) -// Effect: If the block is in the cachetable, then pin it and return it. -// Otherwise fetch the data (but don't pin it, since we'll just end up pinning it again later), and return TOKUDB_TRY_AGAIN. +// See cachetable.h. { CACHETABLE ct = cf->cachetable; - cachetable_lock(ct); - // - // Even though there is a risk that cachetable_wait_write may wait on a bunch - // of I/O to complete, we call this because if we - // are in this situation where a lot of data is being evicted on writer threads - // then we are in a screw case anyway. - // - cachetable_wait_write(ct); - PAIR p; - for (p = ct->table[fullhash&(ct->table_size-1)]; p; p = p->hash_chain) { - if (p->key.b==key.b && p->cachefile==cf) { - // - // In Doofenshmirtz, we keep the root to leaf path pinned - // as we perform a query on a dictionary at any given time. - // This implies that only ONE query client can ever be - // in get_and_pin_nonblocking for this dictionary. - // So, if there is a write lock grabbed - // on the PAIR that we want to lock, then some expensive operation - // MUST be happening (read from disk, write to disk, flush, etc...), - // and we should run the unlockers. - // Otherwise, if there is no write lock grabbed, we know there will - // be no stall, so we grab the lock and return to the user - // - if (!nb_mutex_writers(&p->value_nb_mutex) && - (!may_modify_value || resolve_checkpointing_fast(p))) - { - nb_mutex_lock(&p->value_nb_mutex, &ct->mutex); - if (may_modify_value && p->checkpoint_pending) { - write_locked_pair_for_checkpoint(ct, p); - } - pair_touch(p); - // release the cachetable lock before calling pf_req_callback - // helps scalability for in-memory workloads by not holding the cachetable lock - // when calling pf_req_callback, and if possible, returns the PAIR to the user without - // reacquiring the cachetable lock - cachetable_unlock(ct); - bool partial_fetch_required = pf_req_callback(p->value_data,read_extraargs); - // - // Just because the PAIR exists does necessarily mean the all the data the caller requires - // is in memory. A partial fetch may be required, which is evaluated above - // if the variable is true, a partial fetch is required so we must grab the PAIR's write lock - // and then call a callback to retrieve what we need - // - if (partial_fetch_required) { - cachetable_lock(ct); - run_unlockers(unlockers); // The contract says the unlockers are run with the ct lock being held. - // Now wait for the I/O to occur. - do_partial_fetch(ct, cf, p, pf_callback, read_extraargs, false); - cachetable_unlock(ct); - return TOKUDB_TRY_AGAIN; - } - else { - *value = p->value_data; - return 0; - } + +try_again: + + PAIR p = ct->list.find_pair(cf, key, fullhash); + if (p == NULL) { + // Not found + ct->list.read_list_unlock(); + ct->list.write_list_lock(); + p = ct->list.find_pair(cf, key, fullhash); + if (p != NULL) { + // we just did another search with the write list lock and + // found the pair this means that in between our + // releasing the read list lock and grabbing the write list lock, + // another thread snuck in and inserted the PAIR into + // the cachetable. For simplicity, we just return + // to the top and restart the function + ct->list.write_list_unlock(); + ct->list.read_list_lock(); + goto try_again; + } + + p = cachetable_insert_at( + ct, + cf, + key, + zero_value, + fullhash, + zero_attr, + write_callback, + CACHETABLE_CLEAN + ); + assert(p); + pair_lock(p); + nb_mutex_lock(&p->value_nb_mutex, &p->mutex); + pair_unlock(p); + run_unlockers(unlockers); // we hold the write list_lock. + ct->list.write_list_unlock(); + + // at this point, only the pair is pinned, + // and no pair mutex held + uint64_t t0 = get_tnow(); + cachetable_fetch_pair(ct, cf, p, fetch_callback, read_extraargs, false); + cachetable_miss++; + cachetable_misstime += get_tnow() - t0; + + if (ct->ev.should_client_thread_sleep()) { + ct->ev.wait_for_cache_pressure_to_subside(); + } + if (ct->ev.should_client_wake_eviction_thread()) { + ct->ev.signal_eviction_thread(); + } + + // We need to be holding the read list lock on exit, + // and we don't want to hold during our wait for + // cache pressure to subside. + ct->list.read_list_lock(); + return TOKUDB_TRY_AGAIN; + } + else { + // + // In Doofenshmirtz, we keep the root to leaf path pinned + // as we perform a query on a dictionary at any given time. + // This implies that only ONE query client can ever be + // in get_and_pin_nonblocking for this dictionary. + // So, if there is a write lock grabbed + // on the PAIR that we want to lock, then some expensive operation + // MUST be happening (read from disk, write to disk, flush, etc...), + // and we should run the unlockers. + // Otherwise, if there is no write lock grabbed, we know there will + // be no stall, so we grab the lock and return to the user + // + pair_lock(p); + if (nb_mutex_writers(&p->value_nb_mutex)) { + // The pair's mutex is released in this function call: + pin_and_release_pair(p, ct, may_modify_value, unlockers); + return TOKUDB_TRY_AGAIN; + } + nb_mutex_lock(&p->value_nb_mutex, &p->mutex); + pair_unlock(p); + if (may_modify_value) { + bool checkpoint_pending = get_checkpoint_pending(p, &ct->list); + bool is_checkpointing_fast = resolve_checkpointing_fast( + p, + checkpoint_pending + ); + + if (!is_checkpointing_fast) { + run_unlockers(unlockers); } - else { - run_unlockers(unlockers); // The contract says the unlockers are run with the ct lock being held. - // Now wait for the I/O to occur. - nb_mutex_lock(&p->value_nb_mutex, &ct->mutex); - if (may_modify_value && p->checkpoint_pending) { - write_locked_pair_for_checkpoint(ct, p); - } + + // We hold the read list lock throughout this call. + // This is O.K. because in production, this function + // should always put the write on a background thread. + write_locked_pair_for_checkpoint(ct, p, checkpoint_pending); + if (!is_checkpointing_fast) { + pair_lock(p); nb_mutex_unlock(&p->value_nb_mutex); - cachetable_unlock(ct); + pair_unlock(p); + return TOKUDB_TRY_AGAIN; } } - } - assert(p==0); - // Not found - p = cachetable_insert_at( - ct, - cf, - key, - zero_value, - fullhash, - zero_attr, + // At this point, we have pinned the PAIR + // and resolved its checkpointing. The list lock is not held + // and the pair's mutex is not held. Before + // returning the PAIR to the user, we must + // still check for partial fetch + bool partial_fetch_required = pf_req_callback(p->value_data,read_extraargs); + if (partial_fetch_required) { + // TODO(leif): the following comment is probably wrong now + // that we can unpin without the read list lock. + run_unlockers(unlockers); // The contract says the unlockers are run with the ct lock being held. + + // Since we have to do disk I/O we should temporarily + // release the read list lock. + ct->list.read_list_unlock(); + + // Now wait for the I/O to occur. + do_partial_fetch(ct, cf, p, pf_callback, read_extraargs, false); + + if (ct->ev.should_client_thread_sleep()) { + ct->ev.wait_for_cache_pressure_to_subside(); + } + if (ct->ev.should_client_wake_eviction_thread()) { + ct->ev.signal_eviction_thread(); + } + + // We need to be holding the read list lock on exit, + // and we don't want to hold during neither our wait for + // cache pressure to subside, nor our partial fetch. + ct->list.read_list_lock(); + return TOKUDB_TRY_AGAIN; + } + else { + *value = p->value_data; + return 0; + } + } + // We should not get here. Above code should hit a return in all cases. + abort(); +} + +int toku_cachetable_get_and_pin_nonblocking ( + CACHEFILE cf, + CACHEKEY key, + uint32_t fullhash, + void**value, + long* UU(sizep), + CACHETABLE_WRITE_CALLBACK write_callback, + CACHETABLE_FETCH_CALLBACK fetch_callback, + CACHETABLE_PARTIAL_FETCH_REQUIRED_CALLBACK pf_req_callback, + CACHETABLE_PARTIAL_FETCH_CALLBACK pf_callback, + bool may_modify_value, + void *read_extraargs, + UNLOCKERS unlockers + ) +// See cachetable.h. +{ + int r = 0; + toku_cachetable_begin_batched_pin(cf); + r = toku_cachetable_get_and_pin_nonblocking_batched( + cf, + key, + fullhash, + value, + sizep, write_callback, - CACHETABLE_CLEAN - ); - assert(p); - nb_mutex_lock(&p->value_nb_mutex, &ct->mutex); - run_unlockers(unlockers); // we hold the ct mutex. - uint64_t t0 = get_tnow(); - cachetable_fetch_pair(ct, cf, p, fetch_callback, read_extraargs, false); - increment_partitioned_counter(cachetable_miss, 1); - increment_partitioned_counter(cachetable_misstime, get_tnow() - t0); - cachetable_unlock(ct); - return TOKUDB_TRY_AGAIN; + fetch_callback, + pf_req_callback, + pf_callback, + may_modify_value, + read_extraargs, + unlockers + ); + toku_cachetable_end_batched_pin(cf); + return r; } struct cachefile_prefetch_args { @@ -2194,7 +2135,6 @@ static void cachetable_reader(void* extra) { struct cachefile_prefetch_args* cpargs = (struct cachefile_prefetch_args*)extra; CACHEFILE cf = cpargs->p->cachefile; CACHETABLE ct = cf->cachetable; - cachetable_lock(ct); cachetable_fetch_pair( ct, cpargs->p->cachefile, @@ -2203,7 +2143,6 @@ static void cachetable_reader(void* extra) { cpargs->read_extraargs, false ); - cachetable_unlock(ct); bjm_remove_background_job(cf->bjm); toku_free(cpargs); } @@ -2212,9 +2151,7 @@ static void cachetable_partial_reader(void* extra) { struct cachefile_partial_prefetch_args *cpargs = (struct cachefile_partial_prefetch_args*)extra; CACHEFILE cf = cpargs->p->cachefile; CACHETABLE ct = cf->cachetable; - cachetable_lock(ct); do_partial_fetch(ct, cpargs->p->cachefile, cpargs->p, cpargs->pf_callback, cpargs->read_extraargs, false); - cachetable_unlock(ct); bjm_remove_background_job(cf->bjm); toku_free(cpargs); } @@ -2229,23 +2166,30 @@ int toku_cachefile_prefetch(CACHEFILE cf, CACHEKEY key, uint32_t fullhash, // Effect: See the documentation for this function in cachetable.h { int r = 0; + PAIR p = NULL; if (doing_prefetch) { *doing_prefetch = false; } CACHETABLE ct = cf->cachetable; - cachetable_lock(ct); - // lookup - PAIR p; - for (p = ct->table[fullhash&(ct->table_size-1)]; p; p = p->hash_chain) { - if (p->key.b==key.b && p->cachefile==cf) { - pair_touch(p); - break; - } + // if cachetable has too much data, don't bother prefetching + if (ct->ev.should_client_thread_sleep()) { + goto exit; } - + ct->list.read_list_lock(); + // lookup + p = ct->list.find_pair(cf, key, fullhash); // if not found then create a pair in the READING state and fetch it - if (p == 0) { - increment_partitioned_counter(cachetable_prefetches, 1); + if (p == NULL) { + cachetable_prefetches++; + ct->list.read_list_unlock(); + ct->list.write_list_lock(); + p = ct->list.find_pair(cf, key, fullhash); + if (p != NULL) { + pair_lock(p); + ct->list.write_list_unlock(); + goto found_pair; + } + r = bjm_add_background_job(cf->bjm); assert_zero(r); p = cachetable_insert_at( @@ -2259,7 +2203,11 @@ int toku_cachefile_prefetch(CACHEFILE cf, CACHEKEY key, uint32_t fullhash, CACHETABLE_CLEAN ); assert(p); - nb_mutex_lock(&p->value_nb_mutex, &ct->mutex); + pair_lock(p); + nb_mutex_lock(&p->value_nb_mutex, &p->mutex); + pair_unlock(p); + ct->list.write_list_unlock(); + struct cachefile_prefetch_args *MALLOC(cpargs); cpargs->p = p; cpargs->fetch_callback = fetch_callback; @@ -2268,10 +2216,19 @@ int toku_cachefile_prefetch(CACHEFILE cf, CACHEKEY key, uint32_t fullhash, if (doing_prefetch) { *doing_prefetch = true; } + goto exit; } - else if (nb_mutex_users(&p->value_nb_mutex)==0) { + pair_lock(p); + ct->list.read_list_unlock(); + +found_pair: + // at this point, p is found, pair's mutex is grabbed, and + // no list lock is held + if (nb_mutex_users(&p->value_nb_mutex)==0) { + pair_touch(p); // nobody else is using the node, so we should go ahead and prefetch - nb_mutex_lock(&p->value_nb_mutex, &ct->mutex); + nb_mutex_lock(&p->value_nb_mutex, &p->mutex); + pair_unlock(p); bool partial_fetch_required = pf_req_callback(p->value_data, read_extraargs); if (partial_fetch_required) { @@ -2287,82 +2244,24 @@ int toku_cachefile_prefetch(CACHEFILE cf, CACHEKEY key, uint32_t fullhash, } } else { + pair_lock(p); nb_mutex_unlock(&p->value_nb_mutex); + pair_unlock(p); } } - cachetable_unlock(ct); + else { + pair_unlock(p); + } +exit: return 0; } -// effect: Move an object from one key to another key. -// requires: The object is pinned in the table -int toku_cachetable_rename (CACHEFILE cachefile, CACHEKEY oldkey, CACHEKEY newkey) { - CACHETABLE ct = cachefile->cachetable; - PAIR *ptr_to_p,p; - uint32_t fullhash = toku_cachetable_hash(cachefile, oldkey); - cachetable_lock(ct); - for (ptr_to_p = &ct->table[fullhash&(ct->table_size-1)], p = *ptr_to_p; - p; - ptr_to_p = &p->hash_chain, p = *ptr_to_p) { - if (p->key.b==oldkey.b && p->cachefile==cachefile) { - *ptr_to_p = p->hash_chain; - p->key = newkey; - uint32_t new_fullhash = toku_cachetable_hash(cachefile, newkey); - uint32_t nh = new_fullhash&(ct->table_size-1); - p->fullhash = new_fullhash; - p->hash_chain = ct->table[nh]; - ct->table[nh] = p; - cachetable_unlock(ct); - return 0; - } - } - cachetable_unlock(ct); - return -1; -} - void toku_cachefile_verify (CACHEFILE cf) { toku_cachetable_verify(cf->cachetable); } void toku_cachetable_verify (CACHETABLE ct) { - cachetable_lock(ct); - uint32_t num_found = 0; - - // First clear all the verify flags by going through the hash chains - { - uint32_t i; - for (i=0; itable_size; i++) { - PAIR p; - for (p=ct->table[i]; p; p=p->hash_chain) { - num_found++; - } - } - } - assert(num_found == ct->n_in_table); - num_found = 0; - // Now go through the clock chain, make sure everything in the LRU chain is hashed. - { - PAIR p; - bool is_first = true; - for (p=ct->clock_head; ct->clock_head!=NULL && (p!=ct->clock_head || is_first); p=p->clock_next) { - is_first=false; - PAIR p2; - uint32_t fullhash = p->fullhash; - //assert(fullhash==toku_cachetable_hash(p->cachefile, p->key)); - for (p2=ct->table[fullhash&(ct->table_size-1)]; p2; p2=p2->hash_chain) { - if (p2==p) { - /* found it */ - num_found++; - goto next; - } - } - fprintf(stderr, "Something in the clock chain is not hashed\n"); - assert(0); - next:; - } - assert (num_found == ct->n_in_table); - } - cachetable_unlock(ct); + ct->list.verify(); } struct pair_flush_for_close{ @@ -2375,17 +2274,15 @@ static void cachetable_flush_pair_for_close(void* extra) { PAIR p = args->p; CACHEFILE cf = p->cachefile; CACHETABLE ct = cf->cachetable; - cachetable_lock(ct); PAIR_ATTR attr; cachetable_only_write_locked_data( - ct, + &ct->ev, p, false, // not for a checkpoint, as we assert above &attr, false // not a clone ); p->dirty = CACHETABLE_CLEAN; - cachetable_unlock(ct); bjm_remove_background_job(args->bjm); toku_free(args); } @@ -2414,44 +2311,34 @@ static void cachetable_flush_cachefile(CACHETABLE ct, CACHEFILE cf) { // no jobs added to the kibbutz. This implies that the only work other // threads may be doing is work by the writer threads. // - unsigned i; unsigned num_pairs = 0; unsigned list_size = 256; PAIR *list = NULL; XMALLOC_N(list_size, list); + ct->list.read_list_lock(); //Make a list of pairs that belong to this cachefile. - if (cf == NULL) { - for (i=0; i < ct->table_size; i++) { - PAIR p; - for (p = ct->table[i]; p; p = p->hash_chain) { - if (cf == 0 || p->cachefile==cf) { - if (num_pairs == list_size) { - list_size *= 2; - XREALLOC_N(list_size, list); - } - list[num_pairs++] = p; + for (i=0; i < ct->list.m_table_size; i++) { + PAIR p; + for (p = ct->list.m_table[i]; p; p = p->hash_chain) { + if (cf == 0 || p->cachefile == cf) { + if (num_pairs == list_size) { + list_size *= 2; + XREALLOC_N(list_size, list); } + list[num_pairs++] = p; } } - } - else { - for (struct toku_list *next_pair = cf->pairs_for_cachefile.next; next_pair != &cf->pairs_for_cachefile; next_pair = next_pair->next) { - PAIR p = toku_list_struct(next_pair, struct ctpair, next_for_cachefile); - if (num_pairs == list_size) { - list_size *= 2; - XREALLOC_N(list_size, list); - } - list[num_pairs++] = p; - } } + ct->list.read_list_unlock(); // first write out dirty PAIRs BACKGROUND_JOB_MANAGER bjm = NULL; bjm_init(&bjm); for (i=0; i < num_pairs; i++) { PAIR p = list[i]; + pair_lock(p); assert(nb_mutex_users(&p->value_nb_mutex) == 0); assert(nb_mutex_users(&p->disk_nb_mutex) == 0); assert(!p->cloned_value_data); @@ -2463,203 +2350,244 @@ static void cachetable_flush_cachefile(CACHETABLE ct, CACHEFILE cf) { args->bjm = bjm; toku_kibbutz_enq(ct->ct_kibbutz, cachetable_flush_pair_for_close, args); } + pair_unlock(p); } - cachetable_unlock(ct); bjm_wait_for_jobs_to_finish(bjm); bjm_destroy(bjm); - cachetable_lock(ct); // now get rid of everything + ct->list.write_list_lock(); for (i=0; i < num_pairs; i++) { PAIR p = list[i]; + pair_lock(p); assert(nb_mutex_users(&p->value_nb_mutex) == 0); assert(nb_mutex_users(&p->disk_nb_mutex) == 0); assert(!p->cloned_value_data); assert(p->dirty == CACHETABLE_CLEAN); - cachetable_maybe_remove_and_free_pair(ct, p); + // TODO: maybe break up this function + // so that write lock does not need to be held for entire + // free + cachetable_maybe_remove_and_free_pair(&ct->list, &ct->ev, p); } + // assert here that cachefile is flushed by checking + // pair_list and finding no pairs belonging to this cachefile + // Make a list of pairs that belong to this cachefile. + for (i=0; i < ct->list.m_table_size; i++) { + PAIR p; + for (p = ct->list.m_table[i]; p; p = p->hash_chain) { + assert(p->cachefile != cf); + } + } + ct->list.write_list_unlock(); if (cf) { - assert(toku_list_empty(&cf->pairs_for_cachefile)); bjm_reset(cf->bjm); } - - if ((4 * ct->n_in_table < ct->table_size) && (ct->table_size>4)) { - cachetable_rehash(ct, ct->table_size/2); - } toku_free(list); } /* Requires that no locks be held that are used by the checkpoint logic */ void toku_cachetable_minicron_shutdown(CACHETABLE ct) { - int r = toku_minicron_shutdown(&ct->checkpointer); - assert(r==0); - r = toku_minicron_shutdown(&ct->cleaner); + int r = ct->cp.shutdown(); assert(r==0); + ct->cl.destroy(); } /* Require that it all be flushed. */ int toku_cachetable_close (CACHETABLE *ctp) { + int r = 0; CACHETABLE ct=*ctp; - if (!toku_minicron_has_been_shutdown(&ct->checkpointer)) { - // for test code only, production code uses toku_cachetable_minicron_shutdown() - int r = toku_minicron_shutdown(&ct->checkpointer); - assert(r==0); - } - if (!toku_minicron_has_been_shutdown(&ct->cleaner)) { - // for test code only, production code uses toku_cachetable_minicron_shutdown() - int r = toku_minicron_shutdown(&ct->cleaner); - assert(r==0); - } - cachetable_lock(ct); + ct->cp.destroy(); + ct->cl.destroy(); cachetable_flush_cachefile(ct, NULL); - uint32_t i; - for (i=0; itable_size; i++) { - if (ct->table[i]) return -1; + ct->ev.destroy(); + r = ct->list.destroy(); + if (r != 0) { + // This means that there were still pairs in the + // pair list, which is bad. + return -1; } - assert(ct->size_evicting == 0); - rwlock_destroy(&ct->pending_lock); - cachetable_unlock(ct); + ct->cf_list.destroy(); + toku_kibbutz_destroy(ct->client_kibbutz); toku_kibbutz_destroy(ct->ct_kibbutz); toku_kibbutz_destroy(ct->checkpointing_kibbutz); - bjm_destroy(ct->checkpoint_clones_bjm); - toku_cond_destroy(&ct->flow_control_cond); - toku_mutex_destroy(&ct->mutex); - toku_free(ct->table); toku_free(ct->env_dir); toku_free(ct); *ctp = 0; return 0; } -int toku_cachetable_unpin_and_remove ( +static PAIR test_get_pair(CACHEFILE cachefile, CACHEKEY key, uint32_t fullhash, bool have_ct_lock) { + CACHETABLE ct = cachefile->cachetable; + + if (!have_ct_lock) { + ct->list.read_list_lock(); + } + + PAIR p = ct->list.find_pair(cachefile, key, fullhash); + assert(p != NULL); + if (!have_ct_lock) { + ct->list.read_list_unlock(); + } + return p; +} + +//test-only wrapper +int toku_test_cachetable_unpin(CACHEFILE cachefile, CACHEKEY key, uint32_t fullhash, enum cachetable_dirty dirty, PAIR_ATTR attr) { + // By default we don't have the lock + PAIR p = test_get_pair(cachefile, key, fullhash, false); + return toku_cachetable_unpin(cachefile, p, dirty, attr); +} + +//test-only wrapper +int toku_test_cachetable_unpin_ct_prelocked_no_flush(CACHEFILE cachefile, CACHEKEY key, uint32_t fullhash, enum cachetable_dirty dirty, PAIR_ATTR attr) { + // We hold the cachetable mutex. + PAIR p = test_get_pair(cachefile, key, fullhash, true); + return toku_cachetable_unpin_ct_prelocked_no_flush(cachefile, p, dirty, attr); +} + +//test-only wrapper +int toku_test_cachetable_unpin_and_remove ( CACHEFILE cachefile, CACHEKEY key, CACHETABLE_REMOVE_KEY remove_key, + void* remove_key_extra) +{ + uint32_t fullhash = toku_cachetable_hash(cachefile, key); + PAIR p = test_get_pair(cachefile, key, fullhash, false); + return toku_cachetable_unpin_and_remove(cachefile, p, remove_key, remove_key_extra); +} + +int toku_cachetable_unpin_and_remove ( + CACHEFILE cachefile, + PAIR p, + CACHETABLE_REMOVE_KEY remove_key, void* remove_key_extra ) { + invariant_notnull(p); int r = ENOENT; // Removing something already present is OK. CACHETABLE ct = cachefile->cachetable; - PAIR p; - cachetable_lock(ct); - uint32_t fullhash = toku_cachetable_hash(cachefile, key); - for (p=ct->table[fullhash&(ct->table_size-1)]; p; p=p->hash_chain) { - if (p->key.b==key.b && p->cachefile==cachefile) { - p->dirty = CACHETABLE_CLEAN; // clear the dirty bit. We're just supposed to remove it. - assert(nb_mutex_writers(&p->value_nb_mutex)); - // grab disk_nb_mutex to ensure any background thread writing - // out a cloned value completes - nb_mutex_lock(&p->disk_nb_mutex, &ct->mutex); - assert(p->cloned_value_data == NULL); - - // - // take care of key removal - // - bool for_checkpoint = p->checkpoint_pending; - // now let's wipe out the pending bit, because we are - // removing the PAIR - p->checkpoint_pending = false; - // - // Here is a tricky thing. - // Later on in this function, we may release the - // cachetable lock if other threads are blocked - // on this pair, trying to acquire the PAIR lock. - // While the cachetable lock is released, - // we may theoretically begin another checkpoint, or start - // a cleaner thread. - // So, just to be sure this PAIR won't be marked - // for the impending checkpoint, we mark the - // PAIR as clean. For the PAIR to not be picked by the - // cleaner thread, we mark the cachepressure_size to be 0 - // This should not be an issue because we call - // cachetable_remove_pair before - // releasing the cachetable lock. - // - p->dirty = CACHETABLE_CLEAN; - CACHEKEY key_to_remove = key; - p->attr.cache_pressure_size = 0; - // - // callback for removing the key - // for FTNODEs, this leads to calling - // toku_free_blocknum - // - if (remove_key) { - remove_key( - &key_to_remove, - for_checkpoint, - remove_key_extra - ); - } - nb_mutex_unlock(&p->value_nb_mutex); - nb_mutex_unlock(&p->disk_nb_mutex); - // - // As of Dr. Noga, only these threads may be - // blocked waiting to lock this PAIR: - // - the checkpoint thread (because a checkpoint is in progress - // and the PAIR was in the list of pending pairs) - // - a client thread running get_and_pin_nonblocking, who - // ran unlockers, then waited on the PAIR lock. - // While waiting on a PAIR lock, another thread comes in, - // locks the PAIR, and ends up calling unpin_and_remove, - // all while get_and_pin_nonblocking is waiting on the PAIR lock. - // We did not realize this at first, which caused bug #4357 - // The following threads CANNOT be blocked waiting on - // the PAIR lock: - // - a thread trying to run eviction via maybe_flush_some. - // That cannot happen because maybe_flush_some only - // attempts to lock PAIRS that are not locked, and this PAIR - // is locked. - // - cleaner thread, for the same reason as a thread running - // eviction - // - client thread doing a normal get_and_pin. The client is smart - // enough to not try to lock a PAIR that another client thread - // is trying to unpin and remove. Note that this includes work - // done on kibbutzes. - // - writer thread. Writer threads do not grab PAIR locks. They - // get PAIR locks transferred to them by client threads. - // - // first thing we do is remove the PAIR from the various - // cachetable data structures, so no other thread can possibly - // access it. We do not want to risk some other thread - // trying to lock this PAIR if we release the cachetable lock - // below. If some thread is already waiting on the lock, - // then we let that thread grab the lock and finish, but - // we don't want any NEW threads to try to grab the PAIR - // lock. - // - // Because we call cachetable_remove_pair and wait, - // the threads that may be waiting - // on this PAIR lock must be careful to do NOTHING with the PAIR - // As per our analysis above, we only need - // to make sure the checkpoint thread and get_and_pin_nonblocking do - // nothing, and looking at those functions, it is clear they do nothing. - // - cachetable_remove_pair(ct, p); - if (nb_mutex_blocked_writers(&p->value_nb_mutex)>0) { - nb_mutex_wait_for_users( - &p->value_nb_mutex, - &ct->mutex - ); - assert(!p->checkpoint_pending); - assert(p->attr.cache_pressure_size == 0); - } - // just a sanity check - assert(nb_mutex_users(&p->disk_nb_mutex) == 0); - assert(p->cloned_value_data == NULL); - //Remove pair. - cachetable_free_pair(ct, p); - r = 0; - goto done; - } + p->dirty = CACHETABLE_CLEAN; // clear the dirty bit. We're just supposed to remove it. + assert(nb_mutex_writers(&p->value_nb_mutex)); + // grab disk_nb_mutex to ensure any background thread writing + // out a cloned value completes + pair_lock(p); + nb_mutex_lock(&p->disk_nb_mutex, &p->mutex); + pair_unlock(p); + assert(p->cloned_value_data == NULL); + + // + // take care of key removal + // + ct->list.write_list_lock(); + ct->list.read_pending_cheap_lock(); + bool for_checkpoint = p->checkpoint_pending; + // now let's wipe out the pending bit, because we are + // removing the PAIR + p->checkpoint_pending = false; + + // + // Here is a tricky thing. + // Later on in this function, we may release the + // cachetable lock if other threads are blocked + // on this pair, trying to acquire the PAIR lock. + // While the cachetable lock is released, + // we may theoretically begin another checkpoint, or start + // a cleaner thread. + // So, just to be sure this PAIR won't be marked + // for the impending checkpoint, we mark the + // PAIR as clean. For the PAIR to not be picked by the + // cleaner thread, we mark the cachepressure_size to be 0 + // This should not be an issue because we call + // cachetable_remove_pair before + // releasing the cachetable lock. + // + p->dirty = CACHETABLE_CLEAN; + CACHEKEY key_to_remove = p->key; + p->attr.cache_pressure_size = 0; + // + // callback for removing the key + // for FTNODEs, this leads to calling + // toku_free_blocknum + // + if (remove_key) { + remove_key( + &key_to_remove, + for_checkpoint, + remove_key_extra + ); + } + ct->list.read_pending_cheap_unlock(); + + pair_lock(p); + nb_mutex_unlock(&p->value_nb_mutex); + nb_mutex_unlock(&p->disk_nb_mutex); + // + // As of Dr. Noga, only these threads may be + // blocked waiting to lock this PAIR: + // - the checkpoint thread (because a checkpoint is in progress + // and the PAIR was in the list of pending pairs) + // - a client thread running get_and_pin_nonblocking, who + // ran unlockers, then waited on the PAIR lock. + // While waiting on a PAIR lock, another thread comes in, + // locks the PAIR, and ends up calling unpin_and_remove, + // all while get_and_pin_nonblocking is waiting on the PAIR lock. + // We did not realize this at first, which caused bug #4357 + // The following threads CANNOT be blocked waiting on + // the PAIR lock: + // - a thread trying to run eviction via run_eviction. + // That cannot happen because run_eviction only + // attempts to lock PAIRS that are not locked, and this PAIR + // is locked. + // - cleaner thread, for the same reason as a thread running + // eviction + // - client thread doing a normal get_and_pin. The client is smart + // enough to not try to lock a PAIR that another client thread + // is trying to unpin and remove. Note that this includes work + // done on kibbutzes. + // - writer thread. Writer threads do not grab PAIR locks. They + // get PAIR locks transferred to them by client threads. + // + + // first thing we do is remove the PAIR from the various + // cachetable data structures, so no other thread can possibly + // access it. We do not want to risk some other thread + // trying to lock this PAIR if we release the cachetable lock + // below. If some thread is already waiting on the lock, + // then we let that thread grab the lock and finish, but + // we don't want any NEW threads to try to grab the PAIR + // lock. + // + // Because we call cachetable_remove_pair and wait, + // the threads that may be waiting + // on this PAIR lock must be careful to do NOTHING with the PAIR + // As per our analysis above, we only need + // to make sure the checkpoint thread and get_and_pin_nonblocking do + // nothing, and looking at those functions, it is clear they do nothing. + // + cachetable_remove_pair(&ct->list, &ct->ev, p); + ct->list.write_list_unlock(); + if (nb_mutex_blocked_writers(&p->value_nb_mutex)>0) { + nb_mutex_wait_for_users( + &p->value_nb_mutex, + &p->mutex + ); + assert(!p->checkpoint_pending); + assert(p->attr.cache_pressure_size == 0); } - done: - cachetable_unlock(ct); + // just a sanity check + assert(nb_mutex_users(&p->disk_nb_mutex) == 0); + assert(p->cloned_value_data == NULL); + //Remove pair. + pair_unlock(p); + cachetable_free_pair(p); + r = 0; return r; } @@ -2669,9 +2597,7 @@ set_filenum_in_array(const FT &ft, const uint32_t index, FILENUM *const array) { return 0; } -int -log_open_txn (const TOKUTXN &txn, const uint32_t UU(index), CACHETABLE *const ctp); -int +static int log_open_txn (const TOKUTXN &txn, const uint32_t UU(index), CACHETABLE *const ctp) { int r; CACHETABLE ct = *ctp; @@ -2683,13 +2609,12 @@ log_open_txn (const TOKUTXN &txn, const uint32_t UU(index), CACHETABLE *const ct goto cleanup; } else { - ct->checkpoint_num_txns++; + ct->cp.increment_num_txns(); } open_filenums.num = num_filenums; open_filenums.filenums = array; //Fill in open_filenums - r = txn->open_fts.iterate(array); invariant(r==0); switch (toku_txn_get_state(txn)) { @@ -2737,170 +2662,14 @@ cleanup: return 0; } +// Requires: All three checkpoint-relevant locks must be held (see checkpoint.c). +// Algorithm: Write a checkpoint record to the log, noting the LSN of that record. +// Use the begin_checkpoint callback to take necessary snapshots (header, btt) +// Mark every dirty node as "pending." ("Pending" means that the node must be +// written to disk before it can be modified.) int -toku_cachetable_begin_checkpoint (CACHETABLE ct, TOKULOGGER logger) { - // Requires: All three checkpoint-relevant locks must be held (see checkpoint.c). - // Algorithm: Write a checkpoint record to the log, noting the LSN of that record. - // Use the begin_checkpoint callback to take necessary snapshots (header, btt) - // Mark every dirty node as "pending." ("Pending" means that the node must be - // written to disk before it can be modified.) - { - unsigned i; - cachetable_lock(ct); - //Initialize accountability counters - ct->checkpoint_num_files = 0; - ct->checkpoint_num_txns = 0; - - //Make list of cachefiles to be included in checkpoint. - { - CACHEFILE cf; - assert(ct->cachefiles_in_checkpoint==NULL); - for (cf = ct->cachefiles; cf; cf=cf->next) { - // The caller must serialize open, close, and begin checkpoint. - // So we should never see a closing cachefile here. - - // putting this check so that this function may be called - // by cachetable tests - if (cf->note_pin_by_checkpoint) { - int r = cf->note_pin_by_checkpoint(cf, cf->userdata); - assert(r==0); - } - cf->next_in_checkpoint = ct->cachefiles_in_checkpoint; - ct->cachefiles_in_checkpoint = cf; - cf->for_checkpoint = true; - } - } - - if (logger) { - { - LSN begin_lsn={ .lsn = (uint64_t) -1 }; // we'll need to store the lsn of the checkpoint begin in all the trees that are checkpointed. - TXN_MANAGER mgr = toku_logger_get_txn_manager(logger); - TXNID last_xid = toku_txn_manager_get_last_xid(mgr); - int r = toku_log_begin_checkpoint(logger, &begin_lsn, 0, 0, last_xid); - assert(r==0); - ct->lsn_of_checkpoint_in_progress = begin_lsn; - } - // Log all the open files - { - //Must loop through ALL open files (even if not included in checkpoint). - CACHEFILE cf; - for (cf = ct->cachefiles; cf; cf=cf->next) { - if (cf->log_fassociate_during_checkpoint) { - int r = cf->log_fassociate_during_checkpoint(cf, cf->userdata); - ct->checkpoint_num_files++; - assert(r==0); - } - } - } - // Log all the open transactions MUST BE AFTER OPEN FILES - { - int r = toku_txn_manager_iter_over_live_txns( - logger->txn_manager, - &ct - ); - assert(r==0); - } - // Log rollback suppression for all the open files MUST BE AFTER TXNS - { - //Must loop through ALL open files (even if not included in checkpoint). - CACHEFILE cf; - for (cf = ct->cachefiles; cf; cf=cf->next) { - if (cf->log_suppress_rollback_during_checkpoint) { - int r = cf->log_suppress_rollback_during_checkpoint(cf, cf->userdata); - assert(r==0); - } - } - } - } - - unsigned int npending = 0; - // - // Here is why we have the pending lock, and why we take its write lock - // at this point. - // - // First, here is how the pending lock is used: - // - begin checkpoint grabs the write lock - // - threads that write a node to disk grab the read lock - // - // As a result, when we grab the write lock here, we know that - // no writer threads are in the middle of writing a node out to disk. - // - // We are protecting against a race condition between writer - // threads that write nodes to disk, and the beginning of this checkpoint. - // When a writer thread writes a node to disk, the cachetable lock is released, - // allowing a begin_checkpoint to occur. If writer threads and begin checkpoint - // run concurrently, our checkpoint may be incorrect. Here is how. - // - // Here is the specific race condition. Suppose this pending lock does not exist, - // and writer threads may be writing nodes to disk. Take the following scenario: - // Writer thread: - // -grabs cachetable lock, has a dirty PAIR without the checkpoint pending bit set - // -releases the cachetable lock with the intent of writing this node to disk - // Before the writer thread goes any further, the checkpoint thread comes along: - // - marks the dirty PAIR that is about to be written out as pending. - // - copies the current translation table of that contains the PAIR to the inprogress one (see struct block_table) - // At this point, for the checkpoint to be correct, the dirty PAIR - // that is in the process of being written out should be included in the inprogress translation table. This PAIR - // belongs in the checkpoint. - // Now let's go back to the writer thread: - // - because the checkpoint pending bit was not set for the PAIR, the for_checkpoint parameter - // passed into the flush callback is false. - // - as a result, the PAIR is written to disk, the current translation table is updated, but the - // inprogress translation table is NOT updated. - // - the PAIR is marked as clean because it was just written to disk - // Now, when the checkpoint thread gets around to this PAIR, it notices - // that the checkpoint_pending bit is set, but the PAIR is clean. So no I/O is done. - // The checkpoint_pending bit is cleared, without the inprogress translation table ever being - // updated. This is not correct. The result is that the checkpoint does not include the proper - // state of this PAIR. - // - rwlock_write_lock(&ct->pending_lock, &ct->mutex); - bjm_reset(ct->checkpoint_clones_bjm); - for (i=0; i < ct->table_size; i++) { - PAIR p; - for (p = ct->table[i]; p; p=p->hash_chain) { - assert(!p->checkpoint_pending); - //Only include pairs belonging to cachefiles in the checkpoint - if (!p->cachefile->for_checkpoint) continue; - // mark anything that is dirty OR currently in use - // as pending a checkpoint - // - //The rule for the checkpoint_pending bit is as follows: - // - begin_checkpoint may set checkpoint_pending to true - // even though the pair lock on the node is not held. Only the - // cachetable lock is necessary - // - any thread that wants to clear the pending bit must own - // BOTH the cachetable lock and the PAIR lock. Otherwise, - // we may end up clearing the pending bit before the - // current lock is ever released. - if (p->dirty || nb_mutex_writers(&p->value_nb_mutex)) { - p->checkpoint_pending = true; - if (ct->pending_head) { - ct->pending_head->pending_prev = p; - } - p->pending_next = ct->pending_head; - p->pending_prev = NULL; - ct->pending_head = p; - npending++; - } - } - } - rwlock_write_unlock(&ct->pending_lock); - - //begin_checkpoint_userdata must be called AFTER all the pairs are marked as pending. - //Once marked as pending, we own write locks on the pairs, which means the writer threads can't conflict. - { - CACHEFILE cf; - for (cf = ct->cachefiles_in_checkpoint; cf; cf=cf->next_in_checkpoint) { - if (cf->begin_checkpoint_userdata) { - int r = cf->begin_checkpoint_userdata(ct->lsn_of_checkpoint_in_progress, cf->userdata); - assert(r==0); - } - } - } - cachetable_unlock(ct); - } - return 0; +toku_cachetable_begin_checkpoint (CHECKPOINTER cp, TOKULOGGER UU(logger)) { + return cp->begin_checkpoint(); } @@ -2913,108 +2682,19 @@ int toku_cachetable_get_checkpointing_user_data_status (void) { return toku_checkpointing_user_data_status; } +// Requires: The big checkpoint lock must be held (see checkpoint.c). +// Algorithm: Write all pending nodes to disk +// Use checkpoint callback to write snapshot information to disk (header, btt) +// Use end_checkpoint callback to fsync dictionary and log, and to free unused blocks +// Note: If testcallback is null (for testing purposes only), call it after writing dictionary but before writing log int -toku_cachetable_end_checkpoint(CACHETABLE ct, TOKULOGGER logger, - void (*testcallback_f)(void*), void * testextra) { - // Requires: The big checkpoint lock must be held (see checkpoint.c). - // Algorithm: Write all pending nodes to disk - // Use checkpoint callback to write snapshot information to disk (header, btt) - // Use end_checkpoint callback to fsync dictionary and log, and to free unused blocks - // Note: If testcallback is null (for testing purposes only), call it after writing dictionary but before writing log - - int retval = 0; - cachetable_lock(ct); - { - PAIR p; - while ((p = ct->pending_head)!=0) { - ct->pending_head = ct->pending_head->pending_next; - pending_pairs_remove(ct, p); - write_pair_for_checkpoint_thread(ct, p); // if still pending, clear the pending bit and write out the node - // Don't need to unlock and lock cachetable, because the cachetable was unlocked and locked while the flush callback ran. - } - } - assert(!ct->pending_head); - cachetable_unlock(ct); - bjm_wait_for_jobs_to_finish(ct->checkpoint_clones_bjm); - cachetable_lock(ct); - - - { // have just written data blocks, so next write the translation and header for each open dictionary - CACHEFILE cf; - for (cf = ct->cachefiles_in_checkpoint; cf; cf=cf->next_in_checkpoint) { - if (cf->checkpoint_userdata) { - cachetable_unlock(ct); - toku_cachetable_set_checkpointing_user_data_status(1); - int r = cf->checkpoint_userdata(cf, cf->fd, cf->userdata); - toku_cachetable_set_checkpointing_user_data_status(0); - assert(r==0); - cachetable_lock(ct); - } - } - } - - cachetable_unlock(ct); - // For testing purposes only. Dictionary has been fsync-ed to disk but log has not yet been written. - if (testcallback_f) { - testcallback_f(testextra); - } - if (logger) { - int r = toku_log_end_checkpoint(logger, NULL, - 1, // want the end_checkpoint to be fsync'd - ct->lsn_of_checkpoint_in_progress, - 0, - ct->checkpoint_num_files, - ct->checkpoint_num_txns); - assert(r==0); - toku_logger_note_checkpoint(logger, ct->lsn_of_checkpoint_in_progress); - } - cachetable_lock(ct); - - { - // everything has been written to file and fsynced - // ... call checkpoint-end function in block translator - // to free obsolete blocks on disk used by previous checkpoint - CACHEFILE cf; - //cachefiles_in_checkpoint is protected by the checkpoint_safe_lock - for (cf = ct->cachefiles_in_checkpoint; cf; cf=cf->next_in_checkpoint) { - if (cf->end_checkpoint_userdata) { - cachetable_unlock(ct); - int r = cf->end_checkpoint_userdata(cf, cf->fd, cf->userdata); - assert(r==0); - cachetable_lock(ct); - } - } - } - cachetable_unlock(ct); - - { - //Delete list of cachefiles in the checkpoint, - //remove reference - //clear bit saying they're in checkpoint - CACHEFILE cf; - //cachefiles_in_checkpoint is protected by the checkpoint_safe_lock - while ((cf = ct->cachefiles_in_checkpoint)) { - ct->cachefiles_in_checkpoint = cf->next_in_checkpoint; - cf->next_in_checkpoint = NULL; - cf->for_checkpoint = false; - // checking for function existing so that this function - // can be called from cachetable tests - if (cf->note_unpin_by_checkpoint) { - int r = cf->note_unpin_by_checkpoint(cf, cf->userdata); - if (r!=0) { - retval = r; - goto panic; - } - } - } - } - -panic: - return retval; +toku_cachetable_end_checkpoint(CHECKPOINTER cp, TOKULOGGER UU(logger), + void (*testcallback_f)(void*), void* testextra) { + return cp->end_checkpoint(testcallback_f, testextra); } TOKULOGGER toku_cachefile_logger (CACHEFILE cf) { - return cf->cachetable->logger; + return cf->cachetable->cp.get_logger(); } FILENUM toku_cachefile_filenum (CACHEFILE cf) { @@ -3026,18 +2706,20 @@ FILENUM toku_cachefile_filenum (CACHEFILE cf) { int toku_cachetable_assert_all_unpinned (CACHETABLE ct) { uint32_t i; int some_pinned=0; - cachetable_lock(ct); - for (i=0; itable_size; i++) { + ct->list.read_list_lock(); + for (i=0; ilist.m_table_size; i++) { PAIR p; - for (p=ct->table[i]; p; p=p->hash_chain) { + for (p=ct->list.m_table[i]; p; p=p->hash_chain) { + pair_lock(p); assert(nb_mutex_writers(&p->value_nb_mutex)>=0); if (nb_mutex_writers(&p->value_nb_mutex)) { //printf("%s:%d pinned: %" PRId64 " (%p)\n", __FILE__, __LINE__, p->key.b, p->value_data); some_pinned=1; } + pair_unlock(p); } } - cachetable_unlock(ct); + ct->list.read_list_unlock(); return some_pinned; } @@ -3045,69 +2727,77 @@ int toku_cachefile_count_pinned (CACHEFILE cf, int print_them) { assert(cf != NULL); int n_pinned=0; CACHETABLE ct = cf->cachetable; - cachetable_lock(ct); - for (struct toku_list *next_pair = cf->pairs_for_cachefile.next; next_pair != &cf->pairs_for_cachefile; next_pair = next_pair->next) { - PAIR p = toku_list_struct(next_pair, struct ctpair, next_for_cachefile); - assert(nb_mutex_writers(&p->value_nb_mutex) >= 0); - if (nb_mutex_writers(&p->value_nb_mutex)) { - if (print_them) printf("%s:%d pinned: %" PRId64 " (%p)\n", __FILE__, __LINE__, p->key.b, p->value_data); - n_pinned++; + ct->list.read_list_lock(); + + // Iterate over all the pairs to find pairs specific to the + // given cachefile. + for (uint32_t i = 0; i < ct->list.m_table_size; i++) { + for (PAIR p = ct->list.m_table[i]; p; p = p->hash_chain) { + if (p->cachefile == cf) { + pair_lock(p); + assert(nb_mutex_writers(&p->value_nb_mutex) >= 0); + if (nb_mutex_writers(&p->value_nb_mutex)) { + if (print_them) { + printf("%s:%d pinned: %" PRId64 " (%p)\n", + __FILE__, + __LINE__, + p->key.b, + p->value_data); + } + n_pinned++; + } + pair_unlock(p); + } } } - cachetable_unlock(ct); + + ct->list.read_list_unlock(); return n_pinned; } void toku_cachetable_print_state (CACHETABLE ct) { uint32_t i; - cachetable_lock(ct); - for (i=0; itable_size; i++) { - PAIR p = ct->table[i]; + ct->list.read_list_lock(); + for (i=0; ilist.m_table_size; i++) { + PAIR p = ct->list.m_table[i]; if (p != 0) { + pair_lock(p); printf("t[%u]=", i); - for (p=ct->table[i]; p; p=p->hash_chain) { + for (p=ct->list.m_table[i]; p; p=p->hash_chain) { printf(" {%" PRId64 ", %p, dirty=%d, pin=%d, size=%ld}", p->key.b, p->cachefile, (int) p->dirty, nb_mutex_writers(&p->value_nb_mutex), p->attr.size); } printf("\n"); + pair_unlock(p); } } - cachetable_unlock(ct); + ct->list.read_list_unlock(); } void toku_cachetable_get_state (CACHETABLE ct, int *num_entries_ptr, int *hash_size_ptr, long *size_current_ptr, long *size_limit_ptr) { - cachetable_lock(ct); - if (num_entries_ptr) - *num_entries_ptr = ct->n_in_table; - if (hash_size_ptr) - *hash_size_ptr = ct->table_size; - if (size_current_ptr) - *size_current_ptr = ct->size_current; - if (size_limit_ptr) - *size_limit_ptr = ct->size_limit; - cachetable_unlock(ct); + ct->list.get_state(num_entries_ptr, hash_size_ptr); + ct->ev.get_state(size_current_ptr, size_limit_ptr); } int toku_cachetable_get_key_state (CACHETABLE ct, CACHEKEY key, CACHEFILE cf, void **value_ptr, int *dirty_ptr, long long *pin_ptr, long *size_ptr) { - PAIR p; int r = -1; uint32_t fullhash = toku_cachetable_hash(cf, key); - cachetable_lock(ct); - for (p = ct->table[fullhash&(ct->table_size-1)]; p; p = p->hash_chain) { - if (p->key.b == key.b && p->cachefile == cf) { - if (value_ptr) - *value_ptr = p->value_data; - if (dirty_ptr) - *dirty_ptr = p->dirty; - if (pin_ptr) - *pin_ptr = nb_mutex_writers(&p->value_nb_mutex); - if (size_ptr) - *size_ptr = p->attr.size; - r = 0; - break; - } + ct->list.read_list_lock(); + PAIR p = ct->list.find_pair(cf, key, fullhash); + if (p) { + pair_lock(p); + if (value_ptr) + *value_ptr = p->value_data; + if (dirty_ptr) + *dirty_ptr = p->dirty; + if (pin_ptr) + *pin_ptr = nb_mutex_writers(&p->value_nb_mutex); + if (size_ptr) + *size_ptr = p->attr.size; + r = 0; + pair_unlock(p); } - cachetable_unlock(ct); + ct->list.read_list_unlock(); return r; } @@ -3202,13 +2892,6 @@ toku_cachetable_get_fname_in_cwd(CACHETABLE ct, const char * fname_in_env) { return toku_construct_full_name(2, ct->env_dir, fname_in_env); } - -// Returns the limit on the size of the cache table -uint64_t toku_cachetable_get_size_limit(CACHETABLE ct) { - return ct->size_limit; -} - - static long cleaner_thread_rate_pair(PAIR p) { @@ -3217,8 +2900,61 @@ cleaner_thread_rate_pair(PAIR p) static int const CLEANER_N_TO_CHECK = 8; -int -toku_cleaner_thread (void *cachetable_v) +int toku_cleaner_thread_for_test (CACHETABLE ct) { + return ct->cl.run_cleaner(); +} + +int toku_cleaner_thread (void *cleaner_v) { + cleaner* cl = (cleaner *) cleaner_v; + assert(cl); + return cl->run_cleaner(); +} + +///////////////////////////////////////////////////////////////////////// +// +// cleaner methods +// +static_assert(std::is_pod::value, "cleaner isn't POD"); + +void cleaner::init(uint32_t _cleaner_iterations, pair_list* _pl, CACHETABLE _ct) { + // default is no cleaner, for now + toku_minicron_setup(&m_cleaner_cron, 0, toku_cleaner_thread, this); + HELGRIND_VALGRIND_HG_DISABLE_CHECKING(&m_cleaner_iterations, sizeof m_cleaner_iterations); + m_cleaner_iterations = _cleaner_iterations; + m_pl = _pl; + m_ct = _ct; +} + +// this function is allowed to be called multiple times +void cleaner::destroy(void) { + if (!toku_minicron_has_been_shutdown(&m_cleaner_cron)) { + // for test code only, production code uses toku_cachetable_minicron_shutdown() + int r = toku_minicron_shutdown(&m_cleaner_cron); + assert(r==0); + } +} + +uint32_t cleaner::get_iterations(void) { + return m_cleaner_iterations; +} + +void cleaner::set_iterations(uint32_t new_iterations) { + m_cleaner_iterations = new_iterations; +} + +uint32_t cleaner::get_period(void) { + return toku_minicron_get_period(&m_cleaner_cron); +} + +uint32_t cleaner::get_period_unlocked(void) { + return toku_minicron_get_period_unlocked(&m_cleaner_cron); +} + +void cleaner::set_period(uint32_t new_period) { + int r = toku_minicron_change_period(&m_cleaner_cron, new_period); + assert_zero(r); +} + // Effect: runs a cleaner. // // We look through some number of nodes, the first N that we see which are @@ -3229,21 +2965,19 @@ toku_cleaner_thread (void *cachetable_v) // cachefile that we're doing some background work (so a flush won't // start). At this point, we can safely unlock the cachetable, do the // work (callback), and unlock/release our claim to the cachefile. -{ +int cleaner::run_cleaner(void) { int r; - CACHETABLE ct = (CACHETABLE) cachetable_v; - assert(ct); - uint32_t num_iterations = toku_get_cleaner_iterations(ct); + uint32_t num_iterations = this->get_iterations(); for (uint32_t i = 0; i < num_iterations; ++i) { - increment_partitioned_counter(cleaner_executions, 1); - cachetable_lock(ct); + cleaner_executions++; + m_pl->read_list_lock(); PAIR best_pair = NULL; int n_seen = 0; long best_score = 0; - const PAIR first_pair = ct->cleaner_head; + const PAIR first_pair = m_pl->m_cleaner_head; if (first_pair == NULL) { // nothing in the cachetable, just get out now - cachetable_unlock(ct); + m_pl->read_list_unlock(); break; } // here we select a PAIR for cleaning @@ -3257,24 +2991,39 @@ toku_cleaner_thread (void *cachetable_v) // - this is how a thread that is calling unpin_and_remove will prevent // the cleaner thread from picking its PAIR (see comments in that function) do { - if (nb_mutex_users(&ct->cleaner_head->value_nb_mutex) > 0) { - goto next_pair; + pair_lock(m_pl->m_cleaner_head); + if (nb_mutex_users(&m_pl->m_cleaner_head->value_nb_mutex) > 0) { + pair_unlock(m_pl->m_cleaner_head); } - n_seen++; - { - long score = cleaner_thread_rate_pair(ct->cleaner_head); + else { + n_seen++; + long score = 0; + score = cleaner_thread_rate_pair(m_pl->m_cleaner_head); if (score > best_score) { best_score = score; - best_pair = ct->cleaner_head; + // Since we found a new best pair, we need to + // free the old best pair. + if (best_pair) { + pair_unlock(best_pair); + } + best_pair = m_pl->m_cleaner_head; + } + else { + pair_unlock(m_pl->m_cleaner_head); } } - next_pair: - ct->cleaner_head = ct->cleaner_head->clock_next; - } while (ct->cleaner_head != first_pair && n_seen < CLEANER_N_TO_CHECK); + // Advance the cleaner head. + m_pl->m_cleaner_head = m_pl->m_cleaner_head->clock_next; + } while (m_pl->m_cleaner_head != first_pair && n_seen < CLEANER_N_TO_CHECK); + m_pl->read_list_unlock(); + // // at this point, if we have found a PAIR for cleaning, // that is, best_pair != NULL, we do the clean // + // if best_pair !=NULL, then best_pair->mutex is held + // no list lock is held + // if (best_pair) { CACHEFILE cf = best_pair->cachefile; // try to add a background job to the manager @@ -3283,14 +3032,20 @@ toku_cleaner_thread (void *cachetable_v) // becomes a no-op r = bjm_add_background_job(cf->bjm); if (r) { - cachetable_unlock(ct); + pair_unlock(best_pair); continue; } - nb_mutex_lock(&best_pair->value_nb_mutex, &ct->mutex); + nb_mutex_lock(&best_pair->value_nb_mutex, &best_pair->mutex); + pair_unlock(best_pair); // verify a key assumption. assert(cleaner_thread_rate_pair(best_pair) > 0); - if (best_pair->checkpoint_pending) { - write_locked_pair_for_checkpoint(ct, best_pair); + // check the checkpoint_pending bit + m_pl->read_pending_cheap_lock(); + bool checkpoint_pending = best_pair->checkpoint_pending; + best_pair->checkpoint_pending = false; + m_pl->read_pending_cheap_unlock(); + if (checkpoint_pending) { + write_locked_pair_for_checkpoint(m_ct, best_pair, true); } bool cleaner_callback_called = false; @@ -3300,20 +3055,20 @@ toku_cleaner_thread (void *cachetable_v) // in Dr. Noga, but unit tests verify this behavior works properly. if (cleaner_thread_rate_pair(best_pair) > 0) { - cachetable_unlock(ct); r = best_pair->cleaner_callback(best_pair->value_data, best_pair->key, best_pair->fullhash, best_pair->write_extraargs); assert_zero(r); cleaner_callback_called = true; - cachetable_lock(ct); } // The cleaner callback must have unlocked the pair, so we // don't need to unlock it if the cleaner callback is called. if (!cleaner_callback_called) { + pair_lock(best_pair); nb_mutex_unlock(&best_pair->value_nb_mutex); + pair_unlock(best_pair); } // We need to make sure the cachefile sticks around so a close // can't come destroy it. That's the purpose of this @@ -3321,10 +3076,8 @@ toku_cleaner_thread (void *cachetable_v) // cachefile is still valid here, even though the cleaner // callback unlocks the pair. bjm_remove_background_job(cf->bjm); - cachetable_unlock(ct); } else { - cachetable_unlock(ct); // If we didn't find anything this time around the cachetable, // we probably won't find anything if we run around again, so // just break out from the for-loop now and @@ -3335,6 +3088,1305 @@ toku_cleaner_thread (void *cachetable_v) return 0; } +static_assert(std::is_pod::value, "pair_list isn't POD"); + +const uint32_t INITIAL_PAIR_LIST_SIZE = 4; + +// Allocates the hash table of pairs inside this pair list. +// +void pair_list::init() { + m_table_size = INITIAL_PAIR_LIST_SIZE; + m_n_in_table = 0; + m_clock_head = NULL; + m_cleaner_head = NULL; + m_pending_head = NULL; + m_table = NULL; + + + pthread_rwlockattr_t attr; + pthread_rwlockattr_init(&attr); +#if defined(HAVE_PTHREAD_RWLOCKATTR_SETKIND_NP) + pthread_rwlockattr_setkind_np(&attr, PTHREAD_RWLOCK_PREFER_WRITER_NONRECURSIVE_NP); +#else + // TODO: need to figure out how to make writer-preferential rwlocks + // happen on osx +#endif + toku_pthread_rwlock_init(&m_list_lock, &attr); + toku_pthread_rwlock_init(&m_pending_lock_expensive, &attr); + toku_pthread_rwlock_init(&m_pending_lock_cheap, &attr); + XCALLOC_N(m_table_size, m_table); +} + +// Frees the pair_list hash table. It is expected to be empty by +// the time this is called. Returns an error if there are any +// pairs in any of the hash table slots. +int pair_list::destroy() { + // Check if any entries exist in the hash table. + for (uint32_t i = 0; i < m_table_size; ++i) { + if (m_table[i]) { + return -1; + } + } + toku_pthread_rwlock_destroy(&m_list_lock); + toku_pthread_rwlock_destroy(&m_pending_lock_expensive); + toku_pthread_rwlock_destroy(&m_pending_lock_cheap); + toku_free(m_table); + return 0; +} + +// This places the given pair inside of the pair list. +// +// requires caller to have grabbed write lock on list. +// +void pair_list::put(PAIR p) { + // sanity check to make sure that the PAIR does not already exist + PAIR pp = this->find_pair(p->cachefile, p->key, p->fullhash); + assert(pp == NULL); + + this->add_to_clock(p); + uint32_t h = p->fullhash & (m_table_size - 1); + p->hash_chain = m_table[h]; + m_table[h] = p; + m_n_in_table++; + + if (m_n_in_table > m_table_size) { + this->rehash(m_table_size * 2); + } +} + +// This removes the given pair from the pair list. +// +// requires caller to have grabbed write lock on list. +// +void pair_list::evict(PAIR p) { + this->pair_remove(p); + this->pending_pairs_remove(p); + + assert(m_n_in_table > 0); + m_n_in_table--; + + // Remove it from the hash chain. + unsigned int h = p->fullhash&(m_table_size - 1); + m_table[h] = this->remove_from_hash_chain(p, m_table[h]); + + // possibly rehash + if ((4 * m_n_in_table < m_table_size) && m_table_size > 4) { + this->rehash(m_table_size / 2); + } +} + +PAIR pair_list::remove_from_hash_chain (PAIR remove_me, PAIR list) { + if (remove_me == list) { + return list->hash_chain; + } + list->hash_chain = this->remove_from_hash_chain(remove_me, list->hash_chain); + return list; +} + +// +// Remove pair from linked list for cleaner/clock +// +// +// requires caller to have grabbed write lock on list. +// +void pair_list::pair_remove (PAIR p) { + if (p->clock_prev == p) { + assert(m_clock_head == p); + assert(p->clock_next == p); + assert(m_cleaner_head == p); + m_clock_head = NULL; + m_cleaner_head = NULL; + } + else { + if (p == m_clock_head) { + m_clock_head = m_clock_head->clock_next; + } + if (p == m_cleaner_head) { + m_cleaner_head = m_cleaner_head->clock_next; + } + p->clock_prev->clock_next = p->clock_next; + p->clock_next->clock_prev = p->clock_prev; + + } +} + +//Remove a pair from the list of pairs that were marked with the +//pending bit for the in-progress checkpoint. +// +// requires that if the caller is the checkpoint thread, then a read lock +// is grabbed on the list. Otherwise, must have write lock on list. +// +void pair_list::pending_pairs_remove (PAIR p) { + if (p->pending_next) { + p->pending_next->pending_prev = p->pending_prev; + } + if (p->pending_prev) { + p->pending_prev->pending_next = p->pending_next; + } + else if (m_pending_head==p) { + m_pending_head = p->pending_next; + } + p->pending_prev = p->pending_next = NULL; +} + + +// Returns a pair from the pair list, using the given +// pair. If the pair cannot be found, null is returned. +// +// +// requires caller to have grabbed read lock on list. +// +PAIR pair_list::find_pair(CACHEFILE file, CACHEKEY key, uint32_t fullhash) { + PAIR found_pair = nullptr; + for (PAIR p = m_table[fullhash&(m_table_size - 1)]; p; p = p->hash_chain) { + if (p->key.b == key.b && p->cachefile == file) { + found_pair = p; + break; + } + } + return found_pair; +} + +// has ct locked on entry +// This function MUST NOT release and reacquire the cachetable lock +// Its callers (toku_cachetable_put_with_dep_pairs) depend on this behavior. +// +// requires caller to have grabbed write lock on list. +// +void pair_list::rehash (uint32_t newtable_size) { + assert(newtable_size >= 4 && ((newtable_size & (newtable_size - 1))==0)); + PAIR *XCALLOC_N(newtable_size, newtable); + assert(newtable!=0); + uint32_t oldtable_size = m_table_size; + m_table_size = newtable_size; + for (uint32_t i = 0; i < newtable_size; i++) { + newtable[i] = 0; + } + for (uint32_t i = 0; i < oldtable_size; i++) { + PAIR p; + while ((p = m_table[i]) != 0) { + unsigned int h = p->fullhash&(newtable_size - 1); + m_table[i] = p->hash_chain; + p->hash_chain = newtable[h]; + newtable[h] = p; + } + } + toku_free(m_table); + m_table = newtable; +} + +// Add PAIR to linked list shared by cleaner thread and clock +// +// requires caller to have grabbed write lock on list. +// +void pair_list::add_to_clock (PAIR p) { + // requires that p is not currently in the table. + // inserts p into the clock list at the tail. + + p->count = CLOCK_INITIAL_COUNT; + //assert either both head and tail are set or they are both NULL + // tail and head exist + if (m_clock_head) { + assert(m_cleaner_head); + // insert right before the head + p->clock_next = m_clock_head; + p->clock_prev = m_clock_head->clock_prev; + + p->clock_prev->clock_next = p; + p->clock_next->clock_prev = p; + + } + // this is the first element in the list + else { + m_clock_head = p; + p->clock_next = p->clock_prev = m_clock_head; + m_cleaner_head = p; + } +} + +// test function +// +// grabs and releases write list lock +// +void pair_list::verify() { + this->write_list_lock(); + uint32_t num_found = 0; + + // First clear all the verify flags by going through the hash chains + { + uint32_t i; + for (i = 0; i < m_table_size; i++) { + PAIR p; + for (p = m_table[i]; p; p = p->hash_chain) { + num_found++; + } + } + } + assert(num_found == m_n_in_table); + num_found = 0; + // Now go through the clock chain, make sure everything in the LRU chain is hashed. + { + PAIR p; + bool is_first = true; + for (p = m_clock_head; m_clock_head != NULL && (p != m_clock_head || is_first); p=p->clock_next) { + is_first=false; + PAIR p2; + uint32_t fullhash = p->fullhash; + //assert(fullhash==toku_cachetable_hash(p->cachefile, p->key)); + for (p2 = m_table[fullhash&(m_table_size-1)]; p2; p2=p2->hash_chain) { + if (p2==p) { + /* found it */ + num_found++; + goto next; + } + } + fprintf(stderr, "Something in the clock chain is not hashed\n"); + assert(0); + next:; + } + assert (num_found == m_n_in_table); + } + this->write_list_unlock(); +} + +// If given pointers are not null, assign the hash table size of +// this pair list and the number of pairs in this pair list. +// +// +// grabs and releases read list lock +// +void pair_list::get_state(int *num_entries, int *hash_size) { + this->read_list_lock(); + if (num_entries) { + *num_entries = m_n_in_table; + } + if (hash_size) { + *hash_size = m_table_size; + } + this->read_list_unlock(); +} + +void pair_list::read_list_lock() { + toku_pthread_rwlock_rdlock(&m_list_lock); +} + +void pair_list::read_list_unlock() { + toku_pthread_rwlock_rdunlock(&m_list_lock); +} + +void pair_list::write_list_lock() { + toku_pthread_rwlock_wrlock(&m_list_lock); +} + +void pair_list::write_list_unlock() { + toku_pthread_rwlock_wrunlock(&m_list_lock); +} + +void pair_list::read_pending_exp_lock() { + toku_pthread_rwlock_rdlock(&m_pending_lock_expensive); +} + +void pair_list::read_pending_exp_unlock() { + toku_pthread_rwlock_rdunlock(&m_pending_lock_expensive); +} + +void pair_list::write_pending_exp_lock() { + toku_pthread_rwlock_wrlock(&m_pending_lock_expensive); +} + +void pair_list::write_pending_exp_unlock() { + toku_pthread_rwlock_wrunlock(&m_pending_lock_expensive); +} + +void pair_list::read_pending_cheap_lock() { + toku_pthread_rwlock_rdlock(&m_pending_lock_cheap); +} + +void pair_list::read_pending_cheap_unlock() { + toku_pthread_rwlock_rdunlock(&m_pending_lock_cheap); +} + +void pair_list::write_pending_cheap_lock() { + toku_pthread_rwlock_wrlock(&m_pending_lock_cheap); +} + +void pair_list::write_pending_cheap_unlock() { + toku_pthread_rwlock_wrunlock(&m_pending_lock_cheap); +} + + +static_assert(std::is_pod::value, "evictor isn't POD"); + +// +// This is the function that runs eviction on its own thread. +// +static void *eviction_thread(void *evictor_v) { + evictor* CAST_FROM_VOIDP(evictor, evictor_v); + evictor->run_eviction_thread(); + return evictor_v; +} + +// +// Starts the eviction thread, assigns external object references, +// and initializes all counters and condition variables. +// +void evictor::init(long _size_limit, pair_list* _pl, KIBBUTZ _kibbutz, uint32_t eviction_period) { + HELGRIND_VALGRIND_HG_DISABLE_CHECKING(&m_ev_thread_is_running, sizeof m_ev_thread_is_running); + HELGRIND_VALGRIND_HG_DISABLE_CHECKING(&m_size_nonleaf, sizeof m_size_nonleaf); + HELGRIND_VALGRIND_HG_DISABLE_CHECKING(&m_size_current, sizeof m_size_current); + HELGRIND_VALGRIND_HG_DISABLE_CHECKING(&m_size_evicting, sizeof m_size_evicting); + HELGRIND_VALGRIND_HG_DISABLE_CHECKING(&m_size_leaf, sizeof m_size_leaf); + HELGRIND_VALGRIND_HG_DISABLE_CHECKING(&m_size_rollback, sizeof m_size_rollback); + HELGRIND_VALGRIND_HG_DISABLE_CHECKING(&m_size_cachepressure, sizeof m_size_cachepressure); + + m_low_size_watermark = _size_limit; + // these values are selected kind of arbitrarily right now as + // being a percentage more than low_size_watermark, which is provided + // by the caller. + m_low_size_hysteresis = (11 * _size_limit)/10; //10% more + m_high_size_hysteresis = (5 * _size_limit)/4; // 20% more + m_high_size_watermark = (3 * _size_limit)/2; // 50% more + + m_size_reserved = unreservable_memory(_size_limit); + m_size_nonleaf = 0; + m_size_current = 0; + m_size_evicting = 0; + m_size_leaf = 0; + m_size_rollback = 0; + m_size_cachepressure = 0; + + m_pl = _pl; + m_kibbutz = _kibbutz; + toku_mutex_init(&m_ev_thread_lock, NULL); + toku_cond_init(&m_flow_control_cond, NULL); + toku_cond_init(&m_ev_thread_cond, NULL); + m_num_sleepers = 0; + m_ev_thread_is_running = false; + m_period_in_seconds = eviction_period; + + // start the background thread + m_run_thread = true; + m_num_eviction_thread_runs = 0; + int r = toku_pthread_create(&m_ev_thread, NULL, eviction_thread, this); + assert_zero(r); +} + +// +// This stops the eviction thread and clears the condition variable. +// +// NOTE: This should only be called if there are no evictions in progress. +// +void evictor::destroy() { + assert(m_size_evicting == 0); + + // Stop the eviction thread. + toku_mutex_lock(&m_ev_thread_lock); + m_run_thread = false; + this->signal_eviction_thread(); + toku_mutex_unlock(&m_ev_thread_lock); + + void *ret; + int r = toku_pthread_join(m_ev_thread, &ret); + assert_zero(r); + assert(!m_ev_thread_is_running); + + toku_cond_destroy(&m_flow_control_cond); + toku_cond_destroy(&m_ev_thread_cond); + toku_mutex_destroy(&m_ev_thread_lock); +} + +// +// Increases status variables and the current size variable +// of the evictor based on the given pair attribute. +// +void evictor::add_pair_attr(PAIR_ATTR attr) { + assert(attr.is_valid); + add_to_size_current(attr.size); + (void) __sync_fetch_and_add(&m_size_nonleaf, attr.nonleaf_size); + (void) __sync_fetch_and_add(&m_size_leaf, attr.leaf_size); + (void) __sync_fetch_and_add(&m_size_rollback, attr.rollback_size); + (void) __sync_fetch_and_add(&m_size_cachepressure, attr.cache_pressure_size); +} + +// +// Decreases status variables and the current size variable +// of the evictor based on the given pair attribute. +// +void evictor::remove_pair_attr(PAIR_ATTR attr) { + assert(attr.is_valid); + remove_from_size_current(attr.size); + (void) __sync_fetch_and_sub(&m_size_nonleaf, attr.nonleaf_size); + (void) __sync_fetch_and_sub(&m_size_leaf, attr.leaf_size); + (void) __sync_fetch_and_sub(&m_size_rollback, attr.rollback_size); + (void) __sync_fetch_and_sub(&m_size_cachepressure, attr.cache_pressure_size); + assert(m_size_current >= 0); +} + +// +// Updates this evictor's stats to match the "new" pair attribute given +// while also removing the given "old" pair attribute. +// +void evictor::change_pair_attr(PAIR_ATTR old_attr, PAIR_ATTR new_attr) { + this->add_pair_attr(new_attr); + this->remove_pair_attr(old_attr); +} + +// +// Adds the given size to the evictor's estimation of +// the size of the cachetable. +// +void evictor::add_to_size_current(long size) { + (void) __sync_fetch_and_add(&m_size_current, size); +} + +// +// Subtracts the given size from the evictor's current +// approximation of the cachetable size. +// +void evictor::remove_from_size_current(long size) { + (void) __sync_fetch_and_sub(&m_size_current, size); +} + +// +// TODO: (Zardosht) comment this function +// +uint64_t evictor::reserve_memory(double fraction) { + uint64_t reserved_memory = 0; + toku_mutex_lock(&m_ev_thread_lock); + reserved_memory = fraction * (m_low_size_watermark - m_size_reserved); + m_size_reserved += reserved_memory; + (void) __sync_fetch_and_add(&m_size_current, reserved_memory); + this->signal_eviction_thread(); + toku_mutex_unlock(&m_ev_thread_lock); + + if (this->should_client_thread_sleep()) { + this->wait_for_cache_pressure_to_subside(); + } + return reserved_memory; +} + +// +// TODO: (Zardosht) comment this function +// +void evictor::release_reserved_memory(uint64_t reserved_memory){ + (void) __sync_fetch_and_sub(&m_size_current, reserved_memory); + toku_mutex_lock(&m_ev_thread_lock); + m_size_reserved -= reserved_memory; + // signal the eviction thread in order to possibly wake up sleeping clients + if (m_num_sleepers > 0) { + this->signal_eviction_thread(); + } + toku_mutex_unlock(&m_ev_thread_lock); +} + +// +// This function is the eviction thread. It runs for the lifetime of +// the evictor. Goes to sleep for period_in_seconds +// by waiting on m_ev_thread_cond. +// +void evictor::run_eviction_thread(){ + toku_mutex_lock(&m_ev_thread_lock); + while (m_run_thread) { + m_num_eviction_thread_runs++; // for test purposes only + m_ev_thread_is_running = true; + // responsibility of run_eviction to release and + // regrab ev_thread_lock as it sees fit + this->run_eviction(); + m_ev_thread_is_running = false; + + if (m_run_thread) { + // + // sleep until either we are signaled + // via signal_eviction_thread or + // m_period_in_seconds amount of time has passed + // + if (m_period_in_seconds) { + toku_timespec_t wakeup_time; + struct timeval tv; + gettimeofday(&tv, 0); + wakeup_time.tv_sec = tv.tv_sec; + wakeup_time.tv_nsec = tv.tv_usec * 1000LL; + wakeup_time.tv_sec += m_period_in_seconds; + toku_cond_timedwait( + &m_ev_thread_cond, + &m_ev_thread_lock, + &wakeup_time + ); + } + // for test purposes, we have an option of + // not waiting on a period, but rather sleeping indefinitely + else { + toku_cond_wait(&m_ev_thread_cond, &m_ev_thread_lock); + } + } + } + toku_mutex_unlock(&m_ev_thread_lock); +} + +// +// runs eviction. +// on entry, ev_thread_lock is grabbed, on exit, ev_thread_lock must still be grabbed +// it is the responsibility of this function to release and reacquire ev_thread_lock as it sees fit. +// +void evictor::run_eviction(){ + // + // These variables will help us detect if everything in the clock is currently being accessed. + // We must detect this case otherwise we will end up in an infinite loop below. + // + CACHEKEY curr_cachekey; + curr_cachekey.b = INT64_MAX; // create initial value so compiler does not complain + FILENUM curr_filenum; + curr_filenum.fileid = UINT32_MAX; // create initial value so compiler does not complain + bool set_val = false; + bool exited_early = false; + + while (this->eviction_needed()) { + if (m_num_sleepers > 0 && this->should_sleeping_clients_wakeup()) { + toku_cond_broadcast(&m_flow_control_cond); + } + // release ev_thread_lock so that eviction may run without holding mutex + toku_mutex_unlock(&m_ev_thread_lock); + + m_pl->read_list_lock(); + PAIR curr_in_clock = m_pl->m_clock_head; + // if nothing to evict, we need to exit + if (!curr_in_clock) { + m_pl->read_list_unlock(); + toku_mutex_lock(&m_ev_thread_lock); + exited_early = true; + goto exit; + } + if (set_val && + curr_in_clock->key.b == curr_cachekey.b && + curr_in_clock->cachefile->filenum.fileid == curr_filenum.fileid) + { + // we have identified a cycle where everything in the clock is in use + // do not return an error + // just let memory be overfull + m_pl->read_list_unlock(); + toku_mutex_lock(&m_ev_thread_lock); + exited_early = true; + goto exit; + } + bool eviction_run = run_eviction_on_pair(curr_in_clock); + if (eviction_run) { + set_val = false; + } + else if (!set_val) { + set_val = true; + curr_cachekey = m_pl->m_clock_head->key; + curr_filenum = m_pl->m_clock_head->cachefile->filenum; + } + // at this point, either curr_in_clock is still in the list because it has not been fully evicted, + // and we need to move ct->m_clock_head over. Otherwise, curr_in_clock has been fully evicted + // and we do NOT need to move ct->m_clock_head, as the removal of curr_in_clock + // modified ct->m_clock_head + if (m_pl->m_clock_head && (m_pl->m_clock_head == curr_in_clock)) { + m_pl->m_clock_head = m_pl->m_clock_head->clock_next; + } + m_pl->read_list_unlock(); + + toku_mutex_lock(&m_ev_thread_lock); + } + +exit: + if (m_num_sleepers > 0 && (exited_early || this->should_sleeping_clients_wakeup())) { + toku_cond_broadcast(&m_flow_control_cond); + } + return; +} + +// +// NOTE: Cachetable lock held on entry. +// Runs eviction on the given PAIR. This may be a +// partial eviction or full eviction. +// +// on entry, pair mutex is NOT held, but pair list's read list lock +// IS held +// on exit, the same conditions must apply +// +bool evictor::run_eviction_on_pair(PAIR curr_in_clock) { + bool ret_val = false; + // function meant to be called on PAIR that is not being accessed right now + CACHEFILE cf = curr_in_clock->cachefile; + int r = bjm_add_background_job(cf->bjm); + if (r) { + goto exit; + } + pair_lock(curr_in_clock); + if (nb_mutex_users(&curr_in_clock->value_nb_mutex) || + nb_mutex_users(&curr_in_clock->disk_nb_mutex)) + { + pair_unlock(curr_in_clock); + bjm_remove_background_job(cf->bjm); + goto exit; + } + + // now that we have the pair mutex we care about, we can + // release the read list lock and reacquire it at the end of the function + m_pl->read_list_unlock(); + ret_val = true; + if (curr_in_clock->count > 0) { + curr_in_clock->count--; + // call the partial eviction callback + nb_mutex_lock(&curr_in_clock->value_nb_mutex, &curr_in_clock->mutex); + pair_unlock(curr_in_clock); + + void *value = curr_in_clock->value_data; + void* disk_data = curr_in_clock->disk_data; + void *write_extraargs = curr_in_clock->write_extraargs; + enum partial_eviction_cost cost; + long bytes_freed_estimate = 0; + curr_in_clock->pe_est_callback( + value, + disk_data, + &bytes_freed_estimate, + &cost, + write_extraargs + ); + if (cost == PE_CHEAP) { + curr_in_clock->size_evicting_estimate = 0; + this->do_partial_eviction(curr_in_clock); + bjm_remove_background_job(cf->bjm); + } + else if (cost == PE_EXPENSIVE) { + // only bother running an expensive partial eviction + // if it is expected to free space + if (bytes_freed_estimate > 0) { + curr_in_clock->size_evicting_estimate = bytes_freed_estimate; + toku_mutex_lock(&m_ev_thread_lock); + m_size_evicting += bytes_freed_estimate; + toku_mutex_unlock(&m_ev_thread_lock); + toku_kibbutz_enq( + m_kibbutz, + cachetable_partial_eviction, + curr_in_clock + ); + } + else { + pair_lock(curr_in_clock); + nb_mutex_unlock(&curr_in_clock->value_nb_mutex); + pair_unlock(curr_in_clock); + bjm_remove_background_job(cf->bjm); + } + } + else { + assert(false); + } + } + else { + // responsibility of try_evict_pair to eventually remove background job + // pair's mutex is still grabbed here + this->try_evict_pair(curr_in_clock); + } + // regrab the read list lock, because the caller assumes + // that it is held. The contract requires this. + m_pl->read_list_lock(); +exit: + return ret_val; +} + +// +// on entry, pair's mutex is not held, but pair is pinned +// on exit, PAIR is unpinned +// +void evictor::do_partial_eviction(PAIR p) { + PAIR_ATTR new_attr; + PAIR_ATTR old_attr = p->attr; + + p->pe_callback(p->value_data, old_attr, &new_attr, p->write_extraargs); + + this->change_pair_attr(old_attr, new_attr); + p->attr = new_attr; + this->decrease_size_evicting(p->size_evicting_estimate); + pair_lock(p); + nb_mutex_unlock(&p->value_nb_mutex); + pair_unlock(p); +} + +// +// CT lock held on entry +// background job has been added for p->cachefile on entry +// responsibility of this function to make sure that background job is removed +// +// on entry, pair's mutex is held, on exit, the pair's mutex is NOT held +// +void evictor::try_evict_pair(PAIR p) { + CACHEFILE cf = p->cachefile; + // evictions without a write or unpinned pair's that are clean + // can be run in the current thread + + // the only caller, run_eviction_on_pair, should call this function + // only if no one else is trying to use it + assert(!nb_mutex_users(&p->value_nb_mutex)); + nb_mutex_lock(&p->value_nb_mutex, &p->mutex); + // if the PAIR is dirty, the running eviction requires writing the + // PAIR out. if the disk_nb_mutex is grabbed, then running + // eviction requires waiting for the disk_nb_mutex to become available, + // which may be expensive. Hence, if either is true, we + // do the eviction on a writer thread + if (!p->dirty && (nb_mutex_writers(&p->disk_nb_mutex) == 0)) { + p->size_evicting_estimate = 0; + // + // This method will unpin PAIR and release PAIR mutex + // + // because the PAIR is not dirty, we can safely pass + // false for the for_checkpoint parameter + this->evict_pair(p, false); + bjm_remove_background_job(cf->bjm); + } + else { + pair_unlock(p); + toku_mutex_lock(&m_ev_thread_lock); + assert(m_size_evicting >= 0); + p->size_evicting_estimate = p->attr.size; + m_size_evicting += p->size_evicting_estimate; + assert(m_size_evicting >= 0); + toku_mutex_unlock(&m_ev_thread_lock); + toku_kibbutz_enq(m_kibbutz, cachetable_evicter, p); + } +} + +// +// Requires: This thread must hold the write lock (nb_mutex) for the pair. +// The pair's mutex (p->mutex) is also held. +// on exit, neither is held +// +void evictor::evict_pair(PAIR p, bool for_checkpoint) { + if (p->dirty) { + pair_unlock(p); + cachetable_write_locked_pair(this, p, for_checkpoint); + pair_lock(p); + } + // one thing we can do here is extract the size_evicting estimate, + // have decrease_size_evicting take the estimate and not the pair, + // and do this work after we have called + // cachetable_maybe_remove_and_free_pair + this->decrease_size_evicting(p->size_evicting_estimate); + // if we are to remove this pair, we need the write list lock, + // to get it in a way that avoids deadlocks, we must first release + // the pair's mutex, then grab the write list lock, then regrab the + // pair's mutex. The pair cannot go anywhere because + // the pair is still pinned + nb_mutex_lock(&p->disk_nb_mutex, &p->mutex); + pair_unlock(p); + m_pl->write_list_lock(); + pair_lock(p); + nb_mutex_unlock(&p->value_nb_mutex); + nb_mutex_unlock(&p->disk_nb_mutex); + // at this point, we have the pair list's write list lock + // and we have the pair's mutex (p->mutex) held + cachetable_maybe_remove_and_free_pair(m_pl, this, p); + m_pl->write_list_unlock(); +} + +// +// this function handles the responsibilities for writer threads when they +// decrease size_evicting. The responsibilities are: +// - decrease m_size_evicting in a thread safe manner +// - in some circumstances, signal the eviction thread +// +void evictor::decrease_size_evicting(long size_evicting_estimate) { + if (size_evicting_estimate > 0) { + toku_mutex_lock(&m_ev_thread_lock); + int64_t buffer = m_high_size_hysteresis - m_low_size_watermark; + // if size_evicting is transitioning from greater than buffer to below buffer, and + // some client threads are sleeping, we need to wake up the eviction thread. + // Here is why. In this scenario, we are in one of two cases: + // - size_current - size_evicting < low_size_watermark + // If this is true, then size_current < high_size_hysteresis, which + // means we need to wake up sleeping clients + // - size_current - size_evicting > low_size_watermark, + // which means more evictions must be run. + // The consequences of both cases are the responsibility + // of the eviction thread. + // + bool need_to_signal_ev_thread = + (m_num_sleepers > 0) && + !m_ev_thread_is_running && + (m_size_evicting > buffer) && + ((m_size_evicting - size_evicting_estimate) <= buffer); + m_size_evicting -= size_evicting_estimate; + assert(m_size_evicting >= 0); + if (need_to_signal_ev_thread) { + this->signal_eviction_thread(); + } + toku_mutex_unlock(&m_ev_thread_lock); + } +} + +// +// Wait for cache table space to become available +// size_current is number of bytes currently occupied by data (referred to by pairs) +// size_evicting is number of bytes queued up to be evicted +// +void evictor::wait_for_cache_pressure_to_subside() { + toku_mutex_lock(&m_ev_thread_lock); + m_num_sleepers++; + this->signal_eviction_thread(); + toku_cond_wait(&m_flow_control_cond, &m_ev_thread_lock); + m_num_sleepers--; + toku_mutex_unlock(&m_ev_thread_lock); + +} + +// +// Get the status of the current estimated size of the cachetable, +// and the evictor's set limit. +// +void evictor::get_state(long *size_current_ptr, long *size_limit_ptr) { + if (size_current_ptr) { + *size_current_ptr = m_size_current; + } + if (size_limit_ptr) { + *size_limit_ptr = m_low_size_watermark; + } +} + +// +// Force the eviction thread to do some work. +// +// This function does not require any mutex to be held. +// As a result, scheduling is not guaranteed, but that is tolerable. +// +void evictor::signal_eviction_thread() { + toku_cond_signal(&m_ev_thread_cond); +} + +// +// Returns true if the cachetable is so over subscribed, that a client thread should sleep +// +// This function may be called in a thread-unsafe manner. Locks are not +// required to read size_current. The result is that +// the values may be a little off, but we think that is tolerable. +// +bool evictor::should_client_thread_sleep(){ + return m_size_current > m_high_size_watermark; +} + +// +// Returns true if a sleeping client should be woken up because +// the cachetable is not overly subscribed +// +// This function may be called in a thread-unsafe manner. Locks are not +// required to read size_current. The result is that +// the values may be a little off, but we think that is tolerable. +// +bool evictor::should_sleeping_clients_wakeup() { + return m_size_current <= m_high_size_hysteresis; +} + +// +// Returns true if a client thread should try to wake up the eviction +// thread because the client thread has noticed too much data taken +// up in the cachetable. +// +// This function may be called in a thread-unsafe manner. Locks are not +// required to read size_current or size_evicting. The result is that +// the values may be a little off, but we think that is tolerable. +// If the caller wants to ensure that ev_thread_is_running and size_evicting +// are accurate, then the caller must hold ev_thread_lock before +// calling this function. +// +bool evictor::should_client_wake_eviction_thread() { + return + !m_ev_thread_is_running && + ((m_size_current - m_size_evicting) > m_low_size_hysteresis); +} + +// +// Determines if eviction is needed. If the current size of +// the cachetable exceeds the sum of our fixed size limit and +// the amount of data currently being evicted, then eviction is needed +// +bool evictor::eviction_needed() { + return (m_size_current - m_size_evicting) > m_low_size_watermark; +} + +void evictor::fill_engine_status() { + STATUS_VALUE(CT_SIZE_CURRENT) = m_size_current; + STATUS_VALUE(CT_SIZE_LIMIT) = m_low_size_hysteresis; + STATUS_VALUE(CT_SIZE_WRITING) = m_size_evicting; + STATUS_VALUE(CT_SIZE_NONLEAF) = m_size_nonleaf; + STATUS_VALUE(CT_SIZE_LEAF) = m_size_leaf; + STATUS_VALUE(CT_SIZE_ROLLBACK) = m_size_rollback; + STATUS_VALUE(CT_SIZE_CACHEPRESSURE) = m_size_cachepressure; +} + +//////////////////////////////////////////////////////////////////////////////// + +static_assert(std::is_pod::value, "checkpointer isn't POD"); + +// +// Sets the cachetable reference in this checkpointer class, this is temporary. +// +void checkpointer::init(CACHETABLE _ct, + TOKULOGGER _logger, + cachefile_list *files) { + m_ct = _ct; + m_logger = _logger; + m_cf_list = files; + bjm_init(&m_checkpoint_clones_bjm); + + // Default is no checkpointing. + toku_minicron_setup(&m_checkpointer_cron, 0, checkpoint_thread, this); +} + +void checkpointer::destroy() { + if (!this->has_been_shutdown()) { + // for test code only, production code uses toku_cachetable_minicron_shutdown() + int r = this->shutdown(); + assert(r == 0); + } + bjm_destroy(m_checkpoint_clones_bjm); +} + +// +// Sets how often the checkpoint thread will run. +// +int checkpointer::set_checkpoint_period(uint32_t new_period) { + return toku_minicron_change_period(&m_checkpointer_cron, new_period); +} + +// +// Sets how often the checkpoint thread will run. +// +uint32_t checkpointer::get_checkpoint_period() { + return toku_minicron_get_period(&m_checkpointer_cron); +} + +// +// Stops the checkpoint thread. +// +int checkpointer::shutdown() { + return toku_minicron_shutdown(&m_checkpointer_cron); +} + +// +// If checkpointing is running, this returns false. +// +bool checkpointer::has_been_shutdown() { + return toku_minicron_has_been_shutdown(&m_checkpointer_cron); +} + +TOKULOGGER checkpointer::get_logger() { + return m_logger; +} + +void checkpointer::increment_num_txns() { + m_checkpoint_num_txns++; +} + +// +// Update the user data in any cachefiles in our checkpoint list. +// +void checkpointer::update_cachefiles() { + int r = 0; + CACHEFILE cf; + for(cf = m_cf_list->m_head; cf; cf=cf->next) { + if (cf->for_checkpoint && cf->begin_checkpoint_userdata) { + r = cf->begin_checkpoint_userdata(m_lsn_of_checkpoint_in_progress, + cf->userdata); + assert(r == 0); + } + } +} + +// +// Sets up and kicks off a checkpoint. +// +int checkpointer::begin_checkpoint() { + // 1. Initialize the accountability counters. + int r = 0; + m_checkpoint_num_files = 0; + m_checkpoint_num_txns = 0; + + // 2. Make list of cachefiles to be included in the checkpoint. + // TODO: How do we remove the non-lock cachetable reference here? + m_cf_list->read_lock(); + for (CACHEFILE cf = m_cf_list->m_head; cf; cf = cf->next) { + // The caller must serialize open, close, and begin checkpoint. + // So we should never see a closing cachefile here. + // Is there an assert we can add here? + + // Putting this check here so that this method may be called + // by cachetable tests. + if (cf->note_pin_by_checkpoint) { + r = cf->note_pin_by_checkpoint(cf, cf->userdata); + assert(r == 0); + } + cf->for_checkpoint = true; + m_checkpoint_num_files++; + } + m_cf_list->read_unlock(); + + // 3. Create log entries for this checkpoint. + if (m_logger) { + this->log_begin_checkpoint(); + } + + bjm_reset(m_checkpoint_clones_bjm); + + m_ct->list.write_pending_exp_lock(); + m_ct->list.read_list_lock(); + m_cf_list->read_lock(); // needed for update_cachefiles + m_ct->list.write_pending_cheap_lock(); + // 4. Turn on all the relevant checkpoint pending bits. + this->turn_on_pending_bits(); + + // 5. + this->update_cachefiles(); + m_ct->list.write_pending_cheap_unlock(); + m_cf_list->read_unlock(); + m_ct->list.read_list_unlock(); + m_ct->list.write_pending_exp_unlock(); + return r; +} + +// +// Assuming the logger exists, this will write out the folloing +// information to the log. +// +// 1. Writes the BEGIN_CHECKPOINT to the log. +// 2. Writes the list of open dictionaries to the log. +// 3. Writes the list of open transactions to the log. +// 4. Writes the list of dicionaries that have had rollback logs suppresed. +// +// NOTE: This also has the side effecto of setting the LSN +// of checkpoint in progress. +// +void checkpointer::log_begin_checkpoint() { + int r = 0; + + // Write the BEGIN_CHECKPOINT to the log. + LSN begin_lsn={ .lsn = (uint64_t) -1 }; // we'll need to store the lsn of the checkpoint begin in all the trees that are checkpointed. + TXN_MANAGER mgr = toku_logger_get_txn_manager(m_logger); + TXNID last_xid = toku_txn_manager_get_last_xid(mgr); + r = toku_log_begin_checkpoint(m_logger, &begin_lsn, 0, 0, last_xid); + assert(r==0); + m_lsn_of_checkpoint_in_progress = begin_lsn; + + // Log the list of open dictionaries. + for (CACHEFILE cf = m_cf_list->m_head; cf; cf = cf->next) { + if (cf->log_fassociate_during_checkpoint) { + r = cf->log_fassociate_during_checkpoint(cf, cf->userdata); + assert(r == 0); + } + } + + // Write open transactions to the log. + r = toku_txn_manager_iter_over_live_txns ( + m_logger->txn_manager, + &m_ct); + assert(r == 0); + + // Writes list of dictionaries that have had + // rollback logs suppressed. + for (CACHEFILE cf = m_cf_list->m_head; cf; cf = cf->next) { + if (cf->log_suppress_rollback_during_checkpoint) { + r = cf->log_suppress_rollback_during_checkpoint(cf, cf->userdata); + assert(r == 0); + } + } +} + +// +// Sets the pending bits of EVERY PAIR in the cachetable, regardless of +// whether the PAIR is clean or not. It will be the responsibility of +// end_checkpoint or client threads to simply clear the pending bit +// if the PAIR is clean. +// +// On entry and exit , the pair list's read list lock is grabbed, and +// both pending locks are grabbed +// +void checkpointer::turn_on_pending_bits() { + for (uint32_t i = 0; i < m_ct->list.m_table_size; i++) { + PAIR p; + for (p = m_ct->list.m_table[i]; p; p = p->hash_chain) { + assert(!p->checkpoint_pending); + //Only include pairs belonging to cachefiles in the checkpoint + if (!p->cachefile->for_checkpoint) { + continue; + } + // Mark everything as pending a checkpoint + // + // The rule for the checkpoint_pending bit is as follows: + // - begin_checkpoint may set checkpoint_pending to true + // even though the pair lock on the node is not held. + // - any thread that wants to clear the pending bit must own + // the PAIR lock. Otherwise, + // we may end up clearing the pending bit before the + // current lock is ever released. + p->checkpoint_pending = true; + if (m_ct->list.m_pending_head) { + m_ct->list.m_pending_head->pending_prev = p; + } + p->pending_next = m_ct->list.m_pending_head; + p->pending_prev = NULL; + m_ct->list.m_pending_head = p; + } + } +} + +void checkpointer::add_background_job() { + int r = bjm_add_background_job(m_checkpoint_clones_bjm); + assert_zero(r); +} +void checkpointer::remove_background_job() { + bjm_remove_background_job(m_checkpoint_clones_bjm); +} + +int checkpointer::end_checkpoint(void (*testcallback_f)(void*), void* testextra) { + int r = 0; + CACHEFILE *XMALLOC_N(m_checkpoint_num_files, checkpoint_cfs); + + this->fill_checkpoint_cfs(checkpoint_cfs); + this->checkpoint_pending_pairs(); + this->checkpoint_userdata(checkpoint_cfs); + // For testing purposes only. Dictionary has been fsync-ed to disk but log has not yet been written. + if (testcallback_f) { + testcallback_f(testextra); + } + this->log_end_checkpoint(); + this->end_checkpoint_userdata(checkpoint_cfs); + + //Delete list of cachefiles in the checkpoint, + r = this->remove_cachefiles(checkpoint_cfs); + toku_free(checkpoint_cfs); + return r; +} + +void checkpointer::fill_checkpoint_cfs(CACHEFILE* checkpoint_cfs) { + m_cf_list->read_lock(); + uint32_t curr_index = 0; + for (CACHEFILE cf = m_cf_list->m_head; cf; cf = cf->next) { + if (cf->for_checkpoint) { + assert(curr_index < m_checkpoint_num_files); + checkpoint_cfs[curr_index] = cf; + curr_index++; + } + } + assert(curr_index == m_checkpoint_num_files); + m_cf_list->read_unlock(); +} + +void checkpointer::checkpoint_pending_pairs() { + PAIR p; + m_ct->list.read_list_lock(); + while ((p = m_ct->list.m_pending_head)!=0) { + // TODO: Investigate why we move pending head outisde of the pending_pairs_remove() call. + m_ct->list.m_pending_head = m_ct->list.m_pending_head->pending_next; + m_ct->list.pending_pairs_remove(p); + // if still pending, clear the pending bit and write out the node + pair_lock(p); + m_ct->list.read_list_unlock(); + write_pair_for_checkpoint_thread(&m_ct->ev, p); + pair_unlock(p); + m_ct->list.read_list_lock(); + } + assert(!m_ct->list.m_pending_head); + m_ct->list.read_list_unlock(); + bjm_wait_for_jobs_to_finish(m_checkpoint_clones_bjm); +} + +void checkpointer::checkpoint_userdata(CACHEFILE* checkpoint_cfs) { + // have just written data blocks, so next write the translation and header for each open dictionary + for (uint32_t i = 0; i < m_checkpoint_num_files; i++) { + CACHEFILE cf = checkpoint_cfs[i]; + assert(cf->for_checkpoint); + if (cf->checkpoint_userdata) { + toku_cachetable_set_checkpointing_user_data_status(1); + int r = cf->checkpoint_userdata(cf, cf->fd, cf->userdata); + toku_cachetable_set_checkpointing_user_data_status(0); + assert(r==0); + } + } +} + +void checkpointer::log_end_checkpoint() { + if (m_logger) { + int r = toku_log_end_checkpoint(m_logger, NULL, + 1, // want the end_checkpoint to be fsync'd + m_lsn_of_checkpoint_in_progress, + 0, + m_checkpoint_num_files, + m_checkpoint_num_txns); + assert(r==0); + toku_logger_note_checkpoint(m_logger, m_lsn_of_checkpoint_in_progress); + } +} + +void checkpointer::end_checkpoint_userdata(CACHEFILE* checkpoint_cfs) { + // everything has been written to file and fsynced + // ... call checkpoint-end function in block translator + // to free obsolete blocks on disk used by previous checkpoint + //cachefiles_in_checkpoint is protected by the checkpoint_safe_lock + for (uint32_t i = 0; i < m_checkpoint_num_files; i++) { + CACHEFILE cf = checkpoint_cfs[i]; + assert(cf->for_checkpoint); + if (cf->end_checkpoint_userdata) { + int r = cf->end_checkpoint_userdata(cf, cf->fd, cf->userdata); + assert(r==0); + } + } +} + +// +// Deletes all the cachefiles in this checkpointers cachefile list. +// +int checkpointer::remove_cachefiles(CACHEFILE* checkpoint_cfs) { + int r = 0; + // making this a while loop because note_unpin_by_checkpoint may destroy the cachefile + for (uint32_t i = 0; i < m_checkpoint_num_files; i++) { + CACHEFILE cf = checkpoint_cfs[i]; + // Checking for function existing so that this function + // can be called from cachetable tests. + assert(cf->for_checkpoint); + cf->for_checkpoint = false; + if (cf->note_unpin_by_checkpoint) { + // Clear the bit saying theis file is in the checkpoint. + r = cf->note_unpin_by_checkpoint(cf, cf->userdata); + if (r != 0) { + return r; + } + } + } + return r; +} + + +//////////////////////////////////////////////////////// +// +// cachefiles list +// +static_assert(std::is_pod::value, "cachefile_list isn't POD"); + +void cachefile_list::init() { + m_head = NULL; + m_next_filenum_to_use.fileid = 0; + toku_pthread_rwlock_init(&m_lock, NULL); +} + +void cachefile_list::destroy() { + toku_pthread_rwlock_destroy(&m_lock); +} + +void cachefile_list::read_lock() { + toku_pthread_rwlock_rdlock(&m_lock); +} + +void cachefile_list::read_unlock() { + toku_pthread_rwlock_rdunlock(&m_lock); +} + +void cachefile_list::write_lock() { + toku_pthread_rwlock_wrlock(&m_lock); +} + +void cachefile_list::write_unlock() { + toku_pthread_rwlock_wrunlock(&m_lock); +} + void __attribute__((__constructor__)) toku_cachetable_helgrind_ignore(void); void toku_cachetable_helgrind_ignore(void) { diff --git a/ft/cachetable.h b/ft/cachetable.h index 3c1642e01fb..016d3510c09 100644 --- a/ft/cachetable.h +++ b/ft/cachetable.h @@ -9,6 +9,7 @@ #include #include "fttypes.h" +#include "minicron.h" #include "workqueue.h" @@ -27,7 +28,6 @@ typedef BLOCKNUM CACHEKEY; - int toku_set_cleaner_period (CACHETABLE ct, uint32_t new_period); uint32_t toku_get_cleaner_period (CACHETABLE ct); uint32_t toku_get_cleaner_period_unlocked (CACHETABLE ct); @@ -50,6 +50,9 @@ int toku_create_cachetable(CACHETABLE *result, long size_limit, LSN initial_lsn, // Returns: If success, returns 0 and result points to the new cachetable. Otherwise, // returns an error number. +// Returns a pointer to the checkpointer within the given cachetable. +CHECKPOINTER toku_cachetable_get_checkpointer(CACHETABLE ct); + // What is the cachefile that goes with a particular filenum? // During a transaction, we cannot reuse a filenum. int toku_cachefile_of_filenum (CACHETABLE t, FILENUM filenum, CACHEFILE *cf); @@ -63,8 +66,8 @@ int toku_cachefile_of_iname_in_env (CACHETABLE ct, const char *iname_in_env, CAC char * toku_cachefile_fname_in_cwd (CACHEFILE cf); // TODO: #1510 Add comments on how these behave -int toku_cachetable_begin_checkpoint (CACHETABLE ct, TOKULOGGER); -int toku_cachetable_end_checkpoint(CACHETABLE ct, TOKULOGGER logger, +int toku_cachetable_begin_checkpoint (CHECKPOINTER cp, TOKULOGGER); +int toku_cachetable_end_checkpoint(CHECKPOINTER cp, TOKULOGGER logger, void (*testcallback_f)(void*), void * testextra); // Shuts down checkpoint thread @@ -78,9 +81,6 @@ int toku_cachetable_close (CACHETABLE*); /* Flushes everything to disk, and dest // Open a file and bind the file to a new cachefile object. (For use by test programs only.) int toku_cachetable_openf (CACHEFILE *,CACHETABLE, const char */*fname_in_env*/, int flags, mode_t mode); -// Returns the limit on the cachetable size -uint64_t toku_cachetable_get_size_limit(CACHETABLE ct); - // Bind a file to a new cachefile object. int toku_cachetable_openfd (CACHEFILE *,CACHETABLE, int /*fd*/, const char *fname_relative_to_env); /*(used for logging)*/ @@ -128,7 +128,7 @@ typedef void (*CACHETABLE_FLUSH_CALLBACK)(CACHEFILE, int fd, CACHEKEY key, void // Returns: 0 if success, otherwise an error number. The address and size of the object // associated with the key are returned. // Can access fd (fd is protected by a readlock during call) -typedef int (*CACHETABLE_FETCH_CALLBACK)(CACHEFILE, int fd, CACHEKEY key, uint32_t fullhash, void **value_data, void **disk_data, PAIR_ATTR *sizep, int *dirtyp, void *read_extraargs); +typedef int (*CACHETABLE_FETCH_CALLBACK)(CACHEFILE, PAIR p, int fd, CACHEKEY key, uint32_t fullhash, void **value_data, void **disk_data, PAIR_ATTR *sizep, int *dirtyp, void *read_extraargs); // The cachetable calls the partial eviction estimate callback to determine if // partial eviction is a cheap operation that may be called by on the client thread @@ -166,6 +166,11 @@ typedef bool (*CACHETABLE_PARTIAL_FETCH_REQUIRED_CALLBACK)(void *ftnode_pv, void // Returns: 0 if success, otherwise an error number. typedef int (*CACHETABLE_PARTIAL_FETCH_CALLBACK)(void *value_data, void* disk_data, void *read_extraargs, int fd, PAIR_ATTR *sizep); +// The cachetable calls the put callback during a cachetable_put command to provide the opaque PAIR. +// The PAIR can then be used to later unpin the pair. +// Returns: 0 if success, otherwise an error number. +typedef void (*CACHETABLE_PUT_CALLBACK)(void *value_data, PAIR p); + // TODO(leif) XXX TODO XXX typedef int (*CACHETABLE_CLEANER_CALLBACK)(void *ftnode_pv, BLOCKNUM blocknum, uint32_t fullhash, void *write_extraargs); @@ -208,9 +213,9 @@ CACHETABLE toku_cachefile_get_cachetable(CACHEFILE cf); // put something into the cachetable and checkpoint dependent pairs // if the checkpointing is necessary int toku_cachetable_put_with_dep_pairs( - CACHEFILE cachefile, + CACHEFILE cachefile, CACHETABLE_GET_KEY_AND_FULLHASH get_key_and_fullhash, - void*value, + void*value, PAIR_ATTR attr, CACHETABLE_WRITE_CALLBACK write_callback, void *get_key_and_fullhash_extra, @@ -220,10 +225,10 @@ int toku_cachetable_put_with_dep_pairs( uint32_t* dependent_fullhash, //array of fullhashes of dependent pairs enum cachetable_dirty* dependent_dirty, // array stating dirty/cleanness of dependent pairs CACHEKEY* key, - uint32_t* fullhash + uint32_t* fullhash, + CACHETABLE_PUT_CALLBACK put_callback ); - // Put a memory object into the cachetable. // Effects: Lookup the key in the cachetable. If the key is not in the cachetable, // then insert the pair and pin it. Otherwise return an error. Some of the key @@ -232,11 +237,11 @@ int toku_cachetable_put_with_dep_pairs( // error number. int toku_cachetable_put(CACHEFILE cf, CACHEKEY key, uint32_t fullhash, void *value, PAIR_ATTR size, - CACHETABLE_WRITE_CALLBACK write_callback + CACHETABLE_WRITE_CALLBACK write_callback, + CACHETABLE_PUT_CALLBACK put_callback ); - -// Get and pin the memory object of a PAIR, and write dependent pairs to disk +// Get and pin the memory object of a PAIR, and write dependent pairs to disk // if the dependent pairs are pending a checkpoint. // Effects: If the memory object is in the cachetable, acquire a PAIR lock on it. // Otherwise, fetch it from storage by calling the fetch callback. If the fetch @@ -246,14 +251,40 @@ int toku_cachetable_put(CACHEFILE cf, CACHEKEY key, uint32_t fullhash, // then the required PAIRs are written to disk for checkpoint. // KEY PROPERTY OF DEPENDENT PAIRS: They are already locked by the client // Returns: 0 if the memory object is in memory, otherwise an error number. -int toku_cachetable_get_and_pin_with_dep_pairs ( - CACHEFILE cachefile, - CACHEKEY key, - uint32_t fullhash, - void**value, +// Requires: toku_cachetable_begin_batched_pin must have been called before entering this function. +// Requires: toku_cachetable_end_batched_pin must be called after this function. +// Rationale: +// begin_batched_pin and end_batched_pin take and release a read lock on the pair list. +// Normally, that would be done within this get_and_pin, but we want to pin multiple nodes with a single acquisition of the read lock. +int toku_cachetable_get_and_pin_with_dep_pairs_batched ( + CACHEFILE cachefile, + CACHEKEY key, + uint32_t fullhash, + void**value, long *sizep, CACHETABLE_WRITE_CALLBACK write_callback, - CACHETABLE_FETCH_CALLBACK fetch_callback, + CACHETABLE_FETCH_CALLBACK fetch_callback, + CACHETABLE_PARTIAL_FETCH_REQUIRED_CALLBACK pf_req_callback, + CACHETABLE_PARTIAL_FETCH_CALLBACK pf_callback, + bool may_modify_value, + void* read_extraargs, // parameter for fetch_callback, pf_req_callback, and pf_callback + uint32_t num_dependent_pairs, // number of dependent pairs that we may need to checkpoint + CACHEFILE* dependent_cfs, // array of cachefiles of dependent pairs + CACHEKEY* dependent_keys, // array of cachekeys of dependent pairs + uint32_t* dependent_fullhash, //array of fullhashes of dependent pairs + enum cachetable_dirty* dependent_dirty // array stating dirty/cleanness of dependent pairs + ); + +// Effect: call toku_cachetable_get_and_pin_with_dep_pairs_batched once, +// wrapped in begin_batched_pin and end_batched_pin calls. +int toku_cachetable_get_and_pin_with_dep_pairs ( + CACHEFILE cachefile, + CACHEKEY key, + uint32_t fullhash, + void**value, + long *sizep, + CACHETABLE_WRITE_CALLBACK write_callback, + CACHETABLE_FETCH_CALLBACK fetch_callback, CACHETABLE_PARTIAL_FETCH_REQUIRED_CALLBACK pf_req_callback, CACHETABLE_PARTIAL_FETCH_CALLBACK pf_callback, bool may_modify_value, @@ -302,17 +333,44 @@ struct unlockers { UNLOCKERS next; }; -// Effect: If the block is in the cachetable, then return it. -// Otherwise call the release_lock_callback, call the functions in unlockers, fetch the data (but don't pin it, since we'll just end up pinning it again later), -// and return TOKU_DB_TRYAGAIN. -int toku_cachetable_get_and_pin_nonblocking ( - CACHEFILE cf, - CACHEKEY key, - uint32_t fullhash, - void**value, +// Effect: Makes necessary preparations (grabs locks) for pinning multiple nodes. +void toku_cachetable_begin_batched_pin(CACHEFILE cf); + +// Effect: Clean up (release locks) after pinning multiple nodes. +void toku_cachetable_end_batched_pin(CACHEFILE cf); + +// Effect: If the block is in the cachetable, then return it. +// Otherwise call the functions in unlockers, fetch the data (but don't pin it, since we'll just end up pinning it again later), and return TOKUDB_TRY_AGAIN. +// Requires: toku_cachetable_begin_batched_pin must have been called before entering this function. +// Requires: toku_cachetable_end_batched_pin must be called after this function. +// Rationale: +// begin_batched_pin and end_batched_pin take and release a read lock on the pair list. +// Normally, that would be done within this get_and_pin, but we want to pin multiple nodes with a single acquisition of the read lock. +int toku_cachetable_get_and_pin_nonblocking_batched ( + CACHEFILE cf, + CACHEKEY key, + uint32_t fullhash, + void**value, long *sizep, CACHETABLE_WRITE_CALLBACK write_callback, - CACHETABLE_FETCH_CALLBACK fetch_callback, + CACHETABLE_FETCH_CALLBACK fetch_callback, + CACHETABLE_PARTIAL_FETCH_REQUIRED_CALLBACK pf_req_callback __attribute__((unused)), + CACHETABLE_PARTIAL_FETCH_CALLBACK pf_callback __attribute__((unused)), + bool may_modify_value, + void *read_extraargs, // parameter for fetch_callback, pf_req_callback, and pf_callback + UNLOCKERS unlockers + ); + +// Effect: call toku_cachetable_get_and_pin_nonblocking_batched once, +// wrapped in begin_batched_pin and end_batched_pin calls. +int toku_cachetable_get_and_pin_nonblocking ( + CACHEFILE cf, + CACHEKEY key, + uint32_t fullhash, + void**value, + long *sizep, + CACHETABLE_WRITE_CALLBACK write_callback, + CACHETABLE_FETCH_CALLBACK fetch_callback, CACHETABLE_PARTIAL_FETCH_REQUIRED_CALLBACK pf_req_callback __attribute__((unused)), CACHETABLE_PARTIAL_FETCH_CALLBACK pf_callback __attribute__((unused)), bool may_modify_value, @@ -332,21 +390,30 @@ int toku_cachetable_maybe_get_and_pin (CACHEFILE, CACHEKEY, uint32_t /*fullhash* int toku_cachetable_maybe_get_and_pin_clean (CACHEFILE, CACHEKEY, uint32_t /*fullhash*/, void**); // Effect: Like maybe get and pin, but may pin a clean pair. -int toku_cachetable_unpin(CACHEFILE, CACHEKEY, uint32_t fullhash, enum cachetable_dirty dirty, PAIR_ATTR size); +int toku_cachetable_unpin(CACHEFILE, PAIR, enum cachetable_dirty dirty, PAIR_ATTR size); // Effect: Unpin a memory object // Modifies: If the memory object is in the cachetable, then OR the dirty flag, // update the size, and release the read lock on the memory object. // Returns: 0 if success, otherwise returns an error number. // Requires: The ct is locked. -int toku_cachetable_unpin_ct_prelocked_no_flush(CACHEFILE, CACHEKEY, uint32_t fullhash, enum cachetable_dirty dirty, PAIR_ATTR size); +int toku_cachetable_unpin_ct_prelocked_no_flush(CACHEFILE, PAIR, enum cachetable_dirty dirty, PAIR_ATTR size); // Effect: The same as tokud_cachetable_unpin, except that the ct must not be locked. // Requires: The ct is NOT locked. -int toku_cachetable_unpin_and_remove (CACHEFILE, CACHEKEY, CACHETABLE_REMOVE_KEY, void*); /* Removing something already present is OK. */ +int toku_cachetable_unpin_and_remove (CACHEFILE, PAIR, CACHETABLE_REMOVE_KEY, void*); /* Removing something already present is OK. */ // Effect: Remove an object from the cachetable. Don't write it back. // Requires: The object must be pinned exactly once. +// test-only wrapper that use CACHEKEY and fullhash +int toku_test_cachetable_unpin(CACHEFILE, CACHEKEY, uint32_t fullhash, enum cachetable_dirty dirty, PAIR_ATTR size); + +// test-only wrapper that use CACHEKEY and fullhash +int toku_test_cachetable_unpin_ct_prelocked_no_flush(CACHEFILE, CACHEKEY, uint32_t fullhash, enum cachetable_dirty dirty, PAIR_ATTR size); + +// test-only wrapper that use CACHEKEY +int toku_test_cachetable_unpin_and_remove (CACHEFILE, CACHEKEY, CACHETABLE_REMOVE_KEY, void*); /* Removing something already present is OK. */ + int toku_cachefile_prefetch(CACHEFILE cf, CACHEKEY key, uint32_t fullhash, CACHETABLE_WRITE_CALLBACK write_callback, CACHETABLE_FETCH_CALLBACK fetch_callback, @@ -377,11 +444,6 @@ int toku_cachetable_assert_all_unpinned (CACHETABLE); int toku_cachefile_count_pinned (CACHEFILE, int /*printthem*/ ); -// Rename whatever is at oldkey to be newkey. Requires that the object be pinned. -int toku_cachetable_rename (CACHEFILE cachefile, CACHEKEY oldkey, CACHEKEY newkey); - -//int cachetable_fsync_all (CACHETABLE); /* Flush everything to disk, but keep it in cache. */ - // Close the cachefile. // Effects: All of the cached object associated with the cachefile are evicted from // the cachetable. The flush callback is called for each of these objects. The @@ -504,14 +566,9 @@ void remove_background_job_from_cf (CACHEFILE cf); // test-only function extern int toku_cachetable_get_checkpointing_user_data_status(void); -int -toku_cleaner_thread (void *cachetable_v); +// test-only function +int toku_cleaner_thread_for_test (CACHETABLE ct); +int toku_cleaner_thread (void *cleaner_v); -void toku_cachetables_init (void); -// Effect: Initialize the cachetables module. CDall this before calling any other cachetable operations. - -void toku_cachetables_destroy (void); -// Effect: Deinitialize the cachetables module. CDall this after calling any other cachetable operations to free resources that may have been allocated. -// To use the cachetable module again, call toku_cachetables_init() again. #endif diff --git a/ft/checkpoint.cc b/ft/checkpoint.cc index a92d57202a8..371bfbde4e6 100644 --- a/ft/checkpoint.cc +++ b/ft/checkpoint.cc @@ -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) { diff --git a/ft/checkpoint.h b/ft/checkpoint.h index 0b476694945..198bdc2749d 100644 --- a/ft/checkpoint.h +++ b/ft/checkpoint.h @@ -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); diff --git a/ft/compress.cc b/ft/compress.cc index 735a242c86f..1cd9c4d06b3 100644 --- a/ft/compress.cc +++ b/ft/compress.cc @@ -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); diff --git a/ft/ft-cachetable-wrappers.cc b/ft/ft-cachetable-wrappers.cc index 2f0b89b7c3a..63e6a366d83 100644 --- a/ft/ft-cachetable-wrappers.cc +++ b/ft/ft-cachetable-wrappers.cc @@ -9,6 +9,7 @@ #include #include #include +#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() ); diff --git a/ft/ft-cachetable-wrappers.h b/ft/ft-cachetable-wrappers.h index 5f0a2c08d17..b6158015bc1 100644 --- a/ft/ft-cachetable-wrappers.h +++ b/ft/ft-cachetable-wrappers.h @@ -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. diff --git a/ft/ft-flusher.cc b/ft/ft-flusher.cc index aa2aa411e7e..4affd49365a 100644 --- a/ft/ft-flusher.cc +++ b/ft/ft-flusher.cc @@ -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 ); diff --git a/ft/ft-internal.h b/ft/ft-internal.h index 5d10cae7b26..4649d52135d 100644 --- a/ft/ft-internal.h +++ b/ft/ft-internal.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); diff --git a/ft/ft-ops.cc b/ft/ft-ops.cc index d1ad0db7713..eb164c26580 100644 --- a/ft/ft-ops.cc +++ b/ft/ft-ops.cc @@ -764,7 +764,7 @@ toku_ft_status_update_pivot_fetch_reason(struct ftnode_fetch_extra *bfe) } } -int toku_ftnode_fetch_callback (CACHEFILE UU(cachefile), int fd, BLOCKNUM nodename, uint32_t fullhash, +int toku_ftnode_fetch_callback (CACHEFILE UU(cachefile), PAIR p, int fd, BLOCKNUM nodename, uint32_t fullhash, void **ftnode_pv, void** disk_data, PAIR_ATTR *sizep, int *dirtyp, void *extraargs) { assert(extraargs); assert(*ftnode_pv == NULL); @@ -789,6 +789,7 @@ int toku_ftnode_fetch_callback (CACHEFILE UU(cachefile), int fd, BLOCKNUM nodena if (r == 0) { *sizep = make_ftnode_pair_attr(*node); + (*node)->ct_pair = p; *dirtyp = (*node)->dirty; // deserialize could mark the node as dirty (presumably for upgrade) } return r; @@ -1309,7 +1310,7 @@ ft_init_new_root(FT ft, FTNODE nodea, FTNODE nodeb, DBT splitk, CACHEKEY *rootp, //printf("%s:%d put %lld\n", __FILE__, __LINE__, newroot_diskoff); uint32_t fullhash = toku_cachetable_hash(ft->cf, newroot_diskoff); newroot->fullhash = fullhash; - toku_cachetable_put(ft->cf, newroot_diskoff, fullhash, newroot, make_ftnode_pair_attr(newroot), get_write_callbacks_for_node(ft)); + toku_cachetable_put(ft->cf, newroot_diskoff, fullhash, newroot, make_ftnode_pair_attr(newroot), get_write_callbacks_for_node(ft), toku_node_save_ct_pair); //at this point, newroot is associated with newroot_diskoff, nodea is associated with root_blocknum // make newroot_diskoff point to nodea @@ -4287,9 +4288,9 @@ ft_search_node ( #if TOKU_DO_PREFETCH static int -ftnode_fetch_callback_and_free_bfe(CACHEFILE cf, int fd, BLOCKNUM nodename, uint32_t fullhash, void **ftnode_pv, void** UU(disk_data), PAIR_ATTR *sizep, int *dirtyp, void *extraargs) +ftnode_fetch_callback_and_free_bfe(CACHEFILE cf, PAIR p, int fd, BLOCKNUM nodename, uint32_t fullhash, void **ftnode_pv, void** UU(disk_data), PAIR_ATTR *sizep, int *dirtyp, void *extraargs) { - int r = toku_ftnode_fetch_callback(cf, fd, nodename, fullhash, ftnode_pv, disk_data, sizep, dirtyp, extraargs); + int r = toku_ftnode_fetch_callback(cf, p, fd, nodename, fullhash, ftnode_pv, disk_data, sizep, dirtyp, extraargs); struct ftnode_fetch_extra *CAST_FROM_VOIDP(ffe, extraargs); destroy_bfe_for_prefetch(ffe); toku_free(ffe); @@ -4355,8 +4356,7 @@ unlock_ftnode_fun (void *v) { // CT lock is held int r = toku_cachetable_unpin_ct_prelocked_no_flush( brt->ft->cf, - node->thisnodename, - node->fullhash, + node->ct_pair, (enum cachetable_dirty) node->dirty, x->msgs_applied ? make_ftnode_pair_attr(node) : make_invalid_pair_attr() ); @@ -4388,16 +4388,24 @@ ft_search_child(FT_HANDLE brt, FTNODE node, int childnum, ft_search_t *search, F ); bool msgs_applied = false; { - int rr = toku_pin_ftnode(brt, childblocknum, fullhash, - unlockers, - &next_ancestors, bounds, - &bfe, - (node->height == 1), // may_modify_node true iff child is leaf - true, - &childnode, - &msgs_applied); - if (rr==TOKUDB_TRY_AGAIN) return rr; - assert(rr==0); + int rr = toku_pin_ftnode_batched(brt, childblocknum, fullhash, + unlockers, + &next_ancestors, bounds, + &bfe, + (node->height == 1), // may_modify_node true iff child is leaf + true, + (node->height == 1), // end_batch_on_success true iff child is a leaf + &childnode, + &msgs_applied); + if (rr==TOKUDB_TRY_AGAIN) { + // We're going to try again, so we aren't pinning any more + // nodes in this batch. + toku_cachetable_end_batched_pin(brt->ft->cf); + return rr; + } + // We end the batch before applying ancestor messages if we get + // all the way to a leaf. + invariant_zero(rr); } struct unlock_ftnode_extra unlock_extra = {brt,childnode,msgs_applied}; @@ -4565,13 +4573,18 @@ ft_search_node( struct pivot_bounds const * const bounds, bool can_bulk_fetch ) -{ int r = 0; +{ + int r = 0; // assert that we got a valid child_to_search assert(child_to_search >= 0 && child_to_search < node->n_children); // // At this point, we must have the necessary partition available to continue the search // assert(BP_STATE(node,child_to_search) == PT_AVAIL); + // When we enter, we are in a batch. If we search a node but get + // DB_NOTFOUND and need to search the next node, we'll need to start + // another batch. + bool must_begin_batch = false; while (child_to_search >= 0 && child_to_search < node->n_children) { // // Normally, the child we want to use is available, as we checked @@ -4587,6 +4600,10 @@ ft_search_node( } const struct pivot_bounds next_bounds = next_pivot_keys(node, child_to_search, bounds); if (node->height > 0) { + if (must_begin_batch) { + toku_cachetable_begin_batched_pin(brt->ft->cf); + must_begin_batch = false; + } r = ft_search_child( brt, node, @@ -4618,34 +4635,35 @@ ft_search_node( if (r != DB_NOTFOUND) { return r; //Error (or message to quit early, such as TOKUDB_FOUND_BUT_REJECTED or TOKUDB_TRY_AGAIN) } - // we have a new pivotkey - else { - if (node->height == 0) { - // when we run off the end of a basement, try to lock the range up to the pivot. solves #3529 - const DBT *pivot = NULL; - if (search->direction == FT_SEARCH_LEFT) - pivot = next_bounds.upper_bound_inclusive; // left -> right - else - pivot = next_bounds.lower_bound_exclusive; // right -> left - if (pivot) { - int rr = getf(pivot->size, pivot->data, 0, NULL, getf_v, true); - if (rr != 0) - return rr; // lock was not granted - } - } - - // If we got a DB_NOTFOUND then we have to search the next record. Possibly everything present is not visible. - // This way of doing DB_NOTFOUND is a kludge, and ought to be simplified. Something like this is needed for DB_NEXT, but - // for point queries, it's overkill. If we got a DB_NOTFOUND on a point query then we should just stop looking. - // When releasing locks on I/O we must not search the same subtree again, or we won't be guaranteed to make forward progress. - // If we got a DB_NOTFOUND, then the pivot is too small if searching from left to right (too large if searching from right to left). - // So save the pivot key in the search object. - maybe_search_save_bound(node, child_to_search, search); - } // not really necessary, just put this here so that reading the // code becomes simpler. The point is at this point in the code, // we know that we got DB_NOTFOUND and we have to continue assert(r == DB_NOTFOUND); + // we have a new pivotkey + if (node->height == 0) { + // when we run off the end of a basement, try to lock the range up to the pivot. solves #3529 + const DBT *pivot = NULL; + if (search->direction == FT_SEARCH_LEFT) + pivot = next_bounds.upper_bound_inclusive; // left -> right + else + pivot = next_bounds.lower_bound_exclusive; // right -> left + if (pivot) { + int rr = getf(pivot->size, pivot->data, 0, NULL, getf_v, true); + if (rr != 0) + return rr; // lock was not granted + } + } + + // If we got a DB_NOTFOUND then we have to search the next record. Possibly everything present is not visible. + // This way of doing DB_NOTFOUND is a kludge, and ought to be simplified. Something like this is needed for DB_NEXT, but + // for point queries, it's overkill. If we got a DB_NOTFOUND on a point query then we should just stop looking. + // When releasing locks on I/O we must not search the same subtree again, or we won't be guaranteed to make forward progress. + // If we got a DB_NOTFOUND, then the pivot is too small if searching from left to right (too large if searching from right to left). + // So save the pivot key in the search object. + maybe_search_save_bound(node, child_to_search, search); + + // We're about to pin some more nodes, but we thought we were done before. + must_begin_batch = true; if (search->direction == FT_SEARCH_LEFT) { child_to_search++; } @@ -4663,11 +4681,12 @@ toku_ft_search (FT_HANDLE brt, ft_search_t *search, FT_GET_CALLBACK_FUNCTION get { int r; uint trycount = 0; // How many tries did it take to get the result? + FT ft = brt->ft; try_again: trycount++; - assert(brt->ft); + assert(ft); // // Here is how searches work @@ -4698,7 +4717,7 @@ try_again: struct ftnode_fetch_extra bfe; fill_bfe_for_subset_read( &bfe, - brt->ft, + ft, search, &ftcursor->range_lock_left_key, &ftcursor->range_lock_right_key, @@ -4708,12 +4727,17 @@ try_again: ); FTNODE node = NULL; { - toku_ft_grab_treelock(brt->ft); + toku_ft_grab_treelock(ft); uint32_t fullhash; CACHEKEY root_key; - toku_calculate_root_offset_pointer(brt->ft, &root_key, &fullhash); - toku_pin_ftnode_off_client_thread( - brt->ft, + toku_calculate_root_offset_pointer(ft, &root_key, &fullhash); + // Begin a batch of pins here. If a child gets TOKUDB_TRY_AGAIN + // it must immediately end the batch. Otherwise, it must end the + // batch as soon as it pins the leaf. The batch will never be + // ended in this function. + toku_cachetable_begin_batched_pin(ft->cf); + toku_pin_ftnode_off_client_thread_batched( + ft, root_key, fullhash, &bfe, @@ -4722,7 +4746,13 @@ try_again: NULL, &node ); - toku_ft_release_treelock(brt->ft); + if (node->height == 0) { + // The root is a leaf, must end the batch now because we + // won't apply ancestor messages, which is where we usually + // end it. + toku_cachetable_end_batched_pin(ft->cf); + } + toku_ft_release_treelock(ft); } uint tree_height = node->height + 1; // How high is the tree? This is the height of the root node plus one (leaf is at height 0). @@ -5401,7 +5431,7 @@ toku_dump_ftnode (FILE *file, FT_HANDLE brt, BLOCKNUM blocknum, int depth, const } } } - r = toku_cachetable_unpin(brt->ft->cf, blocknum, fullhash, CACHETABLE_CLEAN, make_ftnode_pair_attr(node)); + r = toku_cachetable_unpin(brt->ft->cf, node->ct_pair, CACHETABLE_CLEAN, make_ftnode_pair_attr(node)); assert_zero(r); return result; } @@ -5432,7 +5462,6 @@ int toku_ft_layer_init(void) { partitioned_counters_init(); toku_checkpoint_init(); toku_ft_serialize_layer_init(); - toku_cachetables_init(); toku_mutex_init(&ft_open_close_lock, NULL); exit: return r; @@ -5440,7 +5469,6 @@ exit: void toku_ft_layer_destroy(void) { toku_mutex_destroy(&ft_open_close_lock); - toku_cachetables_destroy(); toku_ft_serialize_layer_destroy(); toku_checkpoint_destroy(); partitioned_counters_destroy(); diff --git a/ft/ft-verify.cc b/ft/ft-verify.cc index d25e47eaca8..eebe6395db8 100644 --- a/ft/ft-verify.cc +++ b/ft/ft-verify.cc @@ -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) ); diff --git a/ft/ft.cc b/ft/ft.cc index bb503fb1f83..26f465f5b03 100644 --- a/ft/ft.cc +++ b/ft/ft.cc @@ -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 diff --git a/ft/ft.h b/ft/ft.h index 805c536de83..fc0001519d9 100644 --- a/ft/ft.h +++ b/ft/ft.h @@ -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); diff --git a/ft/fttypes.h b/ft/fttypes.h index ba297d47197..0cd40d2542e 100644 --- a/ft/fttypes.h +++ b/ft/fttypes.h @@ -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 { diff --git a/ft/log_upgrade.cc b/ft/log_upgrade.cc index 0c0ed93d391..0592a0a5788 100644 --- a/ft/log_upgrade.cc +++ b/ft/log_upgrade.cc @@ -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 diff --git a/ft/memarena.cc b/ft/memarena.cc index 427e86226ce..572ad1a498a 100644 --- a/ft/memarena.cc +++ b/ft/memarena.cc @@ -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(resultmutex); 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. + + } + } } } diff --git a/ft/recover.cc b/ft/recover.cc index 7c388eaf618..0e3a0715970 100644 --- a/ft/recover.cc +++ b/ft/recover.cc @@ -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)); diff --git a/ft/rollback-ct-callbacks.cc b/ft/rollback-ct-callbacks.cc index 4e0444de74b..c7279791ccd 100644 --- a/ft/rollback-ct-callbacks.cc +++ b/ft/rollback-ct-callbacks.cc @@ -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; diff --git a/ft/rollback-ct-callbacks.h b/ft/rollback-ct-callbacks.h index a1224be8f48..937fb0da43c 100644 --- a/ft/rollback-ct-callbacks.h +++ b/ft/rollback-ct-callbacks.h @@ -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), diff --git a/ft/rollback.cc b/ft/rollback.cc index 175ac482257..65080101719 100644 --- a/ft/rollback.cc +++ b/ft/rollback.cc @@ -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); } diff --git a/ft/rollback.h b/ft/rollback.h index fce795f4c8e..d7d92178457 100644 --- a/ft/rollback.h +++ b/ft/rollback.h @@ -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 diff --git a/ft/tests/CMakeLists.txt b/ft/tests/CMakeLists.txt index 3e1f5855068..8c578146f5b 100644 --- a/ft/tests/CMakeLists.txt +++ b/ft/tests/CMakeLists.txt @@ -1,6 +1,6 @@ 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) diff --git a/ft/tests/cachetable-3969.cc b/ft/tests/cachetable-3969.cc index 5bbc927914a..5797eb54779 100644 --- a/ft/tests/cachetable-3969.cc +++ b/ft/tests/cachetable-3969.cc @@ -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 diff --git a/ft/tests/cachetable-4302.cc b/ft/tests/cachetable-4302.cc deleted file mode 100644 index 4bb62c24ce8..00000000000 --- a/ft/tests/cachetable-4302.cc +++ /dev/null @@ -1,174 +0,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-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; -} diff --git a/ft/tests/cachetable-4357.cc b/ft/tests/cachetable-4357.cc index 2240d4a4dc1..069acb18451 100644 --- a/ft/tests/cachetable-4357.cc +++ b/ft/tests/cachetable-4357.cc @@ -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; diff --git a/ft/tests/cachetable-4365.cc b/ft/tests/cachetable-4365.cc index 311c090b37f..e49700cf302 100644 --- a/ft/tests/cachetable-4365.cc +++ b/ft/tests/cachetable-4365.cc @@ -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); diff --git a/ft/tests/cachetable-4545.cc b/ft/tests/cachetable-4545.cc index 075d4b895b2..ddbcc6994d1 100644 --- a/ft/tests/cachetable-4545.cc +++ b/ft/tests/cachetable-4545.cc @@ -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 diff --git a/ft/tests/cachetable-5097.cc b/ft/tests/cachetable-5097.cc index 411b0f427d2..06c4c8dda54 100644 --- a/ft/tests/cachetable-5097.cc +++ b/ft/tests/cachetable-5097.cc @@ -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); } diff --git a/ft/tests/cachetable-all-write.cc b/ft/tests/cachetable-all-write.cc index 2d6776945ce..b17a166c133 100644 --- a/ft/tests/cachetable-all-write.cc +++ b/ft/tests/cachetable-all-write.cc @@ -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); diff --git a/ft/tests/cachetable-checkpoint-pending.cc b/ft/tests/cachetable-checkpoint-pending.cc index 63db470c2bb..a86dab46f43 100644 --- a/ft/tests/cachetable-checkpoint-pending.cc +++ b/ft/tests/cachetable-checkpoint-pending.cc @@ -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); diff --git a/ft/tests/cachetable-checkpoint-pinned-nodes.cc b/ft/tests/cachetable-checkpoint-pinned-nodes.cc index 21cc4b5a28d..224c5b63ad4 100644 --- a/ft/tests/cachetable-checkpoint-pinned-nodes.cc +++ b/ft/tests/cachetable-checkpoint-pinned-nodes.cc @@ -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 // - - 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_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 diff --git a/ft/tests/cachetable-checkpoint-prefetched-nodes.cc b/ft/tests/cachetable-checkpoint-prefetched-nodes.cc index becb025476c..a2274343141 100644 --- a/ft/tests/cachetable-checkpoint-prefetched-nodes.cc +++ b/ft/tests/cachetable-checkpoint-prefetched-nodes.cc @@ -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 diff --git a/ft/tests/cachetable-checkpoint-test.cc b/ft/tests/cachetable-checkpoint-test.cc index 7b08befa631..349f85c591e 100644 --- a/ft/tests/cachetable-checkpoint-test.cc +++ b/ft/tests/cachetable-checkpoint-test.cc @@ -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); diff --git a/ft/tests/cachetable-checkpointer-class.cc b/ft/tests/cachetable-checkpointer-class.cc new file mode 100644 index 00000000000..18fac335be0 --- /dev/null +++ b/ft/tests/cachetable-checkpointer-class.cc @@ -0,0 +1,267 @@ +/* -*- 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; +} + + diff --git a/ft/tests/cachetable-cleaner-checkpoint.cc b/ft/tests/cachetable-cleaner-checkpoint.cc index 1d6fc2dc87d..770c1866440 100644 --- a/ft/tests/cachetable-cleaner-checkpoint.cc +++ b/ft/tests/cachetable-cleaner-checkpoint.cc @@ -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 diff --git a/ft/tests/cachetable-cleaner-checkpoint2.cc b/ft/tests/cachetable-cleaner-checkpoint2.cc index 1a525d781d2..256b7feae97 100644 --- a/ft/tests/cachetable-cleaner-checkpoint2.cc +++ b/ft/tests/cachetable-cleaner-checkpoint2.cc @@ -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 diff --git a/ft/tests/cachetable-cleaner-thread-attrs-accumulate.cc b/ft/tests/cachetable-cleaner-thread-attrs-accumulate.cc index bba9eebccdf..c7a9e334175 100644 --- a/ft/tests/cachetable-cleaner-thread-attrs-accumulate.cc +++ b/ft/tests/cachetable-cleaner-thread-attrs-accumulate.cc @@ -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); diff --git a/ft/tests/cachetable-cleaner-thread-empty-cachetable.cc b/ft/tests/cachetable-cleaner-thread-empty-cachetable.cc index 5baaa4fa749..77e9ccfc1c5 100644 --- a/ft/tests/cachetable-cleaner-thread-empty-cachetable.cc +++ b/ft/tests/cachetable-cleaner-thread-empty-cachetable.cc @@ -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 diff --git a/ft/tests/cachetable-cleaner-thread-everything-pinned.cc b/ft/tests/cachetable-cleaner-thread-everything-pinned.cc index ad992cff734..9f1953b5ce1 100644 --- a/ft/tests/cachetable-cleaner-thread-everything-pinned.cc +++ b/ft/tests/cachetable-cleaner-thread-everything-pinned.cc @@ -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); } diff --git a/ft/tests/cachetable-cleaner-thread-nothing-needs-flushing.cc b/ft/tests/cachetable-cleaner-thread-nothing-needs-flushing.cc index ee4e7ed9c84..5b271a09c20 100644 --- a/ft/tests/cachetable-cleaner-thread-nothing-needs-flushing.cc +++ b/ft/tests/cachetable-cleaner-thread-nothing-needs-flushing.cc @@ -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); } diff --git a/ft/tests/cachetable-cleaner-thread-simple.cc b/ft/tests/cachetable-cleaner-thread-simple.cc index e19fcef5b85..046d05d8654 100644 --- a/ft/tests/cachetable-cleaner-thread-simple.cc +++ b/ft/tests/cachetable-cleaner-thread-simple.cc @@ -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); } diff --git a/ft/tests/cachetable-clock-all-pinned.cc b/ft/tests/cachetable-clock-all-pinned.cc index 4a4edb7a3a7..ed048b216fa 100644 --- a/ft/tests/cachetable-clock-all-pinned.cc +++ b/ft/tests/cachetable-clock-all-pinned.cc @@ -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)); } diff --git a/ft/tests/cachetable-clock-eviction.cc b/ft/tests/cachetable-clock-eviction.cc index 5a43343be66..49ee874c243 100644 --- a/ft/tests/cachetable-clock-eviction.cc +++ b/ft/tests/cachetable-clock-eviction.cc @@ -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 ); diff --git a/ft/tests/cachetable-clock-eviction2.cc b/ft/tests/cachetable-clock-eviction2.cc index d062aef2212..902fac2848b 100644 --- a/ft/tests/cachetable-clock-eviction2.cc +++ b/ft/tests/cachetable-clock-eviction2.cc @@ -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); diff --git a/ft/tests/cachetable-clock-eviction3.cc b/ft/tests/cachetable-clock-eviction3.cc index 5814a3aa5c9..6b795ebc0b4 100644 --- a/ft/tests/cachetable-clock-eviction3.cc +++ b/ft/tests/cachetable-clock-eviction3.cc @@ -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); diff --git a/ft/tests/cachetable-clock-eviction4.cc b/ft/tests/cachetable-clock-eviction4.cc index 00643945670..61d52d5f358 100644 --- a/ft/tests/cachetable-clock-eviction4.cc +++ b/ft/tests/cachetable-clock-eviction4.cc @@ -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 ); diff --git a/ft/tests/cachetable-clone-checkpoint.cc b/ft/tests/cachetable-clone-checkpoint.cc index ff2a69a2a9a..4c40adfa081 100644 --- a/ft/tests/cachetable-clone-checkpoint.cc +++ b/ft/tests/cachetable-clone-checkpoint.cc @@ -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; diff --git a/ft/tests/cachetable-clone-partial-fetch-pinned-node.cc b/ft/tests/cachetable-clone-partial-fetch-pinned-node.cc index dc646267826..887f4f5940c 100644 --- a/ft/tests/cachetable-clone-partial-fetch-pinned-node.cc +++ b/ft/tests/cachetable-clone-partial-fetch-pinned-node.cc @@ -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 diff --git a/ft/tests/cachetable-clone-partial-fetch.cc b/ft/tests/cachetable-clone-partial-fetch.cc index 21a8463b0ca..e49fd3da9e5 100644 --- a/ft/tests/cachetable-clone-partial-fetch.cc +++ b/ft/tests/cachetable-clone-partial-fetch.cc @@ -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 diff --git a/ft/tests/cachetable-clone-pin-nonblocking.cc b/ft/tests/cachetable-clone-pin-nonblocking.cc index 7f54c205114..eb11d29193d 100644 --- a/ft/tests/cachetable-clone-pin-nonblocking.cc +++ b/ft/tests/cachetable-clone-pin-nonblocking.cc @@ -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 diff --git a/ft/tests/cachetable-clone-unpin-remove.cc b/ft/tests/cachetable-clone-unpin-remove.cc index 4218f54fa83..22672a412b8 100644 --- a/ft/tests/cachetable-clone-unpin-remove.cc +++ b/ft/tests/cachetable-clone-unpin-remove.cc @@ -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 diff --git a/ft/tests/cachetable-count-pinned-test.cc b/ft/tests/cachetable-count-pinned-test.cc index 97315a762c8..47edc92c6f3 100644 --- a/ft/tests/cachetable-count-pinned-test.cc +++ b/ft/tests/cachetable-count-pinned-test.cc @@ -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); diff --git a/ft/tests/cachetable-debug-test.cc b/ft/tests/cachetable-debug-test.cc index 894be442d9d..a7132a99543 100644 --- a/ft/tests/cachetable-debug-test.cc +++ b/ft/tests/cachetable-debug-test.cc @@ -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); diff --git a/ft/tests/cachetable-eviction-close-test.cc b/ft/tests/cachetable-eviction-close-test.cc index b76076b39f3..985bc552ae6 100644 --- a/ft/tests/cachetable-eviction-close-test.cc +++ b/ft/tests/cachetable-eviction-close-test.cc @@ -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); diff --git a/ft/tests/cachetable-eviction-close-test2.cc b/ft/tests/cachetable-eviction-close-test2.cc index c1350f54faa..72a841267b9 100644 --- a/ft/tests/cachetable-eviction-close-test2.cc +++ b/ft/tests/cachetable-eviction-close-test2.cc @@ -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); diff --git a/ft/tests/cachetable-eviction-getandpin-test.cc b/ft/tests/cachetable-eviction-getandpin-test.cc index bb2de643a71..fa980479ca9 100644 --- a/ft/tests/cachetable-eviction-getandpin-test.cc +++ b/ft/tests/cachetable-eviction-getandpin-test.cc @@ -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); diff --git a/ft/tests/cachetable-eviction-getandpin-test2.cc b/ft/tests/cachetable-eviction-getandpin-test2.cc index 66f3ee3662b..822d7cdc45b 100644 --- a/ft/tests/cachetable-eviction-getandpin-test2.cc +++ b/ft/tests/cachetable-eviction-getandpin-test2.cc @@ -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); diff --git a/ft/tests/cachetable-evictor-class.cc b/ft/tests/cachetable-evictor-class.cc new file mode 100644 index 00000000000..b26008effb5 --- /dev/null +++ b/ft/tests/cachetable-evictor-class.cc @@ -0,0 +1,228 @@ +/* -*- 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; +} diff --git a/ft/tests/cachetable-fetch-inducing-evictor.cc b/ft/tests/cachetable-fetch-inducing-evictor.cc new file mode 100644 index 00000000000..e46e1676bba --- /dev/null +++ b/ft/tests/cachetable-fetch-inducing-evictor.cc @@ -0,0 +1,117 @@ +/* -*- 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; +} diff --git a/ft/tests/cachetable-flush-during-cleaner.cc b/ft/tests/cachetable-flush-during-cleaner.cc index 83482e8a684..7bcbac2596f 100644 --- a/ft/tests/cachetable-flush-during-cleaner.cc +++ b/ft/tests/cachetable-flush-during-cleaner.cc @@ -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); diff --git a/ft/tests/cachetable-flush-test.cc b/ft/tests/cachetable-flush-test.cc index 1861a01933b..20d396cf1b6 100644 --- a/ft/tests/cachetable-flush-test.cc +++ b/ft/tests/cachetable-flush-test.cc @@ -28,14 +28,14 @@ test_cachetable_def_flush (int n) { for (i=0; isize); *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); diff --git a/ft/tests/cachetable-kibbutz_and_flush_cachefile.cc b/ft/tests/cachetable-kibbutz_and_flush_cachefile.cc index e6d003d3a88..e280034ae12 100644 --- a/ft/tests/cachetable-kibbutz_and_flush_cachefile.cc +++ b/ft/tests/cachetable-kibbutz_and_flush_cachefile.cc @@ -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); } diff --git a/ft/tests/cachetable-partial-fetch.cc b/ft/tests/cachetable-partial-fetch.cc index a36e7775bf7..d68566ad7ef 100644 --- a/ft/tests/cachetable-partial-fetch.cc +++ b/ft/tests/cachetable-partial-fetch.cc @@ -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); diff --git a/ft/tests/cachetable-pin-checkpoint.cc b/ft/tests/cachetable-pin-checkpoint.cc index 046a82cdc37..0358532aeea 100644 --- a/ft/tests/cachetable-pin-checkpoint.cc +++ b/ft/tests/cachetable-pin-checkpoint.cc @@ -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 diff --git a/ft/tests/cachetable-pin-nonblocking-checkpoint-clean.cc b/ft/tests/cachetable-pin-nonblocking-checkpoint-clean.cc index 6e53c110515..c31466ed19b 100644 --- a/ft/tests/cachetable-pin-nonblocking-checkpoint-clean.cc +++ b/ft/tests/cachetable-pin-nonblocking-checkpoint-clean.cc @@ -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 diff --git a/ft/tests/cachetable-prefetch-checkpoint-test.cc b/ft/tests/cachetable-prefetch-checkpoint-test.cc index ddd3ce990aa..f4b7a34bc8a 100644 --- a/ft/tests/cachetable-prefetch-checkpoint-test.cc +++ b/ft/tests/cachetable-prefetch-checkpoint-test.cc @@ -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; iev, 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; i1); 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); diff --git a/ft/tests/cachetable-prefetch2-test.cc b/ft/tests/cachetable-prefetch2-test.cc index 0f6b17c1059..c7a894abe31 100644 --- a/ft/tests/cachetable-prefetch2-test.cc +++ b/ft/tests/cachetable-prefetch2-test.cc @@ -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); diff --git a/ft/tests/cachetable-put-checkpoint.cc b/ft/tests/cachetable-put-checkpoint.cc index 0fee69f7b35..e67988cf646 100644 --- a/ft/tests/cachetable-put-checkpoint.cc +++ b/ft/tests/cachetable-put-checkpoint.cc @@ -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 diff --git a/ft/tests/cachetable-put-test.cc b/ft/tests/cachetable-put-test.cc index dbab9d0cb13..ae5dd8e7be7 100644 --- a/ft/tests/cachetable-put-test.cc +++ b/ft/tests/cachetable-put-test.cc @@ -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); } diff --git a/ft/tests/cachetable-rename-test.cc b/ft/tests/cachetable-rename-test.cc deleted file mode 100644 index 69a7a58a231..00000000000 --- a/ft/tests/cachetable-rename-test.cc +++ /dev/null @@ -1,192 +0,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= KEYLIMIT) { - test_mutex_unlock(); - toku_pthread_yield(); maybe_flush(t); - test_mutex_lock(); - } - assert(n_keys0) { - // 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), ¤t_value, ¤t_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; -} diff --git a/ft/tests/cachetable-scan.cc b/ft/tests/cachetable-scan.cc index 6fed5e34be1..bcda0cc0660 100644 --- a/ft/tests/cachetable-scan.cc +++ b/ft/tests/cachetable-scan.cc @@ -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= N_PRESENT_LIMIT) { - test_mutex_unlock(); toku_pthread_yield(); maybe_flush(thect); test_mutex_lock(); - } - assert(n_present0 && r0) { - // First touch some random ones - test_mutex_lock(); - int whichone = random()%n_present; - CACHEFILE whichcf = present_items[whichone].cf; - CACHEKEY whichkey = present_items[whichone].key; - test_mutex_unlock(); - void *value; - //printf("Touching %d (%lld, %p)\n", whichone, whichkey, whichcf); - uint32_t fhash = toku_cachetable_hash(whichcf, whichkey); - CACHETABLE_WRITE_CALLBACK wc = def_write_callback((void*)(long)whichkey.b); - wc.flush_callback = flush_forchain; - r = toku_cachetable_get_and_pin(whichcf, - whichkey, - fhash, - &value, - NULL, - wc, - fetch_forchain, - def_pf_req_callback, - def_pf_callback, - true, - (void*)(long)whichkey.b - ); - assert(r==0); - r = toku_cachetable_unpin(whichcf, - whichkey, - fhash, - CACHETABLE_CLEAN, make_pair_attr(test_object_size)); - assert(r==0); - } - - i += 1+ random()%100; - int fnum = i%N_FILES; - // i is always incrementing, so we need not worry about inserting a duplicate - // if i is a duplicate, cachetable_put will return -1 - // printf("%s:%d Add {%ld,%p}\n", __SRCFILE__, __LINE__, i, f[fnum]); - uint32_t fhash = toku_cachetable_hash(f[fnum], make_blocknum(i)); - CACHETABLE_WRITE_CALLBACK wc = def_write_callback((void *)i); - wc.flush_callback = flush_forchain; - r = toku_cachetable_put(f[fnum], make_blocknum(i), fhash, (void*)i, make_pair_attr(test_object_size), wc); - assert(r==0 || r==-1); - if (r==0) { - item_becomes_present(ct, f[fnum], make_blocknum(i)); - //print_ints(); - //cachetable_print_state(ct); - r = toku_cachetable_unpin(f[fnum], make_blocknum(i), fhash, CACHETABLE_CLEAN, make_pair_attr(test_object_size)); - assert(r==0); - } - - // now that we have a clock instead of an LRU, there - // is no guarantee that PAIR stays in cachetable - - //long long pinned; - //r = toku_cachetable_get_key_state(ct, make_blocknum(i), f[fnum], 0, 0, &pinned, 0); - //assert(r==0); - //assert(pinned == 0); - verify_cachetable_against_present(); - - if (random()%10==0) { - i = random()%N_FILES; - //printf("Close %d (%p), now n_present=%d\n", i, f[i], n_present); - //print_ints(); - CACHEFILE oldcf=f[i]; - r = toku_cachefile_close(&f[i], 0, false, ZERO_LSN); assert(r==0); - file_is_not_present(oldcf); - r = toku_cachetable_openf(&f[i], ct, fname[i], O_RDWR, S_IRWXU|S_IRWXG|S_IRWXO); assert(r==0); - } - } - for (i=0; i 0) { i = random() % nkeys; uint32_t hi = toku_cachetable_hash(f1, make_blocknum(testkeys[i].b)); - r = toku_cachetable_unpin_and_remove(f1, testkeys[i], NULL, NULL); + r = toku_test_cachetable_unpin_and_remove(f1, testkeys[i], NULL, NULL); assert(r == 0); toku_cachefile_verify(f1); @@ -69,13 +70,6 @@ cachetable_unpin_and_remove_test (int n) { testkeys[i] = testkeys[nkeys-1]; nkeys -= 1; } - // verify that all are really removed - for (i=0; i0; 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); +} +enum unpin_evictor_test_type { + unpin_increase, + unpin_decrease, + unpin_invalid_attr +}; + +static void +unpin_and_evictor_test(enum unpin_evictor_test_type test_type) { + int r; + CACHETABLE ct; + int test_limit = 4; + 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); + evictor_test_helpers::set_hysteresis_limits(&ct->ev, test_limit, test_limit); + evictor_test_helpers::disable_ev_thread(&ct->ev); + + void* value2; + long size2; + CACHETABLE_WRITE_CALLBACK wc = def_write_callback(NULL); + // this should put in the cachetable a pair of size 8 + r = toku_cachetable_get_and_pin( + f1, + make_blocknum(1), + 1, + &value2, + &size2, + wc, + def_fetch, + def_pf_req_callback, + def_pf_callback, + true, + 0 + ); + assert(r==0); + // + // now we unpin, + // if we increase the size, we should catch a sleep + // if we don't increase the size, there should be no sleep + // if we pass in an invalid pair_attr, there should be no sleep. + // + uint64_t old_num_ev_runs = 0; + uint64_t new_num_ev_runs = 0; + if (test_type == unpin_increase) { + old_num_ev_runs = evictor_test_helpers::get_num_eviction_runs(&ct->ev); + r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(9)); + new_num_ev_runs = evictor_test_helpers::get_num_eviction_runs(&ct->ev); + assert(new_num_ev_runs > old_num_ev_runs); + } + else if (test_type == unpin_decrease || test_type == unpin_invalid_attr) { + old_num_ev_runs = evictor_test_helpers::get_num_eviction_runs(&ct->ev); + r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8)); + new_num_ev_runs = evictor_test_helpers::get_num_eviction_runs(&ct->ev); + assert(new_num_ev_runs == old_num_ev_runs); + } + else { + assert(false); + } + + toku_cachetable_verify(ct); r = toku_cachefile_close(&f1, 0, false, ZERO_LSN); assert(r == 0); r = toku_cachetable_close(&ct); assert(r == 0 && ct == 0); } @@ -58,5 +121,8 @@ int test_main(int argc, const char *argv[]) { default_parse_args(argc, argv); cachetable_unpin_test(8); + unpin_and_evictor_test(unpin_increase); + unpin_and_evictor_test(unpin_decrease); + unpin_and_evictor_test(unpin_invalid_attr); return 0; } diff --git a/ft/tests/cachetable-writer-thread-limit.cc b/ft/tests/cachetable-writer-thread-limit.cc index 8da68ba80ac..3cbf25e3840 100644 --- a/ft/tests/cachetable-writer-thread-limit.cc +++ b/ft/tests/cachetable-writer-thread-limit.cc @@ -48,10 +48,10 @@ cachetable_test (void) { for (int64_t i = 0; i < num_entries; i++) { CACHETABLE_WRITE_CALLBACK wc = def_write_callback(NULL); wc.flush_callback = flush; - 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); int curr_size = __sync_fetch_and_add(&total_size, 1); assert(curr_size <= test_limit + test_limit/2+1); - r = toku_cachetable_unpin(f1, make_blocknum(i), i, CACHETABLE_DIRTY, make_pair_attr(4)); + r = toku_test_cachetable_unpin(f1, make_blocknum(i), i, CACHETABLE_DIRTY, make_pair_attr(4)); } r = toku_cachefile_close(&f1, 0, false, ZERO_LSN); assert(r == 0); diff --git a/ft/tests/dbufio-test-destroy.cc b/ft/tests/dbufio-test-destroy.cc index b13fbe86e3a..ecef5017083 100644 --- a/ft/tests/dbufio-test-destroy.cc +++ b/ft/tests/dbufio-test-destroy.cc @@ -13,11 +13,11 @@ enum { N = 5 }; enum { M = 10 }; -static void test1 (size_t chars_per_file, size_t bytes_per_read) { +static void test1 (size_t chars_per_file, size_t UU(bytes_per_read)) { int fds[N]; char fnames[N][100]; size_t n_read[N]; - int n_live=N; + int UU(n_live)=N; for (int i=0; ibp[0].ptr.u.nonleaf) == 0); assert(toku_bnc_n_entries(node->bp[1].ptr.u.nonleaf) > 0); toku_unpin_ftnode(t->ft, node); - r = toku_checkpoint(ct, NULL, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT); + r = toku_checkpoint(cp, NULL, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT); assert_zero(r); toku_pin_node_with_min_bfe(&node, node_internal, t); assert(!node->dirty); @@ -214,7 +215,7 @@ doit (void) { assert(toku_bnc_n_entries(node->bp[1].ptr.u.nonleaf) == 0); // now let's do a flush with an empty buffer, make sure it is ok toku_unpin_ftnode(t->ft, node); - r = toku_checkpoint(ct, NULL, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT); + r = toku_checkpoint(cp, NULL, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT); assert_zero(r); toku_pin_node_with_min_bfe(&node, node_internal, t); assert(!node->dirty); @@ -280,7 +281,7 @@ doit (void) { 0 ); assert_zero(r); - r = toku_checkpoint(ct, NULL, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT); + r = toku_checkpoint(cp, NULL, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT); assert_zero(r); toku_pin_node_with_min_bfe(&node, node_internal, t); for (int i = 0; i < 20; i++) { diff --git a/ft/tests/test.h b/ft/tests/test.h index 9fef52dc09e..1bfe8ae6a65 100644 --- a/ft/tests/test.h +++ b/ft/tests/test.h @@ -12,7 +12,7 @@ #include "ft-ops.h" #include - +#include "cachetable-internal.h" #define CKERR(r) ({ int __r = r; if (__r!=0) fprintf(stderr, "%s:%d error %d %s\n", __FILE__, __LINE__, __r, strerror(r)); assert(__r==0); }) @@ -167,6 +167,7 @@ static UU() bool def_pf_req_callback(void* UU(ftnode_pv), void* UU(read_extraarg static UU() int def_fetch (CACHEFILE f __attribute__((__unused__)), + PAIR UU(p), int UU(fd), CACHEKEY k __attribute__((__unused__)), uint32_t fullhash __attribute__((__unused__)), @@ -182,6 +183,30 @@ def_fetch (CACHEFILE f __attribute__((__unused__)), return 0; } +static UU() void +put_callback_nop( + void *UU(v), + PAIR UU(p)) { +} + +static UU() int +fetch_die( + CACHEFILE UU(thiscf), + PAIR UU(p), + 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 UU() int def_cleaner_callback( @@ -206,6 +231,29 @@ static UU() CACHETABLE_WRITE_CALLBACK def_write_callback(void* write_extraargs) return wc; } +class evictor_test_helpers { +public: + static void set_hysteresis_limits(evictor* ev, long low_size_watermark, long high_size_watermark) { + ev->m_low_size_watermark = low_size_watermark; + ev->m_low_size_hysteresis = low_size_watermark; + ev->m_high_size_hysteresis = high_size_watermark; + ev->m_high_size_watermark = high_size_watermark; + } + static void disable_ev_thread(evictor* ev) { + toku_mutex_lock(&ev->m_ev_thread_lock); + ev->m_period_in_seconds = 0; + // signal eviction thread so that it wakes up + // and then sleeps indefinitely + ev->signal_eviction_thread(); + toku_mutex_unlock(&ev->m_ev_thread_lock); + // sleep for one second to ensure eviction thread picks up new period + usleep(1*1024*1024); + } + static uint64_t get_num_eviction_runs(evictor* ev) { + return ev->m_num_eviction_thread_runs; + } +}; + int verbose=0; static inline void diff --git a/ft/tests/test3681.cc b/ft/tests/test3681.cc index 22b184c8f09..deac90e1c8e 100644 --- a/ft/tests/test3681.cc +++ b/ft/tests/test3681.cc @@ -52,8 +52,9 @@ static void *startb (void *n) { assert(n==NULL); int count=0; while (!done) { - int r = toku_checkpoint(ct, NULL, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT); assert(r==0); - count++; + CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct); + int r = toku_checkpoint(cp, NULL, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT); assert(r==0); + count++; } printf("count=%d\n", count); return NULL; diff --git a/ft/tests/upgrade_test_simple.cc b/ft/tests/upgrade_test_simple.cc index 427ecd65b04..347fb1c6f1e 100644 --- a/ft/tests/upgrade_test_simple.cc +++ b/ft/tests/upgrade_test_simple.cc @@ -108,7 +108,8 @@ with_open_tree(const char *fname, tree_cb cb, void *cb_extra) r2 = cb(t, ct, cb_extra); r = toku_verify_ft(t); CKERR(r); - 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); CKERR(r); r = toku_close_ft_handle_nolsn(t, 0); CKERR(r); diff --git a/ft/tests/xid_lsn_independent.cc b/ft/tests/xid_lsn_independent.cc index 0ffdc53c5c9..c72108ac0f2 100644 --- a/ft/tests/xid_lsn_independent.cc +++ b/ft/tests/xid_lsn_independent.cc @@ -88,8 +88,8 @@ static void test_xid_lsn_independent(int N) { CKERR(r); toku_txn_close_txn(txn3); } - - r = toku_checkpoint(ct, logger, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT); + CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct); + r = toku_checkpoint(cp, logger, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT); CKERR(r); r = toku_close_ft_handle_nolsn(brt, NULL); CKERR(r); diff --git a/src/lock_tree/locktree.cc b/src/lock_tree/locktree.cc index 3e5bf4ba52f..3cfcc56bb30 100644 --- a/src/lock_tree/locktree.cc +++ b/src/lock_tree/locktree.cc @@ -2664,7 +2664,7 @@ toku_ltm_set_lock_wait_time(toku_ltm *mgr, uint64_t lock_wait_time_msec) { if (lock_wait_time_msec == UINT64_MAX) mgr->lock_wait_time = max_timeval; else - mgr->lock_wait_time = (struct timeval) { (time_t) lock_wait_time_msec / 1000, (__suseconds_t) (lock_wait_time_msec % 1000) * 1000 }; + mgr->lock_wait_time = (struct timeval) { (time_t) lock_wait_time_msec / 1000, (SUSECONDS) (lock_wait_time_msec % 1000) * 1000 }; } void diff --git a/src/lock_tree/locktree.h b/src/lock_tree/locktree.h index 39133343f6d..f8ef4c93875 100644 --- a/src/lock_tree/locktree.h +++ b/src/lock_tree/locktree.h @@ -11,6 +11,12 @@ #include #include +#if defined(__APPLE__) +typedef suseconds_t SUSECONDS; +#else +typedef __suseconds_t SUSECONDS; +#endif + /** \file locktree.h \brief Lock trees: header and comments diff --git a/src/tests/CMakeLists.txt b/src/tests/CMakeLists.txt index 30d529c2300..e6ac53a3232 100644 --- a/src/tests/CMakeLists.txt +++ b/src/tests/CMakeLists.txt @@ -234,7 +234,6 @@ if(BUILD_TESTING) test_logflush test_multiple_checkpoints_block_commit test_query - test_rand_insert_mult_db test_redirect_func test_row_size_supported test_stress1 @@ -657,7 +656,6 @@ if(BUILD_TESTING) ydb/test_archive1.tdb ydb/test_logmax.tdb ydb/test_query.tdb - ydb/test_rand_insert_mult_db.tdb ydb/test_txn_abort5.tdb ydb/test_txn_abort5a.tdb ydb/test_txn_abort6.tdb diff --git a/src/tests/drd.suppressions b/src/tests/drd.suppressions index b0adfd39eb7..2bb788c46cf 100644 --- a/src/tests/drd.suppressions +++ b/src/tests/drd.suppressions @@ -57,6 +57,13 @@ drd:ConflictingAccess fun:clone } +{ + EvictorSignalUnlocked + drd:CondRaceErr + fun:pthread_cond_signal@* + fun:_ZL16toku_cond_signalP9toku_cond + fun:_ZN7evictor22signal_eviction_threadEv +} { diff --git a/src/tests/queries_with_deletes.cc b/src/tests/queries_with_deletes.cc index 2c868a2dfe7..e76c9c44cb2 100644 --- a/src/tests/queries_with_deletes.cc +++ b/src/tests/queries_with_deletes.cc @@ -22,7 +22,7 @@ int test_main (int argc, char * const argv[]) { r = db_env_create(&env, 0); CKERR(r); env->set_errfile(env, stderr); // set a cachetable size of 10K - uint32_t cachesize = 10*1024; + uint32_t cachesize = 100*1024; // as part of #4503, arbitrarily increasing sizze of cachetable // the idea is to make it small enough such that all data // cannot fit in the cachetable, but big enough such that diff --git a/src/tests/test_rand_insert_mult_db.cc b/src/tests/test_rand_insert_mult_db.cc deleted file mode 100644 index 80b8ffafcae..00000000000 --- a/src/tests/test_rand_insert_mult_db.cc +++ /dev/null @@ -1,72 +0,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 Tokutek Inc. All rights reserved." -#include "test.h" - -#include -#include - -#include -#include -#include -#include - - -static void -test_mult_insert (int num_dbs, int num_elements) { - - DB_TXN * const null_txn = 0; - int r; - - r = system("rm -rf " ENVDIR); - CKERR(r); - r=toku_os_mkdir(ENVDIR, S_IRWXU+S_IRWXG+S_IRWXO); assert(r==0); - - /* create the dup database file */ - DB_ENV *env; - r = db_env_create(&env, 0); assert(r == 0); - r = env->set_cachesize(env, 0, 4096*4, 1); assert(r==0); - r = env->open(env, ENVDIR, DB_CREATE+DB_PRIVATE+DB_INIT_MPOOL, 0); assert(r == 0); - - DB* dbs[num_dbs]; - for (int i = 0; i < num_dbs; i++) { - r = db_create(&dbs[i], env, 0); - assert(r == 0); - r = dbs[i]->set_flags(dbs[i], 0); - assert(r == 0); - r = dbs[i]->set_pagesize(dbs[i], 4096); - assert(r == 0); - char curr_name[15]; - sprintf(curr_name, "main_%d", i); - r = dbs[i]->open(dbs[i], null_txn, curr_name, 0, DB_BTREE, DB_CREATE, 0666); - assert(r == 0); - } - - - /* insert n/2 pairs */ - for (int i=0; iput(dbs[j], null_txn, dbt_init(&key, &rand_key, sizeof rand_key), dbt_init(&val, &i, sizeof i), 0); - assert(r == 0); - } - } - - /* reopen the database to force nonleaf buffering */ - for (int i = 0; i < num_dbs; i++) { - r = dbs[i]->close(dbs[i], 0); - assert(r == 0); - } - - r = env->close(env, 0); assert(r == 0); -} - -int -test_main(int argc, char *const argv[]) { - parse_args(argc, argv); - test_mult_insert(50, 1000); - - return 0; -} diff --git a/src/ydb.cc b/src/ydb.cc index e6dcad20aab..0e78246ad43 100644 --- a/src/ydb.cc +++ b/src/ydb.cc @@ -741,6 +741,7 @@ env_open(DB_ENV * env, const char *home, uint32_t flags, int mode) { int r; bool newenv; // true iff creating a new environment uint32_t unused_flags=flags; + CHECKPOINTER cp; if (env_opened(env)) { r = toku_ydb_do_error(env, EINVAL, "The environment is already open\n"); @@ -996,7 +997,8 @@ env_open(DB_ENV * env, const char *home, uint32_t flags, int mode) { r = locked_txn_commit(txn, 0); assert_zero(r); } - r = toku_checkpoint(env->i->cachetable, env->i->logger, NULL, NULL, NULL, NULL, STARTUP_CHECKPOINT); + cp = toku_cachetable_get_checkpointer(env->i->cachetable); + r = toku_checkpoint(cp, env->i->logger, NULL, NULL, NULL, NULL, STARTUP_CHECKPOINT); assert_zero(r); env_fs_poller(env); // get the file system state at startup env_fs_init_minicron(env); @@ -1060,7 +1062,8 @@ env_close(DB_ENV * env, uint32_t flags) { if (env->i->cachetable) { toku_cachetable_minicron_shutdown(env->i->cachetable); if (env->i->logger) { - r = toku_checkpoint(env->i->cachetable, env->i->logger, NULL, NULL, NULL, NULL, SHUTDOWN_CHECKPOINT); + CHECKPOINTER cp = toku_cachetable_get_checkpointer(env->i->cachetable); + r = toku_checkpoint(cp, env->i->logger, NULL, NULL, NULL, NULL, SHUTDOWN_CHECKPOINT); if (r) { err_msg = "Cannot close environment (error during checkpoint)\n"; toku_ydb_do_error(env, r, "%s", err_msg); @@ -1073,7 +1076,7 @@ env_close(DB_ENV * env, uint32_t flags) { goto panic_and_quit_early; } //Do a second checkpoint now that the rollback cachefile is closed. - r = toku_checkpoint(env->i->cachetable, env->i->logger, NULL, NULL, NULL, NULL, SHUTDOWN_CHECKPOINT); + r = toku_checkpoint(cp, env->i->logger, NULL, NULL, NULL, NULL, SHUTDOWN_CHECKPOINT); if (r) { err_msg = "Cannot close environment (error during checkpoint)\n"; toku_ydb_do_error(env, r, "%s", err_msg); @@ -1427,7 +1430,8 @@ env_set_verbose(DB_ENV * env, uint32_t UU(which), int UU(onoff)) { static int toku_env_txn_checkpoint(DB_ENV * env, uint32_t kbyte __attribute__((__unused__)), uint32_t min __attribute__((__unused__)), uint32_t flags __attribute__((__unused__))) { - int r = toku_checkpoint(env->i->cachetable, env->i->logger, + CHECKPOINTER cp = toku_cachetable_get_checkpointer(env->i->cachetable); + int r = toku_checkpoint(cp, env->i->logger, checkpoint_callback_f, checkpoint_callback_extra, checkpoint_callback2_f, checkpoint_callback2_extra, CLIENT_CHECKPOINT); diff --git a/src/ydb_txn.cc b/src/ydb_txn.cc index 30db09acf7d..60fef32fc83 100644 --- a/src/ydb_txn.cc +++ b/src/ydb_txn.cc @@ -259,7 +259,8 @@ locked_txn_commit_with_progress(DB_TXN *txn, uint32_t flags, TXN_PROGRESS_POLL_FUNCTION poll, void* poll_extra) { TOKUTXN ttxn = db_txn_struct_i(txn)->tokutxn; if (toku_txn_requires_checkpoint(ttxn)) { - toku_checkpoint(txn->mgrp->i->cachetable, txn->mgrp->i->logger, NULL, NULL, NULL, NULL, TXN_COMMIT_CHECKPOINT); + CHECKPOINTER cp = toku_cachetable_get_checkpointer(txn->mgrp->i->cachetable); + toku_checkpoint(cp, txn->mgrp->i->logger, NULL, NULL, NULL, NULL, TXN_COMMIT_CHECKPOINT); } bool holds_mo_lock = false; if (!toku_txn_is_read_only(db_txn_struct_i(txn)->tokutxn)) {