Commit 7850ebec authored by Olivier Cros's avatar Olivier Cros

Move source code from neo/ to neo/lib

In order to prepare the eggification of the different neo parts, we created a
new neo/lib module, containing all of the main neo's functions. It allows to
make neo a virtual namespace, and so not containing module code anymore.




git-svn-id: https://svn.erp5.org/repos/neo/trunk@2615 71dcc9de-d417-0410-9af5-da40c76e7ee4
parent 49a524f1
#
# Copyright (C) 2006-2010 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, write to the Free Software
# Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
import logging as logging_std
PREFIX = '%(asctime)s %(levelname)-9s %(name)-10s'
SUFFIX = ' [%(module)14s:%(lineno)3d] %(message)s'
def setupLog(name='NEO', filename=None, verbose=False):
global logging
if verbose:
level = logging_std.DEBUG
else:
level = logging_std.INFO
fmt = PREFIX + SUFFIX
logging = logging_std.getLogger(name.upper())
for handler in logging.handlers[:]:
logging.removeHandler(handler)
if filename is None:
handler = logging_std.StreamHandler()
else:
handler = logging_std.FileHandler(filename)
handler.setFormatter(logging_std.Formatter(fmt))
logging.setLevel(level)
logging.addHandler(handler)
logging.propagate = 0
# Create default logger
setupLog()
import pkg_resources
pkg_resources.declare_namespace(__name__)
......@@ -15,19 +15,19 @@
# along with this program; if not, write to the Free Software
# Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
import neo
import neo.lib
from neo.node import NodeManager
from neo.event import EventManager
from neo.connection import ListeningConnection
from neo.exception import PrimaryFailure
from neo.lib.node import NodeManager
from neo.lib.event import EventManager
from neo.lib.connection import ListeningConnection
from neo.lib.exception import PrimaryFailure
from neo.admin.handler import AdminEventHandler, MasterEventHandler, \
MasterRequestEventHandler
from neo.connector import getConnectorHandler
from neo.bootstrap import BootstrapManager
from neo.pt import PartitionTable
from neo.protocol import NodeTypes, NodeStates, Packets, Errors
from neo.live_debug import register as registerLiveDebugger
from neo.lib.connector import getConnectorHandler
from neo.lib.bootstrap import BootstrapManager
from neo.lib.pt import PartitionTable
from neo.lib.protocol import NodeTypes, NodeStates, Packets, Errors
from neo.lib.live_debug import register as registerLiveDebugger
class Dispatcher:
"""Dispatcher use to redirect master request to handler"""
......@@ -68,7 +68,7 @@ class Application(object):
self.server = config.getBind()
self.master_addresses = config.getMasters()
neo.logging.debug('IP address is %s, port is %d', *(self.server))
neo.lib.logging.debug('IP address is %s, port is %d', *(self.server))
# The partition table is initialized after getting the number of
# partitions.
......@@ -105,7 +105,7 @@ class Application(object):
while True:
self.em.poll(1)
except PrimaryFailure:
neo.logging.error('primary master is down')
neo.lib.logging.error('primary master is down')
def connectToPrimary(self):
......
......@@ -17,11 +17,11 @@
import neo
from neo.handler import EventHandler
from neo import protocol
from neo.protocol import Packets, Errors
from neo.exception import PrimaryFailure
from neo.util import dump
from neo.lib.handler import EventHandler
from neo.lib import protocol
from neo.lib.protocol import Packets, Errors
from neo.lib.exception import PrimaryFailure
from neo.lib.util import dump
def forward_ask(klass):
def wrapper(self, conn, *args, **kw):
......@@ -42,7 +42,7 @@ class AdminEventHandler(EventHandler):
"""This class deals with events for administrating cluster."""
def askPartitionList(self, conn, min_offset, max_offset, uuid):
neo.logging.info("ask partition list from %s to %s for %s" %
neo.lib.logging.info("ask partition list from %s to %s for %s" %
(min_offset, max_offset, dump(uuid)))
app = self.app
# check we have one pt otherwise ask it to PMN
......@@ -61,7 +61,7 @@ class AdminEventHandler(EventHandler):
def askNodeList(self, conn, node_type):
neo.logging.info("ask node list for %s" %(node_type))
neo.lib.logging.info("ask node list for %s" %(node_type))
def node_filter(n):
return n.getType() is node_type
node_list = self.app.nm.getList(node_filter)
......@@ -70,7 +70,7 @@ class AdminEventHandler(EventHandler):
conn.answer(p)
def setNodeState(self, conn, uuid, state, modify_partition_table):
neo.logging.info("set node state for %s-%s" %(dump(uuid), state))
neo.lib.logging.info("set node state for %s-%s" %(dump(uuid), state))
node = self.app.nm.getByUUID(uuid)
if node is None:
raise protocol.ProtocolError('invalid uuid')
......@@ -144,7 +144,7 @@ class MasterEventHandler(EventHandler):
def answerNodeInformation(self, conn):
# XXX: This will no more exists when the initialization module will be
# implemented for factorize code (as done for bootstrap)
neo.logging.debug("answerNodeInformation")
neo.lib.logging.debug("answerNodeInformation")
def notifyPartitionChanges(self, conn, ptid, cell_list):
self.app.pt.update(ptid, cell_list, self.app.nm)
......@@ -176,12 +176,12 @@ class MasterRequestEventHandler(EventHandler):
client_conn.answer(packet)
def answerClusterState(self, conn, state):
neo.logging.info("answerClusterState for a conn")
neo.lib.logging.info("answerClusterState for a conn")
self.app.cluster_state = state
self._answerNeoCTL(conn, Packets.AnswerClusterState(state))
def answerPartitionTable(self, conn, ptid, row_list):
neo.logging.info("answerPartitionTable for a conn")
neo.lib.logging.info("answerPartitionTable for a conn")
client_conn, kw = self.app.dispatcher.pop(conn.getPeerId())
# sent client the partition table
self.app.sendPartitionTable(client_conn)
......
......@@ -19,9 +19,9 @@ from ZODB import BaseStorage, ConflictResolution, POSException
from zope.interface import implements
import ZODB.interfaces
from neo import setupLog
from neo.util import add64
from neo.protocol import ZERO_TID
from neo.lib import setupLog
from neo.lib.util import add64
from neo.lib.protocol import ZERO_TID
from neo.client.app import Application
from neo.client.exception import NEOStorageNotFoundError
from neo.client.exception import NEOStorageDoesNotExistError
......@@ -255,7 +255,8 @@ class Storage(BaseStorage.BaseStorage,
def pack(self, t, referencesf, gc=False):
if gc:
neo.logging.warning('Garbage Collection is not available in NEO, '
neo.lib.logging.warning(
'Garbage Collection is not available in NEO, '
'please use an external tool. Packing without GC.')
self.app.pack(t)
......
This diff is collapsed.
......@@ -15,8 +15,8 @@
# along with this program; if not, write to the Free Software
# Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
from neo.handler import EventHandler
from neo.protocol import ProtocolError
from neo.lib.handler import EventHandler
from neo.lib.protocol import ProtocolError
class BaseHandler(EventHandler):
"""Base class for client-side EventHandler implementations."""
......
......@@ -15,12 +15,12 @@
# along with this program; if not, write to the Free Software
# Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
import neo
import neo.lib
from neo.client.handlers import BaseHandler, AnswerBaseHandler
from neo.pt import MTPartitionTable as PartitionTable
from neo.protocol import NodeTypes, NodeStates, ProtocolError
from neo.util import dump
from neo.lib.pt import MTPartitionTable as PartitionTable
from neo.lib.protocol import NodeTypes, NodeStates, ProtocolError
from neo.lib.util import dump
from neo.client.exception import NEOStorageError
class PrimaryBootstrapHandler(AnswerBaseHandler):
......@@ -43,7 +43,7 @@ class PrimaryBootstrapHandler(AnswerBaseHandler):
if your_uuid is None:
raise ProtocolError('No UUID supplied')
app.uuid = your_uuid
neo.logging.info('Got an UUID: %s', dump(app.uuid))
neo.lib.logging.info('Got an UUID: %s', dump(app.uuid))
node = app.nm.getByAddress(conn.getAddress())
conn.setUUID(uuid)
......@@ -66,7 +66,7 @@ class PrimaryBootstrapHandler(AnswerBaseHandler):
if primary_node is None:
# I don't know such a node. Probably this information
# is old. So ignore it.
neo.logging.warning('Unknown primary master UUID: %s. ' \
neo.lib.logging.warning('Unknown primary master UUID: %s. ' \
'Ignoring.' % dump(primary_uuid))
else:
app.primary_master_node = primary_node
......@@ -94,7 +94,7 @@ class PrimaryNotificationsHandler(BaseHandler):
def connectionClosed(self, conn):
app = self.app
neo.logging.critical("connection to primary master node closed")
neo.lib.logging.critical("connection to primary master node closed")
conn.close()
app.master_conn = None
app.primary_master_node = None
......@@ -104,19 +104,19 @@ class PrimaryNotificationsHandler(BaseHandler):
app = self.app
if app.master_conn is not None:
assert conn is app.master_conn
neo.logging.critical("connection timeout to primary master node " \
"expired")
neo.lib.logging.critical(
"connection timeout to primary master node expired")
BaseHandler.timeoutExpired(self, conn)
def peerBroken(self, conn):
app = self.app
if app.master_conn is not None:
assert conn is app.master_conn
neo.logging.critical("primary master node is broken")
neo.lib.logging.critical("primary master node is broken")
BaseHandler.peerBroken(self, conn)
def stopOperation(self, conn):
neo.logging.critical("master node ask to stop operation")
neo.lib.logging.critical("master node ask to stop operation")
def invalidateObjects(self, conn, tid, oid_list):
app = self.app
......
......@@ -18,10 +18,10 @@
from ZODB.TimeStamp import TimeStamp
from ZODB.POSException import ConflictError
import neo
import neo.lib
from neo.client.handlers import BaseHandler, AnswerBaseHandler
from neo.protocol import NodeTypes, ProtocolError, LockState
from neo.util import dump
from neo.lib.protocol import NodeTypes, ProtocolError, LockState
from neo.lib.util import dump
from neo.client.exception import NEOStorageError, NEOStorageNotFoundError
from neo.client.exception import NEOStorageDoesNotExistError
......@@ -74,7 +74,7 @@ class StorageAnswersHandler(AnswerBaseHandler):
local_var = self.app.local_var
object_stored_counter_dict = local_var.object_stored_counter_dict[oid]
if conflicting:
neo.logging.info('%r report a conflict for %r with %r', conn,
neo.lib.logging.info('%r report a conflict for %r with %r', conn,
dump(oid), dump(serial))
conflict_serial_dict = local_var.conflict_serial_dict
if serial in object_stored_counter_dict:
......@@ -96,7 +96,7 @@ class StorageAnswersHandler(AnswerBaseHandler):
raise NEOStorageError('Wrong TID, transaction not started')
def answerTIDsFrom(self, conn, tid_list):
neo.logging.debug('Get %d TIDs from %r', len(tid_list), conn)
neo.lib.logging.debug('Get %d TIDs from %r', len(tid_list), conn)
assert not self.app.local_var.tids_from.intersection(set(tid_list))
self.app.local_var.tids_from.update(tid_list)
......@@ -144,7 +144,7 @@ class StorageAnswersHandler(AnswerBaseHandler):
raise ConflictError, 'Lock wait timeout for oid %s on %r' % (
dump(oid), conn)
elif status == LockState.GRANTED:
neo.logging.info('Store of oid %s was successful, but after ' \
neo.lib.logging.info('Store of oid %s was successful, but after ' \
'timeout.', dump(oid))
# XXX: Not sure what to do in this case yet, for now do nothing.
else:
......
......@@ -18,7 +18,7 @@
from ZODB import BaseStorage
from zope.interface import implements
import ZODB.interfaces
from neo.util import u64, add64
from neo.lib.util import u64, add64
from neo.client.exception import NEOStorageCreationUndoneError
from neo.client.exception import NEOStorageNotFoundError
......
......@@ -16,8 +16,8 @@
# Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
from threading import Thread, Event, enumerate as thread_enum
from neo.locking import Lock
import neo
from neo.lib.locking import Lock
import neo.lib
class _ThreadedPoll(Thread):
"""Polling thread."""
......@@ -34,7 +34,7 @@ class _ThreadedPoll(Thread):
self._stop = Event()
def run(self):
neo.logging.debug('Started %s', self)
neo.lib.logging.debug('Started %s', self)
while not self.stopping():
# First check if we receive any new message from other node
try:
......@@ -42,8 +42,8 @@ class _ThreadedPoll(Thread):
# interrupt this call when stopping.
self.em.poll(1)
except:
self.neo.logging.error('poll raised, retrying', exc_info=1)
self.neo.logging.debug('Threaded poll stopped')
self.neo.lib.logging.error('poll raised, retrying', exc_info=1)
self.neo.lib.logging.debug('Threaded poll stopped')
self._stop.clear()
def stop(self):
......@@ -110,7 +110,7 @@ def psThreadedPoll(log=None):
Logs alive ThreadedPoll threads.
"""
if log is None:
log = neo.logging.debug
log = neo.lib.logging.debug
for thread in thread_enum():
if not isinstance(thread, ThreadedPoll):
continue
......
......@@ -18,12 +18,12 @@
import time
from random import shuffle
import neo
from neo.locking import RLock
from neo.protocol import NodeTypes, Packets
from neo.connection import MTClientConnection, ConnectionClosed
import neo.lib
from neo.lib.locking import RLock
from neo.lib.protocol import NodeTypes, Packets
from neo.lib.connection import MTClientConnection, ConnectionClosed
from neo.client.exception import NEOStorageError
from neo.profiling import profiler_decorator
from neo.lib.profiling import profiler_decorator
# How long before we might retry a connection to a node to which connection
# failed in the past.
......@@ -61,7 +61,7 @@ class ConnectionPool(object):
assert addr is not None
app = self.app
app.setNodeReady()
neo.logging.debug('trying to connect to %s - %s', node,
neo.lib.logging.debug('trying to connect to %s - %s', node,
node.getState())
conn = MTClientConnection(app.em, app.storage_event_handler, addr,
connector=app.connector_handler(), dispatcher=app.dispatcher)
......@@ -70,7 +70,7 @@ class ConnectionPool(object):
try:
if conn.getConnector() is None:
# This happens, if a connection could not be established.
neo.logging.error('Connection to %r failed', node)
neo.lib.logging.error('Connection to %r failed', node)
self.notifyFailure(node)
return None
......@@ -84,15 +84,15 @@ class ConnectionPool(object):
app._waitMessage(conn, msg_id,
handler=app.storage_bootstrap_handler)
except ConnectionClosed:
neo.logging.error('Connection to %r failed', node)
neo.lib.logging.error('Connection to %r failed', node)
self.notifyFailure(node)
return None
if app.isNodeReady():
neo.logging.info('Connected %r', node)
neo.lib.logging.info('Connected %r', node)
return conn
else:
neo.logging.info('%r not ready', node)
neo.lib.logging.info('%r not ready', node)
self.notifyFailure(node)
return NOT_READY
......@@ -107,7 +107,7 @@ class ConnectionPool(object):
not self.app.dispatcher.registered(conn):
del self.connection_dict[conn.getUUID()]
conn.close()
neo.logging.debug('_dropConnections : connection to ' \
neo.lib.logging.debug('_dropConnections : connection to ' \
'storage node %s:%d closed', *(conn.getAddress()))
if len(self.connection_dict) <= self.max_pool_size:
break
......
#
# Copyright (C) 2006-2010 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, write to the Free Software
# Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
import logging as logging_std
PREFIX = '%(asctime)s %(levelname)-9s %(name)-10s'
SUFFIX = ' [%(module)14s:%(lineno)3d] %(message)s'
def setupLog(name='NEO', filename=None, verbose=False):
global logging
if verbose:
level = logging_std.DEBUG
else:
level = logging_std.INFO
fmt = PREFIX + SUFFIX
logging = logging_std.getLogger(name.upper())
for handler in logging.handlers[:]:
logging.removeHandler(handler)
if filename is None:
handler = logging_std.StreamHandler()
else:
handler = logging_std.FileHandler(filename)
handler.setFormatter(logging_std.Formatter(fmt))
logging.setLevel(level)
logging.addHandler(handler)
logging.propagate = 0
# Create default logger
setupLog()
......@@ -17,7 +17,7 @@
ATTRIBUTE_TRACKER_ENABLED = False
from neo.locking import LockUser
from neo.lib.locking import LockUser
"""
Usage example:
......
......@@ -18,10 +18,10 @@
import neo
from time import sleep
from neo.handler import EventHandler
from neo.protocol import Packets
from neo.util import dump
from neo.connection import ClientConnection
from neo.lib.handler import EventHandler
from neo.lib.protocol import Packets
from neo.lib.util import dump
from neo.lib.connection import ClientConnection
NO_SERVER = ('0.0.0.0', 0)
......@@ -106,7 +106,7 @@ class BootstrapManager(EventHandler):
conn.close()
return
neo.logging.info('connected to a primary master node')
neo.lib.logging.info('connected to a primary master node')
conn.ask(Packets.RequestIdentification(self.node_type,
self.uuid, self.server, self.name))
......@@ -120,7 +120,7 @@ class BootstrapManager(EventHandler):
if self.uuid != your_uuid:
# got an uuid from the primary master
self.uuid = your_uuid
neo.logging.info('Got a new UUID : %s' % dump(self.uuid))
neo.lib.logging.info('Got a new UUID : %s' % dump(self.uuid))
conn.setUUID(uuid)
def getPrimaryConnection(self, connector_handler):
......@@ -128,7 +128,7 @@ class BootstrapManager(EventHandler):
Primary lookup/connection process.
Returns when the connection is made.
"""
neo.logging.info('connecting to a primary master node')
neo.lib.logging.info('connecting to a primary master node')
em, nm = self.app.em, self.app.nm
index = 0
self.current = nm.getMasterList()[0]
......
......@@ -16,7 +16,7 @@
# Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
from ConfigParser import SafeConfigParser
from neo import util
from neo.lib import util
class ConfigurationManager(object):
......
......@@ -17,19 +17,19 @@
from time import time
import neo
from neo.locking import RLock
import neo.lib
from neo.lib.locking import RLock
from neo.protocol import PacketMalformedError, Packets, ParserState
from neo.connector import ConnectorException, ConnectorTryAgainException, \
from neo.lib.protocol import PacketMalformedError, Packets, ParserState
from neo.lib.connector import ConnectorException, ConnectorTryAgainException, \
ConnectorInProgressException, ConnectorConnectionRefusedException, \
ConnectorConnectionClosedException
from neo.util import dump
from neo.logger import PACKET_LOGGER
from neo.lib.util import dump
from neo.lib.logger import PACKET_LOGGER
from neo import attributeTracker
from neo.util import ReadBuffer
from neo.profiling import profiler_decorator
from neo.lib import attributeTracker
from neo.lib.util import ReadBuffer
from neo.lib.profiling import profiler_decorator
PING_DELAY = 6
PING_TIMEOUT = 5
......@@ -62,7 +62,7 @@ def lockCheckWrapper(func):
def wrapper(self, *args, **kw):
if not self._lock._is_owned():
import traceback
neo.logging.warning('%s called on %s instance without being ' \
neo.lib.logging.warning('%s called on %s instance without being ' \
'locked. Stack:\n%s', func.func_code.co_name,
self.__class__.__name__, ''.join(traceback.format_stack()))
# Call anyway
......@@ -163,7 +163,7 @@ class HandlerSwitcher(object):
assert len(self._pending) == 1 or self._pending[0][0]
PACKET_LOGGER.dispatch(connection, packet, 'from')
if connection.isClosed() and packet.ignoreOnClosedConnection():
neo.logging.debug('Ignoring packet %r on closed connection %r',
neo.lib.logging.debug('Ignoring packet %r on closed connection %r',
packet, connection)
return
msg_id = packet.getId()
......@@ -177,7 +177,8 @@ class HandlerSwitcher(object):
if klass and isinstance(packet, klass) or packet.isError():
handler.packetReceived(connection, packet)
else:
neo.logging.error('Unexpected answer %r in %r', packet, connection)
neo.lib.logging.error(
'Unexpected answer %r in %r', packet, connection)
notification = Packets.Notify('Unexpected answer: %r' % packet)
try:
connection.notify(notification)
......@@ -188,7 +189,8 @@ class HandlerSwitcher(object):
# apply a pending handler if no more answers are pending
while len(self._pending) > 1 and not self._pending[0][0]:
del self._pending[0]
neo.logging.debug('Apply handler %r on %r', self._pending[0][1],
neo.lib.logging.debug(
'Apply handler %r on %r', self._pending[0][1],
connection)
if timeout == self._next_timeout:
self._updateNextTimeout()
......@@ -286,12 +288,13 @@ class BaseConnection(object):
if handlers.isPending():
msg_id = handlers.checkTimeout(self, t)
if msg_id is not None:
neo.logging.info('timeout for #0x%08x with %r', msg_id, self)
neo.lib.logging.info(
'timeout for #0x%08x with %r', msg_id, self)
self.close()
self.getHandler().timeoutExpired(self)
elif self._timeout.hardExpired(t):
# critical time reach or pong not received, abort
neo.logging.info('timeout with %r', self)
neo.lib.logging.info('timeout with %r', self)
self.notify(Packets.Notify('Timeout'))
self.abort()
self.getHandler().timeoutExpired(self)
......@@ -345,9 +348,9 @@ class BaseConnection(object):
def setHandler(self, handler):
if self._handlers.setHandler(handler):
neo.logging.debug('Set handler %r on %r', handler, self)
neo.lib.logging.debug('Set handler %r on %r', handler, self)
else:
neo.logging.debug('Delay handler %r on %r', handler, self)
neo.lib.logging.debug('Delay handler %r on %r', handler, self)
def getEventManager(self):
return self.em
......@@ -386,7 +389,7 @@ class ListeningConnection(BaseConnection):
"""A listen connection."""
def __init__(self, event_manager, handler, addr, connector, **kw):
neo.logging.debug('listening to %s:%d', *addr)
neo.lib.logging.debug('listening to %s:%d', *addr)
BaseConnection.__init__(self, event_manager, handler,
addr=addr, connector=connector)
self.connector.makeListeningConnection(addr)
......@@ -395,7 +398,7 @@ class ListeningConnection(BaseConnection):
def readable(self):
try:
new_s, addr = self.connector.getNewConnection()
neo.logging.debug('accepted a connection from %s:%d', *addr)
neo.lib.logging.debug('accepted a connection from %s:%d', *addr)
handler = self.getHandler()
new_conn = ServerConnection(self.getEventManager(), handler,
connector=new_s, addr=addr)
......@@ -458,7 +461,7 @@ class Connection(BaseConnection):
return next_id
def close(self):
neo.logging.debug('closing a connector for %r', self)
neo.lib.logging.debug('closing a connector for %r', self)
BaseConnection.close(self)
if self._on_close is not None:
self._on_close()
......@@ -469,7 +472,7 @@ class Connection(BaseConnection):
def abort(self):
"""Abort dealing with this connection."""
neo.logging.debug('aborting a connector for %r', self)
neo.lib.logging.debug('aborting a connector for %r', self)
self.aborted = True
def writable(self):
......@@ -556,17 +559,19 @@ class Connection(BaseConnection):
except ConnectorConnectionClosedException:
# connection resetted by peer, according to the man, this error
# should not occurs but it seems it's false
neo.logging.debug('Connection reset by peer: %r', self.connector)
neo.lib.logging.debug(
'Connection reset by peer: %r', self.connector)
self._closure()
except:
neo.logging.debug('Unknown connection error: %r', self.connector)
neo.lib.logging.debug(
'Unknown connection error: %r', self.connector)
self._closure()
# unhandled connector exception
raise
else:
if not data:
neo.logging.debug('Connection %r closed in recv',
self.connector)
neo.lib.logging.debug(
'Connection %r closed in recv', self.connector)
self._closure()
return
self.read_buf.append(data)
......@@ -583,16 +588,18 @@ class Connection(BaseConnection):
pass
except ConnectorConnectionClosedException:
# connection resetted by peer
neo.logging.debug('Connection reset by peer: %r', self.connector)
neo.lib.logging.debug(
'Connection reset by peer: %r', self.connector)
self._closure()
except:
neo.logging.debug('Unknown connection error: %r', self.connector)
neo.lib.logging.debug(
'Unknown connection error: %r', self.connector)
# unhandled connector exception
self._closure()
raise
else:
if not n:
neo.logging.debug('Connection %r closed in send',
neo.lib.logging.debug('Connection %r closed in send',
self.connector)
self._closure()
return
......
......@@ -15,8 +15,8 @@
# along with this program; if not, write to the Free Software
# Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
from neo.locking import Lock
from neo.profiling import profiler_decorator
from neo.lib.locking import Lock
from neo.lib.profiling import profiler_decorator
EMPTY = {}
NOBODY = []
......
......@@ -16,9 +16,9 @@
# Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
from time import time
import neo
from neo.epoll import Epoll
from neo.profiling import profiler_decorator
import neo.lib
from neo.lib.epoll import Epoll
from neo.lib.profiling import profiler_decorator
class EpollEventManager(object):
"""This class manages connections and events based on epoll(5)."""
......@@ -190,13 +190,13 @@ class EpollEventManager(object):
self.epoll.modify(fd, fd in self.reader_set, 0)
def log(self):
neo.logging.info('Event Manager:')
neo.logging.info(' Readers: %r', [x for x in self.reader_set])
neo.logging.info(' Writers: %r', [x for x in self.writer_set])
neo.logging.info(' Connections:')
neo.lib.logging.info('Event Manager:')
neo.lib.logging.info(' Readers: %r', [x for x in self.reader_set])
neo.lib.logging.info(' Writers: %r', [x for x in self.writer_set])
neo.lib.logging.info(' Connections:')
pending_set = set(self._pending_processing)
for fd, conn in self.connection_dict.items():
neo.logging.info(' %r: %r (pending=%r)', fd, conn,
neo.lib.logging.info(' %r: %r (pending=%r)', fd, conn,
conn in pending_set)
......
......@@ -15,9 +15,9 @@
# along with this program; if not, write to the Free Software
# Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
import neo
from neo.protocol import NodeStates, ErrorCodes, Packets, Errors
from neo.protocol import PacketMalformedError, UnexpectedPacketError, \
import neo.lib
from neo.lib.protocol import NodeStates, ErrorCodes, Packets, Errors
from neo.lib.protocol import PacketMalformedError, UnexpectedPacketError, \
BrokenNodeDisallowedError, NotReadyError, ProtocolError
......@@ -40,7 +40,7 @@ class EventHandler(object):
else:
message = 'unexpected packet: %s in %s' % (message,
self.__class__.__name__)
neo.logging.error(message)
neo.lib.logging.error(message)
conn.answer(Errors.ProtocolError(message))
conn.abort()
self.peerBroken(conn)
......@@ -58,7 +58,7 @@ class EventHandler(object):
except UnexpectedPacketError, e:
self.__unexpectedPacket(conn, packet, *e.args)
except PacketMalformedError:
neo.logging.error('malformed packet from %r', conn)
neo.lib.logging.error('malformed packet from %r', conn)
conn.notify(Packets.Notify('Malformed packet: %r' % (packet, )))
conn.abort()
self.peerBroken(conn)
......@@ -82,7 +82,7 @@ class EventHandler(object):
def checkClusterName(self, name):
# raise an exception if the fiven name mismatch the current cluster name
if self.app.name != name:
neo.logging.error('reject an alien cluster')
neo.lib.logging.error('reject an alien cluster')
raise ProtocolError('invalid cluster name')
......@@ -94,32 +94,32 @@ class EventHandler(object):
def connectionStarted(self, conn):
"""Called when a connection is started."""
neo.logging.debug('connection started for %r', conn)
neo.lib.logging.debug('connection started for %r', conn)
def connectionCompleted(self, conn):
"""Called when a connection is completed."""
neo.logging.debug('connection completed for %r', conn)
neo.lib.logging.debug('connection completed for %r', conn)
def connectionFailed(self, conn):
"""Called when a connection failed."""
neo.logging.debug('connection failed for %r', conn)
neo.lib.logging.debug('connection failed for %r', conn)
def connectionAccepted(self, conn):
"""Called when a connection is accepted."""
def timeoutExpired(self, conn):
"""Called when a timeout event occurs."""
neo.logging.debug('timeout expired for %r', conn)
neo.lib.logging.debug('timeout expired for %r', conn)
self.connectionLost(conn, NodeStates.TEMPORARILY_DOWN)
def connectionClosed(self, conn):
"""Called when a connection is closed by the peer."""
neo.logging.debug('connection closed for %r', conn)
neo.lib.logging.debug('connection closed for %r', conn)
self.connectionLost(conn, NodeStates.TEMPORARILY_DOWN)
def peerBroken(self, conn):
"""Called when a peer is broken."""
neo.logging.error('%r is broken', conn)
neo.lib.logging.error('%r is broken', conn)
self.connectionLost(conn, NodeStates.BROKEN)
def connectionLost(self, conn, new_state):
......@@ -131,7 +131,7 @@ class EventHandler(object):
# Packet handlers.
def notify(self, conn, message):
neo.logging.info('notification from %r: %s', conn, message)
neo.lib.logging.info('notification from %r: %s', conn, message)
def requestIdentification(self, conn, node_type,
uuid, address, name):
......@@ -403,19 +403,19 @@ class EventHandler(object):
def protocolError(self, conn, message):
# the connection should have been closed by the remote peer
neo.logging.error('protocol error: %s' % (message,))
neo.lib.logging.error('protocol error: %s' % (message,))
def timeoutError(self, conn, message):
neo.logging.error('timeout error: %s' % (message,))
neo.lib.logging.error('timeout error: %s' % (message,))
def brokenNodeDisallowedError(self, conn, message):
raise RuntimeError, 'broken node disallowed error: %s' % (message,)
def alreadyPendingError(self, conn, message):
neo.logging.error('already pending error: %s' % (message, ))
neo.lib.logging.error('already pending error: %s' % (message, ))
def ack(self, conn, message):
neo.logging.debug("no error message : %s" % (message))
neo.lib.logging.debug("no error message : %s" % (message))
# Fetch tables initialization
......
......@@ -16,10 +16,10 @@
# Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
import neo
from neo.protocol import PacketMalformedError
from neo.util import dump
from neo.handler import EventHandler
from neo.profiling import profiler_decorator
from neo.lib.protocol import PacketMalformedError
from neo.lib.util import dump
from neo.lib.handler import EventHandler
from neo.lib.profiling import profiler_decorator
LOGGER_ENABLED = False
......@@ -37,7 +37,7 @@ class PacketLogger(object):
klass = packet.getType()
uuid = dump(conn.getUUID())
ip, port = conn.getAddress()
neo.logging.debug('#0x%08x %-30s %s %s (%s:%d)', packet.getId(),
neo.lib.logging.debug('#0x%08x %-30s %s %s (%s:%d)', packet.getId(),
packet.__class__.__name__, direction, uuid, ip, port)
# look for custom packet logger
logger = self.packet_dispatch_table.get(klass, None)
......@@ -48,11 +48,11 @@ class PacketLogger(object):
try:
args = packet.decode() or ()
except PacketMalformedError:
neo.logging.warning("Can't decode packet for logging")
neo.lib.logging.warning("Can't decode packet for logging")
return
log_message = logger(conn, *args)
if log_message is not None:
neo.logging.debug('#0x%08x %s', packet.getId(), log_message)
neo.lib.logging.debug('#0x%08x %s', packet.getId(), log_message)
def error(self, conn, code, message):
return "%s (%s)" % (code, message)
......@@ -64,7 +64,7 @@ class PacketLogger(object):
else:
address = '?'
node = (dump(uuid), node_type, address, state)
neo.logging.debug(' ! %s | %8s | %22s | %s' % node)
neo.lib.logging.debug(' ! %s | %8s | %22s | %s' % node)
PACKET_LOGGER = PacketLogger()
if not LOGGER_ENABLED:
......
......@@ -17,11 +17,11 @@
from time import time
import neo
from neo.util import dump
from neo.protocol import NodeTypes, NodeStates
import neo.lib
from neo.lib.util import dump
from neo.lib.protocol import NodeTypes, NodeStates
from neo import attributeTracker
from neo.lib import attributeTracker
class Node(object):
"""This class represents a node."""
......@@ -263,7 +263,7 @@ class NodeManager(object):
def add(self, node):
if node in self._node_set:
neo.logging.warning('adding a known node %r, ignoring', node)
neo.lib.logging.warning('adding a known node %r, ignoring', node)
return
self._node_set.add(node)
self._updateAddress(node, None)
......@@ -274,7 +274,7 @@ class NodeManager(object):
def remove(self, node):
if node not in self._node_set:
neo.logging.warning('removing unknown node %r, ignoring', node)
neo.lib.logging.warning('removing unknown node %r, ignoring', node)
return
self._node_set.remove(node)
self.__drop(self._address_dict, node.getAddress())
......@@ -446,12 +446,12 @@ class NodeManager(object):
log_args = (node_type, dump(uuid), addr, state)
if node is None:
if state == NodeStates.DOWN:
neo.logging.debug('NOT creating node %s %s %s %s',
neo.lib.logging.debug('NOT creating node %s %s %s %s',
*log_args)
else:
node = self._createNode(klass, address=addr, uuid=uuid,
state=state)
neo.logging.debug('creating node %r', node)
neo.lib.logging.debug('creating node %r', node)
else:
assert isinstance(node, klass), 'node %r is not ' \
'of expected type: %r' % (node, klass)
......@@ -460,14 +460,15 @@ class NodeManager(object):
'Discrepancy between node_by_uuid (%r) and ' \
'node_by_addr (%r)' % (node_by_uuid, node_by_addr)
if state == NodeStates.DOWN:
neo.logging.debug('droping node %r (%r), found with %s ' \
neo.lib.logging.debug(
'droping node %r (%r), found with %s ' \
'%s %s %s', node, node.isConnected(), *log_args)
if node.isConnected():
# cut this connection, node removed by handler
node.getConnection().close()
self.remove(node)
else:
neo.logging.debug('updating node %r to %s %s %s %s',
neo.lib.logging.debug('updating node %r to %s %s %s %s',
node, *log_args)
node.setUUID(uuid)
node.setAddress(addr)
......@@ -475,12 +476,12 @@ class NodeManager(object):
self.log()
def log(self):
neo.logging.info('Node manager : %d nodes' % len(self._node_set))
neo.lib.logging.info('Node manager : %d nodes' % len(self._node_set))
for node in sorted(list(self._node_set)):
uuid = dump(node.getUUID()) or '-' * 32
address = node.getAddress() or ''
if address:
address = '%s:%d' % address
neo.logging.info(' * %32s | %8s | %22s | %s' % (
neo.lib.logging.info(' * %32s | %8s | %22s | %s' % (
uuid, node.getType(), address, node.getState()))
......@@ -17,10 +17,10 @@
from struct import pack, unpack, error, calcsize
from socket import inet_ntoa, inet_aton
from neo.profiling import profiler_decorator
from neo.lib.profiling import profiler_decorator
from cStringIO import StringIO
from neo.util import Enum
from neo.lib.util import Enum
# The protocol version (major, minor).
PROTOCOL_VERSION = (4, 1)
......
......@@ -17,10 +17,10 @@
import neo
from neo import protocol
from neo.protocol import CellStates
from neo.util import dump, u64
from neo.locking import RLock
from neo.lib import protocol
from neo.lib.protocol import CellStates
from neo.lib.util import dump, u64
from neo.lib.locking import RLock
class PartitionTableException(Exception):
"""
......@@ -218,7 +218,7 @@ class PartitionTable(object):
# the node must be known by the node manager
assert node is not None
self.setCell(offset, node, state)
neo.logging.debug('partition table loaded')
neo.lib.logging.debug('partition table loaded')
self.log()
def update(self, ptid, cell_list, nm):
......@@ -228,14 +228,14 @@ class PartitionTable(object):
is not known, it is created in the node manager and set as unavailable
"""
if ptid <= self._id:
neo.logging.warning('ignoring older partition changes')
neo.lib.logging.warning('ignoring older partition changes')
return
self._id = ptid
for offset, uuid, state in cell_list:
node = nm.getByUUID(uuid)
assert node is not None, 'No node found for uuid %r' % (dump(uuid), )
self.setCell(offset, node, state)
neo.logging.debug('partition table updated')
neo.lib.logging.debug('partition table updated')
self.log()
def filled(self):
......@@ -243,7 +243,7 @@ class PartitionTable(object):
def log(self):
for line in self._format():
neo.logging.debug(line)
neo.lib.logging.debug(line)
def format(self):
return '\n'.join(self._format())
......
......@@ -19,13 +19,13 @@ import neo
import os, sys
from time import time
from neo import protocol
from neo.protocol import UUID_NAMESPACES, ZERO_TID
from neo.protocol import ClusterStates, NodeStates, NodeTypes, Packets
from neo.node import NodeManager
from neo.event import EventManager
from neo.connection import ListeningConnection, ClientConnection
from neo.exception import ElectionFailure, PrimaryFailure, OperationFailure
from neo.lib import protocol
from neo.lib.protocol import UUID_NAMESPACES, ZERO_TID
from neo.lib.protocol import ClusterStates, NodeStates, NodeTypes, Packets
from neo.lib.node import NodeManager
from neo.lib.event import EventManager
from neo.lib.connection import ListeningConnection, ClientConnection
from neo.lib.exception import ElectionFailure, PrimaryFailure, OperationFailure
from neo.master.handlers import election, identification, secondary
from neo.master.handlers import storage, client, shutdown
from neo.master.handlers import administration
......@@ -33,10 +33,10 @@ from neo.master.pt import PartitionTable
from neo.master.transactions import TransactionManager
from neo.master.verification import VerificationManager
from neo.master.recovery import RecoveryManager
from neo.util import dump
from neo.connector import getConnectorHandler
from neo.lib.util import dump
from neo.lib.connector import getConnectorHandler
from neo.live_debug import register as registerLiveDebugger
from neo.lib.live_debug import register as registerLiveDebugger
class Application(object):
"""The master node application."""
......@@ -61,7 +61,7 @@ class Application(object):
for address in config.getMasters():
self.nm.createMaster(address=address)
neo.logging.debug('IP address is %s, port is %d', *(self.server))
neo.lib.logging.debug('IP address is %s, port is %d', *(self.server))
# Partition table
replicas, partitions = config.getReplicas(), config.getPartitions()
......@@ -70,10 +70,10 @@ class Application(object):
if partitions <= 0:
raise RuntimeError, 'partitions must be more than zero'
self.pt = PartitionTable(partitions, replicas)
neo.logging.info('Configuration:')
neo.logging.info('Partitions: %d', partitions)
neo.logging.info('Replicas : %d', replicas)
neo.logging.info('Name : %s', self.name)
neo.lib.logging.info('Configuration:')
neo.lib.logging.info('Partitions: %d', partitions)
neo.lib.logging.info('Replicas : %d', replicas)
neo.lib.logging.info('Name : %s', self.name)
self.listening_conn = None
self.primary = None
......@@ -86,7 +86,7 @@ class Application(object):
if uuid is None or uuid == '':
uuid = self.getNewUUID(NodeTypes.MASTER)
self.uuid = uuid
neo.logging.info('UUID : %s', dump(uuid))
neo.lib.logging.info('UUID : %s', dump(uuid))
# election related data
self.unconnected_master_node_set = set()
......@@ -107,7 +107,7 @@ class Application(object):
try:
self._run()
except:
neo.logging.info('\nPre-mortem informations:')
neo.lib.logging.info('\nPre-mortem informations:')
self.log()
raise
......@@ -145,7 +145,7 @@ class Application(object):
others while attempting to connect to other master nodes at the
same time. Note that storage nodes and client nodes may connect
to self as well as master nodes."""
neo.logging.info('begin the election of a primary master')
neo.lib.logging.info('begin the election of a primary master')
self.unconnected_master_node_set.clear()
self.negotiating_master_node_set.clear()
......@@ -199,7 +199,7 @@ class Application(object):
for node in self.nm.getMasterList():
if not node.isRunning() and node.getLastStateChange() + \
expiration < current_time:
neo.logging.info('%s is down' % (node, ))
neo.lib.logging.info('%s is down' % (node, ))
node.setDown()
self.unconnected_master_node_set.discard(
node.getAddress())
......@@ -222,7 +222,7 @@ class Application(object):
Broadcast the announce that I'm the primary
"""
# I am the primary.
neo.logging.debug('I am the primary, sending an announcement')
neo.lib.logging.debug('I am the primary, sending an announcement')
for conn in self.em.getClientList():
conn.notify(Packets.AnnouncePrimary())
conn.abort()
......@@ -239,7 +239,7 @@ class Application(object):
"""
Ask other masters to reelect a primary after an election failure.
"""
neo.logging.error('election failed: %s', (m, ))
neo.lib.logging.error('election failed: %s', (m, ))
# Ask all connected nodes to reelect a single primary master.
for conn in self.em.getClientList():
......@@ -288,7 +288,7 @@ class Application(object):
def broadcastPartitionChanges(self, cell_list, selector=None):
"""Broadcast a Notify Partition Changes packet."""
neo.logging.debug('broadcastPartitionChanges')
neo.lib.logging.debug('broadcastPartitionChanges')
if not cell_list:
return
if not selector:
......@@ -308,7 +308,8 @@ class Application(object):
def broadcastLastOID(self):
oid = self.tm.getLastOID()
neo.logging.debug('Broadcast last OID to storages : %s' % dump(oid))
neo.lib.logging.debug(
'Broadcast last OID to storages : %s' % dump(oid))
packet = Packets.NotifyLastOID(oid)
for node in self.nm.getStorageList(only_identified=True):
node.notify(packet)
......@@ -319,7 +320,7 @@ class Application(object):
and stop the service only if a catastrophy happens or the user commits
a shutdown.
"""
neo.logging.info('provide service')
neo.lib.logging.info('provide service')
em = self.em
self.tm.reset()
......@@ -332,7 +333,7 @@ class Application(object):
except OperationFailure:
# If not operational, send Stop Operation packets to storage
# nodes and client nodes. Abort connections to client nodes.
neo.logging.critical('No longer operational')
neo.lib.logging.critical('No longer operational')
for node in self.nm.getIdentifiedList():
if node.isStorage() or node.isClient():
node.notify(Packets.StopOperation())
......@@ -345,7 +346,8 @@ class Application(object):
return
def playPrimaryRole(self):
neo.logging.info('play the primary role with %r', self.listening_conn)
neo.lib.logging.info(
'play the primary role with %r', self.listening_conn)
# i'm the primary, send the announcement
self._announcePrimary()
......@@ -382,7 +384,7 @@ class Application(object):
"""
I play a secondary role, thus only wait for a primary master to fail.
"""
neo.logging.info('play the secondary role with %r',
neo.lib.logging.info('play the secondary role with %r',
self.listening_conn)
# Wait for an announcement. If this is too long, probably
......@@ -496,7 +498,7 @@ class Application(object):
self.em.poll(1)
if self.cluster_state != ClusterStates.RUNNING:
neo.logging.info("asking all nodes to shutdown")
neo.lib.logging.info("asking all nodes to shutdown")
# This code sends packets but never polls, so they never reach
# network.
for node in self.nm.getIdentifiedList():
......@@ -533,7 +535,7 @@ class Application(object):
# always accept admin nodes
node_ctor = self.nm.createAdmin
handler = administration.AdministrationHandler(self)
neo.logging.info('Accept an admin %s' % (dump(uuid), ))
neo.lib.logging.info('Accept an admin %s' % (dump(uuid), ))
elif node_type == NodeTypes.MASTER:
if node is None:
# unknown master, rejected
......@@ -541,15 +543,15 @@ class Application(object):
# always put other master in waiting state
node_ctor = self.nm.createMaster
handler = secondary.SecondaryMasterHandler(self)
neo.logging.info('Accept a master %s' % (dump(uuid), ))
neo.lib.logging.info('Accept a master %s' % (dump(uuid), ))
elif node_type == NodeTypes.CLIENT:
# refuse any client before running
if self.cluster_state != ClusterStates.RUNNING:
neo.logging.info('Reject a connection from a client')
neo.lib.logging.info('Reject a connection from a client')
raise protocol.NotReadyError
node_ctor = self.nm.createClient
handler = client.ClientServiceHandler(self)
neo.logging.info('Accept a client %s' % (dump(uuid), ))
neo.lib.logging.info('Accept a client %s' % (dump(uuid), ))
elif node_type == NodeTypes.STORAGE:
node_ctor = self.nm.createStorage
if self._current_manager is not None:
......@@ -557,7 +559,8 @@ class Application(object):
(uuid, state, handler) = identify(uuid, node)
else:
(uuid, state, handler) = self.identifyStorageNode(uuid, node)
neo.logging.info('Accept a storage %s (%s)' % (dump(uuid), state))
neo.lib.logging.info('Accept a storage %s (%s)' %
(dump(uuid), state))
return (uuid, node, state, handler, node_ctor)
def onTransactionCommitted(self, txn):
......
......@@ -17,15 +17,16 @@
import neo
from neo.handler import EventHandler
from neo.protocol import NodeTypes, NodeStates, Packets
from neo.util import dump
from neo.lib.handler import EventHandler
from neo.lib.protocol import NodeTypes, NodeStates, Packets
from neo.lib.util import dump
class MasterHandler(EventHandler):
"""This class implements a generic part of the event handlers."""
def protocolError(self, conn, message):
neo.logging.error('Protocol error %s %s', message, conn.getAddress())
neo.lib.logging.error(
'Protocol error %s %s', message, conn.getAddress())
def askPrimary(self, conn):
app = self.app
......@@ -94,7 +95,7 @@ class BaseServiceHandler(MasterHandler):
if new_state != NodeStates.BROKEN and was_pending:
# 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
neo.logging.info('drop a pending node from the node manager')
neo.lib.logging.info('drop a pending node from the node manager')
self.app.nm.remove(node)
self.app.broadcastNodesInformation([node])
# clean node related data in specialized handlers
......
......@@ -18,9 +18,9 @@
import neo
from neo.master.handlers import MasterHandler
from neo.protocol import ClusterStates, NodeStates, Packets, ProtocolError
from neo.protocol import Errors
from neo.util import dump
from neo.lib.protocol import ClusterStates, NodeStates, Packets, ProtocolError
from neo.lib.protocol import Errors
from neo.lib.util import dump
CLUSTER_STATE_WORKFLOW = {
# destination: sources
......@@ -63,7 +63,7 @@ class AdministrationHandler(MasterHandler):
self.app.shutdown()
def setNodeState(self, conn, uuid, state, modify_partition_table):
neo.logging.info("set node state for %s-%s : %s" %
neo.lib.logging.info("set node state for %s-%s : %s" %
(dump(uuid), state, modify_partition_table))
app = self.app
node = app.nm.getByUUID(uuid)
......@@ -119,7 +119,7 @@ class AdministrationHandler(MasterHandler):
def addPendingNodes(self, conn, uuid_list):
uuids = ', '.join([dump(uuid) for uuid in uuid_list])
neo.logging.debug('Add nodes %s' % uuids)
neo.lib.logging.debug('Add nodes %s' % uuids)
app = self.app
nm = app.nm
em = app.em
......@@ -136,11 +136,11 @@ class AdministrationHandler(MasterHandler):
uuid_set = uuid_set.intersection(set(uuid_list))
# nothing to do
if not uuid_set:
neo.logging.warning('No nodes added')
neo.lib.logging.warning('No nodes added')
conn.answer(Errors.Ack('No nodes added'))
return
uuids = ', '.join([dump(uuid) for uuid in uuid_set])
neo.logging.info('Adding nodes %s' % uuids)
neo.lib.logging.info('Adding nodes %s' % uuids)
# switch nodes to running state
node_list = [nm.getByUUID(uuid) for uuid in uuid_set]
for node in node_list:
......
......@@ -15,11 +15,11 @@
# along with this program; if not, write to the Free Software
# Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
import neo
import neo.lib
from neo.protocol import NodeStates, Packets, ProtocolError
from neo.lib.protocol import NodeStates, Packets, ProtocolError
from neo.master.handlers import MasterHandler
from neo.util import dump
from neo.lib.util import dump
from neo.master.transactions import DelayedError
class ClientServiceHandler(MasterHandler):
......
......@@ -15,14 +15,15 @@
# along with this program; if not, write to the Free Software
# Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
import neo
import neo.lib
from neo.protocol import NodeTypes, Packets
from neo.protocol import NotReadyError, ProtocolError, UnexpectedPacketError
from neo.protocol import BrokenNodeDisallowedError
from neo.lib.protocol import NodeTypes, Packets
from neo.lib.protocol import NotReadyError, ProtocolError,
UnexpectedPacketError
from neo.lib.protocol import BrokenNodeDisallowedError
from neo.master.handlers import MasterHandler
from neo.exception import ElectionFailure
from neo.util import dump
from neo.lib.exception import ElectionFailure
from neo.lib.util import dump
class ClientElectionHandler(MasterHandler):
......@@ -90,7 +91,7 @@ class ClientElectionHandler(MasterHandler):
node = app.nm.getByAddress(conn.getAddress())
if node_type != NodeTypes.MASTER:
# The peer is not a master node!
neo.logging.error('%r is not a master node', conn)
neo.lib.logging.error('%r is not a master node', conn)
app.nm.remove(node)
app.negotiating_master_node_set.discard(node.getAddress())
conn.close()
......@@ -99,7 +100,8 @@ class ClientElectionHandler(MasterHandler):
if your_uuid != app.uuid:
# uuid conflict happened, accept the new one and restart election
app.uuid = your_uuid
neo.logging.info('UUID conflict, new UUID: %s', dump(your_uuid))
neo.lib.logging.info('UUID conflict, new UUID: %s',
dump(your_uuid))
raise ElectionFailure, 'new uuid supplied'
conn.setUUID(uuid)
......@@ -138,7 +140,8 @@ class ClientElectionHandler(MasterHandler):
if primary_node is None:
# I don't know such a node. Probably this information
# is old. So ignore it.
neo.logging.warning('received an unknown primary node UUID')
neo.lib.logging.warning(
'received an unknown primary node UUID')
else:
# Whatever the situation is, I trust this master.
app.primary = False
......@@ -196,11 +199,11 @@ class ServerElectionHandler(MasterHandler):
self.checkClusterName(name)
app = self.app
if node_type != NodeTypes.MASTER:
neo.logging.info('reject a connection from a non-master')
neo.lib.logging.info('reject a connection from a non-master')
raise NotReadyError
node = app.nm.getByAddress(address)
if node is None:
neo.logging.error('unknown master node: %s' % (address, ))
neo.lib.logging.error('unknown master node: %s' % (address, ))
raise ProtocolError('unknown master node')
# If this node is broken, reject it.
if node.getUUID() == uuid:
......@@ -236,5 +239,5 @@ class ServerElectionHandler(MasterHandler):
app.primary_master_node = node
app.unconnected_master_node_set.clear()
app.negotiating_master_node_set.clear()
neo.logging.info('%s is the primary', node)
neo.lib.logging.info('%s is the primary', node)
......@@ -17,15 +17,16 @@
import neo
from neo.protocol import NodeTypes, Packets
from neo.protocol import BrokenNodeDisallowedError, ProtocolError
from neo.lib.protocol import NodeTypes, Packets
from neo.lib.protocol import BrokenNodeDisallowedError, ProtocolError
from neo.master.handlers import MasterHandler
class IdentificationHandler(MasterHandler):
"""This class deals with messages from the admin node only"""
def nodeLost(self, conn, node):
neo.logging.warning('lost a node in IdentificationHandler : %s' % node)
neo.lib.logging.warning('
lost a node in IdentificationHandler : %s' % node)
def requestIdentification(self, conn, node_type, uuid, address, name):
......
......@@ -16,8 +16,8 @@
# Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
from neo.master.handlers import MasterHandler
from neo.exception import ElectionFailure, PrimaryFailure
from neo.protocol import NodeTypes, Packets
from neo.lib.exception import ElectionFailure, PrimaryFailure
from neo.lib.protocol import NodeTypes, Packets
class SecondaryMasterHandler(MasterHandler):
""" Handler used by primary to handle secondary masters"""
......
......@@ -15,8 +15,8 @@
# along with this program; if not, write to the Free Software
# Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
import neo
from neo import protocol
import neo.lib
from neo.lib import protocol
from neo.master.handlers import BaseServiceHandler
class ShutdownHandler(BaseServiceHandler):
......@@ -24,15 +24,15 @@ class ShutdownHandler(BaseServiceHandler):
def requestIdentification(self, conn, node_type,
uuid, address, name):
neo.logging.error('reject any new connection')
neo.lib.logging.error('reject any new connection')
raise protocol.ProtocolError('cluster is shutting down')
def askPrimary(self, conn):
neo.logging.error('reject any new demand for primary master')
neo.lib.logging.error('reject any new demand for primary master')
raise protocol.ProtocolError('cluster is shutting down')
def askBeginTransaction(self, conn, tid):
neo.logging.error('reject any new demand for new tid')
neo.lib.logging.error('reject any new demand for new tid')
raise protocol.ProtocolError('cluster is shutting down')
......@@ -15,15 +15,15 @@
# along with this program; if not, write to the Free Software
# Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
import neo
import neo.lib
from neo.protocol import ProtocolError
from neo.protocol import Packets
from neo.lib.protocol import ProtocolError
from neo.lib.protocol import Packets
from neo.master.handlers import BaseServiceHandler
from neo.exception import OperationFailure
from neo.util import dump
from neo.connector import ConnectorConnectionClosedException
from neo.pt import PartitionTableException
from neo.lib.exception import OperationFailure
from neo.lib.util import dump
from neo.lib.connector import ConnectorConnectionClosedException
from neo.lib.pt import PartitionTableException
class StorageServiceHandler(BaseServiceHandler):
......@@ -40,7 +40,7 @@ class StorageServiceHandler(BaseServiceHandler):
conn.notify(Packets.StartOperation())
def nodeLost(self, conn, node):
neo.logging.info('storage node lost')
neo.lib.logging.info('storage node lost')
assert not node.isRunning(), node.getState()
if not self.app.pt.operational():
......@@ -71,7 +71,7 @@ class StorageServiceHandler(BaseServiceHandler):
def notifyReplicationDone(self, conn, offset):
node = self.app.nm.getByUUID(conn.getUUID())
neo.logging.debug("%s is up for offset %s" % (node, offset))
neo.lib.logging.debug("%s is up for offset %s" % (node, offset))
try:
cell_list = self.app.pt.setUpToDate(node, offset)
except PartitionTableException, e:
......
......@@ -15,12 +15,13 @@
# along with this program; if not, write to the Free Software
# Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
import neo.pt
import neo.lib.pt
from struct import pack, unpack
from neo.protocol import CellStates
from neo.pt import PartitionTableException
from neo.lib.protocol import CellStates
from neo.lib.pt import PartitionTableException
from neo.lib.pt import PartitionTable
class PartitionTable(neo.pt.PartitionTable):
class PartitionTable(PartitionTable):
"""This class manages a partition table for the primary master node"""
def setID(self, id):
......@@ -53,7 +54,7 @@ class PartitionTable(neo.pt.PartitionTable):
row = []
for _ in xrange(repeats):
node = node_list[index]
row.append(neo.pt.Cell(node))
row.append(neo.lib.pt.Cell(node))
self.count_dict[node] = self.count_dict.get(node, 0) + 1
index += 1
if index == len(node_list):
......@@ -87,7 +88,8 @@ class PartitionTable(neo.pt.PartitionTable):
node_list = [c.getNode() for c in row]
n = self.findLeastUsedNode(node_list)
if n is not None:
row.append(neo.pt.Cell(n, CellStates.OUT_OF_DATE))
row.append(neo.lib.pt.Cell(n,
CellStates.OUT_OF_DATE))
self.count_dict[n] += 1
cell_list.append((offset, n.getUUID(),
CellStates.OUT_OF_DATE))
......@@ -180,7 +182,7 @@ class PartitionTable(neo.pt.PartitionTable):
continue
if num_cells <= self.nr:
row.append(neo.pt.Cell(node, CellStates.OUT_OF_DATE))
row.append(neo.lib.pt.Cell(node, CellStates.OUT_OF_DATE))
cell_list.append((offset, node.getUUID(),
CellStates.OUT_OF_DATE))
node_count += 1
......@@ -200,7 +202,7 @@ class PartitionTable(neo.pt.PartitionTable):
CellStates.FEEDING))
# Don't count a feeding cell.
self.count_dict[max_cell.getNode()] -= 1
row.append(neo.pt.Cell(node, CellStates.OUT_OF_DATE))
row.append(neo.lib.pt.Cell(node, CellStates.OUT_OF_DATE))
cell_list.append((offset, node.getUUID(),
CellStates.OUT_OF_DATE))
node_count += 1
......@@ -281,7 +283,7 @@ class PartitionTable(neo.pt.PartitionTable):
node = self.findLeastUsedNode([cell.getNode() for cell in row])
if node is None:
break
row.append(neo.pt.Cell(node, CellStates.OUT_OF_DATE))
row.append(neo.lib.pt.Cell(node, CellStates.OUT_OF_DATE))
changed_cell_list.append((offset, node.getUUID(),
CellStates.OUT_OF_DATE))
self.count_dict[node] += 1
......
......@@ -18,9 +18,9 @@
from struct import pack
import neo
from neo.util import dump
from neo.protocol import Packets, ProtocolError, ClusterStates, NodeStates
from neo.protocol import NotReadyError, ZERO_OID, ZERO_TID
from neo.lib.util import dump
from neo.lib.protocol import Packets, ProtocolError, ClusterStates, NodeStates
from neo.lib.protocol import NotReadyError, ZERO_OID, ZERO_TID
from neo.master.handlers import MasterHandler
REQUIRED_NODE_NUMBER = 1
......@@ -43,7 +43,7 @@ class RecoveryManager(MasterHandler):
Returns the handler for storage nodes
"""
if uuid is None and not self.app._startup_allowed:
neo.logging.info('reject empty storage node')
neo.lib.logging.info('reject empty storage node')
raise NotReadyError
return (uuid, NodeStates.RUNNING, self)
......@@ -54,7 +54,7 @@ class RecoveryManager(MasterHandler):
back the latest partition table or make a new table from scratch,
if this is the first time.
"""
neo.logging.info('begin the recovery of the status')
neo.lib.logging.info('begin the recovery of the status')
self.app.changeClusterState(ClusterStates.RECOVERING)
em = self.app.em
......@@ -66,7 +66,7 @@ class RecoveryManager(MasterHandler):
while not self.app._startup_allowed:
em.poll(1)
neo.logging.info('startup allowed')
neo.lib.logging.info('startup allowed')
# build a new partition table
if self.app.pt.getID() is None:
......@@ -81,13 +81,14 @@ class RecoveryManager(MasterHandler):
self.app.broadcastNodesInformation(refused_node_set)
self.app.setLastTransaction(self.app.tm.getLastTID())
neo.logging.debug('cluster starts with loid=%s and this partition ' \
'table :', dump(self.app.tm.getLastOID()))
neo.lib.logging.debug(
'cluster starts with loid=%s and this partition ' \
'table :', dump(self.app.tm.getLastOID()))
self.app.pt.log()
def buildFromScratch(self):
nm, em, pt = self.app.nm, self.app.em, self.app.pt
neo.logging.debug('creating a new partition table, wait for a ' \
neo.lib.logging.debug('creating a new partition table, wait for a ' \
'storage node')
# wait for some empty storage nodes, their are accepted
while len(nm.getStorageList()) < REQUIRED_NODE_NUMBER:
......@@ -131,7 +132,7 @@ class RecoveryManager(MasterHandler):
def answerPartitionTable(self, conn, ptid, row_list):
if ptid != self.target_ptid:
# If this is not from a target node, ignore it.
neo.logging.warn('Got %s while waiting %s', dump(ptid),
neo.lib.logging.warn('Got %s while waiting %s', dump(ptid),
dump(self.target_ptid))
return
try:
......
......@@ -17,10 +17,10 @@
from time import time, gmtime
from struct import pack, unpack
from neo.protocol import ZERO_TID
from neo.lib.protocol import ZERO_TID
from datetime import timedelta, datetime
from neo.util import dump, u64, p64
import neo
from neo.lib.util import dump, u64, p64
import neo.lib
TID_LOW_OVERFLOW = 2**32
TID_LOW_MAX = TID_LOW_OVERFLOW - 1
......@@ -347,7 +347,8 @@ class TransactionManager(object):
else:
tid = self._nextTID(ttid, divisor)
self._queue.append((node.getUUID(), ttid))
neo.logging.debug('Finish TXN %s for %s (was %s)', dump(tid), node, dump(ttid))
neo.lib.logging.debug('Finish TXN %s for %s (was %s)',
dump(tid), node, dump(ttid))
txn = Transaction(node, ttid, tid, oid_list, uuid_list, msg_id)
self._ttid_dict[ttid] = txn
self._node_dict.setdefault(node, {})[ttid] = txn
......@@ -418,7 +419,7 @@ class TransactionManager(object):
"""
Abort pending transactions initiated by a node
"""
neo.logging.debug('Abort TXN for %s', node)
neo.lib.logging.debug('Abort TXN for %s', node)
uuid = node.getUUID()
# XXX: this loop is usefull only during an import
for nuuid, ntid in list(self._queue):
......@@ -433,7 +434,7 @@ class TransactionManager(object):
del self._node_dict[node]
def log(self):
neo.logging.info('Transactions:')
neo.lib.logging.info('Transactions:')
for txn in self._ttid_dict.itervalues():
neo.logging.info(' %r', txn)
neo.lib.logging.info(' %r', txn)
......@@ -16,8 +16,8 @@
# Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
import neo
from neo.util import dump
from neo.protocol import ClusterStates, Packets, NodeStates
from neo.lib.util import dump
from neo.lib.protocol import ClusterStates, Packets, NodeStates
from neo.master.handlers import BaseServiceHandler
......@@ -117,11 +117,11 @@ class VerificationManager(BaseServiceHandler):
em, nm = self.app.em, self.app.nm
# wait for any missing node
neo.logging.debug('waiting for the cluster to be operational')
neo.lib.logging.debug('waiting for the cluster to be operational')
while not self.app.pt.operational():
em.poll(1)
neo.logging.info('start to verify data')
neo.lib.logging.info('start to verify data')
# Gather all unfinished transactions.
self._askStorageNodesAndWait(Packets.AskUnfinishedTransactions(),
......@@ -197,7 +197,7 @@ class VerificationManager(BaseServiceHandler):
def answerUnfinishedTransactions(self, conn, tid_list):
uuid = conn.getUUID()
neo.logging.info('got unfinished transactions %s from %r',
neo.lib.logging.info('got unfinished transactions %s from %r',
[dump(tid) for tid in tid_list], conn)
if not self._gotAnswerFrom(uuid):
return
......@@ -222,19 +222,19 @@ class VerificationManager(BaseServiceHandler):
def tidNotFound(self, conn, message):
uuid = conn.getUUID()
neo.logging.info('TID not found: %s', message)
neo.lib.logging.info('TID not found: %s', message)
if not self._gotAnswerFrom(uuid):
return
self._oid_set = None
def answerObjectPresent(self, conn, oid, tid):
uuid = conn.getUUID()
neo.logging.info('object %s:%s found', dump(oid), dump(tid))
neo.lib.logging.info('object %s:%s found', dump(oid), dump(tid))
self._gotAnswerFrom(uuid)
def oidNotFound(self, conn, message):
uuid = conn.getUUID()
neo.logging.info('OID not found: %s', message)
neo.lib.logging.info('OID not found: %s', message)
app = self.app
if not self._gotAnswerFrom(uuid):
return
......
......@@ -16,8 +16,8 @@
# Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
from neo.neoctl.neoctl import NeoCTL, NotReadyException
from neo.util import bin, dump
from neo.protocol import ClusterStates, NodeStates, NodeTypes
from neo.lib.util import bin, dump
from neo.lib.protocol import ClusterStates, NodeStates, NodeTypes
action_dict = {
'print': {
......
......@@ -15,8 +15,8 @@
# along with this program; if not, write to the Free Software
# Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
from neo.handler import EventHandler
from neo.protocol import ErrorCodes, Packets
from neo.lib.handler import EventHandler
from neo.lib.protocol import ErrorCodes, Packets
class CommandEventHandler(EventHandler):
""" Base handler for command """
......
......@@ -15,11 +15,11 @@
# along with this program; if not, write to the Free Software
# Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
from neo.connector import getConnectorHandler
from neo.connection import ClientConnection
from neo.event import EventManager
from neo.lib.connector import getConnectorHandler
from neo.lib.connection import ClientConnection
from neo.lib.event import EventManager
from neo.neoctl.handler import CommandEventHandler
from neo.protocol import ClusterStates, NodeStates, ErrorCodes, Packets
from neo.lib.protocol import ClusterStates, NodeStates, ErrorCodes, Packets
class NotReadyException(Exception):
pass
......
import pkg_resources
pkg_resources.declare_namespace(__name__)
......@@ -17,8 +17,8 @@
# Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
from optparse import OptionParser
from neo import setupLog
from neo.config import ConfigurationManager
from neo.lib import setupLog
from neo.lib.config import ConfigurationManager
parser = OptionParser()
parser.add_option('-u', '--uuid', help='specify an UUID to use for this ' \
......
......@@ -18,7 +18,7 @@
import sys
from optparse import OptionParser
from neo import setupLog
from neo.lib import setupLog
parser = OptionParser()
parser.add_option('-v', '--verbose', action = 'store_true',
......
......@@ -17,8 +17,8 @@
# Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
from optparse import OptionParser
from neo import setupLog
from neo.config import ConfigurationManager
from neo.lib import setupLog
from neo.lib.config import ConfigurationManager
parser = OptionParser()
parser.add_option('-v', '--verbose', action = 'store_true',
......
......@@ -23,7 +23,7 @@ import logging
import time
import os
from neo import setupLog
from neo.lib import setupLog
# register options
parser = OptionParser()
......
......@@ -19,8 +19,8 @@
# Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
from optparse import OptionParser
from neo import setupLog
from neo.config import ConfigurationManager
from neo.lib import setupLog
from neo.lib.config import ConfigurationManager
parser = OptionParser()
......
......@@ -19,23 +19,23 @@ import neo
import sys
from collections import deque
from neo.protocol import NodeTypes, CellStates, Packets
from neo.node import NodeManager
from neo.event import EventManager
from neo.connection import ListeningConnection
from neo.exception import OperationFailure, PrimaryFailure
from neo.lib.protocol import NodeTypes, CellStates, Packets
from neo.lib.node import NodeManager
from neo.lib.event import EventManager
from neo.lib.connection import ListeningConnection
from neo.lib.exception import OperationFailure, PrimaryFailure
from neo.storage.handlers import identification, verification, initialization
from neo.storage.handlers import master, hidden
from neo.storage.replicator import Replicator
from neo.storage.database import buildDatabaseManager
from neo.storage.transactions import TransactionManager
from neo.storage.exception import AlreadyPendingError
from neo.connector import getConnectorHandler
from neo.pt import PartitionTable
from neo.util import dump
from neo.bootstrap import BootstrapManager
from neo.lib.connector import getConnectorHandler
from neo.lib.pt import PartitionTable
from neo.lib.util import dump
from neo.lib.bootstrap import BootstrapManager
from neo.live_debug import register as registerLiveDebugger
from neo.lib.live_debug import register as registerLiveDebugger
class Application(object):
"""The storage node application."""
......@@ -59,7 +59,7 @@ class Application(object):
# set the bind address
self.server = config.getBind()
neo.logging.debug('IP address is %s, port is %d', *(self.server))
neo.lib.logging.debug('IP address is %s, port is %d', *(self.server))
# The partition table is initialized after getting the number of
# partitions.
......@@ -131,12 +131,12 @@ class Application(object):
# create a partition table
self.pt = PartitionTable(num_partitions, num_replicas)
neo.logging.info('Configuration loaded:')
neo.logging.info('UUID : %s', dump(self.uuid))
neo.logging.info('PTID : %s', dump(ptid))
neo.logging.info('Name : %s', self.name)
neo.logging.info('Partitions: %s', num_partitions)
neo.logging.info('Replicas : %s', num_replicas)
neo.lib.logging.info('Configuration loaded:')
neo.lib.logging.info('UUID : %s', dump(self.uuid))
neo.lib.logging.info('PTID : %s', dump(ptid))
neo.lib.logging.info('Name : %s', self.name)
neo.lib.logging.info('Partitions: %s', num_partitions)
neo.lib.logging.info('Replicas : %s', num_replicas)
def loadPartitionTable(self):
"""Load a partition table from the database."""
......@@ -161,7 +161,7 @@ class Application(object):
try:
self._run()
except:
neo.logging.info('\nPre-mortem informations:')
neo.lib.logging.info('\nPre-mortem informations:')
self.log()
raise
......@@ -202,10 +202,10 @@ class Application(object):
self.doOperation()
raise RuntimeError, 'should not reach here'
except OperationFailure, msg:
neo.logging.error('operation stopped: %s', msg)
neo.lib.logging.error('operation stopped: %s', msg)
except PrimaryFailure, msg:
self.replicator.masterLost()
neo.logging.error('primary master is down: %s', msg)
neo.lib.logging.error('primary master is down: %s', msg)
self.master_node = None
def connectToPrimary(self):
......@@ -230,7 +230,7 @@ class Application(object):
(node, conn, uuid, num_partitions, num_replicas) = data
self.master_node = node
self.master_conn = conn
neo.logging.info('I am %s', dump(uuid))
neo.lib.logging.info('I am %s', dump(uuid))
self.uuid = uuid
self.dm.setUUID(uuid)
......@@ -252,7 +252,7 @@ class Application(object):
def verifyData(self):
"""Verify data under the control by a primary master node.
Connections from client nodes may not be accepted at this stage."""
neo.logging.info('verifying data')
neo.lib.logging.info('verifying data')
handler = verification.VerificationHandler(self)
self.master_conn.setHandler(handler)
......@@ -263,7 +263,7 @@ class Application(object):
def initialize(self):
""" Retreive partition table and node informations from the primary """
neo.logging.debug('initializing...')
neo.lib.logging.debug('initializing...')
_poll = self._poll
handler = initialization.InitializationHandler(self)
self.master_conn.setHandler(handler)
......@@ -285,7 +285,7 @@ class Application(object):
def doOperation(self):
"""Handle everything, including replications and transactions."""
neo.logging.info('doing operation')
neo.lib.logging.info('doing operation')
_poll = self._poll
......@@ -309,7 +309,7 @@ class Application(object):
def wait(self):
# change handler
neo.logging.info("waiting in hidden state")
neo.lib.logging.info("waiting in hidden state")
_poll = self._poll
handler = hidden.HiddenHandler(self)
for conn in self.em.getConnectionList():
......@@ -355,9 +355,9 @@ class Application(object):
def logQueuedEvents(self):
if self.event_queue is None:
return
neo.logging.info("Pending events:")
neo.lib.logging.info("Pending events:")
for key, event, _msg_id, _conn, args in self.event_queue:
neo.logging.info(' %r:%r: %r:%r %r %r', key, event.__name__,
neo.lib.logging.info(' %r:%r: %r:%r %r %r', key, event.__name__,
_msg_id, _conn, args)
def shutdown(self, erase=False):
......
......@@ -15,7 +15,7 @@
# along with this program; if not, write to the Free Software
# Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
from neo.exception import DatabaseFailure
from neo.lib.exception import DatabaseFailure
from neo.storage.database.manager import DatabaseManager
DATABASE_MANAGER_DICT = {}
......
......@@ -21,11 +21,11 @@ Not persistent ! (no data retained after process exit)
"""
from BTrees.OOBTree import OOBTree as _OOBTree
import neo
import neo.lib
from neo.storage.database import DatabaseManager
from neo.protocol import CellStates
from neo import util
from neo.lib.protocol import CellStates
from neo.lib import util
# The only purpose of this value (and code using it) is to avoid creating
# arbitrarily-long lists of values when cleaning up dictionaries.
......@@ -266,7 +266,7 @@ class BTreeDatabaseManager(DatabaseManager):
except KeyError:
raise IndexError(value_serial)
if value is None:
neo.logging.info("Multiple levels of indirection when " \
neo.lib.logging.info("Multiple levels of indirection when " \
"searching for object data for oid %d at tid %d. This " \
"causes suboptimal performance." % (oid, value_serial))
value_serial, compression, checksum, value = self._getObjectData(
......@@ -522,7 +522,7 @@ class BTreeDatabaseManager(DatabaseManager):
raise CreationUndone
_, _, value, value_serial = self._obj[oid][value_serial]
if value is None:
neo.logging.info("Multiple levels of indirection when " \
neo.lib.logging.info("Multiple levels of indirection when " \
"searching for object data for oid %d at tid %d. This " \
"causes suboptimal performance." % (oid, value_serial))
length = self._getObjectLength(oid, value_serial)
......
......@@ -15,8 +15,8 @@
# along with this program; if not, write to the Free Software
# Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
from neo import util
from neo.exception import DatabaseFailure
from neo.lib import util
from neo.lib.exception import DatabaseFailure
class CreationUndone(Exception):
pass
......
......@@ -18,15 +18,15 @@
import MySQLdb
from MySQLdb import OperationalError
from MySQLdb.constants.CR import SERVER_GONE_ERROR, SERVER_LOST
import neo
import neo.lib
from array import array
import string
from neo.storage.database import DatabaseManager
from neo.storage.database.manager import CreationUndone
from neo.exception import DatabaseFailure
from neo.protocol import CellStates, ZERO_OID, ZERO_TID
from neo import util
from neo.lib.exception import DatabaseFailure
from neo.lib.protocol import CellStates, ZERO_OID, ZERO_TID
from neo.lib import util
LOG_QUERIES = False
......@@ -68,7 +68,8 @@ class MySQLDatabaseManager(DatabaseManager):
kwd = {'db' : self.db, 'user' : self.user}
if self.passwd is not None:
kwd['passwd'] = self.passwd
neo.logging.info('connecting to MySQL on the database %s with user %s',
neo.lib.logging.info(
'connecting to MySQL on the database %s with user %s',
self.db, self.user)
self.conn = MySQLdb.connect(**kwd)
self.conn.autocommit(False)
......@@ -78,12 +79,12 @@ class MySQLDatabaseManager(DatabaseManager):
def _commit(self):
if LOG_QUERIES:
neo.logging.debug('committing...')
neo.lib.logging.debug('committing...')
self.conn.commit()
def _rollback(self):
if LOG_QUERIES:
neo.logging.debug('aborting...')
neo.lib.logging.debug('aborting...')
self.conn.rollback()
def query(self, query):
......@@ -97,7 +98,7 @@ class MySQLDatabaseManager(DatabaseManager):
c = '\\x%02x' % ord(c)
printable_char_list.append(c)
query_part = ''.join(printable_char_list)
neo.logging.debug('querying %s...', query_part)
neo.lib.logging.debug('querying %s...', query_part)
conn.query(query)
r = conn.store_result()
......@@ -114,7 +115,7 @@ class MySQLDatabaseManager(DatabaseManager):
except OperationalError, m:
if m[0] in (SERVER_GONE_ERROR, SERVER_LOST):
neo.logging.info('the MySQL server is gone; reconnecting')
neo.lib.logging.info('the MySQL server is gone; reconnecting')
self._connect()
return self.query(query)
raise DatabaseFailure('MySQL error %d: %s' % (m[0], m[1]))
......@@ -324,7 +325,7 @@ class MySQLDatabaseManager(DatabaseManager):
})
compression, checksum, value, next_value_serial = r[0]
if value is None:
neo.logging.info("Multiple levels of indirection when " \
neo.lib.logging.info("Multiple levels of indirection when " \
"searching for object data for oid %d at tid %d. This " \
"causes suboptimal performance." % (oid, value_serial))
value_serial, compression, checksum, value = self._getObjectData(
......@@ -634,7 +635,7 @@ class MySQLDatabaseManager(DatabaseManager):
(self._getPartition(oid), oid, value_serial))
length, value_serial = r[0]
if length is None:
neo.logging.info("Multiple levels of indirection when " \
neo.lib.logging.info("Multiple levels of indirection when " \
"searching for object data for oid %d at tid %d. This " \
"causes suboptimal performance." % (oid, value_serial))
length = self._getObjectLength(oid, value_serial)
......
......@@ -17,11 +17,11 @@
import neo
from neo.handler import EventHandler
from neo import protocol
from neo.util import dump
from neo.exception import PrimaryFailure, OperationFailure
from neo.protocol import NodeStates, NodeTypes, Packets, Errors
from neo.lib.handler import EventHandler
from neo.lib import protocol
from neo.lib.util import dump
from neo.lib.exception import PrimaryFailure, OperationFailure
from neo.lib.protocol import NodeStates, NodeTypes, Packets, Errors
class BaseMasterHandler(EventHandler):
......@@ -35,7 +35,7 @@ class BaseMasterHandler(EventHandler):
raise PrimaryFailure('re-election occurs')
def notifyClusterInformation(self, conn, state):
neo.logging.warning('ignoring notify cluster information in %s' %
neo.lib.logging.warning('ignoring notify cluster information in %s' %
self.__class__.__name__)
def notifyLastOID(self, conn, oid):
......@@ -48,7 +48,7 @@ class BaseMasterHandler(EventHandler):
for node_type, addr, uuid, state in node_list:
if uuid == self.app.uuid:
# This is me, do what the master tell me
neo.logging.info("I was told I'm %s" %(state))
neo.lib.logging.info("I was told I'm %s" %(state))
if state in (NodeStates.DOWN, NodeStates.TEMPORARILY_DOWN,
NodeStates.BROKEN):
conn.close()
......@@ -57,7 +57,8 @@ class BaseMasterHandler(EventHandler):
elif state == NodeStates.HIDDEN:
raise OperationFailure
elif node_type == NodeTypes.CLIENT and state != NodeStates.RUNNING:
neo.logging.info('Notified of non-running client, abort (%r)',
neo.lib.logging.info(
'Notified of non-running client, abort (%r)',
dump(uuid))
self.app.tm.abortFor(uuid)
......@@ -86,14 +87,14 @@ class BaseClientAndStorageOperationHandler(EventHandler):
return
o = self._askObject(oid, serial, tid)
if o is None:
neo.logging.debug('oid = %s does not exist', dump(oid))
neo.lib.logging.debug('oid = %s does not exist', dump(oid))
p = Errors.OidDoesNotExist(dump(oid))
elif o is False:
neo.logging.debug('oid = %s not found', dump(oid))
neo.lib.logging.debug('oid = %s not found', dump(oid))
p = Errors.OidNotFound(dump(oid))
else:
serial, next_serial, compression, checksum, data, data_serial = o
neo.logging.debug('oid = %s, serial = %s, next_serial = %s',
neo.lib.logging.debug('oid = %s, serial = %s, next_serial = %s',
dump(oid), dump(serial), dump(next_serial))
p = Packets.AnswerObject(oid, serial, next_serial,
compression, checksum, data, data_serial)
......
......@@ -15,10 +15,10 @@
# along with this program; if not, write to the Free Software
# Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
import neo
from neo import protocol
from neo.util import dump
from neo.protocol import Packets, LockState, Errors
import neo.lib
from neo.lib import protocol
from neo.lib.util import dump
from neo.lib.protocol import Packets, LockState, Errors
from neo.storage.handlers import BaseClientAndStorageOperationHandler
from neo.storage.transactions import ConflictError, DelayedError
from neo.storage.exception import AlreadyPendingError
......@@ -51,7 +51,7 @@ class ClientOperationHandler(BaseClientAndStorageOperationHandler):
data_serial, ttid, unlock, request_time):
if ttid not in self.app.tm:
# transaction was aborted, cancel this event
neo.logging.info('Forget store of %s:%s by %s delayed by %s',
neo.lib.logging.info('Forget store of %s:%s by %s delayed by %s',
dump(oid), dump(serial), dump(ttid),
dump(self.app.tm.getLockingTID(oid)))
# send an answer as the client side is waiting for it
......@@ -80,7 +80,7 @@ class ClientOperationHandler(BaseClientAndStorageOperationHandler):
if SLOW_STORE is not None:
duration = time.time() - request_time
if duration > SLOW_STORE:
neo.logging.info('StoreObject delay: %.02fs', duration)
neo.lib.logging.info('StoreObject delay: %.02fs', duration)
conn.answer(Packets.AnswerStoreObject(0, oid, serial))
def askStoreObject(self, conn, oid, serial,
......@@ -140,7 +140,8 @@ class ClientOperationHandler(BaseClientAndStorageOperationHandler):
def askHasLock(self, conn, ttid, oid):
locking_tid = self.app.tm.getLockingTID(oid)
neo.logging.info('%r check lock of %r:%r', conn, dump(ttid), dump(oid))
neo.lib.logging.info('%r check lock of %r:%r', conn,
dump(ttid), dump(oid))
if locking_tid is None:
state = LockState.NOT_LOCKED
elif locking_tid is ttid:
......@@ -167,7 +168,8 @@ class ClientOperationHandler(BaseClientAndStorageOperationHandler):
def _askCheckCurrentSerial(self, conn, ttid, serial, oid, request_time):
if ttid not in self.app.tm:
# transaction was aborted, cancel this event
neo.logging.info('Forget serial check of %s:%s by %s delayed by '
neo.lib.logging.info(
'Forget serial check of %s:%s by %s delayed by '
'%s', dump(oid), dump(serial), dump(ttid),
dump(self.app.tm.getLockingTID(oid)))
# send an answer as the client side is waiting for it
......@@ -190,7 +192,7 @@ class ClientOperationHandler(BaseClientAndStorageOperationHandler):
if SLOW_STORE is not None:
duration = time.time() - request_time
if duration > SLOW_STORE:
neo.logging.info('CheckCurrentSerial delay: %.02fs',
neo.lib.logging.info('CheckCurrentSerial delay: %.02fs',
duration)
conn.answer(Packets.AnswerCheckCurrentSerial(0, oid, serial))
......@@ -15,10 +15,10 @@
# along with this program; if not, write to the Free Software
# Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
import neo
import neo.lib
from neo.storage.handlers import BaseMasterHandler
from neo.protocol import CellStates
from neo.lib.protocol import CellStates
class HiddenHandler(BaseMasterHandler):
"""This class implements a generic part of the event handlers."""
......@@ -29,7 +29,7 @@ class HiddenHandler(BaseMasterHandler):
app = self.app
if ptid <= app.pt.getID():
# Ignore this packet.
neo.logging.debug('ignoring older partition changes')
neo.lib.logging.debug('ignoring older partition changes')
return
# update partition table in memory and the database
......
......@@ -15,18 +15,18 @@
# along with this program; if not, write to the Free Software
# Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
import neo
import neo.lib
from neo.handler import EventHandler
from neo.protocol import NodeTypes, Packets, NotReadyError
from neo.protocol import ProtocolError, BrokenNodeDisallowedError
from neo.util import dump
from neo.lib.handler import EventHandler
from neo.lib.protocol import NodeTypes, Packets, NotReadyError
from neo.lib.protocol import ProtocolError, BrokenNodeDisallowedError
from neo.lib.util import dump
class IdentificationHandler(EventHandler):
""" Handler used for incoming connections during operation state """
def connectionLost(self, conn, new_state):
neo.logging.warning('A connection was lost during identification')
neo.lib.logging.warning('A connection was lost during identification')
def requestIdentification(self, conn, node_type,
uuid, address, name):
......@@ -54,7 +54,7 @@ class IdentificationHandler(EventHandler):
from neo.storage.handlers.storage import StorageOperationHandler
handler = StorageOperationHandler
if node is None:
neo.logging.error('reject an unknown storage node %s',
neo.lib.logging.error('reject an unknown storage node %s',
dump(uuid))
raise NotReadyError
else:
......
......@@ -15,10 +15,10 @@
# along with this program; if not, write to the Free Software
# Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
import neo
import neo.lib
from neo.storage.handlers import BaseMasterHandler
from neo import protocol
from neo.lib import protocol
class InitializationHandler(BaseMasterHandler):
......@@ -35,7 +35,7 @@ class InitializationHandler(BaseMasterHandler):
pt.load(ptid, row_list, self.app.nm)
if not pt.filled():
raise protocol.ProtocolError('Partial partition table received')
neo.logging.debug('Got the partition table :')
neo.lib.logging.debug('Got the partition table :')
self.app.pt.log()
# Install the partition table into the database for persistency.
cell_list = []
......@@ -48,7 +48,8 @@ class InitializationHandler(BaseMasterHandler):
unassigned_set.remove(offset)
# delete objects database
if unassigned_set:
neo.logging.debug('drop data for partitions %r' % unassigned_set)
neo.lib.logging.debug(
'drop data for partitions %r' % unassigned_set)
app.dm.dropPartitions(num_partitions, unassigned_set)
app.dm.setPartitionTable(ptid, cell_list)
......@@ -66,5 +67,5 @@ class InitializationHandler(BaseMasterHandler):
# packets in between (or even before asking for node information).
# - this handler will be changed after receiving answerPartitionTable
# and before handling the next packet
neo.logging.debug('ignoring notifyPartitionChanges during '\
neo.lib.logging.debug('ignoring notifyPartitionChanges during '\
'initialization')
......@@ -15,9 +15,9 @@
# along with this program; if not, write to the Free Software
# Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
import neo
from neo.util import dump
from neo.protocol import CellStates, Packets, ProtocolError
import neo.lib
from neo.lib.util import dump
from neo.lib.protocol import CellStates, Packets, ProtocolError
from neo.storage.handlers import BaseMasterHandler
......@@ -36,7 +36,7 @@ class MasterOperationHandler(BaseMasterHandler):
app = self.app
if ptid <= app.pt.getID():
# Ignore this packet.
neo.logging.debug('ignoring older partition changes')
neo.lib.logging.debug('ignoring older partition changes')
return
# update partition table in memory and the database
......@@ -68,9 +68,9 @@ class MasterOperationHandler(BaseMasterHandler):
def askPack(self, conn, tid):
app = self.app
neo.logging.info('Pack started, up to %s...', dump(tid))
neo.lib.logging.info('Pack started, up to %s...', dump(tid))
app.dm.pack(tid, app.tm.updateObjectDataForPack)
neo.logging.info('Pack finished.')
neo.lib.logging.info('Pack finished.')
if not conn.isClosed():
conn.answer(Packets.AnswerPack(True))
......@@ -16,11 +16,11 @@
# along with this program; if not, write to the Free Software
# Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
import neo
import neo.lib
from neo.handler import EventHandler
from neo.protocol import Packets, ZERO_TID, ZERO_OID
from neo.util import add64
from neo.lib.handler import EventHandler
from neo.lib.protocol import Packets, ZERO_TID, ZERO_OID
from neo.lib.util import add64
# TODO: benchmark how different values behave
RANGE_LENGTH = 4000
......@@ -82,11 +82,13 @@ class ReplicationHandler(EventHandler):
"""This class handles events for replications."""
def connectionLost(self, conn, new_state):
neo.logging.error('replication is stopped due to a connection lost')
neo.lib.logging.error(
'replication is stopped due to a connection lost')
self.app.replicator.storageLost()
def connectionFailed(self, conn):
neo.logging.error('replication is stopped due to connection failure')
neo.lib.logging.error(
'replication is stopped due to connection failure')
self.app.replicator.storageLost()
def acceptIdentification(self, conn, node_type,
......
......@@ -16,7 +16,7 @@
# Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
from neo.storage.handlers import BaseClientAndStorageOperationHandler
from neo.protocol import Packets
from neo.lib.protocol import Packets
class StorageOperationHandler(BaseClientAndStorageOperationHandler):
......
......@@ -18,9 +18,9 @@
import neo
from neo.storage.handlers import BaseMasterHandler
from neo.protocol import Packets, Errors, ProtocolError
from neo.util import dump
from neo.exception import OperationFailure
from neo.lib.protocol import Packets, Errors, ProtocolError
from neo.lib.util import dump
from neo.lib.exception import OperationFailure
class VerificationHandler(BaseMasterHandler):
"""This class deals with events for a verification phase."""
......@@ -48,7 +48,7 @@ class VerificationHandler(BaseMasterHandler):
app = self.app
if ptid <= app.pt.getID():
# Ignore this packet.
neo.logging.debug('ignoring older partition changes')
neo.lib.logging.debug('ignoring older partition changes')
return
# update partition table in memory and the database
app.pt.update(ptid, cell_list, app.nm)
......
......@@ -15,13 +15,13 @@
# along with this program; if not, write to the Free Software
# Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
import neo
import neo.lib
from random import choice
from neo.storage.handlers import replication
from neo.protocol import NodeTypes, NodeStates, Packets
from neo.connection import ClientConnection
from neo.util import dump
from neo.lib.protocol import NodeTypes, NodeStates, Packets
from neo.lib.connection import ClientConnection
from neo.lib.util import dump
class Partition(object):
"""This class abstracts the state of a partition."""
......@@ -210,7 +210,7 @@ class Replicator(object):
def setCriticalTID(self, tid):
"""This is a callback from MasterOperationHandler."""
neo.logging.debug('setting critical TID %s to %s', dump(tid),
neo.lib.logging.debug('setting critical TID %s to %s', dump(tid),
', '.join([str(p.getRID()) for p in self.critical_tid_list]))
for partition in self.critical_tid_list:
partition.setCriticalTID(tid)
......@@ -224,7 +224,7 @@ class Replicator(object):
def setUnfinishedTIDList(self, tid_list):
"""This is a callback from MasterOperationHandler."""
neo.logging.debug('setting unfinished TIDs %s',
neo.lib.logging.debug('setting unfinished TIDs %s',
','.join([dump(tid) for tid in tid_list]))
self.waiting_for_unfinished_tids = False
self.unfinished_tid_list = tid_list
......@@ -245,13 +245,13 @@ class Replicator(object):
node = choice(node_list)
except IndexError:
# Not operational.
neo.logging.error('not operational', exc_info = 1)
neo.lib.logging.error('not operational', exc_info = 1)
self.current_partition = None
return
addr = node.getAddress()
if addr is None:
neo.logging.error("no address known for the selected node %s" %
neo.lib.logging.error("no address known for the selected node %s" %
(dump(node.getUUID()), ))
return
if self.current_connection is not None:
......@@ -299,19 +299,19 @@ class Replicator(object):
if self.replication_done and \
not self.current_connection.isPending():
# finish a replication
neo.logging.info('replication is done for %s' %
neo.lib.logging.info('replication is done for %s' %
(self.current_partition.getRID(), ))
self._finishReplication()
return
if self.waiting_for_unfinished_tids:
# Still waiting.
neo.logging.debug('waiting for unfinished tids')
neo.lib.logging.debug('waiting for unfinished tids')
return
if self.unfinished_tid_list is None:
# Ask pending transactions.
neo.logging.debug('asking unfinished tids')
neo.lib.logging.debug('asking unfinished tids')
self._askUnfinishedTIDs()
return
......@@ -327,7 +327,7 @@ class Replicator(object):
else:
# Not yet.
self.unfinished_tid_list = None
neo.logging.debug('not ready yet')
neo.lib.logging.debug('not ready yet')
return
self._startReplication()
......
......@@ -16,9 +16,9 @@
# Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
from time import time
import neo
from neo.util import dump
from neo.protocol import ZERO_TID
import neo.lib
from neo.lib.util import dump
from neo.lib.protocol import ZERO_TID
class ConflictError(Exception):
"""
......@@ -228,7 +228,7 @@ class TransactionManager(object):
# check if the object if locked
locking_tid = self._store_lock_dict.get(oid)
if locking_tid == ttid and unlock:
neo.logging.info('Deadlock resolution on %r:%r', dump(oid),
neo.lib.logging.info('Deadlock resolution on %r:%r', dump(oid),
dump(ttid))
# A duplicate store means client is resolving a deadlock, so
# drop the lock it held on this object, and drop object data for
......@@ -249,7 +249,7 @@ class TransactionManager(object):
if previous_serial is None:
# XXX: use some special serial when previous store was not
# an undo ? Maybe it should just not happen.
neo.logging.info('Transaction %s storing %s more than '
neo.lib.logging.info('Transaction %s storing %s more than '
'once', dump(ttid), dump(oid))
else:
previous_serial = None
......@@ -258,15 +258,16 @@ class TransactionManager(object):
if history_list:
previous_serial = history_list[0][0]
if previous_serial is not None and previous_serial != serial:
neo.logging.info('Resolvable conflict on %r:%r', dump(oid),
dump(ttid))
neo.lib.logging.info('Resolvable conflict on %r:%r',
dump(oid), dump(ttid))
raise ConflictError(previous_serial)
neo.logging.info('Transaction %s storing %s', dump(ttid), dump(oid))
neo.lib.logging.info('Transaction %s storing %s',
dump(ttid), dump(oid))
self._store_lock_dict[oid] = ttid
elif locking_tid > ttid:
# We have a smaller TID than locking transaction, so we are older:
# enter waiting queue so we are handled when lock gets released.
neo.logging.info('Store delayed for %r:%r by %r', dump(oid),
neo.lib.logging.info('Store delayed for %r:%r by %r', dump(oid),
dump(ttid), dump(locking_tid))
raise DelayedError
else:
......@@ -275,7 +276,7 @@ class TransactionManager(object):
# hold locks that older transaction is waiting upon. Make client
# release locks & reacquire them by notifying it of the possible
# deadlock.
neo.logging.info('Possible deadlock on %r:%r with %r',
neo.lib.logging.info('Possible deadlock on %r:%r with %r',
dump(oid), dump(ttid), dump(locking_tid))
raise ConflictError(ZERO_TID)
......@@ -324,7 +325,7 @@ class TransactionManager(object):
del self._store_lock_dict[oid]
except KeyError:
# all locks might not have been acquiredwhen aborting
neo.logging.warning('%s write lock was not held by %s',
neo.lib.logging.warning('%s write lock was not held by %s',
dump(oid), dump(ttid))
# remove the transaction
uuid = transaction.getUUID()
......@@ -352,15 +353,15 @@ class TransactionManager(object):
return oid in self._load_lock_dict
def log(self):
neo.logging.info("Transactions:")
neo.lib.logging.info("Transactions:")
for txn in self._transaction_dict.values():
neo.logging.info(' %r', txn)
neo.logging.info(' Read locks:')
neo.lib.logging.info(' %r', txn)
neo.lib.logging.info(' Read locks:')
for oid, ttid in self._load_lock_dict.items():
neo.logging.info(' %r by %r', dump(oid), dump(ttid))
neo.logging.info(' Write locks:')
neo.lib.logging.info(' %r by %r', dump(oid), dump(ttid))
neo.lib.logging.info(' Write locks:')
for oid, ttid in self._store_lock_dict.items():
neo.logging.info(' %r by %r', dump(oid), dump(ttid))
neo.lib.logging.info(' %r by %r', dump(oid), dump(ttid))
def updateObjectDataForPack(self, oid, orig_serial, new_serial,
getObjectData):
......
......@@ -23,8 +23,8 @@ import tempfile
import MySQLdb
import neo
from mock import Mock
from neo import protocol
from neo.protocol import Packets
from neo.lib import protocol
from neo.lib.protocol import Packets
from time import time, gmtime
from struct import pack, unpack
......@@ -409,7 +409,7 @@ connector_cpt = 0
class DoNothingConnector(Mock):
def __init__(self, s=None):
neo.logging.info("initializing connector")
neo.lib.logging.info("initializing connector")
self.desc = globals()['connector_cpt']
globals()['connector_cpt'] = globals()['connector_cpt']+ 1
self.packet_cpt = 0
......
......@@ -24,8 +24,8 @@ from neo.tests import NeoUnitTestBase
from neo.client.app import Application, RevisionIndex
from neo.client.exception import NEOStorageError, NEOStorageNotFoundError
from neo.client.exception import NEOStorageDoesNotExistError
from neo.protocol import Packet, Packets, Errors, INVALID_TID
from neo.util import makeChecksum
from neo.lib.protocol import Packet, Packets, Errors, INVALID_TID
from neo.lib.util import makeChecksum
import time
def _getMasterConnection(self):
......
......@@ -18,8 +18,8 @@
import unittest
from mock import Mock, ReturnValues
from neo.tests import NeoUnitTestBase
from neo.pt import PartitionTable
from neo.protocol import NodeTypes, NodeStates
from neo.lib.pt import PartitionTable
from neo.lib.protocol import NodeTypes, NodeStates
from neo.client.handlers.master import PrimaryBootstrapHandler
from neo.client.handlers.master import PrimaryNotificationsHandler, \
PrimaryAnswersHandler
......
......@@ -18,7 +18,7 @@
import unittest
from mock import Mock
from neo.tests import NeoUnitTestBase
from neo.protocol import NodeTypes, LockState
from neo.lib.protocol import NodeTypes, LockState
from neo.client.handlers.storage import StorageBootstrapHandler, \
StorageAnswersHandler
from neo.client.exception import NEOStorageError, NEOStorageNotFoundError
......
......@@ -29,8 +29,8 @@ import traceback
import threading
from neo.neoctl.neoctl import NeoCTL, NotReadyException
from neo.protocol import ClusterStates, NodeTypes, CellStates, NodeStates
from neo.util import dump
from neo.lib.protocol import ClusterStates, NodeTypes, CellStates, NodeStates
from neo.lib.util import dump
from neo.tests import DB_ADMIN, DB_PASSWD, NeoTestBase
from neo.client.Storage import Storage
......
......@@ -17,7 +17,7 @@
import unittest
from neo.tests.functional import NEOCluster, NEOFunctionalTest
from neo.protocol import NodeStates
from neo.lib.protocol import NodeStates
MASTER_NODE_COUNT = 3
......
......@@ -20,7 +20,7 @@ import transaction
from persistent import Persistent
from neo.tests.functional import NEOCluster, NEOFunctionalTest
from neo.protocol import ClusterStates, NodeStates
from neo.lib.protocol import ClusterStates, NodeStates
from MySQLdb import ProgrammingError
from MySQLdb.constants.ER import NO_SUCH_TABLE
......
......@@ -19,7 +19,7 @@ import unittest
from mock import Mock
from struct import pack, unpack
from neo.tests import NeoUnitTestBase
from neo.protocol import NodeTypes, NodeStates, Packets
from neo.lib.protocol import NodeTypes, NodeStates, Packets
from neo.master.handlers.client import ClientServiceHandler
from neo.master.app import Application
......
......@@ -17,13 +17,13 @@
import unittest
from mock import Mock
from neo import protocol
from neo.lib import protocol
from neo.tests import NeoUnitTestBase
from neo.protocol import Packet, NodeTypes, NodeStates
from neo.lib.protocol import Packet, NodeTypes, NodeStates
from neo.master.handlers.election import ClientElectionHandler, ServerElectionHandler
from neo.master.app import Application
from neo.exception import ElectionFailure
from neo.connection import ClientConnection
from neo.lib.exception import ElectionFailure
from neo.lib.connection import ClientConnection
# patch connection so that we can register _addPacket messages
# in mock object
......
......@@ -19,7 +19,7 @@ import unittest
from mock import Mock
from neo.tests import NeoUnitTestBase
from neo.master.app import Application
from neo.util import p64, u64
from neo.lib.util import p64, u64
class MasterAppTests(NeoUnitTestBase):
......
......@@ -18,9 +18,9 @@
import unittest
from mock import Mock
from neo.tests import NeoUnitTestBase
from neo.protocol import NodeStates, CellStates
from neo.lib.protocol import NodeStates, CellStates
from neo.master.pt import PartitionTable
from neo.node import StorageNode
from neo.lib.node import StorageNode
class MasterPartitionTableTests(NeoUnitTestBase):
......
......@@ -18,7 +18,7 @@
import unittest
from struct import pack, unpack
from neo.tests import NeoUnitTestBase
from neo.protocol import NodeTypes, NodeStates, CellStates
from neo.lib.protocol import NodeTypes, NodeStates, CellStates
from neo.master.recovery import RecoveryManager
from neo.master.app import Application
......
......@@ -19,11 +19,11 @@ import unittest
from mock import Mock
from struct import pack
from neo.tests import NeoUnitTestBase
from neo.protocol import NodeTypes, NodeStates, Packets
from neo.lib.protocol import NodeTypes, NodeStates, Packets
from neo.master.handlers.storage import StorageServiceHandler
from neo.master.handlers.client import ClientServiceHandler
from neo.master.app import Application
from neo.exception import OperationFailure
from neo.lib.exception import OperationFailure
class MasterStorageHandlerTests(NeoUnitTestBase):
......
......@@ -19,7 +19,7 @@ import unittest
from mock import Mock
from struct import pack, unpack
from neo.tests import NeoUnitTestBase
from neo.protocol import ZERO_TID
from neo.lib.protocol import ZERO_TID
from neo.master.transactions import Transaction, TransactionManager
from neo.master.transactions import packTID, unpackTID, addTID, DelayedError
......
......@@ -18,7 +18,7 @@
import unittest
from struct import pack, unpack
from neo.tests import NeoUnitTestBase
from neo.protocol import NodeTypes, NodeStates
from neo.lib.protocol import NodeTypes, NodeStates
from neo.master.verification import VerificationManager, VerificationFailure
from neo.master.app import Application
......
......@@ -22,9 +22,9 @@ from neo.tests import NeoUnitTestBase
from neo.storage.app import Application
from neo.storage.transactions import ConflictError, DelayedError
from neo.storage.handlers.client import ClientOperationHandler
from neo.protocol import INVALID_PARTITION
from neo.protocol import INVALID_TID, INVALID_OID
from neo.protocol import Packets, LockState
from neo.lib.protocol import INVALID_PARTITION
from neo.lib.protocol import INVALID_TID, INVALID_OID
from neo.lib.protocol import Packets, LockState
class StorageClientHandlerTests(NeoUnitTestBase):
......
......@@ -18,8 +18,9 @@
import unittest
from mock import Mock
from neo.tests import NeoUnitTestBase
from neo.protocol import NodeTypes, NotReadyError, BrokenNodeDisallowedError
from neo.pt import PartitionTable
from neo.lib.protocol import NodeTypes, NotReadyError,
BrokenNodeDisallowedError
from neo.lib.pt import PartitionTable
from neo.storage.app import Application
from neo.storage.handlers.identification import IdentificationHandler
......
......@@ -18,11 +18,11 @@
import unittest
from mock import Mock
from neo.tests import NeoUnitTestBase
from neo.pt import PartitionTable
from neo.lib.pt import PartitionTable
from neo.storage.app import Application
from neo.storage.handlers.initialization import InitializationHandler
from neo.protocol import CellStates, ProtocolError
from neo.exception import PrimaryFailure
from neo.lib.protocol import CellStates, ProtocolError
from neo.lib.exception import PrimaryFailure
class StorageInitializationHandlerTests(NeoUnitTestBase):
......
......@@ -21,10 +21,10 @@ from collections import deque
from neo.tests import NeoUnitTestBase
from neo.storage.app import Application
from neo.storage.handlers.master import MasterOperationHandler
from neo.exception import PrimaryFailure, OperationFailure
from neo.pt import PartitionTable
from neo.protocol import CellStates, ProtocolError, Packets
from neo.protocol import INVALID_TID, INVALID_OID
from neo.lib.exception import PrimaryFailure, OperationFailure
from neo.lib.pt import PartitionTable
from neo.lib.protocol import CellStates, ProtocolError, Packets
from neo.lib.protocol import INVALID_TID, INVALID_OID
class StorageMasterHandlerTests(NeoUnitTestBase):
......
......@@ -26,7 +26,7 @@ from neo.storage.handlers.replication import ReplicationHandler
from neo.storage.handlers.replication import RANGE_LENGTH
from neo.storage.handlers.storage import StorageOperationHandler
from neo.storage.replicator import Replicator
from neo.protocol import ZERO_OID, ZERO_TID
from neo.lib.protocol import ZERO_OID, ZERO_TID
MAX_TRANSACTIONS = 10000
MAX_OBJECTS = 100000
......
......@@ -17,9 +17,9 @@
import unittest
from mock import Mock
from neo.util import add64
from neo.lib.util import add64
from neo.tests import NeoUnitTestBase
from neo.protocol import Packets, ZERO_OID, ZERO_TID
from neo.lib.protocol import Packets, ZERO_OID, ZERO_TID
from neo.storage.handlers.replication import ReplicationHandler
from neo.storage.handlers.replication import RANGE_LENGTH, MIN_RANGE_LENGTH
......
......@@ -19,7 +19,7 @@ import unittest
from mock import Mock, ReturnValues
from neo.tests import NeoUnitTestBase
from neo.storage.replicator import Replicator, Partition, Task
from neo.protocol import CellStates, NodeStates, Packets
from neo.lib.protocol import CellStates, NodeStates, Packets
class StorageReplicatorTests(NeoUnitTestBase):
......
......@@ -19,10 +19,10 @@ import unittest
from mock import Mock, ReturnValues
from neo.tests import NeoUnitTestBase
from neo.storage.app import Application
from neo.protocol import CellStates
from neo.lib.protocol import CellStates
from collections import deque
from neo.pt import PartitionTable
from neo.util import dump
from neo.lib.pt import PartitionTable
from neo.lib.util import dump
from neo.storage.exception import AlreadyPendingError
class StorageAppTests(NeoUnitTestBase):
......
......@@ -17,10 +17,10 @@
import unittest
from mock import Mock
from neo.util import dump, p64, u64
from neo.protocol import CellStates, ZERO_OID, ZERO_TID
from neo.lib.util import dump, p64, u64
from neo.lib.protocol import CellStates, ZERO_OID, ZERO_TID
from neo.tests import NeoUnitTestBase
from neo.exception import DatabaseFailure
from neo.lib.exception import DatabaseFailure
from neo.storage.database.mysqldb import MySQLDatabaseManager
MAX_TID = '\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFE' # != INVALID_TID
......
......@@ -21,8 +21,8 @@ from collections import deque
from neo.tests import NeoUnitTestBase
from neo.storage.app import Application
from neo.storage.handlers.storage import StorageOperationHandler
from neo.protocol import INVALID_PARTITION, Packets
from neo.protocol import INVALID_TID, INVALID_OID
from neo.lib.protocol import INVALID_PARTITION, Packets
from neo.lib.protocol import INVALID_TID, INVALID_OID
class StorageStorageHandlerTests(NeoUnitTestBase):
......
......@@ -18,7 +18,7 @@
import unittest
import MySQLdb
from mock import Mock
from neo.exception import DatabaseFailure
from neo.lib.exception import DatabaseFailure
from neo.tests.storage.testStorageDBTests import StorageDBTests
from neo.storage.database.mysqldb import MySQLDatabaseManager
......
......@@ -18,12 +18,12 @@
import unittest
from mock import Mock
from neo.tests import NeoUnitTestBase
from neo.pt import PartitionTable
from neo.lib.pt import PartitionTable
from neo.storage.app import Application
from neo.storage.handlers.verification import VerificationHandler
from neo.protocol import CellStates, ErrorCodes
from neo.exception import PrimaryFailure, OperationFailure
from neo.util import p64, u64
from neo.lib.protocol import CellStates, ErrorCodes
from neo.lib.exception import PrimaryFailure, OperationFailure
from neo.lib.util import p64, u64
class StorageVerificationHandlerTests(NeoUnitTestBase):
......
......@@ -19,8 +19,8 @@ import unittest
from mock import Mock
from neo.tests import NeoUnitTestBase
from neo.storage.app import Application
from neo.bootstrap import BootstrapManager
from neo.protocol import NodeTypes
from neo.lib.bootstrap import BootstrapManager
from neo.lib.protocol import NodeTypes
class BootstrapManagerTests(NeoUnitTestBase):
......
......@@ -17,17 +17,17 @@
import unittest
from time import time
from mock import Mock
from neo.connection import ListeningConnection, Connection, \
from neo.lib.connection import ListeningConnection, Connection, \
ClientConnection, ServerConnection, MTClientConnection, \
HandlerSwitcher, Timeout, PING_DELAY, PING_TIMEOUT, OnTimeout
from neo.connector import getConnectorHandler, registerConnectorHandler
from neo.lib.connector import getConnectorHandler, registerConnectorHandler
from neo.tests import DoNothingConnector
from neo.connector import ConnectorException, ConnectorTryAgainException, \
from neo.lib.connector import ConnectorException, ConnectorTryAgainException, \
ConnectorInProgressException, ConnectorConnectionRefusedException
from neo.protocol import Packets, ParserState
from neo.lib.protocol import Packets, ParserState
from neo.tests import NeoUnitTestBase
from neo.util import ReadBuffer
from neo.locking import Queue
from neo.lib.util import ReadBuffer
from neo.lib.locking import Queue
class ConnectionTests(NeoUnitTestBase):
......
......@@ -17,7 +17,7 @@
from mock import Mock
from neo.tests import NeoTestBase
from neo.dispatcher import Dispatcher, ForgottenPacket
from neo.lib.dispatcher import Dispatcher, ForgottenPacket
from Queue import Queue
import unittest
......
......@@ -18,8 +18,8 @@ import unittest
from mock import Mock
from time import time
from neo.tests import NeoUnitTestBase
from neo.epoll import Epoll
from neo.event import EpollEventManager
from neo.lib.epoll import Epoll
from neo.lib.event import EpollEventManager
class EventTests(NeoUnitTestBase):
......
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
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