Skip to content

Commit

Permalink
luigid logging: Log less and change logger name (#1636)
Browse files Browse the repository at this point in the history
I think it makes sense to log less, because currently I think INFO level
should not be verbose. In particular "No workers connected" message is
verbose and going to be said many more times than it will be actually
removed (when used with assistants).

Further, I think now is a sane time to change the name of the logger to
the "recommended" `getLogger(__name__)` style. Because PR
#1633 was just merged and it's less likely that people have
relied on the logger name prior to that.
  • Loading branch information
Tarrasch committed Apr 11, 2016
1 parent a8e64fe commit 0318565
Showing 1 changed file with 5 additions and 5 deletions.
10 changes: 5 additions & 5 deletions luigi/scheduler.py
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@
from luigi.task_status import DISABLED, DONE, FAILED, PENDING, RUNNING, SUSPENDED, UNKNOWN
from luigi.task import Config

logger = logging.getLogger("luigi.server")
logger = logging.getLogger(__name__)


class Scheduler(object):
Expand Down Expand Up @@ -414,8 +414,8 @@ def update_status(self, task, config):
# Mark tasks with no remaining active stakeholders for deletion
if not task.stakeholders:
if task.remove is None:
logger.info("Task %r has stakeholders %r but none remain connected -> might remove "
"task in %s seconds", task.id, task.stakeholders, config.remove_delay)
logger.debug("Task %r has stakeholders %r but none remain connected -> might remove "
"task in %s seconds", task.id, task.stakeholders, config.remove_delay)
task.remove = time.time() + config.remove_delay

# Re-enable task after the disable time expires
Expand Down Expand Up @@ -531,7 +531,7 @@ def _prune_workers(self):
remove_workers = []
for worker in self._state.get_active_workers():
if worker.prune(self._config):
logger.info("Worker %s timed out (no contact for >=%ss)", worker, self._config.worker_disconnect_delay)
logger.debug("Worker %s timed out (no contact for >=%ss)", worker, self._config.worker_disconnect_delay)
remove_workers.append(worker.id)

self._state.inactivate_workers(remove_workers)
Expand Down Expand Up @@ -914,7 +914,7 @@ def dep_func(t):

task = self._state.get_task(task_id)
if task is None or not task.family:
logger.warn('Missing task for id [%s]', task_id)
logger.debug('Missing task for id [%s]', task_id)

# NOTE : If a dependency is missing from self._state there is no way to deduce the
# task family and parameters.
Expand Down

0 comments on commit 0318565

Please sign in to comment.