Use proper translating helper for logging
Add hacking rule borrowed from keystone to make sure we don't regress and fix all the issues found by the hacking check. Change-Id: I41635fdd83c3e04d04f9849a72c49ccb5ac42875
This commit is contained in:
parent
385a768900
commit
588d795b3c
oslo_messaging
@ -26,6 +26,7 @@ import six
|
|||||||
|
|
||||||
import oslo_messaging
|
import oslo_messaging
|
||||||
from oslo_messaging._i18n import _
|
from oslo_messaging._i18n import _
|
||||||
|
from oslo_messaging._i18n import _LE
|
||||||
from oslo_messaging import _utils as utils
|
from oslo_messaging import _utils as utils
|
||||||
|
|
||||||
LOG = logging.getLogger(__name__)
|
LOG = logging.getLogger(__name__)
|
||||||
@ -84,7 +85,7 @@ class RPCException(Exception):
|
|||||||
except Exception:
|
except Exception:
|
||||||
# kwargs doesn't match a variable in the message
|
# kwargs doesn't match a variable in the message
|
||||||
# log the issue and the kwargs
|
# log the issue and the kwargs
|
||||||
LOG.exception(_('Exception in string format operation'))
|
LOG.exception(_LE('Exception in string format operation'))
|
||||||
for name, value in six.iteritems(kwargs):
|
for name, value in six.iteritems(kwargs):
|
||||||
LOG.error("%s: %s", name, value)
|
LOG.error("%s: %s", name, value)
|
||||||
# at least get the core message out if something happened
|
# at least get the core message out if something happened
|
||||||
@ -169,7 +170,7 @@ def serialize_remote_exception(failure_info, log_failure=True):
|
|||||||
tb = traceback.format_exception(*failure_info)
|
tb = traceback.format_exception(*failure_info)
|
||||||
failure = failure_info[1]
|
failure = failure_info[1]
|
||||||
if log_failure:
|
if log_failure:
|
||||||
LOG.error(_("Returning exception %s to caller"),
|
LOG.error(_LE("Returning exception %s to caller"),
|
||||||
six.text_type(failure))
|
six.text_type(failure))
|
||||||
LOG.error(tb)
|
LOG.error(tb)
|
||||||
|
|
||||||
|
@ -32,6 +32,8 @@ from oslo_messaging._drivers import amqpdriver
|
|||||||
from oslo_messaging._drivers import base
|
from oslo_messaging._drivers import base
|
||||||
from oslo_messaging._drivers import common as rpc_common
|
from oslo_messaging._drivers import common as rpc_common
|
||||||
from oslo_messaging._i18n import _
|
from oslo_messaging._i18n import _
|
||||||
|
from oslo_messaging._i18n import _LE
|
||||||
|
from oslo_messaging._i18n import _LI
|
||||||
from oslo_messaging import exceptions
|
from oslo_messaging import exceptions
|
||||||
|
|
||||||
qpid_codec = importutils.try_import("qpid.codec010")
|
qpid_codec = importutils.try_import("qpid.codec010")
|
||||||
@ -219,7 +221,7 @@ class ConsumerBase(object):
|
|||||||
self._unpack_json_msg(message)
|
self._unpack_json_msg(message)
|
||||||
self.callback(QpidMessage(self.session, message))
|
self.callback(QpidMessage(self.session, message))
|
||||||
except Exception:
|
except Exception:
|
||||||
LOG.exception(_("Failed to process message... skipping it."))
|
LOG.exception(_LE("Failed to process message... skipping it."))
|
||||||
self.session.acknowledge(message)
|
self.session.acknowledge(message)
|
||||||
|
|
||||||
def get_receiver(self):
|
def get_receiver(self):
|
||||||
@ -574,13 +576,13 @@ class Connection(object):
|
|||||||
LOG.error(msg)
|
LOG.error(msg)
|
||||||
raise exceptions.MessageDeliveryFailure(msg)
|
raise exceptions.MessageDeliveryFailure(msg)
|
||||||
else:
|
else:
|
||||||
msg = _("Unable to connect to AMQP server on %(broker)s: "
|
msg = _LE("Unable to connect to AMQP server on "
|
||||||
"%(e)s. Sleeping %(delay)s seconds") % msg_dict
|
"%(broker)s: %(e)s. Sleeping %(delay)s seconds")
|
||||||
LOG.error(msg)
|
LOG.error(msg, msg_dict)
|
||||||
time.sleep(delay)
|
time.sleep(delay)
|
||||||
delay = min(delay + 1, 5)
|
delay = min(delay + 1, 5)
|
||||||
else:
|
else:
|
||||||
LOG.info(_('Connected to AMQP server on %s'), broker['host'])
|
LOG.info(_LI('Connected to AMQP server on %s'), broker['host'])
|
||||||
break
|
break
|
||||||
|
|
||||||
self.session = self.connection.session()
|
self.session = self.connection.session()
|
||||||
@ -643,8 +645,8 @@ class Connection(object):
|
|||||||
"""
|
"""
|
||||||
def _connect_error(exc):
|
def _connect_error(exc):
|
||||||
log_info = {'topic': topic, 'err_str': exc}
|
log_info = {'topic': topic, 'err_str': exc}
|
||||||
LOG.error(_("Failed to declare consumer for topic '%(topic)s': "
|
LOG.error(_LE("Failed to declare consumer for topic '%(topic)s': "
|
||||||
"%(err_str)s"), log_info)
|
"%(err_str)s"), log_info)
|
||||||
|
|
||||||
def _declare_consumer():
|
def _declare_consumer():
|
||||||
consumer = consumer_cls(self.driver_conf, self.session, topic,
|
consumer = consumer_cls(self.driver_conf, self.session, topic,
|
||||||
@ -666,7 +668,7 @@ class Connection(object):
|
|||||||
|
|
||||||
def _error_callback(exc):
|
def _error_callback(exc):
|
||||||
timer.check_return(_raise_timeout, exc)
|
timer.check_return(_raise_timeout, exc)
|
||||||
LOG.exception(_('Failed to consume message from queue: %s'), exc)
|
LOG.exception(_LE('Failed to consume message from queue: %s'), exc)
|
||||||
|
|
||||||
def _consume():
|
def _consume():
|
||||||
# NOTE(sileht):
|
# NOTE(sileht):
|
||||||
@ -691,8 +693,8 @@ class Connection(object):
|
|||||||
try:
|
try:
|
||||||
self._lookup_consumer(nxt_receiver).consume()
|
self._lookup_consumer(nxt_receiver).consume()
|
||||||
except Exception:
|
except Exception:
|
||||||
LOG.exception(_("Error processing message. "
|
LOG.exception(_LE("Error processing message. "
|
||||||
"Skipping it."))
|
"Skipping it."))
|
||||||
|
|
||||||
self.ensure(_error_callback, _consume)
|
self.ensure(_error_callback, _consume)
|
||||||
|
|
||||||
@ -701,7 +703,7 @@ class Connection(object):
|
|||||||
|
|
||||||
def _connect_error(exc):
|
def _connect_error(exc):
|
||||||
log_info = {'topic': topic, 'err_str': exc}
|
log_info = {'topic': topic, 'err_str': exc}
|
||||||
LOG.exception(_("Failed to publish message to topic "
|
LOG.exception(_LE("Failed to publish message to topic "
|
||||||
"'%(topic)s': %(err_str)s"), log_info)
|
"'%(topic)s': %(err_str)s"), log_info)
|
||||||
|
|
||||||
def _publisher_send():
|
def _publisher_send():
|
||||||
|
@ -260,8 +260,8 @@ class Consumer(object):
|
|||||||
try:
|
try:
|
||||||
self.callback(RabbitMessage(message))
|
self.callback(RabbitMessage(message))
|
||||||
except Exception:
|
except Exception:
|
||||||
LOG.exception(_("Failed to process message"
|
LOG.exception(_LE("Failed to process message"
|
||||||
" ... skipping it."))
|
" ... skipping it."))
|
||||||
message.ack()
|
message.ack()
|
||||||
|
|
||||||
|
|
||||||
@ -605,7 +605,7 @@ class Connection(object):
|
|||||||
retry = None
|
retry = None
|
||||||
|
|
||||||
def on_error(exc, interval):
|
def on_error(exc, interval):
|
||||||
LOG.debug(_("Received recoverable error from kombu:"),
|
LOG.debug("Received recoverable error from kombu:",
|
||||||
exc_info=True)
|
exc_info=True)
|
||||||
|
|
||||||
recoverable_error_callback and recoverable_error_callback(exc)
|
recoverable_error_callback and recoverable_error_callback(exc)
|
||||||
@ -681,7 +681,7 @@ class Connection(object):
|
|||||||
self._set_current_channel(channel)
|
self._set_current_channel(channel)
|
||||||
return ret
|
return ret
|
||||||
except recoverable_errors as exc:
|
except recoverable_errors as exc:
|
||||||
LOG.debug(_("Received recoverable error from kombu:"),
|
LOG.debug("Received recoverable error from kombu:",
|
||||||
exc_info=True)
|
exc_info=True)
|
||||||
error_callback and error_callback(exc)
|
error_callback and error_callback(exc)
|
||||||
self._set_current_channel(None)
|
self._set_current_channel(None)
|
||||||
@ -823,8 +823,8 @@ class Connection(object):
|
|||||||
|
|
||||||
def _connect_error(exc):
|
def _connect_error(exc):
|
||||||
log_info = {'topic': consumer.routing_key, 'err_str': exc}
|
log_info = {'topic': consumer.routing_key, 'err_str': exc}
|
||||||
LOG.error(_("Failed to declare consumer for topic '%(topic)s': "
|
LOG.error(_LE("Failed to declare consumer for topic '%(topic)s': "
|
||||||
"%(err_str)s"), log_info)
|
"%(err_str)s"), log_info)
|
||||||
|
|
||||||
def _declare_consumer():
|
def _declare_consumer():
|
||||||
consumer.declare(self)
|
consumer.declare(self)
|
||||||
@ -852,7 +852,7 @@ class Connection(object):
|
|||||||
|
|
||||||
def _error_callback(exc):
|
def _error_callback(exc):
|
||||||
_recoverable_error_callback(exc)
|
_recoverable_error_callback(exc)
|
||||||
LOG.error(_('Failed to consume message from queue: %s'),
|
LOG.error(_LE('Failed to consume message from queue: %s'),
|
||||||
exc)
|
exc)
|
||||||
|
|
||||||
def _consume():
|
def _consume():
|
||||||
@ -948,8 +948,8 @@ class Connection(object):
|
|||||||
|
|
||||||
def _error_callback(exc):
|
def _error_callback(exc):
|
||||||
log_info = {'topic': exchange.name, 'err_str': exc}
|
log_info = {'topic': exchange.name, 'err_str': exc}
|
||||||
LOG.error(_("Failed to publish message to topic "
|
LOG.error(_LE("Failed to publish message to topic "
|
||||||
"'%(topic)s': %(err_str)s"), log_info)
|
"'%(topic)s': %(err_str)s"), log_info)
|
||||||
LOG.debug('Exception', exc_info=exc)
|
LOG.debug('Exception', exc_info=exc)
|
||||||
|
|
||||||
method = functools.partial(method, exchange, msg, routing_key, timeout)
|
method = functools.partial(method, exchange, msg, routing_key, timeout)
|
||||||
|
@ -36,7 +36,7 @@ from stevedore import driver
|
|||||||
from oslo_messaging._drivers import base
|
from oslo_messaging._drivers import base
|
||||||
from oslo_messaging._drivers import common as rpc_common
|
from oslo_messaging._drivers import common as rpc_common
|
||||||
from oslo_messaging._executors import impl_pooledexecutor # FIXME(markmc)
|
from oslo_messaging._executors import impl_pooledexecutor # FIXME(markmc)
|
||||||
from oslo_messaging._i18n import _, _LE, _LW
|
from oslo_messaging._i18n import _, _LE, _LI, _LW
|
||||||
from oslo_messaging._drivers import pool
|
from oslo_messaging._drivers import pool
|
||||||
|
|
||||||
|
|
||||||
@ -102,7 +102,7 @@ def _serialize(data):
|
|||||||
return jsonutils.dumps(data, ensure_ascii=True)
|
return jsonutils.dumps(data, ensure_ascii=True)
|
||||||
except TypeError:
|
except TypeError:
|
||||||
with excutils.save_and_reraise_exception():
|
with excutils.save_and_reraise_exception():
|
||||||
LOG.error(_("JSON serialization failed."))
|
LOG.error(_LE("JSON serialization failed."))
|
||||||
|
|
||||||
|
|
||||||
def _deserialize(data):
|
def _deserialize(data):
|
||||||
@ -388,7 +388,7 @@ class InternalContext(object):
|
|||||||
rpc_common.serialize_remote_exception(e._exc_info,
|
rpc_common.serialize_remote_exception(e._exc_info,
|
||||||
log_failure=False)}
|
log_failure=False)}
|
||||||
except Exception:
|
except Exception:
|
||||||
LOG.error(_("Exception during message handling"))
|
LOG.error(_LE("Exception during message handling"))
|
||||||
return {'exc':
|
return {'exc':
|
||||||
rpc_common.serialize_remote_exception(sys.exc_info())}
|
rpc_common.serialize_remote_exception(sys.exc_info())}
|
||||||
|
|
||||||
@ -471,7 +471,7 @@ class ZmqBaseReactor(ConsumerBase):
|
|||||||
def register(self, proxy, in_addr, zmq_type_in,
|
def register(self, proxy, in_addr, zmq_type_in,
|
||||||
in_bind=True, subscribe=None):
|
in_bind=True, subscribe=None):
|
||||||
|
|
||||||
LOG.info(_("Registering reactor"))
|
LOG.info(_LI("Registering reactor"))
|
||||||
|
|
||||||
if zmq_type_in not in (zmq.PULL, zmq.SUB):
|
if zmq_type_in not in (zmq.PULL, zmq.SUB):
|
||||||
raise RPCException("Bad input socktype")
|
raise RPCException("Bad input socktype")
|
||||||
@ -483,11 +483,11 @@ class ZmqBaseReactor(ConsumerBase):
|
|||||||
self.proxies[inq] = proxy
|
self.proxies[inq] = proxy
|
||||||
self.sockets.append(inq)
|
self.sockets.append(inq)
|
||||||
|
|
||||||
LOG.info(_("In reactor registered"))
|
LOG.info(_LI("In reactor registered"))
|
||||||
|
|
||||||
def consume_in_thread(self):
|
def consume_in_thread(self):
|
||||||
def _consume(sock):
|
def _consume(sock):
|
||||||
LOG.info(_("Consuming socket"))
|
LOG.info(_LI("Consuming socket"))
|
||||||
while not sock.closed:
|
while not sock.closed:
|
||||||
self.consume(sock)
|
self.consume(sock)
|
||||||
|
|
||||||
@ -539,7 +539,7 @@ class ZmqProxy(ZmqBaseReactor):
|
|||||||
|
|
||||||
if topic not in self.topic_proxy:
|
if topic not in self.topic_proxy:
|
||||||
def publisher(waiter):
|
def publisher(waiter):
|
||||||
LOG.info(_("Creating proxy for topic: %s"), topic)
|
LOG.info(_LI("Creating proxy for topic: %s"), topic)
|
||||||
|
|
||||||
try:
|
try:
|
||||||
# The topic is received over the network,
|
# The topic is received over the network,
|
||||||
@ -577,14 +577,14 @@ class ZmqProxy(ZmqBaseReactor):
|
|||||||
try:
|
try:
|
||||||
wait_sock_creation.wait()
|
wait_sock_creation.wait()
|
||||||
except RPCException:
|
except RPCException:
|
||||||
LOG.error(_("Topic socket file creation failed."))
|
LOG.error(_LE("Topic socket file creation failed."))
|
||||||
return
|
return
|
||||||
|
|
||||||
try:
|
try:
|
||||||
self.topic_proxy[topic].put_nowait(data)
|
self.topic_proxy[topic].put_nowait(data)
|
||||||
except eventlet.queue.Full:
|
except eventlet.queue.Full:
|
||||||
LOG.error(_("Local per-topic backlog buffer full for topic "
|
LOG.error(_LE("Local per-topic backlog buffer full for topic "
|
||||||
"%s. Dropping message."), topic)
|
"%s. Dropping message."), topic)
|
||||||
|
|
||||||
def consume_in_thread(self):
|
def consume_in_thread(self):
|
||||||
"""Runs the ZmqProxy service."""
|
"""Runs the ZmqProxy service."""
|
||||||
@ -599,8 +599,8 @@ class ZmqProxy(ZmqBaseReactor):
|
|||||||
except os.error:
|
except os.error:
|
||||||
if not os.path.isdir(ipc_dir):
|
if not os.path.isdir(ipc_dir):
|
||||||
with excutils.save_and_reraise_exception():
|
with excutils.save_and_reraise_exception():
|
||||||
LOG.error(_("Required IPC directory does not exist at"
|
LOG.error(_LE("Required IPC directory does not exist at"
|
||||||
" %s"), ipc_dir)
|
" %s"), ipc_dir)
|
||||||
try:
|
try:
|
||||||
self.register(consumption_proxy,
|
self.register(consumption_proxy,
|
||||||
consume_in,
|
consume_in,
|
||||||
@ -608,11 +608,11 @@ class ZmqProxy(ZmqBaseReactor):
|
|||||||
except zmq.ZMQError:
|
except zmq.ZMQError:
|
||||||
if os.access(ipc_dir, os.X_OK):
|
if os.access(ipc_dir, os.X_OK):
|
||||||
with excutils.save_and_reraise_exception():
|
with excutils.save_and_reraise_exception():
|
||||||
LOG.error(_("Permission denied to IPC directory at"
|
LOG.error(_LE("Permission denied to IPC directory at"
|
||||||
" %s"), ipc_dir)
|
" %s"), ipc_dir)
|
||||||
with excutils.save_and_reraise_exception():
|
with excutils.save_and_reraise_exception():
|
||||||
LOG.error(_("Could not create ZeroMQ receiver daemon. "
|
LOG.error(_LE("Could not create ZeroMQ receiver daemon. "
|
||||||
"Socket may already be in use."))
|
"Socket may already be in use."))
|
||||||
|
|
||||||
super(ZmqProxy, self).consume_in_thread()
|
super(ZmqProxy, self).consume_in_thread()
|
||||||
|
|
||||||
@ -664,7 +664,7 @@ class ZmqReactor(ZmqBaseReactor):
|
|||||||
# Unmarshal only after verifying the message.
|
# Unmarshal only after verifying the message.
|
||||||
ctx = RpcContext.unmarshal(data[3])
|
ctx = RpcContext.unmarshal(data[3])
|
||||||
else:
|
else:
|
||||||
LOG.error(_("ZMQ Envelope version unsupported or unknown."))
|
LOG.error(_LE("ZMQ Envelope version unsupported or unknown."))
|
||||||
return
|
return
|
||||||
|
|
||||||
self.pool.spawn_n(self.process, proxy, ctx, request)
|
self.pool.spawn_n(self.process, proxy, ctx, request)
|
||||||
@ -692,7 +692,7 @@ class Connection(rpc_common.Connection):
|
|||||||
topic = '.'.join((topic.split('.', 1)[0], CONF.rpc_zmq_host))
|
topic = '.'.join((topic.split('.', 1)[0], CONF.rpc_zmq_host))
|
||||||
|
|
||||||
if topic in self.topics:
|
if topic in self.topics:
|
||||||
LOG.info(_("Skipping topic registration. Already registered."))
|
LOG.info(_LI("Skipping topic registration. Already registered."))
|
||||||
return
|
return
|
||||||
|
|
||||||
# Receive messages from (local) proxy
|
# Receive messages from (local) proxy
|
||||||
|
@ -23,6 +23,7 @@ import eventlet
|
|||||||
from oslo_config import cfg
|
from oslo_config import cfg
|
||||||
|
|
||||||
from oslo_messaging._i18n import _
|
from oslo_messaging._i18n import _
|
||||||
|
from oslo_messaging._i18n import _LI
|
||||||
|
|
||||||
matchmaker_opts = [
|
matchmaker_opts = [
|
||||||
cfg.IntOpt('matchmaker_heartbeat_freq',
|
cfg.IntOpt('matchmaker_heartbeat_freq',
|
||||||
@ -211,7 +212,7 @@ class HeartbeatMatchMakerBase(MatchMakerBase):
|
|||||||
self.hosts.discard(host)
|
self.hosts.discard(host)
|
||||||
self.backend_unregister(key, '.'.join((key, host)))
|
self.backend_unregister(key, '.'.join((key, host)))
|
||||||
|
|
||||||
LOG.info(_("Matchmaker unregistered: %(key)s, %(host)s"),
|
LOG.info(_LI("Matchmaker unregistered: %(key)s, %(host)s"),
|
||||||
{'key': key, 'host': host})
|
{'key': key, 'host': host})
|
||||||
|
|
||||||
def start_heartbeat(self):
|
def start_heartbeat(self):
|
||||||
|
@ -23,7 +23,7 @@ import logging
|
|||||||
from oslo_config import cfg
|
from oslo_config import cfg
|
||||||
|
|
||||||
from oslo_messaging._drivers import matchmaker as mm
|
from oslo_messaging._drivers import matchmaker as mm
|
||||||
from oslo_messaging._i18n import _
|
from oslo_messaging._i18n import _LW
|
||||||
|
|
||||||
matchmaker_opts = [
|
matchmaker_opts = [
|
||||||
# Matchmaker ring file
|
# Matchmaker ring file
|
||||||
@ -71,8 +71,8 @@ class RoundRobinRingExchange(RingExchange):
|
|||||||
def run(self, key):
|
def run(self, key):
|
||||||
if not self._ring_has(key):
|
if not self._ring_has(key):
|
||||||
LOG.warn(
|
LOG.warn(
|
||||||
_("No key defining hosts for topic '%s', "
|
_LW("No key defining hosts for topic '%s', "
|
||||||
"see ringfile"), key
|
"see ringfile"), key
|
||||||
)
|
)
|
||||||
return []
|
return []
|
||||||
host = next(self.ring0[key])
|
host = next(self.ring0[key])
|
||||||
@ -89,8 +89,8 @@ class FanoutRingExchange(RingExchange):
|
|||||||
nkey = key.split('fanout~')[1:][0]
|
nkey = key.split('fanout~')[1:][0]
|
||||||
if not self._ring_has(nkey):
|
if not self._ring_has(nkey):
|
||||||
LOG.warn(
|
LOG.warn(
|
||||||
_("No key defining hosts for topic '%s', "
|
_LW("No key defining hosts for topic '%s', "
|
||||||
"see ringfile"), nkey
|
"see ringfile"), nkey
|
||||||
)
|
)
|
||||||
return []
|
return []
|
||||||
return map(lambda x: (key + '.' + x, x), self.ring[nkey])
|
return map(lambda x: (key + '.' + x, x), self.ring[nkey])
|
||||||
|
@ -14,6 +14,8 @@
|
|||||||
|
|
||||||
import re
|
import re
|
||||||
|
|
||||||
|
import ast
|
||||||
|
import six
|
||||||
|
|
||||||
oslo_namespace_imports_dot = re.compile(r"import[\s]+oslo[.][^\s]+")
|
oslo_namespace_imports_dot = re.compile(r"import[\s]+oslo[.][^\s]+")
|
||||||
oslo_namespace_imports_from_dot = re.compile(r"from[\s]+oslo[.]")
|
oslo_namespace_imports_from_dot = re.compile(r"from[\s]+oslo[.]")
|
||||||
@ -38,5 +40,306 @@ def check_oslo_namespace_imports(logical_line):
|
|||||||
yield(0, msg)
|
yield(0, msg)
|
||||||
|
|
||||||
|
|
||||||
|
class BaseASTChecker(ast.NodeVisitor):
|
||||||
|
"""Provides a simple framework for writing AST-based checks.
|
||||||
|
|
||||||
|
Subclasses should implement visit_* methods like any other AST visitor
|
||||||
|
implementation. When they detect an error for a particular node the
|
||||||
|
method should call ``self.add_error(offending_node)``. Details about
|
||||||
|
where in the code the error occurred will be pulled from the node
|
||||||
|
object.
|
||||||
|
|
||||||
|
Subclasses should also provide a class variable named CHECK_DESC to
|
||||||
|
be used for the human readable error message.
|
||||||
|
|
||||||
|
"""
|
||||||
|
|
||||||
|
def __init__(self, tree, filename):
|
||||||
|
"""This object is created automatically by pep8.
|
||||||
|
|
||||||
|
:param tree: an AST tree
|
||||||
|
:param filename: name of the file being analyzed
|
||||||
|
(ignored by our checks)
|
||||||
|
"""
|
||||||
|
self._tree = tree
|
||||||
|
self._errors = []
|
||||||
|
|
||||||
|
def run(self):
|
||||||
|
"""Called automatically by pep8."""
|
||||||
|
self.visit(self._tree)
|
||||||
|
return self._errors
|
||||||
|
|
||||||
|
def add_error(self, node, message=None):
|
||||||
|
"""Add an error caused by a node to the list of errors for pep8."""
|
||||||
|
message = message or self.CHECK_DESC
|
||||||
|
error = (node.lineno, node.col_offset, message, self.__class__)
|
||||||
|
self._errors.append(error)
|
||||||
|
|
||||||
|
|
||||||
|
class CheckForLoggingIssues(BaseASTChecker):
|
||||||
|
|
||||||
|
DEBUG_CHECK_DESC = 'O324 Using translated string in debug logging'
|
||||||
|
NONDEBUG_CHECK_DESC = 'O325 Not using translating helper for logging'
|
||||||
|
EXCESS_HELPER_CHECK_DESC = 'O326 Using hints when _ is necessary'
|
||||||
|
LOG_MODULES = ('logging')
|
||||||
|
I18N_MODULES = (
|
||||||
|
'oslo_messaging._i18n._',
|
||||||
|
'oslo_messaging._i18n._LI',
|
||||||
|
'oslo_messaging._i18n._LW',
|
||||||
|
'oslo_messaging._i18n._LE',
|
||||||
|
'oslo_messaging._i18n._LC',
|
||||||
|
)
|
||||||
|
TRANS_HELPER_MAP = {
|
||||||
|
'debug': None,
|
||||||
|
'info': '_LI',
|
||||||
|
'warn': '_LW',
|
||||||
|
'warning': '_LW',
|
||||||
|
'error': '_LE',
|
||||||
|
'exception': '_LE',
|
||||||
|
'critical': '_LC',
|
||||||
|
}
|
||||||
|
|
||||||
|
def __init__(self, tree, filename):
|
||||||
|
super(CheckForLoggingIssues, self).__init__(tree, filename)
|
||||||
|
|
||||||
|
self.logger_names = []
|
||||||
|
self.logger_module_names = []
|
||||||
|
self.i18n_names = {}
|
||||||
|
|
||||||
|
# NOTE(dstanek): this kinda accounts for scopes when talking
|
||||||
|
# about only leaf node in the graph
|
||||||
|
self.assignments = {}
|
||||||
|
|
||||||
|
def generic_visit(self, node):
|
||||||
|
"""Called if no explicit visitor function exists for a node."""
|
||||||
|
for field, value in ast.iter_fields(node):
|
||||||
|
if isinstance(value, list):
|
||||||
|
for item in value:
|
||||||
|
if isinstance(item, ast.AST):
|
||||||
|
item._parent = node
|
||||||
|
self.visit(item)
|
||||||
|
elif isinstance(value, ast.AST):
|
||||||
|
value._parent = node
|
||||||
|
self.visit(value)
|
||||||
|
|
||||||
|
def _filter_imports(self, module_name, alias):
|
||||||
|
"""Keeps lists of logging and i18n imports
|
||||||
|
|
||||||
|
"""
|
||||||
|
if module_name in self.LOG_MODULES:
|
||||||
|
self.logger_module_names.append(alias.asname or alias.name)
|
||||||
|
elif module_name in self.I18N_MODULES:
|
||||||
|
self.i18n_names[alias.asname or alias.name] = alias.name
|
||||||
|
|
||||||
|
def visit_Import(self, node):
|
||||||
|
for alias in node.names:
|
||||||
|
self._filter_imports(alias.name, alias)
|
||||||
|
return super(CheckForLoggingIssues, self).generic_visit(node)
|
||||||
|
|
||||||
|
def visit_ImportFrom(self, node):
|
||||||
|
for alias in node.names:
|
||||||
|
full_name = '%s.%s' % (node.module, alias.name)
|
||||||
|
self._filter_imports(full_name, alias)
|
||||||
|
return super(CheckForLoggingIssues, self).generic_visit(node)
|
||||||
|
|
||||||
|
def _find_name(self, node):
|
||||||
|
"""Return the fully qualified name or a Name or Attribute."""
|
||||||
|
if isinstance(node, ast.Name):
|
||||||
|
return node.id
|
||||||
|
elif (isinstance(node, ast.Attribute)
|
||||||
|
and isinstance(node.value, (ast.Name, ast.Attribute))):
|
||||||
|
method_name = node.attr
|
||||||
|
obj_name = self._find_name(node.value)
|
||||||
|
if obj_name is None:
|
||||||
|
return None
|
||||||
|
return obj_name + '.' + method_name
|
||||||
|
elif isinstance(node, six.string_types):
|
||||||
|
return node
|
||||||
|
else: # could be Subscript, Call or many more
|
||||||
|
return None
|
||||||
|
|
||||||
|
def visit_Assign(self, node):
|
||||||
|
"""Look for 'LOG = logging.getLogger'
|
||||||
|
|
||||||
|
This handles the simple case:
|
||||||
|
name = [logging_module].getLogger(...)
|
||||||
|
|
||||||
|
- or -
|
||||||
|
|
||||||
|
name = [i18n_name](...)
|
||||||
|
|
||||||
|
And some much more comple ones:
|
||||||
|
name = [i18n_name](...) % X
|
||||||
|
|
||||||
|
- or -
|
||||||
|
|
||||||
|
self.name = [i18n_name](...) % X
|
||||||
|
|
||||||
|
"""
|
||||||
|
attr_node_types = (ast.Name, ast.Attribute)
|
||||||
|
|
||||||
|
if (len(node.targets) != 1
|
||||||
|
or not isinstance(node.targets[0], attr_node_types)):
|
||||||
|
# say no to: "x, y = ..."
|
||||||
|
return super(CheckForLoggingIssues, self).generic_visit(node)
|
||||||
|
|
||||||
|
target_name = self._find_name(node.targets[0])
|
||||||
|
|
||||||
|
if (isinstance(node.value, ast.BinOp) and
|
||||||
|
isinstance(node.value.op, ast.Mod)):
|
||||||
|
if (isinstance(node.value.left, ast.Call) and
|
||||||
|
isinstance(node.value.left.func, ast.Name) and
|
||||||
|
node.value.left.func.id in self.i18n_names):
|
||||||
|
# NOTE(dstanek): this is done to match cases like:
|
||||||
|
# `msg = _('something %s') % x`
|
||||||
|
node = ast.Assign(value=node.value.left)
|
||||||
|
|
||||||
|
if not isinstance(node.value, ast.Call):
|
||||||
|
# node.value must be a call to getLogger
|
||||||
|
self.assignments.pop(target_name, None)
|
||||||
|
return super(CheckForLoggingIssues, self).generic_visit(node)
|
||||||
|
|
||||||
|
# is this a call to an i18n function?
|
||||||
|
if (isinstance(node.value.func, ast.Name)
|
||||||
|
and node.value.func.id in self.i18n_names):
|
||||||
|
self.assignments[target_name] = node.value.func.id
|
||||||
|
return super(CheckForLoggingIssues, self).generic_visit(node)
|
||||||
|
|
||||||
|
if (not isinstance(node.value.func, ast.Attribute)
|
||||||
|
or not isinstance(node.value.func.value, attr_node_types)):
|
||||||
|
# function must be an attribute on an object like
|
||||||
|
# logging.getLogger
|
||||||
|
return super(CheckForLoggingIssues, self).generic_visit(node)
|
||||||
|
|
||||||
|
object_name = self._find_name(node.value.func.value)
|
||||||
|
func_name = node.value.func.attr
|
||||||
|
|
||||||
|
if (object_name in self.logger_module_names
|
||||||
|
and func_name == 'getLogger'):
|
||||||
|
self.logger_names.append(target_name)
|
||||||
|
|
||||||
|
return super(CheckForLoggingIssues, self).generic_visit(node)
|
||||||
|
|
||||||
|
def visit_Call(self, node):
|
||||||
|
"""Look for the 'LOG.*' calls.
|
||||||
|
|
||||||
|
"""
|
||||||
|
|
||||||
|
# obj.method
|
||||||
|
if isinstance(node.func, ast.Attribute):
|
||||||
|
obj_name = self._find_name(node.func.value)
|
||||||
|
if isinstance(node.func.value, ast.Name):
|
||||||
|
method_name = node.func.attr
|
||||||
|
elif isinstance(node.func.value, ast.Attribute):
|
||||||
|
obj_name = self._find_name(node.func.value)
|
||||||
|
method_name = node.func.attr
|
||||||
|
else: # could be Subscript, Call or many more
|
||||||
|
return super(CheckForLoggingIssues, self).generic_visit(node)
|
||||||
|
|
||||||
|
# must be a logger instance and one of the support logging methods
|
||||||
|
if (obj_name not in self.logger_names
|
||||||
|
or method_name not in self.TRANS_HELPER_MAP):
|
||||||
|
return super(CheckForLoggingIssues, self).generic_visit(node)
|
||||||
|
|
||||||
|
# the call must have arguments
|
||||||
|
if not len(node.args):
|
||||||
|
return super(CheckForLoggingIssues, self).generic_visit(node)
|
||||||
|
|
||||||
|
if method_name == 'debug':
|
||||||
|
self._process_debug(node)
|
||||||
|
elif method_name in self.TRANS_HELPER_MAP:
|
||||||
|
self._process_non_debug(node, method_name)
|
||||||
|
|
||||||
|
return super(CheckForLoggingIssues, self).generic_visit(node)
|
||||||
|
|
||||||
|
def _process_debug(self, node):
|
||||||
|
msg = node.args[0] # first arg to a logging method is the msg
|
||||||
|
|
||||||
|
# if first arg is a call to a i18n name
|
||||||
|
if (isinstance(msg, ast.Call)
|
||||||
|
and isinstance(msg.func, ast.Name)
|
||||||
|
and msg.func.id in self.i18n_names):
|
||||||
|
self.add_error(msg, message=self.DEBUG_CHECK_DESC)
|
||||||
|
|
||||||
|
# if the first arg is a reference to a i18n call
|
||||||
|
elif (isinstance(msg, ast.Name)
|
||||||
|
and msg.id in self.assignments
|
||||||
|
and not self._is_raised_later(node, msg.id)):
|
||||||
|
self.add_error(msg, message=self.DEBUG_CHECK_DESC)
|
||||||
|
|
||||||
|
def _process_non_debug(self, node, method_name):
|
||||||
|
msg = node.args[0] # first arg to a logging method is the msg
|
||||||
|
|
||||||
|
# if first arg is a call to a i18n name
|
||||||
|
if isinstance(msg, ast.Call):
|
||||||
|
try:
|
||||||
|
func_name = msg.func.id
|
||||||
|
except AttributeError:
|
||||||
|
# in the case of logging only an exception, the msg function
|
||||||
|
# will not have an id associated with it, for instance:
|
||||||
|
# LOG.warning(six.text_type(e))
|
||||||
|
return
|
||||||
|
|
||||||
|
# the function name is the correct translation helper
|
||||||
|
# for the logging method
|
||||||
|
if func_name == self.TRANS_HELPER_MAP[method_name]:
|
||||||
|
return
|
||||||
|
|
||||||
|
# the function name is an alias for the correct translation
|
||||||
|
# helper for the loggine method
|
||||||
|
if (self.i18n_names[func_name] ==
|
||||||
|
self.TRANS_HELPER_MAP[method_name]):
|
||||||
|
return
|
||||||
|
|
||||||
|
self.add_error(msg, message=self.NONDEBUG_CHECK_DESC)
|
||||||
|
|
||||||
|
# if the first arg is not a reference to the correct i18n hint
|
||||||
|
elif isinstance(msg, ast.Name):
|
||||||
|
|
||||||
|
# FIXME(dstanek): to make sure more robust we should be checking
|
||||||
|
# all names passed into a logging method. we can't right now
|
||||||
|
# because:
|
||||||
|
# 1. We have code like this that we'll fix when dealing with the %:
|
||||||
|
# msg = _('....') % {}
|
||||||
|
# LOG.warn(msg)
|
||||||
|
# 2. We also do LOG.exception(e) in several places. I'm not sure
|
||||||
|
# exactly what we should be doing about that.
|
||||||
|
if msg.id not in self.assignments:
|
||||||
|
return
|
||||||
|
|
||||||
|
helper_method_name = self.TRANS_HELPER_MAP[method_name]
|
||||||
|
if (self.assignments[msg.id] != helper_method_name
|
||||||
|
and not self._is_raised_later(node, msg.id)):
|
||||||
|
self.add_error(msg, message=self.NONDEBUG_CHECK_DESC)
|
||||||
|
elif (self.assignments[msg.id] == helper_method_name
|
||||||
|
and self._is_raised_later(node, msg.id)):
|
||||||
|
self.add_error(msg, message=self.EXCESS_HELPER_CHECK_DESC)
|
||||||
|
|
||||||
|
def _is_raised_later(self, node, name):
|
||||||
|
|
||||||
|
def find_peers(node):
|
||||||
|
node_for_line = node._parent
|
||||||
|
for _field, value in ast.iter_fields(node._parent._parent):
|
||||||
|
if isinstance(value, list) and node_for_line in value:
|
||||||
|
return value[value.index(node_for_line) + 1:]
|
||||||
|
continue
|
||||||
|
return []
|
||||||
|
|
||||||
|
peers = find_peers(node)
|
||||||
|
for peer in peers:
|
||||||
|
if isinstance(peer, ast.Raise):
|
||||||
|
if (isinstance(peer.type, ast.Call) and
|
||||||
|
len(peer.type.args) > 0 and
|
||||||
|
isinstance(peer.type.args[0], ast.Name) and
|
||||||
|
name in (a.id for a in peer.type.args)):
|
||||||
|
return True
|
||||||
|
else:
|
||||||
|
return False
|
||||||
|
elif isinstance(peer, ast.Assign):
|
||||||
|
if name in (t.id for t in peer.targets):
|
||||||
|
return False
|
||||||
|
|
||||||
|
|
||||||
def factory(register):
|
def factory(register):
|
||||||
|
register(CheckForLoggingIssues)
|
||||||
register(check_oslo_namespace_imports)
|
register(check_oslo_namespace_imports)
|
||||||
|
@ -21,7 +21,7 @@ import six
|
|||||||
from stevedore import dispatch
|
from stevedore import dispatch
|
||||||
import yaml
|
import yaml
|
||||||
|
|
||||||
from oslo_messaging._i18n import _
|
from oslo_messaging._i18n import _LI, _LW
|
||||||
from oslo_messaging.notify import notifier
|
from oslo_messaging.notify import notifier
|
||||||
|
|
||||||
|
|
||||||
@ -77,7 +77,7 @@ class RoutingDriver(notifier._Driver):
|
|||||||
invoke_on_load=True,
|
invoke_on_load=True,
|
||||||
invoke_args=None)
|
invoke_args=None)
|
||||||
if not list(self.plugin_manager):
|
if not list(self.plugin_manager):
|
||||||
LOG.warning(_("Failed to load any notifiers for %s"),
|
LOG.warning(_LW("Failed to load any notifiers for %s"),
|
||||||
self.NOTIFIER_PLUGIN_NAMESPACE)
|
self.NOTIFIER_PLUGIN_NAMESPACE)
|
||||||
|
|
||||||
def _get_drivers_for_message(self, group, event_type, priority):
|
def _get_drivers_for_message(self, group, event_type, priority):
|
||||||
@ -114,7 +114,8 @@ class RoutingDriver(notifier._Driver):
|
|||||||
"""Emit the notification.
|
"""Emit the notification.
|
||||||
"""
|
"""
|
||||||
# accepted_drivers is passed in as a result of the map() function
|
# accepted_drivers is passed in as a result of the map() function
|
||||||
LOG.info(_("Routing '%(event)s' notification to '%(driver)s' driver"),
|
LOG.info(_LI("Routing '%(event)s' notification to '%(driver)s' "
|
||||||
|
"driver"),
|
||||||
{'event': message.get('event_type'), 'driver': ext.name})
|
{'event': message.get('event_type'), 'driver': ext.name})
|
||||||
ext.obj.notify(context, message, priority, retry)
|
ext.obj.notify(context, message, priority, retry)
|
||||||
|
|
||||||
|
@ -29,7 +29,7 @@ import sys
|
|||||||
|
|
||||||
import six
|
import six
|
||||||
|
|
||||||
from oslo_messaging._i18n import _
|
from oslo_messaging._i18n import _LE
|
||||||
from oslo_messaging import _utils as utils
|
from oslo_messaging import _utils as utils
|
||||||
from oslo_messaging import localcontext
|
from oslo_messaging import localcontext
|
||||||
from oslo_messaging import serializer as msg_serializer
|
from oslo_messaging import serializer as msg_serializer
|
||||||
@ -147,7 +147,7 @@ class RPCDispatcher(object):
|
|||||||
except Exception as e:
|
except Exception as e:
|
||||||
# sys.exc_info() is deleted by LOG.exception().
|
# sys.exc_info() is deleted by LOG.exception().
|
||||||
exc_info = sys.exc_info()
|
exc_info = sys.exc_info()
|
||||||
LOG.error(_('Exception during message handling: %s'), e,
|
LOG.error(_LE('Exception during message handling: %s'), e,
|
||||||
exc_info=exc_info)
|
exc_info=exc_info)
|
||||||
incoming.reply(failure=exc_info)
|
incoming.reply(failure=exc_info)
|
||||||
# NOTE(dhellmann): Remove circular object reference
|
# NOTE(dhellmann): Remove circular object reference
|
||||||
|
@ -29,7 +29,6 @@ from oslo_service import service
|
|||||||
from stevedore import driver
|
from stevedore import driver
|
||||||
|
|
||||||
from oslo_messaging._drivers import base as driver_base
|
from oslo_messaging._drivers import base as driver_base
|
||||||
from oslo_messaging._i18n import _
|
|
||||||
from oslo_messaging._i18n import _LW
|
from oslo_messaging._i18n import _LW
|
||||||
from oslo_messaging import _utils
|
from oslo_messaging import _utils
|
||||||
from oslo_messaging import exceptions
|
from oslo_messaging import exceptions
|
||||||
@ -176,9 +175,9 @@ class MessageHandlingServer(service.ServiceBase):
|
|||||||
# NOTE(dims): Need to change this to raise RuntimeError after
|
# NOTE(dims): Need to change this to raise RuntimeError after
|
||||||
# verifying/fixing other openstack projects (like Neutron)
|
# verifying/fixing other openstack projects (like Neutron)
|
||||||
# work ok with this change
|
# work ok with this change
|
||||||
LOG.warn(_("wait() should be called after stop() as it "
|
LOG.warn(_LW("wait() should be called after stop() as it "
|
||||||
"waits for existing messages to finish "
|
"waits for existing messages to finish "
|
||||||
"processing"))
|
"processing"))
|
||||||
|
|
||||||
if self._executor is not None:
|
if self._executor is not None:
|
||||||
self._executor.wait()
|
self._executor.wait()
|
||||||
|
Loading…
x
Reference in New Issue
Block a user