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). ...@@ -4,43 +4,6 @@ or promised features of NEO (marked with N).
All the listed bugs will be fixed with high priority. 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 (N) A backup cell may be wrongly marked as corrupted while checking replicas
---------------------------------------------------------------------------- ----------------------------------------------------------------------------
......
...@@ -206,8 +206,9 @@ lists: ...@@ -206,8 +206,9 @@ lists:
users discussion users discussion
- `neo-dev <http://mail.tiolive.com/mailman/listinfo/neo-dev>`_: - `neo-dev <http://mail.tiolive.com/mailman/listinfo/neo-dev>`_:
developers discussion 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 Commercial Support
================== ==================
......
...@@ -15,16 +15,13 @@ ...@@ -15,16 +15,13 @@
General General
- Review XXX/TODO code tags (CODE) - 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 - 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 could become UP_TO_DATE with appropriate backup_tid, so that the cluster
stays operational. (FEATURE) stays operational. (FEATURE)
- Finish renaming UUID into NID everywhere (CODE) - 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 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 some reasons is immediately rejected with the 'not ready' error code.
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.
This is mainly the case for : This is mainly the case for :
- Client rejected before the cluster is operational - Client rejected before the cluster is operational
- Empty storages rejected during recovery process - Empty storages rejected during recovery process
...@@ -41,36 +38,20 @@ ...@@ -41,36 +38,20 @@
- Clarify handler methods to call when a connection is accepted from a - Clarify handler methods to call when a connection is accepted from a
listening conenction and when remote node is identified listening conenction and when remote node is identified
(cf. neo/lib/bootstrap.py). (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() - Review PENDING/HIDDEN/SHUTDOWN states, don't use notifyNodeInformation()
to do a state-switch, use a exception-based mechanism ? (CODE) to do a state-switch, use a exception-based mechanism ? (CODE)
- Review handler split (CODE) - Review handler split (CODE)
The current handler split is the result of small incremental changes. A The current handler split is the result of small incremental changes. A
global review is required to make them square. 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 - Review transactional isolation of various methods
Some methods might not implement proper transaction isolation when they Some methods might not implement proper transaction isolation when they
should. An example is object history (undoLog), which can see data should. An example is object history (undoLog), which can see data
committed by future transactions. committed by future transactions.
- Add a 'devid' storage configuration so that master do not distribute - Add a 'devid' storage configuration so that master do not distribute
replicated partitions on storages with same 'devid'. 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 Storage
- Use libmysqld instead of a stand-alone MySQL server. - 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 - In backup mode, 2 simultaneous replication should be possible so that: NOTE
- outdated cells does not block backup for too long time - outdated cells does not block backup for too long time
- constantly modified partitions does not prevent outdated cells to - constantly modified partitions does not prevent outdated cells to
...@@ -78,9 +59,7 @@ ...@@ -78,9 +59,7 @@
Current behaviour is undefined and the above 2 scenarios may happen. Current behaviour is undefined and the above 2 scenarios may happen.
- Create a specialized PartitionTable that know the database and replicator - Create a specialized PartitionTable that know the database and replicator
to remove duplicates and remove logic from handlers (CODE) to remove duplicates and remove logic from handlers (CODE)
- Consider insert multiple objects at time in the database, with taking care - Make listening address and port optional, and if they are not provided
of maximum SQL request size allowed. (SPEED)
- Make listening address and port optionnal, and if they are not provided
listen on all interfaces on any available port. listen on all interfaces on any available port.
- Make replication speed configurable (HIGH AVAILABILITY) - Make replication speed configurable (HIGH AVAILABILITY)
In its current implementation, replication runs at lowest priority, to In its current implementation, replication runs at lowest priority, to
...@@ -125,15 +104,11 @@ ...@@ -125,15 +104,11 @@
instead of parsing the whole partition table. (SPEED) instead of parsing the whole partition table. (SPEED)
Client Client
- Race conditions on the partition table ?
(update by the poll thread vs. access by other threads)
- Merge Application into Storage (SPEED) - Merge Application into Storage (SPEED)
- Optimize cache.py by rewriting it either in C or Cython (LOAD LATENCY) - Optimize cache.py by rewriting it either in C or Cython (LOAD LATENCY)
- Use generic bootstrap module (CODE) - 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 Admin
- Make admin node able to monitor multiple clusters simultaneously - Make admin node able to monitor multiple clusters simultaneously
...@@ -141,6 +116,7 @@ ...@@ -141,6 +116,7 @@
- Add ctl command to list last transactions, like fstail for FileStorage. - Add ctl command to list last transactions, like fstail for FileStorage.
Tests Tests
- Split neo/tests/threaded/test.py
- Use another mock library: Python 3.3+ has unittest.mock, which is - Use another mock library: Python 3.3+ has unittest.mock, which is
available for earlier versions at https://pypi.python.org/pypi/mock available for earlier versions at https://pypi.python.org/pypi/mock
......
...@@ -58,7 +58,8 @@ partitions: 12 ...@@ -58,7 +58,8 @@ partitions: 12
# - MySQL: [user[:password]@]database[unix_socket] # - MySQL: [user[:password]@]database[unix_socket]
# Database must be created manually. # Database must be created manually.
# - SQLite: path # - 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 node
[admin] [admin]
...@@ -67,7 +68,7 @@ bind: 127.0.0.1:9999 ...@@ -67,7 +68,7 @@ bind: 127.0.0.1:9999
# common section. # common section.
;logfile: ~/log/admin.log ;logfile: ~/log/admin.log
# Nodes can have their own certicates. # Nodes can have their own certificates.
;cert = admin.crt ;cert = admin.crt
;key = admin.key ;key = admin.key
......
...@@ -139,6 +139,6 @@ class Application(BaseApplication): ...@@ -139,6 +139,6 @@ class Application(BaseApplication):
pass pass
row_list.append((offset, row)) row_list.append((offset, row))
except IndexError: except IndexError:
conn.notify(Errors.ProtocolError('invalid partition table offset')) conn.send(Errors.ProtocolError('invalid partition table offset'))
else: else:
conn.answer(Packets.AnswerPartitionList(self.pt.getID(), row_list)) conn.answer(Packets.AnswerPartitionList(self.pt.getID(), row_list))
...@@ -89,16 +89,16 @@ class Storage(BaseStorage.BaseStorage, ...@@ -89,16 +89,16 @@ class Storage(BaseStorage.BaseStorage,
""" """
Note: never blocks in NEO. 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): def tpc_vote(self, transaction):
return self.app.tpc_vote(transaction, self.tryToResolveConflict) return self.app.tpc_vote(transaction)
def tpc_abort(self, transaction): def tpc_abort(self, transaction):
return self.app.tpc_abort(transaction) return self.app.tpc_abort(transaction)
def tpc_finish(self, transaction, f=None): 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): def store(self, oid, serial, data, version, transaction):
assert version == '', 'Versions are not supported' assert version == '', 'Versions are not supported'
...@@ -128,7 +128,7 @@ class Storage(BaseStorage.BaseStorage, ...@@ -128,7 +128,7 @@ class Storage(BaseStorage.BaseStorage,
# undo # undo
def undo(self, transaction_id, txn): 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): def undoLog(self, first=0, last=-20, filter=None):
return self.app.undoLog(first, last, filter) return self.app.undoLog(first, last, filter)
...@@ -167,8 +167,7 @@ class Storage(BaseStorage.BaseStorage, ...@@ -167,8 +167,7 @@ class Storage(BaseStorage.BaseStorage,
def importFrom(self, source, start=None, stop=None, preindex=None): def importFrom(self, source, start=None, stop=None, preindex=None):
""" Allow import only a part of the source storage """ """ Allow import only a part of the source storage """
return self.app.importFrom(source, start, stop, return self.app.importFrom(self, source, start, stop, preindex)
self.tryToResolveConflict, preindex)
def pack(self, t, referencesf, gc=False): def pack(self, t, referencesf, gc=False):
if gc: if gc:
......
This diff is collapsed.
...@@ -19,6 +19,9 @@ from ZODB import POSException ...@@ -19,6 +19,9 @@ from ZODB import POSException
class NEOStorageError(POSException.StorageError): class NEOStorageError(POSException.StorageError):
pass pass
class NEOStorageReadRetry(NEOStorageError):
pass
class NEOStorageNotFoundError(NEOStorageError): class NEOStorageNotFoundError(NEOStorageError):
pass pass
......
...@@ -147,10 +147,10 @@ class PrimaryNotificationsHandler(MTEventHandler): ...@@ -147,10 +147,10 @@ class PrimaryNotificationsHandler(MTEventHandler):
logging.critical(msg) logging.critical(msg)
app.master_conn = None app.master_conn = None
for txn_context in app.txn_contexts(): for txn_context in app.txn_contexts():
txn_context['error'] = msg txn_context.error = msg
try: try:
del app.pt app.__dict__.pop('pt').clear()
except AttributeError: except KeyError:
pass pass
app.primary_master_node = None app.primary_master_node = None
super(PrimaryNotificationsHandler, self).connectionClosed(conn) super(PrimaryNotificationsHandler, self).connectionClosed(conn)
...@@ -158,6 +158,10 @@ class PrimaryNotificationsHandler(MTEventHandler): ...@@ -158,6 +158,10 @@ class PrimaryNotificationsHandler(MTEventHandler):
def stopOperation(self, conn): def stopOperation(self, conn):
logging.critical("master node ask to stop operation") 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): def invalidateObjects(self, conn, tid, oid_list):
app = self.app app = self.app
if app.ignore_invalidations: if app.ignore_invalidations:
...@@ -182,9 +186,9 @@ class PrimaryNotificationsHandler(MTEventHandler): ...@@ -182,9 +186,9 @@ class PrimaryNotificationsHandler(MTEventHandler):
if self.app.pt.filled(): # XXX wrong if self.app.pt.filled(): # XXX wrong
self.app.pt.update(ptid, cell_list, self.app.nm) 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( super(PrimaryNotificationsHandler, self).notifyNodeInformation(
conn, node_list) conn, timestamp, node_list)
# XXX: 'update' automatically closes DOWN nodes. Do we really want # XXX: 'update' automatically closes DOWN nodes. Do we really want
# to do the same thing for nodes in other non-running states ? # to do the same thing for nodes in other non-running states ?
getByUUID = self.app.nm.getByUUID getByUUID = self.app.nm.getByUUID
...@@ -194,6 +198,13 @@ class PrimaryNotificationsHandler(MTEventHandler): ...@@ -194,6 +198,13 @@ class PrimaryNotificationsHandler(MTEventHandler):
if node and node.isConnected(): if node and node.isConnected():
node.getConnection().close() 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): class PrimaryAnswersHandler(AnswerBaseHandler):
""" Handle that process expected packets from the primary master """ """ Handle that process expected packets from the primary master """
...@@ -204,6 +215,10 @@ class PrimaryAnswersHandler(AnswerBaseHandler): ...@@ -204,6 +215,10 @@ class PrimaryAnswersHandler(AnswerBaseHandler):
oid_list.reverse() oid_list.reverse()
self.app.new_oid_list = oid_list 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): def answerTransactionFinished(self, conn, _, tid):
self.app.setHandlerData(tid) self.app.setHandlerData(tid)
......
This diff is collapsed.
...@@ -14,27 +14,18 @@ ...@@ -14,27 +14,18 @@
# You should have received a copy of the GNU General Public License # You should have received a copy of the GNU General Public License
# along with this program. If not, see <http://www.gnu.org/licenses/>. # along with this program. If not, see <http://www.gnu.org/licenses/>.
import time import random, time
from random import shuffle
from neo.lib import logging from neo.lib import logging
from neo.lib.locking import Lock from neo.lib.locking import Lock
from neo.lib.protocol import NodeTypes, Packets from neo.lib.protocol import NodeTypes, Packets
from neo.lib.connection import MTClientConnection, ConnectionClosed from neo.lib.connection import MTClientConnection, ConnectionClosed
from neo.lib.exception import NodeNotReady 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 # How long before we might retry a connection to a node to which connection
# failed in the past. # failed in the past.
MAX_FAILURE_AGE = 600 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): class ConnectionPool(object):
"""This class manages a pool of connections to storage nodes.""" """This class manages a pool of connections to storage nodes."""
...@@ -67,55 +58,24 @@ class ConnectionPool(object): ...@@ -67,55 +58,24 @@ class ConnectionPool(object):
else: else:
logging.info('Connected %r', node) logging.info('Connected %r', node)
return conn return conn
self.notifyFailure(node)
def notifyFailure(self, node):
self.node_failure_dict[node.getUUID()] = time.time() + MAX_FAILURE_AGE 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() uuid = cell.getUUID()
# First, prefer a connected node.
if uuid in self.connection_dict: 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) failure = self.node_failure_dict.get(uuid)
if failure: if failure:
if time.time() < 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) self.node_failure_dict.pop(uuid, None)
return CELL_GOOD return 1 + random()
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
def getConnForNode(self, node): def getConnForNode(self, node):
"""Return a locked connection object to a given 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): ...@@ -38,6 +38,7 @@ class BootstrapManager(EventHandler):
self.num_replicas = None self.num_replicas = None
self.num_partitions = None self.num_partitions = None
self.current = None self.current = None
app.nm.reset()
uuid = property(lambda self: self.app.uuid) uuid = property(lambda self: self.app.uuid)
......
...@@ -106,7 +106,7 @@ class ConfigurationManager(object): ...@@ -106,7 +106,7 @@ class ConfigurationManager(object):
return self.__get('engine', True) return self.__get('engine', True)
def getWait(self): def getWait(self):
# BUG # XXX: see also DatabaseManager.__init__
return self.__get('wait') return self.__get('wait')
def getDynamicMasterList(self): def getDynamicMasterList(self):
......
...@@ -22,7 +22,7 @@ from .connector import ConnectorException, ConnectorDelayedConnection ...@@ -22,7 +22,7 @@ from .connector import ConnectorException, ConnectorDelayedConnection
from .locking import RLock from .locking import RLock
from .protocol import uuid_str, Errors, \ from .protocol import uuid_str, Errors, \
PacketMalformedError, Packets, ParserState PacketMalformedError, Packets, ParserState
from .util import ReadBuffer from .util import dummy_read_buffer, ReadBuffer
CRITICAL_TIMEOUT = 30 CRITICAL_TIMEOUT = 30
...@@ -115,16 +115,17 @@ class HandlerSwitcher(object): ...@@ -115,16 +115,17 @@ class HandlerSwitcher(object):
def _handle(self, connection, packet): # NOTE incoming packet -> handle -> dispatch ... def _handle(self, connection, packet): # NOTE incoming packet -> handle -> dispatch ...
assert len(self._pending) == 1 or self._pending[0][0] assert len(self._pending) == 1 or self._pending[0][0]
logging.packet(connection, packet, False) 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', logging.debug('Ignoring packet %r on closed connection %r',
packet, connection) packet, connection)
return return
msg_id = packet.getId() if not packet.isResponse(): # notification
(request_dict, handler) = self._pending[0] # XXX: If there are several handlers, which one to use ?
# notifications are not expected self._pending[0][1].packetReceived(connection, packet)
if not packet.isResponse():
handler.packetReceived(connection, packet)
return return
msg_id = packet.getId()
request_dict, handler = self._pending[0]
# checkout the expected answer class # checkout the expected answer class
try: try:
klass, _, _, kw = request_dict.pop(msg_id) klass, _, _, kw = request_dict.pop(msg_id)
...@@ -137,7 +138,7 @@ class HandlerSwitcher(object): ...@@ -137,7 +138,7 @@ class HandlerSwitcher(object):
logging.error('Unexpected answer %r in %r', packet, connection) logging.error('Unexpected answer %r in %r', packet, connection)
if not connection.isClosed(): if not connection.isClosed():
notification = Packets.Notify('Unexpected answer: %r' % packet) notification = Packets.Notify('Unexpected answer: %r' % packet)
connection.notify(notification) connection.send(notification)
connection.abort() connection.abort()
# handler.peerBroken(connection) # handler.peerBroken(connection)
# apply a pending handler if no more answers are pending # apply a pending handler if no more answers are pending
...@@ -354,7 +355,7 @@ class Connection(BaseConnection): ...@@ -354,7 +355,7 @@ class Connection(BaseConnection):
r.append(('len(queue)', len(self._queue))) r.append(('len(queue)', len(self._queue)))
if self._on_close is not None: if self._on_close is not None:
r.append(('on_close', getattr(self._on_close, '__name__', '?'))) 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)) if getattr(self, x))
return r, flags return r, flags
...@@ -383,7 +384,7 @@ class Connection(BaseConnection): ...@@ -383,7 +384,7 @@ class Connection(BaseConnection):
if self.server: if self.server:
del self.idle del self.idle
self.client = False self.client = False
self.notify(Packets.CloseClient()) self.send(Packets.CloseClient())
else: else:
self.close() self.close()
...@@ -449,6 +450,7 @@ class Connection(BaseConnection): ...@@ -449,6 +450,7 @@ class Connection(BaseConnection):
return return
logging.debug('aborting a connector for %r', self) logging.debug('aborting a connector for %r', self)
self.aborted = True self.aborted = True
self.read_buf = dummy_read_buffer
if self._on_close is not None: if self._on_close is not None:
self._on_close() self._on_close()
self._on_close = None self._on_close = None
...@@ -486,8 +488,6 @@ class Connection(BaseConnection): ...@@ -486,8 +488,6 @@ class Connection(BaseConnection):
except PacketMalformedError, e: except PacketMalformedError, e:
logging.error('malformed packet from %r: %s', self, e) logging.error('malformed packet from %r: %s', self, e)
self._closure() self._closure()
if self.aborted:
self.em.removeReader(self)
return not not self._queue return not not self._queue
def hasPendingMessages(self): def hasPendingMessages(self):
...@@ -552,14 +552,12 @@ class Connection(BaseConnection): ...@@ -552,14 +552,12 @@ class Connection(BaseConnection):
self.em.addWriter(self) self.em.addWriter(self)
logging.packet(self, packet, True) logging.packet(self, packet, True)
def notify(self, packet): def send(self, packet, msg_id=None):
""" Then a packet with a new ID """ """ Then a packet with a new ID """
if self.isClosed(): if self.isClosed():
raise ConnectionClosed raise ConnectionClosed
msg_id = self._getNextId() packet.setId(self._getNextId() if msg_id is None else msg_id)
packet.setId(msg_id)
self._addPacket(packet) self._addPacket(packet)
return msg_id
def ask(self, packet, timeout=CRITICAL_TIMEOUT, on_timeout=None, **kw): def ask(self, packet, timeout=CRITICAL_TIMEOUT, on_timeout=None, **kw):
""" """
...@@ -580,14 +578,14 @@ class Connection(BaseConnection): ...@@ -580,14 +578,14 @@ class Connection(BaseConnection):
self.em.wakeup() self.em.wakeup()
return msg_id 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 """ """ Answer to a packet by re-using its ID for the packet answer """
assert packet.isResponse(), packet
if self.isClosed(): if self.isClosed():
if packet.ignoreOnClosedConnection() and not packet.isError():
raise ConnectionClosed raise ConnectionClosed
if msg_id is None: return
msg_id = self.getPeerId() packet.setId(self.peer_id)
packet.setId(msg_id)
assert packet.isResponse(), packet
self._addPacket(packet) self._addPacket(packet)
def idle(self): def idle(self):
...@@ -678,7 +676,7 @@ class MTConnectionType(type): ...@@ -678,7 +676,7 @@ class MTConnectionType(type):
if __debug__: if __debug__:
for name in 'answer',: for name in 'answer',:
setattr(cls, name, cls.lockCheckWrapper(name)) setattr(cls, name, cls.lockCheckWrapper(name))
for name in 'close', 'notify': for name in 'close', 'send':
setattr(cls, name, cls.__class__.lockWrapper(cls, name)) setattr(cls, name, cls.__class__.lockWrapper(cls, name))
for name in ('_delayedConnect', 'onTimeout', for name in ('_delayedConnect', 'onTimeout',
'process', 'readable', 'writable'): 'process', 'readable', 'writable'):
...@@ -725,29 +723,17 @@ class MTClientConnection(ClientConnection): ...@@ -725,29 +723,17 @@ class MTClientConnection(ClientConnection):
with lock: with lock:
super(MTClientConnection, self).__init__(*args, **kwargs) 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, def ask(self, packet, timeout=CRITICAL_TIMEOUT, on_timeout=None,
queue=None, **kw): queue=None, **kw):
with self.lock: 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 queue is None:
if type(packet) is not Packets.Ping: if type(packet) is Packets.Ping:
raise TypeError, 'Only Ping packet can be asked ' \ return self._ask(packet, timeout, on_timeout, **kw)
'without a queue, got a %r.' % (packet, ) raise TypeError('Only Ping packet can be asked'
else: ' without a queue, got a %r.' % packet)
msg_id = self._ask(packet, timeout, on_timeout, **kw)
self.dispatcher.register(self, msg_id, queue) 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 return msg_id
...@@ -19,17 +19,15 @@ from .locking import Lock, Empty ...@@ -19,17 +19,15 @@ from .locking import Lock, Empty
EMPTY = {} EMPTY = {}
NOBODY = [] NOBODY = []
class ForgottenPacket(object): @apply
""" class _ConnectionClosed(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): handler_method_name = 'connectionClosed'
return self.msg_id decode = tuple
class getId(object):
def __eq__(self, other):
return True
def giant_lock(func): def giant_lock(func):
def wrapped(self, *args, **kw): def wrapped(self, *args, **kw):
...@@ -88,7 +86,7 @@ class Dispatcher: ...@@ -88,7 +86,7 @@ class Dispatcher:
def unregister(self, conn): def unregister(self, conn):
""" Unregister a connection and put fake packet in queues to unlock """ 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() self.lock_acquire()
try: try:
message_table = self.message_table.pop(id(conn), EMPTY) message_table = self.message_table.pop(id(conn), EMPTY)
...@@ -101,25 +99,10 @@ class Dispatcher: ...@@ -101,25 +99,10 @@ class Dispatcher:
continue continue
queue_id = id(queue) queue_id = id(queue)
if queue_id not in notified_set: if queue_id not in notified_set:
queue.put((conn, None, None)) queue.put((conn, _ConnectionClosed, EMPTY))
notified_set.add(queue_id) notified_set.add(queue_id)
_decrefQueue(queue) _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 @giant_lock
def forget_queue(self, queue, flush_queue=True): def forget_queue(self, queue, flush_queue=True):
""" """
...@@ -137,9 +120,7 @@ class Dispatcher: ...@@ -137,9 +120,7 @@ class Dispatcher:
found += 1 found += 1
message_table[msg_id] = NOBODY message_table[msg_id] = NOBODY
refcount = self.queue_dict.pop(id(queue), 0) refcount = self.queue_dict.pop(id(queue), 0)
if refcount != found: assert refcount == found, (refcount, found)
raise ValueError('We hit a refcount bug: %s queue uses ' \
'expected, %s found' % (refcount, found))
if flush_queue: if flush_queue:
get = queue.get get = queue.get
while True: while True:
......
...@@ -135,6 +135,11 @@ class EpollEventManager(object): ...@@ -135,6 +135,11 @@ class EpollEventManager(object):
self._closeAppend(connector.shutdown()) self._closeAppend(connector.shutdown())
if self._closeAcquire(0): if self._closeAcquire(0):
self._closeRelease() self._closeRelease()
return
if close:
# 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 def isIdle(self): # NOTE
return not (self._pending_processing or self.writer_set) return not (self._pending_processing or self.writer_set)
......
...@@ -15,14 +15,21 @@ ...@@ -15,14 +15,21 @@
# along with this program. If not, see <http://www.gnu.org/licenses/>. # along with this program. If not, see <http://www.gnu.org/licenses/>.
import sys import sys
from collections import deque
from operator import itemgetter
from . import logging from . import logging
from .connection import ConnectionClosed
from .protocol import ( from .protocol import (
NodeStates, Packets, Errors, BackendNotImplemented, NodeStates, Packets, Errors, BackendNotImplemented,
BrokenNodeDisallowedError, NotReadyError, PacketMalformedError, BrokenNodeDisallowedError, NonReadableCell, NotReadyError,
ProtocolError, UnexpectedPacketError) PacketMalformedError, ProtocolError, UnexpectedPacketError)
from .util import cached_property from .util import cached_property
class DelayEvent(Exception):
pass
class EventHandler(object): class EventHandler(object):
"""This class handles events.""" """This class handles events."""
...@@ -64,6 +71,9 @@ class EventHandler(object): ...@@ -64,6 +71,9 @@ class EventHandler(object):
raise UnexpectedPacketError('no handler found') raise UnexpectedPacketError('no handler found')
args = packet.decode() or () args = packet.decode() or ()
method(conn, *args, **kw) method(conn, *args, **kw)
except DelayEvent, e:
assert not kw, kw
self.getEventQueue().queueEvent(method, conn, args, *e.args)
except UnexpectedPacketError, e: except UnexpectedPacketError, e:
if not conn.isClosed(): if not conn.isClosed():
self.__unexpectedPacket(conn, packet, *e.args) self.__unexpectedPacket(conn, packet, *e.args)
...@@ -92,6 +102,8 @@ class EventHandler(object): ...@@ -92,6 +102,8 @@ class EventHandler(object):
conn.answer(Errors.BackendNotImplemented( conn.answer(Errors.BackendNotImplemented(
"%s.%s does not implement %s" "%s.%s does not implement %s"
% (m.im_class.__module__, m.im_class.__name__, m.__name__))) % (m.im_class.__module__, m.im_class.__name__, m.__name__)))
except NonReadableCell, e:
conn.answer(Errors.NonReadableCell())
except AssertionError: except AssertionError:
e = sys.exc_info() e = sys.exc_info()
try: try:
...@@ -165,9 +177,9 @@ class EventHandler(object): ...@@ -165,9 +177,9 @@ class EventHandler(object):
return return
conn.close() conn.close()
def notifyNodeInformation(self, conn, node_list): def notifyNodeInformation(self, conn, *args):
app = self.app app = self.app
app.nm.update(app, node_list) app.nm.update(app, *args)
def ping(self, conn): def ping(self, conn):
conn.answer(Packets.Pong()) conn.answer(Packets.Pong())
...@@ -207,9 +219,6 @@ class EventHandler(object): ...@@ -207,9 +219,6 @@ class EventHandler(object):
def brokenNodeDisallowedError(self, conn, message): def brokenNodeDisallowedError(self, conn, message):
raise RuntimeError, 'broken node disallowed error: %s' % (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): def ack(self, conn, message):
logging.debug("no error message: %s", message) logging.debug("no error message: %s", message)
...@@ -235,7 +244,7 @@ class MTEventHandler(EventHandler): ...@@ -235,7 +244,7 @@ class MTEventHandler(EventHandler):
self.dispatch(conn, packet, kw) self.dispatch(conn, packet, kw)
kw = {} kw = {}
if not (self.dispatcher.dispatch(conn, packet.getId(), packet, 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' raise ProtocolError('Unexpected response packet from %r: %r'
% (conn, packet)) % (conn, packet))
else: else:
...@@ -264,3 +273,104 @@ class AnswerBaseHandler(EventHandler): ...@@ -264,3 +273,104 @@ class AnswerBaseHandler(EventHandler):
def acceptIdentification(*args): def acceptIdentification(*args):
pass 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 @@ ...@@ -14,13 +14,6 @@
# You should have received a copy of the GNU General Public License # You should have received a copy of the GNU General Public License
# along with this program. If not, see <http://www.gnu.org/licenses/>. # 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 collections import deque
from functools import wraps from functools import wraps
from logging import getLogger, Formatter, Logger, StreamHandler, \ from logging import getLogger, Formatter, Logger, StreamHandler, \
...@@ -215,6 +208,13 @@ class NEOLogger(Logger): ...@@ -215,6 +208,13 @@ class NEOLogger(Logger):
self._setup(filename, reset) self._setup(filename, reset)
__del__ = setup __del__ = setup
def fork(self):
with self:
pid = os.fork()
if pid:
return pid
self._setup()
def isEnabledFor(self, level): def isEnabledFor(self, level):
return True return True
...@@ -246,7 +246,7 @@ class NEOLogger(Logger): ...@@ -246,7 +246,7 @@ class NEOLogger(Logger):
self._emit(record) self._emit(record)
self.commit() self.commit()
else: else:
self._record_size += RECORD_SIZE + len(record.msg) self._record_size += RECORD_SIZE + len(record.msg or '')
q = self._record_queue q = self._record_queue
q.append(record) q.append(record)
if record.levelno < WARNING: if record.levelno < WARNING:
......
...@@ -19,8 +19,9 @@ from os.path import exists, getsize ...@@ -19,8 +19,9 @@ from os.path import exists, getsize
import json import json
from . import attributeTracker, logging from . import attributeTracker, logging
from .handler import DelayEvent, EventQueue
from .protocol import formatNodeList, uuid_str, \ from .protocol import formatNodeList, uuid_str, \
NodeTypes, NodeStates, ProtocolError NodeTypes, NodeStates, NotReadyError, ProtocolError
class Node(object): class Node(object):
...@@ -39,17 +40,20 @@ class Node(object): ...@@ -39,17 +40,20 @@ class Node(object):
self._last_state_change = time() self._last_state_change = time()
manager.add(self) manager.add(self)
def notify(self, packet): @property
def send(self):
assert self.isConnected(), 'Not connected' 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' 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' assert self.isConnected(), 'Not connected'
self._connection.answer(packet, msg_id) return self._connection.answer
def getLastStateChange(self): def getLastStateChange(self):
return self._last_state_change return self._last_state_change
...@@ -232,7 +236,7 @@ class MasterDB(object): ...@@ -232,7 +236,7 @@ class MasterDB(object):
def __iter__(self): def __iter__(self):
return iter(self._set) return iter(self._set)
class NodeManager(object): class NodeManager(EventQueue):
"""This class manages node status.""" """This class manages node status."""
_master_db = None _master_db = None
...@@ -255,9 +259,14 @@ class NodeManager(object): ...@@ -255,9 +259,14 @@ class NodeManager(object):
self._master_db = db = MasterDB(master_db) self._master_db = db = MasterDB(master_db)
for addr in db: for addr in db:
self.createMaster(address=addr) self.createMaster(address=addr)
self.reset()
close = __init__ close = __init__
def reset(self):
EventQueue.__init__(self)
self._timestamp = 0
def add(self, node): def add(self, node):
if node in self._node_set: if node in self._node_set:
logging.warning('adding a known node %r, ignoring', node) logging.warning('adding a known node %r, ignoring', node)
...@@ -350,9 +359,22 @@ class NodeManager(object): ...@@ -350,9 +359,22 @@ class NodeManager(object):
return self._address_dict.get(address, None) return self._address_dict.get(address, None)
def getByUUID(self, uuid, *id_timestamp): 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) node = self._uuid_dict.get(uuid)
if not id_timestamp or node and (node.id_timestamp,) == id_timestamp: 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 return node
def _createNode(self, klass, address=None, uuid=None, **kw): def _createNode(self, klass, address=None, uuid=None, **kw):
...@@ -389,7 +411,9 @@ class NodeManager(object): ...@@ -389,7 +411,9 @@ class NodeManager(object):
def createFromNodeType(self, node_type, **kw): def createFromNodeType(self, node_type, **kw):
return self._createNode(NODE_TYPE_MAPPING[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 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: for node_type, addr, uuid, state, id_timestamp in node_list:
# This should be done here (although klass might not be used in this # This should be done here (although klass might not be used in this
...@@ -427,6 +451,8 @@ class NodeManager(object): ...@@ -427,6 +451,8 @@ class NodeManager(object):
# reconnect to the master because they cleared their # reconnect to the master because they cleared their
# partition table upon disconnection. # partition table upon disconnection.
node.getConnection().close() node.getConnection().close()
if app.uuid != uuid:
app.pt.dropNode(node)
self.remove(node) self.remove(node)
continue continue
logging.debug('updating node %r to %s %s %s %s %s', logging.debug('updating node %r to %s %s %s %s %s',
...@@ -441,14 +467,17 @@ class NodeManager(object): ...@@ -441,14 +467,17 @@ class NodeManager(object):
# For the first notification, we receive a full list of nodes from # For the first notification, we receive a full list of nodes from
# the master. Remove all unknown nodes from a previous connection. # the master. Remove all unknown nodes from a previous connection.
for node in node_set - self._node_set: for node in node_set - self._node_set:
app.pt.dropNode(node)
self.remove(node) self.remove(node)
self.log() self.log()
self.executeQueuedEvents()
def log(self): def log(self):
logging.info('Node manager : %u nodes', len(self._node_set)) logging.info('Node manager : %u nodes', len(self._node_set))
if self._node_set: if self._node_set:
logging.info('\n'.join(formatNodeList( logging.info('\n'.join(formatNodeList(
map(Node.asTuple, self._node_set), ' * '))) map(Node.asTuple, self._node_set), ' * ')))
self.logQueuedEvents()
# node_type -> node_klass # node_type -> node_klass
@apply @apply
......
...@@ -20,7 +20,7 @@ import traceback ...@@ -20,7 +20,7 @@ import traceback
from cStringIO import StringIO from cStringIO import StringIO
from struct import Struct from struct import Struct
PROTOCOL_VERSION = 9 PROTOCOL_VERSION = 12
# Size restrictions. # Size restrictions.
MIN_PACKET_SIZE = 10 MIN_PACKET_SIZE = 10
...@@ -71,11 +71,12 @@ def ErrorCodes(): ...@@ -71,11 +71,12 @@ def ErrorCodes():
OID_DOES_NOT_EXIST OID_DOES_NOT_EXIST
PROTOCOL_ERROR PROTOCOL_ERROR
BROKEN_NODE BROKEN_NODE
ALREADY_PENDING
REPLICATION_ERROR REPLICATION_ERROR
CHECKING_ERROR CHECKING_ERROR
BACKEND_NOT_IMPLEMENTED BACKEND_NOT_IMPLEMENTED
NON_READABLE_CELL
READ_ONLY_ACCESS READ_ONLY_ACCESS
INCOMPLETE_TRANSACTION
@Enum @Enum
def ClusterStates(): def ClusterStates():
...@@ -146,12 +147,6 @@ def CellStates(): ...@@ -146,12 +147,6 @@ def CellStates():
# readable nor writable. # readable nor writable.
CORRUPTED CORRUPTED
@Enum
def LockState():
NOT_LOCKED
GRANTED
GRANTED_TO_OTHER
# used for logging # used for logging
node_state_prefix_dict = { node_state_prefix_dict = {
NodeStates.RUNNING: 'R', NodeStates.RUNNING: 'R',
...@@ -223,6 +218,19 @@ class BrokenNodeDisallowedError(ProtocolError): ...@@ -223,6 +218,19 @@ class BrokenNodeDisallowedError(ProtocolError):
class BackendNotImplemented(Exception): class BackendNotImplemented(Exception):
""" Method not implemented by backend storage """ """ 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): class Packet(object):
""" """
Base class for any packet definition. The _fmt class attribute must be Base class for any packet definition. The _fmt class attribute must be
...@@ -405,6 +413,19 @@ class PStructItemOrNone(PStructItem): ...@@ -405,6 +413,19 @@ class PStructItemOrNone(PStructItem):
value = reader(self.size) value = reader(self.size)
return None if value == self._None else self.unpack(value)[0] 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): class PList(PStructItem):
""" """
A list of homogeneous items A list of homogeneous items
...@@ -817,6 +838,12 @@ class UnfinishedTransactions(Packet): ...@@ -817,6 +838,12 @@ class UnfinishedTransactions(Packet):
Ask unfinished transactions S -> PM. Ask unfinished transactions S -> PM.
Answer unfinished transactions PM -> S. Answer unfinished transactions PM -> S.
""" """
_fmt = PStruct('ask_unfinished_transactions',
PList('row_list',
PNumber('offset'),
),
)
_answer = PStruct('answer_unfinished_transactions', _answer = PStruct('answer_unfinished_transactions',
PTID('max_tid'), PTID('max_tid'),
PList('tid_list', PList('tid_list',
...@@ -870,6 +897,18 @@ class BeginTransaction(Packet): ...@@ -870,6 +897,18 @@ class BeginTransaction(Packet):
PTID('tid'), 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): class FinishTransaction(Packet):
""" """
Finish a transaction. C -> PM. Finish a transaction. C -> PM.
...@@ -944,14 +983,60 @@ class GenerateOIDs(Packet): ...@@ -944,14 +983,60 @@ class GenerateOIDs(Packet):
PFOidList, 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): class StoreObject(Packet):
""" """
Ask to store an object. Send an OID, an original serial, a current Ask to store an object. Send an OID, an original serial, a current
transaction ID, and data. C -> S. transaction ID, and data. C -> S.
Answer if an object has been stored. If an object is in conflict, As for IStorage, 'serial' is ZERO_TID for new objects.
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.
""" """
_fmt = PStruct('ask_store_object', _fmt = PStruct('ask_store_object',
POID('oid'), POID('oid'),
...@@ -961,21 +1046,19 @@ class StoreObject(Packet): ...@@ -961,21 +1046,19 @@ class StoreObject(Packet):
PString('data'), PString('data'),
PTID('data_serial'), PTID('data_serial'),
PTID('tid'), PTID('tid'),
PBoolean('unlock'),
) )
_answer = PStruct('answer_store_object', _answer = PStruct('answer_store_object',
PBoolean('conflicting'), PTID('conflict'),
POID('oid'),
PTID('serial'),
) )
class AbortTransaction(Packet): class AbortTransaction(Packet):
""" """
Abort a transaction. C -> S, PM. Abort a transaction. C -> S and C -> PM -> S.
""" """
_fmt = PStruct('abort_transaction', _fmt = PStruct('abort_transaction',
PTID('tid'), PTID('tid'),
PFUUIDList, # unused for * -> S
) )
class StoreTransaction(Packet): class StoreTransaction(Packet):
...@@ -1159,6 +1242,7 @@ class NotifyNodeInformation(Packet): ...@@ -1159,6 +1242,7 @@ class NotifyNodeInformation(Packet):
Notify information about one or more nodes. PM -> Any. Notify information about one or more nodes. PM -> Any.
""" """
_fmt = PStruct('notify_node_informations', _fmt = PStruct('notify_node_informations',
PFloat('id_timestamp'),
PFNodeList, PFNodeList,
) )
...@@ -1244,22 +1328,6 @@ class ObjectUndoSerial(Packet): ...@@ -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): class CheckCurrentSerial(Packet):
""" """
Verifies if given serial is current for object oid in the database, and Verifies if given serial is current for object oid in the database, and
...@@ -1271,16 +1339,12 @@ class CheckCurrentSerial(Packet): ...@@ -1271,16 +1339,12 @@ class CheckCurrentSerial(Packet):
""" """
_fmt = PStruct('ask_check_current_serial', _fmt = PStruct('ask_check_current_serial',
PTID('tid'), PTID('tid'),
PTID('serial'),
POID('oid'),
)
_answer = PStruct('answer_store_object',
PBoolean('conflicting'),
POID('oid'), POID('oid'),
PTID('serial'), PTID('serial'),
) )
_answer = StoreObject._answer
class Pack(Packet): class Pack(Packet):
""" """
Request a pack at given TID. Request a pack at given TID.
...@@ -1662,6 +1726,8 @@ class Packets(dict): ...@@ -1662,6 +1726,8 @@ class Packets(dict):
ValidateTransaction) ValidateTransaction)
AskBeginTransaction, AnswerBeginTransaction = register( AskBeginTransaction, AnswerBeginTransaction = register(
BeginTransaction) BeginTransaction)
FailedVote = register(
FailedVote)
AskFinishTransaction, AnswerTransactionFinished = register( AskFinishTransaction, AnswerTransactionFinished = register(
FinishTransaction, ignore_when_closed=False) FinishTransaction, ignore_when_closed=False)
AskLockInformation, AnswerInformationLocked = register( AskLockInformation, AnswerInformationLocked = register(
...@@ -1672,6 +1738,12 @@ class Packets(dict): ...@@ -1672,6 +1738,12 @@ class Packets(dict):
UnlockInformation) UnlockInformation)
AskNewOIDs, AnswerNewOIDs = register( AskNewOIDs, AnswerNewOIDs = register(
GenerateOIDs) GenerateOIDs)
NotifyDeadlock = register(
Deadlock)
AskRebaseTransaction, AnswerRebaseTransaction = register(
RebaseTransaction)
AskRebaseObject, AnswerRebaseObject = register(
RebaseObject)
AskStoreObject, AnswerStoreObject = register( AskStoreObject, AnswerStoreObject = register(
StoreObject) StoreObject)
AbortTransaction = register( AbortTransaction = register(
...@@ -1710,8 +1782,6 @@ class Packets(dict): ...@@ -1710,8 +1782,6 @@ class Packets(dict):
ClusterState) ClusterState)
AskObjectUndoSerial, AnswerObjectUndoSerial = register( AskObjectUndoSerial, AnswerObjectUndoSerial = register(
ObjectUndoSerial) ObjectUndoSerial)
AskHasLock, AnswerHasLock = register(
HasLock)
AskTIDsFrom, AnswerTIDsFrom = register( AskTIDsFrom, AnswerTIDsFrom = register(
TIDListFrom) TIDListFrom)
AskPack, AnswerPack = register( AskPack, AnswerPack = register(
...@@ -1781,3 +1851,8 @@ def formatNodeList(node_list, prefix='', _sort_key=itemgetter(2)): ...@@ -1781,3 +1851,8 @@ def formatNodeList(node_list, prefix='', _sort_key=itemgetter(2)):
for i in xrange(len(node_list[0]) - 1)) for i in xrange(len(node_list[0]) - 1))
return map((prefix + t + '%s').__mod__, node_list) return map((prefix + t + '%s').__mod__, node_list)
return () return ()
NotifyNodeInformation._neolog = staticmethod(lambda timestamp, node_list:
((timestamp,), formatNodeList(node_list, ' ! ')))
Error._neolog = staticmethod(lambda *args: ((), ("%s (%s)" % args,)))
...@@ -15,12 +15,10 @@ ...@@ -15,12 +15,10 @@
# along with this program. If not, see <http://www.gnu.org/licenses/>. # along with this program. If not, see <http://www.gnu.org/licenses/>.
import math import math
from functools import wraps
from . import logging, protocol from . import logging, protocol
from .locking import Lock
from .protocol import uuid_str, CellStates from .protocol import uuid_str, CellStates
from .util import u64 from .util import u64
from .locking import RLock
class PartitionTableException(Exception): class PartitionTableException(Exception):
""" """
...@@ -81,6 +79,10 @@ class Cell(object): ...@@ -81,6 +79,10 @@ class Cell(object):
class PartitionTable(object): class PartitionTable(object):
"""This class manages a partition table.""" """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): def __init__(self, num_partitions, num_replicas):
self._id = None self._id = None
self.np = num_partitions self.np = num_partitions
...@@ -163,7 +165,7 @@ class PartitionTable(object): ...@@ -163,7 +165,7 @@ class PartitionTable(object):
if cell.getUUID() == uuid: if cell.getUUID() == uuid:
return cell return cell
def setCell(self, offset, node, state): def _setCell(self, offset, node, state):
if state == CellStates.DISCARDED: if state == CellStates.DISCARDED:
return self.removeCell(offset, node) return self.removeCell(offset, node)
if node.isBroken() or node.isDown(): if node.isBroken() or node.isDown():
...@@ -182,7 +184,6 @@ class PartitionTable(object): ...@@ -182,7 +184,6 @@ class PartitionTable(object):
row.append(Cell(node, state)) row.append(Cell(node, state))
if state != CellStates.FEEDING: if state != CellStates.FEEDING:
self.count_dict[node] += 1 self.count_dict[node] += 1
return offset, node.getUUID(), state
def removeCell(self, offset, node): def removeCell(self, offset, node):
row = self.partition_list[offset] row = self.partition_list[offset]
...@@ -193,7 +194,10 @@ class PartitionTable(object): ...@@ -193,7 +194,10 @@ class PartitionTable(object):
if not cell.isFeeding(): if not cell.isFeeding():
self.count_dict[node] -= 1 self.count_dict[node] -= 1
break 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): def load(self, ptid, row_list, nm):
""" """
...@@ -209,30 +213,38 @@ class PartitionTable(object): ...@@ -209,30 +213,38 @@ class PartitionTable(object):
node = nm.getByUUID(uuid) node = nm.getByUUID(uuid)
# the node must be known by the node manager # the node must be known by the node manager
assert node is not None assert node is not None
self.setCell(offset, node, state) self._setCell(offset, node, state)
logging.debug('partition table loaded (ptid=%s)', ptid) logging.debug('partition table loaded (ptid=%s)', ptid)
self.log() self.log()
def update(self, ptid, cell_list, nm): def update(self, ptid, cell_list, nm):
""" """
Update the partition with the cell list supplied. Ignore those changes Update the partition with the cell list supplied. If a node
if the partition table ID is not greater than the current one. If a node
is not known, it is created in the node manager and set as unavailable is not known, it is created in the node manager and set as unavailable
""" """
if ptid <= self._id: assert self._id < ptid, (self._id, ptid)
logging.warning('ignoring older partition changes')
return
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: for offset, uuid, state in cell_list:
node = nm.getByUUID(uuid) node = nm.getByUUID(uuid)
assert node is not None, 'No node found for uuid ' + uuid_str(uuid) assert node is not None, 'No node found for uuid ' + uuid_str(uuid)
self.setCell(offset, node, state) self._setCell(offset, node, state)
logging.debug('partition table updated (ptid=%s)', ptid) self.logUpdated()
self.log() if not all(cell.isReadable() for cell in readable_list):
logging.warning(self._first_outdated_message)
def filled(self): def filled(self):
return self.num_filled_rows == self.np return self.num_filled_rows == self.np
def logUpdated(self):
logging.debug('partition table updated (ptid=%s)', self._id)
self.log()
def log(self): def log(self):
logging.debug(self.format()) logging.debug(self.format())
...@@ -258,15 +270,16 @@ class PartitionTable(object): ...@@ -258,15 +270,16 @@ class PartitionTable(object):
partition on the line (here, line length is 11 to keep the docstring partition on the line (here, line length is 11 to keep the docstring
width under 80 column). width under 80 column).
""" """
node_list = sorted(self.count_dict)
result = ['pt: node %u: %s, %s' % (i, uuid_str(node.getUUID()), result = ['pt: node %u: %s, %s' % (i, uuid_str(node.getUUID()),
protocol.node_state_prefix_dict[node.getState()]) 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 append = result.append
line = [] line = []
max_line_len = 20 # XXX: hardcoded number of partitions per line max_line_len = 20 # XXX: hardcoded number of partitions per line
prefix = 0 prefix = 0
prefix_len = int(math.ceil(math.log10(self.np))) 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: if len(line) == max_line_len:
append('pt: %0*u: %s' % (prefix_len, prefix, '|'.join(line))) append('pt: %0*u: %s' % (prefix_len, prefix, '|'.join(line)))
line = [] line = []
...@@ -276,8 +289,7 @@ class PartitionTable(object): ...@@ -276,8 +289,7 @@ class PartitionTable(object):
append('pt: %0*u: %s' % (prefix_len, prefix, '|'.join(line))) append('pt: %0*u: %s' % (prefix_len, prefix, '|'.join(line)))
return result return result
def formatRows(self): def _formatRows(self, node_list):
node_list = sorted(self.count_dict)
cell_state_dict = protocol.cell_state_prefix_dict cell_state_dict = protocol.cell_state_prefix_dict
for row in self.partition_list: for row in self.partition_list:
if row is None: if row is None:
...@@ -287,12 +299,14 @@ class PartitionTable(object): ...@@ -287,12 +299,14 @@ class PartitionTable(object):
for x in row} for x in row}
yield ''.join(cell_dict.get(x, '.') for x in node_list) yield ''.join(cell_dict.get(x, '.') for x in node_list)
def operational(self): def operational(self, exclude_list=()):
if not self.filled(): if not self.filled():
return False return False
for row in self.partition_list: for row in self.partition_list:
for cell in row: for cell in row:
if cell.isReadable() and cell.getNode().isRunning(): if cell.isReadable():
node = cell.getNode()
if node.isRunning() and node.getUUID() not in exclude_list:
break break
else: else:
return False return False
...@@ -308,38 +322,22 @@ class PartitionTable(object): ...@@ -308,38 +322,22 @@ class PartitionTable(object):
getRow = self.getRow getRow = self.getRow
return [(x, getRow(x)) for x in xrange(self.np)] 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): class MTPartitionTable(PartitionTable):
""" Thread-safe aware version of the partition table, override only methods """ Thread-safe aware version of the partition table, override only methods
used in the client """ used in the client """
def __init__(self, *args, **kwargs): def __init__(self, *args, **kw):
self._lock = RLock() self._lock = Lock()
PartitionTable.__init__(self, *args, **kwargs) PartitionTable.__init__(self, *args, **kw)
def lock(self):
self._lock.acquire()
def unlock(self):
self._lock.release()
@thread_safe def update(self, *args, **kw):
def setCell(self, *args, **kwargs): with self._lock:
return PartitionTable.setCell(self, *args, **kwargs) return PartitionTable.update(self, *args, **kw)
@thread_safe def clear(self, *args, **kw):
def clear(self, *args, **kwargs): with self._lock:
return PartitionTable.clear(self, *args, **kwargs) return PartitionTable.clear(self, *args, **kw)
@thread_safe def operational(self, *args, **kw):
def operational(self, *args, **kwargs): with self._lock:
return PartitionTable.operational(self, *args, **kwargs) return PartitionTable.operational(self, *args, **kw)
...@@ -17,9 +17,8 @@ ...@@ -17,9 +17,8 @@
import thread, threading, weakref import thread, threading, weakref
from . import logging from . import logging
from .app import BaseApplication from .app import BaseApplication
from .connection import ConnectionClosed
from .debug import register as registerLiveDebugger from .debug import register as registerLiveDebugger
from .dispatcher import Dispatcher, ForgottenPacket from .dispatcher import Dispatcher
from .locking import SimpleQueue from .locking import SimpleQueue
class app_set(weakref.WeakSet): class app_set(weakref.WeakSet):
...@@ -141,17 +140,8 @@ class ThreadedApplication(BaseApplication): ...@@ -141,17 +140,8 @@ class ThreadedApplication(BaseApplication):
_handlePacket = self._handlePacket _handlePacket = self._handlePacket
while True: while True:
qconn, qpacket, kw = get(True) qconn, qpacket, kw = get(True)
is_forgotten = isinstance(qpacket, ForgottenPacket) if conn is qconn and msg_id == qpacket.getId(): # NOTE selector on msg_id
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) _handlePacket(qconn, qpacket, kw, handler)
break break
if not is_forgotten and qpacket is not None:
_handlePacket(qconn, qpacket, kw) _handlePacket(qconn, qpacket, kw)
return self.getHandlerData() return self.getHandlerData()
...@@ -214,6 +214,8 @@ class ReadBuffer(object): ...@@ -214,6 +214,8 @@ class ReadBuffer(object):
self.size = 0 self.size = 0
self.content.clear() self.content.clear()
dummy_read_buffer = ReadBuffer()
dummy_read_buffer.append = lambda _: None
class cached_property(object): class cached_property(object):
""" """
......
This diff is collapsed.
...@@ -190,7 +190,7 @@ class BackupApplication(object): ...@@ -190,7 +190,7 @@ class BackupApplication(object):
"ask %s to replicate partition %u up to %s from %s", "ask %s to replicate partition %u up to %s from %s",
uuid_str(cell.getUUID()), offset, dump(tid), uuid_str(cell.getUUID()), offset, dump(tid),
uuid_str(primary_node.getUUID())) uuid_str(primary_node.getUUID()))
cell.getNode().getConnection().notify(p) cell.getNode().send(p)
trigger_set.add(primary_node) trigger_set.add(primary_node)
for node in trigger_set: for node in trigger_set:
self.triggerBackup(node) self.triggerBackup(node)
...@@ -255,7 +255,7 @@ class BackupApplication(object): ...@@ -255,7 +255,7 @@ class BackupApplication(object):
cell.replicating = tid cell.replicating = tid
for node, untouched_dict in untouched_dict.iteritems(): for node, untouched_dict in untouched_dict.iteritems():
if app.isStorageReady(node.getUUID()): 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: for node in trigger_set:
self.triggerBackup(node) self.triggerBackup(node)
count = sum(map(len, self.tid_list)) count = sum(map(len, self.tid_list))
...@@ -291,8 +291,7 @@ class BackupApplication(object): ...@@ -291,8 +291,7 @@ class BackupApplication(object):
source_dict[offset] = addr source_dict[offset] = addr
logging.debug("ask %s to replicate partition %u up to %s from %r", logging.debug("ask %s to replicate partition %u up to %s from %r",
uuid_str(node.getUUID()), offset, dump(tid), addr) uuid_str(node.getUUID()), offset, dump(tid), addr)
node.getConnection().notify(Packets.Replicate( # NOTE Mb -> Sb (notify to trigger replicate up to tid) node.send(Packets.Replicate(tid, self.name, source_dict)) # NOTE Mb -> Sb (notify to trigger replicate up to tid)
tid, self.name, source_dict))
# NOTE feedback from Sb -> Mb a partition (requested by invalidatePartitions->triggerBackup) has been replicated # NOTE feedback from Sb -> Mb a partition (requested by invalidatePartitions->triggerBackup) has been replicated
def notifyReplicationDone(self, node, offset, tid): def notifyReplicationDone(self, node, offset, tid):
...@@ -310,7 +309,7 @@ class BackupApplication(object): ...@@ -310,7 +309,7 @@ class BackupApplication(object):
last_tid = app.getLastTransaction() last_tid = app.getLastTransaction()
if tid < last_tid: if tid < last_tid:
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", logging.debug("partition %u: updating backup_tid of %r to %s",
offset, cell, dump(tid)) offset, cell, dump(tid))
cell.backup_tid = tid cell.backup_tid = tid
...@@ -334,7 +333,7 @@ class BackupApplication(object): ...@@ -334,7 +333,7 @@ class BackupApplication(object):
"ask %s to replicate partition %u up to %s from %s", "ask %s to replicate partition %u up to %s from %s",
uuid_str(node.getUUID()), offset, dump(max_tid), uuid_str(node.getUUID()), offset, dump(max_tid),
uuid_str(primary_node.getUUID())) uuid_str(primary_node.getUUID()))
node.notify(Packets.Replicate(max_tid, '', node.send(Packets.Replicate(max_tid, '',
{offset: primary_node.getAddress()})) {offset: primary_node.getAddress()}))
else: else:
if app.getClusterState() == ClusterStates.BACKINGUP: if app.getClusterState() == ClusterStates.BACKINGUP:
...@@ -350,5 +349,5 @@ class BackupApplication(object): ...@@ -350,5 +349,5 @@ class BackupApplication(object):
"ask %s to replicate partition %u up to %s from %s", "ask %s to replicate partition %u up to %s from %s",
uuid_str(cell.getUUID()), offset, uuid_str(cell.getUUID()), offset,
dump(tid), uuid_str(node.getUUID())) dump(tid), uuid_str(node.getUUID()))
cell.getNode().notify(p) cell.getNode().send(p)
return result return result
...@@ -14,6 +14,7 @@ ...@@ -14,6 +14,7 @@
# You should have received a copy of the GNU General Public License # You should have received a copy of the GNU General Public License
# along with this program. If not, see <http://www.gnu.org/licenses/>. # 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 import logging
from neo.lib.exception import StoppedOperation from neo.lib.exception import StoppedOperation
from neo.lib.handler import EventHandler from neo.lib.handler import EventHandler
...@@ -98,7 +99,7 @@ class MasterHandler(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.getMasterList())
node_list.extend(n.asTuple() for n in nm.getClientList()) node_list.extend(n.asTuple() for n in nm.getClientList())
node_list.extend(n.asTuple() for n in nm.getStorageList()) 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): def askPartitionTable(self, conn):
pt = self.app.pt pt = self.app.pt
...@@ -115,7 +116,7 @@ class BaseServiceHandler(MasterHandler): ...@@ -115,7 +116,7 @@ class BaseServiceHandler(MasterHandler):
def connectionCompleted(self, conn, new): def connectionCompleted(self, conn, new):
self._notifyNodeInformation(conn) self._notifyNodeInformation(conn)
pt = self.app.pt 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): def connectionLost(self, conn, new_state):
app = self.app app = self.app
...@@ -145,7 +146,3 @@ class BaseServiceHandler(MasterHandler): ...@@ -145,7 +146,3 @@ class BaseServiceHandler(MasterHandler):
app.broadcastPartitionChanges(app.pt.outdate(node)) app.broadcastPartitionChanges(app.pt.outdate(node))
if not app.pt.operational(): if not app.pt.operational():
raise StoppedOperation raise StoppedOperation
def notifyReady(self, conn):
self.app.setStorageReady(conn.getUUID())
...@@ -17,7 +17,7 @@ ...@@ -17,7 +17,7 @@
import random import random
from . import MasterHandler from . import MasterHandler
from ..app import StateChangedException from ..app import monotonic_time, StateChangedException
from neo.lib import logging from neo.lib import logging
from neo.lib.exception import StoppedOperation from neo.lib.exception import StoppedOperation
from neo.lib.pt import PartitionTableException from neo.lib.pt import PartitionTableException
...@@ -103,7 +103,8 @@ class AdministrationHandler(MasterHandler): ...@@ -103,7 +103,8 @@ class AdministrationHandler(MasterHandler):
node.setState(state) node.setState(state)
if node.isConnected(): if node.isConnected():
# notify itself so it can shutdown # 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 # close to avoid handle the closure as a connection lost
node.getConnection().abort() node.getConnection().abort()
if keep: if keep:
...@@ -121,7 +122,8 @@ class AdministrationHandler(MasterHandler): ...@@ -121,7 +122,8 @@ class AdministrationHandler(MasterHandler):
# ignores non-running nodes # ignores non-running nodes
assert not node.isRunning() assert not node.isRunning()
if node.isConnected(): if node.isConnected():
node.notify(Packets.NotifyNodeInformation([node.asTuple()])) node.send(Packets.NotifyNodeInformation(
monotonic_time(), [node.asTuple()]))
app.broadcastNodesInformation([node]) app.broadcastNodesInformation([node])
def addPendingNodes(self, conn, uuid_list): def addPendingNodes(self, conn, uuid_list):
...@@ -139,10 +141,9 @@ class AdministrationHandler(MasterHandler): ...@@ -139,10 +141,9 @@ class AdministrationHandler(MasterHandler):
for node in app.nm.getStorageList() for node in app.nm.getStorageList()
if node.isPending() and node.getUUID() in uuid_list)) if node.isPending() and node.getUUID() in uuid_list))
if node_list: if node_list:
p = Packets.StartOperation(bool(app.backup_tid)) # NOTE ...
for node in node_list: for node in node_list:
node.setRunning() node.setRunning()
node.notify(p) app.startStorage(node)
app.broadcastNodesInformation(node_list) app.broadcastNodesInformation(node_list)
conn.answer(Errors.Ack('Nodes added: %s' % conn.answer(Errors.Ack('Nodes added: %s' %
', '.join(uuid_str(x.getUUID()) for x in node_list))) ', '.join(uuid_str(x.getUUID()) for x in node_list)))
...@@ -160,7 +161,7 @@ class AdministrationHandler(MasterHandler): ...@@ -160,7 +161,7 @@ class AdministrationHandler(MasterHandler):
node_list.append(node) node_list.append(node)
repair = Packets.NotifyRepair(*args) repair = Packets.NotifyRepair(*args)
for node in node_list: for node in node_list:
node.notify(repair) node.send(repair)
conn.answer(Errors.Ack('')) conn.answer(Errors.Ack(''))
def tweakPartitionTable(self, conn, uuid_list): def tweakPartitionTable(self, conn, uuid_list):
...@@ -224,6 +225,6 @@ class AdministrationHandler(MasterHandler): ...@@ -224,6 +225,6 @@ class AdministrationHandler(MasterHandler):
).getAddress() ).getAddress()
else: else:
source = '', None source = '', None
node.getConnection().notify(Packets.CheckPartition( node.send(Packets.CheckPartition(
offset, source, min_tid, max_tid)) offset, source, min_tid, max_tid))
conn.answer(Errors.Ack('')) conn.answer(Errors.Ack(''))
...@@ -14,7 +14,9 @@ ...@@ -14,7 +14,9 @@
# You should have received a copy of the GNU General Public License # You should have received a copy of the GNU General Public License
# along with this program. If not, see <http://www.gnu.org/licenses/>. # 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 neo.lib.protocol import NodeStates, Packets, ProtocolError, MAX_TID, Errors
from ..app import monotonic_time
from . import MasterHandler from . import MasterHandler
class ClientServiceHandler(MasterHandler): class ClientServiceHandler(MasterHandler):
...@@ -36,54 +38,55 @@ class ClientServiceHandler(MasterHandler): ...@@ -36,54 +38,55 @@ class ClientServiceHandler(MasterHandler):
node_list = [nm.getByUUID(conn.getUUID()).asTuple()] # for id_timestamp 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.getMasterList())
node_list.extend(n.asTuple() for n in nm.getStorageList()) 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): def askBeginTransaction(self, conn, tid):
""" """
A client request a TID, nothing is kept about it until the finish. A client request a TID, nothing is kept about it until the finish.
""" """
app = self.app 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()) 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): def askNewOIDs(self, conn, num_oids):
conn.answer(Packets.AnswerNewOIDs(self.app.tm.getNextOIDList(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): def askFinishTransaction(self, conn, ttid, oid_list, checked_list):
app = self.app app = self.app
pt = app.pt tid, node_list = app.tm.prepare(
app,
# 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(
ttid, ttid,
app.tm.prepare(
ttid,
pt.getPartitions(),
oid_list, oid_list,
{x.getUUID() for x in identified_node_list}, checked_list,
conn.getPeerId(), conn.getPeerId(),
),
) )
for node in identified_node_list: if tid:
p = Packets.AskLockInformation(ttid, tid)
for node in node_list:
node.ask(p, timeout=60) 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): def askFinalTID(self, conn, ttid):
tm = self.app.tm tm = self.app.tm
...@@ -112,9 +115,24 @@ class ClientServiceHandler(MasterHandler): ...@@ -112,9 +115,24 @@ class ClientServiceHandler(MasterHandler):
else: else:
conn.answer(Packets.AnswerPack(False)) conn.answer(Packets.AnswerPack(False))
def abortTransaction(self, conn, tid): def abortTransaction(self, conn, tid, uuid_list):
# BUG: The replicator may wait this transaction to be finished. # Consider a failure when the connection between the storage and the
self.app.tm.abort(tid, conn.getUUID()) # 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 # like ClientServiceHandler but read-only & only for tid <= backup_tid
......
...@@ -56,7 +56,7 @@ class BaseElectionHandler(EventHandler): ...@@ -56,7 +56,7 @@ class BaseElectionHandler(EventHandler):
class ClientElectionHandler(BaseElectionHandler): class ClientElectionHandler(BaseElectionHandler):
def notifyNodeInformation(self, conn, node_list): def notifyNodeInformation(self, conn, timestamp, node_list):
# XXX: For the moment, do nothing because # XXX: For the moment, do nothing because
# we'll close this connection and reconnect. # we'll close this connection and reconnect.
pass pass
......
...@@ -14,10 +14,10 @@ ...@@ -14,10 +14,10 @@
# You should have received a copy of the GNU General Public License # You should have received a copy of the GNU General Public License
# along with this program. If not, see <http://www.gnu.org/licenses/>. # along with this program. If not, see <http://www.gnu.org/licenses/>.
from time import time
from neo.lib import logging from neo.lib import logging
from neo.lib.protocol import ClusterStates, NodeStates, NodeTypes, \ from neo.lib.protocol import ClusterStates, NodeStates, NodeTypes, \
NotReadyError, ProtocolError, uuid_str NotReadyError, ProtocolError, uuid_str
from ..app import monotonic_time
from . import MasterHandler from . import MasterHandler
class IdentificationHandler(MasterHandler): class IdentificationHandler(MasterHandler):
...@@ -92,7 +92,7 @@ class IdentificationHandler(MasterHandler): ...@@ -92,7 +92,7 @@ class IdentificationHandler(MasterHandler):
uuid=uuid, address=address) uuid=uuid, address=address)
else: else:
node.setUUID(uuid) node.setUUID(uuid)
node.id_timestamp = time() node.id_timestamp = monotonic_time()
node.setState(state) node.setState(state)
node.setConnection(conn) node.setConnection(conn)
conn.setHandler(handler) conn.setHandler(handler)
......
...@@ -15,6 +15,7 @@ ...@@ -15,6 +15,7 @@
# along with this program. If not, see <http://www.gnu.org/licenses/>. # along with this program. If not, see <http://www.gnu.org/licenses/>.
import sys import sys
from ..app import monotonic_time
from . import MasterHandler from . import MasterHandler
from neo.lib.handler import EventHandler from neo.lib.handler import EventHandler
from neo.lib.exception import ElectionFailure, PrimaryFailure from neo.lib.exception import ElectionFailure, PrimaryFailure
...@@ -25,10 +26,11 @@ class SecondaryMasterHandler(MasterHandler): ...@@ -25,10 +26,11 @@ class SecondaryMasterHandler(MasterHandler):
""" Handler used by primary to handle secondary masters""" """ Handler used by primary to handle secondary masters"""
def connectionLost(self, conn, new_state): def connectionLost(self, conn, new_state):
node = self.app.nm.getByUUID(conn.getUUID()) app = self.app
assert node is not None if app.listening_conn: # if running
node = app.nm.getByUUID(conn.getUUID())
node.setDown() node.setDown()
self.app.broadcastNodesInformation([node]) app.broadcastNodesInformation([node])
def announcePrimary(self, conn): def announcePrimary(self, conn):
raise ElectionFailure, 'another primary arises' raise ElectionFailure, 'another primary arises'
...@@ -38,18 +40,18 @@ class SecondaryMasterHandler(MasterHandler): ...@@ -38,18 +40,18 @@ class SecondaryMasterHandler(MasterHandler):
def _notifyNodeInformation(self, conn): def _notifyNodeInformation(self, conn):
node_list = [n.asTuple() for n in self.app.nm.getMasterList()] 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): class PrimaryHandler(EventHandler):
""" Handler used by secondaries to handle primary master""" """ Handler used by secondaries to handle primary master"""
def connectionLost(self, conn, new_state): def connectionLost(self, conn, new_state):
self.app.primary_master_node.setDown() self.connectionFailed(conn)
raise PrimaryFailure, 'primary master is dead'
def connectionFailed(self, conn): def connectionFailed(self, conn):
self.app.primary_master_node.setDown() 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): def connectionCompleted(self, conn):
app = self.app app = self.app
...@@ -72,8 +74,9 @@ class PrimaryHandler(EventHandler): ...@@ -72,8 +74,9 @@ class PrimaryHandler(EventHandler):
def notifyClusterInformation(self, conn, state): def notifyClusterInformation(self, conn, state):
self.app.cluster_state = state self.app.cluster_state = state
def notifyNodeInformation(self, conn, node_list): def notifyNodeInformation(self, conn, timestamp, node_list):
super(PrimaryHandler, self).notifyNodeInformation(conn, node_list) super(PrimaryHandler, self).notifyNodeInformation(
conn, timestamp, node_list)
for node_type, _, uuid, state, _ in node_list: for node_type, _, uuid, state, _ in node_list:
assert node_type == NodeTypes.MASTER, node_type assert node_type == NodeTypes.MASTER, node_type
if uuid == self.app.uuid and state == NodeStates.UNKNOWN: if uuid == self.app.uuid and state == NodeStates.UNKNOWN:
......
...@@ -15,9 +15,11 @@ ...@@ -15,9 +15,11 @@
# along with this program. If not, see <http://www.gnu.org/licenses/>. # along with this program. If not, see <http://www.gnu.org/licenses/>.
from neo.lib import logging 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.exception import StoppedOperation
from neo.lib.pt import PartitionTableException from neo.lib.pt import PartitionTableException
from neo.lib.util import dump
from . import BaseServiceHandler from . import BaseServiceHandler
...@@ -26,18 +28,22 @@ class StorageServiceHandler(BaseServiceHandler): ...@@ -26,18 +28,22 @@ class StorageServiceHandler(BaseServiceHandler):
def connectionCompleted(self, conn, new): def connectionCompleted(self, conn, new):
app = self.app app = self.app
uuid = conn.getUUID()
app.setStorageNotReady(uuid)
if new: if new:
super(StorageServiceHandler, self).connectionCompleted(conn, new) super(StorageServiceHandler, self).connectionCompleted(conn, new)
if app.nm.getByUUID(uuid).isRunning(): # node may be PENDING node = app.nm.getByUUID(conn.getUUID())
conn.notify(Packets.StartOperation(app.backup_tid)) 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): def connectionLost(self, conn, new_state):
app = self.app app = self.app
node = app.nm.getByUUID(conn.getUUID()) uuid = conn.getUUID()
node = app.nm.getByUUID(uuid)
super(StorageServiceHandler, self).connectionLost(conn, new_state) super(StorageServiceHandler, self).connectionLost(conn, new_state)
app.tm.storageLost(conn.getUUID()) app.setStorageNotReady(uuid)
app.tm.storageLost(uuid)
if (app.getClusterState() == ClusterStates.BACKINGUP if (app.getClusterState() == ClusterStates.BACKINGUP
# Also check if we're exiting, because backup_app is not usable # Also check if we're exiting, because backup_app is not usable
# in this case. Maybe cluster state should be set to something # in this case. Maybe cluster state should be set to something
...@@ -47,7 +53,7 @@ class StorageServiceHandler(BaseServiceHandler): ...@@ -47,7 +53,7 @@ class StorageServiceHandler(BaseServiceHandler):
if app.packing is not None: if app.packing is not None:
self.answerPack(conn, False) self.answerPack(conn, False)
def askUnfinishedTransactions(self, conn): def askUnfinishedTransactions(self, conn, offset_list):
app = self.app app = self.app
if app.backup_tid: if app.backup_tid:
last_tid = app.pt.getBackupTid(min) last_tid = app.pt.getBackupTid(min)
...@@ -60,6 +66,10 @@ class StorageServiceHandler(BaseServiceHandler): ...@@ -60,6 +66,10 @@ class StorageServiceHandler(BaseServiceHandler):
pending_list = app.tm.registerForNotification(conn.getUUID()) pending_list = app.tm.registerForNotification(conn.getUUID())
p = Packets.AnswerUnfinishedTransactions(last_tid, pending_list) p = Packets.AnswerUnfinishedTransactions(last_tid, pending_list)
conn.answer(p) 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): def answerInformationLocked(self, conn, ttid):
self.app.tm.lock(ttid, conn.getUUID()) self.app.tm.lock(ttid, conn.getUUID())
...@@ -77,7 +87,8 @@ class StorageServiceHandler(BaseServiceHandler): ...@@ -77,7 +87,8 @@ class StorageServiceHandler(BaseServiceHandler):
def notifyReplicationDone(self, conn, offset, tid): def notifyReplicationDone(self, conn, offset, tid):
app = self.app app = self.app
node = app.nm.getByUUID(conn.getUUID()) uuid = conn.getUUID()
node = app.nm.getByUUID(uuid)
if app.backup_tid: if app.backup_tid:
cell_list = app.backup_app.notifyReplicationDone(node, offset, tid) cell_list = app.backup_app.notifyReplicationDone(node, offset, tid)
if not cell_list: if not cell_list:
...@@ -85,11 +96,15 @@ class StorageServiceHandler(BaseServiceHandler): ...@@ -85,11 +96,15 @@ class StorageServiceHandler(BaseServiceHandler):
else: else:
try: try:
cell_list = self.app.pt.setUpToDate(node, offset) cell_list = self.app.pt.setUpToDate(node, offset)
if not cell_list:
raise ProtocolError('Non-outdated partition')
except PartitionTableException, e: except PartitionTableException, e:
raise ProtocolError(str(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) self.app.broadcastPartitionChanges(cell_list)
def answerPack(self, conn, status): def answerPack(self, conn, status):
...@@ -100,5 +115,5 @@ class StorageServiceHandler(BaseServiceHandler): ...@@ -100,5 +115,5 @@ class StorageServiceHandler(BaseServiceHandler):
if not uid_set: if not uid_set:
app.packing = None app.packing = None
if not client.isClosed(): 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 @@ ...@@ -16,17 +16,25 @@
from collections import defaultdict from collections import defaultdict
import neo.lib.pt import neo.lib.pt
from neo.lib import logging
from neo.lib.protocol import CellStates, ZERO_TID from neo.lib.protocol import CellStates, ZERO_TID
class Cell(neo.lib.pt.Cell): class Cell(neo.lib.pt.Cell):
replicating = ZERO_TID replicating = ZERO_TID
updatable = False
def setState(self, state): def setState(self, state):
readable = self.isReadable() readable = self.isReadable()
super(Cell, self).setState(state) 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: try:
del self.backup_tid, self.replicating del self.backup_tid, self.replicating
except AttributeError: except AttributeError:
...@@ -147,7 +155,7 @@ class PartitionTable(neo.lib.pt.PartitionTable): ...@@ -147,7 +155,7 @@ class PartitionTable(neo.lib.pt.PartitionTable):
if node is None: if node is None:
node = nm.createStorage(uuid=uuid) node = nm.createStorage(uuid=uuid)
new_nodes.append(node.asTuple()) new_nodes.append(node.asTuple())
self.setCell(offset, node, state) self._setCell(offset, node, state)
return new_nodes return new_nodes
def setUpToDate(self, node, offset): def setUpToDate(self, node, offset):
...@@ -156,20 +164,22 @@ class PartitionTable(neo.lib.pt.PartitionTable): ...@@ -156,20 +164,22 @@ class PartitionTable(neo.lib.pt.PartitionTable):
# check the partition is assigned and known as outdated # check the partition is assigned and known as outdated
for cell in self.getCellList(offset): for cell in self.getCellList(offset):
if cell.getUUID() == uuid: if cell.getUUID() == uuid:
if cell.isOutOfDate(): if cell.isOutOfDate() and cell.updatable:
break break
return return
else: else:
raise neo.lib.pt.PartitionTableException('Non-assigned partition') raise neo.lib.pt.PartitionTableException('Non-assigned partition')
# update the partition table # 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. # If the partition contains a feeding cell, drop it now.
for feeding_cell in self.getCellList(offset): for feeding_cell in self.getCellList(offset):
if feeding_cell.isFeeding(): if feeding_cell.isFeeding():
cell_list.append(self.removeCell(offset, node = feeding_cell.getNode()
feeding_cell.getNode())) self.removeCell(offset, node)
cell_list.append((offset, node.getUUID(), CellStates.DISCARDED))
break break
return cell_list return cell_list
...@@ -276,6 +286,9 @@ class PartitionTable(neo.lib.pt.PartitionTable): ...@@ -276,6 +286,9 @@ class PartitionTable(neo.lib.pt.PartitionTable):
to serve. This allows a cluster restart. to serve. This allows a cluster restart.
""" """
change_list = [] 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): for offset, row in enumerate(self.partition_list):
lost = lost_node lost = lost_node
cell_list = [] cell_list = []
...@@ -290,8 +303,16 @@ class PartitionTable(neo.lib.pt.PartitionTable): ...@@ -290,8 +303,16 @@ class PartitionTable(neo.lib.pt.PartitionTable):
cell.setState(CellStates.OUT_OF_DATE) cell.setState(CellStates.OUT_OF_DATE)
change_list.append((offset, cell.getUUID(), change_list.append((offset, cell.getUUID(),
CellStates.OUT_OF_DATE)) CellStates.OUT_OF_DATE))
if fully_readable and change_list:
logging.warning(self._first_outdated_message)
return change_list 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): def iterNodeCell(self, node):
for offset, row in enumerate(self.partition_list): for offset, row in enumerate(self.partition_list):
for cell in row: for cell in row:
......
...@@ -16,6 +16,7 @@ ...@@ -16,6 +16,7 @@
from neo.lib import logging from neo.lib import logging
from neo.lib.protocol import Packets, ProtocolError, ClusterStates, NodeStates from neo.lib.protocol import Packets, ProtocolError, ClusterStates, NodeStates
from .app import monotonic_time
from .handlers import MasterHandler from .handlers import MasterHandler
...@@ -88,7 +89,7 @@ class RecoveryManager(MasterHandler): ...@@ -88,7 +89,7 @@ class RecoveryManager(MasterHandler):
truncate = Packets.Truncate(app.truncate_tid) truncate = Packets.Truncate(app.truncate_tid)
for node in node_list: for node in node_list:
conn = node.getConnection() conn = node.getConnection()
conn.notify(truncate) conn.send(truncate)
self.connectionCompleted(conn, False) self.connectionCompleted(conn, False)
continue continue
node_list = pt.getConnectedNodeList() node_list = pt.getConnectedNodeList()
...@@ -170,7 +171,8 @@ class RecoveryManager(MasterHandler): ...@@ -170,7 +171,8 @@ class RecoveryManager(MasterHandler):
new_nodes = app.pt.load(ptid, row_list, app.nm) new_nodes = app.pt.load(ptid, row_list, app.nm)
except IndexError: except IndexError:
raise ProtocolError('Invalid offset') raise ProtocolError('Invalid offset')
self._notifyAdmins(Packets.NotifyNodeInformation(new_nodes), self._notifyAdmins(
Packets.NotifyNodeInformation(monotonic_time(), new_nodes),
Packets.SendPartitionTable(ptid, row_list)) Packets.SendPartitionTable(ptid, row_list))
self.ask_pt = () self.ask_pt = ()
uuid = conn.getUUID() uuid = conn.getUUID()
...@@ -180,4 +182,4 @@ class RecoveryManager(MasterHandler): ...@@ -180,4 +182,4 @@ class RecoveryManager(MasterHandler):
def _notifyAdmins(self, *packets): def _notifyAdmins(self, *packets):
for node in self.app.nm.getAdminList(only_identified=True): for node in self.app.nm.getAdminList(only_identified=True):
for packet in packets: for packet in packets:
node.notify(packet) node.send(packet)
...@@ -18,29 +18,31 @@ from collections import deque ...@@ -18,29 +18,31 @@ from collections import deque
from time import time from time import time
from struct import pack, unpack from struct import pack, unpack
from neo.lib import logging 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 from neo.lib.util import dump, u64, addTID, tidFromTime
class DelayedError(Exception):
pass
class Transaction(object): class Transaction(object):
""" """
A pending transaction A pending transaction
""" """
locking_tid = ZERO_TID
_tid = None _tid = None
_msg_id = None _msg_id = None
_oid_list = None _oid_list = None
_failed = frozenset()
_prepared = False _prepared = False
# uuid dict hold flag to known who has locked the transaction # uuid dict hold flag to known who has locked the transaction
_uuid_set = None _uuid_set = None
_lock_wait_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 Prepare the transaction, set OIDs and UUIDs related to it
""" """
self._node = node self._node = node
self._storage_readiness = storage_readiness
self._ttid = ttid self._ttid = ttid
self._birth = time() self._birth = time()
# store storage uuids that must be notified at commit # store storage uuids that must be notified at commit
...@@ -113,13 +115,13 @@ class Transaction(object): ...@@ -113,13 +115,13 @@ class Transaction(object):
""" """
return list(self._notification_set) 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._tid = tid
self._oid_list = oid_list self._oid_list = oid_list
self._msg_id = msg_id self._msg_id = msg_id
self._uuid_set = set(uuid_list) self._uuid_set = uuid_set
self._lock_wait_uuid_set = set(uuid_list) self._lock_wait_uuid_set = uuid_set.copy()
self._prepared = True self._prepared = True
def storageLost(self, uuid): def storageLost(self, uuid):
...@@ -163,7 +165,7 @@ class Transaction(object): ...@@ -163,7 +165,7 @@ class Transaction(object):
return not self._lock_wait_uuid_set return not self._lock_wait_uuid_set
class TransactionManager(object): class TransactionManager(EventQueue):
""" """
Manage current transactions Manage current transactions
""" """
...@@ -173,6 +175,7 @@ class TransactionManager(object): ...@@ -173,6 +175,7 @@ class TransactionManager(object):
self.reset() self.reset()
def reset(self): def reset(self):
EventQueue.__init__(self)
# ttid -> transaction # ttid -> transaction
self._ttid_dict = {} self._ttid_dict = {}
self._last_oid = ZERO_OID self._last_oid = ZERO_OID
...@@ -195,6 +198,7 @@ class TransactionManager(object): ...@@ -195,6 +198,7 @@ class TransactionManager(object):
except ValueError: except ValueError:
pass pass
del self._ttid_dict[ttid] del self._ttid_dict[ttid]
self.executeQueuedEvents()
def __contains__(self, ttid): def __contains__(self, ttid):
""" """
...@@ -285,7 +289,7 @@ class TransactionManager(object): ...@@ -285,7 +289,7 @@ class TransactionManager(object):
txn.registerForNotification(uuid) txn.registerForNotification(uuid)
return self._ttid_dict.keys() return self._ttid_dict.keys()
def begin(self, node, tid=None): def begin(self, node, storage_readiness, tid=None):
""" """
Generate a new TID Generate a new TID
""" """
...@@ -297,38 +301,116 @@ class TransactionManager(object): ...@@ -297,38 +301,116 @@ class TransactionManager(object):
# last TID. # last TID.
self._queue.append(tid) self._queue.append(tid)
self.setLastTID(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) logging.debug('Begin %s', txn)
return tid 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 Prepare a transaction to be finished
""" """
txn = self[ttid] 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 # maybe not the fastest but _queue should be often small
if ttid in self._queue: if ttid in self._queue:
tid = ttid tid = ttid
else: else:
tid = self._nextTID(ttid, divisor) tid = self._nextTID(ttid, pt.getPartitions())
self._queue.append(ttid) self._queue.append(ttid)
logging.debug('Finish TXN %s for %s (was %s)', logging.debug('Finish TXN %s for %s (was %s)',
dump(tid), txn.getNode(), dump(ttid)) 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 # check if greater and foreign OID was stored
if oid_list: if oid_list:
self.setLastOID(max(oid_list)) self.setLastOID(max(oid_list))
return tid return tid, node_list
def abort(self, ttid, uuid): def abort(self, ttid, uuid):
""" """
Abort a transaction Abort a transaction
""" """
logging.debug('Abort TXN %s for %s', dump(ttid), uuid_str(uuid)) 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" raise ProtocolError("commit already requested for ttid %s"
% dump(ttid)) % dump(ttid))
del self[ttid] del self[ttid]
return txn._notification_set
def lock(self, ttid, uuid): def lock(self, ttid, uuid):
""" """
...@@ -350,7 +432,7 @@ class TransactionManager(object): ...@@ -350,7 +432,7 @@ class TransactionManager(object):
for ttid, txn in self._ttid_dict.iteritems(): for ttid, txn in self._ttid_dict.iteritems():
if txn.storageLost(uuid) and self._queue[0] == ttid: if txn.storageLost(uuid) and self._queue[0] == ttid:
unlock = True unlock = True
# do not break: we must call forget() on all transactions # do not break: we must call storageLost() on all transactions
if unlock: if unlock:
self._unlockPending() self._unlockPending()
...@@ -370,6 +452,7 @@ class TransactionManager(object): ...@@ -370,6 +452,7 @@ class TransactionManager(object):
break break
del queue[0], self._ttid_dict[ttid] del queue[0], self._ttid_dict[ttid]
self._on_commit(txn) self._on_commit(txn)
self.executeQueuedEvents()
def clientLost(self, node): def clientLost(self, node):
for txn in self._ttid_dict.values(): for txn in self._ttid_dict.values():
...@@ -380,4 +463,4 @@ class TransactionManager(object): ...@@ -380,4 +463,4 @@ class TransactionManager(object):
logging.info('Transactions:') logging.info('Transactions:')
for txn in self._ttid_dict.itervalues(): for txn in self._ttid_dict.itervalues():
logging.info(' %r', txn) logging.info(' %r', txn)
self.logQueuedEvents()
...@@ -131,7 +131,7 @@ class VerificationManager(BaseServiceHandler): ...@@ -131,7 +131,7 @@ class VerificationManager(BaseServiceHandler):
if uuid_set: if uuid_set:
packet = Packets.ValidateTransaction(ttid, tid) packet = Packets.ValidateTransaction(ttid, tid)
for node in getIdentifiedList(pool_set=uuid_set): for node in getIdentifiedList(pool_set=uuid_set):
node.notify(packet) node.send(packet)
def answerLastIDs(self, conn, loid, ltid): def answerLastIDs(self, conn, loid, ltid):
self._uuid_set.remove(conn.getUUID()) self._uuid_set.remove(conn.getUUID())
......
...@@ -20,7 +20,6 @@ ...@@ -20,7 +20,6 @@
import bz2, gzip, errno, optparse, os, signal, sqlite3, sys, time import bz2, gzip, errno, optparse, os, signal, sqlite3, sys, time
from bisect import insort from bisect import insort
from logging import getLevelName from logging import getLevelName
from functools import partial
comp_dict = dict(bz2=bz2.BZ2File, gz=gzip.GzipFile) comp_dict = dict(bz2=bz2.BZ2File, gz=gzip.GzipFile)
...@@ -94,11 +93,6 @@ class Log(object): ...@@ -94,11 +93,6 @@ class Log(object):
exec bz2.decompress(text) in g exec bz2.decompress(text) in g
for x in 'uuid_str', 'Packets', 'PacketMalformedError': for x in 'uuid_str', 'Packets', 'PacketMalformedError':
setattr(self, x, g[x]) setattr(self, x, g[x])
try:
self.notifyNodeInformation = partial(g['formatNodeList'],
prefix=' ! ')
except KeyError:
self.notifyNodeInformation = None
try: try:
self._next_protocol, = q("SELECT date FROM protocol WHERE date>?", self._next_protocol, = q("SELECT date FROM protocol WHERE date>?",
(date,)).next() (date,)).next()
...@@ -131,8 +125,8 @@ class Log(object): ...@@ -131,8 +125,8 @@ class Log(object):
body = None body = None
msg = ['#0x%04x %-30s %s' % (msg_id, msg, peer)] msg = ['#0x%04x %-30s %s' % (msg_id, msg, peer)]
if body is not None: if body is not None:
logger = getattr(self, p.handler_method_name, None) log = getattr(p, '_neolog', None)
if logger or self._decode_all: if log or self._decode_all:
p = p() p = p()
p._id = msg_id p._id = msg_id
p._body = body p._body = body
...@@ -141,15 +135,13 @@ class Log(object): ...@@ -141,15 +135,13 @@ class Log(object):
except self.PacketMalformedError: except self.PacketMalformedError:
msg.append("Can't decode packet") msg.append("Can't decode packet")
else: else:
if logger: if log:
msg += logger(*args) args, extra = log(*args)
elif args: msg += extra
msg = '%s \t| %r' % (msg[0], args), if args and self._decode_all:
msg[0] += ' \t| ' + repr(args)
return date, name, 'PACKET', msg return date, name, 'PACKET', msg
def error(self, code, message):
return "%s (%s)" % (code, message),
def emit_many(log_list): def emit_many(log_list):
log_list = [(log, iter(log).next) for log in 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:]): ...@@ -33,6 +33,7 @@ if filter(re.compile(r'--coverage$|-\w*c').match, sys.argv[1:]):
coverage.neotestrunner = [] coverage.neotestrunner = []
coverage.start() coverage.start()
from neo.lib import logging
from neo.tests import getTempDirectory, NeoTestBase, Patch, \ from neo.tests import getTempDirectory, NeoTestBase, Patch, \
__dict__ as neo_tests__dict__ __dict__ as neo_tests__dict__
from neo.tests.benchmark import BenchmarkRunner from neo.tests.benchmark import BenchmarkRunner
...@@ -45,7 +46,6 @@ UNIT_TEST_MODULES = [ ...@@ -45,7 +46,6 @@ UNIT_TEST_MODULES = [
'neo.tests.testConnection', 'neo.tests.testConnection',
'neo.tests.testHandler', 'neo.tests.testHandler',
'neo.tests.testNodes', 'neo.tests.testNodes',
'neo.tests.testDispatcher',
'neo.tests.testUtil', 'neo.tests.testUtil',
'neo.tests.testPT', 'neo.tests.testPT',
# master application # master application
...@@ -66,7 +66,7 @@ UNIT_TEST_MODULES = [ ...@@ -66,7 +66,7 @@ UNIT_TEST_MODULES = [
# client application # client application
'neo.tests.client.testClientApp', 'neo.tests.client.testClientApp',
'neo.tests.client.testMasterHandler', 'neo.tests.client.testMasterHandler',
'neo.tests.client.testConnectionPool', 'neo.tests.client.testZODBURI',
# light functional tests # light functional tests
'neo.tests.threaded.test', 'neo.tests.threaded.test',
'neo.tests.threaded.testImporter', 'neo.tests.threaded.testImporter',
...@@ -236,6 +236,9 @@ class TestRunner(BenchmarkRunner): ...@@ -236,6 +236,9 @@ class TestRunner(BenchmarkRunner):
parser.add_option('-C', '--cov-unit', action='store_true', parser.add_option('-C', '--cov-unit', action='store_true',
help='Same as -c but output 1 file per test,' help='Same as -c but output 1 file per test,'
' in the temporary test directory') ' 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, parser.add_option('-l', '--loop', type='int', default=1,
help='Repeat tests several times') help='Repeat tests several times')
parser.add_option('-f', '--functional', action='store_true', parser.add_option('-f', '--functional', action='store_true',
...@@ -280,6 +283,7 @@ Environment Variables: ...@@ -280,6 +283,7 @@ Environment Variables:
sys.exit('Nothing to run, please give one of -f, -u, -z') sys.exit('Nothing to run, please give one of -f, -u, -z')
options.unit = options.functional = options.zodb = True options.unit = options.functional = options.zodb = True
return dict( return dict(
log = options.log,
loop = options.loop, loop = options.loop,
unit = options.unit, unit = options.unit,
functional = options.functional, functional = options.functional,
...@@ -292,6 +296,8 @@ Environment Variables: ...@@ -292,6 +296,8 @@ Environment Variables:
def start(self): def start(self):
config = self._config config = self._config
logging.backlog(max_packet=1<<20,
**({'max_size': None} if config.log else {}))
only = config.only only = config.only
# run requested tests # run requested tests
runner = NeoTestRunner(config.title or 'Neo', config.verbosity) runner = NeoTestRunner(config.title or 'Neo', config.verbosity)
......
...@@ -28,7 +28,6 @@ from neo.lib.util import dump ...@@ -28,7 +28,6 @@ from neo.lib.util import dump
from neo.lib.bootstrap import BootstrapManager from neo.lib.bootstrap import BootstrapManager
from .checker import Checker from .checker import Checker
from .database import buildDatabaseManager from .database import buildDatabaseManager
from .exception import AlreadyPendingError
from .handlers import identification, initialization from .handlers import identification, initialization
from .handlers import master, hidden from .handlers import master, hidden
from .replicator import Replicator from .replicator import Replicator
...@@ -39,13 +38,14 @@ from neo.lib.debug import register as registerLiveDebugger ...@@ -39,13 +38,14 @@ from neo.lib.debug import register as registerLiveDebugger
class Application(BaseApplication): class Application(BaseApplication):
"""The storage node application.""" """The storage node application."""
checker = replicator = tm = None
def __init__(self, config): def __init__(self, config):
super(Application, self).__init__( super(Application, self).__init__(
config.getSSL(), config.getDynamicMasterList()) config.getSSL(), config.getDynamicMasterList())
# set the cluster name # set the cluster name
self.name = config.getCluster() self.name = config.getCluster()
self.tm = TransactionManager(self)
self.dm = buildDatabaseManager(config.getAdapter(), self.dm = buildDatabaseManager(config.getAdapter(),
(config.getDatabase(), config.getEngine(), config.getWait()), (config.getDatabase(), config.getEngine(), config.getWait()),
) )
...@@ -62,20 +62,13 @@ class Application(BaseApplication): ...@@ -62,20 +62,13 @@ class Application(BaseApplication):
# partitions. # partitions.
self.pt = None self.pt = None
self.checker = Checker(self)
self.replicator = Replicator(self)
self.listening_conn = None self.listening_conn = None
self.master_conn = None self.master_conn = None
self.master_node = None self.master_node = None
# operation related data # operation related data
self.event_queue = None
self.event_queue_dict = None
self.operational = False self.operational = False
# ready is True when operational and got all informations
self.ready = False
self.dm.setup(reset=config.getReset()) self.dm.setup(reset=config.getReset())
self.loadConfiguration() self.loadConfiguration()
...@@ -95,8 +88,8 @@ class Application(BaseApplication): ...@@ -95,8 +88,8 @@ class Application(BaseApplication):
def log(self): def log(self):
self.em.log() self.em.log()
self.logQueuedEvents()
self.nm.log() self.nm.log()
if self.tm:
self.tm.log() self.tm.log()
if self.pt is not None: if self.pt is not None:
self.pt.log() self.pt.log()
...@@ -137,19 +130,17 @@ class Application(BaseApplication): ...@@ -137,19 +130,17 @@ class Application(BaseApplication):
def loadPartitionTable(self): def loadPartitionTable(self):
"""Load a partition table from the database.""" """Load a partition table from the database."""
self.pt.clear()
ptid = self.dm.getPTID() ptid = self.dm.getPTID()
cell_list = self.dm.getPartitionTable() if ptid is None:
new_cell_list = [] return
for offset, uuid, state in cell_list: cell_list = []
# convert from int to Enum for offset, uuid, state in self.dm.getPartitionTable():
state = CellStates[state]
# register unknown nodes # register unknown nodes
if self.nm.getByUUID(uuid) is None: if self.nm.getByUUID(uuid) is None:
self.nm.createStorage(uuid=uuid) self.nm.createStorage(uuid=uuid)
new_cell_list.append((offset, uuid, state)) cell_list.append((offset, uuid, CellStates[state]))
# load the partition table in manager self.pt.update(ptid, cell_list, self.nm)
self.pt.clear()
self.pt.update(ptid, new_cell_list, self.nm)
def run(self): def run(self):
try: try:
...@@ -173,10 +164,9 @@ class Application(BaseApplication): ...@@ -173,10 +164,9 @@ class Application(BaseApplication):
# Connect to a primary master node, verify data, and # Connect to a primary master node, verify data, and
# start the operation. This cycle will be executed permanently, # start the operation. This cycle will be executed permanently,
# until the user explicitly requests a shutdown. # until the user explicitly requests a shutdown.
self.operational = False
while True: while True:
self.cluster_state = None self.cluster_state = None
self.ready = False
self.operational = False
if self.master_node is None: if self.master_node is None:
# look for the primary master # look for the primary master
self.connectToPrimary() self.connectToPrimary()
...@@ -184,13 +174,9 @@ class Application(BaseApplication): ...@@ -184,13 +174,9 @@ class Application(BaseApplication):
node = self.nm.getByUUID(self.uuid) node = self.nm.getByUUID(self.uuid)
if node is not None and node.isHidden(): if node is not None and node.isHidden():
self.wait() self.wait()
# drop any client node self.checker = Checker(self)
for conn in self.em.getConnectionList(): self.replicator = Replicator(self)
if conn not in (self.listening_conn, self.master_conn): self.tm = TransactionManager(self)
conn.close()
# create/clear event queue
self.event_queue = deque()
self.event_queue_dict = {}
try: try:
self.initialize() self.initialize()
self.doOperation() self.doOperation()
...@@ -200,7 +186,15 @@ class Application(BaseApplication): ...@@ -200,7 +186,15 @@ class Application(BaseApplication):
except PrimaryFailure, msg: except PrimaryFailure, msg:
logging.error('primary master is down: %s', msg) logging.error('primary master is down: %s', msg)
finally: 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): def connectToPrimary(self):
"""Find a primary master node, and connect to it. """Find a primary master node, and connect to it.
...@@ -212,11 +206,6 @@ class Application(BaseApplication): ...@@ -212,11 +206,6 @@ class Application(BaseApplication):
at this stage.""" at this stage."""
pt = self.pt 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 # search, find, connect and identify to the primary master
bootstrap = BootstrapManager(self, NodeTypes.STORAGE, self.server) bootstrap = BootstrapManager(self, NodeTypes.STORAGE, self.server)
self.master_node, self.master_conn, num_partitions, num_replicas = \ self.master_node, self.master_conn, num_partitions, num_replicas = \
...@@ -246,9 +235,7 @@ class Application(BaseApplication): ...@@ -246,9 +235,7 @@ class Application(BaseApplication):
self.master_conn.setHandler(initialization.InitializationHandler(self)) self.master_conn.setHandler(initialization.InitializationHandler(self))
while not self.operational: while not self.operational:
_poll() _poll()
self.ready = True self.master_conn.send(Packets.NotifyReady())
self.replicator.populate()
self.master_conn.notify(Packets.NotifyReady())
def doOperation(self): def doOperation(self):
"""Handle everything, including replications and transactions.""" """Handle everything, including replications and transactions."""
...@@ -258,12 +245,11 @@ class Application(BaseApplication): ...@@ -258,12 +245,11 @@ class Application(BaseApplication):
_poll = self.em._poll _poll = self.em._poll
isIdle = self.em.isIdle isIdle = self.em.isIdle
handler = master.MasterOperationHandler(self) self.master_conn.setHandler(master.MasterOperationHandler(self))
self.master_conn.setHandler(handler) self.replicator.populate()
# Forget all unfinished data. # Forget all unfinished data.
self.dm.dropUnfinishedData() self.dm.dropUnfinishedData()
self.tm.reset()
self.task_queue = task_queue = deque() self.task_queue = task_queue = deque()
try: try:
...@@ -282,13 +268,6 @@ class Application(BaseApplication): ...@@ -282,13 +268,6 @@ class Application(BaseApplication):
poll() poll()
finally: finally:
del self.task_queue 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): def changeClusterState(self, state):
self.cluster_state = state self.cluster_state = state
...@@ -309,46 +288,6 @@ class Application(BaseApplication): ...@@ -309,46 +288,6 @@ class Application(BaseApplication):
if not node.isHidden(): if not node.isHidden():
break 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): def newTask(self, iterator):
try: try:
iterator.next() iterator.next()
......
...@@ -109,7 +109,7 @@ class Checker(object): ...@@ -109,7 +109,7 @@ class Checker(object):
self.source = source self.source = source
def start(): def start():
if app.tm.isLockedTid(max_tid): if app.tm.isLockedTid(max_tid):
app.queueEvent(start) app.tm.read_queue.queueEvent(start)
return return
args = partition, CHECK_COUNT, min_tid, max_tid args = partition, CHECK_COUNT, min_tid, max_tid
p = Packets.AskCheckTIDRange(*args) p = Packets.AskCheckTIDRange(*args)
...@@ -181,7 +181,7 @@ class Checker(object): ...@@ -181,7 +181,7 @@ class Checker(object):
uuid_list.append(conn.getUUID()) uuid_list.append(conn.getUUID())
self.app.closeClient(conn) self.app.closeClient(conn)
p = Packets.NotifyPartitionCorrupted(self.partition, uuid_list) 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: if len(self.conn_dict) <= 1:
logging.warning("check of partition %u aborted", self.partition) logging.warning("check of partition %u aborted", self.partition)
self.queue.clear() self.queue.clear()
......
...@@ -283,11 +283,11 @@ class ImporterDatabaseManager(DatabaseManager): ...@@ -283,11 +283,11 @@ class ImporterDatabaseManager(DatabaseManager):
super(ImporterDatabaseManager, self).__init__(*args, **kw) super(ImporterDatabaseManager, self).__init__(*args, **kw)
implements(self, """_getNextTID checkSerialRange checkTIDRange implements(self, """_getNextTID checkSerialRange checkTIDRange
deleteObject deleteTransaction dropPartitions getLastTID deleteObject deleteTransaction dropPartitions getLastTID
getReplicationObjectList getTIDList nonempty""".split()) getReplicationObjectList _getTIDList nonempty""".split())
_uncommitted_data = property( _getPartition = property(lambda self: self.db._getPartition)
lambda self: self.db._uncommitted_data, _getReadablePartition = property(lambda self: self.db._getReadablePartition)
lambda self, value: setattr(self.db, "_uncommitted_data", value)) _uncommitted_data = property(lambda self: self.db._uncommitted_data)
def _parse(self, database): def _parse(self, database):
config = SafeConfigParser() config = SafeConfigParser()
...@@ -300,11 +300,12 @@ class ImporterDatabaseManager(DatabaseManager): ...@@ -300,11 +300,12 @@ class ImporterDatabaseManager(DatabaseManager):
self.compress = main.get('compress', 1) self.compress = main.get('compress', 1)
self.db = buildDatabaseManager(main['adapter'], self.db = buildDatabaseManager(main['adapter'],
(main['database'], main.get('engine'), main['wait'])) (main['database'], main.get('engine'), main['wait']))
for x in """query erase getConfiguration _setConfiguration for x in """getConfiguration _setConfiguration setNumPartitions
getPartitionTable changePartitionTable query erase getPartitionTable changePartitionTable
getUnfinishedTIDDict dropUnfinishedData abortTransaction getUnfinishedTIDDict dropUnfinishedData abortTransaction
storeTransaction lockTransaction unlockTransaction storeTransaction lockTransaction unlockTransaction
storeData getOrphanList _pruneData deferCommit loadData storeData getOrphanList _pruneData deferCommit
dropPartitionsTemporary
""".split(): """.split():
setattr(self, x, getattr(self.db, x)) setattr(self, x, getattr(self.db, x))
...@@ -315,21 +316,14 @@ class ImporterDatabaseManager(DatabaseManager): ...@@ -315,21 +316,14 @@ class ImporterDatabaseManager(DatabaseManager):
self.db.commit() self.db.commit()
self._last_commit = time.time() 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): def close(self):
self.db.close() self.db.close()
if isinstance(self.zodb, list): # _setup called if isinstance(self.zodb, list): # _setup called
for zodb in self.zodb: for zodb in self.zodb:
zodb.close() zodb.close()
def _setup(self): def setup(self, reset=0):
self.db._setup() self.db.setup(reset)
zodb_state = self.getConfiguration("zodb") zodb_state = self.getConfiguration("zodb")
if zodb_state: if zodb_state:
logging.warning("Ignoring configuration file for oid mapping." logging.warning("Ignoring configuration file for oid mapping."
......
...@@ -14,14 +14,14 @@ ...@@ -14,14 +14,14 @@
# You should have received a copy of the GNU General Public License # You should have received a copy of the GNU General Public License
# along with this program. If not, see <http://www.gnu.org/licenses/>. # along with this program. If not, see <http://www.gnu.org/licenses/>.
import threading import struct, threading
from collections import defaultdict from collections import defaultdict
from contextlib import contextmanager from contextlib import contextmanager
from functools import wraps from functools import wraps
from neo.lib import logging, util from neo.lib import logging, util
from neo.lib.exception import DatabaseFailure from neo.lib.exception import DatabaseFailure
from neo.lib.interfaces import abstract, requires 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 X = 0
...@@ -59,7 +59,7 @@ class DatabaseManager(object): ...@@ -59,7 +59,7 @@ class DatabaseManager(object):
_deferred = 0 _deferred = 0
_duplicating = _repairing = None _duplicating = _repairing = None
def __init__(self, database, engine=None, wait=0): def __init__(self, database, engine=None, wait=None):
""" """
Initialize the object. Initialize the object.
""" """
...@@ -68,17 +68,15 @@ class DatabaseManager(object): ...@@ -68,17 +68,15 @@ class DatabaseManager(object):
raise ValueError("Unsupported engine: %r not in %r" raise ValueError("Unsupported engine: %r not in %r"
% (engine, self.ENGINES)) % (engine, self.ENGINES))
self._engine = engine 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._parse(database)
self._connect() self._connect()
def __getattr__(self, attr): def __getattr__(self, attr):
if attr == "_getPartition": if self._duplicating is None:
np = self.getNumPartitions()
value = lambda x: x % np
elif self._duplicating is None:
return self.__getattribute__(attr) return self.__getattribute__(attr)
else:
value = getattr(self._duplicating, attr) value = getattr(self._duplicating, attr)
setattr(self, attr, value) setattr(self, attr, value)
return value return value
...@@ -105,19 +103,10 @@ class DatabaseManager(object): ...@@ -105,19 +103,10 @@ class DatabaseManager(object):
def _connect(self): def _connect(self):
"""Connect to the database""" """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 @abstract
def erase(self): def erase(self):
"""""" """"""
@abstract
def _setup(self): def _setup(self):
"""To be overridden by the backend to set up a database """To be overridden by the backend to set up a database
...@@ -128,6 +117,16 @@ class DatabaseManager(object): ...@@ -128,6 +117,16 @@ class DatabaseManager(object):
Keys are data ids and values are number of references. 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 @abstract
def nonempty(self, table): def nonempty(self, table):
"""Check whether table is empty or return None if it does not exist""" """Check whether table is empty or return None if it does not exist"""
...@@ -222,7 +221,7 @@ class DatabaseManager(object): ...@@ -222,7 +221,7 @@ class DatabaseManager(object):
""" """
self.setConfiguration('partitions', num_partitions) self.setConfiguration('partitions', num_partitions)
try: try:
del self._getPartition del self._getPartition, self._getReadablePartition
except AttributeError: except AttributeError:
pass pass
...@@ -260,15 +259,6 @@ class DatabaseManager(object): ...@@ -260,15 +259,6 @@ class DatabaseManager(object):
if ptid is not None: if ptid is not None:
return int(ptid) 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): def getBackupTID(self):
return util.bin(self.getConfiguration('backup_tid')) return util.bin(self.getConfiguration('backup_tid'))
...@@ -299,7 +289,7 @@ class DatabaseManager(object): ...@@ -299,7 +289,7 @@ class DatabaseManager(object):
return -1 return -1
@abstract @abstract
def getPartitionTable(self): def getPartitionTable(self, *nid):
"""Return a whole partition table as a sequence of rows. Each row """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 is again a tuple of an offset (row ID), the NID of a storage
node, and a cell state.""" node, and a cell state."""
...@@ -409,21 +399,67 @@ class DatabaseManager(object): ...@@ -409,21 +399,67 @@ class DatabaseManager(object):
compression, checksum, data, compression, checksum, data,
None if data_serial is None else util.p64(data_serial)) None if data_serial is None else util.p64(data_serial))
@abstract @contextmanager
def changePartitionTable(self, ptid, cell_list, reset=False): 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 """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), 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 the NID of a storage node, and a cell state. If reset is True,
Table ID must be stored as well. If reset is True, existing data existing data is first thrown away.
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 @abstract
def dropPartitions(self, offset_list): def dropPartitions(self, offset_list):
"""Delete all data for specified partitions""" """Delete all data for specified partitions"""
@abstract def _getUnfinishedDataIdList(self):
"""Drop any unfinished data from a database."""
@requires(_getUnfinishedDataIdList)
def dropUnfinishedData(self): def dropUnfinishedData(self):
"""Drop any unfinished data from a database.""" """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 @abstract
def storeTransaction(self, tid, object_list, transaction, temporary = True): def storeTransaction(self, tid, object_list, transaction, temporary = True):
...@@ -469,6 +505,11 @@ class DatabaseManager(object): ...@@ -469,6 +505,11 @@ class DatabaseManager(object):
no hash collision. no hash collision.
""" """
@abstract
def loadData(self, data_id):
"""Inverse of storeData
"""
def holdData(self, checksum_or_id, *args): def holdData(self, checksum_or_id, *args):
"""Store raw data of temporary object """Store raw data of temporary object
...@@ -499,8 +540,7 @@ class DatabaseManager(object): ...@@ -499,8 +540,7 @@ class DatabaseManager(object):
else: else:
del refcount[data_id] del refcount[data_id]
if prune: if prune:
self._pruneData(data_id_list) return self._pruneData(data_id_list)
self.commit()
@fallback @fallback
def _getDataTID(self, oid, tid=None, before_tid=None): def _getDataTID(self, oid, tid=None, before_tid=None):
...@@ -570,11 +610,14 @@ class DatabaseManager(object): ...@@ -570,11 +610,14 @@ class DatabaseManager(object):
return current_tid, current_tid return current_tid, current_tid
return current_tid, tid return current_tid, tid
if transaction_object: if transaction_object:
try:
current_tid = current_data_tid = u64(transaction_object[2]) current_tid = current_data_tid = u64(transaction_object[2])
except struct.error:
current_tid = current_data_tid = tid
else: else:
current_tid, current_data_tid = getDataTID(before_tid=ltid) current_tid, current_data_tid = getDataTID(before_tid=ltid)
if current_tid is None: if current_tid is None:
return (None, None, False) return None, None, False
found_undone_tid, undone_data_tid = getDataTID(tid=undone_tid) found_undone_tid, undone_data_tid = getDataTID(tid=undone_tid)
assert found_undone_tid is not None, (oid, undone_tid) assert found_undone_tid is not None, (oid, undone_tid)
is_current = undone_data_tid in (current_data_tid, tid) is_current = undone_data_tid in (current_data_tid, tid)
...@@ -677,12 +720,20 @@ class DatabaseManager(object): ...@@ -677,12 +720,20 @@ class DatabaseManager(object):
min_tid and min_oid and below max_tid, for given partition, min_tid and min_oid and below max_tid, for given partition,
sorted in ascending order.""" 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, """Return a list of TIDs in ascending order from an offset,
at most the specified length. The list of partitions are passed at most the specified length. The list of partitions are passed
to filter out non-applicable TIDs.""" 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 @abstract
def getReplicationTIDList(self, min_tid, max_tid, length, partition): def getReplicationTIDList(self, min_tid, max_tid, length, partition):
"""Return a list of TIDs in ascending order from an initial tid value, """Return a list of TIDs in ascending order from an initial tid value,
......
This diff is collapsed.
...@@ -219,7 +219,9 @@ class SQLiteDatabaseManager(DatabaseManager): ...@@ -219,7 +219,9 @@ class SQLiteDatabaseManager(DatabaseManager):
else: else:
q("REPLACE INTO config VALUES (?,?)", (key, str(value))) 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") return self.query("SELECT * FROM pt")
# A test with a table of 20 million lines and SQLite 3.8.7.1 shows that # A test with a table of 20 million lines and SQLite 3.8.7.1 shows that
...@@ -260,7 +262,7 @@ class SQLiteDatabaseManager(DatabaseManager): ...@@ -260,7 +262,7 @@ class SQLiteDatabaseManager(DatabaseManager):
# even though ttid is a constant. # even though ttid is a constant.
for tid, in self.query("SELECT tid FROM trans" for tid, in self.query("SELECT tid FROM trans"
" WHERE partition=? AND tid>=? AND ttid=? LIMIT 1", " WHERE partition=? AND tid>=? AND ttid=? LIMIT 1",
(self._getPartition(ttid), ttid, ttid)): (self._getReadablePartition(ttid), ttid, ttid)):
return util.p64(tid) return util.p64(tid)
def getLastObjectTID(self, oid): def getLastObjectTID(self, oid):
...@@ -268,7 +270,7 @@ class SQLiteDatabaseManager(DatabaseManager): ...@@ -268,7 +270,7 @@ class SQLiteDatabaseManager(DatabaseManager):
r = self.query("SELECT tid FROM obj" r = self.query("SELECT tid FROM obj"
" WHERE partition=? AND oid=?" " WHERE partition=? AND oid=?"
" ORDER BY tid DESC LIMIT 1", " ORDER BY tid DESC LIMIT 1",
(self._getPartition(oid), oid)).fetchone() (self._getReadablePartition(oid), oid)).fetchone()
return r and util.p64(r[0]) return r and util.p64(r[0])
def _getNextTID(self, *args): # partition, oid, tid def _getNextTID(self, *args): # partition, oid, tid
...@@ -279,7 +281,7 @@ class SQLiteDatabaseManager(DatabaseManager): ...@@ -279,7 +281,7 @@ class SQLiteDatabaseManager(DatabaseManager):
def _getObject(self, oid, tid=None, before_tid=None): def _getObject(self, oid, tid=None, before_tid=None):
q = self.query q = self.query
partition = self._getPartition(oid) partition = self._getReadablePartition(oid)
sql = ('SELECT tid, compression, data.hash, value, value_tid' sql = ('SELECT tid, compression, data.hash, value, value_tid'
' FROM obj LEFT JOIN data ON obj.data_id = data.id' ' FROM obj LEFT JOIN data ON obj.data_id = data.id'
' WHERE partition=? AND oid=?') ' WHERE partition=? AND oid=?')
...@@ -300,7 +302,7 @@ class SQLiteDatabaseManager(DatabaseManager): ...@@ -300,7 +302,7 @@ class SQLiteDatabaseManager(DatabaseManager):
return (serial, self._getNextTID(partition, oid, serial), return (serial, self._getNextTID(partition, oid, serial),
compression, checksum, data, value_serial) compression, checksum, data, value_serial)
def changePartitionTable(self, ptid, cell_list, reset=False): def _changePartitionTable(self, cell_list, reset=False):
q = self.query q = self.query
if reset: if reset:
q("DELETE FROM pt") q("DELETE FROM pt")
...@@ -315,7 +317,6 @@ class SQLiteDatabaseManager(DatabaseManager): ...@@ -315,7 +317,6 @@ class SQLiteDatabaseManager(DatabaseManager):
if state != CellStates.DISCARDED: if state != CellStates.DISCARDED:
q("INSERT OR FAIL INTO pt VALUES (?,?,?)", q("INSERT OR FAIL INTO pt VALUES (?,?,?)",
(offset, nid, int(state))) (offset, nid, int(state)))
self.setPTID(ptid)
def dropPartitions(self, offset_list): def dropPartitions(self, offset_list):
where = " WHERE partition=?" where = " WHERE partition=?"
...@@ -328,12 +329,15 @@ class SQLiteDatabaseManager(DatabaseManager): ...@@ -328,12 +329,15 @@ class SQLiteDatabaseManager(DatabaseManager):
q("DELETE FROM trans" + where, args) q("DELETE FROM trans" + where, args)
self._pruneData(data_id_list) 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 q = self.query
data_id_list = [x for x, in q("SELECT data_id FROM tobj") if x] q("DELETE FROM tobj" + where)
q("DELETE FROM tobj") q("DELETE FROM ttrans" + where)
q("DELETE FROM ttrans")
self.releaseData(data_id_list, True)
def storeTransaction(self, tid, object_list, transaction, temporary=True): def storeTransaction(self, tid, object_list, transaction, temporary=True):
u64 = util.u64 u64 = util.u64
...@@ -404,8 +408,12 @@ class SQLiteDatabaseManager(DatabaseManager): ...@@ -404,8 +408,12 @@ class SQLiteDatabaseManager(DatabaseManager):
return r return r
raise 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): def _getDataTID(self, oid, tid=None, before_tid=None):
partition = self._getPartition(oid) partition = self._getReadablePartition(oid)
sql = 'SELECT tid, value_tid FROM obj' \ sql = 'SELECT tid, value_tid FROM obj' \
' WHERE partition=? AND oid=?' ' WHERE partition=? AND oid=?'
if tid is not None: if tid is not None:
...@@ -447,7 +455,6 @@ class SQLiteDatabaseManager(DatabaseManager): ...@@ -447,7 +455,6 @@ class SQLiteDatabaseManager(DatabaseManager):
def deleteTransaction(self, tid): def deleteTransaction(self, tid):
tid = util.u64(tid) tid = util.u64(tid)
getPartition = self._getPartition
self.query("DELETE FROM trans WHERE partition=? AND tid=?", self.query("DELETE FROM trans WHERE partition=? AND tid=?",
(self._getPartition(tid), tid)) (self._getPartition(tid), tid))
...@@ -486,7 +493,7 @@ class SQLiteDatabaseManager(DatabaseManager): ...@@ -486,7 +493,7 @@ class SQLiteDatabaseManager(DatabaseManager):
q = self.query q = self.query
r = q("SELECT oids, user, description, ext, packed, ttid" r = q("SELECT oids, user, description, ext, packed, ttid"
" FROM trans WHERE partition=? AND tid=?", " FROM trans WHERE partition=? AND tid=?",
(self._getPartition(tid), tid)).fetchone() (self._getReadablePartition(tid), tid)).fetchone()
if not r and all: if not r and all:
r = q("SELECT oids, user, description, ext, packed, ttid" r = q("SELECT oids, user, description, ext, packed, ttid"
" FROM ttrans WHERE tid=?", (tid,)).fetchone() " FROM ttrans WHERE tid=?", (tid,)).fetchone()
...@@ -506,7 +513,8 @@ class SQLiteDatabaseManager(DatabaseManager): ...@@ -506,7 +513,8 @@ class SQLiteDatabaseManager(DatabaseManager):
FROM obj LEFT JOIN data ON obj.data_id = data.id FROM obj LEFT JOIN data ON obj.data_id = data.id
WHERE partition=? AND oid=? AND tid>=? WHERE partition=? AND oid=? AND tid>=?
ORDER BY tid DESC LIMIT ?,?""", ORDER BY tid DESC LIMIT ?,?""",
(self._getPartition(oid), oid, self._getPackTID(), offset, length)) (self._getReadablePartition(oid), oid,
self._getPackTID(), offset, length))
] or None ] or None
def getReplicationObjectList(self, min_tid, max_tid, length, partition, def getReplicationObjectList(self, min_tid, max_tid, length, partition,
...@@ -521,12 +529,11 @@ class SQLiteDatabaseManager(DatabaseManager): ...@@ -521,12 +529,11 @@ class SQLiteDatabaseManager(DatabaseManager):
ORDER BY tid ASC, oid ASC LIMIT ?""", ORDER BY tid ASC, oid ASC LIMIT ?""",
(partition, u64(max_tid), min_tid, u64(min_oid), min_tid, length))] (partition, u64(max_tid), min_tid, u64(min_oid), min_tid, length))]
def getTIDList(self, offset, length, partition_list): def _getTIDList(self, offset, length, partition_list):
p64 = util.p64 return (t[0] for t in self.query(
return [p64(t[0]) for t in self.query("""\ "SELECT tid FROM trans WHERE `partition` in (%s)"
SELECT tid FROM trans WHERE partition in (%s) " ORDER BY tid DESC LIMIT %d,%d"
ORDER BY tid DESC LIMIT %d,%d""" % (','.join(map(str, partition_list)), offset, length)))
% (','.join(map(str, partition_list)), offset, length))]
def getReplicationTIDList(self, min_tid, max_tid, length, partition): def getReplicationTIDList(self, min_tid, max_tid, length, partition):
u64 = util.u64 u64 = util.u64
...@@ -544,7 +551,7 @@ class SQLiteDatabaseManager(DatabaseManager): ...@@ -544,7 +551,7 @@ class SQLiteDatabaseManager(DatabaseManager):
# transaction referencing its value at max_serial or above. # transaction referencing its value at max_serial or above.
# If there is, copy value to the first future transaction. Any further # If there is, copy value to the first future transaction. Any further
# reference is just updated to point to the new data location. # reference is just updated to point to the new data location.
partition = self._getPartition(oid) partition = self._getReadablePartition(oid)
value_serial = None value_serial = None
q = self.query q = self.query
for T in '', 't': for T in '', 't':
...@@ -565,7 +572,7 @@ class SQLiteDatabaseManager(DatabaseManager): ...@@ -565,7 +572,7 @@ class SQLiteDatabaseManager(DatabaseManager):
p64 = util.p64 p64 = util.p64
tid = util.u64(tid) tid = util.u64(tid)
updatePackFuture = self._updatePackFuture updatePackFuture = self._updatePackFuture
getPartition = self._getPartition getPartition = self._getReadablePartition
q = self.query q = self.query
self._setPackTID(tid) self._setPackTID(tid)
for count, oid, max_serial in q("SELECT COUNT(*) - 1, oid, MAX(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 ...@@ -18,9 +18,20 @@ import weakref
from neo.lib import logging from neo.lib import logging
from neo.lib.handler import EventHandler from neo.lib.handler import EventHandler
from neo.lib.exception import PrimaryFailure, StoppedOperation 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): def connectionLost(self, conn, new_state):
if self.app.listening_conn: # if running if self.app.listening_conn: # if running
...@@ -36,10 +47,11 @@ class BaseMasterHandler(EventHandler): ...@@ -36,10 +47,11 @@ class BaseMasterHandler(EventHandler):
def notifyClusterInformation(self, conn, state): def notifyClusterInformation(self, conn, state):
self.app.changeClusterState(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 """Store information on nodes, only if this is sent by a primary
master node.""" 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: for node_type, _, uuid, state, _ in node_list:
if uuid == self.app.uuid: if uuid == self.app.uuid:
# This is me, do what the master tell me # This is me, do what the master tell me
...@@ -55,8 +67,17 @@ class BaseMasterHandler(EventHandler): ...@@ -55,8 +67,17 @@ class BaseMasterHandler(EventHandler):
uuid_str(uuid)) uuid_str(uuid))
self.app.tm.abortFor(uuid) self.app.tm.abortFor(uuid)
def answerUnfinishedTransactions(self, conn, *args, **kw): def notifyPartitionChanges(self, conn, ptid, cell_list):
self.app.replicator.setUnfinishedTIDList(*args, **kw) """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): def askFinalTID(self, conn, ttid):
conn.answer(Packets.AnswerFinalTID(self.app.dm.getFinalTID(ttid))) conn.answer(Packets.AnswerFinalTID(self.app.dm.getFinalTID(ttid)))
......
This diff is collapsed.
...@@ -27,12 +27,16 @@ class IdentificationHandler(EventHandler): ...@@ -27,12 +27,16 @@ class IdentificationHandler(EventHandler):
def connectionLost(self, conn, new_state): def connectionLost(self, conn, new_state):
logging.warning('A connection was lost during identification') 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, def requestIdentification(self, conn, node_type, uuid, address, name,
id_timestamp): id_timestamp):
self.checkClusterName(name) self.checkClusterName(name)
app = self.app app = self.app
# reject any incoming connections if not ready # reject any incoming connections if not ready
if not app.ready: if not app.operational:
raise NotReadyError raise NotReadyError
if uuid is None: if uuid is None:
if node_type != NodeTypes.STORAGE: if node_type != NodeTypes.STORAGE:
...@@ -43,12 +47,6 @@ class IdentificationHandler(EventHandler): ...@@ -43,12 +47,6 @@ class IdentificationHandler(EventHandler):
if uuid == app.uuid: if uuid == app.uuid:
raise ProtocolError("uuid conflict or loopback connection") raise ProtocolError("uuid conflict or loopback connection")
node = app.nm.getByUUID(uuid, id_timestamp) 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(): if node.isBroken():
raise BrokenNodeDisallowedError raise BrokenNodeDisallowedError
# choose the handler according to the node type # choose the handler according to the node type
......
...@@ -23,24 +23,26 @@ class InitializationHandler(BaseMasterHandler): ...@@ -23,24 +23,26 @@ class InitializationHandler(BaseMasterHandler):
def sendPartitionTable(self, conn, ptid, row_list): def sendPartitionTable(self, conn, ptid, row_list):
app = self.app app = self.app
pt = app.pt pt = app.pt
pt.load(ptid, row_list, self.app.nm) pt.load(ptid, row_list, app.nm)
if not pt.filled(): if not pt.filled():
raise ProtocolError('Partial partition table received') raise ProtocolError('Partial partition table received')
# Install the partition table into the database for persistence. # Install the partition table into the database for persistence.
cell_list = [] cell_list = []
num_partitions = pt.getPartitions() offset_list = xrange(pt.getPartitions())
unassigned_set = set(xrange(num_partitions)) unassigned_set = set(offset_list)
for offset in xrange(num_partitions): for offset in offset_list:
for cell in pt.getCellList(offset): for cell in pt.getCellList(offset):
cell_list.append((offset, cell.getUUID(), cell.getState())) cell_list.append((offset, cell.getUUID(), cell.getState()))
if cell.getUUID() == app.uuid: if cell.getUUID() == app.uuid:
unassigned_set.remove(offset) unassigned_set.remove(offset)
# delete objects database # delete objects database
dm = app.dm
if unassigned_set: if unassigned_set:
logging.debug('drop data for partitions %r', 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): def truncate(self, conn, tid):
dm = self.app.dm dm = self.app.dm
......
...@@ -31,25 +31,6 @@ class MasterOperationHandler(BaseMasterHandler): ...@@ -31,25 +31,6 @@ class MasterOperationHandler(BaseMasterHandler):
dm._setBackupTID(dm.getLastIDs()[0] or ZERO_TID) dm._setBackupTID(dm.getLastIDs()[0] or ZERO_TID)
dm.commit() 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): def askLockInformation(self, conn, ttid, tid):
self.app.tm.lock(ttid, tid) self.app.tm.lock(ttid, tid)
conn.answer(Packets.AnswerInformationLocked(ttid)) conn.answer(Packets.AnswerInformationLocked(ttid))
...@@ -64,6 +45,9 @@ class MasterOperationHandler(BaseMasterHandler): ...@@ -64,6 +45,9 @@ class MasterOperationHandler(BaseMasterHandler):
logging.info('Pack finished.') logging.info('Pack finished.')
conn.answer(Packets.AnswerPack(True)) 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): def replicate(self, conn, tid, upstream_name, source_dict):
self.app.replicator.backup(tid, {p: a and (a, upstream_name) self.app.replicator.backup(tid, {p: a and (a, upstream_name)
for p, a in source_dict.iteritems()}) for p, a in source_dict.iteritems()})
......
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
...@@ -9,9 +9,6 @@ from email.mime.text import MIMEText ...@@ -9,9 +9,6 @@ from email.mime.text import MIMEText
MAIL_SERVER = '127.0.0.1:25' MAIL_SERVER = '127.0.0.1:25'
from neo.lib import logging
logging.backlog()
class AttributeDict(dict): class AttributeDict(dict):
def __getattr__(self, item): def __getattr__(self, item):
......
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
...@@ -158,7 +158,7 @@ class NEOProcess(object): ...@@ -158,7 +158,7 @@ class NEOProcess(object):
if args: if args:
os.close(w) os.close(w)
os.kill(os.getpid(), signal.SIGSTOP) os.kill(os.getpid(), signal.SIGSTOP)
self.pid = os.fork() self.pid = logging.fork()
if self.pid: if self.pid:
# Wait that the signal to kill the child is set up. # Wait that the signal to kill the child is set up.
os.close(w) os.close(w)
...@@ -168,8 +168,6 @@ class NEOProcess(object): ...@@ -168,8 +168,6 @@ class NEOProcess(object):
else: else:
# Child # Child
try: try:
# release SQLite debug log
logging.setup()
signal.signal(signal.SIGTERM, lambda *args: sys.exit()) signal.signal(signal.SIGTERM, lambda *args: sys.exit())
if coverage: if coverage:
coverage.stop() coverage.stop()
...@@ -482,6 +480,7 @@ class NEOCluster(object): ...@@ -482,6 +480,7 @@ class NEOCluster(object):
master_nodes=master_nodes, master_nodes=master_nodes,
name=self.cluster_name, name=self.cluster_name,
**kw) **kw)
result.app.max_reconnection_to_master = 10
self.zodb_storage_list.append(result) self.zodb_storage_list.append(result)
return result return result
......
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
...@@ -71,7 +71,7 @@ class MasterStorageHandlerTests(NeoUnitTestBase): ...@@ -71,7 +71,7 @@ class MasterStorageHandlerTests(NeoUnitTestBase):
self.checkNoPacketSent(client_conn) self.checkNoPacketSent(client_conn)
self.assertEqual(self.app.packing[2], {conn2.getUUID()}) self.assertEqual(self.app.packing[2], {conn2.getUUID()})
self.service.answerPack(conn2, False) self.service.answerPack(conn2, False)
packet = self.checkAnswerPacket(client_conn, Packets.AnswerPack) packet = self.checkNotifyPacket(client_conn, Packets.AnswerPack)
# TODO: verify packet peer id # TODO: verify packet peer id
self.assertTrue(packet.decode()[0]) self.assertTrue(packet.decode()[0])
self.assertEqual(self.app.packing, None) self.assertEqual(self.app.packing, None)
......
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
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