Commit f49c11a3 authored by Kirill Smelkov's avatar Kirill Smelkov

bigfile/virtmem: Do loadblk() with virtmem lock released

loadblk() calls are potentially slow and external code that serve the cal can
take other locks in addition to virtmem lock taken by virtmem subsystem. If
that "other locks" are also taken before external code calls e.g.
fileh_invalidate_page() in different codepath a deadlock can happen, e.g.

      T1                  T2

      page-access         invalidation-from-server received
      V -> loadblk
                          Z   <- ClientStorage.invalidateTransaction()
      Z -> zeo.load
                          V   <- fileh_invalidate_page

The solution to avoid deadlock is to call loadblk() with virtmem lock released
and upon loadblk() completion recheck virtmem data structures carefully.

To make that happen:

- new page state is introduces:

    PAGE_LOADING                (file content loading is  in progress)

- virtmem releases virt_lock before calling loadblk() when serving pagefault

- because loading is now done with virtmem lock released, now:

1. After loading completes we need to recheck fileh/vma data structures

   The recheck is done in full - vma_on_pagefault() just asks its driver (see
   VM_RETRY and VM_HANDLED codes) to retry handling the fault completely. This
   should work as the freshly loaded page was just inserted into fileh->pagemap
   and should be found there in the cache on next lookup.

   On the other hand this also works correctly, if there was concurrent change
   - e.g. vma was unmapped while we were loading the data - in that case the
   fault will be also processed correctly - but loaded data will stay in
   fileh->pagemap (and if not used will be evicted as not-needed
   eventually by RAM reclaim).

2. Similar to retrying mechanism is used for cases when two threads
   concurrently access the same page and would both try to load corresponding
   block - only one thread issues the actual loadblk() and another waits for load
   to complete with polling and VM_RETRY.

3. To correctly invalidate loading-in-progress pages another new page state
   is introduced:

    PAGE_LOADING_INVALIDATED    (file content loading was in progress
                                 while request to invalidate the page came in)

   which fileh_invalidate_page() uses to propagate invalidation message to
   loadblk() caller.

4. Blocks loading can now happen in parallel with other block loading and
   other virtmem operations - e.g. invalidation. For such cases tests are added
   to test_thread.py

5. virtmem lock now becomes just regular lock, instead of being previously
   recursive.

   For virtmem lock to be recursive was needed for cases, when code under
   loadblk() could trigger other virtmem calls, e.g. due to GC and calling
   another VMA dtor that would want to lock virtmem, but virtmem lock was
   already held.

   This is no longer needed.

6. To catch double faults we now cannot use just on static variable
   in_on_pagefault. That variable thus becomes thread-local.

7. Old test in test_thread to "test that access vs access don't overlap" no
   longer holds true - and is thus removed.

/cc @Tyagov, @klaus
parent 0231a65d
......@@ -82,25 +82,40 @@ static void on_pagefault(int sig, siginfo_t *si, void *_uc)
* coredump (see comments wrt SA_NODEFER in pagefault_init()), but anyway -
* better check just in case.
*
* NOTE since we are under virtmem lock, here we can use just one static
* variable, instead of several per-thread ones. */
static int in_on_pagefault;
* NOTE it is ok to use __thread in synchronous sighandler - even if TLS
* block is allocated dynamically at runtime, we can overlap with such
* allocation only if SIGSEGV happens in that original TLS allocation,
* which should not happen, and thus it is already a bug somewhere in
* thread datatructures. */
static __thread int in_on_pagefault;
BUG_ON(in_on_pagefault);
++in_on_pagefault;
/* (1) addr -> vma ;lookup VMA covering faulting memory address */
vma = virt_lookup_vma(si->si_addr);
if (!vma) {
--in_on_pagefault;
virt_unlock();
goto dont_handle; /* fault outside registered file slices */
/* vma_on_pagefault() can tell us to retry handling the fault, e.g. after a
* page has been loaded. Loop until pagefault is handled */
while (1) {
VMFaultResult vmres;
/* (1) addr -> vma ;lookup VMA covering faulting memory address */
vma = virt_lookup_vma(si->si_addr);
if (!vma) {
--in_on_pagefault;
virt_unlock();
goto dont_handle; /* fault outside registered file slices */
}
/* now, since we found faulting address in registered memory areas, we know
* we should serve this pagefault. */
vmres = vma_on_pagefault(vma, (uintptr_t)si->si_addr, write);
/* see if pagefault handled or should be retried */
if (vmres == VM_HANDLED)
break;
if (vmres == VM_RETRY)
continue;
BUG(); /* unreachable */
}
/* now, since we found faulting address in registered memory areas, we know
* we should serve this pagefault. */
vma_on_pagefault(vma, (uintptr_t)si->si_addr, write);
/* pagefault served - restore and return from sighandler */
--in_on_pagefault;
virt_unlock();
......
......@@ -16,10 +16,10 @@
#
# See COPYING file for full licensing terms.
from wendelin.bigfile import BigFile
from threading import Thread
from threading import Thread, Lock
from time import sleep
from wendelin.bigfile.tests.test_basic import bchr_py2
from wendelin.bigfile.tests.test_basic import bord_py3
from six.moves import _thread
# Notify channel for
......@@ -50,50 +50,187 @@ class NotifyChannel:
# Synthetic bigfile that verifies there is no concurrent calls to loadblk
class XBigFile(BigFile):
# XXX hack, hardcoded
MB = 1024*1024
PS = 2*MB
def __new__(cls, blksize):
obj = BigFile.__new__(cls, blksize)
obj.loadblk_counter = 0
return obj
def loadblk(self, blk, buf):
assert self.loadblk_counter == 0
# Test that it is possible to take a lock both:
# - from-under virtmem, and
# - on top of virtmem
# and not deadlock.
#
# ( this happens e.g. with ZEO:
#
# T1 T2
#
# page-access invalidation-from-server received
# V -> loadblk
# Z <- ClientStorage.invalidateTransaction()
# Z -> zeo.load
# V <- fileh_invalidate_page
def test_thread_lock_vs_virtmem_lock():
Z = Lock()
c12 = NotifyChannel() # T1 -> T2
c21 = NotifyChannel() # T2 -> T1
class ZLockBigFile(BigFile):
def __new__(cls, blksize):
obj = BigFile.__new__(cls, blksize)
obj.cycle = 0
return obj
def loadblk(self, blk, buf):
tell, wait = c12.tell, c21.wait
# on the first cycle we synchronize with invalidate in T2
if self.cycle == 0:
tell('T1-V-under')
wait('T2-Z-taken')
# this will deadlock, if V is plain lock and calling from under-virtmem
# is done with V held
Z.acquire()
Z.release()
self.cycle += 1
f = ZLockBigFile(PS)
fh = f.fileh_open()
vma = fh.mmap(0, 1)
m = memoryview(vma)
# sleep with increased conter - to give chance for other threads to try
# to load the same block and catch that
self.loadblk_counter += 1
sleep(3)
def T1():
m[0] # calls ZLockBigFile.loadblk()
# nothing to do - we just leave blk as is (all zeros)
self.loadblk_counter -= 1
assert self.loadblk_counter == 0
def T2():
tell, wait = c21.tell, c12.wait
# XXX hack, hardcoded
MB = 1024*1024
PS = 2*MB
wait('T1-V-under')
Z.acquire()
tell('T2-Z-taken')
fh.invalidate_page(0)
Z.release()
t1, t2 = Thread(target=T1), Thread(target=T2)
t1.start(); t2.start()
t1.join(); t2.join()
# multiple access from several threads to the same page - block loaded only once
def test_thread_multiaccess_sameblk():
d = {} # blk -> #loadblk(blk)
class CountBigFile(BigFile):
def loadblk(self, blk, buf):
d[blk] = d.get(blk, 0) + 1
# make sure other threads has time and high probability to overlap
# loadblk/loadblk
sleep(1)
f = CountBigFile(PS)
fh = f.fileh_open()
vma = fh.mmap(0, 1)
m = memoryview(vma)
def test_thread_basic():
f = XBigFile(PS)
def T():
m[0] # calls CountBigFile.loadblk()
t1, t2 = Thread(target=T), Thread(target=T)
t1.start(); t2.start()
t1.join(); t2.join()
assert d[0] == 1
# multiple access from several threads to different pages - blocks loaded in parallel
def test_thread_multiaccess_parallel():
# tid -> (T0 -> T<tid>, T<tid> -> T0)
channels = {}
class SyncBigFile(BigFile):
def loadblk(self, blk, buf):
# tell driver we are in loadblk and wait untill it says us to go
cin, cout = channels[_thread.get_ident()]
cout.tell('ready')
cin.wait('go')
f = SyncBigFile(PS)
fh = f.fileh_open()
vma = fh.mmap(0, 4)
vma = fh.mmap(0, 2)
m = memoryview(vma)
# simple test that access vs access don't overlap
# ( look for assert & sleep in XBigFile.loadblk() )
Q = []
def access0():
Q.append(m[0])
t1 = Thread(target=access0)
t2 = Thread(target=access0)
def T1():
channels[_thread.get_ident()] = (NotifyChannel(), NotifyChannel())
m[0*PS]
def T2():
channels[_thread.get_ident()] = (NotifyChannel(), NotifyChannel())
m[1*PS]
t1, t2 = Thread(target=T1), Thread(target=T2)
t1.start(); t2.start()
while len(channels) != 2:
pass
c01, c10 = channels[t1.ident]
c02, c20 = channels[t2.ident]
c10.wait('ready'); c20.wait('ready')
c01.tell('go'); c02.tell('go')
t1.join(); t2.join()
# loading vs invalidate in another thread
def test_thread_load_vs_invalidate():
c12 = NotifyChannel() # T1 -> T2
c21 = NotifyChannel() # T2 -> T1
class RetryBigFile(BigFile):
def __new__(cls, blksize):
obj = BigFile.__new__(cls, blksize)
obj.cycle = 0
return obj
def loadblk(self, blk, buf):
tell, wait = c12.tell, c21.wait
bufmem = memoryview(buf)
# on the first cycle we synchronize with invalidate in T2
if self.cycle == 0:
tell('T1-loadblk0-ready')
wait('T1-loadblk0-go')
# here we know request to invalidate this page came in and this
# '1' should be ignored by virtmem
bufmem[0] = bord_py3(b'1')
# this is code for consequent "after-invalidate" loadblk
# '2' should be returned to clients
else:
bufmem[0] = bord_py3(b'2')
self.cycle += 1
f = RetryBigFile(PS)
fh = f.fileh_open()
vma = fh.mmap(0, 1)
m = memoryview(vma)
def T1():
assert m[0] == bord_py3(b'2')
t1.start()
t2.start()
def T2():
tell, wait = c21.tell, c12.wait
t1.join()
t2.join()
wait('T1-loadblk0-ready')
fh.invalidate_page(0)
tell('T1-loadblk0-go')
assert Q == [bchr_py2(0), bchr_py2(0)]
t1, t2 = Thread(target=T1), Thread(target=T2)
t1.start(); t2.start()
t1.join(); t2.join()
......@@ -221,6 +221,25 @@ int M(VMA *vma, pgoff_t idx) { return bitmap_test_bit(vma->page_ismappedv, idx)
} while (0)
/* vma_on_pagefault() assumes virtmem_lock is taken by caller and can ask it to
* retry. Handle fault to the end, like on_pagefault() does. */
void xvma_on_pagefault(VMA *vma, uintptr_t addr, int write) {
virt_lock();
while (1) {
VMFaultResult vmres;
vmres = vma_on_pagefault(vma, addr, write);
if (vmres == VM_HANDLED)
break;
if (vmres == VM_RETRY)
continue;
fail("Unexpected return code from vma_on_pagefault: %i", vmres);
}
virt_unlock();
}
/* test access to file mappings via explicit vma_on_pagefault() calls */
void test_file_access_synthetic(void)
{
......@@ -317,7 +336,7 @@ void test_file_access_synthetic(void)
/* simulate read access to page[0] - it should load it */
diag("read page[0]");
vma_on_pagefault(vma, vma->addr_start + 0*PS, 0);
xvma_on_pagefault(vma, vma->addr_start + 0*PS, 0);
ok1( M(vma, 0)); B(vma, 0*PSb); MUST_FAULT( B(vma, 0*PSb) = 10 );
ok1(!M(vma, 1)); MUST_FAULT( B(vma, 1*PSb) ); MUST_FAULT( B(vma, 1*PSb) = 11 );
......@@ -341,7 +360,7 @@ void test_file_access_synthetic(void)
/* simulate write access to page[2] - it should load it and mark page dirty */
diag("write page[2]");
vma_on_pagefault(vma, vma->addr_start + 2*PS, 1);
xvma_on_pagefault(vma, vma->addr_start + 2*PS, 1);
ok1( M(vma, 0)); B(vma, 0*PSb); MUST_FAULT( B(vma, 0*PSb) = 10 );
ok1(!M(vma, 1)); MUST_FAULT( B(vma, 1*PSb) ); MUST_FAULT( B(vma, 1*PSb) = 11 );
......@@ -369,7 +388,7 @@ void test_file_access_synthetic(void)
/* read access to page[3] - load */
diag("read page[3]");
vma_on_pagefault(vma, vma->addr_start + 3*PS, 0);
xvma_on_pagefault(vma, vma->addr_start + 3*PS, 0);
ok1( M(vma, 0)); B(vma, 0*PSb); MUST_FAULT( B(vma, 0*PSb) = 10 );
ok1(!M(vma, 1)); MUST_FAULT( B(vma, 1*PSb) ); MUST_FAULT( B(vma, 1*PSb) = 11 );
......@@ -401,7 +420,7 @@ void test_file_access_synthetic(void)
/* write access to page[0] - upgrade loaded -> dirty */
diag("write page[0]");
vma_on_pagefault(vma, vma->addr_start + 0*PS, 1);
xvma_on_pagefault(vma, vma->addr_start + 0*PS, 1);
ok1( M(vma, 0)); B(vma, 0*PSb); B(vma, 0*PSb) = 10;
ok1(!M(vma, 1)); MUST_FAULT( B(vma, 1*PSb) ); MUST_FAULT( B(vma, 1*PSb) = 11 );
......@@ -436,7 +455,7 @@ void test_file_access_synthetic(void)
* RAMLimited with 3 allocated pages max). Evicted will be page[3] - as it
* is the only PAGE_LOADED page. */
diag("read page[1]");
vma_on_pagefault(vma, vma->addr_start + 1*PS, 0);
xvma_on_pagefault(vma, vma->addr_start + 1*PS, 0);
ok1( M(vma, 0)); B(vma, 0*PSb); B(vma, 0*PSb) = 10;
ok1( M(vma, 1)); B(vma, 1*PSb); MUST_FAULT( B(vma, 1*PSb) = 11 );
......@@ -557,7 +576,7 @@ void test_file_access_synthetic(void)
/* read access to page[2] - should map it R/W - the page is in PAGE_DIRTY state */
diag("read page[2]");
vma_on_pagefault(vma, vma->addr_start + 2*PS, 0);
xvma_on_pagefault(vma, vma->addr_start + 2*PS, 0);
ok1(!M(vma, 0)); MUST_FAULT( B(vma, 0*PSb) ); MUST_FAULT( B(vma, 0*PSb) = 10 );
ok1(!M(vma, 1)); MUST_FAULT( B(vma, 1*PSb) ); MUST_FAULT( B(vma, 1*PSb) = 11 );
......@@ -612,7 +631,7 @@ void test_file_access_synthetic(void)
/* read page[3] (so that we have 1 PAGE_LOADED besides PAGE_DIRTY pages) */
ok1(!pagemap_get(&fh->pagemap, 103));
vma_on_pagefault(vma, vma->addr_start + 3*PS, 0);
xvma_on_pagefault(vma, vma->addr_start + 3*PS, 0);
page3 = pagemap_get(&fh->pagemap, 103);
ok1(page3);
......@@ -635,12 +654,12 @@ void test_file_access_synthetic(void)
/* prepare state (2 dirty pages, only 1 mapped) */
void mkdirty2() {
vma_on_pagefault(vma, vma->addr_start + 0*PS, 1); /* write page[0] */
vma_on_pagefault(vma, vma->addr_start + 2*PS, 1); /* write page[2] */
xvma_on_pagefault(vma, vma->addr_start + 0*PS, 1); /* write page[0] */
xvma_on_pagefault(vma, vma->addr_start + 2*PS, 1); /* write page[2] */
vma_unmap(vma);
err = fileh_mmap(vma, fh, 100, 4);
ok1(!err);
vma_on_pagefault(vma, vma->addr_start + 2*PS, 0);
xvma_on_pagefault(vma, vma->addr_start + 2*PS, 0);
ok1(!M(vma, 0)); MUST_FAULT( B(vma, 0*PSb) ); MUST_FAULT( B(vma, 0*PSb) = 10 );
ok1(!M(vma, 1)); MUST_FAULT( B(vma, 1*PSb) ); MUST_FAULT( B(vma, 1*PSb) = 11 );
......@@ -808,7 +827,7 @@ void test_file_access_synthetic(void)
ok1(page3->lru.prev == &ram->lru_list);
/* read page[3] back */
vma_on_pagefault(vma, vma->addr_start + 3*PS, 0);
xvma_on_pagefault(vma, vma->addr_start + 3*PS, 0);
ok1(!M(vma, 0)); MUST_FAULT( B(vma, 0*PSb) ); MUST_FAULT( B(vma, 0*PSb) = 10 );
ok1(!M(vma, 1)); MUST_FAULT( B(vma, 1*PSb) ); MUST_FAULT( B(vma, 1*PSb) = 11 );
......
......@@ -54,11 +54,8 @@ static int __ram_reclaim(RAM *ram);
/* global lock which protects manipulating virtmem data structures
*
* NOTE not scalable, but this is temporary solution - as we are going to move
* memory managment back into the kernel, where it is done properly.
*
* NOTE type is recursive to support deleting virtmem object via python
* finalizers (triggered either from decref or from gc - both from sighandler). */
static pthread_mutex_t virtmem_lock = PTHREAD_RECURSIVE_MUTEX_INITIALIZER_NP;
* memory managment back into the kernel, where it is done properly. */
static pthread_mutex_t virtmem_lock = PTHREAD_ERRORCHECK_MUTEX_INITIALIZER_NP;
static const VirtGilHooks *virtmem_gilhooks;
void *virt_gil_ensure_unlocked(void)
......@@ -175,6 +172,9 @@ void fileh_close(BigFileH *fileh)
/* drop all pages (dirty or not) associated with this fileh */
pagemap_for_each(page, &fileh->pagemap) {
/* it's an error to close fileh to mapping of which an access is
* currently being done in another thread */
BUG_ON(page->state == PAGE_LOADING);
page_drop_memory(page);
list_del(&page->lru);
bzero(page, sizeof(*page)); /* just in case */
......@@ -417,8 +417,20 @@ void fileh_invalidate_page(BigFileH *fileh, pgoff_t pgoffset)
virt_lock();
page = pagemap_get(&fileh->pagemap, pgoffset);
if (page)
page_drop_memory(page);
if (page) {
/* for pages where loading is in progress, we just remove the page from
* pagemap and mark it to be dropped by their loaders after it is done.
* In the mean time, as pagemap entry is now empty, on next access to
* the memory the page will be created/loaded anew */
if (page->state == PAGE_LOADING) {
pagemap_del(&fileh->pagemap, pgoffset);
page->state = PAGE_LOADING_INVALIDATED;
}
/* else we just make sure to drop page memory */
else {
page_drop_memory(page);
}
}
virt_unlock();
sigsegv_restore(&save_sigset);
......@@ -535,7 +547,7 @@ void *mem_xvalloc(void *addr, size_t len)
/* pagefault entry when we know request came to our memory area
*
* (virtmem_lock already taken by caller) */
void vma_on_pagefault(VMA *vma, uintptr_t addr, int write)
VMFaultResult vma_on_pagefault(VMA *vma, uintptr_t addr, int write)
{
pgoff_t pagen;
Page *page;
......@@ -571,8 +583,8 @@ void vma_on_pagefault(VMA *vma, uintptr_t addr, int write)
pagemap_set(&fileh->pagemap, pagen, page);
}
/* (5) if page was not yet loaded - load it */
if (page->state < PAGE_LOADED) {
/* (5a) if page was not yet loaded - start loading it */
if (page->state == PAGE_EMPTY) {
/* NOTE if we load data in-place, there would be a race with concurrent
* access to the page here - after first enabling memory-access to
* the page, other threads could end up reading corrupt data, while
......@@ -589,13 +601,15 @@ void vma_on_pagefault(VMA *vma, uintptr_t addr, int write)
blk_t blk;
void *pageram;
int err;
BigFile *file;
/*
* if pagesize < blksize - need to prepare several adjacent pages for blk;
* if pagesize > blksize - will need to either 1) rescan which blk got
* dirty, or 2) store not-even-touched blocks adjacent to modified one.
*/
TODO (fileh->file->blksize != page_size(page));
file = fileh->file;
TODO (file->blksize != page_size(page));
// FIXME doing this mmap-to-temp/unmap is somewhat costly. Better
// constantly have whole RAM mapping somewhere R/W and load there.
......@@ -618,7 +632,18 @@ void vma_on_pagefault(VMA *vma, uintptr_t addr, int write)
/* loadblk() -> pageram memory */
blk = page->f_pgoffset; // NOTE because blksize = pagesize
err = fileh->file->file_ops->loadblk(fileh->file, blk, pageram);
/* mark page as loading and unlock virtmem before calling loadblk()
*
* that call is potentially slow and external code can take other
* locks. If that "other locks" are also taken before external code
* calls e.g. fileh_invalidate_page() in different codepath a deadlock
* can happen. */
page->state = PAGE_LOADING;
virt_unlock();
err = file->file_ops->loadblk(file, blk, pageram);
/* TODO on error -> try to throw exception somehow to the caller, so
* that it can abort current transaction, but not die.
*
......@@ -627,9 +652,51 @@ void vma_on_pagefault(VMA *vma, uintptr_t addr, int write)
*/
TODO (err);
/* relock virtmem */
virt_lock();
xmunmap(pageram, page_size(page));
page->state = PAGE_LOADED;
/* if the page was invalidated while we were loading it, we have to drop
* it's memory and the Page structure completely - invalidater already
* removed it from pagemap */
if (page->state == PAGE_LOADING_INVALIDATED) {
page_drop_memory(page);
list_del(&page->lru);
bzero(page, sizeof(*page)); /* just in case */
free(page);
}
/* else just mark the page as loaded ok */
else
page->state = PAGE_LOADED;
/* we have to retry the whole fault, because the vma could have been
* changed while we were loading page with virtmem lock released */
return VM_RETRY;
}
/* (5b) page is currently being loaded by another thread - wait for load to complete
*
* NOTE a page is protected from being concurently loaded by two threads at
* the same time via:
*
* - virtmem lock - we get/put pages from fileh->pagemap only under it
* - page->state is set PAGE_LOADING for loading in progress pages
* - such page is inserted in fileh->pagepam
*
* so if second thread faults at the same memory page, and the page is
* still loading, it will find the page in PAGE_LOADING state and will just
* wait for it to complete. */
if (page->state == PAGE_LOADING) {
/* XXX polling instead of proper completion */
void *gilstate;
virt_unlock();
gilstate = virt_gil_ensure_unlocked();
usleep(10000); // XXX with 1000 uslepp still busywaits
virt_gil_retake_if_waslocked(gilstate);
virt_lock();
return VM_RETRY;
}
/* (6) page data ready. Mmap it atomically into vma address space, or mprotect
......@@ -666,7 +733,7 @@ void vma_on_pagefault(VMA *vma, uintptr_t addr, int write)
* (7) access to page prepared - now it is ok to return from signal handler
* - the caller will re-try executing faulting instruction.
*/
return;
return VM_HANDLED;
}
......@@ -690,7 +757,8 @@ static int __ram_reclaim(RAM *ram)
hlru = hlru->next;
scanned++;
/* can release ram only from loaded non-dirty pages */
/* can release ram only from loaded non-dirty pages
* NOTE PAGE_LOADING pages are not dropped - they just continue to load */
if (page->state == PAGE_LOADED) {
page_drop_memory(page);
batch--;
......@@ -766,6 +834,10 @@ static void page_drop_memory(Page *page)
/* Memory for this page goes out. 1) unmap it from all mmaps */
struct list_head *hmmap;
/* NOTE we try not to drop memory for loading-in-progress pages.
* so if this is called for such a page - it is a bug. */
BUG_ON(page->state == PAGE_LOADING);
if (page->state == PAGE_EMPTY)
return;
......
......@@ -76,8 +76,12 @@ typedef struct BigFileH BigFileH;
/* Page - describes fixed-size item of physical RAM associated with content from fileh */
enum PageState {
PAGE_EMPTY = 0, /* file content has not been loaded yet */
PAGE_LOADED = 1, /* file content has been loaded and was not modified */
PAGE_DIRTY = 2, /* file content has been loaded and was modified */
PAGE_LOADING = 1, /* file content loading is in progress */
PAGE_LOADING_INVALIDATED
= 2, /* file content loading was in progress
while request to invalidate the page came in */
PAGE_LOADED = 3, /* file content has been loaded and was not modified */
PAGE_DIRTY = 4, /* file content has been loaded and was modified */
};
typedef enum PageState PageState;
......@@ -220,7 +224,7 @@ void fileh_dirty_discard(BigFileH *fileh);
*
* Make sure that page corresponding to pgoffset is not present in fileh memory.
*
* The page could be in either dirty or loaded or empty state. In all
* The page could be in either dirty or loaded/loading or empty state. In all
* cases page transitions to empty state and its memory is forgotten.
*
* ( Such invalidation is needed to synchronize fileh memory, when we know a
......@@ -236,7 +240,12 @@ void fileh_invalidate_page(BigFileH *fileh, pgoff_t pgoffset);
*
* (clients call this indirectly via triggering SIGSEGV on read/write to memory)
*/
void vma_on_pagefault(VMA *vma, uintptr_t addr, int write);
enum VMFaultResult {
VM_HANDLED = 0, /* pagefault handled */
VM_RETRY = 1, /* pagefault handled partly - handling have to be retried */
};
typedef enum VMFaultResult VMFaultResult;
VMFaultResult vma_on_pagefault(VMA *vma, uintptr_t addr, int write);
int pagefault_init(void); /* in pagefault.c */
......
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