Commit b0734a4c authored by Kirill Smelkov's avatar Kirill Smelkov

Merge branch 'master' into t

* master: (76 commits)
  Silently ignore answers to packets that aren't ignored on closed connection
  Do not process any packet for aborted connections
  Fix sorting of delayed events
  Do never parse any packet from aborted connection
  fixup! Add file descriptor and aborted flag to __repr__ of connections
  fixup! qa: add a basic assertion in Patch to detect when patched code changes
  qa: fix occasional deadlock when starting subprocesses in functional tests
  client: Add support for zodburi
  bug: on exist/crash, storage space for non-voted data may be leaked
  storage: fix commit activity when cells are discarded or when they become readable
  client: speed up cell sorting on read-access
  qa: new ConnectionFilter.retry()
  Fix race when tweak touches partitions that are being reported as replicated
  qa: add a basic assertion in Patch to detect when patched code changes
  Forbid read-accesses to cells that are actually non-readable
  Fix potential EMFILE when retrying to connect indefinitely
  The partition table must forget dropped nodes
  master: make sure that storage nodes have an up-to-date PT/NM when they're added
  In STOPPING cluster state, really wait for all transaction to be finished
  master: fix random crashes on shutdown when using several master nodes
  ...
parents e8f81e9d 4a82657b
......@@ -4,43 +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()
-------------------------------------------------------------------------
Storage nodes are only queried once at most and if all (for the requested
partition) failed, the client raises instead of asking the master whether it
had an up-to-date partition table (and retry if useful).
In the case of undoLog(), incomplete results may be returned.
(N) Storage does not discard answers from aborted replications
--------------------------------------------------------------
In some cases, this can lead to data corruption (wrong AnswerFetch*) or crashes
(e.g. KeyError because self.current_partition is None at the beginning of
Replicator.fetchObjects).
The assumption that aborting the replication of a partition implies the closure
of the connection turned out to be wrong, e.g. when a partition is aborted by a
third party, like CORRUPTED/DISCARDED event from the master.
Workaround: do not replicate or tweak while checking replicas.
Currently, this can be reproduced by running testBackupNodeLost
(neo.tests.threaded.testReplication.ReplicationTests) many times.
(N) A backup cell may be wrongly marked as corrupted while checking replicas
----------------------------------------------------------------------------
......
......@@ -206,8 +206,9 @@ lists:
users discussion
- `neo-dev <http://mail.tiolive.com/mailman/listinfo/neo-dev>`_:
developers discussion
- `neo-report <http://mail.tiolive.com/mailman/listinfo/neo-report>`_:
automated test results (read-only list)
Automated test results are published at
https://www.erp5.com/quality/integration/P-ERP5.Com.Unit%20Tests/Base_viewListMode?proxy_form_id=WebSection_viewERP5UnitTestForm&proxy_field_id=listbox&proxy_field_selection_name=WebSection_viewERP5UnitTestForm_listbox_selection&reset=1&listbox_title=NEO-%25
Commercial Support
==================
......
......@@ -15,16 +15,13 @@
General
- Review XXX/TODO code tags (CODE)
- Coverage for functional tests (i.e. collect results from subprocesses)
- When all cells are OUT_OF_DATE in backup mode, the one with most data
could become UP_TO_DATE with appropriate backup_tid, so that the cluster
stays operational. (FEATURE)
- Finish renaming UUID into NID everywhere (CODE)
- Implements delayed connection acceptation.
- Delayed connection acceptation even when a storage node is not ready ?
Currently, any node that connects too early to another that is busy for
some reasons is immediately rejected with the 'not ready' error code. This
should be replaced by a queue in the listening node that keep a pool a
nodes that will be accepted late, when the conditions will be satisfied.
some reasons is immediately rejected with the 'not ready' error code.
This is mainly the case for :
- Client rejected before the cluster is operational
- Empty storages rejected during recovery process
......@@ -41,36 +38,20 @@
- Clarify handler methods to call when a connection is accepted from a
listening conenction and when remote node is identified
(cf. neo/lib/bootstrap.py).
- Choose how to handle a storage integrity verification when it comes back.
Do the replication process, the verification stage, with or without
unfinished transactions, cells have to set as outdated, if yes, should the
partition table changes be broadcasted ? (BANDWITH, SPEED)
- Make SIGINT on primary master change cluster in STOPPING state.
- Review PENDING/HIDDEN/SHUTDOWN states, don't use notifyNodeInformation()
to do a state-switch, use a exception-based mechanism ? (CODE)
- Review handler split (CODE)
The current handler split is the result of small incremental changes. A
global review is required to make them square.
- Review node notifications. Eg. A storage don't have to be notified of new
clients but only when one is lost.
- Review transactional isolation of various methods
Some methods might not implement proper transaction isolation when they
should. An example is object history (undoLog), which can see data
committed by future transactions.
- Add a 'devid' storage configuration so that master do not distribute
replicated partitions on storages with same 'devid'.
- Fix race conditions between client-to-storage operations and moved
partitions. Currently, reads succeed because feeding nodes don't delete
anything while the cluster is operational, for performance reasons:
deletion of dropped partitions must be reimplemented in a scalable way.
(HIGH AVAILABILITY)
Storage
- Use libmysqld instead of a stand-alone MySQL server.
- Notify master when storage becomes available for clients (LATENCY)
Currently, storage presence is broadcasted to client nodes too early, as
the storage node would refuse them until it has only up-to-date data (not
only up-to-date cells, but also a partition table and node states).
- In backup mode, 2 simultaneous replication should be possible so that: NOTE
- outdated cells does not block backup for too long time
- constantly modified partitions does not prevent outdated cells to
......@@ -78,9 +59,7 @@
Current behaviour is undefined and the above 2 scenarios may happen.
- Create a specialized PartitionTable that know the database and replicator
to remove duplicates and remove logic from handlers (CODE)
- Consider insert multiple objects at time in the database, with taking care
of maximum SQL request size allowed. (SPEED)
- Make listening address and port optionnal, and if they are not provided
- Make listening address and port optional, and if they are not provided
listen on all interfaces on any available port.
- Make replication speed configurable (HIGH AVAILABILITY)
In its current implementation, replication runs at lowest priority, to
......@@ -125,15 +104,11 @@
instead of parsing the whole partition table. (SPEED)
Client
- Race conditions on the partition table ?
(update by the poll thread vs. access by other threads)
- Merge Application into Storage (SPEED)
- Optimize cache.py by rewriting it either in C or Cython (LOAD LATENCY)
- Use generic bootstrap module (CODE)
- If too many storage nodes are dead, the client should check the partition
table hasn't changed by pinging the master and retry if necessary.
- Implement IStorageRestoreable (ZODB API) in order to preserve data
serials (i.e. undo information).
- Fix and reenable deadlock avoidance (SPEED). This is required for NOTE
neo.threaded.test.Test.testDeadlockAvoidance
Admin
- Make admin node able to monitor multiple clusters simultaneously
......@@ -141,6 +116,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
......
......@@ -58,7 +58,8 @@ partitions: 12
# - MySQL: [user[:password]@]database[unix_socket]
# Database must be created manually.
# - SQLite: path
# engine: Optionnal parameter for MySQL. Can be InnoDB (default) or TokuDB.
# engine: Optionnal parameter for MySQL.
# Can be InnoDB (default), RocksDB or TokuDB.
# Admin node
[admin]
......@@ -67,7 +68,7 @@ bind: 127.0.0.1:9999
# common section.
;logfile: ~/log/admin.log
# Nodes can have their own certicates.
# Nodes can have their own certificates.
;cert = admin.crt
;key = admin.key
......
......@@ -139,6 +139,6 @@ class Application(BaseApplication):
pass
row_list.append((offset, row))
except IndexError:
conn.notify(Errors.ProtocolError('invalid partition table offset'))
conn.send(Errors.ProtocolError('invalid partition table offset'))
else:
conn.answer(Packets.AnswerPartitionList(self.pt.getID(), row_list))
......@@ -89,16 +89,16 @@ class Storage(BaseStorage.BaseStorage,
"""
Note: never blocks in NEO.
"""
return self.app.tpc_begin(transaction, tid, status)
return self.app.tpc_begin(self, transaction, tid, status)
def tpc_vote(self, transaction):
return self.app.tpc_vote(transaction, self.tryToResolveConflict)
return self.app.tpc_vote(transaction)
def tpc_abort(self, transaction):
return self.app.tpc_abort(transaction)
def tpc_finish(self, transaction, f=None):
return self.app.tpc_finish(transaction, self.tryToResolveConflict, f)
return self.app.tpc_finish(transaction, f)
def store(self, oid, serial, data, version, transaction):
assert version == '', 'Versions are not supported'
......@@ -128,7 +128,7 @@ class Storage(BaseStorage.BaseStorage,
# undo
def undo(self, transaction_id, txn):
return self.app.undo(transaction_id, txn, self.tryToResolveConflict)
return self.app.undo(transaction_id, txn)
def undoLog(self, first=0, last=-20, filter=None):
return self.app.undoLog(first, last, filter)
......@@ -167,8 +167,7 @@ class Storage(BaseStorage.BaseStorage,
def importFrom(self, source, start=None, stop=None, preindex=None):
""" Allow import only a part of the source storage """
return self.app.importFrom(source, start, stop,
self.tryToResolveConflict, preindex)
return self.app.importFrom(self, source, start, stop, preindex)
def pack(self, t, referencesf, gc=False):
if gc:
......
......@@ -16,13 +16,10 @@
from cPickle import dumps, loads
from zlib import compress, decompress
from random import shuffle
import heapq
import time
from functools import partial
from ZODB.POSException import UndoError, StorageTransactionError, ConflictError
from ZODB.POSException import ReadConflictError
from ZODB.POSException import UndoError, ConflictError, ReadConflictError
from . import OLD_ZODB
if OLD_ZODB:
from ZODB.ConflictResolution import ResolvedSerial
......@@ -32,15 +29,15 @@ from neo.lib import logging
from neo.lib.protocol import NodeTypes, Packets, \
INVALID_PARTITION, MAX_TID, ZERO_HASH, ZERO_TID
from neo.lib.util import makeChecksum, dump
from neo.lib.locking import Empty, Lock, SimpleQueue
from neo.lib.locking import Empty, Lock
from neo.lib.connection import MTClientConnection, ConnectionClosed
from .exception import NEOStorageError, NEOStorageCreationUndoneError
from .exception import NEOStorageNotFoundError
from .exception import (NEOStorageError, NEOStorageCreationUndoneError,
NEOStorageReadRetry, NEOStorageNotFoundError, NEOPrimaryMasterLost)
from .handlers import storage, master
from neo.lib.dispatcher import ForgottenPacket
from neo.lib.threaded_app import ThreadedApplication
from .cache import ClientCache
from .pool import ConnectionPool
from .transactions import TransactionContainer
from neo.lib.util import p64, u64, parseMasterList
CHECKED_SERIAL = object()
......@@ -54,44 +51,14 @@ if SignalHandler:
SignalHandler.registerHandler(signal.SIGUSR2, logging.reopen)
class TransactionContainer(dict):
def pop(self, txn):
return dict.pop(self, id(txn), None)
def get(self, txn):
try:
return self[id(txn)]
except KeyError:
raise StorageTransactionError("unknown transaction %r" % txn)
def new(self, txn):
key = id(txn)
if key in self:
raise StorageTransactionError("commit of transaction %r"
" already started" % txn)
context = self[key] = {
'queue': SimpleQueue(),
'txn': txn,
'ttid': None,
'data_dict': {},
'data_size': 0,
'cache_dict': {},
'cache_size': 0,
'object_base_serial_dict': {},
'object_serial_dict': {},
'object_stored_counter_dict': {},
'conflict_serial_dict': {},
'resolved_conflict_serial_dict': {},
'involved_nodes': set(),
'checked_nodes': set(),
}
return context
class Application(ThreadedApplication):
"""The client node application."""
# For tests only. Do not touch. We want tpc_finish to always recover when
# the transaction is really committed, no matter for how long the master
# is unreachable.
max_reconnection_to_master = float('inf')
def __init__(self, master_nodes, name, compress=True, **kw):
super(Application, self).__init__(parseMasterList(master_nodes),
name, **kw)
......@@ -165,13 +132,11 @@ class Application(ThreadedApplication):
conn, packet, kw = get(block)
except Empty:
break
if packet is None or isinstance(packet, ForgottenPacket):
# connection was closed or some packet was forgotten
continue
block = False
try:
_handlePacket(conn, packet, kw)
except ConnectionClosed:
except (ConnectionClosed, NEOStorageReadRetry):
# We also catch NEOStorageReadRetry for ObjectUndoSerial.
pass
def _waitAnyTransactionMessage(self, txn_context, block=True):
......@@ -179,13 +144,15 @@ class Application(ThreadedApplication):
Just like _waitAnyMessage, but for per-transaction exchanges, rather
than per-thread.
"""
queue = txn_context['queue']
queue = txn_context.queue
self.setHandlerData(txn_context)
try:
self._waitAnyMessage(queue, block=block)
finally:
# Don't leave access to thread context, even if a raise happens.
self.setHandlerData(None)
if txn_context.conflict_dict:
self._handleConflicts(txn_context)
def _askStorage(self, conn, packet, **kw):
""" Send a request to a storage node and process its answer """
......@@ -217,12 +184,14 @@ class Application(ThreadedApplication):
logging.debug('connecting to primary master...')
self.start()
index = -1
fail_count = 0
ask = self._ask
handler = self.primary_bootstrap_handler
while 1:
self.ignore_invalidations = True
# Get network connection to primary master
while 1:
while fail_count < self.max_reconnection_to_master:
self.nm.reset()
if self.primary_master_node is not None:
# If I know a primary master node, pinpoint it.
self.trying_master_node = self.primary_master_node
......@@ -242,11 +211,15 @@ class Application(ThreadedApplication):
try:
ask(conn, p, handler=handler)
except ConnectionClosed:
fail_count += 1
continue
# If we reached the primary master node, mark as connected
if self.primary_master_node is not None and \
self.primary_master_node is self.trying_master_node:
break
else:
raise NEOPrimaryMasterLost(
"Too many connection failures to the primary master")
logging.info('Connected to %s', self.primary_master_node)
try:
# Request identification and required informations to be
......@@ -260,6 +233,7 @@ class Application(ThreadedApplication):
except ConnectionClosed:
logging.error('Connection to %s lost', self.trying_master_node)
self.primary_master_node = None
fail_count += 1
logging.info("Connected and ready")
return conn
......@@ -290,6 +264,40 @@ class Application(ThreadedApplication):
# return the last OID used, this is inaccurate
return int(u64(self.last_oid))
def _askStorageForRead(self, object_id, packet, askStorage=None):
cp = self.cp
pt = self.pt
if type(object_id) is str:
object_id = pt.getPartition(object_id)
if askStorage is None:
askStorage = self._askStorage
# Failure condition with minimal overhead: most of the time, only the
# following line is executed. In case of storage errors, we retry each
# node at least once, without looping forever.
failed = 0
while 1:
cell_list = pt.getCellList(object_id, True)
cell_list.sort(key=cp.getCellSortKey)
for cell in cell_list:
node = cell.getNode()
conn = cp.getConnForNode(node)
if conn is not None:
try:
return askStorage(conn, packet)
except ConnectionClosed:
pass
except NEOStorageReadRetry, e:
if e.args[0]:
continue
failed += 1
if not pt.filled():
raise NEOPrimaryMasterLost
if len(cell_list) < failed: # too many failures
raise NEOStorageError('no storage available')
# Do not retry too quickly, for example
# when there's an incoming PT update.
self.sync()
def load(self, oid, tid=None, before_tid=None):
"""
Internal method which manage load, loadSerial and loadBefore.
......@@ -365,23 +373,20 @@ class Application(ThreadedApplication):
return data, tid, next_tid
def _loadFromStorage(self, oid, at_tid, before_tid):
packet = Packets.AskObject(oid, at_tid, before_tid)
for node, conn in self.cp.iterateForObject(oid, readable=True):
try:
tid, next_tid, compression, checksum, data, data_tid \
= self._askStorage(conn, packet)
except ConnectionClosed:
continue
def askStorage(conn, packet):
tid, next_tid, compression, checksum, data, data_tid \
= self._askStorage(conn, packet)
if data or checksum != ZERO_HASH:
if checksum != makeChecksum(data):
logging.error('wrong checksum from %s for oid %s',
conn, dump(oid))
continue
raise NEOStorageReadRetry(False)
return (decompress(data) if compression else data,
tid, next_tid, data_tid)
raise NEOStorageCreationUndoneError(dump(oid))
raise NEOStorageError("storage down or corrupted data")
return self._askStorageForRead(oid,
Packets.AskObject(oid, at_tid, before_tid),
askStorage)
def _loadFromCache(self, oid, at_tid=None, before_tid=None):
"""
......@@ -393,7 +398,7 @@ class Application(ThreadedApplication):
return result
return self._cache.load(oid, before_tid)
def tpc_begin(self, transaction, tid=None, status=' '):
def tpc_begin(self, storage, transaction, tid=None, status=' '):
"""Begin a new transaction."""
# First get a transaction, only one is allowed at a time
txn_context = self._txn_container.new(transaction)
......@@ -402,16 +407,18 @@ class Application(ThreadedApplication):
if answer_ttid is None:
raise NEOStorageError('tpc_begin failed')
assert tid in (None, answer_ttid), (tid, answer_ttid)
txn_context['ttid'] = answer_ttid
txn_context.Storage = storage
txn_context.ttid = answer_ttid
def store(self, oid, serial, data, version, transaction):
"""Store object."""
logging.debug('storing oid %s serial %s', dump(oid), dump(serial))
if not serial: # BBB
serial = ZERO_TID
self._store(self._txn_container.get(transaction), oid, serial, data)
def _store(self, txn_context, oid, serial, data, data_serial=None,
unlock=False):
ttid = txn_context['ttid']
def _store(self, txn_context, oid, serial, data, data_serial=None):
ttid = txn_context.ttid
if data is None:
# This is some undo: either a no-data object (undoing object
# creation) or a back-pointer to an earlier revision (going back to
......@@ -433,73 +440,35 @@ class Application(ThreadedApplication):
compression = 0
compressed_data = data
checksum = makeChecksum(compressed_data)
txn_context['data_size'] += size
on_timeout = partial(
self.onStoreTimeout,
txn_context=txn_context,
oid=oid,
)
txn_context.data_size += size
# Store object in tmp cache
txn_context['data_dict'][oid] = data
# Store data on each node
txn_context['object_stored_counter_dict'][oid] = {}
txn_context['object_base_serial_dict'].setdefault(oid, serial)
txn_context['object_serial_dict'][oid] = serial
queue = txn_context['queue']
involved_nodes = txn_context['involved_nodes']
add_involved_nodes = involved_nodes.add
packet = Packets.AskStoreObject(oid, serial, compression,
checksum, compressed_data, data_serial, ttid, unlock)
for node, conn in self.cp.iterateForObject(oid):
try:
conn.ask(packet, on_timeout=on_timeout, queue=queue)
add_involved_nodes(node)
except ConnectionClosed:
continue
if not involved_nodes:
raise NEOStorageError("Store failed")
checksum, compressed_data, data_serial, ttid)
txn_context.data_dict[oid] = data, txn_context.write(
self, packet, oid, oid=oid, serial=serial)
while txn_context['data_size'] >= self._cache._max_size:
while txn_context.data_size >= self._cache._max_size:
self._waitAnyTransactionMessage(txn_context)
self._waitAnyTransactionMessage(txn_context, False)
def onStoreTimeout(self, conn, msg_id, txn_context, oid):
# NOTE: this method is called from poll thread, don't use
# thread-specific value !
txn_context.setdefault('timeout_dict', {})[oid] = msg_id
# Ask the storage if someone locks the object.
# By sending a message with a smaller timeout,
# the connection will be kept open.
conn.ask(Packets.AskHasLock(txn_context['ttid'], oid),
timeout=5, queue=txn_context['queue'])
def _handleConflicts(self, txn_context, tryToResolveConflict):
result = []
append = result.append
# Check for conflicts
data_dict = txn_context['data_dict']
object_base_serial_dict = txn_context['object_base_serial_dict']
object_serial_dict = txn_context['object_serial_dict']
conflict_serial_dict = txn_context['conflict_serial_dict'].copy()
txn_context['conflict_serial_dict'].clear()
resolved_conflict_serial_dict = txn_context[
'resolved_conflict_serial_dict']
for oid, conflict_serial_set in conflict_serial_dict.iteritems():
conflict_serial = max(conflict_serial_set)
serial = object_serial_dict[oid]
if ZERO_TID in conflict_serial_set:
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]
except KeyError:
data = txn_context['cache_dict'][oid]
else:
def _handleConflicts(self, txn_context):
data_dict = txn_context.data_dict
pop_conflict = txn_context.conflict_dict.popitem
resolved_dict = txn_context.resolved_dict
tryToResolveConflict = txn_context.Storage.tryToResolveConflict
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
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,
......@@ -508,68 +477,54 @@ 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.
# XXX: currently, brute-force is implemented: we send
# object data again.
# WARNING: not maintained code
logging.info('Deadlock avoidance triggered on %r:%r',
dump(oid), dump(serial))
for store_oid, store_data in data_dict.iteritems():
store_serial = object_serial_dict[store_oid]
if store_data is CHECKED_SERIAL:
self._checkCurrentSerialInTransaction(txn_context,
store_oid, store_serial)
else:
if store_data is None:
# Some undo
logging.warning('Deadlock avoidance cannot reliably'
' work with undo, this must be implemented.')
conflict_serial = ZERO_TID
break
self._store(txn_context, store_oid, store_serial,
store_data, unlock=True)
else:
continue
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)
if data is CHECKED_SERIAL:
raise ReadConflictError(oid=oid, serials=(conflict_serial,
serial))
# TODO: data can be None if a conflict happens during undo
if data:
txn_context['data_size'] -= len(data)
resolved_serial_set = resolved_conflict_serial_dict.setdefault(
oid, set())
if resolved_serial_set and conflict_serial <= max(
resolved_serial_set):
# A later serial has already been resolved, skip.
resolved_serial_set.update(conflict_serial_set)
continue
txn_context.data_size -= len(data)
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))
logging.info(
'Conflict resolution failed for %s@%s with %s',
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),
dump(conflict_serial))
logging.info(
'Conflict resolution succeeded for %s@%s with %s',
dump(oid), dump(serial), dump(conflict_serial))
# Mark this conflict as resolved
resolved_serial_set.update(conflict_serial_set)
# Base serial changes too, as we resolved a conflict
object_base_serial_dict[oid] = conflict_serial
resolved_dict[oid] = conflict_serial
# Try to store again
self._store(txn_context, oid, conflict_serial, new_data)
append(oid)
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)
return result
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
......@@ -579,106 +534,97 @@ class Application(ThreadedApplication):
while pending(queue):
_waitAnyMessage(queue)
def waitStoreResponses(self, txn_context, tryToResolveConflict):
result = []
append = result.append
resolved_oid_set = set()
update = resolved_oid_set.update
_handleConflicts = self._handleConflicts
queue = txn_context['queue']
conflict_serial_dict = txn_context['conflict_serial_dict']
def waitStoreResponses(self, txn_context):
queue = txn_context.queue
pending = self.dispatcher.pending
_waitAnyTransactionMessage = self._waitAnyTransactionMessage
while pending(queue) or conflict_serial_dict:
# Note: handler data can be overwritten by _handleConflicts
# so we must set it for each iteration.
while pending(queue):
_waitAnyTransactionMessage(txn_context)
if conflict_serial_dict:
conflicts = _handleConflicts(txn_context,
tryToResolveConflict)
if conflicts:
update(conflicts)
# Check for never-stored objects, and update result for all others
for oid, store_dict in \
txn_context['object_stored_counter_dict'].iteritems():
if not store_dict:
logging.error('tpc_store failed')
raise NEOStorageError('tpc_store failed')
elif oid in resolved_oid_set:
append((oid, ResolvedSerial) if OLD_ZODB else oid)
return result
if txn_context.data_dict:
raise NEOStorageError('could not store/check all oids')
def tpc_vote(self, transaction, tryToResolveConflict):
def tpc_vote(self, transaction):
"""Store current transaction."""
txn_context = self._txn_container.get(transaction)
result = self.waitStoreResponses(txn_context, tryToResolveConflict)
ttid = txn_context['ttid']
# Store data on each node
assert not txn_context['data_dict'], txn_context
self.waitStoreResponses(txn_context)
ttid = txn_context.ttid
packet = Packets.AskStoreTransaction(ttid, str(transaction.user),
str(transaction.description), dumps(transaction._extension),
txn_context['cache_dict'])
queue = txn_context['queue']
trans_nodes = []
for node, conn in self.cp.iterateForObject(ttid):
logging.debug("voting transaction %s on %s", dump(ttid),
dump(conn.getUUID()))
txn_context.cache_dict)
queue = txn_context.queue
involved_nodes = txn_context.involved_nodes
# Ask in parallel all involved storage nodes to commit object metadata.
# Nodes that store the transaction metadata get a special packet.
trans_nodes = txn_context.write(self, packet, ttid)
packet = Packets.AskVoteTransaction(ttid)
for uuid, status in involved_nodes.iteritems():
if status == 1 and uuid not in trans_nodes:
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,
# this will happen during tpc_finish.
failed = [node.getUUID()
for node in self.nm.getStorageList()
if node.isRunning() and involved_nodes.get(node.getUUID()) == 2]
if failed:
try:
conn.ask(packet, queue=queue)
self._askPrimary(Packets.FailedVote(ttid, failed))
except ConnectionClosed:
continue
trans_nodes.append(node)
# check at least one storage node accepted
if trans_nodes:
involved_nodes = txn_context['involved_nodes']
packet = Packets.AskVoteTransaction(ttid)
for node in involved_nodes.difference(trans_nodes):
conn = self.cp.getConnForNode(node)
if conn is not None:
try:
conn.ask(packet, queue=queue)
except ConnectionClosed:
pass
involved_nodes.update(trans_nodes)
self.waitResponses(queue)
txn_context['voted'] = None
# We must not go further if connection to master was lost since
# tpc_begin, to lower the probability of failing during tpc_finish.
# IDEA: We can improve in 2 opposite directions:
# - In the case of big transactions, it would be useful to
# also detect failures earlier.
# - If possible, recover from master failure.
if 'error' in txn_context:
raise NEOStorageError(txn_context['error'])
return result
logging.error('tpc_vote failed')
raise NEOStorageError('tpc_vote failed')
pass
txn_context.voted = True
# We must not go further if connection to master was lost since
# tpc_begin, to lower the probability of failing during tpc_finish.
# IDEA: We can improve in 2 opposite directions:
# - In the case of big transactions, it would be useful to
# also detect failures earlier.
# - If possible, recover from master failure.
if txn_context.error:
raise NEOStorageError(txn_context.error)
if OLD_ZODB:
return [(oid, ResolvedSerial)
for oid in txn_context.resolved_dict]
return txn_context.resolved_dict
def tpc_abort(self, transaction):
"""Abort current transaction."""
txn_context = self._txn_container.pop(transaction)
if txn_context is None:
return
p = Packets.AbortTransaction(txn_context['ttid'])
# cancel transaction on all those nodes
nodes = map(self.cp.getConnForNode,
txn_context['involved_nodes'] |
txn_context['checked_nodes'])
nodes.append(self.master_conn)
for conn in nodes:
if conn is not None:
try:
conn.notify(p)
except ConnectionClosed:
pass
# We want that the involved nodes abort a transaction after any
# other packet sent by the client for this transaction. IOW, if we
# already have a connection with a storage node, potentially with
# a pending write, aborting only via the master may lead to a race
# condition. The consequence would be that storage nodes lock oids
# forever.
p = Packets.AbortTransaction(txn_context.ttid, ())
for uuid in txn_context.involved_nodes:
try:
self.cp.connection_dict[uuid].send(p)
except (KeyError, ConnectionClosed):
pass
# Because we want to be sure that the involved nodes are notified,
# we still have to send the full list to the master. Most of the
# time, the storage nodes get 2 AbortTransaction packets, and the
# second one is rarely useful. Another option would be that the
# storage nodes keep a list of aborted transactions, but the
# difficult part would be to avoid a memory leak.
try:
notify = self.master_conn.send
except AttributeError:
pass
else:
try:
notify(Packets.AbortTransaction(txn_context.ttid,
txn_context.involved_nodes))
except ConnectionClosed:
pass
# We don't need to flush queue, as it won't be reused by future
# transactions (deleted on next line & indexed by transaction object
# instance).
self.dispatcher.forget_queue(txn_context['queue'], flush_queue=False)
self.dispatcher.forget_queue(txn_context.queue, flush_queue=False)
def tpc_finish(self, transaction, tryToResolveConflict, f=None):
def tpc_finish(self, transaction, f=None):
"""Finish current transaction
To avoid inconsistencies between several databases involved in the
......@@ -696,19 +642,19 @@ class Application(ThreadedApplication):
if any failure happens.
"""
txn_container = self._txn_container
if 'voted' not in txn_container.get(transaction):
self.tpc_vote(transaction, tryToResolveConflict)
if not txn_container.get(transaction).voted:
self.tpc_vote(transaction)
checked_list = []
self._load_lock_acquire()
try:
# Call finish on master
txn_context = txn_container.pop(transaction)
cache_dict = txn_context['cache_dict']
cache_dict = txn_context.cache_dict
checked_list = [oid for oid, data in cache_dict.iteritems()
if data is CHECKED_SERIAL]
for oid in checked_list:
del cache_dict[oid]
ttid = txn_context['ttid']
ttid = txn_context.ttid
p = Packets.AskFinishTransaction(ttid, cache_dict, checked_list)
try:
tid = self._askPrimary(p, cache_dict=cache_dict, callback=f)
......@@ -732,20 +678,17 @@ class Application(ThreadedApplication):
pass
if tid == MAX_TID:
while 1:
for _, conn in self.cp.iterateForObject(
ttid, readable=True):
try:
return self._askStorage(conn, p)
except ConnectionClosed:
pass
self._getMasterConnection()
try:
return self._askStorageForRead(ttid, p)
except NEOPrimaryMasterLost:
pass
elif tid:
return tid
except Exception:
logging.exception("Failed to get final tid for TXN %s",
dump(ttid))
def undo(self, undone_tid, txn, tryToResolveConflict):
def undo(self, undone_tid, txn):
txn_context = self._txn_container.get(txn)
txn_info, txn_ext = self._getTransactionInformation(undone_tid)
txn_oid_list = txn_info['oids']
......@@ -764,31 +707,40 @@ class Application(ThreadedApplication):
# is)
getCellList = self.pt.getCellList
getCellSortKey = self.cp.getCellSortKey
getConnForCell = self.cp.getConnForCell
getConnForNode = self.cp.getConnForNode
queue = self._thread_container.queue
ttid = txn_context['ttid']
ttid = txn_context.ttid
undo_object_tid_dict = {}
snapshot_tid = p64(u64(self.last_tid) + 1)
for partition, oid_list in partition_oid_dict.iteritems():
cell_list = getCellList(partition, readable=True)
# We do want to shuffle before getting one with the smallest
# key, so that all cells with the same (smallest) key has
# identical chance to be chosen.
shuffle(cell_list)
storage_conn = getConnForCell(min(cell_list, key=getCellSortKey))
storage_conn.ask(Packets.AskObjectUndoSerial(ttid,
snapshot_tid, undone_tid, oid_list),
queue=queue, undo_object_tid_dict=undo_object_tid_dict)
# Wait for all AnswerObjectUndoSerial. We might get OidNotFoundError,
# meaning that objects in transaction's oid_list do not exist any
# longer. This is the symptom of a pack, so forbid undoing transaction
# when it happens.
try:
self.waitResponses(queue)
except NEOStorageNotFoundError:
self.dispatcher.forget_queue(queue)
raise UndoError('non-undoable transaction')
kw = {
'queue': queue,
'partition_oid_dict': partition_oid_dict,
'undo_object_tid_dict': undo_object_tid_dict,
}
while partition_oid_dict:
for partition, oid_list in partition_oid_dict.iteritems():
cell_list = [cell
for cell in getCellList(partition, readable=True)
# Exclude nodes that may have missed previous resolved
# conflicts. For example, if a network failure happened
# only between the client and the storage, the latter would
# still be readable until we commit.
if txn_context.involved_nodes.get(cell.getUUID(), 0) < 2]
storage_conn = getConnForNode(
min(cell_list, key=getCellSortKey).getNode())
storage_conn.ask(Packets.AskObjectUndoSerial(ttid,
snapshot_tid, undone_tid, oid_list),
partition=partition, **kw)
# Wait for all AnswerObjectUndoSerial. We might get
# OidNotFoundError, meaning that objects in transaction's oid_list
# do not exist any longer. This is the symptom of a pack, so forbid
# undoing transaction when it happens.
try:
self.waitResponses(queue)
except NEOStorageNotFoundError:
self.dispatcher.forget_queue(queue)
raise UndoError('non-undoable transaction')
# Send undo data to all storage nodes.
for oid in txn_oid_list:
......@@ -800,7 +752,11 @@ class Application(ThreadedApplication):
# object. This is an undo conflict, try to resolve it.
try:
# Load the latest version we are supposed to see
data = self.load(oid, current_serial)[0]
if current_serial == ttid:
# XXX: see TODO below
data = txn_context.cache_dict[oid]
else:
data = self.load(oid, current_serial)[0]
# Load the version we were undoing to
undo_data = self.load(oid, undo_serial)[0]
except NEOStorageNotFoundError:
......@@ -808,14 +764,20 @@ class Application(ThreadedApplication):
'conflict')
# Resolve conflict
try:
data = tryToResolveConflict(oid, current_serial,
undone_tid, undo_data, data)
data = txn_context.Storage.tryToResolveConflict(
oid, current_serial, undone_tid, undo_data, data)
except ConflictError:
raise UndoError('Some data were modified by a later ' \
'transaction', oid)
undo_serial = None
# TODO: The situation is similar to deadlock avoidance.
# Reenable the cache size limit to avoid OOM when there's
# a huge amount conflicting data, and get the data back
# from the storage when it's not in cache_dict anymore.
txn_context.cache_size = - float('inf')
self._store(txn_context, oid, current_serial, data, undo_serial)
self.waitStoreResponses(txn_context)
return None, txn_oid_list
def _insertMetadata(self, txn_info, extension):
......@@ -824,18 +786,8 @@ class Application(ThreadedApplication):
def _getTransactionInformation(self, tid):
packet = Packets.AskTransactionInformation(tid)
for node, conn in self.cp.iterateForObject(tid, readable=True):
try:
txn_info, txn_ext = self._askStorage(conn, packet)
except ConnectionClosed:
continue
except NEOStorageNotFoundError:
# TID not found
continue
break
else:
raise NEOStorageError('Transaction %r not found' % (tid, ))
return (txn_info, txn_ext)
return self._askStorageForRead(tid,
Packets.AskTransactionInformation(tid))
def undoLog(self, first, last, filter=None, block=0):
# XXX: undoLog is broken
......@@ -856,6 +808,9 @@ class Application(ThreadedApplication):
conn.ask(packet, queue=queue, tid_set=tid_set)
# Wait for answers from all storages.
# TODO: Results are incomplete when readable cells move concurrently
# from one storage to another. We detect when this happens and
# retry.
self.waitResponses(queue)
# Reorder tids
......@@ -884,15 +839,8 @@ class Application(ThreadedApplication):
tid_list = []
# request a tid list for each partition
for offset in xrange(self.pt.getPartitions()):
p = Packets.AskTIDsFrom(start, stop, limit, offset)
for node, conn in self.cp.iterateForObject(offset, readable=True):
try:
r = self._askStorage(conn, p)
break
except ConnectionClosed:
pass
else:
raise NEOStorageError('transactionLog failed')
r = self._askStorageForRead(offset,
Packets.AskTIDsFrom(start, stop, limit, offset))
if r:
tid_list = list(heapq.merge(tid_list, r))
if len(tid_list) >= limit:
......@@ -909,17 +857,10 @@ class Application(ThreadedApplication):
return (tid, txn_list)
def history(self, oid, size=1, filter=None):
# Get history informations for object first
packet = Packets.AskObjectHistory(oid, 0, size)
for node, conn in self.cp.iterateForObject(oid, readable=True):
try:
history_list = self._askStorage(conn, packet)
except ConnectionClosed:
continue
# Now that we have object informations, get txn informations
result = []
# history_list is already sorted descending (by the storage)
for serial, size in history_list:
result = []
# history_list is already sorted descending (by the storage)
for serial, size in self._askStorageForRead(oid, packet):
txn_info, txn_ext = self._getTransactionInformation(serial)
# create history dict
txn_info.pop('id')
......@@ -931,10 +872,9 @@ class Application(ThreadedApplication):
if filter is None or filter(txn_info):
result.append(txn_info)
self._insertMetadata(txn_info, txn_ext)
return result
return result
def importFrom(self, source, start, stop, tryToResolveConflict,
preindex=None):
def importFrom(self, storage, source, start, stop, preindex=None):
# TODO: The main difference with BaseStorage implementation is that
# preindex can't be filled with the result 'store' (tid only
# known after 'tpc_finish'. This method could be dropped if we
......@@ -944,15 +884,15 @@ class Application(ThreadedApplication):
preindex = {}
for transaction in source.iterator(start, stop):
tid = transaction.tid
self.tpc_begin(transaction, tid, transaction.status)
self.tpc_begin(storage, transaction, tid, transaction.status)
for r in transaction:
oid = r.oid
pre = preindex.get(oid)
self.store(oid, pre, r.data, r.version, transaction)
preindex[oid] = tid
conflicted = self.tpc_vote(transaction, tryToResolveConflict)
conflicted = self.tpc_vote(transaction)
assert not conflicted, conflicted
real_tid = self.tpc_finish(transaction, tryToResolveConflict)
real_tid = self.tpc_finish(transaction)
assert real_tid == tid, (real_tid, tid)
from .iterator import iterator
......@@ -983,24 +923,13 @@ class Application(ThreadedApplication):
self._txn_container.get(transaction), oid, serial)
def _checkCurrentSerialInTransaction(self, txn_context, oid, serial):
ttid = txn_context['ttid']
txn_context['object_serial_dict'][oid] = serial
# Placeholders
queue = txn_context['queue']
txn_context['object_stored_counter_dict'][oid] = {}
ttid = txn_context.ttid
# ZODB.Connection performs calls 'checkCurrentSerialInTransaction'
# after stores, and skips oids that have been successfully stored.
assert oid not in txn_context['cache_dict'], (oid, txn_context)
txn_context['data_dict'].setdefault(oid, CHECKED_SERIAL)
checked_nodes = txn_context['checked_nodes']
packet = Packets.AskCheckCurrentSerial(ttid, serial, oid)
for node, conn in self.cp.iterateForObject(oid):
try:
conn.ask(packet, queue=queue)
except ConnectionClosed:
continue
checked_nodes.add(node)
if not checked_nodes:
raise NEOStorageError("checkCurrent failed")
assert oid not in txn_context.cache_dict, oid
assert oid not in txn_context.data_dict, oid
packet = Packets.AskCheckCurrentSerial(ttid, oid, serial)
txn_context.data_dict[oid] = CHECKED_SERIAL, txn_context.write(
self, packet, oid, 0, oid=oid, serial=serial)
self._waitAnyTransactionMessage(txn_context, False)
......@@ -19,6 +19,9 @@ from ZODB import POSException
class NEOStorageError(POSException.StorageError):
pass
class NEOStorageReadRetry(NEOStorageError):
pass
class NEOStorageNotFoundError(NEOStorageError):
pass
......
......@@ -147,10 +147,10 @@ class PrimaryNotificationsHandler(MTEventHandler):
logging.critical(msg)
app.master_conn = None
for txn_context in app.txn_contexts():
txn_context['error'] = msg
txn_context.error = msg
try:
del app.pt
except AttributeError:
app.__dict__.pop('pt').clear()
except KeyError:
pass
app.primary_master_node = None
super(PrimaryNotificationsHandler, self).connectionClosed(conn)
......@@ -158,6 +158,10 @@ class PrimaryNotificationsHandler(MTEventHandler):
def stopOperation(self, conn):
logging.critical("master node ask to stop operation")
def notifyClusterInformation(self, conn, state):
# TODO: on shutdown, abort any transaction that is not voted
logging.info("cluster switching to %s state", state)
def invalidateObjects(self, conn, tid, oid_list):
app = self.app
if app.ignore_invalidations:
......@@ -182,9 +186,9 @@ class PrimaryNotificationsHandler(MTEventHandler):
if self.app.pt.filled(): # XXX wrong
self.app.pt.update(ptid, cell_list, self.app.nm)
def notifyNodeInformation(self, conn, node_list):
def notifyNodeInformation(self, conn, timestamp, node_list):
super(PrimaryNotificationsHandler, self).notifyNodeInformation(
conn, node_list)
conn, timestamp, node_list)
# XXX: 'update' automatically closes DOWN nodes. Do we really want
# to do the same thing for nodes in other non-running states ?
getByUUID = self.app.nm.getByUUID
......@@ -194,6 +198,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 """
......@@ -204,6 +215,10 @@ class PrimaryAnswersHandler(AnswerBaseHandler):
oid_list.reverse()
self.app.new_oid_list = oid_list
def incompleteTransaction(self, conn, message):
raise NEOStorageError("storage nodes for which vote failed can not be"
" disconnected without making the cluster non-operational")
def answerTransactionFinished(self, conn, _, tid):
self.app.setHandlerData(tid)
......
......@@ -14,17 +14,18 @@
# 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 ZODB.POSException import ConflictError
from neo.lib import logging
from neo.lib.protocol import LockState, ZERO_TID
from neo.lib.util import dump
from neo.lib.protocol import Packets, uuid_str
from neo.lib.util import dump, makeChecksum
from neo.lib.exception import NodeNotReady
from neo.lib.handler import MTEventHandler
from . import AnswerBaseHandler
from ..transactions import Transaction
from ..exception import NEOStorageError, NEOStorageNotFoundError
from ..exception import NEOStorageDoesNotExistError
from ..exception import NEOStorageReadRetry, NEOStorageDoesNotExistError
class StorageEventHandler(MTEventHandler):
......@@ -32,7 +33,7 @@ class StorageEventHandler(MTEventHandler):
node = self.app.nm.getByAddress(conn.getAddress())
assert node is not None
self.app.cp.removeConnection(node)
self.app.dispatcher.unregister(conn)
super(StorageEventHandler, self).connectionLost(conn, new_state)
def connectionFailed(self, conn):
# Connection to a storage node failed
......@@ -62,60 +63,98 @@ class StorageAnswersHandler(AnswerBaseHandler):
def answerObject(self, conn, oid, *args):
self.app.setHandlerData(args)
def answerStoreObject(self, conn, conflicting, oid, serial):
def answerStoreObject(self, conn, conflict, oid, serial):
txn_context = self.app.getHandlerData()
object_stored_counter_dict = txn_context[
'object_stored_counter_dict'][oid]
if conflicting:
if conflict:
# Conflicts can not be resolved now because 'conn' is locked.
# We must postpone the resolution (by queuing the conflict in
# 'conflict_dict') to avoid any deadlock with another thread that
# also resolves a conflict successfully to the same storage nodes.
# Warning: if a storage (S1) is much faster than another (S2), then
# we may process entirely a conflict with S1 (i.e. we received the
# answer to the store of the resolved object on S1) before we
# receive the conflict answer from the first store on S2.
logging.info('%r report a conflict for %r with %r',
conn, dump(oid), dump(serial))
logging.info('%s reports a conflict on %s:%s with %s',
uuid_str(conn.getUUID()), dump(oid),
dump(txn_context.ttid), dump(conflict))
# If this conflict is not already resolved, mark it for
# resolution.
if serial not in txn_context[
'resolved_conflict_serial_dict'].get(oid, ()):
if serial in object_stored_counter_dict and serial != ZERO_TID:
raise NEOStorageError('Storages %s accepted object %s'
' for serial %s but %s reports a conflict for it.' % (
map(dump, object_stored_counter_dict[serial]),
dump(oid), dump(serial), dump(conn.getUUID())))
conflict_serial_dict = txn_context['conflict_serial_dict']
conflict_serial_dict.setdefault(oid, set()).add(serial)
if txn_context.resolved_dict.get(oid, '') < conflict:
txn_context.conflict_dict[oid] = serial, conflict
else:
uuid_set = object_stored_counter_dict.get(serial)
if uuid_set is None: # store to first storage node
object_stored_counter_dict[serial] = uuid_set = set()
try:
data = txn_context['data_dict'].pop(oid)
except KeyError: # multiple undo
assert txn_context['cache_dict'][oid] is None, oid
else:
if type(data) is str:
size = len(data)
txn_context['data_size'] -= size
size += txn_context['cache_size']
if size < self.app._cache._max_size:
txn_context['cache_size'] = size
else:
# Do not cache data past cache max size, as it
# would just flush it on tpc_finish. This also
# prevents memory errors for big transactions.
data = None
txn_context['cache_dict'][oid] = data
else: # replica
assert oid not in txn_context['data_dict'], oid
uuid_set.add(conn.getUUID())
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 serial <= txn_context.conflict_dict.get(oid, ('',))[0]:
# Another node already reported this conflict or a newer,
# by answering to this rebase or to the previous store.
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
answerVoteTransaction = answerStoreTransaction
def connectionClosed(self, conn):
txn_context = self.app.getHandlerData()
if type(txn_context) is Transaction:
txn_context.nodeLost(self.app, conn.getUUID())
super(StorageAnswersHandler, self).connectionClosed(conn)
def answerTIDsFrom(self, conn, tid_list):
logging.debug('Get %u TIDs from %r', len(tid_list), conn)
self.app.setHandlerData(tid_list)
......@@ -148,43 +187,17 @@ class StorageAnswersHandler(AnswerBaseHandler):
# This can happen when requiring txn informations
raise NEOStorageNotFoundError(message)
def nonReadableCell(self, conn, message):
logging.info('non readable cell')
raise NEOStorageReadRetry(True)
def answerTIDs(self, conn, tid_list, tid_set):
tid_set.update(tid_list)
def answerObjectUndoSerial(self, conn, object_tid_dict,
undo_object_tid_dict):
def answerObjectUndoSerial(self, conn, object_tid_dict, partition,
partition_oid_dict, undo_object_tid_dict):
del partition_oid_dict[partition]
undo_object_tid_dict.update(object_tid_dict)
def answerFinalTID(self, conn, tid):
self.app.setHandlerData(tid)
def answerHasLock(self, conn, oid, status):
store_msg_id = self.app.getHandlerData()['timeout_dict'].pop(oid)
if status == LockState.GRANTED_TO_OTHER:
# Stop expecting the timed-out store request.
self.app.dispatcher.forget(conn, store_msg_id)
# Object is locked by another transaction, and we have waited until
# timeout. To avoid a deadlock, abort current transaction (we might
# be locking objects the other transaction is waiting for).
raise ConflictError, 'Lock wait timeout for oid %s on %r' % (
dump(oid), conn)
# HasLock design required that storage is multi-threaded so that
# it can answer to AskHasLock while processing store requests.
# This means that the 2 cases (granted to us or nobody) are legitimate,
# either because it gave us the lock but is/was slow to store our data,
# or because the storage took a lot of time processing a previous
# store (and did not even considered our lock request).
# XXX: But storage nodes are still mono-threaded, so they should
# only answer with GRANTED_TO_OTHER (if they reply!), except
# maybe in very rare cases of race condition. Only log for now.
# This also means that most of the time, if the storage is slow
# to process some store requests, HasLock will timeout in turn
# and the connector will be closed.
# Anyway, it's not clear that HasLock requests are useful.
# Are store requests potentially long to process ? If not,
# we should simply raise a ConflictError on store timeout.
logging.info('Store of oid %s delayed (storage overload ?)', dump(oid))
def alreadyPendingError(self, conn, message):
pass
......@@ -14,27 +14,18 @@
# You should have received a copy of the GNU General Public License
# along with this program. If not, see <http://www.gnu.org/licenses/>.
import time
from random import shuffle
import random, time
from neo.lib import logging
from neo.lib.locking import Lock
from neo.lib.protocol import NodeTypes, Packets
from neo.lib.connection import MTClientConnection, ConnectionClosed
from neo.lib.exception import NodeNotReady
from .exception import NEOPrimaryMasterLost, NEOStorageError
from .exception import NEOPrimaryMasterLost
# How long before we might retry a connection to a node to which connection
# failed in the past.
MAX_FAILURE_AGE = 600
# Cell list sort keys
# We are connected to storage node hosting cell, high priority
CELL_CONNECTED = -1
# normal priority
CELL_GOOD = 0
# Storage node hosting cell failed recently, low priority
CELL_FAILED = 1
class ConnectionPool(object):
"""This class manages a pool of connections to storage nodes."""
......@@ -67,55 +58,24 @@ class ConnectionPool(object):
else:
logging.info('Connected %r', node)
return conn
self.notifyFailure(node)
def notifyFailure(self, node):
self.node_failure_dict[node.getUUID()] = time.time() + MAX_FAILURE_AGE
def getCellSortKey(self, cell):
def getCellSortKey(self, cell, random=random.random):
# The use of 'random' suffles cells to randomise node to access.
uuid = cell.getUUID()
# First, prefer a connected node.
if uuid in self.connection_dict:
return CELL_CONNECTED
return random()
# Then one that didn't fail recently.
failure = self.node_failure_dict.get(uuid)
if failure:
if time.time() < failure:
return CELL_FAILED
# At last, order by date of connection failure.
return failure
# Do not use 'del' statement: we didn't lock, so another
# thread might have removed uuid from node_failure_dict.
self.node_failure_dict.pop(uuid, None)
return CELL_GOOD
def getConnForCell(self, cell):
return self.getConnForNode(cell.getNode())
def iterateForObject(self, object_id, readable=False):
""" Iterate over nodes managing an object """
pt = self.app.pt
if type(object_id) is str:
object_id = pt.getPartition(object_id)
cell_list = pt.getCellList(object_id, readable)
if not cell_list:
raise NEOStorageError('no storage available')
getConnForNode = self.getConnForNode
while 1:
new_cell_list = []
# Shuffle to randomise node to access...
shuffle(cell_list)
# ...and sort with non-unique keys, to prioritise ranges of
# randomised entries.
cell_list.sort(key=self.getCellSortKey)
for cell in cell_list:
node = cell.getNode()
conn = getConnForNode(node)
if conn is not None:
yield node, conn
# Re-check if node is running, as our knowledge of its
# state can have changed during connection attempt.
elif node.isRunning():
new_cell_list.append(cell)
if not new_cell_list:
break
cell_list = new_cell_list
if self.app.master_conn is None:
raise NEOPrimaryMasterLost
return 1 + random()
def getConnForNode(self, node):
"""Return a locked connection object to a given node
......
#
# Copyright (C) 2017 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
# as published by the Free Software Foundation; either version 2
# of the License, or (at your option) any later version.
#
# This program is distributed in the hope that it will be useful,
# but WITHOUT ANY WARRANTY; without even the implied warranty of
# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
# GNU General Public License for more details.
#
# 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 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
def __init__(self, txn):
self.queue = SimpleQueue()
self.txn = txn
# data being stored
self.data_dict = {} # {oid: (value, [node_id])}
# data stored: this will go to the cache on tpc_finish
self.cache_dict = {} # {oid: value}
# conflicts to resolve
self.conflict_dict = {} # {oid: (base_serial, serial)}
# resolved conflicts
self.resolved_dict = {} # {oid: serial}
# Keys are node ids instead of Node objects because a node may
# disappear from the cluster. In any case, we always have to check
# if the id is still known by the NodeManager.
# 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
involved = self.involved_nodes
object_id = pt.getPartition(object_id)
for cell in pt.getCellList(object_id):
node = cell.getNode()
uuid = node.getUUID()
status = involved.get(uuid, -1)
if status < store:
involved[uuid] = store
elif status > 1:
continue
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
except ConnectionClosed:
pass
involved[uuid] = 2
if uuid_list:
return uuid_list
raise NEOStorageError(
'no storage available for write to partition %s' % object_id)
def written(self, app, uuid, oid):
# When a node that is being disconnected by the master because it was
# not part of the transaction that caused a conflict, we may receive a
# positive answer (not to be confused with lockless stores) before the
# conflict. Because we have no way to identify such case, we must keep
# the data in self.data_dict until all nodes have answered so we remain
# able to resolve conflicts.
try:
data, uuid_list = self.data_dict[oid]
uuid_list.remove(uuid)
except KeyError:
# 1. store to S1 and S2
# 2. S2 reports a conflict
# 3. store to S1 and S2 # conflict resolution
# 4. S1 does not report a conflict (lockless)
# 5. S2 answers before S1 for the second store
return
except ValueError:
# The most common case for this exception is because nodeLost()
# tries all oids blindly. Other possible cases:
# - like above (KeyError), but with S2 answering last
# - answer to resolved conflict before the first answer from a
# node that was being disconnected by the master
return
if uuid_list:
return
del self.data_dict[oid]
if type(data) is str:
size = len(data)
self.data_size -= size
size += self.cache_size
if size < app._cache._max_size:
self.cache_size = size
else:
# Do not cache data past cache max size, as it
# would just flush it on tpc_finish. This also
# prevents memory errors for big transactions.
data = None
self.cache_dict[oid] = data
def nodeLost(self, app, uuid):
self.involved_nodes[uuid] = 2
for oid in list(self.data_dict):
self.written(app, uuid, oid)
class TransactionContainer(dict):
# IDEA: Drop this container and use the new set_data/data API on
# transactions (requires transaction >= 1.6).
def pop(self, txn):
return dict.pop(self, id(txn), None)
def get(self, txn):
try:
return self[id(txn)]
except KeyError:
raise StorageTransactionError("unknown transaction %r" % txn)
def new(self, txn):
key = id(txn)
if key in self:
raise StorageTransactionError("commit of transaction %r"
" already started" % txn)
context = self[key] = Transaction(txn)
return context
#
# Copyright (C) 2017 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
# as published by the Free Software Foundation; either version 2
# of the License, or (at your option) any later version.
#
# This program is distributed in the hope that it will be useful,
# but WITHOUT ANY WARRANTY; without even the implied warranty of
# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
# GNU General Public License for more details.
#
# You should have received a copy of the GNU General Public License
# along with this program. If not, see <http://www.gnu.org/licenses/>.
"""NEO URI resolver for zodburi
URI format:
neo://name@master1,master2,...,masterN?options
"""
import ZODB.config
import ZConfig
from cStringIO import StringIO
from collections import OrderedDict
from urlparse import urlsplit, parse_qsl
# neo_zconf_options returns set of zconfig options supported by NEO storage
def neo_zconf_options():
neo_schema = """<schema>
<import package="ZODB" />
<import package="neo.client" />
</schema>"""
neo_schema = StringIO(neo_schema)
neo_schema = ZConfig.loadSchemaFile(neo_schema)
neo_storage_zconf = neo_schema.gettype('NeoStorage')
options = {k for k, _ in neo_storage_zconf}
assert 'master_nodes' in options
assert 'name' in options
return options
# canonical_opt_name returns "oPtion_nAme" as "option-name"
def canonical_opt_name(name):
return name.lower().replace('_', '-')
# worker entrypoint for resolve_uri and tests
def _resolve_uri(uri):
scheme, netloc, path, query, frag = urlsplit(uri)
if scheme != "neo":
raise ValueError("invalid uri: %s : expected neo:// scheme" % uri)
if path != "":
raise ValueError("invalid uri: %s : non-empty path" % uri)
if frag != "":
raise ValueError("invalid uri: %s : non-empty fragment" % uri)
# extract master list and name from netloc
name, masterloc = netloc.split('@', 1)
master_list = masterloc.split(',')
neokw = OrderedDict()
neokw['master_nodes'] = ' '.join(master_list)
neokw['name'] = name
# get options from query: only those that are defined by NEO schema go to
# storage - rest are returned as database options
dbkw = {}
neo_options = neo_zconf_options()
for k, v in OrderedDict(parse_qsl(query)).items():
if k in neo_options:
neokw[k] = v
else:
# it might be option for storage, but not in canonical form e.g.
# read_only -> read-only (zodburi world settled on using "_" and
# ZConfig world on "-" as separators)
k2 = canonical_opt_name(k)
if k2 in neo_options:
neokw[k2] = v
# else keep this kv as db option
else:
dbkw[k] = v
# now we have everything. Let ZConfig do actual work for validation options
# and borning the storage
neozconf = """%import neo.client
<NEOStorage>
"""
for k, v in neokw.items():
neozconf += " %s\t%s\n" % (k, v)
neozconf += "</NEOStorage>\n"
return neozconf, dbkw
# resolve_uri resolves uri according to neo:// schema.
# see module docstring for uri format.
def resolve_uri(uri):
neozconf, dbkw = _resolve_uri(uri)
def factory():
return ZODB.config.storageFromString(neozconf)
return factory, dbkw
......@@ -38,6 +38,7 @@ class BootstrapManager(EventHandler):
self.num_replicas = None
self.num_partitions = None
self.current = None
app.nm.reset()
uuid = property(lambda self: self.app.uuid)
......
......@@ -106,7 +106,7 @@ class ConfigurationManager(object):
return self.__get('engine', True)
def getWait(self):
# BUG
# XXX: see also DatabaseManager.__init__
return self.__get('wait')
def getDynamicMasterList(self):
......
......@@ -22,7 +22,7 @@ from .connector import ConnectorException, ConnectorDelayedConnection
from .locking import RLock
from .protocol import uuid_str, Errors, \
PacketMalformedError, Packets, ParserState
from .util import ReadBuffer
from .util import dummy_read_buffer, ReadBuffer
CRITICAL_TIMEOUT = 30
......@@ -115,16 +115,17 @@ class HandlerSwitcher(object):
def _handle(self, connection, packet): # NOTE incoming packet -> handle -> dispatch ...
assert len(self._pending) == 1 or self._pending[0][0]
logging.packet(connection, packet, False)
if connection.isClosed() and packet.ignoreOnClosedConnection():
if connection.isClosed() and (connection.isAborted() or
packet.ignoreOnClosedConnection()):
logging.debug('Ignoring packet %r on closed connection %r',
packet, connection)
return
msg_id = packet.getId()
(request_dict, handler) = self._pending[0]
# notifications are not expected
if not packet.isResponse():
handler.packetReceived(connection, packet)
if not packet.isResponse(): # notification
# XXX: If there are several handlers, which one to use ?
self._pending[0][1].packetReceived(connection, packet)
return
msg_id = packet.getId()
request_dict, handler = self._pending[0]
# checkout the expected answer class
try:
klass, _, _, kw = request_dict.pop(msg_id)
......@@ -137,7 +138,7 @@ class HandlerSwitcher(object):
logging.error('Unexpected answer %r in %r', packet, connection)
if not connection.isClosed():
notification = Packets.Notify('Unexpected answer: %r' % packet)
connection.notify(notification)
connection.send(notification)
connection.abort()
# handler.peerBroken(connection)
# apply a pending handler if no more answers are pending
......@@ -354,7 +355,7 @@ class Connection(BaseConnection):
r.append(('len(queue)', len(self._queue)))
if self._on_close is not None:
r.append(('on_close', getattr(self._on_close, '__name__', '?')))
flags.extend(x for x in ('aborted', 'connecting', 'client', 'server')
flags.extend(x for x in ('connecting', 'client', 'server')
if getattr(self, x))
return r, flags
......@@ -383,7 +384,7 @@ class Connection(BaseConnection):
if self.server:
del self.idle
self.client = False
self.notify(Packets.CloseClient())
self.send(Packets.CloseClient())
else:
self.close()
......@@ -449,6 +450,7 @@ class Connection(BaseConnection):
return
logging.debug('aborting a connector for %r', self)
self.aborted = True
self.read_buf = dummy_read_buffer
if self._on_close is not None:
self._on_close()
self._on_close = None
......@@ -486,8 +488,6 @@ class Connection(BaseConnection):
except PacketMalformedError, e:
logging.error('malformed packet from %r: %s', self, e)
self._closure()
if self.aborted:
self.em.removeReader(self)
return not not self._queue
def hasPendingMessages(self):
......@@ -552,14 +552,12 @@ class Connection(BaseConnection):
self.em.addWriter(self)
logging.packet(self, packet, True)
def notify(self, packet):
def send(self, packet, msg_id=None):
""" Then a packet with a new ID """
if self.isClosed():
raise ConnectionClosed
msg_id = self._getNextId()
packet.setId(msg_id)
packet.setId(self._getNextId() if msg_id is None else msg_id)
self._addPacket(packet)
return msg_id
def ask(self, packet, timeout=CRITICAL_TIMEOUT, on_timeout=None, **kw):
"""
......@@ -580,14 +578,14 @@ class Connection(BaseConnection):
self.em.wakeup()
return msg_id
def answer(self, packet, msg_id=None):
def answer(self, packet):
""" Answer to a packet by re-using its ID for the packet answer """
if self.isClosed():
raise ConnectionClosed
if msg_id is None:
msg_id = self.getPeerId()
packet.setId(msg_id)
assert packet.isResponse(), packet
if self.isClosed():
if packet.ignoreOnClosedConnection() and not packet.isError():
raise ConnectionClosed
return
packet.setId(self.peer_id)
self._addPacket(packet)
def idle(self):
......@@ -678,7 +676,7 @@ class MTConnectionType(type):
if __debug__:
for name in 'answer',:
setattr(cls, name, cls.lockCheckWrapper(name))
for name in 'close', 'notify':
for name in 'close', 'send':
setattr(cls, name, cls.__class__.lockWrapper(cls, name))
for name in ('_delayedConnect', 'onTimeout',
'process', 'readable', 'writable'):
......@@ -725,29 +723,17 @@ class MTClientConnection(ClientConnection):
with lock:
super(MTClientConnection, self).__init__(*args, **kwargs)
# Alias without lock (cheaper than super())
_ask = ClientConnection.ask.__func__
def ask(self, packet, timeout=CRITICAL_TIMEOUT, on_timeout=None,
queue=None, **kw):
with self.lock:
if self.isClosed():
raise ConnectionClosed
# XXX: Here, we duplicate Connection.ask because we need to call
# self.dispatcher.register after setId is called and before
# _addPacket is called.
msg_id = self._getNextId()
packet.setId(msg_id)
if queue is None:
if type(packet) is not Packets.Ping:
raise TypeError, 'Only Ping packet can be asked ' \
'without a queue, got a %r.' % (packet, )
else:
self.dispatcher.register(self, msg_id, queue)
self._addPacket(packet)
handlers = self._handlers
t = None if handlers.isPending() else time()
handlers.emit(packet, timeout, on_timeout, kw)
if not self._queue:
next_timeout = self._next_timeout
self.updateTimeout(t)
if self._next_timeout < next_timeout:
self.em.wakeup()
return msg_id
if type(packet) is Packets.Ping:
return self._ask(packet, timeout, on_timeout, **kw)
raise TypeError('Only Ping packet can be asked'
' without a queue, got a %r.' % packet)
msg_id = self._ask(packet, timeout, on_timeout, **kw)
self.dispatcher.register(self, msg_id, queue)
return msg_id
......@@ -19,17 +19,15 @@ from .locking import Lock, Empty
EMPTY = {}
NOBODY = []
class ForgottenPacket(object):
"""
Instances of this class will be pushed to queue when an expected answer
is being forgotten. Its purpose is similar to pushing "None" when
connection is closed, but the meaning is different.
"""
def __init__(self, msg_id):
self.msg_id = msg_id
def getId(self):
return self.msg_id
@apply
class _ConnectionClosed(object):
handler_method_name = 'connectionClosed'
decode = tuple
class getId(object):
def __eq__(self, other):
return True
def giant_lock(func):
def wrapped(self, *args, **kw):
......@@ -88,7 +86,7 @@ class Dispatcher:
def unregister(self, conn):
""" Unregister a connection and put fake packet in queues to unlock
threads excepting responses from that connection """
threads expecting responses from that connection """
self.lock_acquire()
try:
message_table = self.message_table.pop(id(conn), EMPTY)
......@@ -101,25 +99,10 @@ class Dispatcher:
continue
queue_id = id(queue)
if queue_id not in notified_set:
queue.put((conn, None, None))
queue.put((conn, _ConnectionClosed, EMPTY))
notified_set.add(queue_id)
_decrefQueue(queue)
@giant_lock
def forget(self, conn, msg_id):
""" Forget about a specific message for a specific connection.
Actually makes it "expected by nobody", so we know we can ignore it,
and not detect it as an error. """
message_table = self.message_table[id(conn)]
queue = message_table[msg_id]
if queue is NOBODY:
raise KeyError, 'Already expected by NOBODY: %r, %r' % (
conn, msg_id)
queue.put((conn, ForgottenPacket(msg_id), None))
self.queue_dict[id(queue)] -= 1
message_table[msg_id] = NOBODY
return queue
@giant_lock
def forget_queue(self, queue, flush_queue=True):
"""
......@@ -137,9 +120,7 @@ class Dispatcher:
found += 1
message_table[msg_id] = NOBODY
refcount = self.queue_dict.pop(id(queue), 0)
if refcount != found:
raise ValueError('We hit a refcount bug: %s queue uses ' \
'expected, %s found' % (refcount, found))
assert refcount == found, (refcount, found)
if flush_queue:
get = queue.get
while True:
......
......@@ -131,10 +131,15 @@ class EpollEventManager(object):
else:
self.reader_set.discard(fd)
self.writer_set.discard(fd)
if close:
self._closeAppend(connector.shutdown())
if self._closeAcquire(0):
self._closeRelease()
return
if close:
self._closeAppend(connector.shutdown())
if self._closeAcquire(0):
self._closeRelease()
# The connection is not registered, so do not wait for epoll
# to wake up (which may not even happen, and lead to EMFILE).
connector.shutdown()()
def isIdle(self): # NOTE
return not (self._pending_processing or self.writer_set)
......
......@@ -15,14 +15,21 @@
# along with this program. If not, see <http://www.gnu.org/licenses/>.
import sys
from collections import deque
from operator import itemgetter
from . import logging
from .connection import ConnectionClosed
from .protocol import (
NodeStates, Packets, Errors, BackendNotImplemented,
BrokenNodeDisallowedError, NotReadyError, PacketMalformedError,
ProtocolError, UnexpectedPacketError)
BrokenNodeDisallowedError, NonReadableCell, NotReadyError,
PacketMalformedError, ProtocolError, UnexpectedPacketError)
from .util import cached_property
class DelayEvent(Exception):
pass
class EventHandler(object):
"""This class handles events."""
......@@ -64,6 +71,9 @@ class EventHandler(object):
raise UnexpectedPacketError('no handler found')
args = packet.decode() or ()
method(conn, *args, **kw)
except DelayEvent, e:
assert not kw, kw
self.getEventQueue().queueEvent(method, conn, args, *e.args)
except UnexpectedPacketError, e:
if not conn.isClosed():
self.__unexpectedPacket(conn, packet, *e.args)
......@@ -92,6 +102,8 @@ class EventHandler(object):
conn.answer(Errors.BackendNotImplemented(
"%s.%s does not implement %s"
% (m.im_class.__module__, m.im_class.__name__, m.__name__)))
except NonReadableCell, e:
conn.answer(Errors.NonReadableCell())
except AssertionError:
e = sys.exc_info()
try:
......@@ -165,9 +177,9 @@ class EventHandler(object):
return
conn.close()
def notifyNodeInformation(self, conn, node_list):
def notifyNodeInformation(self, conn, *args):
app = self.app
app.nm.update(app, node_list)
app.nm.update(app, *args)
def ping(self, conn):
conn.answer(Packets.Pong())
......@@ -207,9 +219,6 @@ class EventHandler(object):
def brokenNodeDisallowedError(self, conn, message):
raise RuntimeError, 'broken node disallowed error: %s' % (message,)
def alreadyPendingError(self, conn, message):
logging.error('already pending error: %s', message)
def ack(self, conn, message):
logging.debug("no error message: %s", message)
......@@ -235,7 +244,7 @@ class MTEventHandler(EventHandler):
self.dispatch(conn, packet, kw)
kw = {}
if not (self.dispatcher.dispatch(conn, packet.getId(), packet, kw)
or type(packet) is Packets.Pong):
or type(packet) is Packets.Pong or conn.isClosed()):
raise ProtocolError('Unexpected response packet from %r: %r'
% (conn, packet))
else:
......@@ -264,3 +273,104 @@ class AnswerBaseHandler(EventHandler):
def acceptIdentification(*args):
pass
def connectionClosed(self, conn):
raise ConnectionClosed
class _DelayedConnectionEvent(EventHandler):
# WARNING: This assumes that the connection handler does not change.
handler_method_name = '_func'
__new__ = object.__new__
def __init__(self, func, conn, args):
self._args = args
self._conn = conn
self._func = func
self._msg_id = conn.getPeerId()
def __call__(self):
conn = self._conn
if not conn.isClosed():
msg_id = conn.getPeerId()
try:
self.dispatch(conn, self)
finally:
conn.setPeerId(msg_id)
def __repr__(self):
return '<%s: 0x%x %s>' % (self._func.__name__, self._msg_id, self._conn)
def decode(self):
return self._args
def getEventQueue(self):
raise
def getId(self):
return self._msg_id
class EventQueue(object):
def __init__(self):
self._event_queue = []
self._executing_event = -1
sortQueuedEvents = (lambda key=itemgetter(0): lambda self:
self._event_queue.sort(key=key))()
def queueEvent(self, func, conn=None, args=(), key=None):
assert self._executing_event < 0, self._executing_event
self._event_queue.append((key, func if conn is None else
_DelayedConnectionEvent(func, conn, args)))
if key is not None:
self.sortQueuedEvents()
def sortAndExecuteQueuedEvents(self):
if self._executing_event < 0:
self.sortQueuedEvents()
self.executeQueuedEvents()
else:
# We can't sort events when they're being processed.
self._executing_event = 1
def executeQueuedEvents(self):
# Not reentrant. When processing a queued event, calling this method
# only tells the caller to retry all events from the beginning, because
# events for the same connection must be processed in chronological
# order.
queue = self._event_queue
if queue: # return quickly if the queue is empty
self._executing_event += 1
if self._executing_event:
return
done = []
while 1:
try:
for i, event in enumerate(queue):
try:
event[1]()
done.append(i)
except DelayEvent:
pass
if self._executing_event:
break
else:
break
finally:
while done:
del queue[done.pop()]
self._executing_event = 0
# What sortAndExecuteQueuedEvents could not do immediately
# is done here:
if event[0] is not None:
self.sortQueuedEvents()
self._executing_event = -1
def logQueuedEvents(self):
if self._event_queue:
logging.info(" Pending events:")
for event in self._event_queue:
logging.info(' %r', event)
......@@ -14,13 +14,6 @@
# You should have received a copy of the GNU General Public License
# along with this program. If not, see <http://www.gnu.org/licenses/>.
# WARNING: Log rotating should not be implemented here.
# SQLite does not access database only by file descriptor,
# and an OperationalError exception would be raised if a log is emitted
# between a rename and a reopen.
# Fortunately, SQLite allow multiple process to access the same DB,
# so an external tool should be able to dump and empty tables.
from collections import deque
from functools import wraps
from logging import getLogger, Formatter, Logger, StreamHandler, \
......@@ -215,6 +208,13 @@ class NEOLogger(Logger):
self._setup(filename, reset)
__del__ = setup
def fork(self):
with self:
pid = os.fork()
if pid:
return pid
self._setup()
def isEnabledFor(self, level):
return True
......@@ -246,7 +246,7 @@ class NEOLogger(Logger):
self._emit(record)
self.commit()
else:
self._record_size += RECORD_SIZE + len(record.msg)
self._record_size += RECORD_SIZE + len(record.msg or '')
q = self._record_queue
q.append(record)
if record.levelno < WARNING:
......
......@@ -19,8 +19,9 @@ from os.path import exists, getsize
import json
from . import attributeTracker, logging
from .handler import DelayEvent, EventQueue
from .protocol import formatNodeList, uuid_str, \
NodeTypes, NodeStates, ProtocolError
NodeTypes, NodeStates, NotReadyError, ProtocolError
class Node(object):
......@@ -39,17 +40,20 @@ class Node(object):
self._last_state_change = time()
manager.add(self)
def notify(self, packet):
@property
def send(self):
assert self.isConnected(), 'Not connected'
self._connection.notify(packet)
return self._connection.send
def ask(self, packet, *args, **kw):
@property
def ask(self):
assert self.isConnected(), 'Not connected'
self._connection.ask(packet, *args, **kw)
return self._connection.ask
def answer(self, packet, msg_id=None):
@property
def answer(self):
assert self.isConnected(), 'Not connected'
self._connection.answer(packet, msg_id)
return self._connection.answer
def getLastStateChange(self):
return self._last_state_change
......@@ -232,7 +236,7 @@ class MasterDB(object):
def __iter__(self):
return iter(self._set)
class NodeManager(object):
class NodeManager(EventQueue):
"""This class manages node status."""
_master_db = None
......@@ -255,9 +259,14 @@ class NodeManager(object):
self._master_db = db = MasterDB(master_db)
for addr in db:
self.createMaster(address=addr)
self.reset()
close = __init__
def reset(self):
EventQueue.__init__(self)
self._timestamp = 0
def add(self, node):
if node in self._node_set:
logging.warning('adding a known node %r, ignoring', node)
......@@ -350,10 +359,23 @@ class NodeManager(object):
return self._address_dict.get(address, None)
def getByUUID(self, uuid, *id_timestamp):
""" Return the node that match with a given UUID """
"""Return the node that matches with a given UUID
If an id timestamp is passed, DelayEvent is raised if identification
must be delayed. This is because we rely only on the notifications from
the master to recognize nodes (otherwise, we could get id conflicts)
and such notifications may be late in some cases, even when the master
expects us to not reject the connection.
"""
node = self._uuid_dict.get(uuid)
if not id_timestamp or node and (node.id_timestamp,) == id_timestamp:
return node
if id_timestamp:
id_timestamp, = id_timestamp
if not node or node.id_timestamp != id_timestamp:
if self._timestamp < id_timestamp:
raise DelayEvent
# The peer got disconnected from the master.
raise NotReadyError('unknown by master')
return node
def _createNode(self, klass, address=None, uuid=None, **kw):
by_address = self.getByAddress(address)
......@@ -389,7 +411,9 @@ class NodeManager(object):
def createFromNodeType(self, node_type, **kw):
return self._createNode(NODE_TYPE_MAPPING[node_type], **kw)
def update(self, app, node_list):
def update(self, app, timestamp, node_list):
assert self._timestamp < timestamp, (self._timestamp, timestamp)
self._timestamp = timestamp
node_set = self._node_set.copy() if app.id_timestamp is None else None
for node_type, addr, uuid, state, id_timestamp in node_list:
# This should be done here (although klass might not be used in this
......@@ -427,6 +451,8 @@ class NodeManager(object):
# reconnect to the master because they cleared their
# partition table upon disconnection.
node.getConnection().close()
if app.uuid != uuid:
app.pt.dropNode(node)
self.remove(node)
continue
logging.debug('updating node %r to %s %s %s %s %s',
......@@ -441,14 +467,17 @@ class NodeManager(object):
# For the first notification, we receive a full list of nodes from
# the master. Remove all unknown nodes from a previous connection.
for node in node_set - self._node_set:
app.pt.dropNode(node)
self.remove(node)
self.log()
self.executeQueuedEvents()
def log(self):
logging.info('Node manager : %u nodes', len(self._node_set))
if self._node_set:
logging.info('\n'.join(formatNodeList(
map(Node.asTuple, self._node_set), ' * ')))
self.logQueuedEvents()
# node_type -> node_klass
@apply
......
......@@ -20,7 +20,7 @@ import traceback
from cStringIO import StringIO
from struct import Struct
PROTOCOL_VERSION = 9
PROTOCOL_VERSION = 12
# Size restrictions.
MIN_PACKET_SIZE = 10
......@@ -71,11 +71,12 @@ def ErrorCodes():
OID_DOES_NOT_EXIST
PROTOCOL_ERROR
BROKEN_NODE
ALREADY_PENDING
REPLICATION_ERROR
CHECKING_ERROR
BACKEND_NOT_IMPLEMENTED
NON_READABLE_CELL
READ_ONLY_ACCESS
INCOMPLETE_TRANSACTION
@Enum
def ClusterStates():
......@@ -146,12 +147,6 @@ def CellStates():
# readable nor writable.
CORRUPTED
@Enum
def LockState():
NOT_LOCKED
GRANTED
GRANTED_TO_OTHER
# used for logging
node_state_prefix_dict = {
NodeStates.RUNNING: 'R',
......@@ -223,6 +218,19 @@ class BrokenNodeDisallowedError(ProtocolError):
class BackendNotImplemented(Exception):
""" Method not implemented by backend storage """
class NonReadableCell(Exception):
"""Read-access to a cell that is actually non-readable
This happens in case of race condition at processing partition table
updates: client's PT is older or newer than storage's. The latter case is
possible because the master must validate any end of replication, which
means that the storage node can't anticipate the PT update (concurrently,
there may be a first tweaks that moves the replicated cell to another node,
and a second one that moves it back).
On such event, the client must retry, preferably another cell.
"""
class Packet(object):
"""
Base class for any packet definition. The _fmt class attribute must be
......@@ -405,6 +413,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
......@@ -817,6 +838,12 @@ class UnfinishedTransactions(Packet):
Ask unfinished transactions S -> PM.
Answer unfinished transactions PM -> S.
"""
_fmt = PStruct('ask_unfinished_transactions',
PList('row_list',
PNumber('offset'),
),
)
_answer = PStruct('answer_unfinished_transactions',
PTID('max_tid'),
PList('tid_list',
......@@ -870,6 +897,18 @@ class BeginTransaction(Packet):
PTID('tid'),
)
class FailedVote(Packet):
"""
Report storage nodes for which vote failed. C -> M
True is returned if it's still possible to finish the transaction.
"""
_fmt = PStruct('failed_vote',
PTID('tid'),
PFUUIDList,
)
_answer = Error
class FinishTransaction(Packet):
"""
Finish a transaction. C -> PM.
......@@ -944,14 +983,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.
Answer if an object has been stored. If an object is in conflict,
a serial of the conflicting transaction is returned. In this case,
if this serial is newer than the current transaction ID, a client
node must not try to resolve the conflict. S -> C.
As for IStorage, 'serial' is ZERO_TID for new objects.
"""
_fmt = PStruct('ask_store_object',
POID('oid'),
......@@ -961,21 +1046,19 @@ class StoreObject(Packet):
PString('data'),
PTID('data_serial'),
PTID('tid'),
PBoolean('unlock'),
)
_answer = PStruct('answer_store_object',
PBoolean('conflicting'),
POID('oid'),
PTID('serial'),
PTID('conflict'),
)
class AbortTransaction(Packet):
"""
Abort a transaction. C -> S, PM.
Abort a transaction. C -> S and C -> PM -> S.
"""
_fmt = PStruct('abort_transaction',
PTID('tid'),
PFUUIDList, # unused for * -> S
)
class StoreTransaction(Packet):
......@@ -1159,6 +1242,7 @@ class NotifyNodeInformation(Packet):
Notify information about one or more nodes. PM -> Any.
"""
_fmt = PStruct('notify_node_informations',
PFloat('id_timestamp'),
PFNodeList,
)
......@@ -1244,22 +1328,6 @@ class ObjectUndoSerial(Packet):
),
)
class HasLock(Packet):
"""
Ask a storage is oid is locked by another transaction.
C -> S
Answer whether a transaction holds the write lock for requested object.
"""
_fmt = PStruct('has_load_lock',
PTID('tid'),
POID('oid'),
)
_answer = PStruct('answer_has_lock',
POID('oid'),
PEnum('lock_state', LockState),
)
class CheckCurrentSerial(Packet):
"""
Verifies if given serial is current for object oid in the database, and
......@@ -1271,16 +1339,12 @@ class CheckCurrentSerial(Packet):
"""
_fmt = PStruct('ask_check_current_serial',
PTID('tid'),
PTID('serial'),
POID('oid'),
)
_answer = PStruct('answer_store_object',
PBoolean('conflicting'),
POID('oid'),
PTID('serial'),
)
_answer = StoreObject._answer
class Pack(Packet):
"""
Request a pack at given TID.
......@@ -1662,6 +1726,8 @@ class Packets(dict):
ValidateTransaction)
AskBeginTransaction, AnswerBeginTransaction = register(
BeginTransaction)
FailedVote = register(
FailedVote)
AskFinishTransaction, AnswerTransactionFinished = register(
FinishTransaction, ignore_when_closed=False)
AskLockInformation, AnswerInformationLocked = register(
......@@ -1672,6 +1738,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(
......@@ -1710,8 +1782,6 @@ class Packets(dict):
ClusterState)
AskObjectUndoSerial, AnswerObjectUndoSerial = register(
ObjectUndoSerial)
AskHasLock, AnswerHasLock = register(
HasLock)
AskTIDsFrom, AnswerTIDsFrom = register(
TIDListFrom)
AskPack, AnswerPack = register(
......@@ -1781,3 +1851,8 @@ def formatNodeList(node_list, prefix='', _sort_key=itemgetter(2)):
for i in xrange(len(node_list[0]) - 1))
return map((prefix + t + '%s').__mod__, node_list)
return ()
NotifyNodeInformation._neolog = staticmethod(lambda timestamp, node_list:
((timestamp,), formatNodeList(node_list, ' ! ')))
Error._neolog = staticmethod(lambda *args: ((), ("%s (%s)" % args,)))
......@@ -15,12 +15,10 @@
# along with this program. If not, see <http://www.gnu.org/licenses/>.
import math
from functools import wraps
from . import logging, protocol
from .locking import Lock
from .protocol import uuid_str, CellStates
from .util import u64
from .locking import RLock
class PartitionTableException(Exception):
"""
......@@ -81,6 +79,10 @@ class Cell(object):
class PartitionTable(object):
"""This class manages a partition table."""
# Flushing logs whenever a cell becomes out-of-date would flood them.
_first_outdated_message = \
'a cell became non-readable whereas all cells were readable'
def __init__(self, num_partitions, num_replicas):
self._id = None
self.np = num_partitions
......@@ -163,7 +165,7 @@ class PartitionTable(object):
if cell.getUUID() == uuid:
return cell
def setCell(self, offset, node, state):
def _setCell(self, offset, node, state):
if state == CellStates.DISCARDED:
return self.removeCell(offset, node)
if node.isBroken() or node.isDown():
......@@ -182,7 +184,6 @@ class PartitionTable(object):
row.append(Cell(node, state))
if state != CellStates.FEEDING:
self.count_dict[node] += 1
return offset, node.getUUID(), state
def removeCell(self, offset, node):
row = self.partition_list[offset]
......@@ -193,7 +194,10 @@ class PartitionTable(object):
if not cell.isFeeding():
self.count_dict[node] -= 1
break
return (offset, node.getUUID(), CellStates.DISCARDED)
def dropNode(self, node):
count = self.count_dict.pop(node, None)
assert not count, (node, count)
def load(self, ptid, row_list, nm):
"""
......@@ -209,30 +213,38 @@ class PartitionTable(object):
node = nm.getByUUID(uuid)
# the node must be known by the node manager
assert node is not None
self.setCell(offset, node, state)
self._setCell(offset, node, state)
logging.debug('partition table loaded (ptid=%s)', ptid)
self.log()
def update(self, ptid, cell_list, nm):
"""
Update the partition with the cell list supplied. Ignore those changes
if the partition table ID is not greater than the current one. If a node
Update the partition with the cell list supplied. If a node
is not known, it is created in the node manager and set as unavailable
"""
if ptid <= self._id:
logging.warning('ignoring older partition changes')
return
assert self._id < ptid, (self._id, ptid)
self._id = ptid
readable_list = []
for row in self.partition_list:
if not all(cell.isReadable() for cell in row):
del readable_list[:]
break
readable_list += row
for offset, uuid, state in cell_list:
node = nm.getByUUID(uuid)
assert node is not None, 'No node found for uuid ' + uuid_str(uuid)
self.setCell(offset, node, state)
logging.debug('partition table updated (ptid=%s)', ptid)
self.log()
self._setCell(offset, node, state)
self.logUpdated()
if not all(cell.isReadable() for cell in readable_list):
logging.warning(self._first_outdated_message)
def filled(self):
return self.num_filled_rows == self.np
def logUpdated(self):
logging.debug('partition table updated (ptid=%s)', self._id)
self.log()
def log(self):
logging.debug(self.format())
......@@ -258,15 +270,16 @@ class PartitionTable(object):
partition on the line (here, line length is 11 to keep the docstring
width under 80 column).
"""
node_list = sorted(self.count_dict)
result = ['pt: node %u: %s, %s' % (i, uuid_str(node.getUUID()),
protocol.node_state_prefix_dict[node.getState()])
for i, node in enumerate(sorted(self.count_dict))]
for i, node in enumerate(node_list)]
append = result.append
line = []
max_line_len = 20 # XXX: hardcoded number of partitions per line
prefix = 0
prefix_len = int(math.ceil(math.log10(self.np)))
for offset, row in enumerate(self.formatRows()):
for offset, row in enumerate(self._formatRows(node_list)):
if len(line) == max_line_len:
append('pt: %0*u: %s' % (prefix_len, prefix, '|'.join(line)))
line = []
......@@ -276,8 +289,7 @@ class PartitionTable(object):
append('pt: %0*u: %s' % (prefix_len, prefix, '|'.join(line)))
return result
def formatRows(self):
node_list = sorted(self.count_dict)
def _formatRows(self, node_list):
cell_state_dict = protocol.cell_state_prefix_dict
for row in self.partition_list:
if row is None:
......@@ -287,13 +299,15 @@ class PartitionTable(object):
for x in row}
yield ''.join(cell_dict.get(x, '.') for x in node_list)
def operational(self):
def operational(self, exclude_list=()):
if not self.filled():
return False
for row in self.partition_list:
for cell in row:
if cell.isReadable() and cell.getNode().isRunning():
break
if cell.isReadable():
node = cell.getNode()
if node.isRunning() and node.getUUID() not in exclude_list:
break
else:
return False
return True
......@@ -308,38 +322,22 @@ class PartitionTable(object):
getRow = self.getRow
return [(x, getRow(x)) for x in xrange(self.np)]
def thread_safe(method):
def wrapper(self, *args, **kwargs):
self.lock()
try:
return method(self, *args, **kwargs)
finally:
self.unlock()
return wraps(method)(wrapper)
class MTPartitionTable(PartitionTable):
""" Thread-safe aware version of the partition table, override only methods
used in the client """
def __init__(self, *args, **kwargs):
self._lock = RLock()
PartitionTable.__init__(self, *args, **kwargs)
def lock(self):
self._lock.acquire()
def unlock(self):
self._lock.release()
def __init__(self, *args, **kw):
self._lock = Lock()
PartitionTable.__init__(self, *args, **kw)
@thread_safe
def setCell(self, *args, **kwargs):
return PartitionTable.setCell(self, *args, **kwargs)
def update(self, *args, **kw):
with self._lock:
return PartitionTable.update(self, *args, **kw)
@thread_safe
def clear(self, *args, **kwargs):
return PartitionTable.clear(self, *args, **kwargs)
def clear(self, *args, **kw):
with self._lock:
return PartitionTable.clear(self, *args, **kw)
@thread_safe
def operational(self, *args, **kwargs):
return PartitionTable.operational(self, *args, **kwargs)
def operational(self, *args, **kw):
with self._lock:
return PartitionTable.operational(self, *args, **kw)
......@@ -17,9 +17,8 @@
import thread, threading, weakref
from . import logging
from .app import BaseApplication
from .connection import ConnectionClosed
from .debug import register as registerLiveDebugger
from .dispatcher import Dispatcher, ForgottenPacket
from .dispatcher import Dispatcher
from .locking import SimpleQueue
class app_set(weakref.WeakSet):
......@@ -141,17 +140,8 @@ class ThreadedApplication(BaseApplication):
_handlePacket = self._handlePacket
while True:
qconn, qpacket, kw = get(True)
is_forgotten = isinstance(qpacket, ForgottenPacket)
if conn is qconn:
# check fake packet
if qpacket is None:
raise ConnectionClosed
if msg_id == qpacket.getId(): # NOTE selector on msg_id
if is_forgotten:
raise ValueError, 'ForgottenPacket for an ' \
'explicitly expected packet.'
_handlePacket(qconn, qpacket, kw, handler)
break
if not is_forgotten and qpacket is not None:
_handlePacket(qconn, qpacket, kw)
if conn is qconn and msg_id == qpacket.getId(): # NOTE selector on msg_id
_handlePacket(qconn, qpacket, kw, handler)
break
_handlePacket(qconn, qpacket, kw)
return self.getHandlerData()
......@@ -214,6 +214,8 @@ class ReadBuffer(object):
self.size = 0
self.content.clear()
dummy_read_buffer = ReadBuffer()
dummy_read_buffer.append = lambda _: None
class cached_property(object):
"""
......
......@@ -29,6 +29,16 @@ from neo.lib.exception import ElectionFailure, PrimaryFailure, StoppedOperation
class StateChangedException(Exception): pass
_previous_time = 0
def monotonic_time():
global _previous_time
now = time()
if _previous_time < now:
_previous_time = now
else:
_previous_time = now = _previous_time + 1e-3
return now
from .backup_app import BackupApplication
from .handlers import election, identification, secondary
from .handlers import administration, client, storage
......@@ -41,6 +51,7 @@ from .verification import VerificationManager
class Application(BaseApplication):
"""The master node application."""
packing = None
storage_readiness = 0
# Latest completely committed TID
last_transaction = ZERO_TID
backup_tid = None
......@@ -56,7 +67,8 @@ class Application(BaseApplication):
self.server = config.getBind()
self.autostart = config.getAutostart()
self.storage_readiness = set()
self.storage_ready_dict = {}
self.storage_starting_set = set()
for master_address in config.getMasters():
self.nm.createMaster(address=master_address)
......@@ -196,7 +208,7 @@ class Application(BaseApplication):
# Ask all connected nodes to reelect a single primary master.
for conn in self.em.getClientList():
conn.notify(Packets.ReelectPrimary())
conn.send(Packets.ReelectPrimary())
conn.abort()
# Wait until the connections are closed.
......@@ -240,22 +252,27 @@ class Application(BaseApplication):
continue
node_dict[NodeTypes.MASTER].append(node_info)
now = monotonic_time()
# send at most one non-empty notification packet per node
for node in self.nm.getIdentifiedList():
node_list = node_dict.get(node.getType())
if node_list and node.isRunning() and node is not exclude:
node.notify(Packets.NotifyNodeInformation(node_list))
# We don't skip pending storage nodes because we don't send them
# the full list of nodes when they're added, and it's also quite
# useful to notify them about new masters.
if node_list and node is not exclude:
node.send(Packets.NotifyNodeInformation(now, node_list))
def broadcastPartitionChanges(self, cell_list):
"""Broadcast a Notify Partition Changes packet."""
logging.debug('broadcastPartitionChanges')
if cell_list:
self.pt.log()
ptid = self.pt.setNextID()
self.pt.logUpdated()
packet = Packets.NotifyPartitionChanges(ptid, cell_list)
for node in self.nm.getIdentifiedList():
if node.isRunning() and not node.isMaster():
node.notify(packet)
# As for broadcastNodesInformation, we don't send the full PT
# when pending storage nodes are added, so keep them notified.
if not node.isMaster():
node.send(packet)
def provideService(self):
"""
......@@ -280,7 +297,7 @@ class Application(BaseApplication):
for node in self.nm.getStorageList(only_identified=True):
tid_dict[node.getUUID()] = tid
if node.isRunning():
node.notify(packet)
node.send(packet)
self.pt.setBackupTidDict(tid_dict)
def playPrimaryRole(self):
......@@ -292,7 +309,7 @@ class Application(BaseApplication):
if conn.isListening():
conn.setHandler(identification.IdentificationHandler(self))
else:
conn.notify(packet)
conn.send(packet)
# Primary master should rather establish connections to all
# secondaries, rather than the other way around. This requires
# a bit more work when a new master joins a cluster but makes
......@@ -356,16 +373,19 @@ class Application(BaseApplication):
truncate = Packets.Truncate(*e.args) if e.args else None
# Automatic restart except if we truncate or retry to.
self._startup_allowed = not (self.truncate_tid or truncate)
self.storage_readiness = 0
self.storage_ready_dict.clear()
self.storage_starting_set.clear()
node_list = []
for node in self.nm.getIdentifiedList():
if node.isStorage() or node.isClient():
conn = node.getConnection()
conn.notify(Packets.StopOperation())
conn.send(Packets.StopOperation())
if node.isClient():
conn.abort()
continue
if truncate:
conn.notify(truncate)
conn.send(truncate)
if node.isRunning():
node.setPending()
node_list.append(node)
......@@ -398,6 +418,7 @@ class Application(BaseApplication):
conn.close()
# Reconnect to primary master node.
self.nm.reset()
primary_handler = secondary.PrimaryHandler(self)
ClientConnection(self, primary_handler, self.primary_master_node)
......@@ -437,14 +458,15 @@ class Application(BaseApplication):
notification_packet = Packets.NotifyClusterInformation(state)
for node in self.nm.getIdentifiedList():
conn = node.getConnection()
conn.notify(notification_packet)
conn.send(notification_packet)
if node.isClient():
if state == ClusterStates.RUNNING:
handler = self.client_service_handler
elif state == ClusterStates.BACKINGUP:
handler = self.client_ro_service_handler
else:
conn.abort()
if state != ClusterStates.STOPPING:
conn.abort()
continue
elif node.isStorage() and storage_handler:
handler = storage_handler
......@@ -473,15 +495,18 @@ class Application(BaseApplication):
def shutdown(self):
"""Close all connections and exit"""
self.changeClusterState(ClusterStates.STOPPING)
self.listening_conn.close()
for conn in self.em.getConnectionList():
node = self.nm.getByUUID(conn.getUUID())
if node is None or not node.isIdentified():
conn.close()
# No need to change handlers in order to reject RequestIdentification
# & AskBeginTransaction packets because they won't be any:
# the only remaining connected peers are identified non-clients
# and we don't accept new connections anymore.
# Marking a fictional storage node as starting operation blocks any
# request to start a new transaction. Do this way has 2 advantages:
# - It's simpler than changing the handler of all clients,
# which is anyway not supported by EventQueue.
# - Returning an error code would cause activity on client side for
# nothing.
# What's important is to not abort during the second phase of commits
# and for this, clients must even be able to reconnect, in case of
# failure during tpc_finish.
# We're rarely involved in vote, so we have to trust clients that they
# abort any transaction that is still in the first phase.
self.storage_starting_set.add(None)
try:
# wait for all transaction to be finished
while self.tm.hasPending():
......@@ -490,14 +515,15 @@ class Application(BaseApplication):
logging.critical('No longer operational')
logging.info("asking remaining nodes to shutdown")
self.listening_conn.close()
handler = EventHandler(self)
for node in self.nm.getConnectedList():
conn = node.getConnection()
if node.isStorage():
conn.setHandler(handler)
conn.notify(Packets.NotifyNodeInformation(((
node.getType(), node.getAddress(), node.getUUID(),
NodeStates.TEMPORARILY_DOWN, None),)))
conn.send(Packets.NotifyNodeInformation(monotonic_time(), ((
node.getType(), node.getAddress(), node.getUUID(),
NodeStates.TEMPORARILY_DOWN, None),)))
conn.abort()
elif conn.pending():
conn.abort()
......@@ -527,19 +553,18 @@ class Application(BaseApplication):
transaction_node = txn.getNode()
invalidate_objects = Packets.InvalidateObjects(tid, txn.getOIDList())
for client_node in self.nm.getClientList(only_identified=True):
c = client_node.getConnection()
if client_node is transaction_node:
c.answer(Packets.AnswerTransactionFinished(ttid, tid),
msg_id=txn.getMessageId()) # NOTE msgid: out-of-order answer
client_node.send(Packets.AnswerTransactionFinished(ttid, tid),
msg_id=txn.getMessageId()) # NOTE msgid: out-of-order answer
else:
# NOTE notifies clients sequentially & irregardless of whether client was subscribed
c.notify(invalidate_objects)
client_node.send(invalidate_objects)
# Unlock Information to relevant storage nodes.
notify_unlock = Packets.NotifyUnlockInformation(ttid)
getByUUID = self.nm.getByUUID
for storage_uuid in txn.getUUIDList():
getByUUID(storage_uuid).getConnection().notify(notify_unlock)
getByUUID(storage_uuid).send(notify_unlock)
# Notify storage that have replications blocked by this transaction,
# and clients that try to recover from a failure during tpc_finish.
......@@ -550,7 +575,7 @@ class Application(BaseApplication):
# There should be only 1 client interested.
node.answer(Packets.AnswerFinalTID(tid))
else:
node.notify(notify_finished)
node.send(notify_finished)
assert self.last_transaction < tid, (self.last_transaction, tid)
self.setLastTransaction(tid)
......@@ -562,11 +587,27 @@ class Application(BaseApplication):
self.last_transaction = tid
def setStorageNotReady(self, uuid):
self.storage_readiness.discard(uuid)
self.storage_starting_set.discard(uuid)
self.storage_ready_dict.pop(uuid, None)
self.tm.executeQueuedEvents()
def startStorage(self, node):
node.send(Packets.StartOperation(self.backup_tid))
uuid = node.getUUID()
assert uuid not in self.storage_starting_set
if uuid not in self.storage_ready_dict:
self.storage_starting_set.add(uuid)
def setStorageReady(self, uuid):
self.storage_readiness.add(uuid)
self.storage_starting_set.remove(uuid)
assert uuid not in self.storage_ready_dict, self.storage_ready_dict
self.storage_readiness = self.storage_ready_dict[uuid] = \
self.storage_readiness + 1
self.tm.executeQueuedEvents()
def isStorageReady(self, uuid):
return uuid in self.storage_readiness
return uuid in self.storage_ready_dict
def getStorageReadySet(self, readiness=float('inf')):
return {k for k, v in self.storage_ready_dict.iteritems()
if v <= readiness}
......@@ -190,7 +190,7 @@ class BackupApplication(object):
"ask %s to replicate partition %u up to %s from %s",
uuid_str(cell.getUUID()), offset, dump(tid),
uuid_str(primary_node.getUUID()))
cell.getNode().getConnection().notify(p)
cell.getNode().send(p)
trigger_set.add(primary_node)
for node in trigger_set:
self.triggerBackup(node)
......@@ -255,7 +255,7 @@ class BackupApplication(object):
cell.replicating = tid
for node, untouched_dict in untouched_dict.iteritems():
if app.isStorageReady(node.getUUID()):
node.notify(Packets.Replicate(tid, '', untouched_dict)) # NOTE Mb -> Sb (notify tid brings no new data)
node.send(Packets.Replicate(tid, '', untouched_dict)) # NOTE Mb -> Sb (notify tid brings no new data)
for node in trigger_set:
self.triggerBackup(node)
count = sum(map(len, self.tid_list))
......@@ -291,8 +291,7 @@ class BackupApplication(object):
source_dict[offset] = addr
logging.debug("ask %s to replicate partition %u up to %s from %r",
uuid_str(node.getUUID()), offset, dump(tid), addr)
node.getConnection().notify(Packets.Replicate( # NOTE Mb -> Sb (notify to trigger replicate up to tid)
tid, self.name, source_dict))
node.send(Packets.Replicate(tid, self.name, source_dict)) # NOTE Mb -> Sb (notify to trigger replicate up to tid)
# NOTE feedback from Sb -> Mb a partition (requested by invalidatePartitions->triggerBackup) has been replicated
def notifyReplicationDone(self, node, offset, tid):
......@@ -310,7 +309,7 @@ class BackupApplication(object):
last_tid = app.getLastTransaction()
if tid < last_tid:
tid = last_tid
node.notify(Packets.Replicate(tid, '', {offset: None}))
node.send(Packets.Replicate(tid, '', {offset: None}))
logging.debug("partition %u: updating backup_tid of %r to %s",
offset, cell, dump(tid))
cell.backup_tid = tid
......@@ -334,7 +333,7 @@ class BackupApplication(object):
"ask %s to replicate partition %u up to %s from %s",
uuid_str(node.getUUID()), offset, dump(max_tid),
uuid_str(primary_node.getUUID()))
node.notify(Packets.Replicate(max_tid, '',
node.send(Packets.Replicate(max_tid, '',
{offset: primary_node.getAddress()}))
else:
if app.getClusterState() == ClusterStates.BACKINGUP:
......@@ -350,5 +349,5 @@ class BackupApplication(object):
"ask %s to replicate partition %u up to %s from %s",
uuid_str(cell.getUUID()), offset,
dump(tid), uuid_str(node.getUUID()))
cell.getNode().notify(p)
cell.getNode().send(p)
return result
......@@ -14,6 +14,7 @@
# 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 ..app import monotonic_time
from neo.lib import logging
from neo.lib.exception import StoppedOperation
from neo.lib.handler import EventHandler
......@@ -98,7 +99,7 @@ class MasterHandler(EventHandler):
node_list.extend(n.asTuple() for n in nm.getMasterList())
node_list.extend(n.asTuple() for n in nm.getClientList())
node_list.extend(n.asTuple() for n in nm.getStorageList())
conn.notify(Packets.NotifyNodeInformation(node_list))
conn.send(Packets.NotifyNodeInformation(monotonic_time(), node_list))
def askPartitionTable(self, conn):
pt = self.app.pt
......@@ -115,7 +116,7 @@ class BaseServiceHandler(MasterHandler):
def connectionCompleted(self, conn, new):
self._notifyNodeInformation(conn)
pt = self.app.pt
conn.notify(Packets.SendPartitionTable(pt.getID(), pt.getRowList()))
conn.send(Packets.SendPartitionTable(pt.getID(), pt.getRowList()))
def connectionLost(self, conn, new_state):
app = self.app
......@@ -145,7 +146,3 @@ class BaseServiceHandler(MasterHandler):
app.broadcastPartitionChanges(app.pt.outdate(node))
if not app.pt.operational():
raise StoppedOperation
def notifyReady(self, conn):
self.app.setStorageReady(conn.getUUID())
......@@ -17,7 +17,7 @@
import random
from . import MasterHandler
from ..app import StateChangedException
from ..app import monotonic_time, StateChangedException
from neo.lib import logging
from neo.lib.exception import StoppedOperation
from neo.lib.pt import PartitionTableException
......@@ -103,7 +103,8 @@ class AdministrationHandler(MasterHandler):
node.setState(state)
if node.isConnected():
# notify itself so it can shutdown
node.notify(Packets.NotifyNodeInformation([node.asTuple()]))
node.send(Packets.NotifyNodeInformation(
monotonic_time(), [node.asTuple()]))
# close to avoid handle the closure as a connection lost
node.getConnection().abort()
if keep:
......@@ -121,7 +122,8 @@ class AdministrationHandler(MasterHandler):
# ignores non-running nodes
assert not node.isRunning()
if node.isConnected():
node.notify(Packets.NotifyNodeInformation([node.asTuple()]))
node.send(Packets.NotifyNodeInformation(
monotonic_time(), [node.asTuple()]))
app.broadcastNodesInformation([node])
def addPendingNodes(self, conn, uuid_list):
......@@ -139,10 +141,9 @@ class AdministrationHandler(MasterHandler):
for node in app.nm.getStorageList()
if node.isPending() and node.getUUID() in uuid_list))
if node_list:
p = Packets.StartOperation(bool(app.backup_tid)) # NOTE ...
for node in node_list:
node.setRunning()
node.notify(p)
app.startStorage(node)
app.broadcastNodesInformation(node_list)
conn.answer(Errors.Ack('Nodes added: %s' %
', '.join(uuid_str(x.getUUID()) for x in node_list)))
......@@ -160,7 +161,7 @@ class AdministrationHandler(MasterHandler):
node_list.append(node)
repair = Packets.NotifyRepair(*args)
for node in node_list:
node.notify(repair)
node.send(repair)
conn.answer(Errors.Ack(''))
def tweakPartitionTable(self, conn, uuid_list):
......@@ -224,6 +225,6 @@ class AdministrationHandler(MasterHandler):
).getAddress()
else:
source = '', None
node.getConnection().notify(Packets.CheckPartition(
node.send(Packets.CheckPartition(
offset, source, min_tid, max_tid))
conn.answer(Errors.Ack(''))
......@@ -14,7 +14,9 @@
# 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 neo.lib.handler import DelayEvent
from neo.lib.protocol import NodeStates, Packets, ProtocolError, MAX_TID, Errors
from ..app import monotonic_time
from . import MasterHandler
class ClientServiceHandler(MasterHandler):
......@@ -36,54 +38,55 @@ class ClientServiceHandler(MasterHandler):
node_list = [nm.getByUUID(conn.getUUID()).asTuple()] # for id_timestamp
node_list.extend(n.asTuple() for n in nm.getMasterList())
node_list.extend(n.asTuple() for n in nm.getStorageList())
conn.notify(Packets.NotifyNodeInformation(node_list))
conn.send(Packets.NotifyNodeInformation(monotonic_time(), node_list))
def askBeginTransaction(self, conn, tid):
"""
A client request a TID, nothing is kept about it until the finish.
"""
app = self.app
# Delay new transaction as long as we are waiting for NotifyReady
# answers, otherwise we can know if the client is expected to commit
# the transaction in full to all these storage nodes.
if app.storage_starting_set:
raise DelayEvent
node = app.nm.getByUUID(conn.getUUID())
conn.answer(Packets.AnswerBeginTransaction(app.tm.begin(node, tid)))
tid = app.tm.begin(node, app.storage_readiness, tid)
conn.answer(Packets.AnswerBeginTransaction(tid))
def askNewOIDs(self, conn, num_oids):
conn.answer(Packets.AnswerNewOIDs(self.app.tm.getNextOIDList(num_oids)))
def getEventQueue(self):
# for askBeginTransaction & failedVote
return self.app.tm
def failedVote(self, conn, *args):
app = self.app
conn.answer((Errors.Ack if app.tm.vote(app, *args) else
Errors.IncompleteTransaction)())
def askFinishTransaction(self, conn, ttid, oid_list, checked_list):
app = self.app
pt = app.pt
# Collect partitions related to this transaction.
getPartition = pt.getPartition
partition_set = set(map(getPartition, oid_list))
partition_set.update(map(getPartition, checked_list))
partition_set.add(getPartition(ttid))
# Collect the UUIDs of nodes related to this transaction.
uuid_list = filter(app.isStorageReady, {cell.getUUID()
for part in partition_set
for cell in pt.getCellList(part)
if cell.getNodeState() != NodeStates.HIDDEN})
if not uuid_list:
raise ProtocolError('No storage node ready for transaction')
identified_node_list = app.nm.getIdentifiedList(pool_set=set(uuid_list))
# Request locking data.
# build a new set as we may not send the message to all nodes as some
# might be not reachable at that time
p = Packets.AskLockInformation(
tid, node_list = app.tm.prepare(
app,
ttid,
app.tm.prepare(
ttid,
pt.getPartitions(),
oid_list,
{x.getUUID() for x in identified_node_list},
conn.getPeerId(),
),
oid_list,
checked_list,
conn.getPeerId(),
)
for node in identified_node_list:
node.ask(p, timeout=60)
if tid:
p = Packets.AskLockInformation(ttid, tid)
for node in node_list:
node.ask(p, timeout=60)
else:
conn.answer(Errors.IncompleteTransaction())
# It's simpler to abort automatically rather than asking the client
# to send a notification on tpc_abort, since it would have keep the
# transaction longer in list of transactions.
# This should happen so rarely that we don't try to minimize the
# number of abort notifications by looking the modified partitions.
self.abortTransaction(conn, ttid, app.getStorageReadySet())
def askFinalTID(self, conn, ttid):
tm = self.app.tm
......@@ -112,9 +115,24 @@ class ClientServiceHandler(MasterHandler):
else:
conn.answer(Packets.AnswerPack(False))
def abortTransaction(self, conn, tid):
# BUG: The replicator may wait this transaction to be finished.
self.app.tm.abort(tid, conn.getUUID())
def abortTransaction(self, conn, tid, uuid_list):
# Consider a failure when the connection between the storage and the
# client breaks while the answer to the first write is sent back.
# In other words, the client can not know the exact set of nodes that
# know this transaction, and it sends us all nodes it considered for
# writing.
# We must also add those that are waiting for this transaction to be
# finished (returned by tm.abort), because they may have join the
# cluster after that the client started to abort.
app = self.app
involved = app.tm.abort(tid, conn.getUUID())
involved.update(uuid_list)
involved.intersection_update(app.getStorageReadySet())
if involved:
p = Packets.AbortTransaction(tid, ())
getByUUID = app.nm.getByUUID
for involved in involved:
getByUUID(involved).send(p)
# like ClientServiceHandler but read-only & only for tid <= backup_tid
......
......@@ -56,7 +56,7 @@ class BaseElectionHandler(EventHandler):
class ClientElectionHandler(BaseElectionHandler):
def notifyNodeInformation(self, conn, node_list):
def notifyNodeInformation(self, conn, timestamp, node_list):
# XXX: For the moment, do nothing because
# we'll close this connection and reconnect.
pass
......
......@@ -14,10 +14,10 @@
# 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 time import time
from neo.lib import logging
from neo.lib.protocol import ClusterStates, NodeStates, NodeTypes, \
NotReadyError, ProtocolError, uuid_str
from ..app import monotonic_time
from . import MasterHandler
class IdentificationHandler(MasterHandler):
......@@ -92,7 +92,7 @@ class IdentificationHandler(MasterHandler):
uuid=uuid, address=address)
else:
node.setUUID(uuid)
node.id_timestamp = time()
node.id_timestamp = monotonic_time()
node.setState(state)
node.setConnection(conn)
conn.setHandler(handler)
......
......@@ -15,6 +15,7 @@
# along with this program. If not, see <http://www.gnu.org/licenses/>.
import sys
from ..app import monotonic_time
from . import MasterHandler
from neo.lib.handler import EventHandler
from neo.lib.exception import ElectionFailure, PrimaryFailure
......@@ -25,10 +26,11 @@ class SecondaryMasterHandler(MasterHandler):
""" Handler used by primary to handle secondary masters"""
def connectionLost(self, conn, new_state):
node = self.app.nm.getByUUID(conn.getUUID())
assert node is not None
node.setDown()
self.app.broadcastNodesInformation([node])
app = self.app
if app.listening_conn: # if running
node = app.nm.getByUUID(conn.getUUID())
node.setDown()
app.broadcastNodesInformation([node])
def announcePrimary(self, conn):
raise ElectionFailure, 'another primary arises'
......@@ -38,18 +40,18 @@ class SecondaryMasterHandler(MasterHandler):
def _notifyNodeInformation(self, conn):
node_list = [n.asTuple() for n in self.app.nm.getMasterList()]
conn.notify(Packets.NotifyNodeInformation(node_list))
conn.send(Packets.NotifyNodeInformation(monotonic_time(), node_list))
class PrimaryHandler(EventHandler):
""" Handler used by secondaries to handle primary master"""
def connectionLost(self, conn, new_state):
self.app.primary_master_node.setDown()
raise PrimaryFailure, 'primary master is dead'
self.connectionFailed(conn)
def connectionFailed(self, conn):
self.app.primary_master_node.setDown()
raise PrimaryFailure, 'primary master is dead'
if self.app.listening_conn: # if running
raise PrimaryFailure('primary master is dead')
def connectionCompleted(self, conn):
app = self.app
......@@ -72,8 +74,9 @@ class PrimaryHandler(EventHandler):
def notifyClusterInformation(self, conn, state):
self.app.cluster_state = state
def notifyNodeInformation(self, conn, node_list):
super(PrimaryHandler, self).notifyNodeInformation(conn, node_list)
def notifyNodeInformation(self, conn, timestamp, node_list):
super(PrimaryHandler, self).notifyNodeInformation(
conn, timestamp, node_list)
for node_type, _, uuid, state, _ in node_list:
assert node_type == NodeTypes.MASTER, node_type
if uuid == self.app.uuid and state == NodeStates.UNKNOWN:
......
......@@ -15,9 +15,11 @@
# along with this program. If not, see <http://www.gnu.org/licenses/>.
from neo.lib import logging
from neo.lib.protocol import CellStates, ClusterStates, Packets, ProtocolError
from neo.lib.protocol import (CellStates, ClusterStates, Packets, ProtocolError,
uuid_str)
from neo.lib.exception import StoppedOperation
from neo.lib.pt import PartitionTableException
from neo.lib.util import dump
from . import BaseServiceHandler
......@@ -26,18 +28,22 @@ class StorageServiceHandler(BaseServiceHandler):
def connectionCompleted(self, conn, new):
app = self.app
uuid = conn.getUUID()
app.setStorageNotReady(uuid)
if new:
super(StorageServiceHandler, self).connectionCompleted(conn, new)
if app.nm.getByUUID(uuid).isRunning(): # node may be PENDING
conn.notify(Packets.StartOperation(app.backup_tid))
node = app.nm.getByUUID(conn.getUUID())
if node.isRunning(): # node may be PENDING
app.startStorage(node)
def notifyReady(self, conn):
self.app.setStorageReady(conn.getUUID())
def connectionLost(self, conn, new_state):
app = self.app
node = app.nm.getByUUID(conn.getUUID())
uuid = conn.getUUID()
node = app.nm.getByUUID(uuid)
super(StorageServiceHandler, self).connectionLost(conn, new_state)
app.tm.storageLost(conn.getUUID())
app.setStorageNotReady(uuid)
app.tm.storageLost(uuid)
if (app.getClusterState() == ClusterStates.BACKINGUP
# Also check if we're exiting, because backup_app is not usable
# in this case. Maybe cluster state should be set to something
......@@ -47,7 +53,7 @@ class StorageServiceHandler(BaseServiceHandler):
if app.packing is not None:
self.answerPack(conn, False)
def askUnfinishedTransactions(self, conn):
def askUnfinishedTransactions(self, conn, offset_list):
app = self.app
if app.backup_tid:
last_tid = app.pt.getBackupTid(min)
......@@ -60,6 +66,10 @@ class StorageServiceHandler(BaseServiceHandler):
pending_list = app.tm.registerForNotification(conn.getUUID())
p = Packets.AnswerUnfinishedTransactions(last_tid, pending_list)
conn.answer(p)
app.pt.updatable(conn.getUUID(), offset_list)
def notifyDeadlock(self, conn, *args):
self.app.tm.deadlock(conn.getUUID(), *args)
def answerInformationLocked(self, conn, ttid):
self.app.tm.lock(ttid, conn.getUUID())
......@@ -77,7 +87,8 @@ class StorageServiceHandler(BaseServiceHandler):
def notifyReplicationDone(self, conn, offset, tid):
app = self.app
node = app.nm.getByUUID(conn.getUUID())
uuid = conn.getUUID()
node = app.nm.getByUUID(uuid)
if app.backup_tid:
cell_list = app.backup_app.notifyReplicationDone(node, offset, tid)
if not cell_list:
......@@ -85,11 +96,15 @@ class StorageServiceHandler(BaseServiceHandler):
else:
try:
cell_list = self.app.pt.setUpToDate(node, offset)
if not cell_list:
raise ProtocolError('Non-outdated partition')
except PartitionTableException, e:
raise ProtocolError(str(e))
logging.debug("%s is up for offset %s", node, offset)
if not cell_list:
logging.info("ignored late notification that"
" %s has replicated partition %s up to %s",
uuid_str(uuid), offset, dump(tid))
return
logging.debug("%s is up for partition %s (tid=%s)",
uuid_str(uuid), offset, dump(tid))
self.app.broadcastPartitionChanges(cell_list)
def answerPack(self, conn, status):
......@@ -100,5 +115,5 @@ class StorageServiceHandler(BaseServiceHandler):
if not uid_set:
app.packing = None
if not client.isClosed():
client.answer(Packets.AnswerPack(True), msg_id=msg_id) # NOTE msg_id: out-of-order answer
client.send(Packets.AnswerPack(True), msg_id) # NOTE msg_id: out-of-order answer
......@@ -16,17 +16,25 @@
from collections import defaultdict
import neo.lib.pt
from neo.lib import logging
from neo.lib.protocol import CellStates, ZERO_TID
class Cell(neo.lib.pt.Cell):
replicating = ZERO_TID
updatable = False
def setState(self, state):
readable = self.isReadable()
super(Cell, self).setState(state)
if readable and not self.isReadable():
if self.isReadable():
return
try:
del self.updatable
except AttributeError:
pass
if readable:
try:
del self.backup_tid, self.replicating
except AttributeError:
......@@ -147,7 +155,7 @@ class PartitionTable(neo.lib.pt.PartitionTable):
if node is None:
node = nm.createStorage(uuid=uuid)
new_nodes.append(node.asTuple())
self.setCell(offset, node, state)
self._setCell(offset, node, state)
return new_nodes
def setUpToDate(self, node, offset):
......@@ -156,20 +164,22 @@ class PartitionTable(neo.lib.pt.PartitionTable):
# check the partition is assigned and known as outdated
for cell in self.getCellList(offset):
if cell.getUUID() == uuid:
if cell.isOutOfDate():
if cell.isOutOfDate() and cell.updatable:
break
return
else:
raise neo.lib.pt.PartitionTableException('Non-assigned partition')
# update the partition table
cell_list = [self.setCell(offset, node, CellStates.UP_TO_DATE)]
self._setCell(offset, node, CellStates.UP_TO_DATE)
cell_list = [(offset, uuid, CellStates.UP_TO_DATE)]
# If the partition contains a feeding cell, drop it now.
for feeding_cell in self.getCellList(offset):
if feeding_cell.isFeeding():
cell_list.append(self.removeCell(offset,
feeding_cell.getNode()))
node = feeding_cell.getNode()
self.removeCell(offset, node)
cell_list.append((offset, node.getUUID(), CellStates.DISCARDED))
break
return cell_list
......@@ -276,6 +286,9 @@ class PartitionTable(neo.lib.pt.PartitionTable):
to serve. This allows a cluster restart.
"""
change_list = []
fully_readable = all(cell.isReadable()
for row in self.partition_list
for cell in row)
for offset, row in enumerate(self.partition_list):
lost = lost_node
cell_list = []
......@@ -290,8 +303,16 @@ class PartitionTable(neo.lib.pt.PartitionTable):
cell.setState(CellStates.OUT_OF_DATE)
change_list.append((offset, cell.getUUID(),
CellStates.OUT_OF_DATE))
if fully_readable and change_list:
logging.warning(self._first_outdated_message)
return change_list
def updatable(self, uuid, offset_list):
for offset in offset_list:
for cell in self.partition_list[offset]:
if cell.getUUID() == uuid and not cell.isReadable():
cell.updatable = True
def iterNodeCell(self, node):
for offset, row in enumerate(self.partition_list):
for cell in row:
......
......@@ -16,6 +16,7 @@
from neo.lib import logging
from neo.lib.protocol import Packets, ProtocolError, ClusterStates, NodeStates
from .app import monotonic_time
from .handlers import MasterHandler
......@@ -88,7 +89,7 @@ class RecoveryManager(MasterHandler):
truncate = Packets.Truncate(app.truncate_tid)
for node in node_list:
conn = node.getConnection()
conn.notify(truncate)
conn.send(truncate)
self.connectionCompleted(conn, False)
continue
node_list = pt.getConnectedNodeList()
......@@ -170,8 +171,9 @@ class RecoveryManager(MasterHandler):
new_nodes = app.pt.load(ptid, row_list, app.nm)
except IndexError:
raise ProtocolError('Invalid offset')
self._notifyAdmins(Packets.NotifyNodeInformation(new_nodes),
Packets.SendPartitionTable(ptid, row_list))
self._notifyAdmins(
Packets.NotifyNodeInformation(monotonic_time(), new_nodes),
Packets.SendPartitionTable(ptid, row_list))
self.ask_pt = ()
uuid = conn.getUUID()
app.backup_tid = self.backup_tid_dict[uuid]
......@@ -180,4 +182,4 @@ class RecoveryManager(MasterHandler):
def _notifyAdmins(self, *packets):
for node in self.app.nm.getAdminList(only_identified=True):
for packet in packets:
node.notify(packet)
node.send(packet)
......@@ -18,29 +18,31 @@ from collections import deque
from time import time
from struct import pack, unpack
from neo.lib import logging
from neo.lib.protocol import ProtocolError, uuid_str, ZERO_OID, ZERO_TID
from neo.lib.handler import DelayEvent, EventQueue
from neo.lib.protocol import Packets, ProtocolError, uuid_str, \
ZERO_OID, ZERO_TID
from neo.lib.util import dump, u64, addTID, tidFromTime
class DelayedError(Exception):
pass
class Transaction(object):
"""
A pending transaction
"""
locking_tid = ZERO_TID
_tid = None
_msg_id = None
_oid_list = None
_failed = frozenset()
_prepared = False
# uuid dict hold flag to known who has locked the transaction
_uuid_set = None
_lock_wait_uuid_set = None
def __init__(self, node, ttid):
def __init__(self, node, storage_readiness, ttid):
"""
Prepare the transaction, set OIDs and UUIDs related to it
"""
self._node = node
self._storage_readiness = storage_readiness
self._ttid = ttid
self._birth = time()
# store storage uuids that must be notified at commit
......@@ -113,13 +115,13 @@ class Transaction(object):
"""
return list(self._notification_set)
def prepare(self, tid, oid_list, uuid_list, msg_id):
def prepare(self, tid, oid_list, uuid_set, msg_id):
self._tid = tid
self._oid_list = oid_list
self._msg_id = msg_id
self._uuid_set = set(uuid_list)
self._lock_wait_uuid_set = set(uuid_list)
self._uuid_set = uuid_set
self._lock_wait_uuid_set = uuid_set.copy()
self._prepared = True
def storageLost(self, uuid):
......@@ -163,7 +165,7 @@ class Transaction(object):
return not self._lock_wait_uuid_set
class TransactionManager(object):
class TransactionManager(EventQueue):
"""
Manage current transactions
"""
......@@ -173,6 +175,7 @@ class TransactionManager(object):
self.reset()
def reset(self):
EventQueue.__init__(self)
# ttid -> transaction
self._ttid_dict = {}
self._last_oid = ZERO_OID
......@@ -195,6 +198,7 @@ class TransactionManager(object):
except ValueError:
pass
del self._ttid_dict[ttid]
self.executeQueuedEvents()
def __contains__(self, ttid):
"""
......@@ -285,7 +289,7 @@ class TransactionManager(object):
txn.registerForNotification(uuid)
return self._ttid_dict.keys()
def begin(self, node, tid=None):
def begin(self, node, storage_readiness, tid=None):
"""
Generate a new TID
"""
......@@ -297,38 +301,116 @@ class TransactionManager(object):
# last TID.
self._queue.append(tid)
self.setLastTID(tid)
txn = self._ttid_dict[tid] = Transaction(node, tid)
txn = self._ttid_dict[tid] = Transaction(node, storage_readiness, tid)
logging.debug('Begin %s', txn)
return tid
def prepare(self, ttid, divisor, oid_list, uuid_list, msg_id):
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.send(Packets.NotifyDeadlock(ttid, locking_tid))
def vote(self, app, ttid, uuid_list):
"""
Check that the transaction can be voted
when the client reports failed nodes.
"""
txn = self[ttid]
# The client does not know which nodes are not expected to have
# transactions in full. Let's filter out them.
failed = app.getStorageReadySet(txn._storage_readiness)
failed.intersection_update(uuid_list)
if failed:
operational = app.pt.operational
if not operational(failed):
# No way to commit this transaction because there are
# non-replicated storage nodes with failed stores.
return False
failed = failed.copy()
for t in self._ttid_dict.itervalues():
failed |= t._failed
if not operational(failed):
# Other transactions were voted and unless they're aborted,
# we won't be able to finish this one, because that would make
# the cluster non-operational. Let's tell the caller to retry
# later.
raise DelayEvent
# Allow the client to finish the transaction,
# even if it will disconnect storage nodes.
txn._failed = failed
return True
def prepare(self, app, ttid, oid_list, checked_list, msg_id):
"""
Prepare a transaction to be finished
"""
txn = self[ttid]
pt = app.pt
failed = txn._failed
if failed and not pt.operational(failed):
return None, None
ready = app.getStorageReadySet(txn._storage_readiness)
getPartition = pt.getPartition
partition_set = set(map(getPartition, oid_list))
partition_set.update(map(getPartition, checked_list))
partition_set.add(getPartition(ttid))
node_list = []
uuid_set = set()
for partition in partition_set:
for cell in pt.getCellList(partition):
node = cell.getNode()
if node.isIdentified():
uuid = node.getUUID()
if uuid in uuid_set:
continue
if uuid in failed:
# This will commit a new PT with outdated cells before
# locking the transaction, which is important during
# the verification phase.
node.getConnection().close()
elif uuid in ready:
uuid_set.add(uuid)
node_list.append(node)
# A node that was not ready at the beginning of the transaction
# can't have readable cells. And if we're still operational without
# the 'failed' nodes, then there must still be 1 node in 'ready'
# that is UP.
assert node_list, (ready, failed)
# maybe not the fastest but _queue should be often small
if ttid in self._queue:
tid = ttid
else:
tid = self._nextTID(ttid, divisor)
tid = self._nextTID(ttid, pt.getPartitions())
self._queue.append(ttid)
logging.debug('Finish TXN %s for %s (was %s)',
dump(tid), txn.getNode(), dump(ttid))
txn.prepare(tid, oid_list, uuid_list, msg_id)
txn.prepare(tid, oid_list, uuid_set, msg_id)
# check if greater and foreign OID was stored
if oid_list:
self.setLastOID(max(oid_list))
return tid
return tid, node_list
def abort(self, ttid, uuid):
"""
Abort a transaction
"""
logging.debug('Abort TXN %s for %s', dump(ttid), uuid_str(uuid))
if self[ttid].isPrepared():
txn = self[ttid]
if txn.isPrepared():
raise ProtocolError("commit already requested for ttid %s"
% dump(ttid))
del self[ttid]
return txn._notification_set
def lock(self, ttid, uuid):
"""
......@@ -350,7 +432,7 @@ class TransactionManager(object):
for ttid, txn in self._ttid_dict.iteritems():
if txn.storageLost(uuid) and self._queue[0] == ttid:
unlock = True
# do not break: we must call forget() on all transactions
# do not break: we must call storageLost() on all transactions
if unlock:
self._unlockPending()
......@@ -370,6 +452,7 @@ class TransactionManager(object):
break
del queue[0], self._ttid_dict[ttid]
self._on_commit(txn)
self.executeQueuedEvents()
def clientLost(self, node):
for txn in self._ttid_dict.values():
......@@ -380,4 +463,4 @@ class TransactionManager(object):
logging.info('Transactions:')
for txn in self._ttid_dict.itervalues():
logging.info(' %r', txn)
self.logQueuedEvents()
......@@ -131,7 +131,7 @@ class VerificationManager(BaseServiceHandler):
if uuid_set:
packet = Packets.ValidateTransaction(ttid, tid)
for node in getIdentifiedList(pool_set=uuid_set):
node.notify(packet)
node.send(packet)
def answerLastIDs(self, conn, loid, ltid):
self._uuid_set.remove(conn.getUUID())
......
......@@ -20,7 +20,6 @@
import bz2, gzip, errno, optparse, os, signal, sqlite3, sys, time
from bisect import insort
from logging import getLevelName
from functools import partial
comp_dict = dict(bz2=bz2.BZ2File, gz=gzip.GzipFile)
......@@ -94,11 +93,6 @@ class Log(object):
exec bz2.decompress(text) in g
for x in 'uuid_str', 'Packets', 'PacketMalformedError':
setattr(self, x, g[x])
try:
self.notifyNodeInformation = partial(g['formatNodeList'],
prefix=' ! ')
except KeyError:
self.notifyNodeInformation = None
try:
self._next_protocol, = q("SELECT date FROM protocol WHERE date>?",
(date,)).next()
......@@ -131,8 +125,8 @@ class Log(object):
body = None
msg = ['#0x%04x %-30s %s' % (msg_id, msg, peer)]
if body is not None:
logger = getattr(self, p.handler_method_name, None)
if logger or self._decode_all:
log = getattr(p, '_neolog', None)
if log or self._decode_all:
p = p()
p._id = msg_id
p._body = body
......@@ -141,15 +135,13 @@ class Log(object):
except self.PacketMalformedError:
msg.append("Can't decode packet")
else:
if logger:
msg += logger(*args)
elif args:
msg = '%s \t| %r' % (msg[0], args),
if log:
args, extra = log(*args)
msg += extra
if args and self._decode_all:
msg[0] += ' \t| ' + repr(args)
return date, name, 'PACKET', msg
def error(self, code, message):
return "%s (%s)" % (code, message),
def emit_many(log_list):
log_list = [(log, iter(log).next) for log in log_list]
......
......@@ -33,6 +33,7 @@ if filter(re.compile(r'--coverage$|-\w*c').match, sys.argv[1:]):
coverage.neotestrunner = []
coverage.start()
from neo.lib import logging
from neo.tests import getTempDirectory, NeoTestBase, Patch, \
__dict__ as neo_tests__dict__
from neo.tests.benchmark import BenchmarkRunner
......@@ -45,7 +46,6 @@ UNIT_TEST_MODULES = [
'neo.tests.testConnection',
'neo.tests.testHandler',
'neo.tests.testNodes',
'neo.tests.testDispatcher',
'neo.tests.testUtil',
'neo.tests.testPT',
# master application
......@@ -66,7 +66,7 @@ UNIT_TEST_MODULES = [
# client application
'neo.tests.client.testClientApp',
'neo.tests.client.testMasterHandler',
'neo.tests.client.testConnectionPool',
'neo.tests.client.testZODBURI',
# light functional tests
'neo.tests.threaded.test',
'neo.tests.threaded.testImporter',
......@@ -236,6 +236,9 @@ class TestRunner(BenchmarkRunner):
parser.add_option('-C', '--cov-unit', action='store_true',
help='Same as -c but output 1 file per test,'
' in the temporary test directory')
parser.add_option('-L', '--log', action='store_true',
help='Force all logs to be emitted immediately and keep'
' packet body in logs of successful threaded tests')
parser.add_option('-l', '--loop', type='int', default=1,
help='Repeat tests several times')
parser.add_option('-f', '--functional', action='store_true',
......@@ -280,6 +283,7 @@ Environment Variables:
sys.exit('Nothing to run, please give one of -f, -u, -z')
options.unit = options.functional = options.zodb = True
return dict(
log = options.log,
loop = options.loop,
unit = options.unit,
functional = options.functional,
......@@ -292,6 +296,8 @@ Environment Variables:
def start(self):
config = self._config
logging.backlog(max_packet=1<<20,
**({'max_size': None} if config.log else {}))
only = config.only
# run requested tests
runner = NeoTestRunner(config.title or 'Neo', config.verbosity)
......
......@@ -28,7 +28,6 @@ from neo.lib.util import dump
from neo.lib.bootstrap import BootstrapManager
from .checker import Checker
from .database import buildDatabaseManager
from .exception import AlreadyPendingError
from .handlers import identification, initialization
from .handlers import master, hidden
from .replicator import Replicator
......@@ -39,13 +38,14 @@ from neo.lib.debug import register as registerLiveDebugger
class Application(BaseApplication):
"""The storage node application."""
checker = replicator = tm = None
def __init__(self, config):
super(Application, self).__init__(
config.getSSL(), config.getDynamicMasterList())
# set the cluster name
self.name = config.getCluster()
self.tm = TransactionManager(self)
self.dm = buildDatabaseManager(config.getAdapter(),
(config.getDatabase(), config.getEngine(), config.getWait()),
)
......@@ -62,20 +62,13 @@ class Application(BaseApplication):
# partitions.
self.pt = None
self.checker = Checker(self)
self.replicator = Replicator(self)
self.listening_conn = None
self.master_conn = None
self.master_node = None
# operation related data
self.event_queue = None
self.event_queue_dict = None
self.operational = False
# ready is True when operational and got all informations
self.ready = False
self.dm.setup(reset=config.getReset())
self.loadConfiguration()
......@@ -95,9 +88,9 @@ class Application(BaseApplication):
def log(self):
self.em.log()
self.logQueuedEvents()
self.nm.log()
self.tm.log()
if self.tm:
self.tm.log()
if self.pt is not None:
self.pt.log()
......@@ -137,19 +130,17 @@ class Application(BaseApplication):
def loadPartitionTable(self):
"""Load a partition table from the database."""
self.pt.clear()
ptid = self.dm.getPTID()
cell_list = self.dm.getPartitionTable()
new_cell_list = []
for offset, uuid, state in cell_list:
# convert from int to Enum
state = CellStates[state]
if ptid is None:
return
cell_list = []
for offset, uuid, state in self.dm.getPartitionTable():
# register unknown nodes
if self.nm.getByUUID(uuid) is None:
self.nm.createStorage(uuid=uuid)
new_cell_list.append((offset, uuid, state))
# load the partition table in manager
self.pt.clear()
self.pt.update(ptid, new_cell_list, self.nm)
cell_list.append((offset, uuid, CellStates[state]))
self.pt.update(ptid, cell_list, self.nm)
def run(self):
try:
......@@ -173,10 +164,9 @@ class Application(BaseApplication):
# Connect to a primary master node, verify data, and
# start the operation. This cycle will be executed permanently,
# until the user explicitly requests a shutdown.
self.operational = False
while True:
self.cluster_state = None
self.ready = False
self.operational = False
if self.master_node is None:
# look for the primary master
self.connectToPrimary()
......@@ -184,13 +174,9 @@ class Application(BaseApplication):
node = self.nm.getByUUID(self.uuid)
if node is not None and node.isHidden():
self.wait()
# drop any client node
for conn in self.em.getConnectionList():
if conn not in (self.listening_conn, self.master_conn):
conn.close()
# create/clear event queue
self.event_queue = deque()
self.event_queue_dict = {}
self.checker = Checker(self)
self.replicator = Replicator(self)
self.tm = TransactionManager(self)
try:
self.initialize()
self.doOperation()
......@@ -200,7 +186,15 @@ class Application(BaseApplication):
except PrimaryFailure, msg:
logging.error('primary master is down: %s', msg)
finally:
self.checker = Checker(self)
self.operational = False
# When not ready, we reject any incoming connection so for
# consistency, we also close any connection except that to the
# master. This includes connections to other storage nodes and any
# replication is aborted, whether we are feeding or out-of-date.
for conn in self.em.getConnectionList():
if conn not in (self.listening_conn, self.master_conn):
conn.close()
del self.checker, self.replicator, self.tm
def connectToPrimary(self):
"""Find a primary master node, and connect to it.
......@@ -212,11 +206,6 @@ class Application(BaseApplication):
at this stage."""
pt = self.pt
# First of all, make sure that I have no connection.
for conn in self.em.getConnectionList():
if not conn.isListening():
conn.close()
# search, find, connect and identify to the primary master
bootstrap = BootstrapManager(self, NodeTypes.STORAGE, self.server)
self.master_node, self.master_conn, num_partitions, num_replicas = \
......@@ -246,9 +235,7 @@ class Application(BaseApplication):
self.master_conn.setHandler(initialization.InitializationHandler(self))
while not self.operational:
_poll()
self.ready = True
self.replicator.populate()
self.master_conn.notify(Packets.NotifyReady())
self.master_conn.send(Packets.NotifyReady())
def doOperation(self):
"""Handle everything, including replications and transactions."""
......@@ -258,12 +245,11 @@ class Application(BaseApplication):
_poll = self.em._poll
isIdle = self.em.isIdle
handler = master.MasterOperationHandler(self)
self.master_conn.setHandler(handler)
self.master_conn.setHandler(master.MasterOperationHandler(self))
self.replicator.populate()
# Forget all unfinished data.
self.dm.dropUnfinishedData()
self.tm.reset()
self.task_queue = task_queue = deque()
try:
......@@ -282,13 +268,6 @@ class Application(BaseApplication):
poll()
finally:
del self.task_queue
# XXX: Although no handled exception should happen between
# replicator.populate() and the beginning of this 'try'
# clause, the replicator should be reset in a safer place.
self.replicator = Replicator(self)
# Abort any replication, whether we are feeding or out-of-date.
for node in self.nm.getStorageList(only_identified=True):
node.getConnection().close()
def changeClusterState(self, state):
self.cluster_state = state
......@@ -309,46 +288,6 @@ class Application(BaseApplication):
if not node.isHidden():
break
def queueEvent(self, some_callable, conn=None, args=(), key=None,
raise_on_duplicate=True):
event_queue_dict = self.event_queue_dict
n = event_queue_dict.get(key)
if n and raise_on_duplicate:
raise AlreadyPendingError()
msg_id = None if conn is None else conn.getPeerId()
self.event_queue.append((key, some_callable, msg_id, conn, args))
if key is not None:
event_queue_dict[key] = n + 1 if n else 1
def executeQueuedEvents(self):
p = self.event_queue.popleft
event_queue_dict = self.event_queue_dict
for _ in xrange(len(self.event_queue)):
key, some_callable, msg_id, conn, args = p()
if key is not None:
n = event_queue_dict[key] - 1
if n:
event_queue_dict[key] = n
else:
del event_queue_dict[key]
if conn is None:
some_callable(*args)
elif not conn.isClosed():
orig_msg_id = conn.getPeerId()
try:
conn.setPeerId(msg_id)
some_callable(conn, *args)
finally:
conn.setPeerId(orig_msg_id)
def logQueuedEvents(self):
if self.event_queue is None:
return
logging.info("Pending events:")
for key, event, _msg_id, _conn, args in self.event_queue:
logging.info(' %r:%r: %r:%r %r %r', key, event.__name__,
_msg_id, _conn, args)
def newTask(self, iterator):
try:
iterator.next()
......
......@@ -109,7 +109,7 @@ class Checker(object):
self.source = source
def start():
if app.tm.isLockedTid(max_tid):
app.queueEvent(start)
app.tm.read_queue.queueEvent(start)
return
args = partition, CHECK_COUNT, min_tid, max_tid
p = Packets.AskCheckTIDRange(*args)
......@@ -181,7 +181,7 @@ class Checker(object):
uuid_list.append(conn.getUUID())
self.app.closeClient(conn)
p = Packets.NotifyPartitionCorrupted(self.partition, uuid_list)
self.app.master_conn.notify(p)
self.app.master_conn.send(p)
if len(self.conn_dict) <= 1:
logging.warning("check of partition %u aborted", self.partition)
self.queue.clear()
......
......@@ -283,11 +283,11 @@ class ImporterDatabaseManager(DatabaseManager):
super(ImporterDatabaseManager, self).__init__(*args, **kw)
implements(self, """_getNextTID checkSerialRange checkTIDRange
deleteObject deleteTransaction dropPartitions getLastTID
getReplicationObjectList getTIDList nonempty""".split())
getReplicationObjectList _getTIDList nonempty""".split())
_uncommitted_data = property(
lambda self: self.db._uncommitted_data,
lambda self, value: setattr(self.db, "_uncommitted_data", value))
_getPartition = property(lambda self: self.db._getPartition)
_getReadablePartition = property(lambda self: self.db._getReadablePartition)
_uncommitted_data = property(lambda self: self.db._uncommitted_data)
def _parse(self, database):
config = SafeConfigParser()
......@@ -300,11 +300,12 @@ class ImporterDatabaseManager(DatabaseManager):
self.compress = main.get('compress', 1)
self.db = buildDatabaseManager(main['adapter'],
(main['database'], main.get('engine'), main['wait']))
for x in """query erase getConfiguration _setConfiguration
getPartitionTable changePartitionTable
for x in """getConfiguration _setConfiguration setNumPartitions
query erase getPartitionTable changePartitionTable
getUnfinishedTIDDict dropUnfinishedData abortTransaction
storeTransaction lockTransaction unlockTransaction
storeData getOrphanList _pruneData deferCommit
loadData storeData getOrphanList _pruneData deferCommit
dropPartitionsTemporary
""".split():
setattr(self, x, getattr(self.db, x))
......@@ -315,21 +316,14 @@ class ImporterDatabaseManager(DatabaseManager):
self.db.commit()
self._last_commit = time.time()
def setNumPartitions(self, num_partitions):
self.db.setNumPartitions(num_partitions)
try:
del self._getPartition
except AttributeError:
pass
def close(self):
self.db.close()
if isinstance(self.zodb, list): # _setup called
for zodb in self.zodb:
zodb.close()
def _setup(self):
self.db._setup()
def setup(self, reset=0):
self.db.setup(reset)
zodb_state = self.getConfiguration("zodb")
if zodb_state:
logging.warning("Ignoring configuration file for oid mapping."
......
......@@ -14,14 +14,14 @@
# You should have received a copy of the GNU General Public License
# along with this program. If not, see <http://www.gnu.org/licenses/>.
import threading
import struct, threading
from collections import defaultdict
from contextlib import contextmanager
from functools import wraps
from neo.lib import logging, util
from neo.lib.exception import DatabaseFailure
from neo.lib.interfaces import abstract, requires
from neo.lib.protocol import ZERO_TID
from neo.lib.protocol import CellStates, NonReadableCell, ZERO_TID
X = 0
......@@ -59,7 +59,7 @@ class DatabaseManager(object):
_deferred = 0
_duplicating = _repairing = None
def __init__(self, database, engine=None, wait=0):
def __init__(self, database, engine=None, wait=None):
"""
Initialize the object.
"""
......@@ -68,18 +68,16 @@ class DatabaseManager(object):
raise ValueError("Unsupported engine: %r not in %r"
% (engine, self.ENGINES))
self._engine = engine
self._wait = wait
# XXX: Maybe the default should be to retry indefinitely.
# But for unit tests, we really want to never retry.
self._wait = wait or 0
self._parse(database)
self._connect()
def __getattr__(self, attr):
if attr == "_getPartition":
np = self.getNumPartitions()
value = lambda x: x % np
elif self._duplicating is None:
if self._duplicating is None:
return self.__getattribute__(attr)
else:
value = getattr(self._duplicating, attr)
value = getattr(self._duplicating, attr)
setattr(self, attr, value)
return value
......@@ -105,19 +103,10 @@ class DatabaseManager(object):
def _connect(self):
"""Connect to the database"""
def setup(self, reset=0):
"""Set up a database, discarding existing data first if reset is True
"""
if reset:
self.erase()
self._uncommitted_data = defaultdict(int)
self._setup()
@abstract
def erase(self):
""""""
@abstract
def _setup(self):
"""To be overridden by the backend to set up a database
......@@ -128,6 +117,16 @@ class DatabaseManager(object):
Keys are data ids and values are number of references.
"""
@requires(_setup)
def setup(self, reset=0):
"""Set up a database, discarding existing data first if reset is True
"""
if reset:
self.erase()
self._readable_set = set()
self._uncommitted_data = defaultdict(int)
self._setup()
@abstract
def nonempty(self, table):
"""Check whether table is empty or return None if it does not exist"""
......@@ -222,7 +221,7 @@ class DatabaseManager(object):
"""
self.setConfiguration('partitions', num_partitions)
try:
del self._getPartition
del self._getPartition, self._getReadablePartition
except AttributeError:
pass
......@@ -260,15 +259,6 @@ class DatabaseManager(object):
if ptid is not None:
return int(ptid)
def setPTID(self, ptid):
"""
Store a Partition Table ID into a database.
"""
if ptid is not None:
assert isinstance(ptid, (int, long)), ptid
ptid = str(ptid)
self.setConfiguration('ptid', ptid)
def getBackupTID(self):
return util.bin(self.getConfiguration('backup_tid'))
......@@ -299,7 +289,7 @@ class DatabaseManager(object):
return -1
@abstract
def getPartitionTable(self):
def getPartitionTable(self, *nid):
"""Return a whole partition table as a sequence of rows. Each row
is again a tuple of an offset (row ID), the NID of a storage
node, and a cell state."""
......@@ -409,21 +399,67 @@ class DatabaseManager(object):
compression, checksum, data,
None if data_serial is None else util.p64(data_serial))
@abstract
def changePartitionTable(self, ptid, cell_list, reset=False):
@contextmanager
def replicated(self, offset):
readable_set = self._readable_set
assert offset not in readable_set
readable_set.add(offset)
try:
yield
finally:
readable_set.remove(offset)
def _changePartitionTable(self, cell_list, reset=False):
"""Change a part of a partition table. The list of cells is
a tuple of tuples, each of which consists of an offset (row ID),
the NID of a storage node, and a cell state. The Partition
Table ID must be stored as well. If reset is True, existing data
is first thrown away."""
the NID of a storage node, and a cell state. If reset is True,
existing data is first thrown away.
"""
@requires(_changePartitionTable)
def changePartitionTable(self, ptid, cell_list, reset=False):
readable_set = self._readable_set
if reset:
readable_set.clear()
np = self.getNumPartitions()
def _getPartition(x, np=np):
return x % np
def _getReadablePartition(x, np=np, r=readable_set):
x %= np
if x in r:
return x
raise NonReadableCell
self._getPartition = _getPartition
self._getReadablePartition = _getReadablePartition
me = self.getUUID()
for offset, nid, state in cell_list:
if nid == me:
if CellStates.UP_TO_DATE != state != CellStates.FEEDING:
readable_set.discard(offset)
else:
readable_set.add(offset)
self._changePartitionTable(cell_list, reset)
assert isinstance(ptid, (int, long)), ptid
self._setConfiguration('ptid', str(ptid))
@abstract
def dropPartitions(self, offset_list):
"""Delete all data for specified partitions"""
@abstract
def _getUnfinishedDataIdList(self):
"""Drop any unfinished data from a database."""
@requires(_getUnfinishedDataIdList)
def dropUnfinishedData(self):
"""Drop any unfinished data from a database."""
data_id_list = self._getUnfinishedDataIdList()
self.dropPartitionsTemporary()
self.releaseData(data_id_list, True)
self.commit()
@abstract
def dropPartitionsTemporary(self, offset_list=None):
"""Drop partitions from temporary tables"""
@abstract
def storeTransaction(self, tid, object_list, transaction, temporary = True):
......@@ -469,6 +505,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
......@@ -499,8 +540,7 @@ class DatabaseManager(object):
else:
del refcount[data_id]
if prune:
self._pruneData(data_id_list)
self.commit()
return self._pruneData(data_id_list)
@fallback
def _getDataTID(self, oid, tid=None, before_tid=None):
......@@ -570,11 +610,14 @@ class DatabaseManager(object):
return current_tid, current_tid
return current_tid, tid
if transaction_object:
current_tid = current_data_tid = u64(transaction_object[2])
try:
current_tid = current_data_tid = u64(transaction_object[2])
except struct.error:
current_tid = current_data_tid = tid
else:
current_tid, current_data_tid = getDataTID(before_tid=ltid)
if current_tid is None:
return (None, None, False)
if current_tid is None:
return None, None, False
found_undone_tid, undone_data_tid = getDataTID(tid=undone_tid)
assert found_undone_tid is not None, (oid, undone_tid)
is_current = undone_data_tid in (current_data_tid, tid)
......@@ -677,12 +720,20 @@ class DatabaseManager(object):
min_tid and min_oid and below max_tid, for given partition,
sorted in ascending order."""
@abstract
def getTIDList(self, offset, length, partition_list):
def _getTIDList(self, offset, length, partition_list):
"""Return a list of TIDs in ascending order from an offset,
at most the specified length. The list of partitions are passed
to filter out non-applicable TIDs."""
@requires(_getTIDList)
def getTIDList(self, offset, length, partition_list):
if partition_list:
if self._readable_set.issuperset(partition_list):
return map(util.p64, self._getTIDList(
offset, length, partition_list))
raise NonReadableCell
return ()
@abstract
def getReplicationTIDList(self, min_tid, max_tid, length, partition):
"""Return a list of TIDs in ascending order from an initial tid value,
......
......@@ -20,6 +20,9 @@ from MySQLdb import DataError, IntegrityError, \
OperationalError, ProgrammingError
from MySQLdb.constants.CR import SERVER_GONE_ERROR, SERVER_LOST
from MySQLdb.constants.ER import DATA_TOO_LONG, DUP_ENTRY, NO_SUCH_TABLE
# BBB: the following 2 constants were added to mysqlclient 1.3.8
DROP_LAST_PARTITION = 1508
SAME_NAME_PARTITION = 1517
from array import array
from hashlib import sha1
import os
......@@ -46,7 +49,7 @@ class MySQLDatabaseManager(DatabaseManager):
"""This class manages a database on MySQL."""
VERSION = 1
ENGINES = "InnoDB", "TokuDB"
ENGINES = "InnoDB", "RocksDB", "TokuDB"
_engine = ENGINES[0] # default engine
# Disabled even on MySQL 5.1-5.5 and MariaDB 5.2-5.3 because
......@@ -77,14 +80,21 @@ class MySQLDatabaseManager(DatabaseManager):
timeout_at = None
else:
timeout_at = time.time() + self._wait
last = None
while True:
try:
self.conn = MySQLdb.connect(**kwd)
break
except Exception:
if timeout_at is not None and time.time() >= timeout_at:
except Exception as e:
if None is not timeout_at <= time.time():
raise
logging.exception('Connection to MySQL failed, retrying.')
e = str(e)
if last == e:
log = logging.debug
else:
last = e
log = logging.exception
log('Connection to MySQL failed, retrying.')
time.sleep(1)
self._active = 0
self._config = {}
......@@ -110,6 +120,10 @@ class MySQLDatabaseManager(DatabaseManager):
if LOG_QUERIES:
logging.debug('querying %s...',
getPrintableQuery(query.split('\n', 1)[0][:70]))
# Try 3 times at most. When it fails too often for the same
# query then the disconnection is likely caused by this query.
# We don't want to enter into an infinite loop.
retry = 2
while 1:
conn = self.conn
try:
......@@ -121,12 +135,15 @@ class MySQLDatabaseManager(DatabaseManager):
for d in row])
for row in r.fetch_row(r.num_rows())])
break
except OperationalError, m:
if self._active or m[0] not in (SERVER_GONE_ERROR, SERVER_LOST):
except OperationalError as m:
code, m = m.args
if self._active or SERVER_GONE_ERROR != code != SERVER_LOST \
or not retry:
raise DatabaseFailure('MySQL error %d: %s\nQuery: %s'
% (m[0], m[1], getPrintableQuery(query[:1000])))
% (code, m, getPrintableQuery(query[:1000])))
logging.info('the MySQL server is gone; reconnecting')
self._connect()
retry -= 1
r = query.split(None, 1)[0]
if r in ("INSERT", "REPLACE", "DELETE", "UPDATE"):
self._active = 1
......@@ -145,8 +162,8 @@ class MySQLDatabaseManager(DatabaseManager):
def nonempty(self, table):
try:
return bool(self.query("SELECT 1 FROM %s LIMIT 1" % table))
except ProgrammingError, (code, _):
if code != NO_SUCH_TABLE:
except ProgrammingError as e:
if e.args[0] != NO_SUCH_TABLE:
raise
def _setup(self):
......@@ -276,13 +293,15 @@ class MySQLDatabaseManager(DatabaseManager):
sql = "REPLACE INTO config VALUES ('%s', '%s')" % (k, e(value))
try:
q(sql)
except DataError, (code, _):
if code != DATA_TOO_LONG or len(value) < 256 or key != "zodb":
except DataError as e:
if e.args[0] != DATA_TOO_LONG or len(value) < 256 or key != "zodb":
raise
q("ALTER TABLE config MODIFY value VARBINARY(%s) NULL" % len(value))
q(sql)
def getPartitionTable(self):
def getPartitionTable(self, *nid):
if nid:
return self.query("SELECT rid, state FROM pt WHERE nid=%u" % nid)
return self.query("SELECT * FROM pt")
def getLastTID(self, max_tid):
......@@ -312,7 +331,7 @@ class MySQLDatabaseManager(DatabaseManager):
# MariaDB is smart enough to realize that 'ttid' is constant.
r = self.query("SELECT tid FROM trans"
" WHERE `partition`=%s AND tid>=ttid AND ttid=%s LIMIT 1"
% (self._getPartition(ttid), ttid))
% (self._getReadablePartition(ttid), ttid))
if r:
return util.p64(r[0][0])
......@@ -321,7 +340,7 @@ class MySQLDatabaseManager(DatabaseManager):
r = self.query("SELECT tid FROM obj"
" WHERE `partition`=%d AND oid=%d"
" ORDER BY tid DESC LIMIT 1"
% (self._getPartition(oid), oid))
% (self._getReadablePartition(oid), oid))
return util.p64(r[0][0]) if r else None
def _getNextTID(self, *args): # partition, oid, tid
......@@ -337,7 +356,7 @@ class MySQLDatabaseManager(DatabaseManager):
def _getObject(self, oid, tid=None, before_tid=None):
q = self.query
partition = self._getPartition(oid)
partition = self._getReadablePartition(oid)
sql = ('SELECT tid, compression, data.hash, value, value_tid'
' FROM obj LEFT JOIN data ON (obj.data_id = data.id)'
' WHERE `partition` = %d AND oid = %d') % (partition, oid)
......@@ -360,7 +379,7 @@ class MySQLDatabaseManager(DatabaseManager):
return (serial, self._getNextTID(partition, oid, serial),
compression, checksum, data, value_serial)
def changePartitionTable(self, ptid, cell_list, reset=False):
def _changePartitionTable(self, cell_list, reset=False):
offset_list = []
q = self.query
if reset:
......@@ -376,7 +395,6 @@ class MySQLDatabaseManager(DatabaseManager):
q("INSERT INTO pt VALUES (%d, %d, %d)"
" ON DUPLICATE KEY UPDATE state = %d"
% (offset, nid, state, state))
self.setPTID(ptid)
if self._use_partition:
for offset in offset_list:
add = """ALTER TABLE %%s ADD PARTITION (
......@@ -384,8 +402,8 @@ class MySQLDatabaseManager(DatabaseManager):
for table in 'trans', 'obj':
try:
self.conn.query(add % table)
except OperationalError, (code, _):
if code != 1517: # duplicate partition name
except OperationalError as e:
if e.args[0] != SAME_NAME_PARTITION:
raise
def dropPartitions(self, offset_list):
......@@ -408,16 +426,19 @@ class MySQLDatabaseManager(DatabaseManager):
for table in 'trans', 'obj':
try:
self.conn.query(drop % table)
except OperationalError, (code, _):
if code != 1508: # already dropped
except OperationalError as e:
if e.args[0] != DROP_LAST_PARTITION:
raise
def dropUnfinishedData(self):
def _getUnfinishedDataIdList(self):
return [x for x, in self.query("SELECT data_id FROM tobj") if x]
def dropPartitionsTemporary(self, offset_list=None):
where = "" if offset_list is None else \
" WHERE `partition` IN (%s)" % ','.join(map(str, offset_list))
q = self.query
data_id_list = [x for x, in q("SELECT data_id FROM tobj") if x]
q("DELETE FROM tobj")
q("DELETE FROM ttrans")
self.releaseData(data_id_list, True)
q("DELETE FROM tobj" + where)
q("DELETE FROM ttrans" + where)
def storeTransaction(self, tid, object_list, transaction, temporary = True):
e = self.escape
......@@ -535,8 +556,8 @@ class MySQLDatabaseManager(DatabaseManager):
try:
self.query("INSERT INTO data VALUES (NULL, '%s', %d, '%s')" %
(checksum, compression, e(data)))
except IntegrityError, (code, _):
if code == DUP_ENTRY:
except IntegrityError as e:
if e.args[0] == DUP_ENTRY:
(r, d), = self.query("SELECT id, value FROM data"
" WHERE hash='%s' AND compression=%s"
% (checksum, compression))
......@@ -545,12 +566,21 @@ 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):
sql = ('SELECT tid, value_tid FROM obj'
' WHERE `partition` = %d AND oid = %d'
) % (self._getPartition(oid), oid)
) % (self._getReadablePartition(oid), oid)
if tid is not None:
sql += ' AND tid = %d' % tid
elif before_tid is not None:
......@@ -589,7 +619,6 @@ class MySQLDatabaseManager(DatabaseManager):
def deleteTransaction(self, tid):
tid = util.u64(tid)
getPartition = self._getPartition
self.query("DELETE FROM trans WHERE `partition`=%s AND tid=%s" %
(self._getPartition(tid), tid))
......@@ -623,7 +652,7 @@ class MySQLDatabaseManager(DatabaseManager):
q = self.query
r = q("SELECT oids, user, description, ext, packed, ttid"
" FROM trans WHERE `partition` = %d AND tid = %d"
% (self._getPartition(tid), tid))
% (self._getReadablePartition(tid), tid))
if not r and all:
r = q("SELECT oids, user, description, ext, packed, ttid"
" FROM ttrans WHERE tid = %d" % tid)
......@@ -643,7 +672,8 @@ class MySQLDatabaseManager(DatabaseManager):
" FROM obj LEFT JOIN data ON (obj.data_id = data.id)"
" WHERE `partition` = %d AND oid = %d AND tid >= %d"
" ORDER BY tid DESC LIMIT %d, %d" %
(self._getPartition(oid), oid, self._getPackTID(), offset, length))
(self._getReadablePartition(oid), oid,
self._getPackTID(), offset, length))
if r:
return [(p64(tid), length or 0) for tid, length in r]
......@@ -659,12 +689,11 @@ class MySQLDatabaseManager(DatabaseManager):
partition, u64(max_tid), min_tid, u64(min_oid), min_tid, length))
return [(p64(serial), p64(oid)) for serial, oid in r]
def getTIDList(self, offset, length, partition_list):
q = self.query
r = q("""SELECT tid FROM trans WHERE `partition` in (%s)
ORDER BY tid DESC LIMIT %d,%d""" \
% (','.join(map(str, partition_list)), offset, length))
return [util.p64(t[0]) for t in r]
def _getTIDList(self, offset, length, partition_list):
return (t[0] for t in self.query(
"SELECT tid FROM trans WHERE `partition` in (%s)"
" ORDER BY tid DESC LIMIT %d,%d"
% (','.join(map(str, partition_list)), offset, length)))
def getReplicationTIDList(self, min_tid, max_tid, length, partition):
u64 = util.u64
......@@ -694,7 +723,7 @@ class MySQLDatabaseManager(DatabaseManager):
# reference is just updated to point to the new data location.
value_serial = None
kw = {
'partition': self._getPartition(oid),
'partition': self._getReadablePartition(oid),
'oid': oid,
'orig_tid': orig_serial,
'max_tid': max_serial,
......@@ -718,7 +747,7 @@ class MySQLDatabaseManager(DatabaseManager):
p64 = util.p64
tid = util.u64(tid)
updatePackFuture = self._updatePackFuture
getPartition = self._getPartition
getPartition = self._getReadablePartition
q = self.query
self._setPackTID(tid)
for count, oid, max_serial in q("SELECT COUNT(*) - 1, oid, MAX(tid)"
......
......@@ -219,7 +219,9 @@ class SQLiteDatabaseManager(DatabaseManager):
else:
q("REPLACE INTO config VALUES (?,?)", (key, str(value)))
def getPartitionTable(self):
def getPartitionTable(self, *nid):
if nid:
return self.query("SELECT rid, state FROM pt WHERE nid=?", nid)
return self.query("SELECT * FROM pt")
# A test with a table of 20 million lines and SQLite 3.8.7.1 shows that
......@@ -260,7 +262,7 @@ class SQLiteDatabaseManager(DatabaseManager):
# even though ttid is a constant.
for tid, in self.query("SELECT tid FROM trans"
" WHERE partition=? AND tid>=? AND ttid=? LIMIT 1",
(self._getPartition(ttid), ttid, ttid)):
(self._getReadablePartition(ttid), ttid, ttid)):
return util.p64(tid)
def getLastObjectTID(self, oid):
......@@ -268,7 +270,7 @@ class SQLiteDatabaseManager(DatabaseManager):
r = self.query("SELECT tid FROM obj"
" WHERE partition=? AND oid=?"
" ORDER BY tid DESC LIMIT 1",
(self._getPartition(oid), oid)).fetchone()
(self._getReadablePartition(oid), oid)).fetchone()
return r and util.p64(r[0])
def _getNextTID(self, *args): # partition, oid, tid
......@@ -279,7 +281,7 @@ class SQLiteDatabaseManager(DatabaseManager):
def _getObject(self, oid, tid=None, before_tid=None):
q = self.query
partition = self._getPartition(oid)
partition = self._getReadablePartition(oid)
sql = ('SELECT tid, compression, data.hash, value, value_tid'
' FROM obj LEFT JOIN data ON obj.data_id = data.id'
' WHERE partition=? AND oid=?')
......@@ -300,7 +302,7 @@ class SQLiteDatabaseManager(DatabaseManager):
return (serial, self._getNextTID(partition, oid, serial),
compression, checksum, data, value_serial)
def changePartitionTable(self, ptid, cell_list, reset=False):
def _changePartitionTable(self, cell_list, reset=False):
q = self.query
if reset:
q("DELETE FROM pt")
......@@ -315,7 +317,6 @@ class SQLiteDatabaseManager(DatabaseManager):
if state != CellStates.DISCARDED:
q("INSERT OR FAIL INTO pt VALUES (?,?,?)",
(offset, nid, int(state)))
self.setPTID(ptid)
def dropPartitions(self, offset_list):
where = " WHERE partition=?"
......@@ -328,12 +329,15 @@ class SQLiteDatabaseManager(DatabaseManager):
q("DELETE FROM trans" + where, args)
self._pruneData(data_id_list)
def dropUnfinishedData(self):
def _getUnfinishedDataIdList(self):
return [x for x, in self.query("SELECT data_id FROM tobj") if x]
def dropPartitionsTemporary(self, offset_list=None):
where = "" if offset_list is None else \
" WHERE `partition` IN (%s)" % ','.join(map(str, offset_list))
q = self.query
data_id_list = [x for x, in q("SELECT data_id FROM tobj") if x]
q("DELETE FROM tobj")
q("DELETE FROM ttrans")
self.releaseData(data_id_list, True)
q("DELETE FROM tobj" + where)
q("DELETE FROM ttrans" + where)
def storeTransaction(self, tid, object_list, transaction, temporary=True):
u64 = util.u64
......@@ -404,8 +408,12 @@ 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)
partition = self._getReadablePartition(oid)
sql = 'SELECT tid, value_tid FROM obj' \
' WHERE partition=? AND oid=?'
if tid is not None:
......@@ -447,7 +455,6 @@ class SQLiteDatabaseManager(DatabaseManager):
def deleteTransaction(self, tid):
tid = util.u64(tid)
getPartition = self._getPartition
self.query("DELETE FROM trans WHERE partition=? AND tid=?",
(self._getPartition(tid), tid))
......@@ -486,7 +493,7 @@ class SQLiteDatabaseManager(DatabaseManager):
q = self.query
r = q("SELECT oids, user, description, ext, packed, ttid"
" FROM trans WHERE partition=? AND tid=?",
(self._getPartition(tid), tid)).fetchone()
(self._getReadablePartition(tid), tid)).fetchone()
if not r and all:
r = q("SELECT oids, user, description, ext, packed, ttid"
" FROM ttrans WHERE tid=?", (tid,)).fetchone()
......@@ -506,7 +513,8 @@ class SQLiteDatabaseManager(DatabaseManager):
FROM obj LEFT JOIN data ON obj.data_id = data.id
WHERE partition=? AND oid=? AND tid>=?
ORDER BY tid DESC LIMIT ?,?""",
(self._getPartition(oid), oid, self._getPackTID(), offset, length))
(self._getReadablePartition(oid), oid,
self._getPackTID(), offset, length))
] or None
def getReplicationObjectList(self, min_tid, max_tid, length, partition,
......@@ -521,12 +529,11 @@ class SQLiteDatabaseManager(DatabaseManager):
ORDER BY tid ASC, oid ASC LIMIT ?""",
(partition, u64(max_tid), min_tid, u64(min_oid), min_tid, length))]
def getTIDList(self, offset, length, partition_list):
p64 = util.p64
return [p64(t[0]) for t in self.query("""\
SELECT tid FROM trans WHERE partition in (%s)
ORDER BY tid DESC LIMIT %d,%d"""
% (','.join(map(str, partition_list)), offset, length))]
def _getTIDList(self, offset, length, partition_list):
return (t[0] for t in self.query(
"SELECT tid FROM trans WHERE `partition` in (%s)"
" ORDER BY tid DESC LIMIT %d,%d"
% (','.join(map(str, partition_list)), offset, length)))
def getReplicationTIDList(self, min_tid, max_tid, length, partition):
u64 = util.u64
......@@ -544,7 +551,7 @@ class SQLiteDatabaseManager(DatabaseManager):
# transaction referencing its value at max_serial or above.
# If there is, copy value to the first future transaction. Any further
# reference is just updated to point to the new data location.
partition = self._getPartition(oid)
partition = self._getReadablePartition(oid)
value_serial = None
q = self.query
for T in '', 't':
......@@ -565,7 +572,7 @@ class SQLiteDatabaseManager(DatabaseManager):
p64 = util.p64
tid = util.u64(tid)
updatePackFuture = self._updatePackFuture
getPartition = self._getPartition
getPartition = self._getReadablePartition
q = self.query
self._setPackTID(tid)
for count, oid, max_serial in q("SELECT COUNT(*) - 1, oid, MAX(tid)"
......
#
# Copyright (C) 2010-2017 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
# as published by the Free Software Foundation; either version 2
# of the License, or (at your option) any later version.
#
# This program is distributed in the hope that it will be useful,
# but WITHOUT ANY WARRANTY; without even the implied warranty of
# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
# GNU General Public License for more details.
#
# You should have received a copy of the GNU General Public License
# along with this program. If not, see <http://www.gnu.org/licenses/>.
class AlreadyPendingError(Exception):
pass
......@@ -18,9 +18,20 @@ import weakref
from neo.lib import logging
from neo.lib.handler import EventHandler
from neo.lib.exception import PrimaryFailure, StoppedOperation
from neo.lib.protocol import uuid_str, NodeStates, NodeTypes, Packets
from neo.lib.protocol import (uuid_str,
NodeStates, NodeTypes, Packets, ProtocolError)
class BaseMasterHandler(EventHandler):
class BaseHandler(EventHandler):
def notifyTransactionFinished(self, conn, ttid, max_tid):
app = self.app
app.tm.abort(ttid)
app.replicator.transactionFinished(ttid, max_tid)
def abortTransaction(self, conn, ttid, _):
self.notifyTransactionFinished(conn, ttid, None)
class BaseMasterHandler(BaseHandler):
def connectionLost(self, conn, new_state):
if self.app.listening_conn: # if running
......@@ -36,10 +47,11 @@ class BaseMasterHandler(EventHandler):
def notifyClusterInformation(self, conn, state):
self.app.changeClusterState(state)
def notifyNodeInformation(self, conn, node_list):
def notifyNodeInformation(self, conn, timestamp, node_list):
"""Store information on nodes, only if this is sent by a primary
master node."""
super(BaseMasterHandler, self).notifyNodeInformation(conn, node_list)
super(BaseMasterHandler, self).notifyNodeInformation(
conn, timestamp, node_list)
for node_type, _, uuid, state, _ in node_list:
if uuid == self.app.uuid:
# This is me, do what the master tell me
......@@ -55,8 +67,17 @@ class BaseMasterHandler(EventHandler):
uuid_str(uuid))
self.app.tm.abortFor(uuid)
def answerUnfinishedTransactions(self, conn, *args, **kw):
self.app.replicator.setUnfinishedTIDList(*args, **kw)
def notifyPartitionChanges(self, conn, ptid, cell_list):
"""This is very similar to Send Partition Table, except that
the information is only about changes from the previous."""
app = self.app
if ptid != 1 + app.pt.getID():
raise ProtocolError('wrong partition table id')
app.pt.update(ptid, cell_list, app.nm)
app.dm.changePartitionTable(ptid, cell_list)
if app.operational:
app.replicator.notifyPartitionChanges(cell_list)
app.dm.commit()
def askFinalTID(self, conn, ttid):
conn.answer(Packets.AnswerFinalTID(self.app.dm.getFinalTID(ttid)))
......
......@@ -15,19 +15,19 @@
# along with this program. If not, see <http://www.gnu.org/licenses/>.
from neo.lib import logging
from neo.lib.handler import EventHandler
from neo.lib.handler import DelayEvent
from neo.lib.util import dump, makeChecksum, add64
from neo.lib.protocol import Packets, LockState, Errors, ProtocolError, \
from neo.lib.protocol import Packets, Errors, NonReadableCell, ProtocolError, \
ZERO_HASH, INVALID_PARTITION
from ..transactions import ConflictError, DelayedError, NotRegisteredError
from ..exception import AlreadyPendingError
from ..transactions import ConflictError, NotRegisteredError
from . import BaseHandler
import time
# Log stores taking (incl. lock delays) more than this many seconds.
# Set to None to disable.
SLOW_STORE = 2
class ClientOperationHandler(EventHandler):
class ClientOperationHandler(BaseHandler):
def askTransactionInformation(self, conn, tid):
t = self.app.dm.getTransaction(tid)
......@@ -38,12 +38,14 @@ class ClientOperationHandler(EventHandler):
t[4], t[0])
conn.answer(p)
def getEventQueue(self):
# for read rpc
return self.app.tm.read_queue
def askObject(self, conn, oid, serial, tid):
app = self.app
if app.tm.loadLocked(oid):
# Delay the response.
app.queueEvent(self.askObject, conn, (oid, serial, tid))
return
raise DelayEvent
o = app.dm.getObject(oid, serial, tid)
try:
serial, next_serial, compression, checksum, data, data_serial = o
......@@ -58,9 +60,6 @@ class ClientOperationHandler(EventHandler):
compression, checksum, data, data_serial)
conn.answer(p)
def abortTransaction(self, conn, ttid):
self.app.tm.abort(ttid)
def askStoreTransaction(self, conn, ttid, *txn_info):
self.app.tm.register(conn, ttid)
self.app.tm.vote(ttid, txn_info)
......@@ -71,41 +70,31 @@ class ClientOperationHandler(EventHandler):
conn.answer(Packets.AnswerVoteTransaction())
def _askStoreObject(self, conn, oid, serial, compression, checksum, data,
data_serial, ttid, unlock, request_time):
data_serial, ttid, request_time):
try:
self.app.tm.storeObject(ttid, serial, oid, compression,
checksum, data, data_serial, unlock)
checksum, data, data_serial)
except ConflictError, err:
# resolvable or not
conn.answer(Packets.AnswerStoreObject(1, oid, err.tid))
except DelayedError:
# locked by a previous transaction, retry later
# If we are unlocking, we want queueEvent to raise
# AlreadyPendingError, to avoid making client wait for an unneeded
# response.
try:
self.app.queueEvent(self._askStoreObject, conn, (oid, serial,
compression, checksum, data, data_serial, ttid,
unlock, request_time), key=(oid, ttid),
raise_on_duplicate=unlock)
except AlreadyPendingError:
conn.answer(Errors.AlreadyPending(dump(oid)))
conn.answer(Packets.AnswerStoreObject(err.tid))
return
except NonReadableCell:
logging.info('Ignore store of %s:%s by %s: unassigned partition',
dump(oid), dump(serial), dump(ttid))
except NotRegisteredError:
# transaction was aborted, cancel this event
logging.info('Forget store of %s:%s by %s delayed by %s',
dump(oid), dump(serial), dump(ttid),
dump(self.app.tm.getLockingTID(oid)))
# send an answer as the client side is waiting for it
conn.answer(Packets.AnswerStoreObject(0, oid, serial))
else:
if SLOW_STORE is not None:
if request_time and SLOW_STORE is not None:
duration = time.time() - request_time
if duration > SLOW_STORE:
logging.info('StoreObject delay: %.02fs', duration)
conn.answer(Packets.AnswerStoreObject(0, oid, serial))
conn.answer(Packets.AnswerStoreObject(None))
def askStoreObject(self, conn, oid, serial,
compression, checksum, data, data_serial, ttid, unlock):
compression, checksum, data, data_serial, ttid):
if 1 < compression:
raise ProtocolError('invalid compression value')
# register the transaction
......@@ -116,8 +105,34 @@ class ClientOperationHandler(EventHandler):
assert data_serial is None
else:
checksum = data = None
self._askStoreObject(conn, oid, serial, compression, checksum, data,
data_serial, ttid, unlock, time.time())
try:
self._askStoreObject(conn, oid, serial, compression,
checksum, data, data_serial, ttid, None)
except DelayEvent, e:
# locked by a previous transaction, retry later
self.app.tm.queueEvent(self._askStoreObject, conn, (oid, serial,
compression, checksum, data, data_serial, ttid, time.time()),
*e.args)
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, e:
# locked by a previous transaction, retry later
self.app.tm.queueEvent(self._askRebaseObject,
conn, (ttid, oid, time.time()), *e.args)
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(
......@@ -159,25 +174,12 @@ class ClientOperationHandler(EventHandler):
p = Packets.AnswerObjectUndoSerial(object_tid_dict)
conn.answer(p)
def askHasLock(self, conn, ttid, oid):
locking_tid = self.app.tm.getLockingTID(oid)
logging.info('%r check lock of %r:%r', conn, dump(ttid), dump(oid))
if locking_tid is None:
state = LockState.NOT_LOCKED
elif locking_tid is ttid:
state = LockState.GRANTED
else:
state = LockState.GRANTED_TO_OTHER
conn.answer(Packets.AnswerHasLock(oid, state))
def askObjectHistory(self, conn, oid, first, last):
if first >= last:
raise ProtocolError('invalid offsets')
app = self.app
if app.tm.loadLocked(oid):
# Delay the response.
app.queueEvent(self.askObjectHistory, conn, (oid, first, last))
return
raise DelayEvent
history_list = app.dm.getObjectHistory(oid, first, last - first)
if history_list is None:
p = Errors.OidNotFound(dump(oid))
......@@ -185,36 +187,36 @@ class ClientOperationHandler(EventHandler):
p = Packets.AnswerObjectHistory(oid, history_list)
conn.answer(p)
def askCheckCurrentSerial(self, conn, ttid, serial, oid):
def askCheckCurrentSerial(self, conn, ttid, oid, serial):
self.app.tm.register(conn, ttid)
self._askCheckCurrentSerial(conn, ttid, serial, oid, time.time())
try:
self._askCheckCurrentSerial(conn, ttid, oid, serial, None)
except DelayEvent, e:
# locked by a previous transaction, retry later
self.app.tm.queueEvent(self._askCheckCurrentSerial,
conn, (ttid, oid, serial, time.time()), *e.args)
def _askCheckCurrentSerial(self, conn, ttid, serial, oid, request_time):
def _askCheckCurrentSerial(self, conn, ttid, oid, serial, request_time):
try:
self.app.tm.checkCurrentSerial(ttid, serial, oid)
self.app.tm.checkCurrentSerial(ttid, oid, serial)
except ConflictError, err:
# resolvable or not
conn.answer(Packets.AnswerCheckCurrentSerial(1, oid, err.tid))
except DelayedError:
# locked by a previous transaction, retry later
try:
self.app.queueEvent(self._askCheckCurrentSerial, conn, (ttid,
serial, oid, request_time), key=(oid, ttid))
except AlreadyPendingError:
conn.answer(Errors.AlreadyPending(dump(oid)))
conn.answer(Packets.AnswerCheckCurrentSerial(err.tid))
return
except NonReadableCell:
logging.info('Ignore check of %s:%s by %s: unassigned partition',
dump(oid), dump(serial), dump(ttid))
except NotRegisteredError:
# transaction was aborted, cancel this event
logging.info('Forget serial check of %s:%s by %s delayed by %s',
dump(oid), dump(serial), dump(ttid),
dump(self.app.tm.getLockingTID(oid)))
# send an answer as the client side is waiting for it
conn.answer(Packets.AnswerCheckCurrentSerial(0, oid, serial))
else:
if SLOW_STORE is not None:
if request_time and SLOW_STORE is not None:
duration = time.time() - request_time
if duration > SLOW_STORE:
logging.info('CheckCurrentSerial delay: %.02fs', duration)
conn.answer(Packets.AnswerCheckCurrentSerial(0, oid, serial))
conn.answer(Packets.AnswerCheckCurrentSerial(None))
# like ClientOperationHandler but read-only & only for tid <= backup_tid
......@@ -224,11 +226,12 @@ class ClientReadOnlyOperationHandler(ClientOperationHandler):
conn.answer(Errors.ReadOnlyAccess(
'read-only access because cluster is in backuping mode'))
abortTransaction = _readOnly
askStoreTransaction = _readOnly
askVoteTransaction = _readOnly
askStoreObject = _readOnly
askFinalTID = _readOnly
askRebaseObject = _readOnly
askRebaseTransaction = _readOnly
# takes write lock & is only used when going to commit
askCheckCurrentSerial = _readOnly
......
......@@ -27,12 +27,16 @@ class IdentificationHandler(EventHandler):
def connectionLost(self, conn, new_state):
logging.warning('A connection was lost during identification')
def getEventQueue(self):
# for requestIdentification
return self.app.nm
def requestIdentification(self, conn, node_type, uuid, address, name,
id_timestamp):
self.checkClusterName(name)
app = self.app
# reject any incoming connections if not ready
if not app.ready:
if not app.operational:
raise NotReadyError
if uuid is None:
if node_type != NodeTypes.STORAGE:
......@@ -43,12 +47,6 @@ class IdentificationHandler(EventHandler):
if uuid == app.uuid:
raise ProtocolError("uuid conflict or loopback connection")
node = app.nm.getByUUID(uuid, id_timestamp)
if node is None:
# Do never create node automatically, or we could get id
# conflicts. We must only rely on the notifications from the
# master to recognize nodes. So this is not always an error:
# maybe there are incoming notifications.
raise NotReadyError('unknown node: retry later')
if node.isBroken():
raise BrokenNodeDisallowedError
# choose the handler according to the node type
......
......@@ -23,24 +23,26 @@ class InitializationHandler(BaseMasterHandler):
def sendPartitionTable(self, conn, ptid, row_list):
app = self.app
pt = app.pt
pt.load(ptid, row_list, self.app.nm)
pt.load(ptid, row_list, app.nm)
if not pt.filled():
raise ProtocolError('Partial partition table received')
# Install the partition table into the database for persistence.
cell_list = []
num_partitions = pt.getPartitions()
unassigned_set = set(xrange(num_partitions))
for offset in xrange(num_partitions):
offset_list = xrange(pt.getPartitions())
unassigned_set = set(offset_list)
for offset in offset_list:
for cell in pt.getCellList(offset):
cell_list.append((offset, cell.getUUID(), cell.getState()))
if cell.getUUID() == app.uuid:
unassigned_set.remove(offset)
# delete objects database
dm = app.dm
if unassigned_set:
logging.debug('drop data for partitions %r', unassigned_set)
app.dm.dropPartitions(unassigned_set)
dm.dropPartitions(unassigned_set)
app.dm.changePartitionTable(ptid, cell_list, reset=True)
dm.changePartitionTable(ptid, cell_list, reset=True)
dm.commit()
def truncate(self, conn, tid):
dm = self.app.dm
......
......@@ -31,25 +31,6 @@ class MasterOperationHandler(BaseMasterHandler):
dm._setBackupTID(dm.getLastIDs()[0] or ZERO_TID)
dm.commit()
def notifyTransactionFinished(self, conn, *args, **kw):
self.app.replicator.transactionFinished(*args, **kw)
def notifyPartitionChanges(self, conn, ptid, cell_list):
"""This is very similar to Send Partition Table, except that
the information is only about changes from the previous."""
app = self.app
if ptid <= app.pt.getID():
# Ignore this packet.
logging.debug('ignoring older partition changes')
return
# update partition table in memory and the database
app.pt.update(ptid, cell_list, app.nm)
app.dm.changePartitionTable(ptid, cell_list)
# Check changes for replications
app.replicator.notifyPartitionChanges(cell_list)
def askLockInformation(self, conn, ttid, tid):
self.app.tm.lock(ttid, tid)
conn.answer(Packets.AnswerInformationLocked(ttid))
......@@ -64,6 +45,9 @@ class MasterOperationHandler(BaseMasterHandler):
logging.info('Pack finished.')
conn.answer(Packets.AnswerPack(True))
def answerUnfinishedTransactions(self, conn, *args, **kw):
self.app.replicator.setUnfinishedTIDList(*args, **kw)
def replicate(self, conn, tid, upstream_name, source_dict):
self.app.replicator.backup(tid, {p: a and (a, upstream_name)
for p, a in source_dict.iteritems()})
......
......@@ -17,13 +17,13 @@
import weakref
from functools import wraps
from neo.lib.connection import ConnectionClosed
from neo.lib.handler import EventHandler
from neo.lib.handler import DelayEvent, EventHandler
from neo.lib.protocol import Errors, NodeStates, Packets, ProtocolError, \
ZERO_HASH
def checkConnectionIsReplicatorConnection(func):
def wrapper(self, conn, *args, **kw):
if self.app.replicator.getCurrentConnection() is conn:
if self.app.replicator.isReplicatingConnection(conn):
return func(self, conn, *args, **kw)
return wraps(func)(wrapper)
......@@ -46,7 +46,7 @@ class StorageOperationHandler(EventHandler):
def connectionLost(self, conn, new_state):
app = self.app
if app.listening_conn and conn.isClient():
if app.operational and conn.isClient():
# XXX: Connection and Node should merged.
uuid = conn.getUUID()
if uuid:
......@@ -62,7 +62,7 @@ class StorageOperationHandler(EventHandler):
# Client
def connectionFailed(self, conn):
if self.app.listening_conn:
if self.app.operational:
self.app.replicator.abort()
def _acceptIdentification(self, node, *args):
......@@ -143,18 +143,20 @@ class StorageOperationHandler(EventHandler):
# Server (all methods must set connection as server so that it isn't closed
# if client tasks are finished)
def getEventQueue(self):
return self.app.tm
@checkFeedingConnection(check=True)
def askCheckTIDRange(self, conn, *args):
app = self.app
if app.tm.isLockedTid(args[3]): # max_tid
app.queueEvent(self.askCheckTIDRange, conn, args)
return
raise DelayEvent
msg_id = conn.getPeerId()
conn = weakref.proxy(conn)
def check():
r = app.dm.checkTIDRange(*args)
try:
conn.answer(Packets.AnswerCheckTIDRange(*r), msg_id) # NOTE msg_id: out-of-order answer
conn.send(Packets.AnswerCheckTIDRange(*r), msg_id) # NOTE msg_id: out-of-order answer
except (weakref.ReferenceError, ConnectionClosed):
pass
yield
......@@ -170,7 +172,7 @@ class StorageOperationHandler(EventHandler):
def check():
r = app.dm.checkSerialRange(*args)
try:
conn.answer(Packets.AnswerCheckSerialRange(*r), msg_id) # NOTE msg_id: out-of-order answer
conn.send(Packets.AnswerCheckSerialRange(*r), msg_id) # NOTE msg_id: out-of-order answer
except (weakref.ReferenceError, ConnectionClosed):
pass
yield
......@@ -188,9 +190,7 @@ class StorageOperationHandler(EventHandler):
# NotifyTransactionFinished(M->S) + AskFetchTransactions(S->S)
# is faster than
# NotifyUnlockInformation(M->S)
app.queueEvent(self.askFetchTransactions, conn,
(partition, length, min_tid, max_tid, tid_list))
return
raise DelayEvent
msg_id = conn.getPeerId()
conn = weakref.proxy(conn)
peer_tid_set = set(tid_list)
......@@ -207,14 +207,15 @@ class StorageOperationHandler(EventHandler):
else:
t = dm.getTransaction(tid)
if t is None:
conn.answer(Errors.ReplicationError(
"partition %u dropped" % partition))
conn.send(Errors.ReplicationError(
"partition %u dropped"
% partition), msg_id)
return
oid_list, user, desc, ext, packed, ttid = t
conn.notify(Packets.AddTransaction(
tid, user, desc, ext, packed, ttid, oid_list))
conn.send(Packets.AddTransaction(tid, user,
desc, ext, packed, ttid, oid_list), msg_id)
yield
conn.answer(Packets.AnswerFetchTransactions(
conn.send(Packets.AnswerFetchTransactions(
pack_tid, next_tid, peer_tid_set), msg_id) # NOTE msg_id: out-of-order answer
yield
except (weakref.ReferenceError, ConnectionClosed):
......@@ -251,12 +252,14 @@ class StorageOperationHandler(EventHandler):
continue
object = dm.getObject(oid, serial)
if not object:
conn.answer(Errors.ReplicationError(
"partition %u dropped or truncated" % partition))
conn.send(Errors.ReplicationError(
"partition %u dropped or truncated"
% partition), msg_id)
return
conn.notify(Packets.AddObject(oid, serial, *object[2:]))
conn.send(Packets.AddObject(oid, serial, *object[2:]),
msg_id)
yield
conn.answer(Packets.AnswerFetchObjects(
conn.send(Packets.AnswerFetchObjects(
pack_tid, next_tid, next_oid, object_dict), msg_id) # NOTE msg_id: out-of-order answer
yield
except (weakref.ReferenceError, ConnectionClosed):
......
......@@ -29,7 +29,7 @@ partitions.
2 parts, done sequentially:
- Transaction (metadata) replication
- Object (data) replication
- Object (metadata+data) replication
Both parts follow the same mechanism:
- The range of data to replicate is split into chunks of FETCH_COUNT items
......@@ -37,15 +37,52 @@ Both parts follow the same mechanism:
- For every chunk, the requesting node sends to seeding node the list of items
it already has.
- Before answering, the seeding node sends 1 packet for every missing item.
For items that are already on the replicating node, there is no check that
values matches.
- The seeding node finally answers with the list of items to delete (usually
empty).
Replication is partial, starting from the greatest stored tid in the partition:
- For transactions, this tid is excluded from replication.
- For objects, this tid is included unless the storage already knows it has
all oids for it.
There is no check that item values on both nodes matches.
Internal replication, which is similar to RAID1 (and as opposed to asynchronous
replication to a backup cluster) requires extra care with respect to
transactions. The transition of a cell from OUT_OF_DATE to UP_TO_DATE is done
is several steps.
A replicating node can not depend on other nodes to fetch the data
recently/being committed because that can not be done atomically: it could miss
writes between the processing of its request by a source node and the reception
of the answer.
Therefore, outdated cells are writable: a storage node asks the master for
transactions being committed and then it is expected to fully receive from the
client any transaction that is started after this answer.
Which has in turn other consequences:
- The client must not fail to write to a storage node after the above request
to the master: for this, the storage must have announced it is ready, and it
must delay identification of unknown clients (those for which it hasn't
received yet a notification from the master).
- Writes must be accepted blindly (i.e. without taking a write-lock) when a
storage node lacks the data to check for conflicts. This is possible because
1 up-to-date cell (for each partition) is enough to do these checks.
- Because the client can not reliably know if a storage node is expected to
receive a transaction in full, all writes must succeed.
- Even if the replication is finished, we have to wait that we don't have any
lockless writes left before announcing to the master that we're up-to-date.
To sum up:
1. ask unfinished transactions -> (last_transaction, ttid_list)
2. replicate to last_transaction
3. wait for all ttid_list to be finished -> new last_transaction
4. replicate to last_transaction
5. no lockless write anymore, except to (oid, ttid) that were already
stored/checked without taking a lock
6. wait for all transactions with lockless writes to be finished
7. announce we're up-to-date
For any failed write, the client marks the storage node as failed and stops
writing to it for the transaction. Unless there's no failed write, vote ends
with an extra request to the master: the transaction will only succeed if the
failed nodes can be disconnected, forcing them to replicate the missing data.
TODO: Packing and replication currently fail when they happen at the same time.
"""
......@@ -74,6 +111,12 @@ class Partition(object):
class Replicator(object):
# When the replication of a partition is aborted, the connection to the
# feeding node may still be open, e.g. on PT update from the master. In
# such case, replication is also aborted on the other side but there may
# be a few incoming packets that must be discarded.
_conn_msg_id = None
current_node = None
current_partition = None
......@@ -85,10 +128,9 @@ class Replicator(object):
if node is not None and node.isConnected(True):
return node.getConnection()
# XXX: We can't replicate unfinished transactions but do we need such
# complex code ? Backup mechanism does not rely on this: instead
# the upstream storage delays the answer. Maybe we can do the same
# for internal replication.
def isReplicatingConnection(self, conn):
return conn is self.getCurrentConnection() and \
conn.getPeerId() == self._conn_msg_id
def setUnfinishedTIDList(self, max_tid, ttid_list, offset_list):
"""This is a callback from MasterOperationHandler."""
......@@ -103,14 +145,39 @@ class Replicator(object):
self.replicate_dict[offset] = max_tid
self._nextPartition()
def transactionFinished(self, ttid, max_tid):
def transactionFinished(self, ttid, max_tid=None):
""" Callback from MasterOperationHandler """
self.ttid_set.remove(ttid)
try:
self.ttid_set.remove(ttid)
except KeyError:
assert max_tid is None, max_tid
return
min_ttid = min(self.ttid_set) if self.ttid_set else INVALID_TID
for offset, p in self.partition_dict.iteritems():
if p.max_ttid and p.max_ttid < min_ttid:
p.max_ttid = None
self.replicate_dict[offset] = max_tid
if p.max_ttid:
if max_tid:
# Filling replicate_dict while there are still unfinished
# transactions for this partition is not the most
# efficient (due to the overhead of potentially replicating
# the last transactions in several times), but that's a
# simple way to make sure it is filled even if the
# remaining unfinished transactions are aborted.
self.replicate_dict[offset] = max_tid
if p.max_ttid < min_ttid:
# no more unfinished transaction for this partition
if not (offset == self.current_partition
or offset in self.replicate_dict):
logging.debug(
"All unfinished transactions have been aborted."
" Mark partition %u as already fully replicated",
offset)
# We don't have anymore the previous value of
# self.replicate_dict[offset], but p.max_ttid is not
# wrong. Anyway here, we're not in backup mode and this
# value will be ignored.
# XXX: see NonReadableCell.__doc__
self.app.tm.replicated(offset, p.max_ttid)
p.max_ttid = None
self._nextPartition()
def getBackupTID(self):
......@@ -136,7 +203,7 @@ class Replicator(object):
app = self.app
pt = app.pt
uuid = app.uuid
self.partition_dict = p = {}
self.partition_dict = {}
self.replicate_dict = {}
self.source_dict = {}
self.ttid_set = set()
......@@ -160,13 +227,14 @@ class Replicator(object):
p.next_trans = p.next_obj = next_tid
p.max_ttid = None
if outdated_list:
self.app.master_conn.ask(Packets.AskUnfinishedTransactions(),
offset_list=outdated_list)
self.app.tm.replicating(outdated_list)
def notifyPartitionChanges(self, cell_list):
"""This is a callback from MasterOperationHandler."""
abort = False
added_list = []
discarded_list = []
readable_list = []
app = self.app
last_tid, last_trans_dict, last_obj_dict, _ = app.dm.getLastIDs()
for offset, uuid, state in cell_list:
......@@ -179,6 +247,7 @@ class Replicator(object):
self.replicate_dict.pop(offset, None)
self.source_dict.pop(offset, None)
abort = abort or self.current_partition == offset
discarded_list.append(offset)
elif state == CellStates.OUT_OF_DATE:
assert offset not in self.partition_dict
self.partition_dict[offset] = p = Partition()
......@@ -189,9 +258,17 @@ class Replicator(object):
p.next_obj = last_obj_dict.get(offset, ZERO_TID)
p.max_ttid = INVALID_TID
added_list.append(offset)
else:
assert state in (CellStates.UP_TO_DATE,
CellStates.FEEDING), state
readable_list.append(offset)
tm = app.tm
if added_list:
self.app.master_conn.ask(Packets.AskUnfinishedTransactions(),
offset_list=added_list)
tm.replicating(added_list)
if discarded_list:
tm.discarded(discarded_list)
if readable_list:
tm.readable(readable_list)
if abort:
self.abort()
......@@ -232,6 +309,8 @@ class Replicator(object):
if self.current_partition is not None or not self.replicate_dict:
return
app = self.app
assert app.master_conn and app.operational, (
app.master_conn, app.operational)
# Start replicating the partition which is furthest behind,
# to increase the overall backup_tid as soon as possible.
# Then prefer a partition with no unfinished transaction.
......@@ -241,7 +320,8 @@ class Replicator(object):
try:
addr, name = self.source_dict[offset]
except KeyError:
assert app.pt.getCell(offset, app.uuid).isOutOfDate()
assert app.pt.getCell(offset, app.uuid).isOutOfDate(), (
offset, app.pt.getCell(offset, app.uuid).getState())
node = random.choice([cell.getNode()
for cell in app.pt.getCellList(offset, readable=True)
if cell.getNodeState() == NodeStates.RUNNING])
......@@ -296,7 +376,7 @@ class Replicator(object):
max_tid = self.replicate_tid
tid_list = self.app.dm.getReplicationTIDList(min_tid, max_tid,
FETCH_COUNT, offset)
self.current_node.getConnection().ask(Packets.AskFetchTransactions(
self._conn_msg_id = self.current_node.ask(Packets.AskFetchTransactions(
offset, FETCH_COUNT, min_tid, max_tid, tid_list))
def fetchObjects(self, min_tid=None, min_oid=ZERO_OID):
......@@ -315,19 +395,21 @@ class Replicator(object):
object_dict[serial].append(oid)
except KeyError:
object_dict[serial] = [oid]
self.current_node.getConnection().ask(Packets.AskFetchObjects(
self._conn_msg_id = self.current_node.ask(Packets.AskFetchObjects(
offset, FETCH_COUNT, min_tid, max_tid, min_oid, object_dict))
def finish(self):
offset = self.current_partition
tid = self.replicate_tid
del self.current_partition, self.replicate_tid
del self.current_partition, self._conn_msg_id, self.replicate_tid
p = self.partition_dict[offset]
p.next_obj = add64(tid, 1)
self.updateBackupTID()
if not p.max_ttid:
p = Packets.NotifyReplicationDone(offset, tid)
self.app.master_conn.notify(p)
if p.max_ttid or offset in self.replicate_dict and \
offset not in self.source_dict:
logging.debug("unfinished transactions: %r", self.ttid_set)
else:
self.app.tm.replicated(offset, tid)
logging.debug("partition %u replicated up to %s from %r",
offset, dump(tid), self.current_node)
self.getCurrentConnection().setReconnectionNoDelay()
......@@ -338,6 +420,7 @@ class Replicator(object):
if offset is None:
return
del self.current_partition
self._conn_msg_id = None
logging.warning('replication aborted for partition %u%s',
offset, message and ' (%s)' % message)
if offset in self.partition_dict:
......@@ -354,24 +437,19 @@ class Replicator(object):
else: # partition removed
self._nextPartition()
def cancel(self):
offset = self.current_partition
if offset is not None:
logging.info('cancel replication of partition %u', offset)
del self.current_partition
try:
self.replicate_dict.setdefault(offset, self.replicate_tid)
del self.replicate_tid
except AttributeError:
pass
self.getCurrentConnection().close()
def stop(self):
# Close any open connection to an upstream storage,
# possibly aborting current replication.
node = self.current_node
if node is not None is node.getUUID():
self.cancel()
offset = self.current_partition
if offset is not None:
logging.info('cancel replication of partition %u', offset)
del self.current_partition
if self._conn_msg_id is not None:
self.replicate_dict.setdefault(offset, self.replicate_tid)
del self._conn_msg_id, self.replicate_tid
self.getCurrentConnection().close()
# Cancel all replication orders from upstream cluster.
for offset in self.replicate_dict.keys():
addr, name = self.source_dict.get(offset, (None, None))
......
......@@ -16,8 +16,10 @@
from time import time
from neo.lib import logging
from neo.lib.handler import DelayEvent, EventQueue
from neo.lib.util import dump
from neo.lib.protocol import ProtocolError, uuid_str, ZERO_TID
from neo.lib.protocol import Packets, ProtocolError, NonReadableCell, \
uuid_str, MAX_TID
class ConflictError(Exception):
"""
......@@ -30,11 +32,6 @@ class ConflictError(Exception):
self.tid = tid
class DelayedError(Exception):
"""
Raised when an object is locked by a previous transaction
"""
class NotRegisteredError(Exception):
"""
Raised when a ttid is not registered
......@@ -44,53 +41,148 @@ class Transaction(object):
"""
Container for a pending transaction
"""
_delayed = {}
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>" \
% (self.__class__.__name__,
dump(self.tid),
return "<%s(%s, locking_tid=%s, tid=%s, age=%.2fs) at 0x%x>" % (
self.__class__.__name__,
uuid_str(self.uuid),
dump(self.locking_tid),
dump(self.tid),
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 logDelay(self, ttid, locked, oid_serial):
if self._delayed.get(oid_serial) != locked:
if self._delayed:
self._delayed[oid_serial] = locked
else:
self._delayed = {oid_serial: locked}
logging.info('Lock delayed for %s:%s by %s',
dump(oid_serial[0]), dump(ttid), dump(locked))
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
def cancel(self, oid):
try:
return self.store_dict.pop(oid)[1]
except KeyError:
self.checked_set.remove(oid)
class TransactionManager(object):
class TransactionManager(EventQueue):
"""
Manage pending transaction and locks
"""
def __init__(self, app):
EventQueue.__init__(self)
self.read_queue = EventQueue()
self._app = app
self._transaction_dict = {}
self._store_lock_dict = {}
self._load_lock_dict = {}
self._replicated = {}
self._replicating = set()
from neo.lib.util import u64
np = app.pt.getPartitions()
self.getPartition = lambda oid: u64(oid) % np
def discarded(self, offset_list):
self._replicating.difference_update(offset_list)
for offset in offset_list:
self._replicated.pop(offset, None)
getPartition = self.getPartition
for oid_dict in self._load_lock_dict, self._store_lock_dict:
for oid in oid_dict.keys():
if getPartition(oid) in offset_list:
del oid_dict[oid]
data_id_list = []
for transaction in self._transaction_dict.itervalues():
serial_dict = transaction.serial_dict
oid_list = [oid for oid in serial_dict
if getPartition(oid) in offset_list]
for oid in oid_list:
del serial_dict[oid]
try:
data_id_list.append(transaction.store_dict.pop(oid)[1])
except KeyError:
pass
transaction.lockless.difference_update(oid_list)
self._app.dm.dropPartitionsTemporary(offset_list)
self._app.dm.releaseData(data_id_list, True)
# notifyPartitionChanges will commit
self.executeQueuedEvents()
self.read_queue.executeQueuedEvents()
def readable(self, offset_list):
for offset in offset_list:
tid = self._replicated.pop(offset, None)
assert tid is None, (offset, tid)
def replicating(self, offset_list):
self._replicating.update(offset_list)
isdisjoint = set(offset_list).isdisjoint
assert isdisjoint(self._replicated), (offset_list, self._replicated)
assert isdisjoint(map(self.getPartition, self._store_lock_dict)), (
offset_list, self._store_lock_dict)
p = Packets.AskUnfinishedTransactions(offset_list)
self._app.master_conn.ask(p, offset_list=offset_list)
def replicated(self, partition, tid):
# also called for readable cells in BACKINGUP state
self._replicating.discard(partition)
self._replicated[partition] = tid
self._notifyReplicated()
def _notifyReplicated(self):
getPartition = self.getPartition
store_lock_dict = self._store_lock_dict
replicated = self._replicated
notify = {x[0] for x in replicated.iteritems() if x[1]}
# We sort transactions so that in case of muliple stores/checks for the
# 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
assert txn.lockless.issubset(txn.serial_dict), (
txn.lockless, txn.serial_dict)
for oid in txn.lockless:
partition = getPartition(oid)
if replicated.get(partition):
if store_lock_dict.get(oid, ttid) != ttid:
# We have a "multi-lock" store, i.e. an
# initially-lockless store to a partition that became
# replicated.
notify.discard(partition)
store_lock_dict[oid] = ttid
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.
for partition in notify:
self._app.master_conn.send(Packets.NotifyReplicationDone(
partition, replicated[partition]))
replicated[partition] = None
for oid, ttid in store_lock_dict.iteritems():
if getPartition(oid) in notify:
# Use 'discard' instead of 'remove', for oids that were
# locked after that the partition was replicated.
self._transaction_dict[ttid].lockless.discard(oid)
def register(self, conn, ttid):
"""
......@@ -111,13 +203,73 @@ class TransactionManager(object):
except KeyError:
return None
def reset(self):
"""
Reset the transaction manager
"""
self._transaction_dict.clear()
self._store_lock_dict.clear()
self._load_lock_dict.clear()
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. They must also be sorted again because
# our locking tid has changed.
self.sortAndExecuteQueuedEvents()
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):
"""
......@@ -132,7 +284,9 @@ class TransactionManager(object):
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)
......@@ -152,7 +306,7 @@ class TransactionManager(object):
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):
......@@ -178,83 +332,131 @@ class TransactionManager(object):
def getLockingTID(self, oid):
return self._store_lock_dict.get(oid)
def lockObject(self, ttid, serial, oid, unlock=False):
def lockObject(self, ttid, serial, oid):
"""
Take a write lock on given object, checking that "serial" is
current.
Raises:
DelayedError
DelayEvent
ConflictError
"""
# check if the object if locked
locking_tid = self._store_lock_dict.get(oid)
if locking_tid == ttid and unlock:
logging.info('Deadlock resolution on %r:%r', dump(oid), dump(ttid))
# A duplicate store means client is resolving a deadlock, so
# drop the lock it held on this object, and drop object data for
# consistency.
del self._store_lock_dict[oid]
data_id = self._transaction_dict[ttid].cancel(oid)
if data_id:
self._app.dm.pruneData((data_id,))
# Give a chance to pending events to take that lock now.
self._app.executeQueuedEvents()
# Attemp to acquire lock again.
locking_tid = self._store_lock_dict.get(oid)
if locking_tid is None:
previous_serial = None
elif locking_tid == ttid:
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
# However, this transaction may have begun after we started to
# 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:
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.
transaction.logDelay(ttid, locked, (oid, serial))
# 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)
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
# new transactions.
# For the cluster, we're still out-of-date and like above,
# at least 1 other (readable) cell checks for conflicts.
return
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.
logging.info('Deadlock on %s:%s with %s',
dump(oid), dump(ttid), dump(locked))
# 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.send(Packets.NotifyDeadlock(
ttid, transaction.locking_tid))
self._rebase(transaction, ttid)
raise DelayEvent(transaction)
# If previous store was an undo, next store must be based on
# undo target.
previous_serial = self._transaction_dict[ttid].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:
# XXX: use some special serial when previous store was not
# an undo ? Maybe it should just not happen.
# 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
# resolution waits for invalidations from the master (to then
# load the saved data), which are sent after the notification
# we are down, and the client would stop writing to us.
logging.info('Transaction %s storing %s more than once',
dump(ttid), dump(oid))
elif locking_tid < ttid:
# 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.
logging.info('Store delayed for %r:%r by %r', dump(oid),
dump(ttid), dump(locking_tid))
raise DelayedError
return
elif transaction.locking_tid == MAX_TID:
# Deadlock avoidance. Still no new locking_tid from the client.
raise DelayEvent(transaction)
else:
# 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. Make client release
# locks & reacquire them by notifying it of the possible deadlock.
logging.info('Possible deadlock on %r:%r with %r',
dump(oid), dump(ttid), dump(locking_tid))
raise ConflictError(ZERO_TID)
# XXX: Consider locking before reporting a conflict:
# - That would speed up the case of cascading conflict resolution
# by avoiding incremental resolution, assuming that the time to
# resolve a conflict is often constant: "C+A vs. B -> C+A+B"
# rarely costs more than "C+A vs. C+B -> C+A+B".
# - That would slow down of cascading unresolvable conflicts but
# if that happens, the application should be reviewed.
if previous_serial is None:
previous_serial = self._app.dm.getLastObjectTID(oid)
try:
previous_serial = self._app.dm.getLastObjectTID(oid)
except NonReadableCell:
partition = self.getPartition(oid)
if partition not in self._replicated:
# Either the partition is discarded or we haven't yet
# received the notification from the master that the
# partition is assigned to us. In the latter case, we're
# not expected to have the partition in full.
# We'll return a successful answer to the client, which
# is fine because there's at least one other cell that is
# readable for this oid.
raise
with self._app.dm.replicated(partition):
previous_serial = self._app.dm.getLastObjectTID(oid)
# Locking before reporting a conflict would speed up the case of
# cascading conflict resolution by avoiding incremental resolution,
# assuming that the time to resolve a conflict is often constant:
# "C+A vs. B -> C+A+B" rarely costs more than "C+A vs. C+B -> C+A+B".
# However, this would be against the optimistic principle of ZODB.
if previous_serial is not None and previous_serial != serial:
logging.info('Resolvable conflict on %r:%r',
dump(oid), dump(ttid))
assert serial < previous_serial, (serial, previous_serial)
logging.info('Conflict on %s:%s with %s',
dump(oid), dump(ttid), dump(previous_serial))
raise ConflictError(previous_serial)
logging.debug('Transaction %s storing %s', dump(ttid), dump(oid))
logging.debug('Transaction %s locking %s', dump(ttid), dump(oid))
self._store_lock_dict[oid] = ttid
def checkCurrentSerial(self, ttid, serial, oid):
def checkCurrentSerial(self, ttid, oid, serial):
try:
transaction = self._transaction_dict[ttid]
except KeyError:
raise NotRegisteredError
self.lockObject(ttid, serial, oid, unlock=True)
transaction.check(oid)
self.lockObject(ttid, serial, oid)
transaction.serial_dict[oid] = serial
def storeObject(self, ttid, serial, oid, compression, checksum, data,
value_serial, unlock=False):
value_serial):
"""
Store an object received from client node
"""
......@@ -262,7 +464,8 @@ class TransactionManager(object):
transaction = self._transaction_dict[ttid]
except KeyError:
raise NotRegisteredError
self.lockObject(ttid, serial, oid, unlock=unlock)
self.lockObject(ttid, serial, oid)
transaction.serial_dict[oid] = serial
# store object
if data is None:
data_id = None
......@@ -270,6 +473,43 @@ class TransactionManager(object):
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.
# This can also happen when a partition is dropped.
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
......@@ -278,9 +518,8 @@ class TransactionManager(object):
Note: does not alter persistent content.
"""
if ttid not in self._transaction_dict:
# the tid may be unknown as the transaction is aborted on every node
# of the partition, even if no data was received (eg. conflict on
# another node)
assert not even_if_locked
# See how the master processes AbortTransaction from the client.
return
logging.debug('Abort TXN %s', dump(ttid))
transaction = self._transaction_dict[ttid]
......@@ -294,22 +533,42 @@ class TransactionManager(object):
dm.abortTransaction(ttid)
dm.releaseData([x[1] for x in transaction.store_dict.itervalues()],
True)
dm.commit()
# 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)))
write_locking_tid = self._store_lock_dict.pop(oid)
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 == bool(
self._replicated.get(self.getPartition(oid))), 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:
self._notifyReplicated()
# some locks were released, some pending locks may now succeed
self._app.executeQueuedEvents()
self.read_queue.executeQueuedEvents()
self.executeQueuedEvents()
def abortFor(self, uuid):
"""
......@@ -334,10 +593,12 @@ class TransactionManager(object):
logging.info(' %s %r', dump(ttid), txn)
logging.info(' Read locks:')
for oid, ttid in self._load_lock_dict.iteritems():
logging.info(' %r by %r', dump(oid), dump(ttid))
logging.info(' %s by %s', dump(oid), dump(ttid))
logging.info(' Write locks:')
for oid, ttid in self._store_lock_dict.iteritems():
logging.info(' %r by %r', dump(oid), dump(ttid))
logging.info(' %s by %s', dump(oid), dump(ttid))
self.logQueuedEvents()
self.read_queue.logQueuedEvents()
def updateObjectDataForPack(self, oid, orig_serial, new_serial, data_id):
lock_tid = self.getLockingTID(oid)
......
......@@ -37,6 +37,7 @@ from time import time
from struct import pack, unpack
from unittest.case import _ExpectedFailure, _UnexpectedSuccess
try:
from transaction.interfaces import IDataManager
from ZODB.utils import newTid
except ImportError:
pass
......@@ -74,7 +75,6 @@ SSL = os.path.dirname(__file__) + os.sep
SSL = SSL + "ca.crt", SSL + "node.crt", SSL + "node.key"
logging.default_root_handler.handle = lambda record: None
logging.backlog(None, 1<<20)
debug.register()
# prevent "signal only works in main thread" errors in subprocesses
......@@ -326,7 +326,7 @@ class NeoUnitTestBase(NeoTestBase):
def checkNoPacketSent(self, conn):
""" check if no packet were sent """
self._checkNoPacketSend(conn, 'notify')
self._checkNoPacketSend(conn, 'send')
self._checkNoPacketSend(conn, 'answer')
self._checkNoPacketSend(conn, 'ask')
......@@ -372,23 +372,57 @@ class NeoUnitTestBase(NeoTestBase):
def checkNotifyPacket(self, conn, packet_type, packet_number=0):
""" Check if a notify-packet with the right type is sent """
calls = conn.mockGetNamedCalls('notify')
calls = conn.mockGetNamedCalls('send')
packet = calls.pop(packet_number).getParam(0)
self.assertTrue(isinstance(packet, protocol.Packet))
self.assertEqual(type(packet), packet_type)
return packet
class TransactionalResource(object):
class _sortKey(object):
def __init__(self, last):
self._last = last
def __cmp__(self, other):
assert type(self) is not type(other), other
return 1 if self._last else -1
def __init__(self, txn, last, **kw):
self.sortKey = lambda: self._sortKey(last)
for k in kw:
assert callable(IDataManager.get(k)), k
self.__dict__.update(kw)
txn.get().join(self)
def __call__(self, func):
name = func.__name__
assert callable(IDataManager.get(name)), name
setattr(self, name, func)
return func
def __getattr__(self, attr):
if callable(IDataManager.get(attr)):
return lambda *_: None
return self.__getattribute__(attr)
class Patch(object):
"""
Patch attributes and revert later automatically.
Usage:
with Patch(someObject, attrToPatch=newValue) as patch:
with Patch(someObject, [new,] attrToPatch=newValue) as patch:
[... code that runs with patches ...]
[... code that runs without patch ...]
The 'new' positional parameter defaults to False and it must be equal to
not hasattr(someObject, 'attrToPatch')
It is an assertion to detect when a Patch is obsolete.
' as patch' is optional: 'patch.revert()' can be used to revert patches
in the middle of the 'with' clause.
......@@ -400,7 +434,7 @@ class Patch(object):
In this case, patches are automatically reverted when 'patch' is deleted.
For patched callables, the new one receives the original value as first
argument.
argument if 'new' is True.
Alternative usage:
......@@ -415,25 +449,31 @@ class Patch(object):
applied = False
def __new__(cls, patched, **patch):
def __new__(cls, patched, *args, **patch):
if patch:
return object.__new__(cls)
def patch(func):
self = cls(patched, **{func.__name__: func})
self = cls(patched, *args, **{func.__name__: func})
self.apply()
return self
return patch
def __init__(self, patched, **patch):
def __init__(self, patched, *args, **patch):
new, = args or (0,)
(name, patch), = patch.iteritems()
self._patched = patched
self._name = name
if callable(patch):
wrapped = getattr(patched, name, None)
func = patch
patch = lambda *args, **kw: func(wrapped, *args, **kw)
if callable(wrapped):
patch = wraps(wrapped)(patch)
try:
wrapped = getattr(patched, name)
except AttributeError:
assert new, (patched, name)
else:
assert not new, (patched, name)
if callable(patch):
func = patch
patch = lambda *args, **kw: func(wrapped, *args, **kw)
if callable(wrapped):
patch = wraps(wrapped)(patch)
self._patch = patch
try:
orig = patched.__dict__[name]
......
......@@ -9,9 +9,6 @@ from email.mime.text import MIMEText
MAIL_SERVER = '127.0.0.1:25'
from neo.lib import logging
logging.backlog()
class AttributeDict(dict):
def __getattr__(self, item):
......
......@@ -16,64 +16,30 @@
import unittest
from ..mock import Mock
from ZODB.POSException import StorageTransactionError, ConflictError
from ZODB.POSException import StorageTransactionError
from .. import NeoUnitTestBase, buildUrlFromString
from neo.client.app import Application
from neo.client.cache import test as testCache
from neo.client.exception import NEOStorageError, NEOStorageNotFoundError
from neo.lib.protocol import NodeTypes, Packets, Errors, UUID_NAMESPACES
from neo.lib.util import makeChecksum
def _getMasterConnection(self):
if self.master_conn is None:
self.last_tid = None
self.uuid = 1 + (UUID_NAMESPACES[NodeTypes.CLIENT] << 24)
self.num_partitions = 10
self.num_replicas = 1
self.pt = Mock({'getCellList': ()})
self.master_conn = Mock()
return self.master_conn
def _ask(self, conn, packet, handler=None, **kw):
self.setHandlerData(None)
conn.ask(packet, **kw)
if handler is None:
raise NotImplementedError
else:
handler.dispatch(conn, conn.fakeReceived())
return self.getHandlerData()
def failing_tryToResolveConflict(oid, conflict_serial, serial, data):
raise ConflictError
from neo.client.exception import NEOStorageError
from neo.lib.protocol import NodeTypes, UUID_NAMESPACES
class ClientApplicationTests(NeoUnitTestBase):
def setUp(self):
NeoUnitTestBase.setUp(self)
# apply monkey patches
self._getMasterConnection = Application._getMasterConnection
self._ask = Application._ask
Application._getMasterConnection = _getMasterConnection
Application._ask = _ask
self._to_stop_list = []
def _tearDown(self, success):
# stop threads
for app in self._to_stop_list:
app.close()
# restore environment
Application._ask = self._ask
Application._getMasterConnection = self._getMasterConnection
NeoUnitTestBase._tearDown(self, success)
# some helpers
def checkAskObject(self, conn):
return self.checkAskPacket(conn, Packets.AskObject)
def _begin(self, app, txn, tid):
txn_context = app._txn_container.new(txn)
txn_context['ttid'] = tid
txn_context.ttid = tid
return txn_context
def getApp(self, master_nodes=None, name='test', **kw):
......@@ -104,61 +70,6 @@ class ClientApplicationTests(NeoUnitTestBase):
testCache = testCache
def test_load(self):
app = self.getApp()
cache = app._cache
oid = self.makeOID()
tid1 = self.makeTID(1)
tid2 = self.makeTID(2)
tid3 = self.makeTID(3)
tid4 = self.makeTID(4)
# connection to SN close
self.assertFalse(oid in cache._oid_dict)
conn = Mock({'getAddress': ('', 0)})
app.cp = Mock({'iterateForObject': [(Mock(), conn)]})
def fakeReceived(packet):
packet.setId(0)
conn.fakeReceived = iter((packet,)).next
def fakeObject(oid, serial, next_serial, data):
fakeReceived(Packets.AnswerObject(oid, serial, next_serial, 0,
makeChecksum(data), data, None))
return data, serial, next_serial
fakeReceived(Errors.OidNotFound(''))
#Application._waitMessage = self._waitMessage
# XXX: test disabled because of an infinite loop
# self.assertRaises(NEOStorageError, app.load, oid, None, tid2)
# self.checkAskObject(conn)
#Application._waitMessage = _waitMessage
# object not found in NEO -> NEOStorageNotFoundError
self.assertFalse(oid in cache._oid_dict)
fakeReceived(Errors.OidNotFound(''))
self.assertRaises(NEOStorageNotFoundError, app.load, oid)
self.checkAskObject(conn)
r1 = fakeObject(oid, tid1, tid3, 'FOO')
self.assertEqual(r1, app.load(oid, None, tid2))
self.checkAskObject(conn)
for t in tid2, tid3:
self.assertEqual(cache._load(oid, t).tid, tid1)
self.assertEqual(r1, app.load(oid, tid1))
self.assertEqual(r1, app.load(oid, None, tid3))
self.assertRaises(StandardError, app.load, oid, tid2)
self.assertRaises(StopIteration, app.load, oid)
self.checkAskObject(conn)
r2 = fakeObject(oid, tid3, None, 'BAR')
self.assertEqual(r2, app.load(oid, None, tid4))
self.checkAskObject(conn)
self.assertEqual(r2, app.load(oid))
self.assertEqual(r2, app.load(oid, tid3))
cache.invalidate(oid, tid4)
self.assertRaises(StopIteration, app.load, oid)
self.checkAskObject(conn)
self.assertEqual(len(cache._oid_dict[oid]), 2)
def test_store1(self):
app = self.getApp()
oid = self.makeOID(11)
......@@ -182,11 +93,8 @@ class ClientApplicationTests(NeoUnitTestBase):
tid = self.makeTID()
txn = self.makeTransactionObject()
app.master_conn = Mock()
conn = Mock()
self.assertRaises(StorageTransactionError, app.undo, tid,
txn, failing_tryToResolveConflict)
self.assertRaises(StorageTransactionError, app.undo, tid, txn)
# no packet sent
self.checkNoPacketSent(conn)
self.checkNoPacketSent(app.master_conn)
def test_connectToPrimaryNode(self):
......
#
# Copyright (C) 2009-2017 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
# as published by the Free Software Foundation; either version 2
# of the License, or (at your option) any later version.
#
# This program is distributed in the hope that it will be useful,
# but WITHOUT ANY WARRANTY; without even the implied warranty of
# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
# GNU General Public License for more details.
#
# You should have received a copy of the GNU General Public License
# along with this program. If not, see <http://www.gnu.org/licenses/>.
import time, unittest
from ..mock import Mock
from .. import NeoUnitTestBase
from neo.client.app import ConnectionPool
from neo.client.exception import NEOStorageError
from neo.client import pool
from neo.lib.util import p64
class ConnectionPoolTests(NeoUnitTestBase):
# TODO: test getConnForNode (requires splitting complex functionalities)
def test_CellSortKey(self):
cp = ConnectionPool(None)
node_uuid_1 = self.getStorageUUID()
node_uuid_2 = self.getStorageUUID()
node_uuid_3 = self.getStorageUUID()
# We are connected to node 1
cp.connection_dict[node_uuid_1] = None
def uuid_now(func, uuid, now):
pool.time = Mock({'time': now})
try:
return func(Mock({'getUUID': uuid}))
finally:
pool.time = time
# A connection to node 3 failed, will be forgotten at 5
uuid_now(cp.notifyFailure, node_uuid_3, 5 - pool.MAX_FAILURE_AGE)
def getCellSortKey(*args):
return uuid_now(cp.getCellSortKey, *args)
# At 0, key values are not ambiguous
self.assertTrue(getCellSortKey(node_uuid_1, 0) < getCellSortKey(
node_uuid_2, 0) < getCellSortKey(node_uuid_3, 0))
# At 10, nodes 2 and 3 have the same key value
self.assertTrue(getCellSortKey(node_uuid_1, 10) < getCellSortKey(
node_uuid_2, 10))
self.assertEqual(getCellSortKey(node_uuid_2, 10), getCellSortKey(
node_uuid_3, 10))
def test_iterateForObject_noStorageAvailable(self):
# no node available
oid = p64(1)
app = Mock()
app.pt = Mock({'getCellList': []})
pool = ConnectionPool(app)
self.assertRaises(NEOStorageError, pool.iterateForObject(oid).next)
if __name__ == '__main__':
unittest.main()
#
# Copyright (C) 2017 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
# as published by the Free Software Foundation; either version 2
# of the License, or (at your option) any later version.
#
# This program is distributed in the hope that it will be useful,
# but WITHOUT ANY WARRANTY; without even the implied warranty of
# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
# GNU General Public License for more details.
#
# You should have received a copy of the GNU General Public License
# along with this program. If not, see <http://www.gnu.org/licenses/>.
import unittest
from neo.client.zodburi import _resolve_uri
testv = [
# [] of (uri, zconf_ok, dbkw_ok)
("neo://dbname@master",
"""\
master_nodes\tmaster
name\tdbname
""",
{}),
("neo://db2@master1:port1,master2:port2,master3:port3",
"""\
master_nodes\tmaster1:port1 master2:port2 master3:port3
name\tdb2
""",
{}),
("neo://db3@master1,master2:port2?read_only=true",
"""\
master_nodes\tmaster1 master2:port2
name\tdb3
read-only\ttrue
""",
{}),
("neo://db4@[2001:67c:1254:2a::1]:1234,master2:port2?read_only=false"
"&compress=true&logfile=xxx&alpha=111&dynamic_master_list=zzz&ca=qqq"
"&cert=rrr&key=sss&beta=222",
"""\
master_nodes\t[2001:67c:1254:2a::1]:1234 master2:port2
name\tdb4
read-only\tfalse
compress\ttrue
logfile\txxx
dynamic_master_list\tzzz
ca\tqqq
cert\trrr
key\tsss
""",
{"alpha": "111", "beta": "222"}),
]
class ZODBURITests(unittest.TestCase):
def test_zodburi(self):
# invalid schema / path / fragment
self.assertRaises(ValueError, _resolve_uri, "http://db@master")
self.assertRaises(ValueError, _resolve_uri, "neo://db@master/path")
self.assertRaises(ValueError, _resolve_uri, "neo://db@master#frag")
# db @ master not fully specified
self.assertRaises(ValueError, _resolve_uri, "neo://master")
# verify zodburi resolver produces expected zconfig
for uri, zconf_ok, dbkw_ok in testv:
zconf_ok = "%import neo.client\n<NEOStorage>\n" + zconf_ok + \
"</NEOStorage>\n"
zconf, dbkw = _resolve_uri(uri)
self.assertMultiLineEqual(zconf, zconf_ok)
self.assertEqual(dbkw, dbkw_ok)
if __name__ == '__main__':
unittest.main()
......@@ -158,7 +158,7 @@ class NEOProcess(object):
if args:
os.close(w)
os.kill(os.getpid(), signal.SIGSTOP)
self.pid = os.fork()
self.pid = logging.fork()
if self.pid:
# Wait that the signal to kill the child is set up.
os.close(w)
......@@ -168,8 +168,6 @@ class NEOProcess(object):
else:
# Child
try:
# release SQLite debug log
logging.setup()
signal.signal(signal.SIGTERM, lambda *args: sys.exit())
if coverage:
coverage.stop()
......@@ -482,6 +480,7 @@ class NEOCluster(object):
master_nodes=master_nodes,
name=self.cluster_name,
**kw)
result.app.max_reconnection_to_master = 10
self.zodb_storage_list.append(result)
return result
......
......@@ -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()
......@@ -254,33 +211,6 @@ class ClientTests(NEOFunctionalTest):
self.__checkTree(neo_conn.root()['trees'])
self.assertEqual(dump, self.__dump(neo_db.storage))
def testLockTimeout(self):
""" Hold a lock on an object to block a second transaction """
def test():
self.neo = NEOCluster(['test_neo1'], replicas=0,
temp_dir=self.getTempDirectory())
self.neo.start()
# BUG: The following 2 lines creates 2 app, i.e. 2 TCP connections
# to the storage, so there may be a race condition at network
# level and 'st2.store' may be effective before 'st1.store'.
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())
st2.tpc_begin(t2)
st1.tpc_begin(t1)
st1.store(oid, rev, data, '', t1)
# this store will be delayed
st2.store(oid, rev, data, '', t2)
# the vote will timeout as t1 never release the lock
self.assertRaises(ConflictError, st2.tpc_vote, t2)
self.runWithTimeout(40, test)
def testIPv6Client(self):
""" Test the connectivity of an IPv6 connection for neo client """
......@@ -297,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
......
......@@ -19,8 +19,8 @@ from ..mock import Mock
from .. import NeoUnitTestBase
from neo.lib.util import p64
from neo.lib.protocol import NodeTypes, NodeStates, Packets
from neo.master.handlers.client import ClientServiceHandler
from neo.master.app import Application
from neo.master.handlers.client import ClientServiceHandler
class MasterClientHandlerTests(NeoUnitTestBase):
......@@ -39,8 +39,6 @@ class MasterClientHandlerTests(NeoUnitTestBase):
# define some variable to simulate client and storage node
self.client_port = 11022
self.storage_port = 10021
self.master_port = 10010
self.master_address = ('127.0.0.1', self.master_port)
self.client_address = ('127.0.0.1', self.client_port)
self.storage_address = ('127.0.0.1', self.storage_port)
self.storage_uuid = self.getStorageUUID()
......@@ -63,105 +61,6 @@ class MasterClientHandlerTests(NeoUnitTestBase):
)
return uuid
def checkAnswerBeginTransaction(self, conn):
return self.checkAnswerPacket(conn, Packets.AnswerBeginTransaction)
# Tests
def test_07_askBeginTransaction(self):
tid1 = self.getNextTID()
tid2 = self.getNextTID()
service = self.service
tm_org = self.app.tm
self.app.tm = tm = Mock({
'begin': '\x00\x00\x00\x00\x00\x00\x00\x01',
})
# client call it
client_uuid = self.identifyToMasterNode(node_type=NodeTypes.CLIENT, port=self.client_port)
client_node = self.app.nm.getByUUID(client_uuid)
conn = self.getFakeConnection(client_uuid, self.client_address)
service.askBeginTransaction(conn, None)
calls = tm.mockGetNamedCalls('begin')
self.assertEqual(len(calls), 1)
calls[0].checkArgs(client_node, None)
self.checkAnswerBeginTransaction(conn)
# Client asks for a TID
conn = self.getFakeConnection(client_uuid, self.client_address)
self.app.tm = tm_org
service.askBeginTransaction(conn, tid1)
calls = tm.mockGetNamedCalls('begin')
self.assertEqual(len(calls), 1)
calls[0].checkArgs(client_node, None)
packet = self.checkAnswerBeginTransaction(conn)
self.assertEqual(packet.decode(), (tid1, ))
def test_08_askNewOIDs(self):
service = self.service
oid1, oid2 = p64(1), p64(2)
self.app.tm.setLastOID(oid1)
# client call it
client_uuid = self.identifyToMasterNode(node_type=NodeTypes.CLIENT, port=self.client_port)
conn = self.getFakeConnection(client_uuid, self.client_address)
for node in self.app.nm.getStorageList():
conn = self.getFakeConnection(node.getUUID(), node.getAddress())
node.setConnection(conn)
service.askNewOIDs(conn, 1)
self.assertTrue(self.app.tm.getLastOID() > oid1)
def test_09_askFinishTransaction(self):
service = self.service
# do the right job
client_uuid = self.identifyToMasterNode(node_type=NodeTypes.CLIENT, port=self.client_port)
storage_uuid = self.storage_uuid
storage_conn = self.getFakeConnection(storage_uuid,
self.storage_address, is_server=True)
storage2_uuid = self.identifyToMasterNode(port=10022)
storage2_conn = self.getFakeConnection(storage2_uuid,
(self.storage_address[0], self.storage_address[1] + 1),
is_server=True)
self.app.setStorageReady(storage2_uuid)
conn = self.getFakeConnection(client_uuid, self.client_address)
self.app.pt = Mock({
'getPartition': 0,
'getCellList': [
Mock({'getUUID': storage_uuid}),
Mock({'getUUID': storage2_uuid}),
],
'getPartitions': 2,
})
ttid = self.getNextTID()
service.askBeginTransaction(conn, ttid)
conn = self.getFakeConnection(client_uuid, self.client_address)
self.app.nm.getByUUID(storage_uuid).setConnection(storage_conn)
# No packet sent if storage node is not ready
self.assertFalse(self.app.isStorageReady(storage_uuid))
service.askFinishTransaction(conn, ttid, (), ())
self.checkNoPacketSent(storage_conn)
# ...but AskLockInformation is sent if it is ready
self.app.setStorageReady(storage_uuid)
self.assertTrue(self.app.isStorageReady(storage_uuid))
service.askFinishTransaction(conn, ttid, (), ())
self.checkAskPacket(storage_conn, Packets.AskLockInformation)
self.assertEqual(len(self.app.tm.registerForNotification(storage_uuid)), 1)
txn = self.app.tm[ttid]
pending_ttid = list(self.app.tm.registerForNotification(storage_uuid))[0]
self.assertEqual(ttid, pending_ttid)
self.assertEqual(len(txn.getOIDList()), 0)
self.assertEqual(len(txn.getUUIDList()), 1)
def test_connectionClosed(self):
# give a client uuid which have unfinished transactions
client_uuid = self.identifyToMasterNode(node_type=NodeTypes.CLIENT,
port = self.client_port)
conn = self.getFakeConnection(client_uuid, self.client_address)
self.app.listening_conn = object() # mark as running
lptid = self.app.pt.getID()
self.assertEqual(self.app.nm.getByUUID(client_uuid).getState(),
NodeStates.RUNNING)
self.service.connectionClosed(conn)
# node must be have been remove, and no more transaction must remains
self.assertEqual(self.app.nm.getByUUID(client_uuid), None)
self.assertEqual(lptid, self.app.pt.getID())
def test_askPack(self):
self.assertEqual(self.app.packing, None)
self.app.nm.createClient()
......
......@@ -19,18 +19,12 @@ from ..mock import Mock
from neo.lib import protocol
from .. import NeoUnitTestBase
from neo.lib.protocol import NodeTypes, NodeStates, Packets
from neo.master.app import Application
from neo.master.handlers.election import ClientElectionHandler, \
ServerElectionHandler
from neo.master.app import Application
from neo.lib.exception import ElectionFailure
from neo.lib.connection import ClientConnection
# patch connection so that we can register _addPacket messages
# in mock object
def _addPacket(self, packet):
if self.connector is not None:
self.connector._addPacket(packet)
class MasterClientElectionTestBase(NeoUnitTestBase):
def setUp(self):
......@@ -67,13 +61,6 @@ class MasterClientElectionTests(MasterClientElectionTestBase):
self.election = ClientElectionHandler(self.app)
self.app.unconnected_master_node_set = set()
self.app.negotiating_master_node_set = set()
# apply monkey patches
ClientConnection._addPacket = _addPacket
def _tearDown(self, success):
# restore patched methods
del ClientConnection._addPacket
NeoUnitTestBase._tearDown(self, success)
def _checkUnconnected(self, node):
addr = node.getAddress()
......@@ -223,13 +210,6 @@ class MasterServerElectionTests(MasterClientElectionTestBase):
self.client_address = (self.local_ip, 1000)
self.storage_address = (self.local_ip, 2000)
self.master_address = (self.local_ip, 3000)
# apply monkey patches
ClientConnection._addPacket = _addPacket
def _tearDown(self, success):
NeoUnitTestBase._tearDown(self, success)
# restore environment
del ClientConnection._addPacket
def test_requestIdentification1(self):
""" A non-master node request identification """
......
......@@ -50,9 +50,6 @@ class MasterAppTests(NeoUnitTestBase):
master.setConnection(master_conn)
storage.setConnection(storage_conn)
client.setConnection(client_conn)
master.setRunning()
client.setRunning()
storage.setRunning()
self.app.nm.add(storage)
self.app.nm.add(client)
......@@ -87,15 +84,6 @@ class MasterAppTests(NeoUnitTestBase):
self.checkNoPacketSent(master_conn)
self.checkNotifyNodeInformation(storage_conn)
# node not running, don't send informations
client.setPending()
self.app.broadcastNodesInformation([s_node])
# check conn
self.assertFalse(client_conn.mockGetNamedCalls('notify'))
self.checkNoPacketSent(master_conn)
self.checkNotifyNodeInformation(storage_conn)
if __name__ == '__main__':
unittest.main()
......@@ -70,15 +70,15 @@ class MasterPartitionTableTests(NeoUnitTestBase):
num_partitions = 5
num_replicas = 3
pt = PartitionTable(num_partitions, num_replicas)
pt.setCell(0, sn1, CellStates.OUT_OF_DATE)
pt._setCell(0, sn1, CellStates.OUT_OF_DATE)
sn1.setState(NodeStates.RUNNING)
pt.setCell(1, sn2, CellStates.UP_TO_DATE)
pt._setCell(1, sn2, CellStates.UP_TO_DATE)
sn2.setState(NodeStates.TEMPORARILY_DOWN)
pt.setCell(2, sn3, CellStates.UP_TO_DATE)
pt._setCell(2, sn3, CellStates.UP_TO_DATE)
sn3.setState(NodeStates.DOWN)
pt.setCell(3, sn4, CellStates.UP_TO_DATE)
pt._setCell(3, sn4, CellStates.UP_TO_DATE)
sn4.setState(NodeStates.BROKEN)
pt.setCell(4, sn5, CellStates.UP_TO_DATE)
pt._setCell(4, sn5, CellStates.UP_TO_DATE)
sn5.setState(NodeStates.RUNNING)
# outdate nodes
cells_outdated = pt.outdate()
......@@ -118,12 +118,12 @@ class MasterPartitionTableTests(NeoUnitTestBase):
sn = [self.createStorage(None, i + 1, NodeStates.RUNNING)
for i in xrange(3)]
pt = PartitionTable(3, 0)
pt.setCell(0, sn[0], CellStates.OUT_OF_DATE)
pt.setCell(1, sn[1], CellStates.FEEDING)
pt.setCell(1, sn[2], CellStates.OUT_OF_DATE)
pt.setCell(2, sn[0], CellStates.OUT_OF_DATE)
pt.setCell(2, sn[1], CellStates.FEEDING)
pt.setCell(2, sn[2], CellStates.UP_TO_DATE)
pt._setCell(0, sn[0], CellStates.OUT_OF_DATE)
pt._setCell(1, sn[1], CellStates.FEEDING)
pt._setCell(1, sn[2], CellStates.OUT_OF_DATE)
pt._setCell(2, sn[0], CellStates.OUT_OF_DATE)
pt._setCell(2, sn[1], CellStates.FEEDING)
pt._setCell(2, sn[2], CellStates.UP_TO_DATE)
self.assertEqual(sorted(pt.dropNodeList(sn[:1], True)), [
(0, 1, CellStates.DISCARDED),
......@@ -137,7 +137,7 @@ class MasterPartitionTableTests(NeoUnitTestBase):
(2, 3, CellStates.DISCARDED)])
self.assertRaises(PartitionTableException, pt.dropNodeList, sn[1:2])
pt.setCell(1, sn[2], CellStates.UP_TO_DATE)
pt._setCell(1, sn[2], CellStates.UP_TO_DATE)
self.assertEqual(sorted(pt.dropNodeList(sn[1:2])), [
(1, 2, CellStates.DISCARDED),
(2, 2, CellStates.DISCARDED)])
......@@ -212,6 +212,9 @@ class MasterPartitionTableTests(NeoUnitTestBase):
self.assertEqual(self._pt_states(pt), self._pt_states(new_pt))
def update(self, pt, change_list=None):
offset_list = range(pt.np)
for node in pt.count_dict:
pt.updatable(node.getUUID(), offset_list)
if change_list is None:
for offset, row in enumerate(pt.partition_list):
for cell in list(row):
......@@ -233,24 +236,24 @@ class MasterPartitionTableTests(NeoUnitTestBase):
for i in xrange(5)]
pt = PartitionTable(5, 2)
# part 0
pt.setCell(0, sn[0], CellStates.DISCARDED)
pt.setCell(0, sn[1], CellStates.UP_TO_DATE)
pt._setCell(0, sn[0], CellStates.DISCARDED)
pt._setCell(0, sn[1], CellStates.UP_TO_DATE)
# part 1
pt.setCell(1, sn[0], CellStates.FEEDING)
pt.setCell(1, sn[1], CellStates.FEEDING)
pt.setCell(1, sn[2], CellStates.OUT_OF_DATE)
pt._setCell(1, sn[0], CellStates.FEEDING)
pt._setCell(1, sn[1], CellStates.FEEDING)
pt._setCell(1, sn[2], CellStates.OUT_OF_DATE)
# part 2
pt.setCell(2, sn[0], CellStates.FEEDING)
pt.setCell(2, sn[1], CellStates.UP_TO_DATE)
pt.setCell(2, sn[2], CellStates.UP_TO_DATE)
pt._setCell(2, sn[0], CellStates.FEEDING)
pt._setCell(2, sn[1], CellStates.UP_TO_DATE)
pt._setCell(2, sn[2], CellStates.UP_TO_DATE)
# part 3
pt.setCell(3, sn[0], CellStates.UP_TO_DATE)
pt.setCell(3, sn[1], CellStates.UP_TO_DATE)
pt.setCell(3, sn[2], CellStates.UP_TO_DATE)
pt.setCell(3, sn[3], CellStates.UP_TO_DATE)
pt._setCell(3, sn[0], CellStates.UP_TO_DATE)
pt._setCell(3, sn[1], CellStates.UP_TO_DATE)
pt._setCell(3, sn[2], CellStates.UP_TO_DATE)
pt._setCell(3, sn[3], CellStates.UP_TO_DATE)
# part 4
pt.setCell(4, sn[0], CellStates.UP_TO_DATE)
pt.setCell(4, sn[4], CellStates.UP_TO_DATE)
pt._setCell(4, sn[0], CellStates.UP_TO_DATE)
pt._setCell(4, sn[4], CellStates.UP_TO_DATE)
count_dict = defaultdict(int)
change_list = self.tweak(pt)
......
......@@ -71,7 +71,7 @@ class MasterStorageHandlerTests(NeoUnitTestBase):
self.checkNoPacketSent(client_conn)
self.assertEqual(self.app.packing[2], {conn2.getUUID()})
self.service.answerPack(conn2, False)
packet = self.checkAnswerPacket(client_conn, Packets.AnswerPack)
packet = self.checkNotifyPacket(client_conn, Packets.AnswerPack)
# TODO: verify packet peer id
self.assertTrue(packet.decode()[0])
self.assertEqual(self.app.packing, None)
......
......@@ -24,66 +24,11 @@ from neo.master.transactions import TransactionManager
class testTransactionManager(NeoUnitTestBase):
def makeOID(self, i):
return pack('!Q', i)
def makeNode(self, node_type):
uuid = self.getNewUUID(node_type)
node = Mock({'getUUID': uuid, '__hash__': uuid, '__repr__': 'FakeNode'})
return uuid, node
def test_storageLost(self):
client1 = Mock({'__hash__': 1})
client2 = Mock({'__hash__': 2})
client3 = Mock({'__hash__': 3})
storage_1_uuid = self.getStorageUUID()
storage_2_uuid = self.getStorageUUID()
oid_list = [self.makeOID(1), ]
tm = TransactionManager(None)
# Transaction 1: 2 storage nodes involved, one will die and the other
# already answered node lock
msg_id_1 = 1
ttid1 = tm.begin(client1)
tid1 = tm.prepare(ttid1, 1, oid_list,
[storage_1_uuid, storage_2_uuid], msg_id_1)
tm.lock(ttid1, storage_2_uuid)
t1 = tm[ttid1]
self.assertFalse(t1.locked())
# Storage 1 dies:
# t1 is over
self.assertTrue(t1.storageLost(storage_1_uuid))
self.assertEqual(t1.getUUIDList(), [storage_2_uuid])
del tm[ttid1]
# Transaction 2: 2 storage nodes involved, one will die
msg_id_2 = 2
ttid2 = tm.begin(client2)
tid2 = tm.prepare(ttid2, 1, oid_list,
[storage_1_uuid, storage_2_uuid], msg_id_2)
t2 = tm[ttid2]
self.assertFalse(t2.locked())
# Storage 1 dies:
# t2 still waits for storage 2
self.assertFalse(t2.storageLost(storage_1_uuid))
self.assertEqual(t2.getUUIDList(), [storage_2_uuid])
self.assertTrue(t2.lock(storage_2_uuid))
del tm[ttid2]
# Transaction 3: 1 storage node involved, which won't die
msg_id_3 = 3
ttid3 = tm.begin(client3)
tid3 = tm.prepare(ttid3, 1, oid_list, [storage_2_uuid, ],
msg_id_3)
t3 = tm[ttid3]
self.assertFalse(t3.locked())
# Storage 1 dies:
# t3 doesn't care
self.assertFalse(t3.storageLost(storage_1_uuid))
self.assertEqual(t3.getUUIDList(), [storage_2_uuid])
self.assertTrue(t3.lock(storage_2_uuid))
del tm[ttid3]
def testTIDUtils(self):
"""
Tests packTID/unpackTID/addTID.
......@@ -110,53 +55,14 @@ class testTransactionManager(NeoUnitTestBase):
unpackTID(addTID(packTID((2010, 11, 30, 23, 59), 2**32 - 1), 1)),
((2010, 12, 1, 0, 0), 0))
def testTransactionLock(self):
"""
Transaction lock is present to ensure invalidation TIDs are sent in
strictly increasing order.
Note: this implementation might change later, for more parallelism.
"""
client_uuid, client = self.makeNode(NodeTypes.CLIENT)
tm = TransactionManager(None)
# With a requested TID, lock spans from begin to remove
ttid1 = self.getNextTID()
ttid2 = self.getNextTID()
tid1 = tm.begin(client, ttid1)
self.assertEqual(tid1, ttid1)
del tm[ttid1]
# Without a requested TID, lock spans from prepare to remove only
ttid3 = tm.begin(client)
ttid4 = tm.begin(client) # Doesn't raise
node = Mock({'getUUID': client_uuid, '__hash__': 0})
tid4 = tm.prepare(ttid4, 1, [], [], 0)
del tm[ttid4]
tm.prepare(ttid3, 1, [], [], 0)
def testClientDisconectsAfterBegin(self):
client_uuid1, node1 = self.makeNode(NodeTypes.CLIENT)
tm = TransactionManager(None)
tid1 = self.getNextTID()
tid2 = self.getNextTID()
tm.begin(node1, tid1)
tm.begin(node1, 0, tid1)
tm.clientLost(node1)
self.assertTrue(tid1 not in tm)
def testUnlockPending(self):
callback = Mock()
uuid1, node1 = self.makeNode(NodeTypes.CLIENT)
uuid2, node2 = self.makeNode(NodeTypes.CLIENT)
storage_uuid = self.getStorageUUID()
tm = TransactionManager(callback)
ttid1 = tm.begin(node1)
ttid2 = tm.begin(node2)
tid1 = tm.prepare(ttid1, 1, [], [storage_uuid], 0)
tid2 = tm.prepare(ttid2, 1, [], [storage_uuid], 0)
tm.lock(ttid2, storage_uuid)
# txn 2 is still blocked by txn 1
self.assertEqual(len(callback.getNamedCalls('__call__')), 0)
tm.lock(ttid1, storage_uuid)
# both transactions are unlocked when txn 1 is fully locked
self.assertEqual(len(callback.getNamedCalls('__call__')), 2)
if __name__ == '__main__':
unittest.main()
......@@ -20,7 +20,7 @@ from .. import NeoUnitTestBase
from neo.storage.app import Application
from neo.storage.handlers.client import ClientOperationHandler
from neo.lib.util import p64
from neo.lib.protocol import INVALID_TID, Packets, LockState
from neo.lib.protocol import INVALID_TID, Packets
class StorageClientHandlerTests(NeoUnitTestBase):
......@@ -100,24 +100,5 @@ class StorageClientHandlerTests(NeoUnitTestBase):
self.operation.askObjectUndoSerial(conn, tid, ltid, undone_tid, oid_list)
self.checkErrorPacket(conn)
def test_askHasLock(self):
tid_1 = self.getNextTID()
tid_2 = self.getNextTID()
oid = self.getNextTID()
def getLockingTID(oid):
return locking_tid
self.app.tm.getLockingTID = getLockingTID
for locking_tid, status in (
(None, LockState.NOT_LOCKED),
(tid_1, LockState.GRANTED),
(tid_2, LockState.GRANTED_TO_OTHER),
):
conn = self._getConnection()
self.operation.askHasLock(conn, tid_1, oid)
p_oid, p_status = self.checkAnswerPacket(conn,
Packets.AnswerHasLock).decode()
self.assertEqual(oid, p_oid)
self.assertEqual(status, p_status)
if __name__ == "__main__":
unittest.main()
......@@ -28,7 +28,7 @@ class StorageIdentificationHandlerTests(NeoUnitTestBase):
config = self.getStorageConfiguration(master_number=1)
self.app = Application(config)
self.app.name = 'NEO'
self.app.ready = True
self.app.operational = True
self.app.pt = PartitionTable(4, 1)
self.identification = IdentificationHandler(self.app)
......
......@@ -16,12 +16,11 @@
import unittest
from ..mock import Mock
from collections import deque
from .. import NeoUnitTestBase
from neo.storage.app import Application
from neo.storage.handlers.master import MasterOperationHandler
from neo.lib.pt import PartitionTable
from neo.lib.protocol import CellStates
from neo.lib.protocol import CellStates, ProtocolError
class StorageMasterHandlerTests(NeoUnitTestBase):
......@@ -31,10 +30,6 @@ class StorageMasterHandlerTests(NeoUnitTestBase):
# create an application object
config = self.getStorageConfiguration(master_number=1)
self.app = Application(config)
self.app.transaction_dict = {}
self.app.store_lock_dict = {}
self.app.load_lock_dict = {}
self.app.event_queue = deque()
# handler
self.operation = MasterOperationHandler(self.app)
# set pmn
......@@ -60,7 +55,8 @@ class StorageMasterHandlerTests(NeoUnitTestBase):
app.replicator = Mock({})
self.app.pt = Mock({'getID': 1})
count = len(self.app.nm.getList())
self.operation.notifyPartitionChanges(conn, 0, ())
self.assertRaises(ProtocolError, self.operation.notifyPartitionChanges,
conn, 0, ())
self.assertEqual(self.app.pt.getID(), 1)
self.assertEqual(len(self.app.nm.getList()), count)
calls = self.app.replicator.mockGetNamedCalls('removePartition')
......@@ -83,18 +79,18 @@ class StorageMasterHandlerTests(NeoUnitTestBase):
app.nm.createStorage(uuid=uuid1)
app.nm.createStorage(uuid=uuid2)
app.nm.createStorage(uuid=uuid3)
ptid1, ptid2 = (1, 2)
self.assertNotEqual(ptid1, ptid2)
app.pt = PartitionTable(3, 1)
app.pt._id = 1
ptid = 2
app.dm = Mock({ })
app.replicator = Mock({})
self.operation.notifyPartitionChanges(conn, ptid2, cells)
self.operation.notifyPartitionChanges(conn, ptid, cells)
# ptid set
self.assertEqual(app.pt.getID(), ptid2)
self.assertEqual(app.pt.getID(), ptid)
# dm call
calls = self.app.dm.mockGetNamedCalls('changePartitionTable')
self.assertEqual(len(calls), 1)
calls[0].checkArgs(ptid2, cells)
calls[0].checkArgs(ptid, cells)
if __name__ == "__main__":
unittest.main()
......@@ -19,9 +19,7 @@ from ..mock import Mock
from .. import NeoUnitTestBase
from neo.storage.app import Application
from neo.lib.protocol import CellStates
from collections import deque
from neo.lib.pt import PartitionTable
from neo.storage.exception import AlreadyPendingError
class StorageAppTests(NeoUnitTestBase):
......@@ -31,8 +29,6 @@ class StorageAppTests(NeoUnitTestBase):
# create an application object
config = self.getStorageConfiguration(master_number=1)
self.app = Application(config)
self.app.event_queue = deque()
self.app.event_queue_dict = {}
def _tearDown(self, success):
self.app.close()
......@@ -66,8 +62,8 @@ class StorageAppTests(NeoUnitTestBase):
storage = self.app.nm.createStorage(uuid=storage_uuid)
client_uuid = self.getClientUUID()
self.app.pt.setCell(0, master, CellStates.UP_TO_DATE)
self.app.pt.setCell(0, storage, CellStates.UP_TO_DATE)
self.app.pt._setCell(0, master, CellStates.UP_TO_DATE)
self.app.pt._setCell(0, storage, CellStates.UP_TO_DATE)
self.assertEqual(len(self.app.pt.getNodeSet()), 2)
self.assertFalse(self.app.pt.filled())
for x in xrange(num_partitions):
......@@ -83,8 +79,8 @@ class StorageAppTests(NeoUnitTestBase):
self.assertFalse(self.app.pt.hasOffset(x))
# add some node
self.app.pt.setCell(0, master, CellStates.UP_TO_DATE)
self.app.pt.setCell(0, storage, CellStates.UP_TO_DATE)
self.app.pt._setCell(0, master, CellStates.UP_TO_DATE)
self.app.pt._setCell(0, storage, CellStates.UP_TO_DATE)
self.assertEqual(len(self.app.pt.getNodeSet()), 2)
self.assertFalse(self.app.pt.filled())
for x in xrange(num_partitions):
......@@ -121,26 +117,6 @@ class StorageAppTests(NeoUnitTestBase):
self.assertTrue(cell_list[0].getUUID() in (master_uuid, storage_uuid))
self.assertTrue(cell_list[1].getUUID() in (master_uuid, storage_uuid))
def test_02_queueEvent(self):
self.assertEqual(len(self.app.event_queue), 0)
msg_id = 1325136
event = Mock({'__repr__': 'event'})
conn = Mock({'__repr__': 'conn', 'getPeerId': msg_id})
key = 'foo'
self.app.queueEvent(event, conn, ("test", ), key=key)
self.assertEqual(len(self.app.event_queue), 1)
_key, _event, _msg_id, _conn, args = self.app.event_queue[0]
self.assertEqual(key, _key)
self.assertEqual(msg_id, _msg_id)
self.assertEqual(len(args), 1)
self.assertEqual(args[0], "test")
self.assertRaises(AlreadyPendingError, self.app.queueEvent, event,
conn, ("test2", ), key=key)
self.assertEqual(len(self.app.event_queue), 1)
self.app.queueEvent(event, conn, ("test3", ), key=key,
raise_on_duplicate=False)
self.assertEqual(len(self.app.event_queue), 2)
if __name__ == '__main__':
unittest.main()
......@@ -72,6 +72,7 @@ class StorageDBTests(NeoUnitTestBase):
db.changePartitionTable(1,
[(i, uuid, CellStates.UP_TO_DATE) for i in xrange(num_partitions)],
reset=True)
db.commit()
def checkConfigEntry(self, get_call, set_call, value):
# generic test for all configuration entries accessors
......@@ -101,10 +102,6 @@ class StorageDBTests(NeoUnitTestBase):
db = self.getDB()
self.checkConfigEntry(db.getName, db.setName, 'TEST_NAME')
def test_15_PTID(self):
db = self.getDB()
self.checkConfigEntry(db.getPTID, db.setPTID, 1)
def test_getPartitionTable(self):
db = self.getDB()
uuid1, uuid2 = self.getStorageUUID(), self.getStorageUUID()
......
......@@ -15,7 +15,8 @@
# along with this program. If not, see <http://www.gnu.org/licenses/>.
import unittest
from MySQLdb import OperationalError
from MySQLdb import NotSupportedError, OperationalError
from MySQLdb.constants.ER import UNKNOWN_STORAGE_ENGINE
from ..mock import Mock
from neo.lib.exception import DatabaseFailure
from neo.lib.util import p64
......@@ -35,7 +36,13 @@ class StorageMySQLdbTests(StorageDBTests):
db = MySQLDatabaseManager(database, self.engine)
self.assertEqual(db.db, DB_PREFIX + '0')
self.assertEqual(db.user, DB_USER)
db.setup(reset)
try:
db.setup(reset)
except NotSupportedError as m:
code, m = m.args
if code != UNKNOWN_STORAGE_ENGINE:
raise
raise unittest.SkipTest(m)
return db
def test_query1(self):
......@@ -98,7 +105,7 @@ class StorageMySQLdbTests(StorageDBTests):
self.db.query(x)
# Reconnection cleared the cache of the config table,
# so fill it again with required values before we patch query().
self.db.getNumPartitions()
self.db._getPartition
# Check MySQLDatabaseManager._max_allowed_packet
query_list = []
self.db.query = lambda query: query_list.append(EXTRA + len(query))
......@@ -119,6 +126,11 @@ class StorageMySQLdbTests(StorageDBTests):
self.assertEqual(len(query_list), count)
class StorageMySQLdbRocksDBTests(StorageMySQLdbTests):
engine = "RocksDB"
class StorageMySQLdbTokuDBTests(StorageMySQLdbTests):
engine = "TokuDB"
......
......@@ -28,7 +28,7 @@ class TransactionManagerTests(NeoUnitTestBase):
self.app = Mock()
# no history
self.app.dm = Mock({'getObjectHistory': []})
self.app.pt = Mock({'isAssigned': True})
self.app.pt = Mock({'isAssigned': True, 'getPartitions': 2})
self.app.em = Mock({'setTimeout': None})
self.manager = TransactionManager(self.app)
......
#
# Copyright (C) 2009-2017 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
# as published by the Free Software Foundation; either version 2
# of the License, or (at your option) any later version.
#
# This program is distributed in the hope that it will be useful,
# but WITHOUT ANY WARRANTY; without even the implied warranty of
# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
# GNU General Public License for more details.
#
# 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 . import NeoTestBase
from neo.lib.dispatcher import Dispatcher, ForgottenPacket
from Queue import Queue
import unittest
class DispatcherTests(NeoTestBase):
def setUp(self):
NeoTestBase.setUp(self)
self.dispatcher = Dispatcher()
def testForget(self):
conn = object()
queue = Queue()
MARKER = object()
# Register an expectation
self.dispatcher.register(conn, 1, queue)
# ...and forget about it, returning registered queue
forgotten_queue = self.dispatcher.forget(conn, 1)
self.assertTrue(queue is forgotten_queue, (queue, forgotten_queue))
# A ForgottenPacket must have been put in the queue
queue_conn, packet, kw = queue.get(block=False)
self.assertTrue(isinstance(packet, ForgottenPacket), packet)
# ...with appropriate packet id
self.assertEqual(packet.getId(), 1)
# ...and appropriate connection
self.assertTrue(conn is queue_conn, (conn, queue_conn))
# If forgotten twice, it must raise a KeyError
self.assertRaises(KeyError, self.dispatcher.forget, conn, 1)
# Event arrives, return value must be True (it was expected)
self.assertTrue(self.dispatcher.dispatch(conn, 1, MARKER, {}))
# ...but must not have reached the queue
self.assertTrue(queue.empty())
# Register an expectation
self.dispatcher.register(conn, 1, queue)
# ...and forget about it
self.dispatcher.forget(conn, 1)
queue.get(block=False)
# No exception must happen if connection is lost.
self.dispatcher.unregister(conn)
# Forgotten message's queue must not have received a "None"
self.assertTrue(queue.empty())
if __name__ == '__main__':
unittest.main()
......@@ -163,8 +163,10 @@ class NodeManagerTests(NeoUnitTestBase):
(NodeTypes.ADMIN, self.admin.getAddress(), self.admin.getUUID(),
NodeStates.UNKNOWN, None),
)
app = Mock()
app.pt = Mock()
# update manager content
manager.update(Mock(), node_list)
manager.update(app, time(), node_list)
# - the client gets down
self.checkClients([])
# - master change it's address
......
......@@ -53,7 +53,7 @@ class PartitionTableTests(NeoUnitTestBase):
self.assertEqual(len(pt.partition_list[x]), 0)
# add a cell to an empty row
self.assertFalse(pt.count_dict.has_key(sn1))
pt.setCell(0, sn1, CellStates.UP_TO_DATE)
pt._setCell(0, sn1, CellStates.UP_TO_DATE)
self.assertTrue(pt.count_dict.has_key(sn1))
self.assertEqual(pt.count_dict[sn1], 1)
for x in xrange(num_partitions):
......@@ -64,14 +64,14 @@ class PartitionTableTests(NeoUnitTestBase):
else:
self.assertEqual(len(pt.partition_list[x]), 0)
# try to add to a nonexistent partition
self.assertRaises(IndexError, pt.setCell, 10, sn1, CellStates.UP_TO_DATE)
self.assertRaises(IndexError, pt._setCell, 10, sn1, CellStates.UP_TO_DATE)
# if we add in discards state, must be removed
pt.setCell(0, sn1, CellStates.DISCARDED)
pt._setCell(0, sn1, CellStates.DISCARDED)
for x in xrange(num_partitions):
self.assertEqual(len(pt.partition_list[x]), 0)
self.assertEqual(pt.count_dict[sn1], 0)
# add a feeding node into empty row
pt.setCell(0, sn1, CellStates.FEEDING)
pt._setCell(0, sn1, CellStates.FEEDING)
self.assertTrue(pt.count_dict.has_key(sn1))
self.assertEqual(pt.count_dict[sn1], 0)
for x in xrange(num_partitions):
......@@ -82,7 +82,7 @@ class PartitionTableTests(NeoUnitTestBase):
else:
self.assertEqual(len(pt.partition_list[x]), 0)
# re-add it as feeding, nothing change
pt.setCell(0, sn1, CellStates.FEEDING)
pt._setCell(0, sn1, CellStates.FEEDING)
self.assertTrue(pt.count_dict.has_key(sn1))
self.assertEqual(pt.count_dict[sn1], 0)
for x in xrange(num_partitions):
......@@ -93,7 +93,7 @@ class PartitionTableTests(NeoUnitTestBase):
else:
self.assertEqual(len(pt.partition_list[x]), 0)
# now add it as up to date
pt.setCell(0, sn1, CellStates.UP_TO_DATE)
pt._setCell(0, sn1, CellStates.UP_TO_DATE)
self.assertTrue(pt.count_dict.has_key(sn1))
self.assertEqual(pt.count_dict[sn1], 1)
for x in xrange(num_partitions):
......@@ -105,18 +105,18 @@ class PartitionTableTests(NeoUnitTestBase):
self.assertEqual(len(pt.partition_list[x]), 0)
# now add broken and down state, must not be taken into account
pt.setCell(0, sn1, CellStates.DISCARDED)
pt._setCell(0, sn1, CellStates.DISCARDED)
for x in xrange(num_partitions):
self.assertEqual(len(pt.partition_list[x]), 0)
self.assertEqual(pt.count_dict[sn1], 0)
sn1.setState(NodeStates.BROKEN)
self.assertRaises(PartitionTableException, pt.setCell,
self.assertRaises(PartitionTableException, pt._setCell,
0, sn1, CellStates.UP_TO_DATE)
for x in xrange(num_partitions):
self.assertEqual(len(pt.partition_list[x]), 0)
self.assertEqual(pt.count_dict[sn1], 0)
sn1.setState(NodeStates.DOWN)
self.assertRaises(PartitionTableException, pt.setCell,
self.assertRaises(PartitionTableException, pt._setCell,
0, sn1, CellStates.UP_TO_DATE)
for x in xrange(num_partitions):
self.assertEqual(len(pt.partition_list[x]), 0)
......@@ -134,7 +134,7 @@ class PartitionTableTests(NeoUnitTestBase):
self.assertEqual(len(pt.partition_list[x]), 0)
# add a cell to an empty row
self.assertFalse(pt.count_dict.has_key(sn1))
pt.setCell(0, sn1, CellStates.UP_TO_DATE)
pt._setCell(0, sn1, CellStates.UP_TO_DATE)
self.assertTrue(pt.count_dict.has_key(sn1))
self.assertEqual(pt.count_dict[sn1], 1)
for x in xrange(num_partitions):
......@@ -148,7 +148,7 @@ class PartitionTableTests(NeoUnitTestBase):
for x in xrange(num_partitions):
self.assertEqual(len(pt.partition_list[x]), 0)
# add a feeding cell
pt.setCell(0, sn1, CellStates.FEEDING)
pt._setCell(0, sn1, CellStates.FEEDING)
self.assertTrue(pt.count_dict.has_key(sn1))
self.assertEqual(pt.count_dict[sn1], 0)
for x in xrange(num_partitions):
......@@ -170,19 +170,19 @@ class PartitionTableTests(NeoUnitTestBase):
uuid1 = self.getStorageUUID()
server1 = ("127.0.0.1", 19001)
sn1 = self.createStorage(server1, uuid1)
pt.setCell(0, sn1, CellStates.UP_TO_DATE)
pt._setCell(0, sn1, CellStates.UP_TO_DATE)
uuid2 = self.getStorageUUID()
server2 = ("127.0.0.2", 19001)
sn2 = self.createStorage(server2, uuid2)
pt.setCell(0, sn2, CellStates.OUT_OF_DATE)
pt._setCell(0, sn2, CellStates.OUT_OF_DATE)
uuid3 = self.getStorageUUID()
server3 = ("127.0.0.3", 19001)
sn3 = self.createStorage(server3, uuid3)
pt.setCell(0, sn3, CellStates.FEEDING)
pt._setCell(0, sn3, CellStates.FEEDING)
uuid4 = self.getStorageUUID()
server4 = ("127.0.0.4", 19001)
sn4 = self.createStorage(server4, uuid4)
pt.setCell(0, sn4, CellStates.DISCARDED) # won't be added
pt._setCell(0, sn4, CellStates.DISCARDED) # won't be added
# now checks result
self.assertEqual(len(pt.partition_list[0]), 3)
for x in xrange(num_partitions):
......@@ -216,15 +216,15 @@ class PartitionTableTests(NeoUnitTestBase):
uuid1 = self.getStorageUUID()
server1 = ("127.0.0.1", 19001)
sn1 = self.createStorage(server1, uuid1)
pt.setCell(0, sn1, CellStates.UP_TO_DATE)
pt._setCell(0, sn1, CellStates.UP_TO_DATE)
uuid2 = self.getStorageUUID()
server2 = ("127.0.0.2", 19001)
sn2 = self.createStorage(server2, uuid2)
pt.setCell(1, sn2, CellStates.OUT_OF_DATE)
pt._setCell(1, sn2, CellStates.OUT_OF_DATE)
uuid3 = self.getStorageUUID()
server3 = ("127.0.0.3", 19001)
sn3 = self.createStorage(server3, uuid3)
pt.setCell(2, sn3, CellStates.FEEDING)
pt._setCell(2, sn3, CellStates.FEEDING)
# now checks result
self.assertEqual(len(pt.partition_list[0]), 1)
self.assertEqual(len(pt.partition_list[1]), 1)
......@@ -246,19 +246,19 @@ class PartitionTableTests(NeoUnitTestBase):
uuid1 = self.getStorageUUID()
server1 = ("127.0.0.1", 19001)
sn1 = self.createStorage(server1, uuid1)
pt.setCell(0, sn1, CellStates.UP_TO_DATE)
pt._setCell(0, sn1, CellStates.UP_TO_DATE)
uuid2 = self.getStorageUUID()
server2 = ("127.0.0.2", 19001)
sn2 = self.createStorage(server2, uuid2)
pt.setCell(0, sn2, CellStates.OUT_OF_DATE)
pt._setCell(0, sn2, CellStates.OUT_OF_DATE)
uuid3 = self.getStorageUUID()
server3 = ("127.0.0.3", 19001)
sn3 = self.createStorage(server3, uuid3)
pt.setCell(0, sn3, CellStates.FEEDING)
pt._setCell(0, sn3, CellStates.FEEDING)
uuid4 = self.getStorageUUID()
server4 = ("127.0.0.4", 19001)
sn4 = self.createStorage(server4, uuid4)
pt.setCell(0, sn4, CellStates.DISCARDED) # won't be added
pt._setCell(0, sn4, CellStates.DISCARDED) # won't be added
# must get only two node as feeding and discarded not taken
# into account
self.assertEqual(pt.getNodeSet(True), {sn1, sn3})
......@@ -276,7 +276,7 @@ class PartitionTableTests(NeoUnitTestBase):
server1 = ("127.0.0.1", 19001)
sn1 = self.createStorage(server1, uuid1)
for x in xrange(num_partitions):
pt.setCell(x, sn1, CellStates.UP_TO_DATE)
pt._setCell(x, sn1, CellStates.UP_TO_DATE)
self.assertEqual(pt.num_filled_rows, num_partitions)
self.assertTrue(pt.filled())
......@@ -288,7 +288,7 @@ class PartitionTableTests(NeoUnitTestBase):
uuid1 = self.getStorageUUID()
server1 = ("127.0.0.1", 19001)
sn1 = self.createStorage(server1, uuid1)
pt.setCell(0, sn1, CellStates.UP_TO_DATE)
pt._setCell(0, sn1, CellStates.UP_TO_DATE)
# now test
self.assertTrue(pt.hasOffset(0))
self.assertFalse(pt.hasOffset(1))
......@@ -307,7 +307,7 @@ class PartitionTableTests(NeoUnitTestBase):
# adding a node in all partition
sn1 = createStorage()
for x in xrange(num_partitions):
pt.setCell(x, sn1, CellStates.UP_TO_DATE)
pt._setCell(x, sn1, CellStates.UP_TO_DATE)
self.assertTrue(pt.filled())
# it's up to date and running, so operational
sn1.setState(NodeStates.RUNNING)
......@@ -319,7 +319,7 @@ class PartitionTableTests(NeoUnitTestBase):
# adding a node in all partition
sn1 = createStorage()
for x in xrange(num_partitions):
pt.setCell(x, sn1, CellStates.FEEDING)
pt._setCell(x, sn1, CellStates.FEEDING)
self.assertTrue(pt.filled())
# it's feeding and running, so operational
sn1.setState(NodeStates.RUNNING)
......@@ -333,7 +333,7 @@ class PartitionTableTests(NeoUnitTestBase):
sn1 = createStorage()
sn1.setState(NodeStates.TEMPORARILY_DOWN)
for x in xrange(num_partitions):
pt.setCell(x, sn1, CellStates.FEEDING)
pt._setCell(x, sn1, CellStates.FEEDING)
self.assertTrue(pt.filled())
# it's up to date and not running, so not operational
self.assertFalse(pt.operational())
......@@ -345,7 +345,7 @@ class PartitionTableTests(NeoUnitTestBase):
# adding a node in all partition
sn1 = createStorage()
for x in xrange(num_partitions):
pt.setCell(x, sn1, CellStates.OUT_OF_DATE)
pt._setCell(x, sn1, CellStates.OUT_OF_DATE)
self.assertTrue(pt.filled())
# it's not up to date and running, so not operational
self.assertFalse(pt.operational())
......@@ -358,18 +358,18 @@ class PartitionTableTests(NeoUnitTestBase):
uuid1 = self.getStorageUUID()
server1 = ("127.0.0.1", 19001)
sn1 = self.createStorage(server1, uuid1)
pt.setCell(0, sn1, CellStates.UP_TO_DATE)
pt.setCell(1, sn1, CellStates.UP_TO_DATE)
pt.setCell(2, sn1, CellStates.UP_TO_DATE)
pt._setCell(0, sn1, CellStates.UP_TO_DATE)
pt._setCell(1, sn1, CellStates.UP_TO_DATE)
pt._setCell(2, sn1, CellStates.UP_TO_DATE)
uuid2 = self.getStorageUUID()
server2 = ("127.0.0.2", 19001)
sn2 = self.createStorage(server2, uuid2)
pt.setCell(0, sn2, CellStates.UP_TO_DATE)
pt.setCell(1, sn2, CellStates.UP_TO_DATE)
pt._setCell(0, sn2, CellStates.UP_TO_DATE)
pt._setCell(1, sn2, CellStates.UP_TO_DATE)
uuid3 = self.getStorageUUID()
server3 = ("127.0.0.3", 19001)
sn3 = self.createStorage(server3, uuid3)
pt.setCell(0, sn3, CellStates.UP_TO_DATE)
pt._setCell(0, sn3, CellStates.UP_TO_DATE)
# test
row_0 = pt.getRow(0)
self.assertEqual(len(row_0), 3)
......
......@@ -131,10 +131,6 @@ class Serialized(object):
The epoll object of each node is hooked so that thread switching happens
before polling for network activity. An extra epoll object is used to
detect which node has a readable epoll object.
XXX: It seems wrong to rely only on epoll as way to know if there are
pending network messages. I had rare random failures due to tic()
returning prematurely.
"""
check_timeout = False
......@@ -183,7 +179,13 @@ class Serialized(object):
p.set_trace(sys._getframe(3))
@classmethod
def tic(cls, step=-1, check_timeout=(), quiet=False):
def tic(cls, step=-1, check_timeout=(), quiet=False,
# BUG: We overuse epoll as a way to know if there are pending
# network messages. Sometimes, and this is more visible with
# a single-core CPU, other threads are still busy and haven't
# sent anything yet on the network. This causes tic() to
# return prematurely. Passing a non-zero value is a hack.
timeout=0):
# If you're in a pdb here, 'n' switches to another thread
# (the following lines are not supposed to be debugged into)
with cls._tic_lock, cls.pdb():
......@@ -203,7 +205,7 @@ class Serialized(object):
app.em.wakeup()
del app
while step: # used as step=-1 (default), =1 (twice), =2 (once)
event_list = cls._epoll.poll(0)
event_list = cls._epoll.poll(timeout)
if not event_list:
break
step -= 1
......@@ -277,13 +279,17 @@ class TestSerialized(Serialized): # NOTE used only in .NeoCTL
r = self._epoll.poll(0)
if r:
return r
Serialized.tic(step=1)
Serialized.tic(step=1, timeout=.001)
raise Exception("tic is looping forever")
return self._epoll.poll(timeout)
class Node(object):
@staticmethod
def convertInitArgs(**kw):
return {'get' + k.capitalize(): v for k, v in kw.iteritems()}
def getConnectionList(self, *peers):
addr = lambda c: c and (c.addr if c.is_server else c.getAddress())
addr_set = {addr(c.connector) for peer in peers
......@@ -355,11 +361,15 @@ class ServerNode(Node):
def getVirtualAddress(self):
return self._init_args['address']
def resetNode(self):
def resetNode(self, **kw):
assert not self.is_alive()
kw = self._init_args
kw = self.convertInitArgs(**kw)
init_args = self._init_args
init_args['getReset'] = False
assert set(kw).issubset(init_args), (kw, init_args)
init_args.update(kw)
self.close()
self.__init__(**kw)
self.__init__(**init_args)
def start(self):
Serialized(self)
......@@ -399,10 +409,6 @@ class StorageApplication(ServerNode, neo.storage.app.Application):
dm = type('', (), {'close': lambda self: None})()
def resetNode(self, clear_database=False):
self._init_args['getReset'] = clear_database
super(StorageApplication, self).resetNode()
def _afterRun(self):
super(StorageApplication, self)._afterRun()
try:
......@@ -430,9 +436,14 @@ class StorageApplication(ServerNode, neo.storage.app.Application):
class ClientApplication(Node, neo.client.app.Application):
max_reconnection_to_master = 10
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:
......@@ -453,6 +464,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, lambda: key(cell)), random.random()))
class NeoCTL(neo.neoctl.app.NeoCTL):
def __init__(self, *args, **kw):
......@@ -534,6 +549,11 @@ class ConnectionFilter(object):
return True
return False
@classmethod
def retry(cls):
with cls.lock:
cls._retry()
@classmethod
def _retry(cls):
for conn, queue in cls.filter_queue.items():
......@@ -545,11 +565,12 @@ class ConnectionFilter(object):
break
else:
if conn.isClosed():
return
# Use the thread that created the packet to reinject it,
# to avoid a race condition on Connector.queued.
conn.em.wakeup(lambda conn=conn, packet=packet:
conn.isClosed() or cls._addPacket(conn, packet))
queue.clear()
else:
# Use the thread that created the packet to reinject it,
# to avoid a race condition on Connector.queued.
conn.em.wakeup(lambda conn=conn, packet=packet:
conn.isClosed() or cls._addPacket(conn, packet))
continue
break
else:
......@@ -564,7 +585,8 @@ class ConnectionFilter(object):
def remove(self, *filters):
with self.lock:
for filter in filters:
del self.filter_dict[filter]
for p in self.filter_dict.pop(filter):
p.revert()
self._retry()
def discard(self, *filters):
......@@ -605,7 +627,7 @@ class NEOCluster(object):
for i in TIC_LOOP:
if lock(False):
return True
Serialized.tic(step=1, quiet=True)
Serialized.tic(step=1, quiet=True, timeout=.001)
raise Exception("tic is looping forever")
return lock(False)
self._lock = _lock
......@@ -661,10 +683,10 @@ class NEOCluster(object):
master_list = [MasterApplication.newAddress()
for _ in xrange(master_count)]
self.master_nodes = ' '.join('%s:%s' % x for x in master_list)
weak_self = weakref.proxy(self)
kw = dict(cluster=weak_self, getReplicas=replicas, getAdapter=adapter,
getPartitions=partitions, getReset=clear_databases,
getSSL=self.SSL)
kw = Node.convertInitArgs(replicas=replicas, adapter=adapter,
partitions=partitions, reset=clear_databases)
kw['cluster'] = weak_self = weakref.proxy(self)
kw['getSSL'] = self.SSL
if upstream is not None:
self.upstream = weakref.proxy(upstream)
kw.update(getUpstreamCluster=upstream.name,
......@@ -748,6 +770,10 @@ class NEOCluster(object):
def primary_master(self):
master, = [master for master in self.master_list if master.primary]
return master
@property
def cache_size(self):
return self.client._cache._max_size
###
def __enter__(self):
......@@ -778,7 +804,7 @@ class NEOCluster(object):
assert state in (ClusterStates.RUNNING, ClusterStates.BACKINGUP), state
self.enableStorageList(storage_list)
def stop(self, clear_database=False, __print_exc=traceback.print_exc):
def stop(self, clear_database=False, __print_exc=traceback.print_exc, **kw):
if self.started:
del self.started
logging.debug("stopping %s", self)
......@@ -806,11 +832,11 @@ class NEOCluster(object):
raise
else:
for node_type in 'master', 'storage', 'admin':
kw = {}
reset_kw = kw.copy()
if node_type == 'storage':
kw['clear_database'] = clear_database
reset_kw['reset'] = clear_database
for node in getattr(self, node_type + '_list'):
node.resetNode(**kw)
node.resetNode(**reset_kw)
def _newClient(self):
return ClientApplication(name=self.name, master_nodes=self.master_nodes,
......@@ -863,13 +889,15 @@ class NEOCluster(object):
self.neoctl.enableStorageList([x.uuid for x in storage_list])
Serialized.tic()
for node in storage_list:
assert self.getNodeState(node) == NodeStates.RUNNING
state = self.getNodeState(node)
assert state == NodeStates.RUNNING, state
def join(self, thread_list, timeout=5):
timeout += time.time()
while thread_list:
assert time.time() < timeout, thread_list
Serialized.tic()
# Map with repr before that threads become unprintable.
assert time.time() < timeout, map(repr, thread_list)
Serialized.tic(timeout=.001)
thread_list = [t for t in thread_list if t.is_alive()]
def getNodeState(self, node):
......@@ -917,10 +945,6 @@ class NEOCluster(object):
txn = transaction.TransactionManager()
return txn, (self.db if db is None else db).open(txn)
def extraCellSortKey(self, key): # XXX unused?
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)
......@@ -964,7 +988,7 @@ class NEOThreadedTest(NeoTestBase):
def _tearDown(self, success):
super(NEOThreadedTest, self)._tearDown(success)
ServerNode.resetPorts()
if success:
if success and logging._max_size is not None:
with logging as db:
db.execute("UPDATE packet SET body=NULL")
db.execute("VACUUM")
......@@ -993,13 +1017,12 @@ class NEOThreadedTest(NeoTestBase):
return obj
return unpickler
class newThread(threading.Thread):
class newPausedThread(threading.Thread):
def __init__(self, func, *args, **kw):
threading.Thread.__init__(self)
self.__target = func, args, kw
self.daemon = True
self.start()
def run(self):
try:
......@@ -1007,6 +1030,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)
......@@ -1015,12 +1040,64 @@ class NEOThreadedTest(NeoTestBase):
del self.__exc_info
raise etype, value, tb
class newThread(newPausedThread):
def __init__(self, *args, **kw):
NEOThreadedTest.newPausedThread.__init__(self, *args, **kw)
self.start()
def commitWithStorageFailure(self, client, txn):
with Patch(client, _getFinalTID=lambda *_: None):
self.assertRaises(ConnectionClosed, txn.commit)
def assertPartitionTable(self, cluster, stats):
self.assertEqual(stats, '|'.join(cluster.admin.pt.formatRows()))
def assertPartitionTable(self, cluster, stats, pt_node=None):
pt = (pt_node or cluster.admin).pt
index = [x.uuid for x in cluster.storage_list].index
self.assertEqual(stats, '|'.join(pt._formatRows(sorted(
pt.count_dict, key=lambda x: index(x.getUUID())))))
@staticmethod
def noConnection(jar, storage):
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):
......
......@@ -20,25 +20,31 @@ 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
from transaction.interfaces import TransientError
from ZODB import DB, POSException
from ZODB.DB import TransactionalUndo
from neo.storage.transactions import TransactionManager, \
DelayedError, ConflictError
from neo.lib.connection import ServerConnection, MTClientConnection
from neo.storage.transactions import TransactionManager, ConflictError
from neo.lib.connection import ConnectionClosed, \
ServerConnection, MTClientConnection
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, \
ZERO_OID, ZERO_TID
from .. import expectedFailure, Patch
from . import LockLock, NEOThreadedTest, with_cluster
Packet, uuid_str, ZERO_OID, ZERO_TID
from .. import expectedFailure, Patch, TransactionalResource
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
from neo.client.transactions import Transaction
from neo.master.handlers.client import ClientServiceHandler
from neo.storage.handlers.client import ClientOperationHandler
from neo.storage.handlers.identification import IdentificationHandler
from neo.storage.handlers.initialization import InitializationHandler
class PCounter(Persistent):
......@@ -58,6 +64,8 @@ class Test(NEOThreadedTest):
def testBasicStore(self, cluster):
if 1:
storage = cluster.getZODBStorage()
storage.sync()
storage.app.max_reconnection_to_master = 0
data_info = {}
compressible = 'x' * 20
compressed = compress(compressible)
......@@ -150,37 +158,45 @@ class Test(NEOThreadedTest):
self.assertEqual((x['tid'], x['size']), expected.pop())
self.assertFalse(expected)
@with_cluster()
def testUndoConflict(self, cluster, conflict_during_store=False):
def _testUndoConflict(self, cluster, *inc):
def waitResponses(orig, *args):
orig(*args)
p.revert()
ob.value += 3
t.commit()
if 1:
t, c = cluster.getTransaction()
c.root()[0] = ob = PCounterWithResolution()
t.commit()
ob.value += 1
t, c = cluster.getTransaction()
c.root()[0] = ob = PCounterWithResolution()
t.commit()
tids = []
for x in inc:
ob.value += x
t.commit()
undo = TransactionalUndo(cluster.db, (ob._p_serial,))
txn = transaction.Transaction()
undo.tpc_begin(txn)
if conflict_during_store:
with Patch(cluster.client, waitResponses=waitResponses) as p:
undo.commit(txn)
else:
ob.value += 3
t.commit()
undo.commit(txn)
undo.tpc_vote(txn)
undo.tpc_finish(txn)
t.begin()
self.assertEqual(ob.value, 3)
tids.append(ob._p_serial)
undo = TransactionalUndo(cluster.db, tids)
txn = transaction.Transaction()
undo.tpc_begin(txn)
ob.value += 5
with Patch(cluster.client, waitResponses=waitResponses) as p:
undo.commit(txn)
undo.tpc_vote(txn)
undo.tpc_finish(txn)
t.begin()
self.assertEqual(ob.value, 5)
return ob
@with_cluster()
def testUndoConflictSmallCache(self, cluster):
big = 'x' * cluster.cache_size
def resolve(orig, *args):
state = orig(*args)
state['x'] = big
return state
with Patch(PCounterWithResolution, _p_resolveConflict=resolve):
self.assertEqual(self._testUndoConflict(cluster, 1, 3).x, big)
@expectedFailure(POSException.ConflictError) # TODO recheck
def testUndoConflictDuringStore(self):
self.testUndoConflict(True)
@with_cluster()
def testUndoConflictDuringStore(self, cluster):
self._testUndoConflict(cluster, 1)
@with_cluster()
def testStorageDataLock(self, cluster):
......@@ -231,6 +247,25 @@ class Test(NEOThreadedTest):
data_info[key] -= 1
self.assertEqual(data_info, cluster.storage.getDataLockInfo())
@with_cluster()
def testStorageDataLock2(self, cluster):
storage = cluster.getZODBStorage()
def t(data):
oid = storage.new_oid()
txn = transaction.Transaction()
storage.tpc_begin(txn)
storage.store(oid, None, data, '', txn)
return txn
t1 = t('foo')
storage.tpc_finish(t('bar'))
s = cluster.storage
s.stop()
cluster.join((s,))
s.resetNode()
storage.app.max_reconnection_to_master = 0
self.assertRaises(NEOPrimaryMasterLost, storage.tpc_vote, t1)
expectedFailure(self.assertFalse)(s.dm.getOrphanList())
@with_cluster(storage_count=1)
def testDelayedUnlockInformation(self, cluster):
except_list = []
......@@ -254,7 +289,7 @@ class Test(NEOThreadedTest):
ob._p_changed = 1
t.commit()
self.assertNotIn(delayUnlockInformation, m2s)
self.assertEqual(except_list, [DelayedError])
self.assertEqual(except_list, [DelayEvent])
@with_cluster(storage_count=2, replicas=1)
def _testDeadlockAvoidance(self, cluster, scenario):
......@@ -320,9 +355,8 @@ class Test(NEOThreadedTest):
# 2: C1 commits
# 3: C2 resolves conflict
self.assertEqual(self._testDeadlockAvoidance([2, 4]),
[DelayedError, DelayedError, ConflictError, ConflictError])
[DelayEvent, DelayEvent, ConflictError, ConflictError])
@expectedFailure(POSException.ConflictError) # XXX recheck
def testDeadlockAvoidance(self):
# This test fail because deadlock avoidance is not fully implemented.
# 0: C1 -> S1
......@@ -331,7 +365,7 @@ class Test(NEOThreadedTest):
# 3: C2 commits
# 4: C1 resolves conflict
self.assertEqual(self._testDeadlockAvoidance([1, 3]),
[DelayedError, ConflictError, "???" ])
[DelayEvent, DelayEvent, DelayEvent, ConflictError])
@with_cluster()
def testConflictResolutionTriggered2(self, cluster):
......@@ -368,12 +402,12 @@ class Test(NEOThreadedTest):
resolved = []
last = lambda txn: txn._extension['last'] # BBB
def _handleConflicts(orig, txn_context, *args):
resolved.append(last(txn_context['txn']))
return orig(txn_context, *args)
def tpc_vote(orig, transaction, *args):
def _handleConflicts(orig, txn_context):
resolved.append(last(txn_context.txn))
orig(txn_context)
def tpc_vote(orig, transaction):
(l3 if last(transaction) else l2)()
return orig(transaction, *args)
return orig(transaction)
with Patch(cluster.client, _handleConflicts=_handleConflicts):
with LockLock() as l3, Patch(cluster.client, tpc_vote=tpc_vote):
with LockLock() as l2:
......@@ -416,9 +450,11 @@ class Test(NEOThreadedTest):
l.acquire()
idle = []
def askObject(orig, *args):
orig(*args)
idle.append(cluster.storage.em.isIdle())
l.release()
try:
orig(*args)
finally:
idle.append(cluster.storage.em.isIdle())
l.release()
if 1:
t, c = cluster.getTransaction()
r = c.root()
......@@ -449,16 +485,63 @@ class Test(NEOThreadedTest):
conn = s0.getConnection()
self.assertFalse(conn.isClosed())
getCellSortKey = cluster.client.cp.getCellSortKey
self.assertEqual(getCellSortKey(s0), CELL_CONNECTED)
self.assertEqual(getCellSortKey(s0, int), 0)
cluster.neoctl.dropNode(s0.getUUID())
self.assertEqual([s1], cluster.client.nm.getStorageList())
self.assertTrue(conn.isClosed())
self.assertEqual(getCellSortKey(s0), CELL_GOOD)
self.assertEqual(getCellSortKey(s0, int), 1)
# XXX: the test originally checked that 'unregister' method
# was called (even if it's useless in this case),
# but we would need an API to do that easily.
self.assertFalse(cluster.client.dispatcher.registered(conn))
@with_cluster(replicas=2)
def test_notifyPartitionChanges(self, cluster):
cluster.db
s0, s1, s2 = cluster.storage_list
s1.stop()
cluster.join((s1,))
s1.resetNode()
# This checks that s1 processes any PT update
# (by MasterOperationHandler) even if it receives one before the
# AnswerUnfinishedTransactions packet.
with ConnectionFilter() as f:
f.delayAskUnfinishedTransactions()
s1.start()
self.tic()
s2.stop()
cluster.join((s2,))
self.tic()
self.assertPartitionTable(cluster, 'UUO', s1)
@with_cluster()
def testStartOperation(self, cluster):
t, c = cluster.getTransaction()
c.root()._p_changed = 1
cluster.storage.stop()
cluster.join(cluster.storage_list)
cluster.storage.resetNode()
delayed = []
def delayConnection(conn, packet):
return conn in delayed
def startOperation(orig, self, conn, backup):
assert not delayed, delayed
delayed.append(conn)
orig(self, conn, backup)
def askBeginTransaction(orig, *args):
f.discard(delayConnection)
orig(*args)
with ConnectionFilter() as f, \
Patch(InitializationHandler, startOperation=startOperation), \
Patch(cluster.master.client_service_handler,
askBeginTransaction=askBeginTransaction) as p:
f.add(delayConnection)
cluster.storage.start()
self.tic()
t.commit()
self.assertNotIn(delayConnection, f)
self.assertTrue(delayed)
@with_cluster(replicas=1, partitions=10)
def testRestartWithMissingStorage(self, cluster):
# translated from neo.tests.functional.testStorage.StorageTest
......@@ -668,6 +751,7 @@ class Test(NEOThreadedTest):
c.root()._p_changed = 1
with Patch(storage.tm, lock=lambda *_: sys.exit()):
self.commitWithStorageFailure(cluster.client, t)
cluster.join((storage,))
self.assertRaises(DatabaseFailure, storage.resetNode)
@with_cluster(replicas=1)
......@@ -754,29 +838,51 @@ class Test(NEOThreadedTest):
@with_cluster(master_count=3, partitions=10, replicas=1, storage_count=3)
def testShutdown(self, cluster):
# NOTE vvv
# declare themselves primary too quickly. The consequence is
# often an endless tic loop.
if 1:
# fill DB a little
t, c = cluster.getTransaction()
c.root()[''] = ''
t.commit()
# declare themselves primary too quickly, but issues seem to be
# only reproducible with SSL enabled.
self._testShutdown(cluster)
def _testShutdown(self, cluster):
def before_finish(_):
# tell admin to shutdown the cluster
cluster.neoctl.setClusterState(ClusterStates.STOPPING)
# all nodes except clients should exit
cluster.join(cluster.master_list
+ cluster.storage_list
+ cluster.admin_list)
self.tic()
l = threading.Lock(); l.acquire()
with ConnectionFilter() as f:
# Make we sure that we send t2/BeginTransaction
# before t1/AskFinishTransaction
@f.delayAskBeginTransaction
def delay(_):
l.release()
return False
t2.start()
l.acquire()
t1, c1 = cluster.getTransaction()
ob = c1.root()['1'] = PCounter()
t1.commit()
ob.value += 1
TransactionalResource(t1, 0, tpc_finish=before_finish)
t2, c2 = cluster.getTransaction()
c2.root()['2'] = None
t2 = self.newPausedThread(t2.commit)
with Patch(cluster.client, _connectToPrimaryNode=lambda *_:
self.fail("unexpected reconnection to master")):
t1.commit()
self.assertRaises(ConnectionClosed, t2.join)
# all nodes except clients should exit
cluster.join(cluster.master_list
+ cluster.storage_list
+ cluster.admin_list)
cluster.stop() # stop and reopen DB to check partition tables
dm = cluster.storage_list[0].dm
self.assertEqual(1, dm.getPTID())
pt = list(dm.getPartitionTable())
self.assertEqual(20, len(pt))
for _, _, state in pt:
self.assertEqual(state, CellStates.UP_TO_DATE)
for s in cluster.storage_list[1:]:
self.assertEqual(s.dm.getPTID(), 1)
self.assertEqual(list(s.dm.getPartitionTable()), pt)
cluster.start()
pt = cluster.admin.pt
self.assertEqual(1, pt.getID())
for row in pt.partition_list:
for cell in row:
self.assertEqual(cell.getState(), CellStates.UP_TO_DATE)
t, c = cluster.getTransaction()
self.assertEqual(c.root()['1'].value, 1)
self.assertNotIn('2', c.root())
@with_cluster()
def testInternalInvalidation(self, cluster):
......@@ -819,12 +925,12 @@ class Test(NEOThreadedTest):
with cluster.newClient() as client:
cache = cluster.client._cache
txn = transaction.Transaction()
client.tpc_begin(txn)
client.tpc_begin(None, txn)
client.store(x1._p_oid, x1._p_serial, y, '', txn)
# Delay invalidation for x
with cluster.master.filterConnection(cluster.client) as m2c:
m2c.delayInvalidateObjects()
tid = client.tpc_finish(txn, None)
tid = client.tpc_finish(txn)
# Change to x is committed. Testing connection must ask the
# storage node to return original value of x, even if we
# haven't processed yet any invalidation for x.
......@@ -856,9 +962,9 @@ class Test(NEOThreadedTest):
# to be processed.
# Now modify x to receive an invalidation for it.
txn = transaction.Transaction()
client.tpc_begin(txn)
client.tpc_begin(None, txn)
client.store(x2._p_oid, tid, x, '', txn) # value=0
tid = client.tpc_finish(txn, None)
tid = client.tpc_finish(txn)
t1.begin() # make sure invalidation is processed
# Resume processing of answer from storage. An entry should be
# added in cache for x=1 with a fixed next_tid (i.e. not None)
......@@ -881,9 +987,9 @@ class Test(NEOThreadedTest):
t = self.newThread(t1.begin)
ll()
txn = transaction.Transaction()
client.tpc_begin(txn)
client.tpc_begin(None, txn)
client.store(x2._p_oid, tid, y, '', txn)
tid = client.tpc_finish(txn, None)
tid = client.tpc_finish(txn)
client.close()
self.assertEqual(invalidations(c1), {x1._p_oid})
t.join()
......@@ -906,8 +1012,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
......@@ -949,9 +1054,9 @@ class Test(NEOThreadedTest):
# modify x with another client
with cluster.newClient() as client:
txn = transaction.Transaction()
client.tpc_begin(txn)
client.tpc_begin(None, txn)
client.store(x1._p_oid, x1._p_serial, y, '', txn)
tid = client.tpc_finish(txn, None)
tid = client.tpc_finish(txn)
#self.tic() # NOTE ----//----
# Check reconnection to the master and storage.
......@@ -966,11 +1071,11 @@ class Test(NEOThreadedTest):
if 1:
client = cluster.client
txn = transaction.Transaction()
client.tpc_begin(txn)
client.tpc_begin(None, txn)
txn_context = client._txn_container.get(txn)
txn_context['ttid'] = add64(txn_context['ttid'], 1)
txn_context.ttid = add64(txn_context.ttid, 1)
self.assertRaises(POSException.StorageError,
client.tpc_finish, txn, None)
client.tpc_finish, txn)
@with_cluster()
def testStorageFailureDuringTpcFinish(self, cluster):
......@@ -1093,18 +1198,30 @@ class Test(NEOThreadedTest):
@with_cluster()
def testRecycledClientUUID(self, cluster):
def notReady(orig, *args):
m2s.discard(delayNotifyInformation)
return orig(*args)
if 1:
cluster.getTransaction()
with cluster.master.filterConnection(cluster.storage) as m2s:
delayNotifyInformation = m2s.delayNotifyNodeInformation()
cluster.client.master_conn.close()
with cluster.newClient() as client, Patch(
client.storage_bootstrap_handler, notReady=notReady):
x = client.load(ZERO_TID)
self.assertNotIn(delayNotifyInformation, m2s)
l = threading.Semaphore(0)
idle = []
def requestIdentification(orig, *args):
try:
orig(*args)
finally:
idle.append(cluster.storage.em.isIdle())
l.release()
cluster.db
with cluster.master.filterConnection(cluster.storage) as m2s:
delayNotifyInformation = m2s.delayNotifyNodeInformation()
cluster.client.master_conn.close()
with cluster.newClient() as client:
with Patch(IdentificationHandler,
requestIdentification=requestIdentification):
load = self.newThread(client.load, ZERO_TID)
l.acquire()
m2s.remove(delayNotifyInformation) # 2 packets pending
# Identification of the second client is retried
# after each processed notification:
l.acquire() # first client down
l.acquire() # new client up
load.join()
self.assertEqual(idle, [1, 1, 0])
@with_cluster(start_cluster=0, storage_count=3, autostart=3)
def testAutostart(self, cluster):
......@@ -1146,6 +1263,7 @@ class Test(NEOThreadedTest):
sys.exit()
def askPartitionTable(orig, self, conn):
p.revert()
del conn._queue[:] # XXX
conn.close()
if 1:
with Patch(cluster.master.pt, make=make), \
......@@ -1209,6 +1327,36 @@ class Test(NEOThreadedTest):
for s in cluster.storage_list:
self.assertEqual(s.dm.getLastIDs()[0], truncate_tid)
def testConnectionAbort(self):
with self.getLoopbackConnection() as client:
poll = client.em.poll
while client.connecting:
poll(1)
server, = (c for c in client.em.connection_dict.itervalues()
if c.isServer())
client.send(Packets.NotifyReady())
def writable(orig):
p.revert()
r = orig()
client.send(Packets.Ping())
return r
def process(orig):
self.assertFalse(server.aborted)
r = orig()
self.assertTrue(server.aborted)
server.em.removeWriter(server)
return r
with Patch(client, writable=writable) as p, \
Patch(server, process=process):
poll(0)
poll(0)
server.em.addWriter(server)
self.assertIsNot(server.connector, None)
poll(0)
self.assertIs(server.connector, None)
poll(0)
self.assertIs(client.connector, None)
def testConnectionTimeout(self):
with self.getLoopbackConnection() as conn:
conn.KEEP_ALIVE
......@@ -1283,7 +1431,7 @@ class Test(NEOThreadedTest):
s2c.append(self)
ll()
def connectToStorage(client):
next(client.cp.iterateForObject(0))
client._askStorageForRead(0, None, lambda *_: None)
if 1:
Ca = cluster.client
Ca.pt # only connect to the master
......@@ -1341,11 +1489,11 @@ class Test(NEOThreadedTest):
reports a conflict after that this conflict was fully resolved with
another node.
"""
def answerStoreObject(orig, conn, conflicting, *args):
if not conflicting:
def answerStoreObject(orig, conn, conflict, oid, serial):
if not conflict:
p.revert()
ll()
orig(conn, conflicting, *args)
orig(conn, conflict, oid, serial)
if 1:
s0, s1 = cluster.storage_list
t1, c1 = cluster.getTransaction()
......@@ -1363,5 +1511,715 @@ class Test(NEOThreadedTest):
ll()
t.join()
@with_cluster()
def testSameNewOidAndConflictOnBigValue(self, cluster):
storage = cluster.getZODBStorage()
oid = storage.new_oid()
txn = transaction.Transaction()
storage.tpc_begin(txn)
storage.store(oid, None, 'foo', '', txn)
storage.tpc_vote(txn)
storage.tpc_finish(txn)
txn = transaction.Transaction()
storage.tpc_begin(txn)
self.assertRaises(POSException.ConflictError, storage.store,
oid, None, '*' * cluster.cache_size, '', txn)
@with_cluster(replicas=1)
def testConflictWithOutOfDateCell(self, cluster):
"""
C1 S1 S0 C2
begin down begin
U <------- commit
up (remaining out-of-date due to suspended replication)
store ---> O (stored lockless)
`--------------> conflict
resolve -> stored lockless
`------------> locked
committed
"""
s0, s1 = cluster.storage_list
t1, c1 = cluster.getTransaction()
c1.root()['x'] = x = PCounterWithResolution()
t1.commit()
s1.stop()
cluster.join((s1,))
x.value += 1
t2, c2 = cluster.getTransaction()
c2.root()['x'].value += 2
t2.commit()
with ConnectionFilter() as f:
f.delayAskFetchTransactions()
s1.resetNode()
s1.start()
self.tic()
t1.commit()
@with_cluster(replicas=1)
def testReplicaDisconnectionDuringCommit(self, cluster):
"""
S0 C S1
<------- c1+=1 -->
<------- c2+=2 --> C-S1 closed
<------- c3+=3
U U
finish O
U
down
loads <--
"""
count = [0]
def ask(orig, self, packet, **kw):
if (isinstance(packet, Packets.AskStoreObject)
and self.getUUID() == s1.uuid):
count[0] += 1
if count[0] == 2:
self.close()
return orig(self, packet, **kw)
s0, s1 = cluster.storage_list
t, c = cluster.getTransaction()
r = c.root()
for x in xrange(3):
r[x] = PCounter()
t.commit()
for x in xrange(3):
r[x].value += x
with ConnectionFilter() as f, Patch(MTClientConnection, ask=ask):
f.delayAskFetchTransactions()
t.commit()
self.assertEqual(count[0], 2)
self.assertPartitionTable(cluster, 'UO')
self.tic()
s0.stop()
cluster.join((s0,))
cluster.client._cache.clear()
value_list = []
for x in xrange(3):
r[x]._p_deactivate()
value_list.append(r[x].value)
self.assertEqual(value_list, range(3))
@with_cluster(replicas=1, partitions=3, storage_count=3)
def testMasterArbitratingVote(self, cluster):
"""
p\S 1 2 3
0 U U .
1 . U U
2 U . U
With the above setup, check when a client C1 fails to connect to S2
and another C2 fails to connect to S1.
For the first 2 scenarios:
- C1 first votes (the master accepts)
- C2 vote is delayed until C1 decides to finish or abort
"""
def delayAbort(conn, packet):
return isinstance(packet, Packets.AbortTransaction)
def c1_vote(txn):
def vote(orig, *args):
try:
return orig(*args)
finally:
ll()
with LockLock() as ll, Patch(cluster.master.tm, vote=vote):
commit2.start()
ll()
if c1_aborts:
raise Exception
pt = [{x.getUUID() for x in x}
for x in cluster.master.pt.partition_list]
cluster.storage_list.sort(key=lambda x:
(x.uuid not in pt[0], x.uuid in pt[1]))
pt = 'UU.|.UU|U.U'
self.assertPartitionTable(cluster, pt)
s1, s2, s3 = cluster.storage_list
t1, c1 = cluster.getTransaction()
with cluster.newClient(1) as db:
t2, c2 = cluster.getTransaction(db)
with self.noConnection(c1, s2), self.noConnection(c2, s1):
cluster.client.cp.connection_dict[s2.uuid].close()
self.tic()
for c1_aborts in 0, 1:
# 0: C1 finishes, C2 vote fails
# 1: C1 aborts, C2 finishes
#
# Although we try to modify the same oid, there's no
# conflict because each storage node sees a single
# and different transaction: vote to storages is done
# in parallel, and the master must be involved as an
# arbitrator, which ultimately rejects 1 of the 2
# transactions, preferably before the second phase of
# the commit.
t1.begin(); c1.root()._p_changed = 1
t2.begin(); c2.root()._p_changed = 1
commit2 = self.newPausedThread(t2.commit)
TransactionalResource(t1, 1, tpc_vote=c1_vote)
with ConnectionFilter() as f:
if not c1_aborts:
f.add(delayAbort)
f.delayAskFetchTransactions(lambda _:
f.discard(delayAbort))
try:
t1.commit()
self.assertFalse(c1_aborts)
except Exception:
self.assertTrue(c1_aborts)
try:
commit2.join()
self.assertTrue(c1_aborts)
except NEOStorageError:
self.assertFalse(c1_aborts)
self.tic()
self.assertPartitionTable(cluster,
'OU.|.UU|O.U' if c1_aborts else 'UO.|.OU|U.U')
self.tic()
self.assertPartitionTable(cluster, pt)
# S3 fails while C1 starts to finish
with ConnectionFilter() as f:
f.add(lambda conn, packet: conn.getUUID() == s3.uuid and
isinstance(packet, Packets.AcceptIdentification))
t1.begin(); c1.root()._p_changed = 1
TransactionalResource(t1, 0, tpc_finish=lambda *_:
cluster.master.nm.getByUUID(s3.uuid)
.getConnection().close())
self.assertRaises(NEOStorageError, t1.commit)
self.assertPartitionTable(cluster, 'UU.|.UO|U.O')
self.tic()
self.assertPartitionTable(cluster, pt)
@with_cluster()
def testAbortTransaction(self, cluster):
t, c = cluster.getTransaction()
r = c.root()
r._p_changed = 1
def abort(_):
raise Exception
TransactionalResource(t, 0, tpc_vote=abort)
with cluster.client.filterConnection(cluster.storage) as cs:
cs.delayAskStoreObject()
self.assertRaises(Exception, t.commit)
t.begin()
r._p_changed = 1
t.commit()
@with_cluster(replicas=1)
def testPartialConflict(self, cluster):
"""
This scenario proves that the client must keep the data of a modified
oid until it is successfully stored to all storages. Indeed, if a
concurrent transaction fails to commit to all storage nodes, we must
handle inconsistent results from replicas.
C1 S1 S2 C2
no connection between S1 and C2
store ---> locked <------ commit
`--------------> conflict
"""
def begin1(*_):
t2.commit()
f.add(delayAnswerStoreObject, Patch(Transaction, written=written))
def delayAnswerStoreObject(conn, packet):
return (isinstance(packet, Packets.AnswerStoreObject)
and getattr(conn.getHandler(), 'app', None) is s)
def written(orig, *args):
orig(*args)
f.remove(delayAnswerStoreObject)
def sync(orig):
mc1.remove(delayMaster)
orig()
s1 = cluster.storage_list[0]
t1, c1 = cluster.getTransaction()
c1.root()['x'] = x = PCounterWithResolution()
t1.commit()
with cluster.newClient(1) as db:
t2, c2 = cluster.getTransaction(db)
with self.noConnection(c2, s1):
for s in cluster.storage_list:
logging.info("late answer from %s", uuid_str(s.uuid))
x.value += 1
c2.root()['x'].value += 2
TransactionalResource(t1, 1, tpc_begin=begin1)
s1m, = s1.getConnectionList(cluster.master)
try:
s1.em.removeReader(s1m)
with ConnectionFilter() as f, \
cluster.master.filterConnection(
cluster.client) as mc1:
f.delayAskFetchTransactions()
delayMaster = mc1.delayNotifyNodeInformation(
Patch(cluster.client, sync=sync))
t1.commit()
self.assertPartitionTable(cluster, 'OU')
finally:
s1.em.addReader(s1m)
self.tic()
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):
"""Transaction checking an oid more than once
1. t1 < t2
2. t1 deadlocks, t2 gets all locks
3. t2 < t1 < t3
4. t2 finishes: conflict on A, t1 locks C, t3 locks B
5. t1 rebases B -> second deadlock
6. t1 resolves A
7. t3 resolves A -> deadlock, and t1 locks B
8. t1 rebases B whereas it was already locked
"""
def changes(*r):
for r in r:
r['a'].value += 1
self.readCurrent(r['b'])
self.readCurrent(r['c'])
t = []
def vote_t2(*args, **kw):
yield 0
t.append(threading.currentThread())
def tic_t1(*args, **kw):
# Make sure t2 finishes before rebasing B,
# so that B is locked by a newer transaction (t3).
t[0].join()
yield 0
end = self._testComplexDeadlockAvoidanceWithOneStorage(changes,
(0, 1, 1, 0, 1, 1, 0, 0, 2, 2, 2, 2, 1, vote_t2, tic_t1),
('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(replicas=1)
def testNotifyReplicated2(self, cluster):
s0, s1 = cluster.storage_list
s1.stop()
cluster.join((s1,))
s1.resetNode()
t1, c1 = cluster.getTransaction()
r = c1.root()
for x in 'ab':
r[x] = PCounterWithResolution()
t1.commit()
r['a'].value += 1
r['b'].value += 2
t2, c2 = cluster.getTransaction()
r = c2.root()
r['a'].value += 3
r['b'].value += 4
thread = self.newPausedThread(t2.commit)
def t2_b(*args, **kw):
self.assertPartitionTable(cluster, 'UO')
f.remove(delay)
self.tic()
self.assertPartitionTable(cluster, 'UO')
yield 0
def t2_vote(*args, **kw):
self.tic()
self.assertPartitionTable(cluster, 'UU')
yield 0
with ConnectionFilter() as f, \
self.thread_switcher((thread,),
(1, 0, 1, 1, t2_b, 0, 0, 1, t2_vote, 0, 0),
('tpc_begin', 'tpc_begin', 1, 1, 2, 2,
'RebaseTransaction', 'RebaseTransaction', 'StoreTransaction',
'AnswerRebaseTransaction', 'AnswerRebaseTransaction',
)) as end:
delay = f.delayAskFetchTransactions()
s1.start()
self.tic()
t1.commit()
thread.join()
t2.begin()
self.assertEqual([4, 6], [r[x].value for x in 'ab'])
@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()
def testDelayedStoreOrdering(self, cluster):
"""
By processing delayed stores (EventQueue) in the order of their locking
tid, we minimize the number deadlocks. Here, we trigger a first
deadlock, so that the delayed check for t1 is reordered after that of
t3.
"""
t1, c1 = cluster.getTransaction()
r = c1.root()
for x in 'abcd':
r[x] = PCounter()
t1.commit()
r['a'].value += 1
self.readCurrent(r['d'])
t2, c2 = cluster.getTransaction()
r = c2.root()
r['b'].value += 1
self.readCurrent(r['d'])
t3, c3 = cluster.getTransaction()
r = c3.root()
r['c'].value += 1
self.readCurrent(r['d'])
threads = map(self.newPausedThread, (t2.commit, t3.commit))
with self.thread_switcher(threads, (1, 2, 0, 1, 2, 1, 0, 2, 0, 1, 2),
('tpc_begin', 'tpc_begin', 'tpc_begin', 1, 2, 3, 4, 4, 4,
'RebaseTransaction', 'StoreTransaction')) as end:
t1.commit()
for t in threads:
t.join()
self.assertEqual(end, {
0: ['AnswerRebaseTransaction', 'StoreTransaction'],
2: ['StoreTransaction']})
@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()
......@@ -15,10 +15,8 @@
# along with this program. If not, see <http://www.gnu.org/licenses/>.
from logging import getLogger, INFO, DEBUG
import random, sys, threading, time
import random
import sys
import time
import transaction
from ZODB.POSException import ReadOnlyError, POSKeyError
import unittest
......@@ -34,10 +32,11 @@ from neo.lib.connection import ClientConnection
from neo.lib.event import EventManager
from neo.lib.protocol import CellStates, ClusterStates, Packets, \
ZERO_OID, ZERO_TID, MAX_TID, uuid_str
from neo.lib.util import p64
from .. import expectedFailure, Patch
from neo.lib.util import p64, u64
from .. import expectedFailure, Patch, TransactionalResource
from . import ConnectionFilter, NEOCluster, NEOThreadedTest, \
predictable_random, with_cluster
from .test import PCounter, PCounterWithResolution # XXX
# dump log to stderr
"""
......@@ -91,6 +90,16 @@ class ReplicationTests(NEOThreadedTest):
checked += 1
return checked
def checkReplicas(self, cluster):
pt = cluster.primary_master.pt
storage_dict = {x.uuid: x for x in cluster.storage_list}
for offset in xrange(pt.getPartitions()):
checksum_list = [
self.checksumPartition(storage_dict[x.getUUID()], offset)
for x in pt.getCellList(offset)]
self.assertEqual(1, len(set(checksum_list)),
(offset, checksum_list))
def testBackupNormalCase(self):
np = 7
nr = 2
......@@ -458,12 +467,9 @@ class ReplicationTests(NEOThreadedTest):
@with_cluster(start_cluster=0, partitions=2, storage_count=2)
def testClientReadingDuringTweak(self, cluster):
# XXX: Currently, the test passes because data of dropped cells are not
# deleted while the cluster is operational: this is only done
# during the RECOVERING phase. But we'll want to be able to free
# disk space without service interruption, and for this the client
# may have to retry reading data from the new cells. If s0 deleted
# all data for partition 1, the test would fail with a POSKeyError.
def sync(orig):
m2c.remove(delay)
orig()
s0, s1 = cluster.storage_list
if 1:
cluster.start([s0])
......@@ -479,9 +485,145 @@ class ReplicationTests(NEOThreadedTest):
cluster.neoctl.enableStorageList([s1.uuid])
cluster.neoctl.tweakPartitionTable()
with cluster.master.filterConnection(cluster.client) as m2c:
m2c.delayNotifyPartitionChanges()
delay = m2c.delayNotifyPartitionChanges()
self.tic()
with Patch(cluster.client, sync=sync):
self.assertEqual('foo', storage.load(oid)[0])
self.assertNotIn(delay, m2c)
@with_cluster(start_cluster=False, storage_count=3, partitions=3)
def testAbortingReplication(self, cluster):
s1, s2, s3 = cluster.storage_list
cluster.start((s1, s2))
t, c = cluster.getTransaction()
r = c.root()
for x in 'ab':
r[x] = PCounter()
t.commit()
cluster.stop(replicas=1)
cluster.start((s1, s2))
with ConnectionFilter() as f:
f.delayAddObject()
cluster.neoctl.tweakPartitionTable()
s3.start()
self.tic()
cluster.neoctl.enableStorageList((s3.uuid,))
cluster.neoctl.tweakPartitionTable()
self.tic()
self.tic()
for s in cluster.storage_list:
self.assertTrue(s.is_alive())
self.checkReplicas(cluster)
@with_cluster(start_cluster=0, replicas=1, storage_count=4, partitions=2)
def testTweakVsReplication(self, cluster, done=False):
S = cluster.storage_list
cluster.start(S[:1])
t, c = cluster.getTransaction()
ob = c.root()[''] = PCounterWithResolution()
t.commit()
self.assertEqual(1, u64(ob._p_oid))
for s in S[1:]:
s.start()
self.tic()
def tweak():
self.tic()
self.assertFalse(delay_list)
self.assertPartitionTable(cluster, 'UU|UO')
f.delayAskFetchObjects()
cluster.enableStorageList(S[2:])
cluster.neoctl.tweakPartitionTable()
self.tic()
self.assertPartitionTable(cluster, 'UU..|F.OO')
with ConnectionFilter() as f, cluster.moduloTID(1), \
Patch(S[1].replicator,
_nextPartitionSortKey=lambda orig, offset: offset):
delay_list = [1, 0]
delay = (f.delayNotifyReplicationDone if done else
f.delayAnswerFetchObjects)(lambda _: delay_list.pop())
cluster.enableStorageList((S[1],))
cluster.neoctl.tweakPartitionTable()
ob._p_changed = 1
if done:
tweak()
t.commit()
else:
t2, c2 = cluster.getTransaction()
c2.root()['']._p_changed = 1
l = threading.Lock(); l.acquire()
TransactionalResource(t2, 0, tpc_vote=lambda _: l.release())
t2 = self.newPausedThread(t2.commit)
self.tic()
@TransactionalResource(t, 0)
def tpc_vote(_):
t2.start()
l.acquire()
f.remove(delay)
tweak()
t.commit()
t2.join()
cluster.neoctl.dropNode(S[2].uuid)
cluster.neoctl.dropNode(S[3].uuid)
cluster.neoctl.tweakPartitionTable()
if done:
f.remove(delay)
self.tic()
self.assertPartitionTable(cluster, 'UU|UO')
self.tic()
self.assertPartitionTable(cluster, 'UU|UU')
self.checkReplicas(cluster)
def testTweakVsReplicationDone(self):
self.testTweakVsReplication(True)
@with_cluster(start_cluster=0, storage_count=2, partitions=2)
def testCommitVsDiscardedCell(self, cluster):
s0, s1 = cluster.storage_list
cluster.start((s0,))
t, c = cluster.getTransaction()
ob = c.root()[''] = PCounterWithResolution()
t.commit()
self.assertEqual(1, u64(ob._p_oid))
s1.start()
self.tic()
nonlocal_ = []
with ConnectionFilter() as f:
delay = f.delayNotifyReplicationDone()
cluster.enableStorageList((s1,))
cluster.neoctl.tweakPartitionTable()
self.tic()
self.assertPartitionTable(cluster, 'U.|FO')
t2, c2 = cluster.getTransaction()
c2.root()[''].value += 3
l = threading.Lock(); l.acquire()
@TransactionalResource(t2, 0)
def tpc_vote(_):
self.tic()
l.release()
t2 = self.newPausedThread(t2.commit)
@TransactionalResource(t, 0, tpc_finish=lambda _:
f.remove(nonlocal_.pop(0)))
def tpc_vote(_):
t2.start()
l.acquire()
nonlocal_.append(f.delayNotifyPartitionChanges())
f.remove(delay)
self.tic()
self.assertEqual('foo', storage.load(oid)[0])
self.assertPartitionTable(cluster, 'U.|.U', cluster.master)
nonlocal_.append(cluster.master.pt.getID())
ob.value += 2
t.commit()
t2.join()
self.tic()
self.assertPartitionTable(cluster, 'U.|.U')
self.assertEqual(cluster.master.pt.getID(), nonlocal_.pop())
t.begin()
self.assertEqual(ob.value, 5)
# get the second to last tid (for which ob=2)
tid2 = s1.dm.getObject(ob._p_oid, None, ob._p_serial)[0]
# s0 must not have committed anything for partition 1
with s0.dm.replicated(1):
self.assertFalse(s0.dm.getObject(ob._p_oid, tid2))
@with_cluster(start_cluster=0, replicas=1)
def testResumingReplication(self, cluster):
......@@ -511,39 +653,53 @@ class ReplicationTests(NEOThreadedTest):
cluster.join((s0,))
t0, t1, t2 = c.db().storage.iterator()
@with_cluster(start_cluster=0, replicas=1)
def testReplicationBlockedByUnfinished(self, cluster):
if 1:
s0, s1 = cluster.storage_list
cluster.start(storage_list=(s0,))
storage = cluster.getZODBStorage()
oid = storage.new_oid()
@with_cluster(start_cluster=0, replicas=1, partitions=2)
def testReplicationBlockedByUnfinished1(self, cluster,
delay_replication=False):
s0, s1 = cluster.storage_list
cluster.start(storage_list=(s0,))
storage = cluster.getZODBStorage()
oid = storage.new_oid()
with ConnectionFilter() as f, cluster.moduloTID(1 - u64(oid) % 2):
if delay_replication:
delay_replication = f.delayAnswerFetchObjects()
tid = None
expected = 'UO'
for n in 1, 0:
# On first iteration, the transaction will block replication
expected = 'U|U'
for n in xrange(3):
# On second iteration, the transaction will block replication
# until tpc_finish.
# We do a second iteration as a quick check that the cluster
# We do a last iteration as a quick check that the cluster
# remains functional after such a scenario.
txn = transaction.Transaction()
storage.tpc_begin(txn)
tid = storage.store(oid, tid, 'foo', '', txn)
if n:
tid = storage.store(oid, tid, str(n), '', txn)
if n == 1:
# Start the outdated storage.
s1.start()
self.tic()
cluster.enableStorageList((s1,))
cluster.neoctl.tweakPartitionTable()
expected = 'UO|UO'
self.tic()
self.assertPartitionTable(cluster, expected)
storage.tpc_vote(txn)
self.assertPartitionTable(cluster, expected)
tid = storage.tpc_finish(txn)
self.tic() # replication resumes and ends
expected = 'UU'
if n == 1:
if delay_replication:
self.tic()
self.assertPartitionTable(cluster, expected)
f.remove(delay_replication)
delay_replication = None
self.tic() # replication resumes and ends
expected = 'UU|UU'
self.assertPartitionTable(cluster, expected)
self.assertEqual(cluster.neoctl.getClusterState(),
ClusterStates.RUNNING)
self.checkPartitionReplicated(s0, s1, 0)
def testReplicationBlockedByUnfinished2(self):
self.testReplicationBlockedByUnfinished1(True)
@with_cluster(partitions=5, replicas=2, storage_count=3)
def testCheckReplicas(self, cluster):
......
......@@ -17,7 +17,7 @@
import unittest
from neo.lib.protocol import Packets
from .. import SSL
from . import NEOCluster, test, testReplication
from . import NEOCluster, with_cluster, test, testReplication
class SSLMixin:
......@@ -36,6 +36,14 @@ class SSLTests(SSLMixin, test.Test):
testDeadlockAvoidance = None # XXX why this fails?
testUndoConflict = testUndoConflictDuringStore = None # XXX why this fails?
if 1:
testShutdownWithSeveralMasterNodes = unittest.skip("fails randomly")(
test.Test.testShutdown.__func__)
@with_cluster(partitions=10, replicas=1, storage_count=3)
def testShutdown(self, cluster):
self._testShutdown(cluster)
def testAbortConnection(self, after_handshake=1):
with self.getLoopbackConnection() as conn:
conn.ask(Packets.Ping())
......
......@@ -25,7 +25,7 @@ else:
class ZODBTestCase(TestCase):
def setUp(self, cluster_kw={}):
def setUp(self):
super(ZODBTestCase, self).setUp()
storages = int(os.getenv('NEO_TEST_ZODB_STORAGES', 1))
kw = {
......@@ -34,7 +34,6 @@ class ZODBTestCase(TestCase):
'partitions': int(os.getenv('NEO_TEST_ZODB_PARTITIONS', 1)),
'db_list': ['%s%u' % (DB_PREFIX, i) for i in xrange(storages)],
}
kw.update(cluster_kw)
if functional:
kw['temp_dir'] = self.getTempDirectory()
self.neo = NEOCluster(**kw)
......
......@@ -22,14 +22,21 @@ from ZODB.tests.StorageTestBase import StorageTestBase
from . import ZODBTestCase
from .. import Patch, threaded
class TIC_LOOP(object):
def __init__(self):
self.stop = time.time() + 10
def __iter__(self):
def tic_loop(t=time.time, x=self.stop):
while t() < x:
yield
return tic_loop()
class BasicTests(ZODBTestCase, StorageTestBase, BasicStorage):
def check_checkCurrentSerialInTransaction(self):
x = time.time() + 10
def tic_loop():
while time.time() < x:
yield
with Patch(threaded, TIC_LOOP=tic_loop()):
with Patch(threaded, TIC_LOOP=TIC_LOOP()):
super(BasicTests, self).check_checkCurrentSerialInTransaction()
if __name__ == "__main__":
......
......@@ -26,9 +26,6 @@ from . import ZODBTestCase
class PackableTests(ZODBTestCase, StorageTestBase,
PackableStorageWithOptionalGC, PackableUndoStorage):
def setUp(self):
super(PackableTests, self).setUp(cluster_kw={'adapter': 'MySQL'})
checkPackAllRevisions = expectedFailure()(
PackableStorageWithOptionalGC.checkPackAllRevisions)
checkPackUndoLog = expectedFailure()(PackableUndoStorage.checkPackUndoLog)
......
......@@ -28,14 +28,6 @@ class UndoTests(ZODBTestCase, StorageTestBase, TransactionalUndoStorage,
checkTransactionalUndoAfterPack = expectedFailure()(
TransactionalUndoStorage.checkTransactionalUndoAfterPack)
for x in ('checkUndoMultipleConflictResolution',
'checkUndoMultipleConflictResolutionReversed'):
try:
setattr(UndoTests, x,
expectedFailure(KeyError)(getattr(TransactionalUndoStorage, x)))
except AttributeError:
pass
if __name__ == "__main__":
suite = unittest.makeSuite(UndoTests, 'check')
unittest.main(defaultTest='suite')
......
......@@ -84,6 +84,9 @@ setup(
'neosimple=neo.scripts.simple:main',
'stat_zodb=neo.tests.stat_zodb:main',
],
'zodburi.resolvers': [
'neo = neo.client.zodburi:resolve_uri [client]',
],
},
extras_require = extras_require,
package_data = {
......
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