Commit fb4bfb32 authored by Kirill Smelkov's avatar Kirill Smelkov

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

Like with loadblk (see f49c11a3 "bigfile/virtmem: Do loadblk() with
virtmem lock released" for the reference) storeblk() calls are
potentially slow and external code that serves the call 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.
with fileh_invalidate_page() in different codepath - a deadlock can happen:

      T1                  T2

      commit              invalidation-from-server received
      V -> storeblk
                          Z   <- ClientStorage.invalidateTransaction()
      Z -> zeo.store
                          V   <- fileh_invalidate_page (of unrelated page)

The solution to avoid deadlock, like for loadblk case, is to call storeblk()
with virtmem lock released.

However unlike loadblk which can be invoked at any time, storeblk is
invoked at commit time only so for storeblk case we handle rules for making
sure virtmem stays consistent after virtmem lock is retaken differently:

1. We disallow several parallel writeouts for one fileh. This way dirty
   pages handling logic can not mess up. This restriction is also
   consistent with ZODB 2 phase commit protocol where for a transaction
   commit logic is invoked/handled from only 1 thread.

2. For the same reason we disallow discard while writeout is in
   progress. This is also consistent with ZODB 2 phase commit protocol
   where txn.tpc_abort() is not expected to be called at the same time
   with txn.commit().

3. While writeout is in progress, for that fileh we disallow pages
   modifications and pages invalidations - because both operations would
   change at least fileh dirty pages list which is iterated over by
   writeout code with releasing/retaking the virtmem lock. By
   disallowing them we make sure fileh dirty pages list stays constant
   during whole fileh writeout.

   This restrictions are also consistent with ZODB commit semantics:

   - while an object is being stored into ZODB it is not expected it
     will be further modified or explicitly invalidated by client via
     ._p_invalidate()

   - server initiated invalidations come into effect only at transaction
     boundaries - when new transaction is started, not during commit time.

Also since now storeblk is called with virtmem lock released, for buffer
to store we no longer can use present page mapping in some vma directly,
because while virtmem lock is released that mappings can go away.

Fixes: #6
parent b0d1e540
...@@ -15,7 +15,7 @@ ...@@ -15,7 +15,7 @@
# warranty of MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. # warranty of MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.
# #
# See COPYING file for full licensing terms. # See COPYING file for full licensing terms.
from wendelin.bigfile import BigFile from wendelin.bigfile import BigFile, WRITEOUT_STORE
from threading import Thread, Lock from threading import Thread, Lock
from time import sleep from time import sleep
...@@ -69,6 +69,16 @@ PS = 2*MB ...@@ -69,6 +69,16 @@ PS = 2*MB
# Z <- ClientStorage.invalidateTransaction() # Z <- ClientStorage.invalidateTransaction()
# Z -> zeo.load # Z -> zeo.load
# V <- fileh_invalidate_page (possibly of unrelated page) # V <- fileh_invalidate_page (possibly of unrelated page)
#
# --------
# and similarly for storeblk:
#
# T1 T2
#
# commit same as ^^^
# V -> storeblk
#
# Z -> zeo.store
def test_thread_lock_vs_virtmem_lock(): def test_thread_lock_vs_virtmem_lock():
Z = Lock() Z = Lock()
c12 = NotifyChannel() # T1 -> T2 c12 = NotifyChannel() # T1 -> T2
...@@ -79,7 +89,7 @@ def test_thread_lock_vs_virtmem_lock(): ...@@ -79,7 +89,7 @@ def test_thread_lock_vs_virtmem_lock():
obj = BigFile.__new__(cls, blksize) obj = BigFile.__new__(cls, blksize)
return obj return obj
def loadblk(self, blk, buf): def Zsync_and_lockunlock(self):
tell, wait = c12.tell, c21.wait tell, wait = c12.tell, c21.wait
# synchronize with invalidate in T2 # synchronize with invalidate in T2
...@@ -91,6 +101,11 @@ def test_thread_lock_vs_virtmem_lock(): ...@@ -91,6 +101,11 @@ def test_thread_lock_vs_virtmem_lock():
Z.acquire() Z.acquire()
Z.release() Z.release()
def loadblk(self, blk, buf):
self.Zsync_and_lockunlock()
def storeblk(self, blk, buf):
self.Zsync_and_lockunlock()
f = ZLockBigFile(PS) f = ZLockBigFile(PS)
fh = f.fileh_open() fh = f.fileh_open()
...@@ -101,16 +116,26 @@ def test_thread_lock_vs_virtmem_lock(): ...@@ -101,16 +116,26 @@ def test_thread_lock_vs_virtmem_lock():
def T1(): def T1():
m[0] # calls ZLockBigFile.loadblk() m[0] # calls ZLockBigFile.loadblk()
tell, wait = c12.tell, c21.wait
wait('T2-Z-released')
m[0] = bord_py3(b'1') # make page dirty
fh.dirty_writeout(WRITEOUT_STORE) # calls ZLockBigFile.storeblk()
def T2(): def T2():
tell, wait = c21.tell, c12.wait tell, wait = c21.tell, c12.wait
wait('T1-V-under') # cycle 0: vs loadblk in T0
Z.acquire() # cycle 1: vs storeblk in T0
tell('T2-Z-taken') for _ in range(2):
wait('T1-V-under')
Z.acquire()
tell('T2-Z-taken')
fh2.invalidate_page(0) # NOTE invalidating page _not_ of fh
Z.release()
fh2.invalidate_page(0) # NOTE invalidating page _not_ of fh tell('T2-Z-released')
Z.release()
t1, t2 = Thread(target=T1), Thread(target=T2) t1, t2 = Thread(target=T1), Thread(target=T2)
......
...@@ -150,6 +150,7 @@ int fileh_open(BigFileH *fileh, BigFile *file, RAM *ram) ...@@ -150,6 +150,7 @@ int fileh_open(BigFileH *fileh, BigFile *file, RAM *ram)
fileh->file = file; fileh->file = file;
INIT_LIST_HEAD(&fileh->mmaps); INIT_LIST_HEAD(&fileh->mmaps);
INIT_LIST_HEAD(&fileh->dirty_pages); INIT_LIST_HEAD(&fileh->dirty_pages);
fileh->writeout_inprogress = 0;
pagemap_init(&fileh->pagemap, ilog2_exact(ram->pagesize)); pagemap_init(&fileh->pagemap, ilog2_exact(ram->pagesize));
out: out:
...@@ -172,6 +173,9 @@ void fileh_close(BigFileH *fileh) ...@@ -172,6 +173,9 @@ void fileh_close(BigFileH *fileh)
// fileh, but mapping exists - real fileh release is delayed to last unmap ? // fileh, but mapping exists - real fileh release is delayed to last unmap ?
BUG_ON(!list_empty(&fileh->mmaps)); BUG_ON(!list_empty(&fileh->mmaps));
/* it's an error to close fileh while writeout is in progress */
BUG_ON(fileh->writeout_inprogress);
/* drop all pages (dirty or not) associated with this fileh */ /* drop all pages (dirty or not) associated with this fileh */
pagemap_for_each(page, &fileh->pagemap) { pagemap_for_each(page, &fileh->pagemap) {
/* it's an error to close fileh to mapping of which an access is /* it's an error to close fileh to mapping of which an access is
...@@ -328,6 +332,10 @@ int fileh_dirty_writeout(BigFileH *fileh, enum WriteoutFlags flags) ...@@ -328,6 +332,10 @@ int fileh_dirty_writeout(BigFileH *fileh, enum WriteoutFlags flags)
sigsegv_block(&save_sigset); sigsegv_block(&save_sigset);
virt_lock(); virt_lock();
/* concurrent writeouts are not allowed */
BUG_ON(fileh->writeout_inprogress);
fileh->writeout_inprogress = 1;
/* pages are stored (if stored) in sorted order */ /* pages are stored (if stored) in sorted order */
if (flags & WRITEOUT_STORE) if (flags & WRITEOUT_STORE)
list_sort(&fileh->dirty_pages, hpage_indirty_cmp_bypgoffset, NULL); list_sort(&fileh->dirty_pages, hpage_indirty_cmp_bypgoffset, NULL);
...@@ -343,35 +351,29 @@ int fileh_dirty_writeout(BigFileH *fileh, enum WriteoutFlags flags) ...@@ -343,35 +351,29 @@ int fileh_dirty_writeout(BigFileH *fileh, enum WriteoutFlags flags)
blk_t blk = page->f_pgoffset; // NOTE assumes blksize = pagesize blk_t blk = page->f_pgoffset; // NOTE assumes blksize = pagesize
void *pagebuf; void *pagebuf;
int mapped_tmp = 0;
if (!page->refcnt) { /* mmap page temporarily somewhere
/* page not mmaped anywhere - mmap it temporarily somewhere */ *
pagebuf = page_mmap(page, NULL, PROT_READ); * ( we cannot use present page mapping in some vma directly,
TODO(!pagebuf); // XXX err * because while storeblk is called with virtmem lock released that
mapped_tmp = 1; * mapping can go away ) */
} pagebuf = page_mmap(page, NULL, PROT_READ);
TODO(!pagebuf); // XXX err
else {
/* some vma mmaps page - use that memory directly */ /* unlock virtmem before calling storeblk()
*
/* XXX this assumes there is small #vma and is ugly - in general it * that call is potentially slow and external code can take other
* should be simpler via back-pointers from page? */ * locks. If that "other locks" are also taken before external code
pagebuf = NULL; * calls e.g. fileh_invalidate_page() in different codepath a deadlock
list_for_each(hmmap, &fileh->mmaps) { * can happen. (similar to loadblk case) */
VMA *vma = list_entry(hmmap, typeof(*vma), same_fileh); virt_unlock();
if (vma_page_ismapped(vma, page)) {
pagebuf = vma_page_addr(vma, page);
break;
}
}
BUG_ON(!pagebuf);
}
err = file->file_ops->storeblk(file, blk, pagebuf); err = file->file_ops->storeblk(file, blk, pagebuf);
if (mapped_tmp) /* relock virtmem */
xmunmap(pagebuf, page_size(page)); virt_lock();
xmunmap(pagebuf, page_size(page));
if (err) if (err)
goto out; goto out;
...@@ -395,6 +397,8 @@ int fileh_dirty_writeout(BigFileH *fileh, enum WriteoutFlags flags) ...@@ -395,6 +397,8 @@ int fileh_dirty_writeout(BigFileH *fileh, enum WriteoutFlags flags)
if (flags & WRITEOUT_MARKSTORED) if (flags & WRITEOUT_MARKSTORED)
BUG_ON(!list_empty(&fileh->dirty_pages)); BUG_ON(!list_empty(&fileh->dirty_pages));
fileh->writeout_inprogress = 0;
out: out:
virt_unlock(); virt_unlock();
sigsegv_restore(&save_sigset); sigsegv_restore(&save_sigset);
...@@ -411,6 +415,8 @@ void fileh_dirty_discard(BigFileH *fileh) ...@@ -411,6 +415,8 @@ void fileh_dirty_discard(BigFileH *fileh)
sigsegv_block(&save_sigset); sigsegv_block(&save_sigset);
virt_lock(); virt_lock();
/* discard is not allowed to run in parallel to writeout */
BUG_ON(fileh->writeout_inprogress);
list_for_each_safe(hpage, hpage_next, &fileh->dirty_pages) { list_for_each_safe(hpage, hpage_next, &fileh->dirty_pages) {
page = list_entry(hpage, typeof(*page), in_dirty); page = list_entry(hpage, typeof(*page), in_dirty);
...@@ -438,6 +444,9 @@ void fileh_invalidate_page(BigFileH *fileh, pgoff_t pgoffset) ...@@ -438,6 +444,9 @@ void fileh_invalidate_page(BigFileH *fileh, pgoff_t pgoffset)
sigsegv_block(&save_sigset); sigsegv_block(&save_sigset);
virt_lock(); virt_lock();
/* it's an error to invalidate fileh while writeout is in progress */
BUG_ON(fileh->writeout_inprogress);
page = pagemap_get(&fileh->pagemap, pgoffset); page = pagemap_get(&fileh->pagemap, pgoffset);
if (page) { if (page) {
/* for pages where loading is in progress, we just remove the page from /* for pages where loading is in progress, we just remove the page from
...@@ -660,7 +669,7 @@ VMFaultResult vma_on_pagefault(VMA *vma, uintptr_t addr, int write) ...@@ -660,7 +669,7 @@ VMFaultResult vma_on_pagefault(VMA *vma, uintptr_t addr, int write)
* that call is potentially slow and external code can take other * that call is potentially slow and external code can take other
* locks. If that "other locks" are also taken before external code * locks. If that "other locks" are also taken before external code
* calls e.g. fileh_invalidate_page() in different codepath a deadlock * calls e.g. fileh_invalidate_page() in different codepath a deadlock
* can happen. */ * can happen. (similar to storeblk case) */
page->state = PAGE_LOADING; page->state = PAGE_LOADING;
virt_unlock(); virt_unlock();
...@@ -742,8 +751,12 @@ VMFaultResult vma_on_pagefault(VMA *vma, uintptr_t addr, int write) ...@@ -742,8 +751,12 @@ VMFaultResult vma_on_pagefault(VMA *vma, uintptr_t addr, int write)
} }
// XXX also call page->markdirty() ? // XXX also call page->markdirty() ?
if (newstate == PAGE_DIRTY && newstate != page->state) if (newstate == PAGE_DIRTY && newstate != page->state) {
/* it is not allowed to modify pages while writeout is in progress */
BUG_ON(fileh->writeout_inprogress);
list_add_tail(&page->in_dirty, &fileh->dirty_pages); list_add_tail(&page->in_dirty, &fileh->dirty_pages);
}
page->state = max(page->state, newstate); page->state = max(page->state, newstate);
/* mark page as used recently */ /* mark page as used recently */
...@@ -859,6 +872,8 @@ static void page_drop_memory(Page *page) ...@@ -859,6 +872,8 @@ static void page_drop_memory(Page *page)
/* NOTE we try not to drop memory for loading-in-progress pages. /* 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. */ * so if this is called for such a page - it is a bug. */
BUG_ON(page->state == PAGE_LOADING); BUG_ON(page->state == PAGE_LOADING);
/* same for storing-in-progress */
BUG_ON(page->fileh->writeout_inprogress && page->state == PAGE_DIRTY);
if (page->state == PAGE_EMPTY) if (page->state == PAGE_EMPTY)
return; return;
......
...@@ -67,6 +67,9 @@ struct BigFileH { ...@@ -67,6 +67,9 @@ struct BigFileH {
/* fileh dirty pages */ /* fileh dirty pages */
struct list_head dirty_pages; /* _ -> page->in_dirty */ struct list_head dirty_pages; /* _ -> page->in_dirty */
/* whether writeout is currently in progress */
int writeout_inprogress;
}; };
typedef struct BigFileH BigFileH; typedef struct BigFileH BigFileH;
...@@ -153,6 +156,7 @@ int fileh_open(BigFileH *fileh, BigFile *file, RAM *ram); ...@@ -153,6 +156,7 @@ int fileh_open(BigFileH *fileh, BigFile *file, RAM *ram);
/* close fileh /* close fileh
* *
* it's an error to call fileh_close with existing mappings * it's an error to call fileh_close with existing mappings
* it's an error to call fileh_close while writeout for fileh is in progress
*/ */
void fileh_close(BigFileH *fileh); void fileh_close(BigFileH *fileh);
...@@ -205,6 +209,12 @@ enum WriteoutFlags { ...@@ -205,6 +209,12 @@ enum WriteoutFlags {
* *
* No guarantee is made about atomicity - e.g. if this call fails, some * No guarantee is made about atomicity - e.g. if this call fails, some
* pages could be written and some left in memory in dirty state. * pages could be written and some left in memory in dirty state.
*
* it's an error for a given fileh to call several fileh_dirty_writeout() in
* parallel.
*
* it's an error for a given fileh to modify its pages while writeout is in
* progress: until fileh_dirty_writeout(... | WRITEOUT_STORE) has finished.
*/ */
int fileh_dirty_writeout(BigFileH *fileh, enum WriteoutFlags flags); int fileh_dirty_writeout(BigFileH *fileh, enum WriteoutFlags flags);
...@@ -216,6 +226,9 @@ int fileh_dirty_writeout(BigFileH *fileh, enum WriteoutFlags flags); ...@@ -216,6 +226,9 @@ int fileh_dirty_writeout(BigFileH *fileh, enum WriteoutFlags flags);
* - it is unmapped from all mmaps; * - it is unmapped from all mmaps;
* - its content is discarded; * - its content is discarded;
* - its backing memory is released to OS. * - its backing memory is released to OS.
*
* it's an error for a given fileh to call fileh_dirty_discard() while writeout
* is in progress.
*/ */
void fileh_dirty_discard(BigFileH *fileh); void fileh_dirty_discard(BigFileH *fileh);
...@@ -230,6 +243,9 @@ void fileh_dirty_discard(BigFileH *fileh); ...@@ -230,6 +243,9 @@ void fileh_dirty_discard(BigFileH *fileh);
* *
* ( Such invalidation is needed to synchronize fileh memory, when we know a * ( Such invalidation is needed to synchronize fileh memory, when we know a
* file was changed externally ) * file was changed externally )
*
* it's an error to call fileh_invalidate_page() while writeout for fileh is in
* progress.
*/ */
void fileh_invalidate_page(BigFileH *fileh, pgoff_t pgoffset); void fileh_invalidate_page(BigFileH *fileh, pgoff_t pgoffset);
......
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