Commit 460cf445 authored by Yoshinori Okuji's avatar Yoshinori Okuji

Rewrite step one

git-svn-id: https://svn.erp5.org/repos/neo/branches/prototype3@22 71dcc9de-d417-0410-9af5-da40c76e7ee4
parent eab036aa
This diff is collapsed.
import logging
from select import select
from time import time
class IdleEvent(object):
"""This class represents an event called when a connection is waiting for
a message too long."""
def __init__(self, conn, msg_id, timeout, additional_timeout):
self._conn = conn
self._id = msg_id
t = time()
self._time = t + timeout
self._critical_time = t + timeout + additional_timeout
self._additional_timeout = additional_timeout
def getId(self):
return self._id
def getTime(self):
return self._time
def getCriticalTime(self):
return self._critical_time
def __call__(self, t):
conn = self._conn
if t > self._critical_time:
logging.info('timeout with %s:%d', *(conn.getAddress()))
conn.getHandler().timeoutExpired(conn)
conn.close()
return True
elif t > self._time:
if self._additional_timeout > 5:
self._additional_timeout -= 5
conn.expectMessage(self._id, 5, self._additional_timeout)
# Start a keep-alive packet.
logging.info('sending a ping to %s:%d', *(conn.getAddress()))
msg_id = conn.getNextId()
conn.addPacket(Packet().ping(msg_id))
conn.expectMessage(msg_id, 5, 0)
else:
conn.expectMessage(self._id, self._additional_timeout, 0)
return True
return False
class EventManager(object):
"""This class manages connections and events."""
def __init__(self):
self.connection_dict = {}
self.reader_set = set([])
self.writer_set = set([])
self.exc_list = []
self.event_list = []
self.prev_time = time()
def getConnectionList(self):
return self.connection_dict.values()
def register(self, conn):
self.connection_dict[conn.getSocket()] = conn
def unregister(self, conn):
del self.connection_dict[conn.getSocket()]
def poll(self, timeout = 1):
rlist, wlist, xlist = select(self.reader_set, self.writer_set, self.exc_list,
timeout)
for s in rlist:
conn = self.connection_dict[s]
conn.readable()
for s in wlist:
conn = self.connection_dict[s]
conn.writable()
# Check idle events. Do not check them out too often, because this
# is somehow heavy.
event_list = self.event_list
if event_list:
t = time()
if t - self.prev_time >= 1:
self.prev_time = t
event_list.sort(key = lambda event: event.getTime())
for event in tuple(event_list):
if event(t):
event_list.pop(0)
else:
break
def addIdleEvent(self, event):
self.event_list.append(event)
def removeIdleEvent(self, event):
try:
self.event_list.remove(event)
except ValueError:
pass
def addReader(self, conn):
self.reader_set.add(conn.getSocket())
def removeReader(self, conn):
self.reader_set.discard(conn.getSocket())
def addWriter(self, conn):
self.writer_set.add(conn.getSocket())
def removeWriter(self, conn):
self.writer_set.discard(conn.getSocket())
import logging
from protocol import Packet, ProtocolError
from connection import ServerConnection
from protocol import ERROR, REQUEST_NODE_IDENTIFICATION, ACCEPT_NODE_IDENTIFICATION, \
PING, PONG, ASK_PRIMARY_MASTER, ANSWER_PRIMARY_MASTER, ANNOUNCE_PRIMARY_MASTER, \
REELECT_PRIMARY_MASTER, NOTIFY_NODE_INFORMATION, START_OPERATION, \
STOP_OPERATION, ASK_FINISHING_TRANSACTIONS, ANSWER_FINISHING_TRANSACTIONS, \
FINISH_TRANSACTIONS, \
NOT_READY_CODE, OID_NOT_FOUND_CODE, SERIAL_NOT_FOUND_CODE, TID_NOT_FOUND_CODE, \
PROTOCOL_ERROR_CODE, TIMEOUT_ERROR_CODE, BROKEN_NODE_DISALLOWED_CODE, \
INTERNAL_ERROR_CODE
class EventHandler(object):
"""This class handles events."""
def __init__(self):
self.initPacketDispatchTable()
self.initErrorDispatchTable()
def connectionStarted(self, conn):
"""Called when a connection is started."""
pass
def connectionCompleted(self, conn):
"""Called when a connection is completed."""
pass
def connectionFailed(self, conn):
"""Called when a connection failed."""
pass
def connectionAccepted(self, conn, s, addr):
"""Called when a connection is accepted."""
new_conn = ServerConnection(conn.getEventManager(), conn.getHandler(),
s = s, addr = addr)
# A request for a node identification should arrive.
new_conn.expectMessage(timeout = 10, additional_timeout = 0)
def timeoutExpired(self, conn):
"""Called when a timeout event occurs."""
pass
def connectionClosed(self, conn):
"""Called when a connection is closed by the peer."""
pass
def packetReceived(self, conn, packet):
"""Called when a packet is received."""
self.dispatch(conn, packet)
def packetMalformed(self, conn, packet, error_message):
"""Called when a packet is malformed."""
logging.info('malformed packet: %s', error_message)
conn.addPacket(Packet().protocolError(packet.getId(), error_message))
conn.abort()
self.peerBroken(conn)
def peerBroken(self, conn):
"""Called when a peer is broken."""
logging.error('%s:%d is broken', *(conn.getAddress()))
def dispatch(self, conn, packet):
"""This is a helper method to handle various packet types."""
t = packet.getType()
try:
method = self.packet_dispatch_table[t]
args = packet.decode()
method(conn, packet, *args)
except ValueError:
self.handleUnexpectedPacket(conn, packet)
except ProtocolError, m:
self.packetMalformed(conn, packet, m[1])
def handleUnexpectedPacket(self, conn, packet, message = None):
"""Handle an unexpected packet."""
if message is None:
message = 'unexpected packet type %d' % packet.getType()
else:
message = 'unexpected packet: ' + message
logging.info('%s', message)
conn.addPacket(Packet().protocolError(packet.getId(), message))
conn.abort()
self.peerBroken(conn)
# Packet handlers.
def handleError(self, conn, packet, code, message):
try:
method = self.error_dispatch_table[code]
method(conn, packet, message)
except ValueError:
self.handleUnexpectedPacket(conn, packet, message)
def handleRequestNodeIdentification(self, conn, packet, node_type,
uuid, ip_address, port, name):
self.handleUnexpectedPacket(conn, packet)
def handleAcceptNodeIdentification(self, conn, packet, node_type,
uuid, ip_address, port):
self.handleUnexpectedPacket(conn, packet)
def handlePing(self, conn, packet):
logging.info('got a ping packet; am I overloaded?')
conn.addPacket(Packet().pong(packet.getId()))
def handlePong(self, conn, packet):
pass
def handleAskPrimaryNode(self, conn, packet):
self.handleUnexpectedPacket(conn, packet)
def handleAnswerPrimaryNode(self, conn, packet, primary_uuid, known_master_list):
self.handleUnexpectedPacket(conn, packet)
def handleAnnouncePrimaryMaster(self, conn, packet):
self.handleUnexpectedPacket(conn, packet)
def handleReelectPrimaryMaster(self, conn, packet):
self.handleUnexpectedPacket(conn, packet)
def handleNotifyNodeInformation(self, conn, packet, node_list):
self.handleUnexpectedPacket(conn, packet)
# Error packet handlers.
handleNotReady = handleUnexpectedPacket
handleOidNotFound = handleUnexpectedPacket
handleSerialNotFound = handleUnexpectedPacket
handleTidNotFound = handleUnexpectedPacket
def handleProtocolError(self, conn, packet, message):
raise RuntimeError, 'protocol error: %s' % (message,)
def handleTimeoutError(self, conn, packet, message):
raise RuntimeError, 'timeout error: %s' % (message,)
def handleBrokenNodeDisallowedError(self, conn, packet, message):
raise RuntimeError, 'broken node disallowed error: %s' % (message,)
def handleInternalError(self, conn, packet, message):
self.peerBroken(conn)
conn.close()
def initPacketDispatchTable(self):
d = {}
d[ERROR] = self.handleError
d[REQUEST_NODE_IDENTIFICATION] = self.handleRequestNodeIdentification
d[ACCEPT_NODE_IDENTIFICATION] = self.handleAcceptNodeIdentification
d[PING] = self.handlePing
d[PONG] = self.handlePong
d[ASK_PRIMARY_MASTER] = self.handleAskPrimaryMaster
d[ANSWER_PRIMARY_MASTER] = self.handleAnswerPrimaryMaster
d[ANNOUNCE_PRIMARY_MASTER] = self.handleAnnouncePrimaryMaster
d[REELECT_PRIMARY_MASTER] = self.handleReelectPrimaryMaster
d[NOTIFY_NODE_INFORMATION] = self.handleNotifyNodeInformation
self.packet_dispatch_table = d
def initErrorDispatchTable(self):
d = {}
d[NOT_READY_CODE] = self.handleNotReady
d[OID_NOT_FOUND_CODE] = self.handleOidNotFound
d[SERIAL_NOT_FOUND_CODE] = self.handleSerialNotFound
d[TID_NOT_FOUND_CODE] = self.handleTidNotFound
d[PROTOCOL_ERROR_CODE] = self.handleProtocolError
d[TIMEOUT_ERROR_CODE] = self.handleTimeoutError
d[BROKEN_NODE_DISALLOWED_CODE] = self.handleBrokenNodeDisallowedError
d[INTERNAL_ERROR_CODE] = self.handleInternalError
self.error_dispatch_table = d
This diff is collapsed.
# Default parameters.
[DEFAULT] [DEFAULT]
# The list of master nodes.
master_nodes: 127.0.0.1:10010 127.0.0.1:10011 127.0.0.1:10012 master_nodes: 127.0.0.1:10010 127.0.0.1:10011 127.0.0.1:10012
#replicas: 1 # The number of replicas.
#partitions: 1009 replicas: 1
#name: main # The number of partitions.
partitions: 1009
# The name of this cluster.
name: main
# The user name for the database.
user: neo
# The password for the database.
password: neo
# The first master.
[master1] [master1]
database: master1
user: neo
#password:
server: 127.0.0.1:10010 server: 127.0.0.1:10010
# The second master.
[master2] [master2]
database: master2
user: neo
server: 127.0.0.1:10011 server: 127.0.0.1:10011
# The third master.
[master3] [master3]
database: master3
user: neo
server: 127.0.0.1:10012 server: 127.0.0.1:10012
# The first storage.
[storage1]
database: neo1
server: 127.0.0.1:10020
...@@ -19,15 +19,13 @@ ...@@ -19,15 +19,13 @@
# Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. # Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
from optparse import OptionParser from optparse import OptionParser
from master import Application from master.app import Application
import logging import logging
# FIXME should be configurable
logging.basicConfig(level = logging.DEBUG)
parser = OptionParser() parser = OptionParser()
parser.add_option('-i', '--initialize', action = 'store_true', parser.add_option('-v', '--verbose', action = 'store_true',
help = 'initialize the database') help = 'print verbose messages')
parser.add_option('-c', '--config', help = 'specify a configuration file') parser.add_option('-c', '--config', help = 'specify a configuration file')
parser.add_option('-s', '--section', help = 'specify a configuration section') parser.add_option('-s', '--section', help = 'specify a configuration section')
...@@ -36,9 +34,10 @@ parser.add_option('-s', '--section', help = 'specify a configuration section') ...@@ -36,9 +34,10 @@ parser.add_option('-s', '--section', help = 'specify a configuration section')
config = options.config or 'neo.conf' config = options.config or 'neo.conf'
section = options.section or 'master' section = options.section or 'master'
app = Application(config, section) if options.verbose:
logging.basicConfig(level = logging.DEBUG)
if options.initialize: else:
app.initializeDatabase() logging.basicConfig(level = logging.ERROR)
app = Application(config, section)
app.run() app.run()
from time import time from time import time
from protocol import RUNNING_STATE, TEMPORARILY_DOWN_STATE, DOWN_STATE, BROKEN_STATE from protocol import RUNNING_STATE, TEMPORARILY_DOWN_STATE, DOWN_STATE, BROKEN_STATE, \
MASTER_NODE_TYPE, STORAGE_NODE_TYPE, CLIENT_NODE_TYPE
class Node(object): class Node(object):
"""This class represents a node.""" """This class represents a node."""
def __init__(self, ip_address = None, port = None, uuid = None): def __init__(self, server = None, uuid = None):
self.state = RUNNING_STATE self.state = RUNNING_STATE
self.ip_address = ip_address self.server = server
self.port = port
self.uuid = uuid self.uuid = uuid
self.manager = None self.manager = None
self.last_state_change = time() self.last_state_change = time()
...@@ -27,16 +27,15 @@ class Node(object): ...@@ -27,16 +27,15 @@ class Node(object):
self.state = new_state self.state = new_state
self.last_state_change = time() self.last_state_change = time()
def setServer(self, ip_address, port): def setServer(self, server):
if self.ip_address is not None: if self.server is not None:
self.manager.unregisterServer(self) self.manager.unregisterServer(self)
self.ip_address = ip_address self.server = server
self.port = port
self.manager.registerServer(self) self.manager.registerServer(self)
def getServer(self): def getServer(self):
return self.ip_address, self.port return self.server
def setUUID(self, uuid): def setUUID(self, uuid):
if self.uuid is not None: if self.uuid is not None:
...@@ -48,17 +47,23 @@ class Node(object): ...@@ -48,17 +47,23 @@ class Node(object):
def getUUID(self): def getUUID(self):
return self.uuid return self.uuid
def getNodeType(self):
raise NotImplementedError
class MasterNode(Node): class MasterNode(Node):
"""This class represents a master node.""" """This class represents a master node."""
pass def getNodeType(self):
return MASTER_NODE_TYPE
class StorageNode(Node): class StorageNode(Node):
"""This class represents a storage node.""" """This class represents a storage node."""
pass def getNodeType(self):
return STORAGE_NODE_TYPE
class ClientNode(Node): class ClientNode(Node):
"""This class represents a client node.""" """This class represents a client node."""
pass def getNodeType(self):
return CLIENT_NODE_TYPE
class NodeManager(object): class NodeManager(object):
"""This class manages node status.""" """This class manages node status."""
...@@ -71,7 +76,7 @@ class NodeManager(object): ...@@ -71,7 +76,7 @@ class NodeManager(object):
def add(self, node): def add(self, node):
node.setManager(self) node.setManager(self)
self.node_list.append(node) self.node_list.append(node)
if node.getServer()[0] is not None: if node.getServer() is not None:
self.registerServer(node) self.registerServer(node)
if node.getUUID() is not None: if node.getUUID() is not None:
self.registerUUID(node) self.registerUUID(node)
...@@ -113,8 +118,8 @@ class NodeManager(object): ...@@ -113,8 +118,8 @@ class NodeManager(object):
def getClientNodeList(self): def getClientNodeList(self):
return self.getNodeList(filter = lambda node: isinstance(node, ClientNode)) return self.getNodeList(filter = lambda node: isinstance(node, ClientNode))
def getNodeByServer(self, ip_address, port): def getNodeByServer(self, server):
return self.server_dict.get((ip_address, port)) return self.server_dict.get(server)
def getNodeByUUID(self, uuid): def getNodeByUUID(self, uuid):
return self.uuid_dict.get(uuid) return self.uuid_dict.get(uuid)
...@@ -21,14 +21,12 @@ ASK_PRIMARY_MASTER = 0x0003 ...@@ -21,14 +21,12 @@ ASK_PRIMARY_MASTER = 0x0003
ANSWER_PRIMARY_MASTER = 0x8003 ANSWER_PRIMARY_MASTER = 0x8003
ANNOUNCE_PRIMARY_MASTER = 0x0004 ANNOUNCE_PRIMARY_MASTER = 0x0004
REELECT_PRIMARY_MASTER = 0x0005 REELECT_PRIMARY_MASTER = 0x0005
NOTIFY_NODE_STATE_CHANGE = 0x0006 NOTIFY_NODE_INFORMATION = 0x0006
SEND_NODE_INFORMATION = 0x0007 START_OPERATION = 0x0007
START_OPERATION = 0x0008 STOP_OPERATION = 0x0008
STOP_OPERATION = 0x0009 ASK_FINISHING_TRANSACTIONS = 0x0009
ASK_FINISHING_TRANSACTIONS = 0x000a ANSWER_FINISHING_TRANSACTIONS = 0x8009
ANSWER_FINISHING_TRANSACTIONS = 0x800a FINISH_TRANSACTIONS = 0x000a
FINISH_TRANSACTIONS = 0x000b
# Error codes. # Error codes.
NOT_READY_CODE = 1 NOT_READY_CODE = 1
...@@ -63,7 +61,7 @@ INVALID_OID = '\0\0\0\0\0\0\0\0' ...@@ -63,7 +61,7 @@ INVALID_OID = '\0\0\0\0\0\0\0\0'
class ProtocolError(Exception): pass class ProtocolError(Exception): pass
class Packet: class Packet(object):
"""A packet.""" """A packet."""
_id = None _id = None
...@@ -152,16 +150,16 @@ class Packet: ...@@ -152,16 +150,16 @@ class Packet:
self._body = pack('!H16s4sH', node_type, uuid, inet_aton(ip_address), port) self._body = pack('!H16s4sH', node_type, uuid, inet_aton(ip_address), port)
return self return self
def askPrimaryMaster(self, msg_id, ltid, loid): def askPrimaryMaster(self, msg_id):
self._id = msg_id self._id = msg_id
self._type = ASK_PRIMARY_MASTER self._type = ASK_PRIMARY_MASTER
self._body = ltid + loid self._body = ''
return self return self
def answerPrimaryMaster(self, msg_id, ltid, loid, primary_uuid, known_master_list): def answerPrimaryMaster(self, msg_id, primary_uuid, known_master_list):
self._id = msg_id self._id = msg_id
self._type = ANSWER_PRIMARY_MASTER self._type = ANSWER_PRIMARY_MASTER
body = [ltid, loid, primary_uuid, pack('!L', len(known_master_list))] body = [primary_uuid, pack('!L', len(known_master_list))]
for master in known_master_list: for master in known_master_list:
body.append(pack('!4sH16s', inet_aton(master[0]), master[1], master[2])) body.append(pack('!4sH16s', inet_aton(master[0]), master[1], master[2]))
self._body = ''.join(body) self._body = ''.join(body)
...@@ -179,21 +177,9 @@ class Packet: ...@@ -179,21 +177,9 @@ class Packet:
self._body = '' self._body = ''
return self return self
def notifyNodeStateChange(self, msg_id, node_type, ip_address, port, uuid, state): def notifyNodeInformation(self, msg_id, node_list):
self._id = msg_id
self._type = NOTIFY_NODE_STATE_CHANGE
self._body = pack('!H4sH16sH', node_type, inet_aton(ip_address), port, uuid, state)
return self
def askNodeInformation(self, msg_id):
self._id = msg_id
self._type = ASK_NODE_INFORMATION
self._body = ''
return self
def answerNodeInformation(self, msg_id, node_list):
self._id = msg_id self._id = msg_id
self._type = ANSWER_NODE_INFORMATION self._type = NOTIFY_NODE_INFORMATION
body = [pack('!L', len(node_list))] body = [pack('!L', len(node_list))]
for node_type, ip_address, port, uuid, state in node_list: for node_type, ip_address, port, uuid, state in node_list:
body.append(pack('!H4sH16sH', node_type, inet_aton(ip_address), port, body.append(pack('!H4sH16sH', node_type, inet_aton(ip_address), port,
...@@ -261,16 +247,12 @@ class Packet: ...@@ -261,16 +247,12 @@ class Packet:
decode_table[ACCEPT_NODE_IDENTIFICATION] = _decodeAcceptNodeIdentification decode_table[ACCEPT_NODE_IDENTIFICATION] = _decodeAcceptNodeIdentification
def _decodeAskPrimaryMaster(self): def _decodeAskPrimaryMaster(self):
try: pass
ltid, loid = unpack('!8s8s', self._body)
except:
raise ProtocolError(self, 'invalid ask primary master')
return ltid, loid
decode_table[ASK_PRIMARY_MASTER] = _decodeAskPrimaryMaster decode_table[ASK_PRIMARY_MASTER] = _decodeAskPrimaryMaster
def _decodeAnswerPrimaryMaster(self): def _decodeAnswerPrimaryMaster(self):
try: try:
ltid, loid, primary_uuid, n = unpack('!8s8s16sL', self._body[:36]) primary_uuid, n = unpack('!16sL', self._body[:36])
known_master_list = [] known_master_list = []
for i in xrange(n): for i in xrange(n):
ip_address, port, uuid = unpack('!4sH16s', self._body[36+i*22:58+i*22]) ip_address, port, uuid = unpack('!4sH16s', self._body[36+i*22:58+i*22])
...@@ -278,7 +260,7 @@ class Packet: ...@@ -278,7 +260,7 @@ class Packet:
known_master_list.append((ip_address, port, uuid)) known_master_list.append((ip_address, port, uuid))
except: except:
raise ProtocolError(self, 'invalid answer primary master') raise ProtocolError(self, 'invalid answer primary master')
return ltid, loid, primary_uuid, known_master_list return primary_uuid, known_master_list
decode_table[ANSWER_PRIMARY_MASTER] = _decodeAnswerPrimaryMaster decode_table[ANSWER_PRIMARY_MASTER] = _decodeAnswerPrimaryMaster
def _decodeAnnouncePrimaryMaster(self): def _decodeAnnouncePrimaryMaster(self):
...@@ -289,24 +271,7 @@ class Packet: ...@@ -289,24 +271,7 @@ class Packet:
pass pass
decode_table[REELECT_PRIMARY_MASTER] = _decodeReelectPrimaryMaster decode_table[REELECT_PRIMARY_MASTER] = _decodeReelectPrimaryMaster
def _decodeNotifyNodeStateChange(self): def _decodeNotifyNodeInformation(self):
try:
node_type, ip_address, port, uuid, state = unpack('!H4sH16sH', self._body[:26])
ip_address = inet_ntoa(ip_address)
except:
raise ProtocolError(self, 'invalid notify node state change')
if node_type not in VALID_NODE_TYPE_LIST:
raise ProtocolError(self, 'invalid node type %d' % node_type)
if state not in VALID_NODE_STATE_LIST:
raise ProtocolError(self, 'invalid node state %d' % state)
return node_type, ip_address, port, uuid, state
decode_table[NOTIFY_NODE_STATE_CHANGE] = _decodeNotifyNodeStateChange
def _decodeAskNodeInformation(self):
pass
decode_table[ASK_NODE_INFORMATION] = _decodeAskNodeInformation
def _decodeAnswerNodeInformation(self):
try: try:
n = unpack('!L', self._body[:4]) n = unpack('!L', self._body[:4])
node_list = [] node_list = []
...@@ -324,4 +289,4 @@ class Packet: ...@@ -324,4 +289,4 @@ class Packet:
except: except:
raise ProtocolError(self, 'invalid answer node information') raise ProtocolError(self, 'invalid answer node information')
return node_list return node_list
decode_table[ANSWER_NODE_INFORMATION] = _decodeAnswerNodeInformation decode_table[NOTIFY_NODE_INFORMATION] = _decodeNotifyNodeInformation
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