Commit 2ca7c335 authored by Julien Muchembled's avatar Julien Muchembled

master: improve algorithm to tweak the partition table

The most important change is that it does not discard readable cells too
quickly anymore. A partition can now have multiple FEEDING cells, to avoid
going below the wanted level of replication.

The new algorithm is also better at minimizing the amount replication.
parent ca58ccd7
......@@ -174,8 +174,9 @@ class AdministrationHandler(MasterHandler):
ClusterStates.BACKINGUP):
raise ProtocolError('Can not tweak partition table in %s state'
% state)
app.broadcastPartitionChanges(app.pt.tweak(
map(app.nm.getByUUID, uuid_list)))
app.broadcastPartitionChanges(app.pt.tweak([node
for node in app.nm.getStorageList()
if node.getUUID() in uuid_list or not node.isRunning()]))
conn.answer(Errors.Ack(''))
def truncate(self, conn, tid):
......
......@@ -14,7 +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 collections import defaultdict
from collections import Counter, defaultdict
import neo.lib.pt
from neo.lib import logging
from neo.lib.protocol import CellStates, ZERO_TID
......@@ -43,16 +43,6 @@ class Cell(neo.lib.pt.Cell):
neo.lib.pt.Cell = Cell
class MappedNode(object):
def __init__(self, node):
self.node = node
self.assigned = set()
def __getattr__(self, attr):
return getattr(self.node, attr)
class PartitionTable(neo.lib.pt.PartitionTable):
"""This class manages a partition table for the primary master node"""
......@@ -68,32 +58,14 @@ class PartitionTable(neo.lib.pt.PartitionTable):
def make(self, node_list):
"""Make a new partition table from scratch."""
# start with the first PTID
self._id = 1
# First, filter the list of nodes.
node_list = [n for n in node_list if n.isRunning() \
and n.getUUID() is not None]
if len(node_list) == 0:
# Impossible.
raise RuntimeError, 'cannot make a partition table with an ' \
'empty storage node list'
# Take it into account that the number of storage nodes may be less
# than the number of replicas.
repeats = min(self.nr + 1, len(node_list))
index = 0
for offset in xrange(self.np):
row = []
for _ in xrange(repeats):
node = node_list[index]
row.append(Cell(node))
self.count_dict[node] = self.count_dict.get(node, 0) + 1
index += 1
if index == len(node_list):
index = 0
self.partition_list[offset] = row
self.num_filled_rows = self.np
assert self._id is None and node_list, (self._id, node_list)
for node in node_list:
assert node.isRunning() and node.getUUID() is not None, node
self.addNodeList(node_list)
self.tweak()
for node, count in self.count_dict.items():
if not count:
del self.count_dict[node]
def dropNodeList(self, node_list, simulate=False):
partition_list = []
......@@ -161,8 +133,9 @@ class PartitionTable(neo.lib.pt.PartitionTable):
def setUpToDate(self, node, offset):
"""Set a cell as up-to-date"""
uuid = node.getUUID()
# check the partition is assigned and known as outdated
for cell in self.getCellList(offset):
# Check the partition is assigned and known as outdated.
row = self.partition_list[offset]
for cell in row:
if cell.getUUID() == uuid:
if cell.isOutOfDate() and cell.updatable:
break
......@@ -170,17 +143,26 @@ class PartitionTable(neo.lib.pt.PartitionTable):
else:
raise neo.lib.pt.PartitionTableException('Non-assigned partition')
# update the partition table
# Update the partition table.
self._setCell(offset, node, CellStates.UP_TO_DATE)
cell_list = [(offset, uuid, CellStates.UP_TO_DATE)]
# If the partition contains a feeding cell, drop it now.
for feeding_cell in self.getCellList(offset):
if feeding_cell.isFeeding():
node = feeding_cell.getNode()
self.removeCell(offset, node)
cell_list.append((offset, node.getUUID(), CellStates.DISCARDED))
break
# Do no keep too many feeding cells.
readable_list = filter(Cell.isReadable, row)
iter_feeding = (cell.getNode() for cell in readable_list
if cell.isFeeding())
# If all cells are readable, we can now drop all feeding cells.
if len(readable_list) != len(row):
# Else we normally discard at most 1 cell. In the case that cells
# became non-readable since the last tweak, we want to avoid going
# below the wanted number of replicas. Also first try to discard
# feeding cells from nodes that it was decided to drop.
iter_feeding = sorted(iter_feeding, key=lambda node: not all(
cell.isFeeding() for _, cell in self.iterNodeCell(node)
))[:max(0, len(readable_list) - self.nr)]
for node in iter_feeding:
self.removeCell(offset, node)
cell_list.append((offset, node.getUUID(), CellStates.DISCARDED))
return cell_list
......@@ -196,87 +178,193 @@ class PartitionTable(neo.lib.pt.PartitionTable):
def tweak(self, drop_list=()):
"""Optimize partition table
This is done by computing a minimal diff between current partition table
and what make() would do.
This reassigns cells in 3 ways:
- Discard cells of nodes listed in 'drop_list'. For partitions with too
few readable cells, some cells are instead marked as FEEDING. This is
a preliminary step to drop these nodes, otherwise the partition table
could become non-operational.
- Other nodes must have the same number of cells, off by 1.
- When a transaction creates new objects (oids are roughly allocated
sequentially), we expect better performance by maximizing the number
of involved nodes (i.e. parallelizing writes).
Examples of optimal partition tables with np=10, nr=1 and 5 nodes:
UU... ..UU.
..UU. U...U
U...U .UU..
.UU.. ...UU
...UU UU...
UU... ..UU.
..UU. U...U
U...U .UU..
.UU.. ...UU
...UU UU...
The above 2 PT only differ by permutation of nodes, and this method
plays on it to minimize the resulting amount of replication.
For performance reasons, this algorithm uses a heuristic.
When (np * nr) is not a multiple of the number of nodes, some nodes
have 1 extra cell compared to other. In such case, other optimal PT
could be considered by rotation of the partitions. Actually np times
more, but it's not worth it since they don't differ enough (if np is
big enough) and we don't already do an exhaustive search.
Example with np=3, nr=1 and 2 nodes:
U. .U U.
.U U. U.
U. U. .U
"""
assigned_dict = {x: {} for x in self.count_dict}
readable_list = [set() for x in xrange(self.np)]
# Collect some data in a usable form for the rest of the method.
node_list = {node: {} for node in self.count_dict
if node not in drop_list}
drop_list = defaultdict(list)
for offset, row in enumerate(self.partition_list):
for cell in row:
if cell.isReadable():
readable_list[offset].add(cell)
assigned_dict[cell.getNode()][offset] = cell
pt = PartitionTable(self.np, self.nr)
drop_list = set(drop_list).intersection(assigned_dict)
node_set = {MappedNode(x) for x in assigned_dict
if x not in drop_list}
pt.make(node_set)
for offset, row in enumerate(pt.partition_list):
for cell in row:
if cell.isReadable():
cell.getNode().assigned.add(offset)
def map_nodes():
node_list = []
for node, assigned in assigned_dict.iteritems():
if node in drop_list:
yield node, frozenset()
continue
readable = {offset for offset, cell in assigned.iteritems()
if cell.isReadable()}
# the criterion on UUID is purely cosmetic
node_list.append((len(readable), len(assigned),
-node.getUUID(), readable, node))
node_list.sort(reverse=1)
for _, _, _, readable, node in node_list:
assigned = assigned_dict[node]
mapped = min(node_set, key=lambda m: (
len(m.assigned.symmetric_difference(assigned)),
len(m.assigned ^ readable)))
node_set.remove(mapped)
yield node, mapped.assigned
assert not node_set
changed_list = []
uptodate_set = set()
remove_dict = defaultdict(list)
for node, mapped in map_nodes():
uuid = node.getUUID()
assigned = assigned_dict[node]
for offset, cell in assigned.iteritems():
if offset in mapped:
if cell.isReadable():
uptodate_set.add(offset)
readable_list[offset].remove(cell)
if cell.isFeeding():
self.count_dict[node] += 1
state = CellStates.UP_TO_DATE
cell.setState(state)
changed_list.append((offset, uuid, state))
cell_dict = node_list.get(cell.getNode())
if cell_dict is None:
drop_list[offset].append(cell)
else:
if not cell.isFeeding():
self.count_dict[node] -= 1
remove_dict[offset].append(cell)
for offset in mapped.difference(assigned):
self.count_dict[node] += 1
state = CellStates.OUT_OF_DATE
self.partition_list[offset].append(Cell(node, state))
changed_list.append((offset, uuid, state))
count_dict = self.count_dict.copy()
for offset, cell_list in remove_dict.iteritems():
cell_dict[offset] = cell
# The sort by node id is cosmetic, to prefer result like the first one
# in __doc__.
node_list = sorted(node_list.iteritems(), key=lambda x: x[0].getUUID())
# Generate an optimal PT.
node_count = len(node_list)
repeats = min(self.nr + 1, node_count)
x = [[] for _ in xrange(node_count)]
i = 0
for offset in xrange(self.np):
for _ in xrange(repeats):
x[i % node_count].append(offset)
i += 1
option_dict = Counter(map(tuple, x))
# Strategies to find the "best" permutation of nodes.
def node_options():
# The second part of the key goes with the above cosmetic sort.
option_list = sorted(option_dict, key=lambda x: (-len(x), x))
# 1. Search for solution that does not cause extra replication.
# This is important because tweak() must does nothing if it's
# called a second time whereas the list of nodes hasn't changed.
result = []
for i, (_, cell_dict) in enumerate(node_list):
option = {offset for offset, cell in cell_dict.iteritems()
if not cell.isFeeding()}
x = filter(option.issubset, option_list)
if not x:
break
result.append((i, x))
else:
yield result
# 2. We have to move cells. Evaluating all options would have
# a complexity of O(node_count!), which is clearly too slow,
# so we use a heuristic.
# For each node, we compare the resulting amount of replication
# in the best (min_cost) and worst (max_cost) case, and we first
# iterate over nodes with the biggest difference. This minimizes
# the impact of bad allocation patterns for the last nodes.
result = []
np_complement = frozenset(xrange(self.np)).difference
for i, (_, cell_dict) in enumerate(node_list):
cost_list = []
for x, option in enumerate(option_list):
discard = [0, 0]
for offset in np_complement(option):
cell = cell_dict.get(offset)
if cell:
discard[cell.isReadable()] += 1
cost_list.append(((discard[1], discard[0]), x))
cost_list.sort()
min_cost = cost_list[0][0]
max_cost = cost_list[-1][0]
result.append((
min_cost[0] - max_cost[0],
min_cost[1] - max_cost[1],
i, [option_list[x[1]] for x in cost_list]))
result.sort()
yield result
# The main loop, which is where we evaluate options.
new = [] # the solution
stack = [] # data recursion
def options():
return iter(node_options[len(new)][-1])
for node_options in node_options(): # for each strategy
iter_option = options()
while 1:
try:
option = next(iter_option)
except StopIteration: # 1st strategy only
if new:
iter_option = stack.pop()
option_dict[new.pop()] += 1
continue
break
if option_dict[option]:
new.append(option)
if len(new) == len(node_list):
break
stack.append(iter_option)
iter_option = options()
option_dict[option] -= 1
if new:
break
else:
raise AssertionError
# Apply the solution.
if self._id is None:
self._id = 1
self.num_filled_rows = self.np
new_state = CellStates.UP_TO_DATE
else:
new_state = CellStates.OUT_OF_DATE
changed_list = []
outdated_list = [repeats] * self.np
discard_list = defaultdict(list)
for i, offset_list in enumerate(new):
node, cell_dict = node_list[node_options[i][-2]]
for offset in offset_list:
cell = cell_dict.pop(offset, None)
if cell is None:
self.count_dict[node] += 1
self.partition_list[offset].append(Cell(node, new_state))
changed_list.append((offset, node.getUUID(), new_state))
elif cell.isReadable():
if cell.isFeeding():
cell.setState(CellStates.UP_TO_DATE)
changed_list.append((offset, node.getUUID(),
CellStates.UP_TO_DATE))
outdated_list[offset] -= 1
for offset, cell in cell_dict.iteritems():
discard_list[offset].append(cell)
for offset, drop_list in drop_list.iteritems():
discard_list[offset] += drop_list
# We have sorted cells to discard in order to first deallocate nodes
# in drop_list, and have feeding cells in other nodes.
# The following loop also makes sure not to discard cells too quickly,
# by keeping a minimum of 'repeats' readable cells.
for offset, outdated in enumerate(outdated_list):
row = self.partition_list[offset]
feeding = None if offset in uptodate_set else min(
readable_list[offset], key=lambda x: count_dict[x.getNode()])
for cell in cell_list:
if cell is feeding:
count_dict[cell.getNode()] += 1
for cell in discard_list[offset]:
if outdated and cell.isReadable():
outdated -= 1
if cell.isFeeding():
continue
state = CellStates.FEEDING
cell.setState(state)
else:
self.count_dict[cell.getNode()] -= 1
state = CellStates.DISCARDED
row.remove(cell)
changed_list.append((offset, cell.getUUID(), state))
assert self.num_filled_rows == len(filter(None, self.partition_list))
assert self.operational(), changed_list
return changed_list
def outdate(self, lost_node=None):
......
......@@ -190,6 +190,11 @@ class NeoTestBase(unittest.TestCase):
"Mock objects can't be compared with '==' or '!='"
return super(NeoTestBase, self).assertEqual(first, second, msg=msg)
def assertPartitionTable(self, pt, expected, key=None):
self.assertEqual(
expected if isinstance(expected, str) else '|'.join(expected),
'|'.join(pt._formatRows(sorted(pt.count_dict, key=key))))
class NeoUnitTestBase(NeoTestBase):
""" Base class for neo tests, implements common checks """
......
......@@ -14,9 +14,10 @@
# You should have received a copy of the GNU General Public License
# along with this program. If not, see <http://www.gnu.org/licenses/>.
import unittest
import random, time, unittest
from collections import defaultdict
from .. import NeoUnitTestBase
from neo.lib import logging
from neo.lib.protocol import NodeStates, CellStates
from neo.lib.pt import PartitionTableException
from neo.master.pt import PartitionTable
......@@ -45,7 +46,7 @@ class MasterPartitionTableTests(NeoUnitTestBase):
self.assertEqual(len(pt.getRow(x)), 0)
self.assertFalse(pt.operational())
self.assertFalse(pt.filled())
self.assertRaises(RuntimeError, pt.make, [])
self.assertRaises(AssertionError, pt.make, [])
self.assertFalse(pt.operational())
self.assertFalse(pt.filled())
......@@ -132,77 +133,35 @@ class MasterPartitionTableTests(NeoUnitTestBase):
(1, 2, CellStates.DISCARDED),
(2, 2, CellStates.DISCARDED)])
pt._setCell(0, sn[0], CellStates.UP_TO_DATE)
self.assertEqual(self.tweak(pt), [(2, 3, CellStates.FEEDING)])
def test_16_make(self):
num_partitions = 5
num_replicas = 1
pt = PartitionTable(num_partitions, num_replicas)
# add nodes
uuid1 = self.getStorageUUID()
server1 = ("127.0.0.1", 19001)
sn1 = self.createStorage(server1, uuid1, NodeStates.RUNNING)
# add not running node
uuid2 = self.getStorageUUID()
server2 = ("127.0.0.2", 19001)
sn2 = self.createStorage(server2, uuid2)
sn2.setState(NodeStates.DOWN)
# add node without uuid
server3 = ("127.0.0.3", 19001)
sn3 = self.createStorage(server3, None, NodeStates.RUNNING)
# add clear node
uuid4 = self.getStorageUUID()
server4 = ("127.0.0.4", 19001)
sn4 = self.createStorage(server4, uuid4, NodeStates.RUNNING)
uuid5 = self.getStorageUUID()
server5 = ("127.0.0.5", 1900)
sn5 = self.createStorage(server5, uuid5, NodeStates.RUNNING)
# make the table
pt.make([sn1, sn2, sn3, sn4, sn5])
# check it's ok, only running nodes and node with uuid
# must be present
for x in xrange(num_partitions):
cells = pt.getCellList(x)
self.assertEqual(len(cells), 2)
nodes = [x.getNode() for x in cells]
for node in nodes:
self.assertTrue(node in (sn1, sn4, sn5))
self.assertTrue(node not in (sn2, sn3))
self.assertTrue(pt.filled())
self.assertTrue(pt.operational())
# create a pt with less nodes
pt.clear()
self.assertFalse(pt.filled())
self.assertFalse(pt.operational())
pt.make([sn1])
# check it's ok
for x in xrange(num_partitions):
cells = pt.getCellList(x)
self.assertEqual(len(cells), 1)
nodes = [x.getNode() for x in cells]
for node in nodes:
self.assertEqual(node, sn1)
self.assertTrue(pt.filled())
self.assertTrue(pt.operational())
def _pt_states(self, pt):
node_dict = defaultdict(list)
for offset, row in enumerate(pt.partition_list):
for cell in row:
state_list = node_dict[cell.getNode()]
if state_list:
self.assertTrue(state_list[-1][0] < offset)
state_list.append((offset, str(cell.getState())[0]))
return map(dict, sorted(node_dict.itervalues()))
def checkPT(self, pt, exclude_empty=False):
new_pt = PartitionTable(pt.np, pt.nr)
new_pt.make(node for node, count in pt.count_dict.iteritems()
if count or not exclude_empty)
self.assertEqual(self._pt_states(pt), self._pt_states(new_pt))
node_list = [self.createStorage(
("127.0.0.1", 19000 + i), self.getStorageUUID(),
NodeStates.RUNNING)
for i in xrange(4)]
for np, nr, expected in (
(3, 0, 'U..|.U.|..U'),
(5, 1, 'UU..|..UU|UU..|..UU|UU..'),
(9, 2, 'UUU.|UU.U|U.UU|.UUU|UUU.|UU.U|U.UU|.UUU|UUU.'),
):
pt = PartitionTable(np, nr)
pt.make(node_list)
self.assertPartitionTable(pt, expected)
self.assertTrue(pt.filled())
self.assertTrue(pt.operational())
# create a pt with less nodes
pt.clear()
self.assertFalse(pt.filled())
self.assertFalse(pt.operational())
pt.make(node_list[:1])
self.assertPartitionTable(pt, '|'.join('U' * np))
self.assertTrue(pt.filled())
self.assertTrue(pt.operational())
def update(self, pt, change_list=None):
offset_list = range(pt.np)
offset_list = xrange(pt.np)
for node in pt.count_dict:
pt.updatable(node.getUUID(), offset_list)
if change_list is None:
......@@ -215,9 +174,11 @@ class MasterPartitionTableTests(NeoUnitTestBase):
for offset, uuid, state in change_list:
if state is CellStates.OUT_OF_DATE:
pt.setUpToDate(node_dict[uuid], offset)
pt.log()
def tweak(self, pt, drop_list=()):
change_list = pt.tweak(drop_list)
pt.log()
self.assertFalse(pt.tweak(drop_list))
return change_list
......@@ -225,6 +186,7 @@ class MasterPartitionTableTests(NeoUnitTestBase):
sn = [self.createStorage(None, i + 1, NodeStates.RUNNING)
for i in xrange(5)]
pt = PartitionTable(5, 2)
pt.setID(1)
# part 0
pt._setCell(0, sn[0], CellStates.DISCARDED)
pt._setCell(0, sn[1], CellStates.UP_TO_DATE)
......@@ -246,45 +208,108 @@ class MasterPartitionTableTests(NeoUnitTestBase):
pt._setCell(4, sn[4], CellStates.UP_TO_DATE)
count_dict = defaultdict(int)
self.assertPartitionTable(pt, (
'.U...',
'FFO..',
'FUU..',
'UUUU.',
'U...U'))
change_list = self.tweak(pt)
self.assertPartitionTable(pt, (
'.UO.O',
'UU.O.',
'UFU.O',
'.UUU.',
'U..OU'))
for offset, uuid, state in change_list:
count_dict[state] += 1
self.assertEqual(count_dict, {CellStates.DISCARDED: 3,
self.assertEqual(count_dict, {CellStates.DISCARDED: 2,
CellStates.FEEDING: 1,
CellStates.OUT_OF_DATE: 5,
CellStates.UP_TO_DATE: 3})
self.update(pt, change_list)
self.checkPT(pt)
self.update(pt)
self.assertPartitionTable(pt, (
'.UU.U',
'UU.U.',
'U.U.U',
'.UUU.',
'U..UU'))
self.assertRaises(PartitionTableException, pt.dropNodeList, sn[1:4])
self.assertEqual(6, len(pt.dropNodeList(sn[1:3], True)))
self.assertEqual(3, len(pt.dropNodeList([sn[1]])))
pt.addNodeList([sn[1]])
self.assertPartitionTable(pt, (
'..U.U',
'U..U.',
'U.U.U',
'..UU.',
'U..UU'))
change_list = self.tweak(pt)
self.assertPartitionTable(pt, (
'.OU.U',
'UO.U.',
'U.U.U',
'.OUU.',
'U..UU'))
self.assertEqual(3, len(change_list))
self.update(pt, change_list)
self.checkPT(pt)
for np, i in (12, 0), (12, 1), (13, 2):
for np, i, expected in (
(12, 0, ('U...|.U..|..U.|...U|'
'U...|.U..|..U.|...U|'
'U...|.U..|..U.|...U',)),
(12, 1, ('UU...|..UU.|U...U|.UU..|...UU|'
'UU...|..UU.|U...U|.UU..|...UU|'
'UU...|..UU.',)),
(13, 2, ('U.UU.|.U.UU|UUU..|..UUU|UU..U|'
'U.UU.|.U.UU|UUU..|..UUU|UU..U|'
'U.UU.|.U.UU|UUU..',
'UUU..|U..UU|.UUU.|UU..U|..UUU|'
'UUU..|U..UU|.UUU.|UU..U|..UUU|'
'UUU..|U..UU|.UUU.')),
):
pt = PartitionTable(np, i)
i += 1
pt.make(sn[:i])
pt.log()
for n in sn[i:i+3]:
self.assertEqual([n], pt.addNodeList([n]))
self.update(pt, self.tweak(pt))
self.checkPT(pt)
self.assertPartitionTable(pt, expected[0])
pt.clear()
pt.make(sn[:i])
for n in sn[i:i+3]:
self.assertEqual([n], pt.addNodeList([n]))
self.tweak(pt)
self.update(pt)
self.checkPT(pt)
self.assertPartitionTable(pt, expected[-1])
pt = PartitionTable(7, 0)
pt.make(sn[:1])
pt.addNodeList(sn[1:3])
self.assertPartitionTable(pt, 'U..|U..|U..|U..|U..|U..|U..')
self.update(pt, self.tweak(pt, sn[:1]))
self.checkPT(pt, True)
self.assertPartitionTable(pt, '.U.|..U|.U.|..U|.U.|..U|.U.')
def test_18_tweak(self):
s = repr(time.time())
logging.info("using seed %r", s)
r = random.Random(s)
sn_count = 11
sn = [self.createStorage(None, i + 1, NodeStates.RUNNING)
for i in xrange(sn_count)]
pt = PartitionTable(1000, 2)
pt.setID(1)
for offset in xrange(pt.np):
state = CellStates.UP_TO_DATE
k = r.randrange(1, sn_count)
for s in r.sample(sn, k):
pt._setCell(offset, s, state)
if k * r.random() < 1:
state = CellStates.OUT_OF_DATE
pt.log()
self.tweak(pt)
self.update(pt)
if __name__ == '__main__':
......
......@@ -1025,11 +1025,11 @@ class NEOThreadedTest(NeoTestBase):
with Patch(client, _getFinalTID=lambda *_: None):
self.assertRaises(ConnectionClosed, txn.commit)
def assertPartitionTable(self, cluster, stats, pt_node=None):
pt = (pt_node or cluster.admin).pt
def assertPartitionTable(self, cluster, expected, pt_node=None):
index = [x.uuid for x in cluster.storage_list].index
self.assertEqual(stats, '|'.join(pt._formatRows(sorted(
pt.count_dict, key=lambda x: index(x.getUUID())))))
super(NEOThreadedTest, self).assertPartitionTable(
(pt_node or cluster.admin).pt, expected,
lambda x: index(x.getUUID()))
@staticmethod
def noConnection(jar, storage):
......
......@@ -317,13 +317,13 @@ class ReplicationTests(NEOThreadedTest):
s2.start()
self.tic()
cluster.enableStorageList([s2])
# 2 UP_TO_DATE cells should become FEEDING,
# and be dropped only when the replication is done,
# 2 UP_TO_DATE cells become FEEDING:
# they are dropped only when the replication is done,
# so that 1 storage can still die without data loss.
with Patch(s0.dm, changePartitionTable=changePartitionTable):
cluster.neoctl.tweakPartitionTable()
self.tic()
expectedFailure(self.assertEqual)(cluster.neoctl.getClusterState(),
self.assertEqual(cluster.neoctl.getClusterState(),
ClusterStates.RUNNING)
@with_cluster(start_cluster=0, partitions=3, replicas=1, storage_count=3)
......
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