Commit f9a0c181 authored by Kirill Smelkov's avatar Kirill Smelkov

Merge tag 'v1.11'

NEO 1.11

* tag 'v1.11': (52 commits)
  Release version 1.11
  Fix short descriptions of neoctl & neomigrate in their headers
  Update copyright year
  qa: new tool to stress-test NEO
  master: fix typo in comment
  Fix error handling when setting up a listening connector
  Fix incomplete/incorrect mapping of node ids in logs
  Fix log corruption on rotation in multi-threaded applications (e.g. client)
  sqlite: optimize storage of metadata
  neolog: do not die when a table is corrupted
  neolog: add support for zstd-compressed logs
  neolog: do not hardcode default value of -L option in help message
  fixup! New log format to show node id (and optionally cluster name) in node column
  New log format to show node id (and optionally cluster name) in node column
  fixup! client: discard late answers to lockless writes
  client: fix race condition between Storage.load() and invalidations
  client: fix race condition in refcounting dispatched answer packets
  More RTMIN+2 (log) information for clients and connections
  storage: check for conflicts when notifying that the a partition is replicated
  storage: clarify several assertions
  qa: new expectedFailure testcase method
  client: merge ConnectionPool inside Application
  client: prepare merge of ConnectionPool inside Application
  client: fix AssertionError when trying to reconnect too quickly after an error
  qa: fix attributeTracker
  storage: fix storage leak when an oid is stored several times within a transaction
  client: discard late answers to lockless writes
  qa: in threaded tests, log queued packets when "tic is looping forever"
  In logs, dump the partition table in a more compact and readable way
  storage: fix write-locking bug when a deadlock happens at the end of a replication
  client: log_flush most exceptions raised from Application to ZODB
  client: fix assertion failure in case of conflict + storage disconnection
  client: simplify connection management in transaction contexts
  client: also vote to nodes that only check serials
  qa: deindent code
  Bump protocol version
  client: fix undetected disconnections to storage nodes during commit
  Fix data corruption due to undetected conflicts after storage failures
  master: notify replicating nodes of aborted watched transactions
  New neoctl command to flush the logs of all nodes in the cluster
  storage: fix premature write-locking during rebase when replication ends
  client: fix race condition when a storage connection is closed just after identification
  storage: relax assertion
  comments, unused import
  storage: fix write-lock leak
  client: fix possible corruption in case of network failure with a storage
  qa: comment about potential freeze when a functional test ends
  storage: fix assertion failure in case of connection reset with a client node
  qa: document a rare random failure in testExport
  debug: add script to trace all accesses to the client cache
  Use argparse instead of optparse
  neolog: use argparse instead of optparse
  Add comment about dormant bug when sending a lot of data to a slow node
  client: make clearer that max_size attribute is used from outside ClientCache
parents b3e24d58 48d936cb
......@@ -6,5 +6,6 @@
/build/
/dist/
/htmlcov/
/neo/tests/ConflictFree.py
/neo/tests/mock.py
/neoppod.egg-info/
Change History
==============
1.11 (2019-03-11)
-----------------
This release continues the work in v1.8 to stabilize NEO. A new 'stress'
tool was added: it kills storage nodes and resets TCP connections randomly,
while causing high concurrency activity. It revealed many bugs of all kinds,
including crashes and corruptions. Most of them happened after network
disconnection. In order to fix them all, several improvements have also been
done to logging:
- New neoctl command to flush the logs of all nodes in the cluster.
- In logs, dump the partition table in a more compact and readable way.
- client: log_flush most exceptions raised from Application to ZODB
- More RTMIN+2 (log) information for clients and connections.
- New log format to show node id (and optionally cluster name) in node column.
- neolog: add support for zstd-compressed logs.
- neolog: do not die when a table is corrupted.
Other changes:
- sqlite: optimize storage of metadata (the speed up in v1.9 about indexing
'obj' primarily by 'oid' was only effective for MySQL).
- Fix error handling when setting up a listening connector.
- The command line parsing of all executables has been completely rewritten,
fixing a few minor bugs.
1.10 (2018-07-16)
-----------------
......
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -15,7 +15,7 @@
# along with this program. If not, see <http://www.gnu.org/licenses/>.
from neo.lib import logging
from neo.lib.app import BaseApplication
from neo.lib.app import BaseApplication, buildOptionParser
from neo.lib.connection import ListeningConnection
from neo.lib.exception import PrimaryFailure
from .handler import AdminEventHandler, MasterEventHandler, \
......@@ -25,24 +25,36 @@ from neo.lib.pt import PartitionTable
from neo.lib.protocol import ClusterStates, Errors, NodeTypes, Packets
from neo.lib.debug import register as registerLiveDebugger
@buildOptionParser
class Application(BaseApplication):
"""The storage node application."""
@classmethod
def _buildOptionParser(cls):
_ = cls.option_parser
_.description = "NEO Admin node"
cls.addCommonServerOptions('admin', '127.0.0.1:9999')
_ = _.group('admin')
_.int('u', 'uuid',
help="specify an UUID to use for this process (testing purpose)")
def __init__(self, config):
super(Application, self).__init__(
config.getSSL(), config.getDynamicMasterList())
for address in config.getMasters():
config.get('ssl'), config.get('dynamic_master_list'))
for address in config['masters']:
self.nm.createMaster(address=address)
self.name = config.getCluster()
self.server = config.getBind()
self.name = config['cluster']
self.server = config['bind']
logging.debug('IP address is %s, port is %d', *self.server)
# The partition table is initialized after getting the number of
# partitions.
self.pt = None
self.uuid = config.getUUID()
self.uuid = config.get('uuid')
logging.node(self.name, self.uuid)
self.request_handler = MasterRequestEventHandler(self)
self.master_event_handler = MasterEventHandler(self)
self.cluster_state = None
......
#
# Copyright (C) 2009-2017 Nexedi SA
# Copyright (C) 2009-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -62,6 +62,11 @@ class AdminEventHandler(EventHandler):
master_node = self.app.master_node
conn.answer(Packets.AnswerPrimary(master_node.getUUID()))
@check_primary_master
def flushLog(self, conn):
self.app.master_conn.send(Packets.FlushLog())
super(AdminEventHandler, self).flushLog(conn)
askLastIDs = forward_ask(Packets.AskLastIDs)
askLastTransaction = forward_ask(Packets.AskLastTransaction)
addPendingNodes = forward_ask(Packets.AddPendingNodes)
......
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -15,6 +15,7 @@
# along with this program. If not, see <http://www.gnu.org/licenses/>.
from ZODB import BaseStorage, ConflictResolution, POSException
from ZODB.POSException import ConflictError, UndoError
from zope.interface import implementer
import ZODB.interfaces
......@@ -81,31 +82,68 @@ class Storage(BaseStorage.BaseStorage,
return self.app.load(oid)[:2]
except NEOStorageNotFoundError:
raise POSException.POSKeyError(oid)
except Exception:
logging.exception('oid=%r', oid)
raise
def new_oid(self):
try:
return self.app.new_oid()
except Exception:
logging.exception('')
raise
def tpc_begin(self, transaction, tid=None, status=' '):
"""
Note: never blocks in NEO.
"""
try:
return self.app.tpc_begin(self, transaction, tid, status)
except Exception:
logging.exception('transaction=%r, tid=%r', transaction, tid)
raise
def tpc_vote(self, transaction):
try:
return self.app.tpc_vote(transaction)
except ConflictError:
raise
except Exception:
logging.exception('transaction=%r', transaction)
raise
def tpc_abort(self, transaction):
try:
return self.app.tpc_abort(transaction)
except Exception:
logging.exception('transaction=%r', transaction)
raise
def tpc_finish(self, transaction, f=None):
try:
return self.app.tpc_finish(transaction, f)
except Exception:
logging.exception('transaction=%r', transaction)
raise
def store(self, oid, serial, data, version, transaction):
assert version == '', 'Versions are not supported'
try:
return self.app.store(oid, serial, data, version, transaction)
except ConflictError:
raise
except Exception:
logging.exception('oid=%r, serial=%r, transaction=%r',
oid, serial, transaction)
raise
def deleteObject(self, oid, serial, transaction):
try:
self.app.store(oid, serial, None, None, transaction)
except Exception:
logging.exception('oid=%r, serial=%r, transaction=%r',
oid, serial, transaction)
raise
# multiple revisions
def loadSerial(self, oid, serial):
......@@ -113,6 +151,9 @@ class Storage(BaseStorage.BaseStorage,
return self.app.load(oid, serial)[0]
except NEOStorageNotFoundError:
raise POSException.POSKeyError(oid)
except Exception:
logging.exception('oid=%r, serial=%r', oid, serial)
raise
def loadBefore(self, oid, tid):
try:
......@@ -121,6 +162,9 @@ class Storage(BaseStorage.BaseStorage,
raise POSException.POSKeyError(oid)
except NEOStorageNotFoundError:
return None
except Exception:
logging.exception('oid=%r, tid=%r', oid, tid)
raise
@property
def iterator(self):
......@@ -128,10 +172,20 @@ class Storage(BaseStorage.BaseStorage,
# undo
def undo(self, transaction_id, txn):
try:
return self.app.undo(transaction_id, txn)
except (ConflictError, UndoError):
raise
except Exception:
logging.exception('transaction_id=%r, txn=%r', transaction_id, txn)
raise
def undoLog(self, first=0, last=-20, filter=None):
try:
return self.app.undoLog(first, last, filter)
except Exception:
logging.exception('first=%r, last=%r', first, last)
raise
def supportsUndo(self):
return True
......@@ -141,10 +195,17 @@ class Storage(BaseStorage.BaseStorage,
data, serial, _ = self.app.load(oid)
except NEOStorageNotFoundError:
raise POSException.POSKeyError(oid)
except Exception:
logging.exception('oid=%r', oid)
raise
return data, serial, ''
def __len__(self):
try:
return self.app.getObjectCount()
except Exception:
logging.exception('')
raise
def registerDB(self, db, limit=None):
self.app.registerDB(db, limit)
......@@ -154,19 +215,30 @@ class Storage(BaseStorage.BaseStorage,
return self.app.history(oid, *args, **kw)
except NEOStorageNotFoundError:
raise POSException.POSKeyError(oid)
except Exception:
logging.exception('oid=%r', oid)
raise
def sync(self):
return self.app.sync()
def copyTransactionsFrom(self, source, verbose=False):
""" Zope compliant API """
try:
return self.app.importFrom(self, source)
except Exception:
logging.exception('source=%r', source)
raise
def pack(self, t, referencesf, gc=False):
if gc:
logging.warning('Garbage Collection is not available in NEO,'
' please use an external tool. Packing without GC.')
try:
self.app.pack(t)
except Exception:
logging.exception('pack_time=%r', t)
raise
def lastSerial(self):
# seems unused
......@@ -198,6 +270,14 @@ class Storage(BaseStorage.BaseStorage,
return self.app.getLastTID(oid)
except NEOStorageNotFoundError:
raise KeyError
except Exception:
logging.exception('oid=%r', oid)
raise
def checkCurrentSerialInTransaction(self, oid, serial, transaction):
try:
self.app.checkCurrentSerialInTransaction(oid, serial, transaction)
except Exception:
logging.exception('oid=%r, serial=%r, transaction=%r',
oid, serial, transaction)
raise
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -15,6 +15,7 @@
# along with this program. If not, see <http://www.gnu.org/licenses/>.
import heapq
import random
import time
try:
......@@ -35,17 +36,21 @@ from neo.lib.protocol import NodeTypes, Packets, \
from neo.lib.util import makeChecksum, dump
from neo.lib.locking import Empty, Lock
from neo.lib.connection import MTClientConnection, ConnectionClosed
from neo.lib.exception import NodeNotReady
from .exception import (NEOStorageError, NEOStorageCreationUndoneError,
NEOStorageReadRetry, NEOStorageNotFoundError, NEOPrimaryMasterLost)
from .handlers import storage, master
from neo.lib.threaded_app import ThreadedApplication
from .cache import ClientCache
from .pool import ConnectionPool
from .transactions import TransactionContainer
from neo.lib.util import p64, u64, parseMasterList
CHECKED_SERIAL = object()
# How long before we might retry a connection to a node to which connection
# failed in the past.
MAX_FAILURE_AGE = 600
try:
from Signals.Signals import SignalHandler
except ImportError:
......@@ -68,7 +73,6 @@ class Application(ThreadedApplication):
name, **kw)
# Internal Attributes common to all thread
self._db = None
self.cp = ConnectionPool(self)
self.primary_master_node = None
self.trying_master_node = None
......@@ -102,6 +106,9 @@ class Application(ThreadedApplication):
# _connecting_to_master_node is used to prevent simultaneous master
# node connection attempts
self._connecting_to_master_node = Lock()
# same for storage nodes
self._connecting_to_storage_node = Lock()
self._node_failure_dict = {}
self.compress = getCompress(compress)
def __getattr__(self, attr):
......@@ -116,6 +123,8 @@ class Application(ThreadedApplication):
def log(self):
super(Application, self).log()
logging.info("%r", self._cache)
for txn_context in self._txn_container.itervalues():
logging.info("%r", txn_context)
@property
def txn_contexts(self):
......@@ -246,6 +255,53 @@ class Application(ThreadedApplication):
logging.info("Connected and ready")
return conn
def getStorageConnection(self, node):
conn = node._connection # XXX
if node.isRunning() if conn is None else not node._identified:
with self._connecting_to_storage_node:
conn = node._connection # XXX
if conn is None:
return self._connectToStorageNode(node)
return conn
def _connectToStorageNode(self, node):
if self.master_conn is None:
raise NEOPrimaryMasterLost
conn = MTClientConnection(self, self.storage_event_handler, node,
dispatcher=self.dispatcher)
p = Packets.RequestIdentification(NodeTypes.CLIENT,
self.uuid, None, self.name, (), self.id_timestamp)
try:
self._ask(conn, p, handler=self.storage_bootstrap_handler)
except ConnectionClosed:
logging.error('Connection to %r failed', node)
except NodeNotReady:
logging.info('%r not ready', node)
else:
logging.info('Connected %r', node)
# Make sure this node will be considered for the next reads
# even if there was a previous recent failure.
self._node_failure_dict.pop(node.getUUID(), None)
return conn
self._node_failure_dict[node.getUUID()] = time.time() + MAX_FAILURE_AGE
def getCellSortKey(self, cell, random=random.random):
# Prefer a node that didn't fail recently.
failure = self._node_failure_dict.get(cell.getUUID())
if failure:
if time.time() < failure:
# Or 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(cell.getUUID(), None)
# A random one, connected or not, is a trivial and quite efficient way
# to distribute the load evenly. On write accesses, a client connects
# to all nodes of touched cells, but before that, or if a client is
# specialized to only do read-only accesses, it should not limit
# itself to only use the first connected nodes.
return random()
def registerDB(self, db, limit):
self._db = db
......@@ -274,7 +330,6 @@ class Application(ThreadedApplication):
return int(u64(self.last_oid))
def _askStorageForRead(self, object_id, packet, askStorage=None):
cp = self.cp
pt = self.pt
# BBB: On Py2, it can be a subclass of bytes (binary from zodbpickle).
if isinstance(object_id, bytes):
......@@ -287,10 +342,10 @@ class Application(ThreadedApplication):
failed = 0
while 1:
cell_list = pt.getCellList(object_id, True)
cell_list.sort(key=cp.getCellSortKey)
cell_list.sort(key=self.getCellSortKey)
for cell in cell_list:
node = cell.getNode()
conn = cp.getConnForNode(node)
conn = self.getStorageConnection(node)
if conn is not None:
try:
return askStorage(conn, packet)
......@@ -355,8 +410,16 @@ class Application(ThreadedApplication):
if result:
return result
self._loading_oid = oid
self._loading_invalidated = []
finally:
release()
# While the cache lock is released, an arbitrary number of
# invalidations may be processed, for this oid or not. And at this
# precise moment, if both tid and before_tid are None (which is
# unlikely to happen with recent ZODB), self.last_tid can be any
# new tid. Since we can get any serial from storage, fixing
# next_tid requires to keep a list of all possible serials.
# When not bound to a ZODB Connection, load() may be the
# first method called and last_tid may still be None.
# This happens, for example, when opening the DB.
......@@ -368,12 +431,11 @@ class Application(ThreadedApplication):
acquire()
try:
if self._loading_oid:
# Common case (no race condition).
self._cache.store(oid, data, tid, next_tid)
elif self._loading_invalidated:
# oid has just been invalidated.
if not next_tid:
next_tid = self._loading_invalidated
for t in self._loading_invalidated:
if tid < t:
next_tid = t
break
self._cache.store(oid, data, tid, next_tid)
# Else, we just reconnected to the master.
finally:
......@@ -445,9 +507,9 @@ class Application(ThreadedApplication):
packet = Packets.AskStoreObject(oid, serial, compression,
checksum, compressed_data, data_serial, ttid)
txn_context.data_dict[oid] = data, serial, txn_context.write(
self, packet, oid, oid=oid)
self, packet, oid, oid=oid, serial=serial)
while txn_context.data_size >= self._cache._max_size:
while txn_context.data_size >= self._cache.max_size:
self._waitAnyTransactionMessage(txn_context)
self._waitAnyTransactionMessage(txn_context, False)
......@@ -482,8 +544,7 @@ class Application(ThreadedApplication):
' with new locking TID %s', dump(ttid), dump(serial))
txn_context.locking_tid = serial
packet = Packets.AskRebaseTransaction(ttid, serial)
for uuid, status in txn_context.involved_nodes.iteritems():
if status < 2:
for uuid in txn_context.conn_dict:
self._askStorageForWrite(txn_context, uuid, packet)
else:
if data is CHECKED_SERIAL:
......@@ -515,15 +576,14 @@ class Application(ThreadedApplication):
self._store(txn_context, oid, serial, data)
def _askStorageForWrite(self, txn_context, uuid, packet):
node = self.nm.getByUUID(uuid)
if node is not None:
conn = self.cp.getConnForNode(node)
if conn is not None:
conn = txn_context.conn_dict[uuid]
try:
return conn.ask(packet, queue=txn_context.queue)
except AttributeError:
if conn is not None:
raise
except ConnectionClosed:
pass
txn_context.involved_nodes[uuid] = 2
txn_context.conn_dict[uuid] = None
def waitResponses(self, queue):
"""Wait for all requests to be answered (or their connection to be
......@@ -554,21 +614,39 @@ class Application(ThreadedApplication):
packet = Packets.AskStoreTransaction(ttid, str(transaction.user),
str(transaction.description), ext, txn_context.cache_dict)
queue = txn_context.queue
involved_nodes = txn_context.involved_nodes
conn_dict = txn_context.conn_dict
# Ask in parallel all involved storage nodes to commit object metadata.
# Nodes that store the transaction metadata get a special packet.
trans_nodes = txn_context.write(self, packet, ttid)
packet = Packets.AskVoteTransaction(ttid)
for uuid, status in involved_nodes.iteritems():
if status == 1 and uuid not in trans_nodes:
for uuid in conn_dict:
if uuid not in trans_nodes:
self._askStorageForWrite(txn_context, uuid, packet)
self.waitResponses(txn_context.queue)
# If there are failed nodes, ask the master whether they can be
# disconnected while keeping the cluster operational. If possible,
# this will happen during tpc_finish.
failed = [node.getUUID()
self.waitStoreResponses(txn_context)
if None in conn_dict.itervalues(): # unlikely
# If some writes failed, we must first check whether
# all oids have been locked by at least one node.
failed = {node.getUUID(): node.isRunning()
for node in self.nm.getStorageList()
if node.isRunning() and involved_nodes.get(node.getUUID()) == 2]
if conn_dict.get(node.getUUID(), 0) is None}
if txn_context.lockless_dict:
getCellList = self.pt.getCellList
for offset, uuid_set in txn_context.lockless_dict.iteritems():
for cell in getCellList(offset):
uuid = cell.getUUID()
if not (uuid in failed or uuid in uuid_set):
break
else:
# Very unlikely. Instead of raising, we could recover
# the transaction by doing something similar to
# deadlock avoidance; that would be done before voting.
# But it's not worth the complexity.
raise NEOStorageError(
'partition %s not fully write-locked' % offset)
failed = [uuid for uuid, running in failed.iteritems() if running]
# If there are running nodes for which some writes failed, ask the
# master whether they can be disconnected while keeping the cluster
# operational. If possible, this will happen during tpc_finish.
if failed:
try:
self._askPrimary(Packets.FailedVote(ttid, failed))
......@@ -600,10 +678,11 @@ class Application(ThreadedApplication):
# condition. The consequence would be that storage nodes lock oids
# forever.
p = Packets.AbortTransaction(txn_context.ttid, ())
for uuid in txn_context.involved_nodes:
for conn in txn_context.conn_dict.itervalues():
if conn is not None:
try:
self.cp.connection_dict[uuid].send(p)
except (KeyError, ConnectionClosed):
conn.send(p)
except ConnectionClosed:
pass
# Because we want to be sure that the involved nodes are notified,
# we still have to send the full list to the master. Most of the
......@@ -618,7 +697,7 @@ class Application(ThreadedApplication):
else:
try:
notify(Packets.AbortTransaction(txn_context.ttid,
txn_context.involved_nodes))
txn_context.conn_dict))
except ConnectionClosed:
pass
# We don't need to flush queue, as it won't be reused by future
......@@ -708,8 +787,8 @@ class Application(ThreadedApplication):
# Ask storage the undo serial (serial at which object's previous data
# is)
getCellList = self.pt.getCellList
getCellSortKey = self.cp.getCellSortKey
getConnForNode = self.cp.getConnForNode
getCellSortKey = self.getCellSortKey
getConnForNode = self.getStorageConnection
queue = self._thread_container.queue
ttid = txn_context.ttid
undo_object_tid_dict = {}
......@@ -727,7 +806,7 @@ class Application(ThreadedApplication):
# conflicts. For example, if a network failure happened
# only between the client and the storage, the latter would
# still be readable until we commit.
if txn_context.involved_nodes.get(cell.getUUID(), 0) < 2]
if txn_context.conn_dict.get(cell.getUUID(), 0) is not None]
storage_conn = getConnForNode(
min(cell_list, key=getCellSortKey).getNode())
storage_conn.ask(Packets.AskObjectUndoSerial(ttid,
......@@ -799,7 +878,7 @@ class Application(ThreadedApplication):
packet = Packets.AskTIDs(first, last, INVALID_PARTITION)
tid_set = set()
for storage_node in self.pt.getNodeSet(True):
conn = self.cp.getConnForNode(storage_node)
conn = self.getStorageConnection(storage_node)
if conn is None:
continue
conn.ask(packet, queue=queue, tid_set=tid_set)
......@@ -934,6 +1013,6 @@ class Application(ThreadedApplication):
assert oid not in txn_context.data_dict, oid
packet = Packets.AskCheckCurrentSerial(ttid, oid, serial)
txn_context.data_dict[oid] = CHECKED_SERIAL, serial, txn_context.write(
self, packet, oid, 0, oid=oid)
self, packet, oid, oid=oid, serial=serial)
self._waitAnyTransactionMessage(txn_context, False)
#
# Copyright (C) 2011-2017 Nexedi SA
# Copyright (C) 2011-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -64,7 +64,7 @@ class ClientCache(object):
- The history queue only contains items with counter > 0
"""
__slots__ = ('_life_time', '_max_history_size', '_max_size',
__slots__ = ('max_size', '_life_time', '_max_history_size',
'_queue_list', '_oid_dict', '_time', '_size', '_history_size',
'_nhit', '_nmiss')
......@@ -72,7 +72,7 @@ class ClientCache(object):
max_size=20*1024*1024):
self._life_time = life_time
self._max_history_size = max_history_size
self._max_size = max_size
self.max_size = max_size
self.clear()
def clear(self):
......@@ -94,7 +94,7 @@ class ClientCache(object):
[self._history_size] + [
sum(1 for _ in self._iterQueue(level))
for level in xrange(1, len(self._queue_list))],
self._life_time, self._max_history_size, self._max_size)
self._life_time, self._max_history_size, self.max_size)
def _iterQueue(self, level):
"""for debugging purpose"""
......@@ -168,7 +168,7 @@ class ClientCache(object):
# XXX It might be better to adjust the level according to the object
# size. See commented factor for example.
item.level = 1 + int(_log(counter, 2)
# * (1.01 - len(item.data) / self._max_size)
# * (1.01 - len(item.data) / self.max_size)
)
self._add(item)
......@@ -212,7 +212,7 @@ class ClientCache(object):
def store(self, oid, data, tid, next_tid):
"""Store a new data record in the cache"""
size = len(data)
max_size = self._max_size
max_size = self.max_size
if size < max_size:
item = self._load(oid, next_tid)
if item:
......@@ -331,7 +331,7 @@ def test(self):
# Test late invalidations.
cache.clear()
cache.store(1, '10*', 10, None)
cache._max_size = cache._size
cache.max_size = cache._size
cache.store(2, '10', 10, 15)
self.assertEqual(cache._queue_list[0].oid, 1)
cache.store(2, '15', 15, None)
......
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -127,8 +127,7 @@ class PrimaryNotificationsHandler(MTEventHandler):
for oid in oid_list:
invalidate(oid, tid)
if oid == loading:
app._loading_oid = None
app._loading_invalidated = tid
app._loading_invalidated.append(tid)
db = app.getDB()
if db is not None:
db.invalidate(tid, oid_list)
......
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -18,7 +18,8 @@ from ZODB.TimeStamp import TimeStamp
from neo.lib import logging
from neo.lib.compress import decompress_list
from neo.lib.protocol import Packets, uuid_str
from neo.lib.connection import ConnectionClosed
from neo.lib.protocol import Packets, uuid_str, ZERO_TID
from neo.lib.util import dump, makeChecksum
from neo.lib.exception import NodeNotReady
from neo.lib.handler import MTEventHandler
......@@ -29,19 +30,6 @@ from ..exception import NEOStorageReadRetry, NEOStorageDoesNotExistError
class StorageEventHandler(MTEventHandler):
def connectionLost(self, conn, new_state):
node = self.app.nm.getByAddress(conn.getAddress())
assert node is not None
self.app.cp.removeConnection(node)
super(StorageEventHandler, self).connectionLost(conn, new_state)
def connectionFailed(self, conn):
# Connection to a storage node failed
node = self.app.nm.getByAddress(conn.getAddress())
assert node is not None
self.app.cp.removeConnection(node)
super(StorageEventHandler, self).connectionFailed(conn)
def _acceptIdentification(*args):
pass
......@@ -58,9 +46,12 @@ class StorageAnswersHandler(AnswerBaseHandler):
def answerObject(self, conn, oid, *args):
self.app.setHandlerData(args)
def answerStoreObject(self, conn, conflict, oid):
def answerStoreObject(self, conn, conflict, oid, serial):
txn_context = self.app.getHandlerData()
if conflict:
if conflict == ZERO_TID:
txn_context.written(self.app, conn.getUUID(), oid, serial)
return
# Conflicts can not be resolved now because 'conn' is locked.
# We must postpone the resolution (by queuing the conflict in
# 'conflict_dict') to avoid any deadlock with another thread that
......@@ -94,7 +85,7 @@ class StorageAnswersHandler(AnswerBaseHandler):
conn.ask(Packets.AskRebaseObject(ttid, oid),
queue=queue, oid=oid)
except ConnectionClosed:
txn_context.involved_nodes[conn.getUUID()] = 2
txn_context.conn_dict[conn.getUUID()] = None
def answerRebaseObject(self, conn, conflict, oid):
if conflict:
......@@ -106,8 +97,10 @@ class StorageAnswersHandler(AnswerBaseHandler):
cached = txn_context.cache_dict.pop(oid)
except KeyError:
if resolved:
# We should still be waiting for an answer from this node.
assert conn.uuid in txn_context.data_dict[oid][2]
# We should still be waiting for an answer from this node,
# unless we lost connection.
assert conn.uuid in txn_context.data_dict[oid][2] or \
txn_context.conn_dict[conn.uuid] is None
return
assert oid in txn_context.data_dict
if serial <= txn_context.conflict_dict.get(oid, ''):
......@@ -135,7 +128,7 @@ class StorageAnswersHandler(AnswerBaseHandler):
if cached:
assert cached == data
txn_context.cache_size -= size
txn_context.data_dict[oid] = data, serial, None
txn_context.data_dict[oid] = data, serial, []
txn_context.conflict_dict[oid] = conflict
def answerStoreTransaction(self, conn):
......@@ -144,6 +137,7 @@ class StorageAnswersHandler(AnswerBaseHandler):
answerVoteTransaction = answerStoreTransaction
def connectionClosed(self, conn):
# only called if we were waiting for an answer
txn_context = self.app.getHandlerData()
if type(txn_context) is Transaction:
txn_context.nodeLost(self.app, conn.getUUID())
......
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2006-2017 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
# as published by the Free Software Foundation; either version 2
# of the License, or (at your option) any later version.
#
# This program is distributed in the hope that it will be useful,
# but WITHOUT ANY WARRANTY; without even the implied warranty of
# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
# GNU General Public License for more details.
#
# You should have received a copy of the GNU General Public License
# along with this program. If not, see <http://www.gnu.org/licenses/>.
import random, time
from neo.lib import logging
from neo.lib.locking import Lock
from neo.lib.protocol import NodeTypes, Packets
from neo.lib.connection import MTClientConnection, ConnectionClosed
from neo.lib.exception import NodeNotReady
from .exception import NEOPrimaryMasterLost
# How long before we might retry a connection to a node to which connection
# failed in the past.
MAX_FAILURE_AGE = 600
class ConnectionPool(object):
"""This class manages a pool of connections to storage nodes."""
def __init__(self, app):
self.app = app
self.connection_dict = {}
# Define a lock in order to create one connection to
# a storage node at a time to avoid multiple connections
# to the same node.
self._lock = Lock()
self.node_failure_dict = {}
def _initNodeConnection(self, node):
"""Init a connection to a given storage node."""
app = self.app
if app.master_conn is None:
raise NEOPrimaryMasterLost
conn = MTClientConnection(app, app.storage_event_handler, node,
dispatcher=app.dispatcher)
p = Packets.RequestIdentification(NodeTypes.CLIENT,
app.uuid, None, app.name, (), app.id_timestamp)
try:
app._ask(conn, p, handler=app.storage_bootstrap_handler)
except ConnectionClosed:
logging.error('Connection to %r failed', node)
except NodeNotReady:
logging.info('%r not ready', node)
else:
logging.info('Connected %r', node)
# Make sure this node will be considered for the next reads
# even if there was a previous recent failure.
self.node_failure_dict.pop(node.getUUID(), None)
return conn
self.node_failure_dict[node.getUUID()] = time.time() + MAX_FAILURE_AGE
def getCellSortKey(self, cell, random=random.random):
# Prefer a node that didn't fail recently.
failure = self.node_failure_dict.get(cell.getUUID())
if failure:
if time.time() < failure:
# Or 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(cell.getUUID(), None)
# A random one, connected or not, is a trivial and quite efficient way
# to distribute the load evenly. On write accesses, a client connects
# to all nodes of touched cells, but before that, or if a client is
# specialized to only do read-only accesses, it should not limit
# itself to only use the first connected nodes.
return random()
def getConnForNode(self, node):
"""Return a locked connection object to a given node
If no connection exists, create a new one"""
if node.isRunning():
uuid = node.getUUID()
try:
# Already connected to node
return self.connection_dict[uuid]
except KeyError:
with self._lock:
# Second lookup, if another thread initiated connection
# while we were waiting for connection lock.
try:
return self.connection_dict[uuid]
except KeyError:
# Create new connection to node
conn = self._initNodeConnection(node)
if conn is not None:
self.connection_dict[uuid] = conn
return conn
def removeConnection(self, node):
self.connection_dict.pop(node.getUUID(), None)
def closeAll(self):
with self._lock:
while 1:
try:
conn = self.connection_dict.popitem()[1]
except KeyError:
break
conn.setReconnectionNoDelay()
conn.close()
#
# Copyright (C) 2017 Nexedi SA
# Copyright (C) 2017-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -14,10 +14,12 @@
# You should have received a copy of the GNU General Public License
# along with this program. If not, see <http://www.gnu.org/licenses/>.
from collections import defaultdict
from ZODB.POSException import StorageTransactionError
from neo.lib.connection import ConnectionClosed
from neo.lib.locking import SimpleQueue
from neo.lib.protocol import Packets
from neo.lib.util import dump
from .exception import NEOStorageError
@apply
......@@ -35,6 +37,7 @@ class Transaction(object):
locking_tid = None
voted = False
ttid = None # XXX: useless, except for testBackupReadOnlyAccess
lockless_dict = None # {partition: {uuid}}
def __init__(self, txn):
self.queue = SimpleQueue()
......@@ -47,72 +50,85 @@ class Transaction(object):
self.conflict_dict = {} # {oid: 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}
# involved storage nodes; connection is None is connection was lost
self.conn_dict = {} # {node_id: connection}
def __repr__(self):
error = self.error
return ("<%s ttid=%s locking_tid=%s voted=%u"
" #queue=%s #writing=%s #written=%s%s>") % (
self.__class__.__name__,
dump(self.ttid), dump(self.locking_tid), self.voted,
len(self.queue._queue), len(self.data_dict), len(self.cache_dict),
' error=%r' % error if error else '')
def wakeup(self, conn):
self.queue.put((conn, _WakeupPacket, {}))
def write(self, app, packet, object_id, store=1, **kw):
def write(self, app, packet, object_id, **kw):
uuid_list = []
pt = app.pt
involved = self.involved_nodes
conn_dict = self.conn_dict
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:
try:
conn = conn_dict[uuid]
except KeyError:
conn = conn_dict[uuid] = app.getStorageConnection(node)
if 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.
# for the other nodes. The condition on kw is to
# distinguish whether we're writing an oid or
# 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 AttributeError:
if conn is not None:
raise
except ConnectionClosed:
pass
involved[uuid] = 2
conn_dict[uuid] = None
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
def written(self, app, uuid, oid, lockless=None):
# When a node 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, serial, uuid_list = self.data_dict[oid]
try:
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
# tries all oids blindly.
# Another possible case is when we receive several positive answers
# from a node that is being disconnected by the master, whereas the
# first one (at least) should actually be conflict answer.
return
if lockless:
if lockless != serial: # late lockless write
assert lockless < serial, (lockless, serial)
uuid_list.append(uuid)
return
# It's safe to do this after the above excepts: either the cell is
# already marked as lockless or the node will be reported as failed.
lockless = self.lockless_dict
if not lockless:
lockless = self.lockless_dict = defaultdict(set)
lockless[app.pt.getPartition(oid)].add(uuid)
if oid in self.conflict_dict:
# In the case of a rebase, uuid_list may not contain the id
# of the node reporting a conflict.
return
if uuid_list:
return
......@@ -121,7 +137,7 @@ class Transaction(object):
size = len(data)
self.data_size -= size
size += self.cache_size
if size < app._cache._max_size:
if size < app._cache.max_size:
self.cache_size = size
else:
# Do not cache data past cache max size, as it
......@@ -131,8 +147,13 @@ class Transaction(object):
self.cache_dict[oid] = data
def nodeLost(self, app, uuid):
self.involved_nodes[uuid] = 2
# The following line is sometimes redundant
# with the one in `except ConnectionClosed:` clauses.
self.conn_dict[uuid] = None
for oid in list(self.data_dict):
# Exclude case of 1 conflict error immediately followed by a
# connection loss, possibly with lockless writes to replicas.
if oid not in self.conflict_dict:
self.written(app, uuid, oid)
......
#
# Copyright (C) 2017 Nexedi SA
# Copyright (C) 2017-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
......@@ -45,50 +45,12 @@ if IF == 'pdb':
#('ZPublisher.Publish', 'publish_module_standard'),
)
import errno, socket, threading, weakref
# Unfortunately, IPython does not always print to given stdout.
#from neo.lib.debug import getPdb
import socket, threading, weakref
from neo.lib.debug import PdbSocket
# We don't use the one from neo.lib.debug because unfortunately,
# IPython does not always print to given stdout.
from pdb import Pdb as getPdb
class Socket(object):
def __init__(self, socket):
# In case that the default timeout is not None.
socket.settimeout(None)
self._socket = socket
self._buf = ''
def write(self, data):
self._socket.send(data)
def readline(self):
recv = self._socket.recv
data = self._buf
while True:
i = 1 + data.find('\n')
if i:
self._buf = data[i:]
return data[:i]
d = recv(4096)
data += d
if not d:
self._buf = ''
return data
def flush(self):
pass
def closed(self):
self._socket.setblocking(0)
try:
self._socket.recv(0)
return True
except socket.error, (err, _):
if err != errno.EAGAIN:
raise
self._socket.setblocking(1)
return False
def pdb():
s = socket.socket(socket.AF_INET, socket.SOCK_STREAM)
try:
......@@ -98,7 +60,7 @@ if IF == 'pdb':
s.listen(0)
print 'Listening to %u' % s.getsockname()[1]
sys.stdout.flush() # BBB: On Python 3, print() takes a 'flush' arg.
_socket = Socket(s.accept()[0])
_socket = PdbSocket(s.accept()[0])
finally:
s.close()
try:
......@@ -155,9 +117,12 @@ if IF == 'pdb':
if BP:
setupBreakPoints(BP)
else:
threading.Thread(target=pdb).start()
threading.Thread(target=pdb, name='pdb').start()
elif IF == 'frames':
# WARNING: Because of https://bugs.python.org/issue17094, the output is
# usually incorrect for subprocesses started by the functional
# test framework.
import traceback
write = sys.stderr.write
for thread_id, frame in sys._current_frames().iteritems():
......@@ -178,3 +143,68 @@ elif IF == 'profile':
prof = cProfile.Profile()
threading.Timer(DURATION, stop, (prof, path)).start()
prof.enable()
elif IF == 'trace-cache':
from struct import Struct
from .client.cache import ClientCache
from .lib.protocol import uuid_str, ZERO_TID as z64
class CacheTracer(object):
_pack2 = Struct('!B8s8s').pack
_pack4 = Struct('!B8s8s8sL').pack
def __init__(self, cache, path):
self._cache = cache
self._trace_file = open(path, 'a')
def close(self):
self._trace_file.close()
return self._cache
def _trace(self, op, x, y=z64, z=z64):
self._trace_file.write(self._pack(op, x, y, z))
def __repr__(self):
return repr(self._cache)
@property
def max_size(self):
return self._cache.max_size
def clear(self):
self._trace_file.write('\0')
self._cache.clear()
def clear_current(self):
self._trace_file.write('\1')
self._cache.clear_current()
def load(self, oid, before_tid=None):
r = self._cache.load(oid, before_tid)
self._trace_file.write(self._pack2(
3 if r else 2, oid, before_tid or z64))
return r
def store(self, oid, data, tid, next_tid):
self._trace_file.write(self._pack4(
4, oid, tid, next_tid or z64, len(data)))
self._cache.store(oid, data, tid, next_tid)
def invalidate(self, oid, tid):
self._trace_file.write(self._pack2(5, oid, tid))
self._cache.invalidate(oid, tid)
@defer
def profile(app):
app._cache_lock_acquire()
try:
cache = app._cache
if type(cache) is ClientCache:
app._cache = CacheTracer(cache, '%s-%s.neo-cache-trace' %
(app.name, uuid_str(app.uuid)))
app._cache.clear()
else:
app._cache = cache.close()
finally:
app._cache_lock_release()
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2015-2017 Nexedi SA
# Copyright (C) 2015-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -14,16 +14,57 @@
# You should have received a copy of the GNU General Public License
# along with this program. If not, see <http://www.gnu.org/licenses/>.
from . import logging
from . import logging, util
from .config import OptionList
from .event import EventManager
from .node import NodeManager
def buildOptionParser(cls):
parser = cls.option_parser = cls.OptionList()
_ = parser.path
_('l', 'logfile',
help="log debugging information to specified SQLite DB")
_('ca', help="(SSL) certificate authority in PEM format")
_('cert', help="(SSL) certificate in PEM format")
_('key', help="(SSL) private key in PEM format")
cls._buildOptionParser()
return cls
class BaseApplication(object):
class OptionList(OptionList):
def parse(self, argv=None):
config = OptionList.parse(self, argv)
ssl = (
config.pop('ca', None),
config.pop('cert', None),
config.pop('key', None),
)
if any(ssl):
config['ssl'] = ssl
return config
server = None
ssl = None
@classmethod
def addCommonServerOptions(cls, section, bind, masters='127.0.0.1:10000'):
_ = cls.option_parser.group('server node')
_.path('f', 'file', help='specify a configuration file')
_('s', 'section', default=section,
help='specify a configuration section')
_('c', 'cluster', required=True, help='the cluster name')
_('m', 'masters', default=masters, parse=util.parseMasterList,
help='master node list')
_('b', 'bind', default=bind,
parse=lambda x: util.parseNodeAddress(x, 0),
help='the local address to bind to')
_.path('D', 'dynamic-master-list',
help='path of the file containing dynamic master node list')
def __init__(self, ssl=None, dynamic_master_list=None):
if ssl:
if not all(ssl):
......@@ -60,3 +101,8 @@ class BaseApplication(object):
self.nm.close()
self.em.close()
self.__dict__.clear()
def setUUID(self, uuid):
if self.uuid != uuid:
self.uuid = uuid
logging.node(self.name, uuid)
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2018 Nexedi SA
# Copyright (C) 2018-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -14,139 +14,187 @@
# You should have received a copy of the GNU General Public License
# along with this program. If not, see <http://www.gnu.org/licenses/>.
import os
from optparse import OptionParser
from ConfigParser import SafeConfigParser, NoOptionError
from . import util
from .util import parseNodeAddress
def getOptionParser():
parser = OptionParser()
parser.add_option('-l', '--logfile',
help='log debugging information to specified SQLite DB')
parser.add_option('--ca', help='certificate authority in PEM format')
parser.add_option('--cert', help='certificate in PEM format')
parser.add_option('--key', help='private key in PEM format')
return parser
def getServerOptionParser():
parser = getOptionParser()
parser.add_option('-f', '--file', help='specify a configuration file')
parser.add_option('-s', '--section', help='specify a configuration section')
parser.add_option('-c', '--cluster', help='the cluster name')
parser.add_option('-m', '--masters', help='master node list')
parser.add_option('-b', '--bind', help='the local address to bind to')
parser.add_option('-D', '--dynamic-master-list',
help='path of the file containing dynamic master node list')
return parser
class ConfigurationManager(object):
"""
Configuration manager that load options from a configuration file and
command line arguments
"""
def __init__(self, defaults, options, section):
self.argument_list = options = {k: v
for k, v in options.__dict__.iteritems()
if v is not None}
self.defaults = defaults
config_file = options.pop('file', None)
if config_file:
self.parser = SafeConfigParser(defaults)
self.parser.read(config_file)
else:
self.parser = None
self.section = options.pop('section', section)
def __get(self, key, optional=False):
value = self.argument_list.get(key)
if value is None:
if self.parser is None:
value = self.defaults.get(key)
import argparse, os, sys
from functools import wraps
from ConfigParser import SafeConfigParser
class _Required(object):
def __init__(self, *args):
self._option_list, self._name = args
def __nonzero__(self):
with_required = self._option_list._with_required
return with_required is not None and self._name not in with_required
class _Option(object):
def __init__(self, *args, **kw):
if len(args) > 1:
self.short, self.name = args
else:
self.name, = args
self.__dict__.update(kw)
def _asArgparse(self, parser, option_list):
kw = self._argument_kw()
args = ['--' + self.name]
try:
value = self.parser.get(self.section, key)
except NoOptionError:
args.insert(0, '-' + self.short)
except AttributeError:
pass
if value is None and not optional:
raise RuntimeError("Option '%s' is undefined'" % (key, ))
kw['help'] = self.help
action = parser.add_argument(*args, **kw)
if action.required:
assert not hasattr(self, 'default')
action.required = _Required(option_list, self.name)
def fromConfigFile(self, cfg, section):
return self(cfg.get(section, self.name.replace('-', '_')))
@staticmethod
def parse(value):
return value
class BoolOption(_Option):
def _argument_kw(self):
return {'action': 'store_true'}
def __call__(self, value):
return value
def __getPath(self, *args, **kw):
path = self.__get(*args, **kw)
if path:
return os.path.expanduser(path)
def fromConfigFile(self, cfg, section):
return cfg.getboolean(section, self.name)
def getLogfile(self):
return self.__getPath('logfile', True)
class Option(_Option):
def getSSL(self):
r = [self.__getPath(key, True) for key in ('ca', 'cert', 'key')]
if any(r):
return r
@property
def __name__(self):
return self.type.__name__
def getMasters(self):
""" Get the master node list except itself """
return util.parseMasterList(self.__get('masters'))
def _argument_kw(self):
kw = {'type': self}
for x in 'default', 'metavar', 'required', 'choices':
try:
kw[x] = getattr(self, x)
except AttributeError:
pass
return kw
@staticmethod
def type(value):
if value:
return value
raise argparse.ArgumentTypeError('value is empty')
def getBind(self):
""" Get the address to bind to """
bind = self.__get('bind')
return parseNodeAddress(bind, 0)
def __call__(self, value):
return self.type(value)
def getDisableDropPartitions(self):
return self.__get('disable_drop_partitions', True)
class OptionGroup(object):
def getDatabase(self):
return self.__get('database')
def __init__(self, description=None):
self.description = description
self._options = []
def getEngine(self):
return self.__get('engine', True)
def _asArgparse(self, parser, option_list):
g = parser.add_argument_group(self.description)
for option in self._options:
option._asArgparse(g, option_list)
def set_defaults(self, **kw):
option_dict = self.getOptionDict()
for k, v in kw.iteritems():
option_dict[k].default = v
def getOptionDict(self):
option_dict = {}
for option in self._options:
if isinstance(option, OptionGroup):
option_dict.update(option.getOptionDict())
else:
option_dict[option.name.replace('-', '_')] = option
return option_dict
def getWait(self):
# XXX: see also DatabaseManager.__init__
return self.__get('wait')
def __call__(self, *args, **kw):
self._options.append(Option(*args, **kw))
def getDynamicMasterList(self):
return self.__getPath('dynamic_master_list', optional=True)
def __option_type(t):
return wraps(t)(lambda self, *args, **kw: self(type=t, *args, **kw))
def getAdapter(self):
return self.__get('adapter')
float = __option_type(float)
int = __option_type(int)
path = __option_type(os.path.expanduser)
def getCluster(self):
cluster = self.__get('cluster')
assert cluster != '', "Cluster name must be non-empty"
return cluster
def bool(self, *args, **kw):
self._options.append(BoolOption(*args, **kw))
def getReplicas(self):
return int(self.__get('replicas'))
class Argument(Option):
def getPartitions(self):
return int(self.__get('partitions'))
def __init__(self, name, **kw):
super(Argument, self).__init__(name, **kw)
def getReset(self):
# only from command line
return self.argument_list.get('reset', False)
def _asArgparse(self, parser, option_list):
kw = {'help': self.help, 'type': self}
for x in 'default', 'metavar', 'nargs', 'choices':
try:
kw[x] = getattr(self, x)
except AttributeError:
pass
parser.add_argument(self.name, **kw)
def getUUID(self):
# only from command line
uuid = self.argument_list.get('uuid', None)
if uuid:
return int(uuid)
class OptionList(OptionGroup):
def getUpstreamCluster(self):
return self.__get('upstream_cluster', True)
_with_required = None
def getUpstreamMasters(self):
return util.parseMasterList(self.__get('upstream_masters'))
def argument(self, *args, **kw):
self._options.append(Argument(*args, **kw))
def getAutostart(self):
n = self.__get('autostart', True)
if n:
return int(n)
def group(self, description):
group = OptionGroup(description)
self._options.append(group)
return group
def getDedup(self):
return self.__get('dedup', True)
def parse(self, argv=None):
parser = argparse.ArgumentParser(description=self.description,
argument_default=argparse.SUPPRESS,
formatter_class=argparse.ArgumentDefaultsHelpFormatter)
for option in self._options:
option._asArgparse(parser, self)
_format_help = parser.format_help
def format_help():
self._with_required = ()
try:
return _format_help()
finally:
del self._with_required
parser.format_help = format_help
if argv is None:
argv = sys.argv[1:]
args = parser.parse_args(argv)
option_dict = self.getOptionDict()
try:
config_file = args.file
except AttributeError:
d = ()
else:
cfg = SafeConfigParser()
cfg.read(config_file)
section = args.section
d = {}
for name in cfg.options(section):
try:
option = option_dict[name]
except KeyError:
continue
d[name] = option.fromConfigFile(cfg, section)
parser.set_defaults(**d)
self._with_required = d
try:
args = parser.parse_args(argv)
finally:
del self._with_required
return {name: option.parse(getattr(args, name))
for name, option in option_dict.iteritems()
if hasattr(args, name)}
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -335,6 +335,7 @@ class Connection(BaseConnection):
return r, flags
def setOnClose(self, callback):
assert not self.isClosed(), self
self._on_close = callback
def isClient(self):
......
#
# Copyright (C) 2009-2017 Nexedi SA
# Copyright (C) 2009-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -34,6 +34,7 @@ class SocketConnector(object):
is_closed = is_server = None
connect_limit = {}
CONNECT_LIMIT = 1 # XXX actually this is (RE-)CONNECT_THROTTLE
KEEPALIVE = 60, 3, 10
SOMAXCONN = 5 # for threaded tests
def __new__(cls, addr, s=None):
......@@ -66,9 +67,10 @@ class SocketConnector(object):
# The following 3 lines are specific to Linux. It seems that OSX
# has similar options (TCP_KEEPALIVE/TCP_KEEPINTVL/TCP_KEEPCNT),
# and Windows has SIO_KEEPALIVE_VALS (fixed count of 10).
s.setsockopt(socket.IPPROTO_TCP, socket.TCP_KEEPIDLE, 60)
s.setsockopt(socket.IPPROTO_TCP, socket.TCP_KEEPCNT, 3)
s.setsockopt(socket.IPPROTO_TCP, socket.TCP_KEEPINTVL, 10)
idle, cnt, intvl = self.KEEPALIVE
s.setsockopt(socket.IPPROTO_TCP, socket.TCP_KEEPIDLE, idle)
s.setsockopt(socket.IPPROTO_TCP, socket.TCP_KEEPCNT, cnt)
s.setsockopt(socket.IPPROTO_TCP, socket.TCP_KEEPINTVL, intvl)
s.setsockopt(socket.SOL_SOCKET, socket.SO_KEEPALIVE, 1)
# disable Nagle algorithm to reduce latency
s.setsockopt(socket.IPPROTO_TCP, socket.TCP_NODELAY, 1)
......@@ -127,6 +129,7 @@ class SocketConnector(object):
self._bind(self.addr)
self.socket.listen(self.SOMAXCONN)
except socket.error, e:
self.is_closed = True
self.socket.close()
self._error('listen', e)
......@@ -174,7 +177,11 @@ class SocketConnector(object):
self._error('recv')
def send(self):
# XXX: unefficient for big packets
# XXX: Inefficient for big packets. In any case, we should make sure
# that 'msg' does not exceed 2GB with SSL (OverflowError).
# Before commit 1a064725b81a702a124d672dba2bcae498980c76,
# this happened when many big AddObject packets were sent
# for a single replication chunk.
msg = ''.join(self.queued)
if msg:
try:
......@@ -216,12 +223,13 @@ class SocketConnector(object):
def __repr__(self):
if self.is_closed is None:
state = 'never opened'
state = ', never opened'
else:
if self.is_closed:
state = 'closed '
state = ', closed '
else:
state = 'opened '
state = ' fileno %s %s, opened ' % (
self.socket_fd, self.getAddress())
if self.is_server is None:
state += 'listening'
else:
......@@ -230,9 +238,7 @@ class SocketConnector(object):
else:
state += 'to '
state += str(self.addr)
return '<%s at 0x%x fileno %s %s, %s>' % (self.__class__.__name__,
id(self), '?' if self.is_closed else self.socket_fd,
self.getAddress(), state)
return '<%s at 0x%x%s>' % (self.__class__.__name__, id(self), state)
class SocketConnectorIPv4(SocketConnector):
" Wrapper for IPv4 sockets"
......
#
# Copyright (C) 2010-2017 Nexedi SA
# Copyright (C) 2010-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -14,11 +14,7 @@
# You should have received a copy of the GNU General Public License
# along with this program. If not, see <http://www.gnu.org/licenses/>.
import traceback
import signal
import imp
import os
import sys
import errno, imp, os, signal, socket, sys, traceback
from functools import wraps
import neo
......@@ -82,9 +78,55 @@ def winpdb(depth=0):
os.abort()
def register(on_log=None):
try:
if on_log is not None:
@safe_handler
def on_log_signal(signum, signal):
on_log()
signal.signal(signal.SIGRTMIN+2, on_log_signal)
signal.signal(signal.SIGRTMIN+3, debugHandler)
except ValueError: # signal only works in main thread
pass
class PdbSocket(object):
def __init__(self, socket):
# In case that the default timeout is not None.
socket.settimeout(None)
self._socket = socket
self._buf = ''
def close(self):
self._socket.close()
def write(self, data):
self._socket.send(data)
def readline(self):
recv = self._socket.recv
data = self._buf
while True:
i = 1 + data.find('\n')
if i:
self._buf = data[i:]
return data[:i]
d = recv(4096)
data += d
if not d:
self._buf = ''
return data
def flush(self):
pass
def closed(self):
self._socket.setblocking(0)
try:
self._socket.recv(0)
return True
except socket.error, (err, _):
if err != errno.EAGAIN:
raise
self._socket.setblocking(1)
return False
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -87,13 +87,11 @@ class Dispatcher:
def unregister(self, conn):
""" Unregister a connection and put fake packet in queues to unlock
threads expecting responses from that connection """
notified_set = set()
_decrefQueue = self._decrefQueue
self.lock_acquire()
try:
message_table = self.message_table.pop(id(conn), EMPTY)
finally:
self.lock_release()
notified_set = set()
_decrefQueue = self._decrefQueue
for queue in message_table.itervalues():
if queue is NOBODY:
continue
......@@ -102,6 +100,8 @@ class Dispatcher:
queue.put((conn, _ConnectionClosed, EMPTY))
notified_set.add(queue_id)
_decrefQueue(queue)
finally:
self.lock_release()
@giant_lock
def forget_queue(self, queue, flush_queue=True):
......
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -314,6 +314,11 @@ class EpollEventManager(object):
for fd, conn in self.connection_dict.items():
logging.info(' %r: %r (pending=%r)', fd, conn,
conn in pending_set)
for request_dict, handler in conn._handlers._pending:
handler = handler.__class__.__name__
for msg_id, (klass, kw) in sorted(request_dict.items()):
logging.info(' #0x%04x %s (%s)', msg_id,
klass.__name__, handler)
# Default to EpollEventManager.
......
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -175,9 +175,7 @@ class EventHandler(object):
if your_uuid is None:
raise ProtocolError('No UUID supplied')
logging.info('connected to a primary master node')
if app.uuid != your_uuid:
app.uuid = your_uuid
logging.info('Got a new UUID: %s', uuid_str(your_uuid))
app.setUUID(your_uuid)
app.id_timestamp = None
elif node.getUUID() != uuid or app.uuid != your_uuid != None:
raise ProtocolError('invalid uuids')
......@@ -201,6 +199,9 @@ class EventHandler(object):
if not conn.client:
conn.close()
def flushLog(self, conn):
logging.flush()
# Error packet handlers.
def error(self, conn, code, message, **kw):
......
#
# Copyright (C) 2015-2017 Nexedi SA
# Copyright (C) 2015-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
......@@ -26,7 +26,7 @@ from Queue import Empty
class LockUser(object):
def __init__(self, message, level=0):
def __init__(self, message=None, level=0):
t = threading.currentThread()
ident = getattr(t, 'node_name', t.name)
# This class is instantiated from a place desiring to known what
......@@ -42,6 +42,7 @@ class LockUser(object):
# current Neo directory structure.
path = os.path.join('...', *path.split(os.path.sep)[-3:])
self.time = time()
if message is not None:
self.ident = "%s@%r %s:%s %s" % (
ident, self.time, path, line_number, line)
self.note(message)
......
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -22,6 +22,9 @@ from time import time
from traceback import format_exception
import bz2, inspect, neo, os, signal, sqlite3, sys, threading
from .util import nextafter
INF = float('inf')
# Stats for storage node of matrix test (py2.7:SQLite)
RECORD_SIZE = ( 234360832 # extra memory used
- 16777264 # sum of raw data ('msg' attribute)
......@@ -61,9 +64,8 @@ class NEOLogger(Logger):
self.parent = root = getLogger()
if not root.handlers:
root.addHandler(self.default_root_handler)
self._db = None
self._record_queue = deque()
self._record_size = 0
self.__reset()
self._nid_dict = {}
self._async = set()
l = threading.Lock()
self._acquire = l.acquire
......@@ -77,6 +79,12 @@ class NEOLogger(Logger):
self._release = _release
self.backlog()
def __reset(self):
self._db = None
self._node = {}
self._record_queue = deque()
self._record_size = 0
def __enter__(self):
self._acquire()
return self._db
......@@ -96,7 +104,7 @@ class NEOLogger(Logger):
if self._db is None:
return
q = self._db.execute
if not q("SELECT id FROM packet LIMIT 1").fetchone():
if not q("SELECT 1 FROM packet LIMIT 1").fetchone():
q("DROP TABLE protocol")
# DROP TABLE already replaced previous data with zeros,
# so VACUUM is not really useful. But here, it should be free.
......@@ -151,9 +159,7 @@ class NEOLogger(Logger):
if self._db is not None:
self._db.close()
if not filename:
self._db = None
self._record_queue.clear()
self._record_size = 0
self.__reset()
return
if filename:
self._db = sqlite3.connect(filename, check_same_thread=False)
......@@ -163,45 +169,52 @@ class NEOLogger(Logger):
if 1: # Not only when logging everything,
# but also for interoperability with logrotate.
q("PRAGMA journal_mode = MEMORY")
for t, columns in (('log', (
"level INTEGER NOT NULL",
"pathname TEXT",
"lineno INTEGER",
"msg TEXT",
)),
('packet', (
"msg_id INTEGER NOT NULL",
"code INTEGER NOT NULL",
"peer TEXT NOT NULL",
"body BLOB",
))):
if reset:
for t in 'log', 'packet':
q('DROP TABLE IF EXISTS ' + t)
q("""CREATE TABLE IF NOT EXISTS log (
id INTEGER PRIMARY KEY AUTOINCREMENT,
date REAL NOT NULL,
name TEXT,
level INTEGER NOT NULL,
pathname TEXT,
lineno INTEGER,
msg TEXT)
q('DROP TABLE IF EXISTS %s1' % t)
elif (2, 'name', 'TEXT', 0, None, 0) in q(
"PRAGMA table_info(%s)" % t):
q("ALTER TABLE %s RENAME TO %s1" % (t, t))
columns = (
"date REAL PRIMARY KEY",
"node INTEGER",
) + columns
q("CREATE TABLE IF NOT EXISTS %s (\n %s) WITHOUT ROWID"
% (t, ',\n '.join(columns)))
q("""CREATE TABLE IF NOT EXISTS protocol (
date REAL PRIMARY KEY,
text BLOB NOT NULL) WITHOUT ROWID
""")
q("""CREATE INDEX IF NOT EXISTS _log_i1 ON log(date)""")
q("""CREATE TABLE IF NOT EXISTS packet (
id INTEGER PRIMARY KEY AUTOINCREMENT,
date REAL NOT NULL,
q("""CREATE TABLE IF NOT EXISTS node (
id INTEGER PRIMARY KEY,
name TEXT,
msg_id INTEGER NOT NULL,
code INTEGER NOT NULL,
peer TEXT NOT NULL,
body BLOB)
""")
q("""CREATE INDEX IF NOT EXISTS _packet_i1 ON packet(date)""")
q("""CREATE TABLE IF NOT EXISTS protocol (
date REAL PRIMARY KEY NOT NULL,
text BLOB NOT NULL)
cluster TEXT,
nid INTEGER)
""")
with open(inspect.getsourcefile(p)) as p:
p = buffer(bz2.compress(p.read()))
for t, in q("SELECT text FROM protocol ORDER BY date DESC"):
if p == t:
break
else:
try:
t = self._record_queue[0].created
except IndexError:
t = time()
with self._db:
q("INSERT INTO protocol VALUES (?,?)", (t, p))
x = q("SELECT text FROM protocol ORDER BY date DESC LIMIT 1"
).fetchone()
if (x and x[0]) != p:
# In case of multithreading, we can have locally unsorted
# records so we can't find the oldest one (it may not be
# pushed to queue): let's use 0 on log rotation.
x = time() if x else 0
q("INSERT INTO protocol VALUES (?,?)", (x, p))
self._db.commit()
self._node = {x[1:]: x[0] for x in q("SELECT * FROM node")}
def setup(self, filename=None, reset=False):
with self:
......@@ -219,6 +232,20 @@ class NEOLogger(Logger):
return True
def _emit(self, r):
try:
nid = self._node[r._node]
except KeyError:
if r._node == (None, None, None):
nid = None
else:
try:
nid = 1 + max(x for x in self._node.itervalues()
if x is not None)
except ValueError:
nid = 0
self._db.execute("INSERT INTO node VALUES (?,?,?,?)",
(nid,) + r._node)
self._node[r._node] = nid
if type(r) is PacketRecord:
ip, port = r.addr
peer = ('%s %s ([%s]:%s)' if ':' in ip else '%s %s (%s:%s)') % (
......@@ -231,15 +258,22 @@ class NEOLogger(Logger):
"""
if msg is not None:
msg = buffer(msg)
self._db.execute("INSERT INTO packet VALUES (NULL,?,?,?,?,?,?)",
(r.created, r._name, r.msg_id, r.code, peer, msg))
q = "INSERT INTO packet VALUES (?,?,?,?,?,?)"
x = [r.created, nid, r.msg_id, r.code, peer, msg]
else:
pathname = os.path.relpath(r.pathname, *neo.__path__)
self._db.execute("INSERT INTO log VALUES (NULL,?,?,?,?,?,?)",
(r.created, r._name, r.levelno, pathname, r.lineno, r.msg))
q = "INSERT INTO log VALUES (?,?,?,?,?,?)"
x = [r.created, nid, r.levelno, pathname, r.lineno, r.msg]
while 1:
try:
self._db.execute(q, x)
break
except sqlite3.IntegrityError:
x[0] = nextafter(x[0], INF)
def _queue(self, record):
record._name = self.name and str(self.name)
name = self.name and str(self.name)
record._node = (name,) + self._nid_dict.get(name, (None, None))
self._acquire()
try:
if self._max_size is None:
......@@ -286,6 +320,18 @@ class NEOLogger(Logger):
addr=connection.getAddress(),
msg=body))
def node(self, *cluster_nid):
name = self.name and str(self.name)
prev = self._nid_dict.get(name)
if prev != cluster_nid:
from .protocol import uuid_str
self.info('Node ID: %s', uuid_str(cluster_nid[1]))
self._nid_dict[name] = cluster_nid
@property
def resetNids(self):
return self._nid_dict.clear
logging = NEOLogger()
signal.signal(signal.SIGRTMIN, lambda signum, frame: logging.flush())
......
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -118,8 +118,8 @@ class Node(object):
if connection.isServer():
self.setIdentified()
else:
assert force is not None, \
attributeTracker.whoSet(self, '_connection')
assert force is not None, (conn,
attributeTracker.whoSet(self, '_connection'))
# The test on peer_id is there to protect against buggy nodes.
# XXX: handler comparison does not cover all cases: there may
# be a pending handler change, which won't be detected, or a future
......@@ -130,6 +130,10 @@ class Node(object):
# the full-fledged functionality, and it is simpler this way.
if not force or conn.getPeerId() is not None or \
type(conn.getHandler()) is not type(connection.getHandler()):
# It may also happen in case of a network failure that is only
# noticed by the peer. We'd like to accept the new connection
# immediately but it's quite complicated. At worst (keepalive
# packets dropped), 'conn' will be closed in ~ 1 minute.
raise ProtocolError("already connected")
def on_closed():
self._connection = connection
......@@ -137,7 +141,6 @@ class Node(object):
self.setIdentified()
conn.setOnClose(on_closed)
conn.close()
assert not connection.isClosed(), connection
connection.setOnClose(self.onConnectionClosed)
def getConnection(self):
......
#
# Copyright (C) 2015-2017 Nexedi SA
# Copyright (C) 2015-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -22,7 +22,7 @@ from struct import Struct
# The protocol version must be increased whenever upgrading a node may require
# to upgrade other nodes. It is encoded as a 4-bytes big-endian integer and
# the high order byte 0 is different from TLS Handshake (0x16).
PROTOCOL_VERSION = 4
PROTOCOL_VERSION = 5
ENCODED_VERSION = Struct('!L').pack(PROTOCOL_VERSION)
# Avoid memory errors on corrupted data.
......@@ -1631,6 +1631,13 @@ class Truncate(Packet):
_answer = Error
class FlushLog(Packet):
"""
Request all nodes to flush their logs.
:nodes: ctl -> A -> M -> *
"""
_next_code = 0
def register(request, ignore_when_closed=None):
......@@ -1806,6 +1813,8 @@ class Packets(dict):
AddObject)
Truncate = register(
Truncate)
FlushLog = register(
FlushLog)
def Errors():
registry_dict = {}
......
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -15,6 +15,8 @@
# along with this program. If not, see <http://www.gnu.org/licenses/>.
import math
from collections import defaultdict
from functools import partial
from . import logging, protocol
from .locking import Lock
from .protocol import uuid_str, CellStates
......@@ -256,39 +258,32 @@ class PartitionTable(object):
def _format(self):
"""Help debugging partition table management.
Output sample:
pt: node 0: S1, R
pt: node 1: S2, R
pt: node 2: S3, R
pt: node 3: S4, R
pt: 00: .UU.|U..U|.UU.|U..U|.UU.|U..U|.UU.|U..U|.UU.|U..U|.UU.
pt: 11: U..U|.UU.|U..U|.UU.|U..U|.UU.|U..U|.UU.|U..U|.UU.|U..U
Here, there are 4 nodes in RUNNING state.
The first partition has 2 replicas in UP_TO_DATE state, on nodes 1 and
2 (nodes 0 and 3 are displayed as unused for that partition by
displaying a dot).
The first number on the left represents the number of the first
partition on the line (here, line length is 11 to keep the docstring
width under 80 column).
Output sample (np=48, nr=0, just after a 3rd node is added):
pt: 10v 20v 30v 40v
pt: S1 R U.FOF.U.FOF.U.FOF.U.FOF.U.FOF.U.FOF.U.FOF.U.FOF.
pt: S2 R .U.FOF.U.FOF.U.FOF.U.FOF.U.FOF.U.FOF.U.FOF.U.FOF
pt: S3 R ..O..O..O..O..O..O..O..O..O..O..O..O..O..O..O..O
The first line helps to locate a nth partition ('v' is an bottom arrow)
and it is omitted when the table has less than 10 partitions.
"""
node_list = sorted(self.count_dict)
result = ['pt: node %u: %s, %s' % (i, uuid_str(node.getUUID()),
protocol.node_state_prefix_dict[node.getState()])
for i, node in enumerate(node_list)]
append = result.append
line = []
max_line_len = 20 # XXX: hardcoded number of partitions per line
prefix = 0
prefix_len = int(math.ceil(math.log10(self.np)))
for offset, row in enumerate(self._formatRows(node_list)):
if len(line) == max_line_len:
append('pt: %0*u: %s' % (prefix_len, prefix, '|'.join(line)))
line = []
prefix = offset
line.append(row)
if line:
append('pt: %0*u: %s' % (prefix_len, prefix, '|'.join(line)))
if not node_list:
return ()
cell_state_dict = protocol.cell_state_prefix_dict
node_dict = defaultdict(partial(bytearray, '.' * self.np))
for offset, row in enumerate(self.partition_list):
for cell in row:
node_dict[cell.getNode()][offset] = \
cell_state_dict[cell.getState()]
n = len(uuid_str(node_list[-1].getUUID()))
result = [''.join('%9sv' % x if x else 'pt:' + ' ' * (5 + n)
for x in xrange(0, self.np, 10))
] if self.np > 10 else []
result.extend('pt: %-*s %s %s' % (n, uuid_str(node.getUUID()),
protocol.node_state_prefix_dict[node.getState()],
node_dict[node])
for node in node_list)
return result
def _formatRows(self, node_list):
......
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -15,9 +15,8 @@
# along with this program. If not, see <http://www.gnu.org/licenses/>.
import thread, threading, weakref
from . import logging
from . import debug, logging
from .app import BaseApplication
from .debug import register as registerLiveDebugger
from .dispatcher import Dispatcher
from .locking import SimpleQueue
......@@ -28,7 +27,10 @@ class app_set(weakref.WeakSet):
app.log()
app_set = app_set()
registerLiveDebugger(app_set.on_log)
def registerLiveDebugger():
debug.register(app_set.on_log)
registerLiveDebugger()
class ThreadContainer(threading.local):
......
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -23,6 +23,20 @@ from Queue import deque
from struct import pack, unpack, Struct
from time import gmtime
# https://stackoverflow.com/a/6163157
def nextafter():
global nextafter
from ctypes import CDLL, util as ctypes_util, c_double
from time import time
_libm = CDLL(ctypes_util.find_library('m'))
nextafter = _libm.nextafter
nextafter.restype = c_double
nextafter.argtypes = c_double, c_double
x = time()
y = nextafter(x, float('inf'))
assert x < y and (x+y)/2 in (x,y), (x, y)
nextafter()
TID_LOW_OVERFLOW = 2**32
TID_LOW_MAX = TID_LOW_OVERFLOW - 1
SECOND_PER_TID_LOW = 60.0 / TID_LOW_OVERFLOW
......
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -18,10 +18,10 @@ import sys
from collections import defaultdict
from time import time
from neo.lib import logging
from neo.lib.app import BaseApplication
from neo.lib import logging, util
from neo.lib.app import BaseApplication, buildOptionParser
from neo.lib.debug import register as registerLiveDebugger
from neo.lib.protocol import uuid_str, UUID_NAMESPACES, ZERO_TID
from neo.lib.protocol import UUID_NAMESPACES, ZERO_TID
from neo.lib.protocol import ClusterStates, NodeStates, NodeTypes, Packets
from neo.lib.handler import EventHandler
from neo.lib.connection import ListeningConnection, ClientConnection
......@@ -47,6 +47,7 @@ from .transactions import TransactionManager
from .verification import VerificationManager
@buildOptionParser
class Application(BaseApplication):
"""The master node application."""
packing = None
......@@ -57,7 +58,7 @@ class Application(BaseApplication):
backup_app = None
truncate_tid = None
def uuid(self, uuid):
def setUUID(self, uuid):
node = self.nm.getByUUID(uuid)
if node is not self._node:
if node:
......@@ -65,33 +66,56 @@ class Application(BaseApplication):
if node.isConnected(True):
node.getConnection().close()
self._node.setUUID(uuid)
uuid = property(lambda self: self._node.getUUID(), uuid)
logging.node(self.name, uuid)
uuid = property(lambda self: self._node.getUUID(), setUUID)
@property
def election(self):
if self.primary and self.cluster_state == ClusterStates.RECOVERING:
return self.primary
@classmethod
def _buildOptionParser(cls):
_ = cls.option_parser
_.description = "NEO Master node"
cls.addCommonServerOptions('master', '127.0.0.1:10000', '')
_ = _.group('master')
_.int('r', 'replicas', default=0, help="replicas number")
_.int('p', 'partitions', default=100, help="partitions number")
_.int('A', 'autostart',
help="minimum number of pending storage nodes to automatically"
" start new cluster (to avoid unwanted recreation of the"
" cluster, this should be the total number of storage nodes)")
_('C', 'upstream-cluster',
help='the name of cluster to backup')
_('M', 'upstream-masters', parse=util.parseMasterList,
help='list of master nodes in the cluster to backup')
_.int('u', 'uuid',
help="specify an UUID to use for this process (testing purpose)")
def __init__(self, config):
super(Application, self).__init__(
config.getSSL(), config.getDynamicMasterList())
config.get('ssl'), config.get('dynamic_master_list'))
self.tm = TransactionManager(self.onTransactionCommitted)
self.name = config.getCluster()
self.server = config.getBind()
self.autostart = config.getAutostart()
self.name = config['cluster']
self.server = config['bind']
self.autostart = config.get('autostart')
self.storage_ready_dict = {}
self.storage_starting_set = set()
for master_address in config.getMasters():
for master_address in config['masters']:
self.nm.createMaster(address=master_address)
self._node = self.nm.createMaster(address=self.server,
uuid=config.getUUID())
uuid=config.get('uuid'))
logging.node(self.name, self.uuid)
logging.debug('IP address is %s, port is %d', *self.server)
# Partition table
replicas, partitions = config.getReplicas(), config.getPartitions()
replicas = config['replicas']
partitions = config['partitions']
if replicas < 0:
raise RuntimeError, 'replicas must be a positive integer'
if partitions <= 0:
......@@ -107,13 +131,13 @@ class Application(BaseApplication):
self._current_manager = None
# backup
upstream_cluster = config.getUpstreamCluster()
upstream_cluster = config.get('upstream_cluster')
if upstream_cluster:
if upstream_cluster == self.name:
raise ValueError("upstream cluster name must be"
" different from cluster name")
self.backup_app = BackupApplication(self, upstream_cluster,
config.getUpstreamMasters())
config['upstream_masters'])
self.administration_handler = administration.AdministrationHandler(
self)
......@@ -242,7 +266,6 @@ class Application(BaseApplication):
if self.uuid is None:
self.uuid = self.getNewUUID(None, self.server, NodeTypes.MASTER)
logging.info('My UUID: ' + uuid_str(self.uuid))
self._node.setRunning()
self._node.id_timestamp = None
self.primary = monotonic_time()
......@@ -575,3 +598,12 @@ class Application(BaseApplication):
def getStorageReadySet(self, readiness=float('inf')):
return {k for k, v in self.storage_ready_dict.iteritems()
if v <= readiness}
def notifyTransactionAborted(self, ttid, uuids):
uuid_set = self.getStorageReadySet()
uuid_set.intersection_update(uuids)
if uuid_set:
p = Packets.AbortTransaction(ttid, ())
getByUUID = self.nm.getByUUID
for uuid in uuid_set:
getByUUID(uuid).send(p)
# -*- coding: utf-8 -*-
#
# Copyright (C) 2012-2017 Nexedi SA
# Copyright (C) 2012-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -82,6 +82,11 @@ class BackupApplication(object):
self.nm.close()
del self.__dict__
def setUUID(self, uuid):
if self.uuid != uuid:
self.uuid = uuid
logging.info('Upstream Node ID: %s', uuid_str(uuid))
def log(self):
self.nm.log()
if self.pt is not None:
......
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -46,6 +46,13 @@ class AdministrationHandler(MasterHandler):
if node is not None:
self.app.nm.remove(node)
def flushLog(self, conn):
p = Packets.FlushLog()
for node in self.app.nm.getConnectedList():
c = node.getConnection()
c is conn or c.send(p)
super(AdministrationHandler, self).flushLog(conn)
def setClusterState(self, conn, state):
app = self.app
# check request
......
#
# Copyright (C) 2012-2017 Nexedi SA
# Copyright (C) 2012-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -27,7 +27,8 @@ class ClientServiceHandler(MasterHandler):
app = self.app
node = app.nm.getByUUID(conn.getUUID())
assert node is not None, conn
app.tm.clientLost(node)
for x in app.tm.clientLost(node):
app.notifyTransactionAborted(*x)
node.setUnknown()
app.broadcastNodesInformation([node])
......@@ -37,7 +38,7 @@ class ClientServiceHandler(MasterHandler):
"""
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
# answers, otherwise we can't know if the client is expected to commit
# the transaction in full to all these storage nodes.
if app.storage_starting_set:
raise DelayEvent
......@@ -121,12 +122,7 @@ class ClientServiceHandler(MasterHandler):
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)
app.notifyTransactionAborted(tid, involved)
# like ClientServiceHandler but read-only & only for tid <= backup_tid
......
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -444,7 +444,9 @@ class TransactionManager(EventQueue):
def clientLost(self, node):
for txn in self._ttid_dict.values():
if txn.clientLost(node):
del self[txn.getTTID()]
tid = txn.getTTID()
del self[tid]
yield tid, txn.getNotificationUUIDList()
def log(self):
logging.info('Transactions:')
......
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -14,6 +14,7 @@
# You should have received a copy of the GNU General Public License
# along with this program. If not, see <http://www.gnu.org/licenses/>.
import sys
from .neoctl import NeoCTL, NotReadyException
from neo.lib.util import p64, u64, tidFromTime, timeStringFromTID
from neo.lib.protocol import uuid_str, formatNodeList, \
......@@ -38,6 +39,7 @@ action_dict = {
'kill': 'killNode',
'prune_orphan': 'pruneOrphan',
'truncate': 'truncate',
'flush_log': 'flushLog',
}
# ex 'S2' -> u32
......@@ -253,6 +255,15 @@ class TerminalNeoCTL(object):
partition_dict = dict.fromkeys(xrange(np), source)
self.neoctl.checkReplicas(partition_dict, min_tid, max_tid)
def flushLog(self, params):
"""
Ask all nodes in the cluster to flush their logs.
If there are backup clusters, only their primary masters flush.
"""
assert not params
self.neoctl.flushLog()
class Application(object):
"""The storage node application."""
......@@ -267,18 +278,18 @@ class Application(object):
# state (RUNNING, DOWN...) and modify the partition if asked
# set cluster name [shutdown|operational] : either shutdown the
# cluster or mark it as operational
if not args:
return self.usage()
current_action = action_dict
level = 0
while current_action is not None and \
level < len(args) and \
try:
while level < len(args) and \
isinstance(current_action, dict):
current_action = current_action.get(args[level])
current_action = current_action[args[level]]
level += 1
action = None
if isinstance(current_action, basestring):
action = getattr(self.neoctl, current_action, None)
if action is None:
return self.usage('unknown command')
except KeyError:
sys.exit('invalid command: ' + ' '.join(args))
action = getattr(self.neoctl, current_action)
try:
return action(args[level:])
except NotReadyException, message:
......@@ -313,8 +324,8 @@ class Application(object):
for x in docstring_line_list])
return '\n'.join(result)
def usage(self, message):
output_list = (message, 'Available commands:', self._usage(action_dict),
def usage(self):
output_list = ('Available commands:', self._usage(action_dict),
"TID arguments can be either integers or timestamps as floats,"
" e.g. '257684787499560686', '0x3937af2eeeeeeee' or '1325421296.'"
" for 2012-01-01 12:34:56 UTC")
......
#
# Copyright (C) 2009-2017 Nexedi SA
# Copyright (C) 2009-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -14,7 +14,9 @@
# You should have received a copy of the GNU General Public License
# along with this program. If not, see <http://www.gnu.org/licenses/>.
from neo.lib.app import BaseApplication
import argparse
from neo.lib import util
from neo.lib.app import BaseApplication, buildOptionParser
from neo.lib.connection import ClientConnection, ConnectionClosed
from neo.lib.protocol import ClusterStates, NodeStates, ErrorCodes, Packets
from .handler import CommandEventHandler
......@@ -22,11 +24,24 @@ from .handler import CommandEventHandler
class NotReadyException(Exception):
pass
@buildOptionParser
class NeoCTL(BaseApplication):
connection = None
connected = False
@classmethod
def _buildOptionParser(cls):
# XXX: Use argparse sub-commands.
parser = cls.option_parser
parser.description = "NEO Control node"
parser('a', 'address', default='127.0.0.1:9999',
parse=lambda x: util.parseNodeAddress(x, 9999),
help="address of an admin node")
parser.argument('cmd', nargs=argparse.REMAINDER,
help="command to execute; if not supplied,"
" the list of available commands is displayed")
def __init__(self, address, **kw):
super(NeoCTL, self).__init__(**kw)
self.server = self.nm.createAdmin(address=address)
......@@ -189,3 +204,9 @@ class NeoCTL(BaseApplication):
if response[0] != Packets.Error or response[1] != ErrorCodes.ACK:
raise RuntimeError(response)
return response[2]
def flushLog(self):
conn = self.__getConnection()
conn.send(Packets.FlushLog())
while conn.pending():
self.em.poll(1)
......@@ -2,7 +2,7 @@
#
# neoadmin - run an administrator node of NEO
#
# Copyright (C) 2009-2017 Nexedi SA
# Copyright (C) 2009-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -18,27 +18,15 @@
# along with this program. If not, see <http://www.gnu.org/licenses/>.
from neo.lib import logging
from neo.lib.config import getServerOptionParser, ConfigurationManager
parser = getServerOptionParser()
parser.add_option('-u', '--uuid', help='specify an UUID to use for this ' \
'process')
defaults = dict(
bind = '127.0.0.1:9999',
masters = '127.0.0.1:10000',
)
def main(args=None):
# build configuration dict from command line options
(options, args) = parser.parse_args(args=args)
config = ConfigurationManager(defaults, options, 'admin')
from neo.admin.app import Application
config = Application.option_parser.parse(args)
# setup custom logging
logging.setup(config.getLogfile())
logging.setup(config.get('logfile'))
# and then, load and run the application
from neo.admin.app import Application
app = Application(config)
app.run()
#!/usr/bin/env python
#
# neoadmin - run an administrator node of NEO
# neoctl - command-line interface to an administrator node of NEO
#
# Copyright (C) 2009-2017 Nexedi SA
# Copyright (C) 2009-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -18,30 +18,22 @@
# along with this program. If not, see <http://www.gnu.org/licenses/>.
from neo.lib import logging
from neo.lib.config import getOptionParser
from neo.lib.util import parseNodeAddress
parser = getOptionParser()
parser.add_option('-a', '--address', help = 'specify the address (ip:port) ' \
'of an admin node', default = '127.0.0.1:9999')
def main(args=None):
(options, args) = parser.parse_args(args=args)
if options.address is not None:
address = parseNodeAddress(options.address, 9999)
else:
address = ('127.0.0.1', 9999)
from neo.neoctl.neoctl import NeoCTL
config = NeoCTL.option_parser.parse(args)
if options.logfile:
logfile = config.get('logfile')
if logfile:
# Contrary to daemons, we log everything to disk automatically
# because a user using -l option here:
# - is certainly debugging an issue and wants everything,
# - would not have to time to send SIGRTMIN before neoctl exits.
logging.backlog(None)
logging.setup(options.logfile)
from neo.neoctl.app import Application
logging.setup(logfile)
ssl = options.ca, options.cert, options.key
r = Application(address, ssl=ssl if any(ssl) else None).execute(args)
from neo.neoctl.app import Application
app = Application(config['address'], ssl=config.get('ssl'))
r = app.execute(config['cmd'])
if r is not None:
print r
......@@ -2,7 +2,7 @@
#
# neolog - read a NEO log
#
# Copyright (C) 2012-2017 Nexedi SA
# Copyright (C) 2012-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -17,25 +17,40 @@
# You should have received a copy of the GNU General Public License
# along with this program. If not, see <http://www.gnu.org/licenses/>.
import bz2, gzip, errno, optparse, os, signal, sqlite3, sys, time
import argparse, bz2, gzip, errno, os, signal, sqlite3, sys, time
from bisect import insort
from itertools import chain
from logging import getLevelName
from zlib import decompress
comp_dict = dict(bz2=bz2.BZ2File, gz=gzip.GzipFile, xz='xzcat')
try:
import zstd
except ImportError:
zstdcat = 'zstdcat'
else:
from cStringIO import StringIO
def zstdcat(path):
with open(path, 'rb') as f:
return StringIO(zstd.decompress(f.read()))
comp_dict = dict(bz2=bz2.BZ2File, gz=gzip.GzipFile, xz='xzcat', zst=zstdcat)
class Log(object):
_log_id = _packet_id = -1
_log_date = _packet_date = 0
_protocol_date = None
def __init__(self, db_path, decode=0, date_format=None,
filter_from=None, node_column=True, node_list=None):
filter_from=None, show_cluster=False, no_nid=False,
node_column=True, node_list=None):
self._date_format = '%F %T' if date_format is None else date_format
self._decode = decode
self._filter_from = filter_from
self._no_nid = no_nid
self._node_column = node_column
self._node_list = node_list
self._node_dict = {}
self._show_cluster = show_cluster
name = os.path.basename(db_path)
try:
name, ext = name.rsplit(os.extsep, 1)
......@@ -57,35 +72,78 @@ class Log(object):
self._default_name = name
def __iter__(self):
db = self._db
q = self._db.execute
try:
db.execute("BEGIN")
q("BEGIN")
yield
nl = "SELECT * FROM log WHERE id>?"
np = "SELECT * FROM packet WHERE id>?"
date = self._filter_from
if date:
date = " AND date>=%f" % date
nl += date
np += date
nl = db.execute(nl, (self._log_id,))
np = db.execute(np, (self._packet_id,))
if date and max(self._log_date, self._packet_date) < date:
log_args = packet_args = date,
date = " WHERE date>=?"
else:
self._filter_from = None
log_args = self._log_date,
packet_args = self._packet_date,
date = " WHERE date>?"
old = "SELECT date, name, NULL, NULL, %s FROM %s" + date
new = ("SELECT date, name, cluster, nid, %s"
" FROM %s LEFT JOIN node ON node=id" + date)
log = 'level, pathname, lineno, msg'
pkt = 'msg_id, code, peer, body'
try:
nl = q(new % (log, 'log'), log_args)
except sqlite3.OperationalError:
nl = q(old % (log, 'log'), log_args)
np = q(old % (pkt, 'packet'), packet_args)
else:
np = q(new % (pkt, 'packet'), packet_args)
try:
nl = chain(q(old % (log, 'log1'), log_args), nl)
np = chain(q(old % (pkt, 'packet1'), packet_args), np)
except sqlite3.OperationalError:
pass
try:
p = np.next()
self._reload(p[1])
self._reload(p[0])
except StopIteration:
p = None
for self._log_id, date, name, level, pathname, lineno, msg in nl:
while p and p[1] < date:
except sqlite3.DatabaseError, e:
yield time.time(), None, 'PACKET', self._exc(e)
p = None
try:
for date, name, cluster, nid, level, pathname, lineno, msg in nl:
while p and p[0] < date:
yield self._packet(*p)
p = np.fetchone()
yield date, name, getLevelName(level), msg.splitlines()
try:
p = next(np, None)
except sqlite3.DatabaseError, e:
yield time.time(), None, 'PACKET', self._exc(e)
p = None
self._log_date = date
yield (date, self._node(name, cluster, nid),
getLevelName(level), msg.splitlines())
except sqlite3.DatabaseError, e:
yield time.time(), None, 'LOG', self._exc(e)
if p:
yield self._packet(*p)
try:
for p in np:
yield self._packet(*p)
except sqlite3.DatabaseError, e:
yield time.time(), None, 'PACKET', self._exc(e)
finally:
db.rollback()
self._db.rollback()
@staticmethod
def _exc(e):
return ('%s: %s' % (type(e).__name__, e)).splitlines()
def _node(self, name, cluster, nid):
if nid and not self._no_nid:
name = self.uuid_str(nid)
if self._show_cluster:
name = cluster + '/' + name
return name
def _reload(self, date):
q = self._db.execute
......@@ -143,8 +201,8 @@ class Log(object):
for msg in msg_list:
print prefix + msg
def _packet(self, id, date, name, msg_id, code, peer, body):
self._packet_id = id
def _packet(self, date, name, cluster, nid, msg_id, code, peer, body):
self._packet_date = date
if self._next_protocol <= date:
self._reload(date)
try:
......@@ -174,7 +232,7 @@ class Log(object):
args = self._decompress(args, path)
if args and self._decode:
msg[0] += ' \t| ' + repr(args)
return date, name, 'PACKET', msg
return date, self._node(name, cluster, nid), 'PACKET', msg
def _decompress(self, args, path):
if args:
......@@ -225,38 +283,43 @@ def emit_many(log_list):
insort(event_list, (-event[0], next, emit, event))
def main():
parser = optparse.OptionParser()
parser.add_option('-a', '--all', action="store_true",
parser = argparse.ArgumentParser(description='NEO Log Reader')
_ = parser.add_argument
_('-a', '--all', action="store_true",
help='decode body of packets')
parser.add_option('-A', '--decompress', action="store_true",
_('-A', '--decompress', action="store_true",
help='decompress data when decode body of packets (implies --all)')
parser.add_option('-d', '--date', metavar='FORMAT',
_('-d', '--date', metavar='FORMAT',
help='custom date format, according to strftime(3)')
parser.add_option('-f', '--follow', action="store_true",
_('-f', '--follow', action="store_true",
help='output appended data as the file grows')
parser.add_option('-F', '--flush', action="append", type="int",
_('-F', '--flush', action="append", type=int, metavar='PID',
help='with -f, tell process PID to flush logs approximately N'
' seconds (see -s)', metavar='PID')
parser.add_option('-n', '--node', action="append",
' seconds (see -s)')
_('-n', '--node', action="append",
help='only show log entries from the given node'
' (only useful for logs produced by threaded tests),'
" special value '-' hides the column")
parser.add_option('-s', '--sleep-interval', type="float", default=1,
help='with -f, sleep for approximately N seconds (default 1.0)'
' between iterations', metavar='N')
parser.add_option('--from', dest='filter_from',
help='show records more recent that timestamp N if N > 0,'
' or now+N if N < 0; N can also be a string that is'
' parseable by dateutil ', metavar='N')
options, args = parser.parse_args()
if options.sleep_interval <= 0:
_('-s', '--sleep-interval', type=float, default=1., metavar='N',
help='with -f, sleep for approximately N seconds (default %(default)s)'
' between iterations')
_('--from', dest='filter_from', metavar='N',
help='show records more recent that timestamp N if N > 0, or now+N'
' if N < 0; N can also be a string that is parseable by dateutil')
_('file', nargs='+',
help='log file, compressed (gz, bz2 or xz) or not')
_ = parser.add_mutually_exclusive_group().add_argument
_('-C', '--cluster', action="store_true",
help='show cluster name in node column')
_('-N', '--no-nid', action="store_true",
help='always show node name (instead of NID) in node column')
args = parser.parse_args()
if args.sleep_interval <= 0:
parser.error("sleep_interval must be positive")
if not args:
parser.error("no log specified")
filter_from = options.filter_from
filter_from = args.filter_from
if filter_from:
try:
filter_from = float(options.filter_from)
filter_from = float(args.filter_from)
except ValueError:
from dateutil.parser import parse
x = parse(filter_from)
......@@ -267,24 +330,25 @@ def main():
else:
if filter_from < 0:
filter_from += time.time()
node_list = options.node or []
node_list = args.node or []
try:
node_list.remove('-')
node_column = False
except ValueError:
node_column = True
log_list = [Log(db_path,
2 if options.decompress else 1 if options.all else 0,
options.date, filter_from, node_column, node_list)
for db_path in args]
if options.follow:
2 if args.decompress else 1 if args.all else 0,
args.date, filter_from, args.cluster, args.no_nid,
node_column, node_list)
for db_path in args.file]
if args.follow:
try:
pid_list = options.flush or ()
pid_list = args.flush or ()
while True:
emit_many(log_list)
for pid in pid_list:
os.kill(pid, signal.SIGRTMIN)
time.sleep(options.sleep_interval)
time.sleep(args.sleep_interval)
except KeyboardInterrupt:
pass
else:
......
......@@ -2,7 +2,7 @@
#
# neomaster - run a master node of NEO
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -18,39 +18,15 @@
# along with this program. If not, see <http://www.gnu.org/licenses/>.
from neo.lib import logging
from neo.lib.config import getServerOptionParser, ConfigurationManager
parser = getServerOptionParser()
parser.add_option('-u', '--uuid', help='the node UUID (testing purpose)')
parser.add_option('-r', '--replicas', help = 'replicas number')
parser.add_option('-p', '--partitions', help = 'partitions number')
parser.add_option('-A', '--autostart',
help='minimum number of pending storage nodes to automatically start'
' new cluster (to avoid unwanted recreation of the cluster,'
' this should be the total number of storage nodes)')
parser.add_option('-C', '--upstream-cluster',
help='the name of cluster to backup')
parser.add_option('-M', '--upstream-masters',
help='list of master nodes in cluster to backup')
defaults = dict(
bind = '127.0.0.1:10000',
masters = '',
replicas = 0,
partitions = 100,
)
def main(args=None):
# build configuration dict from command line options
(options, args) = parser.parse_args(args=args)
config = ConfigurationManager(defaults, options, 'master')
from neo.master.app import Application
config = Application.option_parser.parse(args)
# setup custom logging
logging.backlog(max_size=None) # log without delay
logging.setup(config.getLogfile())
logging.setup(config.get('logfile'))
# and then, load and run the application
from neo.master.app import Application
app = Application(config)
app.run()
#!/usr/bin/env python
#
# neomaster - run a master node of NEO
# neomigrate - import/export data between NEO and a FileStorage
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -17,53 +17,62 @@
# You should have received a copy of the GNU General Public License
# along with this program. If not, see <http://www.gnu.org/licenses/>.
from neo.lib.config import getOptionParser
from __future__ import print_function
import time
import os
from neo.lib.app import buildOptionParser
# register options
parser = getOptionParser()
parser.add_option('-s', '--source', help='the source database')
parser.add_option('-d', '--destination', help='the destination database')
parser.add_option('-c', '--cluster', help='the NEO cluster name')
import_warning = (
"WARNING: This is not the recommended way to import data to NEO:"
" you should use the Importer backend instead.\n"
"NEO also does not implement IStorageRestoreable interface, which"
" means that undo information is not preserved when using this tool:"
" conflict resolution could happen when undoing an old transaction."
)
def main(args=None):
# parse options
(options, args) = parser.parse_args(args=args)
source = options.source or None
destination = options.destination or None
cluster = options.cluster or None
@buildOptionParser
class NEOMigrate(object):
from neo.lib.config import OptionList
@classmethod
def _buildOptionParser(cls):
parser = cls.option_parser
parser.description = "NEO <-> FileStorage conversion tool"
parser('c', 'cluster', required=True, help='the NEO cluster name')
parser.bool('q', 'quiet', help='print nothing to standard output')
parser.argument('source', help='the source database')
parser.argument('destination', help='the destination database')
# check options
if source is None or destination is None:
raise RuntimeError('Source and destination databases must be supplied')
if cluster is None:
raise RuntimeError('The NEO cluster name must be supplied')
def __init__(self, config):
self.name = config.pop('cluster')
self.source = config.pop('source')
self.destination = config.pop('destination')
self.quiet = config.pop('quiet', False)
# open storages
from ZODB.FileStorage import FileStorage
from neo.client.Storage import Storage as NEOStorage
if os.path.exists(source):
""" (remove vvv warning from neo/go test output)
print("WARNING: This is not the recommended way to import data to NEO:"
" you should use the Importer backend instead.\n"
"NEO also does not implement IStorageRestoreable interface,"
" which means that undo information is not preserved when using"
" this tool: conflict resolution could happen when undoing an"
" old transaction.")
"""
src = FileStorage(file_name=source, read_only=True)
dst = NEOStorage(master_nodes=destination, name=cluster,
logfile=options.logfile)
if os.path.exists(self.source):
if not self.quiet:
print(import_warning)
self.src = FileStorage(file_name=self.source, read_only=True)
self.dst = NEOStorage(master_nodes=self.destination, name=self.name,
**config)
else:
src = NEOStorage(master_nodes=source, name=cluster,
logfile=options.logfile, read_only=True)
dst = FileStorage(file_name=destination)
self.src = NEOStorage(master_nodes=self.source, name=self.name,
read_only=True, **config)
self.dst = FileStorage(file_name=self.destination)
# do the job
print "Migrating from %s to %s" % (source, destination)
def run(self):
if not self.quiet:
print("Migrating from %s to %s" % (self.source, self.destination))
start = time.time()
dst.copyTransactionsFrom(src)
self.dst.copyTransactionsFrom(self.src)
if not self.quiet:
elapsed = time.time() - start
print "Migration done in %3.5f" % (elapsed, )
print("Migration done in %3.5f" % elapsed)
def main(args=None):
config = NEOMigrate.option_parser.parse(args)
NEOMigrate(config).run()
......@@ -2,7 +2,7 @@
#
# neostorage - run a storage node of NEO
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -18,49 +18,15 @@
# along with this program. If not, see <http://www.gnu.org/licenses/>.
from neo.lib import logging
from neo.lib.config import getServerOptionParser, ConfigurationManager
parser = getServerOptionParser()
parser.add_option('-u', '--uuid', help='specify an UUID to use for this ' \
'process. Previously assigned UUID takes precedence (ie ' \
'you should always use --reset with this switch)')
parser.add_option('-a', '--adapter', help = 'database adapter to use')
parser.add_option('-d', '--database', help = 'database connections string')
parser.add_option('-e', '--engine', help = 'database engine')
parser.add_option('-w', '--wait', help='seconds to wait for backend to be '
'available, before erroring-out (-1 = infinite)', type='float', default=0)
parser.add_option('--dedup', action='store_true',
help = 'enable deduplication of data when setting'
' up a new storage node (for RocksDB, check'
' https://github.com/facebook/mysql-5.6/issues/702)')
parser.add_option('--disable-drop-partitions', action='store_true',
help = 'do not delete data of discarded cells, which is'
' useful for big databases because the current'
' implementation is inefficient (this option should'
' disappear in the future)')
parser.add_option('--reset', action='store_true',
help='remove an existing database if any, and exit')
defaults = dict(
bind = '127.0.0.1',
masters = '127.0.0.1:10000',
adapter = 'MySQL',
)
def main(args=None):
# TODO: Forbid using "reset" along with any unneeded argument.
# "reset" is too dangerous to let user a chance of accidentally
# letting it slip through in a long option list.
# We should drop support configuration files to make such check useful.
(options, args) = parser.parse_args(args=args)
config = ConfigurationManager(defaults, options, 'storage')
from neo.storage.app import Application
config = Application.option_parser.parse(args)
# setup custom logging
logging.setup(config.getLogfile())
logging.setup(config.get('logfile'))
# and then, load and run the application
from neo.storage.app import Application
app = Application(config)
if not config.getReset():
if not config.get('reset'):
app.run()
#!/usr/bin/env python
#
# Copyright (C) 2009-2017 Nexedi SA
# Copyright (C) 2009-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -15,6 +15,7 @@
# You should have received a copy of the GNU General Public License
# along with this program. If not, see <http://www.gnu.org/licenses/>.
import argparse
import traceback
import unittest
import time
......@@ -275,41 +276,43 @@ class NeoTestRunner(unittest.TextTestResult):
class TestRunner(BenchmarkRunner):
def add_options(self, parser):
parser.add_option('-c', '--coverage', action='store_true',
x = parser.add_mutually_exclusive_group().add_argument
x('-c', '--coverage', action='store_true',
help='Enable coverage')
parser.add_option('-C', '--cov-unit', action='store_true',
x('-C', '--cov-unit', action='store_true',
help='Same as -c but output 1 file per test,'
' in the temporary test directory')
parser.add_option('-L', '--log', action='store_true',
_ = parser.add_argument
_('-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,
_('-l', '--loop', type=int, default=1,
help='Repeat tests several times')
parser.add_option('-f', '--functional', action='store_true',
_('-f', '--functional', action='store_true',
help='Functional tests')
parser.add_option('-s', '--stop-on-error', action='store_false',
dest='stop_on_success',
x = parser.add_mutually_exclusive_group().add_argument
x('-s', '--stop-on-error', action='store_false',
dest='stop_on_success', default=None,
help='Continue as long as tests pass successfully.'
' It is usually combined with --loop, to check that tests'
' do not fail randomly.')
parser.add_option('-S', '--stop-on-success', action='store_true',
x('-S', '--stop-on-success', action='store_true', default=None,
help='Opposite of --stop-on-error: stop as soon as a test'
' passes. Details about errors are not printed at exit.')
parser.add_option('-r', '--readable-tid', action='store_true',
_('-r', '--readable-tid', action='store_true',
help='Change master behaviour to generate readable TIDs for easier'
' debugging (rather than from current time).')
parser.add_option('-u', '--unit', action='store_true',
_('-u', '--unit', action='store_true',
help='Unit & threaded tests')
parser.add_option('-z', '--zodb', action='store_true',
_('-z', '--zodb', action='store_true',
help='ZODB test suite running on a NEO')
parser.add_option('-v', '--verbose', action='store_true',
_('-v', '--verbose', action='store_true',
help='Verbose output')
parser.usage += " [[!] module [test...]]"
parser.format_epilog = lambda _: """
Positional:
Filter by given module/test. These arguments are shell patterns.
This implies -ufz if none of this option is passed.
_('only', nargs=argparse.REMAINDER, metavar='[[!] module [test...]]',
help="Filter by given module/test. These arguments are shell"
" patterns. This implies -ufz if none of this option is"
" passed.")
parser.epilog = """
Environment Variables:
NEO_TESTS_ADAPTER Default is SQLite for threaded clusters,
MySQL otherwise.
......@@ -330,25 +333,23 @@ Environment Variables:
NEO_TEST_ZODB_STORAGES default: 1
""" % neo_tests__dict__
def load_options(self, options, args):
if options.coverage and options.cov_unit:
sys.exit('-c conflicts with -C')
if not (options.unit or options.functional or options.zodb):
if not args:
def load_options(self, args):
if not (args.unit or args.functional or args.zodb):
if not args.only:
sys.exit('Nothing to run, please give one of -f, -u, -z')
options.unit = options.functional = options.zodb = True
args.unit = args.functional = args.zodb = True
return dict(
log = options.log,
loop = options.loop,
unit = options.unit,
functional = options.functional,
zodb = options.zodb,
verbosity = 2 if options.verbose else 1,
coverage = options.coverage,
cov_unit = options.cov_unit,
only = args,
stop_on_success = options.stop_on_success,
readable_tid = options.readable_tid,
log = args.log,
loop = args.loop,
unit = args.unit,
functional = args.functional,
zodb = args.zodb,
verbosity = 2 if args.verbose else 1,
coverage = args.coverage,
cov_unit = args.cov_unit,
only = args.only,
stop_on_success = args.stop_on_success,
readable_tid = args.readable_tid,
)
def start(self):
......
#!/usr/bin/env python
#
# Copyright (C) 2011-2017 Nexedi SA
# Copyright (C) 2011-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -15,9 +15,8 @@
# You should have received a copy of the GNU General Public License
# along with this program. If not, see <http://www.gnu.org/licenses/>.
import inspect, random
import argparse, inspect, random
from logging import getLogger, INFO, DEBUG
from optparse import OptionParser
from neo.lib import logging
from neo.tests import functional
#logging.backlog()
......@@ -27,9 +26,9 @@ del logging.default_root_handler.handle
def main():
args, _, _, defaults = inspect.getargspec(functional.NEOCluster.__init__)
option_list = zip(args[-len(defaults):], defaults)
parser = OptionParser(usage="%prog [options] [db...]",
parser = argparse.ArgumentParser(
description="Quickly setup a simple NEO cluster for testing purpose.")
parser.add_option('--seed', help="settings like node ports/uuids and"
parser.add_argument('--seed', help="settings like node ports/uuids and"
" cluster name are random: pass any string to initialize the RNG")
defaults = {}
for option, default in sorted(option_list):
......@@ -40,14 +39,15 @@ def main():
elif default is not None:
defaults[option] = default
if isinstance(default, int):
kw['type'] = "int"
parser.add_option('--' + option, **kw)
kw['type'] = int
parser.add_argument('--' + option, **kw)
parser.set_defaults(**defaults)
options, args = parser.parse_args()
if options.seed:
functional.random = random.Random(options.seed)
parser.add_argument('db', nargs='+')
args = parser.parse_args()
if args.seed:
functional.random = random.Random(args.seed)
getLogger().setLevel(DEBUG)
cluster = functional.NEOCluster(args, **{x: getattr(options, x)
cluster = functional.NEOCluster(args.db, **{x: getattr(args, x)
for x, _ in option_list})
try:
cluster.run()
......
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -18,44 +18,88 @@ import sys
from collections import deque
from neo.lib import logging
from neo.lib.app import BaseApplication
from neo.lib.protocol import uuid_str, \
CellStates, ClusterStates, NodeTypes, Packets
from neo.lib.app import BaseApplication, buildOptionParser
from neo.lib.protocol import CellStates, ClusterStates, NodeTypes, Packets
from neo.lib.connection import ListeningConnection
from neo.lib.exception import StoppedOperation, PrimaryFailure
from neo.lib.pt import PartitionTable
from neo.lib.util import dump
from neo.lib.bootstrap import BootstrapManager
from .checker import Checker
from .database import buildDatabaseManager
from .database import buildDatabaseManager, DATABASE_MANAGER_DICT
from .handlers import identification, initialization, master
from .replicator import Replicator
from .transactions import TransactionManager
from neo.lib.debug import register as registerLiveDebugger
option_defaults = {
'adapter': 'MySQL',
'wait': 0,
}
assert option_defaults['adapter'] in DATABASE_MANAGER_DICT
@buildOptionParser
class Application(BaseApplication):
"""The storage node application."""
checker = replicator = tm = None
@classmethod
def _buildOptionParser(cls):
parser = cls.option_parser
parser.description = "NEO Storage node"
cls.addCommonServerOptions('storage', '127.0.0.1')
_ = parser.group('storage')
_('a', 'adapter', choices=sorted(DATABASE_MANAGER_DICT),
help="database adapter to use")
_('d', 'database', required=True,
help="database connections string")
_.float('w', 'wait',
help="seconds to wait for backend to be available,"
" before erroring-out (-1 = infinite)")
_.bool('disable-drop-partitions',
help="do not delete data of discarded cells, which is useful for"
" big databases because the current implementation is"
" inefficient (this option should disappear in the future)")
_ = parser.group('database creation')
_.int('u', 'uuid',
help="specify an UUID to use for this process. Previously"
" assigned UUID takes precedence (i.e. you should"
" always use reset with this switch)")
_('e', 'engine', help="database engine (MySQL only)")
_.bool('dedup',
help="enable deduplication of data"
" when setting up a new storage node")
# TODO: Forbid using "reset" along with any unneeded argument.
# "reset" is too dangerous to let user a chance of accidentally
# letting it slip through in a long option list.
# It should even be forbidden in configuration files.
_.bool('reset',
help="remove an existing database if any, and exit")
parser.set_defaults(**option_defaults)
def __init__(self, config):
super(Application, self).__init__(
config.getSSL(), config.getDynamicMasterList())
config.get('ssl'), config.get('dynamic_master_list'))
# set the cluster name
self.name = config.getCluster()
self.name = config['cluster']
self.dm = buildDatabaseManager(config.getAdapter(),
(config.getDatabase(), config.getEngine(), config.getWait()),
self.dm = buildDatabaseManager(config['adapter'],
(config['database'], config.get('engine'), config['wait']),
)
self.disable_drop_partitions = config.getDisableDropPartitions()
self.disable_drop_partitions = config.get('disable_drop_partitions',
False)
# load master nodes
for master_address in config.getMasters():
for master_address in config['masters']:
self.nm.createMaster(address=master_address)
# set the bind address
self.server = config.getBind()
self.server = config['bind']
logging.debug('IP address is %s, port is %d', *self.server)
# The partition table is initialized after getting the number of
......@@ -69,13 +113,15 @@ class Application(BaseApplication):
# operation related data
self.operational = False
self.dm.setup(reset=config.getReset(), dedup=config.getDedup())
self.dm.setup(reset=config.get('reset', False),
dedup=config.get('dedup', False))
self.loadConfiguration()
self.devpath = self.dm.getTopologyPath()
# force node uuid from command line argument, for testing purpose only
if config.getUUID() is not None:
self.uuid = config.getUUID()
if 'uuid' in config:
self.uuid = config['uuid']
logging.node(self.name, self.uuid)
registerLiveDebugger(on_log=self.log)
......@@ -111,6 +157,7 @@ class Application(BaseApplication):
# load configuration
self.uuid = dm.getUUID()
logging.node(self.name, self.uuid)
num_partitions = dm.getNumPartitions()
num_replicas = dm.getNumReplicas()
ptid = dm.getPTID()
......@@ -123,7 +170,6 @@ class Application(BaseApplication):
self.pt = PartitionTable(num_partitions, num_replicas)
logging.info('Configuration loaded:')
logging.info('UUID : %s', uuid_str(self.uuid))
logging.info('PTID : %s', dump(ptid))
logging.info('Name : %s', self.name)
logging.info('Partitions: %s', num_partitions)
......@@ -208,9 +254,7 @@ class Application(BaseApplication):
self.devpath)
self.master_node, self.master_conn, num_partitions, num_replicas = \
bootstrap.getPrimaryConnection()
uuid = self.uuid
logging.info('I am %s', uuid_str(uuid))
self.dm.setUUID(uuid)
self.dm.setUUID(self.uuid)
# Reload a partition table from the database. This is necessary
# when a previous primary master died while sending a partition
......
#
# Copyright (C) 2012-2017 Nexedi SA
# Copyright (C) 2012-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2014-2017 Nexedi SA
# Copyright (C) 2014-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -31,6 +31,7 @@ except ImportError:
_protocol = 1
from ZODB.FileStorage import FileStorage
from ..app import option_defaults
from . import buildDatabaseManager, DatabaseFailure
from .manager import DatabaseManager
from neo.lib import compress, logging, patch, util
......@@ -359,8 +360,7 @@ class ImporterDatabaseManager(DatabaseManager):
config = SafeConfigParser()
config.read(os.path.expanduser(database))
sections = config.sections()
# XXX: defaults copy & pasted from elsewhere - refactoring needed
main = self._conf = {'adapter': 'MySQL', 'wait': 0}
main = self._conf = option_defaults.copy()
main.update(config.items(sections.pop(0)))
self.zodb = [(x, dict(config.items(x))) for x in sections]
x = main.get('compress', 'true')
......
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2012-2017 Nexedi SA
# Copyright (C) 2012-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -179,7 +179,8 @@ class SQLiteDatabaseManager(DatabaseManager):
description BLOB NOT NULL,
ext BLOB NOT NULL,
ttid INTEGER NOT NULL,
PRIMARY KEY (partition, tid))
PRIMARY KEY (partition, tid)
) WITHOUT ROWID
"""
# The table "obj" stores committed object metadata.
......@@ -189,7 +190,8 @@ class SQLiteDatabaseManager(DatabaseManager):
tid INTEGER NOT NULL,
data_id INTEGER,
value_tid INTEGER,
PRIMARY KEY (partition, oid, tid))
PRIMARY KEY (partition, oid, tid)
) WITHOUT ROWID
"""
index_dict['obj'] = (
"CREATE INDEX %s ON %s(partition, tid, oid)",
......
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -62,7 +62,8 @@ class BaseMasterHandler(BaseHandler):
elif node_type == NodeTypes.CLIENT and state != NodeStates.RUNNING:
logging.info('Notified of non-running client, abort (%s)',
uuid_str(uuid))
self.app.tm.abortFor(uuid)
# See comment in ClientOperationHandler.connectionClosed
self.app.tm.abortFor(uuid, even_if_voted=True)
def notifyPartitionChanges(self, conn, ptid, cell_list):
"""This is very similar to Send Partition Table, except that
......
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -18,7 +18,7 @@ from neo.lib import logging
from neo.lib.handler import DelayEvent
from neo.lib.util import dump, makeChecksum, add64
from neo.lib.protocol import Packets, Errors, NonReadableCell, ProtocolError, \
ZERO_HASH, INVALID_PARTITION
ZERO_HASH, ZERO_TID, INVALID_PARTITION
from ..transactions import ConflictError, NotRegisteredError
from . import BaseHandler
import time
......@@ -29,6 +29,24 @@ SLOW_STORE = 2
class ClientOperationHandler(BaseHandler):
def connectionClosed(self, conn):
logging.debug('connection closed for %r', conn)
app = self.app
if app.operational:
# Even if in most cases, abortFor is called from both this method
# and BaseMasterHandler.notifyPartitionChanges (especially since
# storage nodes disconnects unknown clients on their own), these 2
# handlers also cover distinct scenarios, so neither of them is
# redundant:
# - A client node may have network issues with this particular
# storage node and remain RUNNING: we may still be involved in
# the second phase so we only abort non-voted transactions here.
# By not taking part to any further deadlock avoidance,
# not releasing write-locks now would lead to a deadlock.
# - A client node may be disconnected from the master, whereas
# there are still voted (and not locked) transactions to abort.
app.tm.abortFor(conn.getUUID())
def askTransactionInformation(self, conn, tid):
t = self.app.dm.getTransaction(tid)
if t is None:
......@@ -72,26 +90,27 @@ class ClientOperationHandler(BaseHandler):
def _askStoreObject(self, conn, oid, serial, compression, checksum, data,
data_serial, ttid, request_time):
try:
self.app.tm.storeObject(ttid, serial, oid, compression,
locked = self.app.tm.storeObject(ttid, serial, oid, compression,
checksum, data, data_serial)
except ConflictError, err:
# resolvable or not
conn.answer(Packets.AnswerStoreObject(err.tid))
return
locked = err.tid
except NonReadableCell:
logging.info('Ignore store of %s:%s by %s: unassigned partition',
dump(oid), dump(serial), dump(ttid))
locked = ZERO_TID
except NotRegisteredError:
# transaction was aborted, cancel this event
logging.info('Forget store of %s:%s by %s delayed by %s',
dump(oid), dump(serial), dump(ttid),
dump(self.app.tm.getLockingTID(oid)))
locked = ZERO_TID
else:
if request_time and SLOW_STORE is not None:
duration = time.time() - request_time
if duration > SLOW_STORE:
logging.info('StoreObject delay: %.02fs', duration)
conn.answer(Packets.AnswerStoreObject(None))
conn.answer(Packets.AnswerStoreObject(locked))
def askStoreObject(self, conn, oid, serial,
compression, checksum, data, data_serial, ttid):
......@@ -198,25 +217,26 @@ class ClientOperationHandler(BaseHandler):
def _askCheckCurrentSerial(self, conn, ttid, oid, serial, request_time):
try:
self.app.tm.checkCurrentSerial(ttid, oid, serial)
locked = self.app.tm.checkCurrentSerial(ttid, oid, serial)
except ConflictError, err:
# resolvable or not
conn.answer(Packets.AnswerCheckCurrentSerial(err.tid))
return
locked = err.tid
except NonReadableCell:
logging.info('Ignore check of %s:%s by %s: unassigned partition',
dump(oid), dump(serial), dump(ttid))
locked = ZERO_TID
except NotRegisteredError:
# transaction was aborted, cancel this event
logging.info('Forget serial check of %s:%s by %s delayed by %s',
dump(oid), dump(serial), dump(ttid),
dump(self.app.tm.getLockingTID(oid)))
locked = ZERO_TID
else:
if request_time and SLOW_STORE is not None:
duration = time.time() - request_time
if duration > SLOW_STORE:
logging.info('CheckCurrentSerial delay: %.02fs', duration)
conn.answer(Packets.AnswerCheckCurrentSerial(None))
conn.answer(Packets.AnswerCheckCurrentSerial(locked))
# like ClientOperationHandler but read-only & only for tid <= backup_tid
......
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -53,16 +53,17 @@ class IdentificationHandler(EventHandler):
handler = ClientReadOnlyOperationHandler
else:
handler = ClientOperationHandler
assert not node.isConnected(), node
assert node.isRunning(), node
force = False
elif node_type == NodeTypes.STORAGE:
handler = StorageOperationHandler
force = app.uuid < uuid
else:
raise ProtocolError('reject non-client-or-storage node')
# apply the handler and set up the connection
handler = handler(self.app)
conn.setHandler(handler)
node.setConnection(conn, app.uuid < uuid)
node.setConnection(conn, force)
# accept the identification and trigger an event
conn.answer(Packets.AcceptIdentification(NodeTypes.STORAGE, uuid and
app.uuid, app.pt.getPartitions(), app.pt.getReplicas(), uuid))
......
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2018 Nexedi SA
# Copyright (C) 2018-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2010-2017 Nexedi SA
# Copyright (C) 2010-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -19,7 +19,7 @@ from neo.lib import logging
from neo.lib.handler import DelayEvent, EventQueue
from neo.lib.util import dump
from neo.lib.protocol import Packets, ProtocolError, NonReadableCell, \
uuid_str, MAX_TID
uuid_str, MAX_TID, ZERO_TID
class ConflictError(Exception):
"""
......@@ -51,7 +51,9 @@ class Transaction(object):
self.uuid = uuid
self.serial_dict = {}
self.store_dict = {}
# We must distinguish lockless stores from deadlocks.
# Remember the oids for which we didn't check for conflict. This is
# used primarily to know when to declare a cell really readable once
# the replication is finished.
self.lockless = set()
def __repr__(self):
......@@ -152,15 +154,13 @@ class TransactionManager(EventQueue):
store_lock_dict = self._store_lock_dict
replicated = self._replicated
notify = {x[0] for x in replicated.iteritems() if x[1]}
# We sort transactions so that in case of muliple stores/checks for the
# same oid, the lock is taken by the highest locking ttid, which will
# delay new transactions.
# We sort transactions so that in case of multiple stores/checks
# for the same oid, the lock is taken by the highest locking ttid,
# which will delay new transactions.
for txn, ttid in sorted((txn, ttid) for ttid, txn in
self._transaction_dict.iteritems()):
if txn.locking_tid == MAX_TID:
break # all remaining transactions are resolving a deadlock
assert txn.lockless.issubset(txn.serial_dict), (
txn.lockless, txn.serial_dict)
ttid, txn.lockless, txn.serial_dict)
for oid in txn.lockless:
partition = getPartition(oid)
if replicated.get(partition):
......@@ -178,11 +178,21 @@ class TransactionManager(EventQueue):
self._app.master_conn.send(Packets.NotifyReplicationDone(
partition, replicated[partition]))
replicated[partition] = None
for oid, ttid in store_lock_dict.iteritems():
if getPartition(oid) in notify:
for oid in [oid for oid in store_lock_dict
if getPartition(oid) in notify]:
ttid = store_lock_dict.pop(oid)
txn = self._transaction_dict[ttid]
# Use 'discard' instead of 'remove', for oids that were
# locked after that the partition was replicated.
self._transaction_dict[ttid].lockless.discard(oid)
txn.lockless.discard(oid)
try:
locked = self.lockObject(ttid, txn.serial_dict[oid], oid)
except ConflictError:
self._unstore(txn, oid)
except (DelayEvent, NonReadableCell), e: # pragma: no cover
raise AssertionError(e)
else:
assert locked, (oid, ttid, txn)
def register(self, conn, ttid):
"""
......@@ -223,14 +233,22 @@ class TransactionManager(EventQueue):
# fixing the store lock.
if locked == ttid:
del self._store_lock_dict[oid]
# However here, we don't try to remember lockless writes: later,
# we may give write-locks to oids that would normally conflict.
# Readable cells prevent such scenario to go wrong.
lockless = transaction.lockless
if locking_tid == MAX_TID:
if lockless:
lockless.clear()
self._notifyReplicated()
else:
# There's nothing to rebase for lockless stores to replicating
# partitions because there's no lock taken yet. In other words,
# rebasing such stores would do nothing. Other lockless stores
# become normal ones: this transaction does not block anymore
# replicated partitions from being marked as UP_TO_DATE.
oid = [oid
for oid in lockless.intersection(transaction.serial_dict)
for oid in lockless
if self.getPartition(oid) not in self._replicating]
if oid:
lockless.difference_update(oid)
......@@ -269,6 +287,8 @@ class TransactionManager(EventQueue):
self.lockObject(ttid, serial, oid)
except ConflictError:
recheck_set.add(oid)
except (DelayEvent, NonReadableCell), e: # pragma: no cover
raise AssertionError(e)
return recheck_set
def vote(self, ttid, txn_info=None):
......@@ -308,6 +328,8 @@ class TransactionManager(EventQueue):
dict.fromkeys(transaction.store_dict, ttid))
if transaction.voted == 2:
self._app.dm.lockTransaction(tid, ttid)
else:
assert transaction.voted
def unlock(self, ttid):
"""
......@@ -352,6 +374,9 @@ class TransactionManager(EventQueue):
# replicate, and we're expected to store it in full.
# Since there's at least 1 other (readable) cell that will do this
# check, we accept this store/check without taking a lock.
if transaction.locking_tid == MAX_TID:
# Deadlock avoidance. Still no new locking_tid from the client.
raise DelayEvent(transaction)
transaction.lockless.add(oid)
return
locked = self._store_lock_dict.get(oid)
......@@ -404,7 +429,7 @@ class TransactionManager(EventQueue):
assert oid in transaction.serial_dict, transaction
logging.info('Transaction %s checking %s more than once',
dump(ttid), dump(oid))
return
return True
if previous_serial is None:
# 2 valid cases:
# - the previous undo resulted in a resolved conflict
......@@ -417,7 +442,7 @@ class TransactionManager(EventQueue):
# we are down, and the client would stop writing to us.
logging.info('Transaction %s storing %s more than once',
dump(ttid), dump(oid))
return
return True
elif transaction.locking_tid == MAX_TID:
# Deadlock avoidance. Still no new locking_tid from the client.
raise DelayEvent(transaction)
......@@ -449,14 +474,18 @@ class TransactionManager(EventQueue):
raise ConflictError(previous_serial)
logging.debug('Transaction %s locking %s', dump(ttid), dump(oid))
self._store_lock_dict[oid] = ttid
return True
def checkCurrentSerial(self, ttid, oid, serial):
try:
transaction = self._transaction_dict[ttid]
except KeyError:
raise NotRegisteredError
self.lockObject(ttid, serial, oid)
assert oid not in transaction.serial_dict
locked = self.lockObject(ttid, serial, oid)
transaction.serial_dict[oid] = serial
if not locked:
return ZERO_TID
def storeObject(self, ttid, serial, oid, compression, checksum, data,
value_serial):
......@@ -467,7 +496,13 @@ class TransactionManager(EventQueue):
transaction = self._transaction_dict[ttid]
except KeyError:
raise NotRegisteredError
self.lockObject(ttid, serial, oid)
locked = self.lockObject(ttid, serial, oid)
if oid in transaction.serial_dict: # initially/still lockless, or undo
# XXX: We'd like to do that before calling lockObject,
# to release resources immediately (data, maybe lock)
# in case of delay/conflict.
# But keeping everything consistent is complicated.
self._unstore(transaction, oid)
transaction.serial_dict[oid] = serial
# store object
if data is None:
......@@ -475,6 +510,8 @@ class TransactionManager(EventQueue):
else:
data_id = self._app.dm.holdData(checksum, oid, data, compression)
transaction.store(oid, data_id, value_serial)
if not locked:
return ZERO_TID
def rebaseObject(self, ttid, oid):
try:
......@@ -499,6 +536,9 @@ class TransactionManager(EventQueue):
except ConflictError, e:
# Move the data back to the client for conflict resolution,
# since the client may not have it anymore.
return serial, e.tid, self._unstore(transaction, oid)
def _unstore(self, transaction, oid):
try:
data_id = transaction.store_dict.pop(oid)[1]
except KeyError: # check current
......@@ -511,7 +551,7 @@ class TransactionManager(EventQueue):
data = dm.loadData(data_id)
dm.releaseData([data_id], True)
del transaction.serial_dict[oid]
return serial, e.tid, data
return data
def abort(self, ttid, even_if_locked=False):
"""
......@@ -550,21 +590,22 @@ class TransactionManager(EventQueue):
# Lockless store (we are replicating this partition),
# or unresolved deadlock.
continue
if ttid != write_locking_tid:
if __debug__:
if ttid == write_locking_tid:
del self._store_lock_dict[oid]
elif __debug__:
other = self._transaction_dict[write_locking_tid]
x = (oid, ttid, write_locking_tid,
self._replicated, transaction.lockless)
lockless = oid in transaction.lockless
assert oid in other.serial_dict and lockless == bool(
self._replicated.get(self.getPartition(oid))), x
if not lockless:
assert not locked, x
continue # unresolved deadlock
assert oid in other.serial_dict, x
if oid in transaction.lockless:
# Several lockless stores for this oid and among them,
# a higher ttid is still pending.
assert transaction < other, x
del self._store_lock_dict[oid]
# There may remain a single lockless store so we'll need
# this partition to be checked in _notifyReplicated.
assert self._replicated.get(self.getPartition(oid)), x
else: # unresolved deadlock
assert not locked, x
# remove the transaction
del self._transaction_dict[ttid]
if self._replicated:
......@@ -573,14 +614,14 @@ class TransactionManager(EventQueue):
self.read_queue.executeQueuedEvents()
self.executeQueuedEvents()
def abortFor(self, uuid):
def abortFor(self, uuid, even_if_voted=False):
"""
Abort any non-locked transaction of a node
"""
logging.debug('Abort for %s', uuid_str(uuid))
# abort any non-locked transaction of this node
for ttid, transaction in self._transaction_dict.items():
if transaction.uuid == uuid:
if transaction.uuid == uuid and (
even_if_voted or not transaction.voted):
self.abort(ttid)
def isLockedTid(self, tid):
......
#
# Copyright (C) 2009-2017 Nexedi SA
# Copyright (C) 2009-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -28,6 +28,7 @@ import weakref
import MySQLdb
import transaction
from contextlib import contextmanager
from ConfigParser import SafeConfigParser
from cStringIO import StringIO
try:
......@@ -85,8 +86,6 @@ SSL = SSL + "ca.crt", SSL + "node.crt", SSL + "node.key"
logging.default_root_handler.handle = lambda record: None
debug.register()
# prevent "signal only works in main thread" errors in subprocesses
debug.register = lambda on_log=None: None
def mockDefaultValue(name, function):
def method(self, *args, **kw):
......@@ -215,6 +214,14 @@ class NeoTestBase(unittest.TestCase):
expected if isinstance(expected, str) else '|'.join(expected),
'|'.join(pt._formatRows(sorted(pt.count_dict, key=key))))
@contextmanager
def expectedFailure(self, exception=AssertionError, regex=None):
with self.assertRaisesRegexp(exception, regex) as cm:
yield
raise _UnexpectedSuccess
# XXX: passing sys.exc_info() causes deadlocks
raise _ExpectedFailure((type(cm.exception), None, None))
class NeoUnitTestBase(NeoTestBase):
""" Base class for neo tests, implements common checks """
......@@ -255,14 +262,14 @@ class NeoUnitTestBase(NeoTestBase):
assert master_number >= 1 and master_number <= 10
masters = ([(self.local_ip, 10010 + i)
for i in xrange(master_number)])
return Mock({
'getCluster': cluster,
'getBind': masters[0],
'getMasters': masters,
'getReplicas': replicas,
'getPartitions': partitions,
'getUUID': uuid,
})
return {
'cluster': cluster,
'bind': masters[0],
'masters': masters,
'replicas': replicas,
'partitions': partitions,
'uuid': uuid,
}
def getStorageConfiguration(self, cluster='main', master_number=2,
index=0, prefix=DB_PREFIX, uuid=None):
......@@ -277,15 +284,15 @@ class NeoUnitTestBase(NeoTestBase):
db = os.path.join(getTempDirectory(), 'test_neo%s.sqlite' % index)
else:
assert False, adapter
return Mock({
'getCluster': cluster,
'getBind': (masters[0], 10020 + index),
'getMasters': masters,
'getDatabase': db,
'getUUID': uuid,
'getReset': False,
'getAdapter': adapter,
})
return {
'cluster': cluster,
'bind': (masters[0], 10020 + index),
'masters': masters,
'database': db,
'uuid': uuid,
'adapter': adapter,
'wait': 0,
}
def getNewUUID(self, node_type):
"""
......
from __future__ import print_function
import argparse
import sys
import smtplib
import optparse
import platform
import datetime
from email.mime.multipart import MIMEMultipart
......@@ -22,28 +22,28 @@ class BenchmarkRunner(object):
def __init__(self):
self._successful = True
self._status = []
parser = optparse.OptionParser()
parser = argparse.ArgumentParser(
formatter_class=argparse.RawDescriptionHelpFormatter)
# register common options
parser.add_option('', '--title')
parser.add_option('', '--mail-to', action='append')
parser.add_option('', '--mail-from')
parser.add_option('', '--mail-server')
parser.add_option('', '--repeat', type='int', default=1)
_ = parser.add_argument
_('--title')
_('--mail-to', action='append')
_('--mail-from')
_('--mail-server')
self.add_options(parser)
# check common arguments
options, args = parser.parse_args()
if bool(options.mail_to) ^ bool(options.mail_from):
args = parser.parse_args()
if bool(args.mail_to) ^ bool(args.mail_from):
sys.exit('Need a sender and recipients to mail report')
mail_server = options.mail_server or MAIL_SERVER
mail_server = args.mail_server or MAIL_SERVER
# check specifics arguments
self._config = AttributeDict()
self._config.update(self.load_options(options, args))
self._config.update(self.load_options(args))
self._config.update(
title = options.title or self.__class__.__name__,
mail_from = options.mail_from,
mail_to = options.mail_to,
title = args.title or self.__class__.__name__,
mail_from = args.mail_from,
mail_to = args.mail_to,
mail_server = mail_server.split(':'),
repeat = options.repeat,
)
def add_status(self, key, value):
......@@ -104,7 +104,7 @@ class BenchmarkRunner(object):
""" Append options to command line parser """
raise NotImplementedError
def load_options(self, options, args):
def load_options(self, args):
""" Check options and return a configuration dict """
raise NotImplementedError
......
#
# Copyright (C) 2009-2017 Nexedi SA
# Copyright (C) 2009-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2009-2017 Nexedi SA
# Copyright (C) 2009-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2017 Nexedi SA
# Copyright (C) 2017-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2011-2017 Nexedi SA
# Copyright (C) 2011-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2009-2017 Nexedi SA
# Copyright (C) 2009-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -118,13 +118,15 @@ class PortAllocator(object):
class Process(object):
_coverage_fd = None
_coverage_prefix = os.path.join(getTempDirectory(), 'coverage-')
_coverage_prefix = None
_coverage_index = 0
on_fork = [logging.resetNids]
pid = 0
def __init__(self, command, arg_dict={}):
def __init__(self, command, *args, **kw):
self.command = command
self.arg_dict = arg_dict
self.args = args
self.arg_dict = kw
def _args(self):
args = []
......@@ -132,6 +134,7 @@ class Process(object):
args.append('--' + arg)
if param is not None:
args.append(str(param))
args += self.args
return args
def start(self):
......@@ -144,6 +147,9 @@ class Process(object):
if coverage:
cls = self.__class__
cls._coverage_index += 1
if not cls._coverage_prefix:
cls._coverage_prefix = os.path.join(
getTempDirectory(), 'coverage-')
coverage_data_path = cls._coverage_prefix + str(cls._coverage_index)
self._coverage_fd, w = os.pipe()
def save_coverage(*args):
......@@ -169,11 +175,21 @@ class Process(object):
from coverage import Coverage
coverage = Coverage(coverage_data_path)
coverage.start()
# XXX: Sometimes, the handler is not called immediately.
# The process is stuck at an unknown place and the test
# never ends. strace unlocks:
# strace: Process 5520 attached
# close(25) = 0
# getpid() = 5520
# kill(5520, SIGSTOP) = 0
# ...
signal.signal(signal.SIGUSR2, save_coverage)
os.close(self._coverage_fd)
os.write(w, '\0')
sys.argv = [command] + args
setproctitle(self.command)
for on_fork in self.on_fork:
on_fork()
self.run()
status = 0
except SystemExit, e:
......@@ -239,8 +255,8 @@ class Process(object):
self.pid = 0
self.child_coverage()
if result:
raise NodeProcessError('%r %r exited with status %r' % (
self.command, self.arg_dict, result))
raise NodeProcessError('%r %r %r exited with status %r' % (
self.command, self.args, self.arg_dict, result))
return result
def stop(self):
......@@ -255,18 +271,18 @@ class Process(object):
class NEOProcess(Process):
def __init__(self, command, uuid, arg_dict):
def __init__(self, command, *args, **kw):
try:
__import__('neo.scripts.' + command, level=0)
except ImportError:
raise NotFound(command + ' not found')
super(NEOProcess, self).__init__(command, arg_dict)
self.setUUID(uuid)
self.setUUID(kw.pop('uuid', None))
super(NEOProcess, self).__init__(command, *args, **kw)
def _args(self):
args = super(NEOProcess, self)._args()
if self.uuid:
args += '--uuid', str(self.uuid)
args[:0] = '--uuid', str(self.uuid)
return args
def run(self):
......@@ -281,6 +297,10 @@ class NEOProcess(Process):
"""
self.uuid = uuid
@property
def logfile(self):
return self.arg_dict['logfile']
class NEOCluster(object):
SSL = None
......@@ -368,7 +388,7 @@ class NEOCluster(object):
if self.SSL:
kw['ca'], kw['cert'], kw['key'] = self.SSL
self.process_dict.setdefault(node_type, []).append(
NEOProcess(command_dict[node_type], uuid, kw))
NEOProcess(command_dict[node_type], uuid=uuid, **kw))
def setupDB(self, clear_databases=True):
if self.adapter == 'MySQL':
......@@ -480,14 +500,15 @@ class NEOCluster(object):
except (AlreadyStopped, NodeProcessError):
pass
def getZODBStorage(self, **kw):
master_nodes = self.master_nodes.replace('/', ' ')
def getClientConfig(self, **kw):
kw['name'] = self.cluster_name
kw['master_nodes'] = self.master_nodes.replace('/', ' ')
if self.SSL:
kw['ca'], kw['cert'], kw['key'] = self.SSL
result = Storage(
master_nodes=master_nodes,
name=self.cluster_name,
**kw)
return kw
def getZODBStorage(self, **kw):
result = Storage(**self.getClientConfig(**kw))
result.app.max_reconnection_to_master = 10
self.zodb_storage_list.append(result)
return result
......@@ -718,6 +739,7 @@ class NEOFunctionalTest(NeoTestBase):
def setupLog(self):
logging.setup(os.path.join(self.getTempDirectory(), 'test.log'))
logging.resetNids()
def getTempDirectory(self):
# build the full path based on test case and current test method
......
#
# Copyright (C) 2009-2017 Nexedi SA
# Copyright (C) 2009-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -14,6 +14,7 @@
# You should have received a copy of the GNU General Public License
# along with this program. If not, see <http://www.gnu.org/licenses/>.
from __future__ import print_function
import os
import unittest
import transaction
......@@ -25,7 +26,7 @@ from neo.lib.util import makeChecksum, u64
from ZODB.FileStorage import FileStorage
from ZODB.tests.StorageTestBase import zodb_pickle
from persistent import Persistent
from . import NEOCluster, NEOFunctionalTest
from . import NEOCluster, NEOFunctionalTest, NEOProcess
TREE_SIZE = 6
......@@ -120,13 +121,13 @@ class ClientTests(NEOFunctionalTest):
self.__checkTree(tree.right, depth)
self.__checkTree(tree.left, depth)
def __getDataFS(self, reset=False):
def __getDataFS(self):
name = os.path.join(self.getTempDirectory(), 'data.fs')
if reset and os.path.exists(name):
if os.path.exists(name):
os.remove(name)
return FileStorage(file_name=name)
def __populate(self, db, tree_size=TREE_SIZE):
def __populate(self, db, tree_size=TREE_SIZE, with_undo=True):
if isinstance(db.storage, FileStorage):
from base64 import b64encode as undo_tid
else:
......@@ -146,6 +147,7 @@ class ClientTests(NEOFunctionalTest):
t2 = db.lastTransaction()
ob.left = left
transaction.commit()
if with_undo:
undo()
t4 = db.lastTransaction()
undo(t2)
......@@ -174,10 +176,37 @@ class ClientTests(NEOFunctionalTest):
(neo_db, neo_conn) = self.neo.getZODBConnection()
self.__checkTree(neo_conn.root()['trees'])
def __dump(self, storage):
return {u64(t.tid): [(u64(o.oid), o.data_txn and u64(o.data_txn),
def testMigrationTool(self):
dfs_storage = self.__getDataFS()
dfs_db = ZODB.DB(dfs_storage)
self.__populate(dfs_db, with_undo=False)
dump = self.__dump(dfs_storage)
fs_path = dfs_storage.__name__
dfs_db.close()
neo = self.neo
neo.start()
kw = {'cluster': neo.cluster_name, 'quiet': None}
master_nodes = neo.master_nodes.replace('/', ' ')
if neo.SSL:
kw['ca'], kw['cert'], kw['key'] = neo.SSL
p = NEOProcess('neomigrate', fs_path, master_nodes, **kw)
p.start()
p.wait()
os.remove(fs_path)
p = NEOProcess('neomigrate', master_nodes, fs_path, **kw)
p.start()
p.wait()
self.assertEqual(dump, self.__dump(FileStorage(fs_path)))
def __dump(self, storage, sorted=sorted):
return {u64(t.tid): sorted((u64(o.oid), o.data_txn and u64(o.data_txn),
None if o.data is None else makeChecksum(o.data))
for o in t]
for o in t)
for t in storage.iterator()}
def testExport(self):
......@@ -186,10 +215,10 @@ class ClientTests(NEOFunctionalTest):
self.neo.start()
(neo_db, neo_conn) = self.neo.getZODBConnection()
self.__populate(neo_db)
dump = self.__dump(neo_db.storage)
dump = self.__dump(neo_db.storage, list)
# copy neo to data fs
dfs_storage = self.__getDataFS(reset=True)
dfs_storage = self.__getDataFS()
neo_storage = self.neo.getZODBStorage()
dfs_storage.copyTransactionsFrom(neo_storage)
......@@ -209,7 +238,10 @@ class ClientTests(NEOFunctionalTest):
self.neo.start()
neo_db, neo_conn = self.neo.getZODBConnection()
self.__checkTree(neo_conn.root()['trees'])
self.assertEqual(dump, self.__dump(neo_db.storage))
# BUG: The following check is sometimes done whereas the import is not
# finished, resulting in a failure because getReplicationTIDList
# is not implemented by the Importer backend.
self.assertEqual(dump, self.__dump(neo_db.storage, list))
def testIPv6Client(self):
""" Test the connectivity of an IPv6 connection for neo client """
......
#
# Copyright (C) 2009-2017 Nexedi SA
# Copyright (C) 2009-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2009-2017 Nexedi SA
# Copyright (C) 2009-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2009-2017 Nexedi SA
# Copyright (C) 2009-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2009-2017 Nexedi SA
# Copyright (C) 2009-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2009-2017 Nexedi SA
# Copyright (C) 2009-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2009-2017 Nexedi SA
# Copyright (C) 2009-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2009-2017 Nexedi SA
# Copyright (C) 2009-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2009-2017 Nexedi SA
# Copyright (C) 2009-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -61,8 +61,8 @@ class testTransactionManager(NeoUnitTestBase):
tid1 = self.getNextTID()
tid2 = self.getNextTID()
tm.begin(node1, 0, tid1)
tm.clientLost(node1)
self.assertTrue(tid1 not in tm)
self.assertEqual(1, len(list(tm.clientLost(node1))))
self.assertNotIn(tid1, tm)
if __name__ == '__main__':
unittest.main()
#
# Copyright (C) 2018 Nexedi SA
# Copyright (C) 2018-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2009-2017 Nexedi SA
# Copyright (C) 2009-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2009-2017 Nexedi SA
# Copyright (C) 2009-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2009-2017 Nexedi SA
# Copyright (C) 2009-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2009-2017 Nexedi SA
# Copyright (C) 2009-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2009-2017 Nexedi SA
# Copyright (C) 2009-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2009-2017 Nexedi SA
# Copyright (C) 2009-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2010-2017 Nexedi SA
# Copyright (C) 2010-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
# tools/stress is split in such a way that this file can be reused to
# implement another tool to stress an existing cluster, which would be filled
# by a real application.
import curses, os, random, re, select, threading, time
from collections import deque
from neo.lib import logging, protocol
from neo.lib.app import BaseApplication
from neo.lib.debug import register as registerLiveDebugger
from neo.lib.exception import PrimaryFailure
from neo.lib.protocol import ClusterStates, NodeStates, NodeTypes, Packets
from neo.admin.app import Application as AdminApplication
from neo.admin.handler import MasterEventHandler
class Handler(MasterEventHandler):
def answerClusterState(self, conn, state):
super(Handler, self).answerClusterState(conn, state)
self.app.refresh('state')
def answerPartitionTable(self, *args):
super(Handler, self).answerPartitionTable(*args)
self.app.refresh('pt')
def sendPartitionTable(self, *args):
raise AssertionError
def notifyPartitionChanges(self, *args):
super(Handler, self).notifyPartitionChanges(*args)
self.app.refresh('pt')
def answerLastIDs(self, conn, *args):
self.app.answerLastIDs(*args)
def notifyNodeInformation(self, conn, timestamp, node_list):
for node_type, addr, uuid, state, id_timestamp in node_list:
if node_type == NodeTypes.CLIENT and state == NodeStates.UNKNOWN:
self.app.clientDown()
break
getStorageList = self.app.nm.getStorageList
before = [node for node in getStorageList() if node.isRunning()]
super(Handler, self).notifyNodeInformation(conn, timestamp, node_list)
self.app.notifyNodeInformation(
{node for node in getStorageList() if node.isRunning()}
.difference(before))
class StressApplication(AdminApplication):
cluster_state = server = uuid = None
listening_conn = True
restart_ratio = float('inf') # no firewall support
_stress = False
def __init__(self, ssl, master_nodes):
BaseApplication.__init__(self, ssl)
for address in master_nodes:
self.nm.createMaster(address=address)
self.pt = None
self.master_event_handler = Handler(self)
self.reset()
registerLiveDebugger(on_log=self.log)
self.failing = set()
self.restart_lock = threading.Lock()
def close(self):
BaseApplication.close(self)
def run(self):
visibility = None
from logging import disable, ERROR
disable(ERROR)
self.stdscr = curses.initscr()
try:
curses.noecho()
curses.cbreak()
self.stdscr.keypad(1)
visibility = curses.curs_set(0)
self._run()
finally:
if visibility:
curses.curs_set(visibility)
self.stdscr.keypad(0)
curses.echo()
curses.nocbreak()
curses.endwin()
def _run(self):
stdscr = self.stdscr
r, w = os.pipe()
l = threading.Lock()
stdscr.nodelay(1)
input_queue = deque()
def input_read():
x = []
while 1:
c = stdscr.getch()
if c < 0:
if x:
input_queue.append(x)
return input_queue
x.append(c)
def input_thread():
try:
poll = select.poll()
poll.register(0, select.POLLIN)
poll.register(r, select.POLLIN)
while 1:
for fd, _ in poll.poll():
if fd:
return
with l:
empty = not input_queue
if input_read() and empty:
self.em.wakeup()
finally:
os.close(r)
t = threading.Thread(target=input_thread)
t.deamon = True
wait = None
try:
t.start()
self.startCluster()
self.refresh('stress', False)
while 1:
self.failing.clear()
try:
self.connectToPrimary()
self.askLastIDs()
while 1:
self.em.poll(1)
with l:
if input_read():
for x in input_queue:
try:
x, = x
except ValueError:
continue
if x == curses.KEY_RESIZE:
self.refresh()
elif x == curses.KEY_F1:
self.stress()
else:
try:
x = chr(x)
except ValueError:
continue
if x == 'q':
return
input_queue.clear()
except PrimaryFailure:
logging.error('primary master is down')
if self.cluster_state == ClusterStates.STOPPING:
break
self.primaryFailure()
finally:
if self._stress:
self.stress()
wait = time.time()
finally:
os.write(w, '\0')
os.close(w)
t.join()
self.stopCluster(wait)
def primaryFailure(self):
raise
def startCluster(self):
raise NotImplementedError
def stopCluster(self, wait):
raise NotImplementedError
def clientDown(self):
send = self.master_conn.send
send(Packets.FlushLog())
send(Packets.SetClusterState(ClusterStates.STOPPING))
def notifyNodeInformation(self, node_list):
for node in node_list:
self.failing.discard(node.getUUID())
def askLastIDs(self):
conn = self.master_conn
if conn:
conn.ask(Packets.AskLastIDs())
def answerLastIDs(self, loid, ltid):
self.loid = loid
self.ltid = ltid
self.em.setTimeout(int(time.time() + 1), self.askLastIDs)
if self._stress:
node_list = self.nm.getStorageList()
random.shuffle(node_list)
fw = []
kill = []
restart_ratio = self.restart_ratio
for node in node_list:
nid = node.getUUID()
if nid in self.failing:
if restart_ratio <= 1:
fw.append(nid)
continue
running = node.isRunning()
if running or restart_ratio <= 1:
self.failing.add(nid)
if self.pt.operational(self.failing):
(kill if running and random.random() < restart_ratio
else fw).append(nid)
if len(self.failing) == self._fault_count:
break
else:
self.failing.remove(nid)
if fw or kill:
for nid in fw:
self.tcpReset(nid)
if kill:
t = threading.Thread(target=self._restart, args=kill)
t.daemon = 1
t.start()
self.refresh('pt', False)
self.refresh('ids')
def _restart(self, *nids):
with self.restart_lock:
self.restartStorages(nids)
def tcpReset(self, nid):
raise NotImplementedError
def restartStorages(self, nids):
raise NotImplementedError
def refresh(self, what=None, do=True):
stdscr = self.stdscr
try:
y = 0
if what in (None, 'stress'):
stdscr.addstr(y, 0, 'stress: %s (toggle with F1)\n'
% ('yes' if self._stress else 'no'))
y += 1
if what in (None, 'state'):
stdscr.addstr(y, 0, 'cluster state: %s\n' % self.cluster_state)
y += 1
if what in (None, 'ids'):
self.refresh_ids(y)
h = stdscr.getyx()[0] - y
clear = self._ids_height - h
if clear:
self._ids_height = h
what = None
else:
clear = None
y += self._ids_height
if what in (None, 'pt'):
pt = self.pt
n = len(str(pt.np-1))
node_list = sorted(pt.count_dict)
attr = curses.A_NORMAL, curses.A_BOLD
stdscr.addstr(y, 0, 'pt id: %s\n %s' % (pt.getID(), ' ' * n))
for node in node_list:
stdscr.addstr(
protocol.node_state_prefix_dict[node.getState()],
attr[node.getUUID() in self.failing])
stdscr.addstr('\n')
x = '%{}s'.format(n)
n = pt.nr + 1
split = re.compile('[^OC]+|[OC]+').findall
for i, r in enumerate(pt._formatRows(node_list)):
stdscr.addstr(x % i, attr[r.count('U') != n])
for i, r in enumerate(split(': %s\n' % r)):
stdscr.addstr(r, attr[i & 1])
if clear:
stdscr.addstr('\n' * clear)
except curses.error:
pass
if do:
stdscr.refresh()
# _ids_height
def refresh_ids(self, y):
raise NotImplementedError
def stress(self):
self._stress = not self._stress
self.refresh('stress')
#
# Copyright (C) 2009-2017 Nexedi SA
# Copyright (C) 2009-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2009-2017 Nexedi SA
# Copyright (C) 2009-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2009-2017 Nexedi SA
# Copyright (C) 2009-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2006-2017 Nexedi SA
# Copyright (C) 2006-2019 Nexedi SA
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2011-2017 Nexedi SA
# Copyright (C) 2011-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -23,10 +23,10 @@ from contextlib import contextmanager
from itertools import count
from functools import partial, wraps
from zlib import decompress
from ..mock import Mock
import transaction, ZODB
import neo.admin.app, neo.master.app, neo.storage.app
import neo.client.app, neo.neoctl.app
from neo.admin.handler import MasterEventHandler
from neo.client import Storage
from neo.lib import logging
from neo.lib.connection import BaseConnection, \
......@@ -34,8 +34,10 @@ from neo.lib.connection import BaseConnection, \
from neo.lib.connector import SocketConnector, ConnectorException
from neo.lib.handler import EventHandler
from neo.lib.locking import SimpleQueue
from neo.lib.protocol import ClusterStates, Enum, NodeStates, NodeTypes, Packets
from neo.lib.protocol import uuid_str, \
ClusterStates, Enum, NodeStates, NodeTypes, Packets
from neo.lib.util import cached_property, parseMasterList, p64
from neo.master.recovery import RecoveryManager
from .. import (getTempDirectory, setupMySQLdb,
ImporterConfigParser, NeoTestBase, Patch,
ADDRESS_TYPE, IP_VERSION_FORMAT_DICT, DB_PREFIX, DB_SOCKET, DB_USER)
......@@ -133,9 +135,12 @@ class Serialized(object):
detect which node has a readable epoll object.
"""
check_timeout = False
_disabled = False
@classmethod
def init(cls):
if cls._disabled:
return
cls._busy = set()
cls._busy_cond = threading.Condition(threading.Lock())
cls._epoll = select.epoll()
......@@ -152,6 +157,8 @@ class Serialized(object):
@classmethod
def stop(cls):
if cls._disabled:
return
assert not cls._fd_dict, ("file descriptor leak (%r)\nThis may happen"
" when a test fails, in which case you can see the real exception"
" by disabling this one." % cls._fd_dict)
......@@ -162,6 +169,25 @@ class Serialized(object):
def _sort_key(cls, fd_event):
return -cls._fd_dict[fd_event[0]]._last # NOTE ._last set to time.time()
@classmethod
@contextmanager
def until(cls, patched=None, **patch):
if cls._disabled:
if patched is None:
yield int
else:
l = threading.Lock()
l.acquire()
(name, patch), = patch.iteritems()
def release():
p.revert()
l.release()
with Patch(patched, **{name: lambda *args, **kw:
patch(release, *args, **kw)}) as p:
yield l.acquire
else:
yield cls.tic
@classmethod
@contextmanager
def pdb(cls):
......@@ -188,6 +214,10 @@ class Serialized(object):
# We also increase SocketConnector.SOMAXCONN in tests so that
# a connection attempt is never delayed inside the kernel.
timeout=0):
if cls._disabled:
if timeout:
time.sleep(timeout)
return
# If you're in a pdb here, 'n' switches to another thread
# (the following lines are not supposed to be debugged into)
with cls._tic_lock, cls.pdb():
......@@ -223,6 +253,8 @@ class Serialized(object):
cls._sched_lock.acquire()
def __init__(self, app, busy=True):
if self._disabled:
return
self._epoll = app.em.epoll
app.em.epoll = self
# XXX: It may have been initialized before the SimpleQueue is patched.
......@@ -281,16 +313,13 @@ class TestSerialized(Serialized): # NOTE used only in .NeoCTL
if r:
return r
Serialized.tic(step=1, timeout=.001)
ConnectionFilter.log()
raise Exception("tic is looping forever")
return self._epoll.poll(timeout)
class Node(object):
@staticmethod
def convertInitArgs(**kw):
return {'get' + k.capitalize(): v for k, v in kw.iteritems()}
def getConnectionList(self, *peers):
addr = lambda c: c and (c.addr if c.is_server else c.getAddress())
addr_set = {addr(c.connector) for peer in peers
......@@ -354,19 +383,18 @@ class ServerNode(Node):
threading.Thread.__init__(self)
self.daemon = True
self.node_name = '%s_%u' % (self.node_type, port)
kw.update(getCluster=name, getBind=address,
getMasters=master_nodes and parseMasterList(master_nodes))
# -> app.__init__() ; Mock serves as config
super(ServerNode, self).__init__(Mock(kw))
kw.update(cluster=name, bind=address,
masters=master_nodes and parseMasterList(master_nodes))
# -> app.__init__()
super(ServerNode, self).__init__(kw)
def getVirtualAddress(self):
return self._init_args['address']
def resetNode(self, **kw):
assert not self.is_alive()
kw = self.convertInitArgs(**kw)
init_args = self._init_args
init_args['getReset'] = False
init_args['reset'] = False
assert set(kw).issubset(init_args), (kw, init_args)
init_args.update(kw)
self.close()
......@@ -382,6 +410,7 @@ class ServerNode(Node):
finally:
self._afterRun()
logging.debug('stopping %r', self)
if isinstance(self.em.epoll, Serialized):
self.em.epoll.exit()
def _afterRun(self):
......@@ -421,7 +450,7 @@ class StorageApplication(ServerNode, neo.storage.app.Application):
self.master_conn.close()
def getAdapter(self):
return self._init_args['getAdapter']
return self._init_args['adapter']
# get {} oid -> nlocks ; !0 - in staging area for commit (and for how many txn), oid from whole DB
def getDataLockInfo(self):
......@@ -444,12 +473,13 @@ class ClientApplication(Node, neo.client.app.Application):
self.poll_thread.node_name = name
# Smaller cache to speed up tests that checks behaviour when it's too
# small. See also NEOCluster.cache_size
self._cache._max_size //= 1024
self._cache.max_size //= 1024
def _run(self):
try:
super(ClientApplication, self)._run()
finally:
if isinstance(self.em.epoll, Serialized):
self.em.epoll.exit()
def start(self):
......@@ -462,13 +492,20 @@ class ClientApplication(Node, neo.client.app.Application):
conn = self._getMasterConnection()
else:
assert isinstance(peer, StorageApplication)
conn = self.cp.getConnForNode(self.nm.getByUUID(peer.uuid))
conn = self.getStorageConnection(self.nm.getByUUID(peer.uuid))
yield conn
def extraCellSortKey(self, key):
return Patch(self.cp, getCellSortKey=lambda orig, cell:
return Patch(self, getCellSortKey=lambda orig, cell:
(orig(cell, lambda: key(cell)), random.random()))
def closeAllStorageConnections(self):
for node in self.nm.getStorageList():
conn = node._connection # XXX
if conn is not None:
conn.setReconnectionNoDelay()
conn.close()
class NeoCTL(neo.neoctl.app.NeoCTL):
def __init__(self, *args, **kw):
......@@ -577,6 +614,20 @@ class ConnectionFilter(object):
else:
del cls.filter_queue[conn]
@classmethod
def log(cls):
try:
if cls.filter_queue:
logging.info('%s:', cls.__name__)
for conn, queue in cls.filter_queue.iteritems():
app = NEOThreadedTest.getConnectionApp(conn)
logging.info(' %s %s:', uuid_str(app.uuid), conn)
for p in queue:
logging.info(' #0x%04x %s',
p.getId(), p.__class__.__name__)
except Exception:
logging.exception('')
def add(self, filter, *patches):
with self.lock:
self.filter_dict[filter] = patches
......@@ -621,6 +672,8 @@ class NEOCluster(object):
def __init__(orig, self): # temporary definition for SimpleQueue patch
orig(self)
if Serialized._disabled:
return
lock = self._lock
def _lock(blocking=True):
if blocking:
......@@ -629,6 +682,7 @@ class NEOCluster(object):
if lock(False):
return True
Serialized.tic(step=1, quiet=True, timeout=.001)
ConnectionFilter.log()
raise Exception("tic is looping forever")
return lock(False)
self._lock = _lock
......@@ -685,15 +739,15 @@ class NEOCluster(object):
master_list = [MasterApplication.newAddress()
for _ in xrange(master_count)]
self.master_nodes = ' '.join('%s:%s' % x for x in master_list)
kw = Node.convertInitArgs(replicas=replicas, adapter=adapter,
kw = dict(replicas=replicas, adapter=adapter,
partitions=partitions, reset=clear_databases, dedup=dedup)
kw['cluster'] = weak_self = weakref.proxy(self)
kw['getSSL'] = self.SSL
kw['ssl'] = self.SSL
if upstream is not None:
self.upstream = weakref.proxy(upstream)
kw.update(getUpstreamCluster=upstream.name,
getUpstreamMasters=parseMasterList(upstream.master_nodes))
self.master_list = [MasterApplication(getAutostart=autostart,
kw.update(upstream_cluster=upstream.name,
upstream_masters=parseMasterList(upstream.master_nodes))
self.master_list = [MasterApplication(autostart=autostart,
address=x, **kw)
for x in master_list]
if db_list is None:
......@@ -715,8 +769,9 @@ class NEOCluster(object):
db = os.path.join(getTempDirectory(), '%s.conf')
with open(db % tuple(db_list), "w") as f:
cfg.write(f)
kw["getAdapter"] = "Importer"
self.storage_list = [StorageApplication(getDatabase=db % x, **kw)
kw["adapter"] = "Importer"
kw['wait'] = 0
self.storage_list = [StorageApplication(database=db % x, **kw)
for x in db_list]
self.admin_list = [AdminApplication(**kw)]
......@@ -769,7 +824,7 @@ class NEOCluster(object):
@property
def cache_size(self):
return self.client._cache._max_size
return self.client._cache.max_size
###
def __enter__(self):
......@@ -782,22 +837,41 @@ class NEOCluster(object):
self.started = True
self._patch()
self.neoctl = NeoCTL(self.admin.getVirtualAddress(), ssl=self.SSL)
for node in self.master_list if master_list is None else master_list:
if master_list is None:
master_list = self.master_list
if storage_list is None:
storage_list = self.storage_list
def answerPartitionTable(release, orig, *args):
orig(*args)
release()
def dispatch(release, orig, handler, *args):
orig(handler, *args)
node_list = handler.app.nm.getStorageList(only_identified=True)
if len(node_list) == len(storage_list) and not any(
node.getConnection().isPending() for node in node_list):
release()
expected_state = (ClusterStates.RECOVERING,) if recovering else (
ClusterStates.RUNNING, ClusterStates.BACKINGUP)
def notifyClusterInformation(release, orig, handler, conn, state):
orig(handler, conn, state)
if state in expected_state:
release()
with Serialized.until(MasterEventHandler,
answerPartitionTable=answerPartitionTable) as tic1, \
Serialized.until(RecoveryManager, dispatch=dispatch) as tic2, \
Serialized.until(MasterEventHandler,
notifyClusterInformation=notifyClusterInformation) as tic3:
for node in master_list:
node.start()
for node in self.admin_list:
node.start()
Serialized.tic()
if storage_list is None:
storage_list = self.storage_list
tic1()
for node in storage_list:
node.start()
Serialized.tic()
if recovering:
expected_state = ClusterStates.RECOVERING
else:
tic2()
if not recovering:
self.startCluster()
Serialized.tic()
expected_state = ClusterStates.RUNNING, ClusterStates.BACKINGUP
tic3()
self.checkStarted(expected_state, storage_list)
def checkStarted(self, expected_state, storage_list=None):
......@@ -989,6 +1063,7 @@ class NEOThreadedTest(NeoTestBase):
self.__run_count[test_id] = 1 + i
if i:
test_id += '-%s' % i
logging._nid_dict.clear()
logging.setup(os.path.join(getTempDirectory(), test_id + '.log'))
return LoggerThreadName()
......@@ -1005,7 +1080,7 @@ class NEOThreadedTest(NeoTestBase):
@contextmanager
def getLoopbackConnection(self):
app = MasterApplication(address=BIND,
getSSL=NEOCluster.SSL, getReplicas=0, getPartitions=1)
ssl=NEOCluster.SSL, replicas=0, partitions=1)
try:
handler = EventHandler(app)
app.listening_conn = ListeningConnection(app, handler, app.server)
......@@ -1069,9 +1144,14 @@ class NEOThreadedTest(NeoTestBase):
@staticmethod
def noConnection(jar, storage):
return Patch(jar.db().storage.app.cp, getConnForNode=lambda orig, node:
return Patch(jar.db().storage.app,
getStorageConnection=lambda orig, node:
None if node.getUUID() == storage.uuid else orig(node))
@staticmethod
def getConnectionApp(conn):
return getattr(conn.getHandler(), 'app', None)
@staticmethod
def readCurrent(ob):
ob._p_activate()
......@@ -1132,12 +1212,16 @@ def predictable_random(seed=None):
return wraps(wrapped)(wrapper)
return decorator
def with_cluster(start_cluster=True, **cluster_kw):
def with_cluster(serialized=True, start_cluster=True, **cluster_kw):
def decorator(wrapped):
def wrapper(self, *args, **kw):
try:
Serialized._disabled = not serialized
with NEOCluster(**cluster_kw) as cluster:
if start_cluster:
cluster.start()
return wrapped(self, cluster, *args, **kw)
finally:
Serialized._disabled = False
return wraps(wrapped)(wrapper)
return decorator
#
# Copyright (C) 2011-2017 Nexedi SA
# Copyright (C) 2011-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -35,9 +35,9 @@ from neo.lib.handler import DelayEvent, EventHandler
from neo.lib import logging
from neo.lib.protocol import (CellStates, ClusterStates, NodeStates, NodeTypes,
Packets, Packet, uuid_str, ZERO_OID, ZERO_TID, MAX_TID)
from .. import expectedFailure, unpickle_state, Patch, TransactionalResource
from .. import unpickle_state, Patch, TransactionalResource
from . import ClientApplication, ConnectionFilter, LockLock, NEOCluster, \
NEOThreadedTest, RandomConflictDict, ThreadId, with_cluster
NEOThreadedTest, RandomConflictDict, Serialized, ThreadId, with_cluster
from neo.lib.util import add64, makeChecksum, p64, u64
from neo.client.exception import NEOPrimaryMasterLost, NEOStorageError
from neo.client.transactions import Transaction
......@@ -46,6 +46,7 @@ from neo.storage.database import DatabaseFailure
from neo.storage.handlers.client import ClientOperationHandler
from neo.storage.handlers.identification import IdentificationHandler
from neo.storage.handlers.initialization import InitializationHandler
from neo.storage.replicator import Replicator
class PCounter(Persistent):
value = 0
......@@ -161,6 +162,7 @@ class Test(NEOThreadedTest):
undo.tpc_finish(txn)
t.begin()
self.assertEqual(ob.value, 5)
self.assertFalse(cluster.storage.dm.getOrphanList())
return ob
@with_cluster()
......@@ -173,9 +175,9 @@ class Test(NEOThreadedTest):
with Patch(PCounterWithResolution, _p_resolveConflict=resolve):
self.assertEqual(self._testUndoConflict(cluster, 1, 3).x, big)
@expectedFailure(POSException.ConflictError)
@with_cluster()
def testUndoConflictDuringStore(self, cluster):
with self.expectedFailure(POSException.ConflictError): \
self._testUndoConflict(cluster, 1)
def testStorageDataLock(self, dedup=False):
......@@ -245,7 +247,8 @@ class Test(NEOThreadedTest):
s.resetNode()
storage.app.max_reconnection_to_master = 0
self.assertRaises(NEOPrimaryMasterLost, storage.tpc_vote, t1)
expectedFailure(self.assertFalse)(s.dm.getOrphanList())
with self.expectedFailure(): \
self.assertFalse(s.dm.getOrphanList())
@with_cluster(storage_count=1)
def testDelayedUnlockInformation(self, cluster):
......@@ -466,7 +469,7 @@ class Test(NEOThreadedTest):
s0, s1 = cluster.client.nm.getStorageList()
conn = s0.getConnection()
self.assertFalse(conn.isClosed())
getCellSortKey = cluster.client.cp.getCellSortKey
getCellSortKey = cluster.client.getCellSortKey
self.assertEqual(getCellSortKey(s0, good), 0)
cluster.neoctl.dropNode(s0.getUUID())
self.assertEqual([s1], cluster.client.nm.getStorageList())
......@@ -740,25 +743,22 @@ class Test(NEOThreadedTest):
@with_cluster(replicas=1)
def testStorageReconnectDuringStore(self, cluster):
if 1:
t, c = cluster.getTransaction()
c.root()[0] = 'ok'
cluster.client.cp.closeAll()
cluster.client.closeAllStorageConnections()
t.commit() # store request
@with_cluster(storage_count=2, partitions=2)
def testStorageReconnectDuringTransactionLog(self, cluster):
if 1:
t, c = cluster.getTransaction()
cluster.client.cp.closeAll()
cluster.client.closeAllStorageConnections()
tid, (t1,) = cluster.client.transactionLog(
ZERO_TID, c.db().lastTransaction(), 10)
@with_cluster(storage_count=2, partitions=2)
def testStorageReconnectDuringUndoLog(self, cluster):
if 1:
t, c = cluster.getTransaction()
cluster.client.cp.closeAll()
cluster.client.closeAllStorageConnections()
t1, = cluster.client.undoLog(0, 10)
@with_cluster(storage_count=2, replicas=1)
......@@ -979,9 +979,48 @@ class Test(NEOThreadedTest):
self.assertFalse(invalidations(c1))
self.assertEqual(x1.value, 1)
@with_cluster(serialized=False)
def testExternalInvalidation2(self, cluster):
t, c = cluster.getTransaction()
r = c.root()
x = r[''] = PCounter()
t.commit()
tid1 = x._p_serial
nonlocal_ = [0, 1]
l1 = threading.Lock(); l1.acquire()
l2 = threading.Lock(); l2.acquire()
def invalidateObjects(orig, *args):
if not nonlocal_[0]:
l1.acquire()
orig(*args)
nonlocal_[0] += 1
if nonlocal_[0] == 2:
l2.release()
def _cache_lock_release(orig):
orig()
if nonlocal_[1]:
nonlocal_[1] = 0
l1.release()
l2.acquire()
with cluster.newClient() as client, \
Patch(client.notifications_handler,
invalidateObjects=invalidateObjects):
client.sync()
with cluster.master.filterConnection(client) as mc2:
mc2.delayInvalidateObjects()
x._p_changed = 1
t.commit()
tid2 = x._p_serial
self.assertEqual((tid1, tid2), client.load(x._p_oid)[1:])
r._p_changed = 1
t.commit()
with Patch(client, _cache_lock_release=_cache_lock_release):
self.assertEqual((tid2, None), client.load(x._p_oid)[1:])
self.assertEqual(nonlocal_, [2, 0])
@with_cluster(storage_count=2, partitions=2)
def testReadVerifyingStorage(self, cluster):
if 1:
s1, s2 = cluster.sortStorageList()
t1, c1 = cluster.getTransaction()
c1.root()['x'] = x = PCounter()
t1.commit()
......@@ -989,12 +1028,11 @@ class Test(NEOThreadedTest):
with cluster.newClient(1) as db:
t2, c2 = cluster.getTransaction(db)
r = c2.root()
r['y'] = None
r._p_changed = 1
self.readCurrent(r['x'])
# Force the new tid to be even, like the modified oid and
# unlike the oid on which we used readCurrent. Thus we check
# that the node containing only the partition 1 is also
# involved in tpc_finish.
# Force the new tid to be even, like the modified oid and unlike
# the oid on which we used readCurrent. Thus we check that the node
# containing only the partition 1 is also involved in tpc_finish.
with cluster.moduloTID(0):
t2.commit()
for storage in cluster.storage_list:
......@@ -1005,11 +1043,35 @@ class Test(NEOThreadedTest):
# transaction before the last one, and clearing the cache before
# reloading x.
c1._storage.load(x._p_oid)
t0, t1, t2 = c1.db().storage.iterator()
self.assertEqual(map(u64, t0.oid_list), [0])
self.assertEqual(map(u64, t1.oid_list), [0, 1])
# Check oid 1 is part of transaction metadata.
self.assertEqual(t2.oid_list, t1.oid_list)
# In particular, check oid 1 is listed in the last transaction.
self.assertEqual([[0], [0, 1], [0, 1]],
[map(u64, t.oid_list) for t in c1.db().storage.iterator()])
# Another test, this time to check we also vote to storage nodes
# that are only involved in checking current serial.
t1.begin()
s2c2, = s2.getConnectionList(db.storage.app)
def t1_vote(txn):
# Check that the storage hasn't answered to the store,
# which means that a lock is still taken for r['x'] by t2.
self.tic()
txn_context = cluster.client._txn_container.get(txn)
empty = txn_context.queue.empty()
ll()
self.assertTrue(empty)
def t2_vote(_):
s2c2.close()
commit1.start()
ll()
TransactionalResource(t1, 0, tpc_vote=t1_vote)
x.value += 1
TransactionalResource(t2, 1, tpc_vote=t2_vote)
r._p_changed = 1
self.readCurrent(r['x'])
with cluster.moduloTID(0), LockLock() as ll:
commit1 = self.newPausedThread(t1.commit)
t2.commit()
commit1.join()
@with_cluster()
def testClientReconnection(self, cluster):
......@@ -1451,11 +1513,11 @@ class Test(NEOThreadedTest):
reports a conflict after that this conflict was fully resolved with
another node.
"""
def answerStoreObject(orig, conn, conflict, oid):
def answerStoreObject(orig, conn, conflict, oid, serial):
if not conflict:
p.revert()
ll()
orig(conn, conflict, oid)
orig(conn, conflict, oid, serial)
if 1:
s0, s1 = cluster.storage_list
t1, c1 = cluster.getTransaction()
......@@ -1602,7 +1664,7 @@ class Test(NEOThreadedTest):
with cluster.newClient(1) as db:
t2, c2 = cluster.getTransaction(db)
with self.noConnection(c1, s2), self.noConnection(c2, s1):
cluster.client.cp.connection_dict[s2.uuid].close()
cluster.client.nm.getByUUID(s2.uuid).getConnection().close()
self.tic()
for c1_aborts in 0, 1:
# 0: C1 finishes, C2 vote fails
......@@ -1652,6 +1714,74 @@ class Test(NEOThreadedTest):
self.tic()
self.assertPartitionTable(cluster, pt)
@with_cluster(partitions=2, storage_count=2)
def testMasterArbitratingVoteAfterFailedVoteTransaction(self, cluster):
"""
Check that the master node arbitrates the vote when a failure happens
at the end of the first phase (VoteTransaction).
"""
t, c = cluster.getTransaction()
tid = cluster.client.last_tid
r = c.root()
r[0] = ''
delayed = []
with cluster.moduloTID(1), ConnectionFilter() as f:
@f.delayAskVoteTransaction
def _(conn):
delayed.append(None)
s, = (s for s in cluster.storage_list if conn.uuid == s.uuid)
conn, = s.getConnectionList(cluster.client)
conn.em.wakeup(conn.close)
return False
self.assertRaises(NEOStorageError, t.commit)
self.tic()
self.assertEqual(len(delayed), 1)
self.assertEqual(tid, cluster.client.last_tid)
self.assertEqual(cluster.neoctl.getClusterState(),
ClusterStates.RUNNING)
@with_cluster(storage_count=2, replicas=1)
def testPartitionNotFullyWriteLocked(self, cluster):
"""
Make sure all oids are write-locked at least once, which is not
guaranteed by just the storage/master nodes when a readable cell
becomes OUT_OF_DATE during a commit. This scenario is special in that
the other readable cell was only writable at the beginning of the
transaction and the replication finished just before the node failure.
The test uses a conflict to detect lockless writes.
"""
s0, s1 = cluster.storage_list
t, c = cluster.getTransaction()
r = c.root()
x = r[''] = PCounterWithResolution()
t.commit()
s1c, = s1.getConnectionList(cluster.client)
s0.stop()
cluster.join((s0,))
s0.resetNode()
x.value += 2
def vote(_):
f.remove(delay)
self.tic()
s1.stop()
cluster.join((s1,))
TransactionalResource(t, 0, tpc_vote=vote)
with ConnectionFilter() as f, cluster.newClient(1) as db:
t2, c2 = cluster.getTransaction(db)
c2.root()[''].value += 3
t2.commit()
f.delayAnswerStoreObject(lambda conn: conn is s1c)
delay = f.delayAskFetchTransactions()
s0.start()
self.tic()
self.assertRaisesRegexp(NEOStorageError,
'^partition 0 not fully write-locked$',
t.commit)
cluster.client._cache.clear()
t.begin()
x._p_deactivate()
self.assertEqual(x.value, 3)
@with_cluster()
def testAbortTransaction(self, cluster):
t, c = cluster.getTransaction()
......@@ -1685,7 +1815,7 @@ class Test(NEOThreadedTest):
f.add(delayAnswerStoreObject, Patch(Transaction, written=written))
def delayAnswerStoreObject(conn, packet):
return (isinstance(packet, Packets.AnswerStoreObject)
and getattr(conn.getHandler(), 'app', None) is s)
and self.getConnectionApp(conn) is s)
def written(orig, *args):
orig(*args)
f.remove(delayAnswerStoreObject)
......@@ -1939,7 +2069,7 @@ class Test(NEOThreadedTest):
def changes(r1, r2, r3):
r1['b'].value = 1
r1['d'].value = 2
r2['a'].value = '*' * r2._p_jar.db().storage._cache._max_size
r2['a'].value = '*' * r2._p_jar.db().storage._cache.max_size
r2['b'].value = 3
r2['c'].value = 4
r3['a'].value = 5
......@@ -2037,7 +2167,7 @@ class Test(NEOThreadedTest):
# Delay the conflict for the second store of 'a' by t3.
delay_conflict = {s0.uuid: [1], s1.uuid: [1,0]}
def delayConflict(conn, packet):
app = conn.getHandler().app
app = self.getConnectionApp(conn)
if (isinstance(packet, Packets.AnswerStoreObject)
and packet.decode()[0]):
conn, = cluster.client.getConnectionList(app)
......@@ -2076,6 +2206,7 @@ class Test(NEOThreadedTest):
1: ['StoreTransaction'],
3: [4, 'StoreTransaction'],
})
self.assertFalse(s1.dm.getOrphanList())
@with_cluster(replicas=1)
def testNotifyReplicated2(self, cluster):
......@@ -2120,6 +2251,319 @@ class Test(NEOThreadedTest):
t2.begin()
self.assertEqual([4, 6], [r[x].value for x in 'ab'])
@with_cluster(replicas=1, partitions=2)
def testNotifyReplicated3(self, cluster):
s0, s1 = cluster.storage_list
t1, c1 = cluster.getTransaction()
r = c1.root()
r[''] = PCounter()
t1.commit()
s1.stop()
cluster.join((s1,))
s1.resetNode()
nonlocal_ = [0]
class Abort(Exception):
pass
with cluster.newClient(1) as db, Patch(Replicator,
_nextPartitionSortKey=lambda orig, self, offset: offset):
t3, c3 = cluster.getTransaction(db)
with ConnectionFilter() as f, self.noConnection(c3, s0):
@f.delayAnswerFetchObjects
def delay(_):
if nonlocal_:
return nonlocal_.pop()
s1.start()
self.tic()
r[''].value += 1
r._p_changed = 1
t2, c2 = cluster.getTransaction()
c2.root()._p_changed = 1
def t1_rebase(*args, **kw):
self.tic()
f.remove(delay)
yield 0
@self.newPausedThread
def commit23():
t2.commit()
c3.root()[''].value += 3
with self.assertRaises(Abort) as cm:
t3.commit()
self.assertTrue(*cm.exception.args)
def t3_commit(txn):
# Check that the storage hasn't answered to the store,
# which means that a lock is still taken for r[''] by t1.
self.tic()
txn_context = db.storage.app._txn_container.get(txn)
raise Abort(txn_context.queue.empty())
TransactionalResource(t3, 1, commit=t3_commit)
with self.thread_switcher((commit23,),
(1, 1, 0, 0, t1_rebase, 0, 0, 0, 1, 1, 1, 1, 0),
('tpc_begin', 'tpc_begin', 0, 1, 0,
'RebaseTransaction', 'RebaseTransaction',
'AnswerRebaseTransaction', 'AnswerRebaseTransaction',
'StoreTransaction', 'tpc_begin', 1, 'tpc_abort')) as end:
self.assertRaises(POSException.ConflictError, t1.commit)
commit23.join()
self.assertEqual(end, {0: ['tpc_abort']})
self.assertPartitionTable(cluster, 'UU|UU')
@with_cluster(partitions=2, storage_count=2)
def testConflictAfterLocklessWrite(self, cluster):
"""
Show that in case of a write to an outdated cell, the client must
discard the answer if it comes after a resolved conflict, as the client
would not have the data anymore to solve a second conflict (deadlock
avoidance). This test reproduces a case where the storage node can't
easily return the correct data back to the client.
The scenario focuses on object A (oid 1) and node s0, which is
initially replicating partition 1:
1. t1 writes A: s1 conflicts and the answer from s0 is delayed
2. t1 writes B: a deadlock is triggered by s0 and internally, the write
of A is not considered lockless anymore
3. replication of partition 1 finishes: A is marked as locked normally
(which can be considered wrong but discarding the write at that
point is not trivial and anyway another write is coming)
4. t1 resolves A: s1 is not yet notified of the deadlock and accepts
5. t1 receives the answer for the first write of A to s1: if it didn't
discard it, it would mark the write of A as completed on all nodes
6. t1 starts resolving the deadlock, s0 conflicts for the second store
and returns that A needs to be rebased (like in 3, questionable)
7. the answer of s0 for the rebasing of A contains data from the first
write and it is processed first: this is not an issue if the client
still has the data (i.e. not moved to transaction cache, or
discarded because the cache is too small)
"""
t1, c1 = cluster.getTransaction()
r = c1.root()
for x in 'ab':
r[x] = PCounterWithResolution()
t1.commit()
cluster.stop(replicas=1)
cluster.start()
s0, s1 = cluster.sortStorageList()
t1, c1 = cluster.getTransaction()
r = c1.root()
r['a'].value += 1
r['b'].value += 2
with cluster.newClient(1) as db, ConnectionFilter() as f:
delayReplication = f.delayAnswerFetchObjects()
delayStore = f.delayAnswerStoreObject(lambda conn:
conn.uuid == cluster.client.uuid and
self.getConnectionApp(conn) is s0)
delayDeadlock = f.delayNotifyDeadlock()
delayRebase = f.delayAnswerRebaseObject(lambda conn:
# to first process the one from s0
self.getConnectionApp(conn) is s1)
cluster.neoctl.tweakPartitionTable()
self.tic()
t2, c2 = cluster.getTransaction(db)
r = c2.root()
r['a'].value += 3 # for a first conflict on t1/s1
t2.commit()
r['b'].value += 4 # for a deadlock on t1/s0
r['a'].value += 5 # for a second conflict on t1/s0
def t1_b(*args, **kw):
self.tic() # make sure t2/b will be processed before t1/b
yield 0
def t1_resolve(*args, **kw):
f.remove(delayReplication)
self.tic()
yield 1
f.remove(delayStore)
self.tic()
f.remove(delayDeadlock)
def t2_vote(*args, **kw):
yield 0
# From now own, prefer reading from s0,
# in case that packets from s1 are blocked by the filter.
no_read.append(s1.uuid)
def t1_end(*args, **kw):
yield 0
f.remove(delayRebase)
commit2 = self.newPausedThread(t2.commit)
no_read = []
with cluster.client.extraCellSortKey(
lambda cell: cell.getUUID() in no_read), \
self.thread_switcher((commit2,),
(1, 1, 0, 0, t1_b, t1_resolve, 0, 0, 0, 0, 1, t2_vote, t1_end),
('tpc_begin', 'tpc_begin', 2, 1, 2, 1, 1,
'RebaseTransaction', 'RebaseTransaction',
'AnswerRebaseTransaction', 'AnswerRebaseTransaction',
'StoreTransaction')) as end:
t1.commit()
commit2.join()
t1.begin()
r = c1.root()
self.assertEqual(r['a'].value, 9)
self.assertEqual(r['b'].value, 6)
t1 = end.pop(0)
self.assertEqual(t1.pop(), 'StoreTransaction')
self.assertEqual(sorted(t1), [1, 2])
self.assertFalse(end)
self.assertPartitionTable(cluster, 'UU|UU')
@with_cluster(start_cluster=0, storage_count=2, replicas=1)
def testLocklessWriteDuringConflictResolution(self, cluster):
"""
This test reproduces a scenario during which the storage node didn't
check for conflicts when notifying the master that a partition is
replicated. The consequence was that it kept having write locks in a
weird state, relying on replicas to do the checking. While this may
work in itself, the code was hazardous and the client can't easily
discard such "positive" answers, or process them early enough.
The scenario focuses on transaction t1 (storing object A)
and node s1 (initially replicating the only partition):
1. t1 stores: conflict on s0, lockless write on s1
2. t2 stores: locked on s0, lockless write on s1
3. t1 resolves: deadlock on s0, packet to s1 delayed
4. t2 commits: on s1, a single lockless write remains and the storage
node notifies the master that it is UP_TO_DATE
5. s1 receives the second store from t1: answer delayed
6. t2 begins a new transaction
7. t1 resolves the deadlock: conflict on s0, s1 asks to rebase
8. t2 stores and vote
9. s0 down
10. while t2 finishes, t1 starts solving the conflict and due to the
way packets are processed, it proceeds as follows:
a. answer in step 5 is received but not processed
b. data asked, but not for the last serial
(so there will be yet another conflict to solve)
c. new data is stored: client waiting for s1
d. answer in step 5 is processed
What happened before:
4. t1 still has the lock
5. locked ("Transaction storing more than once")
10d. store considered successful, and the data won't be there anymore
for the actual conflict
-> assertion failure
Now that the storage nodes discards lockless writes that actually
conflict:
4. t1 does not have the lock anymore
5. conflict
10d. ignored (conflict already resolved)
-> transaction aborted normally
"""
s0, s1 = cluster.storage_list
cluster.start(storage_list=(s0,))
t1, c1 = cluster.getTransaction()
x1 = c1.root()[''] = PCounterWithResolution()
t1.commit()
x1.value += 1
with cluster.newClient(1) as db, ConnectionFilter() as f:
delayReplication = f.delayAnswerFetchObjects()
delayed = []
delayStore = f.delayAskStoreObject(lambda conn:
conn.uuid in delayed and
self.getConnectionApp(conn) is cluster.client)
delayStored = f.delayAnswerStoreObject(lambda conn:
conn.uuid == cluster.client.uuid and
self.getConnectionApp(conn).uuid in delayed)
def load(orig, oid, at_tid, before_tid):
if delayed:
p.revert()
f.remove(delayStored)
s0.stop()
cluster.join((s0,))
self.tic()
return orig(oid, at_tid, before_tid)
s1.start()
self.tic()
cluster.neoctl.enableStorageList([s1.uuid])
cluster.neoctl.tweakPartitionTable()
self.tic()
t2, c2 = cluster.getTransaction(db)
x2 = c2.root()['']
x2.value += 2
t2.commit()
x2.value += 4
def tic1(*args, **kw):
yield 1
self.tic()
def t1_resolve(*args, **kw):
delayed.append(s1.uuid)
f.remove(delayReplication)
self.tic()
yield 1
self.tic()
def t2_begin(*args, **kw):
f.remove(delayStore)
yield 0
@self.newPausedThread
def commit2():
t2.commit()
x2.value += 8
t2.commit()
with Patch(cluster.client, _loadFromStorage=load) as p, \
self.thread_switcher((commit2,),
(1, 0, tic1, 0, t1_resolve, 1, t2_begin, 0, 1, 1, 0),
('tpc_begin', 'tpc_begin', 1, 1, 1, 'StoreTransaction',
'tpc_begin', 'RebaseTransaction', 'RebaseTransaction', 1,
'StoreTransaction')) as end:
self.assertRaisesRegexp(NEOStorageError,
'^partition 0 not fully write-locked$',
t1.commit)
commit2.join()
t1.begin()
self.assertEqual(x1.value, 14)
self.assertPartitionTable(cluster, 'OU')
@with_cluster(partitions=2, storage_count=2)
def testUnstore(self, cluster):
"""
Show that when resolving a conflict after a lockless write, the storage
can't easily discard the data of the previous store, as it would make
internal data inconsistent. This is currently protected by a assertion
when trying to notifying the master that the replication is finished.
"""
t1, c1 = cluster.getTransaction()
r = c1.root()
for x in 'ab':
r[x] = PCounterWithResolution()
t1.commit()
cluster.stop(replicas=1)
cluster.start()
s0, s1 = cluster.sortStorageList()
t1, c1 = cluster.getTransaction()
r = c1.root()
r._p_changed = 1
r['b'].value += 1
with ConnectionFilter() as f:
delayReplication = f.delayAnswerFetchObjects()
cluster.neoctl.tweakPartitionTable()
self.tic()
t2, c2 = cluster.getTransaction()
x = c2.root()['b']
x.value += 2
t2.commit()
delayStore = f.delayAnswerStoreObject()
delayFinish = f.delayAskFinishTransaction()
x.value += 3
commit2 = self.newPausedThread(t2.commit)
def t2_b(*args, **kw):
yield 1
self.tic()
f.remove(delayReplication)
f.remove(delayStore)
self.tic()
def t1_resolve(*args, **kw):
yield 0
self.tic()
f.remove(delayFinish)
with self.thread_switcher((commit2,),
(1, 0, 0, 1, t2_b, 0, t1_resolve),
('tpc_begin', 'tpc_begin', 0, 2, 2, 'StoreTransaction')) as end:
t1.commit()
commit2.join()
t1.begin()
self.assertEqual(c1.root()['b'].value, 6)
self.assertPartitionTable(cluster, 'UU|UU')
self.assertEqual(end, {0: [2, 2, 'StoreTransaction']})
self.assertFalse(s1.dm.getOrphanList())
@with_cluster(storage_count=2, partitions=2)
def testDeadlockAvoidanceBeforeInvolvingAnotherNode(self, cluster):
t1, c1 = cluster.getTransaction()
......@@ -2340,8 +2784,8 @@ class Test(NEOThreadedTest):
self.assertEqual(1, len(getStorageList()))
with Patch(EventHandler, protocolError=lambda *_: sys.exit()):
self.tic()
expectedFailure(self.assertEqual)(neoctl.getClusterState(),
ClusterStates.RUNNING)
with self.expectedFailure(): \
self.assertEqual(neoctl.getClusterState(), ClusterStates.RUNNING)
self.assertEqual({1: NodeStates.RUNNING, 2: NodeStates.RUNNING},
{x[2]: x[3] for x in neoctl.getNodeList(NodeTypes.STORAGE)})
......
#
# Copyright (C) 2018 Nexedi SA
# Copyright (C) 2018-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2014-2017 Nexedi SA
# Copyright (C) 2014-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2012-2017 Nexedi SA
# Copyright (C) 2012-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......@@ -32,7 +32,7 @@ from neo.lib.connection import ClientConnection
from neo.lib.protocol import CellStates, ClusterStates, Packets, \
ZERO_OID, ZERO_TID, MAX_TID, uuid_str
from neo.lib.util import add64, p64, u64
from .. import expectedFailure, Patch, TransactionalResource
from .. import Patch, TransactionalResource
from . import ConnectionFilter, NEOCluster, NEOThreadedTest, \
predictable_random, with_cluster
from .test import PCounter, PCounterWithResolution # XXX
......@@ -733,7 +733,8 @@ class ReplicationTests(NEOThreadedTest):
s0.start()
self.tic()
self.assertEqual(2, s0.sqlCount('obj'))
expectedFailure(self.assertEqual)(2, count)
with self.expectedFailure(): \
self.assertEqual(2, count)
@with_cluster(replicas=1)
def testResumingReplication(self, cluster):
......@@ -1097,7 +1098,7 @@ class ReplicationTests(NEOThreadedTest):
# try to commit something to backup storage and make sure it is
# really read-only
Zb._cache._max_size = 0 # make store() do work in sync way
Zb._cache.max_size = 0 # make store() do work in sync way
txn = transaction.Transaction()
self.assertRaises(ReadOnlyError, Zb.tpc_begin, txn)
self.assertRaises(ReadOnlyError, Zb.new_oid)
......
#
# Copyright (C) 2015-2017 Nexedi SA
# Copyright (C) 2015-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
......@@ -48,7 +48,9 @@ CREATE TABLE trans (
description BLOB NOT NULL,
ext BLOB NOT NULL,
ttid INTEGER NOT NULL,
PRIMARY KEY (partition, tid));
PRIMARY KEY (partition, tid)
) WITHOUT ROWID
;
INSERT INTO "trans" VALUES(1,231616946283203125,0,X'0000000000000000',X'',X'',X'',231616946283203125);
CREATE TABLE ttrans (
partition INTEGER NOT NULL,
......
......@@ -48,7 +48,9 @@ CREATE TABLE trans (
description BLOB NOT NULL,
ext BLOB NOT NULL,
ttid INTEGER NOT NULL,
PRIMARY KEY (partition, tid));
PRIMARY KEY (partition, tid)
) WITHOUT ROWID
;
CREATE TABLE ttrans (
partition INTEGER NOT NULL,
tid INTEGER NOT NULL,
......
......@@ -46,7 +46,9 @@ CREATE TABLE trans (
description BLOB NOT NULL,
ext BLOB NOT NULL,
ttid INTEGER NOT NULL,
PRIMARY KEY (partition, tid));
PRIMARY KEY (partition, tid)
) WITHOUT ROWID
;
CREATE TABLE ttrans (
partition INTEGER NOT NULL,
tid INTEGER NOT NULL,
......
#
# Copyright (C) 2009-2017 Nexedi SA
# Copyright (C) 2009-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2009-2017 Nexedi SA
# Copyright (C) 2009-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2009-2017 Nexedi SA
# Copyright (C) 2009-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2009-2017 Nexedi SA
# Copyright (C) 2009-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2009-2017 Nexedi SA
# Copyright (C) 2009-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2009-2017 Nexedi SA
# Copyright (C) 2009-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2009-2017 Nexedi SA
# Copyright (C) 2009-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2009-2017 Nexedi SA
# Copyright (C) 2009-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2009-2017 Nexedi SA
# Copyright (C) 2009-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2009-2017 Nexedi SA
# Copyright (C) 2009-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2009-2017 Nexedi SA
# Copyright (C) 2009-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2009-2017 Nexedi SA
# Copyright (C) 2009-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2009-2017 Nexedi SA
# Copyright (C) 2009-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2009-2017 Nexedi SA
# Copyright (C) 2009-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#
# Copyright (C) 2009-2017 Nexedi SA
# Copyright (C) 2009-2019 Nexedi SA
#
# This program is free software; you can redistribute it and/or
# modify it under the terms of the GNU General Public License
......
#! /usr/bin/env python
#
# neoadmin - run an administrator node of NEO
# neoctl - command-line interface to an administrator node of NEO
#
# Copyright (C) 2009 Nexedi SA
#
......
......@@ -14,20 +14,35 @@ Topic :: Database
Topic :: Software Development :: Libraries :: Python Modules
"""
mock = 'neo/tests/mock.py'
if not os.path.exists(mock):
import cStringIO, hashlib, subprocess, urllib, zipfile
x = 'pythonmock-0.1.0.zip'
def get3rdParty(name, tag, url, h, extract=lambda content, name: content):
path = 'neo/tests/' + name
if os.path.exists(path):
return
import hashlib, subprocess, urllib
try:
x = subprocess.check_output(('git', 'cat-file', 'blob', x))
x = subprocess.check_output(('git', 'cat-file', 'blob', tag))
except (OSError, subprocess.CalledProcessError):
x = urllib.urlopen(
'http://downloads.sf.net/sourceforge/python-mock/' + x).read()
mock_py = zipfile.ZipFile(cStringIO.StringIO(x)).read('mock.py')
if (hashlib.sha256(mock_py).hexdigest() !=
'c6ed26e4312ed82160016637a9b6f8baa71cf31a67c555d44045a1ef1d60d1bc'):
raise EnvironmentError("SHA checksum mismatch downloading 'mock.py'")
open(mock, 'w').write(mock_py)
x = urllib.urlopen(url).read()
x = extract(x, name)
if hashlib.sha256(x).hexdigest() != h:
raise EnvironmentError("SHA checksum mismatch downloading '%s'" % name)
with open(path, 'wb') as f:
f.write(x)
def unzip(content, name):
import io, zipfile
return zipfile.ZipFile(io.BytesIO(content)).read(name)
x = 'pythonmock-0.1.0.zip'
get3rdParty('mock.py', x,
'http://downloads.sf.net/sourceforge/python-mock/' + x,
'c6ed26e4312ed82160016637a9b6f8baa71cf31a67c555d44045a1ef1d60d1bc',
unzip)
x = 'ConflictFree.py'
get3rdParty(x, '3rdparty/' + x, 'https://lab.nexedi.com/nexedi/erp5'
'/raw/14b0fcdcc31c5791646f9590678ca028f5d221f5/product/ERP5Type/' + x,
'abb7970856540fd02150edd1fa9a3a3e8d0074ec526ab189684ef7ea9b41825f')
zodb_require = ['ZODB3>=3.10dev']
......@@ -42,6 +57,9 @@ extras_require = {
}
extras_require['tests'] = ['coverage', 'zope.testing', 'psutil>=2',
'neoppod[%s]' % ', '.join(extras_require)]
extras_require['stress'] = ['NetfilterQueue', 'gevent', 'neoppod[tests]',
'cython-zstd', # recommended (log rotation)
]
try:
from docutils.core import publish_string
......@@ -60,7 +78,7 @@ else:
setup(
name = 'neoppod',
version = '1.10',
version = '1.11',
description = __doc__.strip(),
author = 'Nexedi SA',
author_email = 'neo-dev@erp5.org',
......
......@@ -16,25 +16,28 @@ class MatrixImportBenchmark(BenchmarkRunner):
_size = None
def add_options(self, parser):
parser.add_option('-d', '--datafs')
parser.add_option('-z', '--zeo', action="store_true")
parser.add_option('', '--min-storages', type='int', default=1)
parser.add_option('', '--max-storages', type='int', default=2)
parser.add_option('', '--min-replicas', type='int', default=0)
parser.add_option('', '--max-replicas', type='int', default=1)
parser.add_option('', '--threaded', action="store_true")
_ = parser.add_argument
_('-d', '--datafs')
_('-z', '--zeo', action="store_true")
_('--min-storages', type=int, default=1)
_('--max-storages', type=int, default=2)
_('--min-replicas', type=int, default=0)
_('--max-replicas', type=int, default=1)
_('--repeat', type=int, default=1)
_('--threaded', action="store_true")
def load_options(self, options, args):
if options.datafs and not os.path.exists(options.datafs):
def load_options(self, args):
if args.datafs and not os.path.exists(args.datafs):
sys.exit('Missing or wrong data.fs argument')
return dict(
datafs = options.datafs,
min_s = options.min_storages,
max_s = options.max_storages,
min_r = options.min_replicas,
max_r = options.max_replicas,
threaded = options.threaded,
zeo = options.zeo,
datafs = args.datafs,
min_s = args.min_storages,
max_s = args.max_storages,
min_r = args.min_replicas,
max_r = args.max_replicas,
repeat = args.repeat,
threaded = args.threaded,
zeo = args.zeo,
)
def start(self):
......
......@@ -16,21 +16,22 @@ class ImportBenchmark(BenchmarkRunner):
""" Test import of a datafs """
def add_options(self, parser):
parser.add_option('-d', '--datafs')
parser.add_option('-m', '--masters')
parser.add_option('-s', '--storages')
parser.add_option('-p', '--partitions')
parser.add_option('-r', '--replicas')
def load_options(self, options, args):
if options.datafs and not os.path.exists(options.datafs):
_ = parser.add_argument
_('-d', '--datafs')
_('-m', '--masters', type=int, default=1)
_('-s', '--storages', type=int, default=1)
_('-p', '--partitions', type=int, default=10)
_('-r', '--replicas', type=int, default=0)
def load_options(self, args):
if args.datafs and not os.path.exists(args.datafs):
sys.exit('Missing or wrong data.fs argument')
return dict(
datafs = options.datafs,
masters = int(options.masters or 1),
storages = int(options.storages or 1),
partitions = int(options.partitions or 10),
replicas = int(options.replicas or 0),
datafs = args.datafs,
masters = args.masters,
storages = args.storages,
partitions = args.partitions,
replicas = args.replicas,
)
def start(self):
......
#!/usr/bin/env python
# -*- coding: utf-8 -*-
from __future__ import division, print_function
import argparse, curses, errno, os, random, select
import signal, socket, subprocess, sys, threading, time
from contextlib import contextmanager
from datetime import datetime
from functools import partial
from multiprocessing import Lock, RawArray
from struct import Struct
from netfilterqueue import NetfilterQueue
import gevent.socket # preload for subprocesses
from neo.client.Storage import Storage
from neo.lib import logging, util
from neo.lib.connector import SocketConnector
from neo.lib.debug import PdbSocket
from neo.lib.node import Node
from neo.lib.protocol import NodeTypes
from neo.lib.util import timeStringFromTID, p64, u64
from neo.storage.app import DATABASE_MANAGER_DICT, \
Application as StorageApplication
from neo.tests import getTempDirectory
from neo.tests.ConflictFree import ConflictFreeLog
from neo.tests.functional import AlreadyStopped, NEOCluster, Process
from neo.tests.stress import StressApplication
from transaction import begin as transaction_begin
from ZODB import DB, POSException
INET = {
socket.AF_INET: ('ip', socket.IPPROTO_IP, socket.IP_TOS),
socket.AF_INET6: ('ip6', socket.IPPROTO_IPV6, socket.IPV6_TCLASS),
}
NFT_TEMPLATE = """\
table %s %s {
chain mangle {
type filter hook input priority -150
policy accept
%s dscp 1 tcp flags & (fin|syn|rst|ack) != syn jump nfqueue
}
chain nfqueue {
%s
}
chain filter {
type filter hook input priority 0
policy accept
meta l4proto tcp %s dscp 1 mark 1 counter reject with tcp reset
}
}"""
SocketConnector.KEEPALIVE = 5, 1, 1
def child_coverage(self):
# XXX: The dance to collect coverage results just before killing
# subprocesses does not work for processes that may run code that
# is not interruptible with Python code (e.g. Lock.acquire).
# For nodes with a single epoll loop, this is usually fine.
# On the other side, coverage support is broken for clients,
# like here: we just do some cleanup for the assertion in __del__
r = self._coverage_fd
if r is not None:
os.close(r)
del self._coverage_fd
Process.child_coverage = child_coverage
def setDSCP(connection, dscp):
connector = connection.getConnector()
_, sol, opt = INET[connector.af_type]
connector.socket.setsockopt(sol, opt, dscp << 2)
def dscpPatch(dscp):
Node_setConnection = Node.setConnection
Node.dscp = dscp
def setConnection(self, connection, force=None):
if self.dscp and self.getType() == NodeTypes.STORAGE:
setDSCP(connection, 1)
return Node_setConnection(self, connection, force)
Node.setConnection = setConnection
class Client(Process):
_fmt = '!I200s'
prev_count = 0
def __init__(self, command, thread_count, **kw):
super(Client, self).__init__(command)
self.config = kw
self.count = RawArray('I', thread_count)
self.thread_count = thread_count
def run(self):
from neo.lib.threaded_app import registerLiveDebugger
registerLiveDebugger() # for on_log
dscpPatch(0)
self._dscp_lock = threading.Lock()
storage = Storage(**self.config)
db = DB(storage=storage)
try:
if self.thread_count == 1:
self.worker(db)
else:
r, w = os.pipe()
try:
for i in xrange(self.thread_count):
t = threading.Thread(target=self.worker,
args=(db, i, w), name='worker-%s' % i)
t.daemon = 1
t.start()
while 1:
try:
os.read(r, 1)
break
except OSError, e:
if e.errno != errno.EINTR:
raise
finally:
os.close(r)
finally:
db.close()
def worker(self, db, i=0, stop=None):
try:
nm = db.storage.app.nm
conn = db.open()
r = conn.root()
count = self.count
name = self.command
if self.thread_count > 1:
name += ':%s' % i
j = 0
k = None
logs = r.values()
pack = Struct(self._fmt).pack
while 1:
txn = transaction_begin()
try:
data = pack(j, name)
for log in random.sample(logs, 2):
log.append(data)
txn.note(name)
self.setDSCP(nm, 1)
try:
txn.commit()
finally:
self.setDSCP(nm, -1)
except (
POSException.StorageError, # XXX: 'already connected' error
POSException.ConflictError, # XXX: same but during conflict resolution
), e:
if 'unexpected packet:' in str(e):
raise
if j != k:
logging.exception('j = %s', j)
k = j
txn.abort()
continue
j += 1
count[i] = j
finally:
if stop is not None:
try:
os.write(stop, '\0')
except OSError:
pass
def setDSCP(self, nm, dscp):
with self._dscp_lock:
prev = Node.dscp
dscp += prev
Node.dscp = dscp
if dscp and prev:
return
for node in nm.getStorageList():
try:
setDSCP(node.getConnection(), dscp)
except (AttributeError, AssertionError,
# XXX: EBADF due to race condition
socket.error):
pass
@classmethod
def check(cls, r):
nodes = {}
hosts = []
buckets = [0, 0]
item_list = []
unpack = Struct(cls._fmt).unpack
def decode(item):
i, host = unpack(item)
return i, host.rstrip('\0')
for log in r.values():
bucket = log._next
if bucket is None:
bucket = log
buckets[:] = bucket._p_estimated_size, 1
while 1:
for item in bucket._log:
i, host = decode(item)
try:
node = nodes[host]
except KeyError:
node = nodes[host] = len(nodes)
hosts.append(host)
item_list.append((i, node))
if bucket is log:
break
buckets[0] += bucket._p_estimated_size
buckets[1] += 1
bucket = bucket._next
item_list.sort()
nodes = [0] * len(nodes)
for i, node in item_list:
j = nodes[node] // 2
if i != j:
#import code; code.interact(banner="", local=locals())
sys.exit('node: %s, expected: %s, stored: %s'
% (hosts[node], j, i))
nodes[node] += 1
for node, host in sorted(enumerate(hosts), key=lambda x: x[1]):
print('%s\t%s' % (nodes[node], host))
print('average bucket size: %f' % (buckets[0] / buckets[1]))
print('target bucket size:', log._bucket_size)
print('number of full buckets:', buckets[1])
@property
def logfile(self):
return self.config['logfile']
class NFQueue(Process):
def __init__(self, queue):
super(NFQueue, self).__init__('nfqueue_%i' % queue)
self.lock = l = Lock(); l.acquire()
self.queue = queue
def run(self):
acquire = self.lock.acquire
delay = self.delay
nfqueue = NetfilterQueue()
if delay:
from gevent import sleep, socket, spawn
from random import random
def callback(packet):
if acquire(0): packet.set_mark(1)
else: sleep(random() * delay)
packet.accept()
callback = partial(spawn, callback)
else:
def callback(packet):
if acquire(0): packet.set_mark(1)
packet.accept()
nfqueue.bind(self.queue, callback)
try:
if delay:
s = socket.fromfd(nfqueue.get_fd(),
socket.AF_UNIX, socket.SOCK_STREAM)
try:
nfqueue.run_socket(s)
finally:
s.close()
else:
while 1:
nfqueue.run() # returns on signal (e.g. SIGWINCH)
finally:
nfqueue.unbind()
class Alarm(threading.Thread):
__interrupt = BaseException()
def __init__(self, signal, timeout):
super(Alarm, self).__init__()
self.__signal = signal
self.__timeout = timeout
def __enter__(self):
self.__r, self.__w = os.pipe()
self.__prev = signal.signal(self.__signal, self.__raise)
self.start()
def __exit__(self, t, v, tb):
try:
try:
os.close(self.__w)
self.join()
finally:
os.close(self.__r)
signal.signal(self.__signal, self.__prev)
return v is self.__interrupt
except BaseException as e:
if e is not self.__interrupt:
raise
def __raise(self, sig, frame):
raise self.__interrupt
def run(self):
if not select.select((self.__r,), (), (), self.__timeout)[0]:
os.kill(os.getpid(), self.__signal)
class NEOCluster(NEOCluster):
def _newProcess(self, node_type, logfile=None, port=None, **kw):
super(NEOCluster, self)._newProcess(node_type, logfile,
port or self.port_allocator.allocate(
self.address_type, self.local_ip),
**kw)
class Application(StressApplication):
_blocking = None
def __init__(self, client_count, thread_count, restart_ratio, logrotate,
*args, **kw):
self.client_count = client_count
self.thread_count = thread_count
self.logrotate = logrotate
self.restart_ratio = restart_ratio
self.cluster = cluster = NEOCluster(*args, **kw)
# Make the firewall also affect connections between storage nodes.
StorageApplication__init__ = StorageApplication.__init__
def __init__(self, config):
dscpPatch(1)
StorageApplication__init__(self, config)
StorageApplication.__init__ = __init__
super(Application, self).__init__(cluster.SSL,
util.parseMasterList(cluster.master_nodes))
self._nft_family = INET[cluster.address_type][0]
self._nft_table = 'stress_%s' % os.getpid()
self._blocked = []
n = kw['replicas']
self._fault_count = len(kw['db_list']) * n // (1 + n)
@property
def name(self):
return self.cluster.cluster_name
def run(self):
super(Application, self).run()
try:
with self.db() as r:
Client.check(r)
finally:
self.cluster.stop()
@contextmanager
def db(self):
cluster = self.cluster
cluster.start()
db, conn = cluster.getZODBConnection()
try:
yield conn.root()
finally:
db.close()
def startCluster(self):
with self.db() as r:
txn = transaction_begin()
for i in xrange(2 * self.client_count * self.thread_count):
r[i] = ConflictFreeLog()
txn.commit()
cluster = self.cluster
process_list = cluster.process_dict[NFQueue] = []
nft_family = self._nft_family
queue = []
for _, (ip, port), nid, _, _ in sorted(cluster.getStorageList(),
key=lambda x: x[2]):
queue.append(
"%s daddr %s tcp dport %s counter queue num %s bypass"
% (nft_family, ip, port, nid))
p = NFQueue(nid)
process_list.append(p)
p.start()
ruleset = NFT_TEMPLATE % (nft_family, self._nft_table,
nft_family, '\n '.join(queue), nft_family)
p = subprocess.Popen(('nft', '-f', '-'), stdin=subprocess.PIPE,
stdout=subprocess.PIPE, stderr=subprocess.STDOUT)
err = p.communicate(ruleset)[0].rstrip()
if p.poll():
sys.exit("Failed to apply the following ruleset:\n%s\n%s"
% (ruleset, err))
process_list = cluster.process_dict[Client] = []
config = cluster.getClientConfig()
self.started = time.time()
for i in xrange(self.client_count):
name = 'client_%i' % i
p = Client(name, self.thread_count,
logfile=os.path.join(cluster.temp_dir, name + '.log'),
**config)
process_list.append(p)
p.start()
if self.logrotate:
t = threading.Thread(target=self._logrotate_thread)
t.daemon = 1
t.start()
def stopCluster(self, wait=None):
self.restart_lock.acquire()
self._cleanFirewall()
process_dict = self.cluster.process_dict
if wait:
# Give time to flush logs before SIGKILL.
wait += 5 - time.time()
if wait > 0:
with Alarm(signal.SIGUSR1, wait):
for x in Client, NodeTypes.STORAGE:
for x in process_dict[x]:
x.wait()
self.cluster.stop()
try:
del process_dict[NFQueue], process_dict[Client]
except KeyError:
pass
def _logrotate_thread(self):
try:
import zstd
except ImportError:
import gzip, shutil
zstd = None
compress = []
rotated = {}
t = time.time()
while 1:
t += self.logrotate
x = t - time.time()
if x > 0:
time.sleep(x)
x = datetime.utcnow().strftime('-%Y%m%d%H%M%S.log')
for p, process_list in self.cluster.process_dict.iteritems():
if p is not NFQueue:
for p in process_list:
log = p.logfile
if os.path.exists(log):
y = rotated.get(log)
if y:
compress.append(y)
y = log[:-4] + x
os.rename(log, y)
rotated[log] = y
try:
p.kill(signal.SIGRTMIN+1)
except AlreadyStopped:
pass
for log in compress:
if zstd:
with open(log, 'rb') as src:
x = zstd.compress(src.read())
y = log + '.zst'
with open(y, 'wb') as dst:
dst.write(x)
else:
y = log + '.gz'
with open(log, 'rb') as src, gzip.open(y, 'wb') as dst:
shutil.copyfileobj(src, dst, 1<<20)
x = os.stat(log)
os.utime(y, (x.st_atime, x.st_mtime))
os.remove(log)
del compress[:]
def tcpReset(self, nid):
p = self.cluster.process_dict[NFQueue][nid-1]
assert p.queue == nid, (p.queue, nid)
try:
p.lock.release()
except ValueError:
pass
def restartStorages(self, nids):
processes = [p for p in self.cluster.getStorageProcessList()
if p.uuid in nids]
for p in processes: p.kill(signal.SIGKILL)
time.sleep(1)
for p in processes: p.wait()
for p in processes: p.start()
def _cleanFirewall(self):
with open(os.devnull, "wb") as f:
subprocess.call(('nft', 'delete', 'table',
self._nft_family, self._nft_table), stderr=f)
_ids_height = 4
def refresh_ids(self, y):
attr = curses.A_NORMAL, curses.A_BOLD
stdscr = self.stdscr
ltid = self.ltid
stdscr.addstr(y, 0,
'last oid: 0x%x\nlast tid: 0x%x (%s)\nclients: '
% (u64(self.loid), u64(ltid), timeStringFromTID(ltid)))
before = after = 0
for i, p in enumerate(self.cluster.process_dict[Client]):
if i:
stdscr.addstr(', ')
count = sum(p.count)
before += p.prev_count
after += count
stdscr.addstr(str(count), attr[p.prev_count==count])
p.prev_count = count
elapsed = time.time() - self.started
s, ms = divmod(int(elapsed * 1000), 1000)
m, s = divmod(s, 60)
stdscr.addstr(' (+%s)\n\t%sm%02u.%03us (%f/s)\n' % (
after - before, m, s, ms, after / elapsed))
def console(port, app):
from pdb import Pdb
cluster = app.cluster
def console(socket):
Pdb(stdin=socket, stdout=socket).set_trace()
app # this is Application instance
s = socket.socket(cluster.address_type, socket.SOCK_STREAM)
# XXX: The following commented line would only work with Python 3, which
# fixes refcounting of sockets (e.g. when there's a call to .accept()).
#Process.on_fork.append(s.close)
s.bind((cluster.local_ip, port))
s.listen(0)
while 1:
t = threading.Thread(target=console, args=(PdbSocket(s.accept()[0]),))
t.daemon = 1
t.start()
class ArgumentDefaultsHelpFormatter(argparse.HelpFormatter):
def _format_action(self, action):
if not (action.help or action.default in (None, argparse.SUPPRESS)):
action.help = '(default: %(default)s)'
return super(ArgumentDefaultsHelpFormatter, self)._format_action(action)
def main():
adapters = sorted(DATABASE_MANAGER_DICT)
adapters.remove('Importer')
default_adapter = 'SQLite'
assert default_adapter in adapters
kw = dict(formatter_class=ArgumentDefaultsHelpFormatter)
parser = argparse.ArgumentParser(**kw)
_ = parser.add_argument
_('-6', '--ipv6', dest='address_type', action='store_const',
default=socket.AF_INET, const=socket.AF_INET6, help='(default: IPv4)')
_('-a', '--adapter', choices=adapters, default=default_adapter)
_('-d', '--datadir', help="(default: same as unit tests)")
_('-l', '--logdir', help="(default: same as --datadir)")
_('-m', '--masters', type=int, default=1)
_('-s', '--storages', type=int, default=8)
_('-p', '--partitions', type=int, default=24)
_('-r', '--replicas', type=int, default=1)
parsers = parser.add_subparsers(dest='command')
def ratio(value):
value = float(value)
if 0 <= value <= 1:
return value
raise argparse.ArgumentTypeError("ratio ∉ [0,1]")
_ = parsers.add_parser('run',
help='Start a new DB and fills it in a way that triggers many conflict'
' resolutions and deadlock avoidances. Stressing the cluster will'
' cause external faults every second, to check that NEO can'
' recover. The ingested data is checked at exit.',
**kw).add_argument
_('-c', '--clients', type=int, default=10,
help='number of client processes')
_('-t', '--threads', type=int, default=1,
help='number of thread workers per client process')
_('-r', '--restart-ratio', type=ratio, default=.5, metavar='RATIO',
help='probability to kill/restart a storage node, rather than just'
' RSTing a TCP connection with this node')
_('-C', '--console', type=int, default=0,
help='console port (localhost) (default: any)')
_('-D', '--delay', type=float, default=.01,
help='randomly delay packets to storage nodes'
' by a duration between 0 and DELAY seconds')
_('-L', '--logrotate', type=float, default=1, metavar='HOUR')
_ = parsers.add_parser('check',
help='Check ingested data.',
**kw).add_argument
_('tid', nargs='?')
_ = parsers.add_parser('bisect',
help='Search for the first TID that contains corrupted data.',
**kw).add_argument
args = parser.parse_args()
db_list = ['stress_neo%s' % x for x in xrange(args.storages)]
if args.datadir:
if args.adapter != 'SQLite':
parser.error('--datadir is only for SQLite adapter')
db_list = [os.path.join(args.datadir, x + '.sqlite') for x in db_list]
kw = dict(db_list=db_list, name='stress',
partitions=args.partitions, replicas=args.replicas,
adapter=args.adapter, address_type=args.address_type,
temp_dir=args.logdir or args.datadir or getTempDirectory())
if args.command == 'run':
NFQueue.delay = args.delay
app = Application(args.clients, args.threads, args.restart_ratio,
int(round(args.logrotate * 3600, 0)), **kw)
t = threading.Thread(target=console, args=(args.console, app))
t.daemon = 1
t.start()
app.run()
return
cluster = NEOCluster(clear_databases=False, **kw)
try:
cluster.start()
storage = cluster.getZODBStorage()
db = DB(storage=storage)
try:
if args.command == 'check':
tid = args.tid
conn = db.open(at=tid and p64(int(tid, 0)))
Client.check(conn.root())
else:
assert args.command == 'bisect'
conn = db.open()
try:
r = conn.root()
r._p_activate()
ok = r._p_serial
finally:
conn.close()
bad = storage.lastTransaction()
while 1:
print('ok: 0x%x, bad: 0x%x' % (u64(ok), u64(bad)))
tid = p64((u64(ok)+u64(bad)) // 2)
if ok == tid:
break
conn = db.open(at=tid)
try:
Client.check(conn.root())
except SystemExit, e:
print(e)
bad = tid
else:
ok = tid
finally:
conn.close()
print('bad: 0x%x (%s)' % (u64(bad), timeStringFromTID(bad)))
finally:
db.close()
finally:
cluster.stop()
if __name__ == '__main__':
sys.exit(main())
Markdown is supported
0%
or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment