Commit 9977173d authored by John Esmet's avatar John Esmet Committed by Yoni Fogel

close[t:4715] merging 4715 to main. at a high level, ft files are removed by...

close[t:4715] merging 4715 to main. at a high level, ft files are removed by marking OPEN ft/cf's as unlink on close, so when the last reference goes away, the file is unlinked. we log this bit during a checkpoint's fassociate to help with aborting a hot index in the future (so it isn't stricly necessary right now).


git-svn-id: file:///svn/toku/tokudb@44003 c7de825b-a66e-492c-adef-691d508d4ae1
parent b565421d
......@@ -96,7 +96,6 @@ ft_set_dirty(FT ft, BOOL for_checkpoint){
}
}
//fd is protected (must be holding fdlock)
static void
maybe_truncate_cachefile(BLOCK_TABLE bt, int fd, FT h, u_int64_t size_needed_before) {
assert(toku_mutex_is_locked(&bt->mutex));
......@@ -106,7 +105,6 @@ maybe_truncate_cachefile(BLOCK_TABLE bt, int fd, FT h, u_int64_t size_needed_bef
toku_maybe_truncate_cachefile(h->cf, fd, new_size_needed);
}
//fd is protected (must be holding fdlock)
void
toku_maybe_truncate_cachefile_on_open(BLOCK_TABLE bt, int fd, FT h) {
lock_for_blocktable(bt);
......@@ -254,7 +252,6 @@ PRNTF("free", i, pair->size, pair->u.diskoff, bt);
// free (offset,len) from checkpoint
// move inprogress to checkpoint (resetting type)
// inprogress = NULL
//fd is protected (must be holding fdlock)
void
toku_block_translation_note_end_checkpoint (BLOCK_TABLE bt, int fd, FT h) {
// Free unused blocks
......
......@@ -348,9 +348,11 @@ struct cachefile {
bool is_flushing; // during cachetable_flush_cachefile, this must be
// true, to prevent the cleaner thread from messing
// with nodes in that cachefile
struct rwlock fdlock; // Protect changing the fd and is_dev_null
// Only write-locked by toku_cachefile_redirect_nullfd()
BOOL is_dev_null; //True if was deleted and redirected to /dev/null (starts out FALSE, can be set to TRUE, can never be set back to FALSE)
// 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;
......@@ -648,11 +650,16 @@ int toku_cachetable_openfd_with_filenum (CACHEFILE *cfptr, CACHETABLE ct, int fd
cachefiles_lock(ct);
for (extant = ct->cachefiles; extant; extant=extant->next) {
if (memcmp(&extant->fileid, &fileid, sizeof(fileid))==0) {
//File is already open (and in cachetable as extant)
// Clients must serialize cachefile open, close, and unlink
// So, during open, we should never see a closing cachefile
// or one that has been marked as unlink on close.
assert(!extant->is_closing);
r = close(fd); // no change for t:2444
assert(!extant->unlink_on_close);
// Reuse an existing cachefile and close the caller's fd, whose
// responsibility has been passed to us.
r = close(fd);
assert(r == 0);
// re-use pre-existing cachefile
*cfptr = extant;
r = 0;
goto exit;
......@@ -674,7 +681,6 @@ int toku_cachetable_openfd_with_filenum (CACHEFILE *cfptr, CACHETABLE ct, int fd
newcf->next = ct->cachefiles;
ct->cachefiles = newcf;
rwlock_init(&newcf->fdlock);
newcf->most_recent_global_checkpoint_that_finished_early = ZERO_LSN;
newcf->for_local_checkpoint = ZERO_LSN;
newcf->checkpoint_state = CS_NOT_IN_PROGRESS;
......@@ -714,7 +720,6 @@ void toku_cachefile_get_workqueue_load (CACHEFILE cf, int *n_in_queue, int *n_th
int toku_cachefile_set_fd (CACHEFILE cf, int fd, const char *fname_in_env) {
int r;
struct fileid fileid;
(void)toku_cachefile_get_and_pin_fd(cf);
r=toku_os_get_unique_file_id(fd, &fileid);
if (r != 0) {
r=errno; close(fd); goto cleanup; // no change for t:2444
......@@ -739,7 +744,6 @@ int toku_cachefile_set_fd (CACHEFILE cf, int fd, const char *fname_in_env) {
cachefile_init_filenum(cf, fd, fname_in_env, fileid);
r = 0;
cleanup:
toku_cachefile_unpin_fd(cf);
return r;
}
......@@ -748,35 +752,16 @@ toku_cachefile_fname_in_env (CACHEFILE cf) {
return cf->fname_in_env;
}
int toku_cachefile_get_and_pin_fd (CACHEFILE cf) {
CACHETABLE ct = cf->cachetable;
cachetable_lock(ct);
rwlock_prefer_read_lock(&cf->fdlock, cf->cachetable->mutex);
cachetable_unlock(ct);
int
toku_cachefile_get_fd (CACHEFILE cf) {
return cf->fd;
}
void toku_cachefile_unpin_fd (CACHEFILE cf) {
CACHETABLE ct = cf->cachetable;
cachetable_lock(ct);
rwlock_read_unlock(&cf->fdlock);
cachetable_unlock(ct);
}
//Must be holding a read or write lock on cf->fdlock
BOOL
toku_cachefile_is_dev_null_unlocked (CACHEFILE cf) {
return cf->is_dev_null;
}
//Must already be holding fdlock (read or write)
int
toku_cachefile_truncate (CACHEFILE cf, toku_off_t new_size) {
int r;
if (toku_cachefile_is_dev_null_unlocked(cf)) r = 0; //Don't truncate /dev/null
else {
r = ftruncate(cf->fd, new_size);
if (r != 0)
if (r != 0) {
r = errno;
}
return r;
......@@ -803,84 +788,77 @@ void toku_cachefile_wait_for_background_work_to_quiesce(CACHEFILE cf) {
wait_on_background_jobs_to_finish(cf);
}
int toku_cachefile_close (CACHEFILE *cfp, char **error_string, BOOL oplsn_valid, LSN oplsn) {
int
toku_cachefile_close(CACHEFILE *cfp, char **error_string, BOOL oplsn_valid, LSN oplsn) {
int r, close_error = 0;
CACHEFILE cf = *cfp;
CACHETABLE ct = cf->cachetable;
// Mark this cachefile as flushing so that cleaner threads know
// not to operate on any of its pairs.
cachetable_lock(ct);
cf->is_flushing = true;
cachetable_unlock(ct);
// There may be reader, writer, or flusher threads on the kibbutz
// that need to do work on pairs for this cf. Before we can close
// the underlying file, we need to wait for them to finish. No new
// work shoudl start because clients of the cachetable are not supposed
// to use a cachefile in parallel with a close, or afterwards.
wait_on_background_jobs_to_finish(cf);
// Hold the cachetable lock while we check some invariants and
// flush the cachefile.
cachetable_lock(ct);
{
//Checkpoint holds a reference, close should be impossible if still in use by a checkpoint.
// Clients should never attempt to close a cachefile that is being
// checkpointed. We notify clients this is happening in the
// note_pin_by_checkpoint callback.
assert(!cf->next_in_checkpoint);
assert(!cf->for_checkpoint);
// Help enforce the client contract that open/close should never
// run in parallel.
assert(!cf->is_closing);
cf->is_closing = TRUE; //Mark this cachefile so that no one will re-use it.
int r;
// cachetable_flush_cachefile() may release and retake cachetable_lock,
// allowing another thread to get into either/both of
// - toku_cachetable_openfd()
// - toku_cachefile_of_iname_and_add_reference()
cf->is_closing = true;
// Flush the cachefile and remove all of its pairs from the cachetable
cachetable_flush_cachefile(ct, cf);
if (0) {
error:
remove_cf_from_cachefiles_list(cf);
assert(!cf->next_in_checkpoint); //checkpoint cannot run on a closing file
assert(!cf->for_checkpoint); //checkpoint cannot run on a closing file
if (cf->fname_in_env) toku_free(cf->fname_in_env);
rwlock_write_lock(&cf->fdlock, ct->mutex);
if ( !toku_cachefile_is_dev_null_unlocked(cf) ) {
int r3 = toku_file_fsync_without_accounting(cf->fd); //t:2444
if (r3!=0) fprintf(stderr, "%s:%d During error handling, could not fsync file r=%d errno=%d\n", __FILE__, __LINE__, r3, errno);
}
int r2 = close(cf->fd);
if (r2!=0) fprintf(stderr, "%s:%d During error handling, could not close file r=%d errno=%d\n", __FILE__, __LINE__, r2, errno);
//assert(r == 0);
rwlock_write_unlock(&cf->fdlock);
rwlock_destroy(&cf->fdlock);
assert(toku_list_empty(&cf->pairs_for_cachefile));
toku_free(cf);
cachetable_unlock(ct);
return r;
}
// Call the close userdata callback to notify the client this cachefile
// and its underlying file are going to be closed
if (cf->close_userdata) {
rwlock_prefer_read_lock(&cf->fdlock, ct->mutex);
r = cf->close_userdata(cf, cf->fd, cf->userdata, error_string, oplsn_valid, oplsn);
rwlock_read_unlock(&cf->fdlock);
if (r!=0) goto error;
close_error = cf->close_userdata(cf, cf->fd, cf->userdata, error_string, oplsn_valid, oplsn);
}
cf->close_userdata = NULL;
cf->checkpoint_userdata = NULL;
cf->begin_checkpoint_userdata = NULL;
cf->end_checkpoint_userdata = NULL;
cf->userdata = NULL;
remove_cf_from_cachefiles_list(cf);
toku_cond_destroy(&cf->background_wait);
rwlock_write_lock(&cf->fdlock, ct->mutex); //Just make sure we can get it.
if (!toku_cachefile_is_dev_null_unlocked(cf)) {
cachetable_unlock(ct);
r = toku_file_fsync_without_accounting(cf->fd); //t:2444
assert(r == 0);
cachetable_lock(ct);
}
rwlock_write_unlock(&cf->fdlock);
rwlock_destroy(&cf->fdlock);
assert(toku_list_empty(&cf->pairs_for_cachefile));
// Don't hold the cachetable lock during fsync/close/unlink, etc
cachetable_unlock(ct);
// fsync and close the fd.
r = toku_file_fsync_without_accounting(cf->fd);
assert(r == 0);
r = close(cf->fd);
assert(r == 0);
cf->fd = -1;
if (cf->fname_in_env) toku_free(cf->fname_in_env);
// Unlink the file if the bit was set
if (cf->unlink_on_close) {
char *fname_in_cwd = toku_cachetable_get_fname_in_cwd(cf->cachetable, cf->fname_in_env);
r = unlink(fname_in_cwd);
assert_zero(r);
toku_free(fname_in_cwd);
}
toku_free(cf->fname_in_env);
toku_free(cf);
return r;
// If close userdata returned nonzero, pass that error code to the caller
if (close_error != 0) {
r = close_error;
}
return r;
}
//
......@@ -1098,7 +1076,6 @@ static void cachetable_free_pair(CACHETABLE ct, PAIR p) {
void *write_extraargs = p->write_extraargs;
PAIR_ATTR old_attr = p->attr;
rwlock_prefer_read_lock(&cachefile->fdlock, ct->mutex);
cachetable_evictions++;
cachetable_unlock(ct);
PAIR_ATTR new_attr = p->attr;
......@@ -1107,7 +1084,6 @@ static void cachetable_free_pair(CACHETABLE ct, PAIR p) {
flush_callback(cachefile, cachefile->fd, key, value, &disk_data, write_extraargs, old_attr, &new_attr, FALSE, FALSE, TRUE, FALSE);
cachetable_lock(ct);
rwlock_read_unlock(&cachefile->fdlock);
ctpair_destroy(p);
}
......@@ -1154,13 +1130,9 @@ static void cachetable_only_write_locked_data(
PAIR_ATTR old_attr = p->attr;
BOOL dowrite = TRUE;
rwlock_prefer_read_lock(&cachefile->fdlock, ct->mutex);
cachetable_unlock(ct);
// write callback
if (toku_cachefile_is_dev_null_unlocked(cachefile)) {
dowrite = FALSE;
}
flush_callback(
cachefile,
cachefile->fd,
......@@ -1182,7 +1154,6 @@ static void cachetable_only_write_locked_data(
ct->size_current -= p->cloned_value_size;
p->cloned_value_size = 0;
}
rwlock_read_unlock(&cachefile->fdlock);
}
......@@ -1972,13 +1943,11 @@ do_partial_fetch(
assert(!p->dirty);
p->state = CTPAIR_READING;
rwlock_prefer_read_lock(&cachefile->fdlock, ct->mutex);
nb_mutex_lock(&p->disk_nb_mutex, ct->mutex);
cachetable_unlock(ct);
int r = pf_callback(p->value_data, p->disk_data, read_extraargs, cachefile->fd, &new_attr);
lazy_assert_zero(r);
cachetable_lock(ct);
rwlock_read_unlock(&cachefile->fdlock);
p->attr = new_attr;
cachetable_change_pair_attr(ct, old_attr, new_attr);
p->state = CTPAIR_IDLE;
......@@ -2018,13 +1987,11 @@ void toku_cachetable_pf_pinned_pair(
assert(p->value_data == value);
assert(nb_mutex_writers(&p->value_nb_mutex));
nb_mutex_lock(&p->disk_nb_mutex, cf->cachetable->mutex);
rwlock_prefer_read_lock(&cf->fdlock, cf->cachetable->mutex);
int fd = cf->fd;
cachetable_unlock(cf->cachetable);
pf_callback(value, p->disk_data, read_extraargs, fd, &attr);
cachetable_lock(cf->cachetable);
nb_mutex_write_unlock(&p->disk_nb_mutex);
rwlock_read_unlock(&cf->fdlock);
cachetable_unlock(cf->cachetable);
}
......@@ -2092,18 +2059,15 @@ static void cachetable_fetch_pair(
WHEN_TRACE_CT(printf("%s:%d CT: fetch_callback(%lld...)\n", __FILE__, __LINE__, key));
rwlock_prefer_read_lock(&cf->fdlock, ct->mutex);
nb_mutex_lock(&p->disk_nb_mutex, ct->mutex);
cachetable_unlock(ct);
int r;
assert(!toku_cachefile_is_dev_null_unlocked(cf));
r = fetch_callback(cf, cf->fd, key, fullhash, &toku_value, &disk_data, &attr, &dirty, read_extraargs);
if (dirty)
p->dirty = CACHETABLE_DIRTY;
cachetable_lock(ct);
rwlock_read_unlock(&cf->fdlock);
// ft-ops.c asserts that get_and_pin succeeds,
// so we might as well just assert it here as opposed
// to trying to support an INVALID state
......@@ -3343,9 +3307,9 @@ toku_cachetable_begin_checkpoint (CACHETABLE ct, TOKULOGGER logger) {
assert(ct->cachefiles_in_checkpoint==NULL);
cachefiles_lock(ct);
for (cf = ct->cachefiles; cf; cf=cf->next) {
assert(!cf->is_closing); //Closing requires ydb lock (or in checkpoint). Cannot happen.
//Incremement reference count of cachefile because we're using it for the checkpoint.
//This will prevent closing during the checkpoint.
// The caller must serialize open, close, and begin checkpoint.
// So we should never see a closing cachefile here.
assert(!cf->is_closing);
// putting this check so that this function may be called
// by cachetable tests
......@@ -3570,7 +3534,6 @@ toku_cachetable_end_checkpoint(CACHETABLE ct, TOKULOGGER logger,
CACHEFILE cf;
for (cf = ct->cachefiles_in_checkpoint; cf; cf=cf->next_in_checkpoint) {
if (cf->checkpoint_userdata) {
rwlock_prefer_read_lock(&cf->fdlock, ct->mutex);
if (!logger || ct->lsn_of_checkpoint_in_progress.lsn != cf->most_recent_global_checkpoint_that_finished_early.lsn) {
assert(ct->lsn_of_checkpoint_in_progress.lsn >= cf->most_recent_global_checkpoint_that_finished_early.lsn);
cachetable_unlock(ct);
......@@ -3585,7 +3548,6 @@ toku_cachetable_end_checkpoint(CACHETABLE ct, TOKULOGGER logger,
else {
assert(cf->checkpoint_state == CS_NOT_IN_PROGRESS);
}
rwlock_read_unlock(&cf->fdlock);
}
}
}
......@@ -3615,11 +3577,9 @@ toku_cachetable_end_checkpoint(CACHETABLE ct, TOKULOGGER logger,
//cachefiles_in_checkpoint is protected by the checkpoint_safe_lock
for (cf = ct->cachefiles_in_checkpoint; cf; cf=cf->next_in_checkpoint) {
if (cf->end_checkpoint_userdata) {
rwlock_prefer_read_lock(&cf->fdlock, ct->mutex);
if (!logger || ct->lsn_of_checkpoint_in_progress.lsn != cf->most_recent_global_checkpoint_that_finished_early.lsn) {
assert(ct->lsn_of_checkpoint_in_progress.lsn >= cf->most_recent_global_checkpoint_that_finished_early.lsn);
cachetable_unlock(ct);
//end_checkpoint fsyncs the fd, which needs the fdlock
assert(cf->checkpoint_state == CS_CALLED_CHECKPOINT);
int r = cf->end_checkpoint_userdata(cf, cf->fd, cf->userdata);
assert(r==0);
......@@ -3627,7 +3587,6 @@ toku_cachetable_end_checkpoint(CACHETABLE ct, TOKULOGGER logger,
cachetable_lock(ct);
}
assert(cf->checkpoint_state == CS_NOT_IN_PROGRESS);
rwlock_read_unlock(&cf->fdlock);
}
}
}
......@@ -3799,46 +3758,31 @@ toku_cachefile_get_cachetable(CACHEFILE cf) {
int
toku_cachefile_fsync(CACHEFILE cf) {
int r;
if (toku_cachefile_is_dev_null_unlocked(cf))
r = 0; //Don't fsync /dev/null
else
r = toku_file_fsync(cf->fd);
return r;
}
int toku_cachefile_redirect_nullfd (CACHEFILE cf) {
int null_fd;
struct fileid fileid;
// Make it so when the cachefile closes, the underlying file is unlinked
void
toku_cachefile_unlink_on_close(CACHEFILE cf) {
assert(!cf->unlink_on_close);
cf->unlink_on_close = true;
}
CACHETABLE ct = cf->cachetable;
cachetable_lock(ct);
rwlock_write_lock(&cf->fdlock, ct->mutex);
null_fd = open(DEV_NULL_FILE, O_WRONLY+O_BINARY);
assert(null_fd>=0);
int r = toku_os_get_unique_file_id(null_fd, &fileid);
assert(r==0);
close(cf->fd); // no change for t:2444
cf->fd = null_fd;
char *saved_fname_in_env = cf->fname_in_env;
cf->fname_in_env = NULL;
cachefile_init_filenum(cf, null_fd, saved_fname_in_env, fileid);
if (saved_fname_in_env) toku_free(saved_fname_in_env);
cf->is_dev_null = TRUE;
rwlock_write_unlock(&cf->fdlock);
cachetable_unlock(ct);
return 0;
// is this cachefile marked as unlink on close?
bool
toku_cachefile_is_unlink_on_close(CACHEFILE cf) {
return cf->unlink_on_close;
}
u_int64_t toku_cachefile_size(CACHEFILE cf) {
int64_t file_size;
int fd = toku_cachefile_get_and_pin_fd(cf);
int fd = toku_cachefile_get_fd(cf);
int r = toku_os_get_file_size(fd, &file_size);
toku_cachefile_unpin_fd(cf);
assert_zero(r);
return file_size;
}
char *
toku_construct_full_name(int count, ...) {
va_list ap;
......@@ -3884,7 +3828,6 @@ cleaner_thread_rate_pair(PAIR p)
static int const CLEANER_N_TO_CHECK = 8;
// FIXME this is global but no one uses it except cachetable.c
int
toku_cleaner_thread (void *cachetable_v)
// Effect: runs a cleaner.
......@@ -3961,22 +3904,11 @@ toku_cleaner_thread (void *cachetable_v)
CACHEFILE cf = best_pair->cachefile;
BOOL cleaner_callback_called = FALSE;
// grab the fdlock, because the cleaner callback
// will access the fd.
rwlock_prefer_read_lock(&cf->fdlock, ct->mutex);
// it's theoretically possible that after writing a PAIR for checkpoint, the
// PAIR's heuristic tells us nothing needs to be done. It is not possible
// in Dr. Noga, but unit tests verify this behavior works properly.
//
// also, because the cleaner thread needs to act as a client
// and honor the same invariants that client threads honor,
// we refuse to call the cleaner callback if the cachefile
// has been redirected to /dev/null, because client threads
// do not call APIs that access the file if the file has been
// redirected to /dev/null
if (!toku_cachefile_is_dev_null_unlocked(cf) &&
cleaner_thread_rate_pair(best_pair) > 0)
if (cleaner_thread_rate_pair(best_pair) > 0)
{
cachetable_unlock(ct);
int r = best_pair->cleaner_callback(best_pair->value_data,
......@@ -3994,7 +3926,6 @@ toku_cleaner_thread (void *cachetable_v)
assert(!best_pair->cq);
nb_mutex_write_unlock(&best_pair->value_nb_mutex);
}
rwlock_read_unlock(&cf->fdlock);
// We need to make sure the cachefile sticks around so a close
// can't come destroy it. That's the purpose of this
// "add/remove_background_job" business, which means the
......
......@@ -413,32 +413,27 @@ int toku_cachefile_flush (CACHEFILE);
// Get the file descriptor associated with the cachefile
// Return the file descriptor
// Grabs a read lock protecting the fd
int toku_cachefile_get_and_pin_fd (CACHEFILE);
int toku_cachefile_get_fd (CACHEFILE);
// Get the iname (within the environment) associated with the cachefile
// Return the filename
char * toku_cachefile_fname_in_env (CACHEFILE cf);
// Releases the read lock (taken by toku_cachefile_get_and_pin_fd) protecting the fd
void toku_cachefile_unpin_fd (CACHEFILE);
// For test programs only.
// Set the cachefile's fd and fname.
// Effect: Bind the cachefile to a new fd and fname. The old fd is closed.
// Returns: 0 if success, otherwise an error number
int toku_cachefile_set_fd (CACHEFILE cf, int fd, const char *fname_relative_to_env);
// Equivalent to toku_cachefile_set_fd to /dev/null but without
// closing the user data.
int toku_cachefile_redirect_nullfd (CACHEFILE cf);
// Make it so when the cachefile closes, the underlying file is unlinked
void toku_cachefile_unlink_on_close(CACHEFILE cf);
// is this cachefile marked as unlink on close?
bool toku_cachefile_is_unlink_on_close(CACHEFILE cf);
// Truncate a cachefile
int toku_cachefile_truncate (CACHEFILE cf, toku_off_t new_size);
//has it been redirected to dev null?
//Must have called toku_cachefile_get_and_pin_fd to hold a lock around this function
BOOL toku_cachefile_is_dev_null_unlocked (CACHEFILE cf);
// Return the logger associated with the cachefile
TOKULOGGER toku_cachefile_logger (CACHEFILE);
......
......@@ -856,9 +856,6 @@ toku_verify_ftnode (FT_HANDLE brt,
int toku_db_badformat(void) __attribute__((__warn_unused_result__));
int toku_ft_remove_on_commit(TOKUTXN child, DBT* iname_dbt_p) __attribute__((__warn_unused_result__));
int toku_ft_remove_now(CACHETABLE ct, DBT* iname_dbt_p) __attribute__((__warn_unused_result__));
typedef enum {
FT_UPGRADE_FOOTPRINT = 0,
FT_UPGRADE_STATUS_NUM_ROWS
......
......@@ -697,7 +697,6 @@ void toku_ftnode_clone_callback(
}
//fd is protected (must be holding fdlock)
void toku_ftnode_flush_callback (
CACHEFILE cachefile,
int fd,
......@@ -765,7 +764,6 @@ toku_ft_status_update_pivot_fetch_reason(struct ftnode_fetch_extra *bfe)
}
}
//fd is protected (must be holding fdlock)
int toku_ftnode_fetch_callback (CACHEFILE UU(cachefile), int fd, BLOCKNUM nodename, u_int32_t fullhash,
void **ftnode_pv, void** disk_data, PAIR_ATTR *sizep, int *dirtyp, void *extraargs) {
assert(extraargs);
......@@ -2568,19 +2566,18 @@ int toku_ft_insert (FT_HANDLE brt, DBT *key, DBT *val, TOKUTXN txn) {
}
int
toku_ft_load_recovery(TOKUTXN txn, char const * old_iname, char const * new_iname, int do_fsync, int do_log, LSN *load_lsn) {
toku_ft_load_recovery(TOKUTXN txn, FILENUM old_filenum, char const * new_iname, int do_fsync, int do_log, LSN *load_lsn) {
int r = 0;
assert(txn);
toku_txn_force_fsync_on_commit(txn); //If the txn commits, the commit MUST be in the log
//before the (old) file is actually unlinked
TOKULOGGER logger = toku_txn_logger(txn);
BYTESTRING old_iname_bs = {.len=strlen(old_iname), .data=(char*)old_iname};
BYTESTRING new_iname_bs = {.len=strlen(new_iname), .data=(char*)new_iname};
r = toku_logger_save_rollback_load(txn, &old_iname_bs, &new_iname_bs);
r = toku_logger_save_rollback_load(txn, old_filenum, &new_iname_bs);
if (r==0 && do_log && logger) {
TXNID xid = toku_txn_get_txnid(txn);
r = toku_log_load(logger, load_lsn, do_fsync, xid, old_iname_bs, new_iname_bs);
r = toku_log_load(logger, load_lsn, do_fsync, xid, old_filenum, new_iname_bs);
}
return r;
}
......@@ -2637,9 +2634,9 @@ toku_ft_optimize (FT_HANDLE brt) {
int
toku_ft_load(FT_HANDLE brt, TOKUTXN txn, char const * new_iname, int do_fsync, LSN *load_lsn) {
int r = 0;
char const * old_iname = toku_cachefile_fname_in_env(brt->ft->cf);
FILENUM old_filenum = toku_cachefile_filenum(brt->ft->cf);
int do_log = 1;
r = toku_ft_load_recovery(txn, old_iname, new_iname, do_fsync, do_log, load_lsn);
r = toku_ft_load_recovery(txn, old_filenum, new_iname, do_fsync, do_log, load_lsn);
return r;
}
......@@ -3126,13 +3123,12 @@ toku_ft_change_descriptor(
// write new_descriptor to header
new_d.dbt = *new_descriptor;
fd = toku_cachefile_get_and_pin_fd (t->ft->cf);
fd = toku_cachefile_get_fd (t->ft->cf);
r = toku_update_descriptor(t->ft, &new_d, fd);
// very infrequent operation, worth precise threadsafe count
if (r == 0) {
STATUS_VALUE(FT_DESCRIPTOR_SET)++;
}
toku_cachefile_unpin_fd(t->ft->cf);
if (r!=0) goto cleanup;
if (update_cmp_descriptor) {
......@@ -3279,9 +3275,8 @@ ft_handle_open(FT_HANDLE t, const char *fname_in_env, int is_create, int only_cr
//Opening a brt may restore to previous checkpoint. Truncate if necessary.
{
int fd = toku_cachefile_get_and_pin_fd (ft->cf);
int fd = toku_cachefile_get_fd (ft->cf);
toku_maybe_truncate_cachefile_on_open(ft->blocktable, fd, ft);
toku_cachefile_unpin_fd(ft->cf);
}
r = 0;
......@@ -5494,91 +5489,74 @@ int toku_ft_handle_set_panic(FT_HANDLE brt, int panic, char *panic_string) {
return toku_ft_set_panic(brt->ft, panic, panic_string);
}
#if 0
int toku_logger_save_rollback_fdelete (TOKUTXN txn, u_int8_t file_was_open, FILENUM filenum, BYTESTRING iname)
int toku_logger_log_fdelete (TOKUTXN txn, const char *fname, FILENUM filenum, u_int8_t was_open)
#endif
// Prepare to remove a dictionary from the database when this transaction is committed:
// - if cachetable has file open, mark it as in use so that cf remains valid until we're done
// - mark transaction as NEED fsync on commit
// - make entry in rollback log
// - make fdelete entry in recovery log
int toku_ft_remove_on_commit(TOKUTXN txn, DBT* iname_in_env_dbt_p) {
assert(txn);
//
// Effect: when the txn commits, the ft's cachefile will be marked as unlink
// on close. see toku_commit_fdelete and how unlink on close works
// in toku_cachefile_close();
// Requires: serialized with begin checkpoint
// this does not need to take the open close lock because
// 1.) the ft/cf cannot go away because we have a live handle.
// 2.) we're not setting the unlink on close bit _here_. that
// happens on txn commit (as the name suggests).
// 3.) we're already holding the multi operation lock to
// synchronize with begin checkpoint.
// Contract: the iname of the ft should never be reused.
int
toku_ft_remove_on_commit(FT_HANDLE handle, TOKUTXN txn) {
int r;
const char *iname_in_env = iname_in_env_dbt_p->data;
CACHEFILE cf = NULL;
u_int8_t was_open = 0;
FILENUM filenum = {0};
CACHEFILE cf;
r = toku_cachefile_of_iname_in_env(txn->logger->ct, iname_in_env, &cf);
if (r == 0) {
was_open = TRUE;
filenum = toku_cachefile_filenum(cf);
FT h = toku_cachefile_get_userdata(cf);
r = toku_txn_note_ft(txn, h);
if (r!=0) return r;
}
else {
assert(r==ENOENT);
}
assert(txn);
cf = handle->ft->cf;
FT ft = toku_cachefile_get_userdata(cf);
toku_txn_force_fsync_on_commit(txn); // If the txn commits, the commit MUST be in the log
// before the file is actually unlinked
{
BYTESTRING iname_in_env_bs = { .len=strlen(iname_in_env), .data = (char*)iname_in_env };
// TODO: toku_txn_note_ft should return void
// Assert success here because note_ft also asserts success internally.
r = toku_txn_note_ft(txn, ft);
assert(r == 0);
// If the txn commits, the commit MUST be in the log before the file is actually unlinked
toku_txn_force_fsync_on_commit(txn);
// make entry in rollback log
r = toku_logger_save_rollback_fdelete(txn, was_open, filenum, &iname_in_env_bs);
assert_zero(r); //On error we would need to remove the CF reference, which is complicated.
}
if (r==0)
FILENUM filenum = toku_cachefile_filenum(cf);
r = toku_logger_save_rollback_fdelete(txn, filenum);
assert_zero(r);
// make entry in recovery log
r = toku_logger_log_fdelete(txn, iname_in_env);
r = toku_logger_log_fdelete(txn, filenum);
return r;
}
// Non-transaction version of fdelete
int toku_ft_remove_now(CACHETABLE ct, DBT* iname_in_env_dbt_p) {
int r;
const char *iname_in_env = iname_in_env_dbt_p->data;
// Non-transactional version of fdelete
//
// Effect: The ft file is unlinked when the handle closes and it's ft is not
// pinned by checkpoint. see toku_remove_ft_ref() and how unlink on
// close works in toku_cachefile_close();
// Requires: serialized with begin checkpoint
void
toku_ft_remove(FT_HANDLE handle) {
CACHEFILE cf;
r = toku_cachefile_of_iname_in_env(ct, iname_in_env, &cf);
if (r == 0) {
r = toku_cachefile_redirect_nullfd(cf);
assert_zero(r);
}
else
assert(r==ENOENT);
char *iname_in_cwd = toku_cachetable_get_fname_in_cwd(ct, iname_in_env_dbt_p->data);
r = unlink(iname_in_cwd); // we need a pathname relative to cwd
assert_zero(r);
toku_free(iname_in_cwd);
return r;
cf = handle->ft->cf;
toku_cachefile_unlink_on_close(cf);
}
int
toku_ft_get_fragmentation(FT_HANDLE brt, TOKU_DB_FRAGMENTATION report) {
int r;
int fd = toku_cachefile_get_and_pin_fd(brt->ft->cf);
int fd = toku_cachefile_get_fd(brt->ft->cf);
toku_ft_lock(brt->ft);
int64_t file_size;
if (toku_cachefile_is_dev_null_unlocked(brt->ft->cf))
r = EINVAL;
else
r = toku_os_get_file_size(fd, &file_size);
if (r==0) {
report->file_size_bytes = file_size;
toku_block_table_get_fragmentation_unlocked(brt->ft->blocktable, report);
}
toku_ft_unlock(brt->ft);
toku_cachefile_unpin_fd(brt->ft->cf);
return r;
}
......
......@@ -133,7 +133,7 @@ int toku_ft_maybe_update(FT_HANDLE brt, const DBT *key, const DBT *update_functi
// Returns 0 if successful
int toku_ft_maybe_update_broadcast(FT_HANDLE brt, const DBT *update_function_extra, TOKUTXN txn, BOOL oplsn_valid, LSN oplsn, BOOL do_logging, BOOL is_resetting_op) __attribute__ ((warn_unused_result));
int toku_ft_load_recovery(TOKUTXN txn, char const * old_iname, char const * new_iname, int do_fsync, int do_log, LSN *load_lsn) __attribute__ ((warn_unused_result));
int toku_ft_load_recovery(TOKUTXN txn, FILENUM old_filenum, char const * new_iname, int do_fsync, int do_log, LSN *load_lsn) __attribute__ ((warn_unused_result));
int toku_ft_load(FT_HANDLE brt, TOKUTXN txn, char const * new_iname, int do_fsync, LSN *get_lsn) __attribute__ ((warn_unused_result));
// 2954
int toku_ft_hot_index_recovery(TOKUTXN txn, FILENUMS filenums, int do_fsync, int do_log, LSN *hot_index_lsn);
......
......@@ -122,8 +122,9 @@ ft_log_fassociate_during_checkpoint (CACHEFILE cf, void *header_v) {
BYTESTRING bs = { strlen(fname_in_env), // don't include the NUL
fname_in_env };
TOKULOGGER logger = toku_cachefile_logger(cf);
FILENUM filenum = toku_cachefile_filenum (cf);
int r = toku_log_fassociate(logger, NULL, 0, filenum, ft->h->flags, bs);
FILENUM filenum = toku_cachefile_filenum(cf);
bool unlink_on_close = toku_cachefile_is_unlink_on_close(cf);
int r = toku_log_fassociate(logger, NULL, 0, filenum, ft->h->flags, bs, unlink_on_close);
return r;
}
......@@ -508,7 +509,7 @@ int toku_read_ft_and_store_in_cachefile (FT_HANDLE brt, CACHEFILE cf, LSN max_ac
FT h;
int r;
{
int fd = toku_cachefile_get_and_pin_fd (cf);
int fd = toku_cachefile_get_fd(cf);
enum deserialize_error_code e = toku_deserialize_ft_from(fd, max_acceptable_lsn, &h);
if (e == DS_XSUM_FAIL) {
fprintf(stderr, "Checksum failure while reading header in file %s.\n", toku_cachefile_fname_in_env(cf));
......@@ -520,7 +521,6 @@ int toku_read_ft_and_store_in_cachefile (FT_HANDLE brt, CACHEFILE cf, LSN max_ac
} else {
assert(false);
}
toku_cachefile_unpin_fd(cf);
}
if (r!=0) return r;
h->cf = cf;
......
......@@ -14,8 +14,16 @@
#include "ft-search.h"
#include "compress.h"
void toku_ft_suppress_rollbacks(FT h, TOKUTXN txn);
// remove a ft, transactionless.
// if the ft is being checkpointed, it will be removed after checkpoint.
void toku_ft_remove(FT_HANDLE handle);
// remove a ft using the given txn. when the txn commits, the ft is removed.
// if the ft is being checkpointed, it will be removed after checkpoint.
int toku_ft_remove_on_commit(FT_HANDLE handle, TOKUTXN txn) __attribute__((__warn_unused_result__));
//Effect: suppresses rollback logs
void toku_ft_suppress_rollbacks(FT h, TOKUTXN txn);
void toku_ft_init_treelock(FT h);
void toku_ft_destroy_treelock(FT h);
......
......@@ -90,7 +90,6 @@ alignup64(u_int64_t a, u_int64_t b) {
//Race condition if ydb lock is split.
//Ydb lock is held when this function is called.
//Not going to truncate and delete (redirect to devnull) at same time.
//Must be holding a read or write lock on fdlock (fd is protected)
void
toku_maybe_truncate_cachefile (CACHEFILE cf, int fd, u_int64_t size_used)
// Effect: If file size >= SIZE+32MiB, reduce file size.
......@@ -101,7 +100,6 @@ toku_maybe_truncate_cachefile (CACHEFILE cf, int fd, u_int64_t size_used)
//the pwrite lock needlessly.
//Check file size after taking lock to avoid race conditions.
int64_t file_size;
if (toku_cachefile_is_dev_null_unlocked(cf)) goto done;
{
int r = toku_os_get_file_size(fd, &file_size);
lazy_assert_zero(r);
......@@ -123,7 +121,6 @@ toku_maybe_truncate_cachefile (CACHEFILE cf, int fd, u_int64_t size_used)
}
toku_unlock_for_pwrite();
}
done:
return;
}
......
......@@ -44,9 +44,7 @@ struct logtype {
const struct logtype rollbacks[] = {
//TODO: #2037 Add dname
{"fdelete", 'U', FA{{"u_int8_t", "file_was_open", 0},
{"FILENUM", "filenum", 0},
{"BYTESTRING", "iname", 0},
{"fdelete", 'U', FA{{"FILENUM", "filenum", 0},
NULLFIELD}},
//TODO: #2037 Add dname
{"fcreate", 'F', FA{{"FILENUM", "filenum", 0},
......@@ -68,7 +66,7 @@ const struct logtype rollbacks[] = {
{"BLOCKNUM", "spilled_tail", 0},
{"u_int32_t", "spilled_tail_hash", 0},
NULLFIELD}},
{"load", 'l', FA{{"BYTESTRING", "old_iname", 0},
{"load", 'l', FA{{"FILENUM", "old_filenum", 0},
{"BYTESTRING", "new_iname", 0},
NULLFIELD}},
// #2954
......@@ -104,6 +102,7 @@ const struct logtype logtypes[] = {
{"fassociate", 'f', FA{{"FILENUM", "filenum", 0},
{"u_int32_t", "treeflags", 0},
{"BYTESTRING", "iname", 0}, // pathname of file
{"u_int8_t", "unlink_on_close", 0},
NULLFIELD}},
//We do not use a TXNINFO struct since recovery log has
//FILENUMS and TOKUTXN has FTs (for open_fts)
......@@ -159,7 +158,7 @@ const struct logtype logtypes[] = {
NULLFIELD}},
//TODO: #2037 Add dname
{"fdelete", 'U', FA{{"TXNID", "xid", 0},
{"BYTESTRING", "iname", 0},
{"FILENUM", "filenum", 0},
NULLFIELD}},
{"enq_insert", 'I', FA{{"FILENUM", "filenum", 0},
{"TXNID", "xid", 0},
......@@ -193,7 +192,7 @@ const struct logtype logtypes[] = {
{"shutdown", 'Q', FA{{"u_int64_t", "timestamp", 0},
NULLFIELD}},
{"load", 'l', FA{{"TXNID", "xid", 0},
{"BYTESTRING", "old_iname", 0},
{"FILENUM", "old_filenum", 0},
{"BYTESTRING", "new_iname", 0},
NULLFIELD}},
// #2954
......
......@@ -860,13 +860,12 @@ int toku_logger_log_fcreate (TOKUTXN txn, const char *fname, FILENUM filenum, u_
}
// fname is the iname
int toku_logger_log_fdelete (TOKUTXN txn, const char *fname) {
// We only do fdelete on open ft's, so we pass the filenum here
int toku_logger_log_fdelete (TOKUTXN txn, FILENUM filenum) {
if (txn==0) return 0;
if (txn->logger->is_panicked) return EINVAL;
BYTESTRING bs = { .len=strlen(fname), .data = (char *) fname };
//No fsync.
int r = toku_log_fdelete (txn->logger, (LSN*)0, 0, toku_txn_get_txnid(txn), bs);
int r = toku_log_fdelete (txn->logger, (LSN*)0, 0, toku_txn_get_txnid(txn), filenum);
return r;
}
......
......@@ -61,7 +61,7 @@ int toku_logger_restart(TOKULOGGER logger, LSN lastlsn);
int toku_logger_maybe_trim_log(TOKULOGGER logger, LSN oldest_open_lsn);
int toku_logger_log_fcreate (TOKUTXN txn, const char *fname, FILENUM filenum, u_int32_t mode, u_int32_t flags, u_int32_t nodesize, u_int32_t basementnodesize, enum toku_compression_method compression_method);
int toku_logger_log_fdelete (TOKUTXN txn, const char *fname);
int toku_logger_log_fdelete (TOKUTXN txn, FILENUM filenum);
int toku_logger_log_fopen (TOKUTXN txn, const char * fname, FILENUM filenum, uint32_t treeflags);
int toku_fread_u_int8_t (FILE *f, u_int8_t *v, struct x1764 *mm, u_int32_t *len);
......
......@@ -438,6 +438,12 @@ static int toku_recover_fassociate (struct logtype_fassociate *l, RECOVER_ENV re
assert(r==0);
}
}
// try to open the file again and if we get it, restore
// the unlink on close bit.
int ret = file_map_find(&renv->fmap, l->filenum, &tuple);
if (ret == 0 && l->unlink_on_close) {
toku_cachefile_unlink_on_close(tuple->ft_handle->ft->cf);
}
break;
case FORWARD_NEWER_CHECKPOINT_END:
if (r == 0) { //IF it is open
......@@ -451,6 +457,7 @@ static int toku_recover_fassociate (struct logtype_fassociate *l, RECOVER_ENV re
return 0;
}
toku_free(fname);
return r;
}
......@@ -850,23 +857,16 @@ static int toku_recover_fdelete (struct logtype_fdelete *l, RECOVER_ENV renv) {
int r = toku_txnid2txn(renv->logger, l->xid, &txn);
assert(r == 0);
assert(txn != NULL);
char *fixediname = fixup_fname(&l->iname);
{ //Only if it exists
toku_struct_stat buf;
r = toku_stat(fixediname, &buf);
if (r==0) {
// txn exists and file exists, so create fdelete rollback entry
DBT iname_dbt;
toku_fill_dbt(&iname_dbt, fixediname, strlen(fixediname)+1);
r = toku_ft_remove_on_commit(txn, &iname_dbt);
assert(r==0);
}
else {
assert(errno==ENOENT);
}
}
toku_free(fixediname);
// if the forward scan in recovery found this file and opened it, we
// need to mark the txn to remove the ft on commit. if the file was
// not found and not opened, we don't need to do anything - the ft
// is already gone, so we're happy.
struct file_map_tuple *tuple;
r = file_map_find(&renv->fmap, l->filenum, &tuple);
if (r == 0) {
r = toku_ft_remove_on_commit(tuple->ft_handle, txn);
}
return 0;
}
......@@ -1141,13 +1141,11 @@ static int toku_recover_load(struct logtype_load *UU(l), RECOVER_ENV UU(renv)) {
r = toku_txnid2txn(renv->logger, l->xid, &txn);
assert(r == 0);
assert(txn!=NULL);
char *old_iname = fixup_fname(&l->old_iname);
char *new_iname = fixup_fname(&l->new_iname);
r = toku_ft_load_recovery(txn, old_iname, new_iname, 0, 0, (LSN*)NULL);
r = toku_ft_load_recovery(txn, l->old_filenum, new_iname, 0, 0, (LSN*)NULL);
assert(r==0);
toku_free(old_iname);
toku_free(new_iname);
return 0;
}
......
......@@ -26,43 +26,33 @@
#define TOKU_DO_COMMIT_CMD_DELETE 1
int
toku_commit_fdelete (u_int8_t file_was_open,
FILENUM filenum, // valid if file_was_open
BYTESTRING bs_fname, // cwd/iname
toku_commit_fdelete (FILENUM filenum,
TOKUTXN txn,
YIELDF UU(yield),
void *UU(yield_v),
LSN UU(oplsn)) //oplsn is the lsn of the commit
{
//TODO: #2037 verify the file is (user) closed
//Remove reference to the fd in the cachetable
CACHEFILE cf;
int r;
if (file_was_open) { // file was open when toku_ft_remove_on_commit() was called
r = toku_cachefile_of_filenum(txn->logger->ct, filenum, &cf);
if (r==ENOENT) { //Missing file on recovered transaction is not an error
CACHEFILE cf;
CACHETABLE ct = txn->logger->ct;
// Try to get the cachefile for this filenum. A missing file on recovery
// is not an error, but a missing file outside of recovery is.
r = toku_cachefile_of_filenum(ct, filenum, &cf);
if (r == ENOENT) {
assert(txn->recovered_from_checkpoint);
r = 0;
goto done;
}
// file must be open now if it was open when the fdelete was logged,
// because the txn that created it opened it and noted it.
// XXX this does not look true "must still be open (toku_ft_remove_on_commit() incremented refcount)"
assert(r == 0);
{
(void)toku_cachefile_get_and_pin_fd(cf);
assert(!toku_cachefile_is_dev_null_unlocked(cf));
FT h = toku_cachefile_get_userdata(cf);
DICTIONARY_ID dict_id = h->dict_id;
assert_zero(r);
// TODO(yoni): this callback may not be necessary. it seems to remove
// the locktree from the environment's ltm, but that shouldn't
// be necessary since that happens once this txn commits
// (which is after this function ends, essentially)
FT ft = toku_cachefile_get_userdata(cf);
DICTIONARY_ID dict_id = ft->dict_id;
toku_logger_call_remove_finalize_callback(txn->logger, dict_id);
toku_cachefile_unpin_fd(cf);
}
r = toku_cachefile_redirect_nullfd(cf);
assert(r==0);
}
{
char *fname_in_env = fixup_fname(&bs_fname);
char *fname_in_cwd = toku_cachetable_get_fname_in_cwd(txn->logger->ct, fname_in_env);
// bug fix for #4718
// bug was introduced in with fix for #3590
......@@ -82,19 +72,23 @@ toku_commit_fdelete (u_int8_t file_was_open,
assert_zero(r);
}
r = unlink(fname_in_cwd);
assert(r==0 || errno==ENOENT);
toku_free(fname_in_env);
toku_free(fname_in_cwd);
}
// Mark the cachefile as unlink on close. There are two ways for close
// to be eventually called on the cachefile:
//
// - when this txn completes, it will release a reference on the
// ft and close it, UNLESS it was pinned by checkpoint
// - if the cf was pinned by checkpoint, an unpin will release the
// final reference and call close. it must be the final reference
// since this txn has exclusive access to dictionary (by the
// directory row lock for its dname) and we would not get this
// far if there were other live handles.
toku_cachefile_unlink_on_close(cf);
done:
return 0;
return r;
}
int
toku_rollback_fdelete (u_int8_t UU(file_was_open),
FILENUM UU(filenum),
BYTESTRING UU(bs_fname),
toku_rollback_fdelete (FILENUM UU(filenum),
TOKUTXN UU(txn),
YIELDF UU(yield),
void* UU(yield_v),
......@@ -117,45 +111,45 @@ toku_commit_fcreate (FILENUM UU(filenum),
int
toku_rollback_fcreate (FILENUM filenum,
BYTESTRING bs_fname, // cwd/iname
BYTESTRING UU(bs_fname),
TOKUTXN txn,
YIELDF UU(yield),
void* UU(yield_v),
LSN UU(oplsn))
{
//TODO: #2037 verify the file is (user) closed
int r;
CACHEFILE cf;
CACHETABLE ct = txn->logger->ct;
//Remove reference to the fd in the cachetable
CACHEFILE cf = NULL;
int r = toku_cachefile_of_filenum(txn->logger->ct, filenum, &cf);
if (r==ENOENT) { //Missing file on recovered transaction is not an error
// Try to get the cachefile for this filenum. A missing file on recovery
// is not an error, but a missing file outside of recovery is.
r = toku_cachefile_of_filenum(ct, filenum, &cf);
if (r == ENOENT) {
assert(txn->recovered_from_checkpoint);
r = 0;
goto done;
}
// file must be open, because the txn that created it opened it and
// noted it,
assert(r == 0);
{
(void)toku_cachefile_get_and_pin_fd(cf);
assert(!toku_cachefile_is_dev_null_unlocked(cf));
FT h = toku_cachefile_get_userdata(cf);
DICTIONARY_ID dict_id = h->dict_id;
toku_logger_call_remove_finalize_callback(txn->logger, dict_id);
toku_cachefile_unpin_fd(cf);
}
r = toku_cachefile_redirect_nullfd(cf);
assert(r==0);
assert_zero(r);
{
char *fname_in_env = fixup_fname(&bs_fname);
char *fname_in_cwd = toku_cachetable_get_fname_in_cwd(txn->logger->ct, fname_in_env);
// TODO(yoni): this callback may not be necessary. it seems to remove
// the locktree from the environment's ltm, but that shouldn't
// be necessary since that happens once this txn commits
// (which is after this function ends, essentially)
FT ft = toku_cachefile_get_userdata(cf);
DICTIONARY_ID dict_id = ft->dict_id;
toku_logger_call_remove_finalize_callback(txn->logger, dict_id);
r = unlink(fname_in_cwd);
assert(r==0 || errno==ENOENT);
toku_free(fname_in_env);
toku_free(fname_in_cwd);
}
// Mark the cachefile as unlink on close. There are two ways for close
// to be eventually called on the cachefile:
//
// - when this txn completes, it will release a reference on the
// ft and close it, UNLESS it was pinned by checkpoint
// - if the cf was pinned by checkpoint, an unpin will release the
// final reference and call close. it must be the final reference
// since this txn has exclusive access to dictionary (by the
// directory row lock for its dname) and we would not get this
// far if there were other live handles.
toku_cachefile_unlink_on_close(cf);
done:
return 0;
}
......@@ -169,7 +163,6 @@ static int find_ft_from_filenum (OMTVALUE v, void *filenumvp) {
return 0;
}
// Input arg reset_root_xid_that_created TRUE means that this operation has changed the definition of this dictionary.
// (Example use is for schema change committed with txn that inserted cmdupdatebroadcast message.)
// The oplsn argument is ZERO_LSN for normal operation. When this function is called for recovery, it has the LSN of
......@@ -189,8 +182,6 @@ static int do_insertion (enum ft_msg_type type, FILENUM filenum, BYTESTRING key,
}
assert(r==0);
(void)toku_cachefile_get_and_pin_fd(cf);
if (!toku_cachefile_is_dev_null_unlocked(cf)) {
OMTVALUE hv=NULL;
r = toku_omt_find_zero(txn->open_fts, find_ft_from_filenum, &filenum, &hv, NULL);
assert(r==0);
......@@ -200,7 +191,7 @@ static int do_insertion (enum ft_msg_type type, FILENUM filenum, BYTESTRING key,
LSN treelsn = toku_ft_checkpoint_lsn(h);
if (oplsn.lsn <= treelsn.lsn) { // if operation was already applied to tree ...
r = 0; // ... do not apply it again.
goto cleanup;
goto done;
}
}
......@@ -219,9 +210,6 @@ static int do_insertion (enum ft_msg_type type, FILENUM filenum, BYTESTRING key,
TXNID new_root_xid_that_created = xids_get_outermost_xid(xids);
toku_reset_root_xid_that_created(h, new_root_xid_that_created);
}
}
cleanup:
toku_cachefile_unpin_fd(cf);
done:
return r;
}
......@@ -432,24 +420,28 @@ toku_rollback_rollinclude (TXNID xid,
}
int
toku_commit_load (BYTESTRING old_iname,
toku_commit_load (FILENUM old_filenum,
BYTESTRING UU(new_iname),
TOKUTXN txn,
YIELDF UU(yield),
void *UU(yield_v),
LSN UU(oplsn))
{
CACHEFILE cf;
int r;
char *fname_in_env = fixup_fname(&old_iname); //Delete old file
r = toku_cachefile_of_iname_in_env(txn->logger->ct, fname_in_env, &cf);
if (r==0) {
r = toku_cachefile_redirect_nullfd(cf);
assert(r==0);
}
else {
assert(r==ENOENT);
CACHEFILE old_cf;
CACHETABLE ct = txn->logger->ct;
// To commit a dictionary load, we delete the old file
//
// Try to get the cachefile for the old filenum. A missing file on recovery
// is not an error, but a missing file outside of recovery is.
r = toku_cachefile_of_filenum(ct, old_filenum, &old_cf);
if (r == ENOENT) {
invariant(txn->recovered_from_checkpoint);
r = 0;
goto done;
}
lazy_assert(r == 0);
// bug fix for #4718
// bug was introduced in with fix for #3590
......@@ -466,42 +458,44 @@ toku_commit_load (BYTESTRING old_iname,
// yielding the ydb lock before performing the fsync.
if (txn->logger) {
r = toku_logger_fsync_if_lsn_not_fsynced(txn->logger, txn->do_fsync_lsn);
assert_zero(r);
lazy_assert(r == 0);
}
char *fname_in_cwd = toku_cachetable_get_fname_in_cwd(txn->logger->ct, fname_in_env);
r = unlink(fname_in_cwd);
assert(r==0 || errno==ENOENT);
toku_free(fname_in_env);
toku_free(fname_in_cwd);
return 0;
toku_cachefile_unlink_on_close(old_cf);
done:
return r;
}
int
toku_rollback_load (BYTESTRING UU(old_iname),
toku_rollback_load (FILENUM UU(old_filenum),
BYTESTRING new_iname,
TOKUTXN txn,
YIELDF UU(yield),
void *UU(yield_v),
LSN UU(oplsn))
{
CACHEFILE cf;
int r;
char *fname_in_env = fixup_fname(&new_iname); //Delete new file
r = toku_cachefile_of_iname_in_env(txn->logger->ct, fname_in_env, &cf);
if (r==0) {
r = toku_cachefile_redirect_nullfd(cf);
assert(r==0);
}
else {
assert(r==ENOENT);
}
char *fname_in_cwd = toku_cachetable_get_fname_in_cwd(txn->logger->ct, fname_in_env);
CACHEFILE new_cf;
CACHETABLE ct = txn->logger->ct;
// To rollback a dictionary load, we delete the new file.
// Try to get the cachefile for the new fname.
char *fname_in_env = fixup_fname(&new_iname);
r = toku_cachefile_of_iname_in_env(ct, fname_in_env, &new_cf);
if (r == ENOENT) {
// It's possible the new iname was never created, so just try to
// unlink it if it's there and ignore the error if it's not.
char *fname_in_cwd = toku_cachetable_get_fname_in_cwd(ct, fname_in_env);
r = unlink(fname_in_cwd);
assert(r==0 || errno==ENOENT);
toku_free(fname_in_env);
assert(r == 0 || errno == ENOENT);
toku_free(fname_in_cwd);
return 0;
r = 0;
} else {
assert_zero(r);
toku_cachefile_unlink_on_close(new_cf);
}
toku_free(fname_in_env);
return r;
}
//2954
......@@ -617,8 +611,7 @@ toku_rollback_change_fdescriptor(FILENUM filenum,
// noted it,
assert(r==0);
fd = toku_cachefile_get_and_pin_fd(cf);
if (!toku_cachefile_is_dev_null_unlocked(cf)) {
fd = toku_cachefile_get_fd(cf);
OMTVALUE hv=NULL;
r = toku_omt_find_zero(txn->open_fts, find_ft_from_filenum, &filenum, &hv, NULL);
assert(r==0);
......@@ -628,8 +621,6 @@ toku_rollback_change_fdescriptor(FILENUM filenum,
toku_fill_dbt(&d.dbt, old_descriptor.data, old_descriptor.len);
r = toku_update_descriptor(h, &d, fd);
assert(r == 0);
}
toku_cachefile_unpin_fd(cf);
done:
return r;
}
......
/* -*- mode: C; c-basic-offset: 4; indent-tabs-mode: nil -*- */
// vim: expandtab:ts=8:sw=4:softtabstop=4:
#ident "$Id: cachetable-simple-verify.c 36689 2011-11-07 22:08:05Z zardosht $"
#ident "Copyright (c) 2007-2011 Tokutek Inc. All rights reserved."
#include "includes.h"
#include "test.h"
CACHEFILE f1;
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); }
//usleep (5*1024*1024);
PAIR_ATTR attr = make_pair_attr(8);
attr.cache_pressure_size = 8;
*new_size = attr;
if (w) {
assert(c);
}
}
static int
cleaner_callback(
void* UU(ftnode_pv),
BLOCKNUM UU(blocknum),
u_int32_t UU(fullhash),
void* UU(extraargs)
)
{
assert(FALSE);
return 0;
}
static void
cachetable_test (void) {
const int test_limit = 12;
int r;
CACHETABLE ct;
r = toku_create_cachetable(&ct, test_limit, ZERO_LSN, NULL_LOGGER); assert(r == 0);
char fname1[] = __SRCFILE__ "test1.dat";
unlink(fname1);
r = toku_cachetable_openf(&f1, ct, fname1, O_RDWR|O_CREAT, S_IRWXU|S_IRWXG|S_IRWXO); assert(r == 0);
void* v1;
//void* v2;
long s1;
//long s2;
CACHETABLE_WRITE_CALLBACK wc = def_write_callback(NULL);
wc.flush_callback = flush;
wc.cleaner_callback = cleaner_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);
PAIR_ATTR attr = make_pair_attr(8);
attr.cache_pressure_size = 8;
r = toku_cachetable_unpin(f1, make_blocknum(1), 1, CACHETABLE_DIRTY, attr);
// test that once we have redirected to /dev/null,
// cleaner callback is NOT called
r = toku_cachefile_redirect_nullfd(f1);
assert_zero(r);
toku_cleaner_thread(ct);
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;
}
......@@ -16,23 +16,20 @@ cachetable_fd_test (void) {
CACHEFILE cf;
r = toku_cachetable_openf(&cf, ct, fname1, O_RDWR|O_CREAT, S_IRWXU|S_IRWXG|S_IRWXO); assert(r == 0);
int fd1 = toku_cachefile_get_and_pin_fd(cf); assert(fd1 >= 0);
toku_cachefile_unpin_fd(cf);
int fd1 = toku_cachefile_get_fd(cf); assert(fd1 >= 0);
// test set to good fd succeeds
char fname2[] = __SRCFILE__ "test2.data";
unlink(fname2);
int fd2 = open(fname2, O_RDWR | O_CREAT, S_IRWXU|S_IRWXG|S_IRWXO); assert(fd2 >= 0 && fd1 != fd2);
r = toku_cachefile_set_fd(cf, fd2, fname2); assert(r == 0);
assert(toku_cachefile_get_and_pin_fd(cf) == fd2);
toku_cachefile_unpin_fd(cf);
assert(toku_cachefile_get_fd(cf) == fd2);
// test set to bogus fd fails
int fd3 = open(DEV_NULL_FILE, O_RDWR); assert(fd3 >= 0);
r = close(fd3); assert(r == 0);
r = toku_cachefile_set_fd(cf, fd3, DEV_NULL_FILE); assert(r != 0);
assert(toku_cachefile_get_and_pin_fd(cf) == fd2);
toku_cachefile_unpin_fd(cf);
assert(toku_cachefile_get_fd(cf) == fd2);
// test the filenum functions
FILENUM fn = toku_cachefile_filenum(cf);
......
......@@ -26,8 +26,7 @@ static void f_flush (CACHEFILE f,
) {
assert(size.size==BLOCKSIZE);
if (write_me) {
toku_os_full_pwrite(toku_cachefile_get_and_pin_fd(f), value, BLOCKSIZE, key.b);
toku_cachefile_unpin_fd(f);
toku_os_full_pwrite(toku_cachefile_get_fd(f), value, BLOCKSIZE, key.b);
}
if (!keep_me) {
toku_free(value);
......@@ -44,8 +43,7 @@ static int f_fetch (CACHEFILE f,
int *dirtyp,
void*extraargs __attribute__((__unused__))) {
void *buf = toku_malloc(BLOCKSIZE);
int r = pread(toku_cachefile_get_and_pin_fd(f), buf, BLOCKSIZE, key.b);
toku_cachefile_unpin_fd(f);
int r = pread(toku_cachefile_get_fd(f), buf, BLOCKSIZE, key.b);
assert(r==BLOCKSIZE);
*value = buf;
*sizep = make_pair_attr(BLOCKSIZE);
......
......@@ -202,10 +202,10 @@ int create_logfiles() {
//begin_checkpoint 'x': lsn=9 timestamp=1251309957584197 crc=cd067878 len=29
r = toku_log_begin_checkpoint(logger, &lsn, NO_FSYNC, 1251309957584197); assert(r==0); cp_txnid = lsn.lsn;
//fassociate 'f': lsn=11 filenum=1 fname={len=4 data="b.db"} crc=a7126035 len=33
r = toku_log_fassociate(logger, &lsn, NO_FSYNC, fn_bname, 0, bs_bname); assert(r==0);
r = toku_log_fassociate(logger, &lsn, NO_FSYNC, fn_bname, 0, bs_bname, 0); assert(r==0);
num_fassociate++;
//fassociate 'f': lsn=12 filenum=0 fname={len=4 data="a.db"} crc=a70c5f35 len=33
r = toku_log_fassociate(logger, &lsn, NO_FSYNC, fn_aname, 0, bs_aname); assert(r==0);
r = toku_log_fassociate(logger, &lsn, NO_FSYNC, fn_aname, 0, bs_aname, 0); assert(r==0);
num_fassociate++;
//xstillopen 's': lsn=10 txnid=7 parent=0 crc=00061816 len=37 <- obsolete
{
......
......@@ -23,7 +23,6 @@ const char *toku_copyright_string = "Copyright (c) 2007-2009 Tokutek Inc. All r
#include "toku_assert.h"
#include "ydb.h"
#include "ydb-internal.h"
#include <ft/ft-internal.h>
#include <ft/ft-flusher.h>
#include <ft/cachetable.h>
#include <ft/log.h>
......@@ -2649,6 +2648,8 @@ toku_env_dbremove(DB_ENV * env, DB_TXN *txn, const char *fname, const char *dbna
assert(dbname == NULL);
if (flags!=0) return EINVAL;
// We check for an open db here as a "fast path" to error.
// We'll need to check again below to be sure.
if (env_is_db_with_dname_open(env, dname))
return toku_ydb_do_error(env, EINVAL, "Cannot remove dictionary with an open handle.\n");
......@@ -2668,41 +2669,54 @@ toku_env_dbremove(DB_ENV * env, DB_TXN *txn, const char *fname, const char *dbna
// get iname
r = toku_db_get(env->i->directory, child, &dname_dbt, &iname_dbt, DB_SERIALIZABLE); // allocates memory for iname
char *iname = iname_dbt.data;
if (r==DB_NOTFOUND)
DB *db = NULL;
if (r == DB_NOTFOUND) {
r = ENOENT;
else if (r==0) {
} else if (r == 0) {
// remove (dname,iname) from directory
r = toku_db_del(env->i->directory, child, &dname_dbt, DB_DELETE_ANY, TRUE);
if (r == 0) {
if (using_txns) {
// this writes an fdelete to the transaction's rollback log.
// it is removed if the child txn aborts after any error case below
r = toku_ft_remove_on_commit(db_txn_struct_i(child)->tokutxn, &iname_dbt);
if (r != 0) {
goto exit;
}
r = toku_db_create(&db, env, 0);
assert_zero(r);
r = db_open_iname(db, txn, iname, 0, 0);
assert_zero(r);
//Now that we have a writelock on dname, verify that there are still no handles open. (to prevent race conditions)
if (r==0 && env_is_db_with_dname_open(env, dname))
if (using_txns) {
// Now that we have a writelock on dname, verify that there are still no handles open. (to prevent race conditions)
if (env_is_db_with_dname_open(env, dname)) {
r = toku_ydb_do_error(env, EINVAL, "Cannot remove dictionary with an open handle.\n");
if (r==0) {
goto exit;
}
// we know a live db handle does not exist.
//
// if the lock tree still exists, try to get a full table
// lock. if we can't get it, then some txn still needs
// the ft and we should return lock not granted.
// otherwise, we're okay in marking this brt as remove on
// use the internally opened db to try and get a table lock
//
// if we can't get it, then some txn needs the ft and we
// should return lock not granted.
//
// otherwise, we're okay in marking this ft as remove on
// commit. no new handles can open for this dictionary
// because the txn has directory write locks on the dname
if (!can_acquire_table_lock(env, child, iname)) {
if (toku_db_pre_acquire_table_lock(db, child) != 0) {
r = DB_LOCK_NOTGRANTED;
}
} else {
// The ft will be removed when the txn commits
r = toku_ft_remove_on_commit(db->i->ft_handle, db_txn_struct_i(child)->tokutxn);
assert_zero(r);
}
}
else {
r = toku_ft_remove_now(env->i->cachetable, &iname_dbt);
assert_zero(r);
}
// Remove the ft without a txn
toku_ft_remove(db->i->ft_handle);
}
}
exit:
if (db) {
int ret = toku_db_close(db);
assert(ret == 0);
}
if (using_txns) {
// close txn
if (r == 0) { // commit
......@@ -2714,13 +2728,12 @@ toku_env_dbremove(DB_ENV * env, DB_TXN *txn, const char *fname, const char *dbna
invariant(r2==0); // TODO panic
}
}
if (iname) toku_free(iname);
if (iname) {
toku_free(iname);
}
return r;
}
static int
env_dbrename_subdb(DB_ENV *env, DB_TXN *txn, const char *fname, const char *dbname, const char *newname, u_int32_t flags) {
int r;
......@@ -2756,7 +2769,9 @@ toku_env_dbrename(DB_ENV *env, DB_TXN *txn, const char *fname, const char *dbnam
const char * dname = fname;
assert(dbname == NULL);
if (flags!=0) return EINVAL;
if (flags != 0) return EINVAL;
// We check for open dnames for the old and new name as a "fast path" to error.
// We will need to check these again later.
if (env_is_db_with_dname_open(env, dname))
return toku_ydb_do_error(env, EINVAL, "Cannot rename dictionary with an open handle.\n");
if (env_is_db_with_dname_open(env, newname))
......@@ -2777,12 +2792,12 @@ toku_env_dbrename(DB_ENV *env, DB_TXN *txn, const char *fname, const char *dbnam
assert_zero(r);
}
char *iname;
// get iname
r = toku_db_get(env->i->directory, child, &old_dname_dbt, &iname_dbt, DB_SERIALIZABLE); // allocates memory for iname
iname = iname_dbt.data;
if (r==DB_NOTFOUND)
char *iname = iname_dbt.data;
if (r == DB_NOTFOUND) {
r = ENOENT;
else if (r==0) {
} else if (r == 0) {
// verify that newname does not already exist
r = db_getf_set(env->i->directory, child, DB_SERIALIZABLE, &new_dname_dbt, ydb_getf_do_nothing, NULL);
if (r == 0) {
......@@ -2791,27 +2806,39 @@ toku_env_dbrename(DB_ENV *env, DB_TXN *txn, const char *fname, const char *dbnam
else if (r == DB_NOTFOUND) {
// remove old (dname,iname) and insert (newname,iname) in directory
r = toku_db_del(env->i->directory, child, &old_dname_dbt, DB_DELETE_ANY, TRUE);
if (r == 0)
if (r != 0) { goto exit; }
r = toku_db_put(env->i->directory, child, &new_dname_dbt, &iname_dbt, 0, TRUE);
if (r != 0) { goto exit; }
//Now that we have writelocks on both dnames, verify that there are still no handles open. (to prevent race conditions)
if (r==0 && env_is_db_with_dname_open(env, dname))
if (env_is_db_with_dname_open(env, dname)) {
r = toku_ydb_do_error(env, EINVAL, "Cannot rename dictionary with an open handle.\n");
if (r == 0) {
// we know a live db handle for the old name does not exist.
goto exit;
}
if (env_is_db_with_dname_open(env, newname)) {
r = toku_ydb_do_error(env, EINVAL, "Cannot rename dictionary; Dictionary with target name has an open handle.\n");
goto exit;
}
// we know a live db handle does not exist.
//
// if the lock tree still exists, try to get a full table
// lock. if we can't get it, then some txn still references
// this dictionary, so we can't proceed.
// use the internally opened db to try and get a table lock
//
// if we can't get it, then some txn needs the ft and we
// should return lock not granted.
//
// otherwise, we're okay in marking this ft as remove on
// commit. no new handles can open for this dictionary
// because the txn has directory write locks on the dname
if (!can_acquire_table_lock(env, child, iname)) {
r = DB_LOCK_NOTGRANTED;
}
}
if (r==0 && env_is_db_with_dname_open(env, newname)) {
r = toku_ydb_do_error(env, EINVAL, "Cannot rename dictionary; Dictionary with target name has an open handle.\n");
}
// We don't do anything at the ft or cachetable layer for rename.
// We just update entries in the environment's directory.
}
}
exit:
if (using_txns) {
// close txn
if (r == 0) { // commit
......@@ -2823,11 +2850,10 @@ toku_env_dbrename(DB_ENV *env, DB_TXN *txn, const char *fname, const char *dbnam
invariant(r2==0); // TODO panic
}
}
if (iname) toku_free(iname);
if (iname) {
toku_free(iname);
}
return r;
}
int
......
Markdown is supported
0%
or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment