Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Add worker config check_unfulfilled_deps. #2189

Merged
merged 5 commits into from
Jul 26, 2017
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
11 changes: 11 additions & 0 deletions doc/configuration.rst
Original file line number Diff line number Diff line change
Expand Up @@ -255,6 +255,17 @@ send-failure-email
handled by the scheduler.
Defaults to true.

check_unfulfilled_deps
If true, the worker checks for completeness of dependencies before running a
task. In case unfulfilled dependencies are detected, an exception is raised
and the task will not run. This mechanism is useful to detect situations
where tasks do not create their outputs properly, or when targets were
removed after the dependency tree was built. It is recommended to disable
this feature only when the completeness checks are known to be bottlenecks,
e.g. when the ``exists()`` calls of the dependencies' outputs are
resource-intensive.
Defaults to true.


[elasticsearch]
---------------
Expand Down
11 changes: 8 additions & 3 deletions luigi/worker.py
Original file line number Diff line number Diff line change
Expand Up @@ -111,7 +111,7 @@ class TaskProcess(multiprocessing.Process):
Mainly for convenience since this is run in a separate process. """

def __init__(self, task, worker_id, result_queue, status_reporter,
use_multiprocessing=False, worker_timeout=0):
use_multiprocessing=False, worker_timeout=0, check_unfulfilled_deps=True):
super(TaskProcess, self).__init__()
self.task = task
self.worker_id = worker_id
Expand All @@ -121,6 +121,7 @@ def __init__(self, task, worker_id, result_queue, status_reporter,
worker_timeout = task.worker_timeout
self.timeout_time = time.time() + worker_timeout if worker_timeout else None
self.use_multiprocessing = use_multiprocessing or self.timeout_time is not None
self.check_unfulfilled_deps = check_unfulfilled_deps

def _run_get_new_deps(self):
self.task.set_tracking_url = self.status_reporter.update_tracking_url
Expand Down Expand Up @@ -168,7 +169,7 @@ def run(self):
# don't care about unfulfilled dependencies, because we are just
# checking completeness of self.task so outputs of dependencies are
# irrelevant.
if not _is_external(self.task):
if self.check_unfulfilled_deps and not _is_external(self.task):
missing = [dep.task_id for dep in self.task.deps() if not dep.complete()]
if missing:
deps = 'dependency' if len(missing) == 1 else 'dependencies'
Expand Down Expand Up @@ -367,6 +368,9 @@ class worker(Config):
no_install_shutdown_handler = BoolParameter(default=False,
description='If true, the SIGUSR1 shutdown handler will'
'NOT be install on the worker')
check_unfulfilled_deps = BoolParameter(default=True,
description='If true, check for completeness of '
'dependencies before running a task')


class KeepAliveThread(threading.Thread):
Expand Down Expand Up @@ -910,7 +914,8 @@ def _create_task_process(self, task):
return TaskProcess(
task, self._id, self._task_result_queue, reporter,
use_multiprocessing=bool(self.worker_processes > 1),
worker_timeout=self._config.timeout
worker_timeout=self._config.timeout,
check_unfulfilled_deps=self._config.check_unfulfilled_deps,
)

def _purge_children(self):
Expand Down
50 changes: 50 additions & 0 deletions test/worker_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -385,6 +385,56 @@ def requires(self):
self.assertTrue(a.complete())
self.assertTrue(b.complete())

def test_check_unfulfilled_deps_config(self):
class A(Task):

i = luigi.IntParameter()

def __init__(self, *args, **kwargs):
super(A, self).__init__(*args, **kwargs)
self.complete_count = 0
self.has_run = False

def complete(self):
self.complete_count += 1
return self.has_run

def run(self):
self.has_run = True

class B(A):

def requires(self):
return A(i=self.i)

# test the enabled features
with Worker(scheduler=self.sch, worker_id='1') as w:
w._config.check_unfulfilled_deps = True
a1 = A(i=1)
b1 = B(i=1)
self.assertTrue(w.add(b1))
self.assertEqual(a1.complete_count, 1)
self.assertEqual(b1.complete_count, 1)
w.run()
self.assertTrue(a1.complete())
self.assertTrue(b1.complete())
self.assertEqual(a1.complete_count, 3)
self.assertEqual(b1.complete_count, 2)

# test the disabled features
with Worker(scheduler=self.sch, worker_id='2') as w:
w._config.check_unfulfilled_deps = False
a2 = A(i=2)
b2 = B(i=2)
self.assertTrue(w.add(b2))
self.assertEqual(a2.complete_count, 1)
self.assertEqual(b2.complete_count, 1)
w.run()
self.assertTrue(a2.complete())
self.assertTrue(b2.complete())
self.assertEqual(a2.complete_count, 2)
self.assertEqual(b2.complete_count, 2)

def test_gets_missed_work(self):
class A(Task):
done = False
Expand Down