Commit 156710c2 authored by Aurel's avatar Aurel

revert previous commit


git-svn-id: https://svn.erp5.org/repos/neo/branches/prototype3@538 71dcc9de-d417-0410-9af5-da40c76e7ee4
parent e93fc41d
...@@ -22,102 +22,131 @@ from struct import unpack ...@@ -22,102 +22,131 @@ from struct import unpack
from collections import deque from collections import deque
from neo.config import ConfigurationManager from neo.config import ConfigurationManager
from neo.protocol import Packet, ProtocolError, node_types, node_states
from neo.protocol import TEMPORARILY_DOWN_STATE, DOWN_STATE, BROKEN_STATE, \ from neo.protocol import TEMPORARILY_DOWN_STATE, DOWN_STATE, BROKEN_STATE, \
INVALID_UUID, INVALID_PTID, partition_cell_states, MASTER_NODE_TYPE INVALID_UUID, INVALID_PTID, partition_cell_states, MASTER_NODE_TYPE
from neo.event import EventManager
from neo.node import NodeManager, MasterNode, StorageNode, ClientNode, AdminNode from neo.node import NodeManager, MasterNode, StorageNode, ClientNode, AdminNode
from neo.connection import ClientConnection from neo.event import EventManager
from neo.connection import ListeningConnection, ClientConnection
from neo.exception import OperationFailure, PrimaryFailure from neo.exception import OperationFailure, PrimaryFailure
from neo.neoctl.handler import CommandEventHandler from neo.admin.handler import MonitoringEventHandler, AdminEventHandler
from neo.connector import getConnectorHandler from neo.connector import getConnectorHandler
from neo.util import bin, dump
from neo import protocol
class Application(object): class Application(object):
"""The storage node application.""" """The storage node application."""
def __init__(self, ip, port, handler): def __init__(self, file, section):
config = ConfigurationManager(file, section)
self.num_partitions = None
self.num_replicas = None
self.name = config.getName()
logging.debug('the name is %s', self.name)
self.connector_handler = getConnectorHandler(config.getConnector())
self.server = config.getServer()
logging.debug('IP address is %s, port is %d', *(self.server))
self.master_node_list = config.getMasterNodeList()
logging.debug('master nodes are %s', self.master_node_list)
self.connector_handler = getConnectorHandler(handler) # Internal attributes.
self.server = (ip, port)
self.em = EventManager() self.em = EventManager()
self.nm = NodeManager()
# The partition table is initialized after getting the number of
# partitions.
self.pt = None
self.uuid = INVALID_UUID
self.primary_master_node = None
self.ptid = INVALID_PTID self.ptid = INVALID_PTID
def execute(self, args): def run(self):
"""Execute the command given.""" """Make sure that the status is sane and start a loop."""
handler = CommandEventHandler(self) if self.num_partitions is not None and self.num_partitions <= 0:
# connect to admin node raise RuntimeError, 'partitions must be more than zero'
conn = None if len(self.name) == 0:
self.trying_admin_node = False raise RuntimeError, 'cluster name must be non-empty'
try:
while 1: for server in self.master_node_list:
self.em.poll(1) self.nm.add(MasterNode(server = server))
if conn is None:
self.trying_admin_node = True # Make a listening port.
logging.info('connecting to address %s:%d', *(self.server)) handler = AdminEventHandler(self)
conn = ClientConnection(self.em, handler, \ ListeningConnection(self.em, handler, addr = self.server,
addr = self.server, connector_handler = self.connector_handler)
connector_handler = self.connector_handler)
if self.trying_admin_node is False: # Connect to a primary master node, verify data, and
break # start the operation. This cycle will be executed permentnly,
# until the user explicitly requests a shutdown.
while 1:
self.connectToPrimaryMaster()
try:
while 1:
self.em.poll(1)
except PrimaryFailure:
logging.error('primary master is down')
# do not trust any longer our informations
self.pt.clear()
self.nm.clear(filter = lambda node: node.getNodeType() != MASTER_NODE_TYPE)
except OperationFailure, msg:
return "FAIL : %s" %(msg,) def connectToPrimaryMaster(self):
"""Find a primary master node, and connect to it.
# here are the possible commands If a primary master node is not elected or ready, repeat
# print pt 1-10 [uuid] : print the partition table for row from 1 to 10 [containing node with uuid] the attempt of a connection periodically.
# print pt all [uuid] : print the partition table for all rows [containing node with uuid]
# print pt 10-0 [uuid] : print the partition table for row 10 to the end [containing node with uuid]
# print node type : print list of node of the given type (STORAGE_NODE_TYPE, MASTER_NODE_TYPE...)
# set node uuid state : set the node for the given uuid to the state (RUNNING_STATE, DOWN_STATE...)
command = args[0]
options = args[1:]
if command == "print":
print_type = options.pop(0)
if print_type == "pt":
offset = options.pop(0)
if offset == "all":
min_offset = 0
max_offset = 0
else:
min_offset = int(offset)
max_offset = int(options.pop(0))
if len(options):
uuid = bin(options.pop(0))
else:
uuid = INVALID_UUID
p = protocol.askPartitionList(min_offset, max_offset, uuid)
elif print_type == "node":
node_type = options.pop(0)
node_type = node_types.getFromStr(node_type)
if node_type is None:
return 'unknown node type'
p = protocol.askNodeList(node_type)
else:
return "unknown command options"
elif command == "set":
set_type = options.pop(0)
if set_type == "node":
uuid = bin(options.pop(0))
state = options.pop(0)
state = node_states.getFromStr(state)
if state is None:
return "unknown state type"
p = protocol.setNodeState(uuid, state)
else:
return "unknown command options"
else:
return "unknown command"
conn.ask(p) Note that I do not accept any connection from non-master nodes
self.result = "" at this stage."""
logging.info('connecting to a primary master node')
handler = MonitoringEventHandler(self)
em = self.em
nm = self.nm
# First of all, make sure that I have no connection.
for conn in em.getConnectionList():
if not conn.isListeningConnection():
conn.close()
index = 0
self.trying_master_node = None
self.primary_master_node = None
self.master_conn = None
t = 0
while 1: while 1:
self.em.poll(1) em.poll(1)
if len(self.result): if self.primary_master_node is not None:
break # If I know which is a primary master node, check if
# close connection # I have a connection to it already.
conn.close() for conn in em.getConnectionList():
return self.result if not conn.isListeningConnection() and not conn.isServerConnection():
uuid = conn.getUUID()
if uuid is not None:
node = nm.getNodeByUUID(uuid)
if node is self.primary_master_node:
logging.info("connected to primary master node %s:%d" % node.getServer())
self.master_conn = conn
# Yes, I have.
return
if self.trying_master_node is None and t + 1 < time():
# Choose a master node to connect to.
if self.primary_master_node is not None:
# If I know a primary master node, pinpoint it.
self.trying_master_node = self.primary_master_node
else:
# Otherwise, check one by one.
master_list = nm.getMasterNodeList()
try:
self.trying_master_node = master_list[index]
except IndexError:
index = 0
self.trying_master_node = master_list[0]
index += 1
print "connecting to %s:%d" % self.trying_master_node.getServer()
ClientConnection(em, handler, \
addr = self.trying_master_node.getServer(),
connector_handler = self.connector_handler)
t = time()
...@@ -26,59 +26,358 @@ from neo.connection import ClientConnection ...@@ -26,59 +26,358 @@ from neo.connection import ClientConnection
from neo import protocol from neo import protocol
from neo.protocol import Packet, UnexpectedPacketError from neo.protocol import Packet, UnexpectedPacketError
from neo.pt import PartitionTable from neo.pt import PartitionTable
from neo.exception import OperationFailure from neo.exception import PrimaryFailure
from neo.util import dump from neo.util import dump
from neo.handler import identification_required, restrict_node_types from neo.handler import identification_required, restrict_node_types
class CommandEventHandler(EventHandler):
""" Base handler for command """ class BaseEventHandler(EventHandler):
""" Base handler for admin node """
def __init__(self, app):
self.app = app
EventHandler.__init__(self)
class AdminEventHandler(BaseEventHandler):
"""This class deals with events for administrating cluster."""
def connectionAccepted(self, conn, s, addr):
"""Called when a connection is accepted."""
# we only accept connection from command tool
BaseEventHandler.connectionAccepted(self, conn, s, addr)
def handleAskPartitionList(self, conn, packet, min_offset, max_offset, uuid):
logging.info("ask partition list from %s to %s for %s" %(min_offset, max_offset, dump(uuid)))
app = self.app
app.pt.log()
row_list = []
if max_offset == 0:
max_offset = app.num_partitions
try:
for offset in xrange(min_offset, max_offset):
row = []
try:
for cell in app.pt.getCellList(offset):
if uuid != INVALID_UUID and cell.getUUID() != uuid:
continue
else:
row.append((cell.getUUID(), cell.getState()))
except TypeError:
pass
row_list.append((offset, row))
except IndexError:
p = protocot.protocolError('invalid partition table offset')
conn.notify(p)
return
print "sending packet", len(row_list)
p = protocol.answerPartitionList(app.ptid, row_list)
conn.notify(p)
def handleAskNodeList(self, conn, packet, node_type):
logging.info("ask node list for %s" %(node_type))
def node_filter(n):
return n.getNodeType() is node_type
node_list = self.app.nm.getNodeList(node_filter)
node_information_list = []
for node in node_list:
ip, port = node.getServer()
node_information_list.append((node.getNodeType(), ip, port, node.getUUID(), node.getState()))
p = protocol.answerNodeList(node_information_list)
conn.ask(p)
def handleSetNodeState(self, conn, packet, uuid, state):
logging.info("set node state for %s-%s" %(dump(uuid), state))
node = self.app.nm.getNodeByUUID(uuid)
if node is None:
p = protocol.protocolError('invalid uuid')
conn.notify(p)
# send information to master node
master_conn = self.app.master_conn
msg_id = master_conn.getNextId()
ip, port = node.getServer()
node_list = [(node.getNodeType(), ip, port, uuid, state),]
p = protocol.notifyNodeInformation(node_list)
master_conn.ask(p)
self.app.notified = False
while not self.app.notified:
self.app.em.poll(1)
node = self.app.nm.getNodeByUUID(uuid)
p = protocol.answerNodeState(node.getUUID(), node.getState())
conn.answer(p, packet)
class MonitoringEventHandler(BaseEventHandler):
"""This class deals with events for monitoring cluster."""
def connectionAccepted(self, conn, s, addr): def connectionAccepted(self, conn, s, addr):
"""Called when a connection is accepted.""" """Called when a connection is accepted."""
raise UnexpectedPacketError raise UnexpectedPacketError
def connectionCompleted(self, conn): def connectionCompleted(self, conn):
# connected to admin node app = self.app
self.app.trying_admin_node = False if app.trying_master_node is None:
# Should not happen.
raise RuntimeError('connection completed while not trying to connect')
p = protocol.requestNodeIdentification(ADMIN_NODE_TYPE,
app.uuid, app.server[0], app.server[1], app.name)
conn.ask(p)
EventHandler.connectionCompleted(self, conn) EventHandler.connectionCompleted(self, conn)
def connectionFailed(self, conn): def connectionFailed(self, conn):
app = self.app
if app.primary_master_node and conn.getUUID() == app.primary_master_node.getUUID():
raise PrimaryFailure
if app.trying_master_node is None:
# Should not happen.
raise RuntimeError('connection failed while not trying to connect')
if app.trying_master_node is app.primary_master_node:
# Tried to connect to a primary master node and failed.
# So this would effectively mean that it is dead.
app.primary_master_node = None
app.trying_master_node = None
EventHandler.connectionFailed(self, conn) EventHandler.connectionFailed(self, conn)
raise OperationFailure, "impossible to connect to admin node %s" %conn.getAddress()
def timeoutExpired(self, conn): def timeoutExpired(self, conn):
app = self.app
if app.primary_master_node and conn.getUUID() == app.primary_master_node.getUUID():
raise PrimaryFailure
if app.trying_master_node is app.primary_master_node:
# If a primary master node timeouts, I should not rely on it.
app.primary_master_node = None
app.trying_master_node = None
EventHandler.timeoutExpired(self, conn) EventHandler.timeoutExpired(self, conn)
raise OperationFailure, "connection to admin node %s timeout" %conn.getAddress()
def connectionClosed(self, conn): def connectionClosed(self, conn):
if self.app.trying_admin_node: app = self.app
raise OperationFailure, "cannot connect to admin node %s:%d" %conn.getAddress()
if app.primary_master_node and conn.getUUID() == app.primary_master_node.getUUID():
raise PrimaryFailure
if app.trying_master_node is app.primary_master_node:
# If a primary master node closes, I should not rely on it.
app.primary_master_node = None
app.trying_master_node = None
EventHandler.connectionClosed(self, conn) EventHandler.connectionClosed(self, conn)
def peerBroken(self, conn): def peerBroken(self, conn):
app = self.app
if app.primary_master_node and conn.getUUID() == app.primary_master_node.getUUID():
raise PrimaryFailure
if app.trying_master_node is app.primary_master_node:
# If a primary master node gets broken, I should not rely
# on it.
app.primary_master_node = None
app.trying_master_node = None
EventHandler.peerBroken(self, conn) EventHandler.peerBroken(self, conn)
raise OperationFailure, "connect to admin node %s broken" %conn.getAddress()
def handleAnswerPartitionList(self, conn, packet, ptid, row_list): def handleNotReady(self, conn, packet, message):
data = "" app = self.app
if len(row_list) == 0: if app.trying_master_node is not None:
data = "No partition" app.trying_master_node = None
else:
for offset, cell_list in row_list: conn.close()
data += "\n%s | " %offset
for uuid, state in cell_list: def handleAcceptNodeIdentification(self, conn, packet, node_type,
data += "%s - %s |" %(dump(uuid), state) uuid, ip_address, port,
self.app.result = data num_partitions, num_replicas, your_uuid):
app = self.app
def handleAnswerNodeList(self, conn, packet, node_list): node = app.nm.getNodeByServer(conn.getAddress())
data = "" if node_type != MASTER_NODE_TYPE:
if len(node_list) == 0: # The peer is not a master node!
data = "No Node" logging.error('%s:%d is not a master node', ip_address, port)
app.nm.remove(node)
conn.close()
return
if conn.getAddress() != (ip_address, port):
# The server address is different! Then why was
# the connection successful?
logging.error('%s:%d is waiting for %s:%d',
conn.getAddress()[0], conn.getAddress()[1],
ip_address, port)
app.nm.remove(node)
conn.close()
return
if app.num_partitions is None:
app.num_partitions = num_partitions
app.num_replicas = num_replicas
app.pt = PartitionTable(num_partitions, num_replicas)
elif app.num_partitions != num_partitions:
raise RuntimeError('the number of partitions is inconsistent')
elif app.num_replicas != num_replicas:
raise RuntimeError('the number of replicas is inconsistent')
conn.setUUID(uuid)
node.setUUID(uuid)
if your_uuid != INVALID_UUID:
# got an uuid from the primary master
app.uuid = your_uuid
# Ask a primary master.
conn.ask(protocol.askPrimaryMaster())
def handleAnswerPrimaryMaster(self, conn, packet, primary_uuid,
known_master_list):
app = self.app
# Register new master nodes.
for ip_address, port, uuid in known_master_list:
addr = (ip_address, port)
n = app.nm.getNodeByServer(addr)
if n is None:
n = MasterNode(server = addr)
app.nm.add(n)
if uuid != INVALID_UUID:
# If I don't know the UUID yet, believe what the peer
# told me at the moment.
if n.getUUID() is None or n.getUUID() != uuid:
n.setUUID(uuid)
if primary_uuid != INVALID_UUID:
primary_node = app.nm.getNodeByUUID(primary_uuid)
if primary_node is None:
# I don't know such a node. Probably this information
# is old. So ignore it.
pass
else:
app.primary_master_node = primary_node
if app.trying_master_node is primary_node:
# I am connected to the right one.
logging.info('connected to a primary master node')
# This is a workaround to prevent handling of
# packets for the verification phase.
else:
app.trying_master_node = None
conn.close()
else: else:
for node_type, ip, port, uuid, state in node_list: if app.primary_master_node is not None:
data += "\n%s - %s - %s:%s - %s" %(node_type, dump(uuid), ip, port, state) # The primary master node is not a primary master node
self.app.result = data # any longer.
app.primary_master_node = None
def handleAnswerNodeState(self, conn, packet, uuid, state):
self.app.result = "Node %s set to state %s" %(dump(uuid), state) app.trying_master_node = None
conn.close()
@identification_required
def handleSendPartitionTable(self, conn, packet, ptid, row_list):
logging.warning("handleSendPartitionTable")
uuid = conn.getUUID()
app = self.app
nm = app.nm
pt = app.pt
node = app.nm.getNodeByUUID(uuid)
# This must be sent only by primary master node
if node.getNodeType() != MASTER_NODE_TYPE:
return
if app.ptid != ptid:
app.ptid = ptid
pt.clear()
for offset, row in row_list:
for uuid, state in row:
node = nm.getNodeByUUID(uuid)
if node is None:
node = StorageNode(uuid = uuid)
node.setState(TEMPORARILY_DOWN_STATE)
nm.add(node)
pt.setCell(offset, node, state)
pt.log()
@identification_required
def handleNotifyPartitionChanges(self, conn, packet, ptid, cell_list):
logging.warning("handleNotifyPartitionChanges")
app = self.app
nm = app.nm
pt = app.pt
uuid = conn.getUUID()
node = app.nm.getNodeByUUID(uuid)
# This must be sent only by primary master node
if node.getNodeType() != MASTER_NODE_TYPE \
or app.primary_master_node is None \
or app.primary_master_node.getUUID() != uuid:
return
if app.ptid >= ptid:
# Ignore this packet.
return
app.ptid = ptid
for offset, uuid, state in cell_list:
node = nm.getNodeByUUID(uuid)
if node is None:
node = StorageNode(uuid = uuid)
if uuid != app.uuid:
node.setState(TEMPORARILY_DOWN_STATE)
nm.add(node)
# if state == DISCARDED_STATE:
# pt.dropNode(node)
# else:
pt.setCell(offset, node, state)
pt.log()
@identification_required
def handleNotifyNodeInformation(self, conn, packet, node_list):
logging.warning("handleNotifyNodeInformation")
uuid = conn.getUUID()
app = self.app
nm = app.nm
node = nm.getNodeByUUID(uuid)
# This must be sent only by a primary master node.
# Note that this may be sent before I know that it is
# a primary master node.
if node.getNodeType() != MASTER_NODE_TYPE:
logging.warn('ignoring notify node information from %s',
dump(uuid))
return
for node_type, ip_address, port, uuid, state in node_list:
# Register new nodes.
addr = (ip_address, port)
if node_type == MASTER_NODE_TYPE:
n = nm.getNodeByServer(addr)
if n is None:
n = MasterNode(server = addr)
nm.add(n)
if uuid != INVALID_UUID:
# If I don't know the UUID yet, believe what the peer
# told me at the moment.
if n.getUUID() is None:
n.setUUID(uuid)
elif node_type == STORAGE_NODE_TYPE:
if uuid == INVALID_UUID:
# No interest.
continue
n = nm.getNodeByUUID(uuid)
if n is None:
n = StorageNode(server = addr, uuid = uuid)
nm.add(n)
else:
n.setServer(addr)
elif node_type == CLIENT_NODE_TYPE:
continue
n.setState(state)
self.app.notified = True
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