Improve sequential write performance of buffer manager.

This patch adds support for "asyncForce" which forces all pages to disk that
were dirtied before the *previous* call call to asyncForce.  This method
only applies to dirty pages in the Linux file cache (not in the Stasis buffer
pool), and is used by new writeback logic that bounds the amount of such
backlog that exists at any given time.

This patch also changes writeback behavior; the dirty page table now sorts
pages by pageid instead of LSN when it is invoked by writeback.  When it is
invoked by log truncation, it still sorts by LSN.
This commit is contained in:
Sears Russell 2011-10-08 22:01:04 +00:00
parent 64653e3cbe
commit b126b45bcc
12 changed files with 239 additions and 36 deletions

View file

@ -403,6 +403,11 @@ static void bhForcePages(stasis_buffer_manager_t* bm, stasis_buffer_manager_hand
stasis_buffer_hash_t * bh = bm->impl;
bh->page_handle->force_file(bh->page_handle);
}
static void bhAsyncForcePages(stasis_buffer_manager_t* bm, stasis_buffer_manager_handle_t *h) {
stasis_buffer_hash_t * bh = bm->impl;
bh->page_handle->async_force_file(bh->page_handle);
}
static void bhForcePageRange(stasis_buffer_manager_t *bm, stasis_buffer_manager_handle_t *h, pageid_t start, pageid_t stop) {
stasis_buffer_hash_t * bh = bm->impl;
bh->page_handle->force_range(bh->page_handle, start, stop);
@ -515,6 +520,7 @@ stasis_buffer_manager_t* stasis_buffer_manager_hash_open(stasis_page_handle_t *
bm->writeBackPage = bhWriteBackPage;
bm->tryToWriteBackPage = bhTryToWriteBackPage;
bm->forcePages = bhForcePages;
bm->asyncForcePages = bhAsyncForcePages;
bm->forcePageRange = bhForcePageRange;
bm->stasis_buffer_manager_close = bhBufDeinit;
bm->stasis_buffer_manager_simulate_crash = bhSimulateBufferManagerCrash;

View file

@ -10,13 +10,6 @@
#include <stasis/pageHandle.h>
#include <stasis/flags.h>
#define CLOCK
//#ifndef NO_CONCURRENT_LRU
//#ifndef CONCURRENT_LRU
//#define CONCURRENT_LRU
//#endif // CONCURRENT_LRU
//#endif // NO_CONCURRENT_LRU
//#define STRESS_TEST_WRITEBACK 1 // if defined, writeback as much as possible, as fast as possible.
typedef struct {
@ -42,7 +35,7 @@ typedef struct {
static inline int needFlush(stasis_buffer_manager_t * bm) {
stasis_buffer_concurrent_hash_t *bh = bm->impl;
pageid_t count = stasis_dirty_page_table_dirty_count(bh->dpt);
const pageid_t needed = 1000; //MAX_BUFFER_SIZE / 5;
pageid_t needed = stasis_dirty_page_count_soft_limit;
if(count > needed) {
DEBUG("Need flush? Dirty: %lld Total: %lld ret = %d\n", count, needed, count > needed);
}
@ -64,9 +57,6 @@ static int chWriteBackPage_helper(stasis_buffer_manager_t* bm, pageid_t pageid,
if(!trywritelock(p->loadlatch,0)) {
ret = EBUSY;
p->needsFlush = 1; // Not atomic. Oh well.
if(p->id != pageid) {
fprintf(stderr, "BUG FIX: %s:%d would have corrupted a latch's state, but did not\n", __FILE__, __LINE__);
}
} else {
if(p->id != pageid) { // it must have been written back...
unlock(p->loadlatch);
@ -123,9 +113,11 @@ static int chWriteBackPage_helper(stasis_buffer_manager_t* bm, pageid_t pageid,
return 0;
}
static int chWriteBackPage(stasis_buffer_manager_t* bm, pageid_t pageid) {
DEBUG("chWriteBackPage called");
return chWriteBackPage_helper(bm,pageid,0); // not hint; for correctness. Block (deadlock?) on contention.
}
static int chTryToWriteBackPage(stasis_buffer_manager_t* bm, pageid_t pageid) {
DEBUG("chTryToWriteBackPage called");
return chWriteBackPage_helper(bm,pageid,1); // just a hint. Return EBUSY on contention.
}
static void * writeBackWorker(void * bmp) {
@ -134,12 +126,14 @@ static void * writeBackWorker(void * bmp) {
pthread_mutex_t mut;
pthread_mutex_init(&mut,0);
while(1) {
while(ch->running && (!needFlush(bm))) {
DEBUG("Sleeping in write back worker (count = %lld)\n", stasis_dirty_page_table_dirty_count(bh->dpt));
pthread_mutex_lock(&mut);
pthread_cond_wait(&ch->needFree, &mut); // XXX Make sure it's OK to have many different mutexes waiting on the same cond.
pthread_mutex_unlock(&mut);
DEBUG("Woke write back worker (count = %lld)\n", stasis_dirty_page_table_dirty_count(bh->dpt));
while(ch->running && stasis_dirty_page_table_dirty_count(ch->dpt) < stasis_dirty_page_low_water_mark) {
if(!needFlush(bm)) {
printf("Sleeping in write back worker (count = %lld)\n", stasis_dirty_page_table_dirty_count(ch->dpt));
pthread_mutex_lock(&mut);
pthread_cond_wait(&ch->needFree, &mut); // XXX Make sure it's OK to have many different mutexes waiting on the same cond.
pthread_mutex_unlock(&mut);
printf("Woke write back worker (count = %lld)\n", stasis_dirty_page_table_dirty_count(ch->dpt));
}
}
if(!ch->running) { break; }
DEBUG("Calling flush\n");
@ -222,6 +216,8 @@ static inline stasis_buffer_concurrent_hash_tls_t * populateTLS(stasis_buffer_ma
int succ =
trywritelock(tls->p->loadlatch,0); // if this blocks, it is because someone else has pinned the page (it can't be due to eviction because the lru is atomic)
if(tls->p->dirty) pthread_cond_signal(&ch->needFree);
if(succ && (
// Work-stealing heuristic: If we don't know that writes are sequential, then write back the page we just encountered.
(!stasis_buffer_manager_hint_writes_are_sequential)
@ -239,8 +235,9 @@ static inline stasis_buffer_concurrent_hash_tls_t * populateTLS(stasis_buffer_ma
// note that we'd like to assert that the page is unpinned here. However, we can't simply look at p->queue, since another thread could be inside the "spooky" quote below.
tmp = 0;
if(tls->p->id >= 0) {
// Page is not in LRU, so we don't have to worry about the case where we
// are in sequential mode, and have to remove/add the page from/to the LRU.
DEBUG("App thread stole work from write back.\n");
// Page is not in LRU, so we don't have to worry about the case where we
// are in sequential mode, and have to remove/add the page from/to the LRU.
ch->page_handle->write(ch->page_handle, tls->p);
}
hashtable_remove_finish(ch->ht, &h); // need to hold bucket lock until page is flushed. Otherwise, another thread could read stale data from the filehandle.
@ -366,6 +363,10 @@ static void chForcePages(stasis_buffer_manager_t* bm, stasis_buffer_manager_hand
stasis_buffer_concurrent_hash_t * ch = bm->impl;
ch->page_handle->force_file(ch->page_handle);
}
static void chAsyncForcePages(stasis_buffer_manager_t* bm, stasis_buffer_manager_handle_t *h) {
stasis_buffer_concurrent_hash_t * ch = bm->impl;
ch->page_handle->async_force_file(ch->page_handle);
}
static void chForcePageRange(stasis_buffer_manager_t *bm, stasis_buffer_manager_handle_t *h, pageid_t start, pageid_t stop) {
stasis_buffer_concurrent_hash_t * ch = bm->impl;
ch->page_handle->force_range(ch->page_handle, start, stop);
@ -421,6 +422,7 @@ stasis_buffer_manager_t* stasis_buffer_manager_concurrent_hash_open(stasis_page_
bm->writeBackPage = chWriteBackPage;
bm->tryToWriteBackPage = chTryToWriteBackPage;
bm->forcePages = chForcePages;
bm->asyncForcePages = chAsyncForcePages;
bm->forcePageRange = chForcePageRange;
bm->stasis_buffer_manager_close = chBufDeinit;
bm->stasis_buffer_manager_simulate_crash = chSimulateBufferManagerCrash;

View file

@ -55,6 +55,7 @@ static void paReleasePage(stasis_buffer_manager_t *bm, Page * p) {
static int paWriteBackPage(stasis_buffer_manager_t *bm, pageid_t p) { return 0; /* no-op */ }
static void paForcePages(stasis_buffer_manager_t * bm, stasis_buffer_manager_handle_t *h) { /* no-op */ }
static void paAsyncForcePages(stasis_buffer_manager_t * bm, stasis_buffer_manager_handle_t *h) { /* no-op */ }
static void paForcePageRange(stasis_buffer_manager_t *bm, stasis_buffer_manager_handle_t *h, pageid_t start, pageid_t stop) { /* no-op */ }
static void paBufDeinit(stasis_buffer_manager_t * bm) {
@ -95,6 +96,7 @@ stasis_buffer_manager_t * stasis_buffer_manager_mem_array_open () {
bm->writeBackPage = paWriteBackPage;
bm->tryToWriteBackPage = paWriteBackPage;
bm->forcePages = paForcePages;
bm->asyncForcePages = paAsyncForcePages;
bm->forcePageRange = paForcePageRange;
bm->stasis_buffer_manager_close = paBufDeinit;
bm->stasis_buffer_manager_simulate_crash = paBufDeinit;

View file

@ -9,10 +9,10 @@
#include <stasis/util/redblack.h>
#include <stasis/util/multiset.h>
#include <stasis/util/latches.h>
#include <stasis/util/time.h>
#include <stasis/flags.h>
#include <stasis/dirtyPageTable.h>
#include <stasis/page.h>
#include <stdio.h>
typedef struct {
@ -47,6 +47,11 @@ struct stasis_dirty_page_table_t {
void stasis_dirty_page_table_set_dirty(stasis_dirty_page_table_t * dirtyPages, Page * p) {
if(!p->dirty) {
while(stasis_dirty_page_table_dirty_count(dirtyPages)
> stasis_dirty_page_count_hard_limit) {
struct timespec ts = stasis_double_to_timespec(0.01);
nanosleep(&ts,0);
}
pthread_mutex_lock(&dirtyPages->mutex);
if(!p->dirty) {
p->dirty = 1;
@ -134,8 +139,9 @@ pageid_t stasis_dirty_page_table_dirty_count(stasis_dirty_page_table_t * dirtyPa
return ATOMIC_READ_32(&dirtyPages->mutex, &dirtyPages->count);
}
int stasis_dirty_page_table_flush_with_target(stasis_dirty_page_table_t * dirtyPages, lsn_t targetLsn ) {
const int stride = 200;
int stasis_dirty_page_table_flush_with_target(stasis_dirty_page_table_t * dirtyPages, lsn_t targetLsn) {
DEBUG("stasis_dirty_page_table_flush_with_target called");
const long stride = stasis_dirty_page_table_flush_quantum;
int all_flushed;
pthread_mutex_lock(&dirtyPages->mutex);
if (targetLsn == LSN_T_MAX) {
@ -150,15 +156,28 @@ int stasis_dirty_page_table_flush_with_target(stasis_dirty_page_table_t * dirtyP
dirtyPages->flushing = 1;
}
// Normally, we will be called by a background thread that wants to maximize
// write back throughput, and sets targetLsn to LSN_T_MAX.
// Sometimes, we are called by log truncation, which wants to prioritize writeback
// of pages that are blocking log truncation.
// If we are writing back for the buffer manager, sort writebacks by page number.
// Otherwise, sort them by the LSN that first dirtied the page.
// TODO: Re-sort LSN ordered pages before passing them to the OS?
struct rbtree * tree = targetLsn == LSN_T_MAX ? dirtyPages->tableByPage
: dirtyPages->tableByLsnAndPage;
long buffered = 0;
do {
dpt_entry dummy = { 0, 0 };
pageid_t vals[stride];
int off = 0;
int strides = 0;
all_flushed = 1;
for(const dpt_entry * e = rblookup(RB_LUGTEQ, &dummy, dirtyPages->tableByLsnAndPage) ;
for(const dpt_entry * e = rblookup(RB_LUGTEQ, &dummy, tree);
e && e->lsn < targetLsn;
e = rblookup(RB_LUGREAT, &dummy, dirtyPages->tableByLsnAndPage)) {
e = rblookup(RB_LUGREAT, &dummy, tree)) {
dummy = *e;
vals[off] = dummy.p;
off++;
@ -166,7 +185,14 @@ int stasis_dirty_page_table_flush_with_target(stasis_dirty_page_table_t * dirtyP
pthread_mutex_unlock(&dirtyPages->mutex);
for(pageid_t i = 0; i < off; i++) {
if (dirtyPages->bufferManager->tryToWriteBackPage(dirtyPages->bufferManager, vals[i]) == EBUSY) {
all_flushed = 0;
all_flushed = 0;
} else {
buffered++;
}
if(buffered == stride) {
DEBUG("Forcing %lld pages A\n", buffered);
buffered = 0;
dirtyPages->bufferManager->asyncForcePages(dirtyPages->bufferManager, 0);
}
}
off = 0;
@ -175,14 +201,18 @@ int stasis_dirty_page_table_flush_with_target(stasis_dirty_page_table_t * dirtyP
}
}
pthread_mutex_unlock(&dirtyPages->mutex);
for(int i = 0; i < off; i++) {
for(pageid_t i = 0; i < off; i++) {
if (dirtyPages->bufferManager->tryToWriteBackPage(dirtyPages->bufferManager, vals[i]) == EBUSY) {
all_flushed = 0;
};
} else {
buffered++;
}
DEBUG("Forcing %lld pages B\n", buffered);
buffered = 0;
dirtyPages->bufferManager->asyncForcePages(dirtyPages->bufferManager, 0);
}
pthread_mutex_lock(&dirtyPages->mutex);
dpt_entry * e = ((dpt_entry*)rbmin(dirtyPages->tableByLsnAndPage));
dpt_entry * e = ((dpt_entry*)rbmin(tree));
if (!all_flushed &&
targetLsn < LSN_T_MAX &&
@ -210,7 +240,7 @@ int stasis_dirty_page_table_flush_with_target(stasis_dirty_page_table_t * dirtyP
ts.tv_sec = tv.tv_sec;
ts.tv_nsec = 1000*tv.tv_usec;
dpt_entry * e = ((dpt_entry*)rbmin(dirtyPages->tableByLsnAndPage));
dpt_entry * e = ((dpt_entry*)rbmin(tree));
if(targetLsn != LSN_T_MAX) { stasis_util_multiset_insert(dirtyPages->outstanding_flush_lsns, targetLsn); }
@ -219,7 +249,7 @@ int stasis_dirty_page_table_flush_with_target(stasis_dirty_page_table_t * dirtyP
all_flushed = 0;
break;
}
e = ((dpt_entry*)rbmin(dirtyPages->tableByLsnAndPage));
e = ((dpt_entry*)rbmin(tree));
}
if(targetLsn != LSN_T_MAX) {
@ -241,6 +271,7 @@ int stasis_dirty_page_table_flush_with_target(stasis_dirty_page_table_t * dirtyP
}
int stasis_dirty_page_table_flush(stasis_dirty_page_table_t * dirtyPages) {
DEBUG("stasis_dirty_page_table_flush called");
return stasis_dirty_page_table_flush_with_target(dirtyPages, LSN_T_MAX);
}

View file

@ -50,10 +50,36 @@ pageid_t stasis_buffer_manager_size = STASIS_BUFFER_MANAGER_SIZE;
#ifdef MAX_BUFFER_SIZE
pageid_t stasis_buffer_manager_size = MAX_BUFFER_SIZE;
#else // MAX_BUFFER_SIZE
pageid_t stasis_buffer_manager_size = 8310784 / PAGE_SIZE; // ~ 82MB
pageid_t stasis_buffer_manager_size = 83107840 / PAGE_SIZE; // ~ 82MB
#endif // MAX_BUFFER_SIZE
#endif // STASIS_BUFFER_MANAGER_SIZE
pageid_t stasis_dirty_page_count_soft_limit=
#ifdef STASIS_DIRTY_PAGE_count_SOFT_LIMIT
STASIS_DIRTY_PAGE_COUNT_SOFT_LIMIT;
#else
16 * 1024 * 1024 / PAGE_SIZE;
#endif
pageid_t stasis_dirty_page_low_water_mark =
#ifdef STASIS_DIRTY_PAGE_LOW_WATER_MARK
STASIS_DIRTY_PAGE_LOW_WATER_MARK;
#else
(8 * 1024 * 1024) / PAGE_SIZE;
#endif
pageid_t stasis_dirty_page_count_hard_limit =
#ifdef STASIS_DIRTY_PAGE_COUNT_HARD_LIMIT
STASIS_DIRTY_PAGE_COUNT_HARD_LIMIT;
#else
(48 * 1024 * 1024) / PAGE_SIZE;
#endif
pageid_t stasis_dirty_page_table_flush_quantum =
#ifdef STASIS_DIRTY_PAGE_TABLE_FLUSH_QUANTUM
STASIS_DIRTY_PAGE_TABLE_FLUSH_QUANTUM;
#else
(16 * 1024 * 1024) / PAGE_SIZE;
#endif
stasis_page_handle_t* (*stasis_page_handle_factory)(stasis_log_t*, stasis_dirty_page_table_t*) =
#ifdef STASIS_PAGE_HANDLE_FACTORY
STASIS_PAGE_HANDLE_FACTORY;

View file

@ -364,6 +364,48 @@ static int file_force(stasis_handle_t * h) {
}
return 0;
}
static int file_async_force(stasis_handle_t *h) {
file_impl * impl = h->impl;
int ret = 0;
if(!(impl->file_flags & O_SYNC)) {
// not opened synchronously; we need to explicitly sync.
pthread_mutex_lock(&impl->mut);
int fd = impl->fd;
pthread_mutex_unlock(&impl->mut);
{
static int warned = 0;
if(!warned) {
printf("Warning: There is a race condition between force_range() and "
" truncate() in file.c (This shouldn't matter in practice, "
"as the logger hasn't moved over to use file.c yet.\n");
warned = 1;
}
}
//#ifdef HAVE_F_SYNC_RANGE
#ifdef HAVE_SYNC_FILE_RANGE
DEBUG("Calling sync_file_range\n");
ret = sync_file_range(fd, 0, 0, SYNC_FILE_RANGE_WRITE);
if(ret) {
int error = errno;
assert(ret == -1);
// With the possible exceptions of ENOMEM and ENOSPACE, all of the sync
// errors are unrecoverable.
h->error = EBADF;
ret = error;
}
#else
#ifdef HAVE_FDATASYNC
DEBUG("file_force_range() is calling fdatasync()\n");
fdatasync(fd);
#else
DEBUG("file_force_range() is calling fsync()\n");
fsync(fd);
#endif
ret = 0;
#endif
}
return ret;
}
static int file_force_range(stasis_handle_t *h, lsn_t start, lsn_t stop) {
file_impl * impl = h->impl;
int ret = 0;
@ -375,10 +417,10 @@ static int file_force_range(stasis_handle_t *h, lsn_t start, lsn_t stop) {
{
static int warned = 0;
if(!warned) {
printf("Warning: There is a race condition between force_range() and "
" truncate() in file.c (This shouldn't matter in practice, "
"as the logger hasn't moved over to use file.c yet.\n");
warned = 1;
printf("Warning: There is a race condition between force_range() and "
" truncate() in file.c (This shouldn't matter in practice, "
"as the logger hasn't moved over to use file.c yet.\n");
warned = 1;
}
}
//#ifdef HAVE_F_SYNC_RANGE
@ -435,6 +477,7 @@ struct stasis_handle_t file_func = {
.read_buffer = file_read_buffer,
.release_read_buffer = file_release_read_buffer,
.force = file_force,
.async_force = file_async_force,
.force_range = file_force_range,
.fallocate = file_fallocate,
.error = 0

View file

@ -306,6 +306,41 @@ static int pfile_force(stasis_handle_t *h) {
TOCK(force_hist);
return 0;
}
static int pfile_async_force(stasis_handle_t *h) {
TICK(force_range_hist);
pfile_impl * impl = h->impl;
#ifdef HAVE_SYNC_FILE_RANGE
// stop of zero syncs to eof.
DEBUG("pfile_force_range calling sync_file_range %lld %lld\n",
start, stop-start); fflush(stdout);
int ret = sync_file_range(impl->fd, 0, 0, SYNC_FILE_RANGE_WAIT_BEFORE);
ret |= sync_file_range(impl->fd, 0, 0, SYNC_FILE_RANGE_WRITE);
if(ret) {
int error = errno;
assert(ret == -1);
// With the possible exceptions of ENOMEM and ENOSPACE, all of the sync
// errors are unrecoverable.
h->error = EBADF;
ret = error;
}
#else
#ifdef HAVE_FDATASYNC
DEBUG("pfile_force_range() is calling fdatasync()\n");
fdatasync(impl->fd);
#else
DEBUG("pfile_force_range() is calling fsync()\n");
fsync(impl->fd);
#endif
int ret = 0;
#endif
if(impl->sequential) {
int err = posix_fadvise(impl->fd, 0, 0, POSIX_FADV_DONTNEED);
if(err) perror("Attempt to pass POSIX_FADV_SEQUENTIAL (for a range of a file) to kernel failed");
}
TOCK(force_range_hist);
return ret;
}
static int pfile_force_range(stasis_handle_t *h, lsn_t start, lsn_t stop) {
TICK(force_range_hist);
pfile_impl * impl = h->impl;
@ -366,6 +401,7 @@ static struct stasis_handle_t pfile_func = {
.read_buffer = pfile_read_buffer,
.release_read_buffer = pfile_release_read_buffer,
.force = pfile_force,
.async_force = pfile_async_force,
.force_range = pfile_force_range,
.fallocate = pfile_fallocate,
.error = 0

View file

@ -62,6 +62,10 @@ static void phForce(stasis_page_handle_t * ph) {
int err = ((stasis_handle_t*)ph->impl)->force(ph->impl);
assert(!err);
}
static void phAsyncForce(stasis_page_handle_t * ph) {
int err = ((stasis_handle_t*)ph->impl)->async_force(ph->impl);
assert(!err);
}
static void phForceRange(stasis_page_handle_t * ph, lsn_t start, lsn_t stop) {
int err = ((stasis_handle_t*)ph->impl)->force_range(ph->impl,start*PAGE_SIZE,stop*PAGE_SIZE);
assert(!err);
@ -99,6 +103,7 @@ stasis_page_handle_t * stasis_page_handle_open(stasis_handle_t * handle,
ret->prefetch_range = phPrefetchRange;
ret->preallocate_range = phPreallocateRange;
ret->force_file = phForce;
ret->async_force_file = phAsyncForce;
ret->force_range = phForceRange;
ret->close = phClose;
ret->dup = phDup;

View file

@ -162,6 +162,20 @@ struct stasis_buffer_manager_t {
a no-op.
*/
void (*forcePages)(struct stasis_buffer_manager_t*, stasis_buffer_manager_handle_t *h);
/**
Asynchronously force pages to disk.
More concretely, this call blocks until the last call to asyncForcePages
has finished writing blocks to disk, schedules pages for writeback, and
(usually) immediately returns.
For various reasons, this is not useful for data integrity, but is
instead useful as a performance hint.
This function is currently implemented using sync_file_range(2). See its
manpage for a discussion of the limitations of this primitive.
*/
void (*asyncForcePages)(struct stasis_buffer_manager_t*, stasis_buffer_manager_handle_t *h);
/**
Force written back pages that fall within a particular range to disk.

View file

@ -26,6 +26,31 @@ extern stasis_log_t* (*stasis_log_factory)(void);
extern stasis_buffer_manager_t* (*stasis_buffer_manager_factory)(stasis_log_t*, stasis_dirty_page_table_t*);
extern pageid_t stasis_buffer_manager_size;
/**
* The number of pages that must be dirty for the writeback thread to
* initiate writeback.
*/
extern pageid_t stasis_dirty_page_count_soft_limit;
/**
* If there are fewer than this many dirty pages, then the writeback thread
* will go to sleep.
*/
extern pageid_t stasis_dirty_page_low_water_mark;
/**
* The number of pages that must be dirty for application threads to block on
* (or initiate) writeback. Since this number causes backpressure on the
* threads that are dirtying pages, dirty pages will never occupy more than
* this number of pages.
*/
extern pageid_t stasis_dirty_page_count_hard_limit;
/**
* The number of pages that should be written back to Linux's file cache
* at a time. We do not force after each quantum, but instead may hint to
* Linux that it should treat the set as a group. Also, any latchesh held
* by writeback are released at least this often.
*/
extern pageid_t stasis_dirty_page_table_flush_quantum;
/**
If this is true, then the only thread that will perform writeback is the
buffer manager writeback thread. It turns out that splitting sequential

View file

@ -183,6 +183,18 @@ typedef struct stasis_handle_t {
returned) may or may not be forced to disk.
*/
int (*force)(struct stasis_handle_t * h);
/**
* Force any writes that were outstanding the last time this was called to
* disk. This does not force things like block allocation information, or
* issue a hardware write barrier, making it essentially useless for
* durability. However, it is useful as a performance hint, as it allows us
* to bound the number of data pages that are outstanding in Linux's I/O
* request queue, which means that we can use it to bound the latency of
* future force() and force_range() operations.
*
* This call is based on sync_file_range(2).
*/
int (*async_force)(struct stasis_handle_t * h);
int (*force_range)(struct stasis_handle_t * h, lsn_t start, lsn_t stop);
int (*fallocate)(struct stasis_handle_t * h, lsn_t off, lsn_t len);
/**

View file

@ -65,6 +65,7 @@ struct stasis_page_handle_t {
as well...)
*/
void (*force_file)(struct stasis_page_handle_t* ph);
void (*async_force_file)(struct stasis_page_handle_t* ph);
/**
* @param start the pageid of the first page to be forced to disk.
* @param stop the pageid of the page after the last page to be forced to disk.