Commit a60e36e8 authored by Julien Muchembled's avatar Julien Muchembled

Check protocol version, on both connection sides, before parsing any packet

This fixes 2 issues:
- Because neoctl connects to admin nodes without requesting identification,
  the protocol version was not checked, which could even be dangerous
  (think of a user asking for information, but the packet sent by neoctl
  could be decoded as a packet to alter data, like Truncate).
- In case of mismatched protocol version, the error was not loggued on the
  node that initiated the connection.

Compatibility is handled as follows:
- For an old node receiving data from a new node, the 2 high order bytes of the
  packet id, which is always 0 for the first packet, is decoded as the packet
  code. Packet 0 has never existed, which results in PacketMalformedError.
- For a new node receiving data from an old node, the id of the first packet,
  which is always 0, is decoded as the version, which results in a version
  mismatch error.

This new protocol also guarantees that there's no conflict with SSL.

For simplification, the packet length does not count the header anymore.
parents 9e54a8e0 da5d7a89
Documentation Documentation
- Clarify node state signification, and consider renaming them in the code.
Ideas:
TEMPORARILY_DOWN becomes UNAVAILABLE
BROKEN is removed ?
- Clarify the use of each error codes: - Clarify the use of each error codes:
- NOT_READY removed (connection kept opened until ready) - NOT_READY removed (connection kept opened until ready)
- Split PROTOCOL_ERROR (BAD IDENTIFICATION, ...) - Split PROTOCOL_ERROR (BAD IDENTIFICATION, ...)
...@@ -25,8 +21,6 @@ ...@@ -25,8 +21,6 @@
This is mainly the case for : This is mainly the case for :
- Client rejected before the cluster is operational - Client rejected before the cluster is operational
- Empty storages rejected during recovery process - Empty storages rejected during recovery process
Masters implies in the election process should still reject any connection
as the primary master is still unknown.
- Implement transaction garbage collection API (FEATURE) - Implement transaction garbage collection API (FEATURE)
NEO packing implementation does not update transaction metadata when NEO packing implementation does not update transaction metadata when
deleting object revisions. This inconsistency must be made possible to deleting object revisions. This inconsistency must be made possible to
...@@ -38,7 +32,7 @@ ...@@ -38,7 +32,7 @@
- Clarify handler methods to call when a connection is accepted from a - Clarify handler methods to call when a connection is accepted from a
listening conenction and when remote node is identified listening conenction and when remote node is identified
(cf. neo/lib/bootstrap.py). (cf. neo/lib/bootstrap.py).
- Review PENDING/HIDDEN/SHUTDOWN states, don't use notifyNodeInformation() - Review PENDING/SHUTDOWN states, don't use notifyNodeInformation()
to do a state-switch, use a exception-based mechanism ? (CODE) to do a state-switch, use a exception-based mechanism ? (CODE)
- Review handler split (CODE) - Review handler split (CODE)
The current handler split is the result of small incremental changes. A The current handler split is the result of small incremental changes. A
......
...@@ -194,17 +194,17 @@ class Application(ThreadedApplication): ...@@ -194,17 +194,17 @@ class Application(ThreadedApplication):
self.nm.reset() self.nm.reset()
if self.primary_master_node is not None: if self.primary_master_node is not None:
# If I know a primary master node, pinpoint it. # If I know a primary master node, pinpoint it.
self.trying_master_node = self.primary_master_node node = self.primary_master_node
self.primary_master_node = None self.primary_master_node = None
else: else:
# Otherwise, check one by one. # Otherwise, check one by one.
master_list = self.nm.getMasterList() master_list = self.nm.getMasterList()
index = (index + 1) % len(master_list) index = (index + 1) % len(master_list)
self.trying_master_node = master_list[index] node = master_list[index]
# Connect to master # Connect to master
conn = MTClientConnection(self, conn = MTClientConnection(self,
self.notifications_handler, self.notifications_handler,
node=self.trying_master_node, node=node,
dispatcher=self.dispatcher) dispatcher=self.dispatcher)
p = Packets.RequestIdentification( p = Packets.RequestIdentification(
NodeTypes.CLIENT, self.uuid, None, self.name, None) NodeTypes.CLIENT, self.uuid, None, self.name, None)
...@@ -212,10 +212,8 @@ class Application(ThreadedApplication): ...@@ -212,10 +212,8 @@ class Application(ThreadedApplication):
ask(conn, p, handler=handler) ask(conn, p, handler=handler)
except ConnectionClosed: except ConnectionClosed:
fail_count += 1 fail_count += 1
continue else:
# If we reached the primary master node, mark as connected self.primary_master_node = node
if self.primary_master_node is not None and \
self.primary_master_node is self.trying_master_node:
break break
else: else:
raise NEOPrimaryMasterLost( raise NEOPrimaryMasterLost(
......
...@@ -15,10 +15,10 @@ ...@@ -15,10 +15,10 @@
# along with this program. If not, see <http://www.gnu.org/licenses/>. # along with this program. If not, see <http://www.gnu.org/licenses/>.
from neo.lib import logging from neo.lib import logging
from neo.lib.exception import PrimaryElected
from neo.lib.handler import MTEventHandler from neo.lib.handler import MTEventHandler
from neo.lib.pt import MTPartitionTable as PartitionTable from neo.lib.pt import MTPartitionTable as PartitionTable
from neo.lib.protocol import NodeStates, ProtocolError from neo.lib.protocol import NodeStates
from neo.lib.util import dump
from . import AnswerBaseHandler from . import AnswerBaseHandler
from ..exception import NEOStorageError from ..exception import NEOStorageError
...@@ -26,10 +26,6 @@ from ..exception import NEOStorageError ...@@ -26,10 +26,6 @@ from ..exception import NEOStorageError
class PrimaryBootstrapHandler(AnswerBaseHandler): class PrimaryBootstrapHandler(AnswerBaseHandler):
""" Bootstrap handler used when looking for the primary master """ """ Bootstrap handler used when looking for the primary master """
def notReady(self, conn, message):
self.app.trying_master_node = None
conn.close()
def answerPartitionTable(self, conn, ptid, row_list): def answerPartitionTable(self, conn, ptid, row_list):
assert row_list assert row_list
self.app.pt.load(ptid, row_list, self.app.nm) self.app.pt.load(ptid, row_list, self.app.nm)
...@@ -40,57 +36,14 @@ class PrimaryBootstrapHandler(AnswerBaseHandler): ...@@ -40,57 +36,14 @@ class PrimaryBootstrapHandler(AnswerBaseHandler):
class PrimaryNotificationsHandler(MTEventHandler): class PrimaryNotificationsHandler(MTEventHandler):
""" Handler that process the notifications from the primary master """ """ Handler that process the notifications from the primary master """
def _acceptIdentification(self, node, uuid, num_partitions, def notPrimaryMaster(self, *args):
num_replicas, your_uuid, primary, known_master_list): try:
app = self.app super(PrimaryNotificationsHandler, self).notPrimaryMaster(*args)
except PrimaryElected, e:
# Register new master nodes. app.primary_master_node, = e.args
found = False
conn_address = node.getAddress()
for node_address, node_uuid in known_master_list:
if node_address == conn_address:
assert uuid == node_uuid, (dump(uuid), dump(node_uuid))
found = True
n = app.nm.getByAddress(node_address)
if n is None:
n = app.nm.createMaster(address=node_address)
if node_uuid is not None and n.getUUID() != node_uuid:
n.setUUID(node_uuid)
assert found, (node, dump(uuid), known_master_list)
conn = node.getConnection()
if primary is not None:
primary_node = app.nm.getByAddress(primary)
if primary_node is None:
# I don't know such a node. Probably this information
# is old. So ignore it.
logging.warning('Unknown primary master: %s. Ignoring.',
primary)
return
else:
if app.trying_master_node is not primary_node:
app.trying_master_node = None
conn.close()
app.primary_master_node = primary_node
else:
if app.primary_master_node is not None:
# The primary master node is not a primary master node
# any longer.
app.primary_master_node = None
app.trying_master_node = None
conn.close()
return
# the master must give an UUID
if your_uuid is None:
raise ProtocolError('No UUID supplied')
app.uuid = your_uuid
logging.info('Got an UUID: %s', dump(app.uuid))
app.id_timestamp = None
# Always create partition table def _acceptIdentification(self, node, num_partitions, num_replicas):
app.pt = PartitionTable(num_partitions, num_replicas) self.app.pt = PartitionTable(num_partitions, num_replicas)
def answerLastTransaction(self, conn, ltid): def answerLastTransaction(self, conn, ltid):
app = self.app app = self.app
...@@ -189,7 +142,7 @@ class PrimaryNotificationsHandler(MTEventHandler): ...@@ -189,7 +142,7 @@ class PrimaryNotificationsHandler(MTEventHandler):
def notifyNodeInformation(self, conn, timestamp, node_list): def notifyNodeInformation(self, conn, timestamp, node_list):
super(PrimaryNotificationsHandler, self).notifyNodeInformation( super(PrimaryNotificationsHandler, self).notifyNodeInformation(
conn, timestamp, node_list) 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 ? # to do the same thing for nodes in other non-running states ?
getByUUID = self.app.nm.getByUUID getByUUID = self.app.nm.getByUUID
for node in node_list: for node in node_list:
......
...@@ -42,13 +42,8 @@ class StorageEventHandler(MTEventHandler): ...@@ -42,13 +42,8 @@ class StorageEventHandler(MTEventHandler):
self.app.cp.removeConnection(node) self.app.cp.removeConnection(node)
super(StorageEventHandler, self).connectionFailed(conn) super(StorageEventHandler, self).connectionFailed(conn)
def _acceptIdentification(self, node, def _acceptIdentification(*args):
uuid, num_partitions, num_replicas, your_uuid, primary, pass
master_list):
assert self.app.master_conn is None or \
primary == self.app.master_conn.getAddress(), (
primary, self.app.master_conn)
assert uuid == node.getUUID(), (uuid, node.getUUID())
class StorageBootstrapHandler(AnswerBaseHandler): class StorageBootstrapHandler(AnswerBaseHandler):
""" Handler used when connecting to a storage node """ """ Handler used when connecting to a storage node """
......
...@@ -99,7 +99,6 @@ class ConnectionPool(object): ...@@ -99,7 +99,6 @@ class ConnectionPool(object):
return conn return conn
def removeConnection(self, node): def removeConnection(self, node):
"""Explicitly remove connection when a node is broken."""
self.connection_dict.pop(node.getUUID(), None) self.connection_dict.pop(node.getUUID(), None)
def closeAll(self): def closeAll(self):
......
...@@ -15,8 +15,9 @@ ...@@ -15,8 +15,9 @@
# along with this program. If not, see <http://www.gnu.org/licenses/>. # along with this program. If not, see <http://www.gnu.org/licenses/>.
from . import logging from . import logging
from .exception import PrimaryElected
from .handler import EventHandler from .handler import EventHandler
from .protocol import uuid_str, Packets from .protocol import Packets
from .connection import ClientConnection from .connection import ClientConnection
...@@ -24,7 +25,6 @@ class BootstrapManager(EventHandler): ...@@ -24,7 +25,6 @@ class BootstrapManager(EventHandler):
""" """
Manage the bootstrap stage, lookup for the primary master then connect to it Manage the bootstrap stage, lookup for the primary master then connect to it
""" """
accepted = False
def __init__(self, app, node_type, server=None): def __init__(self, app, node_type, server=None):
""" """
...@@ -32,85 +32,30 @@ class BootstrapManager(EventHandler): ...@@ -32,85 +32,30 @@ class BootstrapManager(EventHandler):
primary master node, connect to it then returns when the master node primary master node, connect to it then returns when the master node
is ready. is ready.
""" """
self.primary = None
self.server = server self.server = server
self.node_type = node_type self.node_type = node_type
self.num_replicas = None self.num_replicas = None
self.num_partitions = None self.num_partitions = None
self.current = None
app.nm.reset() app.nm.reset()
uuid = property(lambda self: self.app.uuid) uuid = property(lambda self: self.app.uuid)
def announcePrimary(self, conn):
# We found the primary master early enough to be notified of election
# end. Lucky. Anyway, we must carry on with identification request, so
# nothing to do here.
pass
def connectionCompleted(self, conn): def connectionCompleted(self, conn):
"""
Triggered when the network connection is successful.
Now ask who's the primary.
"""
EventHandler.connectionCompleted(self, conn) EventHandler.connectionCompleted(self, conn)
self.current.setRunning()
conn.ask(Packets.RequestIdentification(self.node_type, self.uuid, conn.ask(Packets.RequestIdentification(self.node_type, self.uuid,
self.server, self.app.name, None)) self.server, self.app.name, None))
def connectionFailed(self, conn): def connectionFailed(self, conn):
"""
Triggered when the network connection failed.
Restart bootstrap.
"""
EventHandler.connectionFailed(self, conn) EventHandler.connectionFailed(self, conn)
self.current = None self.current = None
def connectionLost(self, conn, new_state): def connectionLost(self, conn, new_state):
"""
Triggered when an established network connection is lost.
Restart bootstrap.
"""
self.current.setTemporarilyDown()
self.current = None self.current = None
def notReady(self, conn, message): def _acceptIdentification(self, node, num_partitions, num_replicas):
""" assert self.current is node, (self.current, node)
The primary master send this message when it is still not ready to
handle the client node.
Close connection and restart.
"""
conn.close()
def _acceptIdentification(self, node, uuid, num_partitions,
num_replicas, your_uuid, primary, known_master_list):
nm = self.app.nm
# Register new master nodes.
for address, uuid in known_master_list:
master_node = nm.getByAddress(address)
if master_node is None:
master_node = nm.createMaster(address=address)
master_node.setUUID(uuid)
self.primary = nm.getByAddress(primary)
if self.primary is None or self.current is not self.primary:
# three cases here:
# - something goes wrong (unknown UUID)
# - this master doesn't know who's the primary
# - got the primary's uuid, so cut here
node.getConnection().close()
return
logging.info('connected to a primary master node')
self.num_partitions = num_partitions self.num_partitions = num_partitions
self.num_replicas = num_replicas self.num_replicas = num_replicas
if self.uuid != your_uuid:
# got an uuid from the primary master
self.app.uuid = your_uuid
logging.info('Got a new UUID: %s', uuid_str(self.uuid))
self.app.id_timestamp = None
self.accepted = True
def getPrimaryConnection(self): def getPrimaryConnection(self):
""" """
...@@ -122,25 +67,26 @@ class BootstrapManager(EventHandler): ...@@ -122,25 +67,26 @@ class BootstrapManager(EventHandler):
poll = app.em.poll poll = app.em.poll
index = 0 index = 0
self.current = None self.current = None
conn = None
# retry until identified to the primary # retry until identified to the primary
while not self.accepted: while True:
if self.current is None: try:
# conn closed while self.current:
conn = None if self.current.isIdentified():
return (self.current, self.current.getConnection(),
self.num_partitions, self.num_replicas)
poll(1)
except PrimaryElected, e:
if self.current:
self.current.getConnection().close()
self.current, = e.args
index = app.nm.getMasterList().index(self.current)
else:
# select a master # select a master
master_list = app.nm.getMasterList() master_list = app.nm.getMasterList()
index = (index + 1) % len(master_list) index = (index + 1) % len(master_list)
self.current = master_list[index] self.current = master_list[index]
if conn is None: ClientConnection(app, self, self.current)
# open the connection # Note that the connection may be already closed. This happens when
conn = ClientConnection(app, self, self.current) # the kernel reacts so quickly to a closed port that 'connect'
# Yes, the connection may be already closed. This happens when # fails on the first call. In such case, poll(1) would deadlock
# the kernel reacts so quickly to a closed port that 'connect' # if there's no other connection to timeout.
# fails on the first call. In such case, poll(1) would deadlock
# if there's no other connection to timeout.
if conn.isClosed():
continue
# still processing
poll(1)
return self.current, conn, self.num_partitions, self.num_replicas
...@@ -90,9 +90,7 @@ class ConfigurationManager(object): ...@@ -90,9 +90,7 @@ class ConfigurationManager(object):
def getMasters(self): def getMasters(self):
""" Get the master node list except itself """ """ Get the master node list except itself """
masters = self.__get('masters') return util.parseMasterList(self.__get('masters'))
# load master node list except itself
return util.parseMasterList(masters, except_node=self.getBind())
def getBind(self): def getBind(self):
""" Get the address to bind to """ """ Get the address to bind to """
......
...@@ -20,8 +20,7 @@ from time import time ...@@ -20,8 +20,7 @@ from time import time
from . import attributeTracker, logging from . import attributeTracker, logging
from .connector import ConnectorException, ConnectorDelayedConnection from .connector import ConnectorException, ConnectorDelayedConnection
from .locking import RLock from .locking import RLock
from .protocol import uuid_str, Errors, \ from .protocol import uuid_str, Errors, PacketMalformedError, Packets
PacketMalformedError, Packets, ParserState
from .util import dummy_read_buffer, ReadBuffer from .util import dummy_read_buffer, ReadBuffer
CRITICAL_TIMEOUT = 30 CRITICAL_TIMEOUT = 30
...@@ -138,10 +137,9 @@ class HandlerSwitcher(object): ...@@ -138,10 +137,9 @@ class HandlerSwitcher(object):
else: else:
logging.error('Unexpected answer %r in %r', packet, connection) logging.error('Unexpected answer %r in %r', packet, connection)
if not connection.isClosed(): if not connection.isClosed():
connection.send(Packets.Notify( connection.answer(Errors.ProtocolError(
'Unexpected answer: %r' % packet)) 'Unexpected answer: %r' % packet))
connection.abort() connection.abort()
# handler.peerBroken(connection)
finally: finally:
# apply a pending handler if no more answers are pending # apply a pending handler if no more answers are pending
while len(pending) > 1 and not pending[0][0]: while len(pending) > 1 and not pending[0][0]:
...@@ -260,10 +258,12 @@ class BaseConnection(object): ...@@ -260,10 +258,12 @@ class BaseConnection(object):
) )
def setHandler(self, handler): def setHandler(self, handler):
if self._handlers.setHandler(handler): changed = self._handlers.setHandler(handler)
logging.debug('Set handler %r on %r', handler, self) if changed:
logging.debug('Handler changed on %r', self)
else: else:
logging.debug('Delay handler %r on %r', handler, self) logging.debug('Delay handler %r on %r', handler, self)
return changed
def getUUID(self): def getUUID(self):
return None return None
...@@ -317,9 +317,9 @@ class ListeningConnection(BaseConnection): ...@@ -317,9 +317,9 @@ class ListeningConnection(BaseConnection):
if self._ssl: if self._ssl:
conn.connecting = True conn.connecting = True
connector.ssl(self._ssl, conn._connected) connector.ssl(self._ssl, conn._connected)
self.em.addWriter(conn)
else: else:
conn._connected() conn._connected()
self.em.addWriter(conn) # for SSL or ENCODED_VERSION
def getAddress(self): def getAddress(self):
return self.connector.getAddress() return self.connector.getAddress()
...@@ -338,6 +338,7 @@ class Connection(BaseConnection): ...@@ -338,6 +338,7 @@ class Connection(BaseConnection):
server = False server = False
peer_id = None peer_id = None
_next_timeout = None _next_timeout = None
_parser_state = None
_timeout = 0 _timeout = 0
def __init__(self, event_manager, *args, **kw): def __init__(self, event_manager, *args, **kw):
...@@ -348,7 +349,6 @@ class Connection(BaseConnection): ...@@ -348,7 +349,6 @@ class Connection(BaseConnection):
self.uuid = None self.uuid = None
self._queue = [] self._queue = []
self._on_close = None self._on_close = None
self._parser_state = ParserState()
def _getReprInfo(self): def _getReprInfo(self):
r, flags = super(Connection, self)._getReprInfo() r, flags = super(Connection, self)._getReprInfo()
...@@ -465,20 +465,59 @@ class Connection(BaseConnection): ...@@ -465,20 +465,59 @@ class Connection(BaseConnection):
except ConnectorException: except ConnectorException:
self._closure() self._closure()
def _parse(self):
read = self.read_buf.read
version = read(4)
if version is None:
return
from .protocol import (ENCODED_VERSION, MAX_PACKET_SIZE,
PACKET_HEADER_FORMAT, Packets)
if version != ENCODED_VERSION:
logging.warning('Protocol version mismatch with %r', self)
raise ConnectorException
header_size = PACKET_HEADER_FORMAT.size
unpack = PACKET_HEADER_FORMAT.unpack
def parse():
state = self._parser_state
if state is None:
header = read(header_size)
if header is None:
return
msg_id, msg_type, msg_len = unpack(header)
try:
packet_klass = Packets[msg_type]
except KeyError:
raise PacketMalformedError('Unknown packet type')
if msg_len > MAX_PACKET_SIZE:
raise PacketMalformedError('message too big (%d)' % msg_len)
else:
msg_id, packet_klass, msg_len = state
data = read(msg_len)
if data is None:
# Not enough.
if state is None:
self._parser_state = msg_id, packet_klass, msg_len
else:
self._parser_state = None
packet = packet_klass()
packet.setContent(msg_id, data)
return packet
self._parse = parse
return parse()
def readable(self): def readable(self):
"""Called when self is readable.""" """Called when self is readable."""
# last known remote activity # last known remote activity
self._next_timeout = time() + self._timeout self._next_timeout = time() + self._timeout
read_buf = self.read_buf
try: try:
try: try:
if self.connector.receive(read_buf): if self.connector.receive(self.read_buf):
self.em.addWriter(self) self.em.addWriter(self)
finally: finally:
# A connector may read some data # A connector may read some data
# before raising ConnectorException # before raising ConnectorException
while 1: while 1:
packet = Packets.parse(read_buf, self._parser_state) packet = self._parse()
if packet is None: if packet is None:
break break
self._queue.append(packet) self._queue.append(packet)
...@@ -625,9 +664,7 @@ class ClientConnection(Connection): ...@@ -625,9 +664,7 @@ class ClientConnection(Connection):
self.em.register(self) self.em.register(self)
if connected: if connected:
self._maybeConnected() self._maybeConnected()
# A client connection usually has a pending packet to send # There's always the protocol version to send.
# from the beginning. It would be too smart to detect when
# it's not required to poll for writing.
self.em.addWriter(self) self.em.addWriter(self)
def _delayedConnect(self): def _delayedConnect(self):
......
...@@ -19,6 +19,7 @@ import ssl ...@@ -19,6 +19,7 @@ import ssl
import errno import errno
from time import time from time import time
from . import logging from . import logging
from .protocol import ENCODED_VERSION
# Global connector registry. # Global connector registry.
# Fill by calling registerConnectorHandler. # Fill by calling registerConnectorHandler.
...@@ -58,7 +59,7 @@ class SocketConnector(object): ...@@ -58,7 +59,7 @@ class SocketConnector(object):
s.setblocking(0) s.setblocking(0)
# disable Nagle algorithm to reduce latency # disable Nagle algorithm to reduce latency
s.setsockopt(socket.IPPROTO_TCP, socket.TCP_NODELAY, 1) s.setsockopt(socket.IPPROTO_TCP, socket.TCP_NODELAY, 1)
self.queued = [] self.queued = [ENCODED_VERSION]
return self return self
def queue(self, data): def queue(self, data):
......
...@@ -17,7 +17,7 @@ ...@@ -17,7 +17,7 @@
class NeoException(Exception): class NeoException(Exception):
pass pass
class ElectionFailure(NeoException): class PrimaryElected(NeoException):
pass pass
class PrimaryFailure(NeoException): class PrimaryFailure(NeoException):
......
...@@ -19,9 +19,9 @@ from collections import deque ...@@ -19,9 +19,9 @@ from collections import deque
from operator import itemgetter from operator import itemgetter
from . import logging from . import logging
from .connection import ConnectionClosed from .connection import ConnectionClosed
from .protocol import ( from .exception import PrimaryElected
NodeStates, Packets, Errors, BackendNotImplemented, from .protocol import (NodeStates, NodeTypes, Packets, uuid_str,
BrokenNodeDisallowedError, NonReadableCell, NotReadyError, Errors, BackendNotImplemented, NonReadableCell, NotReadyError,
PacketMalformedError, ProtocolError, UnexpectedPacketError) PacketMalformedError, ProtocolError, UnexpectedPacketError)
from .util import cached_property from .util import cached_property
...@@ -59,7 +59,6 @@ class EventHandler(object): ...@@ -59,7 +59,6 @@ class EventHandler(object):
logging.error(message) logging.error(message)
conn.answer(Errors.ProtocolError(message)) conn.answer(Errors.ProtocolError(message))
conn.abort() conn.abort()
# self.peerBroken(conn)
def dispatch(self, conn, packet, kw={}): def dispatch(self, conn, packet, kw={}):
"""This is a helper method to handle various packet types.""" """This is a helper method to handle various packet types."""
...@@ -80,11 +79,6 @@ class EventHandler(object): ...@@ -80,11 +79,6 @@ class EventHandler(object):
except PacketMalformedError, e: except PacketMalformedError, e:
logging.error('malformed packet from %r: %s', conn, e) logging.error('malformed packet from %r: %s', conn, e)
conn.close() conn.close()
# self.peerBroken(conn)
except BrokenNodeDisallowedError:
if not conn.isClosed():
conn.answer(Errors.BrokenNode('go away'))
conn.abort()
except NotReadyError, message: except NotReadyError, message:
if not conn.isClosed(): if not conn.isClosed():
if not message.args: if not message.args:
...@@ -144,12 +138,7 @@ class EventHandler(object): ...@@ -144,12 +138,7 @@ class EventHandler(object):
def connectionClosed(self, conn): def connectionClosed(self, conn):
"""Called when a connection is closed by the peer.""" """Called when a connection is closed by the peer."""
logging.debug('connection closed for %r', conn) logging.debug('connection closed for %r', conn)
self.connectionLost(conn, NodeStates.TEMPORARILY_DOWN) self.connectionLost(conn, NodeStates.DOWN)
#def peerBroken(self, conn):
# """Called when a peer is broken."""
# logging.error('%r is broken', conn)
# # NodeStates.BROKEN
def connectionLost(self, conn, new_state): def connectionLost(self, conn, new_state):
""" this is a method to override in sub-handlers when there is no need """ this is a method to override in sub-handlers when there is no need
...@@ -159,21 +148,41 @@ class EventHandler(object): ...@@ -159,21 +148,41 @@ class EventHandler(object):
# Packet handlers. # Packet handlers.
def acceptIdentification(self, conn, node_type, *args): def notPrimaryMaster(self, conn, primary, known_master_list):
try: nm = self.app.nm
acceptIdentification = self._acceptIdentification for address in known_master_list:
except AttributeError: nm.createMaster(address=address)
raise UnexpectedPacketError('no handler found') if primary is not None:
if conn.isClosed(): primary = known_master_list[primary]
# acceptIdentification received on a closed (probably aborted, assert primary != self.app.server
# actually) connection. Reject any further packet as unexpected. raise PrimaryElected(nm.getByAddress(primary))
conn.setHandler(EventHandler(self.app))
return def _acceptIdentification(*args):
node = self.app.nm.getByAddress(conn.getAddress()) pass
def acceptIdentification(self, conn, node_type, uuid,
num_partitions, num_replicas, your_uuid):
app = self.app
node = app.nm.getByAddress(conn.getAddress())
assert node.getConnection() is conn, (node.getConnection(), conn) assert node.getConnection() is conn, (node.getConnection(), conn)
if node.getType() == node_type: if node.getType() == node_type:
if node_type == NodeTypes.MASTER:
other = app.nm.getByUUID(uuid)
if other is not None:
other.setUUID(None)
node.setUUID(uuid)
node.setRunning()
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.id_timestamp = None
elif node.getUUID() != uuid or app.uuid != your_uuid != None:
raise ProtocolError('invalid uuids')
node.setIdentified() node.setIdentified()
acceptIdentification(node, *args) self._acceptIdentification(node, num_partitions, num_replicas)
return return
conn.close() conn.close()
...@@ -189,9 +198,6 @@ class EventHandler(object): ...@@ -189,9 +198,6 @@ class EventHandler(object):
# to test/maintain underlying connection. # to test/maintain underlying connection.
pass pass
def notify(self, conn, message):
logging.warning('notification from %r: %s', conn, message)
def closeClient(self, conn): def closeClient(self, conn):
conn.server = False conn.server = False
if not conn.client: if not conn.client:
...@@ -216,9 +222,6 @@ class EventHandler(object): ...@@ -216,9 +222,6 @@ class EventHandler(object):
def timeoutError(self, conn, message): def timeoutError(self, conn, message):
logging.error('timeout error: %s', message) logging.error('timeout error: %s', message)
def brokenNodeDisallowedError(self, conn, message):
raise RuntimeError, 'broken node disallowed error: %s' % (message,)
def ack(self, conn, message): def ack(self, conn, message):
logging.debug("no error message: %s", message) logging.debug("no error message: %s", message)
...@@ -268,7 +271,6 @@ class AnswerBaseHandler(EventHandler): ...@@ -268,7 +271,6 @@ class AnswerBaseHandler(EventHandler):
timeoutExpired = unexpectedInAnswerHandler timeoutExpired = unexpectedInAnswerHandler
connectionClosed = unexpectedInAnswerHandler connectionClosed = unexpectedInAnswerHandler
packetReceived = unexpectedInAnswerHandler packetReceived = unexpectedInAnswerHandler
peerBroken = unexpectedInAnswerHandler
protocolError = unexpectedInAnswerHandler protocolError = unexpectedInAnswerHandler
def acceptIdentification(*args): def acceptIdentification(*args):
......
...@@ -31,8 +31,7 @@ class Node(object): ...@@ -31,8 +31,7 @@ class Node(object):
_identified = False _identified = False
id_timestamp = None id_timestamp = None
def __init__(self, manager, address=None, uuid=None, def __init__(self, manager, address=None, uuid=None, state=NodeStates.DOWN):
state=NodeStates.UNKNOWN):
self._state = state self._state = state
self._address = address self._address = address
self._uuid = uuid self._uuid = uuid
...@@ -64,7 +63,7 @@ class Node(object): ...@@ -64,7 +63,7 @@ class Node(object):
def setState(self, new_state): def setState(self, new_state):
if self._state == new_state: if self._state == new_state:
return return
if new_state == NodeStates.DOWN: if new_state == NodeStates.UNKNOWN:
self._manager.remove(self) self._manager.remove(self)
self._state = new_state self._state = new_state
else: else:
...@@ -271,7 +270,7 @@ class NodeManager(EventQueue): ...@@ -271,7 +270,7 @@ class NodeManager(EventQueue):
if node in self._node_set: if node in self._node_set:
logging.warning('adding a known node %r, ignoring', node) logging.warning('adding a known node %r, ignoring', node)
return return
assert not node.isDown(), node assert not node.isUnknown(), node
self._node_set.add(node) self._node_set.add(node)
self._updateAddress(node, None) self._updateAddress(node, None)
self._updateUUID(node, None) self._updateUUID(node, None)
...@@ -321,7 +320,7 @@ class NodeManager(EventQueue): ...@@ -321,7 +320,7 @@ class NodeManager(EventQueue):
set_dict.setdefault(new_key, set()).add(node) set_dict.setdefault(new_key, set()).add(node)
def _updateState(self, node, old_state): 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) self.__updateSet(self._state_dict, old_state, node.getState(), node)
def getList(self, node_filter=None): def getList(self, node_filter=None):
...@@ -414,7 +413,7 @@ class NodeManager(EventQueue): ...@@ -414,7 +413,7 @@ class NodeManager(EventQueue):
def update(self, app, timestamp, node_list): def update(self, app, timestamp, node_list):
assert self._timestamp < timestamp, (self._timestamp, timestamp) assert self._timestamp < timestamp, (self._timestamp, timestamp)
self._timestamp = timestamp self._timestamp = timestamp
node_set = self._node_set.copy() if app.id_timestamp is None else None added_list = [] if app.id_timestamp is None else None
for node_type, addr, uuid, state, id_timestamp in node_list: for node_type, addr, uuid, state, id_timestamp in node_list:
# This should be done here (although klass might not be used in this # This should be done here (although klass might not be used in this
# iteration), as it raises if type is not valid. # iteration), as it raises if type is not valid.
...@@ -423,24 +422,23 @@ class NodeManager(EventQueue): ...@@ -423,24 +422,23 @@ class NodeManager(EventQueue):
# lookup in current table # lookup in current table
node_by_uuid = self.getByUUID(uuid) node_by_uuid = self.getByUUID(uuid)
node_by_addr = self.getByAddress(addr) node_by_addr = self.getByAddress(addr)
node = node_by_uuid or node_by_addr node = node_by_addr or node_by_uuid
log_args = node_type, uuid_str(uuid), addr, state, id_timestamp log_args = node_type, uuid_str(uuid), addr, state, id_timestamp
if node is None: if node is None:
if state == NodeStates.DOWN: assert state != NodeStates.UNKNOWN, (self._node_set,) + log_args
logging.debug('NOT creating node %s %s %s %s %s', *log_args)
continue
node = self._createNode(klass, address=addr, uuid=uuid, node = self._createNode(klass, address=addr, uuid=uuid,
state=state) state=state)
logging.debug('creating node %r', node) logging.debug('creating node %r', node)
else: else:
assert isinstance(node, klass), 'node %r is not ' \ assert isinstance(node, klass), 'node %r is not ' \
'of expected type: %r' % (node, klass) 'of expected type: %r' % (node, klass)
assert None in (node_by_uuid, node_by_addr) or \ if None is not node_by_uuid is not node_by_addr is not None:
node_by_uuid is node_by_addr, \ assert added_list is not None, \
'Discrepancy between node_by_uuid (%r) and ' \ 'Discrepancy between node_by_uuid (%r) and ' \
'node_by_addr (%r)' % (node_by_uuid, node_by_addr) 'node_by_addr (%r)' % (node_by_uuid, node_by_addr)
if state == NodeStates.DOWN: node_by_uuid.setUUID(None)
if state == NodeStates.UNKNOWN:
logging.debug('dropping node %r (%r), found with %s ' logging.debug('dropping node %r (%r), found with %s '
'%s %s %s %s', node, node.isConnected(), *log_args) '%s %s %s %s', node, node.isConnected(), *log_args)
if node.isConnected(): if node.isConnected():
...@@ -451,8 +449,9 @@ class NodeManager(EventQueue): ...@@ -451,8 +449,9 @@ class NodeManager(EventQueue):
# reconnect to the master because they cleared their # reconnect to the master because they cleared their
# partition table upon disconnection. # partition table upon disconnection.
node.getConnection().close() node.getConnection().close()
if app.uuid != uuid: if app.uuid != uuid: # XXX
app.pt.dropNode(node) dropped = app.pt.dropNode(node)
assert dropped, node
self.remove(node) self.remove(node)
continue continue
logging.debug('updating node %r to %s %s %s %s %s', logging.debug('updating node %r to %s %s %s %s %s',
...@@ -463,12 +462,15 @@ class NodeManager(EventQueue): ...@@ -463,12 +462,15 @@ class NodeManager(EventQueue):
node.id_timestamp = id_timestamp node.id_timestamp = id_timestamp
if app.uuid == uuid: if app.uuid == uuid:
app.id_timestamp = id_timestamp app.id_timestamp = id_timestamp
if node_set: if added_list is not None:
added_list.append(node)
if added_list is not None:
assert app.id_timestamp is not None
# For the first notification, we receive a full list of nodes from # For the first notification, we receive a full list of nodes from
# the master. Remove all unknown nodes from a previous connection. # the master. Remove all unknown nodes from a previous connection.
for node in node_set - self._node_set: for node in self._node_set.difference(added_list):
app.pt.dropNode(node) if app.pt.dropNode(node):
self.remove(node) self.remove(node)
self.log() self.log()
self.executeQueuedEvents() self.executeQueuedEvents()
......
...@@ -14,24 +14,21 @@ ...@@ -14,24 +14,21 @@
# You should have received a copy of the GNU General Public License # You should have received a copy of the GNU General Public License
# along with this program. If not, see <http://www.gnu.org/licenses/>. # along with this program. If not, see <http://www.gnu.org/licenses/>.
import socket
import sys import sys
import traceback import traceback
from cStringIO import StringIO from cStringIO import StringIO
from struct import Struct from struct import Struct
PROTOCOL_VERSION = 12 # 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 = 1
ENCODED_VERSION = Struct('!L').pack(PROTOCOL_VERSION)
# Size restrictions. # Avoid memory errors on corrupted data.
MIN_PACKET_SIZE = 10
MAX_PACKET_SIZE = 0x4000000 MAX_PACKET_SIZE = 0x4000000
PACKET_HEADER_FORMAT = Struct('!LHL') PACKET_HEADER_FORMAT = Struct('!LHL')
# Check that header size is the expected value.
# If it is not, it means that struct module result is incompatible with
# "reference" platform (python 2.4 on x86-64).
assert PACKET_HEADER_FORMAT.size == 10, \
'Unsupported platform, packet header length = %i' % \
(PACKET_HEADER_FORMAT.size, )
RESPONSE_MASK = 0x8000 RESPONSE_MASK = 0x8000
class Enum(tuple): class Enum(tuple):
...@@ -70,7 +67,6 @@ def ErrorCodes(): ...@@ -70,7 +67,6 @@ def ErrorCodes():
TID_NOT_FOUND TID_NOT_FOUND
OID_DOES_NOT_EXIST OID_DOES_NOT_EXIST
PROTOCOL_ERROR PROTOCOL_ERROR
BROKEN_NODE
REPLICATION_ERROR REPLICATION_ERROR
CHECKING_ERROR CHECKING_ERROR
BACKEND_NOT_IMPLEMENTED BACKEND_NOT_IMPLEMENTED
...@@ -120,13 +116,10 @@ def NodeTypes(): ...@@ -120,13 +116,10 @@ def NodeTypes():
@Enum @Enum
def NodeStates(): def NodeStates():
RUNNING UNKNOWN
TEMPORARILY_DOWN
DOWN DOWN
BROKEN RUNNING
HIDDEN
PENDING PENDING
UNKNOWN
@Enum @Enum
def CellStates(): def CellStates():
...@@ -150,12 +143,9 @@ def CellStates(): ...@@ -150,12 +143,9 @@ def CellStates():
# used for logging # used for logging
node_state_prefix_dict = { node_state_prefix_dict = {
NodeStates.RUNNING: 'R', NodeStates.RUNNING: 'R',
NodeStates.TEMPORARILY_DOWN: 'T',
NodeStates.DOWN: 'D', NodeStates.DOWN: 'D',
NodeStates.BROKEN: 'B',
NodeStates.HIDDEN: 'H',
NodeStates.PENDING: 'P',
NodeStates.UNKNOWN: 'U', NodeStates.UNKNOWN: 'U',
NodeStates.PENDING: 'P',
} }
# used for logging # used for logging
...@@ -168,16 +158,12 @@ cell_state_prefix_dict = { ...@@ -168,16 +158,12 @@ cell_state_prefix_dict = {
} }
# Other constants. # Other constants.
INVALID_UUID = 0 INVALID_TID = \
INVALID_TID = '\xff' * 8
INVALID_OID = '\xff' * 8 INVALID_OID = '\xff' * 8
INVALID_PARTITION = 0xffffffff INVALID_PARTITION = 0xffffffff
INVALID_ADDRESS_TYPE = socket.AF_UNSPEC
ZERO_HASH = '\0' * 20 ZERO_HASH = '\0' * 20
ZERO_TID = '\0' * 8 ZERO_TID = \
ZERO_OID = '\0' * 8 ZERO_OID = '\0' * 8
OID_LEN = len(INVALID_OID)
TID_LEN = len(INVALID_TID)
MAX_TID = '\x7f' + '\xff' * 7 # SQLite does not accept numbers above 2^63-1 MAX_TID = '\x7f' + '\xff' * 7 # SQLite does not accept numbers above 2^63-1
# High-order byte: # High-order byte:
...@@ -203,17 +189,14 @@ class ProtocolError(Exception): ...@@ -203,17 +189,14 @@ class ProtocolError(Exception):
""" Base class for protocol errors, close the connection """ """ Base class for protocol errors, close the connection """
class PacketMalformedError(ProtocolError): class PacketMalformedError(ProtocolError):
""" Close the connection and set the node as broken""" """Close the connection"""
class UnexpectedPacketError(ProtocolError): class UnexpectedPacketError(ProtocolError):
""" Close the connection and set the node as broken""" """Close the connection"""
class NotReadyError(ProtocolError): class NotReadyError(ProtocolError):
""" Just close the connection """ """ Just close the connection """
class BrokenNodeDisallowedError(ProtocolError):
""" Just close the connection """
class BackendNotImplemented(Exception): class BackendNotImplemented(Exception):
""" Method not implemented by backend storage """ """ Method not implemented by backend storage """
...@@ -279,8 +262,8 @@ class Packet(object): ...@@ -279,8 +262,8 @@ class Packet(object):
def encode(self): def encode(self):
""" Encode a packet as a string to send it over the network """ """ Encode a packet as a string to send it over the network """
content = self._body content = self._body
length = PACKET_HEADER_FORMAT.size + len(content) return (PACKET_HEADER_FORMAT.pack(self._id, self._code, len(content)),
return (PACKET_HEADER_FORMAT.pack(self._id, self._code, length), content) content)
def __len__(self): def __len__(self):
return PACKET_HEADER_FORMAT.size + len(self._body) return PACKET_HEADER_FORMAT.size + len(self._body)
...@@ -562,19 +545,6 @@ class PPTID(PStructItemOrNone): ...@@ -562,19 +545,6 @@ class PPTID(PStructItemOrNone):
_fmt = '!Q' _fmt = '!Q'
_None = Struct(_fmt).pack(0) _None = Struct(_fmt).pack(0)
class PProtocol(PNumber):
"""
The protocol version definition
"""
def _encode(self, writer, version):
writer(self.pack(version))
def _decode(self, reader):
version = self.unpack(reader(self.size))
if version != (PROTOCOL_VERSION,):
raise ProtocolError('protocol version mismatch')
return version
class PChecksum(PItem): class PChecksum(PItem):
""" """
A hash (SHA1) A hash (SHA1)
...@@ -586,12 +556,14 @@ class PChecksum(PItem): ...@@ -586,12 +556,14 @@ class PChecksum(PItem):
def _decode(self, reader): def _decode(self, reader):
return reader(20) return reader(20)
class PUUID(PStructItemOrNone): class PSignedNull(PStructItemOrNone):
_fmt = '!l'
_None = Struct(_fmt).pack(0)
class PUUID(PSignedNull):
""" """
An UUID (node identifier, 4-bytes signed integer) An UUID (node identifier, 4-bytes signed integer)
""" """
_fmt = '!l'
_None = Struct(_fmt).pack(0)
class PTID(PItem): class PTID(PItem):
""" """
...@@ -671,14 +643,6 @@ PFOidList = PList('oid_list', ...@@ -671,14 +643,6 @@ PFOidList = PList('oid_list',
# packets definition # packets definition
class Notify(Packet):
"""
General purpose notification (remote logging)
"""
_fmt = PStruct('notify',
PString('message'),
)
class Error(Packet): class Error(Packet):
""" """
Error is a special type of message, because this can be sent against Error is a special type of message, because this can be sent against
...@@ -709,7 +673,6 @@ class RequestIdentification(Packet): ...@@ -709,7 +673,6 @@ class RequestIdentification(Packet):
poll_thread = True poll_thread = True
_fmt = PStruct('request_identification', _fmt = PStruct('request_identification',
PProtocol('protocol_version'),
PFNodeType, PFNodeType,
PUUID('uuid'), PUUID('uuid'),
PAddress('address'), PAddress('address'),
...@@ -723,25 +686,8 @@ class RequestIdentification(Packet): ...@@ -723,25 +686,8 @@ class RequestIdentification(Packet):
PNumber('num_partitions'), PNumber('num_partitions'),
PNumber('num_replicas'), PNumber('num_replicas'),
PUUID('your_uuid'), PUUID('your_uuid'),
PAddress('primary'),
PList('known_master_list',
PStruct('master',
PAddress('address'),
PUUID('uuid'),
),
),
) )
def __init__(self, *args, **kw):
if args or kw:
# always announce current protocol version
args = list(args)
args.insert(0, PROTOCOL_VERSION)
super(RequestIdentification, self).__init__(*args, **kw)
def decode(self):
return super(RequestIdentification, self).decode()[1:]
class PrimaryMaster(Packet): class PrimaryMaster(Packet):
""" """
Ask current primary master's uuid. CTL -> A. Ask current primary master's uuid. CTL -> A.
...@@ -750,15 +696,16 @@ class PrimaryMaster(Packet): ...@@ -750,15 +696,16 @@ class PrimaryMaster(Packet):
PUUID('primary_uuid'), PUUID('primary_uuid'),
) )
class AnnouncePrimary(Packet): class NotPrimaryMaster(Packet):
"""
Announce a primary master node election. PM -> SM.
"""
class ReelectPrimary(Packet):
""" """
Force a re-election of a primary master node. M -> M. Send list of known master nodes. SM -> Any.
""" """
_fmt = PStruct('not_primary_master',
PSignedNull('primary'),
PList('known_master_list',
PAddress('address'),
),
)
class Recovery(Packet): class Recovery(Packet):
""" """
...@@ -1620,22 +1567,6 @@ def register(request, ignore_when_closed=None): ...@@ -1620,22 +1567,6 @@ def register(request, ignore_when_closed=None):
StaticRegistry[code] = answer StaticRegistry[code] = answer
return (request, answer) return (request, answer)
class ParserState(object):
"""
Parser internal state.
To be considered opaque datatype outside of PacketRegistry.parse .
"""
payload = None
def set(self, payload):
self.payload = payload
def get(self):
return self.payload
def clear(self):
self.payload = None
class Packets(dict): class Packets(dict):
""" """
Packet registry that checks packet code uniqueness and provides an index Packet registry that checks packet code uniqueness and provides an index
...@@ -1647,58 +1578,19 @@ class Packets(dict): ...@@ -1647,58 +1578,19 @@ class Packets(dict):
# this builds a "singleton" # this builds a "singleton"
return type('PacketRegistry', base, d)(StaticRegistry) return type('PacketRegistry', base, d)(StaticRegistry)
def parse(self, buf, state_container):
state = state_container.get()
if state is None:
header = buf.read(PACKET_HEADER_FORMAT.size)
if header is None:
return None
msg_id, msg_type, msg_len = PACKET_HEADER_FORMAT.unpack(header)
try:
packet_klass = self[msg_type]
except KeyError:
raise PacketMalformedError('Unknown packet type')
if msg_len > MAX_PACKET_SIZE:
raise PacketMalformedError('message too big (%d)' % msg_len)
if msg_len < MIN_PACKET_SIZE:
raise PacketMalformedError('message too small (%d)' % msg_len)
msg_len -= PACKET_HEADER_FORMAT.size
else:
msg_id, packet_klass, msg_len = state
data = buf.read(msg_len)
if data is None:
# Not enough.
if state is None:
state_container.set((msg_id, packet_klass, msg_len))
return None
if state:
state_container.clear()
packet = packet_klass()
packet.setContent(msg_id, data)
return packet
# notifications # notifications
Error = register( Error = register(
Error) Error)
RequestIdentification, AcceptIdentification = register( RequestIdentification, AcceptIdentification = register(
RequestIdentification) RequestIdentification, ignore_when_closed=True)
# Code of RequestIdentification packet must never change so that 2
# incompatible nodes can reject themselves gracefully (i.e. comparing
# protocol versions) instead of raising PacketMalformedError.
assert RequestIdentification._code == 1
Ping, Pong = register( Ping, Pong = register(
Ping) Ping)
CloseClient = register( CloseClient = register(
CloseClient) CloseClient)
Notify = register(
Notify)
AskPrimary, AnswerPrimary = register( AskPrimary, AnswerPrimary = register(
PrimaryMaster) PrimaryMaster)
AnnouncePrimary = register( NotPrimaryMaster = register(
AnnouncePrimary) NotPrimaryMaster)
ReelectPrimary = register(
ReelectPrimary)
NotifyNodeInformation = register( NotifyNodeInformation = register(
NotifyNodeInformation) NotifyNodeInformation)
AskRecovery, AnswerRecovery = register( AskRecovery, AnswerRecovery = register(
......
...@@ -168,7 +168,7 @@ class PartitionTable(object): ...@@ -168,7 +168,7 @@ class PartitionTable(object):
def _setCell(self, offset, node, state): def _setCell(self, offset, node, state):
if state == CellStates.DISCARDED: if state == CellStates.DISCARDED:
return self.removeCell(offset, node) return self.removeCell(offset, node)
if node.isBroken() or node.isDown(): if node.isUnknown():
raise PartitionTableException('Invalid node state') raise PartitionTableException('Invalid node state')
self.count_dict.setdefault(node, 0) self.count_dict.setdefault(node, 0)
...@@ -196,8 +196,10 @@ class PartitionTable(object): ...@@ -196,8 +196,10 @@ class PartitionTable(object):
break break
def dropNode(self, node): def dropNode(self, node):
count = self.count_dict.pop(node, None) count = self.count_dict.get(node)
assert not count, (node, count) if count == 0:
del self.count_dict[node]
return not count
def load(self, ptid, row_list, nm): def load(self, ptid, row_list, nm):
""" """
......
...@@ -148,14 +148,9 @@ def parseNodeAddress(address, port_opt=None): ...@@ -148,14 +148,9 @@ def parseNodeAddress(address, port_opt=None):
# or return either raw host & port or getaddrinfo return value. # or return either raw host & port or getaddrinfo return value.
return socket.getaddrinfo(host, port, 0, socket.SOCK_STREAM)[0][4][:2] return socket.getaddrinfo(host, port, 0, socket.SOCK_STREAM)[0][4][:2]
def parseMasterList(masters, except_node=None): def parseMasterList(masters):
assert masters, 'At least one master must be defined' assert masters, 'At least one master must be defined'
master_node_list = [] return map(parseNodeAddress, masters.split())
for node in masters.split():
address = parseNodeAddress(node)
if address != except_node:
master_node_list.append(address)
return master_node_list
class ReadBuffer(object): class ReadBuffer(object):
......
This diff is collapsed.
...@@ -18,9 +18,7 @@ from ..app import monotonic_time ...@@ -18,9 +18,7 @@ from ..app import monotonic_time
from neo.lib import logging from neo.lib import logging
from neo.lib.exception import StoppedOperation from neo.lib.exception import StoppedOperation
from neo.lib.handler import EventHandler from neo.lib.handler import EventHandler
from neo.lib.protocol import (uuid_str, NodeTypes, NodeStates, Packets, from neo.lib.protocol import Packets
BrokenNodeDisallowedError, ProtocolError,
)
class MasterHandler(EventHandler): class MasterHandler(EventHandler):
"""This class implements a generic part of the event handlers.""" """This class implements a generic part of the event handlers."""
...@@ -28,41 +26,10 @@ class MasterHandler(EventHandler): ...@@ -28,41 +26,10 @@ class MasterHandler(EventHandler):
def connectionCompleted(self, conn, new=None): def connectionCompleted(self, conn, new=None):
if new is None: if new is None:
super(MasterHandler, self).connectionCompleted(conn) super(MasterHandler, self).connectionCompleted(conn)
elif new:
self._notifyNodeInformation(conn)
def requestIdentification(self, conn, node_type, uuid, address, name, _): def connectionLost(self, conn, new_state=None):
self.checkClusterName(name) if self.app.listening_conn: # if running
app = self.app self._connectionLost(conn)
node = app.nm.getByUUID(uuid)
if node:
if node_type is NodeTypes.MASTER and not (
None != address == node.getAddress()):
raise ProtocolError
if node.isBroken():
raise BrokenNodeDisallowedError
peer_uuid = self._setupNode(conn, node_type, uuid, address, node)
if app.primary:
primary_address = app.server
elif app.primary_master_node is not None:
primary_address = app.primary_master_node.getAddress()
else:
primary_address = None
known_master_list = [(app.server, app.uuid)]
for n in app.nm.getMasterList():
if n.isBroken():
continue
known_master_list.append((n.getAddress(), n.getUUID()))
conn.answer(Packets.AcceptIdentification(
NodeTypes.MASTER,
app.uuid,
app.pt.getPartitions(),
app.pt.getReplicas(),
peer_uuid,
primary_address,
known_master_list),
)
def askClusterState(self, conn): def askClusterState(self, conn):
state = self.app.getClusterState() state = self.app.getClusterState()
...@@ -84,11 +51,12 @@ class MasterHandler(EventHandler): ...@@ -84,11 +51,12 @@ class MasterHandler(EventHandler):
self.app.getLastTransaction())) self.app.getLastTransaction()))
def _notifyNodeInformation(self, conn): def _notifyNodeInformation(self, conn):
nm = self.app.nm app = self.app
node_list = [] node = app.nm.getByUUID(conn.getUUID())
node_list.extend(n.asTuple() for n in nm.getMasterList()) node_list = app.nm.getList()
node_list.extend(n.asTuple() for n in nm.getClientList()) node_list.remove(node)
node_list.extend(n.asTuple() for n in nm.getStorageList()) node_list = ([node.asTuple()] # for id_timestamp
+ app.getNodeInformationDict(node_list)[node.getType()])
conn.send(Packets.NotifyNodeInformation(monotonic_time(), node_list)) conn.send(Packets.NotifyNodeInformation(monotonic_time(), node_list))
def askPartitionTable(self, conn): def askPartitionTable(self, conn):
...@@ -96,15 +64,10 @@ class MasterHandler(EventHandler): ...@@ -96,15 +64,10 @@ class MasterHandler(EventHandler):
conn.answer(Packets.AnswerPartitionTable(pt.getID(), pt.getRowList())) conn.answer(Packets.AnswerPartitionTable(pt.getID(), pt.getRowList()))
DISCONNECTED_STATE_DICT = {
NodeTypes.STORAGE: NodeStates.TEMPORARILY_DOWN,
}
class BaseServiceHandler(MasterHandler): class BaseServiceHandler(MasterHandler):
"""This class deals with events for a service phase.""" """This class deals with events for a service phase."""
def connectionCompleted(self, conn, new): def connectionCompleted(self, conn, new):
self._notifyNodeInformation(conn)
pt = self.app.pt pt = self.app.pt
conn.send(Packets.SendPartitionTable(pt.getID(), pt.getRowList())) conn.send(Packets.SendPartitionTable(pt.getID(), pt.getRowList()))
...@@ -115,21 +78,16 @@ class BaseServiceHandler(MasterHandler): ...@@ -115,21 +78,16 @@ class BaseServiceHandler(MasterHandler):
return # for example, when a storage is removed by an admin return # for example, when a storage is removed by an admin
assert node.isStorage(), node assert node.isStorage(), node
logging.info('storage node lost') logging.info('storage node lost')
if new_state != NodeStates.BROKEN: if node.isPending():
new_state = DISCONNECTED_STATE_DICT.get(node.getType(),
NodeStates.DOWN)
assert new_state in (NodeStates.TEMPORARILY_DOWN, NodeStates.DOWN,
NodeStates.BROKEN), new_state
assert node.getState() not in (NodeStates.TEMPORARILY_DOWN,
NodeStates.DOWN, NodeStates.BROKEN), (uuid_str(self.app.uuid),
node.whoSetState(), new_state)
was_pending = node.isPending()
node.setState(new_state)
if new_state != NodeStates.BROKEN and was_pending:
# was in pending state, so drop it from the node manager to forget # 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 # it and do not set in running state when it comes back
logging.info('drop a pending node from the node manager') logging.info('drop a pending node from the node manager')
app.nm.remove(node) node.setUnknown()
elif node.isDown():
# Already put in DOWN state by AdministrationHandler.setNodeState
return
else:
node.setDown()
app.broadcastNodesInformation([node]) app.broadcastNodesInformation([node])
if app.truncate_tid: if app.truncate_tid:
raise StoppedOperation raise StoppedOperation
......
...@@ -34,8 +34,8 @@ CLUSTER_STATE_WORKFLOW = { ...@@ -34,8 +34,8 @@ CLUSTER_STATE_WORKFLOW = {
ClusterStates.STARTING_BACKUP), ClusterStates.STARTING_BACKUP),
} }
NODE_STATE_WORKFLOW = { NODE_STATE_WORKFLOW = {
NodeTypes.MASTER: (NodeStates.UNKNOWN,), NodeTypes.MASTER: (NodeStates.DOWN,),
NodeTypes.STORAGE: (NodeStates.UNKNOWN, NodeStates.DOWN), NodeTypes.STORAGE: (NodeStates.DOWN, NodeStates.UNKNOWN),
} }
class AdministrationHandler(MasterHandler): class AdministrationHandler(MasterHandler):
...@@ -95,7 +95,7 @@ class AdministrationHandler(MasterHandler): ...@@ -95,7 +95,7 @@ class AdministrationHandler(MasterHandler):
message = ('state changed' if state_changed else message = ('state changed' if state_changed else
'node already in %s state' % state) 'node already in %s state' % state)
if node.isStorage(): if node.isStorage():
keep = state == NodeStates.UNKNOWN keep = state == NodeStates.DOWN
try: try:
cell_list = app.pt.dropNodeList([node], keep) cell_list = app.pt.dropNodeList([node], keep)
except PartitionTableException, e: except PartitionTableException, e:
......
...@@ -15,30 +15,22 @@ ...@@ -15,30 +15,22 @@
# along with this program. If not, see <http://www.gnu.org/licenses/>. # along with this program. If not, see <http://www.gnu.org/licenses/>.
from neo.lib.handler import DelayEvent from neo.lib.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 ..app import monotonic_time
from . import MasterHandler from . import MasterHandler
class ClientServiceHandler(MasterHandler): class ClientServiceHandler(MasterHandler):
""" Handler dedicated to client during service state """ """ Handler dedicated to client during service state """
def connectionLost(self, conn, new_state): def _connectionLost(self, conn):
# cancel its transactions and forgot the node # cancel its transactions and forgot the node
app = self.app app = self.app
if app.listening_conn: # if running node = app.nm.getByUUID(conn.getUUID())
node = app.nm.getByUUID(conn.getUUID()) assert node is not None, conn
assert node is not None app.tm.clientLost(node)
app.tm.clientLost(node) node.setUnknown()
node.setState(NodeStates.DOWN) app.broadcastNodesInformation([node])
app.broadcastNodesInformation([node]) app.nm.remove(node)
app.nm.remove(node)
def _notifyNodeInformation(self, conn):
nm = self.app.nm
node_list = [nm.getByUUID(conn.getUUID()).asTuple()] # for id_timestamp
node_list.extend(n.asTuple() for n in nm.getMasterList())
node_list.extend(n.asTuple() for n in nm.getStorageList())
conn.send(Packets.NotifyNodeInformation(monotonic_time(), node_list))
def askBeginTransaction(self, conn, tid): def askBeginTransaction(self, conn, tid):
""" """
......
#
# 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/>.
from neo.lib import logging
from neo.lib.protocol import uuid_str, NodeTypes, Packets
from neo.lib.protocol import NotReadyError
from neo.lib.exception import ElectionFailure
from neo.lib.handler import EventHandler
from . import MasterHandler
class BaseElectionHandler(EventHandler):
def _notifyNodeInformation(self, conn):
pass
def reelectPrimary(self, conn):
raise ElectionFailure, 'reelection requested'
def announcePrimary(self, conn):
app = self.app
if app.primary:
# I am also the primary... So restart the election.
raise ElectionFailure, 'another primary arises'
try:
address = app.master_address_dict[conn]
assert conn.isServer()
except KeyError:
address = conn.getAddress()
assert conn.isClient()
app.primary = False
app.primary_master_node = node = app.nm.getByAddress(address)
app.negotiating_master_node_set.clear()
logging.info('%s is the primary', node)
def elect(self, conn, peer_address):
app = self.app
if app.server < peer_address:
app.primary = False
if conn is not None:
app.master_address_dict[conn] = peer_address
app.negotiating_master_node_set.discard(peer_address)
class ClientElectionHandler(BaseElectionHandler):
def notifyNodeInformation(self, conn, timestamp, node_list):
# XXX: For the moment, do nothing because
# we'll close this connection and reconnect.
pass
def connectionFailed(self, conn):
addr = conn.getAddress()
node = self.app.nm.getByAddress(addr)
assert node is not None, (uuid_str(self.app.uuid), addr)
# node may still be in unknown state
self.app.negotiating_master_node_set.discard(addr)
super(ClientElectionHandler, self).connectionFailed(conn)
def connectionCompleted(self, conn):
app = self.app
conn.ask(Packets.RequestIdentification(
NodeTypes.MASTER,
app.uuid,
app.server,
app.name,
None,
))
super(ClientElectionHandler, self).connectionCompleted(conn)
def connectionLost(self, conn, new_state):
# Retry connection. Either the node just died (and we will end up in
# connectionFailed) or it just got elected (and we must not ignore
# that node).
addr = conn.getAddress()
self.app.unconnected_master_node_set.add(addr)
self.app.negotiating_master_node_set.discard(addr)
def _acceptIdentification(self, node, peer_uuid, num_partitions,
num_replicas, your_uuid, primary, known_master_list):
app = self.app
# Register new master nodes.
for address, uuid in known_master_list:
if app.server == address:
# This is self.
assert node.getAddress() != primary or uuid == your_uuid, (
uuid_str(uuid), uuid_str(your_uuid))
continue
n = app.nm.getByAddress(address)
if n is None:
n = app.nm.createMaster(address=address)
if primary is not None:
# The primary master is defined.
if app.primary_master_node is not None \
and app.primary_master_node.getAddress() != primary:
# There are multiple primary master nodes. This is
# dangerous.
raise ElectionFailure, 'multiple primary master nodes'
primary_node = app.nm.getByAddress(primary)
if primary_node is None:
# I don't know such a node. Probably this information
# is old. So ignore it.
logging.warning('received an unknown primary node')
else:
# Whatever the situation is, I trust this master.
app.primary = False
app.primary_master_node = primary_node
# Stop waiting for connections than primary master's to
# complete to exit election phase ASAP.
app.negotiating_master_node_set.clear()
return
self.elect(None, node.getAddress())
class ServerElectionHandler(BaseElectionHandler, MasterHandler):
def _setupNode(self, conn, node_type, uuid, address, node):
app = self.app
if node_type != NodeTypes.MASTER:
logging.info('reject a connection from a non-master')
raise NotReadyError
if node is None is app.nm.getByAddress(address):
app.nm.createMaster(address=address)
self.elect(conn, address)
return uuid
...@@ -15,26 +15,25 @@ ...@@ -15,26 +15,25 @@
# along with this program. If not, see <http://www.gnu.org/licenses/>. # along with this program. If not, see <http://www.gnu.org/licenses/>.
from neo.lib import logging from neo.lib import logging
from neo.lib.exception import PrimaryElected
from neo.lib.handler import EventHandler
from neo.lib.protocol import ClusterStates, NodeStates, NodeTypes, \ from neo.lib.protocol import ClusterStates, NodeStates, NodeTypes, \
NotReadyError, ProtocolError, uuid_str NotReadyError, Packets, ProtocolError, uuid_str
from ..app import monotonic_time from ..app import monotonic_time
from . import MasterHandler
class IdentificationHandler(MasterHandler): class IdentificationHandler(EventHandler):
def requestIdentification(self, conn, *args, **kw): def requestIdentification(self, conn, node_type, uuid,
super(IdentificationHandler, self).requestIdentification(conn, *args, address, name, id_timestamp):
**kw)
handler = conn.getHandler()
assert not isinstance(handler, IdentificationHandler), handler
handler.connectionCompleted(conn, True)
def _setupNode(self, conn, node_type, uuid, address, node):
app = self.app app = self.app
self.checkClusterName(name)
if address == app.server:
raise ProtocolError('address conflict')
node = app.nm.getByUUID(uuid)
by_addr = address and app.nm.getByAddress(address) by_addr = address and app.nm.getByAddress(address)
while 1: while 1:
if by_addr: if by_addr:
if not by_addr.isConnected(): if not by_addr.isIdentified():
if node is by_addr: if node is by_addr:
break break
if not node or uuid < 0: if not node or uuid < 0:
...@@ -43,7 +42,7 @@ class IdentificationHandler(MasterHandler): ...@@ -43,7 +42,7 @@ class IdentificationHandler(MasterHandler):
node = by_addr node = by_addr
break break
elif node: elif node:
if node.isConnected(): if node.isIdentified():
if uuid < 0: if uuid < 0:
# The peer wants a temporary id that's already assigned. # The peer wants a temporary id that's already assigned.
# Let's give it another one. # Let's give it another one.
...@@ -77,7 +76,14 @@ class IdentificationHandler(MasterHandler): ...@@ -77,7 +76,14 @@ class IdentificationHandler(MasterHandler):
uuid is not None and node is not None) uuid is not None and node is not None)
human_readable_node_type = ' storage (%s) ' % (state, ) human_readable_node_type = ' storage (%s) ' % (state, )
elif node_type == NodeTypes.MASTER: elif node_type == NodeTypes.MASTER:
handler = app.secondary_master_handler if app.election:
if id_timestamp and \
(id_timestamp, address) < (app.election, app.server):
raise PrimaryElected(by_addr or
app.nm.createMaster(address=address))
handler = app.election_handler
else:
handler = app.secondary_handler
human_readable_node_type = ' master ' human_readable_node_type = ' master '
elif node_type == NodeTypes.ADMIN: elif node_type == NodeTypes.ADMIN:
handler = app.administration_handler handler = app.administration_handler
...@@ -94,22 +100,43 @@ class IdentificationHandler(MasterHandler): ...@@ -94,22 +100,43 @@ class IdentificationHandler(MasterHandler):
node.setUUID(uuid) node.setUUID(uuid)
node.id_timestamp = monotonic_time() node.id_timestamp = monotonic_time()
node.setState(state) node.setState(state)
node.setConnection(conn)
conn.setHandler(handler) conn.setHandler(handler)
node.setConnection(conn, not node.isIdentified())
app.broadcastNodesInformation([node], node) app.broadcastNodesInformation([node], node)
return uuid
class SecondaryIdentificationHandler(MasterHandler): conn.answer(Packets.AcceptIdentification(
NodeTypes.MASTER,
app.uuid,
app.pt.getPartitions(),
app.pt.getReplicas(),
uuid))
handler._notifyNodeInformation(conn)
handler.connectionCompleted(conn, True)
def announcePrimary(self, conn):
# If we received AnnouncePrimary on a client connection, we might have
# set this handler on server connection, and might receive
# AnnouncePrimary there too. As we cannot reach this without already
# handling a first AnnouncePrimary, we can safely ignore this one.
pass
def _setupNode(self, conn, node_type, uuid, address, node): class SecondaryIdentificationHandler(EventHandler):
# Nothing to do, storage will disconnect when it receives our answer.
# Primary will do the checks.
return uuid
def requestIdentification(self, conn, node_type, uuid,
address, name, id_timestamp):
app = self.app
self.checkClusterName(name)
if address == app.server:
raise ProtocolError('address conflict')
primary = app.primary_master.getAddress()
if primary == address:
primary = None
elif not app.primary_master.isIdentified():
if node_type == NodeTypes.MASTER:
node = app.nm.createMaster(address=address)
if id_timestamp:
conn.close()
raise PrimaryElected(node)
primary = None
# For some cases, we rely on the fact that the remote will not retry
# immediately (see SocketConnector.CONNECT_LIMIT).
known_master_list = [node.getAddress()
for node in app.nm.getMasterList()]
conn.send(Packets.NotPrimaryMaster(
primary and known_master_list.index(primary),
known_master_list))
conn.abort()
...@@ -15,83 +15,81 @@ ...@@ -15,83 +15,81 @@
# along with this program. If not, see <http://www.gnu.org/licenses/>. # along with this program. If not, see <http://www.gnu.org/licenses/>.
import sys import sys
from ..app import monotonic_time
from . import MasterHandler from . import MasterHandler
from neo.lib.handler import EventHandler from neo.lib.exception import PrimaryElected, PrimaryFailure
from neo.lib.exception import ElectionFailure, PrimaryFailure from neo.lib.protocol import ClusterStates, NodeStates, NodeTypes, Packets
from neo.lib.protocol import NodeStates, NodeTypes, Packets, uuid_str
from neo.lib import logging
class SecondaryMasterHandler(MasterHandler):
""" Handler used by primary to handle secondary masters"""
def connectionLost(self, conn, new_state): class SecondaryHandler(MasterHandler):
app = self.app """Handler used by primary to handle secondary masters"""
if app.listening_conn: # if running
node = app.nm.getByUUID(conn.getUUID())
node.setDown()
app.broadcastNodesInformation([node])
def announcePrimary(self, conn):
raise ElectionFailure, 'another primary arises'
def reelectPrimary(self, conn): def _connectionLost(self, conn):
raise ElectionFailure, 'reelection requested' app = self.app
node = app.nm.getByUUID(conn.getUUID())
node.setDown()
app.broadcastNodesInformation([node])
def _notifyNodeInformation(self, conn):
node_list = [n.asTuple() for n in self.app.nm.getMasterList()]
conn.send(Packets.NotifyNodeInformation(monotonic_time(), node_list))
class PrimaryHandler(EventHandler): class ElectionHandler(MasterHandler):
""" Handler used by secondaries to handle primary master""" """Handler used by primary to handle secondary masters during election"""
def connectionLost(self, conn, new_state): def connectionCompleted(self, conn, new=None):
self.connectionFailed(conn) if new is None:
super(ElectionHandler, self).connectionCompleted(conn)
app = self.app
conn.ask(Packets.RequestIdentification(NodeTypes.MASTER,
app.uuid, app.server, app.name, app.election))
def connectionFailed(self, conn): def connectionFailed(self, conn):
self.app.primary_master_node.setDown() super(ElectionHandler, self).connectionFailed(conn)
if self.app.listening_conn: # if running self.connectionLost(conn)
def _acceptIdentification(self, node, *args):
raise PrimaryElected(node)
def _connectionLost(self, *args):
if self.app.primary: # not switching to secondary role
self.app._current_manager.try_secondary = True
def notPrimaryMaster(self, *args):
try:
super(ElectionHandler, self).notPrimaryMaster(*args)
except PrimaryElected, e:
# We keep playing the primary role when the peer does not
# know yet that we won election against the returned node.
if not e.args[0].isIdentified():
raise
# There may be new master nodes. Connect to them.
self.app._current_manager.try_secondary = True
class PrimaryHandler(ElectionHandler):
"""Handler used by secondaries to handle primary master"""
def _acceptIdentification(self, node, num_partitions, num_replicas):
assert self.app.primary_master is node, (self.app.primary_master, node)
def _connectionLost(self, conn):
node = self.app.primary_master
# node is None when switching to primary role
if node and not node.isConnected(True):
raise PrimaryFailure('primary master is dead') raise PrimaryFailure('primary master is dead')
def connectionCompleted(self, conn): def notPrimaryMaster(self, *args):
app = self.app try:
addr = conn.getAddress() super(ElectionHandler, self).notPrimaryMaster(*args)
node = app.nm.getByAddress(addr) except PrimaryElected, e:
# connection successful, set it as running if e.args[0] is not self.app.primary_master:
node.setRunning() raise
conn.ask(Packets.RequestIdentification(
NodeTypes.MASTER,
app.uuid,
app.server,
app.name,
None,
))
super(PrimaryHandler, self).connectionCompleted(conn)
def reelectPrimary(self, conn):
raise ElectionFailure, 'reelection requested'
def notifyClusterInformation(self, conn, state): def notifyClusterInformation(self, conn, state):
self.app.cluster_state = state if state == ClusterStates.STOPPING:
sys.exit()
def notifyNodeInformation(self, conn, timestamp, node_list): def notifyNodeInformation(self, conn, timestamp, node_list):
super(PrimaryHandler, self).notifyNodeInformation( super(PrimaryHandler, self).notifyNodeInformation(
conn, timestamp, node_list) conn, timestamp, node_list)
for node_type, _, uuid, state, _ in node_list: for node_type, _, uuid, state, _ in node_list:
assert node_type == NodeTypes.MASTER, node_type assert node_type == NodeTypes.MASTER, node_type
if uuid == self.app.uuid and state == NodeStates.UNKNOWN: if uuid == self.app.uuid and state == NodeStates.DOWN:
sys.exit() sys.exit()
def _acceptIdentification(self, node, uuid, num_partitions,
num_replicas, your_uuid, primary, known_master_list):
app = self.app
if primary != app.primary_master_node.getAddress():
raise PrimaryFailure('unexpected primary uuid')
if your_uuid != app.uuid:
app.uuid = your_uuid
logging.info('My UUID: ' + uuid_str(your_uuid))
node.setUUID(uuid)
app.id_timestamp = None
...@@ -15,6 +15,7 @@ ...@@ -15,6 +15,7 @@
# along with this program. If not, see <http://www.gnu.org/licenses/>. # along with this program. If not, see <http://www.gnu.org/licenses/>.
from neo.lib import logging from neo.lib import logging
from neo.lib.connection import ClientConnection
from neo.lib.protocol import Packets, ProtocolError, ClusterStates, NodeStates from neo.lib.protocol import Packets, ProtocolError, ClusterStates, NodeStates
from .app import monotonic_time from .app import monotonic_time
from .handlers import MasterHandler from .handlers import MasterHandler
...@@ -47,6 +48,7 @@ class RecoveryManager(MasterHandler): ...@@ -47,6 +48,7 @@ class RecoveryManager(MasterHandler):
TID, and the last Partition Table ID from storage nodes, then get TID, and the last Partition Table ID from storage nodes, then get
back the latest partition table or make a new table from scratch, back the latest partition table or make a new table from scratch,
if this is the first time. if this is the first time.
A new primary master may also arise during this phase.
""" """
logging.info('begin the recovery of the status') logging.info('begin the recovery of the status')
app = self.app app = self.app
...@@ -54,9 +56,30 @@ class RecoveryManager(MasterHandler): ...@@ -54,9 +56,30 @@ class RecoveryManager(MasterHandler):
app.changeClusterState(ClusterStates.RECOVERING) app.changeClusterState(ClusterStates.RECOVERING)
pt.clear() pt.clear()
self.try_secondary = True
# collect the last partition table available # collect the last partition table available
poll = app.em.poll poll = app.em.poll
while 1: while 1:
if self.try_secondary:
# Keep trying to connect to all other known masters,
# to make sure there is a challege between each pair
# of masters in the cluster. If we win, all connections
# opened here will be closed.
self.try_secondary = False
node_list = []
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
# 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.isDown():
node.setDown()
node_list.append(node)
ClientConnection(app, app.election_handler, node)
if node_list:
app.broadcastNodesInformation(node_list)
poll(1) poll(1)
if pt.filled(): if pt.filled():
# A partition table exists, we are starting an existing # A partition table exists, we are starting an existing
...@@ -100,6 +123,17 @@ class RecoveryManager(MasterHandler): ...@@ -100,6 +123,17 @@ class RecoveryManager(MasterHandler):
for node in node_list: for node in node_list:
assert node.isPending(), node assert node.isPending(), node
node.setRunning() node.setRunning()
for node in app.nm.getMasterList():
if not (node is app._node or node.isIdentified()):
if node.isConnected(True):
node.getConnection().close()
assert node.isDown(), node
elif not node.isDown():
assert self.try_secondary, node
node.setDown()
node_list.append(node)
app.broadcastNodesInformation(node_list) app.broadcastNodesInformation(node_list)
if pt.getID() is None: if pt.getID() is None:
......
...@@ -157,10 +157,10 @@ class NeoCTL(BaseApplication): ...@@ -157,10 +157,10 @@ class NeoCTL(BaseApplication):
return self.setClusterState(ClusterStates.VERIFYING) return self.setClusterState(ClusterStates.VERIFYING)
def killNode(self, node): def killNode(self, node):
return self._setNodeState(node, NodeStates.UNKNOWN) return self._setNodeState(node, NodeStates.DOWN)
def dropNode(self, node): def dropNode(self, node):
return self._setNodeState(node, NodeStates.DOWN) return self._setNodeState(node, NodeStates.UNKNOWN)
def getPrimary(self): def getPrimary(self):
""" """
......
...@@ -42,7 +42,6 @@ from neo.tests.benchmark import BenchmarkRunner ...@@ -42,7 +42,6 @@ from neo.tests.benchmark import BenchmarkRunner
# each of them have to import its TestCase classes # each of them have to import its TestCase classes
UNIT_TEST_MODULES = [ UNIT_TEST_MODULES = [
# generic parts # generic parts
'neo.tests.testBootstrap',
'neo.tests.testConnection', 'neo.tests.testConnection',
'neo.tests.testHandler', 'neo.tests.testHandler',
'neo.tests.testNodes', 'neo.tests.testNodes',
...@@ -50,7 +49,6 @@ UNIT_TEST_MODULES = [ ...@@ -50,7 +49,6 @@ UNIT_TEST_MODULES = [
'neo.tests.testPT', 'neo.tests.testPT',
# master application # master application
'neo.tests.master.testClientHandler', 'neo.tests.master.testClientHandler',
'neo.tests.master.testElectionHandler',
'neo.tests.master.testMasterApp', 'neo.tests.master.testMasterApp',
'neo.tests.master.testMasterPT', 'neo.tests.master.testMasterPT',
'neo.tests.master.testRecovery', 'neo.tests.master.testRecovery',
...@@ -61,7 +59,6 @@ UNIT_TEST_MODULES = [ ...@@ -61,7 +59,6 @@ UNIT_TEST_MODULES = [
'neo.tests.storage.testMasterHandler', 'neo.tests.storage.testMasterHandler',
'neo.tests.storage.testStorageApp', 'neo.tests.storage.testStorageApp',
'neo.tests.storage.testStorage' + os.getenv('NEO_TESTS_ADAPTER', 'SQLite'), 'neo.tests.storage.testStorage' + os.getenv('NEO_TESTS_ADAPTER', 'SQLite'),
'neo.tests.storage.testIdentificationHandler',
'neo.tests.storage.testTransactions', 'neo.tests.storage.testTransactions',
# client application # client application
'neo.tests.client.testClientApp', 'neo.tests.client.testClientApp',
......
...@@ -28,8 +28,7 @@ from neo.lib.util import dump ...@@ -28,8 +28,7 @@ from neo.lib.util import dump
from neo.lib.bootstrap import BootstrapManager from neo.lib.bootstrap import BootstrapManager
from .checker import Checker from .checker import Checker
from .database import buildDatabaseManager from .database import buildDatabaseManager
from .handlers import identification, initialization from .handlers import identification, initialization, master
from .handlers import master, hidden
from .replicator import Replicator from .replicator import Replicator
from .transactions import TransactionManager from .transactions import TransactionManager
...@@ -170,10 +169,6 @@ class Application(BaseApplication): ...@@ -170,10 +169,6 @@ class Application(BaseApplication):
if self.master_node is None: if self.master_node is None:
# look for the primary master # look for the primary master
self.connectToPrimary() self.connectToPrimary()
# check my state
node = self.nm.getByUUID(self.uuid)
if node is not None and node.isHidden():
self.wait()
self.checker = Checker(self) self.checker = Checker(self)
self.replicator = Replicator(self) self.replicator = Replicator(self)
self.tm = TransactionManager(self) self.tm = TransactionManager(self)
...@@ -273,20 +268,6 @@ class Application(BaseApplication): ...@@ -273,20 +268,6 @@ class Application(BaseApplication):
if state == ClusterStates.STOPPING_BACKUP: if state == ClusterStates.STOPPING_BACKUP:
self.replicator.stop() self.replicator.stop()
def wait(self):
# change handler
logging.info("waiting in hidden state")
_poll = self._poll
handler = hidden.HiddenHandler(self)
for conn in self.em.getConnectionList():
conn.setHandler(handler)
node = self.nm.getByUUID(self.uuid)
while True:
_poll()
if not node.isHidden():
break
def newTask(self, iterator): def newTask(self, iterator):
try: try:
iterator.next() iterator.next()
......
...@@ -56,12 +56,9 @@ class BaseMasterHandler(BaseHandler): ...@@ -56,12 +56,9 @@ class BaseMasterHandler(BaseHandler):
if uuid == self.app.uuid: if uuid == self.app.uuid:
# This is me, do what the master tell me # This is me, do what the master tell me
logging.info("I was told I'm %s", state) logging.info("I was told I'm %s", state)
if state in (NodeStates.DOWN, NodeStates.TEMPORARILY_DOWN, if state in (NodeStates.UNKNOWN, NodeStates.DOWN):
NodeStates.BROKEN, NodeStates.UNKNOWN): erase = state == NodeStates.UNKNOWN
erase = state == NodeStates.DOWN
self.app.shutdown(erase=erase) self.app.shutdown(erase=erase)
elif state == NodeStates.HIDDEN:
raise StoppedOperation
elif node_type == NodeTypes.CLIENT and state != NodeStates.RUNNING: elif node_type == NodeTypes.CLIENT and state != NodeStates.RUNNING:
logging.info('Notified of non-running client, abort (%s)', logging.info('Notified of non-running client, abort (%s)',
uuid_str(uuid)) uuid_str(uuid))
......
#
# 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/>.
from . import BaseMasterHandler
from neo.lib import logging
from neo.lib.protocol import CellStates
class HiddenHandler(BaseMasterHandler):
"""This class implements a generic part of the event handlers."""
def notifyPartitionChanges(self, conn, ptid, cell_list):
"""This is very similar to Send Partition Table, except that
the information is only about changes from the previous."""
app = self.app
if ptid <= app.pt.getID():
# Ignore this packet.
logging.debug('ignoring older partition changes')
return
# update partition table in memory and the database
app.pt.update(ptid, cell_list, app.nm)
app.dm.changePartitionTable(ptid, cell_list)
# Check changes for replications
for offset, uuid, state in cell_list:
if uuid == app.uuid and app.replicator is not None:
# If this is for myself, this can affect replications.
if state == CellStates.DISCARDED:
app.replicator.removePartition(offset)
elif state == CellStates.OUT_OF_DATE:
app.replicator.addPartition(offset)
def startOperation(self, conn):
self.app.operational = True
...@@ -17,7 +17,7 @@ ...@@ -17,7 +17,7 @@
from neo.lib import logging from neo.lib import logging
from neo.lib.handler import EventHandler from neo.lib.handler import EventHandler
from neo.lib.protocol import NodeTypes, NotReadyError, Packets from neo.lib.protocol import NodeTypes, NotReadyError, Packets
from neo.lib.protocol import ProtocolError, BrokenNodeDisallowedError from neo.lib.protocol import ProtocolError
from .storage import StorageOperationHandler from .storage import StorageOperationHandler
from .client import ClientOperationHandler, ClientReadOnlyOperationHandler from .client import ClientOperationHandler, ClientReadOnlyOperationHandler
...@@ -47,8 +47,6 @@ class IdentificationHandler(EventHandler): ...@@ -47,8 +47,6 @@ class IdentificationHandler(EventHandler):
if uuid == app.uuid: if uuid == app.uuid:
raise ProtocolError("uuid conflict or loopback connection") raise ProtocolError("uuid conflict or loopback connection")
node = app.nm.getByUUID(uuid, id_timestamp) node = app.nm.getByUUID(uuid, id_timestamp)
if node.isBroken():
raise BrokenNodeDisallowedError
# choose the handler according to the node type # choose the handler according to the node type
if node_type == NodeTypes.CLIENT: if node_type == NodeTypes.CLIENT:
if app.dm.getBackupTID(): if app.dm.getBackupTID():
...@@ -67,6 +65,5 @@ class IdentificationHandler(EventHandler): ...@@ -67,6 +65,5 @@ class IdentificationHandler(EventHandler):
node.setConnection(conn, app.uuid < uuid) node.setConnection(conn, app.uuid < uuid)
# accept the identification and trigger an event # accept the identification and trigger an event
conn.answer(Packets.AcceptIdentification(NodeTypes.STORAGE, uuid and conn.answer(Packets.AcceptIdentification(NodeTypes.STORAGE, uuid and
app.uuid, app.pt.getPartitions(), app.pt.getReplicas(), uuid, app.uuid, app.pt.getPartitions(), app.pt.getReplicas(), uuid))
app.master_node.getAddress(), ()))
handler.connectionCompleted(conn) handler.connectionCompleted(conn)
...@@ -18,8 +18,7 @@ import weakref ...@@ -18,8 +18,7 @@ import weakref
from functools import wraps from functools import wraps
from neo.lib.connection import ConnectionClosed from neo.lib.connection import ConnectionClosed
from neo.lib.handler import DelayEvent, EventHandler from neo.lib.handler import DelayEvent, EventHandler
from neo.lib.protocol import Errors, NodeStates, Packets, ProtocolError, \ from neo.lib.protocol import Errors, Packets, ProtocolError, ZERO_HASH
ZERO_HASH
def checkConnectionIsReplicatorConnection(func): def checkConnectionIsReplicatorConnection(func):
def wrapper(self, conn, *args, **kw): def wrapper(self, conn, *args, **kw):
...@@ -53,7 +52,7 @@ class StorageOperationHandler(EventHandler): ...@@ -53,7 +52,7 @@ class StorageOperationHandler(EventHandler):
node = app.nm.getByUUID(uuid) node = app.nm.getByUUID(uuid)
else: else:
node = app.nm.getByAddress(conn.getAddress()) node = app.nm.getByAddress(conn.getAddress())
node.setState(NodeStates.DOWN) node.setUnknown()
replicator = app.replicator replicator = app.replicator
if replicator.current_node is node: if replicator.current_node is node:
replicator.abort() replicator.abort()
......
...@@ -309,10 +309,6 @@ class NeoUnitTestBase(NeoTestBase): ...@@ -309,10 +309,6 @@ class NeoUnitTestBase(NeoTestBase):
""" Check if the ProtocolError exception was raised """ """ Check if the ProtocolError exception was raised """
self.assertRaises(protocol.ProtocolError, method, *args, **kwargs) self.assertRaises(protocol.ProtocolError, method, *args, **kwargs)
def checkNotReadyErrorRaised(self, method, *args, **kwargs):
""" Check if the NotReadyError exception was raised """
self.assertRaises(protocol.NotReadyError, method, *args, **kwargs)
def checkAborted(self, conn): def checkAborted(self, conn):
""" Ensure the connection was aborted """ """ Ensure the connection was aborted """
self.assertEqual(len(conn.mockGetNamedCalls('abort')), 1) self.assertEqual(len(conn.mockGetNamedCalls('abort')), 1)
...@@ -330,16 +326,6 @@ class NeoUnitTestBase(NeoTestBase): ...@@ -330,16 +326,6 @@ class NeoUnitTestBase(NeoTestBase):
self._checkNoPacketSend(conn, 'answer') self._checkNoPacketSend(conn, 'answer')
self._checkNoPacketSend(conn, 'ask') self._checkNoPacketSend(conn, 'ask')
def checkUUIDSet(self, conn, uuid=None, check_intermediate=True):
""" ensure UUID was set on the connection """
calls = conn.mockGetNamedCalls('setUUID')
found_uuid = calls.pop().getParam(0)
if check_intermediate:
for call in calls:
self.assertEqual(found_uuid, call.getParam(0))
if uuid is not None:
self.assertEqual(found_uuid, uuid)
# in check(Ask|Answer|Notify)Packet we return the packet so it can be used # in check(Ask|Answer|Notify)Packet we return the packet so it can be used
# in tests if more accurate checks are required # in tests if more accurate checks are required
......
...@@ -21,7 +21,6 @@ from .. import NeoUnitTestBase, buildUrlFromString ...@@ -21,7 +21,6 @@ from .. import NeoUnitTestBase, buildUrlFromString
from neo.client.app import Application from neo.client.app import Application
from neo.client.cache import test as testCache from neo.client.cache import test as testCache
from neo.client.exception import NEOStorageError from neo.client.exception import NEOStorageError
from neo.lib.protocol import NodeTypes, UUID_NAMESPACES
class ClientApplicationTests(NeoUnitTestBase): class ClientApplicationTests(NeoUnitTestBase):
...@@ -97,63 +96,6 @@ class ClientApplicationTests(NeoUnitTestBase): ...@@ -97,63 +96,6 @@ class ClientApplicationTests(NeoUnitTestBase):
# no packet sent # no packet sent
self.checkNoPacketSent(app.master_conn) self.checkNoPacketSent(app.master_conn)
def test_connectToPrimaryNode(self):
# here we have three master nodes :
# the connection to the first will fail
# the second will have changed
# the third will not be ready
# after the third, the partition table will be operational
# (as if it was connected to the primary master node)
# will raise IndexError at the third iteration
app = self.getApp('127.0.0.1:10010 127.0.0.1:10011')
# TODO: test more connection failure cases
# askLastTransaction
def _ask8(_):
pass
# Sixth packet : askPartitionTable succeeded
def _ask7(_):
app.pt = Mock({'operational': True})
# fifth packet : request node identification succeeded
def _ask6(conn):
app.master_conn = conn
app.uuid = 1 + (UUID_NAMESPACES[NodeTypes.CLIENT] << 24)
app.trying_master_node = app.primary_master_node = Mock({
'getAddress': ('127.0.0.1', 10011),
'__str__': 'Fake master node',
})
# third iteration : node not ready
def _ask4(_):
app.trying_master_node = None
# second iteration : master node changed
def _ask3(_):
app.primary_master_node = Mock({
'getAddress': ('127.0.0.1', 10010),
'__str__': 'Fake master node',
})
# first iteration : connection failed
def _ask2(_):
app.trying_master_node = None
# do nothing for the first call
# Case of an unknown primary_uuid (XXX: handler should probably raise,
# it's not normal for a node to inform of a primary uuid without
# telling us what its address is.)
def _ask1(_):
pass
ask_func_list = [_ask1, _ask2, _ask3, _ask4, _ask6, _ask7, _ask8]
def _ask_base(conn, _, handler=None):
ask_func_list.pop(0)(conn)
app.nm.getByAddress(conn.getAddress())._connection = None
app._ask = _ask_base
# fake environment
app.em.close()
app.em = Mock({'getConnectionList': []})
app.pt = Mock({ 'operational': False})
app.start = lambda: None
app.master_conn = app._connectToPrimaryNode()
self.assertFalse(ask_func_list)
self.assertTrue(app.master_conn is not None)
self.assertTrue(app.pt.operational())
if __name__ == '__main__': if __name__ == '__main__':
unittest.main() unittest.main()
...@@ -609,13 +609,8 @@ class NEOCluster(object): ...@@ -609,13 +609,8 @@ class NEOCluster(object):
self.expectStorageState(process.getUUID(), NodeStates.PENDING, self.expectStorageState(process.getUUID(), NodeStates.PENDING,
*args, **kw) *args, **kw)
def expectUnknown(self, process, *args, **kw): def expectDown(self, process, *args, **kw):
self.expectStorageState(process.getUUID(), NodeStates.UNKNOWN, self.expectStorageState(process.getUUID(), NodeStates.DOWN, *args, **kw)
*args, **kw)
def expectUnavailable(self, process, *args, **kw):
self.expectStorageState(process.getUUID(),
NodeStates.TEMPORARILY_DOWN, *args, **kw)
def expectPrimary(self, uuid=None, *args, **kw): def expectPrimary(self, uuid=None, *args, **kw):
def callback(last_try): def callback(last_try):
...@@ -678,8 +673,7 @@ class NEOCluster(object): ...@@ -678,8 +673,7 @@ class NEOCluster(object):
return current_try, current_try return current_try, current_try
self.expectCondition(callback, *args, **kw) self.expectCondition(callback, *args, **kw)
def expectStorageNotKnown(self, process, *args, **kw): def expectStorageUnknown(self, process, *args, **kw):
# /!\ Not Known != Unknown
process_uuid = process.getUUID() process_uuid = process.getUUID()
def expected_storage_not_known(last_try): def expected_storage_not_known(last_try):
for storage in self.getStorageList(): for storage in self.getStorageList():
......
...@@ -48,7 +48,7 @@ class ClusterTests(NEOFunctionalTest): ...@@ -48,7 +48,7 @@ class ClusterTests(NEOFunctionalTest):
neo.stop() neo.stop()
neo.run(except_storages=(s2, )) neo.run(except_storages=(s2, ))
neo.expectPending(s1) neo.expectPending(s1)
neo.expectUnknown(s2) neo.expectDown(s2)
neo.expectClusterRecovering() neo.expectClusterRecovering()
# Starting missing storage allows cluster to exit Recovery without # Starting missing storage allows cluster to exit Recovery without
# neoctl action. # neoctl action.
...@@ -61,11 +61,11 @@ class ClusterTests(NEOFunctionalTest): ...@@ -61,11 +61,11 @@ class ClusterTests(NEOFunctionalTest):
neo.stop() neo.stop()
neo.run(except_storages=(s2, )) neo.run(except_storages=(s2, ))
neo.expectPending(s1) neo.expectPending(s1)
neo.expectUnknown(s2) neo.expectDown(s2)
neo.expectClusterRecovering() neo.expectClusterRecovering()
neo.startCluster() neo.startCluster()
neo.expectRunning(s1) neo.expectRunning(s1)
neo.expectUnknown(s2) neo.expectDown(s2)
neo.expectClusterRunning() neo.expectClusterRunning()
def testClusterBreaks(self): def testClusterBreaks(self):
...@@ -149,20 +149,20 @@ class ClusterTests(NEOFunctionalTest): ...@@ -149,20 +149,20 @@ class ClusterTests(NEOFunctionalTest):
) )
storages = self.neo.getStorageProcessList() storages = self.neo.getStorageProcessList()
self.neo.run(except_storages=storages) self.neo.run(except_storages=storages)
self.neo.expectStorageNotKnown(storages[0]) self.neo.expectStorageUnknown(storages[0])
self.neo.expectStorageNotKnown(storages[1]) self.neo.expectStorageUnknown(storages[1])
storages[0].start() storages[0].start()
self.neo.expectPending(storages[0]) self.neo.expectPending(storages[0])
self.neo.expectStorageNotKnown(storages[1]) self.neo.expectStorageUnknown(storages[1])
storages[1].start() storages[1].start()
self.neo.expectPending(storages[0]) self.neo.expectPending(storages[0])
self.neo.expectPending(storages[1]) self.neo.expectPending(storages[1])
storages[0].stop() storages[0].stop()
self.neo.expectUnavailable(storages[0]) self.neo.expectDown(storages[0])
self.neo.expectPending(storages[1]) self.neo.expectPending(storages[1])
storages[1].stop() storages[1].stop()
self.neo.expectUnavailable(storages[0]) self.neo.expectDown(storages[0])
self.neo.expectUnavailable(storages[1]) self.neo.expectDown(storages[1])
def test_suite(): def test_suite():
return unittest.makeSuite(ClusterTests) return unittest.makeSuite(ClusterTests)
......
...@@ -59,7 +59,7 @@ class MasterTests(NEOFunctionalTest): ...@@ -59,7 +59,7 @@ class MasterTests(NEOFunctionalTest):
self.assertEqual(len(killed_uuid_list), 1) self.assertEqual(len(killed_uuid_list), 1)
uuid = killed_uuid_list[0] uuid = killed_uuid_list[0]
# Check the state of the primary we just killed # Check the state of the primary we just killed
self.neo.expectMasterState(uuid, (None, NodeStates.UNKNOWN)) self.neo.expectMasterState(uuid, (None, NodeStates.DOWN))
# BUG: The following check expects neoctl to reconnect before # BUG: The following check expects neoctl to reconnect before
# the election finishes. # the election finishes.
self.assertEqual(self.neo.getPrimary(), None) self.assertEqual(self.neo.getPrimary(), None)
...@@ -77,12 +77,13 @@ class MasterTests(NEOFunctionalTest): ...@@ -77,12 +77,13 @@ class MasterTests(NEOFunctionalTest):
killed_uuid_list = self.neo.killSecondaryMaster() killed_uuid_list = self.neo.killSecondaryMaster()
# Test sanity checks. # Test sanity checks.
self.assertEqual(len(killed_uuid_list), 1) self.assertEqual(len(killed_uuid_list), 1)
self.neo.expectMasterState(killed_uuid_list[0], None) self.neo.expectMasterState(killed_uuid_list[0],
self.assertEqual(len(self.neo.getMasterList()), 2) NodeStates.DOWN)
self.assertEqual(len(self.neo.getMasterList()), MASTER_NODE_COUNT)
uuid, = self.neo.killPrimary() uuid, = self.neo.killPrimary()
# Check the state of the primary we just killed # Check the state of the primary we just killed
self.neo.expectMasterState(uuid, (None, NodeStates.UNKNOWN)) self.neo.expectMasterState(uuid, NodeStates.DOWN)
# Check that a primary master arose. # Check that a primary master arose.
self.neo.expectPrimary(timeout=10) self.neo.expectPrimary(timeout=10)
# Check that the uuid really changed. # Check that the uuid really changed.
......
...@@ -168,7 +168,7 @@ class StorageTests(NEOFunctionalTest): ...@@ -168,7 +168,7 @@ class StorageTests(NEOFunctionalTest):
self.neo.neoctl.killNode(started[0].getUUID()) self.neo.neoctl.killNode(started[0].getUUID())
# Cluster still operational. All cells of first storage should be # Cluster still operational. All cells of first storage should be
# outdated. # outdated.
self.neo.expectUnavailable(started[0]) self.neo.expectDown(started[0])
self.neo.expectOudatedCells(2) self.neo.expectOudatedCells(2)
self.neo.expectClusterRunning() self.neo.expectClusterRunning()
...@@ -177,7 +177,7 @@ class StorageTests(NEOFunctionalTest): ...@@ -177,7 +177,7 @@ class StorageTests(NEOFunctionalTest):
started[1].stop() started[1].stop()
# Cluster not operational anymore. Only cells of second storage that # Cluster not operational anymore. Only cells of second storage that
# were shared with the third one should become outdated. # were shared with the third one should become outdated.
self.neo.expectUnavailable(started[1]) self.neo.expectDown(started[1])
self.neo.expectClusterRecovering() self.neo.expectClusterRecovering()
self.neo.expectOudatedCells(3) self.neo.expectOudatedCells(3)
...@@ -198,7 +198,7 @@ class StorageTests(NEOFunctionalTest): ...@@ -198,7 +198,7 @@ class StorageTests(NEOFunctionalTest):
# stop it, the cluster must switch to verification # stop it, the cluster must switch to verification
started[0].stop() started[0].stop()
self.neo.expectUnavailable(started[0]) self.neo.expectDown(started[0])
self.neo.expectClusterRecovering() self.neo.expectClusterRecovering()
# client must have been disconnected # client must have been disconnected
self.assertEqual(len(self.neo.getClientlist()), 0) self.assertEqual(len(self.neo.getClientlist()), 0)
...@@ -224,7 +224,7 @@ class StorageTests(NEOFunctionalTest): ...@@ -224,7 +224,7 @@ class StorageTests(NEOFunctionalTest):
# stop one storage, cluster must remains running # stop one storage, cluster must remains running
started[0].stop() started[0].stop()
self.neo.expectUnavailable(started[0]) self.neo.expectDown(started[0])
self.neo.expectRunning(started[1]) self.neo.expectRunning(started[1])
self.neo.expectRunning(started[2]) self.neo.expectRunning(started[2])
self.neo.expectOudatedCells(number=10) self.neo.expectOudatedCells(number=10)
...@@ -232,17 +232,17 @@ class StorageTests(NEOFunctionalTest): ...@@ -232,17 +232,17 @@ class StorageTests(NEOFunctionalTest):
# stop a second storage, cluster is still running # stop a second storage, cluster is still running
started[1].stop() started[1].stop()
self.neo.expectUnavailable(started[0]) self.neo.expectDown(started[0])
self.neo.expectUnavailable(started[1]) self.neo.expectDown(started[1])
self.neo.expectRunning(started[2]) self.neo.expectRunning(started[2])
self.neo.expectOudatedCells(number=20) self.neo.expectOudatedCells(number=20)
self.neo.expectClusterRunning() self.neo.expectClusterRunning()
# stop the last, cluster died # stop the last, cluster died
started[2].stop() started[2].stop()
self.neo.expectUnavailable(started[0]) self.neo.expectDown(started[0])
self.neo.expectUnavailable(started[1]) self.neo.expectDown(started[1])
self.neo.expectUnavailable(started[2]) self.neo.expectDown(started[2])
self.neo.expectOudatedCells(number=20) self.neo.expectOudatedCells(number=20)
self.neo.expectClusterRecovering() self.neo.expectClusterRecovering()
...@@ -312,7 +312,7 @@ class StorageTests(NEOFunctionalTest): ...@@ -312,7 +312,7 @@ class StorageTests(NEOFunctionalTest):
# kill one storage, it should be set as unavailable # kill one storage, it should be set as unavailable
started[0].stop() started[0].stop()
self.neo.expectUnavailable(started[0]) self.neo.expectDown(started[0])
self.neo.expectRunning(started[1]) self.neo.expectRunning(started[1])
# and the partition table must not change # and the partition table must not change
self.neo.expectAssignedCells(started[0], 10) self.neo.expectAssignedCells(started[0], 10)
...@@ -320,7 +320,7 @@ class StorageTests(NEOFunctionalTest): ...@@ -320,7 +320,7 @@ class StorageTests(NEOFunctionalTest):
# ask neoctl to drop it # ask neoctl to drop it
self.neo.neoctl.dropNode(started[0].getUUID()) 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[0], 0)
self.neo.expectAssignedCells(started[1], 10) self.neo.expectAssignedCells(started[1], 10)
self.assertRaises(RuntimeError, self.neo.neoctl.dropNode, self.assertRaises(RuntimeError, self.neo.neoctl.dropNode,
...@@ -335,7 +335,7 @@ class StorageTests(NEOFunctionalTest): ...@@ -335,7 +335,7 @@ class StorageTests(NEOFunctionalTest):
(started, stopped) = self.__setup(storage_number=2, replicas=1, (started, stopped) = self.__setup(storage_number=2, replicas=1,
pending_number=1, partitions=10) pending_number=1, partitions=10)
self.neo.expectRunning(started[0]) self.neo.expectRunning(started[0])
self.neo.expectStorageNotKnown(stopped[0]) self.neo.expectStorageUnknown(stopped[0])
self.neo.expectOudatedCells(number=0) self.neo.expectOudatedCells(number=0)
# populate the cluster with some data # populate the cluster with some data
...@@ -362,7 +362,7 @@ class StorageTests(NEOFunctionalTest): ...@@ -362,7 +362,7 @@ class StorageTests(NEOFunctionalTest):
# kill the first storage # kill the first storage
started[0].stop() started[0].stop()
self.neo.expectUnavailable(started[0]) self.neo.expectDown(started[0])
self.neo.expectOudatedCells(number=10) self.neo.expectOudatedCells(number=10)
self.neo.expectAssignedCells(started[0], 10) self.neo.expectAssignedCells(started[0], 10)
self.neo.expectAssignedCells(stopped[0], 10) self.neo.expectAssignedCells(stopped[0], 10)
...@@ -371,7 +371,7 @@ class StorageTests(NEOFunctionalTest): ...@@ -371,7 +371,7 @@ class StorageTests(NEOFunctionalTest):
# drop it from partition table # drop it from partition table
self.neo.neoctl.dropNode(started[0].getUUID()) self.neo.neoctl.dropNode(started[0].getUUID())
self.neo.expectStorageNotKnown(started[0]) self.neo.expectStorageUnknown(started[0])
self.neo.expectRunning(stopped[0]) self.neo.expectRunning(stopped[0])
self.neo.expectAssignedCells(started[0], 0) self.neo.expectAssignedCells(started[0], 0)
self.neo.expectAssignedCells(stopped[0], 10) self.neo.expectAssignedCells(stopped[0], 10)
...@@ -395,12 +395,12 @@ class StorageTests(NEOFunctionalTest): ...@@ -395,12 +395,12 @@ class StorageTests(NEOFunctionalTest):
# drop the first then the second storage # drop the first then the second storage
started[0].stop() started[0].stop()
self.neo.expectUnavailable(started[0]) self.neo.expectDown(started[0])
self.neo.expectRunning(started[1]) self.neo.expectRunning(started[1])
self.neo.expectOudatedCells(number=10) self.neo.expectOudatedCells(number=10)
started[1].stop() started[1].stop()
self.neo.expectUnavailable(started[0]) self.neo.expectDown(started[0])
self.neo.expectUnavailable(started[1]) self.neo.expectDown(started[1])
self.neo.expectOudatedCells(number=10) self.neo.expectOudatedCells(number=10)
self.neo.expectClusterRecovering() self.neo.expectClusterRecovering()
# XXX: need to sync with storages first # XXX: need to sync with storages first
...@@ -409,7 +409,7 @@ class StorageTests(NEOFunctionalTest): ...@@ -409,7 +409,7 @@ class StorageTests(NEOFunctionalTest):
# restart the cluster with the first storage killed # restart the cluster with the first storage killed
self.neo.run(except_storages=[started[1]]) self.neo.run(except_storages=[started[1]])
self.neo.expectPending(started[0]) self.neo.expectPending(started[0])
self.neo.expectUnknown(started[1]) self.neo.expectDown(started[1])
self.neo.expectClusterRecovering() self.neo.expectClusterRecovering()
# Cluster doesn't know there are outdated cells # Cluster doesn't know there are outdated cells
self.neo.expectOudatedCells(number=0) self.neo.expectOudatedCells(number=0)
......
This diff is collapsed.
...@@ -63,29 +63,24 @@ class MasterPartitionTableTests(NeoUnitTestBase): ...@@ -63,29 +63,24 @@ class MasterPartitionTableTests(NeoUnitTestBase):
uuid4 = self.getStorageUUID() uuid4 = self.getStorageUUID()
server4 = ("127.0.0.4", 19004) server4 = ("127.0.0.4", 19004)
sn4 = self.createStorage(server4, uuid4) sn4 = self.createStorage(server4, uuid4)
uuid5 = self.getStorageUUID()
server5 = ("127.0.0.5", 19005)
sn5 = self.createStorage(server5, uuid5)
# create partition table # create partition table
num_partitions = 5 num_partitions = 4
num_replicas = 3 num_replicas = 3
pt = PartitionTable(num_partitions, num_replicas) pt = PartitionTable(num_partitions, num_replicas)
pt._setCell(0, sn1, CellStates.OUT_OF_DATE) pt._setCell(0, sn1, CellStates.OUT_OF_DATE)
sn1.setState(NodeStates.RUNNING) sn1.setState(NodeStates.RUNNING)
pt._setCell(1, sn2, CellStates.UP_TO_DATE) 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) pt._setCell(2, sn3, CellStates.UP_TO_DATE)
sn3.setState(NodeStates.DOWN) sn3.setState(NodeStates.UNKNOWN)
pt._setCell(3, sn4, CellStates.UP_TO_DATE) pt._setCell(3, sn4, CellStates.UP_TO_DATE)
sn4.setState(NodeStates.BROKEN) sn4.setState(NodeStates.RUNNING)
pt._setCell(4, sn5, CellStates.UP_TO_DATE)
sn5.setState(NodeStates.RUNNING)
# outdate nodes # outdate nodes
cells_outdated = pt.outdate() cells_outdated = pt.outdate()
self.assertEqual(len(cells_outdated), 3) self.assertEqual(len(cells_outdated), 2)
for offset, uuid, state in cells_outdated: for offset, uuid, state in cells_outdated:
self.assertTrue(offset in (1, 2, 3)) self.assertIn(offset, (1, 2))
self.assertTrue(uuid in (uuid2, uuid3, uuid4)) self.assertIn(uuid, (uuid2, uuid3))
self.assertEqual(state, CellStates.OUT_OF_DATE) self.assertEqual(state, CellStates.OUT_OF_DATE)
# check each cell # check each cell
# part 1, already outdated # part 1, already outdated
...@@ -103,15 +98,10 @@ class MasterPartitionTableTests(NeoUnitTestBase): ...@@ -103,15 +98,10 @@ class MasterPartitionTableTests(NeoUnitTestBase):
self.assertEqual(len(cells), 1) self.assertEqual(len(cells), 1)
cell = cells[0] cell = cells[0]
self.assertEqual(cell.getState(), CellStates.OUT_OF_DATE) self.assertEqual(cell.getState(), CellStates.OUT_OF_DATE)
# part 4, already outdated # part 4, remains running
cells = pt.getCellList(3) cells = pt.getCellList(3)
self.assertEqual(len(cells), 1) self.assertEqual(len(cells), 1)
cell = cells[0] cell = cells[0]
self.assertEqual(cell.getState(), CellStates.OUT_OF_DATE)
# part 5, remains running
cells = pt.getCellList(4)
self.assertEqual(len(cells), 1)
cell = cells[0]
self.assertEqual(cell.getState(), CellStates.UP_TO_DATE) self.assertEqual(cell.getState(), CellStates.UP_TO_DATE)
def test_15_dropNodeList(self): def test_15_dropNodeList(self):
...@@ -156,7 +146,7 @@ class MasterPartitionTableTests(NeoUnitTestBase): ...@@ -156,7 +146,7 @@ class MasterPartitionTableTests(NeoUnitTestBase):
uuid2 = self.getStorageUUID() uuid2 = self.getStorageUUID()
server2 = ("127.0.0.2", 19001) server2 = ("127.0.0.2", 19001)
sn2 = self.createStorage(server2, uuid2) sn2 = self.createStorage(server2, uuid2)
sn2.setState(NodeStates.TEMPORARILY_DOWN) sn2.setState(NodeStates.DOWN)
# add node without uuid # add node without uuid
server3 = ("127.0.0.3", 19001) server3 = ("127.0.0.3", 19001)
sn3 = self.createStorage(server3, None, NodeStates.RUNNING) sn3 = self.createStorage(server3, None, NodeStates.RUNNING)
......
...@@ -94,7 +94,7 @@ class MasterRecoveryTests(NeoUnitTestBase): ...@@ -94,7 +94,7 @@ class MasterRecoveryTests(NeoUnitTestBase):
conn = self.getFakeConnection(uuid, self.storage_port) conn = self.getFakeConnection(uuid, self.storage_port)
offset = 1000000 offset = 1000000
self.assertFalse(self.app.pt.hasOffset(offset)) self.assertFalse(self.app.pt.hasOffset(offset))
cell_list = [(offset, ((uuid, NodeStates.DOWN,),),)] cell_list = [(offset, ((uuid, NodeStates.UNKNOWN,),),)]
node.setPending() node.setPending()
self.checkProtocolErrorRaised(recovery.answerPartitionTable, conn, self.checkProtocolErrorRaised(recovery.answerPartitionTable, conn,
2, cell_list) 2, cell_list)
......
#
# Copyright (C) 2009-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 unittest
from .. import NeoUnitTestBase
from neo.lib.protocol import NodeTypes, BrokenNodeDisallowedError
from neo.lib.pt import PartitionTable
from neo.storage.app import Application
from neo.storage.handlers.identification import IdentificationHandler
class StorageIdentificationHandlerTests(NeoUnitTestBase):
def setUp(self):
NeoUnitTestBase.setUp(self)
config = self.getStorageConfiguration(master_number=1)
self.app = Application(config)
self.app.name = 'NEO'
self.app.operational = True
self.app.pt = PartitionTable(4, 1)
self.identification = IdentificationHandler(self.app)
def _tearDown(self, success):
self.app.close()
del self.app
super(StorageIdentificationHandlerTests, self)._tearDown(success)
def test_requestIdentification3(self):
""" broken nodes must be rejected """
uuid = self.getClientUUID()
conn = self.getFakeConnection(uuid=uuid)
node = self.app.nm.createClient(uuid=uuid)
node.setBroken()
self.assertRaises(BrokenNodeDisallowedError,
self.identification.requestIdentification,
conn,
NodeTypes.CLIENT,
uuid,
None,
self.app.name,
None,
)
if __name__ == "__main__":
unittest.main()
#
# Copyright (C) 2009-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 unittest
from . import NeoUnitTestBase
from neo.storage.app import Application
from neo.lib.bootstrap import BootstrapManager
from neo.lib.protocol import NodeTypes, Packets
class BootstrapManagerTests(NeoUnitTestBase):
def setUp(self):
NeoUnitTestBase.setUp(self)
self.prepareDatabase(number=1)
# create an application object
config = self.getStorageConfiguration()
self.app = Application(config)
self.bootstrap = BootstrapManager(self.app, NodeTypes.STORAGE)
# define some variable to simulate client and storage node
self.master_port = 10010
self.storage_port = 10020
self.num_partitions = 1009
self.num_replicas = 2
def _tearDown(self, success):
self.app.close()
del self.app
super(BootstrapManagerTests, self)._tearDown(success)
# Tests
def testConnectionCompleted(self):
address = ("127.0.0.1", self.master_port)
conn = self.getFakeConnection(address=address)
self.bootstrap.current = self.app.nm.createMaster(address=address)
self.bootstrap.connectionCompleted(conn)
self.checkAskPacket(conn, Packets.RequestIdentification)
def testHandleNotReady(self):
# the primary is not ready
address = ("127.0.0.1", self.master_port)
conn = self.getFakeConnection(address=address)
self.bootstrap.current = self.app.nm.createMaster(address=address)
self.bootstrap.notReady(conn, '')
self.checkClosed(conn)
self.checkNoPacketSent(conn)
if __name__ == "__main__":
unittest.main()
...@@ -21,7 +21,7 @@ from neo.lib import connection, logging ...@@ -21,7 +21,7 @@ from neo.lib import connection, logging
from neo.lib.connection import BaseConnection, ClientConnection, \ from neo.lib.connection import BaseConnection, ClientConnection, \
MTClientConnection, CRITICAL_TIMEOUT MTClientConnection, CRITICAL_TIMEOUT
from neo.lib.handler import EventHandler from neo.lib.handler import EventHandler
from neo.lib.protocol import Packets from neo.lib.protocol import ENCODED_VERSION, Packets
from . import NeoUnitTestBase, Patch from . import NeoUnitTestBase, Patch
...@@ -73,6 +73,7 @@ class ConnectionTests(NeoUnitTestBase): ...@@ -73,6 +73,7 @@ class ConnectionTests(NeoUnitTestBase):
# don't accept any other packet without specifying a queue. # don't accept any other packet without specifying a queue.
self.handler = EventHandler(self.app) self.handler = EventHandler(self.app)
conn = self._makeClientConnection() conn = self._makeClientConnection()
conn.read_buf.append(ENCODED_VERSION)
use_case_list = ( use_case_list = (
# (a) For a single packet sent at T, # (a) For a single packet sent at T,
......
...@@ -19,7 +19,7 @@ from .mock import Mock ...@@ -19,7 +19,7 @@ from .mock import Mock
from . import NeoUnitTestBase from . import NeoUnitTestBase
from neo.lib.handler import EventHandler from neo.lib.handler import EventHandler
from neo.lib.protocol import PacketMalformedError, UnexpectedPacketError, \ from neo.lib.protocol import PacketMalformedError, UnexpectedPacketError, \
BrokenNodeDisallowedError, NotReadyError, ProtocolError NotReadyError, ProtocolError
class HandlerTests(NeoUnitTestBase): class HandlerTests(NeoUnitTestBase):
...@@ -60,14 +60,6 @@ class HandlerTests(NeoUnitTestBase): ...@@ -60,14 +60,6 @@ class HandlerTests(NeoUnitTestBase):
self.setFakeMethod(fake) self.setFakeMethod(fake)
self.handler.dispatch(conn, packet) self.handler.dispatch(conn, packet)
self.checkClosed(conn) self.checkClosed(conn)
# raise BrokenNodeDisallowedError
conn.mockCalledMethods = {}
def fake(c):
raise BrokenNodeDisallowedError
self.setFakeMethod(fake)
self.handler.dispatch(conn, packet)
self.checkErrorPacket(conn)
self.checkAborted(conn)
# raise NotReadyError # raise NotReadyError
conn.mockCalledMethods = {} conn.mockCalledMethods = {}
def fake(c): def fake(c):
......
...@@ -35,7 +35,7 @@ class NodesTests(NeoUnitTestBase): ...@@ -35,7 +35,7 @@ class NodesTests(NeoUnitTestBase):
address = ('127.0.0.1', 10000) address = ('127.0.0.1', 10000)
uuid = self.getNewUUID(None) uuid = self.getNewUUID(None)
node = Node(self.nm, address=address, uuid=uuid) node = Node(self.nm, address=address, uuid=uuid)
self.assertEqual(node.getState(), NodeStates.UNKNOWN) self.assertEqual(node.getState(), NodeStates.DOWN)
self.assertEqual(node.getAddress(), address) self.assertEqual(node.getAddress(), address)
self.assertEqual(node.getUUID(), uuid) self.assertEqual(node.getUUID(), uuid)
self.assertTrue(time() - 1 < node.getLastStateChange() < time()) self.assertTrue(time() - 1 < node.getLastStateChange() < time())
...@@ -43,7 +43,7 @@ class NodesTests(NeoUnitTestBase): ...@@ -43,7 +43,7 @@ class NodesTests(NeoUnitTestBase):
def testState(self): def testState(self):
""" Check if the last changed time is updated when state is changed """ """ Check if the last changed time is updated when state is changed """
node = Node(self.nm) node = Node(self.nm)
self.assertEqual(node.getState(), NodeStates.UNKNOWN) self.assertEqual(node.getState(), NodeStates.DOWN)
self.assertTrue(time() - 1 < node.getLastStateChange() < time()) self.assertTrue(time() - 1 < node.getLastStateChange() < time())
previous_time = node.getLastStateChange() previous_time = node.getLastStateChange()
node.setState(NodeStates.RUNNING) node.setState(NodeStates.RUNNING)
...@@ -156,15 +156,15 @@ class NodeManagerTests(NeoUnitTestBase): ...@@ -156,15 +156,15 @@ class NodeManagerTests(NeoUnitTestBase):
old_uuid = self.storage.getUUID() old_uuid = self.storage.getUUID()
new_uuid = self.getStorageUUID() new_uuid = self.getStorageUUID()
node_list = ( 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.MASTER, new_address, self.master.getUUID(), NodeStates.RUNNING, None),
(NodeTypes.STORAGE, self.storage.getAddress(), new_uuid, (NodeTypes.STORAGE, self.storage.getAddress(), new_uuid,
NodeStates.RUNNING, None), NodeStates.RUNNING, None),
(NodeTypes.ADMIN, self.admin.getAddress(), self.admin.getUUID(), (NodeTypes.ADMIN, self.admin.getAddress(), self.admin.getUUID(),
NodeStates.UNKNOWN, None), NodeStates.DOWN, None),
) )
app = Mock() app = Mock()
app.pt = Mock() app.pt = Mock({'dropNode': True})
# update manager content # update manager content
manager.update(app, time(), node_list) manager.update(app, time(), node_list)
# - the client gets down # - the client gets down
...@@ -180,9 +180,9 @@ class NodeManagerTests(NeoUnitTestBase): ...@@ -180,9 +180,9 @@ class NodeManagerTests(NeoUnitTestBase):
new_storage = storage_list[0] new_storage = storage_list[0]
self.assertNotEqual(new_storage.getUUID(), old_uuid) self.assertNotEqual(new_storage.getUUID(), old_uuid)
self.assertEqual(new_storage.getState(), NodeStates.RUNNING) self.assertEqual(new_storage.getState(), NodeStates.RUNNING)
# admin is still here but in UNKNOWN state # admin is still here but in DOWN state
self.checkNodes([self.master, self.admin, new_storage]) self.checkNodes([self.master, self.admin, new_storage])
self.assertEqual(self.admin.getState(), NodeStates.UNKNOWN) self.assertEqual(self.admin.getState(), NodeStates.DOWN)
class MasterDBTests(NeoUnitTestBase): class MasterDBTests(NeoUnitTestBase):
......
...@@ -34,7 +34,7 @@ class PartitionTableTests(NeoUnitTestBase): ...@@ -34,7 +34,7 @@ class PartitionTableTests(NeoUnitTestBase):
# check getter # check getter
self.assertEqual(cell.getNode(), sn) self.assertEqual(cell.getNode(), sn)
self.assertEqual(cell.getState(), CellStates.OUT_OF_DATE) self.assertEqual(cell.getState(), CellStates.OUT_OF_DATE)
self.assertEqual(cell.getNodeState(), NodeStates.UNKNOWN) self.assertEqual(cell.getNodeState(), NodeStates.DOWN)
self.assertEqual(cell.getUUID(), uuid) self.assertEqual(cell.getUUID(), uuid)
self.assertEqual(cell.getAddress(), server) self.assertEqual(cell.getAddress(), server)
# check state setter # check state setter
...@@ -104,18 +104,12 @@ class PartitionTableTests(NeoUnitTestBase): ...@@ -104,18 +104,12 @@ class PartitionTableTests(NeoUnitTestBase):
else: else:
self.assertEqual(len(pt.partition_list[x]), 0) self.assertEqual(len(pt.partition_list[x]), 0)
# now add broken and down state, must not be taken into account # now add down state, must not be taken into account
pt._setCell(0, sn1, CellStates.DISCARDED) pt._setCell(0, sn1, CellStates.DISCARDED)
for x in xrange(num_partitions): for x in xrange(num_partitions):
self.assertEqual(len(pt.partition_list[x]), 0) self.assertEqual(len(pt.partition_list[x]), 0)
self.assertEqual(pt.count_dict[sn1], 0) self.assertEqual(pt.count_dict[sn1], 0)
sn1.setState(NodeStates.BROKEN) sn1.setState(NodeStates.UNKNOWN)
self.assertRaises(PartitionTableException, pt._setCell,
0, sn1, CellStates.UP_TO_DATE)
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)
self.assertRaises(PartitionTableException, pt._setCell, self.assertRaises(PartitionTableException, pt._setCell,
0, sn1, CellStates.UP_TO_DATE) 0, sn1, CellStates.UP_TO_DATE)
for x in xrange(num_partitions): for x in xrange(num_partitions):
...@@ -331,7 +325,7 @@ class PartitionTableTests(NeoUnitTestBase): ...@@ -331,7 +325,7 @@ class PartitionTableTests(NeoUnitTestBase):
self.assertFalse(pt.operational()) self.assertFalse(pt.operational())
# adding a node in all partition # adding a node in all partition
sn1 = createStorage() sn1 = createStorage()
sn1.setState(NodeStates.TEMPORARILY_DOWN) sn1.setState(NodeStates.DOWN)
for x in xrange(num_partitions): for x in xrange(num_partitions):
pt._setCell(x, sn1, CellStates.FEEDING) pt._setCell(x, sn1, CellStates.FEEDING)
self.assertTrue(pt.filled()) self.assertTrue(pt.filled())
......
...@@ -35,7 +35,7 @@ from neo.lib.connection import BaseConnection, \ ...@@ -35,7 +35,7 @@ from neo.lib.connection import BaseConnection, \
from neo.lib.connector import SocketConnector, ConnectorException from neo.lib.connector import SocketConnector, ConnectorException
from neo.lib.handler import EventHandler from neo.lib.handler import EventHandler
from neo.lib.locking import SimpleQueue from neo.lib.locking import SimpleQueue
from neo.lib.protocol import ClusterStates, NodeStates, NodeTypes, Packets from neo.lib.protocol import ClusterStates, Enum, NodeStates, NodeTypes, Packets
from neo.lib.util import cached_property, parseMasterList, p64 from neo.lib.util import cached_property, parseMasterList, p64
from .. import NeoTestBase, Patch, getTempDirectory, setupMySQLdb, \ from .. import NeoTestBase, Patch, getTempDirectory, setupMySQLdb, \
ADDRESS_TYPE, IP_VERSION_FORMAT_DICT, DB_PREFIX, DB_SOCKET, DB_USER ADDRESS_TYPE, IP_VERSION_FORMAT_DICT, DB_PREFIX, DB_SOCKET, DB_USER
...@@ -336,7 +336,7 @@ class ServerNode(Node): ...@@ -336,7 +336,7 @@ class ServerNode(Node):
self.daemon = True self.daemon = True
self.node_name = '%s_%u' % (self.node_type, port) self.node_name = '%s_%u' % (self.node_type, port)
kw.update(getCluster=name, getBind=address, kw.update(getCluster=name, getBind=address,
getMasters=master_nodes and parseMasterList(master_nodes, address)) getMasters=master_nodes and parseMasterList(master_nodes))
super(ServerNode, self).__init__(Mock(kw)) super(ServerNode, self).__init__(Mock(kw))
def getVirtualAddress(self): def getVirtualAddress(self):
...@@ -745,27 +745,39 @@ class NEOCluster(object): ...@@ -745,27 +745,39 @@ class NEOCluster(object):
def __exit__(self, t, v, tb): def __exit__(self, t, v, tb):
self.stop(None) self.stop(None)
def start(self, storage_list=None, fast_startup=False): def start(self, storage_list=None, master_list=None, recovering=False):
self.started = True self.started = True
self._patch() self._patch()
self.neoctl = NeoCTL(self.admin.getVirtualAddress(), ssl=self.SSL) self.neoctl = NeoCTL(self.admin.getVirtualAddress(), ssl=self.SSL)
for node_type in 'master', 'admin': for node in self.master_list if master_list is None else master_list:
for node in getattr(self, node_type + '_list'): node.start()
node.start() for node in self.admin_list:
node.start()
Serialized.tic() Serialized.tic()
if fast_startup:
self.startCluster()
if storage_list is None: if storage_list is None:
storage_list = self.storage_list storage_list = self.storage_list
for node in storage_list: for node in storage_list:
node.start() node.start()
Serialized.tic() Serialized.tic()
if not fast_startup: if recovering:
expected_state = ClusterStates.RECOVERING
else:
self.startCluster() self.startCluster()
Serialized.tic() Serialized.tic()
expected_state = ClusterStates.RUNNING, ClusterStates.BACKINGUP
self.checkStarted(expected_state, storage_list)
def checkStarted(self, expected_state, storage_list=None):
if isinstance(expected_state, Enum.Item):
expected_state = expected_state,
state = self.neoctl.getClusterState() state = self.neoctl.getClusterState()
assert state in (ClusterStates.RUNNING, ClusterStates.BACKINGUP), state assert state in expected_state, state
self.enableStorageList(storage_list) expected_state = (NodeStates.PENDING
if state == ClusterStates.RECOVERING
else NodeStates.RUNNING)
for node in self.storage_list if storage_list is None else storage_list:
state = self.getNodeState(node)
assert state == expected_state, (node, state)
def stop(self, clear_database=False, __print_exc=traceback.print_exc, **kw): def stop(self, clear_database=False, __print_exc=traceback.print_exc, **kw):
if self.started: if self.started:
......
...@@ -34,8 +34,8 @@ from neo.lib.connection import ConnectionClosed, \ ...@@ -34,8 +34,8 @@ from neo.lib.connection import ConnectionClosed, \
from neo.lib.exception import DatabaseFailure, StoppedOperation from neo.lib.exception import DatabaseFailure, StoppedOperation
from neo.lib.handler import DelayEvent from neo.lib.handler import DelayEvent
from neo.lib import logging from neo.lib import logging
from neo.lib.protocol import CellStates, ClusterStates, NodeStates, Packets, \ from neo.lib.protocol import (CellStates, ClusterStates, NodeStates, NodeTypes,
Packet, uuid_str, ZERO_OID, ZERO_TID Packets, Packet, uuid_str, ZERO_OID, ZERO_TID)
from .. import expectedFailure, Patch, TransactionalResource from .. import expectedFailure, Patch, TransactionalResource
from . import ClientApplication, ConnectionFilter, LockLock, NEOThreadedTest, \ from . import ClientApplication, ConnectionFilter, LockLock, NEOThreadedTest, \
RandomConflictDict, ThreadId, with_cluster RandomConflictDict, ThreadId, with_cluster
...@@ -552,7 +552,8 @@ class Test(NEOThreadedTest): ...@@ -552,7 +552,8 @@ class Test(NEOThreadedTest):
# restart it with one storage only # restart it with one storage only
if 1: if 1:
cluster.start(storage_list=(s1,)) cluster.start(storage_list=(s1,))
self.assertEqual(NodeStates.UNKNOWN, cluster.getNodeState(s2)) self.assertEqual(NodeStates.DOWN,
cluster.getNodeState(s2))
@with_cluster(storage_count=2, partitions=2, replicas=1) @with_cluster(storage_count=2, partitions=2, replicas=1)
def testRestartStoragesWithReplicas(self, cluster): def testRestartStoragesWithReplicas(self, cluster):
...@@ -837,12 +838,6 @@ class Test(NEOThreadedTest): ...@@ -837,12 +838,6 @@ class Test(NEOThreadedTest):
@with_cluster(master_count=3, partitions=10, replicas=1, storage_count=3) @with_cluster(master_count=3, partitions=10, replicas=1, storage_count=3)
def testShutdown(self, cluster): def testShutdown(self, cluster):
# BUG: Due to bugs in election, master nodes sometimes crash, or they
# declare themselves primary too quickly, but issues seem to be
# only reproducible with SSL enabled.
self._testShutdown(cluster)
def _testShutdown(self, cluster):
def before_finish(_): def before_finish(_):
# tell admin to shutdown the cluster # tell admin to shutdown the cluster
cluster.neoctl.setClusterState(ClusterStates.STOPPING) cluster.neoctl.setClusterState(ClusterStates.STOPPING)
...@@ -1225,12 +1220,10 @@ class Test(NEOThreadedTest): ...@@ -1225,12 +1220,10 @@ class Test(NEOThreadedTest):
@with_cluster(start_cluster=0, storage_count=3, autostart=3) @with_cluster(start_cluster=0, storage_count=3, autostart=3)
def testAutostart(self, cluster): def testAutostart(self, cluster):
def startCluster(orig): cluster.start(cluster.storage_list[:2], recovering=True)
getClusterState = cluster.neoctl.getClusterState cluster.storage_list[2].start()
self.assertEqual(ClusterStates.RECOVERING, getClusterState()) self.tic()
cluster.storage_list[2].start() cluster.checkStarted(ClusterStates.RUNNING)
with Patch(cluster, startCluster=startCluster):
cluster.start(cluster.storage_list[:2])
@with_cluster(storage_count=2, partitions=2) @with_cluster(storage_count=2, partitions=2)
def testAbortVotedTransaction(self, cluster): def testAbortVotedTransaction(self, cluster):
...@@ -2219,6 +2212,60 @@ class Test(NEOThreadedTest): ...@@ -2219,6 +2212,60 @@ class Test(NEOThreadedTest):
def testConflictAfterDeadlockWithSlowReplica2(self): def testConflictAfterDeadlockWithSlowReplica2(self):
self.testConflictAfterDeadlockWithSlowReplica1(True) self.testConflictAfterDeadlockWithSlowReplica1(True)
@with_cluster(start_cluster=0, master_count=3)
def testElection(self, cluster):
m0, m1, m2 = cluster.master_list
cluster.start(master_list=(m0,), recovering=True)
getClusterState = cluster.neoctl.getClusterState
m0.em.removeReader(m0.listening_conn)
m1.start()
self.tic()
m2.start()
self.tic()
self.assertTrue(m0.primary)
self.assertTrue(m1.primary)
self.assertFalse(m2.primary)
m0.em.addReader(m0.listening_conn)
with ConnectionFilter() as f:
f.delayAcceptIdentification()
self.tic()
self.tic()
self.assertTrue(m0.primary)
self.assertFalse(m1.primary)
self.assertFalse(m2.primary)
self.assertEqual(getClusterState(), ClusterStates.RECOVERING)
cluster.startCluster()
def stop(node):
node.stop()
cluster.join((node,))
node.resetNode()
stop(m1)
self.tic()
self.assertEqual(getClusterState(), ClusterStates.RUNNING)
self.assertTrue(m0.primary)
self.assertFalse(m2.primary)
stop(m0)
self.tic()
self.assertEqual(getClusterState(), ClusterStates.RUNNING)
self.assertTrue(m2.primary)
# Check for proper update of node ids on first NotifyNodeInformation.
stop(m2)
m0.start()
def update(orig, app, timestamp, node_list):
orig(app, timestamp, sorted(node_list, reverse=1))
with Patch(cluster.storage.nm, update=update):
with ConnectionFilter() as f:
f.add(lambda conn, packet:
isinstance(packet, Packets.RequestIdentification)
and packet.decode()[0] == NodeTypes.STORAGE)
self.tic()
m2.start()
self.tic()
self.tic()
self.assertEqual(getClusterState(), ClusterStates.RUNNING)
self.assertTrue(m0.primary)
self.assertFalse(m2.primary)
if __name__ == "__main__": if __name__ == "__main__":
unittest.main() unittest.main()
...@@ -17,7 +17,7 @@ ...@@ -17,7 +17,7 @@
import unittest import unittest
from neo.lib.protocol import Packets from neo.lib.protocol import Packets
from .. import SSL from .. import SSL
from . import NEOCluster, with_cluster, test, testReplication from . import NEOCluster, test, testReplication
class SSLMixin: class SSLMixin:
...@@ -36,14 +36,6 @@ class SSLTests(SSLMixin, test.Test): ...@@ -36,14 +36,6 @@ class SSLTests(SSLMixin, test.Test):
testDeadlockAvoidance = None testDeadlockAvoidance = None
testUndoConflict = testUndoConflictDuringStore = None testUndoConflict = testUndoConflictDuringStore = None
if 1:
testShutdownWithSeveralMasterNodes = unittest.skip("fails randomly")(
test.Test.testShutdown.__func__)
@with_cluster(partitions=10, replicas=1, storage_count=3)
def testShutdown(self, cluster):
self._testShutdown(cluster)
def testAbortConnection(self, after_handshake=1): def testAbortConnection(self, after_handshake=1):
with self.getLoopbackConnection() as conn: with self.getLoopbackConnection() as conn:
conn.ask(Packets.Ping()) conn.ask(Packets.Ping())
......
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