Commit 87eca1e0 authored by Julien Muchembled's avatar Julien Muchembled

client: replace global load lock by a per-oid one

parent c7cdcf87
...@@ -17,7 +17,7 @@ ...@@ -17,7 +17,7 @@
import heapq import heapq
import random import random
import time import time
from collections import defaultdict
try: try:
from ZODB._compat import dumps, loads, _protocol from ZODB._compat import dumps, loads, _protocol
except ImportError: except ImportError:
...@@ -79,7 +79,7 @@ class Application(ThreadedApplication): ...@@ -79,7 +79,7 @@ class Application(ThreadedApplication):
# no self-assigned UUID, primary master will supply us one # no self-assigned UUID, primary master will supply us one
self._cache = ClientCache() if cache_size is None else \ self._cache = ClientCache() if cache_size is None else \
ClientCache(max_size=cache_size) ClientCache(max_size=cache_size)
self._loading_oid = None self._loading = defaultdict(lambda: (Lock(), []))
self.new_oid_list = () self.new_oid_list = ()
self.last_oid = '\0' * 8 self.last_oid = '\0' * 8
self.storage_event_handler = storage.StorageEventHandler(self) self.storage_event_handler = storage.StorageEventHandler(self)
...@@ -90,19 +90,13 @@ class Application(ThreadedApplication): ...@@ -90,19 +90,13 @@ class Application(ThreadedApplication):
self.notifications_handler = master.PrimaryNotificationsHandler( self) self.notifications_handler = master.PrimaryNotificationsHandler( self)
self._txn_container = TransactionContainer() self._txn_container = TransactionContainer()
# Lock definition : # Lock definition :
# _load_lock is used to make loading and storing atomic
lock = Lock()
self._load_lock_acquire = lock.acquire
self._load_lock_release = lock.release
# _oid_lock is used in order to not call multiple oid # _oid_lock is used in order to not call multiple oid
# generation at the same time # generation at the same time
lock = Lock() lock = Lock()
self._oid_lock_acquire = lock.acquire self._oid_lock_acquire = lock.acquire
self._oid_lock_release = lock.release self._oid_lock_release = lock.release
lock = Lock()
# _cache_lock is used for the client cache # _cache_lock is used for the client cache
self._cache_lock_acquire = lock.acquire self._cache_lock = Lock()
self._cache_lock_release = lock.release
# _connecting_to_master_node is used to prevent simultaneous master # _connecting_to_master_node is used to prevent simultaneous master
# node connection attempts # node connection attempts
self._connecting_to_master_node = Lock() self._connecting_to_master_node = Lock()
...@@ -398,21 +392,28 @@ class Application(ThreadedApplication): ...@@ -398,21 +392,28 @@ class Application(ThreadedApplication):
""" """
# TODO: # TODO:
# - rename parameters (here? and in handlers & packet definitions) # - rename parameters (here? and in handlers & packet definitions)
acquired = False
acquire = self._cache_lock_acquire lock = self._cache_lock
release = self._cache_lock_release
# XXX: Consider using a more fine-grained lock.
self._load_lock_acquire()
try: try:
acquire() while 1:
try: with lock:
result = self._loadFromCache(oid, tid, before_tid) result = self._loadFromCache(oid, tid, before_tid)
if result: if result:
return result return result
self._loading_oid = oid load_lock = self._loading[oid][0]
self._loading_invalidated = [] acquired = load_lock.acquire(0)
finally: # Several concurrent cache misses for the same oid are probably
release() # for the same tid so we use a per-oid lock to avoid asking the
# same data to the storage node.
if acquired:
# The first thread does load from storage,
# and fills cache with the response.
break
# The other threads wait for the first one to complete and
# loop, possibly resulting in a new cache miss if a different
# tid is actually wanted or if the data was too big.
with load_lock:
pass
# While the cache lock is released, an arbitrary number of # While the cache lock is released, an arbitrary number of
# invalidations may be processed, for this oid or not. And at this # invalidations may be processed, for this oid or not. And at this
# precise moment, if both tid and before_tid are None (which is # precise moment, if both tid and before_tid are None (which is
...@@ -428,20 +429,24 @@ class Application(ThreadedApplication): ...@@ -428,20 +429,24 @@ class Application(ThreadedApplication):
# we got from master. # we got from master.
before_tid = p64(u64(self.last_tid) + 1) before_tid = p64(u64(self.last_tid) + 1)
data, tid, next_tid, _ = self._loadFromStorage(oid, tid, before_tid) data, tid, next_tid, _ = self._loadFromStorage(oid, tid, before_tid)
acquire() with lock:
try: loading = self._loading.pop(oid, None)
if self._loading_oid: if loading:
assert loading[0] is load_lock
if not next_tid: if not next_tid:
for t in self._loading_invalidated: for t in loading[1]:
if tid < t: if tid < t:
next_tid = t next_tid = t
break break
self._cache.store(oid, data, tid, next_tid) self._cache.store(oid, data, tid, next_tid)
# Else, we just reconnected to the master. # Else, we just reconnected to the master.
finally: load_lock.release()
release() except:
finally: if acquired:
self._load_lock_release() with lock:
self._loading.pop(oid, None)
load_lock.release()
raise
return data, tid, next_tid return data, tid, next_tid
def _loadFromStorage(self, oid, at_tid, before_tid): def _loadFromStorage(self, oid, at_tid, before_tid):
...@@ -986,11 +991,8 @@ class Application(ThreadedApplication): ...@@ -986,11 +991,8 @@ class Application(ThreadedApplication):
# It should not be otherwise required (clients should be free to load # It should not be otherwise required (clients should be free to load
# old data as long as it is available in cache, event if it was pruned # old data as long as it is available in cache, event if it was pruned
# by a pack), so don't bother invalidating on other clients. # by a pack), so don't bother invalidating on other clients.
self._cache_lock_acquire() with self._cache_lock:
try:
self._cache.clear() self._cache.clear()
finally:
self._cache_lock_release()
def getLastTID(self, oid): def getLastTID(self, oid):
return self.load(oid)[1] return self.load(oid)[1]
......
...@@ -52,8 +52,7 @@ class PrimaryNotificationsHandler(MTEventHandler): ...@@ -52,8 +52,7 @@ class PrimaryNotificationsHandler(MTEventHandler):
# Either we're connecting or we already know the last tid # Either we're connecting or we already know the last tid
# via invalidations. # via invalidations.
assert app.master_conn is None, app.master_conn assert app.master_conn is None, app.master_conn
app._cache_lock_acquire() with app._cache_lock:
try:
if app_last_tid < ltid: if app_last_tid < ltid:
app._cache.clear_current() app._cache.clear_current()
# In the past, we tried not to invalidate the # In the past, we tried not to invalidate the
...@@ -67,9 +66,7 @@ class PrimaryNotificationsHandler(MTEventHandler): ...@@ -67,9 +66,7 @@ class PrimaryNotificationsHandler(MTEventHandler):
app._cache.clear() app._cache.clear()
# Make sure a parallel load won't refill the cache # Make sure a parallel load won't refill the cache
# with garbage. # with garbage.
app._loading_oid = app._loading_invalidated = None app._loading.clear()
finally:
app._cache_lock_release()
db = app.getDB() db = app.getDB()
db is None or db.invalidateCache() db is None or db.invalidateCache()
app.last_tid = ltid app.last_tid = ltid
...@@ -80,22 +77,20 @@ class PrimaryNotificationsHandler(MTEventHandler): ...@@ -80,22 +77,20 @@ class PrimaryNotificationsHandler(MTEventHandler):
app.last_tid = tid app.last_tid = tid
# Update cache # Update cache
cache = app._cache cache = app._cache
app._cache_lock_acquire() with app._cache_lock:
try:
invalidate = app._cache.invalidate invalidate = app._cache.invalidate
loading = app._loading_oid loading_get = app._loading.get
for oid, data in cache_dict.iteritems(): for oid, data in cache_dict.iteritems():
# Update ex-latest value in cache # Update ex-latest value in cache
invalidate(oid, tid) invalidate(oid, tid)
if oid == loading: loading = loading_get(oid)
app._loading_invalidated.append(tid) if loading:
loading[1].append(tid)
if data is not None: if data is not None:
# Store in cache with no next_tid # Store in cache with no next_tid
cache.store(oid, data, tid, None) cache.store(oid, data, tid, None)
if callback is not None: if callback is not None:
callback(tid) callback(tid)
finally:
app._cache_lock_release()
def connectionClosed(self, conn): def connectionClosed(self, conn):
app = self.app app = self.app
...@@ -124,19 +119,17 @@ class PrimaryNotificationsHandler(MTEventHandler): ...@@ -124,19 +119,17 @@ class PrimaryNotificationsHandler(MTEventHandler):
if app.ignore_invalidations: if app.ignore_invalidations:
return return
app.last_tid = tid app.last_tid = tid
app._cache_lock_acquire() with app._cache_lock:
try:
invalidate = app._cache.invalidate invalidate = app._cache.invalidate
loading = app._loading_oid loading_get = app._loading.get
for oid in oid_list: for oid in oid_list:
invalidate(oid, tid) invalidate(oid, tid)
if oid == loading: loading = loading_get(oid)
app._loading_invalidated.append(tid) if loading:
loading[1].append(tid)
db = app.getDB() db = app.getDB()
if db is not None: if db is not None:
db.invalidate(tid, oid_list) db.invalidate(tid, oid_list)
finally:
app._cache_lock_release()
def notifyPartitionChanges(self, conn, ptid, cell_list): def notifyPartitionChanges(self, conn, ptid, cell_list):
if self.app.pt.filled(): if self.app.pt.filled():
......
...@@ -197,8 +197,7 @@ elif IF == 'trace-cache': ...@@ -197,8 +197,7 @@ elif IF == 'trace-cache':
@defer @defer
def profile(app): def profile(app):
app._cache_lock_acquire() with app._cache_lock:
try:
cache = app._cache cache = app._cache
if type(cache) is ClientCache: if type(cache) is ClientCache:
app._cache = CacheTracer(cache, '%s-%s.neo-cache-trace' % app._cache = CacheTracer(cache, '%s-%s.neo-cache-trace' %
...@@ -206,5 +205,3 @@ elif IF == 'trace-cache': ...@@ -206,5 +205,3 @@ elif IF == 'trace-cache':
app._cache.clear() app._cache.clear()
else: else:
app._cache = cache.close() app._cache = cache.close()
finally:
app._cache_lock_release()
...@@ -1072,8 +1072,7 @@ class NEOThreadedTest(NeoTestBase): ...@@ -1072,8 +1072,7 @@ class NEOThreadedTest(NeoTestBase):
def run(self): def run(self):
try: try:
apply(*self.__target) self.__result = apply(*self.__target)
self.__exc_info = None
except: except:
self.__exc_info = sys.exc_info() self.__exc_info = sys.exc_info()
if self.__exc_info[0] is NEOThreadedTest.failureException: if self.__exc_info[0] is NEOThreadedTest.failureException:
...@@ -1081,10 +1080,13 @@ class NEOThreadedTest(NeoTestBase): ...@@ -1081,10 +1080,13 @@ class NEOThreadedTest(NeoTestBase):
def join(self, timeout=None): def join(self, timeout=None):
threading.Thread.join(self, timeout) threading.Thread.join(self, timeout)
if not self.is_alive() and self.__exc_info: if not self.is_alive():
etype, value, tb = self.__exc_info try:
del self.__exc_info return self.__result
raise etype, value, tb except AttributeError:
etype, value, tb = self.__exc_info
del self.__exc_info
raise etype, value, tb
class newThread(newPausedThread): class newThread(newPausedThread):
......
...@@ -1009,7 +1009,7 @@ class Test(NEOThreadedTest): ...@@ -1009,7 +1009,7 @@ class Test(NEOThreadedTest):
x = r[''] = PCounter() x = r[''] = PCounter()
t.commit() t.commit()
tid1 = x._p_serial tid1 = x._p_serial
nonlocal_ = [0, 1] nonlocal_ = [0, 0, 0]
l1 = threading.Lock(); l1.acquire() l1 = threading.Lock(); l1.acquire()
l2 = threading.Lock(); l2.acquire() l2 = threading.Lock(); l2.acquire()
def invalidateObjects(orig, *args): def invalidateObjects(orig, *args):
...@@ -1019,12 +1019,26 @@ class Test(NEOThreadedTest): ...@@ -1019,12 +1019,26 @@ class Test(NEOThreadedTest):
nonlocal_[0] += 1 nonlocal_[0] += 1
if nonlocal_[0] == 2: if nonlocal_[0] == 2:
l2.release() l2.release()
def _cache_lock_release(orig): class CacheLock(object):
orig() def __init__(self, client):
if nonlocal_[1]: self._lock = client._cache_lock
nonlocal_[1] = 0 def __enter__(self):
self._lock.acquire()
def __exit__(self, t, v, tb):
count = nonlocal_[1]
nonlocal_[1] = count + 1
self._lock.release()
if count == 0:
load_same.start()
l2.acquire()
elif count == 1:
load_other.start()
def _loadFromStorage(orig, *args):
count = nonlocal_[2]
nonlocal_[2] = count + 1
if not count:
l1.release() l1.release()
l2.acquire() return orig(*args)
with cluster.newClient() as client, \ with cluster.newClient() as client, \
Patch(client.notifications_handler, Patch(client.notifications_handler,
invalidateObjects=invalidateObjects): invalidateObjects=invalidateObjects):
...@@ -1043,17 +1057,34 @@ class Test(NEOThreadedTest): ...@@ -1043,17 +1057,34 @@ class Test(NEOThreadedTest):
r._p_changed = 1 r._p_changed = 1
t.commit() t.commit()
self.assertEqual(tid1, client.last_tid) self.assertEqual(tid1, client.last_tid)
with Patch(client, _cache_lock_release=_cache_lock_release): load_same = self.newPausedThread(client.load, x._p_oid)
load_other = self.newPausedThread(client.load, r._p_oid)
with Patch(client, _cache_lock=CacheLock(client)), \
Patch(client, _loadFromStorage=_loadFromStorage):
# 1. Just after having found nothing in cache, the worker # 1. Just after having found nothing in cache, the worker
# thread asks the poll thread to get notified about # thread asks the poll thread to get notified about
# invalidations for the loading oid. # invalidations for the loading oid.
# <context switch> # <context switch> (l1)
# 2. Both invalidations are processed. -> last_tid=tid3 # 2. Both invalidations are processed. -> last_tid=tid3
# <context switch> # <context switch> (l2)
# 3. The worker thread loads before tid3+1. # 3. The worker thread loads before tid3+1.
# The poll thread notified [tid2], which must be ignored. # The poll thread notified [tid2], which must be ignored.
self.assertEqual((tid2, None), client.load(x._p_oid)[1:]) # In parallel, 2 other loads are done (both cache misses):
self.assertEqual(nonlocal_, [2, 0]) # - one for the same oid, which waits for first load to
# complete and in particular fill cache, in order to
# avoid asking the same data to the storage node
# - another for a different oid, which doesn't wait, as shown
# by the fact that it returns an old record (i.e. before any
# invalidation packet is processed)
loaded = client.load(x._p_oid)
self.assertEqual((tid2, None), loaded[1:])
self.assertEqual(loaded, load_same.join())
self.assertEqual((tid1, r._p_serial), load_other.join()[1:])
# To summary:
# - 3 concurrent loads starting with cache misses
# - 2 loads from storage
# - 1 load ending with a cache hit
self.assertEqual(nonlocal_, [2, 8, 2])
@with_cluster(storage_count=2, partitions=2) @with_cluster(storage_count=2, partitions=2)
def testReadVerifyingStorage(self, cluster): def testReadVerifyingStorage(self, cluster):
......
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