refs #4606, merge to main

git-svn-id: file:///svn/toku/tokudb@46797 c7de825b-a66e-492c-adef-691d508d4ae1
This commit is contained in:
Zardosht Kasheff 2013-04-17 00:01:03 -04:00 committed by Yoni Fogel
parent 8cc19d274c
commit 7ea6dd4171
121 changed files with 4778 additions and 2960 deletions

View file

@ -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}")

486
ft/cachetable-internal.h Normal file
View file

@ -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;
// <CER> 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.

File diff suppressed because it is too large Load diff

View file

@ -9,6 +9,7 @@
#include <fcntl.h>
#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

View file

@ -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) {

View file

@ -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);

View file

@ -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);

View file

@ -9,6 +9,7 @@
#include <fttypes.h>
#include <ft-flusher.h>
#include <ft-internal.h>
#include "ft.h"
static void
ftnode_get_key_and_fullhash(
@ -57,7 +58,8 @@ cachetable_put_empty_node_with_dep_nodes(
dependent_fullhash,
dependent_dirty_bits,
name,
fullhash);
fullhash,
toku_node_save_ct_pair);
assert_zero(r);
*result = new_node;
@ -131,10 +133,44 @@ toku_pin_ftnode(
bool apply_ancestor_messages, // this bool is probably temporary, for #3972, once we know how range query estimates work, will revisit this
FTNODE *node_p,
bool* msgs_applied)
{
toku_cachetable_begin_batched_pin(brt->ft->cf);
int r = toku_pin_ftnode_batched(
brt,
blocknum,
fullhash,
unlockers,
ancestors,
bounds,
bfe,
may_modify_node,
apply_ancestor_messages,
false,
node_p,
msgs_applied
);
toku_cachetable_end_batched_pin(brt->ft->cf);
return r;
}
int
toku_pin_ftnode_batched(
FT_HANDLE brt,
BLOCKNUM blocknum,
uint32_t fullhash,
UNLOCKERS unlockers,
ANCESTORS ancestors,
const PIVOT_BOUNDS bounds,
FTNODE_FETCH_EXTRA bfe,
bool may_modify_node,
bool apply_ancestor_messages, // this bool is probably temporary, for #3972, once we know how range query estimates work, will revisit this
bool end_batch_on_success,
FTNODE *node_p,
bool* msgs_applied)
{
void *node_v;
*msgs_applied = false;
int r = toku_cachetable_get_and_pin_nonblocking(
int r = toku_cachetable_get_and_pin_nonblocking_batched(
brt->ft->cf,
blocknum,
fullhash,
@ -149,6 +185,9 @@ toku_pin_ftnode(
unlockers);
if (r==0) {
FTNODE node = (FTNODE) node_v;
if (end_batch_on_success) {
toku_cachetable_end_batched_pin(brt->ft->cf);
}
if (apply_ancestor_messages && node->height == 0) {
toku_apply_ancestors_messages_to_node(brt, node, ancestors, bounds, msgs_applied);
}
@ -174,6 +213,31 @@ toku_pin_ftnode_off_client_thread(
uint32_t num_dependent_nodes,
FTNODE* dependent_nodes,
FTNODE *node_p)
{
toku_cachetable_begin_batched_pin(h->cf);
toku_pin_ftnode_off_client_thread_batched(
h,
blocknum,
fullhash,
bfe,
may_modify_node,
num_dependent_nodes,
dependent_nodes,
node_p
);
toku_cachetable_end_batched_pin(h->cf);
}
void
toku_pin_ftnode_off_client_thread_batched(
FT h,
BLOCKNUM blocknum,
uint32_t fullhash,
FTNODE_FETCH_EXTRA bfe,
bool may_modify_node,
uint32_t num_dependent_nodes,
FTNODE* dependent_nodes,
FTNODE *node_p)
{
void *node_v;
CACHEFILE dependent_cf[num_dependent_nodes];
@ -187,7 +251,7 @@ toku_pin_ftnode_off_client_thread(
dependent_dirty_bits[i] = (enum cachetable_dirty) dependent_nodes[i]->dirty;
}
int r = toku_cachetable_get_and_pin_with_dep_pairs(
int r = toku_cachetable_get_and_pin_with_dep_pairs_batched(
h->cf,
blocknum,
fullhash,
@ -232,8 +296,7 @@ toku_unpin_ftnode_off_client_thread(FT ft, FTNODE node)
{
int r = toku_cachetable_unpin(
ft->cf,
node->thisnodename,
node->fullhash,
node->ct_pair,
(enum cachetable_dirty) node->dirty,
make_ftnode_pair_attr(node)
);
@ -253,8 +316,7 @@ toku_unpin_ftnode_read_only(FT_HANDLE brt, FTNODE node)
{
int r = toku_cachetable_unpin(
brt->ft->cf,
node->thisnodename,
node->fullhash,
node->ct_pair,
(enum cachetable_dirty) node->dirty,
make_invalid_pair_attr()
);

View file

@ -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.

View file

@ -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
);

View file

@ -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);

View file

@ -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();

View file

@ -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)
);

View file

@ -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

View file

@ -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);

View file

@ -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 {

View file

@ -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

View file

@ -44,11 +44,11 @@ void memarena_clear (MEMARENA ma) {
static size_t
round_to_page (size_t size) {
const size_t PAGE_SIZE = 4096;
const size_t result = PAGE_SIZE+((size-1)&~(PAGE_SIZE-1));
assert(0==(result&(PAGE_SIZE-1))); // make sure it's aligned
const size_t _PAGE_SIZE = 4096;
const size_t result = _PAGE_SIZE+((size-1)&~(_PAGE_SIZE-1));
assert(0==(result&(_PAGE_SIZE-1))); // make sure it's aligned
assert(result>=size); // make sure it's not too small
assert(result<size+PAGE_SIZE); // make sure we didn't grow by more than a page.
assert(result<size+_PAGE_SIZE); // make sure we didn't grow by more than a page.
return result;
}

View file

@ -40,46 +40,46 @@ minicron_do (void *pv)
struct minicron *CAST_FROM_VOIDP(p, pv);
toku_mutex_lock(&p->mutex);
while (1) {
if (p->do_shutdown) {
toku_mutex_unlock(&p->mutex);
return 0;
}
if (p->period_in_seconds==0) {
// if we aren't supposed to do it then just do an untimed wait.
toku_cond_wait(&p->condvar, &p->mutex);
} else {
// Recompute the wakeup time every time (instead of once per call to f) in case the period changges.
toku_timespec_t wakeup_at = p->time_of_last_call_to_f;
wakeup_at.tv_sec += p->period_in_seconds;
toku_timespec_t now;
toku_gettime(&now);
//printf("wakeup at %.6f (after %d seconds) now=%.6f\n", wakeup_at.tv_sec + wakeup_at.tv_nsec*1e-9, p->period_in_seconds, now.tv_sec + now.tv_nsec*1e-9);
int r = toku_cond_timedwait(&p->condvar, &p->mutex, &wakeup_at);
if (r!=0 && r!=ETIMEDOUT) fprintf(stderr, "%s:%d r=%d (%s)", __FILE__, __LINE__, r, strerror(r));
assert(r==0 || r==ETIMEDOUT);
}
// Now we woke up, and we should figure out what to do
if (p->do_shutdown) {
toku_mutex_unlock(&p->mutex);
return 0;
}
if (p->period_in_seconds >0) {
// maybe do a checkpoint
toku_timespec_t now;
toku_gettime(&now);
toku_timespec_t time_to_call = p->time_of_last_call_to_f;
time_to_call.tv_sec += p->period_in_seconds;
int compare = timespec_compare(&time_to_call, &now);
//printf("compare(%.6f, %.6f)=%d\n", time_to_call.tv_sec + time_to_call.tv_nsec*1e-9, now.tv_sec+now.tv_nsec*1e-9, compare);
if (compare <= 0) {
toku_mutex_unlock(&p->mutex);
int r = p->f(p->arg);
assert(r==0);
toku_mutex_lock(&p->mutex);
toku_gettime(&p->time_of_last_call_to_f); // the period is measured between calls to f.
}
}
if (p->do_shutdown) {
toku_mutex_unlock(&p->mutex);
return 0;
}
if (p->period_in_seconds==0) {
// if we aren't supposed to do it then just do an untimed wait.
toku_cond_wait(&p->condvar, &p->mutex);
} else {
// Recompute the wakeup time every time (instead of once per call to f) in case the period changges.
toku_timespec_t wakeup_at = p->time_of_last_call_to_f;
wakeup_at.tv_sec += p->period_in_seconds;
toku_timespec_t now;
toku_gettime(&now);
//printf("wakeup at %.6f (after %d seconds) now=%.6f\n", wakeup_at.tv_sec + wakeup_at.tv_nsec*1e-9, p->period_in_seconds, now.tv_sec + now.tv_nsec*1e-9);
int r = toku_cond_timedwait(&p->condvar, &p->mutex, &wakeup_at);
if (r!=0 && r!=ETIMEDOUT) fprintf(stderr, "%s:%d r=%d (%s)", __FILE__, __LINE__, r, strerror(r));
assert(r==0 || r==ETIMEDOUT);
}
// Now we woke up, and we should figure out what to do
if (p->do_shutdown) {
toku_mutex_unlock(&p->mutex);
return 0;
}
if (p->period_in_seconds >0) {
// maybe do a checkpoint
toku_timespec_t now;
toku_gettime(&now);
toku_timespec_t time_to_call = p->time_of_last_call_to_f;
time_to_call.tv_sec += p->period_in_seconds;
int compare = timespec_compare(&time_to_call, &now);
//printf("compare(%.6f, %.6f)=%d\n", time_to_call.tv_sec + time_to_call.tv_nsec*1e-9, now.tv_sec+now.tv_nsec*1e-9, compare);
if (compare <= 0) {
toku_mutex_unlock(&p->mutex);
int r = p->f(p->arg);
assert(r==0);
toku_mutex_lock(&p->mutex);
toku_gettime(&p->time_of_last_call_to_f); // the period is measured between calls to f.
}
}
}
}

View file

@ -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));

View file

@ -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;

View file

@ -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),

View file

@ -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);
}

View file

@ -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

View file

@ -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)

View file

@ -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

View file

@ -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;
}

View file

@ -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;

View file

@ -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);

View file

@ -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

View file

@ -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);
}

View file

@ -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);

View file

@ -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);

View file

@ -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

View file

@ -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

View file

@ -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);

View file

@ -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;
}

View file

@ -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

View file

@ -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

View file

@ -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);

View file

@ -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

View file

@ -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);
}

View file

@ -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);
}

View file

@ -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);
}

View file

@ -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));
}

View file

@ -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 );

View file

@ -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);

View file

@ -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);

View file

@ -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 );

View file

@ -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;

View file

@ -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

View file

@ -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

View file

@ -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

View file

@ -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

View file

@ -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);

View file

@ -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);

View file

@ -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);

View file

@ -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);

View file

@ -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);

View file

@ -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);

View file

@ -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;
}

View file

@ -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;
}

View file

@ -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);

View file

@ -28,14 +28,14 @@ test_cachetable_def_flush (int n) {
for (i=0; i<n; i++) {
uint32_t hi;
hi = toku_cachetable_hash(f1, make_blocknum(i));
r = toku_cachetable_put(f1, make_blocknum(i), hi, (void *)(long)i, make_pair_attr(1), wc);
r = toku_cachetable_put(f1, make_blocknum(i), hi, (void *)(long)i, make_pair_attr(1), wc, put_callback_nop);
assert(r == 0);
r = toku_cachetable_unpin(f1, make_blocknum(i), hi, CACHETABLE_CLEAN, make_pair_attr(1));
r = toku_test_cachetable_unpin(f1, make_blocknum(i), hi, CACHETABLE_CLEAN, make_pair_attr(1));
assert(r == 0);
hi = toku_cachetable_hash(f2, make_blocknum(i));
r = toku_cachetable_put(f2, make_blocknum(i), hi, (void *)(long)i, make_pair_attr(1), wc);
r = toku_cachetable_put(f2, make_blocknum(i), hi, (void *)(long)i, make_pair_attr(1), wc, put_callback_nop);
assert(r == 0);
r = toku_cachetable_unpin(f2, make_blocknum(i), hi, CACHETABLE_CLEAN, make_pair_attr(1));
r = toku_test_cachetable_unpin(f2, make_blocknum(i), hi, CACHETABLE_CLEAN, make_pair_attr(1));
assert(r == 0);
}
toku_cachetable_verify(ct);
@ -47,12 +47,12 @@ test_cachetable_def_flush (int n) {
hi = toku_cachetable_hash(f1, make_blocknum(i));
r = toku_cachetable_maybe_get_and_pin(f1, make_blocknum(i), hi, &v);
assert(r == 0 && v == (void *)(long)i);
r = toku_cachetable_unpin(f1, make_blocknum(i), hi, CACHETABLE_CLEAN, make_pair_attr(1));
r = toku_test_cachetable_unpin(f1, make_blocknum(i), hi, CACHETABLE_CLEAN, make_pair_attr(1));
assert(r == 0);
hi = toku_cachetable_hash(f2, make_blocknum(i));
r = toku_cachetable_maybe_get_and_pin(f2, make_blocknum(i), hi, &v);
assert(r == 0 && v == (void *)(long)i);
r = toku_cachetable_unpin(f2, make_blocknum(i), hi, CACHETABLE_CLEAN, make_pair_attr(1));
r = toku_test_cachetable_unpin(f2, make_blocknum(i), hi, CACHETABLE_CLEAN, make_pair_attr(1));
assert(r == 0);
}
@ -70,7 +70,7 @@ test_cachetable_def_flush (int n) {
hi = toku_cachetable_hash(f2, make_blocknum(i));
r = toku_cachetable_maybe_get_and_pin(f2, make_blocknum(i), hi, &v);
assert(r == 0);
r = toku_cachetable_unpin(f2, make_blocknum(i), hi, CACHETABLE_CLEAN, make_pair_attr(1));
r = toku_test_cachetable_unpin(f2, make_blocknum(i), hi, CACHETABLE_CLEAN, make_pair_attr(1));
assert(r == 0);
}

View file

@ -26,18 +26,18 @@ flush (CACHEFILE cf __attribute__((__unused__)),
static int
fetch (
CACHEFILE cf,
CACHEFILE UU(cf),
PAIR UU(p),
int UU(fd),
CACHEKEY key,
uint32_t hash,
uint32_t UU(hash),
void **vptr,
void** UU(dd),
PAIR_ATTR *sizep,
int *dirtyp,
void *extra
void *UU(extra)
)
{
cf = cf; hash = hash; extra = extra;
*sizep = make_pair_attr((long) key.b);
*vptr = toku_malloc(sizep->size);
*dirtyp = 0;
@ -68,7 +68,7 @@ cachetable_getandpin_test (int n) {
assert(r == 0);
assert(size == i);
r = toku_cachetable_unpin(f1, make_blocknum(i), hi, CACHETABLE_CLEAN, make_pair_attr(i));
r = toku_test_cachetable_unpin(f1, make_blocknum(i), hi, CACHETABLE_CLEAN, make_pair_attr(i));
assert(r == 0);
}
toku_cachetable_verify(ct);

View file

@ -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);
}

View file

@ -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);

View file

@ -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

View file

@ -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

View file

@ -17,12 +17,12 @@ const int item_size = 1;
int n_flush, n_write_me, n_keep_me, n_fetch;
static void flush(
CACHEFILE cf,
CACHEFILE UU(cf),
int UU(fd),
CACHEKEY key,
void *value,
CACHEKEY UU(key),
void *UU(value),
void** UU(dd),
void *extraargs,
void *UU(extraargs),
PAIR_ATTR size,
PAIR_ATTR* UU(new_size),
bool write_me,
@ -31,7 +31,6 @@ static void flush(
bool UU(is_clone)
)
{
cf = cf; key = key; value = value; extraargs = extraargs;
// assert(key == make_blocknum((long)value));
assert(size.size == item_size);
n_flush++;
@ -40,18 +39,18 @@ static void flush(
}
static int fetch(
CACHEFILE cf,
CACHEFILE UU(cf),
PAIR UU(p),
int UU(fd),
CACHEKEY key,
uint32_t fullhash,
void **value,
CACHEKEY UU(key),
uint32_t UU(fullhash),
void **UU(value),
void** UU(dd),
PAIR_ATTR *sizep,
PAIR_ATTR *UU(sizep),
int *dirtyp,
void *extraargs
void *UU(extraargs)
)
{
cf = cf; key = key; fullhash = fullhash; value = value; sizep = sizep; extraargs = extraargs;
n_fetch++;
sleep(10);
*value = 0;
@ -93,10 +92,10 @@ static void cachetable_prefetch_checkpoint_test(int n, enum cachetable_dirty dir
for (i=0; i<n; i++) {
CACHEKEY key = make_blocknum(i);
uint32_t hi = toku_cachetable_hash(f1, key);
r = toku_cachetable_put(f1, key, hi, (void *)(long)i, make_pair_attr(1), wc);
r = toku_cachetable_put(f1, key, hi, (void *)(long)i, make_pair_attr(1), wc, put_callback_nop);
assert(r == 0);
r = toku_cachetable_unpin(f1, key, hi, dirty, make_pair_attr(item_size));
r = toku_test_cachetable_unpin(f1, key, hi, dirty, make_pair_attr(item_size));
assert(r == 0);
void *v;
@ -114,8 +113,8 @@ static void cachetable_prefetch_checkpoint_test(int n, enum cachetable_dirty dir
// the checkpoint should cause n writes, but since n <= the cachetable size,
// all items should be kept in the cachetable
n_flush = n_write_me = n_keep_me = n_fetch = 0;
r = toku_checkpoint(ct, NULL, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT);
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
r = toku_checkpoint(cp, NULL, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT);
assert(r == 0);
assert(n_flush == n && n_write_me == n && n_keep_me == n);
@ -127,7 +126,7 @@ static void cachetable_prefetch_checkpoint_test(int n, enum cachetable_dirty dir
r = toku_cachetable_maybe_get_and_pin(f1, key, hi, &v);
if (r != 0)
continue;
r = toku_cachetable_unpin(f1, key, hi, CACHETABLE_CLEAN, make_pair_attr(item_size));
r = toku_test_cachetable_unpin(f1, key, hi, CACHETABLE_CLEAN, make_pair_attr(item_size));
assert(r == 0);
int its_dirty;
@ -144,7 +143,7 @@ static void cachetable_prefetch_checkpoint_test(int n, enum cachetable_dirty dir
// a subsequent checkpoint should cause no flushes, or writes since all of the items are clean
n_flush = n_write_me = n_keep_me = n_fetch = 0;
r = toku_checkpoint(ct, NULL, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT);
r = toku_checkpoint(cp, NULL, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT);
assert(r == 0);
assert(n_flush == 0 && n_write_me == 0 && n_keep_me == 0);

View file

@ -29,6 +29,7 @@ static int fetch_calls = 0;
static int
fetch (CACHEFILE f __attribute__((__unused__)),
PAIR UU(p),
int UU(fd),
CACHEKEY k __attribute__((__unused__)),
uint32_t fullhash __attribute__((__unused__)),

View file

@ -30,6 +30,7 @@ static int fetch_calls = 0;
static int
fetch (CACHEFILE f __attribute__((__unused__)),
PAIR UU(p),
int UU(fd),
CACHEKEY k __attribute__((__unused__)),
uint32_t fullhash __attribute__((__unused__)),
@ -88,7 +89,7 @@ static void cachetable_prefetch_full_test (bool partial_fetch) {
0
);
assert(r==0);
r = toku_cachetable_unpin(f1, key, fullhash, CACHETABLE_CLEAN, make_pair_attr(1));
r = toku_test_cachetable_unpin(f1, key, fullhash, CACHETABLE_CLEAN, make_pair_attr(1));
}
r = toku_cachefile_prefetch(f1, key, fullhash, wc, fetch, def_pf_req_callback, def_pf_callback, 0, NULL);

View file

@ -8,6 +8,7 @@
#include "includes.h"
#include "test.h"
#include "cachetable-internal.h"
static int flush_calls = 0;
static int flush_evict_calls = 0;
@ -28,10 +29,11 @@ flush (CACHEFILE f __attribute__((__unused__)),
bool UU(is_clone)
) {
assert(w == false);
sleep(1);
flush_calls++;
if (keep == false) {
flush_evict_calls++;
if (verbose) printf("%s:%d flush %" PRId64 "\n", __FUNCTION__, __LINE__, k.b);
if (verbose) printf("%s:%d flush %" PRId64 "\n", __FUNCTION__, __LINE__, k.b);
evicted_keys |= 1 << k.b;
}
}
@ -40,6 +42,7 @@ static int fetch_calls = 0;
static int
fetch (CACHEFILE f __attribute__((__unused__)),
PAIR UU(p),
int UU(fd),
CACHEKEY k,
uint32_t fullhash __attribute__((__unused__)),
@ -65,6 +68,8 @@ static void cachetable_prefetch_flowcontrol_test (int cachetable_size_limit) {
int r;
CACHETABLE ct;
r = toku_create_cachetable(&ct, cachetable_size_limit, ZERO_LSN, NULL_LOGGER); assert(r == 0);
evictor_test_helpers::set_hysteresis_limits(&ct->ev, cachetable_size_limit, cachetable_size_limit);
evictor_test_helpers::disable_ev_thread(&ct->ev);
char fname1[] = __SRCFILE__ "test1.dat";
unlink(fname1);
CACHEFILE f1;
@ -75,40 +80,33 @@ static void cachetable_prefetch_flowcontrol_test (int cachetable_size_limit) {
wc.flush_callback = flush;
// prefetch keys 0 .. N-1. they should all fit in the cachetable
for (i=0; i<cachetable_size_limit; i++) {
for (i=0; i<cachetable_size_limit+1; i++) {
CACHEKEY key = make_blocknum(i);
uint32_t fullhash = toku_cachetable_hash(f1, key);
r = toku_cachefile_prefetch(f1, key, fullhash, wc, fetch, def_pf_req_callback, def_pf_callback, 0, NULL);
bool doing_prefetch = false;
r = toku_cachefile_prefetch(f1, key, fullhash, wc, fetch, def_pf_req_callback, def_pf_callback, 0, &doing_prefetch);
assert(doing_prefetch);
toku_cachetable_verify(ct);
}
// wait for all of the blocks to be fetched
sleep(10);
sleep(3);
// prefetch keys N .. 2*N-1. 0 .. N-1 should be evicted.
for (i=i; i<2*cachetable_size_limit; i++) {
for (i=i+1; i<2*cachetable_size_limit; i++) {
CACHEKEY key = make_blocknum(i);
uint32_t fullhash = toku_cachetable_hash(f1, key);
r = toku_cachefile_prefetch(f1, key, fullhash, wc, fetch, def_pf_req_callback, def_pf_callback, 0, NULL);
bool doing_prefetch = false;
r = toku_cachefile_prefetch(f1, key, fullhash, wc, fetch, def_pf_req_callback, def_pf_callback, 0, &doing_prefetch);
assert(!doing_prefetch);
toku_cachetable_verify(ct);
// sleep(1);
}
// wait for everything to finish
sleep(10);
#if 0 //If we flush using reader thread.
assert(flush_evict_calls == cachetable_size_limit);
assert(evicted_keys == (1 << cachetable_size_limit)-1);
#else
assert(flush_evict_calls == 0);
assert(evicted_keys == 0);
#endif
char *error_string;
r = toku_cachefile_close(&f1, &error_string, false, ZERO_LSN); assert(r == 0);
if (verbose) printf("%s:%d 0x%x 0x%x\n", __FUNCTION__, __LINE__,
evicted_keys, (1 << (2*cachetable_size_limit))-1);
assert(evicted_keys == (1 << (2*cachetable_size_limit))-1);
evicted_keys, (1 << (2*cachetable_size_limit))-1);
r = toku_cachetable_close(&ct); assert(r == 0 && ct == 0);
}

View file

@ -29,6 +29,7 @@ flush (CACHEFILE f __attribute__((__unused__)),
static int
fetch (CACHEFILE f __attribute__((__unused__)),
PAIR UU(p),
int UU(fd),
CACHEKEY k __attribute__((__unused__)),
uint32_t fullhash __attribute__((__unused__)),
@ -105,7 +106,7 @@ static void cachetable_prefetch_maybegetandpin_test (bool do_partial_fetch) {
0
);
assert(r==0);
r = toku_cachetable_unpin(f1, key, fullhash, CACHETABLE_CLEAN, make_pair_attr(1));
r = toku_test_cachetable_unpin(f1, key, fullhash, CACHETABLE_CLEAN, make_pair_attr(1));
}
struct timeval tstart;
@ -132,7 +133,7 @@ static void cachetable_prefetch_maybegetandpin_test (bool do_partial_fetch) {
toku_cachetable_verify(ct);
r = toku_cachetable_unpin(f1, key, fullhash, CACHETABLE_CLEAN, make_pair_attr(1));
r = toku_test_cachetable_unpin(f1, key, fullhash, CACHETABLE_CLEAN, make_pair_attr(1));
assert(r == 0);
toku_cachetable_verify(ct);

View file

@ -9,6 +9,7 @@
static int
fetch (CACHEFILE f __attribute__((__unused__)),
PAIR UU(p),
int UU(fd),
CACHEKEY k __attribute__((__unused__)),
uint32_t fullhash __attribute__((__unused__)),
@ -58,7 +59,7 @@ static void cachetable_prefetch_maybegetandpin_test (void) {
assert(i>1);
toku_cachetable_verify(ct);
r = toku_cachetable_unpin(f1, key, fullhash, CACHETABLE_CLEAN, make_pair_attr(1));
r = toku_test_cachetable_unpin(f1, key, fullhash, CACHETABLE_CLEAN, make_pair_attr(1));
assert(r == 0);
toku_cachetable_verify(ct);

View file

@ -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);

View file

@ -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

View file

@ -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);
}

View file

@ -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<n_keys; i++) {
if (keys[i].b==k.b) {
assert(vals[i]==value);
if (!keep_me) {
if (verbose) printf("%s: %d/%d %" PRIx64 "\n", __FUNCTION__, i, n_keys, k.b);
keys[i]=keys[n_keys-1];
vals[i]=vals[n_keys-1];
n_keys--;
test_mutex_unlock();
return;
}
}
}
fprintf(stderr, "Whoops\n");
abort();
test_mutex_unlock();
}
static int r_fetch (CACHEFILE f __attribute__((__unused__)),
int UU(fd),
CACHEKEY key __attribute__((__unused__)),
uint32_t fullhash __attribute__((__unused__)),
void**value __attribute__((__unused__)),
void** UU(dd),
PAIR_ATTR *sizep __attribute__((__unused__)),
int *dirtyp __attribute__((__unused__)),
void*extraargs __attribute__((__unused__))) {
// fprintf(stderr, "Whoops, this should never be called");
return -42;
}
static void test_rename (void) {
CACHETABLE t;
CACHEFILE f;
int i;
int r;
test_mutex_init();
const char fname[] = __SRCFILE__ "rename.dat";
r=toku_create_cachetable(&t, KEYLIMIT, ZERO_LSN, NULL_LOGGER); assert(r==0);
unlink(fname);
r = toku_cachetable_openf(&f, t, fname, O_RDWR|O_CREAT, S_IRWXU|S_IRWXG|S_IRWXO);
assert(r==0);
for (i=0; i<TRIALLIMIT; i++) {
int ra = random()%3;
if (ra<=1) {
// Insert something
CACHEKEY nkey = make_blocknum(random());
long nval = random();
if (verbose) printf("n_keys=%d Insert %08" PRIx64 "\n", n_keys, nkey.b);
uint32_t hnkey = toku_cachetable_hash(f, nkey);
CACHETABLE_WRITE_CALLBACK wc = def_write_callback(NULL);
wc.flush_callback = r_flush;
r = toku_cachetable_put(f, nkey, hnkey,
(void*)nval, make_pair_attr(1),
wc);
assert(r==0);
test_mutex_lock();
while (n_keys >= KEYLIMIT) {
test_mutex_unlock();
toku_pthread_yield(); maybe_flush(t);
test_mutex_lock();
}
assert(n_keys<KEYLIMIT);
keys[n_keys] = nkey;
vals[n_keys] = (void*)nval;
n_keys++;
test_mutex_unlock();
r = toku_cachetable_unpin(f, nkey, hnkey, CACHETABLE_DIRTY, make_pair_attr(1));
assert(r==0);
} else if (ra==2 && n_keys>0) {
// Rename something
int objnum = random()%n_keys;
CACHEKEY nkey = make_blocknum(random());
test_mutex_lock();
CACHEKEY okey = keys[objnum];
test_mutex_unlock();
void *current_value;
long current_size;
if (verbose) printf("Rename %" PRIx64 " to %" PRIx64 "\n", okey.b, nkey.b);
CACHETABLE_WRITE_CALLBACK wc = def_write_callback(NULL);
wc.flush_callback = r_flush;
r = toku_cachetable_get_and_pin(f, okey, toku_cachetable_hash(f, okey), &current_value, &current_size, wc, r_fetch, def_pf_req_callback, def_pf_callback, true, 0);
if (r == -42) continue;
assert(r==0);
r = toku_cachetable_rename(f, okey, nkey);
assert(r==0);
test_mutex_lock();
// assert(objnum < n_keys && keys[objnum] == okey);
// get_and_pin may reorganize the keys[], so we need to find it again
int j;
for (j=0; j < n_keys; j++)
if (keys[j].b == okey.b)
break;
assert(j < n_keys);
keys[j]=nkey;
test_mutex_unlock();
r = toku_cachetable_unpin(f, nkey, toku_cachetable_hash(f, nkey), CACHETABLE_DIRTY, make_pair_attr(1));
}
}
// test rename fails if old key does not exist in the cachetable
CACHEKEY okey, nkey;
while (1) {
okey = make_blocknum(random());
void *v;
r = toku_cachetable_maybe_get_and_pin(f, okey, toku_cachetable_hash(f, okey), &v);
if (r != 0)
break;
r = toku_cachetable_unpin(f, okey, toku_cachetable_hash(f, okey), CACHETABLE_CLEAN, make_pair_attr(1));
assert(r == 0);
}
nkey = make_blocknum(random());
r = toku_cachetable_rename(f, okey, nkey);
assert(r != 0);
r = toku_cachefile_close(&f, 0, false, ZERO_LSN);
assert(r == 0);
r = toku_cachetable_close(&t);
assert(r == 0);
test_mutex_destroy();
assert(n_keys == 0);
}
int
test_main (int argc, const char *argv[]) {
// parse args
default_parse_args(argc, argv);
toku_os_initialize_settings(verbose);
// run tests
int i;
for (i=0; i<1; i++)
test_rename();
return 0;
}

View file

@ -34,6 +34,7 @@ static void f_flush (CACHEFILE f,
}
static int f_fetch (CACHEFILE f,
PAIR UU(p),
int UU(fd),
CACHEKEY key,
uint32_t fullhash __attribute__((__unused__)),
@ -74,8 +75,8 @@ static void writeit (void) {
for (j=0; j<BLOCKSIZE; j++) ((char*)buf)[j]=(char)((i+j)%256);
CACHETABLE_WRITE_CALLBACK wc = def_write_callback(NULL);
wc.flush_callback = f_flush;
r = toku_cachetable_put(f, key, fullhash, buf, make_pair_attr(BLOCKSIZE), wc); assert(r==0);
r = toku_cachetable_unpin(f, key, fullhash, CACHETABLE_CLEAN, make_pair_attr(BLOCKSIZE)); assert(r==0);
r = toku_cachetable_put(f, key, fullhash, buf, make_pair_attr(BLOCKSIZE), wc, put_callback_nop); assert(r==0);
r = toku_test_cachetable_unpin(f, key, fullhash, CACHETABLE_CLEAN, make_pair_attr(BLOCKSIZE)); assert(r==0);
}
gettimeofday(&end, 0);
double diff = toku_tdiff(&end, &start);
@ -98,7 +99,7 @@ static void readit (void) {
CACHETABLE_WRITE_CALLBACK wc = def_write_callback(NULL);
wc.flush_callback = f_flush;
r=toku_cachetable_get_and_pin(f, key, fullhash, &block, &current_size, wc, f_fetch, def_pf_req_callback, def_pf_callback, true, 0); assert(r==0);
r=toku_cachetable_unpin(f, key, fullhash, CACHETABLE_CLEAN, make_pair_attr(BLOCKSIZE)); assert(r==0);
r=toku_test_cachetable_unpin(f, key, fullhash, CACHETABLE_CLEAN, make_pair_attr(BLOCKSIZE)); assert(r==0);
}
r = toku_cachefile_close(&f, 0, false, ZERO_LSN); assert(r == 0);
r = toku_cachetable_close(&t); assert(r == 0);

View file

@ -75,7 +75,7 @@ test_clean (enum cachetable_dirty dirty, bool cloneable) {
wc.clone_callback = cloneable ? clone_callback : NULL;
wc.flush_callback = flush;
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, dirty, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, dirty, make_pair_attr(8));
check_flush = true;
clone_called = false;
@ -83,7 +83,8 @@ test_clean (enum cachetable_dirty dirty, bool cloneable) {
flush_called = false;
// begin checkpoint, since pair is clean, we should not
// have the clone called
r = toku_cachetable_begin_checkpoint(ct, NULL);
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
r = toku_cachetable_begin_checkpoint(cp, NULL);
assert_zero(r);
struct timeval tstart;
struct timeval tend;
@ -91,7 +92,7 @@ test_clean (enum cachetable_dirty dirty, bool cloneable) {
// test that having a pin that passes false for may_modify_value does not stall behind checkpoint
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, false, NULL);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
gettimeofday(&tend, NULL);
assert(tdelta_usec(&tend, &tstart) <= 2000000);
assert(!clone_called);
@ -118,7 +119,7 @@ test_clean (enum cachetable_dirty dirty, bool cloneable) {
}
// at this point, there should be no more dirty writes
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
gettimeofday(&tend, NULL);
if (cloneable || !dirty ) {
assert(tdelta_usec(&tend, &tstart) <= 2000000);
@ -128,7 +129,7 @@ test_clean (enum cachetable_dirty dirty, bool cloneable) {
}
r = toku_cachetable_end_checkpoint(
ct,
cp,
NULL,
NULL,
NULL

View file

@ -63,19 +63,20 @@ test_clean (enum cachetable_dirty dirty, bool cloneable) {
wc.clone_callback = cloneable ? clone_callback : NULL;
wc.flush_callback = flush;
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8));
// begin checkpoint, since pair is clean, we should not
// have the clone called
r = toku_cachetable_begin_checkpoint(ct, NULL);
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
r = toku_cachetable_begin_checkpoint(cp, NULL);
assert_zero(r);
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
// at this point, there should be no more dirty writes
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, dirty, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, dirty, make_pair_attr(8));
usleep(2*1024*1024);
r = toku_cachetable_end_checkpoint(
ct,
cp,
NULL,
NULL,
NULL

View file

@ -27,14 +27,14 @@ cachetable_test (void) {
r = toku_cachetable_maybe_get_and_pin(f1, make_blocknum(1), 1, &v1);
assert(r==-1);
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
// maybe_get_and_pin_clean should succeed, maybe_get_and_pin should fail
r = toku_cachetable_maybe_get_and_pin(f1, make_blocknum(1), 1, &v1);
assert(r==-1);
r = toku_cachetable_maybe_get_and_pin_clean(f1, make_blocknum(1), 1, &v1);
assert(r == 0);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8));
// maybe_get_and_pin_clean should succeed, maybe_get_and_pin should fail
r = toku_cachetable_maybe_get_and_pin(f1, make_blocknum(1), 1, &v1);
assert(r==0);
@ -43,20 +43,21 @@ cachetable_test (void) {
assert(r==-1);
r = toku_cachetable_maybe_get_and_pin_clean(f1, make_blocknum(1), 1, &v1);
assert(r==-1);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8));
// sanity check, this should still succeed, because the PAIR is dirty
r = toku_cachetable_maybe_get_and_pin(f1, make_blocknum(1), 1, &v1);
assert(r==0);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8));
r = toku_cachetable_begin_checkpoint(ct, NULL); assert(r == 0);
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8));
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
r = toku_cachetable_begin_checkpoint(cp, NULL); assert(r == 0);
// now these should fail, because the node should be pending a checkpoint
r = toku_cachetable_maybe_get_and_pin(f1, make_blocknum(1), 1, &v1);
assert(r==-1);
r = toku_cachetable_maybe_get_and_pin(f1, make_blocknum(1), 1, &v1);
assert(r==-1);
r = toku_cachetable_end_checkpoint(
ct,
cp,
NULL,
NULL,
NULL

View file

@ -50,6 +50,7 @@ flush (CACHEFILE f __attribute__((__unused__)),
static int
fetch (CACHEFILE f __attribute__((__unused__)),
PAIR UU(p),
int UU(fd),
CACHEKEY k __attribute__((__unused__)),
uint32_t fullhash __attribute__((__unused__)),
@ -102,11 +103,12 @@ cachetable_test (bool write_first, bool write_second, bool start_checkpoint) {
enum cachetable_dirty cd[2];
cd[0] = write_first ? CACHETABLE_DIRTY : CACHETABLE_CLEAN;
cd[1] = write_second ? CACHETABLE_DIRTY : CACHETABLE_CLEAN;
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
if (start_checkpoint) {
//
// should mark the v1 and v2 as pending
//
r = toku_cachetable_begin_checkpoint(ct, NULL); assert(r==0);
r = toku_cachetable_begin_checkpoint(cp, NULL); assert(r==0);
}
//
// This call should cause a flush for both
@ -139,13 +141,13 @@ cachetable_test (bool write_first, bool write_second, bool start_checkpoint) {
assert(!v2_written);
}
check_me = false;
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_cachetable_unpin(f1, make_blocknum(2), 2, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_cachetable_unpin(f1, make_blocknum(3), 3, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(2), 2, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(3), 3, CACHETABLE_CLEAN, make_pair_attr(8));
if (start_checkpoint) {
r = toku_cachetable_end_checkpoint(
ct,
cp,
NULL,
NULL,
NULL

View file

@ -50,7 +50,7 @@ static void kibbutz_work(void *fe_v)
CACHEFILE CAST_FROM_VOIDP(f1, fe_v);
sleep(2);
foo = true;
int r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
int r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
assert(r==0);
remove_background_job_from_cf(f1);
}
@ -88,13 +88,13 @@ run_test (void) {
assert(r==TOKUDB_TRY_AGAIN);
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
assert(foo);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8)); assert(r==0);
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8)); assert(r==0);
// now make sure we get TOKUDB_TRY_AGAIN when a partial fetch is involved
// first make sure value is there
r = toku_cachetable_get_and_pin_nonblocking(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL, NULL);
assert(r==0);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8)); assert(r==0);
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8)); assert(r==0);
// now make sure that we get TOKUDB_TRY_AGAIN for the partial fetch
r = toku_cachetable_get_and_pin_nonblocking(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, true_def_pf_req_callback, true_def_pf_callback, true, NULL, NULL);
assert(r==TOKUDB_TRY_AGAIN);
@ -105,13 +105,14 @@ run_test (void) {
//
r = toku_cachetable_get_and_pin_nonblocking(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL, NULL);
assert(r==0);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8)); assert(r==0);
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8)); assert(r==0);
// this should mark the PAIR as pending
r = toku_cachetable_begin_checkpoint(ct, NULL); assert(r == 0);
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
r = toku_cachetable_begin_checkpoint(cp, NULL); assert(r == 0);
r = toku_cachetable_get_and_pin_nonblocking(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL, NULL);
assert(r==TOKUDB_TRY_AGAIN);
r = toku_cachetable_end_checkpoint(
ct,
cp,
NULL,
NULL,
NULL

View file

@ -43,7 +43,7 @@ static void kibbutz_work(void *fe_v)
CACHEFILE CAST_FROM_VOIDP(f1, fe_v);
sleep(2);
foo = true;
int r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
int r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
assert(r==0);
remove_background_job_from_cf(f1);
}
@ -70,15 +70,16 @@ run_test (void) {
cachefile_kibbutz_enq(f1, kibbutz_work, f1);
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
assert(foo);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
//now let's do a simple checkpoint test
// first dirty the PAIR
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8));
// now this should mark the pair for checkpoint
r = toku_cachetable_begin_checkpoint(ct, NULL);
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
r = toku_cachetable_begin_checkpoint(cp, NULL);
//
// now we pin the pair again, and verify in flush callback that the pair is being checkpointed
@ -87,12 +88,12 @@ run_test (void) {
flush_called = false;
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
assert(flush_called);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
check_me = false;
r = toku_cachetable_end_checkpoint(
ct,
cp,
NULL,
NULL,
NULL

View file

@ -50,6 +50,7 @@ flush (CACHEFILE f __attribute__((__unused__)),
static int
fetch (CACHEFILE f __attribute__((__unused__)),
PAIR UU(p),
int UU(fd),
CACHEKEY k __attribute__((__unused__)),
uint32_t fullhash __attribute__((__unused__)),
@ -107,11 +108,12 @@ cachetable_test (bool write_first, bool write_second, bool start_checkpoint) {
enum cachetable_dirty cd[2];
cd[0] = write_first ? CACHETABLE_DIRTY : CACHETABLE_CLEAN;
cd[1] = write_second ? CACHETABLE_DIRTY : CACHETABLE_CLEAN;
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
if (start_checkpoint) {
//
// should mark the v1 and v2 as pending
//
r = toku_cachetable_begin_checkpoint(ct, NULL); assert(r==0);
r = toku_cachetable_begin_checkpoint(cp, NULL); assert(r==0);
}
//
// This call should cause a flush for both
@ -135,7 +137,8 @@ cachetable_test (bool write_first, bool write_second, bool start_checkpoint) {
dependent_fullhash,
cd,
&put_key,
&put_fullhash
&put_fullhash,
put_callback_nop
);
assert(r == 0);
assert(put_key.b == 3);
@ -147,13 +150,13 @@ cachetable_test (bool write_first, bool write_second, bool start_checkpoint) {
}
check_me = false;
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_cachetable_unpin(f1, make_blocknum(2), 2, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_cachetable_unpin(f1, make_blocknum(3), 3, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(2), 2, CACHETABLE_CLEAN, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(3), 3, CACHETABLE_CLEAN, make_pair_attr(8));
if (start_checkpoint) {
r = toku_cachetable_end_checkpoint(
ct,
cp,
NULL,
NULL,
NULL

View file

@ -40,10 +40,11 @@ cachetable_test (void) {
//long s2;
CACHETABLE_WRITE_CALLBACK wc = def_write_callback(NULL);
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
r = toku_cachetable_begin_checkpoint(ct, NULL); assert(r == 0);
r = toku_cachetable_unpin_and_remove(f1, make_blocknum(1), remove_key_expect_checkpoint, NULL);
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
r = toku_cachetable_begin_checkpoint(cp, NULL); assert(r == 0);
r = toku_test_cachetable_unpin_and_remove(f1, make_blocknum(1), remove_key_expect_checkpoint, NULL);
r = toku_cachetable_end_checkpoint(
ct,
cp,
NULL,
NULL,
NULL
@ -51,7 +52,7 @@ cachetable_test (void) {
assert(r==0);
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
r = toku_cachetable_unpin_and_remove(f1, make_blocknum(1), remove_key_expect_no_checkpoint, NULL);
r = toku_test_cachetable_unpin_and_remove(f1, make_blocknum(1), remove_key_expect_no_checkpoint, NULL);
toku_cachetable_verify(ct);

View file

@ -22,7 +22,7 @@ cachetable_test (void) {
//long s2;
CACHETABLE_WRITE_CALLBACK wc = def_write_callback(NULL);
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), 1, &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8));
r = toku_test_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, make_pair_attr(8));
toku_cachetable_verify(ct);
r = toku_cachefile_close(&f1, 0, false, ZERO_LSN); assert(r == 0);
r = toku_cachetable_close(&ct); lazy_assert_zero(r);

View file

@ -136,7 +136,7 @@ static struct item *make_item (uint64_t key) {
}
static CACHEKEY did_fetch={-1};
static int fetch (CACHEFILE f, int UU(fd), CACHEKEY key, uint32_t fullhash __attribute__((__unused__)), void**value, void** UU(dd), PAIR_ATTR *sizep __attribute__((__unused__)), int *dirtyp, void*extraargs) {
static int fetch (CACHEFILE f, PAIR UU(p), int UU(fd), CACHEKEY key, uint32_t fullhash __attribute__((__unused__)), void**value, void** UU(dd), PAIR_ATTR *sizep __attribute__((__unused__)), int *dirtyp, void*extraargs) {
if (verbose) printf("Fetch %" PRId64 "\n", key.b);
assert (expect_f==f);
assert((long)extraargs==23);
@ -182,37 +182,37 @@ static void test0 (void) {
uint32_t h7 = toku_cachetable_hash(f, make_blocknum(7));
CACHETABLE_WRITE_CALLBACK wc = def_write_callback(t3);
wc.flush_callback = flush;
r=toku_cachetable_put(f, make_blocknum(1), h1, make_item(1), make_pair_attr(test_object_size), wc); /* 1P */ /* this is the lru list. 1 is pinned. */
r=toku_cachetable_put(f, make_blocknum(1), h1, make_item(1), make_pair_attr(test_object_size), wc, put_callback_nop); /* 1P */ /* this is the lru list. 1 is pinned. */
assert(r==0);
assert(expect_n_flushes==0);
expect_init();
r=toku_cachetable_put(f, make_blocknum(2), h2, make_item(2), make_pair_attr(test_object_size), wc);
r=toku_cachetable_put(f, make_blocknum(2), h2, make_item(2), make_pair_attr(test_object_size), wc, put_callback_nop);
assert(r==0);
r=toku_cachetable_unpin(f, make_blocknum(2), h2, CACHETABLE_DIRTY, make_pair_attr(1)); /* 2U 1P */
r=toku_test_cachetable_unpin(f, make_blocknum(2), h2, CACHETABLE_DIRTY, make_pair_attr(1)); /* 2U 1P */
assert(expect_n_flushes==0);
expect_init();
r=toku_cachetable_put(f, make_blocknum(3), h3, make_item(3), make_pair_attr(test_object_size), wc);
r=toku_cachetable_put(f, make_blocknum(3), h3, make_item(3), make_pair_attr(test_object_size), wc, put_callback_nop);
assert(r==0);
assert(expect_n_flushes==0); /* 3P 2U 1P */ /* 3 is most recently used (pinned), 2 is next (unpinned), 1 is least recent (pinned) */
expect_init();
r=toku_cachetable_put(f, make_blocknum(4), h4, make_item(4), make_pair_attr(test_object_size), wc);
r=toku_cachetable_put(f, make_blocknum(4), h4, make_item(4), make_pair_attr(test_object_size), wc, put_callback_nop);
assert(r==0);
assert(expect_n_flushes==0); /* 4P 3P 2U 1P */
expect_init();
r=toku_cachetable_put(f, make_blocknum(5), h5, make_item(5), make_pair_attr(test_object_size), wc);
r=toku_cachetable_put(f, make_blocknum(5), h5, make_item(5), make_pair_attr(test_object_size), wc, put_callback_nop);
assert(r==0);
r=toku_cachetable_unpin(f, make_blocknum(5), h5, CACHETABLE_DIRTY, make_pair_attr(test_object_size));
r=toku_test_cachetable_unpin(f, make_blocknum(5), h5, CACHETABLE_DIRTY, make_pair_attr(test_object_size));
assert(r==0);
r=toku_cachetable_unpin(f, make_blocknum(3), h3, CACHETABLE_DIRTY, make_pair_attr(test_object_size));
r=toku_test_cachetable_unpin(f, make_blocknum(3), h3, CACHETABLE_DIRTY, make_pair_attr(test_object_size));
assert(r==0);
assert(expect_n_flushes==0); /* 5U 4P 3U 2U 1P */
expect1(2); /* 2 is the oldest unpinned item. */
r=toku_cachetable_put(f, make_blocknum(6), h6, make_item(6), make_pair_attr(test_object_size), wc); /* 6P 5U 4P 3U 1P */
r=toku_cachetable_put(f, make_blocknum(6), h6, make_item(6), make_pair_attr(test_object_size), wc, put_callback_nop); /* 6P 5U 4P 3U 1P */
assert(r==0);
test_mutex_lock();
while (expect_n_flushes != 0) {
@ -222,7 +222,7 @@ static void test0 (void) {
test_mutex_unlock();
expect1(3);
r=toku_cachetable_put(f, make_blocknum(7), h7, make_item(7), make_pair_attr(test_object_size), wc);
r=toku_cachetable_put(f, make_blocknum(7), h7, make_item(7), make_pair_attr(test_object_size), wc, put_callback_nop);
assert(r==0);
test_mutex_lock();
while (expect_n_flushes != 0) {
@ -230,7 +230,7 @@ static void test0 (void) {
}
assert(expect_n_flushes==0);
test_mutex_unlock();
r=toku_cachetable_unpin(f, make_blocknum(7), h7, CACHETABLE_DIRTY, make_pair_attr(test_object_size)); /* 7U 6P 5U 4P 1P */
r=toku_test_cachetable_unpin(f, make_blocknum(7), h7, CACHETABLE_DIRTY, make_pair_attr(test_object_size)); /* 7U 6P 5U 4P 1P */
assert(r==0);
{
@ -247,7 +247,7 @@ static void test0 (void) {
{
void *item_v=0;
r=toku_cachetable_unpin(f, make_blocknum(4), h4, CACHETABLE_DIRTY, make_pair_attr(test_object_size));
r=toku_test_cachetable_unpin(f, make_blocknum(4), h4, CACHETABLE_DIRTY, make_pair_attr(test_object_size));
assert(r==0);
expect1(4);
did_fetch=make_blocknum(-1);
@ -266,13 +266,13 @@ static void test0 (void) {
test_mutex_unlock();
}
r=toku_cachetable_unpin(f, make_blocknum(2), h2, CACHETABLE_DIRTY, make_pair_attr(test_object_size));
r=toku_test_cachetable_unpin(f, make_blocknum(2), h2, CACHETABLE_DIRTY, make_pair_attr(test_object_size));
assert(r==0);
r=toku_cachetable_unpin(f, make_blocknum(5), h5, CACHETABLE_DIRTY, make_pair_attr(test_object_size));
r=toku_test_cachetable_unpin(f, make_blocknum(5), h5, CACHETABLE_DIRTY, make_pair_attr(test_object_size));
assert(r==0);
r=toku_cachetable_unpin(f, make_blocknum(6), h6, CACHETABLE_DIRTY, make_pair_attr(test_object_size));
r=toku_test_cachetable_unpin(f, make_blocknum(6), h6, CACHETABLE_DIRTY, make_pair_attr(test_object_size));
assert(r==0);
r=toku_cachetable_unpin(f, make_blocknum(1), h1, CACHETABLE_DIRTY, make_pair_attr(test_object_size));
r=toku_test_cachetable_unpin(f, make_blocknum(1), h1, CACHETABLE_DIRTY, make_pair_attr(test_object_size));
assert(r==0);
r=toku_cachetable_assert_all_unpinned(t);
assert(r==0);
@ -305,7 +305,7 @@ static void flush_n (CACHEFILE f __attribute__((__unused__)), int UU(fd), CACHEK
int *CAST_FROM_VOIDP(v, value);
assert(*v==0);
}
static int fetch_n (CACHEFILE f __attribute__((__unused__)), int UU(fd), CACHEKEY key __attribute__((__unused__)),
static int fetch_n (CACHEFILE f __attribute__((__unused__)), PAIR UU(p), int UU(fd), CACHEKEY key __attribute__((__unused__)),
uint32_t fullhash __attribute__((__unused__)),
void**value,
void** UU(dd),
@ -339,24 +339,24 @@ static void test_nested_pin (void) {
uint32_t f1hash = toku_cachetable_hash(f, make_blocknum(1));
CACHETABLE_WRITE_CALLBACK wc = def_write_callback(f2);
wc.flush_callback = flush_n;
r = toku_cachetable_put(f, make_blocknum(1), f1hash, &i0, make_pair_attr(1), wc);
r = toku_cachetable_put(f, make_blocknum(1), f1hash, &i0, make_pair_attr(1), wc, put_callback_nop);
assert(r==0);
r = toku_cachetable_unpin(f, make_blocknum(1), f1hash, CACHETABLE_CLEAN, make_pair_attr(test_object_size));
r = toku_test_cachetable_unpin(f, make_blocknum(1), f1hash, CACHETABLE_CLEAN, make_pair_attr(test_object_size));
r = toku_cachetable_get_and_pin(f, make_blocknum(1), f1hash, &vv, NULL, wc, fetch_n, def_pf_req_callback, def_pf_callback, true, f2);
assert(r==0);
assert(vv==&i0);
assert(i0==0);
r = toku_cachetable_unpin(f, make_blocknum(1), f1hash, CACHETABLE_CLEAN, make_pair_attr(test_object_size));
r = toku_test_cachetable_unpin(f, make_blocknum(1), f1hash, CACHETABLE_CLEAN, make_pair_attr(test_object_size));
assert(r==0);
r = toku_cachetable_maybe_get_and_pin(f, make_blocknum(1), f1hash, &vv2);
assert(r==0);
assert(vv2==vv);
r = toku_cachetable_unpin(f, make_blocknum(1), f1hash, CACHETABLE_CLEAN, make_pair_attr(test_object_size));
r = toku_test_cachetable_unpin(f, make_blocknum(1), f1hash, CACHETABLE_CLEAN, make_pair_attr(test_object_size));
assert(r==0);
uint32_t f2hash = toku_cachetable_hash(f, make_blocknum(2));
r = toku_cachetable_put(f, make_blocknum(2), f2hash, &i1, make_pair_attr(test_object_size), wc);
r = toku_cachetable_put(f, make_blocknum(2), f2hash, &i1, make_pair_attr(test_object_size), wc, put_callback_nop);
assert(r==0); // The other one is pinned, but now the cachetable fails gracefully: It allows the pin to happen
r = toku_cachetable_unpin(f, make_blocknum(2), f2hash, CACHETABLE_CLEAN, make_pair_attr(test_object_size));
r = toku_test_cachetable_unpin(f, make_blocknum(2), f2hash, CACHETABLE_CLEAN, make_pair_attr(test_object_size));
assert(r==0);
// toku_os_usleep(1*1000000);
r = toku_cachefile_close(&f, 0, false, ZERO_LSN); assert(r==0);
@ -379,7 +379,7 @@ static void null_flush (CACHEFILE cf __attribute__((__unused__)),
) {
}
static int add123_fetch (CACHEFILE cf, int UU(fd), CACHEKEY key, uint32_t fullhash, void **value,
static int add123_fetch (CACHEFILE cf, PAIR UU(p), int UU(fd), CACHEKEY key, uint32_t fullhash, void **value,
void** UU(dd),
PAIR_ATTR *sizep __attribute__((__unused__)), int * dirtyp, void*extraargs) {
assert(fullhash==toku_cachetable_hash(cf,key));
@ -390,7 +390,7 @@ PAIR_ATTR *sizep __attribute__((__unused__)), int * dirtyp, void*extraargs) {
return 0;
}
static int add222_fetch (CACHEFILE cf, int UU(fd), CACHEKEY key, uint32_t fullhash, void **value,
static int add222_fetch (CACHEFILE cf, PAIR UU(p), int UU(fd), CACHEKEY key, uint32_t fullhash, void **value,
void** UU(dd),
PAIR_ATTR *sizep __attribute__((__unused__)), int * dirtyp, void*extraargs) {
assert(fullhash==toku_cachetable_hash(cf,key));
@ -425,8 +425,8 @@ static void test_multi_filehandles (void) {
CACHETABLE_WRITE_CALLBACK wc = def_write_callback((void*)123);
wc.flush_callback = null_flush;
r = toku_cachetable_put(f1, make_blocknum(1), toku_cachetable_hash(f1, make_blocknum(1)), (void*)124, make_pair_attr(test_object_size), wc); assert(r==0);
r = toku_cachetable_unpin(f1, make_blocknum(1), toku_cachetable_hash(f1, make_blocknum(1)), CACHETABLE_DIRTY, make_pair_attr(0)); assert(r==0);
r = toku_cachetable_put(f1, make_blocknum(1), toku_cachetable_hash(f1, make_blocknum(1)), (void*)124, make_pair_attr(test_object_size), wc, put_callback_nop); assert(r==0);
r = toku_test_cachetable_unpin(f1, make_blocknum(1), toku_cachetable_hash(f1, make_blocknum(1)), CACHETABLE_DIRTY, make_pair_attr(0)); assert(r==0);
r = toku_cachetable_get_and_pin(f2, make_blocknum(1), toku_cachetable_hash(f2, make_blocknum(1)), &v, NULL, wc, add123_fetch, def_pf_req_callback, def_pf_callback, true, (void*)123); assert(r==0);
assert((unsigned long)v==124);
r = toku_cachetable_get_and_pin(f2, make_blocknum(2), toku_cachetable_hash(f2, make_blocknum(2)), &v, NULL, wc, add123_fetch, def_pf_req_callback, def_pf_callback, true, (void*)123); assert(r==0);
@ -436,11 +436,11 @@ static void test_multi_filehandles (void) {
assert((unsigned long)v==224);
// we support only one close for a file handle
r = toku_cachetable_unpin(f1, make_blocknum(1), toku_cachetable_hash(f1, make_blocknum(1)), CACHETABLE_CLEAN, make_pair_attr(0)); assert(r==0);
r = toku_cachetable_unpin(f2, make_blocknum(2), toku_cachetable_hash(f2, make_blocknum(2)), CACHETABLE_CLEAN, make_pair_attr(0)); assert(r==0);
r = toku_test_cachetable_unpin(f1, make_blocknum(1), toku_cachetable_hash(f1, make_blocknum(1)), CACHETABLE_CLEAN, make_pair_attr(0)); assert(r==0);
r = toku_test_cachetable_unpin(f2, make_blocknum(2), toku_cachetable_hash(f2, make_blocknum(2)), CACHETABLE_CLEAN, make_pair_attr(0)); assert(r==0);
r = toku_cachefile_close(&f2, 0, false, ZERO_LSN); assert(r==0);
r = toku_cachetable_unpin(f3, make_blocknum(2), toku_cachetable_hash(f3, make_blocknum(2)), CACHETABLE_CLEAN, make_pair_attr(0)); assert(r==0);
r = toku_test_cachetable_unpin(f3, make_blocknum(2), toku_cachetable_hash(f3, make_blocknum(2)), CACHETABLE_CLEAN, make_pair_attr(0)); assert(r==0);
r = toku_cachefile_close(&f3, 0, false, ZERO_LSN); assert(r==0);
r = toku_cachetable_close(&t); assert(r==0);
@ -464,7 +464,7 @@ static void test_dirty_flush(CACHEFILE f,
if (verbose) printf("test_dirty_flush %p %" PRId64 " %p %ld %u %u\n", f, key.b, value, size.size, (unsigned)do_write, (unsigned)keep);
}
static int test_dirty_fetch(CACHEFILE f, int UU(fd), CACHEKEY key, uint32_t fullhash, void **value_ptr,
static int test_dirty_fetch(CACHEFILE f, PAIR UU(p), int UU(fd), CACHEKEY key, uint32_t fullhash, void **value_ptr,
void** UU(dd),
PAIR_ATTR *size_ptr, int * dirtyp, void *arg) {
*value_ptr = arg;
@ -496,7 +496,7 @@ static void test_dirty(void) {
uint32_t hkey = toku_cachetable_hash(f, key);
CACHETABLE_WRITE_CALLBACK wc = def_write_callback(NULL);
wc.flush_callback = test_dirty_flush;
r = toku_cachetable_put(f, key, hkey, value, make_pair_attr(test_object_size), wc);
r = toku_cachetable_put(f, key, hkey, value, make_pair_attr(test_object_size), wc, put_callback_nop);
assert(r == 0);
// cachetable_print_state(t);
@ -505,7 +505,7 @@ static void test_dirty(void) {
assert(dirty == 1);
assert(pinned == 1);
r = toku_cachetable_unpin(f, key, hkey, CACHETABLE_CLEAN, make_pair_attr(0));
r = toku_test_cachetable_unpin(f, key, hkey, CACHETABLE_CLEAN, make_pair_attr(0));
assert(r == 0);
r = toku_cachetable_get_key_state(t, key, f, &value, &dirty, &pinned, &entry_size);
assert(r == 0);
@ -522,7 +522,7 @@ static void test_dirty(void) {
assert(dirty == 1);
assert(pinned == 1);
r = toku_cachetable_unpin(f, key, hkey, CACHETABLE_CLEAN, make_pair_attr(test_object_size));
r = toku_test_cachetable_unpin(f, key, hkey, CACHETABLE_CLEAN, make_pair_attr(test_object_size));
assert(r == 0);
// cachetable_print_state(t);
@ -544,7 +544,7 @@ static void test_dirty(void) {
assert(dirty == 0);
assert(pinned == 1);
r = toku_cachetable_unpin(f, key, hkey, CACHETABLE_CLEAN, make_pair_attr(test_object_size));
r = toku_test_cachetable_unpin(f, key, hkey, CACHETABLE_CLEAN, make_pair_attr(test_object_size));
assert(r == 0);
// cachetable_print_state(t);
@ -564,7 +564,7 @@ static void test_dirty(void) {
assert(dirty == 0);
assert(pinned == 1);
r = toku_cachetable_unpin(f, key, hkey, CACHETABLE_DIRTY, make_pair_attr(test_object_size));
r = toku_test_cachetable_unpin(f, key, hkey, CACHETABLE_DIRTY, make_pair_attr(test_object_size));
assert(r == 0);
// cachetable_print_state(t);
@ -632,7 +632,7 @@ static void test_size_resize(void) {
CACHETABLE_WRITE_CALLBACK wc = def_write_callback(NULL);
wc.flush_callback = test_size_flush_callback;
r = toku_cachetable_put(f, key, hkey, value, make_pair_attr(size), wc);
r = toku_cachetable_put(f, key, hkey, value, make_pair_attr(size), wc, put_callback_nop);
assert(r == 0);
void *entry_value; int dirty; long long pinned; long entry_size;
@ -644,7 +644,7 @@ static void test_size_resize(void) {
assert(entry_size == size);
long long new_size = 2*size;
r = toku_cachetable_unpin(f, key, hkey, CACHETABLE_CLEAN, make_pair_attr(new_size));
r = toku_test_cachetable_unpin(f, key, hkey, CACHETABLE_CLEAN, make_pair_attr(new_size));
assert(r == 0);
void *current_value;
@ -654,7 +654,7 @@ static void test_size_resize(void) {
assert(current_value == value);
assert(current_size == new_size);
r = toku_cachetable_unpin(f, key, hkey, CACHETABLE_CLEAN, make_pair_attr(new_size));
r = toku_test_cachetable_unpin(f, key, hkey, CACHETABLE_CLEAN, make_pair_attr(new_size));
assert(r == 0);
r = toku_cachefile_close(&f, 0, false, ZERO_LSN);
@ -697,7 +697,7 @@ static void test_size_flush(void) {
uint32_t hkey = toku_cachetable_hash(f, key);
CACHETABLE_WRITE_CALLBACK wc = def_write_callback(NULL);
wc.flush_callback = test_size_flush_callback;
r = toku_cachetable_put(f, key, hkey, value, make_pair_attr(size), wc);
r = toku_cachetable_put(f, key, hkey, value, make_pair_attr(size), wc, put_callback_nop);
assert(r == 0);
int n_entries, hash_size; long size_current, size_limit;
@ -724,7 +724,7 @@ static void test_size_flush(void) {
}
test_mutex_unlock();
r = toku_cachetable_unpin(f, key, hkey, CACHETABLE_CLEAN, make_pair_attr(size));
r = toku_test_cachetable_unpin(f, key, hkey, CACHETABLE_CLEAN, make_pair_attr(size));
assert(r == 0);
}

View file

@ -1,276 +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);
}
static const int test_object_size = 1;
static CACHETABLE ct;
// increasing N_FILES may break test
enum { N_PRESENT_LIMIT = 4, TRIALS=20000, N_FILES=1 };
static int n_present=0;
static struct present_items {
CACHEKEY key;
CACHEFILE cf;
} present_items[N_PRESENT_LIMIT];
static void print_ints(void) __attribute__((__unused__));
static void print_ints(void) {
int i;
for (i=0; i<n_present; i++) {
if (i==0) printf("{"); else printf(",");
printf("{%" PRId64 ",%p}", present_items[i].key.b, present_items[i].cf);
}
printf("}\n");
}
static void item_becomes_present(CACHETABLE thect, CACHEFILE cf, CACHEKEY key) {
test_mutex_lock();
while (n_present >= N_PRESENT_LIMIT) {
test_mutex_unlock(); toku_pthread_yield(); maybe_flush(thect); test_mutex_lock();
}
assert(n_present<N_PRESENT_LIMIT);
present_items[n_present].cf = cf;
present_items[n_present].key = key;
n_present++;
test_mutex_unlock();
}
static void item_becomes_not_present(CACHEFILE cf, CACHEKEY key) {
int i;
//printf("Removing {%4lld %16p}: Initially: ", key, cf); print_ints();
test_mutex_lock();
assert(n_present<=N_PRESENT_LIMIT);
for (i=0; i<n_present; i++) {
if (present_items[i].key.b==key.b) {
present_items[i]=present_items[n_present-1];
n_present--;
test_mutex_unlock();
//printf(" Finally: "); print_ints();
return;
}
}
printf("Whoops, %p,%" PRId64 " was already not present\n", cf ,key.b);
abort();
test_mutex_unlock();
}
static void file_is_not_present(CACHEFILE cf) {
int i;
test_mutex_lock();
for (i=0; i<n_present; i++) {
assert(present_items[i].cf!=cf);
}
test_mutex_unlock();
}
static void flush_forchain (CACHEFILE f __attribute__((__unused__)),
int UU(fd),
CACHEKEY key,
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 __attribute__((__unused__)),
bool for_checkpoint __attribute__((__unused__)),
bool UU(is_clone)
) {
if (keep_me) return;
int *CAST_FROM_VOIDP(v, value);
//toku_cachetable_print_state(ct);
//printf("Flush %lld %d\n", key, (int)value);
assert((long)v==(long)key.b);
item_becomes_not_present(f, key);
//print_ints();
}
static int fetch_forchain (CACHEFILE f, int UU(fd), CACHEKEY key, uint32_t fullhash, void**value,
void** UU(dd),
PAIR_ATTR *sizep __attribute__((__unused__)), int * dirtyp, void*extraargs) {
assert(toku_cachetable_hash(f, key)==fullhash);
assert((long)extraargs==(long)key.b);
*value = (void*)(long)key.b;
*dirtyp = 0;
return 0;
}
static void verify_cachetable_against_present (void) {
int i;
again:
test_mutex_lock();
int my_n_present = n_present;
struct present_items my_present_items[N_PRESENT_LIMIT];
for (i=0; i<n_present; i++)
my_present_items[i] = present_items[i];
test_mutex_unlock();
for (i=0; i<my_n_present; i++) {
void *v;
uint32_t fullhash = toku_cachetable_hash(my_present_items[i].cf, my_present_items[i].key);
int r=toku_cachetable_maybe_get_and_pin_clean(my_present_items[i].cf,
my_present_items[i].key,
toku_cachetable_hash(my_present_items[i].cf, my_present_items[i].key),
&v);
if (r == -1) goto again;
assert(r==0);
r = toku_cachetable_unpin(my_present_items[i].cf, my_present_items[i].key, fullhash, CACHETABLE_CLEAN, make_pair_attr(test_object_size));
}
}
static void test_chaining (void) {
/* Make sure that the hash chain and the LRU list don't get confused. */
CACHEFILE f[N_FILES];
enum { FILENAME_LEN=100 };
char fname[N_FILES][FILENAME_LEN];
int r;
long i, trial;
r = toku_create_cachetable(&ct, N_PRESENT_LIMIT, ZERO_LSN, NULL_LOGGER); assert(r==0);
for (i=0; i<N_FILES; i++) {
r = snprintf(fname[i], FILENAME_LEN, __SRCFILE__ ".%ld.dat", i);
assert(r>0 && r<FILENAME_LEN);
unlink(fname[i]);
r = toku_cachetable_openf(&f[i], ct, fname[i], O_RDWR|O_CREAT, S_IRWXU|S_IRWXG|S_IRWXO); assert(r==0);
}
for (i=0; i<N_PRESENT_LIMIT; i++) {
int fnum = i%N_FILES;
//printf("%s:%d Add %d\n", __SRCFILE__, __LINE__, i);
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);
item_becomes_present(ct, f[fnum], make_blocknum(i));
r = toku_cachetable_unpin(f[fnum], make_blocknum(i), fhash, CACHETABLE_CLEAN, make_pair_attr(test_object_size));
assert(r==0);
//print_ints();
}
for (trial=0; trial<TRIALS; trial++) {
test_mutex_lock();
int my_n_present = n_present;
test_mutex_unlock();
if (my_n_present>0) {
// 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<N_FILES; i++) {
r = toku_cachefile_close(&f[i], 0, false, ZERO_LSN); assert(r==0);
}
r = toku_cachetable_close(&ct); assert(r==0);
}
static void __attribute__((__noreturn__,unused))
usage (const char *progname) {
fprintf(stderr, "Usage:\n %s [-v] [-q]\n", progname);
exit(1);
}
int
test_main (int argc, const char *argv[]) {
default_parse_args(argc, argv);
test_mutex_init();
test_chaining();
test_mutex_destroy();
if (verbose) printf("ok\n");
return 0;
}

View file

@ -7,6 +7,7 @@
static int
fetch (CACHEFILE f __attribute__((__unused__)),
PAIR UU(p),
int UU(fd),
CACHEKEY k __attribute__((__unused__)),
uint32_t fullhash __attribute__((__unused__)),
@ -46,7 +47,7 @@ cachetable_unpin_and_remove_test (int n) {
// put the keys into the cachetable
for (i=0; i<n; i++) {
uint32_t hi = toku_cachetable_hash(f1, make_blocknum(keys[i].b));
r = toku_cachetable_put(f1, make_blocknum(keys[i].b), hi, (void *)(long) keys[i].b, make_pair_attr(1),wc);
r = toku_cachetable_put(f1, make_blocknum(keys[i].b), hi, (void *)(long) keys[i].b, make_pair_attr(1),wc, put_callback_nop);
assert(r == 0);
}
@ -56,7 +57,7 @@ cachetable_unpin_and_remove_test (int n) {
while (nkeys > 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; i<n; i++) {
r = toku_cachetable_unpin_and_remove(f1, keys[i], NULL, NULL);
// assert(r != 0);
if (r == 0) printf("%s:%d warning %d\n", __SRCFILE__, __LINE__, r);
}
// verify that the cachtable is empty
int nentries;
toku_cachetable_get_state(ct, &nentries, NULL, NULL, NULL);
@ -108,9 +102,9 @@ cachetable_put_evict_remove_test (int n) {
// put 0, 1, 2, ... should evict 0
for (i=0; i<n; i++) {
r = toku_cachetable_put(f1, make_blocknum(i), hi[i], (void *)(long)i, make_pair_attr(1), wc);
r = toku_cachetable_put(f1, make_blocknum(i), hi[i], (void *)(long)i, make_pair_attr(1), wc, put_callback_nop);
assert(r == 0);
r = toku_cachetable_unpin(f1, make_blocknum(i), hi[i], CACHETABLE_CLEAN, make_pair_attr(1));
r = toku_test_cachetable_unpin(f1, make_blocknum(i), hi[i], CACHETABLE_CLEAN, make_pair_attr(1));
assert(r == 0);
}
@ -120,7 +114,7 @@ cachetable_put_evict_remove_test (int n) {
assert(r == 0);
// remove 0
r = toku_cachetable_unpin_and_remove(f1, make_blocknum(0), NULL, NULL);
r = toku_test_cachetable_unpin_and_remove(f1, make_blocknum(0), NULL, NULL);
assert(r == 0);
char *error_string;

View file

@ -9,12 +9,13 @@ CACHETABLE ct;
//
// This test exposed a bug (#3970) caught only by Valgrind.
// freed memory was being accessed by toku_cachetable_unpin_and_remove
// freed memory was being accessed by toku_test_cachetable_unpin_and_remove
//
static void *run_end_chkpt(void *arg) {
assert(arg == NULL);
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
int r = toku_cachetable_end_checkpoint(
ct,
cp,
NULL,
NULL,
NULL
@ -40,7 +41,7 @@ run_test (void) {
long s1;
//long s2;
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), toku_cachetable_hash(f1, make_blocknum(1)), &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
toku_cachetable_unpin(
toku_test_cachetable_unpin(
f1,
make_blocknum(1),
toku_cachetable_hash(f1, make_blocknum(1)),
@ -49,7 +50,8 @@ run_test (void) {
);
// now this should mark the pair for checkpoint
r = toku_cachetable_begin_checkpoint(ct, NULL);
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
r = toku_cachetable_begin_checkpoint(cp, NULL);
r = toku_cachetable_get_and_pin(f1, make_blocknum(1), toku_cachetable_hash(f1, make_blocknum(1)), &v1, &s1, wc, def_fetch, def_pf_req_callback, def_pf_callback, true, NULL);
toku_pthread_t mytid;
@ -58,7 +60,7 @@ run_test (void) {
// give checkpoint thread a chance to start waiting on lock
sleep(1);
r = toku_cachetable_unpin_and_remove(f1, make_blocknum(1), NULL, NULL);
r = toku_test_cachetable_unpin_and_remove(f1, make_blocknum(1), NULL, NULL);
assert(r==0);
void* ret;

View file

@ -23,7 +23,7 @@ cachetable_unpin_test (int n) {
uint32_t hi;
CACHETABLE_WRITE_CALLBACK wc = def_write_callback(NULL);
hi = toku_cachetable_hash(f1, make_blocknum(i));
r = toku_cachetable_put(f1, make_blocknum(i), hi, (void *)(long)i, make_pair_attr(1), wc);
r = toku_cachetable_put(f1, make_blocknum(i), hi, (void *)(long)i, make_pair_attr(1), wc, put_callback_nop);
assert(r == 0);
assert(toku_cachefile_count_pinned(f1, 0) == i);
@ -32,24 +32,87 @@ cachetable_unpin_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);
}
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;
}

View file

@ -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);

View file

@ -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; i<N; i++) {
snprintf(fnames[i], 100, "dbufio-test-destroy-file%d.data", i);
unlink(fnames[i]);
@ -46,8 +46,6 @@ static void test1 (size_t chars_per_file, size_t bytes_per_read) {
assert(r==0);
}
n_live = n_live; bytes_per_read = bytes_per_read;
{ int r = panic_dbufio_fileset(bfs, EIO); assert(r == 0); }
{

View file

@ -40,8 +40,8 @@ static void test_it (int N) {
r = toku_txn_commit_txn(txn, false, NULL, NULL); CKERR(r);
toku_txn_close_txn(txn);
r = toku_checkpoint(ct, logger, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT); CKERR(r);
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);
unsigned int rands[N];
@ -63,7 +63,7 @@ static void test_it (int N) {
toku_txn_close_txn(txn);
r = toku_checkpoint(ct, logger, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT); CKERR(r);
r = toku_checkpoint(cp, logger, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT); CKERR(r);
r = toku_close_ft_handle_nolsn(brt, NULL); CKERR(r);
if (verbose) printf("i=%d\n", i);
@ -89,8 +89,7 @@ static void test_it (int N) {
r = toku_txn_commit_txn(txn, false, NULL, NULL); CKERR(r);
toku_txn_close_txn(txn);
r = toku_checkpoint(ct, logger, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT); CKERR(r);
r = toku_checkpoint(cp, logger, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT); CKERR(r);
r = toku_close_ft_handle_nolsn(brt, NULL); CKERR(r);
if (verbose) printf("d=%d\n", i);
@ -106,12 +105,12 @@ static void test_it (int N) {
assert(is_empty);
}
r = toku_checkpoint(ct, logger, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT); CKERR(r);
r = toku_checkpoint(cp, logger, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT); CKERR(r);
r = toku_close_ft_handle_nolsn(brt, NULL); CKERR(r);
r = toku_checkpoint(ct, logger, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT); CKERR(r);
r = toku_checkpoint(cp, logger, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT); CKERR(r);
r = toku_logger_close_rollback(logger, false); CKERR(r);
r = toku_checkpoint(ct, logger, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT); CKERR(r);
r = toku_checkpoint(cp, logger, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT); CKERR(r);
r = toku_cachetable_close(&ct); CKERR(r);
r = toku_logger_close(&logger); assert(r==0);

View file

@ -84,14 +84,15 @@ create_populate_tree(const char *logdir, const char *fname, int n) {
error = toku_close_ft_handle_nolsn(brt, NULL);
assert(error == 0);
error = toku_checkpoint(ct, logger, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT);
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
error = toku_checkpoint(cp, logger, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT);
assert(error == 0);
error = toku_logger_close_rollback(logger, false);
assert(error == 0);
error = toku_checkpoint(ct, logger, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT);
error = toku_checkpoint(cp, logger, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT);
assert(error == 0);
error = toku_logger_shutdown(logger);
@ -192,8 +193,8 @@ test_provdel(const char *logdir, const char *fname, int n) {
error = toku_close_ft_handle_nolsn(brt, NULL);
assert(error == 0);
error = toku_checkpoint(ct, logger, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT);
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
error = toku_checkpoint(cp, logger, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT);
assert(error == 0);
error = toku_logger_close_rollback(logger, false);

View file

@ -89,12 +89,13 @@ create_populate_tree(const char *logdir, const char *fname, int n) {
error = toku_close_ft_handle_nolsn(brt, NULL);
assert(error == 0);
error = toku_checkpoint(ct, logger, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT);
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
error = toku_checkpoint(cp, logger, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT);
assert(error == 0);
error = toku_logger_close_rollback(logger, false);
assert(error == 0);
error = toku_checkpoint(ct, logger, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT);
error = toku_checkpoint(cp, logger, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT);
assert(error == 0);
error = toku_logger_shutdown(logger);

View file

@ -85,11 +85,12 @@ create_populate_tree(const char *logdir, const char *fname, int n) {
error = toku_close_ft_handle_nolsn(brt, NULL);
assert(error == 0);
error = toku_checkpoint(ct, logger, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT);
CHECKPOINTER cp = toku_cachetable_get_checkpointer(ct);
error = toku_checkpoint(cp, logger, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT);
assert(error == 0);
error = toku_logger_close_rollback(logger, false);
assert(error == 0);
error = toku_checkpoint(ct, logger, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT);
error = toku_checkpoint(cp, logger, NULL, NULL, NULL, NULL, CLIENT_CHECKPOINT);
assert(error == 0);
error = toku_logger_shutdown(logger);

Some files were not shown because too many files have changed in this diff Show more