Improve Rally Logging (part 4)

- Remove translations

  Nobody is using translations for Rally and I don't think that
  anybody is going to use it. Target auditory for Rally are
  developers/operators which usually know well english.
  For me this looks like waste of resources, performance
  degradation (cause we are calling _()), complexity
  (+1 thing that you need to know)

- Pass to log already formatted strings

  It's very bad because in case of wrong formatting, it
  doesn't fail instead just writes errors to the logs,
  as well information about trace is lost, so it's super
  hard to fix it

  Log wrapper doesn't allow to use LOG anymore for
  formatting strings

  All places are fixed

- Improve logging of exceptions

LOG.exception() already logs exception, which means it's bad idea to
pass str(e) to it. Instead we should provide clear description of what
happend. Improved few places to write warnings or exceptions in case
of different level of logs. In few places just use LOG.exception

- Part of log messages were improved and simplified

Change-Id: I57a674fc3a2e111798bd61dd8172081989371f2e
This commit is contained in:
Boris Pavlovic 2017-09-25 14:16:59 -07:00
parent 4a7cc010b1
commit 24258bbe4a
26 changed files with 170 additions and 153 deletions

View File

@ -1 +0,0 @@
[python: **.py]

View File

@ -16,7 +16,6 @@
import collections import collections
import threading import threading
from rally.common.i18n import _LW
from rally.common import logging from rally.common import logging
@ -42,9 +41,11 @@ def _consumer(consume, queue):
try: try:
consume(cache, args) consume(cache, args)
except Exception as e: except Exception as e:
LOG.warning(_LW("Failed to consume a task from the queue: %s") % e) msg = "Failed to consume a task from the queue"
if logging.is_debug(): if logging.is_debug():
LOG.exception(e) LOG.exception(msg)
else:
LOG.warning("%s: %s" % (msg, e))
def _publisher(publish, queue): def _publisher(publish, queue):
@ -56,9 +57,11 @@ def _publisher(publish, queue):
try: try:
publish(queue) publish(queue)
except Exception as e: except Exception as e:
LOG.warning(_LW("Failed to publish a task to the queue: %s") % e) msg = "Failed to publish a task to the queue"
if logging.is_debug(): if logging.is_debug():
LOG.exception(e) LOG.exception(msg)
else:
LOG.warning("%s: %s" % (msg, e))
def run(publish, consume, consumers_count=1): def run(publish, consume, consumers_count=1):

View File

@ -35,7 +35,6 @@ from sqlalchemy.orm.exc import NoResultFound
from sqlalchemy.orm import load_only as sa_loadonly from sqlalchemy.orm import load_only as sa_loadonly
from rally.common.db.sqlalchemy import models from rally.common.db.sqlalchemy import models
from rally.common.i18n import _
from rally import consts from rally import consts
from rally import exceptions from rally import exceptions
from rally.task.processing import charts from rally.task.processing import charts
@ -77,7 +76,7 @@ def serialize_data(data):
result[key] = serialize_data(getattr(data, key)) result[key] = serialize_data(getattr(data, key))
return result return result
raise ValueError(_("Can not serialize %s") % data) raise ValueError("Can not serialize %s" % data)
def serialize(fn): def serialize(fn):
@ -216,7 +215,7 @@ class Connection(object):
return isinstance(obj, type) and issubclass(obj, models.RallyBase) return isinstance(obj, type) and issubclass(obj, models.RallyBase)
if not issubclassof_rally_base(model): if not issubclassof_rally_base(model):
raise Exception(_("The model should be a subclass of RallyBase")) raise Exception("The model should be a subclass of RallyBase")
return query return query
@ -325,7 +324,7 @@ class Connection(object):
) )
if not result: if not result:
status = " or ".join(statuses) status = " or ".join(statuses)
msg = _("Task with uuid='%(uuid)s' and in statuses:'" msg = ("Task with uuid='%(uuid)s' and in statuses:'"
"%(statuses)s' not found.'") % {"uuid": uuid, "%(statuses)s' not found.'") % {"uuid": uuid,
"statuses": status} "statuses": status}
raise exceptions.RallyException(msg) raise exceptions.RallyException(msg)

View File

@ -18,6 +18,12 @@ See https://docs.openstack.org/oslo.i18n/latest/user/usage.html .
import oslo_i18n import oslo_i18n
from rally.common import logging
LOG = logging.getLogger(__name__)
LOG.warning("rally.common.i18n module is deprecated and is going to be "
"removed. Please do not import it.")
_translators = oslo_i18n.TranslatorFactory(domain="rally") _translators = oslo_i18n.TranslatorFactory(domain="rally")

View File

@ -104,7 +104,7 @@ class SubunitV2StreamResult(object):
self._tests[t_id]["reason"] = self._skipped_tests[t_id] self._tests[t_id]["reason"] = self._skipped_tests[t_id]
status += ": %s" % self._tests[t_id]["reason"] status += ": %s" % self._tests[t_id]["reason"]
if self._live: if self._live:
self._logger.info("{-} %s ... %s", name, status) self._logger.info("{-} %s ... %s" % (name, status))
self._skipped_tests.pop(t_id) self._skipped_tests.pop(t_id)
@ -222,8 +222,8 @@ class SubunitV2StreamResult(object):
status += ": %s" % reason status += ": %s" % reason
w = "{%s} " % worker.pop().split("-")[1] if worker else "-" w = "{%s} " % worker.pop().split("-")[1] if worker else "-"
self._logger.info( self._logger.info("%s ... %s"
"%s ... %s", w + self._get_test_name(test_id), status) % (w + self._get_test_name(test_id), status))
def filter_tests(self, status): def filter_tests(self, status):
"""Filter tests by given status.""" """Filter tests by given status."""

View File

@ -20,7 +20,6 @@ from oslo_config import cfg
from oslo_log import handlers from oslo_log import handlers
from oslo_log import log as oslogging from oslo_log import log as oslogging
from rally.common.i18n import _
log = __import__("logging") log = __import__("logging")
@ -61,9 +60,28 @@ def setup(product_name, version="unknown"):
class RallyContextAdapter(oslogging.KeywordArgumentAdapter): class RallyContextAdapter(oslogging.KeywordArgumentAdapter):
_msg = "Do not use *args for string formatting for log message: %s"
def _check_args(self, msg, *args):
if args:
self.log(log.WARNING, self._msg % msg)
def debug(self, msg, *args, **kwargs): def debug(self, msg, *args, **kwargs):
self._check_args(msg, *args)
self.log(log.RDEBUG, msg, *args, **kwargs) self.log(log.RDEBUG, msg, *args, **kwargs)
def info(self, msg, *args, **kwargs):
self._check_args(msg, *args)
self.log(log.INFO, msg, *args, **kwargs)
def warning(self, msg, *args, **kwargs):
self._check_args(msg, *args)
self.log(log.WARNING, msg, *args, **kwargs)
def error(self, msg, *args, **kwargs):
self._check_args(msg, *args)
self.log(log.ERROR, msg, *args, **kwargs)
def getLogger(name="unknown", version="unknown"): def getLogger(name="unknown", version="unknown"):
@ -193,11 +211,11 @@ def _log_wrapper(obj, log_function, msg, **kw):
def wrapper(self, *args, **kwargs): def wrapper(self, *args, **kwargs):
params = {"msg": msg % kw, "obj_name": obj.title(), params = {"msg": msg % kw, "obj_name": obj.title(),
"uuid": getattr(self, obj)["uuid"]} "uuid": getattr(self, obj)["uuid"]}
log_function(_("%(obj_name)s %(uuid)s | Starting: %(msg)s") % log_function("%(obj_name)s %(uuid)s | Starting: %(msg)s"
params) % params)
result = f(self, *args, **kwargs) result = f(self, *args, **kwargs)
log_function(_("%(obj_name)s %(uuid)s | Completed: %(msg)s") % log_function("%(obj_name)s %(uuid)s | Completed: %(msg)s"
params) % params)
return result return result
return wrapper return wrapper
return decorator return decorator

View File

@ -17,7 +17,6 @@ import datetime as dt
import jsonschema import jsonschema
from rally.common.i18n import _, _LW
from rally.common import db from rally.common import db
from rally.common import logging from rally.common import logging
from rally import consts from rally import consts
@ -62,9 +61,9 @@ class Deployment(object):
def __getitem__(self, key): def __getitem__(self, key):
# TODO(astudenov): remove this in future releases # TODO(astudenov): remove this in future releases
if key == "admin" or key == "users": if key == "admin" or key == "users":
LOG.warning(_LW("deployment.%s is deprecated in Rally 0.9.0. " LOG.warning("deployment.%s is deprecated in Rally 0.9.0. "
"Use deployment.get_credentials_for('openstack')" "Use deployment.get_credentials_for('openstack')"
"['%s'] to get credentials.") % (key, key)) "['%s'] to get credentials." % (key, key))
return self.get_credentials_for("openstack")[key] return self.get_credentials_for("openstack")[key]
return self.deployment[key] return self.deployment[key]
@ -148,10 +147,9 @@ class Deployment(object):
return {"admin": None, "users": []} return {"admin": None, "users": []}
try: try:
creds = self.deployment["credentials"][namespace][0] creds = self.deployment["credentials"][namespace][0]
except (KeyError, IndexError) as e: except (KeyError, IndexError):
LOG.exception(e) raise exceptions.RallyException(
raise exceptions.RallyException(_( "No credentials found for %s" % namespace)
"No credentials found for %s") % namespace)
admin = creds["admin"] admin = creds["admin"]
credential_cls = credential.get(namespace) credential_cls = credential.get(namespace)

View File

@ -19,7 +19,6 @@ import datetime as dt
import uuid import uuid
from rally.common import db from rally.common import db
from rally.common.i18n import _LE
from rally.common import logging from rally.common import logging
from rally import consts from rally import consts
from rally import exceptions from rally import exceptions
@ -251,17 +250,16 @@ class Task(object):
if current_status in self.NOT_IMPLEMENTED_STAGES_FOR_ABORT: if current_status in self.NOT_IMPLEMENTED_STAGES_FOR_ABORT:
raise exceptions.RallyException( raise exceptions.RallyException(
_LE("Failed to abort task '%(uuid)s'. It doesn't implemented " "Failed to abort task '%(uuid)s'. It doesn't implemented "
"for '%(stages)s' stages. Current task status is " "for '%(stages)s' stages. Current task status is '%(status)s'."
"'%(status)s'.") % % {"uuid": self.task["uuid"], "status": current_status,
{"uuid": self.task["uuid"], "status": current_status,
"stages": ", ".join(self.NOT_IMPLEMENTED_STAGES_FOR_ABORT)}) "stages": ", ".join(self.NOT_IMPLEMENTED_STAGES_FOR_ABORT)})
elif current_status in [consts.TaskStatus.FINISHED, elif current_status in [consts.TaskStatus.FINISHED,
consts.TaskStatus.CRASHED, consts.TaskStatus.CRASHED,
consts.TaskStatus.ABORTED]: consts.TaskStatus.ABORTED]:
raise exceptions.RallyException( raise exceptions.RallyException(
_LE("Failed to abort task '%s', since it already " "Failed to abort task '%s', since it already finished."
"finished.") % self.task["uuid"]) % self.task["uuid"])
new_status = (consts.TaskStatus.SOFT_ABORTING new_status = (consts.TaskStatus.SOFT_ABORTING
if soft else consts.TaskStatus.ABORTING) if soft else consts.TaskStatus.ABORTING)

View File

@ -24,7 +24,6 @@ from oslo_utils import importutils
import six import six
import rally import rally
from rally.common.i18n import _
from rally.common import logging from rally.common import logging
LOG = logging.getLogger(__name__) LOG = logging.getLogger(__name__)
@ -97,7 +96,7 @@ def import_modules_by_entry_point():
def load_plugins(dir_or_file): def load_plugins(dir_or_file):
if os.path.isdir(dir_or_file): if os.path.isdir(dir_or_file):
directory = dir_or_file directory = dir_or_file
LOG.info(_("Loading plugins from directories %s/*") % LOG.info("Loading plugins from directories %s/*" %
directory.rstrip("/")) directory.rstrip("/"))
to_load = [] to_load = []
@ -113,25 +112,25 @@ def load_plugins(dir_or_file):
fp, pathname, descr = imp.find_module(plugin, [directory]) fp, pathname, descr = imp.find_module(plugin, [directory])
imp.load_module(plugin, fp, pathname, descr) imp.load_module(plugin, fp, pathname, descr)
fp.close() fp.close()
LOG.info(_("\t Loaded module with plugins: %s.py") % fullpath) LOG.info("\t Loaded module with plugins: %s.py" % fullpath)
except Exception as e: except Exception as e:
LOG.warning( msg = "\t Failed to load module with plugins %s.py" % fullpath
"\t Failed to load module with plugins %(path)s.py: %(e)s"
% {"path": fullpath, "e": e})
if logging.is_debug(): if logging.is_debug():
LOG.exception(e) LOG.exception(msg)
else:
LOG.warning("%(msg)s: %(e)s" % {"msg": msg, "e": e})
elif os.path.isfile(dir_or_file): elif os.path.isfile(dir_or_file):
plugin_file = dir_or_file plugin_file = dir_or_file
LOG.info(_("Loading plugins from file %s") % plugin_file) LOG.info("Loading plugins from file %s" % plugin_file)
if plugin_file not in sys.path: if plugin_file not in sys.path:
sys.path.append(plugin_file) sys.path.append(plugin_file)
try: try:
plugin_name = os.path.splitext(plugin_file.split("/")[-1])[0] plugin_name = os.path.splitext(plugin_file.split("/")[-1])[0]
imp.load_source(plugin_name, plugin_file) imp.load_source(plugin_name, plugin_file)
LOG.info(_("\t Loaded module with plugins: %s.py") % plugin_name) LOG.info("\t Loaded module with plugins: %s.py" % plugin_name)
except Exception as e: except Exception as e:
LOG.warning(_( msg = "\t Failed to load module with plugins %s" % plugin_file
"\t Failed to load module with plugins %(path)s: %(e)s")
% {"path": plugin_file, "e": e})
if logging.is_debug(): if logging.is_debug():
LOG.exception(e) LOG.exception(msg)
else:
LOG.warning("%(msg)s: %(e)s" % {"msg": msg, "e": e})

View File

@ -15,7 +15,6 @@
import sys import sys
from rally.common.i18n import _
from rally.common.plugin import discover from rally.common.plugin import discover
from rally.common.plugin import info from rally.common.plugin import info
from rally.common.plugin import meta from rally.common.plugin import meta
@ -40,16 +39,15 @@ def base():
""" """
def wrapper(cls): def wrapper(cls):
if not issubclass(cls, Plugin): if not issubclass(cls, Plugin):
raise exceptions.RallyException(_( raise exceptions.RallyException(
"Plugin's Base can be only a subclass of Plugin class.")) "Plugin's Base can be only a subclass of Plugin class.")
parent = cls._get_base() parent = cls._get_base()
if parent != Plugin: if parent != Plugin:
raise exceptions.RallyException(_( raise exceptions.RallyException(
"'%(plugin_cls)s' can not be marked as plugin base, since it " "'%(plugin_cls)s' can not be marked as plugin base, since it "
"inherits from '%(parent)s' which is also plugin base.") % { "inherits from '%(parent)s' which is also plugin base."
"plugin_cls": cls.__name__, % {"plugin_cls": cls.__name__, "parent": parent.__name__})
"parent": parent.__name__})
cls.base_ref = cls cls.base_ref = cls
cls._default_meta_init(True) cls._default_meta_init(True)

View File

@ -181,14 +181,14 @@ class SSH(object):
if session.recv_ready(): if session.recv_ready():
data = session.recv(4096) data = session.recv(4096)
LOG.debug("stdout: %r", data) LOG.debug("stdout: %r" % data)
if stdout is not None: if stdout is not None:
stdout.write(data.decode("utf8")) stdout.write(data.decode("utf8"))
continue continue
if session.recv_stderr_ready(): if session.recv_stderr_ready():
stderr_data = session.recv_stderr(4096) stderr_data = session.recv_stderr(4096)
LOG.debug("stderr: %r", stderr_data) LOG.debug("stderr: %r" % stderr_data)
if stderr is not None: if stderr is not None:
stderr.write(stderr_data.decode("utf8")) stderr.write(stderr_data.decode("utf8"))
continue continue
@ -203,7 +203,7 @@ class SSH(object):
writes = [] writes = []
continue continue
sent_bytes = session.send(data_to_send) sent_bytes = session.send(data_to_send)
LOG.debug("sent: %s", data_to_send[:sent_bytes]) LOG.debug("sent: %s" % data_to_send[:sent_bytes])
data_to_send = data_to_send[sent_bytes:] data_to_send = data_to_send[sent_bytes:]
if session.exit_status_ready(): if session.exit_status_ready():
@ -252,7 +252,7 @@ class SSH(object):
try: try:
return self.execute("uname") return self.execute("uname")
except (socket.error, exceptions.SSHError) as e: except (socket.error, exceptions.SSHError) as e:
LOG.debug("Ssh is still unavailable: %r", e) LOG.debug("Ssh is still unavailable: %r" % e)
time.sleep(interval) time.sleep(interval)
if time.time() > (start_time + timeout): if time.time() > (start_time + timeout):
raise exceptions.SSHTimeout("Timeout waiting for '%s'" % raise exceptions.SSHTimeout("Timeout waiting for '%s'" %

View File

@ -32,7 +32,6 @@ import uuid
from six import moves from six import moves
from rally.common.i18n import _, _LE
from rally.common import logging from rally.common import logging
from rally import exceptions from rally import exceptions
@ -47,7 +46,7 @@ class ImmutableMixin(object):
def __setattr__(self, key, value): def __setattr__(self, key, value):
if self._inited: if self._inited:
raise AttributeError(_("This object is immutable.")) raise AttributeError("This object is immutable.")
super(ImmutableMixin, self).__setattr__(key, value) super(ImmutableMixin, self).__setattr__(key, value)
@ -291,15 +290,15 @@ class RandomNameGeneratorMixin(object):
if len(task_id_part) < length: if len(task_id_part) < length:
LOG.debug("Task ID %(task_id)s cannot be included in a random " LOG.debug("Task ID %(task_id)s cannot be included in a random "
"name because it is too short. Format: %(format)s", "name because it is too short. Format: %(format)s"
{"task_id": task_id, % {"task_id": task_id,
"format": cls.RESOURCE_NAME_FORMAT}) "format": cls.RESOURCE_NAME_FORMAT})
elif any(char not in cls.RESOURCE_NAME_ALLOWED_CHARACTERS elif any(char not in cls.RESOURCE_NAME_ALLOWED_CHARACTERS
for char in task_id_part): for char in task_id_part):
LOG.debug("Task ID %(task_id)s cannot be included in a random " LOG.debug("Task ID %(task_id)s cannot be included in a random "
"name because it includes disallowed characters. " "name because it includes disallowed characters. "
"Allowed characters are: %(chars)s", "Allowed characters are: %(chars)s"
{"task_id": task_id, % {"task_id": task_id,
"chars": cls.RESOURCE_NAME_ALLOWED_CHARACTERS}) "chars": cls.RESOURCE_NAME_ALLOWED_CHARACTERS})
else: else:
return task_id_part return task_id_part
@ -775,10 +774,10 @@ class BackupHelper(object):
def backup(self, original_path): def backup(self, original_path):
if original_path in self._stored_data: if original_path in self._stored_data:
raise exceptions.RallyException( raise exceptions.RallyException(
_LE("Failed to back up %s since it was already stored.") % "Failed to back up %s since it was already stored."
original_path) % original_path)
backup_path = generate_random_path(self._tempdir) backup_path = generate_random_path(self._tempdir)
LOG.debug("Creating backup of %s in %s", (original_path, backup_path)) LOG.debug("Creating backup of %s in %s" % (original_path, backup_path))
try: try:
shutil.copytree(original_path, backup_path, symlinks=True) shutil.copytree(original_path, backup_path, symlinks=True)
except Exception: except Exception:
@ -816,5 +815,5 @@ class BackupHelper(object):
def __del__(self): def __del__(self):
for path in self._stored_data.values(): for path in self._stored_data.values():
if os.path.exists(path): if os.path.exists(path):
LOG.debug("Deleting %s", path) LOG.debug("Deleting %s" % path)
shutil.rmtree(path) shutil.rmtree(path)

View File

@ -256,7 +256,7 @@ class ValidatablePluginMixin(object):
result = traceback.format_exc() result = traceback.format_exc()
if result: if result:
LOG.debug("Result of validator '%s' is not successful for " LOG.debug("Result of validator '%s' is not successful for "
"plugin %s.", validator_cls.get_name(), name) "plugin %s." % (validator_cls.get_name(), name))
results.append(result) results.append(result)
if results: if results:

View File

@ -18,7 +18,6 @@ import abc
import jsonschema import jsonschema
import six import six
from rally.common.i18n import _, _LE
from rally.common import logging from rally.common import logging
from rally.common.plugin import plugin from rally.common.plugin import plugin
from rally import consts from rally import consts
@ -85,9 +84,9 @@ class Engine(plugin.Plugin):
engine_cls = Engine.get(name) engine_cls = Engine.get(name)
return engine_cls(deployment) return engine_cls(deployment)
except exceptions.PluginNotFound: except exceptions.PluginNotFound:
LOG.error(_LE("Deployment %(uuid)s: Deploy engine for %(name)s " LOG.error(
"does not exist.") % "Deployment %(uuid)s: Plugin %(name)s doesn't exist."
{"uuid": deployment["uuid"], "name": name}) % {"uuid": deployment["uuid"], "name": name})
deployment.update_status(consts.DeployStatus.DEPLOY_FAILED) deployment.update_status(consts.DeployStatus.DEPLOY_FAILED)
raise raise
@ -99,15 +98,15 @@ class Engine(plugin.Plugin):
def cleanup(self): def cleanup(self):
"""Cleanup OpenStack deployment.""" """Cleanup OpenStack deployment."""
@logging.log_deploy_wrapper(LOG.info, _("OpenStack cloud deployment.")) @logging.log_deploy_wrapper(LOG.info, "OpenStack cloud deployment.")
def make_deploy(self): def make_deploy(self):
self.deployment.set_started() self.deployment.set_started()
credentials = self.deploy() credentials = self.deploy()
self.deployment.set_completed() self.deployment.set_completed()
return credentials return credentials
@logging.log_deploy_wrapper(LOG.info, _("Destroy cloud and free " @logging.log_deploy_wrapper(LOG.info,
"allocated resources.")) "Destroy cloud and free allocated resources.")
def make_cleanup(self): def make_cleanup(self):
self.deployment.update_status(consts.DeployStatus.CLEANUP_STARTED) self.deployment.update_status(consts.DeployStatus.CLEANUP_STARTED)
self.cleanup() self.cleanup()
@ -121,9 +120,8 @@ class Engine(plugin.Plugin):
exc_info = None exc_info = None
if not issubclass(exc_type, exceptions.InvalidArgumentsException): if not issubclass(exc_type, exceptions.InvalidArgumentsException):
exc_info = (exc_type, exc_value, exc_traceback) exc_info = (exc_type, exc_value, exc_traceback)
LOG.error(_LE("Deployment %(uuid)s: Error has occurred into " LOG.error("Deployment %s: Error has occurred in context "
"context of the deployment"), "of the deployment" % self.deployment["uuid"],
{"uuid": self.deployment["uuid"]},
exc_info=exc_info) exc_info=exc_info)
status = self.deployment["status"] status = self.deployment["status"]
if status in (consts.DeployStatus.DEPLOY_INIT, if status in (consts.DeployStatus.DEPLOY_INIT,

View File

@ -216,9 +216,9 @@ class ContextManager(object):
LOG.info("%(msg)s finished in %(duration)s" LOG.info("%(msg)s finished in %(duration)s"
% {"msg": msg, "duration": timer.duration(fmt=True)}) % {"msg": msg, "duration": timer.duration(fmt=True)})
except Exception: except Exception:
LOG.exception("%(msg)s failed after %(duration)s" LOG.exception(
% {"msg": msg, "%(msg)s failed after %(duration)s"
"duration": timer.duration(fmt=True)}) % {"msg": msg, "duration": timer.duration(fmt=True)})
def __enter__(self): def __enter__(self):
try: try:

View File

@ -22,7 +22,6 @@ import traceback
import jsonschema import jsonschema
from oslo_config import cfg from oslo_config import cfg
from rally.common.i18n import _
from rally.common import logging from rally.common import logging
from rally.common import objects from rally.common import objects
from rally.common import utils from rally.common import utils
@ -252,7 +251,7 @@ class TaskEngine(object):
str(e), str(e),
json.dumps(traceback.format_exc())) json.dumps(traceback.format_exc()))
if logging.is_debug(): if logging.is_debug():
LOG.exception(e) LOG.exception("Invalid Task")
raise exceptions.InvalidTaskException(str(e)) raise exceptions.InvalidTaskException(str(e))
self.task = task self.task = task
@ -339,14 +338,14 @@ class TaskEngine(object):
raise exceptions.InvalidTaskConfig(**kw) raise exceptions.InvalidTaskConfig(**kw)
@logging.log_task_wrapper(LOG.info, _("Task validation of syntax.")) @logging.log_task_wrapper(LOG.info, "Task validation of syntax.")
def _validate_config_syntax(self, config): def _validate_config_syntax(self, config):
for subtask in config.subtasks: for subtask in config.subtasks:
for workload in subtask["workloads"]: for workload in subtask["workloads"]:
self._validate_workload(workload, vtype="syntax") self._validate_workload(workload, vtype="syntax")
@logging.log_task_wrapper(LOG.info, _("Task validation of required " @logging.log_task_wrapper(LOG.info,
"platforms.")) "Task validation of required platforms.")
def _validate_config_platforms(self, config): def _validate_config_platforms(self, config):
# FIXME(andreykurilin): prepare the similar context object to others # FIXME(andreykurilin): prepare the similar context object to others
credentials = self.deployment.get_all_credentials() credentials = self.deployment.get_all_credentials()
@ -358,7 +357,7 @@ class TaskEngine(object):
self._validate_workload( self._validate_workload(
workload, vcontext=ctx, vtype="platform") workload, vcontext=ctx, vtype="platform")
@logging.log_task_wrapper(LOG.info, _("Task validation of semantic.")) @logging.log_task_wrapper(LOG.info, "Task validation of semantic.")
def _validate_config_semantic(self, config): def _validate_config_semantic(self, config):
self.deployment.verify_connections() self.deployment.verify_connections()
validation_ctx = self.deployment.get_validation_context() validation_ctx = self.deployment.get_validation_context()
@ -369,7 +368,7 @@ class TaskEngine(object):
self._validate_workload( self._validate_workload(
workload, vcontext=ctx_obj, vtype="semantic") workload, vcontext=ctx_obj, vtype="semantic")
@logging.log_task_wrapper(LOG.info, _("Task validation.")) @logging.log_task_wrapper(LOG.info, "Task validation.")
def validate(self, only_syntax=False): def validate(self, only_syntax=False):
"""Perform full task configuration validation. """Perform full task configuration validation.
@ -388,7 +387,7 @@ class TaskEngine(object):
self.task.set_failed(type(e).__name__, str(e), exception_info) self.task.set_failed(type(e).__name__, str(e), exception_info)
if (logging.is_debug() and if (logging.is_debug() and
not isinstance(e, exceptions.InvalidTaskConfig)): not isinstance(e, exceptions.InvalidTaskConfig)):
LOG.exception(e) LOG.exception("Invalid Task")
raise exceptions.InvalidTaskException(str(e)) raise exceptions.InvalidTaskException(str(e))
def _prepare_context(self, ctx, scenario_name, owner_id): def _prepare_context(self, ctx, scenario_name, owner_id):
@ -409,7 +408,7 @@ class TaskEngine(object):
} }
return context_obj return context_obj
@logging.log_task_wrapper(LOG.info, _("Running task.")) @logging.log_task_wrapper(LOG.info, "Running task.")
def run(self): def run(self):
"""Run the benchmark according to the test configuration. """Run the benchmark according to the test configuration.
@ -443,11 +442,10 @@ class TaskEngine(object):
except TaskAborted: except TaskAborted:
subtask_obj.update_status(consts.SubtaskStatus.ABORTED) subtask_obj.update_status(consts.SubtaskStatus.ABORTED)
raise raise
except Exception as e: except Exception:
subtask_obj.update_status(consts.SubtaskStatus.CRASHED) subtask_obj.update_status(consts.SubtaskStatus.CRASHED)
# TODO(astudenov): save error to DB # TODO(astudenov): save error to DB
LOG.debug(traceback.format_exc()) LOG.exception("Unexpected exception during the subtask execution")
LOG.exception(e)
# NOTE(astudenov): crash task after exception in subtask # NOTE(astudenov): crash task after exception in subtask
self.task.update_status(consts.TaskStatus.CRASHED) self.task.update_status(consts.TaskStatus.CRASHED)
@ -472,9 +470,9 @@ class TaskEngine(object):
workload_cfg = objects.Workload.to_task(workload) workload_cfg = objects.Workload.to_task(workload)
LOG.info("Running workload: \n" LOG.info("Running workload: \n"
" position = %(position)s\n" " position = %(position)s\n"
" config = %(cfg)s", {"position": workload["position"], " config = %(cfg)s"
"cfg": json.dumps(workload_cfg, % {"position": workload["position"],
indent=3)}) "cfg": json.dumps(workload_cfg, indent=3)})
runner_cls = runner.ScenarioRunner.get(workload["runner"]["type"]) runner_cls = runner.ScenarioRunner.get(workload["runner"]["type"])
runner_obj = runner_cls(self.task, workload["runner"]) runner_obj = runner_cls(self.task, workload["runner"])
@ -486,9 +484,8 @@ class TaskEngine(object):
with context.ContextManager(context_obj): with context.ContextManager(context_obj):
runner_obj.run(workload["name"], context_obj, runner_obj.run(workload["name"], context_obj,
workload["args"]) workload["args"])
except Exception as e: except Exception:
LOG.debug(traceback.format_exc()) LOG.exception("Unexpected exception during the workload execution")
LOG.exception(e)
# TODO(astudenov): save error to DB # TODO(astudenov): save error to DB
@ -696,7 +693,7 @@ class TaskConfig(object):
then "Task config is invalid: " gets prepended to the message twice then "Task config is invalid: " gets prepended to the message twice
""" """
if config is None: if config is None:
raise Exception(_("Input task is empty")) raise Exception("Input task is empty")
self.version = self._get_version(config) self.version = self._get_version(config)
self._validate_version() self._validate_version()
@ -790,8 +787,8 @@ class TaskConfig(object):
def _validate_version(self): def _validate_version(self):
if self.version not in self.CONFIG_SCHEMAS: if self.version not in self.CONFIG_SCHEMAS:
allowed = ", ".join([str(k) for k in self.CONFIG_SCHEMAS]) allowed = ", ".join([str(k) for k in self.CONFIG_SCHEMAS])
msg = (_("Task configuration version {0} is not supported. " msg = ("Task configuration version %s is not supported. "
"Supported versions: {1}")).format(self.version, allowed) "Supported versions: %s") % (self.version, allowed)
raise exceptions.InvalidTaskException(msg) raise exceptions.InvalidTaskException(msg)
def _validate_json(self, config): def _validate_json(self, config):

View File

@ -19,7 +19,6 @@ import threading
import six import six
from rally.common.i18n import _, _LE
from rally.common import logging from rally.common import logging
from rally.common.plugin import plugin from rally.common.plugin import plugin
from rally.common import utils as rutils from rally.common import utils as rutils
@ -95,7 +94,7 @@ class HookExecutor(object):
for trigger_obj in self.triggers[event_type]: for trigger_obj in self.triggers[event_type]:
started = trigger_obj.on_event(event_type, value) started = trigger_obj.on_event(event_type, value)
if started: if started:
LOG.info(_("Hook %s is trigged for Task %s by %s=%s") LOG.info("Hook %s is trigged for Task %s by %s=%s"
% (trigger_obj.hook_cls.__name__, self.task["uuid"], % (trigger_obj.hook_cls.__name__, self.task["uuid"],
event_type, value)) event_type, value))
@ -177,8 +176,7 @@ class HookAction(plugin.Plugin, validation.ValidatablePluginMixin):
with rutils.Timer() as timer: with rutils.Timer() as timer:
self.run() self.run()
except Exception as exc: except Exception as exc:
LOG.error(_LE("Hook %s failed during run."), self.get_name()) LOG.exception("Hook %s failed during run." % self.get_name())
LOG.exception(exc)
self.set_error(*utils.format_exc(exc)) self.set_error(*utils.format_exc(exc))
self._started_at = timer.timestamp() self._started_at = timer.timestamp()
@ -229,7 +227,7 @@ class HookTrigger(plugin.Plugin, validation.ValidatablePluginMixin):
def on_event(self, event_type, value=None): def on_event(self, event_type, value=None):
"""Launch hook on specified event.""" """Launch hook on specified event."""
LOG.info(_("Hook action %s is triggered for Task %s by %s=%s") LOG.info("Hook action %s is triggered for Task %s by %s=%s"
% (self.hook_cls.get_name(), self.task["uuid"], % (self.hook_cls.get_name(), self.task["uuid"],
event_type, value)) event_type, value))
action_cfg = list(self.hook_cfg["action"].values())[0] action_cfg = list(self.hook_cfg["action"].values())[0]

View File

@ -73,7 +73,7 @@ def _run_scenario_once(cls, method_name, context_obj, scenario_kwargs,
except Exception as e: except Exception as e:
error = utils.format_exc(e) error = utils.format_exc(e)
if logging.is_debug(): if logging.is_debug():
LOG.exception(e) LOG.exception("Iteration %s raised Exception" % iteration)
finally: finally:
status = "Error %s: %s" % tuple(error[0:2]) if error else "OK" status = "Error %s: %s" % tuple(error[0:2]) if error else "OK"
LOG.info("Task %(task)s | ITER: %(iteration)s END: %(status)s" % LOG.info("Task %(task)s | ITER: %(iteration)s END: %(status)s" %
@ -101,7 +101,7 @@ def _log_worker_info(**info):
""" """
info_message = "\n\t".join(["%s: %s" % (k, v) info_message = "\n\t".join(["%s: %s" % (k, v)
for k, v in info.items()]) for k, v in info.items()])
LOG.debug("Starting a worker.\n\t%s", info_message) LOG.debug("Starting a worker.\n\t%s" % info_message)
@validation.add_default("jsonschema") @validation.add_default("jsonschema")
@ -352,7 +352,7 @@ class ScenarioRunner(plugin.Plugin, validation.ValidatablePluginMixin):
info_message = "\n\t".join(["%s: %s" % (k, v) info_message = "\n\t".join(["%s: %s" % (k, v)
for k, v in info.items()]) for k, v in info.items()])
LOG.debug("Starting the %(name)s runner (task UUID: %(task)s)." LOG.debug("Starting the %(name)s runner (task UUID: %(task)s)."
"\n\t%(info)s", "\n\t%(info)s"
{"name": self._meta_get("name"), % {"name": self._meta_get("name"),
"task": self.task["uuid"], "task": self.task["uuid"],
"info": info_message}) "info": info_message})

View File

@ -16,7 +16,6 @@
import copy import copy
import random import random
from rally.common.i18n import _
from rally.common import logging from rally.common import logging
from rally.common.objects import task # noqa from rally.common.objects import task # noqa
from rally.common.plugin import plugin from rally.common.plugin import plugin
@ -53,8 +52,8 @@ def configure(name, platform="default", namespace=None, context=None):
def wrapper(cls): def wrapper(cls):
# TODO(boris-42): Drop this check as soon as we refactor rally report # TODO(boris-42): Drop this check as soon as we refactor rally report
if "." not in name.strip("."): if "." not in name.strip("."):
msg = (_("Scenario name must include a dot: '%s'") % name) raise exceptions.RallyException(
raise exceptions.RallyException(msg) "Scenario name must include a dot: '%s'" % name)
for c in context: for c in context:
if "@" not in c: if "@" not in c:

View File

@ -23,7 +23,6 @@ import abc
import six import six
from rally.common.i18n import _
from rally.common.plugin import plugin from rally.common.plugin import plugin
from rally.common import validation from rally.common import validation
@ -78,28 +77,27 @@ class SLAChecker(object):
self_config = self.config.get("sla", {}) self_config = self.config.get("sla", {})
other_config = other.config.get("sla", {}) other_config = other.config.get("sla", {})
if self_config != other_config: if self_config != other_config:
message = _( raise TypeError(
"Error merging SLACheckers with configs %s, %s. " "Error merging SLACheckers with configs %s, %s. "
"Only SLACheckers with the same config could be merged." "Only SLACheckers with the same config could be merged."
) % (self_config, other_config) % (self_config, other_config))
raise TypeError(message)
def results(self): def results(self):
results = [sla.result() for sla in self.sla_criteria] results = [sla.result() for sla in self.sla_criteria]
if self.aborted_on_sla: if self.aborted_on_sla:
results.append(_format_result( results.append(_format_result(
"aborted_on_sla", False, "aborted_on_sla", False,
_("Task was aborted due to SLA failure(s)."))) "Task was aborted due to SLA failure(s)."))
if self.aborted_manually: if self.aborted_manually:
results.append(_format_result( results.append(_format_result(
"aborted_manually", False, "aborted_manually", False,
_("Task was aborted due to abort signal."))) "Task was aborted due to abort signal."))
if self.unexpected_failure: if self.unexpected_failure:
results.append(_format_result( results.append(_format_result(
"something_went_wrong", False, "something_went_wrong", False,
_("Unexpected error: %s") % self.unexpected_failure)) "Unexpected error: %s" % self.unexpected_failure))
return results return results
@ -183,8 +181,6 @@ class SLA(plugin.Plugin, validation.ValidatablePluginMixin):
def validate_type(self, other): def validate_type(self, other):
if type(self) != type(other): if type(self) != type(other):
message = _( raise TypeError(
"Error merging SLAs of types %s, %s. " "Error merging SLAs of types %s, %s. Only SLAs of the same "
"Only SLAs of the same type could be merged." "type could be merged." % (type(self), type(other)))
) % (type(self), type(other))
raise TypeError(message)

View File

@ -22,7 +22,6 @@ import jsonschema
from novaclient import exceptions as nova_exc from novaclient import exceptions as nova_exc
import six import six
from rally.common.i18n import _
from rally.common import logging from rally.common import logging
from rally import consts from rally import consts
from rally import exceptions from rally import exceptions
@ -227,8 +226,8 @@ def wait_for_status(resource, ready_statuses, failure_statuses=None,
delta = current_time - latest_status_update delta = current_time - latest_status_update
LOG.debug( LOG.debug(
"Waiting for resource %(resource)s. Status changed: " "Waiting for resource %(resource)s. Status changed: "
"%(latest)s => %(current)s in %(delta)s", "%(latest)s => %(current)s in %(delta)s"
{"resource": resource_repr, "latest": latest_status, % {"resource": resource_repr, "latest": latest_status,
"current": status, "delta": delta}) "current": status, "delta": delta})
latest_status = status latest_status = status
@ -300,8 +299,8 @@ def check_service_status(client, service_name):
if service.status == "enabled" and service.state == "up": if service.status == "enabled" and service.state == "up":
return True return True
except nova_exc.NotFound: except nova_exc.NotFound:
LOG.warning(_("Unable to retrieve a list of available services from " LOG.warning("Unable to retrieve a list of available services from "
"nova. Pre-Grizzly OpenStack deployment?")) "nova. Pre-Grizzly OpenStack deployment?")
return False return False
return False return False

View File

@ -18,6 +18,8 @@ import functools
from rally.common import logging from rally.common import logging
from rally.common import validation from rally.common import validation
LOG = logging.getLogger(__name__) LOG = logging.getLogger(__name__)
# TODO(astudenov): remove after deprecating all old validators # TODO(astudenov): remove after deprecating all old validators
@ -127,8 +129,8 @@ def deprecated_validator(name, old_validator_name, rally_version):
"Plugin '%s' uses validator 'rally.task.validation.%s' which " "Plugin '%s' uses validator 'rally.task.validation.%s' which "
"is deprecated in favor of '%s' (it should be used " "is deprecated in favor of '%s' (it should be used "
"via new decorator 'rally.common.validation.add') in " "via new decorator 'rally.common.validation.add') in "
"Rally v%s.", "Rally v%s."
plugin_name, old_validator_name, name, rally_version) % (plugin_name, old_validator_name, name, rally_version))
plugin._meta_setdefault("validators", []) plugin._meta_setdefault("validators", [])
plugin._meta_get("validators").append((name, args, kwargs,)) plugin._meta_get("validators").append((name, args, kwargs,))
return plugin return plugin

View File

@ -31,7 +31,6 @@ from alembic import script as alembic_script
from oslo_config import cfg from oslo_config import cfg
import rally.common.db.sqlalchemy.api as s_api import rally.common.db.sqlalchemy.api as s_api
from rally.common.i18n import _LE
from rally.common import logging from rally.common import logging
LOG = logging.getLogger(__name__) LOG = logging.getLogger(__name__)
@ -151,6 +150,6 @@ class BaseWalkMigrationMixin(object):
if check: if check:
check(engine, data) check(engine, data)
except Exception: except Exception:
LOG.error(_LE("Failed to migrate to version {ver} on engine {eng}") LOG.error("Failed to migrate to version %(ver)s on engine %(eng)s"
.format(ver=version, eng=engine)) % {"ver": version, "eng": engine})
raise raise

View File

@ -0,0 +1,18 @@
# All Rights Reserved.
#
# Licensed under the Apache License, Version 2.0 (the "License"); you may
# not use this file except in compliance with the License. You may obtain
# a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
# License for the specific language governing permissions and limitations
# under the License.
# TODO(boris-42): This is used just to check that i18n.py can be imported
# it should be removed as soon as we remove i18n module
from rally.common import i18n # noqa

View File

@ -17,7 +17,6 @@ import logging # noqa
import mock import mock
from rally.common.i18n import _
from rally.common import logging as rally_logging from rally.common import logging as rally_logging
from tests.unit import test from tests.unit import test
@ -228,8 +227,8 @@ class LogTestCase(test.TestCase):
self.assertEqual(4, t.some_method(2, 2)) self.assertEqual(4, t.some_method(2, 2))
params = {"msg": msg % {"a": 10, "b": 20}, "uuid": t.task["uuid"]} params = {"msg": msg % {"a": 10, "b": 20}, "uuid": t.task["uuid"]}
expected = [ expected = [
mock.call(_("Task %(uuid)s | Starting: %(msg)s") % params), mock.call("Task %(uuid)s | Starting: %(msg)s" % params),
mock.call(_("Task %(uuid)s | Completed: %(msg)s") % params) mock.call("Task %(uuid)s | Completed: %(msg)s" % params)
] ]
self.assertEqual(expected, mock_log.mock_calls) self.assertEqual(expected, mock_log.mock_calls)

View File

@ -136,12 +136,7 @@ class ValidationUtilsTestCase(test.TestCase):
self.Plugin = my_deprecated_validator("foo", bar="baz")(self.Plugin) self.Plugin = my_deprecated_validator("foo", bar="baz")(self.Plugin)
self.assertEqual([("new_validator", ("foo",), {"bar": "baz"})], self.assertEqual([("new_validator", ("foo",), {"bar": "baz"})],
self.Plugin._meta_get("validators")) self.Plugin._meta_get("validators"))
mock_log_warning.assert_called_once_with( mock_log_warning.assert_called_once_with(mock.ANY)
"Plugin '%s' uses validator 'rally.task.validation.%s' which is "
"deprecated in favor of '%s' (it should be used via new decorator "
"'rally.common.validation.add') in Rally v%s.",
self.Plugin.get_name(), "deprecated_validator", "new_validator",
"0.10.0")
def _unwrap_validator(self, validator, *args, **kwargs): def _unwrap_validator(self, validator, *args, **kwargs):
name = self.id() name = self.id()