Commit 092992db authored by Julien Muchembled's avatar Julien Muchembled

Implement deadlock avoidance

This is a first version with several optimizations possible:
- improve EventQueue (or implement a specific queue) to minimize deadlocks
- turn the RebaseObject packet into a notification

Sorting oids could also be useful to reduce the probability of deadlocks,
but that would never be enough to avoid them completely, even if there's a
single storage. For example:

1. C1 does a first store (x or y)
2. C2 stores x and y; one is delayed
3. C1 stores the other -> deadlock
   When solving the deadlock, the data of the first store may only
   exist on the storage.

2 functional tests are removed because they're redundant,
either with ZODB tests or with the new threaded tests.
parent cc8d0a7c
......@@ -4,18 +4,6 @@ or promised features of NEO (marked with N).
All the listed bugs will be fixed with high priority.
(Z) Conflict resolution not fully implemented
---------------------------------------------
Even with a single storage node, so-called 'deadlock avoidance' may
happen to in order to resolve conflicts. In such cases, conflicts will not be
resolved even if your _p_resolveConflict() method would succeed, leading to a
normal ConflictError.
Although this should happen rarely enough not to affect performance, this can
be an issue if your application can't afford restarting the transaction,
e.g. because it interacted with external environment.
(N) Storage failure or update may lead to POSException or break undoLog()
-------------------------------------------------------------------------
......
......@@ -125,6 +125,7 @@
- Add ctl command to list last transactions, like fstail for FileStorage.
Tests
- Split neo/tests/threaded/test.py
- Use another mock library: Python 3.3+ has unittest.mock, which is
available for earlier versions at https://pypi.python.org/pypi/mock
......
......@@ -417,24 +417,16 @@ class Application(ThreadedApplication):
while 1:
# We iterate over conflict_dict, and clear it,
# because new items may be added by calls to _store.
# This is also done atomically, to avoid race conditions
# with PrimaryNotificationsHandler.notifyDeadlock
try:
oid, (serial, conflict_serial) = pop_conflict()
except KeyError:
return
if conflict_serial == MAX_TID:
if 1:
# XXX: disable deadlock avoidance code until it is fixed
logging.info('Deadlock avoidance on %r:%r',
dump(oid), dump(serial))
# 'data' parameter of ConflictError is only used to report the
# class of the object. It doesn't matter if 'data' is None
# because the transaction is too big.
try:
data = data_dict[oid][0]
except KeyError:
# succesfully stored on another storage node
data = txn_context.cache_dict[oid]
else:
try:
data = data_dict.pop(oid)[0]
except KeyError:
assert oid is conflict_serial is None, (oid, conflict_serial)
# Storage refused us from taking object lock, to avoid a
# possible deadlock. TID is actually used for some kind of
# "locking priority": when a higher value has the lock,
......@@ -443,11 +435,15 @@ class Application(ThreadedApplication):
# To recover, we must ask storages to release locks we
# hold (to let possibly-competing transactions acquire
# them), and requeue our already-sent store requests.
logging.info('Deadlock avoidance triggered on %r:%r',
dump(oid), dump(serial))
raise NotImplementedError
ttid = txn_context.ttid
logging.info('Deadlock avoidance triggered for TXN %s'
' with new locking TID %s', dump(ttid), dump(serial))
txn_context.locking_tid = serial
packet = Packets.AskRebaseTransaction(ttid, serial)
for uuid, status in txn_context.involved_nodes.iteritems():
if status < 2:
self._askStorageForWrite(txn_context, uuid, packet)
else:
data = data_dict.pop(oid)[0]
if data is CHECKED_SERIAL:
raise ReadConflictError(oid=oid, serials=(conflict_serial,
serial))
......@@ -457,12 +453,17 @@ class Application(ThreadedApplication):
if self.last_tid < conflict_serial:
self.sync() # possible late invalidation (very rare)
try:
new_data = tryToResolveConflict(oid, conflict_serial,
data = tryToResolveConflict(oid, conflict_serial,
serial, data)
except ConflictError:
logging.info('Conflict resolution failed for '
'%r:%r with %r', dump(oid), dump(serial),
dump(conflict_serial))
# With recent ZODB, get_pickle_metadata (from ZODB.utils)
# does not support empty values, so do not pass 'data'
# in this case.
raise ConflictError(oid=oid, serials=(conflict_serial,
serial), data=data or None)
else:
logging.info('Conflict resolution succeeded for '
'%r:%r with %r', dump(oid), dump(serial),
......@@ -470,12 +471,18 @@ class Application(ThreadedApplication):
# Mark this conflict as resolved
resolved_dict[oid] = conflict_serial
# Try to store again
self._store(txn_context, oid, conflict_serial, new_data)
continue
# With recent ZODB, get_pickle_metadata (from ZODB.utils) does
# not support empty values, so do not pass 'data' in this case.
raise ConflictError(oid=oid, serials=(conflict_serial,
serial), data=data or None)
self._store(txn_context, oid, conflict_serial, data)
def _askStorageForWrite(self, txn_context, uuid, packet):
node = self.nm.getByUUID(uuid)
if node is not None:
conn = self.cp.getConnForNode(node)
if conn is not None:
try:
return conn.ask(packet, queue=txn_context.queue)
except ConnectionClosed:
pass
txn_context.involved_nodes[uuid] = 2
def waitResponses(self, queue):
"""Wait for all requests to be answered (or their connection to be
......@@ -510,16 +517,7 @@ class Application(ThreadedApplication):
packet = Packets.AskVoteTransaction(ttid)
for uuid, status in involved_nodes.iteritems():
if status == 1 and uuid not in trans_nodes:
node = self.nm.getByUUID(uuid)
if node is not None:
conn = self.cp.getConnForNode(node)
if conn is not None:
try:
conn.ask(packet, queue=queue)
continue
except ConnectionClosed:
pass
involved_nodes[uuid] = 2
self._askStorageForWrite(txn_context, uuid, packet)
self.waitResponses(txn_context.queue)
# If there are failed nodes, ask the master whether they can be
# disconnected while keeping the cluster operational. If possible,
......
......@@ -194,6 +194,13 @@ class PrimaryNotificationsHandler(MTEventHandler):
if node and node.isConnected():
node.getConnection().close()
def notifyDeadlock(self, conn, ttid, locking_tid):
for txn_context in self.app.txn_contexts():
if txn_context.ttid == ttid:
txn_context.conflict_dict[None] = locking_tid, None
txn_context.wakeup(conn)
break
class PrimaryAnswersHandler(AnswerBaseHandler):
""" Handle that process expected packets from the primary master """
......
......@@ -14,11 +14,12 @@
# You should have received a copy of the GNU General Public License
# along with this program. If not, see <http://www.gnu.org/licenses/>.
from zlib import decompress
from ZODB.TimeStamp import TimeStamp
from neo.lib import logging
from neo.lib.protocol import MAX_TID
from neo.lib.util import dump
from neo.lib.protocol import Packets
from neo.lib.util import dump, makeChecksum
from neo.lib.exception import NodeNotReady
from neo.lib.handler import MTEventHandler
from . import AnswerBaseHandler
......@@ -75,17 +76,75 @@ class StorageAnswersHandler(AnswerBaseHandler):
# receive the conflict answer from the first store on S2.
logging.info('%r report a conflict for %r with %r',
conn, dump(oid), dump(conflict))
if conflict != MAX_TID:
# If this conflict is not already resolved, mark it for
# resolution.
if conflict <= txn_context.resolved_dict.get(oid, ''):
return
txn_context.conflict_dict[oid] = serial, conflict
# If this conflict is not already resolved, mark it for
# resolution.
if txn_context.resolved_dict.get(oid, '') < conflict:
txn_context.conflict_dict[oid] = serial, conflict
else:
txn_context.written(self.app, conn.getUUID(), oid)
answerCheckCurrentSerial = answerStoreObject
def answerRebaseTransaction(self, conn, oid_list):
txn_context = self.app.getHandlerData()
ttid = txn_context.ttid
queue = txn_context.queue
try:
for oid in oid_list:
# We could have an extra parameter to tell the storage if we
# still have the data, and in this case revert what was done
# in Transaction.written. This would save bandwidth in case of
# conflict.
conn.ask(Packets.AskRebaseObject(ttid, oid),
queue=queue, oid=oid)
except ConnectionClosed:
txn_context.involved_nodes[conn.getUUID()] = 2
def answerRebaseObject(self, conn, conflict, oid):
if conflict:
txn_context = self.app.getHandlerData()
serial, conflict, data = conflict
assert serial and serial < conflict, (serial, conflict)
resolved = conflict <= txn_context.resolved_dict.get(oid, '')
try:
cached = txn_context.cache_dict.pop(oid)
except KeyError:
if resolved:
# We should still be waiting for an answer from this node.
assert conn.uuid in txn_context.data_dict[oid][1]
return
assert oid in txn_context.data_dict
if oid in txn_context.conflict_dict:
# Another node already reported the conflict, by answering
# to this rebase or to the previous store.
# Filling conflict_dict again would be a no-op.
assert txn_context.conflict_dict[oid] == (serial, conflict)
return
# A node has not answered yet to a previous store. Do not wait
# it to report the conflict because it may fail before.
else:
# The data for this oid are now back on client side.
# Revert what was done in Transaction.written
assert not resolved
if data is None: # undo or CHECKED_SERIAL
data = cached
else:
compression, checksum, data = data
if checksum != makeChecksum(data):
raise NEOStorageError(
'wrong checksum while getting back data for'
' object %s during rebase of transaction %s'
% (dump(oid), dump(txn_context.ttid)))
if compression:
data = decompress(data)
size = len(data)
txn_context.data_size += size
if cached:
assert cached == data
txn_context.cache_size -= size
txn_context.data_dict[oid] = data, None
txn_context.conflict_dict[oid] = serial, conflict
def answerStoreTransaction(self, conn):
pass
......
......@@ -17,14 +17,22 @@
from ZODB.POSException import StorageTransactionError
from neo.lib.connection import ConnectionClosed
from neo.lib.locking import SimpleQueue
from neo.lib.protocol import Packets
from .exception import NEOStorageError
@apply
class _WakeupPacket(object):
handler_method_name = 'pong'
decode = tuple
getId = int
class Transaction(object):
cache_size = 0 # size of data in cache_dict
data_size = 0 # size of data in data_dict
error = None
locking_tid = None
voted = False
ttid = None # XXX: useless, except for testBackupReadOnlyAccess
......@@ -45,6 +53,9 @@ class Transaction(object):
# status: 0 -> check only, 1 -> store, 2 -> failed
self.involved_nodes = {} # {node_id: status}
def wakeup(self, conn):
self.queue.put((conn, _WakeupPacket, {}))
def write(self, app, packet, object_id, store=1, **kw):
uuid_list = []
pt = app.pt
......@@ -53,7 +64,7 @@ class Transaction(object):
for cell in pt.getCellList(object_id):
node = cell.getNode()
uuid = node.getUUID()
status = involved.setdefault(uuid, store)
status = involved.get(uuid, -1)
if status < store:
involved[uuid] = store
elif status > 1:
......@@ -61,6 +72,13 @@ class Transaction(object):
conn = app.cp.getConnForNode(node)
if conn is not None:
try:
if status < 0 and self.locking_tid and 'oid' in kw:
# A deadlock happened but this node is not aware of it.
# Tell it to write-lock with the same locking tid as
# for the other nodes. The condition on kw is because
# we don't need that for transaction metadata.
conn.ask(Packets.AskRebaseTransaction(
self.ttid, self.locking_tid), queue=self.queue)
conn.ask(packet, queue=self.queue, **kw)
uuid_list.append(uuid)
continue
......
......@@ -398,6 +398,19 @@ class PStructItemOrNone(PStructItem):
value = reader(self.size)
return None if value == self._None else self.unpack(value)[0]
class POption(PStruct):
def _encode(self, writer, value):
if value is None:
writer('\0')
else:
writer('\1')
PStruct._encode(self, writer, value)
def _decode(self, reader):
if '\0\1'.index(reader(1)):
return PStruct._decode(self, reader)
class PList(PStructItem):
"""
A list of homogeneous items
......@@ -949,14 +962,60 @@ class GenerateOIDs(Packet):
PFOidList,
)
class Deadlock(Packet):
"""
Ask master to generate a new TTID that will be used by the client
to rebase a transaction. S -> PM -> C
"""
_fmt = PStruct('notify_deadlock',
PTID('ttid'),
PTID('locking_tid'),
)
class RebaseTransaction(Packet):
"""
Rebase transaction. C -> S.
"""
_fmt = PStruct('ask_rebase_transaction',
PTID('ttid'),
PTID('locking_tid'),
)
_answer = PStruct('answer_rebase_transaction',
PFOidList,
)
class RebaseObject(Packet):
"""
Rebase object. C -> S.
XXX: It is a request packet to simplify the implementation. For more
efficiency, this should be turned into a notification, and the
RebaseTransaction should answered once all objects are rebased
(so that the client can still wait on something).
"""
_fmt = PStruct('ask_rebase_object',
PTID('ttid'),
PTID('oid'),
)
_answer = PStruct('answer_rebase_object',
POption('conflict',
PTID('serial'),
PTID('conflict_serial'),
POption('data',
PBoolean('compression'),
PChecksum('checksum'),
PString('data'),
),
)
)
class StoreObject(Packet):
"""
Ask to store an object. Send an OID, an original serial, a current
transaction ID, and data. C -> S.
As for IStorage, 'serial' is ZERO_TID for new objects.
Answered 'conflict' value means:
- MAX_TID: deadlock
- else: conflict
"""
_fmt = PStruct('ask_store_object',
POID('oid'),
......@@ -1658,6 +1717,12 @@ class Packets(dict):
UnlockInformation)
AskNewOIDs, AnswerNewOIDs = register(
GenerateOIDs)
NotifyDeadlock = register(
Deadlock)
AskRebaseTransaction, AnswerRebaseTransaction = register(
RebaseTransaction)
AskRebaseObject, AnswerRebaseObject = register(
RebaseObject)
AskStoreObject, AnswerStoreObject = register(
StoreObject)
AbortTransaction = register(
......
......@@ -61,6 +61,9 @@ class StorageServiceHandler(BaseServiceHandler):
p = Packets.AnswerUnfinishedTransactions(last_tid, pending_list)
conn.answer(p)
def notifyDeadlock(self, conn, *args):
self.app.tm.deadlock(conn.getUUID(), *args)
def answerInformationLocked(self, conn, ttid):
self.app.tm.lock(ttid, conn.getUUID())
......
......@@ -19,13 +19,15 @@ from time import time
from struct import pack, unpack
from neo.lib import logging
from neo.lib.handler import DelayEvent, EventQueue
from neo.lib.protocol import ProtocolError, uuid_str, ZERO_OID, ZERO_TID
from neo.lib.protocol import Packets, ProtocolError, uuid_str, \
ZERO_OID, ZERO_TID
from neo.lib.util import dump, u64, addTID, tidFromTime
class Transaction(object):
"""
A pending transaction
"""
locking_tid = ZERO_TID
_tid = None
_msg_id = None
_oid_list = None
......@@ -303,6 +305,19 @@ class TransactionManager(EventQueue):
logging.debug('Begin %s', txn)
return tid
def deadlock(self, storage_id, ttid, locking_tid):
try:
txn = self._ttid_dict[ttid]
except KeyError:
return
if txn.locking_tid <= locking_tid:
client = txn.getNode()
txn.locking_tid = locking_tid = self._nextTID()
logging.info('Deadlock avoidance triggered by %s for %s:'
' new locking tid for TXN %s is %s', uuid_str(storage_id),
uuid_str(client.getUUID()), dump(ttid), dump(locking_tid))
client.notify(Packets.NotifyDeadlock(ttid, locking_tid))
def vote(self, app, ttid, uuid_list):
"""
Check that the transaction can be voted
......
......@@ -304,7 +304,7 @@ class ImporterDatabaseManager(DatabaseManager):
getPartitionTable changePartitionTable
getUnfinishedTIDDict dropUnfinishedData abortTransaction
storeTransaction lockTransaction unlockTransaction
storeData getOrphanList _pruneData deferCommit
loadData storeData getOrphanList _pruneData deferCommit
""".split():
setattr(self, x, getattr(self.db, x))
......
......@@ -463,6 +463,11 @@ class DatabaseManager(object):
no hash collision.
"""
@abstract
def loadData(self, data_id):
"""Inverse of storeData
"""
def holdData(self, checksum_or_id, *args):
"""Store raw data of temporary object
......
......@@ -541,6 +541,15 @@ class MySQLDatabaseManager(DatabaseManager):
raise
return self.conn.insert_id()
def loadData(self, data_id):
compression, hash, value = self.query(
"SELECT compression, hash, value FROM data where id=%s"
% data_id)[0]
if compression and compression & 0x80:
compression &= 0x7f
data = ''.join(self._bigData(data))
return compression, hash, value
del _structLL
def _getDataTID(self, oid, tid=None, before_tid=None):
......
......@@ -404,6 +404,10 @@ class SQLiteDatabaseManager(DatabaseManager):
return r
raise
def loadData(self, data_id):
return self.query("SELECT compression, hash, value"
" FROM data where id=?", (data_id,)).fetchone()
def _getDataTID(self, oid, tid=None, before_tid=None):
partition = self._getPartition(oid)
sql = 'SELECT tid, value_tid FROM obj' \
......
......@@ -110,6 +110,26 @@ class ClientOperationHandler(EventHandler):
self.app.tm.queueEvent(self._askStoreObject, conn, (oid, serial,
compression, checksum, data, data_serial, ttid, time.time()))
def askRebaseTransaction(self, conn, *args):
conn.answer(Packets.AnswerRebaseTransaction(
self.app.tm.rebase(conn, *args)))
def askRebaseObject(self, conn, ttid, oid):
try:
self._askRebaseObject(conn, ttid, oid, None)
except DelayEvent:
# locked by a previous transaction, retry later
self.app.tm.queueEvent(self._askRebaseObject,
conn, (ttid, oid, time.time()))
def _askRebaseObject(self, conn, ttid, oid, request_time):
conflict = self.app.tm.rebaseObject(ttid, oid)
if request_time and SLOW_STORE is not None:
duration = time.time() - request_time
if duration > SLOW_STORE:
logging.info('RebaseObject delay: %.02fs', duration)
conn.answer(Packets.AnswerRebaseObject(conflict))
def askTIDsFrom(self, conn, min_tid, max_tid, length, partition):
conn.answer(Packets.AnswerTIDsFrom(self.app.dm.getReplicationTIDList(
min_tid, max_tid, length, partition)))
......@@ -204,6 +224,8 @@ class ClientReadOnlyOperationHandler(ClientOperationHandler):
askVoteTransaction = _readOnly
askStoreObject = _readOnly
askFinalTID = _readOnly
askRebaseObject = _readOnly
askRebaseTransaction = _readOnly
# takes write lock & is only used when going to commit
askCheckCurrentSerial = _readOnly
......
......@@ -41,14 +41,16 @@ class Transaction(object):
Container for a pending transaction
"""
tid = None
has_trans = False
voted = 0
def __init__(self, uuid, ttid):
self._birth = time()
self.locking_tid = ttid
self.uuid = uuid
# Consider using lists.
self.serial_dict = {}
self.store_dict = {}
self.checked_set = set()
# We must distinguish lockless stores from deadlocks.
self.lockless = set()
def __repr__(self):
return "<%s(tid=%r, uuid=%r, age=%.2fs) at 0x%x>" \
......@@ -58,22 +60,23 @@ class Transaction(object):
time() - self._birth,
id(self))
def check(self, oid):
assert oid not in self.store_dict, dump(oid)
assert oid not in self.checked_set, dump(oid)
self.checked_set.add(oid)
def __lt__(self, other):
return self.locking_tid < other.locking_tid
def store(self, oid, data_id, value_serial):
"""
Add an object to the transaction
"""
assert oid not in self.checked_set, dump(oid)
self.store_dict[oid] = oid, data_id, value_serial
class TransactionManager(EventQueue):
"""
Manage pending transaction and locks
XXX: EventQueue is not very suited for deadlocks. It would be more
efficient to sort delayed packets by locking tid in order to minimize
cascaded deadlocks.
"""
def __init__(self, app):
......@@ -111,11 +114,13 @@ class TransactionManager(EventQueue):
replicated = self._replicated
notify = set(replicated)
# We sort transactions so that in case of muliple stores/checks for the
# same oid, the lock is taken by the highest ttid, which will delay
# new transactions.
for ttid, txn in sorted(self._transaction_dict.iteritems()):
for oid in txn.store_dict, txn.checked_set:
for oid in oid:
# same oid, the lock is taken by the highest locking ttid, which will
# delay new transactions.
for txn, ttid in sorted((txn, ttid) for ttid, txn in
self._transaction_dict.iteritems()):
if txn.locking_tid == MAX_TID:
break # all remaining transactions are resolving a deadlock
for oid in txn.lockless.intersection(txn.serial_dict):
partition = getPartition(oid)
if partition in replicated:
if store_lock_dict.get(oid, ttid) != ttid:
......@@ -124,12 +129,16 @@ class TransactionManager(EventQueue):
# replicated.
notify.discard(partition)
store_lock_dict[oid] = ttid
for partition in notify:
# For this partition, all oids of all pending transactions
if notify:
# For these partitions, all oids of all pending transactions
# are now locked normally and we don't rely anymore on other
# readable cells to check locks: we're really up-to-date.
self._app.master_conn.notify(Packets.NotifyReplicationDone(
partition, replicated.pop(partition)))
for partition in notify:
self._app.master_conn.notify(Packets.NotifyReplicationDone(
partition, replicated.pop(partition)))
for oid, ttid in store_lock_dict.iteritems():
if getPartition(oid) in notify:
self._transaction_dict[ttid].lockless.remove(oid)
def register(self, conn, ttid):
"""
......@@ -150,6 +159,73 @@ class TransactionManager(EventQueue):
except KeyError:
return None
def _rebase(self, transaction, ttid, locking_tid=MAX_TID):
# With the default value of locking_tid, this marks the transaction as
# being rebased, in case that the current lock is released (the other
# transaction is aborted or committed) before the client sends us a new
# locking tid: in lockObject, 'locked' will be None but we'll still
# have to delay the store.
transaction.locking_tid = locking_tid
if ttid:
# Remove store locks we have.
# In order to keep all locking data consistent, this must be done
# when the locking tid changes, i.e. from both 'lockObject' (for
# the node that triggered the deadlock) and 'rebase' (for other
# nodes).
for oid, locked in self._store_lock_dict.items():
# If this oid is locked by several transactions (all lockless),
# the following condition is true if we have the highest ttid,
# but in either case, _notifyReplicated will be called below,
# fixing the store lock.
if locked == ttid:
del self._store_lock_dict[oid]
lockless = transaction.lockless
# There's nothing to rebase for lockless stores to replicating
# partitions because there's no lock taken yet. In other words,
# rebasing such stores would do nothing. Other lockless stores
# become normal ones: this transaction does not block anymore
# replicated partitions from being marked as UP_TO_DATE.
oid = [oid
for oid in lockless.intersection(transaction.serial_dict)
if self.getPartition(oid) not in self._replicating]
if oid:
lockless.difference_update(oid)
self._notifyReplicated()
# Some locks were released, some pending locks may now succeed.
# We may even have delayed stores for this transaction, like the one
# that triggered the deadlock.
self.executeQueuedEvents()
def rebase(self, conn, ttid, locking_tid):
self.register(conn, ttid)
transaction = self._transaction_dict[ttid]
if transaction.voted:
raise ProtocolError("TXN %s already voted" % dump(ttid))
# First, get a set copy of serial_dict before _rebase locks oids.
lock_set = set(transaction.serial_dict)
self._rebase(transaction, transaction.locking_tid != MAX_TID and ttid,
locking_tid)
if transaction.locking_tid == MAX_TID:
# New deadlock. There's no point rebasing objects now.
return ()
# We return all oids that can't be relocked trivially
# (the client will use RebaseObject for these oids).
lock_set -= transaction.lockless # see comment in _rebase
recheck_set = lock_set.intersection(self._store_lock_dict)
lock_set -= recheck_set
for oid in lock_set:
try:
serial = transaction.serial_dict[oid]
except KeyError:
# An oid was already being rebased and delayed,
# and it got a conflict during the above call to _rebase.
continue
try:
self.lockObject(ttid, serial, oid)
except ConflictError:
recheck_set.add(oid)
return recheck_set
def vote(self, ttid, txn_info=None):
"""
Store transaction information received from client node
......@@ -163,7 +239,9 @@ class TransactionManager(EventQueue):
if txn_info:
user, desc, ext, oid_list = txn_info
txn_info = oid_list, user, desc, ext, False, ttid
transaction.has_trans = True
transaction.voted = 2
else:
transaction.voted = 1
# store metadata to temporary table
dm = self._app.dm
dm.storeTransaction(ttid, object_list, txn_info)
......@@ -183,7 +261,7 @@ class TransactionManager(EventQueue):
transaction.tid = tid
self._load_lock_dict.update(
dict.fromkeys(transaction.store_dict, ttid))
if transaction.has_trans:
if transaction.voted == 2:
self._app.dm.lockTransaction(tid, ttid)
def unlock(self, ttid):
......@@ -217,8 +295,8 @@ class TransactionManager(EventQueue):
DelayEvent
ConflictError
"""
partition = self.getPartition(oid)
if partition in self._replicating:
transaction = self._transaction_dict[ttid]
if self.getPartition(oid) in self._replicating:
# We're out-of-date so maybe:
# - we don't have all data to check for conflicts
# - we missed stores/check that would lock this one
......@@ -226,21 +304,27 @@ class TransactionManager(EventQueue):
# replicate, and we're expected to store it in full.
# Since there's at least 1 other (readable) cell that will do this
# check, we accept this store/check without taking a lock.
transaction.lockless.add(oid)
return
locked = self._store_lock_dict.get(oid)
if locked:
if locked < ttid:
other = self._transaction_dict[locked]
if other < transaction or other.voted:
# We have a bigger "TTID" than locking transaction, so we are
# younger: enter waiting queue so we are handled when lock gets
# released. We also want to delay (instead of conflict) if the
# client is so faster that it is committing another transaction
# before we processed UnlockInformation from the master.
# Or the locking transaction has already voted and there's no
# risk of deadlock if we delay.
logging.info('Store delayed for %r:%r by %r', dump(oid),
dump(ttid), dump(locked))
# A client may have several stores delayed for the same oid
# but this is not a problem. EventQueue processes them in order
# and only the last one will not result in conflicts (that are
# already resolved).
raise DelayEvent
transaction = self._transaction_dict[ttid]
if partition in self._replicated and (
oid in transaction.store_dict or oid in transaction.checked_set):
if oid in transaction.lockless:
# This is a consequence of not having taken a lock during
# replication. After a ConflictError, we may be asked to "lock"
# it again. The current lock is a special one that only delays
......@@ -248,21 +332,36 @@ class TransactionManager(EventQueue):
# For the cluster, we're still out-of-date and like above,
# at least 1 other (readable) cell checks for conflicts.
return
if locked != ttid:
if other is not transaction:
# We have a smaller "TTID" than locking transaction, so we are
# older: this is a possible deadlock case, as we might already
# hold locks the younger transaction is waiting upon. Tell
# client that all involved storage nodes must rebase the
# already locked oids for this transaction.
# hold locks the younger transaction is waiting upon.
logging.info('Possible deadlock on %r:%r with %r',
dump(oid), dump(ttid), dump(locked))
raise ConflictError(MAX_TID)
# Ask master to give the client a new locking tid, which will
# be used to ask all involved storage nodes to rebase the
# already locked oids for this transaction.
self._app.master_conn.notify(Packets.NotifyDeadlock(
ttid, transaction.locking_tid))
self._rebase(transaction, ttid)
raise DelayEvent
# If previous store was an undo, next store must be based on
# undo target.
previous_serial = transaction.store_dict[oid][2]
try:
previous_serial = transaction.store_dict[oid][2]
except KeyError:
# Similarly to below for store, cascaded deadlock for
# checkCurrentSerial is possible because rebase() may return
# oids for which previous rebaseObject are delayed, or being
# received, and the client will bindly resend them.
assert oid in transaction.serial_dict, transaction
logging.info('Transaction %s checking %s more than once',
dump(ttid), dump(oid))
return
if previous_serial is None:
# The only valid case is when the previous undo resulted in a
# resolved conflict.
# 2 valid cases:
# - the previous undo resulted in a resolved conflict
# - cascaded deadlock resolution
# Otherwise, this should not happen. For example, when being
# disconnected by the master because we missed a transaction,
# a conflict may happen after a first store to us, but the
......@@ -272,6 +371,9 @@ class TransactionManager(EventQueue):
logging.info('Transaction %s storing %s more than once',
dump(ttid), dump(oid))
return
elif transaction.locking_tid == MAX_TID:
# Deadlock avoidance. Still no new locking_tid from the client.
raise DelayEvent
else:
previous_serial = self._app.dm.getLastObjectTID(oid)
# Locking before reporting a conflict would speed up the case of
......@@ -292,7 +394,7 @@ class TransactionManager(EventQueue):
except KeyError:
raise NotRegisteredError
self.lockObject(ttid, serial, oid)
transaction.check(oid)
transaction.serial_dict[oid] = serial
def storeObject(self, ttid, serial, oid, compression, checksum, data,
value_serial):
......@@ -304,6 +406,7 @@ class TransactionManager(EventQueue):
except KeyError:
raise NotRegisteredError
self.lockObject(ttid, serial, oid)
transaction.serial_dict[oid] = serial
# store object
if data is None:
data_id = None
......@@ -311,6 +414,42 @@ class TransactionManager(EventQueue):
data_id = self._app.dm.holdData(checksum, data, compression)
transaction.store(oid, data_id, value_serial)
def rebaseObject(self, ttid, oid):
try:
transaction = self._transaction_dict[ttid]
except KeyError:
logging.info('Forget rebase of %s by %s delayed by %s',
dump(oid), dump(ttid), dump(self.getLockingTID(oid)))
return
try:
serial = transaction.serial_dict[oid]
except KeyError:
# There was a previous rebase for this oid, it was still delayed
# during the second RebaseTransaction, and then a conflict was
# reported when another transaction was committed.
logging.info("no oid %s to rebase for transaction %s",
dump(oid), dump(ttid))
return
assert oid not in transaction.lockless, (oid, transaction.lockless)
try:
self.lockObject(ttid, serial, oid)
except ConflictError, e:
# Move the data back to the client for conflict resolution,
# since the client may not have it anymore.
try:
data_id = transaction.store_dict.pop(oid)[1]
except KeyError: # check current
data = None
else:
if data_id is None:
data = None
else:
dm = self._app.dm
data = dm.loadData(data_id)
dm.releaseData([data_id], True)
del transaction.serial_dict[oid]
return serial, e.tid, data
def abort(self, ttid, even_if_locked=False):
"""
Abort a transaction
......@@ -335,27 +474,33 @@ class TransactionManager(EventQueue):
dm.releaseData([x[1] for x in transaction.store_dict.itervalues()],
True)
# unlock any object
for oid in transaction.store_dict, transaction.checked_set:
for oid in oid:
if locked:
lock_ttid = self._load_lock_dict.pop(oid, None)
assert lock_ttid in (ttid, None), ('Transaction %s tried'
' to release the lock on oid %s, but it was held by %s'
% (dump(ttid), dump(oid), dump(lock_ttid)))
try:
write_locking_tid = self._store_lock_dict.pop(oid)
except KeyError:
# Lockless store (we were replicating this partition).
continue
if ttid < write_locking_tid:
# Several lockless stores for this oid and among them,
# a higher ttid is still pending.
assert self.getPartition(oid) in self._replicated, (
oid, ttid, write_locking_tid, self._replicated)
continue
assert write_locking_tid == ttid, ('Inconsistent locking'
' state: aborting %s:%s but %s has the lock.'
% (dump(ttid), dump(oid), dump(write_locking_tid)))
for oid in transaction.serial_dict:
if locked:
lock_ttid = self._load_lock_dict.pop(oid, None)
assert lock_ttid in (ttid, None), ('Transaction %s tried'
' to release the lock on oid %s, but it was held by %s'
% (dump(ttid), dump(oid), dump(lock_ttid)))
try:
write_locking_tid = self._store_lock_dict[oid]
except KeyError:
# Lockless store (we are replicating this partition),
# or unresolved deadlock.
continue
if ttid != write_locking_tid:
if __debug__:
other = self._transaction_dict[write_locking_tid]
x = (oid, ttid, write_locking_tid,
self._replicated, transaction.lockless)
lockless = oid in transaction.lockless
assert oid in other.serial_dict and lockless == (
self.getPartition(oid) in self._replicated), x
if not lockless:
assert not locked, x
continue # unresolved deadlock
# Several lockless stores for this oid and among them,
# a higher ttid is still pending.
assert transaction < other, x
del self._store_lock_dict[oid]
# remove the transaction
del self._transaction_dict[ttid]
if self._replicated:
......
......@@ -23,7 +23,6 @@ import socket
from struct import pack
from neo.lib.util import makeChecksum, u64
from ZODB.FileStorage import FileStorage
from ZODB.POSException import ConflictError
from ZODB.tests.StorageTestBase import zodb_pickle
from persistent import Persistent
from . import NEOCluster, NEOFunctionalTest
......@@ -41,25 +40,6 @@ class Tree(Persistent):
self.right = Tree(depth)
self.left = Tree(depth)
# simple persistent object with conflict resolution
class PCounter(Persistent):
_value = 0
def value(self):
return self._value
def inc(self):
self._value += 1
class PCounterWithResolution(PCounter):
def _p_resolveConflict(self, old, saved, new):
new['_value'] = saved['_value'] + new['_value']
return new
class PObject(Persistent):
pass
......@@ -93,29 +73,6 @@ class ClientTests(NEOFunctionalTest):
conn = self.db.open(transaction_manager=txn)
return (txn, conn)
def testConflictResolutionTriggered1(self):
""" Check that ConflictError is raised on write conflict """
# create the initial objects
self.__setup()
t, c = self.makeTransaction()
c.root()['without_resolution'] = PCounter()
t.commit()
# first with no conflict resolution
t1, c1 = self.makeTransaction()
t2, c2 = self.makeTransaction()
o1 = c1.root()['without_resolution']
o2 = c2.root()['without_resolution']
self.assertEqual(o1.value(), 0)
self.assertEqual(o2.value(), 0)
o1.inc()
o2.inc()
o2.inc()
t1.commit()
self.assertEqual(o1.value(), 1)
self.assertEqual(o2.value(), 2)
self.assertRaises(ConflictError, t2.commit)
def testIsolationAtZopeLevel(self):
""" Check transaction isolation within zope connection """
self.__setup()
......@@ -270,51 +227,6 @@ class ClientTests(NEOFunctionalTest):
db2, conn2 = self.neo.getZODBConnection()
self.runWithTimeout(40, test)
def testDelayedLocksCancelled(self):
"""
Hold a lock on an object, try to get another lock on the same
object to delay it. Then cancel the second transaction and check
that the lock is not hold when the first transaction ends
"""
def test():
self.neo = NEOCluster(['test_neo1'], replicas=0,
temp_dir=self.getTempDirectory())
self.neo.start()
db1, conn1 = self.neo.getZODBConnection()
db2, conn2 = self.neo.getZODBConnection()
st1, st2 = conn1._storage, conn2._storage
t1, t2 = transaction.Transaction(), transaction.Transaction()
t1.user = t2.user = u'user'
t1.description = t2.description = u'desc'
oid = st1.new_oid()
rev = '\0' * 8
data = zodb_pickle(PObject())
st1.tpc_begin(t1)
st2.tpc_begin(t2)
# t1 own the lock
st1.store(oid, rev, data, '', t1)
# t2 store is delayed
st2.store(oid, rev, data, '', t2)
# cancel t2, should cancel the store too
st2.tpc_abort(t2)
# finish t1, should release the lock
st1.tpc_vote(t1)
st1.tpc_finish(t1)
db3, conn3 = self.neo.getZODBConnection()
st3 = conn3._storage
t3 = transaction.Transaction()
t3.user = u'user'
t3.description = u'desc'
st3.tpc_begin(t3)
# retrieve the last revision
data, serial = st3.load(oid)
# try to store again, should not be delayed
st3.store(oid, serial, data, '', t3)
# the vote should not timeout
st3.tpc_vote(t3)
st3.tpc_finish(t3)
self.runWithTimeout(10, test)
def testGreaterOIDSaved(self):
"""
Store an object with an OID greater than the last generated by the
......
......@@ -413,6 +413,9 @@ class ClientApplication(Node, neo.client.app.Application):
def __init__(self, master_nodes, name, **kw):
super(ClientApplication, self).__init__(master_nodes, name, **kw)
self.poll_thread.node_name = name
# Smaller cache to speed up tests that checks behaviour when it's too
# small. See also NEOCluster.cache_size
self._cache._max_size //= 1024
def _run(self):
try:
......@@ -433,6 +436,10 @@ class ClientApplication(Node, neo.client.app.Application):
conn = self.cp.getConnForNode(self.nm.getByUUID(peer.uuid))
yield conn
def extraCellSortKey(self, key):
return Patch(self.cp, getCellSortKey=lambda orig, cell:
(orig(cell), key(cell)))
class NeoCTL(neo.neoctl.app.NeoCTL):
def __init__(self, *args, **kw):
......@@ -885,10 +892,6 @@ class NEOCluster(object):
txn = transaction.TransactionManager()
return txn, (self.db if db is None else db).open(txn)
def extraCellSortKey(self, key):
return Patch(self.client.cp, getCellSortKey=lambda orig, cell:
(orig(cell), key(cell)))
def moduloTID(self, partition):
"""Force generation of TIDs that will be stored in given partition"""
partition = p64(partition)
......@@ -974,6 +977,8 @@ class NEOThreadedTest(NeoTestBase):
self.__exc_info = None
except:
self.__exc_info = sys.exc_info()
if self.__exc_info[0] is NEOThreadedTest.failureException:
traceback.print_exception(*self.__exc_info)
def join(self, timeout=None):
threading.Thread.join(self, timeout)
......@@ -1003,6 +1008,44 @@ class NEOThreadedTest(NeoTestBase):
return Patch(jar.db().storage.app.cp, getConnForNode=lambda orig, node:
None if node.getUUID() == storage.uuid else orig(node))
@staticmethod
def readCurrent(ob):
ob._p_activate()
ob._p_jar.readCurrent(ob)
class ThreadId(list):
def __call__(self):
try:
return self.index(thread.get_ident())
except ValueError:
i = len(self)
self.append(thread.get_ident())
return i
@apply
class RandomConflictDict(dict):
# One must not depend on how Python iterates over dict keys, because this
# is implementation-defined behaviour. This patch makes sure of that when
# resolving conflicts.
def __new__(cls):
from neo.client.transactions import Transaction
def __init__(orig, self, *args):
orig(self, *args)
assert self.conflict_dict == {}
self.conflict_dict = dict.__new__(cls)
return Patch(Transaction, __init__=__init__)
def popitem(self):
try:
k = random.choice(list(self))
except IndexError:
raise KeyError
return k, self.pop(k)
def predictable_random(seed=None):
# Because we have 2 running threads when client works, we can't
......
......@@ -20,6 +20,8 @@ import threading
import time
import transaction
import unittest
from collections import defaultdict
from contextlib import contextmanager
from thread import get_ident
from zlib import compress
from persistent import Persistent, GHOST
......@@ -32,9 +34,10 @@ from neo.lib.exception import DatabaseFailure, StoppedOperation
from neo.lib.handler import DelayEvent
from neo.lib import logging
from neo.lib.protocol import CellStates, ClusterStates, NodeStates, Packets, \
uuid_str, ZERO_OID, ZERO_TID
Packet, uuid_str, ZERO_OID, ZERO_TID
from .. import expectedFailure, Patch, TransactionalResource
from . import ConnectionFilter, LockLock, NEOThreadedTest, with_cluster
from . import ClientApplication, ConnectionFilter, LockLock, NEOThreadedTest, \
RandomConflictDict, ThreadId, with_cluster
from neo.lib.util import add64, makeChecksum, p64, u64
from neo.client.exception import NEOPrimaryMasterLost, NEOStorageError
from neo.client.pool import CELL_CONNECTED, CELL_GOOD
......@@ -325,7 +328,6 @@ class Test(NEOThreadedTest):
self.assertEqual(self._testDeadlockAvoidance([2, 4]),
[DelayEvent, DelayEvent, ConflictError, ConflictError])
@expectedFailure(POSException.ConflictError)
def testDeadlockAvoidance(self):
# This test fail because deadlock avoidance is not fully implemented.
# 0: C1 -> S1
......@@ -334,7 +336,7 @@ class Test(NEOThreadedTest):
# 3: C2 commits
# 4: C1 resolves conflict
self.assertEqual(self._testDeadlockAvoidance([1, 3]),
[DelayEvent, ConflictError, "???" ])
[DelayEvent, DelayEvent, DelayEvent, ConflictError])
@with_cluster()
def testConflictResolutionTriggered2(self, cluster):
......@@ -911,8 +913,7 @@ class Test(NEOThreadedTest):
t2, c2 = cluster.getTransaction(db)
r = c2.root()
r['y'] = None
r['x']._p_activate()
c2.readCurrent(r['x'])
self.readCurrent(r['x'])
# Force the new tid to be even, like the modified oid and
# unlike the oid on which we used readCurrent. Thus we check
# that the node containing only the partition 1 is also
......@@ -1612,5 +1613,373 @@ class Test(NEOThreadedTest):
self.assertPartitionTable(cluster, 'UU')
self.assertEqual(x.value, 6)
@contextmanager
def thread_switcher(self, threads, order, expected):
self.assertGreaterEqual(len(order), len(expected))
thread_id = ThreadId()
l = [threading.Lock() for l in xrange(len(threads)+1)]
l[0].acquire()
end = defaultdict(list)
order = iter(order)
expected = iter(expected)
def sched(orig, *args, **kw):
i = thread_id()
logging.info('%s: %s%r', i, orig.__name__, args)
try:
x = u64(kw['oid'])
except KeyError:
for x in args:
if isinstance(x, Packet):
x = type(x).__name__
break
else:
x = orig.__name__
try:
j = next(order)
except StopIteration:
end[i].append(x)
j = None
try:
while 1:
l.pop().release()
except IndexError:
pass
else:
try:
self.assertEqual(next(expected), x)
except StopIteration:
end[i].append(x)
try:
if callable(j):
with contextmanager(j)(*args, **kw) as j:
return orig(*args, **kw)
else:
return orig(*args, **kw)
finally:
if i != j is not None:
try:
l[j].release()
except threading.ThreadError:
l[j].acquire()
threads[j-1].start()
if x != 'StoreTransaction':
try:
l[i].acquire()
except IndexError:
pass
def _handlePacket(orig, *args):
if isinstance(args[2], Packets.AnswerRebaseTransaction):
return sched(orig, *args)
return orig(*args)
with RandomConflictDict, \
Patch(Transaction, write=sched), \
Patch(ClientApplication, _handlePacket=_handlePacket), \
Patch(ClientApplication, tpc_abort=sched), \
Patch(ClientApplication, tpc_begin=sched), \
Patch(ClientApplication, _askStorageForWrite=sched):
yield end
self.assertFalse(list(expected))
self.assertFalse(list(order))
@with_cluster()
def _testComplexDeadlockAvoidanceWithOneStorage(self, cluster, changes,
order, expected_packets, expected_values,
except2=POSException.ReadConflictError):
t1, c1 = cluster.getTransaction()
r = c1.root()
oids = []
for x in 'abcd':
r[x] = PCounterWithResolution()
t1.commit()
oids.append(u64(r[x]._p_oid))
# The test relies on the implementation-defined behavior that ZODB
# processes oids by order of registration. It's also simpler with
# oids from a=1 to d=4.
self.assertEqual(oids, range(1, 5))
t2, c2 = cluster.getTransaction()
t3, c3 = cluster.getTransaction()
changes(r, c2.root(), c3.root())
threads = map(self.newPausedThread, (t2.commit, t3.commit))
with self.thread_switcher(threads, order, expected_packets) as end:
t1.commit()
if except2 is None:
threads[0].join()
else:
self.assertRaises(except2, threads[0].join)
threads[1].join()
t3.begin()
r = c3.root()
self.assertEqual(expected_values, [r[x].value for x in 'abcd'])
return dict(end)
def testCascadedDeadlockAvoidanceWithOneStorage1(self):
"""
locking tids: t1 < t2 < t3
1. A2 (t2 stores A)
2. B1, c2 (t2 checks C)
3. A3 (delayed), B3 (delayed), D3 (delayed)
4. C1 -> deadlock: B3
5. d2 -> deadlock: A3
locking tids: t3 < t1 < t2
6. t3 commits
7. t2 rebase: conflicts on A and D
8. t1 rebase: new deadlock on C
9. t2 aborts (D non current)
all locks released for t1, which rebases and resolves conflicts
"""
def changes(r1, r2, r3):
r1['b'].value += 1
r1['c'].value += 2
r2['a'].value += 3
self.readCurrent(r2['c'])
self.readCurrent(r2['d'])
r3['a'].value += 4
r3['b'].value += 5
r3['d'].value += 6
x = self._testComplexDeadlockAvoidanceWithOneStorage(changes,
(1, 1, 0, 1, 2, 2, 2, 2, 0, 1, 2, 1, 0, 0, 1, 0, 0, 1),
('tpc_begin', 'tpc_begin', 1, 2, 3, 'tpc_begin', 1, 2, 4, 3, 4,
'StoreTransaction', 'RebaseTransaction', 'RebaseTransaction',
'AnswerRebaseTransaction', 'AnswerRebaseTransaction',
'RebaseTransaction', 'AnswerRebaseTransaction'),
[4, 6, 2, 6])
try:
x[1].remove(1)
except ValueError:
pass
self.assertEqual(x, {0: [2, 'StoreTransaction'], 1: ['tpc_abort']})
def testCascadedDeadlockAvoidanceWithOneStorage2(self):
def changes(r1, r2, r3):
r1['a'].value += 1
r1['b'].value += 2
r1['c'].value += 3
r2['a'].value += 4
r3['b'].value += 5
r3['c'].value += 6
self.readCurrent(r2['c'])
self.readCurrent(r2['d'])
self.readCurrent(r3['d'])
def unlock(orig, *args):
f.remove(rebase)
return orig(*args)
rebase = f.delayAskRebaseTransaction(
Patch(TransactionManager, unlock=unlock))
with ConnectionFilter() as f:
x = self._testComplexDeadlockAvoidanceWithOneStorage(changes,
(0, 1, 1, 0, 1, 2, 2, 2, 2, 0, 1, 2, 1,
0, 0, 0, 0, 0, 0, 1, 1, 0, 1, 1, 1),
('tpc_begin', 1, 'tpc_begin', 1, 2, 3, 'tpc_begin',
2, 3, 4, 3, 4, 'StoreTransaction', 'RebaseTransaction',
'RebaseTransaction', 'AnswerRebaseTransaction'),
[1, 7, 9, 0])
x[0].sort(key=str)
try:
x[1].remove(1)
except ValueError:
pass
self.assertEqual(x, {
0: [2, 3, 'AnswerRebaseTransaction',
'RebaseTransaction', 'StoreTransaction'],
1: ['AnswerRebaseTransaction','RebaseTransaction',
'AnswerRebaseTransaction', 'tpc_abort'],
})
def testCascadedDeadlockAvoidanceOnCheckCurrent(self):
def changes(*r):
for r in r:
r['a'].value += 1
self.readCurrent(r['b'])
self.readCurrent(r['c'])
def tic_t1(*args, **kw):
yield 0
self.tic()
end = self._testComplexDeadlockAvoidanceWithOneStorage(changes,
(0, 1, 1, 0, 1, 1, 0, 0, 2, 2, 2, 2, 1, tic_t1, 0),
('tpc_begin', 1) * 2, [3, 0, 0, 0], None)
self.assertLessEqual(2, end[0].count('RebaseTransaction'))
def testFailedConflictOnBigValueDuringDeadlockAvoidance(self):
def changes(r1, r2, r3):
r1['b'].value = 1
r1['d'].value = 2
r2['a'].value = '*' * r2._p_jar.db().storage._cache._max_size
r2['b'].value = 3
r2['c'].value = 4
r3['a'].value = 5
self.readCurrent(r3['c'])
self.readCurrent(r3['d'])
with ConnectionFilter() as f:
x = self._testComplexDeadlockAvoidanceWithOneStorage(changes,
(1, 1, 1, 2, 2, 2, 1, 2, 2, 0, 0, 1, 1, 1, 0),
('tpc_begin', 'tpc_begin', 1, 2, 'tpc_begin', 1, 3, 3, 4,
'StoreTransaction', 2, 4, 'RebaseTransaction',
'AnswerRebaseTransaction', 'tpc_abort'),
[5, 1, 0, 2], POSException.ConflictError)
self.assertEqual(x, {0: ['StoreTransaction']})
@with_cluster(replicas=1, partitions=4)
def testNotifyReplicated(self, cluster):
s0, s1 = cluster.storage_list
s1.stop()
cluster.join((s1,))
s1.resetNode()
t1, c1 = cluster.getTransaction()
r = c1.root()
for x in 'abcd':
r[x] = PCounterWithResolution()
t1.commit()
t3, c3 = cluster.getTransaction()
r['c'].value += 1
t1.commit()
r['b'].value += 2
r['a'].value += 3
t2, c2 = cluster.getTransaction()
r = c2.root()
r['a'].value += 4
r['c'].value += 5
r['d'].value += 6
r = c3.root()
r['c'].value += 7
r['a'].value += 8
r['b'].value += 9
t4, c4 = cluster.getTransaction()
r = c4.root()
r['d'].value += 10
threads = map(self.newPausedThread, (t2.commit, t3.commit, t4.commit))
def t3_c(*args, **kw):
yield 1
# We want to resolve the conflict before storing A.
self.tic()
def t3_resolve(*args, **kw):
self.assertPartitionTable(cluster, 'UO|UO|UO|UO')
f.remove(delay)
self.tic()
self.assertPartitionTable(cluster, 'UO|UO|UU|UO')
yield
def t1_rebase(*args, **kw):
self.tic()
self.assertPartitionTable(cluster, 'UO|UU|UU|UO')
yield
def t3_b(*args, **kw):
yield 1
self.tic()
self.assertPartitionTable(cluster, 'UO|UU|UU|UU')
def t4_vote(*args, **kw):
self.tic()
self.assertPartitionTable(cluster, 'UU|UU|UU|UU')
yield 0
with ConnectionFilter() as f, \
self.thread_switcher(threads,
(1, 2, 3, 0, 1, 0, 2, t3_c, 1, 3, 2, t3_resolve, 0, 0, 0,
t1_rebase, 2, t3_b, 3, t4_vote),
('tpc_begin', 'tpc_begin', 'tpc_begin', 'tpc_begin', 2, 1, 1,
3, 3, 4, 4, 3, 1, 'RebaseTransaction', 'RebaseTransaction',
'AnswerRebaseTransaction', 'AnswerRebaseTransaction', 2
)) as end:
delay = f.delayAskFetchTransactions()
s1.start()
self.tic()
t1.commit()
for t in threads:
t.join()
t4.begin()
self.assertEqual([15, 11, 13, 16], [r[x].value for x in 'abcd'])
self.assertEqual([2, 2], map(end.pop(2).count,
['RebaseTransaction', 'AnswerRebaseTransaction']))
self.assertEqual(end, {
0: [1, 'StoreTransaction'],
1: ['StoreTransaction'],
3: [4, 'StoreTransaction'],
})
@with_cluster(storage_count=2, partitions=2)
def testDeadlockAvoidanceBeforeInvolvingAnotherNode(self, cluster):
t1, c1 = cluster.getTransaction()
r = c1.root()
for x in 'abc':
r[x] = PCounterWithResolution()
t1.commit()
r['a'].value += 1
r['c'].value += 2
r['b'].value += 3
t2, c2 = cluster.getTransaction()
r = c2.root()
r['c'].value += 4
r['a'].value += 5
r['b'].value += 6
t = self.newPausedThread(t2.commit)
def t1_b(*args, **kw):
yield 1
self.tic()
with self.thread_switcher((t,), (1, 0, 1, 0, t1_b, 0, 0, 0, 1),
('tpc_begin', 'tpc_begin', 1, 3, 3, 1, 'RebaseTransaction',
2, 'AnswerRebaseTransaction')) as end:
t1.commit()
t.join()
t2.begin()
self.assertEqual([6, 9, 6], [r[x].value for x in 'abc'])
self.assertEqual([2, 2], map(end.pop(1).count,
['RebaseTransaction', 'AnswerRebaseTransaction']))
self.assertEqual(end, {0: ['AnswerRebaseTransaction',
'StoreTransaction', 'VoteTransaction']})
@with_cluster(replicas=1)
def testConflictAfterDeadlockWithSlowReplica1(self, cluster,
slow_rebase=False):
t1, c1 = cluster.getTransaction()
r = c1.root()
for x in 'ab':
r[x] = PCounterWithResolution()
t1.commit()
r['a'].value += 1
r['b'].value += 2
s1 = cluster.storage_list[1]
with cluster.newClient(1) as db, \
(s1.filterConnection(cluster.client) if slow_rebase else
cluster.client.filterConnection(s1)) as f, \
cluster.client.extraCellSortKey(lambda cell:
cell.getUUID() == s1.uuid):
t2, c2 = cluster.getTransaction(db)
r = c2.root()
r['a'].value += 3
self.readCurrent(r['b'])
t = self.newPausedThread(t2.commit)
def tic_t1(*args, **kw):
yield 0
self.tic()
def tic_t2(*args, **kw):
yield 1
self.tic()
def load(orig, *args, **kw):
f.remove(delayStore)
return orig(*args, **kw)
order = [tic_t2, 0, tic_t2, 1, tic_t1, 0, 0, 0, 1, tic_t1, 0]
def t1_resolve(*args, **kw):
yield
f.remove(delay)
if slow_rebase:
order.append(t1_resolve)
delay = f.delayAnswerRebaseObject()
else:
order[-1] = t1_resolve
delay = f.delayAskStoreObject()
with self.thread_switcher((t,), order,
('tpc_begin', 'tpc_begin', 1, 1, 2, 2, 'RebaseTransaction',
'RebaseTransaction', 'AnswerRebaseTransaction',
'StoreTransaction')) as end:
t1.commit()
t.join()
self.assertNotIn(delay, f)
t2.begin()
end[0].sort(key=str)
self.assertEqual(end, {0: [1, 'AnswerRebaseTransaction',
'StoreTransaction']})
self.assertEqual([4, 2], [r[x].value for x in 'ab'])
def testConflictAfterDeadlockWithSlowReplica2(self):
self.testConflictAfterDeadlockWithSlowReplica1(True)
if __name__ == "__main__":
unittest.main()
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