Commit b27db46f authored by Julien Muchembled's avatar Julien Muchembled

Rename node states: DOWN -> UNKNOWN, TEMPORARILY_DOWN -> DOWN

parent f39babe5
Documentation
- Clarify node state signification, and consider renaming them in the code.
Ideas:
TEMPORARILY_DOWN becomes UNAVAILABLE
- Clarify the use of each error codes:
- NOT_READY removed (connection kept opened until ready)
- Split PROTOCOL_ERROR (BAD IDENTIFICATION, ...)
......
......@@ -142,7 +142,7 @@ class PrimaryNotificationsHandler(MTEventHandler):
def notifyNodeInformation(self, conn, timestamp, node_list):
super(PrimaryNotificationsHandler, self).notifyNodeInformation(
conn, timestamp, node_list)
# XXX: 'update' automatically closes DOWN nodes. Do we really want
# XXX: 'update' automatically closes UNKNOWN nodes. Do we really want
# to do the same thing for nodes in other non-running states ?
getByUUID = self.app.nm.getByUUID
for node in node_list:
......
......@@ -138,7 +138,7 @@ class EventHandler(object):
def connectionClosed(self, conn):
"""Called when a connection is closed by the peer."""
logging.debug('connection closed for %r', conn)
self.connectionLost(conn, NodeStates.TEMPORARILY_DOWN)
self.connectionLost(conn, NodeStates.DOWN)
def connectionLost(self, conn, new_state):
""" this is a method to override in sub-handlers when there is no need
......
......@@ -31,8 +31,7 @@ class Node(object):
_identified = False
id_timestamp = None
def __init__(self, manager, address=None, uuid=None,
state=NodeStates.TEMPORARILY_DOWN):
def __init__(self, manager, address=None, uuid=None, state=NodeStates.DOWN):
self._state = state
self._address = address
self._uuid = uuid
......@@ -64,7 +63,7 @@ class Node(object):
def setState(self, new_state):
if self._state == new_state:
return
if new_state == NodeStates.DOWN:
if new_state == NodeStates.UNKNOWN:
self._manager.remove(self)
self._state = new_state
else:
......@@ -271,7 +270,7 @@ class NodeManager(EventQueue):
if node in self._node_set:
logging.warning('adding a known node %r, ignoring', node)
return
assert not node.isDown(), node
assert not node.isUnknown(), node
self._node_set.add(node)
self._updateAddress(node, None)
self._updateUUID(node, None)
......@@ -321,7 +320,7 @@ class NodeManager(EventQueue):
set_dict.setdefault(new_key, set()).add(node)
def _updateState(self, node, old_state):
assert not node.isDown(), node
assert not node.isUnknown(), node
self.__updateSet(self._state_dict, old_state, node.getState(), node)
def getList(self, node_filter=None):
......@@ -427,7 +426,7 @@ class NodeManager(EventQueue):
log_args = node_type, uuid_str(uuid), addr, state, id_timestamp
if node is None:
assert state != NodeStates.DOWN, (self._node_set,) + log_args
assert state != NodeStates.UNKNOWN, (self._node_set,) + log_args
node = self._createNode(klass, address=addr, uuid=uuid,
state=state)
logging.debug('creating node %r', node)
......@@ -439,7 +438,7 @@ class NodeManager(EventQueue):
'Discrepancy between node_by_uuid (%r) and ' \
'node_by_addr (%r)' % (node_by_uuid, node_by_addr)
node_by_uuid.setUUID(None)
if state == NodeStates.DOWN:
if state == NodeStates.UNKNOWN:
logging.debug('dropping node %r (%r), found with %s '
'%s %s %s %s', node, node.isConnected(), *log_args)
if node.isConnected():
......
......@@ -119,9 +119,9 @@ def NodeTypes():
@Enum
def NodeStates():
RUNNING
TEMPORARILY_DOWN
UNKNOWN
DOWN
RUNNING
PENDING
@Enum
......@@ -146,8 +146,8 @@ def CellStates():
# used for logging
node_state_prefix_dict = {
NodeStates.RUNNING: 'R',
NodeStates.TEMPORARILY_DOWN: 'T',
NodeStates.DOWN: 'D',
NodeStates.UNKNOWN: 'U',
NodeStates.PENDING: 'P',
}
......
......@@ -168,7 +168,7 @@ class PartitionTable(object):
def _setCell(self, offset, node, state):
if state == CellStates.DISCARDED:
return self.removeCell(offset, node)
if node.isDown():
if node.isUnknown():
raise PartitionTableException('Invalid node state')
self.count_dict.setdefault(node, 0)
......
......@@ -238,7 +238,7 @@ class Application(BaseApplication):
# If I know any storage node, make sure that they are not in the
# running state, because they are not connected at this stage.
for node in self.nm.getStorageList():
assert node.isTemporarilyDown(), node
assert node.isDown(), node
if self.uuid is None:
self.uuid = self.getNewUUID(None, self.server, NodeTypes.MASTER)
......@@ -340,7 +340,7 @@ class Application(BaseApplication):
try:
if master_conn is None:
for node in self.nm.getMasterList():
node.setTemporarilyDown()
node.setDown()
node = self.primary_master
failed.add(node.getAddress())
if not node.isConnected(True):
......@@ -487,7 +487,7 @@ class Application(BaseApplication):
if node.isStorage():
conn.send(Packets.NotifyNodeInformation(monotonic_time(), ((
node.getType(), node.getAddress(), node.getUUID(),
NodeStates.TEMPORARILY_DOWN, None),)))
NodeStates.DOWN, None),)))
if conn.pending():
conn.abort()
continue
......
......@@ -82,13 +82,12 @@ class BaseServiceHandler(MasterHandler):
# was in pending state, so drop it from the node manager to forget
# it and do not set in running state when it comes back
logging.info('drop a pending node from the node manager')
node.setDown()
elif node.isTemporarilyDown():
# Already put in TEMPORARILY_DOWN state
# by AdministrationHandler.setNodeState
node.setUnknown()
elif node.isDown():
# Already put in DOWN state by AdministrationHandler.setNodeState
return
else:
node.setTemporarilyDown()
node.setDown()
app.broadcastNodesInformation([node])
if app.truncate_tid:
raise StoppedOperation
......
......@@ -34,8 +34,8 @@ CLUSTER_STATE_WORKFLOW = {
ClusterStates.STARTING_BACKUP),
}
NODE_STATE_WORKFLOW = {
NodeTypes.MASTER: (NodeStates.TEMPORARILY_DOWN,),
NodeTypes.STORAGE: (NodeStates.TEMPORARILY_DOWN, NodeStates.DOWN),
NodeTypes.MASTER: (NodeStates.DOWN,),
NodeTypes.STORAGE: (NodeStates.DOWN, NodeStates.UNKNOWN),
}
class AdministrationHandler(MasterHandler):
......@@ -95,7 +95,7 @@ class AdministrationHandler(MasterHandler):
message = ('state changed' if state_changed else
'node already in %s state' % state)
if node.isStorage():
keep = state == NodeStates.TEMPORARILY_DOWN
keep = state == NodeStates.DOWN
try:
cell_list = app.pt.dropNodeList([node], keep)
except PartitionTableException, e:
......
......@@ -15,7 +15,7 @@
# along with this program. If not, see <http://www.gnu.org/licenses/>.
from neo.lib.handler import DelayEvent
from neo.lib.protocol import NodeStates, Packets, ProtocolError, MAX_TID, Errors
from neo.lib.protocol import Packets, ProtocolError, MAX_TID, Errors
from ..app import monotonic_time
from . import MasterHandler
......@@ -28,7 +28,7 @@ class ClientServiceHandler(MasterHandler):
node = app.nm.getByUUID(conn.getUUID())
assert node is not None, conn
app.tm.clientLost(node)
node.setState(NodeStates.DOWN)
node.setUnknown()
app.broadcastNodesInformation([node])
app.nm.remove(node)
......
......@@ -26,7 +26,7 @@ class SecondaryHandler(MasterHandler):
def _connectionLost(self, conn):
app = self.app
node = app.nm.getByUUID(conn.getUUID())
node.setTemporarilyDown()
node.setDown()
app.broadcastNodesInformation([node])
......@@ -91,5 +91,5 @@ class PrimaryHandler(ElectionHandler):
conn, timestamp, node_list)
for node_type, _, uuid, state, _ in node_list:
assert node_type == NodeTypes.MASTER, node_type
if uuid == self.app.uuid and state == NodeStates.TEMPORARILY_DOWN:
if uuid == self.app.uuid and state == NodeStates.DOWN:
sys.exit()
......@@ -71,11 +71,11 @@ class RecoveryManager(MasterHandler):
for node in app.nm.getMasterList():
if not (node is app._node or node.isConnected(True)):
# During recovery, master nodes are not put back in
# TEMPORARILY_DOWN state by handlers. This is done
# DOWN state by handlers. This is done
# entirely in this method (here and after this poll
# loop), to minimize the notification packets.
if not node.isTemporarilyDown():
node.setTemporarilyDown()
if not node.isDown():
node.setDown()
node_list.append(node)
ClientConnection(app, app.election_handler, node)
if node_list:
......@@ -128,10 +128,10 @@ class RecoveryManager(MasterHandler):
if not (node is app._node or node.isIdentified()):
if node.isConnected(True):
node.getConnection().close()
assert node.isTemporarilyDown(), node
elif not node.isTemporarilyDown():
assert node.isDown(), node
elif not node.isDown():
assert self.try_secondary, node
node.setTemporarilyDown()
node.setDown()
node_list.append(node)
app.broadcastNodesInformation(node_list)
......
......@@ -157,10 +157,10 @@ class NeoCTL(BaseApplication):
return self.setClusterState(ClusterStates.VERIFYING)
def killNode(self, node):
return self._setNodeState(node, NodeStates.TEMPORARILY_DOWN)
return self._setNodeState(node, NodeStates.DOWN)
def dropNode(self, node):
return self._setNodeState(node, NodeStates.DOWN)
return self._setNodeState(node, NodeStates.UNKNOWN)
def getPrimary(self):
"""
......
......@@ -56,8 +56,8 @@ class BaseMasterHandler(BaseHandler):
if uuid == self.app.uuid:
# This is me, do what the master tell me
logging.info("I was told I'm %s", state)
if state in (NodeStates.DOWN, NodeStates.TEMPORARILY_DOWN):
erase = state == NodeStates.DOWN
if state in (NodeStates.UNKNOWN, NodeStates.DOWN):
erase = state == NodeStates.UNKNOWN
self.app.shutdown(erase=erase)
elif node_type == NodeTypes.CLIENT and state != NodeStates.RUNNING:
logging.info('Notified of non-running client, abort (%s)',
......
......@@ -18,8 +18,7 @@ import weakref
from functools import wraps
from neo.lib.connection import ConnectionClosed
from neo.lib.handler import DelayEvent, EventHandler
from neo.lib.protocol import Errors, NodeStates, Packets, ProtocolError, \
ZERO_HASH
from neo.lib.protocol import Errors, Packets, ProtocolError, ZERO_HASH
def checkConnectionIsReplicatorConnection(func):
def wrapper(self, conn, *args, **kw):
......@@ -53,7 +52,7 @@ class StorageOperationHandler(EventHandler):
node = app.nm.getByUUID(uuid)
else:
node = app.nm.getByAddress(conn.getAddress())
node.setState(NodeStates.DOWN)
node.setUnknown()
replicator = app.replicator
if replicator.current_node is node:
replicator.abort()
......
......@@ -609,9 +609,8 @@ class NEOCluster(object):
self.expectStorageState(process.getUUID(), NodeStates.PENDING,
*args, **kw)
def expectUnavailable(self, process, *args, **kw):
self.expectStorageState(process.getUUID(),
NodeStates.TEMPORARILY_DOWN, *args, **kw)
def expectDown(self, process, *args, **kw):
self.expectStorageState(process.getUUID(), NodeStates.DOWN, *args, **kw)
def expectPrimary(self, uuid=None, *args, **kw):
def callback(last_try):
......@@ -674,7 +673,7 @@ class NEOCluster(object):
return current_try, current_try
self.expectCondition(callback, *args, **kw)
def expectStorageNotKnown(self, process, *args, **kw):
def expectStorageUnknown(self, process, *args, **kw):
process_uuid = process.getUUID()
def expected_storage_not_known(last_try):
for storage in self.getStorageList():
......
......@@ -48,7 +48,7 @@ class ClusterTests(NEOFunctionalTest):
neo.stop()
neo.run(except_storages=(s2, ))
neo.expectPending(s1)
neo.expectUnavailable(s2)
neo.expectDown(s2)
neo.expectClusterRecovering()
# Starting missing storage allows cluster to exit Recovery without
# neoctl action.
......@@ -61,11 +61,11 @@ class ClusterTests(NEOFunctionalTest):
neo.stop()
neo.run(except_storages=(s2, ))
neo.expectPending(s1)
neo.expectUnavailable(s2)
neo.expectDown(s2)
neo.expectClusterRecovering()
neo.startCluster()
neo.expectRunning(s1)
neo.expectUnavailable(s2)
neo.expectDown(s2)
neo.expectClusterRunning()
def testClusterBreaks(self):
......@@ -149,20 +149,20 @@ class ClusterTests(NEOFunctionalTest):
)
storages = self.neo.getStorageProcessList()
self.neo.run(except_storages=storages)
self.neo.expectStorageNotKnown(storages[0])
self.neo.expectStorageNotKnown(storages[1])
self.neo.expectStorageUnknown(storages[0])
self.neo.expectStorageUnknown(storages[1])
storages[0].start()
self.neo.expectPending(storages[0])
self.neo.expectStorageNotKnown(storages[1])
self.neo.expectStorageUnknown(storages[1])
storages[1].start()
self.neo.expectPending(storages[0])
self.neo.expectPending(storages[1])
storages[0].stop()
self.neo.expectUnavailable(storages[0])
self.neo.expectDown(storages[0])
self.neo.expectPending(storages[1])
storages[1].stop()
self.neo.expectUnavailable(storages[0])
self.neo.expectUnavailable(storages[1])
self.neo.expectDown(storages[0])
self.neo.expectDown(storages[1])
def test_suite():
return unittest.makeSuite(ClusterTests)
......
......@@ -59,7 +59,7 @@ class MasterTests(NEOFunctionalTest):
self.assertEqual(len(killed_uuid_list), 1)
uuid = killed_uuid_list[0]
# Check the state of the primary we just killed
self.neo.expectMasterState(uuid, (None, NodeStates.TEMPORARILY_DOWN))
self.neo.expectMasterState(uuid, (None, NodeStates.DOWN))
# BUG: The following check expects neoctl to reconnect before
# the election finishes.
self.assertEqual(self.neo.getPrimary(), None)
......@@ -78,12 +78,12 @@ class MasterTests(NEOFunctionalTest):
# Test sanity checks.
self.assertEqual(len(killed_uuid_list), 1)
self.neo.expectMasterState(killed_uuid_list[0],
NodeStates.TEMPORARILY_DOWN)
NodeStates.DOWN)
self.assertEqual(len(self.neo.getMasterList()), MASTER_NODE_COUNT)
uuid, = self.neo.killPrimary()
# Check the state of the primary we just killed
self.neo.expectMasterState(uuid, NodeStates.TEMPORARILY_DOWN)
self.neo.expectMasterState(uuid, NodeStates.DOWN)
# Check that a primary master arose.
self.neo.expectPrimary(timeout=10)
# Check that the uuid really changed.
......
......@@ -168,7 +168,7 @@ class StorageTests(NEOFunctionalTest):
self.neo.neoctl.killNode(started[0].getUUID())
# Cluster still operational. All cells of first storage should be
# outdated.
self.neo.expectUnavailable(started[0])
self.neo.expectDown(started[0])
self.neo.expectOudatedCells(2)
self.neo.expectClusterRunning()
......@@ -177,7 +177,7 @@ class StorageTests(NEOFunctionalTest):
started[1].stop()
# Cluster not operational anymore. Only cells of second storage that
# were shared with the third one should become outdated.
self.neo.expectUnavailable(started[1])
self.neo.expectDown(started[1])
self.neo.expectClusterRecovering()
self.neo.expectOudatedCells(3)
......@@ -198,7 +198,7 @@ class StorageTests(NEOFunctionalTest):
# stop it, the cluster must switch to verification
started[0].stop()
self.neo.expectUnavailable(started[0])
self.neo.expectDown(started[0])
self.neo.expectClusterRecovering()
# client must have been disconnected
self.assertEqual(len(self.neo.getClientlist()), 0)
......@@ -224,7 +224,7 @@ class StorageTests(NEOFunctionalTest):
# stop one storage, cluster must remains running
started[0].stop()
self.neo.expectUnavailable(started[0])
self.neo.expectDown(started[0])
self.neo.expectRunning(started[1])
self.neo.expectRunning(started[2])
self.neo.expectOudatedCells(number=10)
......@@ -232,17 +232,17 @@ class StorageTests(NEOFunctionalTest):
# stop a second storage, cluster is still running
started[1].stop()
self.neo.expectUnavailable(started[0])
self.neo.expectUnavailable(started[1])
self.neo.expectDown(started[0])
self.neo.expectDown(started[1])
self.neo.expectRunning(started[2])
self.neo.expectOudatedCells(number=20)
self.neo.expectClusterRunning()
# stop the last, cluster died
started[2].stop()
self.neo.expectUnavailable(started[0])
self.neo.expectUnavailable(started[1])
self.neo.expectUnavailable(started[2])
self.neo.expectDown(started[0])
self.neo.expectDown(started[1])
self.neo.expectDown(started[2])
self.neo.expectOudatedCells(number=20)
self.neo.expectClusterRecovering()
......@@ -312,7 +312,7 @@ class StorageTests(NEOFunctionalTest):
# kill one storage, it should be set as unavailable
started[0].stop()
self.neo.expectUnavailable(started[0])
self.neo.expectDown(started[0])
self.neo.expectRunning(started[1])
# and the partition table must not change
self.neo.expectAssignedCells(started[0], 10)
......@@ -320,7 +320,7 @@ class StorageTests(NEOFunctionalTest):
# ask neoctl to drop it
self.neo.neoctl.dropNode(started[0].getUUID())
self.neo.expectStorageNotKnown(started[0])
self.neo.expectStorageUnknown(started[0])
self.neo.expectAssignedCells(started[0], 0)
self.neo.expectAssignedCells(started[1], 10)
self.assertRaises(RuntimeError, self.neo.neoctl.dropNode,
......@@ -335,7 +335,7 @@ class StorageTests(NEOFunctionalTest):
(started, stopped) = self.__setup(storage_number=2, replicas=1,
pending_number=1, partitions=10)
self.neo.expectRunning(started[0])
self.neo.expectStorageNotKnown(stopped[0])
self.neo.expectStorageUnknown(stopped[0])
self.neo.expectOudatedCells(number=0)
# populate the cluster with some data
......@@ -362,7 +362,7 @@ class StorageTests(NEOFunctionalTest):
# kill the first storage
started[0].stop()
self.neo.expectUnavailable(started[0])
self.neo.expectDown(started[0])
self.neo.expectOudatedCells(number=10)
self.neo.expectAssignedCells(started[0], 10)
self.neo.expectAssignedCells(stopped[0], 10)
......@@ -371,7 +371,7 @@ class StorageTests(NEOFunctionalTest):
# drop it from partition table
self.neo.neoctl.dropNode(started[0].getUUID())
self.neo.expectStorageNotKnown(started[0])
self.neo.expectStorageUnknown(started[0])
self.neo.expectRunning(stopped[0])
self.neo.expectAssignedCells(started[0], 0)
self.neo.expectAssignedCells(stopped[0], 10)
......@@ -395,12 +395,12 @@ class StorageTests(NEOFunctionalTest):
# drop the first then the second storage
started[0].stop()
self.neo.expectUnavailable(started[0])
self.neo.expectDown(started[0])
self.neo.expectRunning(started[1])
self.neo.expectOudatedCells(number=10)
started[1].stop()
self.neo.expectUnavailable(started[0])
self.neo.expectUnavailable(started[1])
self.neo.expectDown(started[0])
self.neo.expectDown(started[1])
self.neo.expectOudatedCells(number=10)
self.neo.expectClusterRecovering()
# XXX: need to sync with storages first
......@@ -409,7 +409,7 @@ class StorageTests(NEOFunctionalTest):
# restart the cluster with the first storage killed
self.neo.run(except_storages=[started[1]])
self.neo.expectPending(started[0])
self.neo.expectUnavailable(started[1])
self.neo.expectDown(started[1])
self.neo.expectClusterRecovering()
# Cluster doesn't know there are outdated cells
self.neo.expectOudatedCells(number=0)
......
......@@ -70,9 +70,9 @@ class MasterPartitionTableTests(NeoUnitTestBase):
pt._setCell(0, sn1, CellStates.OUT_OF_DATE)
sn1.setState(NodeStates.RUNNING)
pt._setCell(1, sn2, CellStates.UP_TO_DATE)
sn2.setState(NodeStates.TEMPORARILY_DOWN)
sn2.setState(NodeStates.DOWN)
pt._setCell(2, sn3, CellStates.UP_TO_DATE)
sn3.setState(NodeStates.DOWN)
sn3.setState(NodeStates.UNKNOWN)
pt._setCell(3, sn4, CellStates.UP_TO_DATE)
sn4.setState(NodeStates.RUNNING)
# outdate nodes
......@@ -146,7 +146,7 @@ class MasterPartitionTableTests(NeoUnitTestBase):
uuid2 = self.getStorageUUID()
server2 = ("127.0.0.2", 19001)
sn2 = self.createStorage(server2, uuid2)
sn2.setState(NodeStates.TEMPORARILY_DOWN)
sn2.setState(NodeStates.DOWN)
# add node without uuid
server3 = ("127.0.0.3", 19001)
sn3 = self.createStorage(server3, None, NodeStates.RUNNING)
......
......@@ -94,7 +94,7 @@ class MasterRecoveryTests(NeoUnitTestBase):
conn = self.getFakeConnection(uuid, self.storage_port)
offset = 1000000
self.assertFalse(self.app.pt.hasOffset(offset))
cell_list = [(offset, ((uuid, NodeStates.DOWN,),),)]
cell_list = [(offset, ((uuid, NodeStates.UNKNOWN,),),)]
node.setPending()
self.checkProtocolErrorRaised(recovery.answerPartitionTable, conn,
2, cell_list)
......
......@@ -35,7 +35,7 @@ class NodesTests(NeoUnitTestBase):
address = ('127.0.0.1', 10000)
uuid = self.getNewUUID(None)
node = Node(self.nm, address=address, uuid=uuid)
self.assertEqual(node.getState(), NodeStates.TEMPORARILY_DOWN)
self.assertEqual(node.getState(), NodeStates.DOWN)
self.assertEqual(node.getAddress(), address)
self.assertEqual(node.getUUID(), uuid)
self.assertTrue(time() - 1 < node.getLastStateChange() < time())
......@@ -43,7 +43,7 @@ class NodesTests(NeoUnitTestBase):
def testState(self):
""" Check if the last changed time is updated when state is changed """
node = Node(self.nm)
self.assertEqual(node.getState(), NodeStates.TEMPORARILY_DOWN)
self.assertEqual(node.getState(), NodeStates.DOWN)
self.assertTrue(time() - 1 < node.getLastStateChange() < time())
previous_time = node.getLastStateChange()
node.setState(NodeStates.RUNNING)
......@@ -156,12 +156,12 @@ class NodeManagerTests(NeoUnitTestBase):
old_uuid = self.storage.getUUID()
new_uuid = self.getStorageUUID()
node_list = (
(NodeTypes.CLIENT, None, self.client.getUUID(), NodeStates.DOWN, None),
(NodeTypes.CLIENT, None, self.client.getUUID(), NodeStates.UNKNOWN, None),
(NodeTypes.MASTER, new_address, self.master.getUUID(), NodeStates.RUNNING, None),
(NodeTypes.STORAGE, self.storage.getAddress(), new_uuid,
NodeStates.RUNNING, None),
(NodeTypes.ADMIN, self.admin.getAddress(), self.admin.getUUID(),
NodeStates.TEMPORARILY_DOWN, None),
NodeStates.DOWN, None),
)
app = Mock()
app.pt = Mock({'dropNode': True})
......@@ -180,9 +180,9 @@ class NodeManagerTests(NeoUnitTestBase):
new_storage = storage_list[0]
self.assertNotEqual(new_storage.getUUID(), old_uuid)
self.assertEqual(new_storage.getState(), NodeStates.RUNNING)
# admin is still here but in TEMPORARILY_DOWN state
# admin is still here but in DOWN state
self.checkNodes([self.master, self.admin, new_storage])
self.assertEqual(self.admin.getState(), NodeStates.TEMPORARILY_DOWN)
self.assertEqual(self.admin.getState(), NodeStates.DOWN)
class MasterDBTests(NeoUnitTestBase):
......
......@@ -34,7 +34,7 @@ class PartitionTableTests(NeoUnitTestBase):
# check getter
self.assertEqual(cell.getNode(), sn)
self.assertEqual(cell.getState(), CellStates.OUT_OF_DATE)
self.assertEqual(cell.getNodeState(), NodeStates.TEMPORARILY_DOWN)
self.assertEqual(cell.getNodeState(), NodeStates.DOWN)
self.assertEqual(cell.getUUID(), uuid)
self.assertEqual(cell.getAddress(), server)
# check state setter
......@@ -109,7 +109,7 @@ class PartitionTableTests(NeoUnitTestBase):
for x in xrange(num_partitions):
self.assertEqual(len(pt.partition_list[x]), 0)
self.assertEqual(pt.count_dict[sn1], 0)
sn1.setState(NodeStates.DOWN)
sn1.setState(NodeStates.UNKNOWN)
self.assertRaises(PartitionTableException, pt._setCell,
0, sn1, CellStates.UP_TO_DATE)
for x in xrange(num_partitions):
......@@ -325,7 +325,7 @@ class PartitionTableTests(NeoUnitTestBase):
self.assertFalse(pt.operational())
# adding a node in all partition
sn1 = createStorage()
sn1.setState(NodeStates.TEMPORARILY_DOWN)
sn1.setState(NodeStates.DOWN)
for x in xrange(num_partitions):
pt._setCell(x, sn1, CellStates.FEEDING)
self.assertTrue(pt.filled())
......
......@@ -552,7 +552,7 @@ class Test(NEOThreadedTest):
# restart it with one storage only
if 1:
cluster.start(storage_list=(s1,))
self.assertEqual(NodeStates.TEMPORARILY_DOWN,
self.assertEqual(NodeStates.DOWN,
cluster.getNodeState(s2))
@with_cluster(storage_count=2, partitions=2, replicas=1)
......
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