Commit 4a2407dc authored by Kirill Smelkov's avatar Kirill Smelkov

Sync with NEO/py v1.9

parents 5bf40022 1b57a7ae
Change History Change History
============== ==============
1.9 (2018-03-13)
----------------
A lot of performance improvements have been done on storage nodes for this
release, and some of them required changes in the storage format. In short,
the migration is done automatically, but you may want to read UPGRADE notes
for more details.
Performance:
- Speed up replication by sending bigger network packets,
and by not getting object next_serial for nothing.
- Speed up reads by indexing 'obj' primarily by 'oid' (instead of 'tid').
- Optimize storage layout of raw data for replication.
Other storage changes:
- Disable data deduplication by default. --dedup option added.
- importer: do not crash if a backup cluster tries to replicate.
- importer: preserve 'packed' flag.
Master:
- Fix possible failure when reading data in a backup cluster with replicas.
- Fix generation of TID.
- Fix resumption of backup replication (internal or not).
Client:
- New 'cache-size' Storage option.
- Cache hit/miss statistics.
- Fix accounting of cache size.
- Preserve 'packed' flag on iteration.
- At startup, or after nodes are back, full load balancing could be prevented
until some data are written.
Other:
- neolog: --from option now also tries to parse with `dateutil`_.
- neolog: add support for xz-compressed logs, using external xzcat commands.
.. _dateutil: https://dateutil.readthedocs.io/
1.8.1 (2017-11-07) 1.8.1 (2017-11-07)
------------------ ------------------
......
NEO 1.9
=======
The `obj` table in MySQL/SQLite backends is automatically upgraded at startup.
Note however that rewritting this table can take a lot of time if you have a
lot of object records.
This migration is reversible but you'd have to dig into the migration code
(``DatabaseManager._migrate2()``) and do manually the opposite SQL queries.
The change in the `data` table is such that no migration is done (like for NEO
1.4, it's optional and it would cost too much). For optimal performance, you
may prefer to set up new nodes (and drop the old ones once the replication
is finished).
Because there's no change in the protocol, it's actually possible to mix nodes
with versions 1.8.1 and 1.9, and avoid any downtime because of the migration.
NEO 1.6 NEO 1.6
======= =======
......
...@@ -136,9 +136,6 @@ class Storage(BaseStorage.BaseStorage, ...@@ -136,9 +136,6 @@ class Storage(BaseStorage.BaseStorage,
def supportsUndo(self): def supportsUndo(self):
return True return True
def supportsTransactionalUndo(self):
return True
def loadEx(self, oid, version): def loadEx(self, oid, version):
try: try:
data, serial, _ = self.app.load(oid) data, serial, _ = self.app.load(oid)
......
...@@ -56,25 +56,28 @@ class ConnectionPool(object): ...@@ -56,25 +56,28 @@ class ConnectionPool(object):
logging.info('%r not ready', node) logging.info('%r not ready', node)
else: else:
logging.info('Connected %r', node) 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 return conn
self.node_failure_dict[node.getUUID()] = time.time() + MAX_FAILURE_AGE self.node_failure_dict[node.getUUID()] = time.time() + MAX_FAILURE_AGE
def getCellSortKey(self, cell, random=random.random): def getCellSortKey(self, cell, random=random.random):
# The use of 'random' suffles cells to randomise node to access. # Prefer a node that didn't fail recently.
uuid = cell.getUUID() failure = self.node_failure_dict.get(cell.getUUID())
# First, prefer a connected node.
if uuid in self.connection_dict:
return random()
# Then one that didn't fail recently.
failure = self.node_failure_dict.get(uuid)
if failure: if failure:
if time.time() < failure: if time.time() < failure:
# At last, order by date of connection failure. # Or order by date of connection failure.
return failure return failure
# Do not use 'del' statement: we didn't lock, so another # Do not use 'del' statement: we didn't lock, so another
# thread might have removed uuid from node_failure_dict. # thread might have removed uuid from node_failure_dict.
self.node_failure_dict.pop(uuid, None) self.node_failure_dict.pop(cell.getUUID(), None)
return 1 + random() # 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): def getConnForNode(self, node):
"""Return a locked connection object to a given node """Return a locked connection object to a given node
......
...@@ -206,12 +206,21 @@ class BackupApplication(object): ...@@ -206,12 +206,21 @@ class BackupApplication(object):
except IndexError: except IndexError:
last_max_tid = prev_tid last_max_tid = prev_tid
if offset in partition_set: if offset in partition_set:
self.tid_list[offset].append(tid) primary_list = []
node_list = [] node_list = []
for cell in pt.getCellList(offset, readable=True): cell_list = pt.getCellList(offset, readable=True)
for cell in cell_list:
node = cell.getNode() node = cell.getNode()
assert node.isConnected(), node assert node.isConnected(), node
if cell.backup_tid == prev_tid: if cell.backup_tid == prev_tid:
if prev_tid == tid:
# Connecting to upstream: any node is that is
# up-to-date wrt upstream is candidate for being
# primary.
assert self.ignore_invalidations
if app.isStorageReady(node.getUUID()):
primary_list.append(node)
continue
# Let's given 4 TID t0,t1,t2,t3: if a cell is only # Let's given 4 TID t0,t1,t2,t3: if a cell is only
# modified by t0 & t3 and has all data for t0, 4 values # modified by t0 & t3 and has all data for t0, 4 values
# are possible for its 'backup_tid' until it replicates # are possible for its 'backup_tid' until it replicates
...@@ -231,12 +240,19 @@ class BackupApplication(object): ...@@ -231,12 +240,19 @@ class BackupApplication(object):
cell.backup_tid, last_max_tid, prev_tid, tid) cell.backup_tid, last_max_tid, prev_tid, tid)
if app.isStorageReady(node.getUUID()): if app.isStorageReady(node.getUUID()):
node_list.append(node) node_list.append(node)
assert node_list
trigger_set.update(node_list)
# Make sure we have a primary storage for this partition. # Make sure we have a primary storage for this partition.
if offset not in self.primary_partition_dict: if offset not in self.primary_partition_dict:
self.primary_partition_dict[offset] = \ self.primary_partition_dict[offset] = \
random.choice(node_list) random.choice(primary_list or node_list)
if node_list:
self.tid_list[offset].append(tid)
if primary_list:
# Resume replication to secondary cells.
self._triggerSecondary(
self.primary_partition_dict[offset],
offset, tid, cell_list)
else:
trigger_set.update(node_list)
else: else:
# Partition not touched, so increase 'backup_tid' of all # Partition not touched, so increase 'backup_tid' of all
# "up-to-date" replicas, without having to replicate. # "up-to-date" replicas, without having to replicate.
...@@ -335,15 +351,18 @@ class BackupApplication(object): ...@@ -335,15 +351,18 @@ class BackupApplication(object):
if app.getClusterState() == ClusterStates.BACKINGUP: if app.getClusterState() == ClusterStates.BACKINGUP:
self.triggerBackup(node) self.triggerBackup(node)
if primary: if primary:
# Notify secondary storages that they can replicate from self._triggerSecondary(node, offset, tid, cell_list)
# primary ones, even if they are already replicating.
p = Packets.Replicate(tid, '', {offset: node.getAddress()})
for cell in cell_list:
if max(cell.backup_tid, cell.replicating) < tid:
cell.replicating = tid
logging.debug(
"ask %s to replicate partition %u up to %s from %s",
uuid_str(cell.getUUID()), offset,
dump(tid), uuid_str(node.getUUID()))
cell.getNode().send(p)
return result return result
def _triggerSecondary(self, node, offset, tid, cell_list):
# Notify secondary storages that they can replicate from
# primary ones, even if they are already replicating.
p = Packets.Replicate(tid, '', {offset: node.getAddress()})
for cell in cell_list:
if max(cell.backup_tid, cell.replicating) < tid:
cell.replicating = tid
logging.debug(
"ask %s to replicate partition %u up to %s from %s",
uuid_str(cell.getUUID()), offset,
dump(tid), uuid_str(node.getUUID()))
cell.getNode().send(p)
...@@ -35,7 +35,7 @@ class BackupHandler(EventHandler): ...@@ -35,7 +35,7 @@ class BackupHandler(EventHandler):
def answerLastTransaction(self, conn, tid): def answerLastTransaction(self, conn, tid):
app = self.app app = self.app
prev_tid = app.app.getLastTransaction() prev_tid = app.app.getLastTransaction()
if prev_tid < tid: if prev_tid <= tid:
# Since we don't know which partitions were modified during our # Since we don't know which partitions were modified during our
# absence, we must force replication on all storages. As long as # absence, we must force replication on all storages. As long as
# they haven't done this first check, our backup tid will remain # they haven't done this first check, our backup tid will remain
...@@ -43,8 +43,12 @@ class BackupHandler(EventHandler): ...@@ -43,8 +43,12 @@ class BackupHandler(EventHandler):
# >= app.app.getLastTransaction() # >= app.app.getLastTransaction()
# < tid # < tid
# but passing 'tid' is good enough. # but passing 'tid' is good enough.
# A special case is when prev_tid == tid: even in this case, we
# must restore the state of the backup app so that any interrupted
# replication (internal or not) is resumed, otherwise the global
# backup_tid could remain stuck to an old tid if upstream is idle.
app.invalidatePartitions(tid, tid, xrange(app.pt.getPartitions())) app.invalidatePartitions(tid, tid, xrange(app.pt.getPartitions()))
elif prev_tid != tid: else:
raise RuntimeError("upstream DB truncated") raise RuntimeError("upstream DB truncated")
app.ignore_invalidations = False app.ignore_invalidations = False
......
...@@ -147,5 +147,5 @@ class ClientReadOnlyServiceHandler(ClientServiceHandler): ...@@ -147,5 +147,5 @@ class ClientReadOnlyServiceHandler(ClientServiceHandler):
# like in MasterHandler but returns backup_tid instead of last_tid # like in MasterHandler but returns backup_tid instead of last_tid
def askLastTransaction(self, conn): def askLastTransaction(self, conn):
assert self.app.backup_tid is not None # we are in BACKUPING mode assert self.app.backup_tid is not None # we are in BACKUPING mode
backup_tid = self.app.pt.getBackupTid() backup_tid = self.app.pt.getBackupTid(min)
conn.answer(Packets.AnswerLastTransaction(backup_tid)) conn.answer(Packets.AnswerLastTransaction(backup_tid))
...@@ -234,29 +234,16 @@ class TransactionManager(EventQueue): ...@@ -234,29 +234,16 @@ class TransactionManager(EventQueue):
tid = tidFromTime(time()) tid = tidFromTime(time())
min_tid = self._last_tid min_tid = self._last_tid
if tid <= min_tid: if tid <= min_tid:
tid = addTID(min_tid, 1) tid = addTID(min_tid, 1)
# We know we won't have room to adjust by decreasing.
try_decrease = False
else:
try_decrease = True
if ttid is not None: if ttid is not None:
assert isinstance(ttid, basestring), repr(ttid) remainder = u64(ttid) % divisor
assert isinstance(divisor, (int, long)), repr(divisor) delta_remainder = remainder - u64(tid) % divisor
ref_remainder = u64(ttid) % divisor if delta_remainder:
remainder = u64(tid) % divisor tid = addTID(tid, delta_remainder)
if ref_remainder != remainder: if tid <= min_tid:
if try_decrease: tid = addTID(tid, divisor)
new_tid = addTID(tid, ref_remainder - divisor - remainder) assert u64(tid) % divisor == remainder, (dump(tid), remainder)
assert u64(new_tid) % divisor == ref_remainder, (dump(new_tid), assert min_tid < tid, (dump(min_tid), dump(tid))
ref_remainder)
if new_tid <= min_tid:
new_tid = addTID(new_tid, divisor)
else:
if ref_remainder > remainder:
ref_remainder += divisor
new_tid = addTID(tid, ref_remainder - remainder)
assert min_tid < new_tid, (dump(min_tid), dump(tid), dump(new_tid))
tid = new_tid
self._last_tid = tid self._last_tid = tid
return self._last_tid return self._last_tid
......
...@@ -406,7 +406,7 @@ class ImporterDatabaseManager(DatabaseManager): ...@@ -406,7 +406,7 @@ class ImporterDatabaseManager(DatabaseManager):
if compression: if compression:
data = compressed_data data = compressed_data
checksum = util.makeChecksum(data) checksum = util.makeChecksum(data)
data_id = self.holdData(util.makeChecksum(data), data, data_id = self.holdData(util.makeChecksum(data), oid, data,
compression) compression)
data_id_list.append(data_id) data_id_list.append(data_id)
object_list.append((oid, data_id, data_tid)) object_list.append((oid, data_id, data_tid))
......
...@@ -161,11 +161,14 @@ class DatabaseManager(object): ...@@ -161,11 +161,14 @@ class DatabaseManager(object):
"The database can not be upgraded because you have unfinished" "The database can not be upgraded because you have unfinished"
" transactions. Use an older version of NEO to verify them.") " transactions. Use an older version of NEO to verify them.")
def _getVersion(self): def migrate(self, *args, **kw):
version = int(self.getConfiguration("version") or 0) version = int(self.getConfiguration("version") or 0)
if self.VERSION < version: if self.VERSION < version:
raise DatabaseFailure("The database can not be downgraded.") raise DatabaseFailure("The database can not be downgraded.")
return version while version < self.VERSION:
version += 1
getattr(self, '_migrate%s' % version)(*args, **kw)
self.setConfiguration("version", version)
def doOperation(self, app): def doOperation(self, app):
pass pass
...@@ -485,7 +488,11 @@ class DatabaseManager(object): ...@@ -485,7 +488,11 @@ class DatabaseManager(object):
existing data is first thrown away. existing data is first thrown away.
""" """
@requires(_changePartitionTable) def _getDataLastId(self, partition):
"""
"""
@requires(_changePartitionTable, _getDataLastId)
def changePartitionTable(self, ptid, cell_list, reset=False): def changePartitionTable(self, ptid, cell_list, reset=False):
readable_set = self._readable_set readable_set = self._readable_set
if reset: if reset:
...@@ -500,6 +507,10 @@ class DatabaseManager(object): ...@@ -500,6 +507,10 @@ class DatabaseManager(object):
raise NonReadableCell raise NonReadableCell
self._getPartition = _getPartition self._getPartition = _getPartition
self._getReadablePartition = _getReadablePartition self._getReadablePartition = _getReadablePartition
d = self._data_last_ids = []
for p in xrange(np):
i = self._getDataLastId(p)
d.append(p << 48 if i is None else i + 1)
me = self.getUUID() me = self.getUUID()
for offset, nid, state in cell_list: for offset, nid, state in cell_list:
if nid == me: if nid == me:
...@@ -567,7 +578,7 @@ class DatabaseManager(object): ...@@ -567,7 +578,7 @@ class DatabaseManager(object):
""" """
@abstract @abstract
def storeData(self, checksum, data, compression): def storeData(self, checksum, oid, data, compression):
"""To be overridden by the backend to store object raw data """To be overridden by the backend to store object raw data
If same data was already stored, the storage only has to check there's If same data was already stored, the storage only has to check there's
......
This diff is collapsed.
This diff is collapsed.
...@@ -113,7 +113,7 @@ class StorageOperationHandler(EventHandler): ...@@ -113,7 +113,7 @@ class StorageOperationHandler(EventHandler):
checksum, data, data_serial): checksum, data, data_serial):
dm = self.app.dm dm = self.app.dm
if data or checksum != ZERO_HASH: if data or checksum != ZERO_HASH:
data_id = dm.storeData(checksum, data, compression) data_id = dm.storeData(checksum, oid, data, compression)
else: else:
data_id = None data_id = None
# Directly store the transaction. # Directly store the transaction.
......
...@@ -470,7 +470,7 @@ class TransactionManager(EventQueue): ...@@ -470,7 +470,7 @@ class TransactionManager(EventQueue):
if data is None: if data is None:
data_id = None data_id = None
else: else:
data_id = self._app.dm.holdData(checksum, data, compression) data_id = self._app.dm.holdData(checksum, oid, data, compression)
transaction.store(oid, data_id, value_serial) transaction.store(oid, data_id, value_serial)
def rebaseObject(self, ttid, oid): def rebaseObject(self, ttid, oid):
......
...@@ -104,6 +104,7 @@ class StorageDBTests(NeoUnitTestBase): ...@@ -104,6 +104,7 @@ class StorageDBTests(NeoUnitTestBase):
def test_getPartitionTable(self): def test_getPartitionTable(self):
db = self.getDB() db = self.getDB()
db.setNumPartitions(3)
uuid1, uuid2 = self.getStorageUUID(), self.getStorageUUID() uuid1, uuid2 = self.getStorageUUID(), self.getStorageUUID()
cell1 = (0, uuid1, CellStates.OUT_OF_DATE) cell1 = (0, uuid1, CellStates.OUT_OF_DATE)
cell2 = (1, uuid1, CellStates.UP_TO_DATE) cell2 = (1, uuid1, CellStates.UP_TO_DATE)
...@@ -124,7 +125,7 @@ class StorageDBTests(NeoUnitTestBase): ...@@ -124,7 +125,7 @@ class StorageDBTests(NeoUnitTestBase):
self._last_ttid = ttid = add64(self._last_ttid, 1) self._last_ttid = ttid = add64(self._last_ttid, 1)
transaction = oid_list, 'user', 'desc', 'ext', False, ttid transaction = oid_list, 'user', 'desc', 'ext', False, ttid
H = "0" * 20 H = "0" * 20
object_list = [(oid, self.db.holdData(H, '', 1), None) object_list = [(oid, self.db.holdData(H, oid, '', 1), None)
for oid in oid_list] for oid in oid_list]
return (transaction, object_list) return (transaction, object_list)
...@@ -203,6 +204,7 @@ class StorageDBTests(NeoUnitTestBase): ...@@ -203,6 +204,7 @@ class StorageDBTests(NeoUnitTestBase):
def test_setPartitionTable(self): def test_setPartitionTable(self):
db = self.getDB() db = self.getDB()
db.setNumPartitions(3)
ptid = 1 ptid = 1
uuid = self.getStorageUUID() uuid = self.getStorageUUID()
cell1 = 0, uuid, CellStates.OUT_OF_DATE cell1 = 0, uuid, CellStates.OUT_OF_DATE
...@@ -452,8 +454,8 @@ class StorageDBTests(NeoUnitTestBase): ...@@ -452,8 +454,8 @@ class StorageDBTests(NeoUnitTestBase):
tid4 = self.getNextTID() tid4 = self.getNextTID()
tid5 = self.getNextTID() tid5 = self.getNextTID()
oid1 = p64(1) oid1 = p64(1)
foo = db.holdData("3" * 20, 'foo', 0) foo = db.holdData("3" * 20, oid1, 'foo', 0)
bar = db.holdData("4" * 20, 'bar', 0) bar = db.holdData("4" * 20, oid1, 'bar', 0)
db.releaseData((foo, bar)) db.releaseData((foo, bar))
db.storeTransaction( db.storeTransaction(
tid1, ( tid1, (
......
...@@ -19,6 +19,7 @@ from MySQLdb import NotSupportedError, OperationalError ...@@ -19,6 +19,7 @@ from MySQLdb import NotSupportedError, OperationalError
from MySQLdb.constants.ER import UNKNOWN_STORAGE_ENGINE from MySQLdb.constants.ER import UNKNOWN_STORAGE_ENGINE
from ..mock import Mock from ..mock import Mock
from neo.lib.exception import DatabaseFailure from neo.lib.exception import DatabaseFailure
from neo.lib.protocol import ZERO_OID
from neo.lib.util import p64 from neo.lib.util import p64
from .. import DB_PREFIX, DB_SOCKET, DB_USER from .. import DB_PREFIX, DB_SOCKET, DB_USER
from .testStorageDBTests import StorageDBTests from .testStorageDBTests import StorageDBTests
...@@ -114,7 +115,7 @@ class StorageMySQLdbTests(StorageDBTests): ...@@ -114,7 +115,7 @@ class StorageMySQLdbTests(StorageDBTests):
self.assertEqual(2, max(len(self.db.escape(chr(x))) self.assertEqual(2, max(len(self.db.escape(chr(x)))
for x in xrange(256))) for x in xrange(256)))
self.assertEqual(2, len(self.db.escape('\0'))) self.assertEqual(2, len(self.db.escape('\0')))
self.db.storeData('\0' * 20, '\0' * (2**24-1), 0) self.db.storeData('\0' * 20, ZERO_OID, '\0' * (2**24-1), 0)
size, = query_list size, = query_list
max_allowed = self.db.__class__._max_allowed_packet max_allowed = self.db.__class__._max_allowed_packet
self.assertTrue(max_allowed - 1024 < size <= max_allowed, size) self.assertTrue(max_allowed - 1024 < size <= max_allowed, size)
...@@ -123,7 +124,7 @@ class StorageMySQLdbTests(StorageDBTests): ...@@ -123,7 +124,7 @@ class StorageMySQLdbTests(StorageDBTests):
self.db._max_allowed_packet = max_allowed_packet self.db._max_allowed_packet = max_allowed_packet
del query_list[:] del query_list[:]
self.db.storeTransaction(p64(0), self.db.storeTransaction(p64(0),
((p64(1<<i),0,None) for i in xrange(10)), None) ((p64(1<<i),1234,None) for i in xrange(10)), None)
self.assertEqual(max(query_list), max_allowed_packet) self.assertEqual(max(query_list), max_allowed_packet)
self.assertEqual(len(query_list), count) self.assertEqual(len(query_list), count)
......
...@@ -57,7 +57,7 @@ class TransactionManagerTests(NeoUnitTestBase): ...@@ -57,7 +57,7 @@ class TransactionManagerTests(NeoUnitTestBase):
self.manager.storeObject(locking_serial, ram_serial, oid, 0, "3" * 20, self.manager.storeObject(locking_serial, ram_serial, oid, 0, "3" * 20,
'bar', None) 'bar', None)
holdData = self.app.dm.mockGetNamedCalls('holdData') holdData = self.app.dm.mockGetNamedCalls('holdData')
self.assertEqual(holdData.pop(0).params, ("3" * 20, 'bar', 0)) self.assertEqual(holdData.pop(0).params, ("3" * 20, oid, 'bar', 0))
orig_object = self.manager.getObjectFromTransaction(locking_serial, orig_object = self.manager.getObjectFromTransaction(locking_serial,
oid) oid)
self.manager.updateObjectDataForPack(oid, orig_serial, None, checksum) self.manager.updateObjectDataForPack(oid, orig_serial, None, checksum)
......
...@@ -480,17 +480,18 @@ class Test(NEOThreadedTest): ...@@ -480,17 +480,18 @@ class Test(NEOThreadedTest):
def test_notifyNodeInformation(self, cluster): def test_notifyNodeInformation(self, cluster):
# translated from MasterNotificationsHandlerTests # translated from MasterNotificationsHandlerTests
# (neo.tests.client.testMasterHandler) # (neo.tests.client.testMasterHandler)
good = [1, 0].pop
if 1: if 1:
cluster.db # open DB cluster.db # open DB
s0, s1 = cluster.client.nm.getStorageList() s0, s1 = cluster.client.nm.getStorageList()
conn = s0.getConnection() conn = s0.getConnection()
self.assertFalse(conn.isClosed()) self.assertFalse(conn.isClosed())
getCellSortKey = cluster.client.cp.getCellSortKey getCellSortKey = cluster.client.cp.getCellSortKey
self.assertEqual(getCellSortKey(s0, int), 0) self.assertEqual(getCellSortKey(s0, good), 0)
cluster.neoctl.dropNode(s0.getUUID()) cluster.neoctl.dropNode(s0.getUUID())
self.assertEqual([s1], cluster.client.nm.getStorageList()) self.assertEqual([s1], cluster.client.nm.getStorageList())
self.assertTrue(conn.isClosed()) self.assertTrue(conn.isClosed())
self.assertEqual(getCellSortKey(s0, int), 1) self.assertEqual(getCellSortKey(s0, good), 1)
# XXX: the test originally checked that 'unregister' method # XXX: the test originally checked that 'unregister' method
# was called (even if it's useless in this case), # was called (even if it's useless in this case),
# but we would need an API to do that easily. # but we would need an API to do that easily.
...@@ -1438,7 +1439,7 @@ class Test(NEOThreadedTest): ...@@ -1438,7 +1439,7 @@ class Test(NEOThreadedTest):
bad = [] bad = []
ok = [] ok = []
def data_args(value): def data_args(value):
return makeChecksum(value), value, 0 return makeChecksum(value), ZERO_OID, value, 0
node_list = [] node_list = []
for i, s in enumerate(cluster.storage_list): for i, s in enumerate(cluster.storage_list):
node_list.append(s.uuid) node_list.append(s.uuid)
......
...@@ -132,10 +132,14 @@ class ReplicationTests(NEOThreadedTest): ...@@ -132,10 +132,14 @@ class ReplicationTests(NEOThreadedTest):
self.assertEqual(backup.neoctl.getClusterState(), self.assertEqual(backup.neoctl.getClusterState(),
ClusterStates.RUNNING) ClusterStates.RUNNING)
# Restart and switch to BACKINGUP mode again.
backup.stop() backup.stop()
backup.start() backup.start()
backup.neoctl.setClusterState(ClusterStates.STARTING_BACKUP) backup.neoctl.setClusterState(ClusterStates.STARTING_BACKUP)
self.tic() self.tic()
# Leave BACKINGUP mode when 1 replica is late. The cluster
# remains in STOPPING_BACKUP state until it catches up.
with backup.master.filterConnection(*backup.storage_list) as f: with backup.master.filterConnection(*backup.storage_list) as f:
f.add(delaySecondary) f.add(delaySecondary)
while not f.filtered_count: while not f.filtered_count:
...@@ -147,6 +151,8 @@ class ReplicationTests(NEOThreadedTest): ...@@ -147,6 +151,8 @@ class ReplicationTests(NEOThreadedTest):
self.assertEqual(np*nr, self.checkBackup(backup, self.assertEqual(np*nr, self.checkBackup(backup,
max_tid=backup.last_tid)) max_tid=backup.last_tid))
# Again but leave BACKINGUP mode when a storage node is
# receiving data from the upstream cluster.
backup.stop() backup.stop()
backup.start() backup.start()
backup.neoctl.setClusterState(ClusterStates.STARTING_BACKUP) backup.neoctl.setClusterState(ClusterStates.STARTING_BACKUP)
...@@ -162,6 +168,48 @@ class ReplicationTests(NEOThreadedTest): ...@@ -162,6 +168,48 @@ class ReplicationTests(NEOThreadedTest):
self.assertEqual(np*nr, self.checkBackup(backup, self.assertEqual(np*nr, self.checkBackup(backup,
max_tid=backup.last_tid)) max_tid=backup.last_tid))
storage = upstream.getZODBStorage()
# Check that replication from upstream is resumed even if
# upstream is idle.
backup.neoctl.setClusterState(ClusterStates.STARTING_BACKUP)
self.tic()
x = backup.master.backup_app.primary_partition_dict
new_oid_storage = x[0]
with upstream.moduloTID(next(p for p, n in x.iteritems()
if n is not new_oid_storage)), \
ConnectionFilter() as f:
f.delayAddObject()
# Transaction that touches 2 primary cells on 2 different
# nodes.
txn = transaction.Transaction()
tid = storage.load(ZERO_OID)[1]
storage.tpc_begin(txn)
storage.store(ZERO_OID, tid, '', '', txn)
storage.tpc_vote(txn)
storage.tpc_finish(txn)
self.tic()
# Stop when exactly 1 of the 2 cells is synced with
# upstream.
backup.stop()
backup.start()
self.assertEqual(np*nr, self.checkBackup(backup,
max_tid=backup.last_tid))
# Check that replication to secondary cells is resumed even if
# upstream is idle.
with backup.master.filterConnection(*backup.storage_list) as f:
f.add(delaySecondary)
txn = transaction.Transaction()
storage.tpc_begin(txn)
storage.tpc_finish(txn)
self.tic()
backup.stop()
backup.start()
self.assertEqual(np*nr, self.checkBackup(backup,
max_tid=backup.last_tid))
@predictable_random() @predictable_random()
def testBackupNodeLost(self): def testBackupNodeLost(self):
"""Check backup cluster can recover after random connection loss """Check backup cluster can recover after random connection loss
......
...@@ -60,7 +60,7 @@ else: ...@@ -60,7 +60,7 @@ else:
setup( setup(
name = 'neoppod', name = 'neoppod',
version = '1.8.1', version = '1.9',
description = __doc__.strip(), description = __doc__.strip(),
author = 'Nexedi SA', author = 'Nexedi SA',
author_email = 'neo-dev@erp5.org', author_email = 'neo-dev@erp5.org',
......
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