Commit 5743cdce authored by Julien Muchembled's avatar Julien Muchembled

neo.lib.logging.* -> logging.*

parent 1fce5cc4
......@@ -14,8 +14,7 @@
# You should have received a copy of the GNU General Public License
# along with this program. If not, see <http://www.gnu.org/licenses/>.
import neo.lib
from neo.lib import logging
from neo.lib.node import NodeManager
from neo.lib.event import EventManager
from neo.lib.connection import ListeningConnection
......@@ -41,7 +40,7 @@ class Application(object):
self.master_addresses, connector_name = config.getMasters()
self.connector_handler = getConnectorHandler(connector_name)
neo.lib.logging.debug('IP address is %s, port is %d', *(self.server))
logging.debug('IP address is %s, port is %d', *self.server)
# The partition table is initialized after getting the number of
# partitions.
......@@ -75,7 +74,7 @@ class Application(object):
try:
self._run()
except:
neo.lib.logging.exception('Pre-mortem data:')
logging.exception('Pre-mortem data:')
self.log()
raise
......@@ -95,7 +94,7 @@ class Application(object):
while True:
self.em.poll(1)
except PrimaryFailure:
neo.lib.logging.error('primary master is down')
logging.error('primary master is down')
def connectToPrimary(self):
"""Find a primary master node, and connect to it.
......
......@@ -14,10 +14,8 @@
# You should have received a copy of the GNU General Public License
# along with this program. If not, see <http://www.gnu.org/licenses/>.
import neo
from neo.lib import logging, protocol
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
......@@ -39,8 +37,8 @@ class AdminEventHandler(EventHandler):
@check_primary_master
def askPartitionList(self, conn, min_offset, max_offset, uuid):
neo.lib.logging.info("ask partition list from %s to %s for %s",
min_offset, max_offset, dump(uuid))
logging.info("ask partition list from %s to %s for %s",
min_offset, max_offset, dump(uuid))
self.app.sendPartitionTable(conn, min_offset, max_offset, uuid)
@check_primary_master
......@@ -50,7 +48,7 @@ class AdminEventHandler(EventHandler):
node_filter = None
else:
node_filter = lambda n: n.getType() is node_type
neo.lib.logging.info("ask list of %s nodes", node_type)
logging.info("ask list of %s nodes", node_type)
node_list = self.app.nm.getList(node_filter)
node_information_list = [node.asTuple() for node in node_list ]
p = Packets.AnswerNodeList(node_information_list)
......@@ -58,7 +56,7 @@ class AdminEventHandler(EventHandler):
@check_primary_master
def setNodeState(self, conn, uuid, state, modify_partition_table):
neo.lib.logging.info("set node state for %s-%s", dump(uuid), state)
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')
......@@ -121,7 +119,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.lib.logging.debug("answerNodeInformation")
logging.debug("answerNodeInformation")
def notifyPartitionChanges(self, conn, ptid, cell_list):
self.app.pt.update(ptid, cell_list, self.app.nm)
......
......@@ -18,8 +18,8 @@ from ZODB import BaseStorage, ConflictResolution, POSException
from zope.interface import implements
import ZODB.interfaces
import neo.lib
from functools import wraps
from neo.lib import logging
from neo.lib.util import add64
from neo.lib.protocol import ZERO_TID
from .app import Application
......@@ -71,7 +71,7 @@ class Storage(BaseStorage.BaseStorage,
if compress is None:
compress = True
if logfile:
neo.lib.logging.setup(logfile)
logging.setup(logfile)
BaseStorage.BaseStorage.__init__(self, 'NEOStorage(%s)' % (name, ))
# Warning: _is_read_only is used in BaseStorage, do not rename it.
self._is_read_only = read_only
......@@ -238,9 +238,8 @@ class Storage(BaseStorage.BaseStorage,
def pack(self, t, referencesf, gc=False):
if gc:
neo.lib.logging.warning(
'Garbage Collection is not available in NEO, '
'please use an external tool. Packing without GC.')
logging.warning('Garbage Collection is not available in NEO,'
' please use an external tool. Packing without GC.')
self.app.pack(t)
def lastSerial(self):
......
......@@ -27,7 +27,7 @@ from ZODB.POSException import ReadConflictError
from ZODB.ConflictResolution import ResolvedSerial
from persistent.TimeStamp import TimeStamp
import neo.lib
from neo.lib import logging
from neo.lib.protocol import NodeTypes, Packets, \
INVALID_PARTITION, ZERO_HASH, ZERO_TID
from neo.lib.event import EventManager
......@@ -278,7 +278,7 @@ class Application(object):
"""
Lookup for the current primary master node
"""
neo.lib.logging.debug('connecting to primary master...')
logging.debug('connecting to primary master...')
ready = False
nm = self.nm
while not ready:
......@@ -309,8 +309,7 @@ class Application(object):
# Query for primary master node
if conn.getConnector() is None:
# This happens if a connection could not be established.
neo.lib.logging.error(
'Connection to master node %s failed',
logging.error('Connection to master node %s failed',
self.trying_master_node)
continue
try:
......@@ -322,15 +321,13 @@ class Application(object):
# If we reached the primary master node, mark as connected
connected = self.primary_master_node is not None and \
self.primary_master_node is self.trying_master_node
neo.lib.logging.info(
'Connected to %s' % (self.primary_master_node, ))
logging.info('Connected to %s', self.primary_master_node)
try:
ready = self.identifyToPrimaryNode(conn)
except ConnectionClosed:
neo.lib.logging.error('Connection to %s lost',
self.trying_master_node)
logging.error('Connection to %s lost', self.trying_master_node)
self.primary_master_node = None
neo.lib.logging.info("Connected and ready")
logging.info("Connected and ready")
return conn
def identifyToPrimaryNode(self, conn):
......@@ -339,7 +336,7 @@ class Application(object):
Might raise ConnectionClosed so that the new primary can be
looked-up again.
"""
neo.lib.logging.info('Initializing from master')
logging.info('Initializing from master')
ask = self._ask
handler = self.primary_bootstrap_handler
ask(conn, Packets.AskNodeInformation(), handler=handler)
......@@ -437,7 +434,7 @@ class Application(object):
if data or checksum != ZERO_HASH:
if checksum != makeChecksum(data):
neo.lib.logging.error('wrong checksum from %s for oid %s',
logging.error('wrong checksum from %s for oid %s',
conn, dump(oid))
continue
if compression:
......@@ -486,8 +483,7 @@ class Application(object):
txn_context = self._txn_container.get(transaction)
if txn_context is None:
raise StorageTransactionError(self, transaction)
neo.lib.logging.debug(
'storing oid %s serial %s', dump(oid), dump(serial))
logging.debug('storing oid %s serial %s', dump(oid), dump(serial))
self._store(txn_context, oid, serial, data)
return None
......@@ -570,7 +566,7 @@ class Application(object):
if ZERO_TID in conflict_serial_set:
if 1:
# XXX: disable deadlock avoidance code until it is fixed
neo.lib.logging.info('Deadlock avoidance on %r:%r',
logging.info('Deadlock avoidance on %r:%r',
dump(oid), dump(serial))
# 'data' parameter of ConflictError is only used to report the
# class of the object. It doesn't matter if 'data' is None
......@@ -591,7 +587,7 @@ class Application(object):
# XXX: currently, brute-force is implemented: we send
# object data again.
# WARNING: not maintained code
neo.lib.logging.info('Deadlock avoidance triggered on %r:%r',
logging.info('Deadlock avoidance triggered on %r:%r',
dump(oid), dump(serial))
for store_oid, store_data in data_dict.iteritems():
store_serial = object_serial_dict[store_oid]
......@@ -601,9 +597,8 @@ class Application(object):
else:
if store_data is None:
# Some undo
neo.lib.logging.warning('Deadlock avoidance cannot'
' reliably work with undo, this must be '
'implemented.')
logging.warning('Deadlock avoidance cannot reliably'
' work with undo, this must be implemented.')
conflict_serial = ZERO_TID
break
self._store(txn_context, store_oid, store_serial,
......@@ -627,7 +622,7 @@ class Application(object):
new_data = tryToResolveConflict(oid, conflict_serial,
serial, data)
if new_data is not None:
neo.lib.logging.info('Conflict resolution succeed for ' \
logging.info('Conflict resolution succeed for '
'%r:%r with %r', dump(oid), dump(serial),
dump(conflict_serial))
# Mark this conflict as resolved
......@@ -639,7 +634,7 @@ class Application(object):
append(oid)
continue
else:
neo.lib.logging.info('Conflict resolution failed for ' \
logging.info('Conflict resolution failed for '
'%r:%r with %r', dump(oid), dump(serial),
dump(conflict_serial))
raise ConflictError(oid=oid, serials=(txn_context['ttid'],
......@@ -681,7 +676,7 @@ class Application(object):
for oid, store_dict in \
txn_context['object_stored_counter_dict'].iteritems():
if not store_dict:
neo.lib.logging.error('tpc_store failed')
logging.error('tpc_store failed')
raise NEOStorageError('tpc_store failed')
elif oid in resolved_oid_set:
append((oid, ResolvedSerial))
......@@ -705,7 +700,7 @@ class Application(object):
txn_context['cache_dict'])
add_involved_nodes = txn_context['involved_nodes'].add
for node, conn in self.cp.iterateForObject(ttid):
neo.lib.logging.debug("voting object %s on %s", dump(ttid),
logging.debug("voting object %s on %s", dump(ttid),
dump(conn.getUUID()))
try:
self._askStorage(conn, packet)
......@@ -716,7 +711,7 @@ class Application(object):
# check at least one storage node accepted
if txn_stored_counter == 0:
neo.lib.logging.error('tpc_vote failed')
logging.error('tpc_vote failed')
raise NEOStorageError('tpc_vote failed')
# Check if master connection is still alive.
# This is just here to lower the probability of detecting a problem
......@@ -746,10 +741,8 @@ class Application(object):
try:
conn.notify(p)
except:
neo.lib.logging.error(
'Exception in tpc_abort while notifying' \
'storage node %r of abortion, ignoring.',
conn, exc_info=1)
logging.exception('Exception in tpc_abort while notifying'
'storage node %r of abortion, ignoring.', conn)
self._getMasterConnection().notify(p)
queue = txn_context['queue']
# We don't need to flush queue, as it won't be reused by future
......@@ -924,7 +917,7 @@ class Application(object):
# Reorder tids
ordered_tids = sorted(tid_set, reverse=True)
neo.lib.logging.debug("UndoLog tids %s", map(dump, ordered_tids))
logging.debug("UndoLog tids %s", map(dump, ordered_tids))
# For each transaction, get info
undo_info = []
append = undo_info.append
......@@ -1045,7 +1038,7 @@ class Application(object):
self.cp.flush()
self.master_conn = None
# Stop polling thread
neo.lib.logging.debug('Stopping %s', self.poll_thread)
logging.debug('Stopping %s', self.poll_thread)
self.poll_thread.stop()
psThreadedPoll()
close = __del__
......
......@@ -14,7 +14,7 @@
# You should have received a copy of the GNU General Public License
# along with this program. If not, see <http://www.gnu.org/licenses/>.
import neo.lib
from neo.lib import logging
from neo.lib.pt import MTPartitionTable as PartitionTable
from neo.lib.protocol import NodeTypes, NodeStates, ProtocolError
from neo.lib.util import dump
......@@ -52,8 +52,8 @@ class PrimaryBootstrapHandler(AnswerBaseHandler):
if primary_node is None:
# I don't know such a node. Probably this information
# is old. So ignore it.
neo.lib.logging.warning('Unknown primary master UUID: %s. ' \
'Ignoring.' % dump(primary_uuid))
logging.warning('Unknown primary master UUID: %s. Ignoring.',
dump(primary_uuid))
return
else:
if app.trying_master_node is not primary_node:
......@@ -74,7 +74,7 @@ class PrimaryBootstrapHandler(AnswerBaseHandler):
if your_uuid is None:
raise ProtocolError('No UUID supplied')
app.uuid = your_uuid
neo.lib.logging.info('Got an UUID: %s', dump(app.uuid))
logging.info('Got an UUID: %s', dump(app.uuid))
# Always create partition table
app.pt = PartitionTable(num_partitions, num_replicas)
......@@ -93,13 +93,13 @@ class PrimaryNotificationsHandler(BaseHandler):
def connectionClosed(self, conn):
app = self.app
if app.master_conn is not None:
neo.lib.logging.critical("connection to primary master node closed")
logging.critical("connection to primary master node closed")
app.master_conn = None
app.primary_master_node = None
super(PrimaryNotificationsHandler, self).connectionClosed(conn)
def stopOperation(self, conn):
neo.lib.logging.critical("master node ask to stop operation")
logging.critical("master node ask to stop operation")
def invalidateObjects(self, conn, tid, oid_list):
app = self.app
......
......@@ -17,7 +17,7 @@
from ZODB.TimeStamp import TimeStamp
from ZODB.POSException import ConflictError
import neo.lib
from neo.lib import logging
from neo.lib.protocol import NodeTypes, ProtocolError, LockState, ZERO_TID
from neo.lib.util import dump
from neo.lib.exception import NodeNotReady
......@@ -71,8 +71,8 @@ class StorageAnswersHandler(AnswerBaseHandler):
# we may process entirely a conflict with S1 (i.e. we received the
# answer to the store of the resolved object on S1) before we
# receive the conflict answer from the first store on S2.
neo.lib.logging.info('%r report a conflict for %r with %r', conn,
dump(oid), dump(serial))
logging.info('%r report a conflict for %r with %r',
conn, dump(oid), dump(serial))
# If this conflict is not already resolved, mark it for
# resolution.
if serial not in txn_context[
......@@ -115,7 +115,7 @@ class StorageAnswersHandler(AnswerBaseHandler):
pass
def answerTIDsFrom(self, conn, tid_list):
neo.lib.logging.debug('Get %d TIDs from %r', len(tid_list), conn)
logging.debug('Get %u TIDs from %r', len(tid_list), conn)
self.app.setHandlerData(tid_list)
def answerTransactionInformation(self, conn, tid,
......@@ -178,8 +178,7 @@ class StorageAnswersHandler(AnswerBaseHandler):
# Anyway, it's not clear that HasLock requests are useful.
# Are store requests potentially long to process ? If not,
# we should simply raise a ConflictError on store timeout.
neo.lib.logging.info('Store of oid %s delayed (storage overload ?)',
dump(oid))
logging.info('Store of oid %s delayed (storage overload ?)', dump(oid))
def alreadyPendingError(self, conn, message):
pass
......
......@@ -16,8 +16,8 @@
from logging import DEBUG, ERROR
from threading import Thread, Event, enumerate as thread_enum
from neo.lib import logging
from neo.lib.locking import Lock
import neo.lib
class _ThreadedPoll(Thread):
"""Polling thread."""
......@@ -29,7 +29,7 @@ class _ThreadedPoll(Thread):
self._stop = Event()
def run(self):
_log = neo.lib.logging.log
_log = logging.log
def log(*args, **kw):
# Ignore errors due to garbage collection on exit
try:
......@@ -112,7 +112,7 @@ def psThreadedPoll(log=None):
Logs alive ThreadedPoll threads.
"""
if log is None:
log = neo.lib.logging.debug
log = logging.debug
for thread in thread_enum():
if not isinstance(thread, ThreadedPoll):
continue
......
......@@ -17,7 +17,7 @@
import time
from random import shuffle
import neo.lib
from neo.lib import logging
from neo.lib.locking import RLock
from neo.lib.protocol import NodeTypes, Packets
from neo.lib.connection import MTClientConnection, ConnectionClosed
......@@ -56,8 +56,7 @@ class ConnectionPool(object):
def _initNodeConnection(self, node):
"""Init a connection to a given storage node."""
app = self.app
neo.lib.logging.debug('trying to connect to %s - %s', node,
node.getState())
logging.debug('trying to connect to %s - %s', node, node.getState())
conn = MTClientConnection(app.em, app.storage_event_handler, node,
connector=app.connector_handler(), dispatcher=app.dispatcher)
p = Packets.RequestIdentification(NodeTypes.CLIENT,
......@@ -65,15 +64,15 @@ class ConnectionPool(object):
try:
app._ask(conn, p, handler=app.storage_bootstrap_handler)
except ConnectionClosed:
neo.lib.logging.error('Connection to %r failed', node)
logging.error('Connection to %r failed', node)
self.notifyFailure(node)
conn = None
except NodeNotReady:
neo.lib.logging.info('%r not ready', node)
logging.info('%r not ready', node)
self.notifyFailure(node)
conn = None
else:
neo.lib.logging.info('Connected %r', node)
logging.info('Connected %r', node)
return conn
@profiler_decorator
......@@ -87,8 +86,8 @@ class ConnectionPool(object):
not self.app.dispatcher.registered(conn):
del self.connection_dict[conn.getUUID()]
conn.close()
neo.lib.logging.debug('_dropConnections : connection to ' \
'storage node %s:%d closed', *(conn.getAddress()))
logging.debug('_dropConnections: connection to '
'storage node %s:%d closed', *conn.getAddress())
if len(self.connection_dict) <= self.max_pool_size:
break
finally:
......
......@@ -14,9 +14,9 @@
# You should have received a copy of the GNU General Public License
# along with this program. If not, see <http://www.gnu.org/licenses/>.
import neo
from time import sleep
from . import logging
from .handler import EventHandler
from .protocol import Packets
from .util import dump
......@@ -109,13 +109,13 @@ class BootstrapManager(EventHandler):
node.getConnection().close()
return
neo.lib.logging.info('connected to a primary master node')
logging.info('connected to a primary master node')
self.num_partitions = num_partitions
self.num_replicas = num_replicas
if self.uuid != your_uuid:
# got an uuid from the primary master
self.uuid = your_uuid
neo.lib.logging.info('Got a new UUID : %s' % dump(self.uuid))
logging.info('Got a new UUID: %s', dump(self.uuid))
self.accepted = True
def getPrimaryConnection(self, connector_handler):
......@@ -123,7 +123,7 @@ class BootstrapManager(EventHandler):
Primary lookup/connection process.
Returns when the connection is made.
"""
neo.lib.logging.info('connecting to a primary master node')
logging.info('connecting to a primary master node')
em, nm = self.app.em, self.app.nm
index = 0
self.current = None
......
......@@ -17,8 +17,7 @@
from functools import wraps
from time import time
import neo.lib
from . import attributeTracker
from . import attributeTracker, logging
from .connector import ConnectorException, ConnectorTryAgainException, \
ConnectorInProgressException, ConnectorConnectionRefusedException, \
ConnectorConnectionClosedException
......@@ -55,8 +54,8 @@ def lockCheckWrapper(func):
def wrapper(self, *args, **kw):
if not self._lock._is_owned():
import traceback
neo.lib.logging.warning('%s called on %s instance without being ' \
'locked. Stack:\n%s', func.func_code.co_name,
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
return func(self, *args, **kw)
......@@ -165,9 +164,9 @@ class HandlerSwitcher(object):
@profiler_decorator
def _handle(self, connection, packet):
assert len(self._pending) == 1 or self._pending[0][0]
neo.lib.logging.packet(connection, packet, False)
logging.packet(connection, packet, False)
if connection.isClosed() and packet.ignoreOnClosedConnection():
neo.lib.logging.debug('Ignoring packet %r on closed connection %r',
logging.debug('Ignoring packet %r on closed connection %r',
packet, connection)
return
msg_id = packet.getId()
......@@ -185,8 +184,7 @@ class HandlerSwitcher(object):
if klass and isinstance(packet, klass) or packet.isError():
handler.packetReceived(connection, packet, kw)
else:
neo.lib.logging.error(
'Unexpected answer %r in %r', packet, connection)
logging.error('Unexpected answer %r in %r', packet, connection)
if not connection.isClosed():
notification = Packets.Notify('Unexpected answer: %r' % packet)
connection.notify(notification)
......@@ -195,8 +193,7 @@ 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.lib.logging.debug(
'Apply handler %r on %r', self._pending[0][1],
logging.debug('Apply handler %r on %r', self._pending[0][1],
connection)
if msg_id == self._next_timeout_msg_id:
self._updateNextTimeout()
......@@ -281,8 +278,8 @@ class BaseConnection(object):
if msg_id is None:
self._base_timeout = t
else:
neo.lib.logging.info('timeout for #0x%08x with %r',
msg_id, self)
logging.info('timeout for #0x%08x with %r',
msg_id, self)
self.close()
else:
self.idle()
......@@ -332,9 +329,9 @@ class BaseConnection(object):
def setHandler(self, handler):
if self._handlers.setHandler(handler):
neo.lib.logging.debug('Set handler %r on %r', handler, self)
logging.debug('Set handler %r on %r', handler, self)
else:
neo.lib.logging.debug('Delay handler %r on %r', handler, self)
logging.debug('Delay handler %r on %r', handler, self)
def getEventManager(self):
return self.em
......@@ -377,7 +374,7 @@ class ListeningConnection(BaseConnection):
"""A listen connection."""
def __init__(self, event_manager, handler, addr, connector, **kw):
neo.lib.logging.debug('listening to %s:%d', *addr)
logging.debug('listening to %s:%d', *addr)
BaseConnection.__init__(self, event_manager, handler,
addr=addr, connector=connector)
self.connector.makeListeningConnection(addr)
......@@ -385,7 +382,7 @@ class ListeningConnection(BaseConnection):
def readable(self):
try:
new_s, addr = self.connector.getNewConnection()
neo.lib.logging.debug('accepted a connection from %s:%d', *addr)
logging.debug('accepted a connection from %s:%d', *addr)
handler = self.getHandler()
new_conn = ServerConnection(self.getEventManager(), handler,
connector=new_s, addr=addr)
......@@ -477,7 +474,7 @@ class Connection(BaseConnection):
def abort(self):
"""Abort dealing with this connection."""
neo.lib.logging.debug('aborting a connector for %r', self)
logging.debug('aborting a connector for %r', self)
self.aborted = True
assert self.write_buf
if self._on_close is not None:
......@@ -574,19 +571,16 @@ 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.lib.logging.debug(
'Connection reset by peer: %r', self.connector)
logging.debug('Connection reset by peer: %r', self.connector)
self._closure()
except:
neo.lib.logging.debug(
'Unknown connection error: %r', self.connector)
logging.debug('Unknown connection error: %r', self.connector)
self._closure()
# unhandled connector exception
raise
else:
if not data:
neo.lib.logging.debug(
'Connection %r closed in recv', self.connector)
logging.debug('Connection %r closed in recv', self.connector)
self._closure()
return
self._base_timeout = time() # last known remote activity
......@@ -604,19 +598,16 @@ class Connection(BaseConnection):
pass
except ConnectorConnectionClosedException:
# connection resetted by peer
neo.lib.logging.debug(
'Connection reset by peer: %r', self.connector)
logging.debug('Connection reset by peer: %r', self.connector)
self._closure()
except:
neo.lib.logging.debug(
'Unknown connection error: %r', self.connector)
logging.debug('Unknown connection error: %r', self.connector)
# unhandled connector exception
self._closure()
raise
else:
if not n:
neo.lib.logging.debug('Connection %r closed in send',
self.connector)
logging.debug('Connection %r closed in send', self.connector)
self._closure()
return
if n == len(msg):
......@@ -637,7 +628,7 @@ class Connection(BaseConnection):
if was_empty:
# enable polling for writing.
self.em.addWriter(self)
neo.lib.logging.packet(self, packet, True)
logging.packet(self, packet, True)
@not_closed
def notify(self, packet):
......
......@@ -15,9 +15,9 @@
# along with this program. If not, see <http://www.gnu.org/licenses/>.
from time import time
import neo.lib
from select import epoll, EPOLLIN, EPOLLOUT, EPOLLERR, EPOLLHUP
from errno import EINTR, EAGAIN
from . import logging
from .profiling import profiler_decorator
class EpollEventManager(object):
......@@ -113,8 +113,8 @@ class EpollEventManager(object):
event_list = self.epoll.poll(timeout)
except IOError, exc:
if exc.errno in (0, EAGAIN):
neo.lib.logging.info('epoll.poll triggered undocumented '
'error %r', exc.errno)
logging.info('epoll.poll triggered undocumented error %r',
exc.errno)
elif exc.errno != EINTR:
raise
event_list = ()
......@@ -206,13 +206,13 @@ class EpollEventManager(object):
self.epoll.modify(fd, fd in self.reader_set and EPOLLIN)
def log(self):
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:')
logging.info('Event Manager:')
logging.info(' Readers: %r', list(self.reader_set))
logging.info(' Writers: %r', list(self.writer_set))
logging.info(' Connections:')
pending_set = set(self._pending_processing)
for fd, conn in self.connection_dict.items():
neo.lib.logging.info(' %r: %r (pending=%r)', fd, conn,
logging.info(' %r: %r (pending=%r)', fd, conn,
conn in pending_set)
......
......@@ -15,7 +15,7 @@
# along with this program. If not, see <http://www.gnu.org/licenses/>.
from functools import wraps
import neo.lib
from . import logging
from .protocol import (
NodeStates, Packets, ErrorCodes, Errors, BrokenNodeDisallowedError,
NotReadyError, PacketMalformedError, ProtocolError, UnexpectedPacketError)
......@@ -38,7 +38,7 @@ class EventHandler(object):
else:
message = 'unexpected packet: %s in %s' % (message,
self.__class__.__name__)
neo.lib.logging.error(message)
logging.error(message)
conn.answer(Errors.ProtocolError(message))
conn.abort()
# self.peerBroken(conn)
......@@ -58,7 +58,7 @@ class EventHandler(object):
self.__unexpectedPacket(conn, packet, *e.args)
except PacketMalformedError:
if not conn.isClosed():
neo.lib.logging.error('malformed packet from %r', conn)
logging.error('malformed packet from %r', conn)
conn.notify(Packets.Notify('Malformed packet: %r' % (packet, )))
conn.abort()