From 23c83e0f8e617a25cca31a88c6af9ab2f53e4439 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Thu, 12 Feb 2015 23:08:15 -0800 Subject: [PATCH 001/246] Improve multilock class and its associated unit test In the multilock unit test check that all values were acquired (as expected). Also use a class constant for the number of threads to spin up when testing timing overlaps and use the module constant for the duration to wait (to try to trigger overlaps) and use a new helper utility function to make it more obvious what is happening in the multilock release method. Also swaps out the usage of time.time in the lock utils unit test to attempt to use monotonic time (when it can be used) so that false positives are unable to happen (using time.time it is possible for time to go backwards and cause a non-existent overlap to appear). Change-Id: Ifb3967e1c1da41e1b7ac0793fc6a99f84de2a907 --- taskflow/tests/unit/test_utils.py | 36 +++++++++ taskflow/tests/unit/test_utils_lock_utils.py | 83 +++++++++++++------- taskflow/utils/lock_utils.py | 12 ++- taskflow/utils/misc.py | 15 ++++ 4 files changed, 111 insertions(+), 35 deletions(-) diff --git a/taskflow/tests/unit/test_utils.py b/taskflow/tests/unit/test_utils.py index ba71cca2..51467529 100644 --- a/taskflow/tests/unit/test_utils.py +++ b/taskflow/tests/unit/test_utils.py @@ -19,6 +19,8 @@ import inspect import random import time +import six + from taskflow import test from taskflow.utils import misc from taskflow.utils import threading_utils @@ -190,6 +192,40 @@ class TestSequenceMinus(test.TestCase): self.assertEqual(result, [2, 1]) +class TestCountdownIter(test.TestCase): + def test_expected_count(self): + upper = 100 + it = misc.countdown_iter(upper) + items = [] + for i in it: + self.assertEqual(upper, i) + upper -= 1 + items.append(i) + self.assertEqual(0, upper) + self.assertEqual(100, len(items)) + + def test_no_count(self): + it = misc.countdown_iter(0) + self.assertEqual(0, len(list(it))) + it = misc.countdown_iter(-1) + self.assertEqual(0, len(list(it))) + + def test_expected_count_custom_decr(self): + upper = 100 + it = misc.countdown_iter(upper, decr=2) + items = [] + for i in it: + self.assertEqual(upper, i) + upper -= 2 + items.append(i) + self.assertEqual(0, upper) + self.assertEqual(50, len(items)) + + def test_invalid_decr(self): + it = misc.countdown_iter(10, -1) + self.assertRaises(ValueError, six.next, it) + + class TestClamping(test.TestCase): def test_simple_clamp(self): result = misc.clamp(1.0, 2.0, 3.0) diff --git a/taskflow/tests/unit/test_utils_lock_utils.py b/taskflow/tests/unit/test_utils_lock_utils.py index 06bef1ee..bb334a2d 100644 --- a/taskflow/tests/unit/test_utils_lock_utils.py +++ b/taskflow/tests/unit/test_utils_lock_utils.py @@ -24,9 +24,10 @@ from taskflow import test from taskflow.test import mock from taskflow.tests import utils as test_utils from taskflow.utils import lock_utils +from taskflow.utils import misc from taskflow.utils import threading_utils -# NOTE(harlowja): Sleep a little so time.time() can not be the same (which will +# NOTE(harlowja): Sleep a little so now() can not be the same (which will # cause false positives when our overlap detection code runs). If there are # real overlaps then they will still exist. NAPPY_TIME = 0.05 @@ -34,6 +35,10 @@ NAPPY_TIME = 0.05 # We will spend this amount of time doing some "fake" work. WORK_TIMES = [(0.01 + x / 100.0) for x in range(0, 5)] +# Try to use a more accurate time for overlap detection (one that should +# never go backwards and cause false positives during overlap detection...). +now = misc.find_monotonic(allow_time_time=True) + def _find_overlaps(times, start, end): overlaps = 0 @@ -52,17 +57,17 @@ def _spawn_variation(readers, writers, max_workers=None): # TODO(harlowja): sometime in the future use a monotonic clock here # to avoid problems that can be caused by ntpd resyncing the clock # while we are actively running. - enter_time = time.time() + enter_time = now() time.sleep(WORK_TIMES[ident % len(WORK_TIMES)]) - exit_time = time.time() + exit_time = now() start_stops.append((lock.READER, enter_time, exit_time)) time.sleep(NAPPY_TIME) def write_func(ident): with lock.write_lock(): - enter_time = time.time() + enter_time = now() time.sleep(WORK_TIMES[ident % len(WORK_TIMES)]) - exit_time = time.time() + exit_time = now() start_stops.append((lock.WRITER, enter_time, exit_time)) time.sleep(NAPPY_TIME) @@ -89,6 +94,8 @@ def _spawn_variation(readers, writers, max_workers=None): class MultilockTest(test.TestCase): + THREAD_COUNT = 20 + def test_empty_error(self): self.assertRaises(ValueError, lock_utils.MultiLock, []) @@ -179,56 +186,63 @@ class MultilockTest(test.TestCase): def test_acquired_pass(self): activated = collections.deque() + acquires = collections.deque() lock1 = threading.Lock() lock2 = threading.Lock() n_lock = lock_utils.MultiLock((lock1, lock2)) def critical_section(): - start = time.time() - time.sleep(0.05) - end = time.time() + start = now() + time.sleep(NAPPY_TIME) + end = now() activated.append((start, end)) def run(): - with n_lock: + with n_lock as gotten: + acquires.append(gotten) critical_section() threads = [] - for _i in range(0, 20): + for _i in range(0, self.THREAD_COUNT): t = threading_utils.daemon_thread(run) threads.append(t) t.start() while threads: t = threads.pop() t.join() + + self.assertEqual(self.THREAD_COUNT, len(acquires)) + self.assertTrue(all(acquires)) for (start, end) in activated: self.assertEqual(1, _find_overlaps(activated, start, end)) - self.assertFalse(lock1.locked()) self.assertFalse(lock2.locked()) def test_acquired_fail(self): activated = collections.deque() + acquires = collections.deque() lock1 = threading.Lock() lock2 = threading.Lock() n_lock = lock_utils.MultiLock((lock1, lock2)) def run(): - with n_lock: - start = time.time() - time.sleep(0.05) - end = time.time() + with n_lock as gotten: + acquires.append(gotten) + start = now() + time.sleep(NAPPY_TIME) + end = now() activated.append((start, end)) def run_fail(): try: - with n_lock: + with n_lock as gotten: + acquires.append(gotten) raise RuntimeError() except RuntimeError: pass threads = [] - for i in range(0, 20): + for i in range(0, self.THREAD_COUNT): if i % 2 == 1: target = run_fail else: @@ -240,6 +254,8 @@ class MultilockTest(test.TestCase): t = threads.pop() t.join() + self.assertEqual(self.THREAD_COUNT, len(acquires)) + self.assertTrue(all(acquires)) for (start, end) in activated: self.assertEqual(1, _find_overlaps(activated, start, end)) self.assertFalse(lock1.locked()) @@ -247,44 +263,52 @@ class MultilockTest(test.TestCase): def test_double_acquire_single(self): activated = collections.deque() + acquires = [] def run(): - start = time.time() - time.sleep(0.05) - end = time.time() + start = now() + time.sleep(NAPPY_TIME) + end = now() activated.append((start, end)) lock1 = threading.RLock() lock2 = threading.RLock() n_lock = lock_utils.MultiLock((lock1, lock2)) - with n_lock: + with n_lock as gotten: + acquires.append(gotten) run() - with n_lock: + with n_lock as gotten: + acquires.append(gotten) run() run() + self.assertTrue(all(acquires)) + self.assertEqual(2, len(acquires)) for (start, end) in activated: self.assertEqual(1, _find_overlaps(activated, start, end)) def test_double_acquire_many(self): activated = collections.deque() + acquires = collections.deque() n_lock = lock_utils.MultiLock((threading.RLock(), threading.RLock())) def critical_section(): - start = time.time() - time.sleep(0.05) - end = time.time() + start = now() + time.sleep(NAPPY_TIME) + end = now() activated.append((start, end)) def run(): - with n_lock: + with n_lock as gotten: + acquires.append(gotten) critical_section() - with n_lock: + with n_lock as gotten: + acquires.append(gotten) critical_section() critical_section() threads = [] - for i in range(0, 20): + for i in range(0, self.THREAD_COUNT): t = threading_utils.daemon_thread(run) threads.append(t) t.start() @@ -292,6 +316,9 @@ class MultilockTest(test.TestCase): t = threads.pop() t.join() + self.assertTrue(all(acquires)) + self.assertEqual(self.THREAD_COUNT * 2, len(acquires)) + self.assertEqual(self.THREAD_COUNT * 3, len(activated)) for (start, end) in activated: self.assertEqual(1, _find_overlaps(activated, start, end)) diff --git a/taskflow/utils/lock_utils.py b/taskflow/utils/lock_utils.py index ea2023e0..1ee924b2 100644 --- a/taskflow/utils/lock_utils.py +++ b/taskflow/utils/lock_utils.py @@ -363,21 +363,19 @@ class MultiLock(object): # Cleans off one level of the stack (this is done so that if there # are multiple __enter__() and __exit__() pairs active that this will # only remove one level (the last one), and not all levels... - leftover = self._lock_stacks[-1] - while leftover: - lock = self._locks[leftover - 1] + for left in misc.countdown_iter(self._lock_stacks[-1]): + lock_idx = left - 1 + lock = self._locks[lock_idx] try: lock.release() except (threading.ThreadError, RuntimeError) as e: # Ensure that we adjust the lock stack under failure so that # if release is attempted again that we do not try to release # the locks we already released... - self._lock_stacks[-1] = leftover + self._lock_stacks[-1] = left raise threading.ThreadError( "Unable to release lock %s/%s due to '%s'" - % (leftover, len(self._locks), e)) - else: - leftover -= 1 + % (left, len(self._locks), e)) # At the end only clear it off, so that under partial failure we don't # lose any locks... self._lock_stacks.pop() diff --git a/taskflow/utils/misc.py b/taskflow/utils/misc.py index 299082ae..b2e967d4 100644 --- a/taskflow/utils/misc.py +++ b/taskflow/utils/misc.py @@ -74,6 +74,21 @@ def find_monotonic(allow_time_time=False): return None +def countdown_iter(start_at, decr=1): + """Generator that decrements after each generation until <= zero. + + NOTE(harlowja): we can likely remove this when we can use an + ``itertools.count`` that takes a step (on py2.6 which we still support + that step parameter does **not** exist and therefore can't be used). + """ + if decr <= 0: + raise ValueError("Decrement value must be greater" + " than zero and not %s" % decr) + while start_at > 0: + yield start_at + start_at -= decr + + def merge_uri(uri, conf): """Merges a parsed uri into the given configuration dictionary. From d1b6776435644330698722143e36c37b9c3ca375 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Sun, 15 Feb 2015 20:03:53 -0800 Subject: [PATCH 002/246] Make the dispatcher handler be an actual type Instead of having the dispatcher target be a tuple or a single callback, have it be an actual type with comments as to what the types fields are and how they are used. This makes it more obvious as to what those fields are and how they are used so that it is easier to understand how the WBE engine and components work. Change-Id: I5541ccd5a7aa6ae73ed9adceeac2c0524e2a1dc9 --- taskflow/engines/worker_based/dispatcher.py | 37 +++++++++++++++++-- taskflow/engines/worker_based/engine.py | 5 ++- taskflow/engines/worker_based/executor.py | 7 ++-- taskflow/engines/worker_based/server.py | 12 +++--- taskflow/engines/worker_based/types.py | 7 ++-- .../unit/worker_based/test_dispatcher.py | 6 +-- .../unit/worker_based/test_message_pump.py | 11 +++--- 7 files changed, 58 insertions(+), 27 deletions(-) diff --git a/taskflow/engines/worker_based/dispatcher.py b/taskflow/engines/worker_based/dispatcher.py index 13470e08..27350731 100644 --- a/taskflow/engines/worker_based/dispatcher.py +++ b/taskflow/engines/worker_based/dispatcher.py @@ -23,6 +23,36 @@ from taskflow.utils import kombu_utils as ku LOG = logging.getLogger(__name__) +class Handler(object): + """Component(s) that will be called on reception of messages.""" + + __slots__ = ['_process_message', '_validator'] + + def __init__(self, process_message, validator=None): + self._process_message = process_message + self._validator = validator + + @property + def process_message(self): + """Main callback that is called to process a received message. + + This is only called after the format has been validated (using + the ``validator`` callback if applicable) and only after the message + has been acknowledged. + """ + return self._process_message + + @property + def validator(self): + """Optional callback that will be activated before processing. + + This callback if present is expected to validate the message and + raise :py:class:`~taskflow.exceptions.InvalidFormat` if the message + is not valid. + """ + return self._validator + + class TypeDispatcher(object): """Receives messages and dispatches to type specific handlers.""" @@ -99,10 +129,9 @@ class TypeDispatcher(object): LOG.warning("Unexpected message type: '%s' in message" " '%s'", message_type, ku.DelayedPretty(message)) else: - if isinstance(handler, (tuple, list)): - handler, validator = handler + if handler.validator is not None: try: - validator(data) + handler.validator(data) except excp.InvalidFormat as e: message.reject_log_error( logger=LOG, errors=(kombu_exc.MessageStateError,)) @@ -115,7 +144,7 @@ class TypeDispatcher(object): if message.acknowledged: LOG.debug("Message '%s' was acknowledged.", ku.DelayedPretty(message)) - handler(data, message) + handler.process_message(data, message) else: message.reject_log_error(logger=LOG, errors=(kombu_exc.MessageStateError,)) diff --git a/taskflow/engines/worker_based/engine.py b/taskflow/engines/worker_based/engine.py index 31dcade1..a22a5d9f 100644 --- a/taskflow/engines/worker_based/engine.py +++ b/taskflow/engines/worker_based/engine.py @@ -36,8 +36,9 @@ class WorkerBasedActionEngine(engine.ActionEngine): of the (PENDING, WAITING) request states. When expired the associated task the request was made for will have its result become a - `RequestTimeout` exception instead of its - normally returned value (or raised exception). + :py:class:`~taskflow.exceptions.RequestTimeout` + exception instead of its normally returned + value (or raised exception). :param transport_options: transport specific options (see: http://kombu.readthedocs.org/ for what these options imply and are expected to be) diff --git a/taskflow/engines/worker_based/executor.py b/taskflow/engines/worker_based/executor.py index 7722432f..b5b7d39b 100644 --- a/taskflow/engines/worker_based/executor.py +++ b/taskflow/engines/worker_based/executor.py @@ -19,6 +19,7 @@ import functools from oslo_utils import timeutils from taskflow.engines.action_engine import executor +from taskflow.engines.worker_based import dispatcher from taskflow.engines.worker_based import protocol as pr from taskflow.engines.worker_based import proxy from taskflow.engines.worker_based import types as wt @@ -44,10 +45,8 @@ class WorkerTaskExecutor(executor.TaskExecutor): self._requests_cache = wt.RequestsCache() self._transition_timeout = transition_timeout type_handlers = { - pr.RESPONSE: [ - self._process_response, - pr.Response.validate, - ], + pr.RESPONSE: dispatcher.Handler(self._process_response, + validator=pr.Response.validate), } self._proxy = proxy.Proxy(uuid, exchange, type_handlers=type_handlers, diff --git a/taskflow/engines/worker_based/server.py b/taskflow/engines/worker_based/server.py index 1c9a605f..1043e879 100644 --- a/taskflow/engines/worker_based/server.py +++ b/taskflow/engines/worker_based/server.py @@ -19,6 +19,7 @@ import functools from oslo_utils import reflection import six +from taskflow.engines.worker_based import dispatcher from taskflow.engines.worker_based import protocol as pr from taskflow.engines.worker_based import proxy from taskflow import logging @@ -38,14 +39,13 @@ class Server(object): url=None, transport=None, transport_options=None, retry_options=None): type_handlers = { - pr.NOTIFY: [ + pr.NOTIFY: dispatcher.Handler( self._delayed_process(self._process_notify), - functools.partial(pr.Notify.validate, response=False), - ], - pr.REQUEST: [ + validator=functools.partial(pr.Notify.validate, + response=False)), + pr.REQUEST: dispatcher.Handler( self._delayed_process(self._process_request), - pr.Request.validate, - ], + validator=pr.Request.validate), } self._executor = executor self._proxy = proxy.Proxy(topic, exchange, diff --git a/taskflow/engines/worker_based/types.py b/taskflow/engines/worker_based/types.py index 3d53e87a..1ee8f4b8 100644 --- a/taskflow/engines/worker_based/types.py +++ b/taskflow/engines/worker_based/types.py @@ -23,6 +23,7 @@ import threading from oslo_utils import reflection import six +from taskflow.engines.worker_based import dispatcher from taskflow.engines.worker_based import protocol as pr from taskflow import logging from taskflow.types import cache as base @@ -165,10 +166,10 @@ class ProxyWorkerFinder(WorkerFinder): self._workers = {} self._uuid = uuid self._proxy.dispatcher.type_handlers.update({ - pr.NOTIFY: [ + pr.NOTIFY: dispatcher.Handler( self._process_response, - functools.partial(pr.Notify.validate, response=True), - ], + validator=functools.partial(pr.Notify.validate, + response=True)), }) self._counter = itertools.count() diff --git a/taskflow/tests/unit/worker_based/test_dispatcher.py b/taskflow/tests/unit/worker_based/test_dispatcher.py index 21fccdcc..af97e485 100644 --- a/taskflow/tests/unit/worker_based/test_dispatcher.py +++ b/taskflow/tests/unit/worker_based/test_dispatcher.py @@ -40,12 +40,12 @@ def mock_acked_message(ack_ok=True, **kwargs): class TestDispatcher(test.TestCase): def test_creation(self): on_hello = mock.MagicMock() - handlers = {'hello': on_hello} + handlers = {'hello': dispatcher.Handler(on_hello)} dispatcher.TypeDispatcher(type_handlers=handlers) def test_on_message(self): on_hello = mock.MagicMock() - handlers = {'hello': on_hello} + handlers = {'hello': dispatcher.Handler(on_hello)} d = dispatcher.TypeDispatcher(type_handlers=handlers) msg = mock_acked_message(properties={'type': 'hello'}) d.on_message("", msg) @@ -70,7 +70,7 @@ class TestDispatcher(test.TestCase): def test_failed_ack(self): on_hello = mock.MagicMock() - handlers = {'hello': on_hello} + handlers = {'hello': dispatcher.Handler(on_hello)} d = dispatcher.TypeDispatcher(type_handlers=handlers) msg = mock_acked_message(ack_ok=False, properties={'type': 'hello'}) diff --git a/taskflow/tests/unit/worker_based/test_message_pump.py b/taskflow/tests/unit/worker_based/test_message_pump.py index d8438131..59bddbda 100644 --- a/taskflow/tests/unit/worker_based/test_message_pump.py +++ b/taskflow/tests/unit/worker_based/test_message_pump.py @@ -16,6 +16,7 @@ from oslo_utils import uuidutils +from taskflow.engines.worker_based import dispatcher from taskflow.engines.worker_based import protocol as pr from taskflow.engines.worker_based import proxy from taskflow import test @@ -35,7 +36,7 @@ class TestMessagePump(test.TestCase): on_notify = mock.MagicMock() on_notify.side_effect = lambda *args, **kwargs: barrier.set() - handlers = {pr.NOTIFY: on_notify} + handlers = {pr.NOTIFY: dispatcher.Handler(on_notify)} p = proxy.Proxy(TEST_TOPIC, TEST_EXCHANGE, handlers, transport='memory', transport_options={ @@ -60,7 +61,7 @@ class TestMessagePump(test.TestCase): on_response = mock.MagicMock() on_response.side_effect = lambda *args, **kwargs: barrier.set() - handlers = {pr.RESPONSE: on_response} + handlers = {pr.RESPONSE: dispatcher.Handler(on_response)} p = proxy.Proxy(TEST_TOPIC, TEST_EXCHANGE, handlers, transport='memory', transport_options={ @@ -96,9 +97,9 @@ class TestMessagePump(test.TestCase): on_request.side_effect = countdown handlers = { - pr.NOTIFY: on_notify, - pr.RESPONSE: on_response, - pr.REQUEST: on_request, + pr.NOTIFY: dispatcher.Handler(on_notify), + pr.RESPONSE: dispatcher.Handler(on_response), + pr.REQUEST: dispatcher.Handler(on_request), } p = proxy.Proxy(TEST_TOPIC, TEST_EXCHANGE, handlers, transport='memory', From eaad72550821f668e82742cf7ad1323d935d2a26 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Sat, 27 Sep 2014 23:31:56 -0700 Subject: [PATCH 003/246] Allow loading conductors via entrypoints Add an easy to use helper function that is exposed to fetch a conductor in a way that matches how engines and persistence backends and jobboards are loaded. This also adjusts the single threaded conductor to be now named blocking conductor and exposes an entrypoint backend fetch() function to load current and future conductors. Change-Id: Id146d847c329d3e8510a8f24c3ec8b918680ddb5 --- doc/source/conductors.rst | 5 +- setup.cfg | 3 + taskflow/conductors/backends/__init__.py | 45 +++++ taskflow/conductors/backends/impl_blocking.py | 175 ++++++++++++++++++ taskflow/conductors/base.py | 8 +- taskflow/conductors/single_threaded.py | 168 ++--------------- .../{test_conductor.py => test_blocking.py} | 20 +- taskflow/utils/deprecation.py | 44 +++-- 8 files changed, 282 insertions(+), 186 deletions(-) create mode 100644 taskflow/conductors/backends/__init__.py create mode 100644 taskflow/conductors/backends/impl_blocking.py rename taskflow/tests/unit/conductor/{test_conductor.py => test_blocking.py} (90%) diff --git a/doc/source/conductors.rst b/doc/source/conductors.rst index 56fb0e0e..16191ff0 100644 --- a/doc/source/conductors.rst +++ b/doc/source/conductors.rst @@ -67,14 +67,15 @@ Interfaces Implementations =============== -.. automodule:: taskflow.conductors.single_threaded +.. automodule:: taskflow.conductors.backends +.. automodule:: taskflow.conductors.backends.impl_blocking Hierarchy ========= .. inheritance-diagram:: taskflow.conductors.base - taskflow.conductors.single_threaded + taskflow.conductors.backends.impl_blocking :parts: 1 .. _railroad conductors: http://en.wikipedia.org/wiki/Conductor_%28transportation%29 diff --git a/setup.cfg b/setup.cfg index fcaff44d..d9ffce21 100644 --- a/setup.cfg +++ b/setup.cfg @@ -37,6 +37,9 @@ packages = taskflow.jobboards = zookeeper = taskflow.jobs.backends.impl_zookeeper:ZookeeperJobBoard +taskflow.conductors = + blocking = taskflow.conductors.backends.impl_blocking:BlockingConductor + taskflow.persistence = dir = taskflow.persistence.backends.impl_dir:DirBackend file = taskflow.persistence.backends.impl_dir:DirBackend diff --git a/taskflow/conductors/backends/__init__.py b/taskflow/conductors/backends/__init__.py new file mode 100644 index 00000000..0fd75305 --- /dev/null +++ b/taskflow/conductors/backends/__init__.py @@ -0,0 +1,45 @@ +# -*- coding: utf-8 -*- + +# Copyright (C) 2014 Yahoo! Inc. 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. + +import logging + +import stevedore.driver + +from taskflow import exceptions as exc + +# NOTE(harlowja): this is the entrypoint namespace, not the module namespace. +CONDUCTOR_NAMESPACE = 'taskflow.conductors' + +LOG = logging.getLogger(__name__) + + +def fetch(kind, name, jobboard, namespace=CONDUCTOR_NAMESPACE, **kwargs): + """Fetch a conductor backend with the given options. + + This fetch method will look for the entrypoint 'kind' in the entrypoint + namespace, and then attempt to instantiate that entrypoint using the + provided name, jobboard and any board specific kwargs. + """ + LOG.debug('Looking for %r conductor driver in %r', kind, namespace) + try: + mgr = stevedore.driver.DriverManager( + namespace, kind, + invoke_on_load=True, + invoke_args=(name, jobboard), + invoke_kwds=kwargs) + return mgr.driver + except RuntimeError as e: + raise exc.NotFound("Could not find conductor %s" % (kind), e) diff --git a/taskflow/conductors/backends/impl_blocking.py b/taskflow/conductors/backends/impl_blocking.py new file mode 100644 index 00000000..c53248ee --- /dev/null +++ b/taskflow/conductors/backends/impl_blocking.py @@ -0,0 +1,175 @@ +# -*- coding: utf-8 -*- + +# 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. + +import six + +from taskflow.conductors import base +from taskflow import exceptions as excp +from taskflow.listeners import logging as logging_listener +from taskflow import logging +from taskflow.types import timing as tt +from taskflow.utils import async_utils +from taskflow.utils import deprecation +from taskflow.utils import threading_utils + +LOG = logging.getLogger(__name__) +WAIT_TIMEOUT = 0.5 +NO_CONSUME_EXCEPTIONS = tuple([ + excp.ExecutionFailure, + excp.StorageFailure, +]) + + +class BlockingConductor(base.Conductor): + """A conductor that runs jobs in its own dispatching loop. + + This conductor iterates over jobs in the provided jobboard (waiting for + the given timeout if no jobs exist) and attempts to claim them, work on + those jobs in its local thread (blocking further work from being claimed + and consumed) and then consume those work units after completetion. This + process will repeat until the conductor has been stopped or other critical + error occurs. + + NOTE(harlowja): consumption occurs even if a engine fails to run due to + a task failure. This is only skipped when an execution failure or + a storage failure occurs which are *usually* correctable by re-running on + a different conductor (storage failures and execution failures may be + transient issues that can be worked around by later execution). If a job + after completing can not be consumed or abandoned the conductor relies + upon the jobboard capabilities to automatically abandon these jobs. + """ + + def __init__(self, name, jobboard, + persistence=None, engine=None, + engine_options=None, wait_timeout=None): + super(BlockingConductor, self).__init__( + name, jobboard, persistence=persistence, + engine=engine, engine_options=engine_options) + if wait_timeout is None: + wait_timeout = WAIT_TIMEOUT + if isinstance(wait_timeout, (int, float) + six.string_types): + self._wait_timeout = tt.Timeout(float(wait_timeout)) + elif isinstance(wait_timeout, tt.Timeout): + self._wait_timeout = wait_timeout + else: + raise ValueError("Invalid timeout literal: %s" % (wait_timeout)) + self._dead = threading_utils.Event() + + @deprecation.removed_kwarg('timeout', + version="0.8", removal_version="?") + def stop(self, timeout=None): + """Requests the conductor to stop dispatching. + + This method can be used to request that a conductor stop its + consumption & dispatching loop. + + The method returns immediately regardless of whether the conductor has + been stopped. + + :param timeout: This parameter is **deprecated** and is present for + backward compatibility **only**. In order to wait for + the conductor to gracefully shut down, :meth:`wait` + should be used instead. + """ + self._wait_timeout.interrupt() + + @property + def dispatching(self): + return not self._dead.is_set() + + def _dispatch_job(self, job): + engine = self._engine_from_job(job) + consume = True + with logging_listener.LoggingListener(engine, log=LOG): + LOG.debug("Dispatching engine %s for job: %s", engine, job) + try: + engine.run() + except excp.WrappedFailure as e: + if all((f.check(*NO_CONSUME_EXCEPTIONS) for f in e)): + consume = False + if LOG.isEnabledFor(logging.WARNING): + if consume: + LOG.warn("Job execution failed (consumption being" + " skipped): %s [%s failures]", job, len(e)) + else: + LOG.warn("Job execution failed (consumption" + " proceeding): %s [%s failures]", job, len(e)) + # Show the failure/s + traceback (if possible)... + for i, f in enumerate(e): + LOG.warn("%s. %s", i + 1, f.pformat(traceback=True)) + except NO_CONSUME_EXCEPTIONS: + LOG.warn("Job execution failed (consumption being" + " skipped): %s", job, exc_info=True) + consume = False + except Exception: + LOG.warn("Job execution failed (consumption proceeding): %s", + job, exc_info=True) + else: + LOG.info("Job completed successfully: %s", job) + return async_utils.make_completed_future(consume) + + def run(self): + self._dead.clear() + try: + while True: + if self._wait_timeout.is_stopped(): + break + dispatched = 0 + for job in self._jobboard.iterjobs(): + if self._wait_timeout.is_stopped(): + break + LOG.debug("Trying to claim job: %s", job) + try: + self._jobboard.claim(job, self._name) + except (excp.UnclaimableJob, excp.NotFound): + LOG.debug("Job already claimed or consumed: %s", job) + continue + consume = False + try: + f = self._dispatch_job(job) + except Exception: + LOG.warn("Job dispatching failed: %s", job, + exc_info=True) + else: + dispatched += 1 + consume = f.result() + try: + if consume: + self._jobboard.consume(job, self._name) + else: + self._jobboard.abandon(job, self._name) + except (excp.JobFailure, excp.NotFound): + if consume: + LOG.warn("Failed job consumption: %s", job, + exc_info=True) + else: + LOG.warn("Failed job abandonment: %s", job, + exc_info=True) + if dispatched == 0 and not self._wait_timeout.is_stopped(): + self._wait_timeout.wait() + finally: + self._dead.set() + + def wait(self, timeout=None): + """Waits for the conductor to gracefully exit. + + This method waits for the conductor to gracefully exit. An optional + timeout can be provided, which will cause the method to return + within the specified timeout. If the timeout is reached, the returned + value will be False. + + :param timeout: Maximum number of seconds that the :meth:`wait` method + should block for. + """ + return self._dead.wait(timeout) diff --git a/taskflow/conductors/base.py b/taskflow/conductors/base.py index f7546c3e..48344c53 100644 --- a/taskflow/conductors/base.py +++ b/taskflow/conductors/base.py @@ -24,7 +24,7 @@ from taskflow.utils import lock_utils @six.add_metaclass(abc.ABCMeta) class Conductor(object): - """Conductors conduct jobs & assist in associated runtime interactions. + """Base for all conductor implementations. Conductors act as entities which extract jobs from a jobboard, assign there work to some engine (using some desired configuration) and then wait @@ -34,8 +34,8 @@ class Conductor(object): period of time will finish up the prior failed conductors work. """ - def __init__(self, name, jobboard, persistence, - engine=None, engine_options=None): + def __init__(self, name, jobboard, + persistence=None, engine=None, engine_options=None): self._name = name self._jobboard = jobboard self._engine = engine @@ -101,7 +101,7 @@ class Conductor(object): @lock_utils.locked def close(self): - """Closes the jobboard, disallowing further use.""" + """Closes the contained jobboard, disallowing further use.""" self._jobboard.close() @abc.abstractmethod diff --git a/taskflow/conductors/single_threaded.py b/taskflow/conductors/single_threaded.py index a52dc347..aa90645f 100644 --- a/taskflow/conductors/single_threaded.py +++ b/taskflow/conductors/single_threaded.py @@ -1,5 +1,7 @@ # -*- coding: utf-8 -*- +# Copyright (C) 2015 Yahoo! Inc. 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 @@ -12,163 +14,15 @@ # License for the specific language governing permissions and limitations # under the License. -import six - -from taskflow.conductors import base -from taskflow import exceptions as excp -from taskflow.listeners import logging as logging_listener -from taskflow import logging -from taskflow.types import timing as tt -from taskflow.utils import async_utils +from taskflow.conductors.backends import impl_blocking from taskflow.utils import deprecation -from taskflow.utils import threading_utils -LOG = logging.getLogger(__name__) -WAIT_TIMEOUT = 0.5 -NO_CONSUME_EXCEPTIONS = tuple([ - excp.ExecutionFailure, - excp.StorageFailure, -]) +# TODO(harlowja): remove this module soon... +deprecation.removed_module(__name__, + replacement_name="the conductor entrypoints", + version="0.8", removal_version="?") - -class SingleThreadedConductor(base.Conductor): - """A conductor that runs jobs in its own dispatching loop. - - This conductor iterates over jobs in the provided jobboard (waiting for - the given timeout if no jobs exist) and attempts to claim them, work on - those jobs in its local thread (blocking further work from being claimed - and consumed) and then consume those work units after completetion. This - process will repeat until the conductor has been stopped or other critical - error occurs. - - NOTE(harlowja): consumption occurs even if a engine fails to run due to - a task failure. This is only skipped when an execution failure or - a storage failure occurs which are *usually* correctable by re-running on - a different conductor (storage failures and execution failures may be - transient issues that can be worked around by later execution). If a job - after completing can not be consumed or abandoned the conductor relies - upon the jobboard capabilities to automatically abandon these jobs. - """ - - def __init__(self, name, jobboard, persistence, - engine=None, engine_options=None, wait_timeout=None): - super(SingleThreadedConductor, self).__init__( - name, jobboard, persistence, - engine=engine, engine_options=engine_options) - if wait_timeout is None: - wait_timeout = WAIT_TIMEOUT - if isinstance(wait_timeout, (int, float) + six.string_types): - self._wait_timeout = tt.Timeout(float(wait_timeout)) - elif isinstance(wait_timeout, tt.Timeout): - self._wait_timeout = wait_timeout - else: - raise ValueError("Invalid timeout literal: %s" % (wait_timeout)) - self._dead = threading_utils.Event() - - @deprecation.removed_kwarg('timeout', - version="0.8", removal_version="?") - def stop(self, timeout=None): - """Requests the conductor to stop dispatching. - - This method can be used to request that a conductor stop its - consumption & dispatching loop. - - The method returns immediately regardless of whether the conductor has - been stopped. - - :param timeout: This parameter is **deprecated** and is present for - backward compatibility **only**. In order to wait for - the conductor to gracefully shut down, :meth:`wait` - should be used instead. - """ - self._wait_timeout.interrupt() - - @property - def dispatching(self): - return not self._dead.is_set() - - def _dispatch_job(self, job): - engine = self._engine_from_job(job) - consume = True - with logging_listener.LoggingListener(engine, log=LOG): - LOG.debug("Dispatching engine %s for job: %s", engine, job) - try: - engine.run() - except excp.WrappedFailure as e: - if all((f.check(*NO_CONSUME_EXCEPTIONS) for f in e)): - consume = False - if LOG.isEnabledFor(logging.WARNING): - if consume: - LOG.warn("Job execution failed (consumption being" - " skipped): %s [%s failures]", job, len(e)) - else: - LOG.warn("Job execution failed (consumption" - " proceeding): %s [%s failures]", job, len(e)) - # Show the failure/s + traceback (if possible)... - for i, f in enumerate(e): - LOG.warn("%s. %s", i + 1, f.pformat(traceback=True)) - except NO_CONSUME_EXCEPTIONS: - LOG.warn("Job execution failed (consumption being" - " skipped): %s", job, exc_info=True) - consume = False - except Exception: - LOG.warn("Job execution failed (consumption proceeding): %s", - job, exc_info=True) - else: - LOG.info("Job completed successfully: %s", job) - return async_utils.make_completed_future(consume) - - def run(self): - self._dead.clear() - try: - while True: - if self._wait_timeout.is_stopped(): - break - dispatched = 0 - for job in self._jobboard.iterjobs(): - if self._wait_timeout.is_stopped(): - break - LOG.debug("Trying to claim job: %s", job) - try: - self._jobboard.claim(job, self._name) - except (excp.UnclaimableJob, excp.NotFound): - LOG.debug("Job already claimed or consumed: %s", job) - continue - consume = False - try: - f = self._dispatch_job(job) - except Exception: - LOG.warn("Job dispatching failed: %s", job, - exc_info=True) - else: - dispatched += 1 - consume = f.result() - try: - if consume: - self._jobboard.consume(job, self._name) - else: - self._jobboard.abandon(job, self._name) - except (excp.JobFailure, excp.NotFound): - if consume: - LOG.warn("Failed job consumption: %s", job, - exc_info=True) - else: - LOG.warn("Failed job abandonment: %s", job, - exc_info=True) - if dispatched == 0 and not self._wait_timeout.is_stopped(): - self._wait_timeout.wait() - finally: - self._dead.set() - - def wait(self, timeout=None): - """Waits for the conductor to gracefully exit. - - This method waits for the conductor to gracefully exit. An optional - timeout can be provided, which will cause the method to return - within the specified timeout. If the timeout is reached, the returned - value will be False. - - :param timeout: Maximum number of seconds that the :meth:`wait` method - should block for. - """ - return self._dead.wait(timeout) +# TODO(harlowja): remove this proxy/legacy class soon... +SingleThreadedConductor = deprecation.moved_inheritable_class( + impl_blocking.BlockingConductor, 'SingleThreadedConductor', + __name__, version="0.8", removal_version="?") diff --git a/taskflow/tests/unit/conductor/test_conductor.py b/taskflow/tests/unit/conductor/test_blocking.py similarity index 90% rename from taskflow/tests/unit/conductor/test_conductor.py rename to taskflow/tests/unit/conductor/test_blocking.py index 8b21e56d..33de7807 100644 --- a/taskflow/tests/unit/conductor/test_conductor.py +++ b/taskflow/tests/unit/conductor/test_blocking.py @@ -19,7 +19,7 @@ import contextlib from zake import fake_client -from taskflow.conductors import single_threaded as stc +from taskflow.conductors import backends from taskflow import engines from taskflow.jobs.backends import impl_zookeeper from taskflow.jobs import base @@ -50,10 +50,13 @@ def test_factory(blowup): return f -class SingleThreadedConductorTest(test_utils.EngineTestBase, test.TestCase): - ComponentBundle = collections.namedtuple('ComponentBundle', - ['board', 'client', - 'persistence', 'conductor']) +ComponentBundle = collections.namedtuple('ComponentBundle', + ['board', 'client', + 'persistence', 'conductor']) + + +class BlockingConductorTest(test_utils.EngineTestBase, test.TestCase): + KIND = 'blocking' def make_components(self, name='testing', wait_timeout=0.1): client = fake_client.FakeClient() @@ -61,9 +64,10 @@ class SingleThreadedConductorTest(test_utils.EngineTestBase, test.TestCase): board = impl_zookeeper.ZookeeperJobBoard(name, {}, client=client, persistence=persistence) - conductor = stc.SingleThreadedConductor(name, board, persistence, - wait_timeout=wait_timeout) - return self.ComponentBundle(board, client, persistence, conductor) + conductor = backends.fetch(self.KIND, name, board, + persistence=persistence, + wait_timeout=wait_timeout) + return ComponentBundle(board, client, persistence, conductor) def test_connection(self): components = self.make_components() diff --git a/taskflow/utils/deprecation.py b/taskflow/utils/deprecation.py index 9426f118..55ed982d 100644 --- a/taskflow/utils/deprecation.py +++ b/taskflow/utils/deprecation.py @@ -118,8 +118,8 @@ class MovedClassProxy(object): type(self).__name__, id(self), wrapped, id(wrapped)) -def _generate_moved_message(prefix, postfix=None, message=None, - version=None, removal_version=None): +def _generate_message(prefix, postfix=None, message=None, + version=None, removal_version=None): message_components = [prefix] if version: message_components.append(" in version '%s'" % version) @@ -143,9 +143,9 @@ def renamed_kwarg(old_name, new_name, message=None, prefix = _KWARG_MOVED_PREFIX_TPL % old_name postfix = _KWARG_MOVED_POSTFIX_TPL % new_name - out_message = _generate_moved_message(prefix, postfix=postfix, - message=message, version=version, - removal_version=removal_version) + out_message = _generate_message(prefix, postfix=postfix, + message=message, version=version, + removal_version=removal_version) def decorator(f): @@ -165,9 +165,9 @@ def removed_kwarg(old_name, message=None, """Decorates a kwarg accepting function to deprecate a removed kwarg.""" prefix = _KWARG_MOVED_PREFIX_TPL % old_name - out_message = _generate_moved_message(prefix, postfix=None, - message=message, version=version, - removal_version=removal_version) + out_message = _generate_message(prefix, postfix=None, + message=message, version=version, + removal_version=removal_version) def decorator(f): @@ -204,7 +204,7 @@ def _moved_decorator(kind, new_attribute_name, message=None, old_name = ".".join((base_name, old_attribute_name)) new_name = ".".join((base_name, new_attribute_name)) prefix = _KIND_MOVED_PREFIX_TPL % (kind, old_name, new_name) - out_message = _generate_moved_message( + out_message = _generate_message( prefix, message=message, version=version, removal_version=removal_version) deprecation(out_message, stacklevel=stacklevel) @@ -215,6 +215,20 @@ def _moved_decorator(kind, new_attribute_name, message=None, return decorator +def removed_module(module_name, replacement_name=None, message=None, + version=None, removal_version=None, stacklevel=4): + prefix = "The '%s' module usage is deprecated" % module_name + if replacement_name: + postfix = ", please use %s instead" % replacement_name + else: + postfix = None + out_message = _generate_message(prefix, + postfix=postfix, message=message, + version=version, + removal_version=removal_version) + deprecation(out_message, stacklevel=stacklevel) + + def moved_property(new_attribute_name, message=None, version=None, removal_version=None, stacklevel=3): """Decorates a *instance* property that was moved to another location.""" @@ -240,9 +254,9 @@ def moved_inheritable_class(new_class, old_class_name, old_module_name, old_name = ".".join((old_module_name, old_class_name)) new_name = reflection.get_class_name(new_class) prefix = _CLASS_MOVED_PREFIX_TPL % (old_name, new_name) - out_message = _generate_moved_message(prefix, - message=message, version=version, - removal_version=removal_version) + out_message = _generate_message(prefix, + message=message, version=version, + removal_version=removal_version) def decorator(f): @@ -273,7 +287,7 @@ def moved_class(new_class, old_class_name, old_module_name, message=None, old_name = ".".join((old_module_name, old_class_name)) new_name = reflection.get_class_name(new_class) prefix = _CLASS_MOVED_PREFIX_TPL % (old_name, new_name) - out_message = _generate_moved_message(prefix, - message=message, version=version, - removal_version=removal_version) + out_message = _generate_message(prefix, + message=message, version=version, + removal_version=removal_version) return MovedClassProxy(new_class, out_message, stacklevel=stacklevel) From d4129479cfaa5170b01c866237ca2c14ba5c4c42 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 11 Feb 2015 13:13:15 -0800 Subject: [PATCH 004/246] Move 'provides' and 'name' to instance attributes Since we are mixing properties and attributes in the atom class it seems better just to standardize on the one that has the most (attributes) and just mention that they are and should always be immutable in the atom class documentation. Depends-On: I18699fc64391445436537854a5e3723f4e89e961 Change-Id: Id03e4229b40b9da5c9a349ee00c13053a9eb75eb --- doc/source/arguments_and_results.rst | 2 +- taskflow/atom.py | 68 ++++++++++++---------------- 2 files changed, 29 insertions(+), 41 deletions(-) diff --git a/doc/source/arguments_and_results.rst b/doc/source/arguments_and_results.rst index 998db889..714ee87f 100644 --- a/doc/source/arguments_and_results.rst +++ b/doc/source/arguments_and_results.rst @@ -215,7 +215,7 @@ name of the value. ... return 42 ... >>> TheAnswerReturningTask(provides='the_answer').provides - set(['the_answer']) + frozenset(['the_answer']) Returning a tuple +++++++++++++++++ diff --git a/taskflow/atom.py b/taskflow/atom.py index 82f7a5e3..2ade20e6 100644 --- a/taskflow/atom.py +++ b/taskflow/atom.py @@ -135,12 +135,28 @@ def _build_arg_mapping(atom_name, reqs, rebind_args, function, do_infer, @six.add_metaclass(abc.ABCMeta) class Atom(object): - """An abstract flow atom that causes a flow to progress (in some manner). + """An unit of work that causes a flow to progress (in some manner). - An atom is a named object that operates with input flow data to perform + An atom is a named object that operates with input data to perform some action that furthers the overall flows progress. It usually also produces some of its own named output as a result of this process. + NOTE(harlowja): there can be no intersection between what this atom + requires and what it produces (since this would be an impossible + dependency to satisfy). + + :param name: Meaningful name for this atom, should be something that is + distinguishable and understandable for notification, + debugging, storing and any other similar purposes. + :param provides: A set, string or list of items that + this will be providing (or could provide) to others, used + to correlate and associate the thing/s this atom + produces, if it produces anything at all. + :param inject: An *immutable* input_name => value dictionary which + specifies any initial inputs that should be automatically + injected into the atoms scope before the atom execution + commences (this allows for providing atom *local* values + that do not need to be provided by other atoms/dependents). :ivar version: An *immutable* version that associates version information with this atom. It can be useful in resuming older versions of atoms. Standard major, minor versioning concepts @@ -156,43 +172,31 @@ class Atom(object): the names that this atom expects (in a way this is like remapping a namespace of another atom into the namespace of this atom). - :param name: Meaningful name for this atom, should be something that is - distinguishable and understandable for notification, - debugging, storing and any other similar purposes. - :param provides: A set, string or list of items that - this will be providing (or could provide) to others, used - to correlate and associate the thing/s this atom - produces, if it produces anything at all. - :param inject: An *immutable* input_name => value dictionary which - specifies any initial inputs that should be automatically - injected into the atoms scope before the atom execution - commences (this allows for providing atom *local* values that - do not need to be provided by other atoms/dependents). :ivar inject: See parameter ``inject``. - :ivar requires: Any inputs this atom requires to function (if applicable). - NOTE(harlowja): there can be no intersection between what - this atom requires and what it produces (since this would - be an impossible dependency to satisfy). - :ivar optional: Any inputs that are optional for this atom's execute - method. - + :ivar name: See parameter ``name``. + :ivar requires: An *immutable* set of inputs this atom requires to + function. + :ivar optional: An *immutable* set of inputs that are optional for this + atom to function. + :ivar provides: An *immutable* set of outputs this atom produces. """ def __init__(self, name=None, provides=None, inject=None): - self._name = name + self.name = name self.save_as = _save_as_to_mapping(provides) self.version = (1, 0) self.inject = inject self.requires = frozenset() self.optional = frozenset() + self.provides = frozenset(self.save_as) + self.rebind = {} def _build_arg_mapping(self, executor, requires=None, rebind=None, auto_extract=True, ignore_list=None): req_arg, opt_arg = _build_arg_mapping(self.name, requires, rebind, executor, auto_extract, ignore_list) - - self.rebind = {} + self.rebind.clear() if opt_arg: self.rebind.update(opt_arg) if req_arg: @@ -203,7 +207,6 @@ class Atom(object): inject_set = set(six.iterkeys(self.inject)) self.requires -= inject_set self.optional -= inject_set - out_of_order = self.provides.intersection(self.requires) if out_of_order: raise exceptions.DependencyFailure( @@ -219,23 +222,8 @@ class Atom(object): def revert(self, *args, **kwargs): """Reverts this atom (undoing any :meth:`execute` side-effects).""" - @property - def name(self): - """A non-unique name for this atom (human readable).""" - return self._name - def __str__(self): return "%s==%s" % (self.name, misc.get_version_string(self)) def __repr__(self): return '<%s %s>' % (reflection.get_class_name(self), self) - - @property - def provides(self): - """Any outputs this atom produces. - - NOTE(harlowja): there can be no intersection between what this atom - requires and what it produces (since this would be an impossible - dependency to satisfy). - """ - return set(self.save_as) From 145599a7540d3d1fd3d70ff071b08de5a2cf73e1 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Mon, 16 Feb 2015 21:55:28 -0800 Subject: [PATCH 005/246] Add todo note for kombu pull request Change-Id: I50f51c616f81772b352fabe430972e08330017c4 --- taskflow/utils/kombu_utils.py | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/taskflow/utils/kombu_utils.py b/taskflow/utils/kombu_utils.py index 8ace067b..9fb3e49e 100644 --- a/taskflow/utils/kombu_utils.py +++ b/taskflow/utils/kombu_utils.py @@ -23,7 +23,12 @@ _MSG_PROPERTIES = tuple([ class DelayedPretty(object): - """Wraps a message and delays prettifying it until requested.""" + """Wraps a message and delays prettifying it until requested. + + TODO(harlowja): remove this when https://github.com/celery/kombu/pull/454/ + is merged and a release is made that contains it (since that pull + request is equivalent and/or better than this). + """ def __init__(self, message): self._message = message @@ -47,8 +52,6 @@ def _prettify_message(message): This provides something decent(ish) for debugging (or other purposes) so that messages are more nice and understandable.... - - TODO(harlowja): submit something into kombu to fix/adjust this. """ if message.content_type is not None: properties = { From 06ffceea77ea65f335cfb5652fb057017e5807ba Mon Sep 17 00:00:00 2001 From: Manish Godara Date: Fri, 13 Feb 2015 15:53:22 -0800 Subject: [PATCH 006/246] Update Flow::__str__ The current implementation seems a little cryptic so updating it to clearly indicate the atom count. Change-Id: I4f97e458a4b80e33b3d6348a3ef632398586ed47 --- taskflow/flow.py | 5 ++--- taskflow/tests/unit/patterns/test_graph_flow.py | 2 +- taskflow/tests/unit/patterns/test_linear_flow.py | 4 ++-- taskflow/tests/unit/patterns/test_unordered_flow.py | 2 +- 4 files changed, 6 insertions(+), 7 deletions(-) diff --git a/taskflow/flow.py b/taskflow/flow.py index cd70e7c9..5eb05825 100644 --- a/taskflow/flow.py +++ b/taskflow/flow.py @@ -96,9 +96,8 @@ class Flow(object): """ def __str__(self): - lines = ["%s: %s" % (reflection.get_class_name(self), self.name)] - lines.append("%s" % (len(self))) - return "; ".join(lines) + return "%s: %s(len=%d)" % (reflection.get_class_name(self), + self.name, len(self)) @property def provides(self): diff --git a/taskflow/tests/unit/patterns/test_graph_flow.py b/taskflow/tests/unit/patterns/test_graph_flow.py index 62dbc287..8dad447d 100644 --- a/taskflow/tests/unit/patterns/test_graph_flow.py +++ b/taskflow/tests/unit/patterns/test_graph_flow.py @@ -37,7 +37,7 @@ class GraphFlowTest(test.TestCase): self.assertEqual(f.requires, set()) self.assertEqual(f.provides, set()) - expected = 'taskflow.patterns.graph_flow.Flow: test; 0' + expected = 'taskflow.patterns.graph_flow.Flow: test(len=0)' self.assertEqual(str(f), expected) def test_graph_flow_add_nothing(self): diff --git a/taskflow/tests/unit/patterns/test_linear_flow.py b/taskflow/tests/unit/patterns/test_linear_flow.py index 23f891a8..48f8f8de 100644 --- a/taskflow/tests/unit/patterns/test_linear_flow.py +++ b/taskflow/tests/unit/patterns/test_linear_flow.py @@ -36,7 +36,7 @@ class LinearFlowTest(test.TestCase): self.assertEqual(f.requires, set()) self.assertEqual(f.provides, set()) - expected = 'taskflow.patterns.linear_flow.Flow: test; 0' + expected = 'taskflow.patterns.linear_flow.Flow: test(len=0)' self.assertEqual(str(f), expected) def test_linear_flow_add_nothing(self): @@ -107,7 +107,7 @@ class LinearFlowTest(test.TestCase): (task2, task3, {'invariant': True}) ]) - expected = 'taskflow.patterns.linear_flow.Flow: test; 3' + expected = 'taskflow.patterns.linear_flow.Flow: test(len=3)' self.assertEqual(str(f), expected) def test_linear_flow_with_retry(self): diff --git a/taskflow/tests/unit/patterns/test_unordered_flow.py b/taskflow/tests/unit/patterns/test_unordered_flow.py index e55cfad0..195516b6 100644 --- a/taskflow/tests/unit/patterns/test_unordered_flow.py +++ b/taskflow/tests/unit/patterns/test_unordered_flow.py @@ -36,7 +36,7 @@ class UnorderedFlowTest(test.TestCase): self.assertEqual(f.requires, set()) self.assertEqual(f.provides, set()) - expected = 'taskflow.patterns.unordered_flow.Flow: test; 0' + expected = 'taskflow.patterns.unordered_flow.Flow: test(len=0)' self.assertEqual(str(f), expected) def test_unordered_flow_add_nothing(self): From 33420090d77608f148ee3f6b48c3c07722e80704 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Tue, 17 Feb 2015 15:39:33 -0800 Subject: [PATCH 007/246] Tweaks to atom documentation Use links instead of raw docs that don't link to the appropriate methods. Adjust to say that the retry atoms really provide and define the strategy to resolve surronding/associated failures. Add link to what a scope is (in the computer science terms usage). Change-Id: I093862c368007ac761cf9d1e3b0f94be36ada13a --- doc/source/atoms.rst | 46 +++++++++++++++++++++++++++----------------- 1 file changed, 28 insertions(+), 18 deletions(-) diff --git a/doc/source/atoms.rst b/doc/source/atoms.rst index f2b75ffa..d29c553c 100644 --- a/doc/source/atoms.rst +++ b/doc/source/atoms.rst @@ -23,9 +23,9 @@ values (requirements) and name outputs (provided values). Task ===== -A :py:class:`task ` (derived from an atom) is the -smallest possible unit of work that can have an execute & rollback sequence -associated with it. These task objects all derive +A :py:class:`task ` (derived from an atom) is a +unit of work that can have an execute & rollback sequence associated with +it (they are *nearly* analogous to functions). These task objects all derive from :py:class:`~taskflow.task.BaseTask` which defines what a task must provide in terms of properties and methods. @@ -48,17 +48,17 @@ Retry ===== A :py:class:`retry ` (derived from an atom) is a special -unit that handles errors, controls flow execution and can (for example) retry -other atoms with other parameters if needed. When an associated atom -fails, these retry units are *consulted* to determine what the resolution -method should be. The goal is that with this *consultation* the retry atom -will suggest a method for getting around the failure (perhaps by retrying, -reverting a single item, or reverting everything contained in the retries -associated scope). +unit of work that handles errors, controls flow execution and can (for +example) retry other atoms with other parameters if needed. When an associated +atom fails, these retry units are *consulted* to determine what the resolution +*strategy* should be. The goal is that with this consultation the retry atom +will suggest a *strategy* for getting around the failure (perhaps by retrying, +reverting a single atom, or reverting everything contained in the retries +associated `scope`_). Currently derivatives of the :py:class:`retry ` base -class must provide a ``on_failure`` method to determine how a failure should -be handled. +class must provide a :py:func:`~taskflow.retry.Retry.on_failure` method to +determine how a failure should be handled. The current enumeration set that can be returned from this method is: @@ -74,12 +74,13 @@ The current enumeration set that can be returned from this method is: * ``REVERT_ALL`` - completely reverts a whole flow. To aid in the reconciliation process the -:py:class:`retry ` base class also mandates ``execute`` -and ``revert`` methods (although subclasses are allowed to define these methods -as no-ops) that can be used by a retry atom to interact with the runtime -execution model (for example, to track the number of times it has been -called which is useful for the :py:class:`~taskflow.retry.ForEach` retry -subclass). +:py:class:`retry ` base class also mandates +:py:func:`~taskflow.retry.Retry.execute` +and :py:func:`~taskflow.retry.Retry.revert` methods (although subclasses +are allowed to define these methods as no-ops) that can be used by a retry +atom to interact with the runtime execution model (for example, to track the +number of times it has been called which is useful for +the :py:class:`~taskflow.retry.ForEach` retry subclass). To avoid recreating common retry patterns the following provided retry subclasses are provided: @@ -94,6 +95,15 @@ subclasses are provided: :py:class:`~taskflow.retry.ForEach` but extracts values from storage instead of the :py:class:`~taskflow.retry.ForEach` constructor. +.. _scope: http://en.wikipedia.org/wiki/Scope_%28computer_science%29 + +.. note:: + + They are *similar* to exception handlers but are made to be *more* capable + due to there ability to *dynamically* choose a reconciliation strategy, + which allows for these atoms to influence subsequent execution(s) and the + inputs any associated atoms require. + Examples -------- From 28490e5cfae912afc1ee3ff95805e5d447ae7c52 Mon Sep 17 00:00:00 2001 From: Dan Krause Date: Wed, 18 Feb 2015 15:33:57 -0600 Subject: [PATCH 008/246] add get_flow_details and get_atom_details to all backends Change-Id: Ic21a883b4eee827d90065f8ecfc5af2599ebece5 --- taskflow/persistence/backends/impl_dir.py | 6 ++++ taskflow/persistence/backends/impl_memory.py | 14 +++++++++ .../persistence/backends/impl_sqlalchemy.py | 30 +++++++++++++++++++ taskflow/persistence/base.py | 10 +++++++ 4 files changed, 60 insertions(+) diff --git a/taskflow/persistence/backends/impl_dir.py b/taskflow/persistence/backends/impl_dir.py index 644ca453..155ffe41 100644 --- a/taskflow/persistence/backends/impl_dir.py +++ b/taskflow/persistence/backends/impl_dir.py @@ -403,3 +403,9 @@ class Connection(base.Connection): def get_logbook(self, book_uuid): return self._run_with_process_lock("book", self._get_logbook, book_uuid) + + def get_flow_details(self, fd_uuid): + return self._get_flow_details(fd_uuid) + + def get_atom_details(self, ad_uuid): + return self._get_atom_details(ad_uuid) diff --git a/taskflow/persistence/backends/impl_memory.py b/taskflow/persistence/backends/impl_memory.py index 5e94afb1..020deff8 100644 --- a/taskflow/persistence/backends/impl_memory.py +++ b/taskflow/persistence/backends/impl_memory.py @@ -234,3 +234,17 @@ class Connection(base.Connection): yield book except KeyError: pass + + def get_flow_details(self, fd_uuid): + try: + with self._lock.read_lock(): + return self._memory.flow_details[fd_uuid] + except KeyError: + raise exc.NotFound("No flow details found '%s'" % fd_uuid) + + def get_atom_details(self, ad_uuid): + try: + with self._lock.read_lock(): + return self._memory.atom_details[ad_uuid] + except KeyError: + raise exc.NotFound("No atom details found '%s'" % ad_uuid) diff --git a/taskflow/persistence/backends/impl_sqlalchemy.py b/taskflow/persistence/backends/impl_sqlalchemy.py index 5796e273..a49d2492 100644 --- a/taskflow/persistence/backends/impl_sqlalchemy.py +++ b/taskflow/persistence/backends/impl_sqlalchemy.py @@ -558,5 +558,35 @@ class Connection(base.Connection): for book in gathered: yield book + def get_flow_details(self, fd_uuid): + try: + flowdetails = self._tables.flowdetails + with self._engine.begin() as conn: + q = (sql.select([flowdetails]). + where(flowdetails.c.uuid == fd_uuid)) + row = conn.execute(q).first() + if not row: + raise exc.NotFound("No flow details found with uuid" + " '%s'" % fd_uuid) + return self._converter.convert_flow_detail(row) + except sa_exc.SQLAlchemyError as e: + raise exc.StorageFailure("Failed getting flow details with" + " uuid '%s'" % fd_uuid, e) + + def get_atom_details(self, ad_uuid): + try: + atomdetails = self._tables.atomdetails + with self._engine.begin() as conn: + q = (sql.select([atomdetails]). + where(atomdetails.c.uuid == ad_uuid)) + row = conn.execute(q).first() + if not row: + raise exc.NotFound("No atom details found with uuid" + " '%s'" % ad_uuid) + return self._converter.convert_atom_detail(row) + except sa_exc.SQLAlchemyError as e: + raise exc.StorageFailure("Failed getting atom details with" + " uuid '%s'" % ad_uuid, e) + def close(self): pass diff --git a/taskflow/persistence/base.py b/taskflow/persistence/base.py index 00fb29be..0ce09259 100644 --- a/taskflow/persistence/base.py +++ b/taskflow/persistence/base.py @@ -118,6 +118,16 @@ class Connection(object): """Return an iterable of logbook objects.""" pass + @abc.abstractmethod + def get_flow_details(self, fd_uuid): + """Fetches a flowdetails object matching the given uuid.""" + pass + + @abc.abstractmethod + def get_atom_details(self, ad_uuid): + """Fetches a atomdetails object matching the given uuid.""" + pass + def _format_atom(atom_detail): return { From c9c0505814a88ecea0c995aa4f0dd97883978559 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Fri, 6 Feb 2015 17:14:06 -0800 Subject: [PATCH 009/246] Use debtcollector library to replace internal utility The new debtcollector library was extracted from this code and has now been released (and made available in the requirements list) so we can go ahead and just use it for most of the cases (minus the proxy moved class) in our code-base to reduce the amount of custom code we have to maintain to perform deprecation activities. Change-Id: Ieeda6ffe282b67a1c1bb4f72e5858d22df0f30a0 --- requirements-py2.txt | 3 + requirements-py3.txt | 3 + taskflow/conductors/single_threaded.py | 4 +- taskflow/engines/base.py | 12 ++- taskflow/engines/helpers.py | 18 ++-- taskflow/listeners/base.py | 15 ++-- taskflow/utils/deprecation.py | 120 ++----------------------- taskflow/utils/misc.py | 10 ++- 8 files changed, 41 insertions(+), 144 deletions(-) diff --git a/requirements-py2.txt b/requirements-py2.txt index 083caec0..0a88a9f8 100644 --- a/requirements-py2.txt +++ b/requirements-py2.txt @@ -28,3 +28,6 @@ jsonschema>=2.0.0,<3.0.0 # For common utilities oslo.utils>=1.2.0 # Apache-2.0 oslo.serialization>=1.2.0 # Apache-2.0 + +# For deprecation of things +debtcollector>=0.2 diff --git a/requirements-py3.txt b/requirements-py3.txt index b04fc0af..033ca710 100644 --- a/requirements-py3.txt +++ b/requirements-py3.txt @@ -22,3 +22,6 @@ jsonschema>=2.0.0,<3.0.0 # For common utilities oslo.utils>=1.2.0 # Apache-2.0 oslo.serialization>=1.2.0 # Apache-2.0 + +# For deprecation of things +debtcollector>=0.2 diff --git a/taskflow/conductors/single_threaded.py b/taskflow/conductors/single_threaded.py index aa90645f..c8ab5a03 100644 --- a/taskflow/conductors/single_threaded.py +++ b/taskflow/conductors/single_threaded.py @@ -14,6 +14,8 @@ # License for the specific language governing permissions and limitations # under the License. +from debtcollector import moves + from taskflow.conductors.backends import impl_blocking from taskflow.utils import deprecation @@ -23,6 +25,6 @@ deprecation.removed_module(__name__, version="0.8", removal_version="?") # TODO(harlowja): remove this proxy/legacy class soon... -SingleThreadedConductor = deprecation.moved_inheritable_class( +SingleThreadedConductor = moves.moved_class( impl_blocking.BlockingConductor, 'SingleThreadedConductor', __name__, version="0.8", removal_version="?") diff --git a/taskflow/engines/base.py b/taskflow/engines/base.py index 7ea8e708..5e2263eb 100644 --- a/taskflow/engines/base.py +++ b/taskflow/engines/base.py @@ -17,11 +17,11 @@ import abc +from debtcollector import moves import six from taskflow import storage from taskflow.types import notifier -from taskflow.utils import deprecation from taskflow.utils import misc @@ -56,8 +56,8 @@ class Engine(object): return self._notifier @property - @deprecation.moved_property('atom_notifier', version="0.6", - removal_version="?") + @moves.moved_property('atom_notifier', version="0.6", + removal_version="?") def task_notifier(self): """The task notifier.""" return self._atom_notifier @@ -113,7 +113,5 @@ class Engine(object): # TODO(harlowja): remove in 0.7 or later... -EngineBase = deprecation.moved_inheritable_class(Engine, - 'EngineBase', __name__, - version="0.6", - removal_version="?") +EngineBase = moves.moved_class(Engine, 'EngineBase', __name__, + version="0.6", removal_version="?") diff --git a/taskflow/engines/helpers.py b/taskflow/engines/helpers.py index 3d8ccf55..43ea8b6e 100644 --- a/taskflow/engines/helpers.py +++ b/taskflow/engines/helpers.py @@ -18,6 +18,7 @@ import contextlib import itertools import traceback +from debtcollector import renames from oslo_utils import importutils from oslo_utils import reflection import six @@ -26,7 +27,6 @@ import stevedore.driver from taskflow import exceptions as exc from taskflow import logging from taskflow.persistence import backends as p_backends -from taskflow.utils import deprecation from taskflow.utils import misc from taskflow.utils import persistence_utils as p_utils @@ -90,14 +90,14 @@ def _extract_engine(**kwargs): lambda frame: frame[0] in _FILE_NAMES, reversed(traceback.extract_stack(limit=3))) stacklevel = sum(1 for _frame in finder) - decorator = deprecation.renamed_kwarg('engine_conf', 'engine', - version="0.6", - removal_version="?", - # Three is added on since the - # decorator adds three of its own - # stack levels that we need to - # hop out of... - stacklevel=stacklevel + 3) + decorator = renames.renamed_kwarg('engine_conf', 'engine', + version="0.6", + removal_version="?", + # Three is added on since the + # decorator adds three of its own + # stack levels that we need to + # hop out of... + stacklevel=stacklevel + 3) return decorator(_compat_extract)(**kwargs) else: return _compat_extract(**kwargs) diff --git a/taskflow/listeners/base.py b/taskflow/listeners/base.py index 4884d90e..75b0db09 100644 --- a/taskflow/listeners/base.py +++ b/taskflow/listeners/base.py @@ -18,6 +18,7 @@ from __future__ import absolute_import import abc +from debtcollector import moves from oslo_utils import excutils import six @@ -25,7 +26,6 @@ from taskflow import logging from taskflow import states from taskflow.types import failure from taskflow.types import notifier -from taskflow.utils import deprecation LOG = logging.getLogger(__name__) @@ -165,10 +165,8 @@ class Listener(object): # TODO(harlowja): remove in 0.7 or later... -ListenerBase = deprecation.moved_inheritable_class(Listener, - 'ListenerBase', __name__, - version="0.6", - removal_version="?") +ListenerBase = moves.moved_class(Listener, 'ListenerBase', __name__, + version="0.6", removal_version="?") @six.add_metaclass(abc.ABCMeta) @@ -213,10 +211,9 @@ class DumpingListener(Listener): # TODO(harlowja): remove in 0.7 or later... -class LoggingBase(deprecation.moved_inheritable_class(DumpingListener, - 'LoggingBase', __name__, - version="0.6", - removal_version="?")): +class LoggingBase(moves.moved_class(DumpingListener, + 'LoggingBase', __name__, + version="0.6", removal_version="?")): def _dump(self, message, *args, **kwargs): self._log(message, *args, **kwargs) diff --git a/taskflow/utils/deprecation.py b/taskflow/utils/deprecation.py index 55ed982d..8bb48b27 100644 --- a/taskflow/utils/deprecation.py +++ b/taskflow/utils/deprecation.py @@ -14,17 +14,11 @@ # License for the specific language governing permissions and limitations # under the License. -import functools import warnings from oslo_utils import reflection import six -_CLASS_MOVED_PREFIX_TPL = "Class '%s' has moved to '%s'" -_KIND_MOVED_PREFIX_TPL = "%s '%s' has moved to '%s'" -_KWARG_MOVED_POSTFIX_TPL = ", please use the '%s' argument instead" -_KWARG_MOVED_PREFIX_TPL = "Using the '%s' argument is deprecated" - def deprecation(message, stacklevel=None): """Warns about some type of deprecation that has been (or will be) made. @@ -137,34 +131,11 @@ def _generate_message(prefix, postfix=None, message=None, return ''.join(message_components) -def renamed_kwarg(old_name, new_name, message=None, - version=None, removal_version=None, stacklevel=3): - """Decorates a kwarg accepting function to deprecate a renamed kwarg.""" - - prefix = _KWARG_MOVED_PREFIX_TPL % old_name - postfix = _KWARG_MOVED_POSTFIX_TPL % new_name - out_message = _generate_message(prefix, postfix=postfix, - message=message, version=version, - removal_version=removal_version) - - def decorator(f): - - @six.wraps(f) - def wrapper(*args, **kwargs): - if old_name in kwargs: - deprecation(out_message, stacklevel=stacklevel) - return f(*args, **kwargs) - - return wrapper - - return decorator - - def removed_kwarg(old_name, message=None, version=None, removal_version=None, stacklevel=3): """Decorates a kwarg accepting function to deprecate a removed kwarg.""" - prefix = _KWARG_MOVED_PREFIX_TPL % old_name + prefix = "Using the '%s' argument is deprecated" % old_name out_message = _generate_message(prefix, postfix=None, message=message, version=version, removal_version=removal_version) @@ -182,41 +153,9 @@ def removed_kwarg(old_name, message=None, return decorator -def _moved_decorator(kind, new_attribute_name, message=None, - version=None, removal_version=None, - stacklevel=3): - """Decorates a method/property that was moved to another location.""" - - def decorator(f): - try: - old_attribute_name = f.__qualname__ - fully_qualified = True - except AttributeError: - old_attribute_name = f.__name__ - fully_qualified = False - - @six.wraps(f) - def wrapper(self, *args, **kwargs): - base_name = reflection.get_class_name(self, fully_qualified=False) - if fully_qualified: - old_name = old_attribute_name - else: - old_name = ".".join((base_name, old_attribute_name)) - new_name = ".".join((base_name, new_attribute_name)) - prefix = _KIND_MOVED_PREFIX_TPL % (kind, old_name, new_name) - out_message = _generate_message( - prefix, message=message, - version=version, removal_version=removal_version) - deprecation(out_message, stacklevel=stacklevel) - return f(self, *args, **kwargs) - - return wrapper - - return decorator - - def removed_module(module_name, replacement_name=None, message=None, version=None, removal_version=None, stacklevel=4): + """Deprecates a module that will be removed/replaced in the future.""" prefix = "The '%s' module usage is deprecated" % module_name if replacement_name: postfix = ", please use %s instead" % replacement_name @@ -229,56 +168,9 @@ def removed_module(module_name, replacement_name=None, message=None, deprecation(out_message, stacklevel=stacklevel) -def moved_property(new_attribute_name, message=None, - version=None, removal_version=None, stacklevel=3): - """Decorates a *instance* property that was moved to another location.""" - - return _moved_decorator('Property', new_attribute_name, message=message, - version=version, removal_version=removal_version, - stacklevel=stacklevel) - - -def moved_inheritable_class(new_class, old_class_name, old_module_name, - message=None, version=None, removal_version=None): - """Deprecates a class that was moved to another location. - - NOTE(harlowja): this creates a new-old type that can be used for a - deprecation period that can be inherited from, the difference between this - and the ``moved_class`` deprecation function is that the proxy from that - function can not be inherited from (thus limiting its use for a more - particular usecase where inheritance is not needed). - - This will emit warnings when the old locations class is initialized, - telling where the new and improved location for the old class now is. - """ - old_name = ".".join((old_module_name, old_class_name)) - new_name = reflection.get_class_name(new_class) - prefix = _CLASS_MOVED_PREFIX_TPL % (old_name, new_name) - out_message = _generate_message(prefix, - message=message, version=version, - removal_version=removal_version) - - def decorator(f): - - # Use the older functools until the following is available: - # - # https://bitbucket.org/gutworth/six/issue/105 - - @functools.wraps(f, assigned=("__name__", "__doc__")) - def wrapper(self, *args, **kwargs): - deprecation(out_message, stacklevel=3) - return f(self, *args, **kwargs) - - return wrapper - - old_class = type(old_class_name, (new_class,), {}) - old_class.__module__ = old_module_name - old_class.__init__ = decorator(old_class.__init__) - return old_class - - -def moved_class(new_class, old_class_name, old_module_name, message=None, - version=None, removal_version=None, stacklevel=3): +def moved_proxy_class(new_class, old_class_name, old_module_name, + message=None, version=None, removal_version=None, + stacklevel=3): """Deprecates a class that was moved to another location. This will emit warnings when the old locations class is initialized, @@ -286,7 +178,7 @@ def moved_class(new_class, old_class_name, old_module_name, message=None, """ old_name = ".".join((old_module_name, old_class_name)) new_name = reflection.get_class_name(new_class) - prefix = _CLASS_MOVED_PREFIX_TPL % (old_name, new_name) + prefix = "Class '%s' has moved to '%s'" % (old_name, new_name) out_message = _generate_message(prefix, message=message, version=version, removal_version=removal_version) diff --git a/taskflow/utils/misc.py b/taskflow/utils/misc.py index b2e967d4..db99fd50 100644 --- a/taskflow/utils/misc.py +++ b/taskflow/utils/misc.py @@ -419,12 +419,14 @@ def ensure_tree(path): raise -Failure = deprecation.moved_class(failure.Failure, 'Failure', __name__, - version="0.6", removal_version="?") +Failure = deprecation.moved_proxy_class(failure.Failure, + 'Failure', __name__, + version="0.6", removal_version="?") -Notifier = deprecation.moved_class(notifier.Notifier, 'Notifier', __name__, - version="0.6", removal_version="?") +Notifier = deprecation.moved_proxy_class(notifier.Notifier, + 'Notifier', __name__, + version="0.6", removal_version="?") @contextlib.contextmanager From 37b72ec7b5333b580d842b4e4dc6f1dacdc351b5 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Tue, 17 Feb 2015 21:39:48 -0800 Subject: [PATCH 010/246] Use the enum library for the retry strategy enumerations Instead of having a set of string constants that are the retry strategy/decision result have it instead be an enumerator that can be more clearly associated in docs and in code that these values are explicitly the strategies that the retry code can use (making it more obvious to users of these constants what they are for). This also updates the retry docs to use this new linkable class when describing the various strategies that can be used/returned. Change-Id: I944e521562be26f5315d7553da8d5820fc284c49 --- doc/source/atoms.rst | 32 +++++++++++++++-------------- requirements-py2.txt | 3 +++ requirements-py3.txt | 3 +++ taskflow/retry.py | 46 ++++++++++++++++++++++++++++++++---------- taskflow/utils/misc.py | 12 +++++++++++ 5 files changed, 70 insertions(+), 26 deletions(-) diff --git a/doc/source/atoms.rst b/doc/source/atoms.rst index d29c553c..c54eb571 100644 --- a/doc/source/atoms.rst +++ b/doc/source/atoms.rst @@ -58,20 +58,11 @@ associated `scope`_). Currently derivatives of the :py:class:`retry ` base class must provide a :py:func:`~taskflow.retry.Retry.on_failure` method to -determine how a failure should be handled. +determine how a failure should be handled. The current enumeration(s) that can +be returned from the :py:func:`~taskflow.retry.Retry.on_failure` method +are defined in an enumeration class described here: -The current enumeration set that can be returned from this method is: - -* ``RETRY`` - retries the surrounding subflow (a retry object is associated - with a flow, which is typically converted into a graph hierarchy at - compilation time) again. - -* ``REVERT`` - reverts only the surrounding subflow but *consult* the - parent atom before doing this to determine if the parent retry object - provides a different reconciliation strategy (retry atoms can be nested, this - is possible since flows themselves can be nested). - -* ``REVERT_ALL`` - completely reverts a whole flow. +.. autoclass:: taskflow.retry.Decision To aid in the reconciliation process the :py:class:`retry ` base class also mandates @@ -177,7 +168,13 @@ Interfaces ========== .. automodule:: taskflow.task -.. automodule:: taskflow.retry +.. autoclass:: taskflow.retry.Retry +.. autoclass:: taskflow.retry.History +.. autoclass:: taskflow.retry.AlwaysRevert +.. autoclass:: taskflow.retry.AlwaysRevertAll +.. autoclass:: taskflow.retry.Times +.. autoclass:: taskflow.retry.ForEach +.. autoclass:: taskflow.retry.ParameterizedForEach Hierarchy ========= @@ -185,5 +182,10 @@ Hierarchy .. inheritance-diagram:: taskflow.atom taskflow.task - taskflow.retry + taskflow.retry.Retry + taskflow.retry.AlwaysRevert + taskflow.retry.AlwaysRevertAll + taskflow.retry.Times + taskflow.retry.ForEach + taskflow.retry.ParameterizedForEach :parts: 1 diff --git a/requirements-py2.txt b/requirements-py2.txt index 083caec0..a60be382 100644 --- a/requirements-py2.txt +++ b/requirements-py2.txt @@ -13,6 +13,9 @@ ordereddict # Python 2->3 compatibility library. six>=1.7.0 +# Enum library made for <= python 3.3 +enum34 + # Very nice graph library networkx>=1.8 diff --git a/requirements-py3.txt b/requirements-py3.txt index b04fc0af..a2ad2858 100644 --- a/requirements-py3.txt +++ b/requirements-py3.txt @@ -10,6 +10,9 @@ pbr>=0.6,!=0.7,<1.0 # Python 2->3 compatibility library. six>=1.7.0 +# Enum library made for <= python 3.3 +enum34 + # Very nice graph library networkx>=1.8 diff --git a/taskflow/retry.py b/taskflow/retry.py index 47ed8ca5..b7135a92 100644 --- a/taskflow/retry.py +++ b/taskflow/retry.py @@ -17,16 +17,38 @@ import abc +import enum import six from taskflow import atom from taskflow import exceptions as exc from taskflow.utils import misc -# Decision results. -REVERT = "REVERT" -REVERT_ALL = "REVERT_ALL" -RETRY = "RETRY" + +@enum.unique +class Decision(misc.StrEnum): + """Decision results/strategy enumeration.""" + + REVERT = "REVERT" + """Reverts only the surrounding/associated subflow. + + This strategy first consults the parent atom before reverting the + associated subflow to determine if the parent retry object provides a + different reconciliation strategy (if no parent retry object exists + then reverting will proceed, if one does exist the parent retry may + override this reconciliation strategy with its own). + """ + + #: Completely reverts the whole flow. + REVERT_ALL = "REVERT_ALL" + + #: Retries the surrounding/associated subflow again. + RETRY = "RETRY" + +# Retain these aliases for a number of releases... +REVERT = Decision.REVERT +REVERT_ALL = Decision.REVERT_ALL +RETRY = Decision.RETRY # Constants passed into revert/execute kwargs. # @@ -108,14 +130,16 @@ class Retry(atom.Atom): This abstract base class is used to inherit from and provide different strategies that will be activated upon execution failures. Since a retry - object is an atom it may also provide :meth:`.execute` and - :meth:`.revert` methods to alter the inputs of connected atoms (depending - on the desired strategy to be used this can be quite useful). + object is an atom it may also provide :meth:`~taskflow.retry.Retry.execute` + and :meth:`~taskflow.retry.Retry.revert` methods to alter the inputs of + connected atoms (depending on the desired strategy to be used this can be + quite useful). - NOTE(harlowja): the :meth:`.execute` and :meth:`.revert` and - :meth:`.on_failure` will automatically be given a ``history`` parameter, - which contains information about the past decisions and outcomes - that have occurred (if available). + NOTE(harlowja): the :meth:`~taskflow.retry.Retry.execute` and + :meth:`~taskflow.retry.Retry.revert` and + :meth:`~taskflow.retry.Retry.on_failure` will automatically be given + a ``history`` parameter, which contains information about the past + decisions and outcomes that have occurred (if available). """ default_provides = None diff --git a/taskflow/utils/misc.py b/taskflow/utils/misc.py index b2e967d4..daac3855 100644 --- a/taskflow/utils/misc.py +++ b/taskflow/utils/misc.py @@ -26,6 +26,7 @@ import threading import time import types +import enum from oslo_serialization import jsonutils from oslo_utils import importutils from oslo_utils import netutils @@ -57,6 +58,17 @@ _MONOTONIC_LOCATIONS = tuple([ ]) +class StrEnum(str, enum.Enum): + """An enumeration that is also a string and can be compared to strings.""" + + def __new__(cls, *args, **kwargs): + for a in args: + if not isinstance(a, str): + raise TypeError("Enumeration '%s' (%s) is not" + " a string" % (a, type(a).__name__)) + return super(StrEnum, cls).__new__(cls, *args, **kwargs) + + def find_monotonic(allow_time_time=False): """Tries to find a monotonic time providing function (and returns it).""" for import_str in _MONOTONIC_LOCATIONS: From 2a1ffb78825c69f1c8d4983d950e736e114af1bf Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Thu, 19 Feb 2015 21:44:22 -0800 Subject: [PATCH 011/246] Remove WBE experimental documentation note Since this engine is actually used and has worked out for folks it seems appropriate to remove the note that say it is experimental and just say that it still has some features that it needs and note that those are being worked on... Change-Id: I25403530069212af1a4131ba4a835f06ad7c31b1 --- doc/source/workers.rst | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/doc/source/workers.rst b/doc/source/workers.rst index 479d02a5..b081dc74 100644 --- a/doc/source/workers.rst +++ b/doc/source/workers.rst @@ -7,10 +7,9 @@ connected via `amqp`_ (or other supported `kombu`_ transports). .. note:: - This engine is under active development and is experimental but it is - usable and does work but is missing some features (please check the - `blueprint page`_ for known issues and plans) that will make it more - production ready. + This engine is under active development and is usable and **does** work + but is missing some features (please check the `blueprint page`_ for + known issues and plans) that will make it more production ready. .. _blueprint page: https://blueprints.launchpad.net/taskflow?searchtext=wbe From f14854ea6a71de8dbb8499fc0ddbbf2c32c5b768 Mon Sep 17 00:00:00 2001 From: OpenStack Proposal Bot Date: Fri, 20 Feb 2015 14:00:20 +0000 Subject: [PATCH 012/246] Updated from global requirements Change-Id: I041318042948fd10a4ec3e9f0ceb6894c3f883fa --- requirements-py2.txt | 2 +- requirements-py3.txt | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/requirements-py2.txt b/requirements-py2.txt index 083caec0..3f1f9729 100644 --- a/requirements-py2.txt +++ b/requirements-py2.txt @@ -11,7 +11,7 @@ pbr>=0.6,!=0.7,<1.0 ordereddict # Python 2->3 compatibility library. -six>=1.7.0 +six>=1.9.0 # Very nice graph library networkx>=1.8 diff --git a/requirements-py3.txt b/requirements-py3.txt index b04fc0af..6809a013 100644 --- a/requirements-py3.txt +++ b/requirements-py3.txt @@ -8,7 +8,7 @@ pbr>=0.6,!=0.7,<1.0 # Packages needed for using this library. # Python 2->3 compatibility library. -six>=1.7.0 +six>=1.9.0 # Very nice graph library networkx>=1.8 From b7eb26c546307879c93f5b9be8fbb7103d033c3c Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 11 Feb 2015 18:22:20 -0800 Subject: [PATCH 013/246] Chain exceptions correctly on py3.x In order to chain exceptions add a helper that can be used to chain exceptions automatically (when able) and use it in the various places we are already creating a new exception with a prior cause so that on py3.x the newly created exception has its 'cause' associated using the syntax available to do chaining in py3.x (which formats nicely as well in that python version). Change-Id: Iffddb27dbfe80816d6032e4b5532a0011ceedc95 --- taskflow/exceptions.py | 37 ++++++++++++- taskflow/tests/unit/test_exceptions.py | 74 ++++++++++++++++++++++++++ 2 files changed, 109 insertions(+), 2 deletions(-) create mode 100644 taskflow/tests/unit/test_exceptions.py diff --git a/taskflow/exceptions.py b/taskflow/exceptions.py index b2b44193..ae7baef4 100644 --- a/taskflow/exceptions.py +++ b/taskflow/exceptions.py @@ -15,17 +15,50 @@ # under the License. import os +import sys import traceback import six +def raise_with_cause(exc_cls, message, *args, **kwargs): + """Helper to raise + chain exceptions (when able) and associate a *cause*. + + NOTE(harlowja): Since in py3.x exceptions can be chained (due to + :pep:`3134`) we should try to raise the desired exception with the given + *cause* (or extract a *cause* from the current stack if able) so that the + exception formats nicely in old and new versions of python. Since py2.x + does **not** support exception chaining (or formatting) our root exception + class has a :py:meth:`~taskflow.exceptions.TaskFlowException.pformat` + method that can be used to get *similar* information instead (and this + function makes sure to retain the *cause* in that case as well so + that the :py:meth:`~taskflow.exceptions.TaskFlowException.pformat` method + shows them). + + :param exc_cls: the :py:class:`~taskflow.exceptions.TaskFlowException` + class to raise. + :param message: the text/str message that will be passed to + the exceptions constructor as its first positional + argument. + :param args: any additional positional arguments to pass to the + exceptions constructor. + :param kwargs: any additional keyword arguments to pass to the + exceptions constructor. + """ + if 'cause' not in kwargs: + exc_type, exc, exc_tb = sys.exc_info() + if exc is not None: + kwargs['cause'] = exc + del(exc_type, exc, exc_tb) + six.raise_from(exc_cls(message, *args, **kwargs), kwargs.get('cause')) + + class TaskFlowException(Exception): """Base class for *most* exceptions emitted from this library. NOTE(harlowja): in later versions of python we can likely remove the need - to have a cause here as PY3+ have implemented PEP 3134 which handles - chaining in a much more elegant manner. + to have a ``cause`` here as PY3+ have implemented :pep:`3134` which + handles chaining in a much more elegant manner. :param message: the exception message, typically some string that is useful for consumers to view when debugging or analyzing diff --git a/taskflow/tests/unit/test_exceptions.py b/taskflow/tests/unit/test_exceptions.py new file mode 100644 index 00000000..d834ce7c --- /dev/null +++ b/taskflow/tests/unit/test_exceptions.py @@ -0,0 +1,74 @@ +# -*- coding: utf-8 -*- + +# Copyright (C) 2013 Yahoo! Inc. 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. + +import six +import testtools + +from taskflow import exceptions as exc +from taskflow import test + + +class TestExceptions(test.TestCase): + def test_cause(self): + capture = None + try: + raise exc.TaskFlowException("broken", cause=IOError("dead")) + except Exception as e: + capture = e + self.assertIsNotNone(capture) + self.assertIsInstance(capture, exc.TaskFlowException) + self.assertIsNotNone(capture.cause) + self.assertIsInstance(capture.cause, IOError) + + def test_cause_pformat(self): + capture = None + try: + raise exc.TaskFlowException("broken", cause=IOError("dead")) + except Exception as e: + capture = e + self.assertIsNotNone(capture) + self.assertGreater(0, len(capture.pformat())) + + def test_raise_with(self): + capture = None + try: + raise IOError('broken') + except Exception: + try: + exc.raise_with_cause(exc.TaskFlowException, 'broken') + except Exception as e: + capture = e + self.assertIsNotNone(capture) + self.assertIsInstance(capture, exc.TaskFlowException) + self.assertIsNotNone(capture.cause) + self.assertIsInstance(capture.cause, IOError) + + @testtools.skipIf(not six.PY3, 'py3.x is not available') + def test_raise_with_cause(self): + capture = None + try: + raise IOError('broken') + except Exception: + try: + exc.raise_with_cause(exc.TaskFlowException, 'broken') + except Exception as e: + capture = e + self.assertIsNotNone(capture) + self.assertIsInstance(capture, exc.TaskFlowException) + self.assertIsNotNone(capture.cause) + self.assertIsInstance(capture.cause, IOError) + self.assertIsNotNone(capture.__cause__) + self.assertIsInstance(capture.__cause__, IOError) From 9b5293f75326e920abc62bc6462deea8c46eb39e Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Fri, 20 Feb 2015 21:59:07 -0800 Subject: [PATCH 014/246] Switch the note about process pool executor to warning To make the comment about using a process pool executor standout more than it does change it to be a warning box instead of a note box. Change-Id: I2561a3b4e35f9e984f09afbc35b7d982dac27192 --- doc/source/engines.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/doc/source/engines.rst b/doc/source/engines.rst index 59a2869e..066db422 100644 --- a/doc/source/engines.rst +++ b/doc/source/engines.rst @@ -172,7 +172,7 @@ using your desired execution model. scalability by reducing thread/process creation and teardown as well as by reusing existing pools (which is a good practice in general). -.. note:: +.. warning:: Running tasks with a `process pool executor`_ is **experimentally** supported. This is mainly due to the `futures backport`_ and From 035b69d5df930d81f896bae7f855a71454b9d0c5 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Sat, 21 Feb 2015 08:58:35 -0800 Subject: [PATCH 015/246] Add pypi link badges Change-Id: I02db326074ee3557dcfe3f2593c657d0d2a324a9 --- README.rst | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/README.rst b/README.rst index c5e1f105..70ff715a 100644 --- a/README.rst +++ b/README.rst @@ -1,6 +1,14 @@ TaskFlow ======== +.. image:: https://pypip.in/version/taskflow/badge.svg + :target: https://pypi.python.org/pypi/taskflow/ + :alt: Latest Version + +.. image:: https://pypip.in/download/taskflow/badge.svg?period=month + :target: https://pypi.python.org/pypi/taskflow/ + :alt: Downloads + A library to do [jobs, tasks, flows] in a highly available, easy to understand and declarative manner (and more!) to be used with OpenStack and other projects. From 12d81c970f058c728051d5e3520cafcc15368a52 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Sat, 21 Feb 2015 19:44:45 -0800 Subject: [PATCH 016/246] Tweak some of the types thread safety docstrings Change-Id: I27f920007f5a311906257809eca26a60f8e9a386 --- taskflow/types/notifier.py | 10 +++++++++- taskflow/types/timing.py | 8 ++++---- 2 files changed, 13 insertions(+), 5 deletions(-) diff --git a/taskflow/types/notifier.py b/taskflow/types/notifier.py index 77dbbc0d..eaf7b02c 100644 --- a/taskflow/types/notifier.py +++ b/taskflow/types/notifier.py @@ -70,7 +70,8 @@ class _Listener(object): def __repr__(self): repr_msg = "%s object at 0x%x calling into '%r'" % ( - reflection.get_class_name(self), id(self), self._callback) + reflection.get_class_name(self, fully_qualified=False), + id(self), self._callback) if self._details_filter is not None: repr_msg += " using details filter '%r'" % self._details_filter return "<%s>" % repr_msg @@ -109,6 +110,13 @@ class Notifier(object): occurring as well as allow a entity to post said notifications to any associated subscribers without having either entity care about how this notification occurs. + + **Not** thread-safe when a single notifier is mutated at the same + time by multiple threads. For example having multiple threads call + into :py:meth:`.register` or :py:meth:`.reset` at the same time could + potentially end badly. It is thread-safe when + only :py:meth:`.notify` calls or other read-only actions (like calling + into :py:meth:`.is_registered`) are occuring at the same time. """ #: Keys that can *not* be used in callbacks arguments diff --git a/taskflow/types/timing.py b/taskflow/types/timing.py index e22b2da0..0dad971f 100644 --- a/taskflow/types/timing.py +++ b/taskflow/types/timing.py @@ -82,10 +82,10 @@ class StopWatch(object): Inspired by: apache-commons-lang java stopwatch. - Not thread-safe (when a single watch is mutated by multiple threads at - the same time). Thread-safe when used by a single thread (not shared) or - when operations are performed in a thread-safe manner on these objects by - wrapping those operations with locks. + **Not** thread-safe (when a single watch is mutated by multiple threads at + the same time). It is thread-safe when used by a single thread (not + shared) or when operations are performed in a thread-safe manner on these + objects by wrapping those operations with locks. """ _STARTED = 'STARTED' _STOPPED = 'STOPPED' From 1267a2a04d089f4d9e89297e60376c285042d460 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Sat, 21 Feb 2015 20:37:13 -0800 Subject: [PATCH 017/246] Bring over pretty_tox.sh from nova/heat/others Change-Id: Ie0373a30b6416eda26b9666c572f8c47089f7219 --- tools/pretty_tox.sh | 17 +++ tools/subunit_trace.py | 308 +++++++++++++++++++++++++++++++++++++++++ 2 files changed, 325 insertions(+) create mode 100755 tools/pretty_tox.sh create mode 100755 tools/subunit_trace.py diff --git a/tools/pretty_tox.sh b/tools/pretty_tox.sh new file mode 100755 index 00000000..5847e4e5 --- /dev/null +++ b/tools/pretty_tox.sh @@ -0,0 +1,17 @@ +#!/usr/bin/env bash + +set -o pipefail + +TESTRARGS=$1 + +# --until-failure is not compatible with --subunit see: +# +# https://bugs.launchpad.net/testrepository/+bug/1411804 +# +# this work around exists until that is addressed +if [[ "$TESTARGS" =~ "until-failure" ]]; then + python setup.py testr --slowest --testr-args="$TESTRARGS" +else + python setup.py testr --slowest --testr-args="--subunit $TESTRARGS" | $(dirname $0)/subunit_trace.py -f +fi + diff --git a/tools/subunit_trace.py b/tools/subunit_trace.py new file mode 100755 index 00000000..9265efaf --- /dev/null +++ b/tools/subunit_trace.py @@ -0,0 +1,308 @@ +#!/usr/bin/env python + +# Copyright 2014 Hewlett-Packard Development Company, L.P. +# Copyright 2014 Samsung Electronics +# 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. + +"""Trace a subunit stream in reasonable detail and high accuracy.""" + +import argparse +import functools +import os +import re +import sys + +import mimeparse +import subunit +import testtools + +DAY_SECONDS = 60 * 60 * 24 +FAILS = [] +RESULTS = {} + + +class Starts(testtools.StreamResult): + + def __init__(self, output): + super(Starts, self).__init__() + self._output = output + + def startTestRun(self): + self._neednewline = False + self._emitted = set() + + def status(self, test_id=None, test_status=None, test_tags=None, + runnable=True, file_name=None, file_bytes=None, eof=False, + mime_type=None, route_code=None, timestamp=None): + super(Starts, self).status( + test_id, test_status, + test_tags=test_tags, runnable=runnable, file_name=file_name, + file_bytes=file_bytes, eof=eof, mime_type=mime_type, + route_code=route_code, timestamp=timestamp) + if not test_id: + if not file_bytes: + return + if not mime_type or mime_type == 'test/plain;charset=utf8': + mime_type = 'text/plain; charset=utf-8' + primary, sub, parameters = mimeparse.parse_mime_type(mime_type) + content_type = testtools.content_type.ContentType( + primary, sub, parameters) + content = testtools.content.Content( + content_type, lambda: [file_bytes]) + text = content.as_text() + if text and text[-1] not in '\r\n': + self._neednewline = True + self._output.write(text) + elif test_status == 'inprogress' and test_id not in self._emitted: + if self._neednewline: + self._neednewline = False + self._output.write('\n') + worker = '' + for tag in test_tags or (): + if tag.startswith('worker-'): + worker = '(' + tag[7:] + ') ' + if timestamp: + timestr = timestamp.isoformat() + else: + timestr = '' + self._output.write('%s: %s%s [start]\n' % + (timestr, worker, test_id)) + self._emitted.add(test_id) + + +def cleanup_test_name(name, strip_tags=True, strip_scenarios=False): + """Clean up the test name for display. + + By default we strip out the tags in the test because they don't help us + in identifying the test that is run to it's result. + + Make it possible to strip out the testscenarios information (not to + be confused with tempest scenarios) however that's often needed to + indentify generated negative tests. + """ + if strip_tags: + tags_start = name.find('[') + tags_end = name.find(']') + if tags_start > 0 and tags_end > tags_start: + newname = name[:tags_start] + newname += name[tags_end + 1:] + name = newname + + if strip_scenarios: + tags_start = name.find('(') + tags_end = name.find(')') + if tags_start > 0 and tags_end > tags_start: + newname = name[:tags_start] + newname += name[tags_end + 1:] + name = newname + + return name + + +def get_duration(timestamps): + start, end = timestamps + if not start or not end: + duration = '' + else: + delta = end - start + duration = '%d.%06ds' % ( + delta.days * DAY_SECONDS + delta.seconds, delta.microseconds) + return duration + + +def find_worker(test): + for tag in test['tags']: + if tag.startswith('worker-'): + return int(tag[7:]) + return 'NaN' + + +# Print out stdout/stderr if it exists, always +def print_attachments(stream, test, all_channels=False): + """Print out subunit attachments. + + Print out subunit attachments that contain content. This + runs in 2 modes, one for successes where we print out just stdout + and stderr, and an override that dumps all the attachments. + """ + channels = ('stdout', 'stderr') + for name, detail in test['details'].items(): + # NOTE(sdague): the subunit names are a little crazy, and actually + # are in the form pythonlogging:'' (with the colon and quotes) + name = name.split(':')[0] + if detail.content_type.type == 'test': + detail.content_type.type = 'text' + if (all_channels or name in channels) and detail.as_text(): + title = "Captured %s:" % name + stream.write("\n%s\n%s\n" % (title, ('~' * len(title)))) + # indent attachment lines 4 spaces to make them visually + # offset + for line in detail.as_text().split('\n'): + stream.write(" %s\n" % line) + + +def show_outcome(stream, test, print_failures=False, failonly=False): + global RESULTS + status = test['status'] + # TODO(sdague): ask lifeless why on this? + if status == 'exists': + return + + worker = find_worker(test) + name = cleanup_test_name(test['id']) + duration = get_duration(test['timestamps']) + + if worker not in RESULTS: + RESULTS[worker] = [] + RESULTS[worker].append(test) + + # don't count the end of the return code as a fail + if name == 'process-returncode': + return + + if status == 'fail': + FAILS.append(test) + stream.write('{%s} %s [%s] ... FAILED\n' % ( + worker, name, duration)) + if not print_failures: + print_attachments(stream, test, all_channels=True) + elif not failonly: + if status == 'success': + stream.write('{%s} %s [%s] ... ok\n' % ( + worker, name, duration)) + print_attachments(stream, test) + elif status == 'skip': + stream.write('{%s} %s ... SKIPPED: %s\n' % ( + worker, name, test['details']['reason'].as_text())) + else: + stream.write('{%s} %s [%s] ... %s\n' % ( + worker, name, duration, test['status'])) + if not print_failures: + print_attachments(stream, test, all_channels=True) + + stream.flush() + + +def print_fails(stream): + """Print summary failure report. + + Currently unused, however there remains debate on inline vs. at end + reporting, so leave the utility function for later use. + """ + if not FAILS: + return + stream.write("\n==============================\n") + stream.write("Failed %s tests - output below:" % len(FAILS)) + stream.write("\n==============================\n") + for f in FAILS: + stream.write("\n%s\n" % f['id']) + stream.write("%s\n" % ('-' * len(f['id']))) + print_attachments(stream, f, all_channels=True) + stream.write('\n') + + +def count_tests(key, value): + count = 0 + for k, v in RESULTS.items(): + for item in v: + if key in item: + if re.search(value, item[key]): + count += 1 + return count + + +def run_time(): + runtime = 0.0 + for k, v in RESULTS.items(): + for test in v: + runtime += float(get_duration(test['timestamps']).strip('s')) + return runtime + + +def worker_stats(worker): + tests = RESULTS[worker] + num_tests = len(tests) + delta = tests[-1]['timestamps'][1] - tests[0]['timestamps'][0] + return num_tests, delta + + +def print_summary(stream): + stream.write("\n======\nTotals\n======\n") + stream.write("Run: %s in %s sec.\n" % (count_tests('status', '.*'), + run_time())) + stream.write(" - Passed: %s\n" % count_tests('status', 'success')) + stream.write(" - Skipped: %s\n" % count_tests('status', 'skip')) + stream.write(" - Failed: %s\n" % count_tests('status', 'fail')) + + # we could have no results, especially as we filter out the process-codes + if RESULTS: + stream.write("\n==============\nWorker Balance\n==============\n") + + for w in range(max(RESULTS.keys()) + 1): + if w not in RESULTS: + stream.write( + " - WARNING: missing Worker %s! " + "Race in testr accounting.\n" % w) + else: + num, time = worker_stats(w) + stream.write(" - Worker %s (%s tests) => %ss\n" % + (w, num, time)) + + +def parse_args(): + parser = argparse.ArgumentParser() + parser.add_argument('--no-failure-debug', '-n', action='store_true', + dest='print_failures', help='Disable printing failure ' + 'debug information in realtime') + parser.add_argument('--fails', '-f', action='store_true', + dest='post_fails', help='Print failure debug ' + 'information after the stream is proccesed') + parser.add_argument('--failonly', action='store_true', + dest='failonly', help="Don't print success items", + default=( + os.environ.get('TRACE_FAILONLY', False) + is not False)) + return parser.parse_args() + + +def main(): + args = parse_args() + stream = subunit.ByteStreamToStreamResult( + sys.stdin, non_subunit_name='stdout') + starts = Starts(sys.stdout) + outcomes = testtools.StreamToDict( + functools.partial(show_outcome, sys.stdout, + print_failures=args.print_failures, + failonly=args.failonly + )) + summary = testtools.StreamSummary() + result = testtools.CopyStreamResult([starts, outcomes, summary]) + result.startTestRun() + try: + stream.run(result) + finally: + result.stopTestRun() + if count_tests('status', '.*') == 0: + print("The test run didn't actually run any tests") + return 1 + if args.post_fails: + print_fails(sys.stdout) + print_summary(sys.stdout) + return (0 if summary.wasSuccessful() else 1) + + +if __name__ == '__main__': + sys.exit(main()) + From a44bc5e958cd6d098b938aeff76adbdab08b6d74 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Sun, 22 Feb 2015 13:16:14 -0800 Subject: [PATCH 018/246] Use oslo.utils encodeutils for encode/decode functions Instead of having similar logic that is nearly the same as what oslo.utils provides just call into the oslo.utils functionality and use it instead. This also allows us to pass a 'errors' keyword parameter and have it be used in oslo.utils to be used when some kind of encoding or decoding issue occurs (by default this is 'strict', as that is the default used by oslo.utils and python itself). Change-Id: Ia73340da7b47e1d450aa4ba41e3905d447691743 --- taskflow/utils/misc.py | 25 ++++++++++++------------- 1 file changed, 12 insertions(+), 13 deletions(-) diff --git a/taskflow/utils/misc.py b/taskflow/utils/misc.py index daac3855..28c4126f 100644 --- a/taskflow/utils/misc.py +++ b/taskflow/utils/misc.py @@ -28,6 +28,7 @@ import types import enum from oslo_serialization import jsonutils +from oslo_utils import encodeutils from oslo_utils import importutils from oslo_utils import netutils from oslo_utils import reflection @@ -210,30 +211,28 @@ def fix_newlines(text, replacement=os.linesep): return replacement.join(text.splitlines()) -def binary_encode(text, encoding='utf-8'): - """Converts a string of into a binary type using given encoding. +def binary_encode(text, encoding='utf-8', errors='strict'): + """Encodes a text string into a binary string using given encoding. - Does nothing if text not unicode string. + Does nothing if data is already a binary string (raises on unknown types). """ if isinstance(text, six.binary_type): return text - elif isinstance(text, six.text_type): - return text.encode(encoding) else: - raise TypeError("Expected binary or string type not '%s'" % type(text)) + return encodeutils.safe_encode(text, encoding=encoding, + errors=errors) -def binary_decode(data, encoding='utf-8'): - """Converts a binary type into a text type using given encoding. +def binary_decode(data, encoding='utf-8', errors='strict'): + """Decodes a binary string into a text string using given encoding. - Does nothing if data is already unicode string. + Does nothing if data is already a text string (raises on unknown types). """ - if isinstance(data, six.binary_type): - return data.decode(encoding) - elif isinstance(data, six.text_type): + if isinstance(data, six.text_type): return data else: - raise TypeError("Expected binary or string type not '%s'" % type(data)) + return encodeutils.safe_decode(data, incoming=encoding, + errors=errors) def decode_json(raw_data, root_types=(dict,)): From 7a1a467a7a3c35c79294ef89797992c0f71a7897 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Sat, 21 Feb 2015 17:04:46 -0800 Subject: [PATCH 019/246] Refactor parts of the periodic worker Perform some adjustments to the periodic worker type to make it easier to later change and make it better in the future. These changes hopefully also reduce complexity and increase understandability. In part this deprecates providing a tombstone via __init__() since that will restrict future enhancements we can make (in retrospect we need more internal control over that argument & type to build better workers). Change-Id: I1965e157c303c2a45b9950e9f4a921c638f57fd1 --- taskflow/tests/unit/test_types.py | 6 +- taskflow/tests/unit/test_utils.py | 18 +++++ taskflow/types/periodic.py | 114 ++++++++++++++++++------------ taskflow/utils/misc.py | 6 ++ test-requirements.txt | 1 + 5 files changed, 95 insertions(+), 50 deletions(-) diff --git a/taskflow/tests/unit/test_types.py b/taskflow/tests/unit/test_types.py index 2151b9e1..5f750d20 100644 --- a/taskflow/tests/unit/test_types.py +++ b/taskflow/tests/unit/test_types.py @@ -533,23 +533,21 @@ class PeriodicTest(test.TestCase): def test_periodic_single(self): barrier = latch.Latch(5) capture = [] - tombstone = tu.Event() @periodic.periodic(0.01) def callee(): barrier.countdown() if barrier.needed == 0: - tombstone.set() + w.stop() capture.append(1) - w = periodic.PeriodicWorker([callee], tombstone=tombstone) + w = periodic.PeriodicWorker([callee]) t = tu.daemon_thread(target=w.start) t.start() t.join() self.assertEqual(0, barrier.needed) self.assertEqual(5, sum(capture)) - self.assertTrue(tombstone.is_set()) def test_immediate(self): capture = [] diff --git a/taskflow/tests/unit/test_utils.py b/taskflow/tests/unit/test_utils.py index 51467529..1477fe5a 100644 --- a/taskflow/tests/unit/test_utils.py +++ b/taskflow/tests/unit/test_utils.py @@ -17,9 +17,11 @@ import collections import inspect import random +import string import time import six +import testscenarios from taskflow import test from taskflow.utils import misc @@ -192,6 +194,22 @@ class TestSequenceMinus(test.TestCase): self.assertEqual(result, [2, 1]) +class TestReversedEnumerate(testscenarios.TestWithScenarios, test.TestCase): + scenarios = [ + ('ten', {'sample': [10, 9, 8, 7, 6, 5, 4, 3, 2, 1]}), + ('empty', {'sample': []}), + ('negative', {'sample': [-1, -2, -3]}), + ('one', {'sample': [1]}), + ('abc', {'sample': ['a', 'b', 'c']}), + ('ascii_letters', {'sample': list(string.ascii_letters)}), + ] + + def test_sample_equivalence(self): + expected = list(reversed(list(enumerate(self.sample)))) + actual = list(misc.reverse_enumerate(self.sample)) + self.assertEqual(expected, actual) + + class TestCountdownIter(test.TestCase): def test_expected_count(self): upper = 100 diff --git a/taskflow/types/periodic.py b/taskflow/types/periodic.py index 9237d9c5..1cce5dd0 100644 --- a/taskflow/types/periodic.py +++ b/taskflow/types/periodic.py @@ -21,6 +21,7 @@ from oslo_utils import reflection import six from taskflow import logging +from taskflow.utils import deprecation from taskflow.utils import misc from taskflow.utils import threading_utils as tu @@ -64,6 +65,51 @@ def periodic(spacing, run_immediately=True): return wrapper +class _Schedule(object): + """Internal heap-based structure that maintains the schedule/ordering.""" + + def __init__(self): + self._ordering = [] + + def push(self, next_run, index): + heapq.heappush(self._ordering, (next_run, index)) + + def push_next(self, cb, index, now=None): + if now is None: + now = _now() + self.push(now + cb._periodic_spacing, index) + + def __len__(self): + return len(self._ordering) + + def pop(self): + return heapq.heappop(self._ordering) + + +def _build(callables): + schedule = _Schedule() + now = None + immediates = [] + # Reverse order is used since these are later popped off (and to + # ensure the popping order is first -> last we need to append them + # in the opposite ordering last -> first). + for i, cb in misc.reverse_enumerate(callables): + if cb._periodic_run_immediately: + immediates.append(i) + else: + if now is None: + now = _now() + schedule.push_next(cb, i, now=now) + return immediates, schedule + + +def _safe_call(cb, kind): + try: + cb() + except Exception: + LOG.warn("Failed to call %s '%r'", kind, cb, exc_info=True) + + class PeriodicWorker(object): """Calls a collection of callables periodically (sleeping as needed...). @@ -96,54 +142,29 @@ class PeriodicWorker(object): callables.append(member) return cls(callables) + @deprecation.removed_kwarg('tombstone', version="0.8", removal_version="?") def __init__(self, callables, tombstone=None): if tombstone is None: self._tombstone = tu.Event() else: - # Allows someone to share an event (if they so want to...) self._tombstone = tombstone - almost_callables = list(callables) - for cb in almost_callables: + self._callables = [] + for cb in callables: if not six.callable(cb): raise ValueError("Periodic callback must be callable") for attr_name in _PERIODIC_ATTRS: if not hasattr(cb, attr_name): raise ValueError("Periodic callback missing required" " attribute '%s'" % attr_name) - self._callables = tuple((cb, reflection.get_callable_name(cb)) - for cb in almost_callables) - self._schedule = [] - now = _now() - for i, (cb, cb_name) in enumerate(self._callables): - spacing = cb._periodic_spacing - next_run = now + spacing - heapq.heappush(self._schedule, (next_run, i)) - self._immediates = self._fetch_immediates(self._callables) + if cb._periodic: + self._callables.append(cb) + self._immediates, self._schedule = _build(self._callables) def __len__(self): return len(self._callables) - @staticmethod - def _fetch_immediates(callables): - immediates = [] - # Reverse order is used since these are later popped off (and to - # ensure the popping order is first -> last we need to append them - # in the opposite ordering last -> first). - for (cb, cb_name) in reversed(callables): - if cb._periodic_run_immediately: - immediates.append((cb, cb_name)) - return immediates - - @staticmethod - def _safe_call(cb, cb_name, kind='periodic'): - try: - cb() - except Exception: - LOG.warn("Failed to call %s callable '%s'", - kind, cb_name, exc_info=True) - def start(self): - """Starts running (will not stop/return until the tombstone is set). + """Starts running (will not return until :py:meth:`.stop` is called). NOTE(harlowja): If this worker has no contained callables this raises a runtime error and does not run since it is impossible to periodically @@ -154,29 +175,30 @@ class PeriodicWorker(object): " without any callables") while not self._tombstone.is_set(): if self._immediates: - cb, cb_name = self._immediates.pop() - LOG.debug("Calling immediate callable '%s'", cb_name) - self._safe_call(cb, cb_name, kind='immediate') + # Run & schedule its next execution. + index = self._immediates.pop() + cb = self._callables[index] + LOG.blather("Calling immediate '%r'", cb) + _safe_call(cb, 'immediate') + self._schedule.push_next(cb, index) else: # Figure out when we should run next (by selecting the # minimum item from the heap, where the minimum should be # the callable that needs to run next and has the lowest # next desired run time). now = _now() - next_run, i = heapq.heappop(self._schedule) + next_run, index = self._schedule.pop() when_next = next_run - now if when_next <= 0: - cb, cb_name = self._callables[i] - spacing = cb._periodic_spacing - LOG.debug("Calling periodic callable '%s' (it runs every" - " %s seconds)", cb_name, spacing) - self._safe_call(cb, cb_name) - # Run again someday... - next_run = now + spacing - heapq.heappush(self._schedule, (next_run, i)) + # Run & schedule its next execution. + cb = self._callables[index] + LOG.blather("Calling periodic '%r' (it runs every" + " %s seconds)", cb, cb._periodic_spacing) + _safe_call(cb, 'periodic') + self._schedule.push_next(cb, index, now=now) else: # Gotta wait... - heapq.heappush(self._schedule, (next_run, i)) + self._schedule.push(next_run, index) self._tombstone.wait(when_next) def stop(self): @@ -186,4 +208,4 @@ class PeriodicWorker(object): def reset(self): """Resets the tombstone and re-queues up any immediate executions.""" self._tombstone.clear() - self._immediates = self._fetch_immediates(self._callables) + self._immediates, self._schedule = _build(self._callables) diff --git a/taskflow/utils/misc.py b/taskflow/utils/misc.py index daac3855..383b3e28 100644 --- a/taskflow/utils/misc.py +++ b/taskflow/utils/misc.py @@ -101,6 +101,12 @@ def countdown_iter(start_at, decr=1): start_at -= decr +def reverse_enumerate(items): + """Like reversed(enumerate(items)) but with less copying/cloning...""" + for i in countdown_iter(len(items)): + yield i - 1, items[i - 1] + + def merge_uri(uri, conf): """Merges a parsed uri into the given configuration dictionary. diff --git a/test-requirements.txt b/test-requirements.txt index dd4f36d0..8c9a83a6 100644 --- a/test-requirements.txt +++ b/test-requirements.txt @@ -6,6 +6,7 @@ hacking<0.11,>=0.10.0 oslotest>=1.2.0 # Apache-2.0 mock>=1.0 testtools>=0.9.36,!=1.2.0 +testscenarios>=0.4 # Used for testing the WBE engine. kombu>=2.5.0 From 1c03996e66af42e72c0925c777dd785830e5bf13 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Tue, 24 Feb 2015 20:13:29 -0800 Subject: [PATCH 020/246] Stick to one space after a period Change-Id: I72666a24883e5b42e1ef76d0146d3da40d376ff8 --- taskflow/conductors/backends/impl_blocking.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/taskflow/conductors/backends/impl_blocking.py b/taskflow/conductors/backends/impl_blocking.py index c53248ee..1f6a9ee6 100644 --- a/taskflow/conductors/backends/impl_blocking.py +++ b/taskflow/conductors/backends/impl_blocking.py @@ -164,9 +164,9 @@ class BlockingConductor(base.Conductor): def wait(self, timeout=None): """Waits for the conductor to gracefully exit. - This method waits for the conductor to gracefully exit. An optional + This method waits for the conductor to gracefully exit. An optional timeout can be provided, which will cause the method to return - within the specified timeout. If the timeout is reached, the returned + within the specified timeout. If the timeout is reached, the returned value will be False. :param timeout: Maximum number of seconds that the :meth:`wait` method From 0a97fb96b5057bcde31b3373a444b531d8d9ee82 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Tue, 24 Feb 2015 16:19:04 -0800 Subject: [PATCH 021/246] Fix lookup scoping multi-match ordering When look-up is occurring the possible provider ordering is used instead of the scope returned provider ordering. This causes incorrect matches when look-up argument names that are produced by multiple providers (at the same scope level) all providing the same requirement name. The scope order should be enforced as the de-facto order and not the order that the storage unit finds (which is hash based and varies depending on hash ordering). Closes-Bug: #1425326 Change-Id: I15f1ee5515758bdc470c0f7dd7a2f616923e5628 --- taskflow/storage.py | 18 +++++----- taskflow/tests/unit/test_engines.py | 34 +++++++++++++++++++ taskflow/tests/unit/test_utils.py | 18 ++++++++++ .../tests/unit/worker_based/test_worker.py | 2 +- taskflow/tests/utils.py | 7 ++++ taskflow/utils/misc.py | 31 +++++++++++++++++ 6 files changed, 101 insertions(+), 9 deletions(-) diff --git a/taskflow/storage.py b/taskflow/storage.py index e96874ac..bcc4b382 100644 --- a/taskflow/storage.py +++ b/taskflow/storage.py @@ -657,15 +657,17 @@ class Storage(object): scope_iter = iter(scope_walker) else: scope_iter = iter([]) - for atom_names in scope_iter: - if not atom_names: - continue - providers = [] - for p in possible_providers: - if p.name in atom_names: - providers.append((p, _get_results(looking_for, p))) + extractor = lambda p: p.name + for names in scope_iter: + # *Always* retain the scope ordering (if any matches + # happen); instead of retaining the possible provider match + # order (which isn't that important and may be different from + # the scope requested ordering). + providers = misc.look_for(names, possible_providers, + extractor=extractor) if providers: - return providers + return [(p, _get_results(looking_for, p)) + for p in providers] return [] with self._lock.read_lock(): diff --git a/taskflow/tests/unit/test_engines.py b/taskflow/tests/unit/test_engines.py index 9176b9d6..04b8fa3a 100644 --- a/taskflow/tests/unit/test_engines.py +++ b/taskflow/tests/unit/test_engines.py @@ -156,6 +156,40 @@ class EngineLinearFlowTest(utils.EngineTestBase): engine = self._make_engine(flow) self.assertRaises(exc.Empty, engine.run) + def test_overlap_parent_sibling_expected_result(self): + flow = lf.Flow('flow-1') + flow.add(utils.ProgressingTask(provides='source')) + flow.add(utils.TaskOneReturn(provides='source')) + subflow = lf.Flow('flow-2') + subflow.add(utils.AddOne()) + flow.add(subflow) + engine = self._make_engine(flow) + engine.run() + results = engine.storage.fetch_all() + self.assertEqual(2, results['result']) + + def test_overlap_parent_expected_result(self): + flow = lf.Flow('flow-1') + flow.add(utils.ProgressingTask(provides='source')) + subflow = lf.Flow('flow-2') + subflow.add(utils.TaskOneReturn(provides='source')) + subflow.add(utils.AddOne()) + flow.add(subflow) + engine = self._make_engine(flow) + engine.run() + results = engine.storage.fetch_all() + self.assertEqual(2, results['result']) + + def test_overlap_sibling_expected_result(self): + flow = lf.Flow('flow-1') + flow.add(utils.ProgressingTask(provides='source')) + flow.add(utils.TaskOneReturn(provides='source')) + flow.add(utils.AddOne()) + engine = self._make_engine(flow) + engine.run() + results = engine.storage.fetch_all() + self.assertEqual(2, results['result']) + def test_sequential_flow_one_task(self): flow = lf.Flow('flow-1').add( utils.ProgressingTask(name='task1') diff --git a/taskflow/tests/unit/test_utils.py b/taskflow/tests/unit/test_utils.py index 1477fe5a..df229fac 100644 --- a/taskflow/tests/unit/test_utils.py +++ b/taskflow/tests/unit/test_utils.py @@ -244,6 +244,24 @@ class TestCountdownIter(test.TestCase): self.assertRaises(ValueError, six.next, it) +class TestLookFor(test.TestCase): + def test_no_matches(self): + hay = [9, 10, 11] + self.assertEqual([], misc.look_for(hay, [1, 2, 3])) + + def test_match_order(self): + hay = [6, 5, 4, 3, 2, 1] + priors = [] + for i in range(0, 6): + priors.append(i + 1) + matches = misc.look_for(hay, priors) + self.assertGreater(0, len(matches)) + self.assertIsSuperAndSubsequence(hay, matches) + hay = [10, 1, 15, 3, 5, 8, 44] + self.assertEqual([1, 15], misc.look_for(hay, [15, 1])) + self.assertEqual([10, 44], misc.look_for(hay, [44, 10])) + + class TestClamping(test.TestCase): def test_simple_clamp(self): result = misc.clamp(1.0, 2.0, 3.0) diff --git a/taskflow/tests/unit/worker_based/test_worker.py b/taskflow/tests/unit/worker_based/test_worker.py index 597a64a4..07095efa 100644 --- a/taskflow/tests/unit/worker_based/test_worker.py +++ b/taskflow/tests/unit/worker_based/test_worker.py @@ -34,7 +34,7 @@ class TestWorker(test.MockTestCase): self.exchange = 'test-exchange' self.topic = 'test-topic' self.threads_count = 5 - self.endpoint_count = 23 + self.endpoint_count = 24 # patch classes self.executor_mock, self.executor_inst_mock = self.patchClass( diff --git a/taskflow/tests/utils.py b/taskflow/tests/utils.py index 9e6e2a34..031dd706 100644 --- a/taskflow/tests/utils.py +++ b/taskflow/tests/utils.py @@ -89,6 +89,13 @@ class DummyTask(task.Task): pass +class AddOne(task.Task): + default_provides = 'result' + + def execute(self, source): + return source + 1 + + class FakeTask(object): def execute(self, **kwargs): diff --git a/taskflow/utils/misc.py b/taskflow/utils/misc.py index 39708b5c..6c5c9de7 100644 --- a/taskflow/utils/misc.py +++ b/taskflow/utils/misc.py @@ -196,6 +196,37 @@ def parse_uri(uri): return netutils.urlsplit(uri) +def look_for(haystack, needles, extractor=None): + """Find items in haystack and returns matches found (in haystack order). + + Given a list of items (the haystack) and a list of items to look for (the + needles) this will look for the needles in the haystack and returns + the found needles (if any). The ordering of the returned needles is in the + order they are located in the haystack. + + Example input and output: + + >>> from taskflow.utils import misc + >>> hay = [3, 2, 1] + >>> misc.look_for(hay, [1, 2]) + [2, 1] + """ + if not haystack: + return [] + if extractor is None: + extractor = lambda v: v + matches = [] + for i, v in enumerate(needles): + try: + matches.append((haystack.index(extractor(v)), i)) + except ValueError: + pass + if not matches: + return [] + else: + return [needles[i] for (_hay_i, i) in sorted(matches)] + + def clamp(value, minimum, maximum, on_clamped=None): """Clamps a value to ensure its >= minimum and <= maximum.""" if minimum > maximum: From 71c97e39b11a52fa73c0de6182feee4cc36f5d75 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Tue, 24 Feb 2015 23:05:04 -0800 Subject: [PATCH 022/246] Provide more contextual information about invalid periodics Instead of just telling the user of the periodic worker that certain callbacks were invalid tell the user which callback was invalid and exactly what attributes were expected but were not found. This makes the errors that much easier to diagnose and to resolve. Change-Id: Iaf8caf5e460e3ff4731ef575e924c9b86a9d5ae1 --- taskflow/types/periodic.py | 37 +++++++++++++++++++------------------ 1 file changed, 19 insertions(+), 18 deletions(-) diff --git a/taskflow/types/periodic.py b/taskflow/types/periodic.py index 1cce5dd0..8be3e10d 100644 --- a/taskflow/types/periodic.py +++ b/taskflow/types/periodic.py @@ -31,12 +31,17 @@ LOG = logging.getLogger(__name__) # which isn't *always* accurate but will suffice). _now = misc.find_monotonic(allow_time_time=True) -# Attributes expected on periodic tagged/decorated functions or methods... -_PERIODIC_ATTRS = tuple([ - '_periodic', - '_periodic_spacing', - '_periodic_run_immediately', -]) + +def _check_attrs(obj): + """Checks that a periodic function/method has all the expected attributes. + + This will return the expected attributes that were **not** found. + """ + missing_attrs = [] + for a in ('_periodic', '_periodic_spacing', '_periodic_run_immediately'): + if not hasattr(obj, a): + missing_attrs.append(a) + return missing_attrs def periodic(spacing, run_immediately=True): @@ -133,12 +138,8 @@ class PeriodicWorker(object): if name.startswith("_") and exclude_hidden: continue if reflection.is_bound_method(member): - consume = True - for attr_name in _PERIODIC_ATTRS: - if not hasattr(member, attr_name): - consume = False - break - if consume: + missing_attrs = _check_attrs(member) + if not missing_attrs: callables.append(member) return cls(callables) @@ -149,13 +150,13 @@ class PeriodicWorker(object): else: self._tombstone = tombstone self._callables = [] - for cb in callables: + for i, cb in enumerate(callables, 1): if not six.callable(cb): - raise ValueError("Periodic callback must be callable") - for attr_name in _PERIODIC_ATTRS: - if not hasattr(cb, attr_name): - raise ValueError("Periodic callback missing required" - " attribute '%s'" % attr_name) + raise ValueError("Periodic callback %s must be callable" % i) + missing_attrs = _check_attrs(cb) + if missing_attrs: + raise ValueError("Periodic callback %s missing required" + " attributes %s" % (i, missing_attrs)) if cb._periodic: self._callables.append(cb) self._immediates, self._schedule = _build(self._callables) From 517fa604c38ea259e7521a43fcc540120576e71c Mon Sep 17 00:00:00 2001 From: Greg Hill Date: Mon, 23 Feb 2015 15:40:43 -0600 Subject: [PATCH 023/246] add jobboard trash method This allows you to move a job to the trash so it will not be re-attempted, but while also leaving the details behind for diagnostic purposes. Change-Id: I3126e8d771e4012241a5fba1cd61c752f87c9952 Implements: blueprint jobboard-garbage-bin --- taskflow/jobs/backends/impl_zookeeper.py | 34 +++++++++++++++++++++++- taskflow/jobs/base.py | 19 +++++++++++++ taskflow/tests/unit/jobs/test_zk_job.py | 30 ++++++++++++++++++++- taskflow/utils/kazoo_utils.py | 22 +-------------- 4 files changed, 82 insertions(+), 23 deletions(-) diff --git a/taskflow/jobs/backends/impl_zookeeper.py b/taskflow/jobs/backends/impl_zookeeper.py index 87ccac63..10f529eb 100644 --- a/taskflow/jobs/backends/impl_zookeeper.py +++ b/taskflow/jobs/backends/impl_zookeeper.py @@ -51,6 +51,7 @@ ALL_JOB_STATES = ( # Transaction support was added in 3.4.0 MIN_ZK_VERSION = (3, 4, 0) LOCK_POSTFIX = ".lock" +TRASH_FOLDER = ".trash" JOB_PREFIX = 'job' @@ -79,7 +80,7 @@ class ZookeeperJob(base.Job): raise ValueError("Only one of 'book_data' or 'book'" " can be provided") self._path = k_paths.normpath(path) - self._lock_path = path + LOCK_POSTFIX + self._lock_path = self._path + LOCK_POSTFIX self._created_on = created_on self._node_not_found = False basename = k_paths.basename(self._path) @@ -330,6 +331,8 @@ class ZookeeperJobBoard(base.NotifyingJobBoard): if not k_paths.isabs(path): raise ValueError("Zookeeper path must be absolute") self._path = path + self._trash_path = self._path.replace(k_paths.basename(self._path), + TRASH_FOLDER) # The backend to load the full logbooks from, since whats sent over # the zookeeper data connection is only the logbook uuid and name, and # not currently the full logbook (later when a zookeeper backend @@ -362,6 +365,10 @@ class ZookeeperJobBoard(base.NotifyingJobBoard): def path(self): return self._path + @property + def trash_path(self): + return self._trash_path + @property def job_count(self): return len(self._known_jobs) @@ -656,6 +663,30 @@ class ZookeeperJobBoard(base.NotifyingJobBoard): txn.delete(job.lock_path, version=lock_stat.version) kazoo_utils.checked_commit(txn) + def trash(self, job, who): + _check_who(who) + with self._wrap(job.uuid, job.path, "Trash failure: %s"): + try: + owner_data = self._get_owner_and_data(job) + lock_data, lock_stat, data, data_stat = owner_data + except k_exceptions.NoNodeError: + raise excp.JobFailure("Can not trash a job %s" + " which we can not determine" + " the owner of" % (job.uuid)) + if lock_data.get("owner") != who: + raise excp.JobFailure("Can not trash a job %s" + " which is not owned by %s" + % (job.uuid, who)) + + trash_path = job.path.replace(self.path, self.trash_path) + value = misc.binary_encode(jsonutils.dumps(data)) + + txn = self._client.transaction() + txn.create(trash_path, value=value) + txn.delete(job.lock_path, version=lock_stat.version) + txn.delete(job.path, version=data_stat.version) + kazoo_utils.checked_commit(txn) + def _state_change_listener(self, state): LOG.debug("Kazoo client has changed to state: %s", state) @@ -725,6 +756,7 @@ class ZookeeperJobBoard(base.NotifyingJobBoard): if self._worker is None and self._emit_notifications: self._worker = futures.ThreadPoolExecutor(max_workers=1) self._client.ensure_path(self.path) + self._client.ensure_path(self.trash_path) if self._job_watcher is None: self._job_watcher = watchers.ChildrenWatch( self._client, diff --git a/taskflow/jobs/base.py b/taskflow/jobs/base.py index eea5b12b..1a5bcf2f 100644 --- a/taskflow/jobs/base.py +++ b/taskflow/jobs/base.py @@ -260,6 +260,25 @@ class JobBoard(object): this must be the same name that was used for claiming this job. """ + @abc.abstractmethod + def trash(self, job, who): + """Trash the provided job. + + Trashing a job signals to others that the job is broken and should not + be reclaimed. This is provided as an option for users to be able to + remove jobs from the board externally. The trashed job details should + be kept around in an alternate location to be reviewed, if desired. + + Only the entity that has claimed that job can trash a job. Any entity + trashing a unclaimed job (or a job they do not own) will cause an + exception. + + :param job: a job on this jobboard that can be trashed (if it does + not exist then a NotFound exception will be raised). + :param who: string that names the entity performing the trashing, + this must be the same name that was used for claiming this job. + """ + @abc.abstractproperty def connected(self): """Returns if this jobboard is connected.""" diff --git a/taskflow/tests/unit/jobs/test_zk_job.py b/taskflow/tests/unit/jobs/test_zk_job.py index afff4123..17385306 100644 --- a/taskflow/tests/unit/jobs/test_zk_job.py +++ b/taskflow/tests/unit/jobs/test_zk_job.py @@ -73,7 +73,7 @@ class ZakeJobboardTest(test.TestCase, base.BoardTestMixin): def setUp(self): super(ZakeJobboardTest, self).setUp() self.client, self.board = self._create_board() - self.bad_paths = [self.board.path] + self.bad_paths = [self.board.path, self.board.trash_path] self.bad_paths.extend(zake_utils.partition_path(self.board.path)) def test_posting_owner_lost(self): @@ -118,6 +118,34 @@ class ZakeJobboardTest(test.TestCase, base.BoardTestMixin): self.client.storage.pop(path) self.assertEqual(states.UNCLAIMED, j.state) + def test_trashing_claimed_job(self): + + with base.connect_close(self.board): + with base.flush(self.client): + j = self.board.post('test', p_utils.temporary_log_book()) + self.assertEqual(states.UNCLAIMED, j.state) + with base.flush(self.client): + self.board.claim(j, self.board.name) + self.assertEqual(states.CLAIMED, j.state) + + with base.flush(self.client): + self.board.trash(j, self.board.name) + + trashed = [] + jobs = [] + paths = list(six.iteritems(self.client.storage.paths)) + for (path, value) in paths: + if path in self.bad_paths: + continue + if path.find(impl_zookeeper.TRASH_FOLDER) > -1: + trashed.append(path) + elif (path.find(self.board._job_base) > -1 + and not path.endswith(impl_zookeeper.LOCK_POSTFIX)): + jobs.append(path) + + self.assertEqual(len(trashed), 1) + self.assertEqual(len(jobs), 0) + def test_posting_received_raw(self): book = p_utils.temporary_log_book() diff --git a/taskflow/utils/kazoo_utils.py b/taskflow/utils/kazoo_utils.py index c2869bdd..f681dc46 100644 --- a/taskflow/utils/kazoo_utils.py +++ b/taskflow/utils/kazoo_utils.py @@ -37,27 +37,7 @@ def _parse_hosts(hosts): def prettify_failures(failures, limit=-1): - """Prettifies a checked commits failures (ignores sensitive data...). - - Example input and output: - - >>> from taskflow.utils import kazoo_utils - >>> conf = {"hosts": ['localhost:2181']} - >>> c = kazoo_utils.make_client(conf) - >>> c.start(timeout=1) - >>> txn = c.transaction() - >>> txn.create("/test") - >>> txn.check("/test", 2) - >>> txn.delete("/test") - >>> try: - ... kazoo_utils.checked_commit(txn) - ... except kazoo_utils.KazooTransactionException as e: - ... print(kazoo_utils.prettify_failures(e.failures, limit=1)) - ... - RolledBackError@Create(path='/test') and 2 more... - >>> c.stop() - >>> c.close() - """ + """Prettifies a checked commits failures (ignores sensitive data...).""" prettier = [] for (op, r) in failures: pretty_op = reflection.get_class_name(op, fully_qualified=False) From 6da46b71d9ac7a3e5d4fbffd3d166e4484de434d Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Tue, 24 Feb 2015 16:19:04 -0800 Subject: [PATCH 024/246] Add specific scoping documentation Adds information into the arguments and result docs about how scoping lookup works and what it implies. Change-Id: I810874dce042ec43fe9e704d6689215e19d67c9c --- doc/source/engines.rst | 44 +++++++++++++++++++++++- taskflow/engines/action_engine/scopes.py | 38 ++++++++++++-------- taskflow/storage.py | 14 -------- 3 files changed, 66 insertions(+), 30 deletions(-) diff --git a/doc/source/engines.rst b/doc/source/engines.rst index 066db422..e2b85fbd 100644 --- a/doc/source/engines.rst +++ b/doc/source/engines.rst @@ -346,6 +346,47 @@ failures have occurred then the engine will have finished and if so desired the :doc:`persistence ` can be used to cleanup any details that were saved for this execution. +Scoping +======= + +During creation of flows it is also important to understand the lookup +strategy (also typically known as `scope`_ resolution) that the engine you +are using will internally use. For example when a task ``A`` provides +result 'a' and a task ``B`` after ``A`` provides a different result 'a' and a +task ``C`` after ``A`` and after ``B`` requires 'a' to run, which one will +be selected? + +Default strategy +---------------- + +When a engine is executing it internally interacts with the +:py:class:`~taskflow.storage.Storage` class +and that class interacts with the a +:py:class:`~taskflow.engines.action_engine.scopes.ScopeWalker` instance +and the :py:class:`~taskflow.storage.Storage` class uses the following +lookup order to find (or fail) a atoms requirement lookup/request: + +#. Injected atom specific arguments. +#. Transient injected arguments. +#. Non-transient injected arguments. +#. First scope visited provider that produces the named result; note that + if multiple providers are found in the same scope the *first* (the scope + walkers yielded ordering defines what *first* means) that produced that + result *and* can be extracted without raising an error is selected as the + provider of the requested requirement. +#. Fails with :py:class:`~taskflow.exceptions.NotFound` if unresolved at this + point (the ``cause`` attribute of this exception may have more details on + why the lookup failed). + +.. note:: + + To examine this this information when debugging it is recommended to + enable the ``BLATHER`` logging level (level 5). At this level the storage + and scope code/layers will log what is being searched for and what is + being found. + +.. _scope: http://en.wikipedia.org/wiki/Scope_%28computer_science%29 + Interfaces ========== @@ -362,7 +403,8 @@ Implementations .. automodule:: taskflow.engines.action_engine.runner .. automodule:: taskflow.engines.action_engine.runtime .. automodule:: taskflow.engines.action_engine.scheduler -.. automodule:: taskflow.engines.action_engine.scopes +.. autoclass:: taskflow.engines.action_engine.scopes.ScopeWalker + :special-members: __iter__ Hierarchy ========= diff --git a/taskflow/engines/action_engine/scopes.py b/taskflow/engines/action_engine/scopes.py index 6b7f9ffd..c55305d0 100644 --- a/taskflow/engines/action_engine/scopes.py +++ b/taskflow/engines/action_engine/scopes.py @@ -44,6 +44,8 @@ def _extract_atoms(node, idx=-1): class ScopeWalker(object): """Walks through the scopes of a atom using a engines compilation. + NOTE(harlowja): for internal usage only. + This will walk the visible scopes that are accessible for the given atom, which can be used by some external entity in some meaningful way, for example to find dependent values... @@ -63,29 +65,35 @@ class ScopeWalker(object): How this works is the following: - We find all the possible predecessors of the given atom, this is useful - since we know they occurred before this atom but it doesn't tell us - the corresponding scope *level* that each predecessor was created in, - so we need to find this information. + We first grab all the predecessors of the given atom (lets call it + ``Y``) by using the :py:class:`~.compiler.Compilation` execution + graph (and doing a reverse breadth-first expansion to gather its + predecessors), this is useful since we know they *always* will + exist (and execute) before this atom but it does not tell us the + corresponding scope *level* (flow, nested flow...) that each + predecessor was created in, so we need to find this information. For that information we consult the location of the atom ``Y`` in the - node hierarchy. We lookup in a reverse order the parent ``X`` of ``Y`` - and traverse backwards from the index in the parent where ``Y`` - occurred, all children in ``X`` that we encounter in this backwards - search (if a child is a flow itself, its atom contents will be - expanded) will be assumed to be at the same scope. This is then a - *potential* single scope, to make an *actual* scope we remove the items - from the *potential* scope that are not predecessors of ``Y`` to form - the *actual* scope. + :py:class:`~.compiler.Compilation` hierarchy/tree. We lookup in a + reverse order the parent ``X`` of ``Y`` and traverse backwards from + the index in the parent where ``Y`` exists to all siblings (and + children of those siblings) in ``X`` that we encounter in this + backwards search (if a sibling is a flow itself, its atom(s) + will be recursively expanded and included). This collection will + then be assumed to be at the same scope. This is what is called + a *potential* single scope, to make an *actual* scope we remove the + items from the *potential* scope that are **not** predecessors + of ``Y`` to form the *actual* scope which we then yield back. Then for additional scopes we continue up the tree, by finding the parent of ``X`` (lets call it ``Z``) and perform the same operation, going through the children in a reverse manner from the index in parent ``Z`` where ``X`` was located. This forms another *potential* scope which we provide back as an *actual* scope after reducing the - potential set by the predecessors of ``Y``. We then repeat this process - until we no longer have any parent nodes (aka have reached the top of - the tree) or we run out of predecessors. + potential set to only include predecessors previously gathered. We + then repeat this process until we no longer have any parent + nodes (aka we have reached the top of the tree) or we run out of + predecessors. """ predecessors = set(self._graph.bfs_predecessors_iter(self._atom)) last = self._node diff --git a/taskflow/storage.py b/taskflow/storage.py index bcc4b382..f367d05e 100644 --- a/taskflow/storage.py +++ b/taskflow/storage.py @@ -673,24 +673,10 @@ class Storage(object): with self._lock.read_lock(): if optional_args is None: optional_args = [] - if atom_name and atom_name not in self._atom_name_to_uuid: raise exceptions.NotFound("Unknown atom name: %s" % atom_name) if not args_mapping: return {} - - # The order of lookup is the following: - # - # 1. Injected atom specific arguments. - # 2. Transient injected arguments. - # 3. Non-transient injected arguments. - # 4. First scope visited group that produces the named result. - # a). The first of that group that actually provided the name - # result is selected (if group size is greater than one). - # - # Otherwise: blowup! (this will also happen if reading or - # extracting an expected result fails, since it is better to fail - # on lookup then provide invalid data from the wrong provider) if atom_name: injected_args = self._injected_args.get(atom_name, {}) else: From b598897d15b727d2fe7b32782d777a233d196b1d Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Mon, 2 Mar 2015 18:49:17 -0800 Subject: [PATCH 025/246] Tweak functor used to find flatteners/storage routines Make both of these finding functions use similar routines that the utility module now provides since the logic that both use can be shared. Change-Id: Ib941b99945d42f5c0d791e9b2a0696d0e62a2388 --- taskflow/engines/action_engine/compiler.py | 29 +++++-------------- taskflow/storage.py | 21 +++++++------- .../tests/unit/action_engine/test_compile.py | 4 +-- taskflow/tests/unit/test_retries.py | 4 +-- taskflow/utils/misc.py | 12 ++++++++ 5 files changed, 34 insertions(+), 36 deletions(-) diff --git a/taskflow/engines/action_engine/compiler.py b/taskflow/engines/action_engine/compiler.py index fb81ba80..2486b32e 100644 --- a/taskflow/engines/action_engine/compiler.py +++ b/taskflow/engines/action_engine/compiler.py @@ -20,7 +20,6 @@ import threading from taskflow import exceptions as exc from taskflow import flow from taskflow import logging -from taskflow import retry from taskflow import task from taskflow.types import graph as gr from taskflow.types import tree as tr @@ -281,34 +280,22 @@ class PatternCompiler(object): self._freeze = freeze self._lock = threading.Lock() self._compilation = None + self._flatten_matchers = [ + ((flow.Flow,), self._flatten_flow), + ((task.BaseTask,), self._flatten_task), + ] def _flatten(self, item, parent): """Flattens a item (pattern, task) into a graph + tree node.""" - functor = self._find_flattener(item, parent) + functor = misc.match_type_handler(item, self._flatten_matchers) + if not functor: + raise TypeError("Unknown item '%s' (%s) requested to flatten" + % (item, type(item))) self._pre_item_flatten(item) graph, node = functor(item, parent) self._post_item_flatten(item, graph, node) return graph, node - def _find_flattener(self, item, parent): - """Locates the flattening function to use to flatten the given item.""" - if isinstance(item, flow.Flow): - return self._flatten_flow - elif isinstance(item, task.BaseTask): - return self._flatten_task - elif isinstance(item, retry.Retry): - if parent is None: - raise TypeError("Retry controller '%s' (%s) must only be used" - " as a flow constructor parameter and not as a" - " root component" % (item, type(item))) - else: - raise TypeError("Retry controller '%s' (%s) must only be used" - " as a flow constructor parameter and not as a" - " flow added component" % (item, type(item))) - else: - raise TypeError("Unknown item '%s' (%s) requested to flatten" - % (item, type(item))) - def _connect_retry(self, retry, graph): graph.add_node(retry) diff --git a/taskflow/storage.py b/taskflow/storage.py index e96874ac..af29218b 100644 --- a/taskflow/storage.py +++ b/taskflow/storage.py @@ -132,6 +132,10 @@ class Storage(object): self._transients = {} self._injected_args = {} self._lock = lock_utils.ReaderWriterLock() + self._ensure_matchers = [ + ((task.BaseTask,), self._ensure_task), + ((retry.Retry,), self._ensure_retry), + ] # NOTE(imelnikov): failure serialization looses information, # so we cache failures here, in atom name -> failure mapping. @@ -168,17 +172,14 @@ class Storage(object): Returns uuid for the atomdetail that is/was created. """ - if isinstance(atom, task.BaseTask): - return self._ensure_task(atom.name, - misc.get_version_string(atom), - atom.save_as) - elif isinstance(atom, retry.Retry): - return self._ensure_retry(atom.name, - misc.get_version_string(atom), - atom.save_as) + functor = misc.match_type_handler(atom, self._ensure_matchers) + if not functor: + raise TypeError("Unknown item '%s' (%s) requested to ensure" + % (atom, type(atom))) else: - raise TypeError("Object of type 'atom' expected not" - " '%s' (%s)" % (atom, type(atom))) + return functor(atom.name, + misc.get_version_string(atom), + atom.save_as) def _ensure_task(self, task_name, task_version, result_mapping): """Ensures there is a taskdetail that corresponds to the task info. diff --git a/taskflow/tests/unit/action_engine/test_compile.py b/taskflow/tests/unit/action_engine/test_compile.py index a290c50b..445fd7c1 100644 --- a/taskflow/tests/unit/action_engine/test_compile.py +++ b/taskflow/tests/unit/action_engine/test_compile.py @@ -34,9 +34,7 @@ class PatternCompileTest(test.TestCase): def test_retry(self): r = retry.AlwaysRevert('r1') - msg_regex = "^Retry controller .* must only be used .*" - self.assertRaisesRegexp(TypeError, msg_regex, - compiler.PatternCompiler(r).compile) + self.assertRaises(TypeError, compiler.PatternCompiler(r).compile) def test_wrong_object(self): msg_regex = '^Unknown item .* requested to flatten' diff --git a/taskflow/tests/unit/test_retries.py b/taskflow/tests/unit/test_retries.py index b459184b..edcc6d8b 100644 --- a/taskflow/tests/unit/test_retries.py +++ b/taskflow/tests/unit/test_retries.py @@ -377,12 +377,12 @@ class RetryTest(utils.EngineTestBase): def test_run_just_retry(self): flow = utils.OneReturnRetry(provides='x') engine = self._make_engine(flow) - self.assertRaisesRegexp(TypeError, 'Retry controller', engine.run) + self.assertRaises(TypeError, engine.run) def test_use_retry_as_a_task(self): flow = lf.Flow('test').add(utils.OneReturnRetry(provides='x')) engine = self._make_engine(flow) - self.assertRaisesRegexp(TypeError, 'Retry controller', engine.run) + self.assertRaises(TypeError, engine.run) def test_resume_flow_that_had_been_interrupted_during_retrying(self): flow = lf.Flow('flow-1', retry.Times(3, 'r1')).add( diff --git a/taskflow/utils/misc.py b/taskflow/utils/misc.py index 39708b5c..a3bbe274 100644 --- a/taskflow/utils/misc.py +++ b/taskflow/utils/misc.py @@ -87,6 +87,18 @@ def find_monotonic(allow_time_time=False): return None +def match_type_handler(item, type_handlers): + """Matches a given items type using the given match types + handlers. + + Returns the handler if a type match occurs, otherwise none. + """ + for (match_types, handler_func) in type_handlers: + if isinstance(item, match_types): + return handler_func + else: + return None + + def countdown_iter(start_at, decr=1): """Generator that decrements after each generation until <= zero. From 3c806b1d6a6aee7825e022bbdda499fe1adca547 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Mon, 2 Mar 2015 14:20:14 -0800 Subject: [PATCH 026/246] Add a frozen checking decorator Since quite a few of the types check for being frozen and disallow mutations on there instances we can take advantage of a common decorator that checks the frozen attribute and raises instead of duplicating the same logic at the start of the mutating methods. Change-Id: I8c81a26d2d39bb9da4f68d64e07f67ac26ee0b08 --- taskflow/types/fsm.py | 10 ++++------ taskflow/types/tree.py | 9 +++++++-- taskflow/utils/misc.py | 17 +++++++++++++++++ 3 files changed, 28 insertions(+), 8 deletions(-) diff --git a/taskflow/types/fsm.py b/taskflow/types/fsm.py index df614767..afb4eb1d 100644 --- a/taskflow/types/fsm.py +++ b/taskflow/types/fsm.py @@ -23,6 +23,7 @@ import six from taskflow import exceptions as excp from taskflow.types import table +from taskflow.utils import misc class _Jump(object): @@ -97,6 +98,7 @@ class FSM(object): return False return self._states[self._current.name]['terminal'] + @misc.disallow_when_frozen(FrozenMachine) def add_state(self, state, terminal=False, on_enter=None, on_exit=None): """Adds a given state to the state machine. @@ -111,8 +113,6 @@ class FSM(object): :param state: state being entered or exited :type state: string """ - if self.frozen: - raise FrozenMachine() if state in self._states: raise excp.Duplicate("State '%s' already defined" % state) if on_enter is not None: @@ -129,6 +129,7 @@ class FSM(object): } self._transitions[state] = OrderedDict() + @misc.disallow_when_frozen(FrozenMachine) def add_reaction(self, state, event, reaction, *args, **kwargs): """Adds a reaction that may get triggered by the given event & state. @@ -149,8 +150,6 @@ class FSM(object): processed (and this process typically repeats) until the state machine reaches a terminal state. """ - if self.frozen: - raise FrozenMachine() if state not in self._states: raise excp.NotFound("Can not add a reaction to event '%s' for an" " undefined state '%s'" % (event, state)) @@ -162,6 +161,7 @@ class FSM(object): raise excp.Duplicate("State '%s' reaction to event '%s'" " already defined" % (state, event)) + @misc.disallow_when_frozen(FrozenMachine) def add_transition(self, start, end, event): """Adds an allowed transition from start -> end for the given event. @@ -169,8 +169,6 @@ class FSM(object): :param end: end of the transition :param event: event that caused the transition """ - if self.frozen: - raise FrozenMachine() if start not in self._states: raise excp.NotFound("Can not add a transition on event '%s' that" " starts in a undefined state '%s'" % (event, diff --git a/taskflow/types/tree.py b/taskflow/types/tree.py index c4273149..d5b4c12e 100644 --- a/taskflow/types/tree.py +++ b/taskflow/types/tree.py @@ -21,6 +21,8 @@ import os import six +from taskflow.utils import misc + class FrozenNode(Exception): """Exception raised when a frozen node is modified.""" @@ -98,9 +100,12 @@ class Node(object): n.freeze() self.frozen = True + @misc.disallow_when_frozen(FrozenNode) def add(self, child): - if self.frozen: - raise FrozenNode() + """Adds a child to this node (appends to left of existing children). + + NOTE(harlowja): this will also set the childs parent to be this node. + """ child.parent = self self._children.append(child) diff --git a/taskflow/utils/misc.py b/taskflow/utils/misc.py index 6c5c9de7..ec9eda80 100644 --- a/taskflow/utils/misc.py +++ b/taskflow/utils/misc.py @@ -227,6 +227,23 @@ def look_for(haystack, needles, extractor=None): return [needles[i] for (_hay_i, i) in sorted(matches)] +def disallow_when_frozen(excp_cls): + """Frozen checking/raising method decorator.""" + + def decorator(f): + + @six.wraps(f) + def wrapper(self, *args, **kwargs): + if self.frozen: + raise excp_cls() + else: + return f(self, *args, **kwargs) + + return wrapper + + return decorator + + def clamp(value, minimum, maximum, on_clamped=None): """Clamps a value to ensure its >= minimum and <= maximum.""" if minimum > maximum: From 53af5d322105f1d07e153d409a3e03c4b967f75a Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Tue, 3 Mar 2015 12:17:01 -0800 Subject: [PATCH 027/246] Allow node finding to not do a deep search When a local search is preferred allow for it to be done by passing in a 'only_direct' option, also allows passing in a 'include_self' option to ignore/include the current node when finding. The defaults of 'include_self' and 'only_direct' work just as they did prior to this change, so that the addition does not affect current usage(s) of this method. Change-Id: I89fb476bee32144092ea141ca3b971118ab633be --- taskflow/tests/unit/test_types.py | 10 ++++++++++ taskflow/types/tree.py | 15 +++++++++++++-- 2 files changed, 23 insertions(+), 2 deletions(-) diff --git a/taskflow/tests/unit/test_types.py b/taskflow/tests/unit/test_types.py index 5f750d20..7c05dd2c 100644 --- a/taskflow/tests/unit/test_types.py +++ b/taskflow/tests/unit/test_types.py @@ -136,6 +136,16 @@ class TreeTest(test.TestCase): root.freeze() self.assertRaises(tree.FrozenNode, root.add, "bird") + def test_find(self): + root = self._make_species() + self.assertIsNone(root.find('monkey', only_direct=True)) + self.assertIsNotNone(root.find('monkey', only_direct=False)) + self.assertIsNotNone(root.find('animal', only_direct=True)) + self.assertIsNotNone(root.find('reptile', only_direct=True)) + self.assertIsNone(root.find('animal', include_self=False)) + self.assertIsNone(root.find('animal', + include_self=False, only_direct=True)) + def test_dfs_itr(self): root = self._make_species() things = list([n.item for n in root.dfs_iter(include_self=True)]) diff --git a/taskflow/types/tree.py b/taskflow/types/tree.py index c4273149..8694c5b5 100644 --- a/taskflow/types/tree.py +++ b/taskflow/types/tree.py @@ -17,6 +17,7 @@ # under the License. import collections +import itertools import os import six @@ -118,7 +119,7 @@ class Node(object): yield node node = node.parent - def find(self, item): + def find(self, item, only_direct=False, include_self=True): """Returns the node for an item if it exists in this node. This will search not only this node but also any children nodes and @@ -126,9 +127,19 @@ class Node(object): object. :param item: item to lookup. + :param only_direct: only look at current node and its direct children. + :param include_self: include the current node during searching. + :returns: the node for an item if it exists in this node """ - for n in self.dfs_iter(include_self=True): + if only_direct: + if include_self: + it = itertools.chain([self], self.reverse_iter()) + else: + it = self.reverse_iter() + else: + it = self.dfs_iter(include_self=include_self) + for n in it: if n.item == item: return n return None From e3879bd72d0b9a91c0e06dea2826c15e1636ec3f Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Tue, 3 Mar 2015 15:50:43 -0800 Subject: [PATCH 028/246] Use compilation helper objects Instead of using instance methods to perform the compilation of tasks and flows instead move the logic of those functions into tiny helper classes that just perform the compilation of specific types they handle. This makes it more easy to document and understand how a flow or task type is compiled down into a graph and a tree node (which may or may not be attached to a parent tree node). Change-Id: Ib74edcad44556a897a4300132066e89d97739814 --- taskflow/engines/action_engine/compiler.py | 175 ++++++++++-------- .../tests/unit/action_engine/test_compile.py | 2 +- 2 files changed, 99 insertions(+), 78 deletions(-) diff --git a/taskflow/engines/action_engine/compiler.py b/taskflow/engines/action_engine/compiler.py index 2486b32e..23e75bf7 100644 --- a/taskflow/engines/action_engine/compiler.py +++ b/taskflow/engines/action_engine/compiler.py @@ -179,8 +179,74 @@ class Linker(object): priors.append((u, v)) +class _TaskCompiler(object): + """Non-recursive compiler of tasks.""" + + @staticmethod + def handles(obj): + return isinstance(obj, task.BaseTask) + + def compile(self, task, parent=None): + graph = gr.DiGraph(name=task.name) + graph.add_node(task) + node = tr.Node(task) + if parent is not None: + parent.add(node) + return graph, node + + +class _FlowCompiler(object): + """Recursive compiler of flows.""" + + @staticmethod + def handles(obj): + return isinstance(obj, flow.Flow) + + def __init__(self, deep_compiler_func, linker): + self._deep_compiler_func = deep_compiler_func + self._linker = linker + + def _connect_retry(self, retry, graph): + graph.add_node(retry) + + # All nodes that have no predecessors should depend on this retry. + nodes_to = [n for n in graph.no_predecessors_iter() if n is not retry] + if nodes_to: + _add_update_edges(graph, [retry], nodes_to, + attr_dict=_RETRY_EDGE_DATA) + + # Add association for each node of graph that has no existing retry. + for n in graph.nodes_iter(): + if n is not retry and flow.LINK_RETRY not in graph.node[n]: + graph.node[n][flow.LINK_RETRY] = retry + + def _decompose_flow(self, flow, parent=None): + """Decomposes a flow into a graph, tree node + decomposed subgraphs.""" + graph = gr.DiGraph(name=flow.name) + node = tr.Node(flow) + if parent is not None: + parent.add(node) + if flow.retry is not None: + node.add(tr.Node(flow.retry)) + decomposed_members = {} + for item in flow: + subgraph, _subnode = self._deep_compiler_func(item, parent=node) + decomposed_members[item] = subgraph + if subgraph.number_of_nodes(): + graph = gr.merge_graphs([graph, subgraph]) + return graph, node, decomposed_members + + def compile(self, flow, parent=None): + graph, node, decomposed_members = self._decompose_flow(flow, + parent=parent) + self._linker.apply_constraints(graph, flow, decomposed_members) + if flow.retry is not None: + self._connect_retry(flow.retry, graph) + return graph, node + + class PatternCompiler(object): - """Compiles a pattern (or task) into a compilation unit. + """Compiles a flow pattern (or task) into a compilation unit. Let's dive into the basic idea for how this works: @@ -188,9 +254,10 @@ class PatternCompiler(object): this object could be a task, or a flow (one of the supported patterns), the end-goal is to produce a :py:class:`.Compilation` object as the result with the needed components. If this is not possible a - :py:class:`~.taskflow.exceptions.CompilationFailure` will be raised (or - in the case where a unknown type is being requested to compile - a ``TypeError`` will be raised). + :py:class:`~.taskflow.exceptions.CompilationFailure` will be raised. + In the case where a **unknown** type is being requested to compile + a ``TypeError`` will be raised and when a duplicate object (one that + has **already** been compiled) is encountered a ``ValueError`` is raised. The complexity of this comes into play when the 'root' is a flow that contains itself other nested flows (and so-on); to compile this object and @@ -280,86 +347,40 @@ class PatternCompiler(object): self._freeze = freeze self._lock = threading.Lock() self._compilation = None - self._flatten_matchers = [ - ((flow.Flow,), self._flatten_flow), - ((task.BaseTask,), self._flatten_task), + self._matchers = [ + _FlowCompiler(self._compile, self._linker), + _TaskCompiler(), ] - def _flatten(self, item, parent): - """Flattens a item (pattern, task) into a graph + tree node.""" - functor = misc.match_type_handler(item, self._flatten_matchers) - if not functor: - raise TypeError("Unknown item '%s' (%s) requested to flatten" + def _compile(self, item, parent=None): + """Compiles a item (pattern, task) into a graph + tree node.""" + for m in self._matchers: + if m.handles(item): + self._pre_item_compile(item) + graph, node = m.compile(item, parent=parent) + self._post_item_compile(item, graph, node) + return graph, node + else: + raise TypeError("Unknown object '%s' (%s) requested to compile" % (item, type(item))) - self._pre_item_flatten(item) - graph, node = functor(item, parent) - self._post_item_flatten(item, graph, node) - return graph, node - def _connect_retry(self, retry, graph): - graph.add_node(retry) - - # All nodes that have no predecessors should depend on this retry. - nodes_to = [n for n in graph.no_predecessors_iter() if n is not retry] - if nodes_to: - _add_update_edges(graph, [retry], nodes_to, - attr_dict=_RETRY_EDGE_DATA) - - # Add association for each node of graph that has no existing retry. - for n in graph.nodes_iter(): - if n is not retry and flow.LINK_RETRY not in graph.node[n]: - graph.node[n][flow.LINK_RETRY] = retry - - def _flatten_task(self, task, parent): - """Flattens a individual task.""" - graph = gr.DiGraph(name=task.name) - graph.add_node(task) - node = tr.Node(task) - if parent is not None: - parent.add(node) - return graph, node - - def _decompose_flow(self, flow, parent): - """Decomposes a flow into a graph, tree node + decomposed subgraphs.""" - graph = gr.DiGraph(name=flow.name) - node = tr.Node(flow) - if parent is not None: - parent.add(node) - if flow.retry is not None: - node.add(tr.Node(flow.retry)) - decomposed_members = {} - for item in flow: - subgraph, _subnode = self._flatten(item, node) - decomposed_members[item] = subgraph - if subgraph.number_of_nodes(): - graph = gr.merge_graphs([graph, subgraph]) - return graph, node, decomposed_members - - def _flatten_flow(self, flow, parent): - """Flattens a flow.""" - graph, node, decomposed_members = self._decompose_flow(flow, parent) - self._linker.apply_constraints(graph, flow, decomposed_members) - if flow.retry is not None: - self._connect_retry(flow.retry, graph) - return graph, node - - def _pre_item_flatten(self, item): - """Called before a item is flattened; any pre-flattening actions.""" + def _pre_item_compile(self, item): + """Called before a item is compiled; any pre-compilation actions.""" if item in self._history: - raise ValueError("Already flattened item '%s' (%s), recursive" - " flattening is not supported" % (item, - type(item))) + raise ValueError("Already compiled item '%s' (%s), duplicate" + " and/or recursive compiling is not" + " supported" % (item, type(item))) self._history.add(item) - def _post_item_flatten(self, item, graph, node): - """Called after a item is flattened; doing post-flattening actions.""" + def _post_item_compile(self, item, graph, node): + """Called after a item is compiled; doing post-compilation actions.""" - def _pre_flatten(self): - """Called before the flattening of the root starts.""" + def _pre_compile(self): + """Called before the compilation of the root starts.""" self._history.clear() - def _post_flatten(self, graph, node): - """Called after the flattening of the root finishes successfully.""" + def _post_compile(self, graph, node): + """Called after the compilation of the root finishes successfully.""" dup_names = misc.get_duplicate_keys(graph.nodes_iter(), key=lambda node: node.name) if dup_names: @@ -387,9 +408,9 @@ class PatternCompiler(object): def compile(self): """Compiles the contained item into a compiled equivalent.""" if self._compilation is None: - self._pre_flatten() - graph, node = self._flatten(self._root, None) - self._post_flatten(graph, node) + self._pre_compile() + graph, node = self._compile(self._root, parent=None) + self._post_compile(graph, node) if self._freeze: graph.freeze() node.freeze() diff --git a/taskflow/tests/unit/action_engine/test_compile.py b/taskflow/tests/unit/action_engine/test_compile.py index 445fd7c1..884cd8d5 100644 --- a/taskflow/tests/unit/action_engine/test_compile.py +++ b/taskflow/tests/unit/action_engine/test_compile.py @@ -37,7 +37,7 @@ class PatternCompileTest(test.TestCase): self.assertRaises(TypeError, compiler.PatternCompiler(r).compile) def test_wrong_object(self): - msg_regex = '^Unknown item .* requested to flatten' + msg_regex = '^Unknown object .* requested to compile' self.assertRaisesRegexp(TypeError, msg_regex, compiler.PatternCompiler(42).compile) From bebc96034c3cd0e0ccc00e937c51b64e8d709d16 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Tue, 3 Mar 2015 16:25:09 -0800 Subject: [PATCH 029/246] Make the worker banner template part of the worker class Instead of having the banner template be part of the worker module, have it be part of the worker class so that anyone that desires to subclass and create new workers can alter the banner template more easily (if they so choose). Change-Id: Id56815ffd726f8952a74f9ab866b1bd006dcfcad --- taskflow/engines/worker_based/worker.py | 71 +++++++++++++------------ 1 file changed, 38 insertions(+), 33 deletions(-) diff --git a/taskflow/engines/worker_based/worker.py b/taskflow/engines/worker_based/worker.py index 2110b92b..5e9ff85e 100644 --- a/taskflow/engines/worker_based/worker.py +++ b/taskflow/engines/worker_based/worker.py @@ -31,36 +31,6 @@ from taskflow.utils import misc from taskflow.utils import threading_utils as tu from taskflow import version -BANNER_TEMPLATE = string.Template(""" -TaskFlow v${version} WBE worker. -Connection details: - Driver = $transport_driver - Exchange = $exchange - Topic = $topic - Transport = $transport_type - Uri = $connection_uri -Powered by: - Executor = $executor_type - Thread count = $executor_thread_count -Supported endpoints:$endpoints -System details: - Hostname = $hostname - Pid = $pid - Platform = $platform - Python = $python - Thread id = $thread_id -""".strip()) -BANNER_TEMPLATE.defaults = { - # These values may not be possible to fetch/known, default to unknown... - 'pid': '???', - 'hostname': '???', - 'executor_thread_count': '???', - 'endpoints': ' %s' % ([]), - # These are static (avoid refetching...) - 'version': version.version_string(), - 'python': sys.version.split("\n", 1)[0].strip(), -} - LOG = logging.getLogger(__name__) @@ -88,6 +58,39 @@ class Worker(object): (see: :py:attr:`~.proxy.Proxy.DEFAULT_RETRY_OPTIONS`) """ + BANNER_TEMPLATE = string.Template(""" +TaskFlow v${version} WBE worker. +Connection details: + Driver = $transport_driver + Exchange = $exchange + Topic = $topic + Transport = $transport_type + Uri = $connection_uri +Powered by: + Executor = $executor_type + Thread count = $executor_thread_count +Supported endpoints:$endpoints +System details: + Hostname = $hostname + Pid = $pid + Platform = $platform + Python = $python + Thread id = $thread_id +""".strip()) + + # See: http://bugs.python.org/issue13173 for why we are doing this... + BANNER_TEMPLATE.defaults = { + # These values may not be possible to fetch/known, default + # to ??? to represent that they are unknown... + 'pid': '???', + 'hostname': '???', + 'executor_thread_count': '???', + 'endpoints': ' %s' % ([]), + # These are static (avoid refetching...) + 'version': version.version_string(), + 'python': sys.version.split("\n", 1)[0].strip(), + } + def __init__(self, exchange, topic, tasks, executor=None, threads_count=None, url=None, transport=None, transport_options=None, @@ -119,7 +122,10 @@ class Worker(object): def _generate_banner(self): """Generates a banner that can be useful to display before running.""" - tpl_params = {} + try: + tpl_params = dict(self.BANNER_TEMPLATE.defaults) + except AttributeError: + tpl_params = {} connection_details = self._server.connection_details transport = connection_details.transport if transport.driver_version: @@ -151,8 +157,7 @@ class Worker(object): pass tpl_params['platform'] = platform.platform() tpl_params['thread_id'] = tu.get_ident() - banner = BANNER_TEMPLATE.substitute(BANNER_TEMPLATE.defaults, - **tpl_params) + banner = self.BANNER_TEMPLATE.substitute(**tpl_params) # NOTE(harlowja): this is needed since the template in this file # will always have newlines that end with '\n' (even on different # platforms due to the way this source file is encoded) so we have From 847d87db6a24641301884c4874bb31587f40dba5 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Sun, 1 Mar 2015 16:22:51 -0800 Subject: [PATCH 030/246] Add a fully functional orderedset Change-Id: I3f71aa86931695bf319b0c042b0beb988a8db437 --- doc/source/types.rst | 5 ++ taskflow/tests/unit/test_types.py | 136 ++++++++++++++++++++++++++++++ taskflow/types/sets.py | 135 +++++++++++++++++++++++++++++ 3 files changed, 276 insertions(+) create mode 100644 taskflow/types/sets.py diff --git a/doc/source/types.rst b/doc/source/types.rst index 47ba7e48..57e10986 100644 --- a/doc/source/types.rst +++ b/doc/source/types.rst @@ -49,6 +49,11 @@ Periodic .. automodule:: taskflow.types.periodic +Sets +==== + +.. automodule:: taskflow.types.sets + Table ===== diff --git a/taskflow/tests/unit/test_types.py b/taskflow/tests/unit/test_types.py index 5f750d20..c69d9ccb 100644 --- a/taskflow/tests/unit/test_types.py +++ b/taskflow/tests/unit/test_types.py @@ -25,6 +25,7 @@ from taskflow.types import fsm from taskflow.types import graph from taskflow.types import latch from taskflow.types import periodic +from taskflow.types import sets from taskflow.types import table from taskflow.types import timing as tt from taskflow.types import tree @@ -495,6 +496,141 @@ class FSMTest(test.TestCase): self.assertRaises(ValueError, m.add_state, 'b', on_exit=2) +class OrderedSetTest(test.TestCase): + + def test_retain_ordering(self): + items = [10, 9, 8, 7] + s = sets.OrderedSet(iter(items)) + self.assertEqual(items, list(s)) + + def test_retain_duplicate_ordering(self): + items = [10, 9, 10, 8, 9, 7, 8] + s = sets.OrderedSet(iter(items)) + self.assertEqual([10, 9, 8, 7], list(s)) + + def test_length(self): + items = [10, 9, 8, 7] + s = sets.OrderedSet(iter(items)) + self.assertEqual(4, len(s)) + + def test_duplicate_length(self): + items = [10, 9, 10, 8, 9, 7, 8] + s = sets.OrderedSet(iter(items)) + self.assertEqual(4, len(s)) + + def test_contains(self): + items = [10, 9, 8, 7] + s = sets.OrderedSet(iter(items)) + for i in items: + self.assertIn(i, s) + + def test_copy(self): + items = [10, 9, 8, 7] + s = sets.OrderedSet(iter(items)) + s2 = s.copy() + self.assertEqual(s, s2) + self.assertEqual(items, list(s2)) + + def test_empty_intersection(self): + s = sets.OrderedSet([1, 2, 3]) + + es = set(s) + + self.assertEqual(es.intersection(), s.intersection()) + + def test_intersection(self): + s = sets.OrderedSet([1, 2, 3]) + s2 = sets.OrderedSet([2, 3, 4, 5]) + + es = set(s) + es2 = set(s2) + + self.assertEqual(es.intersection(es2), s.intersection(s2)) + self.assertEqual(es2.intersection(s), s2.intersection(s)) + + def test_multi_intersection(self): + s = sets.OrderedSet([1, 2, 3]) + s2 = sets.OrderedSet([2, 3, 4, 5]) + s3 = sets.OrderedSet([1, 2]) + + es = set(s) + es2 = set(s2) + es3 = set(s3) + + self.assertEqual(es.intersection(s2, s3), s.intersection(s2, s3)) + self.assertEqual(es2.intersection(es3), s2.intersection(s3)) + + def test_superset(self): + s = sets.OrderedSet([1, 2, 3]) + s2 = sets.OrderedSet([2, 3]) + self.assertTrue(s.issuperset(s2)) + self.assertFalse(s.issubset(s2)) + + def test_subset(self): + s = sets.OrderedSet([1, 2, 3]) + s2 = sets.OrderedSet([2, 3]) + self.assertTrue(s2.issubset(s)) + self.assertFalse(s2.issuperset(s)) + + def test_empty_difference(self): + s = sets.OrderedSet([1, 2, 3]) + + es = set(s) + + self.assertEqual(es.difference(), s.difference()) + + def test_difference(self): + s = sets.OrderedSet([1, 2, 3]) + s2 = sets.OrderedSet([2, 3]) + + es = set(s) + es2 = set(s2) + + self.assertEqual(es.difference(es2), s.difference(s2)) + self.assertEqual(es2.difference(es), s2.difference(s)) + + def test_multi_difference(self): + s = sets.OrderedSet([1, 2, 3]) + s2 = sets.OrderedSet([2, 3]) + s3 = sets.OrderedSet([3, 4, 5]) + + es = set(s) + es2 = set(s2) + es3 = set(s3) + + self.assertEqual(es3.difference(es), s3.difference(s)) + self.assertEqual(es.difference(es3), s.difference(s3)) + self.assertEqual(es2.difference(es, es3), s2.difference(s, s3)) + + def test_empty_union(self): + s = sets.OrderedSet([1, 2, 3]) + + es = set(s) + + self.assertEqual(es.union(), s.union()) + + def test_union(self): + s = sets.OrderedSet([1, 2, 3]) + s2 = sets.OrderedSet([2, 3, 4]) + + es = set(s) + es2 = set(s2) + + self.assertEqual(es.union(es2), s.union(s2)) + self.assertEqual(es2.union(es), s2.union(s)) + + def test_multi_union(self): + s = sets.OrderedSet([1, 2, 3]) + s2 = sets.OrderedSet([2, 3, 4]) + s3 = sets.OrderedSet([4, 5, 6]) + + es = set(s) + es2 = set(s2) + es3 = set(s3) + + self.assertEqual(es.union(es2, es3), s.union(s2, s3)) + + class PeriodicTest(test.TestCase): def test_invalid_periodic(self): diff --git a/taskflow/types/sets.py b/taskflow/types/sets.py new file mode 100644 index 00000000..4ab99da5 --- /dev/null +++ b/taskflow/types/sets.py @@ -0,0 +1,135 @@ +# -*- coding: utf-8 -*- + +# Copyright (C) 2015 Yahoo! Inc. 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. + +import collections +import itertools + +try: + from collections import OrderedDict # noqa +except ImportError: + from ordereddict import OrderedDict # noqa + +import six + + +# Used for values that don't matter in sets backed by dicts... +_sentinel = object() + + +def _merge_in(target, iterable=None, sentinel=_sentinel): + """Merges iterable into the target and returns the target.""" + if iterable is not None: + for value in iterable: + target.setdefault(value, sentinel) + return target + + +class OrderedSet(collections.Set, collections.Hashable): + """A read-only hashable set that retains insertion/initial ordering. + + It should work in all existing places that ``frozenset`` is used. + + See: https://mail.python.org/pipermail/python-ideas/2009-May/004567.html + for an idea thread that *may* eventually (*someday*) result in this (or + similar) code being included in the mainline python codebase (although + the end result of that thread is somewhat discouraging in that regard). + """ + + __slots__ = ['_data'] + + def __init__(self, iterable=None): + self._data = _merge_in(OrderedDict(), iterable) + + def __hash__(self): + return self._hash() + + def __contains__(self, value): + return value in self._data + + def __len__(self): + return len(self._data) + + def __iter__(self): + for value in six.iterkeys(self._data): + yield value + + def __repr__(self): + return "%s(%s)" % (type(self).__name__, list(self)) + + def copy(self): + """Return a shallow copy of a set.""" + it = iter(self) + c = self._from_iterable(it) + return c + + def intersection(self, *sets): + """Return the intersection of two or more sets as a new set. + + (i.e. elements that are common to all of the sets.) + """ + def absorb_it(sets): + for value in iter(self): + matches = 0 + for s in sets: + if value in s: + matches += 1 + else: + break + if matches == len(sets): + yield value + it = absorb_it(sets) + c = self._from_iterable(it) + return c + + def issuperset(self, other): + """Report whether this set contains another set.""" + for value in other: + if value not in self: + return False + return True + + def issubset(self, other): + """Report whether another set contains this set.""" + for value in iter(self): + if value not in other: + return False + return True + + def difference(self, *sets): + """Return the difference of two or more sets as a new set. + + (i.e. all elements that are in this set but not the others.) + """ + def absorb_it(sets): + for value in iter(self): + seen = False + for s in sets: + if value in s: + seen = True + break + if not seen: + yield value + it = absorb_it(sets) + c = self._from_iterable(it) + return c + + def union(self, *sets): + """Return the union of sets as a new set. + + (i.e. all elements that are in either set.) + """ + it = itertools.chain(iter(self), *sets) + return self._from_iterable(it) From db22de960c095aa4c27cc0ad258149f2a0e2cf48 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 4 Mar 2015 11:49:10 -0800 Subject: [PATCH 031/246] Add node removal/disassociate functions When a node desires to be removed from its parent (or a parent wants to remove a child) it is quite useful to provide functions that do just this so that nodes can remove themselves or there children when this kind of usage is desired. Change-Id: I0071e9a7e15219e0cb7b92779e4f5a08596e5d34 --- taskflow/tests/unit/test_types.py | 38 +++++++++++++++++++++++++++++++ taskflow/types/tree.py | 38 +++++++++++++++++++++++++++++++ 2 files changed, 76 insertions(+) diff --git a/taskflow/tests/unit/test_types.py b/taskflow/tests/unit/test_types.py index 7c05dd2c..61ce0056 100644 --- a/taskflow/tests/unit/test_types.py +++ b/taskflow/tests/unit/test_types.py @@ -113,6 +113,44 @@ class TreeTest(test.TestCase): root = tree.Node("josh") self.assertTrue(root.empty()) + def test_removal(self): + root = self._make_species() + self.assertIsNotNone(root.remove('reptile')) + self.assertRaises(ValueError, root.remove, 'reptile') + self.assertIsNone(root.find('reptile')) + + def test_removal_direct(self): + root = self._make_species() + self.assertRaises(ValueError, root.remove, 'human', + only_direct=True) + + def test_removal_self(self): + root = self._make_species() + n = root.find('horse') + self.assertIsNotNone(n.parent) + n.remove('horse', include_self=True) + self.assertIsNone(n.parent) + self.assertIsNone(root.find('horse')) + + def test_disassociate(self): + root = self._make_species() + n = root.find('horse') + self.assertIsNotNone(n.parent) + c = n.disassociate() + self.assertEqual(1, c) + self.assertIsNone(n.parent) + self.assertIsNone(root.find('horse')) + + def test_disassociate_many(self): + root = self._make_species() + n = root.find('horse') + n.parent.add(n) + n.parent.add(n) + c = n.disassociate() + self.assertEqual(3, c) + self.assertIsNone(n.parent) + self.assertIsNone(root.find('horse')) + def test_not_empty(self): root = self._make_species() self.assertFalse(root.empty()) diff --git a/taskflow/types/tree.py b/taskflow/types/tree.py index 34d4ef11..97059d64 100644 --- a/taskflow/types/tree.py +++ b/taskflow/types/tree.py @@ -149,6 +149,44 @@ class Node(object): return n return None + def disassociate(self): + """Removes this node from its parent (if any). + + :returns: occurences of this node that were removed from its parent. + """ + occurrences = 0 + if self.parent is not None: + p = self.parent + self.parent = None + # Remove all instances of this node from its parent. + while True: + try: + p._children.remove(self) + except ValueError: + break + else: + occurrences += 1 + return occurrences + + def remove(self, item, only_direct=False, include_self=True): + """Removes a item from this nodes children. + + This will search not only this node but also any children nodes and + finally if nothing is found then a value error is raised instead of + the normally returned *removed* node object. + + :param item: item to lookup. + :param only_direct: only look at current node and its direct children. + :param include_self: include the current node during searching. + """ + node = self.find(item, only_direct=only_direct, + include_self=include_self) + if node is None: + raise ValueError("Item '%s' not found to remove" % item) + else: + node.disassociate() + return node + def __contains__(self, item): """Returns whether item exists in this node or this nodes children. From 0e2a567b9005fb38a1379028df8377ed2970e160 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 4 Mar 2015 16:37:37 -0800 Subject: [PATCH 032/246] Ensure ordered set is pickleable Since the existing set and frozen sets are pickleable so should this type so that it functions and matches how the the existing types work. Change-Id: I6994b868ecbf0428ab4bc8ab5c2f4486ad53fb49 --- taskflow/tests/unit/test_types.py | 10 ++++++++++ taskflow/types/sets.py | 6 ++++++ 2 files changed, 16 insertions(+) diff --git a/taskflow/tests/unit/test_types.py b/taskflow/tests/unit/test_types.py index c69d9ccb..cbbe2939 100644 --- a/taskflow/tests/unit/test_types.py +++ b/taskflow/tests/unit/test_types.py @@ -18,6 +18,7 @@ import time import networkx as nx import six +from six.moves import cPickle as pickle from taskflow import exceptions as excp from taskflow import test @@ -498,6 +499,15 @@ class FSMTest(test.TestCase): class OrderedSetTest(test.TestCase): + def test_pickleable(self): + items = [10, 9, 8, 7] + s = sets.OrderedSet(items) + self.assertEqual(items, list(s)) + s_bin = pickle.dumps(s) + s2 = pickle.loads(s_bin) + self.assertEqual(s, s2) + self.assertEqual(items, list(s2)) + def test_retain_ordering(self): items = [10, 9, 8, 7] s = sets.OrderedSet(iter(items)) diff --git a/taskflow/types/sets.py b/taskflow/types/sets.py index 4ab99da5..527ad2a7 100644 --- a/taskflow/types/sets.py +++ b/taskflow/types/sets.py @@ -66,6 +66,12 @@ class OrderedSet(collections.Set, collections.Hashable): for value in six.iterkeys(self._data): yield value + def __setstate__(self, items): + self.__init__(iterable=iter(items)) + + def __getstate__(self): + return tuple(self) + def __repr__(self): return "%s(%s)" % (type(self).__name__, list(self)) From c738693ca49798af6d140335757d12408b534341 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 4 Mar 2015 16:14:01 -0800 Subject: [PATCH 033/246] Add warning about transient arguments and worker-based-engines Make sure that the docstring for the inject methods has a big warning block (that will showup as a red-box in the generated docs) that makes sure people are aware of the side-effects of using transient values (that can't be serialized) with the usage of the wbe-engine. Change-Id: I99da0a45e7a57d0826693866952693708d2432ea --- taskflow/storage.py | 49 ++++++++++++++++++++++++++++++++++++++------- 1 file changed, 42 insertions(+), 7 deletions(-) diff --git a/taskflow/storage.py b/taskflow/storage.py index c4250e7c..a499826a 100644 --- a/taskflow/storage.py +++ b/taskflow/storage.py @@ -504,15 +504,34 @@ class Storage(object): self._with_connection(self._save_atom_detail, ad) def inject_atom_args(self, atom_name, pairs): - """Add *transient* values into storage for a specific atom only. + """Add **transient** values into storage for a specific atom only. This method injects a dictionary/pairs of arguments for an atom so that when that atom is scheduled for execution it will have immediate access to these arguments. - NOTE(harlowja): injected atom arguments take precedence over arguments - provided by predecessor atoms or arguments provided by injecting into - the flow scope (using the inject() method). + .. note:: + + Injected atom arguments take precedence over arguments + provided by predecessor atoms or arguments provided by injecting + into the flow scope (using + the :py:meth:`~taskflow.storage.Storage.inject` method). + + .. warning:: + + It should be noted that injected atom arguments (that are scoped + to the atom with the given name) *should* be serializable + whenever possible. This is a **requirement** for the + :doc:`worker based engine ` which **must** + serialize (typically using ``json``) all + atom :py:meth:`~taskflow.atom.Atom.execute` and + :py:meth:`~taskflow.atom.Atom.revert` arguments to + be able to transmit those arguments to the target worker(s). If + the use-case being applied/desired is to later use the worker + based engine then it is highly recommended to ensure all injected + atoms (even transient ones) are serializable to avoid issues + that *may* appear later (when a object turned out to not actually + be serializable). """ if atom_name not in self._atom_name_to_uuid: raise exceptions.NotFound("Unknown atom name: %s" % atom_name) @@ -524,11 +543,27 @@ class Storage(object): """Add values into storage. This method should be used to put flow parameters (requirements that - are not satisfied by any task in the flow) into storage. + are not satisfied by any atom in the flow) into storage. - :param: transient save the data in-memory only instead of persisting + :param transient: save the data in-memory only instead of persisting the data to backend storage (useful for resource-like objects - or similar objects which should *not* be persisted) + or similar objects which can **not** be persisted) + + .. warning:: + + It should be noted that injected flow arguments (that are scoped + to all atoms in this flow) *should* be serializable whenever + possible. This is a **requirement** for + the :doc:`worker based engine ` which **must** + serialize (typically using ``json``) all + atom :py:meth:`~taskflow.atom.Atom.execute` and + :py:meth:`~taskflow.atom.Atom.revert` arguments to + be able to transmit those arguments to the target worker(s). If + the use-case being applied/desired is to later use the worker + based engine then it is highly recommended to ensure all injected + atoms (even transient ones) are serializable to avoid issues + that *may* appear later (when a object turned out to not actually + be serializable). """ def save_persistent(): From 26dc9322a843ea9b0af16397cd907e54c6e58fef Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 4 Mar 2015 19:46:54 -0800 Subject: [PATCH 034/246] Ensure the thread bundle stops in last to first order Instead of stopping in first started to last started which is typically not desired (this is the starting order) we should do the reverse and stop in last to first instead (which is typically the expected stopping order) by default. Change-Id: Ic579438bc549d380c62c4d56c55c168de425adeb --- .../tests/unit/test_utils_threading_utils.py | 46 +++++++++++++++++++ taskflow/utils/threading_utils.py | 8 +++- 2 files changed, 52 insertions(+), 2 deletions(-) diff --git a/taskflow/tests/unit/test_utils_threading_utils.py b/taskflow/tests/unit/test_utils_threading_utils.py index 974285fa..66ef2d09 100644 --- a/taskflow/tests/unit/test_utils_threading_utils.py +++ b/taskflow/tests/unit/test_utils_threading_utils.py @@ -15,6 +15,7 @@ # under the License. import collections +import functools import time from taskflow import test @@ -83,6 +84,51 @@ class TestThreadBundle(test.TestCase): self.assertEqual(self.thread_count, self.bundle.stop()) self.assertEqual(self.thread_count, len(self.bundle)) + def test_start_stop_order(self): + start_events = collections.deque() + death_events = collections.deque() + + def before_start(i, t): + start_events.append((i, 'bs')) + + def before_join(i, t): + death_events.append((i, 'bj')) + self.death.set() + + def after_start(i, t): + start_events.append((i, 'as')) + + def after_join(i, t): + death_events.append((i, 'aj')) + + for i in range(0, self.thread_count): + self.bundle.bind(lambda: tu.daemon_thread(_spinner, self.death), + before_join=functools.partial(before_join, i), + after_join=functools.partial(after_join, i), + before_start=functools.partial(before_start, i), + after_start=functools.partial(after_start, i)) + self.assertEqual(self.thread_count, self.bundle.start()) + self.assertEqual(self.thread_count, len(self.bundle)) + self.assertEqual(self.thread_count, self.bundle.stop()) + self.assertEqual(0, self.bundle.stop()) + self.assertTrue(self.death.is_set()) + + expected_start_events = [] + for i in range(0, self.thread_count): + expected_start_events.extend([ + (i, 'bs'), (i, 'as'), + ]) + self.assertEqual(expected_start_events, list(start_events)) + + expected_death_events = [] + j = self.thread_count - 1 + for _i in range(0, self.thread_count): + expected_death_events.extend([ + (j, 'bj'), (j, 'aj'), + ]) + j -= 1 + self.assertEqual(expected_death_events, list(death_events)) + def test_start_stop(self): events = collections.deque() diff --git a/taskflow/utils/threading_utils.py b/taskflow/utils/threading_utils.py index cea0760d..1f3186bf 100644 --- a/taskflow/utils/threading_utils.py +++ b/taskflow/utils/threading_utils.py @@ -22,6 +22,8 @@ import threading import six from six.moves import _thread +from taskflow.utils import misc + if sys.version_info[0:2] == (2, 6): # This didn't return that was/wasn't set in 2.6, since we actually care @@ -137,7 +139,8 @@ class ThreadBundle(object): """Creates & starts all associated threads (that are not running).""" count = 0 with self._lock: - for i, (builder, thread, started) in enumerate(self._threads): + it = enumerate(self._threads) + for i, (builder, thread, started) in it: if thread and started: continue if not thread: @@ -157,7 +160,8 @@ class ThreadBundle(object): """Stops & joins all associated threads (that have been started).""" count = 0 with self._lock: - for i, (builder, thread, started) in enumerate(self._threads): + it = misc.reverse_enumerate(self._threads) + for i, (builder, thread, started) in it: if not thread or not started: continue self._trigger_callback(builder.before_join, thread) From 9500aee6005a78eca679817942944584f5e869a4 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 4 Mar 2015 17:19:50 -0800 Subject: [PATCH 035/246] Add a bookshelf developer section Instead of having the relevant links, papers, libraries, projects and such listed on the wiki move them to the developer docs (as these are more developer specific/centric). Change-Id: I6c20c0fcbd05bb7a803aa1ccf8294b60e020cb27 --- doc/source/index.rst | 15 +++++++++++ doc/source/shelf.rst | 60 ++++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 75 insertions(+) create mode 100644 doc/source/shelf.rst diff --git a/doc/source/index.rst b/doc/source/index.rst index 7ab0fedd..52b7c7a8 100644 --- a/doc/source/index.rst +++ b/doc/source/index.rst @@ -85,6 +85,21 @@ Miscellaneous types utils +Bookshelf +--------- + +A useful collection of *relevant* links, documents, papers, similar +projects, frameworks and libraries (and more). + +.. note:: + + Please feel free to submit your own additions and/or changes. + +.. toctree:: + :maxdepth: 1 + + shelf + Indices and tables ================== diff --git a/doc/source/shelf.rst b/doc/source/shelf.rst new file mode 100644 index 00000000..1262c033 --- /dev/null +++ b/doc/source/shelf.rst @@ -0,0 +1,60 @@ +Libraries & frameworks +---------------------- + +* `APScheduler`_ (Python) +* `Async`_ (Python) +* `Celery`_ (Python) +* `Graffiti`_ (Python) +* `JobLib`_ (Python) +* `Luigi`_ (Python) +* `Mesos`_ (C/C++) +* `Papy`_ (Python) +* `Parallel Python`_ (Python) +* `RQ`_ (Python) +* `Spiff`_ (Python) +* `TBB Flow`_ (C/C++) + +Languages +--------- + +* `Ani`_ +* `Make`_ +* `Plaid`_ + +Services +-------- + +* `Cloud Dataflow`_ +* `Mistral`_ + +Papers +------ + +* `Advances in Dataflow Programming Languages`_ + +Related paradigms +----------------- + +* `Dataflow programming`_ +* `Programming paradigm(s)`_ + +.. _APScheduler: http://pythonhosted.org/APScheduler/ +.. _Async: http://pypi.python.org/pypi/async +.. _Celery: http://www.celeryproject.org/ +.. _Graffiti: http://github.com/SegFaultAX/graffiti +.. _JobLib: http://pythonhosted.org/joblib/index.html +.. _Luigi: http://github.com/spotify/luigi +.. _RQ: http://python-rq.org/ +.. _Mistral: http://wiki.openstack.org/wiki/Mistral +.. _Mesos: http://mesos.apache.org/ +.. _Parallel Python: http://www.parallelpython.com/ +.. _Spiff: http://github.com/knipknap/SpiffWorkflow +.. _Papy: http://code.google.com/p/papy/ +.. _Make: http://www.gnu.org/software/make/ +.. _Ani: http://code.google.com/p/anic/ +.. _Programming paradigm(s): http://en.wikipedia.org/wiki/Programming_paradigm +.. _Plaid: http://www.cs.cmu.edu/~aldrich/plaid/ +.. _Advances in Dataflow Programming Languages: http://www.cs.ucf.edu/~dcm/Teaching/COT4810-Spring2011/Literature/DataFlowProgrammingLanguages.pdf +.. _Cloud Dataflow: https://cloud.google.com/dataflow/ +.. _TBB Flow: https://www.threadingbuildingblocks.org/tutorial-intel-tbb-flow-graph +.. _Dataflow programming: http://en.wikipedia.org/wiki/Dataflow_programming From 0806ac166655980005154962b2afef7c0a0e3660 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Thu, 5 Mar 2015 10:22:47 -0800 Subject: [PATCH 036/246] Rename the timing listeners to duration listeners These listeners really only capture duration (from when the task started, to when it stopped) and are not really associated with the time a task started or when it stopped so we should rename these to be more appropriatly named. Change-Id: I54ea0fa739fcea0b3e3c155dc38ce49adfc3bb01 --- doc/source/notifications.rst | 8 ++++---- taskflow/listeners/timing.py | 30 +++++++++++++++++++++--------- 2 files changed, 25 insertions(+), 13 deletions(-) diff --git a/doc/source/notifications.rst b/doc/source/notifications.rst index 6bb0ea6b..4e037c26 100644 --- a/doc/source/notifications.rst +++ b/doc/source/notifications.rst @@ -163,12 +163,12 @@ Printing and logging listeners .. autoclass:: taskflow.listeners.printing.PrintingListener -Timing listener ---------------- +Timing listeners +---------------- -.. autoclass:: taskflow.listeners.timing.TimingListener +.. autoclass:: taskflow.listeners.timing.DurationListener -.. autoclass:: taskflow.listeners.timing.PrintingTimingListener +.. autoclass:: taskflow.listeners.timing.PrintingDurationListener Claim listener -------------- diff --git a/taskflow/listeners/timing.py b/taskflow/listeners/timing.py index c0fab524..a75717a7 100644 --- a/taskflow/listeners/timing.py +++ b/taskflow/listeners/timing.py @@ -18,6 +18,8 @@ from __future__ import absolute_import import itertools +from debtcollector import moves + from taskflow import exceptions as exc from taskflow.listeners import base from taskflow import logging @@ -39,7 +41,7 @@ def _printer(message): print(message) -class TimingListener(base.Listener): +class DurationListener(base.Listener): """Listener that captures task duration. It records how long a task took to execute (or fail) @@ -47,9 +49,9 @@ class TimingListener(base.Listener): to task metadata with key ``'duration'``. """ def __init__(self, engine): - super(TimingListener, self).__init__(engine, - task_listen_for=WATCH_STATES, - flow_listen_for=[]) + super(DurationListener, self).__init__(engine, + task_listen_for=WATCH_STATES, + flow_listen_for=[]) self._timers = {} def deregister(self): @@ -86,22 +88,32 @@ class TimingListener(base.Listener): self._record_ending(timer, task_name) -class PrintingTimingListener(TimingListener): - """Listener that prints the start & stop timing as well as recording it.""" +TimingListener = moves.moved_class(DurationListener, + 'TimingListener', __name__, + version="0.8", removal_version="?") + + +class PrintingDurationListener(DurationListener): + """Listener that prints the duration as well as recording it.""" def __init__(self, engine, printer=None): - super(PrintingTimingListener, self).__init__(engine) + super(PrintingDurationListener, self).__init__(engine) if printer is None: self._printer = _printer else: self._printer = printer def _record_ending(self, timer, task_name): - super(PrintingTimingListener, self)._record_ending(timer, task_name) + super(PrintingDurationListener, self)._record_ending(timer, task_name) self._printer("It took task '%s' %0.2f seconds to" " finish." % (task_name, timer.elapsed())) def _task_receiver(self, state, details): - super(PrintingTimingListener, self)._task_receiver(state, details) + super(PrintingDurationListener, self)._task_receiver(state, details) if state in STARTING_STATES: self._printer("'%s' task started." % (details['task_name'])) + + +PrintingTimingListener = moves.moved_class( + PrintingDurationListener, 'PrintingTimingListener', __name__, + version="0.8", removal_version="?") From 217c635ce239d209c5eade07a0c0e4e6e02ebeee Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Thu, 5 Mar 2015 08:24:44 -0800 Subject: [PATCH 037/246] Use links instead of raw block quotes We can just link to the code that is these functions and methods instead of not, so prefer linking. Change-Id: I2469e314c411fcc64dcd5562637fdec80359c9b7 --- taskflow/engines/helpers.py | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/taskflow/engines/helpers.py b/taskflow/engines/helpers.py index 43ea8b6e..2646eea1 100644 --- a/taskflow/engines/helpers.py +++ b/taskflow/engines/helpers.py @@ -134,7 +134,7 @@ def load(flow, store=None, flow_detail=None, book=None, This function creates and prepares an engine to run the provided flow. All that is left after this returns is to run the engine with the - engines ``run()`` method. + engines :py:meth:`~taskflow.engines.base.Engine.run` method. Which engine to load is specified via the ``engine`` parameter. It can be a string that names the engine type to use, or a string that @@ -143,7 +143,8 @@ def load(flow, store=None, flow_detail=None, book=None, Which storage backend to use is defined by the backend parameter. It can be backend itself, or a dictionary that is passed to - ``taskflow.persistence.backends.fetch()`` to obtain a viable backend. + :py:func:`~taskflow.persistence.backends.fetch` to obtain a + viable backend. :param flow: flow to load :param store: dict -- data to put to storage to satisfy flow requirements @@ -198,7 +199,8 @@ def run(flow, store=None, flow_detail=None, book=None, The arguments are interpreted as for :func:`load() `. - :returns: dictionary of all named results (see ``storage.fetch_all()``) + :returns: dictionary of all named + results (see :py:meth:`~.taskflow.storage.Storage.fetch_all`) """ engine = load(flow, store=store, flow_detail=flow_detail, book=book, engine_conf=engine_conf, backend=backend, From fd4772ca84a79c2bea45a6d8571a3644c4055f8a Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Thu, 5 Mar 2015 15:54:47 -0800 Subject: [PATCH 038/246] Specialize checking for overlaps When merging a child graph into it's parents graph we can specialize checking for overlaps to avoid some of overhead of the default subgraph algorithm (which builds a full graph, with edges) since we only care about the number of duplicate nodes (not needing a full subgraph with retained edges). Change-Id: Ib211460c58efca3ddb5a254da11aafe44716a639 --- taskflow/engines/action_engine/compiler.py | 12 +++- taskflow/tests/unit/test_types.py | 65 ++++++++++++++++++++++ taskflow/types/graph.py | 33 +++++++---- 3 files changed, 97 insertions(+), 13 deletions(-) diff --git a/taskflow/engines/action_engine/compiler.py b/taskflow/engines/action_engine/compiler.py index 23e75bf7..6116cb01 100644 --- a/taskflow/engines/action_engine/compiler.py +++ b/taskflow/engines/action_engine/compiler.py @@ -220,6 +220,11 @@ class _FlowCompiler(object): if n is not retry and flow.LINK_RETRY not in graph.node[n]: graph.node[n][flow.LINK_RETRY] = retry + @staticmethod + def _occurence_detector(to_graph, from_graph): + return sum(1 for node in from_graph.nodes_iter() + if node in to_graph) + def _decompose_flow(self, flow, parent=None): """Decomposes a flow into a graph, tree node + decomposed subgraphs.""" graph = gr.DiGraph(name=flow.name) @@ -233,7 +238,12 @@ class _FlowCompiler(object): subgraph, _subnode = self._deep_compiler_func(item, parent=node) decomposed_members[item] = subgraph if subgraph.number_of_nodes(): - graph = gr.merge_graphs([graph, subgraph]) + graph = gr.merge_graphs( + graph, subgraph, + # We can specialize this to be simpler than the default + # algorithm which creates overhead that we don't + # need for our purposes... + overlap_detector=self._occurence_detector) return graph, node, decomposed_members def compile(self, flow, parent=None): diff --git a/taskflow/tests/unit/test_types.py b/taskflow/tests/unit/test_types.py index 7c05dd2c..baaca5b9 100644 --- a/taskflow/tests/unit/test_types.py +++ b/taskflow/tests/unit/test_types.py @@ -78,6 +78,71 @@ class GraphTest(test.TestCase): g.freeze() self.assertRaises(nx.NetworkXError, g.add_node, "c") + def test_merge(self): + g = graph.DiGraph() + g.add_node("a") + g.add_node("b") + + g2 = graph.DiGraph() + g2.add_node('c') + + g3 = graph.merge_graphs(g, g2) + self.assertEqual(3, len(g3)) + + def test_merge_edges(self): + g = graph.DiGraph() + g.add_node("a") + g.add_node("b") + g.add_edge('a', 'b') + + g2 = graph.DiGraph() + g2.add_node('c') + g2.add_node('d') + g2.add_edge('c', 'd') + + g3 = graph.merge_graphs(g, g2) + self.assertEqual(4, len(g3)) + self.assertTrue(g3.has_edge('c', 'd')) + self.assertTrue(g3.has_edge('a', 'b')) + + def test_overlap_detector(self): + g = graph.DiGraph() + g.add_node("a") + g.add_node("b") + g.add_edge('a', 'b') + + g2 = graph.DiGraph() + g2.add_node('a') + g2.add_node('d') + g2.add_edge('a', 'd') + + self.assertRaises(ValueError, + graph.merge_graphs, g, g2) + + def occurence_detector(to_graph, from_graph): + return sum(1 for node in from_graph.nodes_iter() + if node in to_graph) + + self.assertRaises(ValueError, + graph.merge_graphs, g, g2, + overlap_detector=occurence_detector) + + g3 = graph.merge_graphs(g, g2, allow_overlaps=True) + self.assertEqual(3, len(g3)) + self.assertTrue(g3.has_edge('a', 'b')) + self.assertTrue(g3.has_edge('a', 'd')) + + def test_invalid_detector(self): + g = graph.DiGraph() + g.add_node("a") + + g2 = graph.DiGraph() + g2.add_node('c') + + self.assertRaises(ValueError, + graph.merge_graphs, g, g2, + overlap_detector='b') + class TreeTest(test.TestCase): def _make_species(self): diff --git a/taskflow/types/graph.py b/taskflow/types/graph.py index 068a8e20..53eddba6 100644 --- a/taskflow/types/graph.py +++ b/taskflow/types/graph.py @@ -122,26 +122,35 @@ class DiGraph(nx.DiGraph): queue.append(pred_pred) -def merge_graphs(graphs, allow_overlaps=False): - """Merges a bunch of graphs into a single graph.""" - if not graphs: - return None - graph = graphs[0] - for g in graphs[1:]: +def merge_graphs(graph, *graphs, **kwargs): + """Merges a bunch of graphs into a new graph. + + If no additional graphs are provided the first graph is + returned unmodified otherwise the merged graph is returned. + """ + tmp_graph = graph + allow_overlaps = kwargs.get('allow_overlaps', False) + overlap_detector = kwargs.get('overlap_detector') + if overlap_detector is not None and not six.callable(overlap_detector): + raise ValueError("Overlap detection callback expected to be callable") + elif overlap_detector is None: + overlap_detector = (lambda to_graph, from_graph: + len(to_graph.subgraph(from_graph.nodes_iter()))) + for g in graphs: # This should ensure that the nodes to be merged do not already exist # in the graph that is to be merged into. This could be problematic if # there are duplicates. if not allow_overlaps: # Attempt to induce a subgraph using the to be merged graphs nodes # and see if any graph results. - overlaps = graph.subgraph(g.nodes_iter()) - if len(overlaps): + overlaps = overlap_detector(graph, g) + if overlaps: raise ValueError("Can not merge graph %s into %s since there " "are %s overlapping nodes (and we do not " "support merging nodes)" % (g, graph, - len(overlaps))) - # Keep the target graphs name. - name = graph.name + overlaps)) graph = nx.algorithms.compose(graph, g) - graph.name = name + # Keep the first graphs name. + if graphs: + graph.name = tmp_graph.name return graph From b56e453a9b791977769bace6b8afdbd132e65a83 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Fri, 6 Mar 2015 16:56:04 -0800 Subject: [PATCH 039/246] Ensure needed locks is used when reading/setting intention Change-Id: Ia456033d64513b8c1fa8824146c8c0abd2407485 --- taskflow/storage.py | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/taskflow/storage.py b/taskflow/storage.py index c4250e7c..b9b0f2f3 100644 --- a/taskflow/storage.py +++ b/taskflow/storage.py @@ -311,14 +311,16 @@ class Storage(object): def set_atom_intention(self, atom_name, intention): """Sets the intention of an atom given an atoms name.""" - ad = self._atomdetail_by_name(atom_name) - ad.intention = intention - self._with_connection(self._save_atom_detail, ad) + with self._lock.write_lock(): + ad = self._atomdetail_by_name(atom_name) + ad.intention = intention + self._with_connection(self._save_atom_detail, ad) def get_atom_intention(self, atom_name): """Gets the intention of an atom given an atoms name.""" - ad = self._atomdetail_by_name(atom_name) - return ad.intention + with self._lock.read_lock(): + ad = self._atomdetail_by_name(atom_name) + return ad.intention def get_atoms_states(self, atom_names): """Gets all atoms states given a set of names.""" From 1f108da515ae72192792dce066345432c78a7b8b Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Sat, 7 Mar 2015 07:35:19 -0800 Subject: [PATCH 040/246] Set a no-op functor when none is provided Instead of having later checks to check for none when calling the thread bundle callback just initially set it to a no-op functor that does nothing at binding time instead; this avoids the need to do repeated checks at start/stop time. Change-Id: I5ab4f801bf4767c802ea607fdf864d4852e6c84d --- taskflow/utils/threading_utils.py | 42 +++++++++++++++++-------------- 1 file changed, 23 insertions(+), 19 deletions(-) diff --git a/taskflow/utils/threading_utils.py b/taskflow/utils/threading_utils.py index 1f3186bf..b2790657 100644 --- a/taskflow/utils/threading_utils.py +++ b/taskflow/utils/threading_utils.py @@ -82,16 +82,19 @@ _ThreadBuilder = collections.namedtuple('_ThreadBuilder', ['thread_factory', 'before_start', 'after_start', 'before_join', 'after_join']) -_ThreadBuilder.callables = tuple([ - # Attribute name -> none allowed as a valid value... - ('thread_factory', False), - ('before_start', True), - ('after_start', True), - ('before_join', True), - ('after_join', True), +_ThreadBuilder.fields = tuple([ + 'thread_factory', + 'before_start', + 'after_start', + 'before_join', + 'after_join', ]) +def no_op(*args, **kwargs): + """Function that does nothing.""" + + class ThreadBundle(object): """A group/bundle of threads that start/stop together.""" @@ -110,13 +113,19 @@ class ThreadBundle(object): in dead-lock since the lock on this object is not meant to be (and is not) reentrant... """ + if before_start is None: + before_start = no_op + if after_start is None: + after_start = no_op + if before_join is None: + before_join = no_op + if after_join is None: + after_join = no_op builder = _ThreadBuilder(thread_factory, before_start, after_start, before_join, after_join) - for attr_name, none_allowed in builder.callables: + for attr_name in builder.fields: cb = getattr(builder, attr_name) - if cb is None and none_allowed: - continue if not six.callable(cb): raise ValueError("Provided callback for argument" " '%s' must be callable" % attr_name) @@ -130,11 +139,6 @@ class ThreadBundle(object): False, ]) - @staticmethod - def _trigger_callback(callback, thread): - if callback is not None: - callback(thread) - def start(self): """Creates & starts all associated threads (that are not running).""" count = 0 @@ -145,11 +149,11 @@ class ThreadBundle(object): continue if not thread: self._threads[i][1] = thread = builder.thread_factory() - self._trigger_callback(builder.before_start, thread) + builder.before_start(thread) thread.start() count += 1 try: - self._trigger_callback(builder.after_start, thread) + builder.after_start(thread) finally: # Just incase the 'after_start' callback blows up make sure # we always set this... @@ -164,11 +168,11 @@ class ThreadBundle(object): for i, (builder, thread, started) in it: if not thread or not started: continue - self._trigger_callback(builder.before_join, thread) + builder.before_join(thread) thread.join() count += 1 try: - self._trigger_callback(builder.after_join, thread) + builder.after_join(thread) finally: # Just incase the 'after_join' callback blows up make sure # we always set/reset these... From 213fddb5113f1726cf6a854bd74098f974a7459c Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Sat, 7 Mar 2015 15:52:53 -0800 Subject: [PATCH 041/246] Remove condition acquiring for read-only ops In the reader-writer lock we can remove the condition acquisition around read-only operations as these are safe to do without grabbing the lock. Change-Id: I318082887a8584fd90c2f5b03f07e8f5087aa377 --- taskflow/utils/lock_utils.py | 31 ++++++++++--------------------- 1 file changed, 10 insertions(+), 21 deletions(-) diff --git a/taskflow/utils/lock_utils.py b/taskflow/utils/lock_utils.py index 1ee924b2..d7312088 100644 --- a/taskflow/utils/lock_utils.py +++ b/taskflow/utils/lock_utils.py @@ -142,25 +142,17 @@ class ReaderWriterLock(object): @property def has_pending_writers(self): """Returns if there are writers waiting to become the *one* writer.""" - self._cond.acquire() - try: - return bool(self._pending_writers) - finally: - self._cond.release() + return bool(self._pending_writers) def is_writer(self, check_pending=True): """Returns if the caller is the active writer or a pending writer.""" - self._cond.acquire() - try: - me = self._current_thread() - if self._writer is not None and self._writer == me: - return True - if check_pending: - return me in self._pending_writers - else: - return False - finally: - self._cond.release() + me = self._current_thread() + if self._writer == me: + return True + if check_pending: + return me in self._pending_writers + else: + return False @property def owner(self): @@ -177,11 +169,8 @@ class ReaderWriterLock(object): def is_reader(self): """Returns if the caller is one of the readers.""" - self._cond.acquire() - try: - return self._current_thread() in self._readers - finally: - self._cond.release() + me = self._current_thread() + return me in self._readers @contextlib.contextmanager def read_lock(self): From d302b52a4ec19c49adf5aaf4524f9e62017f14d8 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Sun, 8 Mar 2015 10:14:15 -0700 Subject: [PATCH 042/246] Use condition context manager instead of acquire/release Just use the condition in the read/write lock as a context manager instead of doing acquire, try, finally, release to achieve the same thing. Change-Id: I8cfa5e1067882b34ede2e21eefa10b3ecdd09c43 --- taskflow/utils/lock_utils.py | 27 +++++++-------------------- 1 file changed, 7 insertions(+), 20 deletions(-) diff --git a/taskflow/utils/lock_utils.py b/taskflow/utils/lock_utils.py index d7312088..644e8659 100644 --- a/taskflow/utils/lock_utils.py +++ b/taskflow/utils/lock_utils.py @@ -157,15 +157,14 @@ class ReaderWriterLock(object): @property def owner(self): """Returns whether the lock is locked by a writer or reader.""" - self._cond.acquire() - try: + with self._cond: + # Obtain the lock to ensure we get a accurate view of the actual + # owner that isn't likely to change when we are reading it... if self._writer is not None: return self.WRITER if self._readers: return self.READER return None - finally: - self._cond.release() def is_reader(self): """Returns if the caller is one of the readers.""" @@ -186,8 +185,7 @@ class ReaderWriterLock(object): raise RuntimeError("Writer %s can not acquire a read lock" " while holding/waiting for the write lock" % me) - self._cond.acquire() - try: + with self._cond: while True: # No active writer; we are good to become a reader. if self._writer is None: @@ -195,8 +193,6 @@ class ReaderWriterLock(object): break # An active writer; guess we have to wait. self._cond.wait() - finally: - self._cond.release() try: yield self finally: @@ -204,12 +200,9 @@ class ReaderWriterLock(object): # If the current thread acquired two read locks, then it will # still have to remove that other read lock; this allows for # basic reentrancy to be possible. - self._cond.acquire() - try: + with self._cond: self._readers.remove(me) self._cond.notify_all() - finally: - self._cond.release() @contextlib.contextmanager def write_lock(self): @@ -227,8 +220,7 @@ class ReaderWriterLock(object): # Already the writer; this allows for basic reentrancy. yield self else: - self._cond.acquire() - try: + with self._cond: self._pending_writers.append(me) while True: # No readers, and no active writer, am I next?? @@ -237,17 +229,12 @@ class ReaderWriterLock(object): self._writer = self._pending_writers.popleft() break self._cond.wait() - finally: - self._cond.release() try: yield self finally: - self._cond.acquire() - try: + with self._cond: self._writer = None self._cond.notify_all() - finally: - self._cond.release() class MultiLock(object): From 89087fd11dd0354f3e9038ef1438aa18912bc3fa Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Sun, 8 Mar 2015 18:38:38 -0700 Subject: [PATCH 043/246] Add no double writers thread test To ensure that the rw-lock continues to work as expected add a test which spins up a large number of threads and then has them content on a single lock and has those threads record if another thread is also active in the same critical region. Each thread then sleeps (while in critical region) and then releases. This repeats for up to 5 seconds and then finishes; at that point there should have been no simultaneous access (if there was this would indicate the rw-lock is broken). Change-Id: Ia293dda989ad924be3daca01eb79ca04bf60c79b --- taskflow/tests/unit/test_utils_lock_utils.py | 39 ++++++++++++++++++++ 1 file changed, 39 insertions(+) diff --git a/taskflow/tests/unit/test_utils_lock_utils.py b/taskflow/tests/unit/test_utils_lock_utils.py index bb334a2d..30c8c983 100644 --- a/taskflow/tests/unit/test_utils_lock_utils.py +++ b/taskflow/tests/unit/test_utils_lock_utils.py @@ -15,6 +15,7 @@ # under the License. import collections +import random import threading import time @@ -23,6 +24,7 @@ from concurrent import futures from taskflow import test from taskflow.test import mock from taskflow.tests import utils as test_utils +from taskflow.types import timing from taskflow.utils import lock_utils from taskflow.utils import misc from taskflow.utils import threading_utils @@ -330,6 +332,43 @@ class MultilockTest(test.TestCase): class ReadWriteLockTest(test.TestCase): + THREAD_COUNT = 20 + + def test_no_double_writers(self): + lock = lock_utils.ReaderWriterLock() + watch = timing.StopWatch(duration=5) + watch.start() + dups = collections.deque() + active = collections.deque() + + def acquire_check(me): + with lock.write_lock(): + if len(active) >= 1: + dups.append(me) + dups.extend(active) + active.append(me) + try: + time.sleep(random.random() / 100) + finally: + active.remove(me) + + def run(): + me = threading.current_thread() + while not watch.expired(): + acquire_check(me) + + threads = [] + for i in range(0, self.THREAD_COUNT): + t = threading_utils.daemon_thread(run) + threads.append(t) + t.start() + while threads: + t = threads.pop() + t.join() + + self.assertEqual([], list(dups)) + self.assertEqual([], list(active)) + def test_writer_abort(self): lock = lock_utils.ReaderWriterLock() self.assertFalse(lock.owner) From 20fdbba14188c384dc0365d6451834b0887a6835 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Sun, 8 Mar 2015 18:28:44 -0700 Subject: [PATCH 044/246] Add + use read/write lock decorators For when a whole function should be locked by a read or write lock it is quite useful to have decorators that acquire and release and wrap the target function with the provided lock. This change switches the storage module to use these decorators where appropriate (mainly in places where the read or write lock is activated for the duration of the functions call, at which point it is clearer to just use the new decorators instead). Change-Id: I70d2c1ab478e9b7da9446482a4ffb28f6f5227b7 --- taskflow/storage.py | 385 +++++++++++++++++------------------ taskflow/utils/lock_utils.py | 73 ++++++- 2 files changed, 256 insertions(+), 202 deletions(-) diff --git a/taskflow/storage.py b/taskflow/storage.py index 4207a662..449aafb6 100644 --- a/taskflow/storage.py +++ b/taskflow/storage.py @@ -290,53 +290,53 @@ class Storage(object): # do this update. atom_detail.update(conn.update_atom_details(atom_detail)) + @lock_utils.read_locked def get_atom_uuid(self, atom_name): """Gets an atoms uuid given a atoms name.""" - with self._lock.read_lock(): - ad = self._atomdetail_by_name(atom_name) - return ad.uuid + ad = self._atomdetail_by_name(atom_name) + return ad.uuid + @lock_utils.write_locked def set_atom_state(self, atom_name, state): """Sets an atoms state.""" - with self._lock.write_lock(): - ad = self._atomdetail_by_name(atom_name) - ad.state = state - self._with_connection(self._save_atom_detail, ad) + ad = self._atomdetail_by_name(atom_name) + ad.state = state + self._with_connection(self._save_atom_detail, ad) + @lock_utils.read_locked def get_atom_state(self, atom_name): """Gets the state of an atom given an atoms name.""" - with self._lock.read_lock(): - ad = self._atomdetail_by_name(atom_name) - return ad.state + ad = self._atomdetail_by_name(atom_name) + return ad.state + @lock_utils.write_locked def set_atom_intention(self, atom_name, intention): """Sets the intention of an atom given an atoms name.""" - with self._lock.write_lock(): - ad = self._atomdetail_by_name(atom_name) - ad.intention = intention - self._with_connection(self._save_atom_detail, ad) + ad = self._atomdetail_by_name(atom_name) + ad.intention = intention + self._with_connection(self._save_atom_detail, ad) + @lock_utils.read_locked def get_atom_intention(self, atom_name): """Gets the intention of an atom given an atoms name.""" - with self._lock.read_lock(): - ad = self._atomdetail_by_name(atom_name) - return ad.intention + ad = self._atomdetail_by_name(atom_name) + return ad.intention + @lock_utils.read_locked def get_atoms_states(self, atom_names): """Gets all atoms states given a set of names.""" - with self._lock.read_lock(): - return dict((name, (self.get_atom_state(name), - self.get_atom_intention(name))) - for name in atom_names) + return dict((name, (self.get_atom_state(name), + self.get_atom_intention(name))) + for name in atom_names) + @lock_utils.write_locked def _update_atom_metadata(self, atom_name, update_with, expected_type=None): - with self._lock.write_lock(): - ad = self._atomdetail_by_name(atom_name, - expected_type=expected_type) - if update_with: - ad.meta.update(update_with) - self._with_connection(self._save_atom_detail, ad) + ad = self._atomdetail_by_name(atom_name, + expected_type=expected_type) + if update_with: + ad.meta.update(update_with) + self._with_connection(self._save_atom_detail, ad) def update_atom_metadata(self, atom_name, update_with): """Updates a atoms associated metadata. @@ -372,20 +372,21 @@ class Storage(object): self._update_atom_metadata(task_name, update_with, expected_type=logbook.TaskDetail) + @lock_utils.read_locked def get_task_progress(self, task_name): """Get the progress of a task given a tasks name. :param task_name: tasks name :returns: current task progress value """ - with self._lock.read_lock(): - ad = self._atomdetail_by_name(task_name, - expected_type=logbook.TaskDetail) - try: - return ad.meta['progress'] - except KeyError: - return 0.0 + ad = self._atomdetail_by_name(task_name, + expected_type=logbook.TaskDetail) + try: + return ad.meta['progress'] + except KeyError: + return 0.0 + @lock_utils.read_locked def get_task_progress_details(self, task_name): """Get the progress details of a task given a tasks name. @@ -393,13 +394,12 @@ class Storage(object): :returns: None if progress_details not defined, else progress_details dict """ - with self._lock.read_lock(): - ad = self._atomdetail_by_name(task_name, - expected_type=logbook.TaskDetail) - try: - return ad.meta['progress_details'] - except KeyError: - return None + ad = self._atomdetail_by_name(task_name, + expected_type=logbook.TaskDetail) + try: + return ad.meta['progress_details'] + except KeyError: + return None def _check_all_results_provided(self, atom_name, container): """Warn if an atom did not provide some of its expected results. @@ -418,76 +418,75 @@ class Storage(object): LOG.warning("Atom %s did not supply result " "with index %r (name %s)", atom_name, index, name) + @lock_utils.write_locked def save(self, atom_name, data, state=states.SUCCESS): """Put result for atom with id 'uuid' to storage.""" - with self._lock.write_lock(): - ad = self._atomdetail_by_name(atom_name) - ad.put(state, data) - if state == states.FAILURE and isinstance(data, failure.Failure): - # NOTE(imelnikov): failure serialization looses information, - # so we cache failures here, in atom name -> failure mapping. - self._failures[ad.name] = data - else: - self._check_all_results_provided(ad.name, data) - self._with_connection(self._save_atom_detail, ad) + ad = self._atomdetail_by_name(atom_name) + ad.put(state, data) + if state == states.FAILURE and isinstance(data, failure.Failure): + # NOTE(imelnikov): failure serialization looses information, + # so we cache failures here, in atom name -> failure mapping. + self._failures[ad.name] = data + else: + self._check_all_results_provided(ad.name, data) + self._with_connection(self._save_atom_detail, ad) + @lock_utils.write_locked def save_retry_failure(self, retry_name, failed_atom_name, failure): """Save subflow failure to retry controller history.""" - with self._lock.write_lock(): - ad = self._atomdetail_by_name(retry_name, - expected_type=logbook.RetryDetail) - try: - failures = ad.last_failures - except exceptions.NotFound as e: - raise exceptions.StorageFailure("Unable to fetch most recent" - " retry failures so new retry" - " failure can be inserted", e) - else: - if failed_atom_name not in failures: - failures[failed_atom_name] = failure - self._with_connection(self._save_atom_detail, ad) + ad = self._atomdetail_by_name(retry_name, + expected_type=logbook.RetryDetail) + try: + failures = ad.last_failures + except exceptions.NotFound as e: + raise exceptions.StorageFailure("Unable to fetch most recent" + " retry failures so new retry" + " failure can be inserted", e) + else: + if failed_atom_name not in failures: + failures[failed_atom_name] = failure + self._with_connection(self._save_atom_detail, ad) + @lock_utils.write_locked def cleanup_retry_history(self, retry_name, state): """Cleanup history of retry atom with given name.""" - with self._lock.write_lock(): - ad = self._atomdetail_by_name(retry_name, - expected_type=logbook.RetryDetail) - ad.state = state - ad.results = [] - self._with_connection(self._save_atom_detail, ad) + ad = self._atomdetail_by_name(retry_name, + expected_type=logbook.RetryDetail) + ad.state = state + ad.results = [] + self._with_connection(self._save_atom_detail, ad) + @lock_utils.read_locked def _get(self, atom_name, only_last=False): - with self._lock.read_lock(): - ad = self._atomdetail_by_name(atom_name) - if ad.failure is not None: - cached = self._failures.get(atom_name) - if ad.failure.matches(cached): - return cached - return ad.failure - if ad.state not in STATES_WITH_RESULTS: - raise exceptions.NotFound("Result for atom %s is not currently" - " known" % atom_name) - if only_last: - return ad.last_results - else: - return ad.results + ad = self._atomdetail_by_name(atom_name) + if ad.failure is not None: + cached = self._failures.get(atom_name) + if ad.failure.matches(cached): + return cached + return ad.failure + if ad.state not in STATES_WITH_RESULTS: + raise exceptions.NotFound("Result for atom %s is not currently" + " known" % atom_name) + if only_last: + return ad.last_results + else: + return ad.results def get(self, atom_name): """Gets the results for an atom with a given name from storage.""" return self._get(atom_name) + @lock_utils.read_locked def get_failures(self): """Get list of failures that happened with this flow. No order guaranteed. """ - with self._lock.read_lock(): - return self._failures.copy() + return self._failures.copy() def has_failures(self): """Returns True if there are failed tasks in the storage.""" - with self._lock.read_lock(): - return bool(self._failures) + return bool(self._failures) def _reset_atom(self, ad, state): if ad.name == self.injector_name: @@ -498,12 +497,12 @@ class Storage(object): self._failures.pop(ad.name, None) return True + @lock_utils.write_locked def reset(self, atom_name, state=states.PENDING): """Reset atom with given name (if the task is in a given state).""" - with self._lock.write_lock(): - ad = self._atomdetail_by_name(atom_name) - if self._reset_atom(ad, state): - self._with_connection(self._save_atom_detail, ad) + ad = self._atomdetail_by_name(atom_name) + if self._reset_atom(ad, state): + self._with_connection(self._save_atom_detail, ad) def inject_atom_args(self, atom_name, pairs): """Add **transient** values into storage for a specific atom only. @@ -541,6 +540,7 @@ class Storage(object): self._injected_args.setdefault(atom_name, {}) self._injected_args[atom_name].update(pairs) + @lock_utils.write_locked def inject(self, pairs, transient=False): """Add values into storage. @@ -589,13 +589,12 @@ class Storage(object): self._transients.update(pairs) return (_TRANSIENT_PROVIDER, six.iterkeys(self._transients)) - with self._lock.write_lock(): - if transient: - provider_name, names = save_transient() - else: - provider_name, names = save_persistent() - self._set_result_mapping(provider_name, - dict((name, name) for name in names)) + if transient: + provider_name, names = save_transient() + else: + provider_name, names = save_persistent() + self._set_result_mapping(provider_name, + dict((name, name) for name in names)) def _set_result_mapping(self, provider_name, mapping): """Sets the result mapping for a given producer. @@ -615,6 +614,7 @@ class Storage(object): if provider not in entries: entries.append(provider) + @lock_utils.read_locked def fetch(self, name, many_handler=None): """Fetch a named result.""" # By default we just return the first of many (unless provided @@ -622,32 +622,32 @@ class Storage(object): # more meaningful). if many_handler is None: many_handler = lambda values: values[0] - with self._lock.read_lock(): - try: - providers = self._reverse_mapping[name] - except KeyError: - raise exceptions.NotFound("Name %r is not mapped as a" - " produced output by any" - " providers" % name) - values = [] - for provider in providers: - if provider.name is _TRANSIENT_PROVIDER: - values.append(_item_from_single(provider, - self._transients, name)) - else: - try: - container = self._get(provider.name, only_last=True) - except exceptions.NotFound: - pass - else: - values.append(_item_from_single(provider, - container, name)) - if not values: - raise exceptions.NotFound("Unable to find result %r," - " searched %s" % (name, providers)) + try: + providers = self._reverse_mapping[name] + except KeyError: + raise exceptions.NotFound("Name %r is not mapped as a" + " produced output by any" + " providers" % name) + values = [] + for provider in providers: + if provider.name is _TRANSIENT_PROVIDER: + values.append(_item_from_single(provider, + self._transients, name)) else: - return many_handler(values) + try: + container = self._get(provider.name, only_last=True) + except exceptions.NotFound: + pass + else: + values.append(_item_from_single(provider, + container, name)) + if not values: + raise exceptions.NotFound("Unable to find result %r," + " searched %s" % (name, providers)) + else: + return many_handler(values) + @lock_utils.read_locked def fetch_all(self): """Fetch all named results known so far. @@ -657,15 +657,15 @@ class Storage(object): if len(values) > 1: return values return values[0] - with self._lock.read_lock(): - results = {} - for name in six.iterkeys(self._reverse_mapping): - try: - results[name] = self.fetch(name, many_handler=many_handler) - except exceptions.NotFound: - pass - return results + results = {} + for name in six.iterkeys(self._reverse_mapping): + try: + results[name] = self.fetch(name, many_handler=many_handler) + except exceptions.NotFound: + pass + return results + @lock_utils.read_locked def fetch_mapped_args(self, args_mapping, atom_name=None, scope_walker=None, optional_args=None): @@ -708,66 +708,65 @@ class Storage(object): for p in providers] return [] - with self._lock.read_lock(): - if optional_args is None: - optional_args = [] - if atom_name and atom_name not in self._atom_name_to_uuid: - raise exceptions.NotFound("Unknown atom name: %s" % atom_name) - if not args_mapping: - return {} - if atom_name: - injected_args = self._injected_args.get(atom_name, {}) - else: - injected_args = {} - mapped_args = {} - for (bound_name, name) in six.iteritems(args_mapping): - if LOG.isEnabledFor(logging.BLATHER): - if atom_name: - LOG.blather("Looking for %r <= %r for atom named: %s", - bound_name, name, atom_name) - else: - LOG.blather("Looking for %r <= %r", bound_name, name) - if name in injected_args: - value = injected_args[name] - mapped_args[bound_name] = value - LOG.blather("Matched %r <= %r to %r (from injected" - " values)", bound_name, name, value) + if optional_args is None: + optional_args = [] + if atom_name and atom_name not in self._atom_name_to_uuid: + raise exceptions.NotFound("Unknown atom name: %s" % atom_name) + if not args_mapping: + return {} + if atom_name: + injected_args = self._injected_args.get(atom_name, {}) + else: + injected_args = {} + mapped_args = {} + for (bound_name, name) in six.iteritems(args_mapping): + if LOG.isEnabledFor(logging.BLATHER): + if atom_name: + LOG.blather("Looking for %r <= %r for atom named: %s", + bound_name, name, atom_name) else: - try: - possible_providers = self._reverse_mapping[name] - except KeyError: - if bound_name in optional_args: - continue - raise exceptions.NotFound("Name %r is not mapped as a" - " produced output by any" - " providers" % name) - # Reduce the possible providers to one that are allowed. - providers = _locate_providers(name, possible_providers) - if not providers: - raise exceptions.NotFound( - "Mapped argument %r <= %r was not produced" - " by any accessible provider (%s possible" - " providers were scanned)" - % (bound_name, name, len(possible_providers))) - provider, value = _item_from_first_of(providers, name) - mapped_args[bound_name] = value - LOG.blather("Matched %r <= %r to %r (from %s)", - bound_name, name, value, provider) - return mapped_args + LOG.blather("Looking for %r <= %r", bound_name, name) + if name in injected_args: + value = injected_args[name] + mapped_args[bound_name] = value + LOG.blather("Matched %r <= %r to %r (from injected" + " values)", bound_name, name, value) + else: + try: + possible_providers = self._reverse_mapping[name] + except KeyError: + if bound_name in optional_args: + continue + raise exceptions.NotFound("Name %r is not mapped as a" + " produced output by any" + " providers" % name) + # Reduce the possible providers to one that are allowed. + providers = _locate_providers(name, possible_providers) + if not providers: + raise exceptions.NotFound( + "Mapped argument %r <= %r was not produced" + " by any accessible provider (%s possible" + " providers were scanned)" + % (bound_name, name, len(possible_providers))) + provider, value = _item_from_first_of(providers, name) + mapped_args[bound_name] = value + LOG.blather("Matched %r <= %r to %r (from %s)", + bound_name, name, value, provider) + return mapped_args + @lock_utils.write_locked def set_flow_state(self, state): """Set flow details state and save it.""" - with self._lock.write_lock(): - self._flowdetail.state = state - self._with_connection(self._save_flow_detail) + self._flowdetail.state = state + self._with_connection(self._save_flow_detail) + @lock_utils.read_locked def get_flow_state(self): """Get state from flow details.""" - with self._lock.read_lock(): - state = self._flowdetail.state - if state is None: - state = states.PENDING - return state + state = self._flowdetail.state + if state is None: + state = states.PENDING + return state def _translate_into_history(self, ad): failure = None @@ -782,19 +781,19 @@ class Storage(object): failure = ad.failure return retry.History(ad.results, failure=failure) + @lock_utils.read_locked def get_retry_history(self, retry_name): """Fetch a single retrys history.""" - with self._lock.read_lock(): - ad = self._atomdetail_by_name(retry_name, - expected_type=logbook.RetryDetail) - return self._translate_into_history(ad) + ad = self._atomdetail_by_name(retry_name, + expected_type=logbook.RetryDetail) + return self._translate_into_history(ad) + @lock_utils.read_locked def get_retry_histories(self): """Fetch all retrys histories.""" histories = [] - with self._lock.read_lock(): - for ad in self._flowdetail: - if isinstance(ad, logbook.RetryDetail): - histories.append((ad.name, - self._translate_into_history(ad))) + for ad in self._flowdetail: + if isinstance(ad, logbook.RetryDetail): + histories.append((ad.name, + self._translate_into_history(ad))) return histories diff --git a/taskflow/utils/lock_utils.py b/taskflow/utils/lock_utils.py index d7312088..726e10f9 100644 --- a/taskflow/utils/lock_utils.py +++ b/taskflow/utils/lock_utils.py @@ -64,15 +64,10 @@ def locked(*args, **kwargs): activates the given lock or list of locks as a context manager, automatically releasing that lock on exit. - NOTE(harlowja): if no attribute is provided then by default the attribute - named '_lock' is looked for in the instance object this decorator is - attached to. - - NOTE(harlowja): when we get the wrapt module approved we can address the - correctness of this decorator with regards to classmethods, to keep sanity - and correctness it is recommended to avoid using this on classmethods, once - https://review.openstack.org/#/c/94754/ is merged this will be refactored - and that use-case can be provided in a correct manner. + NOTE(harlowja): if no attribute name is provided then by default the + attribute named '_lock' is looked for (this attribute is expected to be + the lock/list of locks object/s) in the instance object this decorator + is attached to. """ def decorator(f): @@ -101,6 +96,66 @@ def locked(*args, **kwargs): return decorator +def read_locked(*args, **kwargs): + """Acquires & releases a read lock around call into decorated method. + + NOTE(harlowja): if no attribute name is provided then by default the + attribute named '_lock' is looked for (this attribute is expected to be + the rw-lock object) in the instance object this decorator is attached to. + """ + + def decorator(f): + attr_name = kwargs.get('lock', '_lock') + + @six.wraps(f) + def wrapper(self, *args, **kwargs): + rw_lock = getattr(self, attr_name) + with rw_lock.read_lock(): + return f(self, *args, **kwargs) + + return wrapper + + # This is needed to handle when the decorator has args or the decorator + # doesn't have args, python is rather weird here... + if kwargs or not args: + return decorator + else: + if len(args) == 1: + return decorator(args[0]) + else: + return decorator + + +def write_locked(*args, **kwargs): + """Acquires & releases a write lock around call into decorated method. + + NOTE(harlowja): if no attribute name is provided then by default the + attribute named '_lock' is looked for (this attribute is expected to be + the rw-lock object) in the instance object this decorator is attached to. + """ + + def decorator(f): + attr_name = kwargs.get('lock', '_lock') + + @six.wraps(f) + def wrapper(self, *args, **kwargs): + rw_lock = getattr(self, attr_name) + with rw_lock.write_lock(): + return f(self, *args, **kwargs) + + return wrapper + + # This is needed to handle when the decorator has args or the decorator + # doesn't have args, python is rather weird here... + if kwargs or not args: + return decorator + else: + if len(args) == 1: + return decorator(args[0]) + else: + return decorator + + class ReaderWriterLock(object): """A reader/writer lock. From 47c026957819bca71fea786cf8f59914d3687fb5 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Mon, 9 Mar 2015 08:42:25 -0700 Subject: [PATCH 045/246] Add another probabilistic rw-lock test This test works like the no-concurrent writer one but tests reader and writes for a duration and makes sure no collisions happen during this period. Change-Id: I11c1b39d34e4c83fa832c3b89e838e0c4635e750 --- taskflow/tests/unit/test_utils_lock_utils.py | 43 ++++++++++++++++++++ 1 file changed, 43 insertions(+) diff --git a/taskflow/tests/unit/test_utils_lock_utils.py b/taskflow/tests/unit/test_utils_lock_utils.py index 30c8c983..3ccb128f 100644 --- a/taskflow/tests/unit/test_utils_lock_utils.py +++ b/taskflow/tests/unit/test_utils_lock_utils.py @@ -369,6 +369,49 @@ class ReadWriteLockTest(test.TestCase): self.assertEqual([], list(dups)) self.assertEqual([], list(active)) + def test_no_concurrent_readers_writers(self): + lock = lock_utils.ReaderWriterLock() + watch = timing.StopWatch(duration=5) + watch.start() + dups = collections.deque() + active = collections.deque() + + def acquire_check(me, reader): + if reader: + lock_func = lock.read_lock + else: + lock_func = lock.write_lock + with lock_func(): + if not reader: + # There should be no-one else currently active, if there + # is ensure we capture them so that we can later blow-up + # the test. + if len(active) >= 1: + dups.append(me) + dups.extend(active) + active.append(me) + try: + time.sleep(random.random() / 100) + finally: + active.remove(me) + + def run(): + me = threading.current_thread() + while not watch.expired(): + acquire_check(me, random.choice([True, False])) + + threads = [] + for i in range(0, self.THREAD_COUNT): + t = threading_utils.daemon_thread(run) + threads.append(t) + t.start() + while threads: + t = threads.pop() + t.join() + + self.assertEqual([], list(dups)) + self.assertEqual([], list(active)) + def test_writer_abort(self): lock = lock_utils.ReaderWriterLock() self.assertFalse(lock.owner) From 00ab6289ab834be3e938c1aed8385361159d7f63 Mon Sep 17 00:00:00 2001 From: Dan Krause Date: Mon, 9 Mar 2015 12:23:24 -0500 Subject: [PATCH 046/246] Writers can now claim a read lock in ReaderWriterLock Change-Id: Ia75398ada8f3cee5d6384e46d7c52daff2d59912 --- taskflow/tests/unit/test_utils_lock_utils.py | 5 +++-- taskflow/utils/lock_utils.py | 15 ++++++++------- 2 files changed, 11 insertions(+), 9 deletions(-) diff --git a/taskflow/tests/unit/test_utils_lock_utils.py b/taskflow/tests/unit/test_utils_lock_utils.py index 30c8c983..cc5bf343 100644 --- a/taskflow/tests/unit/test_utils_lock_utils.py +++ b/taskflow/tests/unit/test_utils_lock_utils.py @@ -532,10 +532,11 @@ class ReadWriteLockTest(test.TestCase): def reader_func(): with lock.read_lock(): - pass + self.assertTrue(lock.is_writer()) + self.assertTrue(lock.is_reader()) with lock.write_lock(): - self.assertRaises(RuntimeError, reader_func) + self.assertIsNone(reader_func()) self.assertFalse(lock.is_reader()) self.assertFalse(lock.is_reader()) diff --git a/taskflow/utils/lock_utils.py b/taskflow/utils/lock_utils.py index 56e18e43..58945c0f 100644 --- a/taskflow/utils/lock_utils.py +++ b/taskflow/utils/lock_utils.py @@ -163,8 +163,8 @@ class ReaderWriterLock(object): to exist for use-cases where it is useful to have such types of locks. Currently a reader can not escalate its read lock to a write lock and - a writer can not acquire a read lock while it owns or is waiting on - the write lock. + a writer can not acquire a read lock while it is waiting on the write + lock. In the future these restrictions may be relaxed. @@ -232,18 +232,19 @@ class ReaderWriterLock(object): Will wait until no active or pending writers. - Raises a RuntimeError if an active or pending writer tries to acquire + Raises a RuntimeError if a pending writer tries to acquire a read lock. """ me = self._current_thread() - if self.is_writer(): + if me in self._pending_writers: raise RuntimeError("Writer %s can not acquire a read lock" - " while holding/waiting for the write lock" + " while waiting for the write lock" % me) with self._cond: while True: - # No active writer; we are good to become a reader. - if self._writer is None: + # No active writer, or we are the writer; + # we are good to become a reader. + if self._writer is None or self._writer == me: self._readers.append(me) break # An active writer; guess we have to wait. From c22c62d64688f9b2f2d4ef956e839d9281dc1f9e Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Mon, 9 Mar 2015 11:24:46 -0700 Subject: [PATCH 047/246] Remove support for 3.3 We don't test against 3.3 in the integrated CI anymore so we likely shouldn't say that we support it anymore since it's not something we are guaranteeing. We still support 3.4 (and that's what we are testing against so we retain that version in the setup.cfg file). Change-Id: Ica9ef26be27994ac86e3304b510cc9261941f527 --- setup.cfg | 1 - 1 file changed, 1 deletion(-) diff --git a/setup.cfg b/setup.cfg index d9ffce21..a45b6dd8 100644 --- a/setup.cfg +++ b/setup.cfg @@ -20,7 +20,6 @@ classifier = Programming Language :: Python :: 2.6 Programming Language :: Python :: 2.7 Programming Language :: Python :: 3 - Programming Language :: Python :: 3.3 Programming Language :: Python :: 3.4 Topic :: Software Development :: Libraries Topic :: System :: Distributed Computing From 3e8eb915b55dfd6577a9cdb313afc71fd6d44b1a Mon Sep 17 00:00:00 2001 From: Dan Krause Date: Tue, 3 Mar 2015 10:57:09 -0600 Subject: [PATCH 048/246] Persistence backend refactor Factors lots of duplicate code out of persistence backends Adds get_flows_for_book to all backends Change-Id: I0434bd4931cd9274876f9e9c92909531f244bcac --- taskflow/persistence/backends/impl_dir.py | 404 +++--------------- taskflow/persistence/backends/impl_memory.py | 293 ++++--------- .../persistence/backends/impl_sqlalchemy.py | 21 +- .../persistence/backends/impl_zookeeper.py | 390 +++-------------- taskflow/persistence/base.py | 5 + taskflow/persistence/path_based.py | 244 +++++++++++ 6 files changed, 474 insertions(+), 883 deletions(-) create mode 100644 taskflow/persistence/path_based.py diff --git a/taskflow/persistence/backends/impl_dir.py b/taskflow/persistence/backends/impl_dir.py index 155ffe41..5ee244f7 100644 --- a/taskflow/persistence/backends/impl_dir.py +++ b/taskflow/persistence/backends/impl_dir.py @@ -15,6 +15,7 @@ # License for the specific language governing permissions and limitations # under the License. +import contextlib import errno import os import shutil @@ -23,25 +24,26 @@ from oslo_serialization import jsonutils import six from taskflow import exceptions as exc -from taskflow import logging -from taskflow.persistence import base -from taskflow.persistence import logbook +from taskflow.persistence import path_based from taskflow.utils import lock_utils from taskflow.utils import misc -LOG = logging.getLogger(__name__) + +@contextlib.contextmanager +def _storagefailure_wrapper(): + try: + yield + except exc.TaskFlowException: + raise + except Exception as e: + if isinstance(e, (IOError, OSError)) and e.errno == errno.ENOENT: + raise exc.NotFound('Item not found: %s' % e.filename, e) + raise exc.StorageFailure("Storage backend internal error", e) -class DirBackend(base.Backend): +class DirBackend(path_based.PathBasedBackend): """A directory and file based backend. - This backend writes logbooks, flow details, and atom details to a provided - base path on the local filesystem. It will create and store those objects - in three key directories (one for logbooks, one for flow details and one - for atom details). It creates those associated directories and then - creates files inside those directories that represent the contents of those - objects for later reading and writing. - This backend does *not* provide true transactional semantics. It does guarantee that there will be no interprocess race conditions when writing and reading by using a consistent hierarchy of file based locks. @@ -54,17 +56,10 @@ class DirBackend(base.Backend): """ def __init__(self, conf): super(DirBackend, self).__init__(conf) - self._path = os.path.abspath(conf['path']) - self._lock_path = os.path.join(self._path, 'locks') - self._file_cache = {} - - @property - def lock_path(self): - return self._lock_path - - @property - def base_path(self): - return self._path + self.file_cache = {} + if not self._path: + raise ValueError("Empty path is disallowed") + self._path = os.path.abspath(self._path) def get_connection(self): return Connection(self) @@ -73,33 +68,13 @@ class DirBackend(base.Backend): pass -class Connection(base.Connection): - def __init__(self, backend): - self._backend = backend - self._file_cache = self._backend._file_cache - self._flow_path = os.path.join(self._backend.base_path, 'flows') - self._atom_path = os.path.join(self._backend.base_path, 'atoms') - self._book_path = os.path.join(self._backend.base_path, 'books') - - def validate(self): - # Verify key paths exist. - paths = [ - self._backend.base_path, - self._backend.lock_path, - self._flow_path, - self._atom_path, - self._book_path, - ] - for p in paths: - if not os.path.isdir(p): - raise RuntimeError("Missing required directory: %s" % (p)) - +class Connection(path_based.PathBasedConnection): def _read_from(self, filename): # This is very similar to the oslo-incubator fileutils module, but # tweaked to not depend on a global cache, as well as tweaked to not # pull-in the oslo logging module (which is a huge pile of code). mtime = os.path.getmtime(filename) - cache_info = self._file_cache.setdefault(filename, {}) + cache_info = self.backend.file_cache.setdefault(filename, {}) if not cache_info or mtime > cache_info.get('mtime', 0): with open(filename, 'rb') as fp: cache_info['data'] = fp.read().decode('utf-8') @@ -111,301 +86,56 @@ class Connection(base.Connection): contents = contents.encode('utf-8') with open(filename, 'wb') as fp: fp.write(contents) - self._file_cache.pop(filename, None) + self.backend.file_cache.pop(filename, None) - def _run_with_process_lock(self, lock_name, functor, *args, **kwargs): - lock_path = os.path.join(self.backend.lock_path, lock_name) - with lock_utils.InterProcessLock(lock_path): + @contextlib.contextmanager + def _path_lock(self, path): + lockfile = self._join_path(path, 'lock') + with lock_utils.InterProcessLock(lockfile) as lock: + with _storagefailure_wrapper(): + yield lock + + def _join_path(self, *parts): + return os.path.join(*parts) + + def _get_item(self, path): + with self._path_lock(path): + item_path = self._join_path(path, 'metadata') + return misc.decode_json(self._read_from(item_path)) + + def _set_item(self, path, value, transaction): + with self._path_lock(path): + item_path = self._join_path(path, 'metadata') + self._write_to(item_path, jsonutils.dumps(value)) + + def _del_tree(self, path, transaction): + with self._path_lock(path): + shutil.rmtree(path) + + def _get_children(self, path): + with _storagefailure_wrapper(): + return [link for link in os.listdir(path) + if os.path.islink(self._join_path(path, link))] + + def _ensure_path(self, path): + with _storagefailure_wrapper(): + misc.ensure_tree(path) + + def _create_link(self, src_path, dest_path, transaction): + with _storagefailure_wrapper(): try: - return functor(*args, **kwargs) - except exc.TaskFlowException: - raise - except Exception as e: - LOG.exception("Failed running locking file based session") - # NOTE(harlowja): trap all other errors as storage errors. - raise exc.StorageFailure("Storage backend internal error", e) - - def _get_logbooks(self): - lb_uuids = [] - try: - lb_uuids = [d for d in os.listdir(self._book_path) - if os.path.isdir(os.path.join(self._book_path, d))] - except EnvironmentError as e: - if e.errno != errno.ENOENT: - raise - for lb_uuid in lb_uuids: - try: - yield self._get_logbook(lb_uuid) - except exc.NotFound: - pass - - def get_logbooks(self): - try: - books = list(self._get_logbooks()) - except EnvironmentError as e: - raise exc.StorageFailure("Unable to fetch logbooks", e) - else: - for b in books: - yield b - - @property - def backend(self): - return self._backend - - def close(self): - pass - - def _save_atom_details(self, atom_detail, ignore_missing): - # See if we have an existing atom detail to merge with. - e_ad = None - try: - e_ad = self._get_atom_details(atom_detail.uuid, lock=False) - except EnvironmentError: - if not ignore_missing: - raise exc.NotFound("No atom details found with id: %s" - % atom_detail.uuid) - if e_ad is not None: - atom_detail = e_ad.merge(atom_detail) - ad_path = os.path.join(self._atom_path, atom_detail.uuid) - ad_data = base._format_atom(atom_detail) - self._write_to(ad_path, jsonutils.dumps(ad_data)) - return atom_detail - - def update_atom_details(self, atom_detail): - return self._run_with_process_lock("atom", - self._save_atom_details, - atom_detail, - ignore_missing=False) - - def _get_atom_details(self, uuid, lock=True): - - def _get(): - ad_path = os.path.join(self._atom_path, uuid) - ad_data = misc.decode_json(self._read_from(ad_path)) - ad_cls = logbook.atom_detail_class(ad_data['type']) - return ad_cls.from_dict(ad_data['atom']) - - if lock: - return self._run_with_process_lock('atom', _get) - else: - return _get() - - def _get_flow_details(self, uuid, lock=True): - - def _get(): - fd_path = os.path.join(self._flow_path, uuid) - meta_path = os.path.join(fd_path, 'metadata') - meta = misc.decode_json(self._read_from(meta_path)) - fd = logbook.FlowDetail.from_dict(meta) - ad_to_load = [] - ad_path = os.path.join(fd_path, 'atoms') - try: - ad_to_load = [f for f in os.listdir(ad_path) - if os.path.islink(os.path.join(ad_path, f))] - except EnvironmentError as e: - if e.errno != errno.ENOENT: - raise - for ad_uuid in ad_to_load: - fd.add(self._get_atom_details(ad_uuid)) - return fd - - if lock: - return self._run_with_process_lock('flow', _get) - else: - return _get() - - def _save_atoms_and_link(self, atom_details, local_atom_path): - for atom_detail in atom_details: - self._save_atom_details(atom_detail, ignore_missing=True) - src_ad_path = os.path.join(self._atom_path, atom_detail.uuid) - target_ad_path = os.path.join(local_atom_path, atom_detail.uuid) - try: - os.symlink(src_ad_path, target_ad_path) - except EnvironmentError as e: + os.symlink(src_path, dest_path) + except OSError as e: if e.errno != errno.EEXIST: raise - def _save_flow_details(self, flow_detail, ignore_missing): - # See if we have an existing flow detail to merge with. - e_fd = None - try: - e_fd = self._get_flow_details(flow_detail.uuid, lock=False) - except EnvironmentError: - if not ignore_missing: - raise exc.NotFound("No flow details found with id: %s" - % flow_detail.uuid) - if e_fd is not None: - e_fd = e_fd.merge(flow_detail) - for ad in flow_detail: - if e_fd.find(ad.uuid) is None: - e_fd.add(ad) - flow_detail = e_fd - flow_path = os.path.join(self._flow_path, flow_detail.uuid) - misc.ensure_tree(flow_path) - self._write_to(os.path.join(flow_path, 'metadata'), - jsonutils.dumps(flow_detail.to_dict())) - if len(flow_detail): - atom_path = os.path.join(flow_path, 'atoms') - misc.ensure_tree(atom_path) - self._run_with_process_lock('atom', - self._save_atoms_and_link, - list(flow_detail), atom_path) - return flow_detail + @contextlib.contextmanager + def _transaction(self): + """This backend doesn't support transactions""" + yield - def update_flow_details(self, flow_detail): - return self._run_with_process_lock("flow", - self._save_flow_details, - flow_detail, - ignore_missing=False) - - def _save_flows_and_link(self, flow_details, local_flow_path): - for flow_detail in flow_details: - self._save_flow_details(flow_detail, ignore_missing=True) - src_fd_path = os.path.join(self._flow_path, flow_detail.uuid) - target_fd_path = os.path.join(local_flow_path, flow_detail.uuid) - try: - os.symlink(src_fd_path, target_fd_path) - except EnvironmentError as e: - if e.errno != errno.EEXIST: - raise - - def _save_logbook(self, book): - # See if we have an existing logbook to merge with. - e_lb = None - try: - e_lb = self._get_logbook(book.uuid) - except exc.NotFound: - pass - if e_lb is not None: - e_lb = e_lb.merge(book) - for fd in book: - if e_lb.find(fd.uuid) is None: - e_lb.add(fd) - book = e_lb - book_path = os.path.join(self._book_path, book.uuid) - misc.ensure_tree(book_path) - self._write_to(os.path.join(book_path, 'metadata'), - jsonutils.dumps(book.to_dict(marshal_time=True))) - if len(book): - flow_path = os.path.join(book_path, 'flows') - misc.ensure_tree(flow_path) - self._run_with_process_lock('flow', - self._save_flows_and_link, - list(book), flow_path) - return book - - def save_logbook(self, book): - return self._run_with_process_lock("book", - self._save_logbook, book) - - def upgrade(self): - - def _step_create(): - for path in (self._book_path, self._flow_path, self._atom_path): - try: - misc.ensure_tree(path) - except EnvironmentError as e: - raise exc.StorageFailure("Unable to create logbooks" - " required child path %s" % path, - e) - - for path in (self._backend.base_path, self._backend.lock_path): - try: - misc.ensure_tree(path) - except EnvironmentError as e: - raise exc.StorageFailure("Unable to create logbooks required" - " path %s" % path, e) - - self._run_with_process_lock("init", _step_create) - - def clear_all(self): - - def _step_clear(): - for d in (self._book_path, self._flow_path, self._atom_path): - if os.path.isdir(d): - shutil.rmtree(d) - - def _step_atom(): - self._run_with_process_lock("atom", _step_clear) - - def _step_flow(): - self._run_with_process_lock("flow", _step_atom) - - def _step_book(): - self._run_with_process_lock("book", _step_flow) - - # Acquire all locks by going through this little hierarchy. - self._run_with_process_lock("init", _step_book) - - def destroy_logbook(self, book_uuid): - - def _destroy_atoms(atom_details): - for atom_detail in atom_details: - atom_path = os.path.join(self._atom_path, atom_detail.uuid) - try: - shutil.rmtree(atom_path) - except EnvironmentError as e: - if e.errno != errno.ENOENT: - raise exc.StorageFailure("Unable to remove atom" - " directory %s" % atom_path, - e) - - def _destroy_flows(flow_details): - for flow_detail in flow_details: - flow_path = os.path.join(self._flow_path, flow_detail.uuid) - self._run_with_process_lock("atom", _destroy_atoms, - list(flow_detail)) - try: - shutil.rmtree(flow_path) - except EnvironmentError as e: - if e.errno != errno.ENOENT: - raise exc.StorageFailure("Unable to remove flow" - " directory %s" % flow_path, - e) - - def _destroy_book(): - book = self._get_logbook(book_uuid) - book_path = os.path.join(self._book_path, book.uuid) - self._run_with_process_lock("flow", _destroy_flows, list(book)) - try: - shutil.rmtree(book_path) - except EnvironmentError as e: - if e.errno != errno.ENOENT: - raise exc.StorageFailure("Unable to remove book" - " directory %s" % book_path, e) - - # Acquire all locks by going through this little hierarchy. - self._run_with_process_lock("book", _destroy_book) - - def _get_logbook(self, book_uuid): - book_path = os.path.join(self._book_path, book_uuid) - meta_path = os.path.join(book_path, 'metadata') - try: - meta = misc.decode_json(self._read_from(meta_path)) - except EnvironmentError as e: - if e.errno == errno.ENOENT: - raise exc.NotFound("No logbook found with id: %s" % book_uuid) - else: - raise - lb = logbook.LogBook.from_dict(meta, unmarshal_time=True) - fd_path = os.path.join(book_path, 'flows') - fd_uuids = [] - try: - fd_uuids = [f for f in os.listdir(fd_path) - if os.path.islink(os.path.join(fd_path, f))] - except EnvironmentError as e: - if e.errno != errno.ENOENT: - raise - for fd_uuid in fd_uuids: - lb.add(self._get_flow_details(fd_uuid)) - return lb - - def get_logbook(self, book_uuid): - return self._run_with_process_lock("book", - self._get_logbook, book_uuid) - - def get_flow_details(self, fd_uuid): - return self._get_flow_details(fd_uuid) - - def get_atom_details(self, ad_uuid): - return self._get_atom_details(ad_uuid) + def validate(self): + with _storagefailure_wrapper(): + for p in (self.flow_path, self.atom_path, self.book_path): + if not os.path.isdir(p): + raise RuntimeError("Missing required directory: %s" % (p)) diff --git a/taskflow/persistence/backends/impl_memory.py b/taskflow/persistence/backends/impl_memory.py index 020deff8..7fc22e6d 100644 --- a/taskflow/persistence/backends/impl_memory.py +++ b/taskflow/persistence/backends/impl_memory.py @@ -15,127 +15,32 @@ # License for the specific language governing permissions and limitations # under the License. -import functools - -import six +import contextlib +import copy +import os from taskflow import exceptions as exc -from taskflow import logging -from taskflow.persistence import base -from taskflow.persistence import logbook +from taskflow.persistence import path_based +from taskflow.types import tree from taskflow.utils import lock_utils -LOG = logging.getLogger(__name__) - -class _Memory(object): - """Where the data is really stored.""" - - def __init__(self): - self.log_books = {} - self.flow_details = {} - self.atom_details = {} - - def clear_all(self): - self.log_books.clear() - self.flow_details.clear() - self.atom_details.clear() - - -class _MemoryHelper(object): - """Helper functionality for the memory backends & connections.""" - - def __init__(self, memory): - self._memory = memory - - @staticmethod - def _fetch_clone_args(incoming): - if isinstance(incoming, (logbook.LogBook, logbook.FlowDetail)): - # We keep our own copy of the added contents of the following - # types so we don't need the clone to retain them directly... - return { - 'retain_contents': False, - } - return {} - - def construct(self, uuid, container): - """Reconstructs a object from the given uuid and storage container.""" - source = container[uuid] - clone_kwargs = self._fetch_clone_args(source) - clone = source['object'].copy(**clone_kwargs) - rebuilder = source.get('rebuilder') - if rebuilder: - for component in map(rebuilder, source['components']): - clone.add(component) - return clone - - def merge(self, incoming, saved_info=None): - """Merges the incoming object into the local memories copy.""" - if saved_info is None: - if isinstance(incoming, logbook.LogBook): - saved_info = self._memory.log_books.setdefault( - incoming.uuid, {}) - elif isinstance(incoming, logbook.FlowDetail): - saved_info = self._memory.flow_details.setdefault( - incoming.uuid, {}) - elif isinstance(incoming, logbook.AtomDetail): - saved_info = self._memory.atom_details.setdefault( - incoming.uuid, {}) - else: - raise TypeError("Unknown how to merge '%s' (%s)" - % (incoming, type(incoming))) - try: - saved_info['object'].merge(incoming) - except KeyError: - clone_kwargs = self._fetch_clone_args(incoming) - saved_info['object'] = incoming.copy(**clone_kwargs) - if isinstance(incoming, logbook.LogBook): - flow_details = saved_info.setdefault('components', set()) - if 'rebuilder' not in saved_info: - saved_info['rebuilder'] = functools.partial( - self.construct, container=self._memory.flow_details) - for flow_detail in incoming: - flow_details.add(self.merge(flow_detail)) - elif isinstance(incoming, logbook.FlowDetail): - atom_details = saved_info.setdefault('components', set()) - if 'rebuilder' not in saved_info: - saved_info['rebuilder'] = functools.partial( - self.construct, container=self._memory.atom_details) - for atom_detail in incoming: - atom_details.add(self.merge(atom_detail)) - return incoming.uuid - - -class MemoryBackend(base.Backend): +class MemoryBackend(path_based.PathBasedBackend): """A in-memory (non-persistent) backend. This backend writes logbooks, flow details, and atom details to in-memory dictionaries and retrieves from those dictionaries as needed. + + This backend does *not* provide true transactional semantics. It does + guarantee that there will be no inter-thread race conditions when + writing and reading by using a read/write locks. """ def __init__(self, conf=None): super(MemoryBackend, self).__init__(conf) - self._memory = _Memory() - self._helper = _MemoryHelper(self._memory) - self._lock = lock_utils.ReaderWriterLock() - - def _construct_from(self, container): - return dict((uuid, self._helper.construct(uuid, container)) - for uuid in six.iterkeys(container)) - - @property - def log_books(self): - with self._lock.read_lock(): - return self._construct_from(self._memory.log_books) - - @property - def flow_details(self): - with self._lock.read_lock(): - return self._construct_from(self._memory.flow_details) - - @property - def atom_details(self): - with self._lock.read_lock(): - return self._construct_from(self._memory.atom_details) + if self._path is None: + self._path = os.sep + self.memory = tree.Node(self._path) + self.lock = lock_utils.ReaderWriterLock() def get_connection(self): return Connection(self) @@ -144,107 +49,79 @@ class MemoryBackend(base.Backend): pass -class Connection(base.Connection): - """A connection to an in-memory backend.""" - +class Connection(path_based.PathBasedConnection): def __init__(self, backend): - self._backend = backend - self._helper = backend._helper - self._memory = backend._memory - self._lock = backend._lock + super(Connection, self).__init__(backend) + self.upgrade() - def upgrade(self): - pass + @contextlib.contextmanager + def _memory_lock(self, write=False): + if write: + lock = self.backend.lock.write_lock + else: + lock = self.backend.lock.read_lock + + with lock(): + try: + yield + except exc.TaskFlowException as e: + raise + except Exception as e: + raise exc.StorageFailure("Storage backend internal error", e) + + def _fetch_node(self, path): + node = self.backend.memory.find(path) + if node is None: + raise exc.NotFound("Item not found %s" % path) + return node + + def _join_path(self, *parts): + return os.path.join(*parts) + + def _get_item(self, path): + with self._memory_lock(): + return copy.deepcopy(self._fetch_node(path).metadata['value']) + + def _set_item(self, path, value, transaction): + value = copy.deepcopy(value) + try: + item_node = self._fetch_node(path) + item_node.metadata.update(value=value) + except exc.NotFound: + dirname, basename = os.path.split(path) + parent_node = self._fetch_node(dirname) + parent_node.add(tree.Node(path, name=basename, value=value)) + + def _del_tree(self, path, transaction): + node = self._fetch_node(path) + node.disassociate() + + def _get_children(self, path): + with self._memory_lock(): + return [node.metadata['name'] for node in self._fetch_node(path)] + + def _ensure_path(self, path): + with self._memory_lock(write=True): + path = os.path.normpath(path) + parts = path.split(os.sep) + node = self.backend.memory + for p in range(len(parts) - 1): + node_path = os.sep.join(parts[:p + 2]) + try: + node = self._fetch_node(node_path) + except exc.NotFound: + node.add(tree.Node(node_path, name=parts[p + 1])) + + def _create_link(self, src_path, dest_path, transaction): + dirname, basename = os.path.split(dest_path) + parent_node = self._fetch_node(dirname) + parent_node.add(tree.Node(dest_path, name=basename, target=src_path)) + + @contextlib.contextmanager + def _transaction(self): + """This just wraps a global write-lock""" + with self._memory_lock(write=True): + yield def validate(self): pass - - @property - def backend(self): - return self._backend - - def close(self): - pass - - def clear_all(self): - with self._lock.write_lock(): - self._memory.clear_all() - - def destroy_logbook(self, book_uuid): - with self._lock.write_lock(): - try: - # Do the same cascading delete that the sql layer does. - book_info = self._memory.log_books.pop(book_uuid) - except KeyError: - raise exc.NotFound("No logbook found with uuid '%s'" - % book_uuid) - else: - while book_info['components']: - flow_uuid = book_info['components'].pop() - flow_info = self._memory.flow_details.pop(flow_uuid) - while flow_info['components']: - atom_uuid = flow_info['components'].pop() - self._memory.atom_details.pop(atom_uuid) - - def update_atom_details(self, atom_detail): - with self._lock.write_lock(): - try: - atom_info = self._memory.atom_details[atom_detail.uuid] - return self._helper.construct( - self._helper.merge(atom_detail, saved_info=atom_info), - self._memory.atom_details) - except KeyError: - raise exc.NotFound("No atom details found with uuid '%s'" - % atom_detail.uuid) - - def update_flow_details(self, flow_detail): - with self._lock.write_lock(): - try: - flow_info = self._memory.flow_details[flow_detail.uuid] - return self._helper.construct( - self._helper.merge(flow_detail, saved_info=flow_info), - self._memory.flow_details) - except KeyError: - raise exc.NotFound("No flow details found with uuid '%s'" - % flow_detail.uuid) - - def save_logbook(self, book): - with self._lock.write_lock(): - return self._helper.construct(self._helper.merge(book), - self._memory.log_books) - - def get_logbook(self, book_uuid): - with self._lock.read_lock(): - try: - return self._helper.construct(book_uuid, - self._memory.log_books) - except KeyError: - raise exc.NotFound("No logbook found with uuid '%s'" - % book_uuid) - - def get_logbooks(self): - # Don't hold locks while iterating... - with self._lock.read_lock(): - book_uuids = set(six.iterkeys(self._memory.log_books)) - for book_uuid in book_uuids: - try: - with self._lock.read_lock(): - book = self._helper.construct(book_uuid, - self._memory.log_books) - yield book - except KeyError: - pass - - def get_flow_details(self, fd_uuid): - try: - with self._lock.read_lock(): - return self._memory.flow_details[fd_uuid] - except KeyError: - raise exc.NotFound("No flow details found '%s'" % fd_uuid) - - def get_atom_details(self, ad_uuid): - try: - with self._lock.read_lock(): - return self._memory.atom_details[ad_uuid] - except KeyError: - raise exc.NotFound("No atom details found '%s'" % ad_uuid) diff --git a/taskflow/persistence/backends/impl_sqlalchemy.py b/taskflow/persistence/backends/impl_sqlalchemy.py index a49d2492..4368b78a 100644 --- a/taskflow/persistence/backends/impl_sqlalchemy.py +++ b/taskflow/persistence/backends/impl_sqlalchemy.py @@ -202,25 +202,25 @@ class Alchemist(object): atom_cls = logbook.atom_detail_class(row.pop('atom_type')) return atom_cls.from_dict(row) - def _atom_query_iter(self, conn, parent_uuid): + def atom_query_iter(self, conn, parent_uuid): q = (sql.select([self._tables.atomdetails]). where(self._tables.atomdetails.c.parent_uuid == parent_uuid)) for row in conn.execute(q): yield self.convert_atom_detail(row) - def _flow_query_iter(self, conn, parent_uuid): + def flow_query_iter(self, conn, parent_uuid): q = (sql.select([self._tables.flowdetails]). where(self._tables.flowdetails.c.parent_uuid == parent_uuid)) for row in conn.execute(q): yield self.convert_flow_detail(row) def populate_book(self, conn, book): - for fd in self._flow_query_iter(conn, book.uuid): + for fd in self.flow_query_iter(conn, book.uuid): book.add(fd) self.populate_flow_detail(conn, fd) def populate_flow_detail(self, conn, fd): - for ad in self._atom_query_iter(conn, fd.uuid): + for ad in self.atom_query_iter(conn, fd.uuid): fd.add(ad) @@ -558,6 +558,19 @@ class Connection(base.Connection): for book in gathered: yield book + def get_flows_for_book(self, book_uuid): + gathered = [] + try: + with contextlib.closing(self._engine.connect()) as conn: + for row in self._converter.flow_query_iter(conn, book_uuid): + flow_details = self._converter.populate_flow_detail(conn, + row) + gathered.append(flow_details) + except sa_exc.DBAPIError as e: + raise exc.StorageFailure("Failed getting flow details", e) + for flow_details in gathered: + yield flow_details + def get_flow_details(self, fd_uuid): try: flowdetails = self._tables.flowdetails diff --git a/taskflow/persistence/backends/impl_zookeeper.py b/taskflow/persistence/backends/impl_zookeeper.py index ae8096f1..b60e93bf 100644 --- a/taskflow/persistence/backends/impl_zookeeper.py +++ b/taskflow/persistence/backends/impl_zookeeper.py @@ -1,6 +1,7 @@ # -*- coding: utf-8 -*- # Copyright (C) 2014 AT&T Labs All Rights Reserved. +# Copyright (C) 2015 Rackspace Hosting 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 @@ -21,30 +22,18 @@ from kazoo.protocol import paths from oslo_serialization import jsonutils from taskflow import exceptions as exc -from taskflow import logging -from taskflow.persistence import base -from taskflow.persistence import logbook +from taskflow.persistence import path_based from taskflow.utils import kazoo_utils as k_utils from taskflow.utils import misc -LOG = logging.getLogger(__name__) -# Transaction support was added in 3.4.0 MIN_ZK_VERSION = (3, 4, 0) -class ZkBackend(base.Backend): - """A zookeeper backend. - - This backend writes logbooks, flow details, and atom details to a provided - base path in zookeeper. It will create and store those objects in three - key directories (one for logbooks, one for flow details and one for atom - details). It creates those associated directories and then creates files - inside those directories that represent the contents of those objects for - later reading and writing. +class ZkBackend(path_based.PathBasedBackend): + """A zookeeper-backed backend. Example configuration:: - conf = { "hosts": "192.168.0.1:2181,192.168.0.2:2181,192.168.0.3:2181", "path": "/taskflow", @@ -52,24 +41,18 @@ class ZkBackend(base.Backend): """ def __init__(self, conf, client=None): super(ZkBackend, self).__init__(conf) - path = str(conf.get("path", "/taskflow")) - if not path: - raise ValueError("Empty zookeeper path is disallowed") - if not paths.isabs(path): + if not self._path: + self._path = '/taskflow' + if not paths.isabs(self._path): raise ValueError("Zookeeper path must be absolute") - self._path = path if client is not None: self._client = client self._owned = False else: - self._client = k_utils.make_client(conf) + self._client = k_utils.make_client(self._conf) self._owned = True self._validated = False - @property - def path(self): - return self._path - def get_connection(self): conn = ZkConnection(self, self._client, self._conf) if not self._validated: @@ -87,52 +70,15 @@ class ZkBackend(base.Backend): raise exc.StorageFailure("Unable to finalize client", e) -class ZkConnection(base.Connection): +class ZkConnection(path_based.PathBasedConnection): def __init__(self, backend, client, conf): - self._backend = backend - self._client = client + super(ZkConnection, self).__init__(backend) self._conf = conf - self._book_path = paths.join(self._backend.path, "books") - self._flow_path = paths.join(self._backend.path, "flow_details") - self._atom_path = paths.join(self._backend.path, "atom_details") + self._client = client with self._exc_wrapper(): # NOOP if already started. self._client.start() - def validate(self): - with self._exc_wrapper(): - try: - if self._conf.get('check_compatible', True): - k_utils.check_compatible(self._client, MIN_ZK_VERSION) - except exc.IncompatibleVersion as e: - raise exc.StorageFailure("Backend storage is not a" - " compatible version", e) - - @property - def backend(self): - return self._backend - - @property - def book_path(self): - return self._book_path - - @property - def flow_path(self): - return self._flow_path - - @property - def atom_path(self): - return self._atom_path - - def close(self): - pass - - def upgrade(self): - """Creates the initial paths (if they already don't exist).""" - with self._exc_wrapper(): - for path in (self.book_path, self.flow_path, self.atom_path): - self._client.ensure_path(path) - @contextlib.contextmanager def _exc_wrapper(self): """Exception context-manager which wraps kazoo exceptions. @@ -146,8 +92,7 @@ class ZkConnection(base.Connection): except self._client.handler.timeout_exception as e: raise exc.StorageFailure("Storage backend timeout", e) except k_exc.SessionExpiredError as e: - raise exc.StorageFailure("Storage backend session" - " has expired", e) + raise exc.StorageFailure("Storage backend session has expired", e) except k_exc.NoNodeError as e: raise exc.NotFound("Storage backend node not found: %s" % e) except k_exc.NodeExistsError as e: @@ -155,273 +100,50 @@ class ZkConnection(base.Connection): except (k_exc.KazooException, k_exc.ZookeeperError) as e: raise exc.StorageFailure("Storage backend internal error", e) - def update_atom_details(self, ad): - """Update a atom detail transactionally.""" - with self._exc_wrapper(): - txn = self._client.transaction() - ad = self._update_atom_details(ad, txn) - k_utils.checked_commit(txn) - return ad + def _join_path(self, *parts): + return paths.join(*parts) - def _update_atom_details(self, ad, txn, create_missing=False): - # Determine whether the desired data exists or not. - ad_path = paths.join(self.atom_path, ad.uuid) - e_ad = None - try: - ad_data, _zstat = self._client.get(ad_path) - except k_exc.NoNodeError: - # Not-existent: create or raise exception. - if not create_missing: - raise exc.NotFound("No atom details found with" - " id: %s" % ad.uuid) - else: - txn.create(ad_path) + def _get_item(self, path): + with self._exc_wrapper(): + data, _ = self._client.get(path) + return misc.decode_json(data) + + def _set_item(self, path, value, transaction): + data = misc.binary_encode(jsonutils.dumps(value)) + if not self._client.exists(path): + transaction.create(path, data) else: - # Existent: read it out. + transaction.set_data(path, data) + + def _del_tree(self, path, transaction): + for child in self._get_children(path): + self._del_tree(self._join_path(path, child), transaction) + transaction.delete(path) + + def _get_children(self, path): + with self._exc_wrapper(): + return self._client.get_children(path) + + def _ensure_path(self, path): + with self._exc_wrapper(): + self._client.ensure_path(path) + + def _create_link(self, src_path, dest_path, transaction): + if not self._client.exists(dest_path): + transaction.create(dest_path) + + @contextlib.contextmanager + def _transaction(self): + transaction = self._client.transaction() + with self._exc_wrapper(): + yield transaction + k_utils.checked_commit(transaction) + + def validate(self): + with self._exc_wrapper(): try: - ad_data = misc.decode_json(ad_data) - ad_cls = logbook.atom_detail_class(ad_data['type']) - e_ad = ad_cls.from_dict(ad_data['atom']) - except KeyError: - pass - - # Update and write it back - if e_ad: - e_ad = e_ad.merge(ad) - else: - e_ad = ad - ad_data = base._format_atom(e_ad) - txn.set_data(ad_path, - misc.binary_encode(jsonutils.dumps(ad_data))) - return e_ad - - def get_atom_details(self, ad_uuid): - """Read a atom detail. - - *Read-only*, so no need of zk transaction. - """ - with self._exc_wrapper(): - return self._get_atom_details(ad_uuid) - - def _get_atom_details(self, ad_uuid): - ad_path = paths.join(self.atom_path, ad_uuid) - try: - ad_data, _zstat = self._client.get(ad_path) - except k_exc.NoNodeError: - raise exc.NotFound("No atom details found with id: %s" % ad_uuid) - else: - ad_data = misc.decode_json(ad_data) - ad_cls = logbook.atom_detail_class(ad_data['type']) - return ad_cls.from_dict(ad_data['atom']) - - def update_flow_details(self, fd): - """Update a flow detail transactionally.""" - with self._exc_wrapper(): - txn = self._client.transaction() - fd = self._update_flow_details(fd, txn) - k_utils.checked_commit(txn) - return fd - - def _update_flow_details(self, fd, txn, create_missing=False): - # Determine whether the desired data exists or not - fd_path = paths.join(self.flow_path, fd.uuid) - try: - fd_data, _zstat = self._client.get(fd_path) - except k_exc.NoNodeError: - # Not-existent: create or raise exception - if create_missing: - txn.create(fd_path) - e_fd = logbook.FlowDetail(name=fd.name, uuid=fd.uuid) - else: - raise exc.NotFound("No flow details found with id: %s" - % fd.uuid) - else: - # Existent: read it out - e_fd = logbook.FlowDetail.from_dict(misc.decode_json(fd_data)) - - # Update and write it back - e_fd = e_fd.merge(fd) - fd_data = e_fd.to_dict() - txn.set_data(fd_path, misc.binary_encode(jsonutils.dumps(fd_data))) - for ad in fd: - ad_path = paths.join(fd_path, ad.uuid) - # NOTE(harlowja): create an entry in the flow detail path - # for the provided atom detail so that a reference exists - # from the flow detail to its atom details. - if not self._client.exists(ad_path): - txn.create(ad_path) - e_fd.add(self._update_atom_details(ad, txn, create_missing=True)) - return e_fd - - def get_flow_details(self, fd_uuid): - """Read a flow detail. - - *Read-only*, so no need of zk transaction. - """ - with self._exc_wrapper(): - return self._get_flow_details(fd_uuid) - - def _get_flow_details(self, fd_uuid): - fd_path = paths.join(self.flow_path, fd_uuid) - try: - fd_data, _zstat = self._client.get(fd_path) - except k_exc.NoNodeError: - raise exc.NotFound("No flow details found with id: %s" % fd_uuid) - - fd = logbook.FlowDetail.from_dict(misc.decode_json(fd_data)) - for ad_uuid in self._client.get_children(fd_path): - fd.add(self._get_atom_details(ad_uuid)) - return fd - - def save_logbook(self, lb): - """Save (update) a log_book transactionally.""" - - def _create_logbook(lb_path, txn): - lb_data = lb.to_dict(marshal_time=True) - txn.create(lb_path, misc.binary_encode(jsonutils.dumps(lb_data))) - for fd in lb: - # NOTE(harlowja): create an entry in the logbook path - # for the provided flow detail so that a reference exists - # from the logbook to its flow details. - txn.create(paths.join(lb_path, fd.uuid)) - fd_path = paths.join(self.flow_path, fd.uuid) - fd_data = jsonutils.dumps(fd.to_dict()) - txn.create(fd_path, misc.binary_encode(fd_data)) - for ad in fd: - # NOTE(harlowja): create an entry in the flow detail path - # for the provided atom detail so that a reference exists - # from the flow detail to its atom details. - txn.create(paths.join(fd_path, ad.uuid)) - ad_path = paths.join(self.atom_path, ad.uuid) - ad_data = base._format_atom(ad) - txn.create(ad_path, - misc.binary_encode(jsonutils.dumps(ad_data))) - return lb - - def _update_logbook(lb_path, lb_data, txn): - e_lb = logbook.LogBook.from_dict(misc.decode_json(lb_data), - unmarshal_time=True) - e_lb = e_lb.merge(lb) - lb_data = e_lb.to_dict(marshal_time=True) - txn.set_data(lb_path, misc.binary_encode(jsonutils.dumps(lb_data))) - for fd in lb: - fd_path = paths.join(lb_path, fd.uuid) - if not self._client.exists(fd_path): - # NOTE(harlowja): create an entry in the logbook path - # for the provided flow detail so that a reference exists - # from the logbook to its flow details. - txn.create(fd_path) - e_fd = self._update_flow_details(fd, txn, create_missing=True) - e_lb.add(e_fd) - return e_lb - - with self._exc_wrapper(): - txn = self._client.transaction() - # Determine whether the desired data exists or not. - lb_path = paths.join(self.book_path, lb.uuid) - try: - lb_data, _zstat = self._client.get(lb_path) - except k_exc.NoNodeError: - # Create a new logbook since it doesn't exist. - e_lb = _create_logbook(lb_path, txn) - else: - # Otherwise update the existing logbook instead. - e_lb = _update_logbook(lb_path, lb_data, txn) - k_utils.checked_commit(txn) - return e_lb - - def _get_logbook(self, lb_uuid): - lb_path = paths.join(self.book_path, lb_uuid) - try: - lb_data, _zstat = self._client.get(lb_path) - except k_exc.NoNodeError: - raise exc.NotFound("No logbook found with id: %s" % lb_uuid) - else: - lb = logbook.LogBook.from_dict(misc.decode_json(lb_data), - unmarshal_time=True) - for fd_uuid in self._client.get_children(lb_path): - lb.add(self._get_flow_details(fd_uuid)) - return lb - - def get_logbook(self, lb_uuid): - """Read a logbook. - - *Read-only*, so no need of zk transaction. - """ - with self._exc_wrapper(): - return self._get_logbook(lb_uuid) - - def get_logbooks(self): - """Read all logbooks. - - *Read-only*, so no need of zk transaction. - """ - with self._exc_wrapper(): - for lb_uuid in self._client.get_children(self.book_path): - yield self._get_logbook(lb_uuid) - - def destroy_logbook(self, lb_uuid): - """Destroy (delete) a log_book transactionally.""" - - def _destroy_atom_details(ad_uuid, txn): - ad_path = paths.join(self.atom_path, ad_uuid) - if not self._client.exists(ad_path): - raise exc.NotFound("No atom details found with id: %s" - % ad_uuid) - txn.delete(ad_path) - - def _destroy_flow_details(fd_uuid, txn): - fd_path = paths.join(self.flow_path, fd_uuid) - if not self._client.exists(fd_path): - raise exc.NotFound("No flow details found with id: %s" - % fd_uuid) - for ad_uuid in self._client.get_children(fd_path): - _destroy_atom_details(ad_uuid, txn) - txn.delete(paths.join(fd_path, ad_uuid)) - txn.delete(fd_path) - - def _destroy_logbook(lb_uuid, txn): - lb_path = paths.join(self.book_path, lb_uuid) - if not self._client.exists(lb_path): - raise exc.NotFound("No logbook found with id: %s" % lb_uuid) - for fd_uuid in self._client.get_children(lb_path): - _destroy_flow_details(fd_uuid, txn) - txn.delete(paths.join(lb_path, fd_uuid)) - txn.delete(lb_path) - - with self._exc_wrapper(): - txn = self._client.transaction() - _destroy_logbook(lb_uuid, txn) - k_utils.checked_commit(txn) - - def clear_all(self, delete_dirs=True): - """Delete all data transactionally.""" - with self._exc_wrapper(): - txn = self._client.transaction() - - # Delete all data under logbook path. - for lb_uuid in self._client.get_children(self.book_path): - lb_path = paths.join(self.book_path, lb_uuid) - for fd_uuid in self._client.get_children(lb_path): - txn.delete(paths.join(lb_path, fd_uuid)) - txn.delete(lb_path) - - # Delete all data under flow detail path. - for fd_uuid in self._client.get_children(self.flow_path): - fd_path = paths.join(self.flow_path, fd_uuid) - for ad_uuid in self._client.get_children(fd_path): - txn.delete(paths.join(fd_path, ad_uuid)) - txn.delete(fd_path) - - # Delete all data under atom detail path. - for ad_uuid in self._client.get_children(self.atom_path): - ad_path = paths.join(self.atom_path, ad_uuid) - txn.delete(ad_path) - - # Delete containing directories. - if delete_dirs: - txn.delete(self.book_path) - txn.delete(self.atom_path) - txn.delete(self.flow_path) - - k_utils.checked_commit(txn) + if self._conf.get('check_compatible', True): + k_utils.check_compatible(self._client, MIN_ZK_VERSION) + except exc.IncompatibleVersion as e: + raise exc.StorageFailure("Backend storage is not a" + " compatible version", e) diff --git a/taskflow/persistence/base.py b/taskflow/persistence/base.py index 0ce09259..a1f120df 100644 --- a/taskflow/persistence/base.py +++ b/taskflow/persistence/base.py @@ -118,6 +118,11 @@ class Connection(object): """Return an iterable of logbook objects.""" pass + @abc.abstractmethod + def get_flows_for_book(self, book_uuid): + """Return an iterable of flowdetails for a given logbook uuid.""" + pass + @abc.abstractmethod def get_flow_details(self, fd_uuid): """Fetches a flowdetails object matching the given uuid.""" diff --git a/taskflow/persistence/path_based.py b/taskflow/persistence/path_based.py new file mode 100644 index 00000000..ea080257 --- /dev/null +++ b/taskflow/persistence/path_based.py @@ -0,0 +1,244 @@ +# -*- coding: utf-8 -*- + +# Copyright (C) 2015 Rackspace Hosting 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. + +import abc +import six + +from taskflow import exceptions as exc +from taskflow.persistence import base +from taskflow.persistence import logbook + + +@six.add_metaclass(abc.ABCMeta) +class PathBasedBackend(base.Backend): + """Base class for persistence backends that address data by path + + Subclasses of this backend write logbooks, flow details, and atom details + to a provided base path in some filesystem-like storage. They will create + and store those objects in three key directories (one for logbooks, one + for flow details and one for atom details). They create those associated + directories and then create files inside those directories that represent + the contents of those objects for later reading and writing. + """ + + def __init__(self, conf): + super(PathBasedBackend, self).__init__(conf) + if conf is None: + conf = {} + self._path = conf.get('path', None) + + @property + def path(self): + return self._path + + +@six.add_metaclass(abc.ABCMeta) +class PathBasedConnection(base.Connection): + def __init__(self, backend): + self._backend = backend + self._book_path = self._join_path(backend.path, "books") + self._flow_path = self._join_path(backend.path, "flow_details") + self._atom_path = self._join_path(backend.path, "atom_details") + + @staticmethod + def _serialize(obj): + if isinstance(obj, logbook.LogBook): + return obj.to_dict(marshal_time=True) + elif isinstance(obj, logbook.FlowDetail): + return obj.to_dict() + elif isinstance(obj, logbook.AtomDetail): + return base._format_atom(obj) + else: + raise exc.StorageFailure("Invalid storage class %s" % type(obj)) + + @staticmethod + def _deserialize(cls, data): + if issubclass(cls, logbook.LogBook): + return cls.from_dict(data, unmarshal_time=True) + elif issubclass(cls, logbook.FlowDetail): + return cls.from_dict(data) + elif issubclass(cls, logbook.AtomDetail): + atom_class = logbook.atom_detail_class(data['type']) + return atom_class.from_dict(data['atom']) + else: + raise exc.StorageFailure("Invalid storage class %s" % cls) + + @property + def backend(self): + return self._backend + + @property + def book_path(self): + return self._book_path + + @property + def flow_path(self): + return self._flow_path + + @property + def atom_path(self): + return self._atom_path + + @abc.abstractmethod + def _join_path(self, *parts): + """Accept path parts, and return a joined path""" + + @abc.abstractmethod + def _get_item(self, path): + """Fetch a single item from the backend""" + + @abc.abstractmethod + def _set_item(self, path, value, transaction): + """Write a single item to the backend""" + + @abc.abstractmethod + def _del_tree(self, path, transaction): + """Recursively deletes a folder from the backend.""" + + @abc.abstractmethod + def _get_children(self, path): + """Get a list of child items of a path""" + + @abc.abstractmethod + def _ensure_path(self, path): + """Recursively ensure that a path (folder) in the backend exists""" + + @abc.abstractmethod + def _create_link(self, src_path, dest_path, transaction): + """Create a symlink-like link between two paths""" + + @abc.abstractmethod + def _transaction(self): + """Context manager that yields a transaction""" + + def _get_obj_path(self, obj): + if isinstance(obj, logbook.LogBook): + path = self.book_path + elif isinstance(obj, logbook.FlowDetail): + path = self.flow_path + elif isinstance(obj, logbook.AtomDetail): + path = self.atom_path + else: + raise exc.StorageFailure("Invalid storage class %s" % type(obj)) + return self._join_path(path, obj.uuid) + + def _update_object(self, obj, transaction, ignore_missing=False): + path = self._get_obj_path(obj) + try: + item_data = self._get_item(path) + existing_obj = self._deserialize(type(obj), item_data) + obj = existing_obj.merge(obj) + except exc.NotFound: + if not ignore_missing: + raise + self._set_item(path, self._serialize(obj), transaction) + return obj + + def get_logbooks(self, lazy=False): + for book_uuid in self._get_children(self.book_path): + yield self.get_logbook(book_uuid, lazy) + + def get_logbook(self, book_uuid, lazy=False): + book_path = self._join_path(self.book_path, book_uuid) + book_data = self._get_item(book_path) + book = self._deserialize(logbook.LogBook, book_data) + if not lazy: + for flow_details in self.get_flows_for_book(book_uuid): + book.add(flow_details) + return book + + def save_logbook(self, book): + book_path = self._get_obj_path(book) + with self._transaction() as transaction: + self._update_object(book, transaction, ignore_missing=True) + for flow_details in book: + flow_path = self._get_obj_path(flow_details) + link_path = self._join_path(book_path, flow_details.uuid) + self._do_update_flow_details(flow_details, transaction, + ignore_missing=True) + self._create_link(flow_path, link_path, transaction) + return book + + def get_flows_for_book(self, book_uuid, lazy=False): + book_path = self._join_path(self.book_path, book_uuid) + for flow_uuid in self._get_children(book_path): + yield self.get_flow_details(flow_uuid, lazy) + + def get_flow_details(self, flow_uuid, lazy=False): + flow_path = self._join_path(self.flow_path, flow_uuid) + flow_data = self._get_item(flow_path) + flow_details = self._deserialize(logbook.FlowDetail, flow_data) + if not lazy: + for atom_details in self.get_atoms_for_flow(flow_uuid): + flow_details.add(atom_details) + return flow_details + + def _do_update_flow_details(self, flow_detail, transaction, + ignore_missing=False): + flow_path = self._get_obj_path(flow_detail) + self._update_object(flow_detail, transaction, ignore_missing) + for atom_details in flow_detail: + atom_path = self._get_obj_path(atom_details) + link_path = self._join_path(flow_path, atom_details.uuid) + self._create_link(atom_path, link_path, transaction) + self._update_object(atom_details, transaction, ignore_missing=True) + return flow_detail + + def update_flow_details(self, flow_detail, ignore_missing=False): + with self._transaction() as transaction: + return self._do_update_flow_details(flow_detail, transaction, + ignore_missing) + + def get_atoms_for_flow(self, flow_uuid): + flow_path = self._join_path(self.flow_path, flow_uuid) + for atom_uuid in self._get_children(flow_path): + yield self.get_atom_details(atom_uuid) + + def get_atom_details(self, atom_uuid): + atom_path = self._join_path(self.atom_path, atom_uuid) + atom_data = self._get_item(atom_path) + return self._deserialize(logbook.AtomDetail, atom_data) + + def update_atom_details(self, atom_detail, ignore_missing=False): + with self._transaction() as transaction: + return self._update_object(atom_detail, transaction, + ignore_missing) + + def _do_destroy_logbook(self, book_uuid, transaction): + book_path = self._join_path(self.book_path, book_uuid) + for flow_uuid in self._get_children(book_path): + flow_path = self._join_path(self.flow_path, flow_uuid) + for atom_uuid in self._get_children(flow_path): + atom_path = self._join_path(self.atom_path, atom_uuid) + self._del_tree(atom_path, transaction) + self._del_tree(flow_path, transaction) + self._del_tree(book_path, transaction) + + def destroy_logbook(self, book_uuid): + with self._transaction() as transaction: + return self._do_destroy_logbook(book_uuid, transaction) + + def clear_all(self): + with self._transaction() as transaction: + for path in (self.book_path, self.flow_path, self.atom_path): + self._del_tree(path, transaction) + + def upgrade(self): + for path in (self.book_path, self.flow_path, self.atom_path): + self._ensure_path(path) + + def close(self): + pass From 1f4e596e2399b3a062b4ff4f85328952c87af210 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Mon, 9 Mar 2015 16:30:46 -0700 Subject: [PATCH 049/246] Denote issue 17911 has been merged/accepted Instead of saying/when or if we can now say definitively that this issue has been coded up and merged. Change-Id: I61f56e2cf878212c1ae9e0c8bbdac1f73193f083 --- taskflow/types/failure.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/taskflow/types/failure.py b/taskflow/types/failure.py index 7df0f68c..35c81daf 100644 --- a/taskflow/types/failure.py +++ b/taskflow/types/failure.py @@ -126,9 +126,9 @@ class Failure(object): to have code ran when this happens, and this can cause issues and side-effects that the receiver would not have intended to have caused). - TODO(harlowja): when/if http://bugs.python.org/issue17911 merges and - becomes available for use we should be able to use that and simplify the - methods and contents of this object. + TODO(harlowja): use parts of http://bugs.python.org/issue17911 and the + backport at https://pypi.python.org/pypi/traceback2/ to (hopefully) + simplify the methods and contents of this object... """ DICT_VERSION = 1 From 1478f52c9ad8c21d9fa5396118bde54ab8cc6bd7 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Mon, 9 Mar 2015 18:27:04 -0700 Subject: [PATCH 050/246] Retain nested causes where/when we can On py3.x we should attempt to retain an exceptions causes (if any) so that they can be examined at a later time. This adds that change into gather and retain this information via a new failure causes property (that is lazily populated). This will aid in the change to traceback2 and the adjustments made in http://bugs.python.org/issue17911 so that we can in the future use its exception class instead. Change-Id: I8cadd7b70c2f638719155d96df4236fc40e18ccf --- taskflow/tests/unit/test_failure.py | 97 +++++++++++++++++++++++++++++ taskflow/types/failure.py | 78 ++++++++++++++++++++++- 2 files changed, 172 insertions(+), 3 deletions(-) diff --git a/taskflow/tests/unit/test_failure.py b/taskflow/tests/unit/test_failure.py index 793274e1..c8f83b9b 100644 --- a/taskflow/tests/unit/test_failure.py +++ b/taskflow/tests/unit/test_failure.py @@ -17,6 +17,8 @@ import sys import six +from six.moves import cPickle as pickle +import testtools from taskflow import exceptions from taskflow import test @@ -311,6 +313,101 @@ class NonAsciiExceptionsTestCase(test.TestCase): self.assertEqual(fail, copied) +@testtools.skipIf(not six.PY3, 'this test only works on python 3.x') +class FailureCausesTest(test.TestCase): + + @classmethod + def _raise_many(cls, messages): + if not messages: + return + msg = messages.pop(0) + e = RuntimeError(msg) + try: + cls._raise_many(messages) + raise e + except RuntimeError as e1: + six.raise_from(e, e1) + + def test_causes(self): + f = None + try: + self._raise_many(["Still still not working", + "Still not working", "Not working"]) + except RuntimeError: + f = failure.Failure() + + self.assertIsNotNone(f) + self.assertEqual(2, len(f.causes)) + self.assertEqual("Still not working", f.causes[0].exception_str) + self.assertEqual("Not working", f.causes[1].exception_str) + + f = f.causes[0] + self.assertEqual(1, len(f.causes)) + self.assertEqual("Not working", f.causes[0].exception_str) + + f = f.causes[0] + self.assertEqual(0, len(f.causes)) + + def test_causes_to_from_dict(self): + f = None + try: + self._raise_many(["Still still not working", + "Still not working", "Not working"]) + except RuntimeError: + f = failure.Failure() + + self.assertIsNotNone(f) + d_f = f.to_dict() + f = failure.Failure.from_dict(d_f) + self.assertEqual(2, len(f.causes)) + self.assertEqual("Still not working", f.causes[0].exception_str) + self.assertEqual("Not working", f.causes[1].exception_str) + + f = f.causes[0] + self.assertEqual(1, len(f.causes)) + self.assertEqual("Not working", f.causes[0].exception_str) + + f = f.causes[0] + self.assertEqual(0, len(f.causes)) + + def test_causes_pickle(self): + f = None + try: + self._raise_many(["Still still not working", + "Still not working", "Not working"]) + except RuntimeError: + f = failure.Failure() + + self.assertIsNotNone(f) + p_f = pickle.dumps(f) + f = pickle.loads(p_f) + + self.assertEqual(2, len(f.causes)) + self.assertEqual("Still not working", f.causes[0].exception_str) + self.assertEqual("Not working", f.causes[1].exception_str) + + f = f.causes[0] + self.assertEqual(1, len(f.causes)) + self.assertEqual("Not working", f.causes[0].exception_str) + + f = f.causes[0] + self.assertEqual(0, len(f.causes)) + + def test_causes_supress_context(self): + f = None + try: + try: + self._raise_many(["Still still not working", + "Still not working", "Not working"]) + except RuntimeError as e: + six.raise_from(e, None) + except RuntimeError: + f = failure.Failure() + + self.assertIsNotNone(f) + self.assertEqual([], list(f.causes)) + + class ExcInfoUtilsTest(test.TestCase): def test_copy_none(self): result = failure._copy_exc_info(None) diff --git a/taskflow/types/failure.py b/taskflow/types/failure.py index 7df0f68c..f251b00e 100644 --- a/taskflow/types/failure.py +++ b/taskflow/types/failure.py @@ -152,8 +152,10 @@ class Failure(object): self._exception_str = exc.exception_message(self._exc_info[1]) self._traceback_str = ''.join( traceback.format_tb(self._exc_info[2])) + self._causes = kwargs.pop('causes', None) else: - self._exc_info = exc_info # may be None + self._causes = kwargs.pop('causes', None) + self._exc_info = exc_info self._exception_str = kwargs.pop('exception_str') self._exc_type_names = tuple(kwargs.pop('exc_type_names', [])) self._traceback_str = kwargs.pop('traceback_str', None) @@ -172,7 +174,8 @@ class Failure(object): return True return (self._exc_type_names == other._exc_type_names and self.exception_str == other.exception_str - and self.traceback_str == other.traceback_str) + and self.traceback_str == other.traceback_str + and self.causes == other.causes) def matches(self, other): """Checks if another object is equivalent to this object. @@ -269,6 +272,66 @@ class Failure(object): return cls return None + @classmethod + def _extract_causes_iter(cls, exc_val): + seen = [exc_val] + causes = [exc_val] + while causes: + exc_val = causes.pop() + if exc_val is None: + continue + # See: https://www.python.org/dev/peps/pep-3134/ for why/what + # these are... + # + # '__cause__' attribute for explicitly chained exceptions + # '__context__' attribute for implicitly chained exceptions + # '__traceback__' attribute for the traceback + # + # See: https://www.python.org/dev/peps/pep-0415/ for why/what + # the '__suppress_context__' is/means/implies... + supress_context = getattr(exc_val, + '__suppress_context__', False) + if supress_context: + attr_lookups = ['__cause__'] + else: + attr_lookups = ['__cause__', '__context__'] + nested_exc_val = None + for attr_name in attr_lookups: + attr_val = getattr(exc_val, attr_name, None) + if attr_val is None: + continue + if attr_val not in seen: + nested_exc_val = attr_val + break + if nested_exc_val is not None: + exc_info = ( + type(nested_exc_val), + nested_exc_val, + getattr(nested_exc_val, '__traceback__', None), + ) + seen.append(nested_exc_val) + causes.append(nested_exc_val) + yield cls(exc_info=exc_info) + + @property + def causes(self): + """Tuple of all *inner* failure *causes* of this failure. + + NOTE(harlowja): Does **not** include the current failure (only + returns connected causes of this failure, if any). This property + is really only useful on 3.x or newer versions of python as older + versions do **not** have associated causes (the tuple will **always** + be empty on 2.x versions of python). + + Refer to :pep:`3134` and :pep:`409` and :pep:`415` for what + this is examining to find failure causes. + """ + if self._causes is not None: + return self._causes + else: + self._causes = tuple(self._extract_causes_iter(self.exception)) + return self._causes + def __str__(self): return self.pformat() @@ -323,6 +386,10 @@ class Failure(object): self._exc_info = tuple(_fill_iter(dct['exc_info'], 3)) else: self._exc_info = None + causes = dct.get('causes') + if causes is not None: + causes = tuple(self.from_dict(d) for d in causes) + self._causes = causes @classmethod def from_dict(cls, data): @@ -332,6 +399,9 @@ class Failure(object): if version != cls.DICT_VERSION: raise ValueError('Invalid dict version of failure object: %r' % version) + causes = data.get('causes') + if causes is not None: + data['causes'] = tuple(cls.from_dict(d) for d in causes) return cls(**data) def to_dict(self): @@ -341,6 +411,7 @@ class Failure(object): 'traceback_str': self.traceback_str, 'exc_type_names': list(self), 'version': self.DICT_VERSION, + 'causes': [f.to_dict() for f in self.causes], } def copy(self): @@ -348,4 +419,5 @@ class Failure(object): return Failure(exc_info=_copy_exc_info(self.exc_info), exception_str=self.exception_str, traceback_str=self.traceback_str, - exc_type_names=self._exc_type_names[:]) + exc_type_names=self._exc_type_names[:], + causes=self._causes) From 8c088696ed7b291e130b6bff107640face4f540b Mon Sep 17 00:00:00 2001 From: Dan Krause Date: Tue, 10 Mar 2015 12:56:35 -0500 Subject: [PATCH 051/246] added update_flow_metadata method to Storage class Change-Id: Ic018b6653acbe1cdba3e2ad20277aa17417c7cd2 --- taskflow/storage.py | 7 +++++++ taskflow/tests/unit/test_storage.py | 6 ++++++ 2 files changed, 13 insertions(+) diff --git a/taskflow/storage.py b/taskflow/storage.py index 449aafb6..9bd3affd 100644 --- a/taskflow/storage.py +++ b/taskflow/storage.py @@ -760,6 +760,13 @@ class Storage(object): self._flowdetail.state = state self._with_connection(self._save_flow_detail) + @lock_utils.write_locked + def update_flow_metadata(self, update_with): + """Update flowdetails metadata and save it.""" + if update_with: + self._flowdetail.meta.update(update_with) + self._with_connection(self._save_flow_detail) + @lock_utils.read_locked def get_flow_state(self): """Get state from flow details.""" diff --git a/taskflow/tests/unit/test_storage.py b/taskflow/tests/unit/test_storage.py index a27a811c..826c0467 100644 --- a/taskflow/tests/unit/test_storage.py +++ b/taskflow/tests/unit/test_storage.py @@ -200,6 +200,12 @@ class StorageTestMixin(object): "^Name 'xxx' is not mapped", s.fetch, 'xxx') + def test_flow_metadata_update(self): + s = self._get_storage() + update_with = {'test_data': True} + s.update_flow_metadata(update_with) + self.assertTrue(s._flowdetail.meta['test_data']) + def test_task_metadata_update_with_none(self): s = self._get_storage() s.ensure_atom(test_utils.NoopTask('my task')) From 7d3ae77dca02c72def7e3124459fe69971bf40a0 Mon Sep 17 00:00:00 2001 From: Dan Krause Date: Tue, 10 Mar 2015 12:35:50 -0500 Subject: [PATCH 052/246] Added EventTimeListner to record when events occur Change-Id: Iaa3342efb1a98109ca5eb382efcd6bde402d437c --- taskflow/listeners/timing.py | 36 +++++++++++++++++++++++++++ taskflow/tests/unit/test_listeners.py | 23 +++++++++++++++++ 2 files changed, 59 insertions(+) diff --git a/taskflow/listeners/timing.py b/taskflow/listeners/timing.py index a75717a7..bf9b5229 100644 --- a/taskflow/listeners/timing.py +++ b/taskflow/listeners/timing.py @@ -17,6 +17,7 @@ from __future__ import absolute_import import itertools +import time from debtcollector import moves @@ -117,3 +118,38 @@ class PrintingDurationListener(DurationListener): PrintingTimingListener = moves.moved_class( PrintingDurationListener, 'PrintingTimingListener', __name__, version="0.8", removal_version="?") + + +class EventTimeListener(base.Listener): + """Writes task, flow, and retry event timestamps to atom metadata.""" + def __init__(self, engine, + task_listen_for=base.DEFAULT_LISTEN_FOR, + flow_listen_for=base.DEFAULT_LISTEN_FOR, + retry_listen_for=base.DEFAULT_LISTEN_FOR): + super(EventTimeListener, self).__init__( + engine, task_listen_for=task_listen_for, + flow_listen_for=flow_listen_for, retry_listen_for=retry_listen_for) + + def _record_atom_event(self, state, atom_name): + meta_update = {'%s-timestamp' % state: time.time()} + try: + # Don't let storage failures throw exceptions in a listener method. + self._engine.storage.update_atom_metadata(atom_name, meta_update) + except exc.StorageFailure: + LOG.warn("Failure to store timestamp %s for atom %s", + meta_update, atom_name, exc_info=True) + + def _flow_receiver(self, state, details): + meta_update = {'%s-timestamp' % state: time.time()} + try: + # Don't let storage failures throw exceptions in a listener method. + self._engine.storage.update_flow_metadata(meta_update) + except exc.StorageFailure: + LOG.warn("Failure to store timestamp %s for flow %s", + meta_update, details['flow_name'], exc_info=True) + + def _task_receiver(self, state, details): + self._record_atom_event(state, details['task_name']) + + def _retry_receiver(self, state, details): + self._record_atom_event(state, details['retry_name']) diff --git a/taskflow/tests/unit/test_listeners.py b/taskflow/tests/unit/test_listeners.py index d6c64bbd..db46b7fb 100644 --- a/taskflow/tests/unit/test_listeners.py +++ b/taskflow/tests/unit/test_listeners.py @@ -235,6 +235,29 @@ class TestTimingListener(test.TestCase, EngineMakerMixin): exc_info=True) +class TestEventTimeListener(test.TestCase, EngineMakerMixin): + def test_event_time(self): + flow = lf.Flow('flow1').add(SleepyTask("task1", sleep_for=0.1)) + engine = self._make_engine(flow) + with timing.EventTimeListener(engine): + engine.run() + t_uuid = engine.storage.get_atom_uuid("task1") + td = engine.storage._flowdetail.find(t_uuid) + self.assertIsNotNone(td) + self.assertIsNotNone(td.meta) + running_field = '%s-timestamp' % states.RUNNING + success_field = '%s-timestamp' % states.SUCCESS + self.assertIn(running_field, td.meta) + self.assertIn(success_field, td.meta) + td_duration = td.meta[success_field] - td.meta[running_field] + self.assertGreaterEqual(0.1, td_duration) + fd_meta = engine.storage._flowdetail.meta + self.assertIn(running_field, fd_meta) + self.assertIn(success_field, fd_meta) + fd_duration = fd_meta[success_field] - fd_meta[running_field] + self.assertGreaterEqual(0.1, fd_duration) + + class TestLoggingListeners(test.TestCase, EngineMakerMixin): def _make_logger(self, level=logging.DEBUG): log = logging.getLogger( From b9b6576a41fbd5be7302a12a160abf8af37b0cc1 Mon Sep 17 00:00:00 2001 From: Dan Krause Date: Tue, 10 Mar 2015 20:50:41 -0500 Subject: [PATCH 053/246] update uses of TimingListener to DurationListener Change-Id: I10ac9a9c82b08d606c253080ca343008880305c4 --- taskflow/examples/timing_listener.py | 2 +- taskflow/tests/unit/test_listeners.py | 10 +++++----- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/taskflow/examples/timing_listener.py b/taskflow/examples/timing_listener.py index ab53a9aa..68d350ff 100644 --- a/taskflow/examples/timing_listener.py +++ b/taskflow/examples/timing_listener.py @@ -55,5 +55,5 @@ class VariableTask(task.Task): f = lf.Flow('root') f.add(VariableTask('a'), VariableTask('b'), VariableTask('c')) e = engines.load(f) -with timing.PrintingTimingListener(e): +with timing.PrintingDurationListener(e): e.run() diff --git a/taskflow/tests/unit/test_listeners.py b/taskflow/tests/unit/test_listeners.py index d6c64bbd..d55f7d3a 100644 --- a/taskflow/tests/unit/test_listeners.py +++ b/taskflow/tests/unit/test_listeners.py @@ -202,14 +202,14 @@ class TestClaimListener(test.TestCase, EngineMakerMixin): self.assertEqual(1, ran_states.count(states.WAITING)) -class TestTimingListener(test.TestCase, EngineMakerMixin): +class TestDurationListener(test.TestCase, EngineMakerMixin): def test_duration(self): with contextlib.closing(impl_memory.MemoryBackend()) as be: flow = lf.Flow("test") flow.add(SleepyTask("test-1", sleep_for=0.1)) (lb, fd) = persistence_utils.temporary_flow_detail(be) e = self._make_engine(flow, fd, be) - with timing.TimingListener(e): + with timing.DurationListener(e): e.run() t_uuid = e.storage.get_atom_uuid("test-1") td = fd.find(t_uuid) @@ -225,11 +225,11 @@ class TestTimingListener(test.TestCase, EngineMakerMixin): flow.add(test_utils.TaskNoRequiresNoReturns("test-1")) (lb, fd) = persistence_utils.temporary_flow_detail(be) e = self._make_engine(flow, fd, be) - timing_listener = timing.TimingListener(e) - with mock.patch.object(timing_listener._engine.storage, + duration_listener = timing.DurationListener(e) + with mock.patch.object(duration_listener._engine.storage, 'update_atom_metadata') as mocked_uam: mocked_uam.side_effect = exc.StorageFailure('Woot!') - with timing_listener: + with duration_listener: e.run() mocked_warn.assert_called_once_with(mock.ANY, mock.ANY, 'test-1', exc_info=True) From 8e624839c07553931d39567e4e778838bc55d7a1 Mon Sep 17 00:00:00 2001 From: Dan Krause Date: Mon, 9 Mar 2015 14:09:55 -0500 Subject: [PATCH 054/246] add _listeners_from_job method to Conductor base Change-Id: I4a58c2fd5a0c648ffe3f81e2170a622b6b0b43f0 --- taskflow/conductors/base.py | 26 ++++++++++++++++++++++---- 1 file changed, 22 insertions(+), 4 deletions(-) diff --git a/taskflow/conductors/base.py b/taskflow/conductors/base.py index 48344c53..33c4441d 100644 --- a/taskflow/conductors/base.py +++ b/taskflow/conductors/base.py @@ -88,10 +88,28 @@ class Conductor(object): store = dict(job.details["store"]) else: store = {} - return engines.load_from_detail(flow_detail, store=store, - engine=self._engine, - backend=self._persistence, - **self._engine_options) + engine = engines.load_from_detail(flow_detail, store=store, + engine=self._engine, + backend=self._persistence, + **self._engine_options) + for listener in self._listeners_from_job(job, engine): + listener.register() + return engine + + def _listeners_from_job(self, job, engine): + """Returns a list of listeners to be attached to an engine. + + This method should be overridden in order to attach listeners to + engines. It will be called once for each job, and the list returned + listeners will be added to the engine for this job. + + :param job: A job instance that is about to be run in an engine. + :param engine: The engine that listeners will be attached to. + :returns: a list of (unregistered) listener instances. + """ + # TODO(dkrause): Create a standard way to pass listeners or + # listener factories over the jobboard + return [] @lock_utils.locked def connect(self): From f0de22c18a9242f8e5d7a57d6d8a0e60161c42df Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Tue, 10 Mar 2015 12:54:00 -0700 Subject: [PATCH 055/246] Allow injected atom args to be persisted Instead of only storing injected atom arguments in memory allow for specifying those to be persisted; so that users who desire this feature can persist them (it defaults to being transient to retain the old API behavior). This also reworks the validating of engine dependencies to be more correct. It removes the validation of these dependencies from the prepare() method and moves them to a new engine validate() method; this allows users to prepare() the engine, then inject there atom non-transient arguments and then validate(); the validation would fail prior to this at preparation time since no injected arguments would exist and the user would not have the ability to inject any that target a specific atom, since the atom detail would not have been created yet (since that is populated in the prepartion method). Change-Id: I2846d0334db32a115592f850d85b206d9e6a3f07 --- doc/source/engines.rst | 7 +- taskflow/engines/action_engine/engine.py | 36 +++-- taskflow/engines/action_engine/runtime.py | 6 +- taskflow/engines/base.py | 19 ++- taskflow/storage.py | 172 +++++++++++++++++++--- taskflow/tests/unit/test_storage.py | 25 ++++ 6 files changed, 223 insertions(+), 42 deletions(-) diff --git a/doc/source/engines.rst b/doc/source/engines.rst index e2b85fbd..3ce19030 100644 --- a/doc/source/engines.rst +++ b/doc/source/engines.rst @@ -366,9 +366,10 @@ and that class interacts with the a and the :py:class:`~taskflow.storage.Storage` class uses the following lookup order to find (or fail) a atoms requirement lookup/request: -#. Injected atom specific arguments. -#. Transient injected arguments. -#. Non-transient injected arguments. +#. Transient injected atom specific arguments. +#. Non-transient injected atom specific arguments. +#. Transient injected arguments (flow specific). +#. Non-transient injected arguments (flow specific). #. First scope visited provider that produces the named result; note that if multiple providers are found in the same scope the *first* (the scope walkers yielded ordering defines what *first* means) that produced that diff --git a/taskflow/engines/action_engine/engine.py b/taskflow/engines/action_engine/engine.py index b40c5fd5..9fa0f5a5 100644 --- a/taskflow/engines/action_engine/engine.py +++ b/taskflow/engines/action_engine/engine.py @@ -20,6 +20,7 @@ import threading from concurrent import futures from oslo_utils import excutils +from oslo_utils import strutils import six from taskflow.engines.action_engine import compiler @@ -119,6 +120,7 @@ class ActionEngine(base.Engine): """ self.compile() self.prepare() + self.validate() runner = self._runtime.runner last_state = None with _start_stop(self._task_executor): @@ -168,10 +170,34 @@ class ActionEngine(base.Engine): def _ensure_storage(self): """Ensure all contained atoms exist in the storage unit.""" + transient = strutils.bool_from_string( + self._options.get('inject_transient', True)) for node in self._compilation.execution_graph.nodes_iter(): self.storage.ensure_atom(node) if node.inject: - self.storage.inject_atom_args(node.name, node.inject) + self.storage.inject_atom_args(node.name, + node.inject, + transient=transient) + + @lock_utils.locked + def validate(self): + if not self._storage_ensured: + raise exc.InvalidState("Can not validate an engine" + " which has not has its storage" + " populated") + # At this point we can check to ensure all dependencies are either + # flow/task provided or storage provided, if there are still missing + # dependencies then this flow will fail at runtime (which we can avoid + # by failing at validation time). + missing = set() + fetch = self.storage.fetch_unsatisfied_args + for node in self._compilation.execution_graph.nodes_iter(): + scope_walker = self._runtime.fetch_scopes_for(node) + missing.update(fetch(node.name, node.rebind, + scope_walker=scope_walker, + optional_args=node.optional)) + if missing: + raise exc.MissingDependencies(self._flow, sorted(missing)) @lock_utils.locked def prepare(self): @@ -186,14 +212,6 @@ class ActionEngine(base.Engine): self._ensure_storage() self._change_state(states.SUSPENDED) self._storage_ensured = True - # At this point we can check to ensure all dependencies are either - # flow/task provided or storage provided, if there are still missing - # dependencies then this flow will fail at runtime (which we can avoid - # by failing at preparation time). - external_provides = set(self.storage.fetch_all().keys()) - missing = self._flow.requires - external_provides - if missing: - raise exc.MissingDependencies(self._flow, sorted(missing)) # Reset everything back to pending (if we were previously reverted). if self.storage.get_flow_state() == states.REVERTED: self._runtime.reset_all() diff --git a/taskflow/engines/action_engine/runtime.py b/taskflow/engines/action_engine/runtime.py index 169a6415..d71ff65c 100644 --- a/taskflow/engines/action_engine/runtime.py +++ b/taskflow/engines/action_engine/runtime.py @@ -67,15 +67,15 @@ class Runtime(object): @misc.cachedproperty def retry_action(self): return ra.RetryAction(self._storage, self._atom_notifier, - self._fetch_scopes_for) + self.fetch_scopes_for) @misc.cachedproperty def task_action(self): return ta.TaskAction(self._storage, - self._atom_notifier, self._fetch_scopes_for, + self._atom_notifier, self.fetch_scopes_for, self._task_executor) - def _fetch_scopes_for(self, atom): + def fetch_scopes_for(self, atom): """Fetches a tuple of the visible scopes for the given atom.""" try: return self._scopes[atom] diff --git a/taskflow/engines/base.py b/taskflow/engines/base.py index 5e2263eb..632f626f 100644 --- a/taskflow/engines/base.py +++ b/taskflow/engines/base.py @@ -92,9 +92,18 @@ class Engine(object): """Performs any pre-run, but post-compilation actions. NOTE(harlowja): During preparation it is currently assumed that the - underlying storage will be initialized, all final dependencies - will be verified, the tasks will be reset and the engine will enter - the PENDING state. + underlying storage will be initialized, the atoms will be reset and + the engine will enter the PENDING state. + """ + + @abc.abstractmethod + def validate(self): + """Performs any pre-run, post-prepare validation actions. + + NOTE(harlowja): During validation all final dependencies + will be verified and ensured. This will by default check that all + atoms have satisfiable requirements (satisfied by some other + provider). """ @abc.abstractmethod @@ -105,8 +114,8 @@ class Engine(object): def suspend(self): """Attempts to suspend the engine. - If the engine is currently running tasks then this will attempt to - suspend future work from being started (currently active tasks can + If the engine is currently running atoms then this will attempt to + suspend future work from being started (currently active atoms can not currently be preempted) and move the engine into a suspend state which can then later be resumed from. """ diff --git a/taskflow/storage.py b/taskflow/storage.py index 449aafb6..d1f67c62 100644 --- a/taskflow/storage.py +++ b/taskflow/storage.py @@ -47,6 +47,13 @@ _TRANSIENT_PROVIDER = object() # can fail extraction during lookup or emit warning on result reception... _EXTRACTION_EXCEPTIONS = (IndexError, KeyError, ValueError, TypeError) +# Atom detail metadata key used to inject atom non-transient injected args. +META_INJECTED = 'injected' + +# Atom detail metadata key(s) used to set atom progress (with any details). +META_PROGRESS = 'progress' +META_PROGRESS_DETAILS = 'progress_details' + class _Provider(object): """A named symbol provider that produces a output at the given index.""" @@ -356,19 +363,19 @@ class Storage(object): :param details: any task specific progress details """ update_with = { - 'progress': progress, + META_PROGRESS: progress, } if details is not None: # NOTE(imelnikov): as we can update progress without # updating details (e.g. automatically from engine) # we save progress value with details, too. if details: - update_with['progress_details'] = { + update_with[META_PROGRESS_DETAILS] = { 'at_progress': progress, 'details': details, } else: - update_with['progress_details'] = None + update_with[META_PROGRESS_DETAILS] = None self._update_atom_metadata(task_name, update_with, expected_type=logbook.TaskDetail) @@ -382,7 +389,7 @@ class Storage(object): ad = self._atomdetail_by_name(task_name, expected_type=logbook.TaskDetail) try: - return ad.meta['progress'] + return ad.meta[META_PROGRESS] except KeyError: return 0.0 @@ -397,7 +404,7 @@ class Storage(object): ad = self._atomdetail_by_name(task_name, expected_type=logbook.TaskDetail) try: - return ad.meta['progress_details'] + return ad.meta[META_PROGRESS_DETAILS] except KeyError: return None @@ -504,8 +511,12 @@ class Storage(object): if self._reset_atom(ad, state): self._with_connection(self._save_atom_detail, ad) - def inject_atom_args(self, atom_name, pairs): - """Add **transient** values into storage for a specific atom only. + def inject_atom_args(self, atom_name, pairs, transient=True): + """Add values into storage for a specific atom only. + + :param transient: save the data in-memory only instead of persisting + the data to backend storage (useful for resource-like objects + or similar objects which can **not** be persisted) This method injects a dictionary/pairs of arguments for an atom so that when that atom is scheduled for execution it will have immediate access @@ -536,10 +547,26 @@ class Storage(object): """ if atom_name not in self._atom_name_to_uuid: raise exceptions.NotFound("Unknown atom name: %s" % atom_name) - with self._lock.write_lock(): + + def save_transient(): self._injected_args.setdefault(atom_name, {}) self._injected_args[atom_name].update(pairs) + def save_persistent(): + ad = self._atomdetail_by_name(atom_name) + injected = ad.meta.get(META_INJECTED) + if not injected: + injected = {} + injected.update(pairs) + ad.meta[META_INJECTED] = injected + self._with_connection(self._save_atom_detail, ad) + + with self._lock.write_lock(): + if transient: + save_transient() + else: + save_persistent() + @lock_utils.write_locked def inject(self, pairs, transient=False): """Add values into storage. @@ -648,11 +675,91 @@ class Storage(object): return many_handler(values) @lock_utils.read_locked - def fetch_all(self): - """Fetch all named results known so far. + def fetch_unsatisfied_args(self, atom_name, args_mapping, + scope_walker=None, optional_args=None): + """Fetch unsatisfied atom arguments using an atoms argument mapping. - NOTE(harlowja): should be used for debugging and testing purposes. + NOTE(harlowja): this takes into account the provided scope walker + atoms who should produce the required value at runtime, as well as + the transient/persistent flow and atom specific injected arguments. + It does **not** check if the providers actually have produced the + needed values; it just checks that they are registered to produce + it in the future. """ + + def _fetch_providers(name): + """Fetchs pair of (default providers, non-default providers).""" + default_providers = [] + non_default_providers = [] + for p in self._reverse_mapping.get(name, []): + if p.name in (_TRANSIENT_PROVIDER, self.injector_name): + default_providers.append(p) + else: + non_default_providers.append(p) + return default_providers, non_default_providers + + def _locate_providers(name): + """Finds the accessible *potential* providers.""" + default_providers, non_default_providers = _fetch_providers(name) + providers = [] + if non_default_providers: + if scope_walker is not None: + scope_iter = iter(scope_walker) + else: + scope_iter = iter([]) + for names in scope_iter: + for p in non_default_providers: + if p.name in names: + providers.append(p) + for p in default_providers: + if p.name is _TRANSIENT_PROVIDER: + results = self._transients + else: + try: + results = self._get(p.name, only_last=True) + except exceptions.NotFound: + results = {} + try: + _item_from_single(p, results, name) + except exceptions.NotFound: + pass + else: + providers.append(p) + return providers + + ad = self._atomdetail_by_name(atom_name) + if optional_args is None: + optional_args = [] + injected_sources = [ + self._injected_args.get(atom_name, {}), + ad.meta.get(META_INJECTED, {}), + ] + missing = set(six.iterkeys(args_mapping)) + for (bound_name, name) in six.iteritems(args_mapping): + if LOG.isEnabledFor(logging.BLATHER): + LOG.blather("Looking for %r <= %r for atom named: %s", + bound_name, name, atom_name) + if bound_name in optional_args: + LOG.blather("Argument %r is optional, skipping", bound_name) + missing.discard(bound_name) + continue + maybe_providers = 0 + for source in injected_sources: + if not source: + continue + if name in source: + maybe_providers += 1 + maybe_providers += len(_locate_providers(name)) + if maybe_providers: + LOG.blather("Atom %s will have %s potential providers" + " of %r <= %r", atom_name, maybe_providers, + bound_name, name) + missing.discard(bound_name) + return missing + + @lock_utils.read_locked + def fetch_all(self): + """Fetch all named results known so far.""" def many_handler(values): if len(values) > 1: return values @@ -671,6 +778,15 @@ class Storage(object): optional_args=None): """Fetch arguments for an atom using an atoms argument mapping.""" + def _extract_first_from(name, sources): + """Extracts/returns first occurence of key in list of dicts.""" + for i, source in enumerate(sources): + if not source: + continue + if name in source: + return (i, source[name]) + raise KeyError(name) + def _get_results(looking_for, provider): """Gets the results saved for a given provider.""" try: @@ -710,14 +826,16 @@ class Storage(object): if optional_args is None: optional_args = [] - if atom_name and atom_name not in self._atom_name_to_uuid: - raise exceptions.NotFound("Unknown atom name: %s" % atom_name) + if atom_name: + ad = self._atomdetail_by_name(atom_name) + injected_sources = [ + self._injected_args.get(atom_name, {}), + ad.meta.get(META_INJECTED, {}), + ] + else: + injected_sources = [] if not args_mapping: return {} - if atom_name: - injected_args = self._injected_args.get(atom_name, {}) - else: - injected_args = {} mapped_args = {} for (bound_name, name) in six.iteritems(args_mapping): if LOG.isEnabledFor(logging.BLATHER): @@ -726,16 +844,26 @@ class Storage(object): bound_name, name, atom_name) else: LOG.blather("Looking for %r <= %r", bound_name, name) - if name in injected_args: - value = injected_args[name] + try: + source_index, value = _extract_first_from(name, + injected_sources) mapped_args[bound_name] = value - LOG.blather("Matched %r <= %r to %r (from injected" - " values)", bound_name, name, value) - else: + if LOG.isEnabledFor(logging.BLATHER): + if source_index == 0: + LOG.blather("Matched %r <= %r to %r (from injected" + " atom-specific transient" + " values)", bound_name, name, value) + else: + LOG.blather("Matched %r <= %r to %r (from injected" + " atom-specific persistent" + " values)", bound_name, name, value) + except KeyError: try: possible_providers = self._reverse_mapping[name] except KeyError: if bound_name in optional_args: + LOG.blather("Argument %r is optional, skipping", + bound_name) continue raise exceptions.NotFound("Name %r is not mapped as a" " produced output by any" diff --git a/taskflow/tests/unit/test_storage.py b/taskflow/tests/unit/test_storage.py index a27a811c..7401282b 100644 --- a/taskflow/tests/unit/test_storage.py +++ b/taskflow/tests/unit/test_storage.py @@ -533,6 +533,31 @@ class StorageTestMixin(object): intention = s.get_atom_intention('my retry') self.assertEqual(intention, states.RETRY) + def test_inject_persistent_missing(self): + t = test_utils.ProgressingTask('my retry', requires=['x']) + s = self._get_storage() + s.ensure_atom(t) + missing = s.fetch_unsatisfied_args(t.name, t.rebind) + self.assertEqual(set(['x']), missing) + s.inject_atom_args(t.name, {'x': 2}, transient=False) + missing = s.fetch_unsatisfied_args(t.name, t.rebind) + self.assertEqual(set(), missing) + args = s.fetch_mapped_args(t.rebind, atom_name=t.name) + self.assertEqual(2, args['x']) + + def test_inject_persistent_and_transient_missing(self): + t = test_utils.ProgressingTask('my retry', requires=['x']) + s = self._get_storage() + s.ensure_atom(t) + missing = s.fetch_unsatisfied_args(t.name, t.rebind) + self.assertEqual(set(['x']), missing) + s.inject_atom_args(t.name, {'x': 2}, transient=False) + s.inject_atom_args(t.name, {'x': 3}, transient=True) + missing = s.fetch_unsatisfied_args(t.name, t.rebind) + self.assertEqual(set(), missing) + args = s.fetch_mapped_args(t.rebind, atom_name=t.name) + self.assertEqual(3, args['x']) + class StorageMemoryTest(StorageTestMixin, test.TestCase): def setUp(self): From 67f0f5146431523ca65a28b3fb66477ff2f490d0 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Mon, 2 Mar 2015 15:00:41 -0800 Subject: [PATCH 056/246] Use ordered[set/dict] to retain ordering Instead of using always using a set/dict which do not retain use a ordered set and a ordered dict for requires, optional, and provides and rebind mappings types so that the ordering of these containers is maintained later when they are used. These ordering can be useful depending on the atom type (such as in a map and reduce tasks). Partial-Bug: 1357117 Change-Id: I365d11bbba4aa221bc36ca15441acecf199b4d56 --- doc/source/arguments_and_results.rst | 8 +- taskflow/atom.py | 142 ++++++++++-------- taskflow/engines/action_engine/compiler.py | 17 ++- taskflow/patterns/graph_flow.py | 13 +- taskflow/tests/unit/test_flow_dependencies.py | 28 ++-- taskflow/tests/unit/test_task.py | 141 +++++++++-------- 6 files changed, 206 insertions(+), 143 deletions(-) diff --git a/doc/source/arguments_and_results.rst b/doc/source/arguments_and_results.rst index 714ee87f..e4f79990 100644 --- a/doc/source/arguments_and_results.rst +++ b/doc/source/arguments_and_results.rst @@ -74,8 +74,8 @@ ignored during inference (as these names have special meaning/usage in python). ... def execute(self, *args, **kwargs): ... pass ... - >>> UniTask().requires - frozenset([]) + >>> sorted(UniTask().requires) + [] .. make vim sphinx highlighter* happy** @@ -214,8 +214,8 @@ name of the value. ... def execute(self): ... return 42 ... - >>> TheAnswerReturningTask(provides='the_answer').provides - frozenset(['the_answer']) + >>> sorted(TheAnswerReturningTask(provides='the_answer').provides) + ['the_answer'] Returning a tuple +++++++++++++++++ diff --git a/taskflow/atom.py b/taskflow/atom.py index 2ade20e6..ebc5bad3 100644 --- a/taskflow/atom.py +++ b/taskflow/atom.py @@ -16,14 +16,28 @@ # under the License. import abc +import collections +import itertools + +try: + from collections import OrderedDict # noqa +except ImportError: + from ordereddict import OrderedDict # noqa from oslo_utils import reflection import six +from six.moves import zip as compat_zip from taskflow import exceptions +from taskflow.types import sets from taskflow.utils import misc +# Helper types tuples... +_sequence_types = (list, tuple, collections.Sequence) +_set_types = (set, collections.Set) + + def _save_as_to_mapping(save_as): """Convert save_as to mapping name => index. @@ -33,25 +47,26 @@ def _save_as_to_mapping(save_as): # outside of code so that it's more easily understandable, since what an # atom returns is pretty crucial for other later operations. if save_as is None: - return {} + return OrderedDict() if isinstance(save_as, six.string_types): # NOTE(harlowja): this means that your atom will only return one item # instead of a dictionary-like object or a indexable object (like a # list or tuple). - return {save_as: None} - elif isinstance(save_as, (tuple, list)): + return OrderedDict([(save_as, None)]) + elif isinstance(save_as, _sequence_types): # NOTE(harlowja): this means that your atom will return a indexable # object, like a list or tuple and the results can be mapped by index # to that tuple/list that is returned for others to use. - return dict((key, num) for num, key in enumerate(save_as)) - elif isinstance(save_as, set): + return OrderedDict((key, num) for num, key in enumerate(save_as)) + elif isinstance(save_as, _set_types): # NOTE(harlowja): in the case where a set is given we will not be - # able to determine the numeric ordering in a reliable way (since it is - # a unordered set) so the only way for us to easily map the result of - # the atom will be via the key itself. - return dict((key, key) for key in save_as) - raise TypeError('Atom provides parameter ' - 'should be str, set or tuple/list, not %r' % save_as) + # able to determine the numeric ordering in a reliable way (since it + # may be an unordered set) so the only way for us to easily map the + # result of the atom will be via the key itself. + return OrderedDict((key, key) for key in save_as) + else: + raise TypeError('Atom provides parameter ' + 'should be str, set or tuple/list, not %r' % save_as) def _build_rebind_dict(args, rebind_args): @@ -62,9 +77,9 @@ def _build_rebind_dict(args, rebind_args): new name onto the required name). """ if rebind_args is None: - return {} + return OrderedDict() elif isinstance(rebind_args, (list, tuple)): - rebind = dict(zip(args, rebind_args)) + rebind = OrderedDict(compat_zip(args, rebind_args)) if len(args) < len(rebind_args): rebind.update((a, a) for a in rebind_args[len(args):]) return rebind @@ -85,11 +100,11 @@ def _build_arg_mapping(atom_name, reqs, rebind_args, function, do_infer, extra arguments (where applicable). """ - # build a list of required arguments based on function signature + # Build a list of required arguments based on function signature. req_args = reflection.get_callable_args(function, required_only=True) all_args = reflection.get_callable_args(function, required_only=False) - # remove arguments that are part of ignore_list + # Remove arguments that are part of ignore list. if ignore_list: for arg in ignore_list: if arg in req_args: @@ -97,39 +112,45 @@ def _build_arg_mapping(atom_name, reqs, rebind_args, function, do_infer, else: ignore_list = [] - required = {} - # add reqs to required mappings + # Build the required names. + required = OrderedDict() + + # Add required arguments to required mappings if inference is enabled. + if do_infer: + required.update((a, a) for a in req_args) + + # Add additional manually provided requirements to required mappings. if reqs: if isinstance(reqs, six.string_types): required.update({reqs: reqs}) else: required.update((a, a) for a in reqs) - # add req_args to required mappings if do_infer is set - if do_infer: - required.update((a, a) for a in req_args) - - # update required mappings based on rebind_args + # Update required mappings values based on rebinding of arguments names. required.update(_build_rebind_dict(req_args, rebind_args)) + # Determine if there are optional arguments that we may or may not take. if do_infer: - opt_args = set(all_args) - set(required) - set(ignore_list) - optional = dict((a, a) for a in opt_args) + opt_args = sets.OrderedSet(all_args) + opt_args = opt_args - set(itertools.chain(six.iterkeys(required), + iter(ignore_list))) + optional = OrderedDict((a, a) for a in opt_args) else: - optional = {} + optional = OrderedDict() + # Check if we are given some extra arguments that we aren't able to accept. if not reflection.accepts_kwargs(function): - extra_args = set(required) - set(all_args) + extra_args = sets.OrderedSet(six.iterkeys(required)) + extra_args -= all_args if extra_args: - extra_args_str = ', '.join(sorted(extra_args)) raise ValueError('Extra arguments given to atom %s: %s' - % (atom_name, extra_args_str)) + % (atom_name, list(extra_args))) # NOTE(imelnikov): don't use set to preserve order in error message missing_args = [arg for arg in req_args if arg not in required] if missing_args: raise ValueError('Missing arguments for atom %s: %s' - % (atom_name, ' ,'.join(missing_args))) + % (atom_name, missing_args)) return required, optional @@ -161,52 +182,53 @@ class Atom(object): with this atom. It can be useful in resuming older versions of atoms. Standard major, minor versioning concepts should apply. - :ivar save_as: An *immutable* output ``resource`` name dictionary this atom - produces that other atoms may depend on this atom providing. - The format is output index (or key when a dictionary - is returned from the execute method) to stored argument - name. - :ivar rebind: An *immutable* input ``resource`` mapping dictionary that - can be used to alter the inputs given to this atom. It is - typically used for mapping a prior atoms output into + :ivar save_as: An *immutable* output ``resource`` name + :py:class:`.OrderedDict` this atom produces that other + atoms may depend on this atom providing. The format is + output index (or key when a dictionary is returned from + the execute method) to stored argument name. + :ivar rebind: An *immutable* input ``resource`` :py:class:`.OrderedDict` + that can be used to alter the inputs given to this atom. It + is typically used for mapping a prior atoms output into the names that this atom expects (in a way this is like remapping a namespace of another atom into the namespace of this atom). :ivar inject: See parameter ``inject``. :ivar name: See parameter ``name``. - :ivar requires: An *immutable* set of inputs this atom requires to - function. - :ivar optional: An *immutable* set of inputs that are optional for this - atom to function. - :ivar provides: An *immutable* set of outputs this atom produces. + :ivar requires: A :py:class:`~taskflow.types.sets.OrderedSet` of inputs + this atom requires to function. + :ivar optional: A :py:class:`~taskflow.types.sets.OrderedSet` of inputs + that are optional for this atom to function. + :ivar provides: A :py:class:`~taskflow.types.sets.OrderedSet` of outputs + this atom produces. """ def __init__(self, name=None, provides=None, inject=None): self.name = name - self.save_as = _save_as_to_mapping(provides) self.version = (1, 0) self.inject = inject - self.requires = frozenset() - self.optional = frozenset() - self.provides = frozenset(self.save_as) - self.rebind = {} + self.save_as = _save_as_to_mapping(provides) + self.requires = sets.OrderedSet() + self.optional = sets.OrderedSet() + self.provides = sets.OrderedSet(self.save_as) + self.rebind = OrderedDict() def _build_arg_mapping(self, executor, requires=None, rebind=None, auto_extract=True, ignore_list=None): - req_arg, opt_arg = _build_arg_mapping(self.name, requires, rebind, - executor, auto_extract, - ignore_list) - self.rebind.clear() - if opt_arg: - self.rebind.update(opt_arg) - if req_arg: - self.rebind.update(req_arg) - self.requires = frozenset(req_arg.values()) - self.optional = frozenset(opt_arg.values()) + required, optional = _build_arg_mapping(self.name, requires, rebind, + executor, auto_extract, + ignore_list=ignore_list) + rebind = OrderedDict() + for (arg_name, bound_name) in itertools.chain(six.iteritems(required), + six.iteritems(optional)): + rebind.setdefault(arg_name, bound_name) + self.rebind = rebind + self.requires = sets.OrderedSet(six.itervalues(required)) + self.optional = sets.OrderedSet(six.itervalues(optional)) if self.inject: - inject_set = set(six.iterkeys(self.inject)) - self.requires -= inject_set - self.optional -= inject_set + inject_keys = frozenset(six.iterkeys(self.inject)) + self.requires -= inject_keys + self.optional -= inject_keys out_of_order = self.provides.intersection(self.requires) if out_of_order: raise exceptions.DependencyFailure( diff --git a/taskflow/engines/action_engine/compiler.py b/taskflow/engines/action_engine/compiler.py index fb81ba80..9235af78 100644 --- a/taskflow/engines/action_engine/compiler.py +++ b/taskflow/engines/action_engine/compiler.py @@ -158,13 +158,22 @@ class Linker(object): " decomposed into an empty graph" % (v, u, u)) for u in u_g.nodes_iter(): for v in v_g.nodes_iter(): - depends_on = u.provides & v.requires + # This is using the intersection() method vs the & + # operator since the latter doesn't work with frozen + # sets (when used in combination with ordered sets). + # + # If this is not done the following happens... + # + # TypeError: unsupported operand type(s) + # for &: 'frozenset' and 'OrderedSet' + depends_on = u.provides.intersection(v.requires) if depends_on: + edge_attrs = { + _EDGE_REASONS: frozenset(depends_on), + } _add_update_edges(graph, [u], [v], - attr_dict={ - _EDGE_REASONS: depends_on, - }) + attr_dict=edge_attrs) else: # Connect nodes with no predecessors in v to nodes with no # successors in the *first* non-empty predecessor of v (thus diff --git a/taskflow/patterns/graph_flow.py b/taskflow/patterns/graph_flow.py index f71f285b..d36a72ff 100644 --- a/taskflow/patterns/graph_flow.py +++ b/taskflow/patterns/graph_flow.py @@ -27,11 +27,20 @@ def _unsatisfied_requires(node, graph, *additional_provided): if not requires: return requires for provided in additional_provided: - requires = requires - provided + # This is using the difference() method vs the - + # operator since the latter doesn't work with frozen + # or regular sets (when used in combination with ordered + # sets). + # + # If this is not done the following happens... + # + # TypeError: unsupported operand type(s) + # for -: 'set' and 'OrderedSet' + requires = requires.difference(provided) if not requires: return requires for pred in graph.bfs_predecessors_iter(node): - requires = requires - pred.provides + requires = requires.difference(pred.provides) if not requires: return requires return requires diff --git a/taskflow/tests/unit/test_flow_dependencies.py b/taskflow/tests/unit/test_flow_dependencies.py index 69f4a8fe..9627a696 100644 --- a/taskflow/tests/unit/test_flow_dependencies.py +++ b/taskflow/tests/unit/test_flow_dependencies.py @@ -27,40 +27,40 @@ class FlowDependenciesTest(test.TestCase): def test_task_without_dependencies(self): flow = utils.TaskNoRequiresNoReturns() - self.assertEqual(flow.requires, set()) - self.assertEqual(flow.provides, set()) + self.assertEqual(set(), flow.requires) + self.assertEqual(set(), flow.provides) def test_task_requires_default_values(self): flow = utils.TaskMultiArg() - self.assertEqual(flow.requires, set(['x', 'y', 'z'])) - self.assertEqual(flow.provides, set()) + self.assertEqual(set(['x', 'y', 'z']), flow.requires) + self.assertEqual(set(), flow.provides, ) def test_task_requires_rebinded_mapped(self): flow = utils.TaskMultiArg(rebind={'x': 'a', 'y': 'b', 'z': 'c'}) - self.assertEqual(flow.requires, set(['a', 'b', 'c'])) - self.assertEqual(flow.provides, set()) + self.assertEqual(set(['a', 'b', 'c']), flow.requires) + self.assertEqual(set(), flow.provides) def test_task_requires_additional_values(self): flow = utils.TaskMultiArg(requires=['a', 'b']) - self.assertEqual(flow.requires, set(['a', 'b', 'x', 'y', 'z'])) - self.assertEqual(flow.provides, set()) + self.assertEqual(set(['a', 'b', 'x', 'y', 'z']), flow.requires) + self.assertEqual(set(), flow.provides) def test_task_provides_values(self): flow = utils.TaskMultiReturn(provides=['a', 'b', 'c']) - self.assertEqual(flow.requires, set()) - self.assertEqual(flow.provides, set(['a', 'b', 'c'])) + self.assertEqual(set(), flow.requires) + self.assertEqual(set(['a', 'b', 'c']), flow.provides) def test_task_provides_and_requires_values(self): flow = utils.TaskMultiArgMultiReturn(provides=['a', 'b', 'c']) - self.assertEqual(flow.requires, set(['x', 'y', 'z'])) - self.assertEqual(flow.provides, set(['a', 'b', 'c'])) + self.assertEqual(set(['x', 'y', 'z']), flow.requires) + self.assertEqual(set(['a', 'b', 'c']), flow.provides) def test_linear_flow_without_dependencies(self): flow = lf.Flow('lf').add( utils.TaskNoRequiresNoReturns('task1'), utils.TaskNoRequiresNoReturns('task2')) - self.assertEqual(flow.requires, set()) - self.assertEqual(flow.provides, set()) + self.assertEqual(set(), flow.requires) + self.assertEqual(set(), flow.provides) def test_linear_flow_requires_values(self): flow = lf.Flow('lf').add( diff --git a/taskflow/tests/unit/test_task.py b/taskflow/tests/unit/test_task.py index 9a9ae1c9..72f98863 100644 --- a/taskflow/tests/unit/test_task.py +++ b/taskflow/tests/unit/test_task.py @@ -52,36 +52,36 @@ class TaskTest(test.TestCase): def test_passed_name(self): my_task = MyTask(name='my name') - self.assertEqual(my_task.name, 'my name') + self.assertEqual('my name', my_task.name) def test_generated_name(self): my_task = MyTask() - self.assertEqual(my_task.name, - '%s.%s' % (__name__, 'MyTask')) + self.assertEqual('%s.%s' % (__name__, 'MyTask'), + my_task.name) def test_task_str(self): my_task = MyTask(name='my') - self.assertEqual(str(my_task), 'my==1.0') + self.assertEqual('my==1.0', str(my_task)) def test_task_repr(self): my_task = MyTask(name='my') - self.assertEqual(repr(my_task), '<%s.MyTask my==1.0>' % __name__) + self.assertEqual('<%s.MyTask my==1.0>' % __name__, repr(my_task)) def test_no_provides(self): my_task = MyTask() - self.assertEqual(my_task.save_as, {}) + self.assertEqual({}, my_task.save_as) def test_provides(self): my_task = MyTask(provides='food') - self.assertEqual(my_task.save_as, {'food': None}) + self.assertEqual({'food': None}, my_task.save_as) def test_multi_provides(self): my_task = MyTask(provides=('food', 'water')) - self.assertEqual(my_task.save_as, {'food': 0, 'water': 1}) + self.assertEqual({'food': 0, 'water': 1}, my_task.save_as) def test_unpack(self): my_task = MyTask(provides=('food',)) - self.assertEqual(my_task.save_as, {'food': 0}) + self.assertEqual({'food': 0}, my_task.save_as) def test_bad_provides(self): self.assertRaisesRegexp(TypeError, '^Atom provides', @@ -89,28 +89,34 @@ class TaskTest(test.TestCase): def test_requires_by_default(self): my_task = MyTask() - self.assertEqual(my_task.rebind, { + expected = { 'spam': 'spam', 'eggs': 'eggs', 'context': 'context' - }) + } + self.assertEqual(expected, + my_task.rebind) + self.assertEqual(set(['spam', 'eggs', 'context']), + my_task.requires) def test_requires_amended(self): my_task = MyTask(requires=('spam', 'eggs')) - self.assertEqual(my_task.rebind, { + expected = { 'spam': 'spam', 'eggs': 'eggs', 'context': 'context' - }) + } + self.assertEqual(expected, my_task.rebind) def test_requires_explicit(self): my_task = MyTask(auto_extract=False, requires=('spam', 'eggs', 'context')) - self.assertEqual(my_task.rebind, { + expected = { 'spam': 'spam', 'eggs': 'eggs', 'context': 'context' - }) + } + self.assertEqual(expected, my_task.rebind) def test_requires_explicit_not_enough(self): self.assertRaisesRegexp(ValueError, '^Missing arguments', @@ -119,36 +125,43 @@ class TaskTest(test.TestCase): def test_requires_ignores_optional(self): my_task = DefaultArgTask() - self.assertEqual(my_task.requires, set(['spam'])) - self.assertEqual(my_task.optional, set(['eggs'])) + self.assertEqual(set(['spam']), my_task.requires) + self.assertEqual(set(['eggs']), my_task.optional) def test_requires_allows_optional(self): my_task = DefaultArgTask(requires=('spam', 'eggs')) - self.assertEqual(my_task.requires, set(['spam', 'eggs'])) - self.assertEqual(my_task.optional, set()) + self.assertEqual(set(['spam', 'eggs']), my_task.requires) + self.assertEqual(set(), my_task.optional) def test_rebind_includes_optional(self): my_task = DefaultArgTask() - self.assertEqual(my_task.rebind, { + expected = { 'spam': 'spam', 'eggs': 'eggs', - }) + } + self.assertEqual(expected, my_task.rebind) def test_rebind_all_args(self): my_task = MyTask(rebind={'spam': 'a', 'eggs': 'b', 'context': 'c'}) - self.assertEqual(my_task.rebind, { + expected = { 'spam': 'a', 'eggs': 'b', 'context': 'c' - }) + } + self.assertEqual(expected, my_task.rebind) + self.assertEqual(set(['a', 'b', 'c']), + my_task.requires) def test_rebind_partial(self): my_task = MyTask(rebind={'spam': 'a', 'eggs': 'b'}) - self.assertEqual(my_task.rebind, { + expected = { 'spam': 'a', 'eggs': 'b', 'context': 'context' - }) + } + self.assertEqual(expected, my_task.rebind) + self.assertEqual(set(['a', 'b', 'context']), + my_task.requires) def test_rebind_unknown(self): self.assertRaisesRegexp(ValueError, '^Extra arguments', @@ -156,26 +169,33 @@ class TaskTest(test.TestCase): def test_rebind_unknown_kwargs(self): task = KwargsTask(rebind={'foo': 'bar'}) - self.assertEqual(task.rebind, { + expected = { 'foo': 'bar', 'spam': 'spam' - }) + } + self.assertEqual(expected, task.rebind) def test_rebind_list_all(self): my_task = MyTask(rebind=('a', 'b', 'c')) - self.assertEqual(my_task.rebind, { + expected = { 'context': 'a', 'spam': 'b', 'eggs': 'c' - }) + } + self.assertEqual(expected, my_task.rebind) + self.assertEqual(set(['a', 'b', 'c']), + my_task.requires) def test_rebind_list_partial(self): my_task = MyTask(rebind=('a', 'b')) - self.assertEqual(my_task.rebind, { + expected = { 'context': 'a', 'spam': 'b', 'eggs': 'eggs' - }) + } + self.assertEqual(expected, my_task.rebind) + self.assertEqual(set(['a', 'b', 'eggs']), + my_task.requires) def test_rebind_list_more(self): self.assertRaisesRegexp(ValueError, '^Extra arguments', @@ -183,11 +203,14 @@ class TaskTest(test.TestCase): def test_rebind_list_more_kwargs(self): task = KwargsTask(rebind=('a', 'b', 'c')) - self.assertEqual(task.rebind, { + expected = { 'spam': 'a', 'b': 'b', 'c': 'c' - }) + } + self.assertEqual(expected, task.rebind) + self.assertEqual(set(['a', 'b', 'c']), + task.requires) def test_rebind_list_bad_value(self): self.assertRaisesRegexp(TypeError, '^Invalid rebind value', @@ -195,13 +218,13 @@ class TaskTest(test.TestCase): def test_default_provides(self): task = DefaultProvidesTask() - self.assertEqual(task.provides, set(['def'])) - self.assertEqual(task.save_as, {'def': None}) + self.assertEqual(set(['def']), task.provides) + self.assertEqual({'def': None}, task.save_as) def test_default_provides_can_be_overridden(self): task = DefaultProvidesTask(provides=('spam', 'eggs')) - self.assertEqual(task.provides, set(['spam', 'eggs'])) - self.assertEqual(task.save_as, {'spam': 0, 'eggs': 1}) + self.assertEqual(set(['spam', 'eggs']), task.provides) + self.assertEqual({'spam': 0, 'eggs': 1}, task.save_as) def test_update_progress_within_bounds(self): values = [0.0, 0.5, 1.0] @@ -213,7 +236,7 @@ class TaskTest(test.TestCase): a_task = ProgressTask() a_task.notifier.register(task.EVENT_UPDATE_PROGRESS, progress_callback) a_task.execute(values) - self.assertEqual(result, values) + self.assertEqual(values, result) @mock.patch.object(task.LOG, 'warn') def test_update_progress_lower_bound(self, mocked_warn): @@ -225,8 +248,8 @@ class TaskTest(test.TestCase): a_task = ProgressTask() a_task.notifier.register(task.EVENT_UPDATE_PROGRESS, progress_callback) a_task.execute([-1.0, -0.5, 0.0]) - self.assertEqual(result, [0.0, 0.0, 0.0]) - self.assertEqual(mocked_warn.call_count, 2) + self.assertEqual([0.0, 0.0, 0.0], result) + self.assertEqual(2, mocked_warn.call_count) @mock.patch.object(task.LOG, 'warn') def test_update_progress_upper_bound(self, mocked_warn): @@ -238,8 +261,8 @@ class TaskTest(test.TestCase): a_task = ProgressTask() a_task.notifier.register(task.EVENT_UPDATE_PROGRESS, progress_callback) a_task.execute([1.0, 1.5, 2.0]) - self.assertEqual(result, [1.0, 1.0, 1.0]) - self.assertEqual(mocked_warn.call_count, 2) + self.assertEqual([1.0, 1.0, 1.0], result) + self.assertEqual(2, mocked_warn.call_count) @mock.patch.object(notifier.LOG, 'warn') def test_update_progress_handler_failure(self, mocked_warn): @@ -256,34 +279,34 @@ class TaskTest(test.TestCase): a_task = MyTask() self.assertRaises(ValueError, a_task.notifier.register, task.EVENT_UPDATE_PROGRESS, None) - self.assertEqual(len(a_task.notifier), 0) + self.assertEqual(0, len(a_task.notifier)) def test_deregister_any_handler(self): a_task = MyTask() - self.assertEqual(len(a_task.notifier), 0) + self.assertEqual(0, len(a_task.notifier)) a_task.notifier.register(task.EVENT_UPDATE_PROGRESS, lambda event_type, details: None) - self.assertEqual(len(a_task.notifier), 1) + self.assertEqual(1, len(a_task.notifier)) a_task.notifier.deregister_event(task.EVENT_UPDATE_PROGRESS) - self.assertEqual(len(a_task.notifier), 0) + self.assertEqual(0, len(a_task.notifier)) def test_deregister_any_handler_empty_listeners(self): a_task = MyTask() - self.assertEqual(len(a_task.notifier), 0) + self.assertEqual(0, len(a_task.notifier)) self.assertFalse(a_task.notifier.deregister_event( task.EVENT_UPDATE_PROGRESS)) - self.assertEqual(len(a_task.notifier), 0) + self.assertEqual(0, len(a_task.notifier)) def test_deregister_non_existent_listener(self): handler1 = lambda event_type, details: None handler2 = lambda event_type, details: None a_task = MyTask() a_task.notifier.register(task.EVENT_UPDATE_PROGRESS, handler1) - self.assertEqual(len(list(a_task.notifier.listeners_iter())), 1) + self.assertEqual(1, len(list(a_task.notifier.listeners_iter()))) a_task.notifier.deregister(task.EVENT_UPDATE_PROGRESS, handler2) - self.assertEqual(len(list(a_task.notifier.listeners_iter())), 1) + self.assertEqual(1, len(list(a_task.notifier.listeners_iter()))) a_task.notifier.deregister(task.EVENT_UPDATE_PROGRESS, handler1) - self.assertEqual(len(list(a_task.notifier.listeners_iter())), 0) + self.assertEqual(0, len(list(a_task.notifier.listeners_iter()))) def test_bind_not_callable(self): a_task = MyTask() @@ -295,8 +318,8 @@ class TaskTest(test.TestCase): a_task = MyTask() a_task.notifier.register(task.EVENT_UPDATE_PROGRESS, handler1) b_task = a_task.copy(retain_listeners=False) - self.assertEqual(len(a_task.notifier), 1) - self.assertEqual(len(b_task.notifier), 0) + self.assertEqual(1, len(a_task.notifier)) + self.assertEqual(0, len(b_task.notifier)) def test_copy_listeners(self): handler1 = lambda event_type, details: None @@ -304,15 +327,15 @@ class TaskTest(test.TestCase): a_task = MyTask() a_task.notifier.register(task.EVENT_UPDATE_PROGRESS, handler1) b_task = a_task.copy() - self.assertEqual(len(b_task.notifier), 1) + self.assertEqual(1, len(b_task.notifier)) self.assertTrue(a_task.notifier.deregister_event( task.EVENT_UPDATE_PROGRESS)) - self.assertEqual(len(a_task.notifier), 0) - self.assertEqual(len(b_task.notifier), 1) + self.assertEqual(0, len(a_task.notifier)) + self.assertEqual(1, len(b_task.notifier)) b_task.notifier.register(task.EVENT_UPDATE_PROGRESS, handler2) listeners = dict(list(b_task.notifier.listeners_iter())) - self.assertEqual(len(listeners[task.EVENT_UPDATE_PROGRESS]), 2) - self.assertEqual(len(a_task.notifier), 0) + self.assertEqual(2, len(listeners[task.EVENT_UPDATE_PROGRESS])) + self.assertEqual(0, len(a_task.notifier)) class FunctorTaskTest(test.TestCase): @@ -320,7 +343,7 @@ class FunctorTaskTest(test.TestCase): def test_creation_with_version(self): version = (2, 0) f_task = task.FunctorTask(lambda: None, version=version) - self.assertEqual(f_task.version, version) + self.assertEqual(version, f_task.version) def test_execute_not_callable(self): self.assertRaises(ValueError, task.FunctorTask, 2) From ad133adea6ee293f0cfc6145a483fa0cfc27faf6 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Tue, 10 Mar 2015 18:01:44 -0700 Subject: [PATCH 057/246] Add + use failure json schema validation Change-Id: Ie3aa386c831459a028ba494570bafd53b998126e --- doc/source/utils.rst | 5 ++ taskflow/engines/worker_based/protocol.py | 71 +++++++++++++++---- taskflow/engines/worker_based/server.py | 30 +------- taskflow/tests/unit/test_failure.py | 31 ++++++++ .../tests/unit/worker_based/test_server.py | 8 +-- taskflow/types/failure.py | 50 +++++++++++++ taskflow/utils/schema_utils.py | 34 +++++++++ 7 files changed, 182 insertions(+), 47 deletions(-) create mode 100644 taskflow/utils/schema_utils.py diff --git a/doc/source/utils.rst b/doc/source/utils.rst index 1f774663..6949ccf0 100644 --- a/doc/source/utils.rst +++ b/doc/source/utils.rst @@ -48,6 +48,11 @@ Persistence .. automodule:: taskflow.utils.persistence_utils +Schema +~~~~~~ + +.. automodule:: taskflow.utils.schema_utils + Threading ~~~~~~~~~ diff --git a/taskflow/engines/worker_based/protocol.py b/taskflow/engines/worker_based/protocol.py index 8a137471..b22d61fe 100644 --- a/taskflow/engines/worker_based/protocol.py +++ b/taskflow/engines/worker_based/protocol.py @@ -18,8 +18,6 @@ import abc import threading from concurrent import futures -import jsonschema -from jsonschema import exceptions as schema_exc from oslo_utils import reflection from oslo_utils import timeutils import six @@ -30,6 +28,7 @@ from taskflow import logging from taskflow.types import failure as ft from taskflow.types import timing as tt from taskflow.utils import lock_utils +from taskflow.utils import schema_utils as su # NOTE(skudriashev): This is protocol states and events, which are not # related to task states. @@ -98,12 +97,6 @@ NOTIFY = 'NOTIFY' REQUEST = 'REQUEST' RESPONSE = 'RESPONSE' -# Special jsonschema validation types/adjustments. -_SCHEMA_TYPES = { - # See: https://github.com/Julian/jsonschema/issues/148 - 'array': (list, tuple), -} - LOG = logging.getLogger(__name__) @@ -166,8 +159,8 @@ class Notify(Message): else: schema = cls.SENDER_SCHEMA try: - jsonschema.validate(data, schema, types=_SCHEMA_TYPES) - except schema_exc.ValidationError as e: + su.schema_validate(data, schema) + except su.ValidationError as e: if response: raise excp.InvalidFormat("%s message response data not of the" " expected format: %s" @@ -358,11 +351,57 @@ class Request(Message): @classmethod def validate(cls, data): try: - jsonschema.validate(data, cls.SCHEMA, types=_SCHEMA_TYPES) - except schema_exc.ValidationError as e: + su.schema_validate(data, cls.SCHEMA) + except su.ValidationError as e: raise excp.InvalidFormat("%s message response data not of the" " expected format: %s" % (cls.TYPE, e.message), e) + else: + # Validate all failure dictionaries that *may* be present... + failures = [] + if 'failures' in data: + failures.extend(six.itervalues(data['failures'])) + result = data.get('result') + if result is not None: + result_data_type, result_data = result + if result_data_type == 'failure': + failures.append(result_data) + for fail_data in failures: + ft.Failure.validate(fail_data) + + @staticmethod + def from_dict(data, task_uuid=None): + """Parses **validated** data before it can be further processed. + + All :py:class:`~taskflow.types.failure.Failure` objects that have been + converted to dict(s) on the remote side will now converted back + to py:class:`~taskflow.types.failure.Failure` objects. + """ + task_cls = data['task_cls'] + task_name = data['task_name'] + action = data['action'] + arguments = data.get('arguments', {}) + result = data.get('result') + failures = data.get('failures') + # These arguments will eventually be given to the task executor + # so they need to be in a format it will accept (and using keyword + # argument names that it accepts)... + arguments = { + 'arguments': arguments, + } + if task_uuid is not None: + arguments['task_uuid'] = task_uuid + if result is not None: + result_data_type, result_data = result + if result_data_type == 'failure': + arguments['result'] = ft.Failure.from_dict(result_data) + else: + arguments['result'] = result_data + if failures is not None: + arguments['failures'] = {} + for task, fail_data in six.iteritems(failures): + arguments['failures'][task] = ft.Failure.from_dict(fail_data) + return (task_cls, task_name, action, arguments) class Response(Message): @@ -455,8 +494,12 @@ class Response(Message): @classmethod def validate(cls, data): try: - jsonschema.validate(data, cls.SCHEMA, types=_SCHEMA_TYPES) - except schema_exc.ValidationError as e: + su.schema_validate(data, cls.SCHEMA) + except su.ValidationError as e: raise excp.InvalidFormat("%s message response data not of the" " expected format: %s" % (cls.TYPE, e.message), e) + else: + state = data['state'] + if state == FAILURE and 'result' in data: + ft.Failure.validate(data['result']) diff --git a/taskflow/engines/worker_based/server.py b/taskflow/engines/worker_based/server.py index 1043e879..99a39895 100644 --- a/taskflow/engines/worker_based/server.py +++ b/taskflow/engines/worker_based/server.py @@ -17,7 +17,6 @@ import functools from oslo_utils import reflection -import six from taskflow.engines.worker_based import dispatcher from taskflow.engines.worker_based import protocol as pr @@ -94,32 +93,6 @@ class Server(object): def connection_details(self): return self._proxy.connection_details - @staticmethod - def _parse_request(task_cls, task_name, action, arguments, result=None, - failures=None, **kwargs): - """Parse request before it can be further processed. - - All `failure.Failure` objects that have been converted to dict on the - remote side will now converted back to `failure.Failure` objects. - """ - # These arguments will eventually be given to the task executor - # so they need to be in a format it will accept (and using keyword - # argument names that it accepts)... - arguments = { - 'arguments': arguments, - } - if result is not None: - data_type, data = result - if data_type == 'failure': - arguments['result'] = ft.Failure.from_dict(data) - else: - arguments['result'] = data - if failures is not None: - arguments['failures'] = {} - for key, data in six.iteritems(failures): - arguments['failures'][key] = ft.Failure.from_dict(data) - return (task_cls, task_name, action, arguments) - @staticmethod def _parse_message(message): """Extracts required attributes out of the messages properties. @@ -201,9 +174,8 @@ class Server(object): # parse request to get task name, action and action arguments try: - bundle = self._parse_request(**request) + bundle = pr.Request.from_dict(request, task_uuid=task_uuid) task_cls, task_name, action, arguments = bundle - arguments['task_uuid'] = task_uuid except ValueError: with misc.capture_failure() as failure: LOG.warn("Failed to parse request contents from message '%s'", diff --git a/taskflow/tests/unit/test_failure.py b/taskflow/tests/unit/test_failure.py index c8f83b9b..fab9cb9a 100644 --- a/taskflow/tests/unit/test_failure.py +++ b/taskflow/tests/unit/test_failure.py @@ -137,6 +137,36 @@ class FromExceptionTestCase(test.TestCase, GeneralFailureObjTestsMixin): class FailureObjectTestCase(test.TestCase): + def test_invalids(self): + f = { + 'exception_str': 'blah', + 'traceback_str': 'blah', + 'exc_type_names': [], + } + self.assertRaises(exceptions.InvalidFormat, + failure.Failure.validate, f) + f = { + 'exception_str': 'blah', + 'exc_type_names': ['Exception'], + } + self.assertRaises(exceptions.InvalidFormat, + failure.Failure.validate, f) + f = { + 'exception_str': 'blah', + 'traceback_str': 'blah', + 'exc_type_names': ['Exception'], + 'version': -1, + } + self.assertRaises(exceptions.InvalidFormat, + failure.Failure.validate, f) + + def test_valid_from_dict_to_dict(self): + f = _captured_failure('Woot!') + d_f = f.to_dict() + failure.Failure.validate(d_f) + f2 = failure.Failure.from_dict(d_f) + self.assertTrue(f.matches(f2)) + def test_dont_catch_base_exception(self): try: raise SystemExit() @@ -358,6 +388,7 @@ class FailureCausesTest(test.TestCase): self.assertIsNotNone(f) d_f = f.to_dict() + failure.Failure.validate(d_f) f = failure.Failure.from_dict(d_f) self.assertEqual(2, len(f.causes)) self.assertEqual("Still not working", f.causes[0].exception_str) diff --git a/taskflow/tests/unit/worker_based/test_server.py b/taskflow/tests/unit/worker_based/test_server.py index fea5d1cc..9b7815c4 100644 --- a/taskflow/tests/unit/worker_based/test_server.py +++ b/taskflow/tests/unit/worker_based/test_server.py @@ -108,7 +108,7 @@ class TestServer(test.MockTestCase): def test_parse_request(self): request = self.make_request() - bundle = server.Server._parse_request(**request) + bundle = pr.Request.from_dict(request) task_cls, task_name, action, task_args = bundle self.assertEqual((task_cls, task_name, action, task_args), (self.task.name, self.task.name, self.task_action, @@ -116,7 +116,7 @@ class TestServer(test.MockTestCase): def test_parse_request_with_success_result(self): request = self.make_request(action='revert', result=1) - bundle = server.Server._parse_request(**request) + bundle = pr.Request.from_dict(request) task_cls, task_name, action, task_args = bundle self.assertEqual((task_cls, task_name, action, task_args), (self.task.name, self.task.name, 'revert', @@ -126,7 +126,7 @@ class TestServer(test.MockTestCase): def test_parse_request_with_failure_result(self): a_failure = failure.Failure.from_exception(Exception('test')) request = self.make_request(action='revert', result=a_failure) - bundle = server.Server._parse_request(**request) + bundle = pr.Request.from_dict(request) task_cls, task_name, action, task_args = bundle self.assertEqual((task_cls, task_name, action, task_args), (self.task.name, self.task.name, 'revert', @@ -137,7 +137,7 @@ class TestServer(test.MockTestCase): failures = {'0': failure.Failure.from_exception(Exception('test1')), '1': failure.Failure.from_exception(Exception('test2'))} request = self.make_request(action='revert', failures=failures) - bundle = server.Server._parse_request(**request) + bundle = pr.Request.from_dict(request) task_cls, task_name, action, task_args = bundle self.assertEqual( (task_cls, task_name, action, task_args), diff --git a/taskflow/types/failure.py b/taskflow/types/failure.py index f251b00e..1c98aa2d 100644 --- a/taskflow/types/failure.py +++ b/taskflow/types/failure.py @@ -23,6 +23,7 @@ from oslo_utils import reflection import six from taskflow import exceptions as exc +from taskflow.utils import schema_utils as su def _copy_exc_info(exc_info): @@ -132,6 +133,47 @@ class Failure(object): """ DICT_VERSION = 1 + #: Expected failure schema (in json schema format). + SCHEMA = { + "$ref": "#/definitions/cause", + "definitions": { + "cause": { + "type": "object", + 'properties': { + 'version': { + "type": "integer", + "minimum": 0, + }, + 'exception_str': { + "type": "string", + }, + 'traceback_str': { + "type": "string", + }, + 'exc_type_names': { + "type": "array", + "items": { + "type": "string", + }, + "minItems": 1, + }, + 'causes': { + "type": "array", + "items": { + "$ref": "#/definitions/cause", + }, + } + }, + "required": [ + "exception_str", + 'traceback_str', + 'exc_type_names', + ], + "additionalProperties": True, + }, + }, + } + def __init__(self, exc_info=None, **kwargs): if not kwargs: if exc_info is None: @@ -169,6 +211,14 @@ class Failure(object): """Creates a failure object from a exception instance.""" return cls((type(exception), exception, None)) + @classmethod + def validate(cls, data): + try: + su.schema_validate(data, cls.SCHEMA) + except su.ValidationError as e: + raise exc.InvalidFormat("Failure data not of the" + " expected format: %s" % (e.message), e) + def _matches(self, other): if self is other: return True diff --git a/taskflow/utils/schema_utils.py b/taskflow/utils/schema_utils.py new file mode 100644 index 00000000..8d7c216e --- /dev/null +++ b/taskflow/utils/schema_utils.py @@ -0,0 +1,34 @@ +# -*- coding: utf-8 -*- + +# Copyright (C) 2015 Yahoo! Inc. 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. + +import jsonschema +from jsonschema import exceptions as schema_exc + +# Special jsonschema validation types/adjustments. +_SCHEMA_TYPES = { + # See: https://github.com/Julian/jsonschema/issues/148 + 'array': (list, tuple), +} + + +# Expose these types so that people don't have to import the same exceptions. +ValidationError = schema_exc.ValidationError +SchemaError = schema_exc.SchemaError + + +def schema_validate(data, schema): + """Validates given data using provided json schema.""" + jsonschema.validate(data, schema, types=_SCHEMA_TYPES) From 646ca59f63296a97eebdd86e1f20135035da140c Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 11 Mar 2015 13:22:31 -0700 Subject: [PATCH 058/246] Allow backend connection config (via fetch) to be a string To make it easier to provide just a connection string to the persistence backend fetch mechanism (without needing to create a dictionary with just a 'connection' key) match the behavior of the other fetch-like functions and accept a string to simplify the interaction with this function. Change-Id: I59cd330c545985fe9782c3049690425ead1c8cf3 --- taskflow/persistence/backends/__init__.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/taskflow/persistence/backends/__init__.py b/taskflow/persistence/backends/__init__.py index f11bfad0..ec0b6238 100644 --- a/taskflow/persistence/backends/__init__.py +++ b/taskflow/persistence/backends/__init__.py @@ -16,6 +16,7 @@ import contextlib +import six from stevedore import driver from taskflow import exceptions as exc @@ -50,6 +51,8 @@ def fetch(conf, namespace=BACKEND_NAMESPACE, **kwargs): is {'a': 'b', 'c': 'd'} to the constructor of that persistence backend instance. """ + if isinstance(conf, six.string_types): + conf = {'connection': conf} backend_name = conf['connection'] try: uri = misc.parse_uri(backend_name) From 9f64c47cb74b27c40135fff79e23d2bd1c6acfdd Mon Sep 17 00:00:00 2001 From: Dan Krause Date: Tue, 10 Mar 2015 21:42:53 -0500 Subject: [PATCH 059/246] lazy loading for logbooks and flowdetails You can now specify a "lazy" argument when asking a persistence backend for a logbook for flowdetail. This will return a logbook with no flowdetails in it, or a flowdetail with no atomdetails in it, respectively. Change-Id: Ie81a19a2107b44e2c35c2a92507bec03eee55c50 --- .../persistence/backends/impl_sqlalchemy.py | 38 +++++++++++++------ taskflow/persistence/base.py | 11 ++++-- taskflow/persistence/path_based.py | 2 +- taskflow/tests/unit/persistence/base.py | 29 ++++++++++++++ 4 files changed, 65 insertions(+), 15 deletions(-) diff --git a/taskflow/persistence/backends/impl_sqlalchemy.py b/taskflow/persistence/backends/impl_sqlalchemy.py index 4368b78a..d4f343f2 100644 --- a/taskflow/persistence/backends/impl_sqlalchemy.py +++ b/taskflow/persistence/backends/impl_sqlalchemy.py @@ -527,7 +527,7 @@ class Connection(base.Connection): raise exc.StorageFailure("Failed saving logbook" " '%s'" % book.uuid, e) - def get_logbook(self, book_uuid): + def get_logbook(self, book_uuid, lazy=False): try: logbooks = self._tables.logbooks with contextlib.closing(self._engine.connect()) as conn: @@ -538,40 +538,42 @@ class Connection(base.Connection): raise exc.NotFound("No logbook found with" " uuid '%s'" % book_uuid) book = self._converter.convert_book(row) - self._converter.populate_book(conn, book) + if not lazy: + self._converter.populate_book(conn, book) return book except sa_exc.DBAPIError as e: raise exc.StorageFailure( "Failed getting logbook '%s'" % book_uuid, e) - def get_logbooks(self): + def get_logbooks(self, lazy=False): gathered = [] try: with contextlib.closing(self._engine.connect()) as conn: q = sql.select([self._tables.logbooks]) for row in conn.execute(q): book = self._converter.convert_book(row) - self._converter.populate_book(conn, book) + if not lazy: + self._converter.populate_book(conn, book) gathered.append(book) except sa_exc.DBAPIError as e: raise exc.StorageFailure("Failed getting logbooks", e) for book in gathered: yield book - def get_flows_for_book(self, book_uuid): + def get_flows_for_book(self, book_uuid, lazy=False): gathered = [] try: with contextlib.closing(self._engine.connect()) as conn: - for row in self._converter.flow_query_iter(conn, book_uuid): - flow_details = self._converter.populate_flow_detail(conn, - row) - gathered.append(flow_details) + for fd in self._converter.flow_query_iter(conn, book_uuid): + if not lazy: + self._converter.populate_flow_detail(conn, fd) + gathered.append(fd) except sa_exc.DBAPIError as e: raise exc.StorageFailure("Failed getting flow details", e) for flow_details in gathered: yield flow_details - def get_flow_details(self, fd_uuid): + def get_flow_details(self, fd_uuid, lazy=False): try: flowdetails = self._tables.flowdetails with self._engine.begin() as conn: @@ -581,7 +583,10 @@ class Connection(base.Connection): if not row: raise exc.NotFound("No flow details found with uuid" " '%s'" % fd_uuid) - return self._converter.convert_flow_detail(row) + fd = self._converter.convert_flow_detail(row) + if not lazy: + self._converter.populate_flow_detail(conn, fd) + return fd except sa_exc.SQLAlchemyError as e: raise exc.StorageFailure("Failed getting flow details with" " uuid '%s'" % fd_uuid, e) @@ -601,5 +606,16 @@ class Connection(base.Connection): raise exc.StorageFailure("Failed getting atom details with" " uuid '%s'" % ad_uuid, e) + def get_atoms_for_flow(self, fd_uuid): + gathered = [] + try: + with contextlib.closing(self._engine.connect()) as conn: + for ad in self._converter.atom_query_iter(conn, fd_uuid): + gathered.append(ad) + except sa_exc.DBAPIError as e: + raise exc.StorageFailure("Failed getting atom details", e) + for atom_details in gathered: + yield atom_details + def close(self): pass diff --git a/taskflow/persistence/base.py b/taskflow/persistence/base.py index a1f120df..aebb7dd9 100644 --- a/taskflow/persistence/base.py +++ b/taskflow/persistence/base.py @@ -109,12 +109,12 @@ class Connection(object): pass @abc.abstractmethod - def get_logbook(self, book_uuid): + def get_logbook(self, book_uuid, lazy=False): """Fetches a logbook object matching the given uuid.""" pass @abc.abstractmethod - def get_logbooks(self): + def get_logbooks(self, lazy=False): """Return an iterable of logbook objects.""" pass @@ -124,7 +124,7 @@ class Connection(object): pass @abc.abstractmethod - def get_flow_details(self, fd_uuid): + def get_flow_details(self, fd_uuid, lazy=False): """Fetches a flowdetails object matching the given uuid.""" pass @@ -133,6 +133,11 @@ class Connection(object): """Fetches a atomdetails object matching the given uuid.""" pass + @abc.abstractmethod + def get_atoms_for_flow(self, fd_uuid): + """Return an iterable of atomdetails for a given flowdetails uuid.""" + pass + def _format_atom(atom_detail): return { diff --git a/taskflow/persistence/path_based.py b/taskflow/persistence/path_based.py index ea080257..d3e2bfef 100644 --- a/taskflow/persistence/path_based.py +++ b/taskflow/persistence/path_based.py @@ -149,7 +149,7 @@ class PathBasedConnection(base.Connection): def get_logbooks(self, lazy=False): for book_uuid in self._get_children(self.book_path): - yield self.get_logbook(book_uuid, lazy) + yield self.get_logbook(book_uuid, lazy=lazy) def get_logbook(self, book_uuid, lazy=False): book_path = self._join_path(self.book_path, book_uuid) diff --git a/taskflow/tests/unit/persistence/base.py b/taskflow/tests/unit/persistence/base.py index 184cf51e..924e62b2 100644 --- a/taskflow/tests/unit/persistence/base.py +++ b/taskflow/tests/unit/persistence/base.py @@ -130,6 +130,22 @@ class PersistenceTestMixin(object): fd2 = lb2.find(fd.uuid) self.assertEqual(fd2.meta.get('test'), 43) + def test_flow_detail_lazy_fetch(self): + lb_id = uuidutils.generate_uuid() + lb_name = 'lb-%s' % (lb_id) + lb = logbook.LogBook(name=lb_name, uuid=lb_id) + fd = logbook.FlowDetail('test', uuid=uuidutils.generate_uuid()) + td = logbook.TaskDetail("detail-1", uuid=uuidutils.generate_uuid()) + td.version = '4.2' + fd.add(td) + lb.add(fd) + with contextlib.closing(self._get_connection()) as conn: + conn.save_logbook(lb) + with contextlib.closing(self._get_connection()) as conn: + fd2 = conn.get_flow_details(fd.uuid, lazy=True) + self.assertEqual(0, len(fd2)) + self.assertEqual(1, len(fd)) + def test_task_detail_save(self): lb_id = uuidutils.generate_uuid() lb_name = 'lb-%s' % (lb_id) @@ -239,6 +255,19 @@ class PersistenceTestMixin(object): self.assertEqual(1, len(lb)) self.assertEqual(fd.name, lb2.find(fd.uuid).name) + def test_logbook_lazy_fetch(self): + lb_id = uuidutils.generate_uuid() + lb_name = 'lb-%s' % (lb_id) + lb = logbook.LogBook(name=lb_name, uuid=lb_id) + fd = logbook.FlowDetail('test', uuid=uuidutils.generate_uuid()) + lb.add(fd) + with contextlib.closing(self._get_connection()) as conn: + conn.save_logbook(lb) + with contextlib.closing(self._get_connection()) as conn: + lb2 = conn.get_logbook(lb_id, lazy=True) + self.assertEqual(0, len(lb2)) + self.assertEqual(1, len(lb)) + def test_logbook_add_task_detail(self): lb_id = uuidutils.generate_uuid() lb_name = 'lb-%s' % (lb_id) From 844c2c69ab34294142978424d9835b0fe1db644d Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 11 Mar 2015 15:25:41 -0700 Subject: [PATCH 060/246] Put underscore in-front of alchemist helper Since this is a internal helper object we should put a underscore infront of its class name to make sure this is obvious (the docstring says the same thing but this will help). Change-Id: I3aeb6a178c8ceda0c36d5f5bb57bb765260cbd3c --- taskflow/persistence/backends/impl_sqlalchemy.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/taskflow/persistence/backends/impl_sqlalchemy.py b/taskflow/persistence/backends/impl_sqlalchemy.py index 4368b78a..24fedc04 100644 --- a/taskflow/persistence/backends/impl_sqlalchemy.py +++ b/taskflow/persistence/backends/impl_sqlalchemy.py @@ -180,7 +180,7 @@ def _ping_listener(dbapi_conn, connection_rec, connection_proxy): raise -class Alchemist(object): +class _Alchemist(object): """Internal <-> external row <-> objects + other helper functions. NOTE(harlowja): for internal usage only. @@ -343,7 +343,7 @@ class Connection(base.Connection): self._engine = backend.engine self._metadata = sa.MetaData() self._tables = tables.fetch(self._metadata) - self._converter = Alchemist(self._tables) + self._converter = _Alchemist(self._tables) @property def backend(self): From 94eb978a642112b92f79510350441f63f405a80d Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 11 Mar 2015 15:36:29 -0700 Subject: [PATCH 061/246] Just use the local conf variable The local conf variable populated by the parent class already ensures the none/falsey case and sets the conf to an empty dictionary (as well as checks if its the expected type and more...); so we should just use that variable instead of duplicating that work. Change-Id: I824263d5253d428f6a803f470e29d99de3dbf42c --- taskflow/persistence/path_based.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/taskflow/persistence/path_based.py b/taskflow/persistence/path_based.py index ea080257..590dc70c 100644 --- a/taskflow/persistence/path_based.py +++ b/taskflow/persistence/path_based.py @@ -36,9 +36,7 @@ class PathBasedBackend(base.Backend): def __init__(self, conf): super(PathBasedBackend, self).__init__(conf) - if conf is None: - conf = {} - self._path = conf.get('path', None) + self._path = self._conf.get('path', None) @property def path(self): From 5996c8f25e2869eace7caa8ed4f8a9fae5f16aad Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 11 Mar 2015 16:41:15 -0700 Subject: [PATCH 062/246] Allow the storage unit to use the right scoping strategy Instead of having the fetch arguments functions need to be provided a scope walker to correctly find the right arguments, which only the internals of the action engine know about provide a default scope walker (that is the same one the action engine internal uses) to the storage unit and have it be the default strategy used so that users need not know how to pass it in (which they should not care about). This allows for users to fetch the same mapped arguments as the internals of the engine will fetch. Change-Id: I1beca532b2b7c7ad98b09265a0c4477658052d16 --- .../engines/action_engine/actions/base.py | 3 +-- .../engines/action_engine/actions/retry.py | 6 ++--- .../engines/action_engine/actions/task.py | 8 ++---- taskflow/engines/action_engine/engine.py | 26 +++++++++++++++++-- taskflow/engines/action_engine/runtime.py | 25 +++++++++++------- taskflow/engines/base.py | 7 ++--- taskflow/storage.py | 20 ++++++++++---- 7 files changed, 62 insertions(+), 33 deletions(-) diff --git a/taskflow/engines/action_engine/actions/base.py b/taskflow/engines/action_engine/actions/base.py index 5595268a..869ef228 100644 --- a/taskflow/engines/action_engine/actions/base.py +++ b/taskflow/engines/action_engine/actions/base.py @@ -32,10 +32,9 @@ SAVE_RESULT_STATES = (states.SUCCESS, states.FAILURE) class Action(object): """An action that handles executing, state changes, ... of atoms.""" - def __init__(self, storage, notifier, walker_factory): + def __init__(self, storage, notifier): self._storage = storage self._notifier = notifier - self._walker_factory = walker_factory @abc.abstractmethod def handles(self, atom): diff --git a/taskflow/engines/action_engine/actions/retry.py b/taskflow/engines/action_engine/actions/retry.py index 05496d96..f69d5a5b 100644 --- a/taskflow/engines/action_engine/actions/retry.py +++ b/taskflow/engines/action_engine/actions/retry.py @@ -44,8 +44,8 @@ def _revert_retry(retry, arguments): class RetryAction(base.Action): """An action that handles executing, state changes, ... of retry atoms.""" - def __init__(self, storage, notifier, walker_factory): - super(RetryAction, self).__init__(storage, notifier, walker_factory) + def __init__(self, storage, notifier): + super(RetryAction, self).__init__(storage, notifier) self._executor = futures.SynchronousExecutor() @staticmethod @@ -53,11 +53,9 @@ class RetryAction(base.Action): return isinstance(atom, retry_atom.Retry) def _get_retry_args(self, retry, addons=None): - scope_walker = self._walker_factory(retry) arguments = self._storage.fetch_mapped_args( retry.rebind, atom_name=retry.name, - scope_walker=scope_walker, optional_args=retry.optional ) history = self._storage.get_retry_history(retry.name) diff --git a/taskflow/engines/action_engine/actions/task.py b/taskflow/engines/action_engine/actions/task.py index 8c64931a..2a11bf8d 100644 --- a/taskflow/engines/action_engine/actions/task.py +++ b/taskflow/engines/action_engine/actions/task.py @@ -28,8 +28,8 @@ LOG = logging.getLogger(__name__) class TaskAction(base.Action): """An action that handles scheduling, state changes, ... of task atoms.""" - def __init__(self, storage, notifier, walker_factory, task_executor): - super(TaskAction, self).__init__(storage, notifier, walker_factory) + def __init__(self, storage, notifier, task_executor): + super(TaskAction, self).__init__(storage, notifier) self._task_executor = task_executor @staticmethod @@ -100,11 +100,9 @@ class TaskAction(base.Action): def schedule_execution(self, task): self.change_state(task, states.RUNNING, progress=0.0) - scope_walker = self._walker_factory(task) arguments = self._storage.fetch_mapped_args( task.rebind, atom_name=task.name, - scope_walker=scope_walker, optional_args=task.optional ) if task.notifier.can_be_registered(task_atom.EVENT_UPDATE_PROGRESS): @@ -126,11 +124,9 @@ class TaskAction(base.Action): def schedule_reversion(self, task): self.change_state(task, states.REVERTING, progress=0.0) - scope_walker = self._walker_factory(task) arguments = self._storage.fetch_mapped_args( task.rebind, atom_name=task.name, - scope_walker=scope_walker, optional_args=task.optional ) task_uuid = self._storage.get_atom_uuid(task.name) diff --git a/taskflow/engines/action_engine/engine.py b/taskflow/engines/action_engine/engine.py index 9fa0f5a5..68bd3458 100644 --- a/taskflow/engines/action_engine/engine.py +++ b/taskflow/engines/action_engine/engine.py @@ -29,6 +29,7 @@ from taskflow.engines.action_engine import runtime from taskflow.engines import base from taskflow import exceptions as exc from taskflow import states +from taskflow import storage from taskflow.types import failure from taskflow.utils import lock_utils from taskflow.utils import misc @@ -89,6 +90,29 @@ class ActionEngine(base.Engine): else: return None + @misc.cachedproperty + def storage(self): + """The storage unit for this engine. + + NOTE(harlowja): the atom argument lookup strategy will change for + this storage unit after + :py:func:`~taskflow.engines.base.Engine.compile` has + completed (since **only** after compilation is the actual structure + known). Before :py:func:`~taskflow.engines.base.Engine.compile` + has completed the atom argument lookup strategy lookup will be + restricted to injected arguments **only** (this will **not** reflect + the actual runtime lookup strategy, which typically will be, but is + not always different). + """ + def _scope_fetcher(atom_name): + if self._compiled: + return self._runtime.fetch_scopes_for(atom_name) + else: + return None + return storage.Storage(self._flow_detail, + backend=self._backend, + scope_fetcher=_scope_fetcher) + def run(self): with lock_utils.try_lock(self._lock) as was_locked: if not was_locked: @@ -192,9 +216,7 @@ class ActionEngine(base.Engine): missing = set() fetch = self.storage.fetch_unsatisfied_args for node in self._compilation.execution_graph.nodes_iter(): - scope_walker = self._runtime.fetch_scopes_for(node) missing.update(fetch(node.name, node.rebind, - scope_walker=scope_walker, optional_args=node.optional)) if missing: raise exc.MissingDependencies(self._flow, sorted(missing)) diff --git a/taskflow/engines/action_engine/runtime.py b/taskflow/engines/action_engine/runtime.py index d71ff65c..d8df4705 100644 --- a/taskflow/engines/action_engine/runtime.py +++ b/taskflow/engines/action_engine/runtime.py @@ -66,24 +66,31 @@ class Runtime(object): @misc.cachedproperty def retry_action(self): - return ra.RetryAction(self._storage, self._atom_notifier, - self.fetch_scopes_for) + return ra.RetryAction(self._storage, + self._atom_notifier) @misc.cachedproperty def task_action(self): return ta.TaskAction(self._storage, - self._atom_notifier, self.fetch_scopes_for, + self._atom_notifier, self._task_executor) - def fetch_scopes_for(self, atom): + def fetch_scopes_for(self, atom_name): """Fetches a tuple of the visible scopes for the given atom.""" try: - return self._scopes[atom] + return self._scopes[atom_name] except KeyError: - walker = sc.ScopeWalker(self.compilation, atom, - names_only=True) - visible_to = tuple(walker) - self._scopes[atom] = visible_to + atom = None + for node in self.analyzer.iterate_all_nodes(): + if node.name == atom_name: + atom = node + break + if atom is not None: + walker = sc.ScopeWalker(self.compilation, atom, + names_only=True) + self._scopes[atom_name] = visible_to = tuple(walker) + else: + visible_to = tuple([]) return visible_to # Various helper methods used by the runtime components; not for public diff --git a/taskflow/engines/base.py b/taskflow/engines/base.py index 632f626f..824d9087 100644 --- a/taskflow/engines/base.py +++ b/taskflow/engines/base.py @@ -20,9 +20,7 @@ import abc from debtcollector import moves import six -from taskflow import storage from taskflow.types import notifier -from taskflow.utils import misc @six.add_metaclass(abc.ABCMeta) @@ -72,10 +70,9 @@ class Engine(object): """The options that were passed to this engine on construction.""" return self._options - @misc.cachedproperty + @abc.abstractproperty def storage(self): - """The storage unit for this flow.""" - return storage.Storage(self._flow_detail, backend=self._backend) + """The storage unit for this engine.""" @abc.abstractmethod def compile(self): diff --git a/taskflow/storage.py b/taskflow/storage.py index d1f67c62..e8d2cddf 100644 --- a/taskflow/storage.py +++ b/taskflow/storage.py @@ -131,7 +131,7 @@ class Storage(object): with it must be avoided) that are *global* to the flow being executed. """ - def __init__(self, flow_detail, backend=None): + def __init__(self, flow_detail, backend=None, scope_fetcher=None): self._result_mappings = {} self._reverse_mapping = {} self._backend = backend @@ -143,6 +143,9 @@ class Storage(object): ((task.BaseTask,), self._ensure_task), ((retry.Retry,), self._ensure_retry), ] + if scope_fetcher is None: + scope_fetcher = lambda atom_name: None + self._scope_fetcher = scope_fetcher # NOTE(imelnikov): failure serialization looses information, # so we cache failures here, in atom name -> failure mapping. @@ -698,7 +701,7 @@ class Storage(object): non_default_providers.append(p) return default_providers, non_default_providers - def _locate_providers(name): + def _locate_providers(name, scope_walker=None): """Finds the accessible *potential* providers.""" default_providers, non_default_providers = _fetch_providers(name) providers = [] @@ -728,6 +731,8 @@ class Storage(object): return providers ad = self._atomdetail_by_name(atom_name) + if scope_walker is None: + scope_walker = self._scope_fetcher(atom_name) if optional_args is None: optional_args = [] injected_sources = [ @@ -749,7 +754,8 @@ class Storage(object): continue if name in source: maybe_providers += 1 - maybe_providers += len(_locate_providers(name)) + providers = _locate_providers(name, scope_walker=scope_walker) + maybe_providers += len(providers) if maybe_providers: LOG.blather("Atom %s will have %s potential providers" " of %r <= %r", atom_name, maybe_providers, @@ -797,7 +803,8 @@ class Storage(object): " by %s but was unable to get at that providers" " results" % (looking_for, provider), e) - def _locate_providers(looking_for, possible_providers): + def _locate_providers(looking_for, possible_providers, + scope_walker=None): """Finds the accessible providers.""" default_providers = [] for p in possible_providers: @@ -832,6 +839,8 @@ class Storage(object): self._injected_args.get(atom_name, {}), ad.meta.get(META_INJECTED, {}), ] + if scope_walker is None: + scope_walker = self._scope_fetcher(atom_name) else: injected_sources = [] if not args_mapping: @@ -869,7 +878,8 @@ class Storage(object): " produced output by any" " providers" % name) # Reduce the possible providers to one that are allowed. - providers = _locate_providers(name, possible_providers) + providers = _locate_providers(name, possible_providers, + scope_walker=scope_walker) if not providers: raise exceptions.NotFound( "Mapped argument %r <= %r was not produced" From f6e20741638ff5ddc033a6338c374da35beb8c5d Mon Sep 17 00:00:00 2001 From: Davanum Srinivas Date: Thu, 12 Mar 2015 07:12:19 -0400 Subject: [PATCH 063/246] Switch to non-namespaced module imports Change-Id: I25686df6a05bf69a89b9b6be169db72eab89d32a --- taskflow/persistence/backends/sqlalchemy/tables.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/taskflow/persistence/backends/sqlalchemy/tables.py b/taskflow/persistence/backends/sqlalchemy/tables.py index 239e6430..81f0b964 100644 --- a/taskflow/persistence/backends/sqlalchemy/tables.py +++ b/taskflow/persistence/backends/sqlalchemy/tables.py @@ -16,9 +16,9 @@ import collections -from oslo.serialization import jsonutils -from oslo.utils import timeutils -from oslo.utils import uuidutils +from oslo_serialization import jsonutils +from oslo_utils import timeutils +from oslo_utils import uuidutils from sqlalchemy import Table, Column, String, ForeignKey, DateTime, Enum from sqlalchemy import types From 6529c1ed64eb1e95fe04bf4372496a774bf20bf0 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 11 Mar 2015 14:25:47 -0700 Subject: [PATCH 064/246] Restructure the in-memory node usage Instead of retaining full paths only retain partial paths; this then allows for finding a parent node and then easily finding a child by using the find() routines. This more naturally emulates a real filesystem using our tree structure and simplifies certain operations to make them more efficient/simple. Change-Id: I207b02c979704a15eac8be46d11a0af61b05d9b9 --- taskflow/persistence/backends/impl_memory.py | 157 +++++++++++++++---- 1 file changed, 123 insertions(+), 34 deletions(-) diff --git a/taskflow/persistence/backends/impl_memory.py b/taskflow/persistence/backends/impl_memory.py index 7fc22e6d..0f5e22b4 100644 --- a/taskflow/persistence/backends/impl_memory.py +++ b/taskflow/persistence/backends/impl_memory.py @@ -25,11 +25,125 @@ from taskflow.types import tree from taskflow.utils import lock_utils +class Filesystem(object): + """An in-memory tree filesystem-like structure.""" + + @staticmethod + def _normpath(path): + if not path.startswith(os.sep): + raise ValueError("This filesystem can only normalize absolute" + " paths: '%s' is not valid" % path) + return os.path.normpath(path) + + def __init__(self): + self._root = tree.Node(os.sep) + + def ensure_path(self, path): + path = self._normpath(path) + # Ignore the root path as we already checked for that; and it + # will always exist/can't be removed anyway... + if path == self._root.item: + return + node = self._root + for piece in self._iter_pieces(path): + child_node = node.find(piece, only_direct=True, + include_self=False) + if child_node is None: + child_node = tree.Node(piece) + node.add(child_node) + node = child_node + + def _fetch_node(self, path): + node = self._root + path = self._normpath(path) + if path == self._root.item: + return node + for piece in self._iter_pieces(path): + node = node.find(piece, only_direct=True, + include_self=False) + if node is None: + raise exc.NotFound("Item not found %s" % path) + return node + + def _get_item(self, path, links=None): + node = self._fetch_node(path) + if 'target' in node.metadata: + # Follow the link (and watch out for loops)... + path = node.metadata['target'] + if links is None: + links = [] + if path in links: + raise ValueError("Recursive link following not" + " allowed (loop %s detected)" + % (links + [path])) + else: + links.append(path) + return self._get_item(path, links=links) + else: + return copy.deepcopy(node.metadata['value']) + + def ls(self, path): + return [node.item for node in self._fetch_node(path)] + + def _iter_pieces(self, path, include_root=False): + if path == self._root.item: + # Check for this directly as the following doesn't work with + # split correctly: + # + # >>> path = "/" + # path.split(os.sep) + # ['', ''] + parts = [] + else: + parts = path.split(os.sep)[1:] + if include_root: + parts.insert(0, self._root.item) + for piece in parts: + yield piece + + def __delitem__(self, path): + node = self._fetch_node(path) + if node is self._root: + raise ValueError("Can not delete '%s'" % self._root.item) + node.disassociate() + + def pformat(self): + return self._root.pformat() + + def symlink(self, src_path, dest_path): + dest_path = self._normpath(dest_path) + src_path = self._normpath(src_path) + dirname, basename = os.path.split(dest_path) + parent_node = self._fetch_node(dirname) + child_node = parent_node.find(basename, + only_direct=True, + include_self=False) + if child_node is None: + child_node = tree.Node(basename) + parent_node.add(child_node) + child_node.metadata['target'] = src_path + + def __getitem__(self, path): + return self._get_item(path) + + def __setitem__(self, path, value): + path = self._normpath(path) + value = copy.deepcopy(value) + try: + item_node = self._fetch_node(path) + item_node.metadata.update(value=value) + except exc.NotFound: + dirname, basename = os.path.split(path) + parent_node = self._fetch_node(dirname) + parent_node.add(tree.Node(basename, value=value)) + + class MemoryBackend(path_based.PathBasedBackend): """A in-memory (non-persistent) backend. - This backend writes logbooks, flow details, and atom details to in-memory - dictionaries and retrieves from those dictionaries as needed. + This backend writes logbooks, flow details, and atom details to a + in-memory filesystem-like structure (rooted by the ``memory`` + instance variable). This backend does *not* provide true transactional semantics. It does guarantee that there will be no inter-thread race conditions when @@ -39,7 +153,7 @@ class MemoryBackend(path_based.PathBasedBackend): super(MemoryBackend, self).__init__(conf) if self._path is None: self._path = os.sep - self.memory = tree.Node(self._path) + self.memory = Filesystem() self.lock = lock_utils.ReaderWriterLock() def get_connection(self): @@ -60,7 +174,6 @@ class Connection(path_based.PathBasedConnection): lock = self.backend.lock.write_lock else: lock = self.backend.lock.read_lock - with lock(): try: yield @@ -69,53 +182,29 @@ class Connection(path_based.PathBasedConnection): except Exception as e: raise exc.StorageFailure("Storage backend internal error", e) - def _fetch_node(self, path): - node = self.backend.memory.find(path) - if node is None: - raise exc.NotFound("Item not found %s" % path) - return node - def _join_path(self, *parts): return os.path.join(*parts) def _get_item(self, path): with self._memory_lock(): - return copy.deepcopy(self._fetch_node(path).metadata['value']) + return self.backend.memory[path] def _set_item(self, path, value, transaction): - value = copy.deepcopy(value) - try: - item_node = self._fetch_node(path) - item_node.metadata.update(value=value) - except exc.NotFound: - dirname, basename = os.path.split(path) - parent_node = self._fetch_node(dirname) - parent_node.add(tree.Node(path, name=basename, value=value)) + self.backend.memory[path] = value def _del_tree(self, path, transaction): - node = self._fetch_node(path) - node.disassociate() + del self.backend.memory[path] def _get_children(self, path): with self._memory_lock(): - return [node.metadata['name'] for node in self._fetch_node(path)] + return self.backend.memory.ls(path) def _ensure_path(self, path): with self._memory_lock(write=True): - path = os.path.normpath(path) - parts = path.split(os.sep) - node = self.backend.memory - for p in range(len(parts) - 1): - node_path = os.sep.join(parts[:p + 2]) - try: - node = self._fetch_node(node_path) - except exc.NotFound: - node.add(tree.Node(node_path, name=parts[p + 1])) + self.backend.memory.ensure_path(path) def _create_link(self, src_path, dest_path, transaction): - dirname, basename = os.path.split(dest_path) - parent_node = self._fetch_node(dirname) - parent_node.add(tree.Node(dest_path, name=basename, target=src_path)) + self.backend.memory.symlink(src_path, dest_path) @contextlib.contextmanager def _transaction(self): From 0a2928f810172e406579059291427b9ddecf843a Mon Sep 17 00:00:00 2001 From: Min Pae Date: Thu, 19 Feb 2015 22:20:42 -0800 Subject: [PATCH 065/246] Added a map and a reduce task - Added a MapFunctorTask and ReduceFunctorTask to mimic the python builtin functions map and reduce, respectively Change-Id: I29cd1249e1671aedfd2d25460d746b8d2154df2b --- taskflow/task.py | 89 +++++++++++++++++++ taskflow/tests/unit/test_functor_task.py | 29 ++++++ taskflow/tests/unit/test_mapfunctor_task.py | 80 +++++++++++++++++ .../tests/unit/test_reducefunctor_task.py | 72 +++++++++++++++ taskflow/tests/unit/test_task.py | 45 ++++++++++ taskflow/tests/unit/test_utils.py | 15 ++++ taskflow/utils/misc.py | 15 ++++ 7 files changed, 345 insertions(+) create mode 100644 taskflow/tests/unit/test_mapfunctor_task.py create mode 100644 taskflow/tests/unit/test_reducefunctor_task.py diff --git a/taskflow/task.py b/taskflow/task.py index 8fa9ffb5..f98508c4 100644 --- a/taskflow/task.py +++ b/taskflow/task.py @@ -1,5 +1,6 @@ # -*- coding: utf-8 -*- +# Copyright 2015 Hewlett-Packard Development Company, L.P. # Copyright (C) 2013 Rackspace Hosting Inc. All Rights Reserved. # Copyright (C) 2013 Yahoo! Inc. All Rights Reserved. # @@ -20,6 +21,7 @@ import copy from oslo_utils import reflection import six +from six.moves import reduce as compat_reduce from taskflow import atom from taskflow import logging @@ -234,3 +236,90 @@ class FunctorTask(BaseTask): return self._revert(*args, **kwargs) else: return None + + +class ReduceFunctorTask(BaseTask): + """General purpose Task to reduce a list by applying a function + + This Task mimics the behavior of Python's built-in reduce function. The + Task takes a functor (lambda or otherwise) and a list. The list is + specified using the requires argument of the Task. When executed, this + task calls reduce with the functor and list as arguments. The resulting + value from the call to reduce is then returned after execution. + """ + def __init__(self, functor, requires, name=None, provides=None, + auto_extract=True, rebind=None, inject=None): + + if not six.callable(functor): + raise ValueError("Function to use for reduce must be callable") + + f_args = reflection.get_callable_args(functor) + if len(f_args) != 2: + raise ValueError("%s arguments were provided. Reduce functor " + "must take exactly 2 arguments." % len(f_args)) + + if not misc.is_iterable(requires): + raise TypeError("%s type was provided for requires. Requires " + "must be an iterable." % type(requires)) + + if len(requires) < 2: + raise ValueError("%s elements were provided. Requires must have " + "at least 2 elements." % len(requires)) + + if name is None: + name = reflection.get_callable_name(functor) + super(ReduceFunctorTask, self).__init__(name=name, provides=provides, + inject=inject) + + self._functor = functor + + self._build_arg_mapping(executor=self.execute, requires=requires, + rebind=rebind, auto_extract=auto_extract) + + def execute(self, *args, **kwargs): + l = [kwargs[r] for r in self.requires] + return compat_reduce(self._functor, l) + + +class MapFunctorTask(BaseTask): + """General purpose Task to map a function to a list + + This Task mimics the behavior of Python's built-in map function. The Task + takes a functor (lambda or otherwise) and a list. The list is specified + using the requires argument of the Task. When executed, this task calls + map with the functor and list as arguments. The resulting list from the + call to map is then returned after execution. + + Each value of the returned list can be bound to individual names using + the provides argument, following taskflow standard behavior. Order is + preserved in the returned list. + """ + + def __init__(self, functor, requires, name=None, provides=None, + auto_extract=True, rebind=None, inject=None): + + if not six.callable(functor): + raise ValueError("Function to use for map must be callable") + + f_args = reflection.get_callable_args(functor) + if len(f_args) != 1: + raise ValueError("%s arguments were provided. Map functor must " + "take exactly 1 argument." % len(f_args)) + + if not misc.is_iterable(requires): + raise TypeError("%s type was provided for requires. Requires " + "must be an iterable." % type(requires)) + + if name is None: + name = reflection.get_callable_name(functor) + super(MapFunctorTask, self).__init__(name=name, provides=provides, + inject=inject) + + self._functor = functor + + self._build_arg_mapping(executor=self.execute, requires=requires, + rebind=rebind, auto_extract=auto_extract) + + def execute(self, *args, **kwargs): + l = [kwargs[r] for r in self.requires] + return list(map(self._functor, l)) diff --git a/taskflow/tests/unit/test_functor_task.py b/taskflow/tests/unit/test_functor_task.py index 676c2c8b..2deed5fe 100644 --- a/taskflow/tests/unit/test_functor_task.py +++ b/taskflow/tests/unit/test_functor_task.py @@ -40,6 +40,11 @@ class BunchOfFunctions(object): raise RuntimeError('Woot!') +five = lambda: 5 + +multiply = lambda x, y: x * y + + class FunctorTaskTest(test.TestCase): def test_simple(self): @@ -63,3 +68,27 @@ class FunctorTaskTest(test.TestCase): self.assertRaisesRegexp(RuntimeError, '^Woot', taskflow.engines.run, flow) self.assertEqual(values, ['one', 'fail', 'revert one']) + + def test_lambda_functors(self): + t = base.FunctorTask + + flow = linear_flow.Flow('test') + flow.add( + t(five, provides='five', name='five'), + t(multiply, provides='product', name='product') + ) + + flow_store = { + 'x': 2, + 'y': 3 + } + + result = taskflow.engines.run(flow, store=flow_store) + + expected = flow_store.copy() + expected.update({ + 'five': 5, + 'product': 6 + }) + + self.assertDictEqual(expected, result) diff --git a/taskflow/tests/unit/test_mapfunctor_task.py b/taskflow/tests/unit/test_mapfunctor_task.py new file mode 100644 index 00000000..50cd8864 --- /dev/null +++ b/taskflow/tests/unit/test_mapfunctor_task.py @@ -0,0 +1,80 @@ +# -*- coding: utf-8 -*- + +# Copyright 2015 Hewlett-Packard Development Company, L.P. +# +# 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. + +import taskflow.engines as engines +from taskflow.patterns import linear_flow +from taskflow import task as base +from taskflow import test + + +def double(x): + return x * 2 + +square = lambda x: x * x + + +class MapFunctorTaskTest(test.TestCase): + + def setUp(self): + super(MapFunctorTaskTest, self).setUp() + + self.flow_store = { + 'a': 1, + 'b': 2, + 'c': 3, + 'd': 4, + 'e': 5, + } + + def test_double_array(self): + expected = self.flow_store.copy() + expected.update({ + 'double_a': 2, + 'double_b': 4, + 'double_c': 6, + 'double_d': 8, + 'double_e': 10, + }) + + requires = self.flow_store.keys() + provides = ["double_%s" % k for k in requires] + + flow = linear_flow.Flow("double array flow") + flow.add(base.MapFunctorTask(double, requires=requires, + provides=provides)) + + result = engines.run(flow, store=self.flow_store) + self.assertDictEqual(expected, result) + + def test_square_array(self): + expected = self.flow_store.copy() + expected.update({ + 'square_a': 1, + 'square_b': 4, + 'square_c': 9, + 'square_d': 16, + 'square_e': 25, + }) + + requires = self.flow_store.keys() + provides = ["square_%s" % k for k in requires] + + flow = linear_flow.Flow("square array flow") + flow.add(base.MapFunctorTask(square, requires=requires, + provides=provides)) + + result = engines.run(flow, store=self.flow_store) + self.assertDictEqual(expected, result) diff --git a/taskflow/tests/unit/test_reducefunctor_task.py b/taskflow/tests/unit/test_reducefunctor_task.py new file mode 100644 index 00000000..9da1b4f7 --- /dev/null +++ b/taskflow/tests/unit/test_reducefunctor_task.py @@ -0,0 +1,72 @@ +# -*- coding: utf-8 -*- + +# Copyright 2015 Hewlett-Packard Development Company, L.P. +# +# 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. + +import taskflow.engines as engines +from taskflow.patterns import linear_flow +from taskflow import task as base +from taskflow import test + + +def sum(x, y): + return x + y + +multiply = lambda x, y: x * y + + +class ReduceFunctorTaskTest(test.TestCase): + + def setUp(self): + super(ReduceFunctorTaskTest, self).setUp() + + self.flow_store = { + 'a': 1, + 'b': 2, + 'c': 3, + 'd': 4, + 'e': 5, + } + + def test_sum_array(self): + expected = self.flow_store.copy() + expected.update({ + 'sum': 15 + }) + + requires = self.flow_store.keys() + provides = 'sum' + + flow = linear_flow.Flow("sum array flow") + flow.add(base.ReduceFunctorTask(sum, requires=requires, + provides=provides)) + + result = engines.run(flow, store=self.flow_store) + self.assertDictEqual(expected, result) + + def test_multiply_array(self): + expected = self.flow_store.copy() + expected.update({ + 'product': 120 + }) + + requires = self.flow_store.keys() + provides = 'product' + + flow = linear_flow.Flow("square array flow") + flow.add(base.ReduceFunctorTask(multiply, requires=requires, + provides=provides)) + + result = engines.run(flow, store=self.flow_store) + self.assertDictEqual(expected, result) diff --git a/taskflow/tests/unit/test_task.py b/taskflow/tests/unit/test_task.py index 72f98863..2f6d4e97 100644 --- a/taskflow/tests/unit/test_task.py +++ b/taskflow/tests/unit/test_task.py @@ -1,5 +1,6 @@ # -*- coding: utf-8 -*- +# Copyright 2015 Hewlett-Packard Development Company, L.P. # Copyright (C) 2013 Yahoo! Inc. All Rights Reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); you may @@ -351,3 +352,47 @@ class FunctorTaskTest(test.TestCase): def test_revert_not_callable(self): self.assertRaises(ValueError, task.FunctorTask, lambda: None, revert=2) + + +class ReduceFunctorTaskTest(test.TestCase): + + def test_invalid_functor(self): + # Functor not callable + self.assertRaises(ValueError, task.ReduceFunctorTask, 2, requires=5) + + # Functor takes no arguments + self.assertRaises(ValueError, task.ReduceFunctorTask, lambda: None, + requires=5) + + # Functor takes too few arguments + self.assertRaises(ValueError, task.ReduceFunctorTask, lambda x: None, + requires=5) + + def test_functor_invalid_requires(self): + # Invalid type, requires is not iterable + self.assertRaises(TypeError, task.ReduceFunctorTask, + lambda x, y: None, requires=1) + + # Too few elements in requires + self.assertRaises(ValueError, task.ReduceFunctorTask, + lambda x, y: None, requires=[1]) + + +class MapFunctorTaskTest(test.TestCase): + + def test_invalid_functor(self): + # Functor not callable + self.assertRaises(ValueError, task.MapFunctorTask, 2, requires=5) + + # Functor takes no arguments + self.assertRaises(ValueError, task.MapFunctorTask, lambda: None, + requires=5) + + # Functor takes too many arguments + self.assertRaises(ValueError, task.MapFunctorTask, lambda x, y: None, + requires=5) + + def test_functor_invalid_requires(self): + # Invalid type, requires is not iterable + self.assertRaises(TypeError, task.MapFunctorTask, lambda x: None, + requires=1) diff --git a/taskflow/tests/unit/test_utils.py b/taskflow/tests/unit/test_utils.py index 1477fe5a..56dc6691 100644 --- a/taskflow/tests/unit/test_utils.py +++ b/taskflow/tests/unit/test_utils.py @@ -271,3 +271,18 @@ class TestClamping(test.TestCase): misc.clamp(2, 0.0, 1.0, on_clamped=on_clamped) self.assertEqual(1, len(calls)) + + +class TestIterable(test.TestCase): + def test_string_types(self): + self.assertFalse(misc.is_iterable('string')) + self.assertFalse(misc.is_iterable(u'string')) + + def test_list(self): + self.assertTrue(misc.is_iterable(list())) + + def test_tuple(self): + self.assertTrue(misc.is_iterable(tuple())) + + def test_dict(self): + self.assertTrue(misc.is_iterable(dict())) diff --git a/taskflow/utils/misc.py b/taskflow/utils/misc.py index 39708b5c..d8eb9c7f 100644 --- a/taskflow/utils/misc.py +++ b/taskflow/utils/misc.py @@ -15,6 +15,7 @@ # License for the specific language governing permissions and limitations # under the License. +import collections import contextlib import datetime import errno @@ -490,3 +491,17 @@ def capture_failure(): raise RuntimeError("No active exception is being handled") else: yield failure.Failure(exc_info=exc_info) + + +def is_iterable(obj): + """Tests an object to to determine whether it is iterable. + + This function will test the specified object to determine whether it is + iterable. String types (both ``str`` and ``unicode``) are ignored and will + return False. + + :param obj: object to be tested for iterable + :return: True if object is iterable and is not a string + """ + return (not isinstance(obj, six.string_types) and + isinstance(obj, collections.Iterable)) From 86021183570a26361cd4ba2fd9536ca713c14ebe Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 11 Mar 2015 17:45:31 -0700 Subject: [PATCH 066/246] Add a in-memory backend dumping example Change-Id: Ie29735a16129cedd574e09ed06b44a98560f0d90 --- taskflow/examples/dump_memory_backend.py | 83 ++++++++++++++++++++ taskflow/persistence/backends/impl_memory.py | 19 +++-- 2 files changed, 94 insertions(+), 8 deletions(-) create mode 100644 taskflow/examples/dump_memory_backend.py diff --git a/taskflow/examples/dump_memory_backend.py b/taskflow/examples/dump_memory_backend.py new file mode 100644 index 00000000..7209dda0 --- /dev/null +++ b/taskflow/examples/dump_memory_backend.py @@ -0,0 +1,83 @@ +# -*- coding: utf-8 -*- + +# Copyright (C) 2015 Yahoo! Inc. 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. + +import logging +import os +import sys + +logging.basicConfig(level=logging.ERROR) + +self_dir = os.path.abspath(os.path.dirname(__file__)) +top_dir = os.path.abspath(os.path.join(os.path.dirname(__file__), + os.pardir, + os.pardir)) +sys.path.insert(0, top_dir) +sys.path.insert(0, self_dir) + +from taskflow import engines +from taskflow.patterns import linear_flow as lf +from taskflow.persistence import backends +from taskflow import task +from taskflow.utils import persistence_utils as pu + +# INTRO: in this example we create a dummy flow with a dummy task, and run +# it using a in-memory backend and pre/post run we dump out the contents +# of the in-memory backends tree structure (which can be quite useful to +# look at for debugging or other analysis). + + +class PrintTask(task.Task): + def execute(self): + print("Running '%s'" % self.name) + + +backend = backends.fetch({ + 'connection': 'memory://', +}) +book, flow_detail = pu.temporary_flow_detail(backend=backend) + +# Make a little flow and run it... +f = lf.Flow('root') +for alpha in ['a', 'b', 'c']: + f.add(PrintTask(alpha)) + +e = engines.load(f, flow_detail=flow_detail, + book=book, backend=backend) +e.compile() +e.prepare() + +print("----------") +print("Before run") +print("----------") +print(backend.memory.pformat()) +print("----------") + +e.run() + +print("---------") +print("After run") +print("---------") +entries = [os.path.join(backend.memory.root_path, child) + for child in backend.memory.ls(backend.memory.root_path)] +while entries: + path = entries.pop() + value = backend.memory[path] + if value: + print("%s -> %s" % (path, value)) + else: + print("%s" % (path)) + entries.extend(os.path.join(path, child) + for child in backend.memory.ls(path)) diff --git a/taskflow/persistence/backends/impl_memory.py b/taskflow/persistence/backends/impl_memory.py index 0f5e22b4..c1faae84 100644 --- a/taskflow/persistence/backends/impl_memory.py +++ b/taskflow/persistence/backends/impl_memory.py @@ -26,17 +26,20 @@ from taskflow.utils import lock_utils class Filesystem(object): - """An in-memory tree filesystem-like structure.""" + """An in-memory filesystem-like structure.""" - @staticmethod - def _normpath(path): - if not path.startswith(os.sep): + #: Root path of the in-memory filesystem. + root_path = os.sep + + @classmethod + def _normpath(cls, path): + if not path.startswith(cls.root_path): raise ValueError("This filesystem can only normalize absolute" " paths: '%s' is not valid" % path) return os.path.normpath(path) def __init__(self): - self._root = tree.Node(os.sep) + self._root = tree.Node(self.root_path, value=None) def ensure_path(self, path): path = self._normpath(path) @@ -49,7 +52,7 @@ class Filesystem(object): child_node = node.find(piece, only_direct=True, include_self=False) if child_node is None: - child_node = tree.Node(piece) + child_node = tree.Node(piece, value=None) node.add(child_node) node = child_node @@ -62,7 +65,7 @@ class Filesystem(object): node = node.find(piece, only_direct=True, include_self=False) if node is None: - raise exc.NotFound("Item not found %s" % path) + raise exc.NotFound("Path '%s' not found" % path) return node def _get_item(self, path, links=None): @@ -119,7 +122,7 @@ class Filesystem(object): only_direct=True, include_self=False) if child_node is None: - child_node = tree.Node(basename) + child_node = tree.Node(basename, value=None) parent_node.add(child_node) child_node.metadata['target'] = src_path From 12c28bb61d8ff178548b3c594575cc7bd50f5c7f Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Thu, 12 Mar 2015 13:11:31 -0700 Subject: [PATCH 067/246] Ensure event time listener is in listeners docs Also expands the docstring so its known what keys this stores under and what data it is recording. Change-Id: Id2949ca6acc27d4bf41e37743f705d653f426c05 --- doc/source/notifications.rst | 7 +++++-- taskflow/listeners/timing.py | 11 ++++++++++- 2 files changed, 15 insertions(+), 3 deletions(-) diff --git a/doc/source/notifications.rst b/doc/source/notifications.rst index 4e037c26..816393ed 100644 --- a/doc/source/notifications.rst +++ b/doc/source/notifications.rst @@ -170,6 +170,8 @@ Timing listeners .. autoclass:: taskflow.listeners.timing.PrintingDurationListener +.. autoclass:: taskflow.listeners.timing.EventTimeListener + Claim listener -------------- @@ -191,6 +193,7 @@ Hierarchy taskflow.listeners.logging.DynamicLoggingListener taskflow.listeners.logging.LoggingListener taskflow.listeners.printing.PrintingListener - taskflow.listeners.timing.PrintingTimingListener - taskflow.listeners.timing.TimingListener + taskflow.listeners.timing.PrintingDurationListener + taskflow.listeners.timing.EventTimeListener + taskflow.listeners.timing.DurationListener :parts: 1 diff --git a/taskflow/listeners/timing.py b/taskflow/listeners/timing.py index bf9b5229..17d289d6 100644 --- a/taskflow/listeners/timing.py +++ b/taskflow/listeners/timing.py @@ -121,7 +121,16 @@ PrintingTimingListener = moves.moved_class( class EventTimeListener(base.Listener): - """Writes task, flow, and retry event timestamps to atom metadata.""" + """Listener that captures task, flow, and retry event timestamps. + + It records how when an event is received (using unix time) to + storage. It saves the timestamps under keys (in atom or flow details + metadata) of the format ``{event}-timestamp`` where ``event`` is the + state/event name that has been received. + + This information can be later extracted/examined to derive durations... + """ + def __init__(self, engine, task_listen_for=base.DEFAULT_LISTEN_FOR, flow_listen_for=base.DEFAULT_LISTEN_FOR, From 526f2872012b42d634faaaee916e1e050d07a5c9 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Thu, 12 Mar 2015 15:18:36 -0700 Subject: [PATCH 068/246] Allow passing 'many_handler' to fetch_all function When many values are found for a given result a callback is called to determine what to return. To match the fetch routine allow for the fetch_all function to pass along a handler (and default to one that is the existing behavior) so that people can extract there own values (if they so choose). Change-Id: I8c105e1a810e8e0c2c210613506c9ce59021009d --- taskflow/storage.py | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/taskflow/storage.py b/taskflow/storage.py index 3cf496aa..4b9b80dd 100644 --- a/taskflow/storage.py +++ b/taskflow/storage.py @@ -644,11 +644,13 @@ class Storage(object): @lock_utils.read_locked def fetch(self, name, many_handler=None): """Fetch a named result.""" - # By default we just return the first of many (unless provided - # a different callback that can translate many results into something - # more meaningful). + def _many_handler(values): + # By default we just return the first of many (unless provided + # a different callback that can translate many results into + # something more meaningful). + return values[0] if many_handler is None: - many_handler = lambda values: values[0] + many_handler = _many_handler try: providers = self._reverse_mapping[name] except KeyError: @@ -758,12 +760,14 @@ class Storage(object): return missing @lock_utils.read_locked - def fetch_all(self): + def fetch_all(self, many_handler=None): """Fetch all named results known so far.""" - def many_handler(values): + def _many_handler(values): if len(values) > 1: return values return values[0] + if many_handler is None: + many_handler = _many_handler results = {} for name in six.iterkeys(self._reverse_mapping): try: From c1f0200780e4ba0347b97d9019728eeb9e079cdc Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Thu, 12 Mar 2015 15:35:37 -0700 Subject: [PATCH 069/246] Add newline to avoid sphinx warning Fixes the following warning that occurs when running sphinx. WARNING: Definition list ends without a blank line; unexpected unindent. Change-Id: I22605dece596e0397216590bc85f9e56f986e609 --- taskflow/persistence/backends/impl_zookeeper.py | 1 + 1 file changed, 1 insertion(+) diff --git a/taskflow/persistence/backends/impl_zookeeper.py b/taskflow/persistence/backends/impl_zookeeper.py index b60e93bf..5626b289 100644 --- a/taskflow/persistence/backends/impl_zookeeper.py +++ b/taskflow/persistence/backends/impl_zookeeper.py @@ -34,6 +34,7 @@ class ZkBackend(path_based.PathBasedBackend): """A zookeeper-backed backend. Example configuration:: + conf = { "hosts": "192.168.0.1:2181,192.168.0.2:2181,192.168.0.3:2181", "path": "/taskflow", From 0e952e65bcf3d115db3d3d40bff6ec7772fe2962 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Thu, 12 Mar 2015 16:02:57 -0700 Subject: [PATCH 070/246] Some small wbe engine doc tweaks Change-Id: I39c8bb58d910f086b22f4a4591796affc75ea242 --- doc/source/workers.rst | 20 +++++++++++--------- 1 file changed, 11 insertions(+), 9 deletions(-) diff --git a/doc/source/workers.rst b/doc/source/workers.rst index b081dc74..01058b7a 100644 --- a/doc/source/workers.rst +++ b/doc/source/workers.rst @@ -17,8 +17,8 @@ Terminology ----------- Client - Code or program or service that uses this library to define flows and - run them via engines. + Code or program or service (or user) that uses this library to define + flows and run them via engines. Transport + protocol Mechanism (and `protocol`_ on top of that mechanism) used to pass information @@ -117,7 +117,7 @@ engine executor in the following manner: 4. The executor gets the task request confirmation from the worker and the task request state changes from the ``PENDING`` to the ``RUNNING`` state. Once a task request is in the ``RUNNING`` state it can't be timed-out (considering - that task execution process may take unpredictable time). + that the task execution process may take an unpredictable amount of time). 5. The executor gets the task execution result from the worker and passes it back to the executor and worker-based engine to finish task processing (this repeats for subsequent tasks). @@ -128,7 +128,9 @@ engine executor in the following manner: json-serializable (they contain references to tracebacks which are not serializable), so they are converted to dicts before sending and converted from dicts after receiving on both executor & worker sides (this - translation is lossy since the traceback won't be fully retained). + translation is lossy since the traceback can't be fully retained, due + to its contents containing internal interpreter references and + details). Protocol ~~~~~~~~ @@ -405,11 +407,11 @@ Limitations locally to avoid transport overhead for very simple tasks (currently it will run even lightweight tasks remotely, which may be non-performant). * Fault detection, currently when a worker acknowledges a task the engine will - wait for the task result indefinitely (a task could take a very long time to - finish). In the future there needs to be a way to limit the duration of a - remote workers execution (and track there liveness) and possibly spawn - the task on a secondary worker if a timeout is reached (aka the first worker - has died or has stopped responding). + wait for the task result indefinitely (a task may take an indeterminate + amount of time to finish). In the future there needs to be a way to limit + the duration of a remote workers execution (and track there liveness) and + possibly spawn the task on a secondary worker if a timeout is reached (aka + the first worker has died or has stopped responding). Interfaces ========== From bf5164e36c819718e7b8bb6cd68f1af4efc95ddb Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Fri, 13 Mar 2015 15:39:10 -0700 Subject: [PATCH 071/246] Use a collections.namedtuple for the request work unit Instead of returning a tuple with hard to understand and read contents a namedtuple can help understand what the tuple is composed of and how it can be used/what it is, so we should prefer to use it when we can for the wbe requested work to perform. Change-Id: I8147814449d04ba9a03730547ac113e4ef7b272a --- taskflow/engines/worker_based/protocol.py | 9 +++++-- taskflow/engines/worker_based/server.py | 31 ++++++++++++----------- 2 files changed, 23 insertions(+), 17 deletions(-) diff --git a/taskflow/engines/worker_based/protocol.py b/taskflow/engines/worker_based/protocol.py index b22d61fe..34dcd506 100644 --- a/taskflow/engines/worker_based/protocol.py +++ b/taskflow/engines/worker_based/protocol.py @@ -15,6 +15,7 @@ # under the License. import abc +import collections import threading from concurrent import futures @@ -171,6 +172,10 @@ class Notify(Message): % (cls.TYPE, e.message), e) +_WorkUnit = collections.namedtuple('_WorkUnit', ['task_cls', 'task_name', + 'action', 'arguments']) + + class Request(Message): """Represents request with execution results. @@ -371,7 +376,7 @@ class Request(Message): @staticmethod def from_dict(data, task_uuid=None): - """Parses **validated** data before it can be further processed. + """Parses **validated** data into a work unit. All :py:class:`~taskflow.types.failure.Failure` objects that have been converted to dict(s) on the remote side will now converted back @@ -401,7 +406,7 @@ class Request(Message): arguments['failures'] = {} for task, fail_data in six.iteritems(failures): arguments['failures'][task] = ft.Failure.from_dict(fail_data) - return (task_cls, task_name, action, arguments) + return _WorkUnit(task_cls, task_name, action, arguments) class Response(Message): diff --git a/taskflow/engines/worker_based/server.py b/taskflow/engines/worker_based/server.py index 99a39895..033bd3c7 100644 --- a/taskflow/engines/worker_based/server.py +++ b/taskflow/engines/worker_based/server.py @@ -172,10 +172,9 @@ class Server(object): reply_callback = functools.partial(self._reply, True, reply_to, task_uuid) - # parse request to get task name, action and action arguments + # Parse the request to get the activity/work to perform. try: - bundle = pr.Request.from_dict(request, task_uuid=task_uuid) - task_cls, task_name, action, arguments = bundle + work = pr.Request.from_dict(request, task_uuid=task_uuid) except ValueError: with misc.capture_failure() as failure: LOG.warn("Failed to parse request contents from message '%s'", @@ -183,34 +182,35 @@ class Server(object): reply_callback(result=failure.to_dict()) return - # get task endpoint + # Now fetch the task endpoint (and action handler on it). try: - endpoint = self._endpoints[task_cls] + endpoint = self._endpoints[work.task_cls] except KeyError: with misc.capture_failure() as failure: LOG.warn("The '%s' task endpoint does not exist, unable" " to continue processing request message '%s'", - task_cls, ku.DelayedPretty(message), exc_info=True) + work.task_cls, ku.DelayedPretty(message), + exc_info=True) reply_callback(result=failure.to_dict()) return else: try: - handler = getattr(endpoint, action) + handler = getattr(endpoint, work.action) except AttributeError: with misc.capture_failure() as failure: LOG.warn("The '%s' handler does not exist on task endpoint" " '%s', unable to continue processing request" - " message '%s'", action, endpoint, + " message '%s'", work.action, endpoint, ku.DelayedPretty(message), exc_info=True) reply_callback(result=failure.to_dict()) return else: try: - task = endpoint.generate(name=task_name) + task = endpoint.generate(name=work.task_name) except Exception: with misc.capture_failure() as failure: LOG.warn("The '%s' task '%s' generation for request" - " message '%s' failed", endpoint, action, + " message '%s' failed", endpoint, work.action, ku.DelayedPretty(message), exc_info=True) reply_callback(result=failure.to_dict()) return @@ -218,7 +218,7 @@ class Server(object): if not reply_callback(state=pr.RUNNING): return - # associate *any* events this task emits with a proxy that will + # Associate *any* events this task emits with a proxy that will # emit them back to the engine... for handling at the engine side # of things... if task.notifier.can_be_registered(nt.Notifier.ANY): @@ -226,22 +226,23 @@ class Server(object): functools.partial(self._on_event, reply_to, task_uuid)) elif isinstance(task.notifier, nt.RestrictedNotifier): - # only proxy the allowable events then... + # Only proxy the allowable events then... for event_type in task.notifier.events_iter(): task.notifier.register(event_type, functools.partial(self._on_event, reply_to, task_uuid)) - # perform the task action + # Perform the task action. try: - result = handler(task, **arguments) + result = handler(task, **work.arguments) except Exception: with misc.capture_failure() as failure: LOG.warn("The '%s' endpoint '%s' execution for request" - " message '%s' failed", endpoint, action, + " message '%s' failed", endpoint, work.action, ku.DelayedPretty(message), exc_info=True) reply_callback(result=failure.to_dict()) else: + # And be done with it! if isinstance(result, ft.Failure): reply_callback(result=result.to_dict()) else: From 52099616307e8cde256469c7f7d641bc3ae414db Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Fri, 13 Mar 2015 16:10:35 -0700 Subject: [PATCH 072/246] Have this example exit non-zero if incorrect results When this examples engine does not produce the expected results have it exit with a non-zero error code so that the users knows (and so that the example testing system fails when this happens). Change-Id: I8c3b80a7dc1c7ef47d7804526346883b24caabc4 --- taskflow/examples/graph_flow.py | 25 +++++++++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/taskflow/examples/graph_flow.py b/taskflow/examples/graph_flow.py index 9f28dc71..862db389 100644 --- a/taskflow/examples/graph_flow.py +++ b/taskflow/examples/graph_flow.py @@ -80,12 +80,37 @@ store = { "y5": 9, } +# This is the expected values that should be created. +unexpected = 0 +expected = [ + ('x1', 4), + ('x2', 12), + ('x3', 16), + ('x4', 21), + ('x5', 20), + ('x6', 41), + ('x7', 82), +] + result = taskflow.engines.run( flow, engine='serial', store=store) print("Single threaded engine result %s" % result) +for (name, value) in expected: + actual = result.get(name) + if actual != value: + sys.stderr.write("%s != %s\n" % (actual, value)) + unexpected += 1 result = taskflow.engines.run( flow, engine='parallel', store=store) print("Multi threaded engine result %s" % result) +for (name, value) in expected: + actual = result.get(name) + if actual != value: + sys.stderr.write("%s != %s\n" % (actual, value)) + unexpected += 1 + +if unexpected: + sys.exit(1) From 7e052e0447a22642ec701267dc61c3634d153d48 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Fri, 13 Mar 2015 17:24:59 -0700 Subject: [PATCH 073/246] Add a log statement pre-validation that dumps graph info Before validation occurs it can be useful to let the developer know the number of nodes, the edge count and the density of the graph that is about to be validated (as huge graphs, or very dense graphs, or a combination of both) will take longer to validate. Change-Id: I7afb15ef617316e561dfdecc5bf05f4c8263d0d1 --- taskflow/engines/action_engine/engine.py | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/taskflow/engines/action_engine/engine.py b/taskflow/engines/action_engine/engine.py index 68bd3458..e7763068 100644 --- a/taskflow/engines/action_engine/engine.py +++ b/taskflow/engines/action_engine/engine.py @@ -19,6 +19,7 @@ import contextlib import threading from concurrent import futures +import networkx as nx from oslo_utils import excutils from oslo_utils import strutils import six @@ -28,12 +29,15 @@ from taskflow.engines.action_engine import executor from taskflow.engines.action_engine import runtime from taskflow.engines import base from taskflow import exceptions as exc +from taskflow import logging from taskflow import states from taskflow import storage from taskflow.types import failure from taskflow.utils import lock_utils from taskflow.utils import misc +LOG = logging.getLogger(__name__) + @contextlib.contextmanager def _start_stop(executor): @@ -213,9 +217,16 @@ class ActionEngine(base.Engine): # flow/task provided or storage provided, if there are still missing # dependencies then this flow will fail at runtime (which we can avoid # by failing at validation time). + execution_graph = self._compilation.execution_graph + if LOG.isEnabledFor(logging.BLATHER): + LOG.blather("Validating scoping and argument visibility for" + " execution graph with %s nodes and %s edges with" + " density %0.3f", execution_graph.number_of_nodes(), + execution_graph.number_of_edges(), + nx.density(execution_graph)) missing = set() fetch = self.storage.fetch_unsatisfied_args - for node in self._compilation.execution_graph.nodes_iter(): + for node in execution_graph.nodes_iter(): missing.update(fetch(node.name, node.rebind, optional_args=node.optional)) if missing: From 87d6f5823d89c82c74f6d3f638e79e4eac7cba07 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Fri, 13 Mar 2015 18:51:34 -0700 Subject: [PATCH 074/246] Rename attribute '_graph' to '_execution_graph' The naming of '_execution_graph' is more appropriate so that people looking at this code are clear on exactly what this graph is and what it is used for. Change-Id: I8d80101a05b34886a87bdc75e38a0457c785fcf4 --- taskflow/engines/action_engine/analyzer.py | 24 +++++++++++----------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/taskflow/engines/action_engine/analyzer.py b/taskflow/engines/action_engine/analyzer.py index d4b181a6..ddae1e97 100644 --- a/taskflow/engines/action_engine/analyzer.py +++ b/taskflow/engines/action_engine/analyzer.py @@ -33,7 +33,7 @@ class Analyzer(object): def __init__(self, compilation, storage): self._storage = storage - self._graph = compilation.execution_graph + self._execution_graph = compilation.execution_graph def get_next_nodes(self, node=None): if node is None: @@ -65,9 +65,9 @@ class Analyzer(object): otherwise it will examine the whole graph. """ if node: - nodes = self._graph.successors(node) + nodes = self._execution_graph.successors(node) else: - nodes = self._graph.nodes_iter() + nodes = self._execution_graph.nodes_iter() available_nodes = [] for node in nodes: @@ -83,9 +83,9 @@ class Analyzer(object): node, otherwise it will examine the whole graph. """ if node: - nodes = self._graph.predecessors(node) + nodes = self._execution_graph.predecessors(node) else: - nodes = self._graph.nodes_iter() + nodes = self._execution_graph.nodes_iter() available_nodes = [] for node in nodes: @@ -102,7 +102,7 @@ class Analyzer(object): return False task_names = [] - for prev_task in self._graph.predecessors(task): + for prev_task in self._execution_graph.predecessors(task): task_names.append(prev_task.name) task_states = self._storage.get_atoms_states(task_names) @@ -118,7 +118,7 @@ class Analyzer(object): return False task_names = [] - for prev_task in self._graph.successors(task): + for prev_task in self._execution_graph.successors(task): task_names.append(prev_task.name) task_states = self._storage.get_atoms_states(task_names) @@ -127,7 +127,7 @@ class Analyzer(object): def iterate_subgraph(self, retry): """Iterates a subgraph connected to given retry controller.""" - for _src, dst in traversal.dfs_edges(self._graph, retry): + for _src, dst in traversal.dfs_edges(self._execution_graph, retry): yield dst def iterate_retries(self, state=None): @@ -135,20 +135,20 @@ class Analyzer(object): If no state is provided it will yield back all retry controllers. """ - for node in self._graph.nodes_iter(): + for node in self._execution_graph.nodes_iter(): if isinstance(node, retry_atom.Retry): if not state or self.get_state(node) == state: yield node def iterate_all_nodes(self): - for node in self._graph.nodes_iter(): + for node in self._execution_graph.nodes_iter(): yield node def find_atom_retry(self, atom): - return self._graph.node[atom].get('retry') + return self._execution_graph.node[atom].get('retry') def is_success(self): - for node in self._graph.nodes_iter(): + for node in self._execution_graph.nodes_iter(): if self.get_state(node) != st.SUCCESS: return False return True From a1f9321c3fda9af6ed4a240091f9c0c4917dc75d Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Fri, 13 Mar 2015 00:03:35 -0700 Subject: [PATCH 075/246] Ensure we register & deregister conductor listeners Instead of just registering engine listeners that were returned, make sure we also deregister them when the engine has either finished or failed. This ensures that if a listener has hold of any resources (or other) that it can clean those up and be sure that its deregister call will be made. Change-Id: Ia1420c435156362698702fed2bda11c2a0fef803 --- taskflow/conductors/backends/impl_blocking.py | 15 +++++++-- taskflow/conductors/base.py | 2 -- taskflow/utils/misc.py | 32 +++++++++++++++++++ 3 files changed, 44 insertions(+), 5 deletions(-) diff --git a/taskflow/conductors/backends/impl_blocking.py b/taskflow/conductors/backends/impl_blocking.py index 1f6a9ee6..b53452af 100644 --- a/taskflow/conductors/backends/impl_blocking.py +++ b/taskflow/conductors/backends/impl_blocking.py @@ -21,6 +21,7 @@ from taskflow import logging from taskflow.types import timing as tt from taskflow.utils import async_utils from taskflow.utils import deprecation +from taskflow.utils import misc from taskflow.utils import threading_utils LOG = logging.getLogger(__name__) @@ -88,11 +89,19 @@ class BlockingConductor(base.Conductor): def dispatching(self): return not self._dead.is_set() + def _listeners_from_job(self, job, engine): + listeners = super(BlockingConductor, self)._listeners_from_job(job, + engine) + listeners.append(logging_listener.LoggingListener(engine, log=LOG)) + return listeners + def _dispatch_job(self, job): engine = self._engine_from_job(job) - consume = True - with logging_listener.LoggingListener(engine, log=LOG): + listeners = self._listeners_from_job(job, engine) + with misc.ListenerStack(LOG) as stack: + stack.register(listeners) LOG.debug("Dispatching engine %s for job: %s", engine, job) + consume = True try: engine.run() except excp.WrappedFailure as e: @@ -117,7 +126,7 @@ class BlockingConductor(base.Conductor): job, exc_info=True) else: LOG.info("Job completed successfully: %s", job) - return async_utils.make_completed_future(consume) + return async_utils.make_completed_future(consume) def run(self): self._dead.clear() diff --git a/taskflow/conductors/base.py b/taskflow/conductors/base.py index 33c4441d..7a6b8ce8 100644 --- a/taskflow/conductors/base.py +++ b/taskflow/conductors/base.py @@ -92,8 +92,6 @@ class Conductor(object): engine=self._engine, backend=self._persistence, **self._engine_options) - for listener in self._listeners_from_job(job, engine): - listener.register() return engine def _listeners_from_job(self, job, engine): diff --git a/taskflow/utils/misc.py b/taskflow/utils/misc.py index 57a28ea5..5b1cb209 100644 --- a/taskflow/utils/misc.py +++ b/taskflow/utils/misc.py @@ -436,6 +436,38 @@ def get_duplicate_keys(iterable, key=None): return duplicates +class ListenerStack(object): + """Listeners that are deregistered on context manager exit. + + TODO(harlowja): replace this with ``contextlib.ExitStack`` or equivalent + in the future (that code is in python3.2+ and in a few backports that + provide nearly equivalent functionality). When/if + https://review.openstack.org/#/c/164222/ merges we should be able to + remove this since listeners are already context managers. + """ + + def __init__(self, log): + self._registered = [] + self._log = log + + def register(self, listeners): + for listener in listeners: + listener.register() + self._registered.append(listener) + + def __enter__(self): + return self + + def __exit__(self, type, value, tb): + while self._registered: + listener = self._registered.pop() + try: + listener.deregister() + except Exception: + self._log.warn("Failed deregistering listener '%s'", + listener, exc_info=True) + + class ExponentialBackoff(object): """An iterable object that will yield back an exponential delay sequence. From d1742b1888f4f9f0cadbcbd6443db4a749c02a7d Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Fri, 13 Mar 2015 20:53:17 -0700 Subject: [PATCH 076/246] Update engine docs with new validation stage Since the validation stage is split off from the preparation stage we need to adjust the docs to reflect this. At the same time some other tweaks are made to the docs in this section to make them more clear and easy to understand. Change-Id: Ia09bf3fde997eaaf41aef5557b48f3cb0dc30642 --- doc/source/engines.rst | 55 ++++++++++++++++++++++++------------------ 1 file changed, 31 insertions(+), 24 deletions(-) diff --git a/doc/source/engines.rst b/doc/source/engines.rst index 3ce19030..5ce22903 100644 --- a/doc/source/engines.rst +++ b/doc/source/engines.rst @@ -221,39 +221,46 @@ are setup. Compiling --------- -During this stage the flow will be converted into an internal graph -representation using a -:py:class:`~taskflow.engines.action_engine.compiler.Compiler` (the default -implementation for patterns is the +During this stage (see :py:func:`~taskflow.engines.base.Engine.compile`) the +flow will be converted into an internal graph representation using a +compiler (the default implementation for patterns is the :py:class:`~taskflow.engines.action_engine.compiler.PatternCompiler`). This class compiles/converts the flow objects and contained atoms into a -`networkx`_ directed graph that contains the equivalent atoms defined in the -flow and any nested flows & atoms as well as the constraints that are created -by the application of the different flow patterns. This graph is then what will -be analyzed & traversed during the engines execution. At this point a few -helper object are also created and saved to internal engine variables (these -object help in execution of atoms, analyzing the graph and performing other -internal engine activities). At the finishing of this stage a +`networkx`_ directed graph (and tree structure) that contains the equivalent +atoms defined in the flow and any nested flows & atoms as well as the +constraints that are created by the application of the different flow +patterns. This graph (and tree) are what will be analyzed & traversed during +the engines execution. At this point a few helper object are also created and +saved to internal engine variables (these object help in execution of +atoms, analyzing the graph and performing other internal engine +activities). At the finishing of this stage a :py:class:`~taskflow.engines.action_engine.runtime.Runtime` object is created which contains references to all needed runtime components. Preparation ----------- -This stage starts by setting up the storage needed for all atoms in the -previously created graph, ensuring that corresponding -:py:class:`~taskflow.persistence.logbook.AtomDetail` (or subclass of) objects -are created for each node in the graph. Once this is done final validation -occurs on the requirements that are needed to start execution and what -:py:class:`~taskflow.storage.Storage` provides. If there is any atom or flow -requirements not satisfied then execution will not be allowed to continue. +This stage (see :py:func:`~taskflow.engines.base.Engine.prepare`) starts by +setting up the storage needed for all atoms in the compiled graph, ensuring +that corresponding :py:class:`~taskflow.persistence.logbook.AtomDetail` (or +subclass of) objects are created for each node in the graph. + +Validation +---------- + +This stage (see :py:func:`~taskflow.engines.base.Engine.validate`) performs +any final validation of the compiled (and now storage prepared) engine. It +compares the requirements that are needed to start execution and +what is currently provided or will be produced in the future. If there are +*any* atom requirements that are not satisfied (no known current provider or +future producer is found) then execution will **not** be allowed to continue. Execution --------- The graph (and helper objects) previously created are now used for guiding -further execution. The flow is put into the ``RUNNING`` :doc:`state ` -and a +further execution (see :py:func:`~taskflow.engines.base.Engine.run`). The +flow is put into the ``RUNNING`` :doc:`state ` and a :py:class:`~taskflow.engines.action_engine.runner.Runner` implementation object starts to take over and begins going through the stages listed below (for a more visual diagram/representation see @@ -262,10 +269,10 @@ the :ref:`engine state diagram `). .. note:: The engine will respect the constraints imposed by the flow. For example, - if Engine is executing a :py:class:`.linear_flow.Flow` then it is - constrained by the dependency-graph which is linear in this case, and hence - using a Parallel Engine may not yield any benefits if one is looking for - concurrency. + if an engine is executing a :py:class:`~taskflow.patterns.linear_flow.Flow` + then it is constrained by the dependency graph which is linear in this + case, and hence using a parallel engine may not yield any benefits if one + is looking for concurrency. Resumption ^^^^^^^^^^ From ac2b1be981e348677e27237170eaf79e6eaac037 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Fri, 13 Mar 2015 21:11:27 -0700 Subject: [PATCH 077/246] Use compatible map and update map/reduce task docs Instead of using the map() function (which depending on the python version may return a list or an iterator prefer to use the six.moves provided one and convert that one to a list; this avoids creating extra lists on versions of python where map() itself returns a list). This also adjusts some of the docstring to match the style and format of other docstrings. Change-Id: I29212016da95da6ca2bc6b3f103d03f7fcabf032 --- taskflow/task.py | 29 +++++++++++++++-------------- 1 file changed, 15 insertions(+), 14 deletions(-) diff --git a/taskflow/task.py b/taskflow/task.py index f98508c4..935fb740 100644 --- a/taskflow/task.py +++ b/taskflow/task.py @@ -21,6 +21,7 @@ import copy from oslo_utils import reflection import six +from six.moves import map as compat_map from six.moves import reduce as compat_reduce from taskflow import atom @@ -239,13 +240,13 @@ class FunctorTask(BaseTask): class ReduceFunctorTask(BaseTask): - """General purpose Task to reduce a list by applying a function + """General purpose Task to reduce a list by applying a function. - This Task mimics the behavior of Python's built-in reduce function. The - Task takes a functor (lambda or otherwise) and a list. The list is - specified using the requires argument of the Task. When executed, this - task calls reduce with the functor and list as arguments. The resulting - value from the call to reduce is then returned after execution. + This Task mimics the behavior of Python's built-in ``reduce`` function. The + Task takes a functor (lambda or otherwise) and a list. The list is + specified using the ``requires`` argument of the Task. When executed, this + task calls ``reduce`` with the functor and list as arguments. The resulting + value from the call to ``reduce`` is then returned after execution. """ def __init__(self, functor, requires, name=None, provides=None, auto_extract=True, rebind=None, inject=None): @@ -282,16 +283,16 @@ class ReduceFunctorTask(BaseTask): class MapFunctorTask(BaseTask): - """General purpose Task to map a function to a list + """General purpose Task to map a function to a list. - This Task mimics the behavior of Python's built-in map function. The Task - takes a functor (lambda or otherwise) and a list. The list is specified - using the requires argument of the Task. When executed, this task calls - map with the functor and list as arguments. The resulting list from the - call to map is then returned after execution. + This Task mimics the behavior of Python's built-in ``map`` function. The + Task takes a functor (lambda or otherwise) and a list. The list is + specified using the ``requires`` argument of the Task. When executed, this + task calls ``map`` with the functor and list as arguments. The resulting + list from the call to ``map`` is then returned after execution. Each value of the returned list can be bound to individual names using - the provides argument, following taskflow standard behavior. Order is + the ``provides`` argument, following taskflow standard behavior. Order is preserved in the returned list. """ @@ -322,4 +323,4 @@ class MapFunctorTask(BaseTask): def execute(self, *args, **kwargs): l = [kwargs[r] for r in self.requires] - return list(map(self._functor, l)) + return list(compat_map(self._functor, l)) From 5d2fb53a34d4400a0599dd54f07482422d06f854 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Sun, 15 Mar 2015 09:35:30 -0700 Subject: [PATCH 078/246] Attempt to extract traceback from exception In py3.x and newer a exception has an attached traceback so in the `from_exception` class method we should try to extract that traceback if we are able to. Change-Id: Ia1c5cfddc259bd9de7cff1b672354670c2f4f839 --- taskflow/types/failure.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/taskflow/types/failure.py b/taskflow/types/failure.py index 406dd4f8..d713098d 100644 --- a/taskflow/types/failure.py +++ b/taskflow/types/failure.py @@ -209,7 +209,12 @@ class Failure(object): @classmethod def from_exception(cls, exception): """Creates a failure object from a exception instance.""" - return cls((type(exception), exception, None)) + exc_info = ( + type(exception), + exception, + getattr(exception, '__traceback__', None) + ) + return cls(exc_info=exc_info) @classmethod def validate(cls, data): From 7c0a7620cc0ff0b987df63ef4a3182a8ec68753d Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Sun, 15 Mar 2015 10:04:30 -0700 Subject: [PATCH 079/246] Ensure we have a 'coverage-package-name' Change-Id: I0963d1e1a02e03ef20edc7cdb1373f4f77b6b1a9 --- tox.ini | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tox.ini b/tox.ini index 44571ff8..faa0a753 100644 --- a/tox.ini +++ b/tox.ini @@ -41,7 +41,7 @@ commands = pylint --rcfile=pylintrc taskflow basepython = python2.7 deps = {[testenv:py27]deps} coverage>=3.6 -commands = python setup.py testr --coverage --testr-args='{posargs}' +commands = python setup.py testr --coverage --coverage-package-name=taskflow --testr-args='{posargs}' [testenv:venv] basepython = python2.7 From ebe3d6de186bb721a98dcbef3e5aead787a93fb4 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Sun, 15 Mar 2015 18:39:24 -0700 Subject: [PATCH 080/246] Just use the class name instead of TYPE constant The TYPE constant is more of an internal protocol detail and is not the best constant to use in exceptions that tell the received data is invalid, instead just prefer and use the class name instead which is more relevant than the TYPE constant. Change-Id: I60652a149eb1a3c3d8523544c355044bbd6fec94 --- taskflow/engines/worker_based/protocol.py | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/taskflow/engines/worker_based/protocol.py b/taskflow/engines/worker_based/protocol.py index 34dcd506..934f837f 100644 --- a/taskflow/engines/worker_based/protocol.py +++ b/taskflow/engines/worker_based/protocol.py @@ -162,14 +162,15 @@ class Notify(Message): try: su.schema_validate(data, schema) except su.ValidationError as e: + cls_name = reflection.get_class_name(cls, fully_qualified=False) if response: raise excp.InvalidFormat("%s message response data not of the" " expected format: %s" - % (cls.TYPE, e.message), e) + % (cls_name, e.message), e) else: raise excp.InvalidFormat("%s message sender data not of the" " expected format: %s" - % (cls.TYPE, e.message), e) + % (cls_name, e.message), e) _WorkUnit = collections.namedtuple('_WorkUnit', ['task_cls', 'task_name', @@ -358,9 +359,10 @@ class Request(Message): try: su.schema_validate(data, cls.SCHEMA) except su.ValidationError as e: + cls_name = reflection.get_class_name(cls, fully_qualified=False) raise excp.InvalidFormat("%s message response data not of the" " expected format: %s" - % (cls.TYPE, e.message), e) + % (cls_name, e.message), e) else: # Validate all failure dictionaries that *may* be present... failures = [] @@ -501,9 +503,10 @@ class Response(Message): try: su.schema_validate(data, cls.SCHEMA) except su.ValidationError as e: + cls_name = reflection.get_class_name(cls, fully_qualified=False) raise excp.InvalidFormat("%s message response data not of the" " expected format: %s" - % (cls.TYPE, e.message), e) + % (cls_name, e.message), e) else: state = data['state'] if state == FAILURE and 'result' in data: From 6671ce47083f445df7e7e43765f09e4ff264cc81 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Tue, 17 Mar 2015 10:04:03 -0700 Subject: [PATCH 081/246] Use the class name instead of the TYPE property in __str__ Change-Id: Ie6fc31b0c89b9eaf9fabc1feb17d697edc5ff522 --- taskflow/engines/worker_based/protocol.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/taskflow/engines/worker_based/protocol.py b/taskflow/engines/worker_based/protocol.py index 934f837f..1ecfcef2 100644 --- a/taskflow/engines/worker_based/protocol.py +++ b/taskflow/engines/worker_based/protocol.py @@ -106,7 +106,8 @@ class Message(object): """Base class for all message types.""" def __str__(self): - return "<%s> %s" % (self.TYPE, self.to_dict()) + cls_name = reflection.get_class_name(self, fully_qualified=False) + return "<%s> %s" % (cls_name, self.to_dict()) @abc.abstractmethod def to_dict(self): From 374d91d34bf8ec59ab627ff87d79183a6069d18c Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Tue, 17 Mar 2015 16:26:38 -0700 Subject: [PATCH 082/246] Give the GC a break Instead of retaining cycles which the GC will have a harder time breaking, give it a break and use the knowledge that the associated objects will always exist while the runtime object does to just use a weakref proxy so that the GC will have a easier time breaking this cycle. Change-Id: I6241b2f33354fa58565835a5f08e5766aa601704 --- taskflow/engines/action_engine/analyzer.py | 6 +++--- taskflow/engines/action_engine/completer.py | 4 +++- taskflow/engines/action_engine/runtime.py | 2 +- taskflow/engines/action_engine/scheduler.py | 4 +++- 4 files changed, 10 insertions(+), 6 deletions(-) diff --git a/taskflow/engines/action_engine/analyzer.py b/taskflow/engines/action_engine/analyzer.py index ddae1e97..9044fd13 100644 --- a/taskflow/engines/action_engine/analyzer.py +++ b/taskflow/engines/action_engine/analyzer.py @@ -31,9 +31,9 @@ class Analyzer(object): the rest of the runtime system. """ - def __init__(self, compilation, storage): - self._storage = storage - self._execution_graph = compilation.execution_graph + def __init__(self, runtime): + self._storage = runtime.storage + self._execution_graph = runtime.compilation.execution_graph def get_next_nodes(self, node=None): if node is None: diff --git a/taskflow/engines/action_engine/completer.py b/taskflow/engines/action_engine/completer.py index 77ba6859..90ed4458 100644 --- a/taskflow/engines/action_engine/completer.py +++ b/taskflow/engines/action_engine/completer.py @@ -14,6 +14,8 @@ # License for the specific language governing permissions and limitations # under the License. +import weakref + from taskflow.engines.action_engine import executor as ex from taskflow import retry as retry_atom from taskflow import states as st @@ -25,7 +27,7 @@ class Completer(object): """Completes atoms using actions to complete them.""" def __init__(self, runtime): - self._runtime = runtime + self._runtime = weakref.proxy(runtime) self._analyzer = runtime.analyzer self._retry_action = runtime.retry_action self._storage = runtime.storage diff --git a/taskflow/engines/action_engine/runtime.py b/taskflow/engines/action_engine/runtime.py index d8df4705..395ce750 100644 --- a/taskflow/engines/action_engine/runtime.py +++ b/taskflow/engines/action_engine/runtime.py @@ -50,7 +50,7 @@ class Runtime(object): @misc.cachedproperty def analyzer(self): - return an.Analyzer(self._compilation, self._storage) + return an.Analyzer(self) @misc.cachedproperty def runner(self): diff --git a/taskflow/engines/action_engine/scheduler.py b/taskflow/engines/action_engine/scheduler.py index 8e3c64b3..20221830 100644 --- a/taskflow/engines/action_engine/scheduler.py +++ b/taskflow/engines/action_engine/scheduler.py @@ -14,6 +14,8 @@ # License for the specific language governing permissions and limitations # under the License. +import weakref + from taskflow import exceptions as excp from taskflow import retry as retry_atom from taskflow import states as st @@ -23,7 +25,7 @@ from taskflow.types import failure class _RetryScheduler(object): def __init__(self, runtime): - self._runtime = runtime + self._runtime = weakref.proxy(runtime) self._retry_action = runtime.retry_action self._storage = runtime.storage From 35794109e982334a13a792d9e7ef653ce1c23954 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Tue, 17 Mar 2015 17:21:04 -0700 Subject: [PATCH 083/246] Always return scope walker instances from `fetch_scopes_for` Instead of returning tuples with fully expanded scopes return walker instances that internally know how to avoid recomputing the visible scopes (they do this by caching each visibility level and looking in the local cache before computing the scope and storing it in the cache). This makes the usage more uniform and avoids returning different items depending on what is found; making the code easier to follow and understand. Also makes the scope walker call to '_extract_atoms' to go via a static method so that if it is ever desired to alter what '_extract_atoms' does it can be more easily done (using standard inheritance). Change-Id: I5916838163e6be843429fe7b89a0b5622e9c2f36 --- taskflow/engines/action_engine/runtime.py | 12 +++--- taskflow/engines/action_engine/scopes.py | 46 ++++++++++++++--------- 2 files changed, 35 insertions(+), 23 deletions(-) diff --git a/taskflow/engines/action_engine/runtime.py b/taskflow/engines/action_engine/runtime.py index d8df4705..b1bd288b 100644 --- a/taskflow/engines/action_engine/runtime.py +++ b/taskflow/engines/action_engine/runtime.py @@ -38,7 +38,7 @@ class Runtime(object): self._task_executor = task_executor self._storage = storage self._compilation = compilation - self._scopes = {} + self._walkers_to_names = {} @property def compilation(self): @@ -76,9 +76,9 @@ class Runtime(object): self._task_executor) def fetch_scopes_for(self, atom_name): - """Fetches a tuple of the visible scopes for the given atom.""" + """Fetches a walker of the visible scopes for the given atom.""" try: - return self._scopes[atom_name] + return self._walkers_to_names[atom_name] except KeyError: atom = None for node in self.analyzer.iterate_all_nodes(): @@ -88,10 +88,10 @@ class Runtime(object): if atom is not None: walker = sc.ScopeWalker(self.compilation, atom, names_only=True) - self._scopes[atom_name] = visible_to = tuple(walker) + self._walkers_to_names[atom_name] = walker else: - visible_to = tuple([]) - return visible_to + walker = None + return walker # Various helper methods used by the runtime components; not for public # consumption... diff --git a/taskflow/engines/action_engine/scopes.py b/taskflow/engines/action_engine/scopes.py index c55305d0..99e1578b 100644 --- a/taskflow/engines/action_engine/scopes.py +++ b/taskflow/engines/action_engine/scopes.py @@ -56,9 +56,14 @@ class ScopeWalker(object): if self._node is None: raise ValueError("Unable to find atom '%s' in compilation" " hierarchy" % atom) + self._level_cache = {} self._atom = atom self._graph = compilation.execution_graph self._names_only = names_only + self._predecessors = None + + #: Function that extracts the *associated* atoms of a given tree node. + _extract_atoms = staticmethod(_extract_atoms) def __iter__(self): """Iterates over the visible scopes. @@ -95,27 +100,34 @@ class ScopeWalker(object): nodes (aka we have reached the top of the tree) or we run out of predecessors. """ - predecessors = set(self._graph.bfs_predecessors_iter(self._atom)) + if self._predecessors is None: + pred_iter = self._graph.bfs_predecessors_iter(self._atom) + self._predecessors = set(pred_iter) + predecessors = self._predecessors.copy() last = self._node - for parent in self._node.path_iter(include_self=False): + for lvl, parent in enumerate(self._node.path_iter(include_self=False)): if not predecessors: break last_idx = parent.index(last.item) - visible = [] - for a in _extract_atoms(parent, idx=last_idx): - if a in predecessors: - predecessors.remove(a) - if not self._names_only: + try: + visible, removals = self._level_cache[lvl] + predecessors = predecessors - removals + except KeyError: + visible = [] + removals = set() + for a in self._extract_atoms(parent, idx=last_idx): + if a in predecessors: + predecessors.remove(a) + removals.add(a) visible.append(a) - else: - visible.append(a.name) - if LOG.isEnabledFor(logging.BLATHER): - if not self._names_only: + self._level_cache[lvl] = (visible, removals) + if LOG.isEnabledFor(logging.BLATHER): visible_names = [a.name for a in visible] - else: - visible_names = visible - LOG.blather("Scope visible to '%s' (limited by parent '%s'" - " index < %s) is: %s", self._atom, - parent.item.name, last_idx, visible_names) - yield visible + LOG.blather("Scope visible to '%s' (limited by parent '%s'" + " index < %s) is: %s", self._atom, + parent.item.name, last_idx, visible_names) + if self._names_only: + yield [a.name for a in visible] + else: + yield visible last = parent From 22fd32fb55d45b5f355a862b2edd78fde3c2086f Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Fri, 13 Mar 2015 14:29:34 -0700 Subject: [PATCH 084/246] Just let the future executors handle the max workers Instead of providing and retaining a thread count in the worker and action engine executors and checking it and handling the none case, we can just let the future types handle this already (which they already do). And when displaying this information in the worker banner use a new future executor attribute that is the maximum number of workers that will be ever created. Change-Id: I765c22936b53cdbb8a90195a764d4c67bcc3f34b --- taskflow/engines/action_engine/executor.py | 7 ++----- taskflow/engines/worker_based/worker.py | 13 +++++-------- taskflow/tests/unit/worker_based/test_worker.py | 14 ++++---------- taskflow/types/futures.py | 10 ++++++++++ 4 files changed, 21 insertions(+), 23 deletions(-) diff --git a/taskflow/engines/action_engine/executor.py b/taskflow/engines/action_engine/executor.py index b271beb8..bd222a55 100644 --- a/taskflow/engines/action_engine/executor.py +++ b/taskflow/engines/action_engine/executor.py @@ -417,11 +417,8 @@ class ParallelTaskExecutor(TaskExecutor): def start(self): if self._own_executor: - if self._max_workers is not None: - max_workers = self._max_workers - else: - max_workers = threading_utils.get_optimal_thread_count() - self._executor = self._create_executor(max_workers=max_workers) + self._executor = self._create_executor( + max_workers=self._max_workers) def stop(self): if self._own_executor: diff --git a/taskflow/engines/worker_based/worker.py b/taskflow/engines/worker_based/worker.py index 5e9ff85e..8a79133f 100644 --- a/taskflow/engines/worker_based/worker.py +++ b/taskflow/engines/worker_based/worker.py @@ -98,13 +98,9 @@ System details: self._topic = topic self._executor = executor self._owns_executor = False - self._threads_count = -1 if self._executor is None: - if threads_count is not None: - self._threads_count = int(threads_count) - else: - self._threads_count = tu.get_optimal_thread_count() - self._executor = futures.ThreadPoolExecutor(self._threads_count) + self._executor = futures.ThreadPoolExecutor( + max_workers=threads_count) self._owns_executor = True self._endpoints = self._derive_endpoints(tasks) self._exchange = exchange @@ -139,8 +135,9 @@ System details: tpl_params['transport_type'] = transport.driver_type tpl_params['connection_uri'] = connection_details.uri tpl_params['executor_type'] = reflection.get_class_name(self._executor) - if self._threads_count != -1: - tpl_params['executor_thread_count'] = self._threads_count + threads_count = getattr(self._executor, 'max_workers', None) + if threads_count is not None: + tpl_params['executor_thread_count'] = threads_count if self._endpoints: pretty_endpoints = [] for ep in self._endpoints: diff --git a/taskflow/tests/unit/worker_based/test_worker.py b/taskflow/tests/unit/worker_based/test_worker.py index 07095efa..a475c51d 100644 --- a/taskflow/tests/unit/worker_based/test_worker.py +++ b/taskflow/tests/unit/worker_based/test_worker.py @@ -33,7 +33,6 @@ class TestWorker(test.MockTestCase): self.broker_url = 'test-url' self.exchange = 'test-exchange' self.topic = 'test-topic' - self.threads_count = 5 self.endpoint_count = 24 # patch classes @@ -42,11 +41,6 @@ class TestWorker(test.MockTestCase): self.server_mock, self.server_inst_mock = self.patchClass( worker.server, 'Server') - # other mocking - self.threads_count_mock = self.patch( - 'taskflow.engines.worker_based.worker.tu.get_optimal_thread_count') - self.threads_count_mock.return_value = self.threads_count - def worker(self, reset_master_mock=False, **kwargs): worker_kwargs = dict(exchange=self.exchange, topic=self.topic, @@ -62,7 +56,7 @@ class TestWorker(test.MockTestCase): self.worker() master_mock_calls = [ - mock.call.executor_class(self.threads_count), + mock.call.executor_class(max_workers=None), mock.call.Server(self.topic, self.exchange, self.executor_inst_mock, [], url=self.broker_url, @@ -70,7 +64,7 @@ class TestWorker(test.MockTestCase): transport=mock.ANY, retry_options=mock.ANY) ] - self.assertEqual(self.master_mock.mock_calls, master_mock_calls) + self.assertEqual(master_mock_calls, self.master_mock.mock_calls) def test_banner_writing(self): buf = six.StringIO() @@ -84,7 +78,7 @@ class TestWorker(test.MockTestCase): self.worker(threads_count=10) master_mock_calls = [ - mock.call.executor_class(10), + mock.call.executor_class(max_workers=10), mock.call.Server(self.topic, self.exchange, self.executor_inst_mock, [], url=self.broker_url, @@ -92,7 +86,7 @@ class TestWorker(test.MockTestCase): transport=mock.ANY, retry_options=mock.ANY) ] - self.assertEqual(self.master_mock.mock_calls, master_mock_calls) + self.assertEqual(master_mock_calls, self.master_mock.mock_calls) def test_creation_with_custom_executor(self): executor_mock = mock.MagicMock(name='executor') diff --git a/taskflow/types/futures.py b/taskflow/types/futures.py index 2a6f7b6f..cfa97214 100644 --- a/taskflow/types/futures.py +++ b/taskflow/types/futures.py @@ -114,6 +114,11 @@ class ThreadPoolExecutor(_thread.ThreadPoolExecutor): # really want to use anyway). super(ThreadPoolExecutor, self).submit) + @property + def max_workers(self): + """The max number of workers that this executor will ever have.""" + return self._max_workers + @property def statistics(self): """:class:`.ExecutorStatistics` about the executors executions.""" @@ -153,6 +158,11 @@ class ProcessPoolExecutor(_process.ProcessPoolExecutor): """Accessor to determine if the executor is alive/active.""" return not self._shutdown_thread + @property + def max_workers(self): + """The max number of workers that this executor will ever have.""" + return self._max_workers + @property def statistics(self): """:class:`.ExecutorStatistics` about the executors executions.""" From 6db6363959a79c5e4c500b43e9c14197fa737589 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 18 Mar 2015 17:22:50 -0700 Subject: [PATCH 085/246] Rename memory backend filesystem -> fake filesystem To ensure that no-one gets confused and believes this is a real filesystem class, rename it to be a name that clearly tells it is not a real filesystem. Change-Id: I47b4018ff50019662492e1865d9cac5e8d43d3bf --- taskflow/persistence/backends/impl_memory.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/taskflow/persistence/backends/impl_memory.py b/taskflow/persistence/backends/impl_memory.py index c1faae84..32525eca 100644 --- a/taskflow/persistence/backends/impl_memory.py +++ b/taskflow/persistence/backends/impl_memory.py @@ -25,7 +25,7 @@ from taskflow.types import tree from taskflow.utils import lock_utils -class Filesystem(object): +class FakeFilesystem(object): """An in-memory filesystem-like structure.""" #: Root path of the in-memory filesystem. @@ -156,7 +156,7 @@ class MemoryBackend(path_based.PathBasedBackend): super(MemoryBackend, self).__init__(conf) if self._path is None: self._path = os.sep - self.memory = Filesystem() + self.memory = FakeFilesystem() self.lock = lock_utils.ReaderWriterLock() def get_connection(self): From dd33ec71f2f57ce11762c81e8f5a609d01fa1710 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 18 Mar 2015 17:54:32 -0700 Subject: [PATCH 086/246] Use binary/encode decode helper routines in dir backend Instead of using smaller versions of those routines just call into the same helper routines that the rest of the code uses. This also allows that backend to take a configuration option that specifies the encoding that will be used for when translating unicode strings into binary (and vice versa); this defaults to utf-8 to retain the existing behavior. Change-Id: Icfcbb22310e163753d8b34af212efc166c82634b --- taskflow/persistence/backends/impl_dir.py | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/taskflow/persistence/backends/impl_dir.py b/taskflow/persistence/backends/impl_dir.py index 5ee244f7..b1b440c2 100644 --- a/taskflow/persistence/backends/impl_dir.py +++ b/taskflow/persistence/backends/impl_dir.py @@ -21,7 +21,6 @@ import os import shutil from oslo_serialization import jsonutils -import six from taskflow import exceptions as exc from taskflow.persistence import path_based @@ -57,6 +56,7 @@ class DirBackend(path_based.PathBasedBackend): def __init__(self, conf): super(DirBackend, self).__init__(conf) self.file_cache = {} + self.encoding = self._conf.get('encoding', 'utf-8') if not self._path: raise ValueError("Empty path is disallowed") self._path = os.path.abspath(self._path) @@ -77,13 +77,14 @@ class Connection(path_based.PathBasedConnection): cache_info = self.backend.file_cache.setdefault(filename, {}) if not cache_info or mtime > cache_info.get('mtime', 0): with open(filename, 'rb') as fp: - cache_info['data'] = fp.read().decode('utf-8') + cache_info['data'] = misc.binary_decode( + fp.read(), encoding=self.backend.encoding) cache_info['mtime'] = mtime return cache_info['data'] def _write_to(self, filename, contents): - if isinstance(contents, six.text_type): - contents = contents.encode('utf-8') + contents = misc.binary_encode(contents, + encoding=self.backend.encoding) with open(filename, 'wb') as fp: fp.write(contents) self.backend.file_cache.pop(filename, None) From da73113f0de7eaae10d3b089d77e6c723b59d7a0 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 18 Mar 2015 21:27:33 -0700 Subject: [PATCH 087/246] Use iteration instead of list(s) when extracting scopes Instead of creating a temporary list(s) during scope extraction just use a generator/iterator instead to avoid the need to create temporary and wasteful list(s) that are not really needed. Change-Id: Ia7f2c36c2cbd6602b90bd32911fc1ff5bcbb1611 --- taskflow/engines/action_engine/scopes.py | 23 ++++++++++++----------- 1 file changed, 12 insertions(+), 11 deletions(-) diff --git a/taskflow/engines/action_engine/scopes.py b/taskflow/engines/action_engine/scopes.py index 99e1578b..5fd7ee6f 100644 --- a/taskflow/engines/action_engine/scopes.py +++ b/taskflow/engines/action_engine/scopes.py @@ -21,24 +21,23 @@ from taskflow import logging LOG = logging.getLogger(__name__) -def _extract_atoms(node, idx=-1): +def _extract_atoms_iter(node, idx=-1): # Always go left to right, since right to left is the pattern order # and we want to go backwards and not forwards through that ordering... if idx == -1: children_iter = node.reverse_iter() else: children_iter = reversed(node[0:idx]) - atoms = [] for child in children_iter: if isinstance(child.item, flow_type.Flow): - atoms.extend(_extract_atoms(child)) + for atom in _extract_atoms_iter(child): + yield atom elif isinstance(child.item, atom_type.Atom): - atoms.append(child.item) + yield child.item else: raise TypeError( "Unknown extraction item '%s' (%s)" % (child.item, type(child.item))) - return atoms class ScopeWalker(object): @@ -63,7 +62,7 @@ class ScopeWalker(object): self._predecessors = None #: Function that extracts the *associated* atoms of a given tree node. - _extract_atoms = staticmethod(_extract_atoms) + _extract_atoms_iter = staticmethod(_extract_atoms_iter) def __iter__(self): """Iterates over the visible scopes. @@ -115,11 +114,13 @@ class ScopeWalker(object): except KeyError: visible = [] removals = set() - for a in self._extract_atoms(parent, idx=last_idx): - if a in predecessors: - predecessors.remove(a) - removals.add(a) - visible.append(a) + for atom in self._extract_atoms_iter(parent, idx=last_idx): + if atom in predecessors: + predecessors.remove(atom) + removals.add(atom) + visible.append(atom) + if not predecessors: + break self._level_cache[lvl] = (visible, removals) if LOG.isEnabledFor(logging.BLATHER): visible_names = [a.name for a in visible] From f7b3c23bec2435f7f4ca9838fc6377d3d5249116 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 18 Mar 2015 22:01:59 -0700 Subject: [PATCH 088/246] Expose the storage backend being used To match how the persistence connections provide access to there backend it seems appropriate to also have the storage layer provide the same property so that users of it can easily access the underlying backend object (if they so need to). Change-Id: I8fdb5e7efc66bba93da2949bb6adad534bf3b127 --- taskflow/storage.py | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/taskflow/storage.py b/taskflow/storage.py index a08706c3..bb9f0a21 100644 --- a/taskflow/storage.py +++ b/taskflow/storage.py @@ -271,6 +271,11 @@ class Storage(object): # This never changes (so no read locking needed). return self._flowdetail.uuid + @property + def backend(self): + # This never changes (so no read locking needed). + return self._backend + def _save_flow_detail(self, conn): # NOTE(harlowja): we need to update our contained flow detail if # the result of the update actually added more (aka another process From 765037dfc028b06b6e49f3547a514af8d27214c8 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Fri, 20 Mar 2015 08:58:20 -0700 Subject: [PATCH 089/246] Ensure docstring on storage properties To ensure that sphinx generates docs for these methods and shows them (without docstrings they do not show up) add some basic docstrings on the exposed storage classes properties. Change-Id: Iab2d2432403c7065fe8cb679e8e64a24ec17b53f --- taskflow/storage.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/taskflow/storage.py b/taskflow/storage.py index bb9f0a21..5386907e 100644 --- a/taskflow/storage.py +++ b/taskflow/storage.py @@ -263,16 +263,19 @@ class Storage(object): @property def flow_name(self): + """The flow detail name this storage unit is associated with.""" # This never changes (so no read locking needed). return self._flowdetail.name @property def flow_uuid(self): + """The flow detail uuid this storage unit is associated with.""" # This never changes (so no read locking needed). return self._flowdetail.uuid @property def backend(self): + """The backend this storage unit is associated with.""" # This never changes (so no read locking needed). return self._backend From 658d1f9882ecf355cba238e60028dc4fe0f64ec0 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Fri, 20 Mar 2015 10:45:35 -0700 Subject: [PATCH 090/246] Ensure we are really setup before being connected Change-Id: I42d9beeca159913a0eed5f9867c8ac0b191ffc3e --- taskflow/jobs/backends/impl_zookeeper.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/taskflow/jobs/backends/impl_zookeeper.py b/taskflow/jobs/backends/impl_zookeeper.py index 10f529eb..55c7b8eb 100644 --- a/taskflow/jobs/backends/impl_zookeeper.py +++ b/taskflow/jobs/backends/impl_zookeeper.py @@ -351,6 +351,7 @@ class ZookeeperJobBoard(base.NotifyingJobBoard): self._job_base = k_paths.join(path, JOB_PREFIX) self._worker = None self._emit_notifications = bool(emit_notifications) + self._connected = False def _emit(self, state, details): # Submit the work to the executor to avoid blocking the kazoo queue. @@ -715,7 +716,7 @@ class ZookeeperJobBoard(base.NotifyingJobBoard): @property def connected(self): - return self._client.connected + return self._connected and self._client.connected @lock_utils.locked(lock='_open_close_lock') def close(self): @@ -729,6 +730,7 @@ class ZookeeperJobBoard(base.NotifyingJobBoard): with self._job_cond: self._known_jobs.clear() LOG.debug("Stopped & cleared local state") + self._connected = False @lock_utils.locked(lock='_open_close_lock') def connect(self, timeout=10.0): @@ -763,6 +765,7 @@ class ZookeeperJobBoard(base.NotifyingJobBoard): self.path, func=self._on_job_posting, allow_session_lost=True) + self._connected = True except excp.IncompatibleVersion: with excutils.save_and_reraise_exception(): try_clean() From 579500d664a8fd85efe4c047f4c1e9397a842cb5 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Fri, 20 Mar 2015 16:09:07 -0700 Subject: [PATCH 091/246] Ensure thread-safety of persistence dir backend Even though this backend is protected by process locks against other processes we still need to protect against threads in the same process; so to do this grab a write lock and use it in the same manner that the memory backend does; as a per-backend write-lock. Change-Id: Ifb86631033aad1ca622be9cf3c3677293d8c56bd --- taskflow/persistence/backends/impl_dir.py | 7 +++++-- taskflow/persistence/backends/impl_memory.py | 2 +- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/taskflow/persistence/backends/impl_dir.py b/taskflow/persistence/backends/impl_dir.py index 5ee244f7..ca229b7a 100644 --- a/taskflow/persistence/backends/impl_dir.py +++ b/taskflow/persistence/backends/impl_dir.py @@ -60,6 +60,7 @@ class DirBackend(path_based.PathBasedBackend): if not self._path: raise ValueError("Empty path is disallowed") self._path = os.path.abspath(self._path) + self.lock = lock_utils.ReaderWriterLock() def get_connection(self): return Connection(self) @@ -131,8 +132,10 @@ class Connection(path_based.PathBasedConnection): @contextlib.contextmanager def _transaction(self): - """This backend doesn't support transactions""" - yield + """This just wraps a global write-lock.""" + lock = self.backend.lock.write_lock + with lock(): + yield def validate(self): with _storagefailure_wrapper(): diff --git a/taskflow/persistence/backends/impl_memory.py b/taskflow/persistence/backends/impl_memory.py index c1faae84..9c10332d 100644 --- a/taskflow/persistence/backends/impl_memory.py +++ b/taskflow/persistence/backends/impl_memory.py @@ -211,7 +211,7 @@ class Connection(path_based.PathBasedConnection): @contextlib.contextmanager def _transaction(self): - """This just wraps a global write-lock""" + """This just wraps a global write-lock.""" with self._memory_lock(write=True): yield From 92d6eba364d219a1419f6f5a3a2f25ff2a9456a1 Mon Sep 17 00:00:00 2001 From: OpenStack Proposal Bot Date: Sat, 21 Mar 2015 00:18:41 +0000 Subject: [PATCH 092/246] Updated from global requirements Change-Id: I974740d76ba4f2141a0c1ea43f3ac09122e280a9 --- requirements-py2.txt | 8 ++++---- requirements-py3.txt | 8 ++++---- test-requirements.txt | 4 ++-- 3 files changed, 10 insertions(+), 10 deletions(-) diff --git a/requirements-py2.txt b/requirements-py2.txt index 61a08161..23c0a2d0 100644 --- a/requirements-py2.txt +++ b/requirements-py2.txt @@ -20,7 +20,7 @@ enum34 networkx>=1.8 # Used for backend storage engine loading. -stevedore>=1.1.0 # Apache-2.0 +stevedore>=1.3.0,<1.4.0 # Apache-2.0 # Backport for concurrent.futures which exists in 3.2+ futures>=2.1.6 @@ -29,8 +29,8 @@ futures>=2.1.6 jsonschema>=2.0.0,<3.0.0 # For common utilities -oslo.utils>=1.2.0 # Apache-2.0 -oslo.serialization>=1.2.0 # Apache-2.0 +oslo.utils>=1.4.0,<1.5.0 # Apache-2.0 +oslo.serialization>=1.4.0,<1.5.0 # Apache-2.0 # For deprecation of things -debtcollector>=0.2 +debtcollector>=0.3.0,<0.4.0 # Apache-2.0 diff --git a/requirements-py3.txt b/requirements-py3.txt index 457462b6..d67ad1a9 100644 --- a/requirements-py3.txt +++ b/requirements-py3.txt @@ -17,14 +17,14 @@ enum34 networkx>=1.8 # Used for backend storage engine loading. -stevedore>=1.1.0 # Apache-2.0 +stevedore>=1.3.0,<1.4.0 # Apache-2.0 # Used for structured input validation jsonschema>=2.0.0,<3.0.0 # For common utilities -oslo.utils>=1.2.0 # Apache-2.0 -oslo.serialization>=1.2.0 # Apache-2.0 +oslo.utils>=1.4.0,<1.5.0 # Apache-2.0 +oslo.serialization>=1.4.0,<1.5.0 # Apache-2.0 # For deprecation of things -debtcollector>=0.2 +debtcollector>=0.3.0,<0.4.0 # Apache-2.0 diff --git a/test-requirements.txt b/test-requirements.txt index 8c9a83a6..23d9ac3a 100644 --- a/test-requirements.txt +++ b/test-requirements.txt @@ -3,7 +3,7 @@ # process, which may cause wedges in the gate later. hacking<0.11,>=0.10.0 -oslotest>=1.2.0 # Apache-2.0 +oslotest>=1.5.1,<1.6.0 # Apache-2.0 mock>=1.0 testtools>=0.9.36,!=1.2.0 testscenarios>=0.4 @@ -30,4 +30,4 @@ psycopg2 # Docs build jobs need these packages. sphinx>=1.1.2,!=1.2.0,!=1.3b1,<1.3 -oslosphinx>=2.2.0 # Apache-2.0 +oslosphinx>=2.5.0,<2.6.0 # Apache-2.0 From 6723a18427aeaa43540a1d47009782478d116c89 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Sat, 21 Mar 2015 00:18:41 +0000 Subject: [PATCH 093/246] Move to using the oslo.utils stop watch The code was moved to the oslo.utils package and we can now use it to avoid having our own type that does the same thing (less duplicated code and more sharing is good). Change-Id: I0545a978083ced75b2ba99280569ca2370756d33 --- taskflow/tests/unit/test_types.py | 123 -------- .../tests/unit/worker_based/test_protocol.py | 22 +- .../tests/unit/worker_based/test_types.py | 9 +- taskflow/types/timing.py | 264 +----------------- 4 files changed, 23 insertions(+), 395 deletions(-) diff --git a/taskflow/tests/unit/test_types.py b/taskflow/tests/unit/test_types.py index e1e07894..809c1926 100644 --- a/taskflow/tests/unit/test_types.py +++ b/taskflow/tests/unit/test_types.py @@ -28,7 +28,6 @@ from taskflow.types import latch from taskflow.types import periodic from taskflow.types import sets from taskflow.types import table -from taskflow.types import timing as tt from taskflow.types import tree from taskflow.utils import threading_utils as tu @@ -276,128 +275,6 @@ class TreeTest(test.TestCase): 'horse', 'human', 'monkey'], things) -class StopWatchTest(test.TestCase): - def setUp(self): - super(StopWatchTest, self).setUp() - tt.StopWatch.set_now_override(now=0) - self.addCleanup(tt.StopWatch.clear_overrides) - - def test_leftover_no_duration(self): - watch = tt.StopWatch() - watch.start() - self.assertRaises(RuntimeError, watch.leftover) - self.assertRaises(RuntimeError, watch.leftover, return_none=False) - self.assertIsNone(watch.leftover(return_none=True)) - - def test_no_states(self): - watch = tt.StopWatch() - self.assertRaises(RuntimeError, watch.stop) - self.assertRaises(RuntimeError, watch.resume) - - def test_bad_expiry(self): - self.assertRaises(ValueError, tt.StopWatch, -1) - - def test_backwards(self): - watch = tt.StopWatch(0.1) - watch.start() - tt.StopWatch.advance_time_seconds(0.5) - self.assertTrue(watch.expired()) - - tt.StopWatch.advance_time_seconds(-1.0) - self.assertFalse(watch.expired()) - self.assertEqual(0.0, watch.elapsed()) - - def test_expiry(self): - watch = tt.StopWatch(0.1) - watch.start() - tt.StopWatch.advance_time_seconds(0.2) - self.assertTrue(watch.expired()) - - def test_not_expired(self): - watch = tt.StopWatch(0.1) - watch.start() - tt.StopWatch.advance_time_seconds(0.05) - self.assertFalse(watch.expired()) - - def test_no_expiry(self): - watch = tt.StopWatch(0.1) - self.assertRaises(RuntimeError, watch.expired) - - def test_elapsed(self): - watch = tt.StopWatch() - watch.start() - tt.StopWatch.advance_time_seconds(0.2) - # NOTE(harlowja): Allow for a slight variation by using 0.19. - self.assertGreaterEqual(0.19, watch.elapsed()) - - def test_no_elapsed(self): - watch = tt.StopWatch() - self.assertRaises(RuntimeError, watch.elapsed) - - def test_no_leftover(self): - watch = tt.StopWatch() - self.assertRaises(RuntimeError, watch.leftover) - watch = tt.StopWatch(1) - self.assertRaises(RuntimeError, watch.leftover) - - def test_pause_resume(self): - watch = tt.StopWatch() - watch.start() - tt.StopWatch.advance_time_seconds(0.05) - watch.stop() - elapsed = watch.elapsed() - self.assertAlmostEqual(elapsed, watch.elapsed()) - watch.resume() - tt.StopWatch.advance_time_seconds(0.05) - self.assertNotEqual(elapsed, watch.elapsed()) - - def test_context_manager(self): - with tt.StopWatch() as watch: - tt.StopWatch.advance_time_seconds(0.05) - self.assertGreater(0.01, watch.elapsed()) - - def test_splits(self): - watch = tt.StopWatch() - watch.start() - self.assertEqual(0, len(watch.splits)) - - watch.split() - self.assertEqual(1, len(watch.splits)) - self.assertEqual(watch.splits[0].elapsed, - watch.splits[0].length) - - tt.StopWatch.advance_time_seconds(0.05) - watch.split() - splits = watch.splits - self.assertEqual(2, len(splits)) - self.assertNotEqual(splits[0].elapsed, splits[1].elapsed) - self.assertEqual(splits[1].length, - splits[1].elapsed - splits[0].elapsed) - - watch.stop() - self.assertEqual(2, len(watch.splits)) - - watch.start() - self.assertEqual(0, len(watch.splits)) - - def test_elapsed_maximum(self): - watch = tt.StopWatch() - watch.start() - - tt.StopWatch.advance_time_seconds(1) - self.assertEqual(1, watch.elapsed()) - - tt.StopWatch.advance_time_seconds(10) - self.assertEqual(11, watch.elapsed()) - self.assertEqual(1, watch.elapsed(maximum=1)) - - watch.stop() - self.assertEqual(11, watch.elapsed()) - tt.StopWatch.advance_time_seconds(10) - self.assertEqual(11, watch.elapsed()) - self.assertEqual(0, watch.elapsed(maximum=-1)) - - class TableTest(test.TestCase): def test_create_valid_no_rows(self): tbl = table.PleasantTable(['Name', 'City', 'State', 'Country']) diff --git a/taskflow/tests/unit/worker_based/test_protocol.py b/taskflow/tests/unit/worker_based/test_protocol.py index 5436df3c..73fe9c8c 100644 --- a/taskflow/tests/unit/worker_based/test_protocol.py +++ b/taskflow/tests/unit/worker_based/test_protocol.py @@ -21,9 +21,9 @@ from taskflow.engines.action_engine import executor from taskflow.engines.worker_based import protocol as pr from taskflow import exceptions as excp from taskflow import test +from taskflow.test import mock from taskflow.tests import utils from taskflow.types import failure -from taskflow.types import timing class TestProtocolValidation(test.TestCase): @@ -94,8 +94,6 @@ class TestProtocol(test.TestCase): def setUp(self): super(TestProtocol, self).setUp() - timing.StopWatch.set_now_override() - self.addCleanup(timing.StopWatch.clear_overrides) self.task = utils.DummyTask() self.task_uuid = 'task-uuid' self.task_action = 'execute' @@ -164,21 +162,27 @@ class TestProtocol(test.TestCase): failures={self.task.name: a_failure.to_dict()}) self.assertEqual(request.to_dict(), expected) - def test_pending_not_expired(self): + @mock.patch('oslo_utils.timeutils.now') + def test_pending_not_expired(self, now): + now.return_value = 0 req = self.request() - timing.StopWatch.set_offset_override(self.timeout - 1) + now.return_value = self.timeout - 1 self.assertFalse(req.expired) - def test_pending_expired(self): + @mock.patch('oslo_utils.timeutils.now') + def test_pending_expired(self, now): + now.return_value = 0 req = self.request() - timing.StopWatch.set_offset_override(self.timeout + 1) + now.return_value = self.timeout + 1 self.assertTrue(req.expired) - def test_running_not_expired(self): + @mock.patch('oslo_utils.timeutils.now') + def test_running_not_expired(self, now): + now.return_value = 0 request = self.request() request.transition(pr.PENDING) request.transition(pr.RUNNING) - timing.StopWatch.set_offset_override(self.timeout + 1) + now.return_value = self.timeout + 1 self.assertFalse(request.expired) def test_set_result(self): diff --git a/taskflow/tests/unit/worker_based/test_types.py b/taskflow/tests/unit/worker_based/test_types.py index 287283cf..095bf1e3 100644 --- a/taskflow/tests/unit/worker_based/test_types.py +++ b/taskflow/tests/unit/worker_based/test_types.py @@ -14,21 +14,19 @@ # License for the specific language governing permissions and limitations # under the License. -from oslo.utils import reflection +from oslo_utils import reflection from taskflow.engines.worker_based import protocol as pr from taskflow.engines.worker_based import types as worker_types from taskflow import test from taskflow.test import mock from taskflow.tests import utils -from taskflow.types import timing class TestRequestCache(test.TestCase): def setUp(self): super(TestRequestCache, self).setUp() - self.addCleanup(timing.StopWatch.clear_overrides) self.task = utils.DummyTask() self.task_uuid = 'task-uuid' self.task_action = 'execute' @@ -45,7 +43,8 @@ class TestRequestCache(test.TestCase): request_kwargs.update(kwargs) return pr.Request(**request_kwargs) - def test_requests_cache_expiry(self): + @mock.patch('oslo_utils.timeutils.now') + def test_requests_cache_expiry(self, now): # Mock out the calls the underlying objects will soon use to return # times that we can control more easily... overrides = [ @@ -53,7 +52,7 @@ class TestRequestCache(test.TestCase): 1, self.timeout + 1, ] - timing.StopWatch.set_now_override(overrides) + now.side_effect = overrides cache = worker_types.RequestsCache() cache[self.task_uuid] = self.request() diff --git a/taskflow/types/timing.py b/taskflow/types/timing.py index 0dad971f..791bce3e 100644 --- a/taskflow/types/timing.py +++ b/taskflow/types/timing.py @@ -14,14 +14,15 @@ # License for the specific language governing permissions and limitations # under the License. -from oslo_utils import reflection +from oslo_utils import timeutils -from taskflow.utils import misc from taskflow.utils import threading_utils -# Find a monotonic providing time (or fallback to using time.time() -# which isn't *always* accurate but will suffice). -_now = misc.find_monotonic(allow_time_time=True) +#: Moved to oslo.utils (just reference them from there until a later time). +Split = timeutils.Split + +#: Moved to oslo.utils (just reference them from there until a later time). +StopWatch = timeutils.StopWatch class Timeout(object): @@ -47,256 +48,3 @@ class Timeout(object): def reset(self): self._event.clear() - - -class Split(object): - """A *immutable* stopwatch split. - - See: http://en.wikipedia.org/wiki/Stopwatch for what this is/represents. - """ - - __slots__ = ['_elapsed', '_length'] - - def __init__(self, elapsed, length): - self._elapsed = elapsed - self._length = length - - @property - def elapsed(self): - """Duration from stopwatch start.""" - return self._elapsed - - @property - def length(self): - """Seconds from last split (or the elapsed time if no prior split).""" - return self._length - - def __repr__(self): - r = reflection.get_class_name(self, fully_qualified=False) - r += "(elapsed=%s, length=%s)" % (self._elapsed, self._length) - return r - - -class StopWatch(object): - """A simple timer/stopwatch helper class. - - Inspired by: apache-commons-lang java stopwatch. - - **Not** thread-safe (when a single watch is mutated by multiple threads at - the same time). It is thread-safe when used by a single thread (not - shared) or when operations are performed in a thread-safe manner on these - objects by wrapping those operations with locks. - """ - _STARTED = 'STARTED' - _STOPPED = 'STOPPED' - - """ - Class variables that should only be used for testing purposes only... - """ - _now_offset = None - _now_override = None - - def __init__(self, duration=None): - if duration is not None: - if duration < 0: - raise ValueError("Duration must be >= 0 and not %s" % duration) - self._duration = duration - else: - self._duration = None - self._started_at = None - self._stopped_at = None - self._state = None - self._splits = [] - - def start(self): - """Starts the watch (if not already started). - - NOTE(harlowja): resets any splits previously captured (if any). - """ - if self._state == self._STARTED: - return self - self._started_at = self._now() - self._stopped_at = None - self._state = self._STARTED - self._splits = [] - return self - - @property - def splits(self): - """Accessor to all/any splits that have been captured.""" - return tuple(self._splits) - - def split(self): - """Captures a split/elapsed since start time (and doesn't stop).""" - if self._state == self._STARTED: - elapsed = self.elapsed() - if self._splits: - length = self._delta_seconds(self._splits[-1].elapsed, elapsed) - else: - length = elapsed - self._splits.append(Split(elapsed, length)) - return self._splits[-1] - else: - raise RuntimeError("Can not create a split time of a stopwatch" - " if it has not been started") - - def restart(self): - """Restarts the watch from a started/stopped state.""" - if self._state == self._STARTED: - self.stop() - self.start() - return self - - @classmethod - def clear_overrides(cls): - """Clears all overrides/offsets. - - **Only to be used for testing (affects all watch instances).** - """ - cls._now_override = None - cls._now_offset = None - - @classmethod - def set_offset_override(cls, offset): - """Sets a offset that is applied to each time fetch. - - **Only to be used for testing (affects all watch instances).** - """ - cls._now_offset = offset - - @classmethod - def advance_time_seconds(cls, offset): - """Advances/sets a offset that is applied to each time fetch. - - NOTE(harlowja): if a previous offset exists (not ``None``) then this - offset will be added onto the existing one (if you want to reset - the offset completely use the :meth:`.set_offset_override` - method instead). - - **Only to be used for testing (affects all watch instances).** - """ - if cls._now_offset is None: - cls.set_offset_override(offset) - else: - cls.set_offset_override(cls._now_offset + offset) - - @classmethod - def set_now_override(cls, now=None): - """Sets time override to use (if none, then current time is fetched). - - NOTE(harlowja): if a list/tuple is provided then the first element of - the list will be used (and removed) each time a time fetch occurs (once - it becomes empty the override/s will no longer be applied). If a - numeric value is provided then it will be used (and never removed - until the override(s) are cleared via the :meth:`.clear_overrides` - method). - - **Only to be used for testing (affects all watch instances).** - """ - if isinstance(now, (list, tuple)): - cls._now_override = list(now) - else: - if now is None: - now = _now() - cls._now_override = now - - @staticmethod - def _delta_seconds(earlier, later): - return max(0.0, later - earlier) - - @classmethod - def _now(cls): - if cls._now_override is not None: - if isinstance(cls._now_override, list): - try: - now = cls._now_override.pop(0) - except IndexError: - now = _now() - else: - now = cls._now_override - else: - now = _now() - if cls._now_offset is not None: - now = now + cls._now_offset - return now - - def elapsed(self, maximum=None): - """Returns how many seconds have elapsed.""" - if self._state not in (self._STOPPED, self._STARTED): - raise RuntimeError("Can not get the elapsed time of a stopwatch" - " if it has not been started/stopped") - if self._state == self._STOPPED: - elapsed = self._delta_seconds(self._started_at, self._stopped_at) - else: - elapsed = self._delta_seconds(self._started_at, self._now()) - if maximum is not None and elapsed > maximum: - elapsed = max(0.0, maximum) - return elapsed - - def __enter__(self): - """Starts the watch.""" - self.start() - return self - - def __exit__(self, type, value, traceback): - """Stops the watch (ignoring errors if stop fails).""" - try: - self.stop() - except RuntimeError: - pass - - def leftover(self, return_none=False): - """Returns how many seconds are left until the watch expires. - - :param return_none: when ``True`` instead of raising a ``RuntimeError`` - when no duration has been set this call will - return ``None`` instead - :type return_none: boolean - :returns: how many seconds left until the watch expires - :rtype: number - """ - if self._state != self._STARTED: - raise RuntimeError("Can not get the leftover time of a stopwatch" - " that has not been started") - if self._duration is None: - if not return_none: - raise RuntimeError("Can not get the leftover time of a watch" - " that has no duration") - else: - return None - return max(0.0, self._duration - self.elapsed()) - - def expired(self): - """Returns if the watch has expired (ie, duration provided elapsed). - - :returns: if the watch has expired - :rtype: boolean - """ - if self._state is None: - raise RuntimeError("Can not check if a stopwatch has expired" - " if it has not been started/stopped") - if self._duration is None: - return False - if self.elapsed() > self._duration: - return True - return False - - def resume(self): - """Resumes the watch from a stopped state.""" - if self._state == self._STOPPED: - self._state = self._STARTED - return self - else: - raise RuntimeError("Can not resume a stopwatch that has not been" - " stopped") - - def stop(self): - """Stops the watch.""" - if self._state == self._STOPPED: - return self - if self._state != self._STARTED: - raise RuntimeError("Can not stop a stopwatch that has not been" - " started") - self._stopped_at = self._now() - self._state = self._STOPPED - return self From 7d79e5236653d159f7a8dd7423851b34373a7d1d Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Fri, 20 Mar 2015 20:55:43 -0700 Subject: [PATCH 094/246] Move to the newer debtcollector provided functions Use the debtcollector library provided variants of these functions and remove our versions to avoid more duplications. Change-Id: I526789f44899afd284803b127bec5bb349d1aa2b --- taskflow/conductors/backends/impl_blocking.py | 5 +-- taskflow/conductors/single_threaded.py | 9 +++-- taskflow/types/periodic.py | 4 +- taskflow/utils/deprecation.py | 38 ------------------- 4 files changed, 9 insertions(+), 47 deletions(-) diff --git a/taskflow/conductors/backends/impl_blocking.py b/taskflow/conductors/backends/impl_blocking.py index b53452af..5156da10 100644 --- a/taskflow/conductors/backends/impl_blocking.py +++ b/taskflow/conductors/backends/impl_blocking.py @@ -12,6 +12,7 @@ # License for the specific language governing permissions and limitations # under the License. +from debtcollector import removals import six from taskflow.conductors import base @@ -20,7 +21,6 @@ from taskflow.listeners import logging as logging_listener from taskflow import logging from taskflow.types import timing as tt from taskflow.utils import async_utils -from taskflow.utils import deprecation from taskflow.utils import misc from taskflow.utils import threading_utils @@ -67,8 +67,7 @@ class BlockingConductor(base.Conductor): raise ValueError("Invalid timeout literal: %s" % (wait_timeout)) self._dead = threading_utils.Event() - @deprecation.removed_kwarg('timeout', - version="0.8", removal_version="?") + @removals.removed_kwarg('timeout', version="0.8", removal_version="?") def stop(self, timeout=None): """Requests the conductor to stop dispatching. diff --git a/taskflow/conductors/single_threaded.py b/taskflow/conductors/single_threaded.py index c8ab5a03..e1fafcce 100644 --- a/taskflow/conductors/single_threaded.py +++ b/taskflow/conductors/single_threaded.py @@ -15,14 +15,15 @@ # under the License. from debtcollector import moves +from debtcollector import removals from taskflow.conductors.backends import impl_blocking -from taskflow.utils import deprecation # TODO(harlowja): remove this module soon... -deprecation.removed_module(__name__, - replacement_name="the conductor entrypoints", - version="0.8", removal_version="?") +removals.removed_module(__name__, + replacement="the conductor entrypoints", + version="0.8", removal_version="?", + stacklevel=4) # TODO(harlowja): remove this proxy/legacy class soon... SingleThreadedConductor = moves.moved_class( diff --git a/taskflow/types/periodic.py b/taskflow/types/periodic.py index 8be3e10d..7314988c 100644 --- a/taskflow/types/periodic.py +++ b/taskflow/types/periodic.py @@ -17,11 +17,11 @@ import heapq import inspect +from debtcollector import removals from oslo_utils import reflection import six from taskflow import logging -from taskflow.utils import deprecation from taskflow.utils import misc from taskflow.utils import threading_utils as tu @@ -143,7 +143,7 @@ class PeriodicWorker(object): callables.append(member) return cls(callables) - @deprecation.removed_kwarg('tombstone', version="0.8", removal_version="?") + @removals.removed_kwarg('tombstone', version="0.8", removal_version="?") def __init__(self, callables, tombstone=None): if tombstone is None: self._tombstone = tu.Event() diff --git a/taskflow/utils/deprecation.py b/taskflow/utils/deprecation.py index 8bb48b27..9faf80cb 100644 --- a/taskflow/utils/deprecation.py +++ b/taskflow/utils/deprecation.py @@ -17,7 +17,6 @@ import warnings from oslo_utils import reflection -import six def deprecation(message, stacklevel=None): @@ -131,43 +130,6 @@ def _generate_message(prefix, postfix=None, message=None, return ''.join(message_components) -def removed_kwarg(old_name, message=None, - version=None, removal_version=None, stacklevel=3): - """Decorates a kwarg accepting function to deprecate a removed kwarg.""" - - prefix = "Using the '%s' argument is deprecated" % old_name - out_message = _generate_message(prefix, postfix=None, - message=message, version=version, - removal_version=removal_version) - - def decorator(f): - - @six.wraps(f) - def wrapper(*args, **kwargs): - if old_name in kwargs: - deprecation(out_message, stacklevel=stacklevel) - return f(*args, **kwargs) - - return wrapper - - return decorator - - -def removed_module(module_name, replacement_name=None, message=None, - version=None, removal_version=None, stacklevel=4): - """Deprecates a module that will be removed/replaced in the future.""" - prefix = "The '%s' module usage is deprecated" % module_name - if replacement_name: - postfix = ", please use %s instead" % replacement_name - else: - postfix = None - out_message = _generate_message(prefix, - postfix=postfix, message=message, - version=version, - removal_version=removal_version) - deprecation(out_message, stacklevel=stacklevel) - - def moved_proxy_class(new_class, old_class_name, old_module_name, message=None, version=None, removal_version=None, stacklevel=3): From dc2fdaf6b980c0a24aa020046dbfe1f408deb1f5 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 18 Mar 2015 22:11:03 -0700 Subject: [PATCH 095/246] Allow using shallow copy instead of deep copy Certain projects (cinder at the least) seems to be storing items that can not be deep copied without issues; so to make it so that we can easily support having this backend be the default backend always used we need to be able to support a shallow copy for when it is needed. Change-Id: I16215b44870cef9ea60d144ba710c5d730e8b284 --- taskflow/persistence/backends/impl_memory.py | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/taskflow/persistence/backends/impl_memory.py b/taskflow/persistence/backends/impl_memory.py index 78ef21c3..a457214a 100644 --- a/taskflow/persistence/backends/impl_memory.py +++ b/taskflow/persistence/backends/impl_memory.py @@ -38,8 +38,12 @@ class FakeFilesystem(object): " paths: '%s' is not valid" % path) return os.path.normpath(path) - def __init__(self): + def __init__(self, deep_copy=True): self._root = tree.Node(self.root_path, value=None) + if deep_copy: + self._copier = copy.deepcopy + else: + self._copier = copy.copy def ensure_path(self, path): path = self._normpath(path) @@ -83,7 +87,7 @@ class FakeFilesystem(object): links.append(path) return self._get_item(path, links=links) else: - return copy.deepcopy(node.metadata['value']) + return self._copier(node.metadata['value']) def ls(self, path): return [node.item for node in self._fetch_node(path)] @@ -131,7 +135,7 @@ class FakeFilesystem(object): def __setitem__(self, path, value): path = self._normpath(path) - value = copy.deepcopy(value) + value = self._copier(value) try: item_node = self._fetch_node(path) item_node.metadata.update(value=value) @@ -156,7 +160,8 @@ class MemoryBackend(path_based.PathBasedBackend): super(MemoryBackend, self).__init__(conf) if self._path is None: self._path = os.sep - self.memory = FakeFilesystem() + self.memory = FakeFilesystem(deep_copy=self._conf.get('deep_copy', + True)) self.lock = lock_utils.ReaderWriterLock() def get_connection(self): From f815480c0968710594904c17c96d66dbeafc9156 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 18 Mar 2015 16:16:09 -0700 Subject: [PATCH 096/246] By default use a in memory backend (when none is provided) Change-Id: I6891d53389f302f104d45d22e489cf66feb85fd8 --- taskflow/persistence/path_based.py | 9 ++++++--- taskflow/storage.py | 20 +++++++++++++++----- 2 files changed, 21 insertions(+), 8 deletions(-) diff --git a/taskflow/persistence/path_based.py b/taskflow/persistence/path_based.py index d193488d..6c065df0 100644 --- a/taskflow/persistence/path_based.py +++ b/taskflow/persistence/path_based.py @@ -45,6 +45,8 @@ class PathBasedBackend(base.Backend): @six.add_metaclass(abc.ABCMeta) class PathBasedConnection(base.Connection): + """Base class for path based backend connections.""" + def __init__(self, backend): self._backend = backend self._book_path = self._join_path(backend.path, "books") @@ -187,7 +189,8 @@ class PathBasedConnection(base.Connection): def _do_update_flow_details(self, flow_detail, transaction, ignore_missing=False): flow_path = self._get_obj_path(flow_detail) - self._update_object(flow_detail, transaction, ignore_missing) + self._update_object(flow_detail, transaction, + ignore_missing=ignore_missing) for atom_details in flow_detail: atom_path = self._get_obj_path(atom_details) link_path = self._join_path(flow_path, atom_details.uuid) @@ -198,7 +201,7 @@ class PathBasedConnection(base.Connection): def update_flow_details(self, flow_detail, ignore_missing=False): with self._transaction() as transaction: return self._do_update_flow_details(flow_detail, transaction, - ignore_missing) + ignore_missing=ignore_missing) def get_atoms_for_flow(self, flow_uuid): flow_path = self._join_path(self.flow_path, flow_uuid) @@ -213,7 +216,7 @@ class PathBasedConnection(base.Connection): def update_atom_details(self, atom_detail, ignore_missing=False): with self._transaction() as transaction: return self._update_object(atom_detail, transaction, - ignore_missing) + ignore_missing=ignore_missing) def _do_destroy_logbook(self, book_uuid, transaction): book_path = self._join_path(self.book_path, book_uuid) diff --git a/taskflow/storage.py b/taskflow/storage.py index 5386907e..fe601c9a 100644 --- a/taskflow/storage.py +++ b/taskflow/storage.py @@ -22,6 +22,7 @@ import six from taskflow import exceptions from taskflow import logging +from taskflow.persistence.backends import impl_memory from taskflow.persistence import logbook from taskflow import retry from taskflow import states @@ -121,6 +122,10 @@ class Storage(object): atom_details, flow_details) for use by engines. This makes it easier to interact with the underlying storage & backend mechanism through this interface rather than accessing those objects directly. + + NOTE(harlowja): if no backend is provided then a in-memory backend will + be automatically used and the provided flow detail object will be placed + into it for the duration of this objects existence. """ injector_name = '_TaskFlow_INJECTOR' @@ -134,6 +139,13 @@ class Storage(object): def __init__(self, flow_detail, backend=None, scope_fetcher=None): self._result_mappings = {} self._reverse_mapping = {} + if backend is None: + # Err on the likely-hood that most people don't make there + # objects able to be deepcopyable (resources, locks and such + # can't be deepcopied)... + backend = impl_memory.MemoryBackend({'deep_copy': False}) + with contextlib.closing(backend.get_connection()) as conn: + conn.update_flow_details(flow_detail, ignore_missing=True) self._backend = backend self._flowdetail = flow_detail self._transients = {} @@ -169,11 +181,9 @@ class Storage(object): dict((name, name) for name in names)) def _with_connection(self, functor, *args, **kwargs): - # NOTE(harlowja): Activate the given function with a backend - # connection, if a backend is provided in the first place, otherwise - # don't call the function. - if self._backend is None: - return + # Run the given functor with a backend connection as its first + # argument (providing the additional positional arguments and keyword + # arguments as subsequent arguments). with contextlib.closing(self._backend.get_connection()) as conn: functor(conn, *args, **kwargs) From cd5413902dbdf8bffc7f74d0c0a689e9d193a4a6 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Mon, 23 Mar 2015 19:02:05 -0700 Subject: [PATCH 097/246] Prefer posixpath to os.path To ensure that the memory fake filesystem works uniformly across distributions prefer to use the posixpath module which always works with '/' and friends instead of the os.path and os.path.sep and friends which may vary depending on operating system used. Since we have tested with the usage of '/' we might as well just restrict to that; and avoid the cross operating system issues that may pop up when using this fake filesystem. In general isolating one self from the operating system specifics is IMHO preferable for this; as it avoids edge cases that we don't care about. Change-Id: I3f61f380e1bcb131bc42b627adf9dfe8a7f2d992 --- taskflow/persistence/backends/impl_memory.py | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/taskflow/persistence/backends/impl_memory.py b/taskflow/persistence/backends/impl_memory.py index a457214a..bb74a401 100644 --- a/taskflow/persistence/backends/impl_memory.py +++ b/taskflow/persistence/backends/impl_memory.py @@ -17,7 +17,7 @@ import contextlib import copy -import os +import posixpath as pp from taskflow import exceptions as exc from taskflow.persistence import path_based @@ -29,14 +29,14 @@ class FakeFilesystem(object): """An in-memory filesystem-like structure.""" #: Root path of the in-memory filesystem. - root_path = os.sep + root_path = pp.sep @classmethod def _normpath(cls, path): if not path.startswith(cls.root_path): raise ValueError("This filesystem can only normalize absolute" " paths: '%s' is not valid" % path) - return os.path.normpath(path) + return pp.normpath(path) def __init__(self, deep_copy=True): self._root = tree.Node(self.root_path, value=None) @@ -98,11 +98,11 @@ class FakeFilesystem(object): # split correctly: # # >>> path = "/" - # path.split(os.sep) + # path.split(pp.sep) # ['', ''] parts = [] else: - parts = path.split(os.sep)[1:] + parts = path.split(pp.sep)[1:] if include_root: parts.insert(0, self._root.item) for piece in parts: @@ -120,7 +120,7 @@ class FakeFilesystem(object): def symlink(self, src_path, dest_path): dest_path = self._normpath(dest_path) src_path = self._normpath(src_path) - dirname, basename = os.path.split(dest_path) + dirname, basename = pp.split(dest_path) parent_node = self._fetch_node(dirname) child_node = parent_node.find(basename, only_direct=True, @@ -140,7 +140,7 @@ class FakeFilesystem(object): item_node = self._fetch_node(path) item_node.metadata.update(value=value) except exc.NotFound: - dirname, basename = os.path.split(path) + dirname, basename = pp.split(path) parent_node = self._fetch_node(dirname) parent_node.add(tree.Node(basename, value=value)) @@ -159,7 +159,7 @@ class MemoryBackend(path_based.PathBasedBackend): def __init__(self, conf=None): super(MemoryBackend, self).__init__(conf) if self._path is None: - self._path = os.sep + self._path = pp.sep self.memory = FakeFilesystem(deep_copy=self._conf.get('deep_copy', True)) self.lock = lock_utils.ReaderWriterLock() @@ -191,7 +191,7 @@ class Connection(path_based.PathBasedConnection): raise exc.StorageFailure("Storage backend internal error", e) def _join_path(self, *parts): - return os.path.join(*parts) + return pp.join(*parts) def _get_item(self, path): with self._memory_lock(): From cdcb248a650640c3614824a3782daf1bd69faaff Mon Sep 17 00:00:00 2001 From: Suneel Bomminayuni Date: Tue, 24 Mar 2015 18:32:25 +0000 Subject: [PATCH 098/246] Adding test to improve CaptureListener coverage Added a unittest to check the 'capture_task' parameter of CaptureListener class Change-Id: I29a9e7d5b6d41ab458ca116cc4cec551c09cbd30 --- taskflow/tests/unit/test_listeners.py | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/taskflow/tests/unit/test_listeners.py b/taskflow/tests/unit/test_listeners.py index 4c6505ed..d5955106 100644 --- a/taskflow/tests/unit/test_listeners.py +++ b/taskflow/tests/unit/test_listeners.py @@ -258,6 +258,18 @@ class TestEventTimeListener(test.TestCase, EngineMakerMixin): self.assertGreaterEqual(0.1, fd_duration) +class TestCapturingListeners(test.TestCase, EngineMakerMixin): + def test_basic_do_not_capture(self): + flow = lf.Flow("test") + flow.add(test_utils.ProgressingTask("task1")) + e = self._make_engine(flow) + with test_utils.CaptureListener(e, capture_task=False) as capturer: + e.run() + expected = ['test.f RUNNING', + 'test.f SUCCESS'] + self.assertEqual(expected, capturer.values) + + class TestLoggingListeners(test.TestCase, EngineMakerMixin): def _make_logger(self, level=logging.DEBUG): log = logging.getLogger( From 5accccbb7c8fc7f4ff2577bffa83fea27d495aea Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Mon, 23 Mar 2015 17:00:07 -0700 Subject: [PATCH 099/246] Add a set of tests to the in memory fake filesystem To ensure this continues to work as expected add some basic functional tests that ensure the existing operations work (and continue to work). Change-Id: I2903bf29a77936d850d667d1fbff179a14a57adb --- .../persistence/test_memory_persistence.py | 62 +++++++++++++++++++ 1 file changed, 62 insertions(+) diff --git a/taskflow/tests/unit/persistence/test_memory_persistence.py b/taskflow/tests/unit/persistence/test_memory_persistence.py index 725b7459..1c9f451b 100644 --- a/taskflow/tests/unit/persistence/test_memory_persistence.py +++ b/taskflow/tests/unit/persistence/test_memory_persistence.py @@ -16,6 +16,7 @@ import contextlib +from taskflow import exceptions as exc from taskflow.persistence import backends from taskflow.persistence.backends import impl_memory from taskflow import test @@ -45,3 +46,64 @@ class MemoryPersistenceTest(test.TestCase, base.PersistenceTestMixin): conf = {'connection': 'memory'} # note no colon with contextlib.closing(backends.fetch(conf)) as be: self.assertIsInstance(be, impl_memory.MemoryBackend) + + +class MemoryFilesystemTest(test.TestCase): + + @staticmethod + def _get_item_path(fs, path): + # TODO(harlowja): is there a better way to do this?? + return fs[path] + + @staticmethod + def _del_item_path(fs, path): + # TODO(harlowja): is there a better way to do this?? + del fs[path] + + def test_set_get_ls(self): + fs = impl_memory.FakeFilesystem() + fs['/d'] = 'd' + fs['/c'] = 'c' + fs['/d/b'] = 'db' + self.assertEqual(2, len(fs.ls('/'))) + self.assertEqual(1, len(fs.ls('/d'))) + self.assertEqual('d', fs['/d']) + self.assertEqual('c', fs['/c']) + self.assertEqual('db', fs['/d/b']) + + def test_ensure_path(self): + fs = impl_memory.FakeFilesystem() + pieces = ['a', 'b', 'c'] + path = "/" + "/".join(pieces) + fs.ensure_path(path) + path = fs.root_path + for i, p in enumerate(pieces): + if i == 0: + path += p + else: + path += "/" + p + self.assertIsNone(fs[path]) + + def test_not_found(self): + fs = impl_memory.FakeFilesystem() + self.assertRaises(exc.NotFound, self._get_item_path, fs, '/c') + + def test_del_root_not_allowed(self): + fs = impl_memory.FakeFilesystem() + self.assertRaises(ValueError, self._del_item_path, fs, '/') + + def test_link_loop_raises(self): + fs = impl_memory.FakeFilesystem() + fs['/b'] = 'c' + fs.symlink('/b', '/b') + self.assertRaises(ValueError, self._get_item_path, fs, '/b') + + def test_ensure_linked_delete(self): + fs = impl_memory.FakeFilesystem() + fs['/b'] = 'd' + fs.symlink('/b', '/c') + self.assertEqual('d', fs['/b']) + self.assertEqual('d', fs['/c']) + del fs['/b'] + self.assertRaises(exc.NotFound, self._get_item_path, fs, '/c') + self.assertRaises(exc.NotFound, self._get_item_path, fs, '/b') From 66a16b240ad4171a21bbe5fe8e514863b2a875e8 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 25 Mar 2015 16:06:42 -0700 Subject: [PATCH 100/246] Add more comments to fake in-memory filesystem Ensure users of this code/objects are aware that only posixpath style paths will work (and not the variant that is used in os.path which can change depending on operating system) as well as adding docstrings on other methods so that they show up in the generated docs. Change-Id: I5a21cb75b2452b9c9d65860b63658a9d0980025f --- taskflow/persistence/backends/impl_memory.py | 26 +++++++++++++++++++- 1 file changed, 25 insertions(+), 1 deletion(-) diff --git a/taskflow/persistence/backends/impl_memory.py b/taskflow/persistence/backends/impl_memory.py index bb74a401..7efe6ec0 100644 --- a/taskflow/persistence/backends/impl_memory.py +++ b/taskflow/persistence/backends/impl_memory.py @@ -26,7 +26,27 @@ from taskflow.utils import lock_utils class FakeFilesystem(object): - """An in-memory filesystem-like structure.""" + """An in-memory filesystem-like structure. + + This filesystem uses posix style paths **only** so users must be careful + to use the ``posixpath`` module instead of the ``os.path`` one which will + vary depending on the operating system which the active python is running + in (the decision to use ``posixpath`` was to avoid the path variations + which are not relevant in an implementation of a in-memory fake + filesystem). + + Example usage: + + >>> from taskflow.persistence.backends import impl_memory + >>> fs = impl_memory.FakeFilesystem() + >>> fs.ensure_path('/a/b/c') + >>> fs['/a/b/c'] = 'd' + >>> print(fs['/a/b/c']) + d + >>> del fs['/a/b/c'] + >>> fs.ls("/a/b") + [] + """ #: Root path of the in-memory filesystem. root_path = pp.sep @@ -46,6 +66,7 @@ class FakeFilesystem(object): self._copier = copy.copy def ensure_path(self, path): + """Ensure the path (and parents) exists.""" path = self._normpath(path) # Ignore the root path as we already checked for that; and it # will always exist/can't be removed anyway... @@ -90,6 +111,7 @@ class FakeFilesystem(object): return self._copier(node.metadata['value']) def ls(self, path): + """Return list of all children of the given path.""" return [node.item for node in self._fetch_node(path)] def _iter_pieces(self, path, include_root=False): @@ -115,9 +137,11 @@ class FakeFilesystem(object): node.disassociate() def pformat(self): + """Pretty format this in-memory filesystem.""" return self._root.pformat() def symlink(self, src_path, dest_path): + """Link the destionation path to the source path.""" dest_path = self._normpath(dest_path) src_path = self._normpath(src_path) dirname, basename = pp.split(dest_path) From 977d19caa3a632e1cd4ed133b35d7ef274fc05d3 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 25 Mar 2015 17:00:54 -0700 Subject: [PATCH 101/246] Make the graph '_unsatisfied_requires' be a staticmethod To make it possible for users to subclass this flow and replace this method (if they so choose to) make it a static method that is directed to the current module level function and call into the staticmethod when used (users can override staticmethods and change them in subclasses). Change-Id: I747c45636596ca5a8ad81dabcbba12ab55ce77d7 --- taskflow/patterns/graph_flow.py | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/taskflow/patterns/graph_flow.py b/taskflow/patterns/graph_flow.py index d36a72ff..bb779cb2 100644 --- a/taskflow/patterns/graph_flow.py +++ b/taskflow/patterns/graph_flow.py @@ -22,7 +22,6 @@ from taskflow.types import graph as gr def _unsatisfied_requires(node, graph, *additional_provided): - """Extracts the unsatisified symbol requirements of a single node.""" requires = set(node.requires) if not requires: return requires @@ -64,6 +63,9 @@ class Flow(flow.Flow): self._graph = gr.DiGraph() self._graph.freeze() + #: Extracts the unsatisified symbol requirements of a single node. + _unsatisfied_requires = staticmethod(_unsatisfied_requires) + def link(self, u, v): """Link existing node u as a runtime dependency of existing node v.""" if not self._graph.has_node(u): @@ -153,8 +155,8 @@ class Flow(flow.Flow): provided[value].append(self._retry) for item in self._graph.nodes_iter(): - for value in _unsatisfied_requires(item, self._graph, - retry_provides): + for value in self._unsatisfied_requires(item, self._graph, + retry_provides): required[value].append(item) for value in item.provides: provided[value].append(item) @@ -168,8 +170,8 @@ class Flow(flow.Flow): # Try to find a valid provider. if resolve_requires: - for value in _unsatisfied_requires(item, tmp_graph, - retry_provides): + for value in self._unsatisfied_requires(item, tmp_graph, + retry_provides): if value in provided: providers = provided[value] if len(providers) > 1: @@ -232,7 +234,8 @@ class Flow(flow.Flow): retry_provides.update(self._retry.provides) g = self._get_subgraph() for item in g.nodes_iter(): - requires.update(_unsatisfied_requires(item, g, retry_provides)) + requires.update(self._unsatisfied_requires(item, g, + retry_provides)) return frozenset(requires) From 30c7c0c010526fb3c37f97aa3372c3c510c51c0b Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Sat, 21 Mar 2015 21:00:22 -0700 Subject: [PATCH 102/246] Add memory backend get() support Instead of requiring users to use the __getitem__ fetch entrypoint and deal with the value not being found provide a dictionary equivalent get() support that handles the not found case and returns a default when this is triggered. Change-Id: Ie3795745515a9d39ef4321fecb78e9b0e7539d72 --- taskflow/persistence/backends/impl_memory.py | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/taskflow/persistence/backends/impl_memory.py b/taskflow/persistence/backends/impl_memory.py index 7efe6ec0..9e265ef5 100644 --- a/taskflow/persistence/backends/impl_memory.py +++ b/taskflow/persistence/backends/impl_memory.py @@ -46,6 +46,8 @@ class FakeFilesystem(object): >>> del fs['/a/b/c'] >>> fs.ls("/a/b") [] + >>> fs.get("/a/b/c", 'blob') + 'blob' """ #: Root path of the in-memory filesystem. @@ -93,6 +95,13 @@ class FakeFilesystem(object): raise exc.NotFound("Path '%s' not found" % path) return node + def get(self, path, default=None): + """Fetch the value of given path (and return default if not found).""" + try: + return self._get_item(path) + except exc.NotFound: + return default + def _get_item(self, path, links=None): node = self._fetch_node(path) if 'target' in node.metadata: From 5544d71bc897906253cf98771129f2997f4e19ae Mon Sep 17 00:00:00 2001 From: Rick van de Loo Date: Sat, 28 Mar 2015 20:13:21 +1100 Subject: [PATCH 103/246] Fix a couple of spelling and grammar errors Some things that popped out while reading the comments/documentation. Change-Id: I0ccecae3381447ede44bb855d91f997349be1562 --- doc/source/arguments_and_results.rst | 2 +- doc/source/atoms.rst | 2 +- doc/source/engines.rst | 12 ++++++------ doc/source/jobs.rst | 4 ++-- doc/source/persistence.rst | 4 ++-- doc/source/states.rst | 6 +++--- doc/source/workers.rst | 2 +- pylintrc | 2 +- taskflow/examples/alphabet_soup.py | 2 +- taskflow/examples/build_a_car.py | 2 +- taskflow/examples/calculate_in_parallel.py | 2 +- taskflow/examples/create_parallel_volume.py | 2 +- taskflow/examples/echo_listener.py | 4 ++-- taskflow/examples/fake_billing.py | 4 ++-- taskflow/examples/hello_world.py | 2 +- taskflow/examples/jobboard_produce_consume_colors.py | 4 ++-- taskflow/examples/persistence_example.py | 10 +++++----- taskflow/examples/resume_vm_boot.py | 2 +- taskflow/examples/resume_volume_create.py | 2 +- taskflow/examples/retry_flow.py | 2 +- taskflow/examples/run_by_iter.py | 2 +- taskflow/examples/run_by_iter_enumerate.py | 4 ++-- taskflow/examples/simple_linear.py | 4 ++-- taskflow/examples/simple_linear_listening.py | 4 ++-- taskflow/examples/simple_linear_pass.py | 2 +- taskflow/examples/simple_map_reduce.py | 4 ++-- taskflow/examples/timing_listener.py | 2 +- taskflow/examples/wbe_event_sender.py | 8 ++++---- taskflow/examples/wbe_mandelbrot.py | 4 ++-- 29 files changed, 53 insertions(+), 53 deletions(-) diff --git a/doc/source/arguments_and_results.rst b/doc/source/arguments_and_results.rst index e4f79990..009086bf 100644 --- a/doc/source/arguments_and_results.rst +++ b/doc/source/arguments_and_results.rst @@ -416,7 +416,7 @@ the following history (printed as a list):: At this point (since the implementation returned ``RETRY``) the |retry.execute| method will be called again and it will receive the same history and it can then return a value that subseqent tasks can use to alter -there behavior. +their behavior. If instead the |retry.execute| method itself raises an exception, the |retry.revert| method of the implementation will be called and diff --git a/doc/source/atoms.rst b/doc/source/atoms.rst index c54eb571..9c2827bc 100644 --- a/doc/source/atoms.rst +++ b/doc/source/atoms.rst @@ -91,7 +91,7 @@ subclasses are provided: .. note:: They are *similar* to exception handlers but are made to be *more* capable - due to there ability to *dynamically* choose a reconciliation strategy, + due to their ability to *dynamically* choose a reconciliation strategy, which allows for these atoms to influence subsequent execution(s) and the inputs any associated atoms require. diff --git a/doc/source/engines.rst b/doc/source/engines.rst index 5ce22903..a3b3867d 100644 --- a/doc/source/engines.rst +++ b/doc/source/engines.rst @@ -29,7 +29,7 @@ Why they exist An engine being *the* core component which actually makes your flows progress is likely a new concept for many programmers so let's describe how it operates in more depth and some of the reasoning behind why it exists. This will -hopefully make it more clear on there value add to the TaskFlow library user. +hopefully make it more clear on their value add to the TaskFlow library user. First though let us discuss something most are familiar already with; the difference between `declarative`_ and `imperative`_ programming models. The @@ -57,7 +57,7 @@ declarative model) allows for the following functionality to become possible: accomplished allows for a *natural* way of resuming by allowing the engine to track the current state and know at which point a workflow is in and how to get back into that state when resumption occurs. -* Enhancing scalability: When a engine is responsible for executing your +* Enhancing scalability: When an engine is responsible for executing your desired work it becomes possible to alter the *how* in the future by creating new types of execution backends (for example the `worker`_ model which does not execute locally). Without the decoupling of the *what* and the *how* it @@ -203,7 +203,7 @@ For further information, please refer to the the following: How they run ============ -To provide a peek into the general process that a engine goes through when +To provide a peek into the general process that an engine goes through when running lets break it apart a little and describe what one of the engine types does while executing (for this we will look into the :py:class:`~taskflow.engines.action_engine.engine.ActionEngine` engine type). @@ -299,7 +299,7 @@ Scheduling This stage selects which atoms are eligible to run by using a :py:class:`~taskflow.engines.action_engine.scheduler.Scheduler` implementation -(the default implementation looks at there intention, checking if predecessor +(the default implementation looks at their intention, checking if predecessor atoms have ran and so-on, using a :py:class:`~taskflow.engines.action_engine.analyzer.Analyzer` helper object as needed) and submits those atoms to a previously provided compatible @@ -335,7 +335,7 @@ above stages will be restarted and resuming will occur). If the engine is suspended while the engine is going through the above stages this will stop any further scheduling stages from occurring and - all currently executing atoms will be allowed to finish (and there results + all currently executing atoms will be allowed to finish (and their results will be saved). Finishing @@ -366,7 +366,7 @@ be selected? Default strategy ---------------- -When a engine is executing it internally interacts with the +When an engine is executing it internally interacts with the :py:class:`~taskflow.storage.Storage` class and that class interacts with the a :py:class:`~taskflow.engines.action_engine.scopes.ScopeWalker` instance diff --git a/doc/source/jobs.rst b/doc/source/jobs.rst index 06f1123e..d25436ce 100644 --- a/doc/source/jobs.rst +++ b/doc/source/jobs.rst @@ -43,7 +43,7 @@ Jobboards jobboards implement the same interface and semantics so that the backend usage is as transparent as possible. This allows deployers or developers of a service that uses TaskFlow to select a jobboard implementation that fits - their setup (and there intended usage) best. + their setup (and their intended usage) best. High level architecture ======================= @@ -218,7 +218,7 @@ Dual-engine jobs ---------------- **What:** Since atoms and engines are not currently `preemptable`_ we can not -force a engine (or the threads/remote workers... it is using to run) to stop +force an engine (or the threads/remote workers... it is using to run) to stop working on an atom (it is general bad behavior to force code to stop without its consent anyway) if it has already started working on an atom (short of doing a ``kill -9`` on the running interpreter). This could cause problems diff --git a/doc/source/persistence.rst b/doc/source/persistence.rst index 53b01aa3..a0731a30 100644 --- a/doc/source/persistence.rst +++ b/doc/source/persistence.rst @@ -70,7 +70,7 @@ from a previous run) they will begin executing only after any dependent inputs are ready. This is done by analyzing the execution graph and looking at predecessor :py:class:`~taskflow.persistence.logbook.AtomDetail` outputs and states (which may have been persisted in a past run). This will result in -either using there previous information or by running those predecessors and +either using their previous information or by running those predecessors and saving their output to the :py:class:`~taskflow.persistence.logbook.FlowDetail` and :py:class:`~taskflow.persistence.base.Backend` objects. This execution, analysis and interaction with the storage objects continues (what is @@ -81,7 +81,7 @@ will have succeeded or failed in its attempt to run the workflow). **Post-execution:** Typically when an engine is done running the logbook would be discarded (to avoid creating a stockpile of useless data) and the backend storage would be told to delete any contents for a given execution. For certain -use-cases though it may be advantageous to retain logbooks and there contents. +use-cases though it may be advantageous to retain logbooks and their contents. A few scenarios come to mind: diff --git a/doc/source/states.rst b/doc/source/states.rst index bba8d203..9cc7a50a 100644 --- a/doc/source/states.rst +++ b/doc/source/states.rst @@ -121,7 +121,7 @@ or if needed will wait for all of the atoms it depends on to complete. .. note:: - A engine running a task also transitions the task to the ``PENDING`` state + An engine running a task also transitions the task to the ``PENDING`` state after it was reverted and its containing flow was restarted or retried. **RUNNING** - When an engine running the task starts to execute the task, the @@ -168,10 +168,10 @@ flow that the retry is associated with by consulting its .. note:: - A engine running a retry also transitions the retry to the ``PENDING`` state + An engine running a retry also transitions the retry to the ``PENDING`` state after it was reverted and its associated flow was restarted or retried. -**RUNNING** - When a engine starts to execute the retry, the engine +**RUNNING** - When an engine starts to execute the retry, the engine transitions the retry to the ``RUNNING`` state, and the retry stays in this state until its :py:meth:`~taskflow.retry.Retry.execute` method returns. diff --git a/doc/source/workers.rst b/doc/source/workers.rst index 01058b7a..38212405 100644 --- a/doc/source/workers.rst +++ b/doc/source/workers.rst @@ -409,7 +409,7 @@ Limitations * Fault detection, currently when a worker acknowledges a task the engine will wait for the task result indefinitely (a task may take an indeterminate amount of time to finish). In the future there needs to be a way to limit - the duration of a remote workers execution (and track there liveness) and + the duration of a remote workers execution (and track their liveness) and possibly spawn the task on a secondary worker if a timeout is reached (aka the first worker has died or has stopped responding). diff --git a/pylintrc b/pylintrc index 7330f262..def9c979 100644 --- a/pylintrc +++ b/pylintrc @@ -12,7 +12,7 @@ variable-rgx=[a-z_][a-z0-9_]{0,30}$ argument-rgx=[a-z_][a-z0-9_]{1,30}$ # Method names should be at least 3 characters long -# and be lowecased with underscores +# and be lowercased with underscores method-rgx=[a-z_][a-z0-9_]{2,50}$ # Don't require docstrings on tests. diff --git a/taskflow/examples/alphabet_soup.py b/taskflow/examples/alphabet_soup.py index a287f538..eb199f8e 100644 --- a/taskflow/examples/alphabet_soup.py +++ b/taskflow/examples/alphabet_soup.py @@ -38,7 +38,7 @@ from taskflow import task # In this example we show how a simple linear set of tasks can be executed -# using local processes (and not threads or remote workers) with minimial (if +# using local processes (and not threads or remote workers) with minimal (if # any) modification to those tasks to make them safe to run in this mode. # # This is useful since it allows further scaling up your workflows when thread diff --git a/taskflow/examples/build_a_car.py b/taskflow/examples/build_a_car.py index 02be020e..a3291817 100644 --- a/taskflow/examples/build_a_car.py +++ b/taskflow/examples/build_a_car.py @@ -38,7 +38,7 @@ ANY = notifier.Notifier.ANY import example_utils as eu # noqa -# INTRO: This examples shows how a graph flow and linear flow can be used +# INTRO: This example shows how a graph flow and linear flow can be used # together to execute dependent & non-dependent tasks by going through the # steps required to build a simplistic car (an assembly line if you will). It # also shows how raw functions can be wrapped into a task object instead of diff --git a/taskflow/examples/calculate_in_parallel.py b/taskflow/examples/calculate_in_parallel.py index 7ab32fae..0d800a60 100644 --- a/taskflow/examples/calculate_in_parallel.py +++ b/taskflow/examples/calculate_in_parallel.py @@ -30,7 +30,7 @@ from taskflow.patterns import linear_flow as lf from taskflow.patterns import unordered_flow as uf from taskflow import task -# INTRO: This examples shows how a linear flow and a unordered flow can be +# INTRO: These examples show how a linear flow and an unordered flow can be # used together to execute calculations in parallel and then use the # result for the next task/s. The adder task is used for all calculations # and argument bindings are used to set correct parameters for each task. diff --git a/taskflow/examples/create_parallel_volume.py b/taskflow/examples/create_parallel_volume.py index c23bf342..9bab0ba3 100644 --- a/taskflow/examples/create_parallel_volume.py +++ b/taskflow/examples/create_parallel_volume.py @@ -35,7 +35,7 @@ from taskflow.listeners import printing from taskflow.patterns import unordered_flow as uf from taskflow import task -# INTRO: This examples shows how unordered_flow can be used to create a large +# INTRO: These examples show how unordered_flow can be used to create a large # number of fake volumes in parallel (or serially, depending on a constant that # can be easily changed). diff --git a/taskflow/examples/echo_listener.py b/taskflow/examples/echo_listener.py index a8eebf60..99871a83 100644 --- a/taskflow/examples/echo_listener.py +++ b/taskflow/examples/echo_listener.py @@ -31,8 +31,8 @@ from taskflow.patterns import linear_flow as lf from taskflow import task # INTRO: This example walks through a miniature workflow which will do a -# simple echo operation; during this execution a listener is assocated with -# the engine to recieve all notifications about what the flow has performed, +# simple echo operation; during this execution a listener is associated with +# the engine to receive all notifications about what the flow has performed, # this example dumps that output to the stdout for viewing (at debug level # to show all the information which is possible). diff --git a/taskflow/examples/fake_billing.py b/taskflow/examples/fake_billing.py index 5d26a2dc..33804b58 100644 --- a/taskflow/examples/fake_billing.py +++ b/taskflow/examples/fake_billing.py @@ -36,8 +36,8 @@ from taskflow.patterns import linear_flow as lf from taskflow import task from taskflow.utils import misc -# INTRO: This example walks through a miniature workflow which simulates a -# the reception of a API request, creation of a database entry, driver +# INTRO: This example walks through a miniature workflow which simulates +# the reception of an API request, creation of a database entry, driver # activation (which invokes a 'fake' webservice) and final completion. # # This example also shows how a function/object (in this class the url sending) diff --git a/taskflow/examples/hello_world.py b/taskflow/examples/hello_world.py index f8e0bb23..caba5275 100644 --- a/taskflow/examples/hello_world.py +++ b/taskflow/examples/hello_world.py @@ -34,7 +34,7 @@ from taskflow.utils import eventlet_utils # INTRO: This is the defacto hello world equivalent for taskflow; it shows how -# a overly simplistic workflow can be created that runs using different +# an overly simplistic workflow can be created that runs using different # engines using different styles of execution (all can be used to run in # parallel if a workflow is provided that is parallelizable). diff --git a/taskflow/examples/jobboard_produce_consume_colors.py b/taskflow/examples/jobboard_produce_consume_colors.py index 80c2acba..6f586030 100644 --- a/taskflow/examples/jobboard_produce_consume_colors.py +++ b/taskflow/examples/jobboard_produce_consume_colors.py @@ -40,7 +40,7 @@ from taskflow.utils import threading_utils # In this example we show how a jobboard can be used to post work for other # entities to work on. This example creates a set of jobs using one producer # thread (typically this would be split across many machines) and then having -# other worker threads with there own jobboards select work using a given +# other worker threads with their own jobboards select work using a given # filters [red/blue] and then perform that work (and consuming or abandoning # the job after it has been completed or failed). @@ -66,7 +66,7 @@ PRODUCER_UNITS = 10 # How many units of work are expected to be produced (used so workers can # know when to stop running and shutdown, typically this would not be a -# a value but we have to limit this examples execution time to be less than +# a value but we have to limit this example's execution time to be less than # infinity). EXPECTED_UNITS = PRODUCER_UNITS * PRODUCERS diff --git a/taskflow/examples/persistence_example.py b/taskflow/examples/persistence_example.py index fe5968fe..c911c2f1 100644 --- a/taskflow/examples/persistence_example.py +++ b/taskflow/examples/persistence_example.py @@ -68,15 +68,15 @@ class ByeTask(task.Task): print("Bye!") -# This generates your flow structure (at this stage nothing is ran). +# This generates your flow structure (at this stage nothing is run). def make_flow(blowup=False): flow = lf.Flow("hello-world") flow.add(HiTask(), ByeTask(blowup)) return flow -# Persist the flow and task state here, if the file/dir exists already blowup -# if not don't blowup, this allows a user to see both the modes and to see +# Persist the flow and task state here, if the file/dir exists already blow up +# if not don't blow up, this allows a user to see both the modes and to see # what is stored in each case. if eu.SQLALCHEMY_AVAILABLE: persist_path = os.path.join(tempfile.gettempdir(), "persisting.db") @@ -91,8 +91,8 @@ else: blowup = True with eu.get_backend(backend_uri) as backend: - # Make a flow that will blowup if the file doesn't exist previously, if it - # did exist, assume we won't blowup (and therefore this shows the undo + # Make a flow that will blow up if the file didn't exist previously, if it + # did exist, assume we won't blow up (and therefore this shows the undo # and redo that a flow will go through). book = logbook.LogBook("my-test") flow = make_flow(blowup=blowup) diff --git a/taskflow/examples/resume_vm_boot.py b/taskflow/examples/resume_vm_boot.py index 4e93f787..8c7d4aea 100644 --- a/taskflow/examples/resume_vm_boot.py +++ b/taskflow/examples/resume_vm_boot.py @@ -44,7 +44,7 @@ from taskflow.utils import persistence_utils as p_utils import example_utils as eu # noqa -# INTRO: This examples shows how a hierarchy of flows can be used to create a +# INTRO: These examples show how a hierarchy of flows can be used to create a # vm in a reliable & resumable manner using taskflow + a miniature version of # what nova does while booting a vm. diff --git a/taskflow/examples/resume_volume_create.py b/taskflow/examples/resume_volume_create.py index 275fa6b8..93025d95 100644 --- a/taskflow/examples/resume_volume_create.py +++ b/taskflow/examples/resume_volume_create.py @@ -39,7 +39,7 @@ from taskflow.utils import persistence_utils as p_utils import example_utils # noqa -# INTRO: This examples shows how a hierarchy of flows can be used to create a +# INTRO: These examples show how a hierarchy of flows can be used to create a # pseudo-volume in a reliable & resumable manner using taskflow + a miniature # version of what cinder does while creating a volume (very miniature). diff --git a/taskflow/examples/retry_flow.py b/taskflow/examples/retry_flow.py index c1cb1a2d..3123aca1 100644 --- a/taskflow/examples/retry_flow.py +++ b/taskflow/examples/retry_flow.py @@ -32,7 +32,7 @@ from taskflow import task # INTRO: In this example we create a retry controller that receives a phone # directory and tries different phone numbers. The next task tries to call Jim -# using the given number. If if is not a Jim's number, the tasks raises an +# using the given number. If it is not a Jim's number, the task raises an # exception and retry controller takes the next number from the phone # directory and retries the call. # diff --git a/taskflow/examples/run_by_iter.py b/taskflow/examples/run_by_iter.py index 4b7b98cc..3a00a102 100644 --- a/taskflow/examples/run_by_iter.py +++ b/taskflow/examples/run_by_iter.py @@ -37,7 +37,7 @@ from taskflow import task from taskflow.utils import persistence_utils -# INTRO: This examples shows how to run a set of engines at the same time, each +# INTRO: This example shows how to run a set of engines at the same time, each # running in different engines using a single thread of control to iterate over # each engine (which causes that engine to advanced to its next state during # each iteration). diff --git a/taskflow/examples/run_by_iter_enumerate.py b/taskflow/examples/run_by_iter_enumerate.py index d954d6aa..07334cc7 100644 --- a/taskflow/examples/run_by_iter_enumerate.py +++ b/taskflow/examples/run_by_iter_enumerate.py @@ -33,10 +33,10 @@ from taskflow.persistence import backends as persistence_backends from taskflow import task from taskflow.utils import persistence_utils -# INTRO: This examples shows how to run a engine using the engine iteration +# INTRO: These examples show how to run an engine using the engine iteration # capability, in between iterations other activities occur (in this case a # value is output to stdout); but more complicated actions can occur at the -# boundary when a engine yields its current state back to the caller. +# boundary when an engine yields its current state back to the caller. class EchoNameTask(task.Task): diff --git a/taskflow/examples/simple_linear.py b/taskflow/examples/simple_linear.py index 495b9633..daa53ddd 100644 --- a/taskflow/examples/simple_linear.py +++ b/taskflow/examples/simple_linear.py @@ -41,8 +41,8 @@ from taskflow import task # taskflow provides via tasks and flows makes it possible for you to easily at # a later time hook in a persistence layer (and then gain the functionality # that offers) when you decide the complexity of adding that layer in -# is 'worth it' for your applications usage pattern (which certain applications -# may not need). +# is 'worth it' for your application's usage pattern (which certain +# applications may not need). class CallJim(task.Task): diff --git a/taskflow/examples/simple_linear_listening.py b/taskflow/examples/simple_linear_listening.py index d14c82c4..deff63c7 100644 --- a/taskflow/examples/simple_linear_listening.py +++ b/taskflow/examples/simple_linear_listening.py @@ -37,7 +37,7 @@ ANY = notifier.Notifier.ANY # a given ~phone~ number (provided as a function input) in a linear fashion # (one after the other). # -# For a workflow which is serial this shows a extremely simple way +# For a workflow which is serial this shows an extremely simple way # of structuring your tasks (the code that does the work) into a linear # sequence (the flow) and then passing the work off to an engine, with some # initial data to be ran in a reliable manner. @@ -92,7 +92,7 @@ engine = taskflow.engines.load(flow, store={ }) # This is where we attach our callback functions to the 2 different -# notification objects that a engine exposes. The usage of a '*' (kleene star) +# notification objects that an engine exposes. The usage of a '*' (kleene star) # here means that we want to be notified on all state changes, if you want to # restrict to a specific state change, just register that instead. engine.notifier.register(ANY, flow_watch) diff --git a/taskflow/examples/simple_linear_pass.py b/taskflow/examples/simple_linear_pass.py index d378418d..683f251b 100644 --- a/taskflow/examples/simple_linear_pass.py +++ b/taskflow/examples/simple_linear_pass.py @@ -31,7 +31,7 @@ from taskflow import engines from taskflow.patterns import linear_flow from taskflow import task -# INTRO: This examples shows how a task (in a linear/serial workflow) can +# INTRO: This example shows how a task (in a linear/serial workflow) can # produce an output that can be then consumed/used by a downstream task. diff --git a/taskflow/examples/simple_map_reduce.py b/taskflow/examples/simple_map_reduce.py index 3a47fdc1..6476b488 100644 --- a/taskflow/examples/simple_map_reduce.py +++ b/taskflow/examples/simple_map_reduce.py @@ -27,9 +27,9 @@ top_dir = os.path.abspath(os.path.join(os.path.dirname(__file__), sys.path.insert(0, top_dir) sys.path.insert(0, self_dir) -# INTRO: this examples shows a simplistic map/reduce implementation where +# INTRO: These examples show a simplistic map/reduce implementation where # a set of mapper(s) will sum a series of input numbers (in parallel) and -# return there individual summed result. A reducer will then use those +# return their individual summed result. A reducer will then use those # produced values and perform a final summation and this result will then be # printed (and verified to ensure the calculation was as expected). diff --git a/taskflow/examples/timing_listener.py b/taskflow/examples/timing_listener.py index 68d350ff..4e73154d 100644 --- a/taskflow/examples/timing_listener.py +++ b/taskflow/examples/timing_listener.py @@ -36,7 +36,7 @@ from taskflow import task # and have variable run time tasks run and show how the listener will print # out how long those tasks took (when they started and when they finished). # -# This shows how timing metrics can be gathered (or attached onto a engine) +# This shows how timing metrics can be gathered (or attached onto an engine) # after a workflow has been constructed, making it easy to gather metrics # dynamically for situations where this kind of information is applicable (or # even adding this information on at a later point in the future when your diff --git a/taskflow/examples/wbe_event_sender.py b/taskflow/examples/wbe_event_sender.py index e5b075ac..9f9dbd82 100644 --- a/taskflow/examples/wbe_event_sender.py +++ b/taskflow/examples/wbe_event_sender.py @@ -36,10 +36,10 @@ from taskflow.utils import threading_utils ANY = notifier.Notifier.ANY -# INTRO: This examples shows how to use a remote workers event notification +# INTRO: These examples show how to use a remote worker's event notification # attribute to proxy back task event notifications to the controlling process. # -# In this case a simple set of events are triggered by a worker running a +# In this case a simple set of events is triggered by a worker running a # task (simulated to be remote by using a kombu memory transport and threads). # Those events that the 'remote worker' produces will then be proxied back to # the task that the engine is running 'remotely', and then they will be emitted @@ -113,10 +113,10 @@ if __name__ == "__main__": workers = [] # These topics will be used to request worker information on; those - # workers will respond with there capabilities which the executing engine + # workers will respond with their capabilities which the executing engine # will use to match pending tasks to a matched worker, this will cause # the task to be sent for execution, and the engine will wait until it - # is finished (a response is recieved) and then the engine will either + # is finished (a response is received) and then the engine will either # continue with other tasks, do some retry/failure resolution logic or # stop (and potentially re-raise the remote workers failure)... worker_topics = [] diff --git a/taskflow/examples/wbe_mandelbrot.py b/taskflow/examples/wbe_mandelbrot.py index c59b85ce..48db5e65 100644 --- a/taskflow/examples/wbe_mandelbrot.py +++ b/taskflow/examples/wbe_mandelbrot.py @@ -111,11 +111,11 @@ def calculate(engine_conf): # an image bitmap file. # And unordered flow is used here since the mandelbrot calculation is an - # example of a embarrassingly parallel computation that we can scatter + # example of an embarrassingly parallel computation that we can scatter # across as many workers as possible. flow = uf.Flow("mandelbrot") - # These symbols will be automatically given to tasks as input to there + # These symbols will be automatically given to tasks as input to their # execute method, in this case these are constants used in the mandelbrot # calculation. store = { From d33b3165b72346885fc3fee9d757b50e85a53935 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Sun, 15 Mar 2015 09:57:40 -0700 Subject: [PATCH 104/246] Just unify having a single requirements.txt file The difference between the py2 and py3 requirements file is marginal and it does not harm py3 from having additional dependencies that aren't used anyway so just have a single requirements file. Change-Id: I969ea61f68a2c7c2d7fb4d38e6e1d3daa59b0d40 --- requirements-py3.txt | 30 ------------------------ requirements-py2.txt => requirements.txt | 0 tox.ini | 11 +++------ 3 files changed, 3 insertions(+), 38 deletions(-) delete mode 100644 requirements-py3.txt rename requirements-py2.txt => requirements.txt (100%) diff --git a/requirements-py3.txt b/requirements-py3.txt deleted file mode 100644 index d67ad1a9..00000000 --- a/requirements-py3.txt +++ /dev/null @@ -1,30 +0,0 @@ -# The order of packages is significant, because pip processes them in the order -# of appearance. Changing the order has an impact on the overall integration -# process, which may cause wedges in the gate later. - -# See: https://bugs.launchpad.net/pbr/+bug/1384919 for why this is here... -pbr>=0.6,!=0.7,<1.0 - -# Packages needed for using this library. - -# Python 2->3 compatibility library. -six>=1.9.0 - -# Enum library made for <= python 3.3 -enum34 - -# Very nice graph library -networkx>=1.8 - -# Used for backend storage engine loading. -stevedore>=1.3.0,<1.4.0 # Apache-2.0 - -# Used for structured input validation -jsonschema>=2.0.0,<3.0.0 - -# For common utilities -oslo.utils>=1.4.0,<1.5.0 # Apache-2.0 -oslo.serialization>=1.4.0,<1.5.0 # Apache-2.0 - -# For deprecation of things -debtcollector>=0.3.0,<0.4.0 # Apache-2.0 diff --git a/requirements-py2.txt b/requirements.txt similarity index 100% rename from requirements-py2.txt rename to requirements.txt diff --git a/tox.ini b/tox.ini index faa0a753..d87ed9a8 100644 --- a/tox.ini +++ b/tox.ini @@ -16,7 +16,8 @@ envlist = cover, usedevelop = True install_command = pip install {opts} {packages} setenv = VIRTUAL_ENV={envdir} -deps = -r{toxinidir}/test-requirements.txt +deps = -r{toxinidir}/requirements.txt + -r{toxinidir}/test-requirements.txt commands = python setup.py testr --slowest --testr-args='{posargs}' [testenv:docs] @@ -33,7 +34,7 @@ commands = flake8 {posargs} [testenv:pylint] setenv = VIRTUAL_ENV={envdir} -deps = -r{toxinidir}/requirements-py2.txt +deps = {[testenv]deps} pylint==0.26.0 commands = pylint --rcfile=pylintrc taskflow @@ -62,14 +63,12 @@ import_exceptions = six.moves [testenv:py26] basepython = python2.6 deps = {[testenv]deps} - -r{toxinidir}/requirements-py2.txt MySQL-python eventlet>=0.15.1 SQLAlchemy>=0.7.8,<=0.8.99 [testenv:py27] deps = {[testenv]deps} - -r{toxinidir}/requirements-py2.txt MySQL-python eventlet>=0.15.1 SQLAlchemy>=0.7.8,<=0.9.99 @@ -81,26 +80,22 @@ commands = [testenv:py33] deps = {[testenv]deps} - -r{toxinidir}/requirements-py3.txt SQLAlchemy>=0.7.8,<=0.9.99 PyMySQL>=0.6.2 [testenv:py34] deps = {[testenv]deps} - -r{toxinidir}/requirements-py3.txt SQLAlchemy>=0.7.8,<=0.9.99 PyMySQL>=0.6.2 [testenv:py26-sa7-mysql] basepython = python2.6 deps = {[testenv]deps} - -r{toxinidir}/requirements-py2.txt SQLAlchemy>=0.7.8,<=0.7.99 MySQL-python [testenv:py27-sa8-mysql] basepython = python2.7 deps = {[testenv]deps} - -r{toxinidir}/requirements-py2.txt SQLAlchemy>=0.8,<=0.8.99 MySQL-python From f11579bba4e3dc30e24cb2f53cc0f7d11fe2192c Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Tue, 24 Mar 2015 15:43:23 -0700 Subject: [PATCH 105/246] Add in memory filesystem clearing Change-Id: Ibd22eddeec87f660e729099f702f52cdd6b828ea --- taskflow/persistence/backends/impl_memory.py | 4 ++++ .../tests/unit/persistence/test_memory_persistence.py | 11 +++++++++++ 2 files changed, 15 insertions(+) diff --git a/taskflow/persistence/backends/impl_memory.py b/taskflow/persistence/backends/impl_memory.py index 7efe6ec0..a2255006 100644 --- a/taskflow/persistence/backends/impl_memory.py +++ b/taskflow/persistence/backends/impl_memory.py @@ -114,6 +114,10 @@ class FakeFilesystem(object): """Return list of all children of the given path.""" return [node.item for node in self._fetch_node(path)] + def clear(self): + for node in list(self._root.reverse_iter()): + node.disassociate() + def _iter_pieces(self, path, include_root=False): if path == self._root.item: # Check for this directly as the following doesn't work with diff --git a/taskflow/tests/unit/persistence/test_memory_persistence.py b/taskflow/tests/unit/persistence/test_memory_persistence.py index 1c9f451b..717d2be6 100644 --- a/taskflow/tests/unit/persistence/test_memory_persistence.py +++ b/taskflow/tests/unit/persistence/test_memory_persistence.py @@ -84,6 +84,17 @@ class MemoryFilesystemTest(test.TestCase): path += "/" + p self.assertIsNone(fs[path]) + def test_clear(self): + fs = impl_memory.FakeFilesystem() + paths = ['/b', '/c', '/a/b/c'] + for p in paths: + fs.ensure_path(p) + for p in paths: + self.assertIsNone(self._get_item_path(fs, p)) + fs.clear() + for p in paths: + self.assertRaises(exc.NotFound, self._get_item_path, fs, p) + def test_not_found(self): fs = impl_memory.FakeFilesystem() self.assertRaises(exc.NotFound, self._get_item_path, fs, '/c') From de68bc2d7238011e741ed2f759ebcc4839d83e67 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Mon, 23 Mar 2015 16:15:38 -0700 Subject: [PATCH 106/246] Allow providing a node stringify function to tree pformat Allow providing a node to string function that will be activated when the root node pformat routine needs to format a node into a string (for viewing purposes); this allows for subclasses or other users of nodes to provide there own function that they can specialize as they see fit (without duplicating the pformat routine itself). To start use it to print out the node link target in the fake in-memory filesystem when a link is encountered. Change-Id: I33b8f1ec8d30d72ae35971e35f5edd84a9145db7 --- taskflow/persistence/backends/impl_memory.py | 11 ++++++++++- taskflow/types/tree.py | 17 +++++++++++------ 2 files changed, 21 insertions(+), 7 deletions(-) diff --git a/taskflow/persistence/backends/impl_memory.py b/taskflow/persistence/backends/impl_memory.py index 7efe6ec0..b9733dd6 100644 --- a/taskflow/persistence/backends/impl_memory.py +++ b/taskflow/persistence/backends/impl_memory.py @@ -19,6 +19,8 @@ import contextlib import copy import posixpath as pp +import six + from taskflow import exceptions as exc from taskflow.persistence import path_based from taskflow.types import tree @@ -136,9 +138,16 @@ class FakeFilesystem(object): raise ValueError("Can not delete '%s'" % self._root.item) node.disassociate() + @staticmethod + def _stringify_node(node): + if 'target' in node.metadata: + return "%s (link to %s)" % (node.item, node.metadata['target']) + else: + return six.text_type(node.item) + def pformat(self): """Pretty format this in-memory filesystem.""" - return self._root.pformat() + return self._root.pformat(stringify_node=self._stringify_node) def symlink(self, src_path, dest_path): """Link the destionation path to the source path.""" diff --git a/taskflow/types/tree.py b/taskflow/types/tree.py index 97059d64..4faa8291 100644 --- a/taskflow/types/tree.py +++ b/taskflow/types/tree.py @@ -200,7 +200,7 @@ class Node(object): # NOTE(harlowja): 0 is the right most index, len - 1 is the left most return self._children[index] - def pformat(self): + def pformat(self, stringify_node=None): """Recursively formats a node into a nice string representation. **Example**:: @@ -220,24 +220,29 @@ class Node(object): |__Mobile |__Mail """ - def _inner_pformat(node, level): + def _inner_pformat(node, level, stringify_node): if level == 0: - yield six.text_type(node.item) + yield stringify_node(node) prefix = self.STARTING_PREFIX else: - yield self.HORIZONTAL_CONN + six.text_type(node.item) + yield self.HORIZONTAL_CONN + stringify_node(node) prefix = self.EMPTY_SPACE_SEP * len(self.HORIZONTAL_CONN) child_count = node.child_count() for (i, child) in enumerate(node): - for (j, text) in enumerate(_inner_pformat(child, level + 1)): + for (j, text) in enumerate(_inner_pformat(child, + level + 1, + stringify_node)): if j == 0 or i + 1 < child_count: text = prefix + self.VERTICAL_CONN + text else: text = prefix + self.EMPTY_SPACE_SEP + text yield text + if stringify_node is None: + # Default to making a unicode string out of the nodes item... + stringify_node = lambda node: six.text_type(node.item) expected_lines = self.child_count(only_direct=False) accumulator = six.StringIO() - for i, line in enumerate(_inner_pformat(self, 0)): + for i, line in enumerate(_inner_pformat(self, 0, stringify_node)): accumulator.write(line) if i < expected_lines: accumulator.write(self.LINE_SEP) From 3204242790060de4b5ea1e0c559a13825d8d9a92 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Mon, 30 Mar 2015 08:28:00 -0700 Subject: [PATCH 107/246] Remove 11635 bug reference This bug is not really related directly to the multiprocessing futures executor so it doesn't seem as appropriate to include in the bug list. Change-Id: Iab2d5c655019740a17e9856e4b309eb5279a8587 --- doc/source/engines.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/doc/source/engines.rst b/doc/source/engines.rst index 5ce22903..988d111f 100644 --- a/doc/source/engines.rst +++ b/doc/source/engines.rst @@ -178,7 +178,7 @@ using your desired execution model. supported. This is mainly due to the `futures backport`_ and the `multiprocessing`_ module that exist in older versions of python not being as up to date (with important fixes such as :pybug:`4892`, - :pybug:`6721`, :pybug:`9205`, :pybug:`11635`, :pybug:`16284`, + :pybug:`6721`, :pybug:`9205`, :pybug:`16284`, :pybug:`22393` and others...) as the most recent python version (which themselves have a variety of ongoing/recent bugs). From bb0af4f81d966d3178f9eb4f3fa65f0bc5dbadf3 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Mon, 30 Mar 2015 12:27:24 -0700 Subject: [PATCH 108/246] Use 'node' terminology instead of 'item' terminology Since graphs are composed of 'nodes' it seems more appropriate to use that terminology (which is known to people with graph experience) instead of the non descriptive and/or standard 'item' terminology. Change-Id: I04a4521386d3cdf7e58fb9fa8cf26c00443c2cf6 --- taskflow/patterns/graph_flow.py | 91 ++++++++++--------- .../tests/unit/patterns/test_graph_flow.py | 6 +- 2 files changed, 50 insertions(+), 47 deletions(-) diff --git a/taskflow/patterns/graph_flow.py b/taskflow/patterns/graph_flow.py index bb779cb2..7d407c8c 100644 --- a/taskflow/patterns/graph_flow.py +++ b/taskflow/patterns/graph_flow.py @@ -69,9 +69,9 @@ class Flow(flow.Flow): def link(self, u, v): """Link existing node u as a runtime dependency of existing node v.""" if not self._graph.has_node(u): - raise ValueError('Item %s not found to link from' % (u)) + raise ValueError("Node '%s' not found to link from" % (u)) if not self._graph.has_node(v): - raise ValueError('Item %s not found to link to' % (v)) + raise ValueError("Node '%s' not found to link to" % (v)) self._swap(self._link(u, v, manual=True)) return self @@ -105,34 +105,38 @@ class Flow(flow.Flow): direct access to the underlying graph). """ if not graph.is_directed_acyclic(): - raise exc.DependencyFailure("No path through the items in the" + raise exc.DependencyFailure("No path through the node(s) in the" " graph produces an ordering that" " will allow for logical" " edge traversal") self._graph = graph.freeze() - def add(self, *items, **kwargs): + def add(self, *nodes, **kwargs): """Adds a given task/tasks/flow/flows to this flow. - :param items: items to add to the flow + :param nodes: node(s) to add to the flow :param kwargs: keyword arguments, the two keyword arguments currently processed are: * ``resolve_requires`` a boolean that when true (the - default) implies that when items are added their - symbol requirements will be matched to existing items - and links will be automatically made to those + default) implies that when node(s) are added their + symbol requirements will be matched to existing + node(s) and links will be automatically made to those providers. If multiple possible providers exist then a AmbiguousDependency exception will be raised. * ``resolve_existing``, a boolean that when true (the - default) implies that on addition of a new item that - existing items will have their requirements scanned - for symbols that this newly added item can provide. + default) implies that on addition of a new node that + existing node(s) will have their requirements scanned + for symbols that this newly added node can provide. If a match is found a link is automatically created - from the newly added item to the requiree. + from the newly added node to the requiree. """ - items = [i for i in items if not self._graph.has_node(i)] - if not items: + + # Let's try to avoid doing any work if we can; since the below code + # after this filter can create more temporary graphs that aren't needed + # if the nodes already exist... + nodes = [i for i in nodes if not self._graph.has_node(i)] + if not nodes: return self # This syntax will *hopefully* be better in future versions of python. @@ -154,23 +158,23 @@ class Flow(flow.Flow): retry_provides.add(value) provided[value].append(self._retry) - for item in self._graph.nodes_iter(): - for value in self._unsatisfied_requires(item, self._graph, + for node in self._graph.nodes_iter(): + for value in self._unsatisfied_requires(node, self._graph, retry_provides): - required[value].append(item) - for value in item.provides: - provided[value].append(item) + required[value].append(node) + for value in node.provides: + provided[value].append(node) - # NOTE(harlowja): Add items and edges to a temporary copy of the + # NOTE(harlowja): Add node(s) and edge(s) to a temporary copy of the # underlying graph and only if that is successful added to do we then # swap with the underlying graph. tmp_graph = gr.DiGraph(self._graph) - for item in items: - tmp_graph.add_node(item) + for node in nodes: + tmp_graph.add_node(node) # Try to find a valid provider. if resolve_requires: - for value in self._unsatisfied_requires(item, tmp_graph, + for value in self._unsatisfied_requires(node, tmp_graph, retry_provides): if value in provided: providers = provided[value] @@ -178,28 +182,28 @@ class Flow(flow.Flow): provider_names = [n.name for n in providers] raise exc.AmbiguousDependency( "Resolution error detected when" - " adding %(item)s, multiple" + " adding '%(node)s', multiple" " providers %(providers)s found for" " required symbol '%(value)s'" - % dict(item=item.name, + % dict(node=node.name, providers=sorted(provider_names), value=value)) else: - self._link(providers[0], item, + self._link(providers[0], node, graph=tmp_graph, reason=value) else: - required[value].append(item) + required[value].append(node) - for value in item.provides: - provided[value].append(item) + for value in node.provides: + provided[value].append(node) # See if what we provide fulfills any existing requiree. if resolve_existing: - for value in item.provides: + for value in node.provides: if value in required: for requiree in list(required[value]): - if requiree is not item: - self._link(item, requiree, + if requiree is not node: + self._link(node, requiree, graph=tmp_graph, reason=value) required[value].remove(requiree) @@ -233,8 +237,8 @@ class Flow(flow.Flow): requires.update(self._retry.requires) retry_provides.update(self._retry.provides) g = self._get_subgraph() - for item in g.nodes_iter(): - requires.update(self._unsatisfied_requires(item, g, + for node in g.nodes_iter(): + requires.update(self._unsatisfied_requires(node, g, retry_provides)) return frozenset(requires) @@ -251,29 +255,28 @@ class TargetedFlow(Flow): self._subgraph = None self._target = None - def set_target(self, target_item): + def set_target(self, target_node): """Set target for the flow. - Any items (tasks or subflows) not needed for the target - item will not be executed. + Any node(s) (tasks or subflows) not needed for the target + node will not be executed. """ - if not self._graph.has_node(target_item): - raise ValueError('Item %s not found' % target_item) - self._target = target_item + if not self._graph.has_node(target_node): + raise ValueError("Node '%s' not found" % target_node) + self._target = target_node self._subgraph = None def reset_target(self): """Reset target for the flow. - All items of the flow will be executed. + All node(s) of the flow will be executed. """ - self._target = None self._subgraph = None - def add(self, *items): + def add(self, *nodes): """Adds a given task/tasks/flow/flows to this flow.""" - super(TargetedFlow, self).add(*items) + super(TargetedFlow, self).add(*nodes) # reset cached subgraph, in case it was affected self._subgraph = None return self diff --git a/taskflow/tests/unit/patterns/test_graph_flow.py b/taskflow/tests/unit/patterns/test_graph_flow.py index 8dad447d..588361c9 100644 --- a/taskflow/tests/unit/patterns/test_graph_flow.py +++ b/taskflow/tests/unit/patterns/test_graph_flow.py @@ -182,14 +182,14 @@ class GraphFlowTest(test.TestCase): task1 = _task('task1') task2 = _task('task2') f = gf.Flow('test').add(task2) - self.assertRaisesRegexp(ValueError, 'Item .* not found to link from', + self.assertRaisesRegexp(ValueError, 'Node .* not found to link from', f.link, task1, task2) def test_graph_flow_link_to_unknown_node(self): task1 = _task('task1') task2 = _task('task2') f = gf.Flow('test').add(task1) - self.assertRaisesRegexp(ValueError, 'Item .* not found to link to', + self.assertRaisesRegexp(ValueError, 'Node .* not found to link to', f.link, task1, task2) def test_graph_flow_link_raises_on_cycle(self): @@ -245,7 +245,7 @@ class TargetedGraphFlowTest(test.TestCase): task1 = _task('task1', provides=['a'], requires=[]) task2 = _task('task2', provides=['b'], requires=['a']) f.add(task1) - self.assertRaisesRegexp(ValueError, '^Item .* not found', + self.assertRaisesRegexp(ValueError, '^Node .* not found', f.set_target, task2) def test_targeted_flow_one_node(self): From 1714cca97583b91b22449d2e18401dfb08f7adfa Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Mon, 30 Mar 2015 14:55:04 -0700 Subject: [PATCH 109/246] Turn 'check_who' into a decorator This helper function seems to be more pretty/nice as a decorator that is used uniformly instead of repeated calling into it from needed functions. Change-Id: If8aa2fb914cf80fdcecc37f044ab129d13f12be8 --- taskflow/jobs/backends/impl_zookeeper.py | 25 +++++++++++++++--------- 1 file changed, 16 insertions(+), 9 deletions(-) diff --git a/taskflow/jobs/backends/impl_zookeeper.py b/taskflow/jobs/backends/impl_zookeeper.py index 55c7b8eb..34ec5b42 100644 --- a/taskflow/jobs/backends/impl_zookeeper.py +++ b/taskflow/jobs/backends/impl_zookeeper.py @@ -55,11 +55,18 @@ TRASH_FOLDER = ".trash" JOB_PREFIX = 'job' -def _check_who(who): - if not isinstance(who, six.string_types): - raise TypeError("Job applicant must be a string type") - if len(who) == 0: - raise ValueError("Job applicant must be non-empty") +def check_who(meth): + """Decorator that checks the expected owner type & value restrictions.""" + + @six.wraps(meth) + def wrapper(self, job, who, *args, **kwargs): + if not isinstance(who, six.string_types): + raise TypeError("Job applicant must be a string type") + if len(who) == 0: + raise ValueError("Job applicant must be non-empty") + return meth(self, job, who, *args, **kwargs) + + return wrapper class ZookeeperJob(base.Job): @@ -535,6 +542,7 @@ class ZookeeperJobBoard(base.NotifyingJobBoard): self._emit(base.POSTED, details={'job': job}) return job + @check_who def claim(self, job, who): def _unclaimable_try_find_owner(cause): try: @@ -547,7 +555,6 @@ class ZookeeperJobBoard(base.NotifyingJobBoard): msg = "Job %s already claimed" % (job.uuid) return excp.UnclaimableJob(msg, cause) - _check_who(who) with self._wrap(job.uuid, job.path, "Claiming failure: %s"): # NOTE(harlowja): post as json which will allow for future changes # more easily than a raw string/text. @@ -626,8 +633,8 @@ class ZookeeperJobBoard(base.NotifyingJobBoard): return (misc.decode_json(lock_data), lock_stat, misc.decode_json(job_data), job_stat) + @check_who def consume(self, job, who): - _check_who(who) with self._wrap(job.uuid, job.path, "Consumption failure: %s"): try: owner_data = self._get_owner_and_data(job) @@ -646,8 +653,8 @@ class ZookeeperJobBoard(base.NotifyingJobBoard): kazoo_utils.checked_commit(txn) self._remove_job(job.path) + @check_who def abandon(self, job, who): - _check_who(who) with self._wrap(job.uuid, job.path, "Abandonment failure: %s"): try: owner_data = self._get_owner_and_data(job) @@ -664,8 +671,8 @@ class ZookeeperJobBoard(base.NotifyingJobBoard): txn.delete(job.lock_path, version=lock_stat.version) kazoo_utils.checked_commit(txn) + @check_who def trash(self, job, who): - _check_who(who) with self._wrap(job.uuid, job.path, "Trash failure: %s"): try: owner_data = self._get_owner_and_data(job) From 918ff7e14454dcbf1c6edeb12da2b6fc7365c7bc Mon Sep 17 00:00:00 2001 From: Greg Hill Date: Mon, 30 Mar 2015 14:41:13 -0500 Subject: [PATCH 110/246] fix renamed class to call super correctly Should prevent this exception from always being thrown: TypeError: super(type, obj): obj must be an instance or subtype of type Change-Id: I074cbc8cadb1d1d087609f82a6486005b63f3593 --- taskflow/listeners/timing.py | 2 +- taskflow/tests/unit/test_listeners.py | 11 +++++++++++ 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/taskflow/listeners/timing.py b/taskflow/listeners/timing.py index 17d289d6..e346f08f 100644 --- a/taskflow/listeners/timing.py +++ b/taskflow/listeners/timing.py @@ -56,7 +56,7 @@ class DurationListener(base.Listener): self._timers = {} def deregister(self): - super(TimingListener, self).deregister() + super(DurationListener, self).deregister() # There should be none that still exist at deregistering time, so log a # warning if there were any that somehow still got left behind... leftover_timers = len(self._timers) diff --git a/taskflow/tests/unit/test_listeners.py b/taskflow/tests/unit/test_listeners.py index d5955106..13aa9503 100644 --- a/taskflow/tests/unit/test_listeners.py +++ b/taskflow/tests/unit/test_listeners.py @@ -203,6 +203,17 @@ class TestClaimListener(test.TestCase, EngineMakerMixin): class TestDurationListener(test.TestCase, EngineMakerMixin): + def test_deregister(self): + """Verify that register and deregister don't blow up""" + with contextlib.closing(impl_memory.MemoryBackend()) as be: + flow = lf.Flow("test") + flow.add(SleepyTask("test-1", sleep_for=0.1)) + (lb, fd) = persistence_utils.temporary_flow_detail(be) + e = self._make_engine(flow, fd, be) + l = timing.DurationListener(e) + l.register() + l.deregister() + def test_duration(self): with contextlib.closing(impl_memory.MemoryBackend()) as be: flow = lf.Flow("test") From d16e111b536f60d3b455b05d20a1849f62a77e8e Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Mon, 23 Mar 2015 20:44:50 -0700 Subject: [PATCH 111/246] Make an attempt at having taskflow exceptions print causes better It is often quite useful to try to see what the contained causes are on versions of python that do not have the native support for this built-in so to make everyones life easier add basic support for traversing the cause list and printing out associated causes when we are able to. Change-Id: Ia0a7e13757a989722291bcc06599d04014706d8c --- taskflow/exceptions.py | 71 ++++++++++++++++---------- taskflow/tests/unit/test_exceptions.py | 38 ++++++++++++++ 2 files changed, 82 insertions(+), 27 deletions(-) diff --git a/taskflow/exceptions.py b/taskflow/exceptions.py index ae7baef4..069a2f2a 100644 --- a/taskflow/exceptions.py +++ b/taskflow/exceptions.py @@ -18,6 +18,7 @@ import os import sys import traceback +from oslo_utils import reflection import six @@ -76,35 +77,51 @@ class TaskFlowException(Exception): def cause(self): return self._cause - def pformat(self, indent=2, indent_text=" "): + def __str__(self): + return self.pformat() + + def _get_message(self): + # We must *not* call into the __str__ method as that will reactivate + # the pformat method, which will end up badly (and doesn't look + # pretty at all); so be careful... + return self.args[0] + + def pformat(self, indent=2, indent_text=" ", show_root_class=False): """Pretty formats a taskflow exception + any connected causes.""" if indent < 0: - raise ValueError("indent must be greater than or equal to zero") - return os.linesep.join(self._pformat(self, [], 0, - indent=indent, - indent_text=indent_text)) - - @classmethod - def _pformat(cls, excp, lines, current_indent, indent=2, indent_text=" "): - line_prefix = indent_text * current_indent - for line in traceback.format_exception_only(type(excp), excp): - # We'll add our own newlines on at the end of formatting. - # - # NOTE(harlowja): the reason we don't search for os.linesep is - # that the traceback module seems to only use '\n' (for some - # reason). - if line.endswith("\n"): - line = line[0:-1] - lines.append(line_prefix + line) - try: - cause = excp.cause - except AttributeError: - pass - else: - if cause is not None: - cls._pformat(cause, lines, current_indent + indent, - indent=indent, indent_text=indent_text) - return lines + raise ValueError("Provided 'indent' must be greater than" + " or equal to zero instead of %s" % indent) + buf = six.StringIO() + if show_root_class: + buf.write(reflection.get_class_name(self, fully_qualified=False)) + buf.write(": ") + buf.write(self._get_message()) + active_indent = indent + next_up = self.cause + while next_up is not None: + buf.write(os.linesep) + if isinstance(next_up, TaskFlowException): + buf.write(indent_text * active_indent) + buf.write(reflection.get_class_name(next_up, + fully_qualified=False)) + buf.write(": ") + buf.write(next_up._get_message()) + else: + lines = traceback.format_exception_only(type(next_up), next_up) + for i, line in enumerate(lines): + buf.write(indent_text * active_indent) + if line.endswith("\n"): + # We'll add our own newlines on... + line = line[0:-1] + buf.write(line) + if i + 1 != len(lines): + buf.write(os.linesep) + active_indent += indent + try: + next_up = next_up.cause + except AttributeError: + next_up = None + return buf.getvalue() # Errors related to storage or operations on storage units. diff --git a/taskflow/tests/unit/test_exceptions.py b/taskflow/tests/unit/test_exceptions.py index d834ce7c..b3590653 100644 --- a/taskflow/tests/unit/test_exceptions.py +++ b/taskflow/tests/unit/test_exceptions.py @@ -56,6 +56,44 @@ class TestExceptions(test.TestCase): self.assertIsNotNone(capture.cause) self.assertIsInstance(capture.cause, IOError) + def test_pformat_str(self): + ex = None + try: + try: + try: + raise IOError("Didn't work") + except IOError: + exc.raise_with_cause(exc.TaskFlowException, + "It didn't go so well") + except exc.TaskFlowException: + exc.raise_with_cause(exc.TaskFlowException, "I Failed") + except exc.TaskFlowException as e: + ex = e + + self.assertIsNotNone(ex) + self.assertIsInstance(ex, exc.TaskFlowException) + self.assertIsInstance(ex.cause, exc.TaskFlowException) + self.assertIsInstance(ex.cause.cause, IOError) + + p_msg = ex.pformat() + p_str_msg = str(ex) + for msg in ["I Failed", "It didn't go so well", "Didn't work"]: + self.assertIn(msg, p_msg) + self.assertIn(msg, p_str_msg) + + def test_pformat_root_class(self): + ex = exc.TaskFlowException("Broken") + self.assertIn("TaskFlowException", + ex.pformat(show_root_class=True)) + self.assertNotIn("TaskFlowException", + ex.pformat(show_root_class=False)) + self.assertIn("Broken", + ex.pformat(show_root_class=True)) + + def test_invalid_pformat_indent(self): + ex = exc.TaskFlowException("Broken") + self.assertRaises(ValueError, ex.pformat, indent=-100) + @testtools.skipIf(not six.PY3, 'py3.x is not available') def test_raise_with_cause(self): capture = None From ce7b1d2c92e901f213f71c4c3e4d9c5f2145ff70 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Tue, 24 Mar 2015 11:39:42 -0700 Subject: [PATCH 112/246] Allow ls() to list recursively (using breadth-first) Make it easier to list the contents of the full in-memory filesystem by allowing the ls() function to take a recursive keyword argument (which by default is false). This makes it easier to analyze the contents of the full in memory filesystem; saving people from creating similar code themselves. Change-Id: I573797945255cb81728e7a86b58768b848110ee7 --- taskflow/examples/dump_memory_backend.py | 7 +---- taskflow/persistence/backends/impl_memory.py | 20 +++++++++++-- .../persistence/test_memory_persistence.py | 29 +++++++++++++++++++ 3 files changed, 48 insertions(+), 8 deletions(-) diff --git a/taskflow/examples/dump_memory_backend.py b/taskflow/examples/dump_memory_backend.py index 7209dda0..2e3aee71 100644 --- a/taskflow/examples/dump_memory_backend.py +++ b/taskflow/examples/dump_memory_backend.py @@ -70,14 +70,9 @@ e.run() print("---------") print("After run") print("---------") -entries = [os.path.join(backend.memory.root_path, child) - for child in backend.memory.ls(backend.memory.root_path)] -while entries: - path = entries.pop() +for path in backend.memory.ls(backend.memory.root_path, recursive=True): value = backend.memory[path] if value: print("%s -> %s" % (path, value)) else: print("%s" % (path)) - entries.extend(os.path.join(path, child) - for child in backend.memory.ls(path)) diff --git a/taskflow/persistence/backends/impl_memory.py b/taskflow/persistence/backends/impl_memory.py index 7efe6ec0..778eb97a 100644 --- a/taskflow/persistence/backends/impl_memory.py +++ b/taskflow/persistence/backends/impl_memory.py @@ -110,9 +110,25 @@ class FakeFilesystem(object): else: return self._copier(node.metadata['value']) - def ls(self, path): + def ls(self, path, recursive=False): """Return list of all children of the given path.""" - return [node.item for node in self._fetch_node(path)] + if not recursive: + return [node.item for node in self._fetch_node(path)] + else: + paths = [] + node = self._fetch_node(path) + for child in node.bfs_iter(): + # Reconstruct the child's path... + hops = [child.item] + for parent in child.path_iter(include_self=False): + hops.append(parent.item) + hops.reverse() + # This avoids getting '//a/b' (duplicated sep at start)... + child_path = pp.sep.join(hops) + if child_path.startswith("//"): + child_path = child_path[1:] + paths.append(child_path) + return paths def _iter_pieces(self, path, include_root=False): if path == self._root.item: diff --git a/taskflow/tests/unit/persistence/test_memory_persistence.py b/taskflow/tests/unit/persistence/test_memory_persistence.py index 1c9f451b..069e88de 100644 --- a/taskflow/tests/unit/persistence/test_memory_persistence.py +++ b/taskflow/tests/unit/persistence/test_memory_persistence.py @@ -71,6 +71,35 @@ class MemoryFilesystemTest(test.TestCase): self.assertEqual('c', fs['/c']) self.assertEqual('db', fs['/d/b']) + def test_ls_recursive(self): + fs = impl_memory.FakeFilesystem() + fs.ensure_path("/d") + fs.ensure_path("/c/d") + fs.ensure_path("/b/c/d") + fs.ensure_path("/a/b/c/d") + contents = fs.ls("/", recursive=True) + self.assertEqual([ + '/a', + '/b', + '/c', + '/d', + '/a/b', + '/b/c', + '/c/d', + '/a/b/c', + '/b/c/d', + '/a/b/c/d', + ], contents) + + def test_ls_recursive_targeted(self): + fs = impl_memory.FakeFilesystem() + fs.ensure_path("/d") + fs.ensure_path("/c/d") + fs.ensure_path("/b/c/d") + fs.ensure_path("/a/b/c/d") + contents = fs.ls("/a/b", recursive=True) + self.assertEqual(['/a/b/c', '/a/b/c/d'], contents) + def test_ensure_path(self): fs = impl_memory.FakeFilesystem() pieces = ['a', 'b', 'c'] From 57d2fff654d5de2c5846b3db80cb93035c71ab76 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Tue, 31 Mar 2015 18:15:01 -0700 Subject: [PATCH 113/246] Switch back to maxdepth 2 The current table of contents at maxdepth 3 is getting a little to big to be easily findable and readable so let's switch it back to maxdepth 2 which is better. Change-Id: I5afdb4396d407e807084196d782111bd8102936b --- doc/source/index.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/doc/source/index.rst b/doc/source/index.rst index 52b7c7a8..d136c16f 100644 --- a/doc/source/index.rst +++ b/doc/source/index.rst @@ -14,7 +14,7 @@ Contents ======== .. toctree:: - :maxdepth: 3 + :maxdepth: 2 atoms arguments_and_results From f7d81c740d1e4510d2c2c2c79bd4f2689a339780 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Tue, 31 Mar 2015 15:45:18 -0700 Subject: [PATCH 114/246] Add a suspension engine section To make it more clear as to what the suspend() method does add a special case section in the engine docs that describes exactly (at a somewhat high-level) what it is doing (and how it does it). Change-Id: If2fbba744e460e99d4bb7da58809a27ca7dd37d6 --- doc/source/engines.rst | 37 ++++++++++++++++++++++++++++++++++--- 1 file changed, 34 insertions(+), 3 deletions(-) diff --git a/doc/source/engines.rst b/doc/source/engines.rst index 3c3d757b..5225f96f 100644 --- a/doc/source/engines.rst +++ b/doc/source/engines.rst @@ -327,7 +327,7 @@ categories, one for if that atom failed and one for if it did not. If the atom failed it may be set to a new intention such as ``RETRY`` or ``REVERT`` (other atoms that were predecessors of this failing atom may also have there intention altered). Once this intention adjustment has happened a -new round of :ref:`scheduling ` occurs and this process repeats +new round of :ref:`scheduling ` occurs and this process repeats until the engine succeeds or fails (if the process running the engine dies the above stages will be restarted and resuming will occur). @@ -335,8 +335,8 @@ above stages will be restarted and resuming will occur). If the engine is suspended while the engine is going through the above stages this will stop any further scheduling stages from occurring and - all currently executing atoms will be allowed to finish (and their results - will be saved). + all currently executing work will be allowed to finish (see + :ref:`suspension `). Finishing --------- @@ -353,6 +353,37 @@ failures have occurred then the engine will have finished and if so desired the :doc:`persistence ` can be used to cleanup any details that were saved for this execution. +Special cases +============= + +.. _suspension: + +Suspension +---------- + +Each engine implements a :py:func:`~taskflow.engines.base.Engine.suspend` +method that can be used to *externally* (or in the future *internally*) request +that the engine stop :ref:`scheduling ` new work. By default what +this performs is a transition of the flow state from ``RUNNING`` into a +``SUSPENDING`` state (which will later transition into a ``SUSPENDED`` state). +Since an engine may be remotely executing atoms (or locally executing them) +and there is currently no preemption what occurs is that the engines +:py:class:`~taskflow.engines.action_engine.runner.Runner` state machine will +detect this transition into ``SUSPENDING`` has occurred and the state +machine will avoid scheduling new work (it will though let active work +continue). After the current work has finished the engine will +transition from ``SUSPENDING`` into ``SUSPENDED`` and return from its +:py:func:`~taskflow.engines.base.Engine.run` method. + + +.. note:: + + When :py:func:`~taskflow.engines.base.Engine.run` is returned from at that + point there *may* (but does not have to be, depending on what was active + when :py:func:`~taskflow.engines.base.Engine.suspend` was called) be + unfinished work in the flow that was not finished (but which can be + resumed at a later point in time). + Scoping ======= From 254d3f97154699a141cdbb8a2f6712321546ee66 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 1 Apr 2015 14:29:17 -0700 Subject: [PATCH 115/246] Put the examples/misc/considerations under a new section Change-Id: I36c36b0ce0407760db0344f509471bb35dca03f3 --- doc/source/index.rst | 3 +++ 1 file changed, 3 insertions(+) diff --git a/doc/source/index.rst b/doc/source/index.rst index d136c16f..2182346b 100644 --- a/doc/source/index.rst +++ b/doc/source/index.rst @@ -29,6 +29,9 @@ Contents jobs conductors +Supplementary +============= + Examples -------- From 227cf52319f14306ddd89a2bef42ac264210ac2e Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Mon, 23 Mar 2015 12:39:15 -0700 Subject: [PATCH 116/246] Make the storage layer more resilent to failures When a storage failure happens the internally maintained flow detail or atom details should not be updated; to ensure this happens update a cloned atom/flow detail first and only if the save succeeds do we update the internal version (and perform any futher work). Change-Id: Ib8da476b6be2c71b663e0fc659b5df24e6b9f095 --- taskflow/storage.py | 227 ++++++++++++++++++++++++-------------------- 1 file changed, 125 insertions(+), 102 deletions(-) diff --git a/taskflow/storage.py b/taskflow/storage.py index fe601c9a..83311e92 100644 --- a/taskflow/storage.py +++ b/taskflow/storage.py @@ -170,22 +170,21 @@ class Storage(object): for ad in self._flowdetail) try: - injector_td = self._atomdetail_by_name( - self.injector_name, - expected_type=logbook.TaskDetail) + source, _clone = self._atomdetail_by_name( + self.injector_name, expected_type=logbook.TaskDetail) except exceptions.NotFound: pass else: - names = six.iterkeys(injector_td.results) - self._set_result_mapping(injector_td.name, - dict((name, name) for name in names)) + names_iter = six.iterkeys(source.results) + self._set_result_mapping(source.name, + dict((name, name) for name in names_iter)) def _with_connection(self, functor, *args, **kwargs): # Run the given functor with a backend connection as its first # argument (providing the additional positional arguments and keyword # arguments as subsequent arguments). with contextlib.closing(self._backend.get_connection()) as conn: - functor(conn, *args, **kwargs) + return functor(conn, *args, **kwargs) def ensure_atom(self, atom): """Ensure that there is an atomdetail in storage for the given atom. @@ -258,18 +257,23 @@ class Storage(object): self._set_result_mapping(retry_name, result_mapping) return retry_id - def _create_atom_detail(self, _detail_cls, name, uuid, task_version=None): + def _create_atom_detail(self, detail_cls, name, uuid, task_version=None): """Add the atom detail to flow detail. Atom becomes known to storage by that name and uuid. Atom state is set to PENDING. """ - ad = _detail_cls(name, uuid) + ad = detail_cls(name, uuid) ad.state = states.PENDING ad.version = task_version - self._flowdetail.add(ad) - self._with_connection(self._save_flow_detail) + # Add the atom detail to the clone, which upon success will be + # updated into the contained flow detail; if it does not get saved + # then no update will happen. + source, clone = self._fetch_flowdetail(clone=True) + clone.add(ad) + self._with_connection(self._save_flow_detail, source, clone) self._atom_name_to_uuid[ad.name] = ad.uuid + return ad @property def flow_name(self): @@ -289,13 +293,21 @@ class Storage(object): # This never changes (so no read locking needed). return self._backend - def _save_flow_detail(self, conn): + def _save_flow_detail(self, conn, original_flow_detail, flow_detail): # NOTE(harlowja): we need to update our contained flow detail if # the result of the update actually added more (aka another process # added item to the flow detail). - self._flowdetail.update(conn.update_flow_details(self._flowdetail)) + original_flow_detail.update(conn.update_flow_details(flow_detail)) + return original_flow_detail - def _atomdetail_by_name(self, atom_name, expected_type=None): + def _fetch_flowdetail(self, clone=False): + source = self._flowdetail + if clone: + return (source, source.copy()) + else: + return (source, source) + + def _atomdetail_by_name(self, atom_name, expected_type=None, clone=False): try: ad = self._flowdetail.find(self._atom_name_to_uuid[atom_name]) except KeyError: @@ -308,47 +320,51 @@ class Storage(object): raise TypeError("Atom %s is not of the expected type: %s" % (atom_name, reflection.get_class_name(expected_type))) - return ad + if clone: + return (ad, ad.copy()) + else: + return (ad, ad) - def _save_atom_detail(self, conn, atom_detail): + def _save_atom_detail(self, conn, original_atom_detail, atom_detail): # NOTE(harlowja): we need to update our contained atom detail if # the result of the update actually added more (aka another process # is also modifying the task detail), since python is by reference # and the contained atom detail will reflect the old state if we don't # do this update. - atom_detail.update(conn.update_atom_details(atom_detail)) + original_atom_detail.update(conn.update_atom_details(atom_detail)) + return original_atom_detail @lock_utils.read_locked def get_atom_uuid(self, atom_name): """Gets an atoms uuid given a atoms name.""" - ad = self._atomdetail_by_name(atom_name) - return ad.uuid + source, _clone = self._atomdetail_by_name(atom_name) + return source.uuid @lock_utils.write_locked def set_atom_state(self, atom_name, state): """Sets an atoms state.""" - ad = self._atomdetail_by_name(atom_name) - ad.state = state - self._with_connection(self._save_atom_detail, ad) + source, clone = self._atomdetail_by_name(atom_name, clone=True) + clone.state = state + self._with_connection(self._save_atom_detail, source, clone) @lock_utils.read_locked def get_atom_state(self, atom_name): """Gets the state of an atom given an atoms name.""" - ad = self._atomdetail_by_name(atom_name) - return ad.state + source, _clone = self._atomdetail_by_name(atom_name) + return source.state @lock_utils.write_locked def set_atom_intention(self, atom_name, intention): """Sets the intention of an atom given an atoms name.""" - ad = self._atomdetail_by_name(atom_name) - ad.intention = intention - self._with_connection(self._save_atom_detail, ad) + source, clone = self._atomdetail_by_name(atom_name, clone=True) + clone.intention = intention + self._with_connection(self._save_atom_detail, source, clone) @lock_utils.read_locked def get_atom_intention(self, atom_name): """Gets the intention of an atom given an atoms name.""" - ad = self._atomdetail_by_name(atom_name) - return ad.intention + source, _clone = self._atomdetail_by_name(atom_name) + return source.intention @lock_utils.read_locked def get_atoms_states(self, atom_names): @@ -360,11 +376,12 @@ class Storage(object): @lock_utils.write_locked def _update_atom_metadata(self, atom_name, update_with, expected_type=None): - ad = self._atomdetail_by_name(atom_name, - expected_type=expected_type) + source, clone = self._atomdetail_by_name(atom_name, + expected_type=expected_type, + clone=True) if update_with: - ad.meta.update(update_with) - self._with_connection(self._save_atom_detail, ad) + clone.meta.update(update_with) + self._with_connection(self._save_atom_detail, source, clone) def update_atom_metadata(self, atom_name, update_with): """Updates a atoms associated metadata. @@ -407,10 +424,10 @@ class Storage(object): :param task_name: tasks name :returns: current task progress value """ - ad = self._atomdetail_by_name(task_name, - expected_type=logbook.TaskDetail) + source, _clone = self._atomdetail_by_name( + task_name, expected_type=logbook.TaskDetail) try: - return ad.meta[META_PROGRESS] + return source.meta[META_PROGRESS] except KeyError: return 0.0 @@ -422,10 +439,10 @@ class Storage(object): :returns: None if progress_details not defined, else progress_details dict """ - ad = self._atomdetail_by_name(task_name, - expected_type=logbook.TaskDetail) + source, _clone = self._atomdetail_by_name( + task_name, expected_type=logbook.TaskDetail) try: - return ad.meta[META_PROGRESS_DETAILS] + return source.meta[META_PROGRESS_DETAILS] except KeyError: return None @@ -449,23 +466,24 @@ class Storage(object): @lock_utils.write_locked def save(self, atom_name, data, state=states.SUCCESS): """Put result for atom with id 'uuid' to storage.""" - ad = self._atomdetail_by_name(atom_name) - ad.put(state, data) + source, clone = self._atomdetail_by_name(atom_name, clone=True) + clone.put(state, data) + result = self._with_connection(self._save_atom_detail, source, clone) if state == states.FAILURE and isinstance(data, failure.Failure): # NOTE(imelnikov): failure serialization looses information, - # so we cache failures here, in atom name -> failure mapping. - self._failures[ad.name] = data + # so we cache failures here, in atom name -> failure mapping so + # that we can later use the better version on fetch/get. + self._failures[result.name] = data else: - self._check_all_results_provided(ad.name, data) - self._with_connection(self._save_atom_detail, ad) + self._check_all_results_provided(result.name, data) @lock_utils.write_locked def save_retry_failure(self, retry_name, failed_atom_name, failure): """Save subflow failure to retry controller history.""" - ad = self._atomdetail_by_name(retry_name, - expected_type=logbook.RetryDetail) + source, clone = self._atomdetail_by_name( + retry_name, expected_type=logbook.RetryDetail, clone=True) try: - failures = ad.last_failures + failures = clone.last_failures except exceptions.NotFound as e: raise exceptions.StorageFailure("Unable to fetch most recent" " retry failures so new retry" @@ -473,32 +491,35 @@ class Storage(object): else: if failed_atom_name not in failures: failures[failed_atom_name] = failure - self._with_connection(self._save_atom_detail, ad) + self._with_connection(self._save_atom_detail, source, clone) @lock_utils.write_locked def cleanup_retry_history(self, retry_name, state): """Cleanup history of retry atom with given name.""" - ad = self._atomdetail_by_name(retry_name, - expected_type=logbook.RetryDetail) - ad.state = state - ad.results = [] - self._with_connection(self._save_atom_detail, ad) + source, clone = self._atomdetail_by_name( + retry_name, expected_type=logbook.RetryDetail, clone=True) + clone.state = state + clone.results = [] + self._with_connection(self._save_atom_detail, source, clone) @lock_utils.read_locked def _get(self, atom_name, only_last=False): - ad = self._atomdetail_by_name(atom_name) - if ad.failure is not None: + source, _clone = self._atomdetail_by_name(atom_name) + if source.failure is not None: cached = self._failures.get(atom_name) - if ad.failure.matches(cached): + if source.failure.matches(cached): + # Try to give the version back that should have the backtrace + # instead of one that has it stripped (since backtraces are not + # serializable). return cached - return ad.failure - if ad.state not in STATES_WITH_RESULTS: + return source.failure + if source.state not in STATES_WITH_RESULTS: raise exceptions.NotFound("Result for atom %s is not currently" " known" % atom_name) if only_last: - return ad.last_results + return source.last_results else: - return ad.results + return source.results def get(self, atom_name): """Gets the results for an atom with a given name from storage.""" @@ -516,21 +537,17 @@ class Storage(object): """Returns True if there are failed tasks in the storage.""" return bool(self._failures) - def _reset_atom(self, ad, state): - if ad.name == self.injector_name: - return False - if ad.state == state: - return False - ad.reset(state) - self._failures.pop(ad.name, None) - return True - @lock_utils.write_locked def reset(self, atom_name, state=states.PENDING): - """Reset atom with given name (if the task is in a given state).""" - ad = self._atomdetail_by_name(atom_name) - if self._reset_atom(ad, state): - self._with_connection(self._save_atom_detail, ad) + """Reset atom with given name (if the atom is not in a given state).""" + if atom_name == self.injector_name: + return + source, clone = self._atomdetail_by_name(atom_name, clone=True) + if source.state == state: + return + clone.reset(state) + result = self._with_connection(self._save_atom_detail, source, clone) + self._failures.pop(result.name, None) def inject_atom_args(self, atom_name, pairs, transient=True): """Add values into storage for a specific atom only. @@ -574,13 +591,13 @@ class Storage(object): self._injected_args[atom_name].update(pairs) def save_persistent(): - ad = self._atomdetail_by_name(atom_name) - injected = ad.meta.get(META_INJECTED) + source, clone = self._atomdetail_by_name(atom_name, clone=True) + injected = source.meta.get(META_INJECTED) if not injected: injected = {} injected.update(pairs) - ad.meta[META_INJECTED] = injected - self._with_connection(self._save_atom_detail, ad) + clone.meta[META_INJECTED] = injected + self._with_connection(self._save_atom_detail, source, clone) with self._lock.write_lock(): if transient: @@ -618,20 +635,22 @@ class Storage(object): def save_persistent(): try: - ad = self._atomdetail_by_name(self.injector_name, - expected_type=logbook.TaskDetail) + source, clone = self._atomdetail_by_name( + self.injector_name, + expected_type=logbook.TaskDetail, + clone=True) except exceptions.NotFound: - uuid = uuidutils.generate_uuid() - self._create_atom_detail(logbook.TaskDetail, - self.injector_name, uuid) - ad = self._atomdetail_by_name(self.injector_name, - expected_type=logbook.TaskDetail) - ad.results = dict(pairs) - ad.state = states.SUCCESS + source = self._create_atom_detail(logbook.TaskDetail, + self.injector_name, + uuidutils.generate_uuid()) + clone = source + clone.results = dict(pairs) + clone.state = states.SUCCESS else: - ad.results.update(pairs) - self._with_connection(self._save_atom_detail, ad) - return (self.injector_name, six.iterkeys(ad.results)) + clone.results.update(pairs) + result = self._with_connection(self._save_atom_detail, + source, clone) + return (self.injector_name, six.iterkeys(result.results)) def save_transient(): self._transients.update(pairs) @@ -641,6 +660,7 @@ class Storage(object): provider_name, names = save_transient() else: provider_name, names = save_persistent() + self._set_result_mapping(provider_name, dict((name, name) for name in names)) @@ -750,14 +770,14 @@ class Storage(object): providers.append(p) return providers - ad = self._atomdetail_by_name(atom_name) + source, _clone = self._atomdetail_by_name(atom_name) if scope_walker is None: scope_walker = self._scope_fetcher(atom_name) if optional_args is None: optional_args = [] injected_sources = [ self._injected_args.get(atom_name, {}), - ad.meta.get(META_INJECTED, {}), + source.meta.get(META_INJECTED, {}), ] missing = set(six.iterkeys(args_mapping)) for (bound_name, name) in six.iteritems(args_mapping): @@ -856,10 +876,10 @@ class Storage(object): if optional_args is None: optional_args = [] if atom_name: - ad = self._atomdetail_by_name(atom_name) + source, _clone = self._atomdetail_by_name(atom_name) injected_sources = [ self._injected_args.get(atom_name, {}), - ad.meta.get(META_INJECTED, {}), + source.meta.get(META_INJECTED, {}), ] if scope_walker is None: scope_walker = self._scope_fetcher(atom_name) @@ -917,20 +937,23 @@ class Storage(object): @lock_utils.write_locked def set_flow_state(self, state): """Set flow details state and save it.""" - self._flowdetail.state = state - self._with_connection(self._save_flow_detail) + source, clone = self._fetch_flowdetail(clone=True) + clone.state = state + self._with_connection(self._save_flow_detail, source, clone) @lock_utils.write_locked def update_flow_metadata(self, update_with): """Update flowdetails metadata and save it.""" if update_with: - self._flowdetail.meta.update(update_with) - self._with_connection(self._save_flow_detail) + source, clone = self._fetch_flowdetail(clone=True) + clone.meta.update(update_with) + self._with_connection(self._save_flow_detail, source, clone) @lock_utils.read_locked def get_flow_state(self): """Get state from flow details.""" - state = self._flowdetail.state + source = self._flowdetail + state = source.state if state is None: state = states.PENDING return state @@ -951,9 +974,9 @@ class Storage(object): @lock_utils.read_locked def get_retry_history(self, retry_name): """Fetch a single retrys history.""" - ad = self._atomdetail_by_name(retry_name, - expected_type=logbook.RetryDetail) - return self._translate_into_history(ad) + source, _clone = self._atomdetail_by_name( + retry_name, expected_type=logbook.RetryDetail) + return self._translate_into_history(source) @lock_utils.read_locked def get_retry_histories(self): From 17b7c219de5091db802d5e8af539e4162666750b Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 1 Apr 2015 17:12:38 -0700 Subject: [PATCH 117/246] Remove link about implementing job garbage binning This link is no longer relevant as we now have a jobboard method that can trash() jobs so that they can not be claimed again. Change-Id: I087a90c16b95e9b80a37863fa79f3348fa448bdf --- doc/source/conductors.rst | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/doc/source/conductors.rst b/doc/source/conductors.rst index 16191ff0..a819c8a0 100644 --- a/doc/source/conductors.rst +++ b/doc/source/conductors.rst @@ -53,11 +53,9 @@ claimable state. #. Forcefully delete jobs that have been failing continuously after a given number of conductor attempts. This can be either done manually or - automatically via scripts (or other associated monitoring). + automatically via scripts (or other associated monitoring) or via + the jobboards :py:func:`~taskflow.jobs.base.JobBoard.trash` method. #. Resolve the internal error's cause (storage backend failure, other...). -#. Help implement `jobboard garbage binning`_. - -.. _jobboard garbage binning: https://blueprints.launchpad.net/taskflow/+spec/jobboard-garbage-bin Interfaces ========== From c553d65c0f2de1401f5252ee51a91846f1e5ed2b Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Thu, 2 Apr 2015 11:54:41 -0700 Subject: [PATCH 118/246] Shrink the bookshelf description This smaller/tweaked version displays better on one line in my browser and is a little more accurate (with less words); so let's use it. Change-Id: I80c09328b62e93ab3f401361b5c9513d4d6de9d8 --- doc/source/index.rst | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/doc/source/index.rst b/doc/source/index.rst index d136c16f..0d673074 100644 --- a/doc/source/index.rst +++ b/doc/source/index.rst @@ -88,8 +88,8 @@ Miscellaneous Bookshelf --------- -A useful collection of *relevant* links, documents, papers, similar -projects, frameworks and libraries (and more). +A useful collection of links, documents, papers, similar +projects, frameworks and libraries. .. note:: From 9cb7181690a64d7d73a6d8d0a970646b109947c8 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Fri, 3 Apr 2015 10:26:16 -0700 Subject: [PATCH 119/246] Add a example showing how to share an executor To ensure that it is easily known how to share a futures executor across engines make an example that just does that so that people can easily reference an example when they need/want to do this. Part of blueprint share-engine-thread Part of blueprint more-examples Change-Id: I7a74da50d076157b9fbccdcbd6dff1b3b9cd93b6 --- doc/source/examples.rst | 12 ++++ taskflow/examples/share_engine_thread.py | 81 ++++++++++++++++++++++++ 2 files changed, 93 insertions(+) create mode 100644 taskflow/examples/share_engine_thread.py diff --git a/doc/source/examples.rst b/doc/source/examples.rst index f1ebdc7e..ceed11a8 100644 --- a/doc/source/examples.rst +++ b/doc/source/examples.rst @@ -176,6 +176,18 @@ Summation mapper(s) and reducer (in parallel) :linenos: :lines: 16- +Sharing a thread pool executor (in parallel) +============================================ + +.. note:: + + Full source located at :example:`share_engine_thread` + +.. literalinclude:: ../../taskflow/examples/share_engine_thread.py + :language: python + :linenos: + :lines: 16- + Storing & emitting a bill ========================= diff --git a/taskflow/examples/share_engine_thread.py b/taskflow/examples/share_engine_thread.py new file mode 100644 index 00000000..5654fa06 --- /dev/null +++ b/taskflow/examples/share_engine_thread.py @@ -0,0 +1,81 @@ +# -*- coding: utf-8 -*- + +# Copyright (C) 2012-2013 Yahoo! Inc. 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. + +import logging +import os +import random +import sys +import time + +logging.basicConfig(level=logging.ERROR) + +top_dir = os.path.abspath(os.path.join(os.path.dirname(__file__), + os.pardir, + os.pardir)) +sys.path.insert(0, top_dir) + +import six + +from taskflow import engines +from taskflow.patterns import unordered_flow as uf +from taskflow import task +from taskflow.types import futures +from taskflow.utils import threading_utils as tu + +# INTRO: in this example we create 2 dummy flow(s) with a 2 dummy task(s), and +# run it using a shared thread pool executor to show how a single executor can +# be used with more than one engine (sharing the execution thread pool between +# them); this allows for saving resources and reusing threads in situations +# where this is benefical. + + +class DelayedTask(task.Task): + def __init__(self, name): + super(DelayedTask, self).__init__(name=name) + self._wait_for = random.random() + + def execute(self): + print("Running '%s' in thread '%s'" % (self.name, tu.get_ident())) + time.sleep(self._wait_for) + + +f1 = uf.Flow("f1") +f1.add(DelayedTask("f1-1")) +f1.add(DelayedTask("f1-2")) + +f2 = uf.Flow("f2") +f2.add(DelayedTask("f2-1")) +f2.add(DelayedTask("f2-2")) + +# Run them all using the same futures (thread-pool based) executor... +with futures.ThreadPoolExecutor() as ex: + e1 = engines.load(f1, engine='parallel', executor=ex) + e2 = engines.load(f2, engine='parallel', executor=ex) + iters = [e1.run_iter(), e2.run_iter()] + # Iterate over a copy (so we can remove from the source list). + cloned_iters = list(iters) + while iters: + # Run a single 'step' of each iterator, forcing each engine to perform + # some work, then yield, and repeat until each iterator is consumed + # and there is no more engine work to be done. + for it in cloned_iters: + try: + six.next(it) + except StopIteration: + try: + iters.remove(it) + except ValueError: + pass From aa8be4b8f5f55cc4e11088dcb122b34345f56cec Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 11 Feb 2015 18:22:20 -0800 Subject: [PATCH 120/246] Use the ability to chain exceptions correctly In the zookeeper jobboard (and persistence backends) we are associating the cause of a new exception when raising a exception. Using the new exception helper we can make this work better on py2.x and py3.x so that the py3.x version has the chain setup correctly (while the py2.x version just uses the class 'cause' attribute instead). Change-Id: Ieeac2f70e1834d4612556565762ffd3be3e5b5a1 --- taskflow/engines/worker_based/protocol.py | 32 +++-- taskflow/jobs/backends/impl_zookeeper.py | 131 ++++++++++-------- taskflow/persistence/backends/impl_dir.py | 8 +- taskflow/persistence/backends/impl_memory.py | 7 +- .../persistence/backends/impl_sqlalchemy.py | 74 +++++----- .../persistence/backends/impl_zookeeper.py | 32 +++-- 6 files changed, 169 insertions(+), 115 deletions(-) diff --git a/taskflow/engines/worker_based/protocol.py b/taskflow/engines/worker_based/protocol.py index 1ecfcef2..867f5369 100644 --- a/taskflow/engines/worker_based/protocol.py +++ b/taskflow/engines/worker_based/protocol.py @@ -165,13 +165,17 @@ class Notify(Message): except su.ValidationError as e: cls_name = reflection.get_class_name(cls, fully_qualified=False) if response: - raise excp.InvalidFormat("%s message response data not of the" - " expected format: %s" - % (cls_name, e.message), e) + excp.raise_with_cause(excp.InvalidFormat, + "%s message response data not of the" + " expected format: %s" % (cls_name, + e.message), + cause=e) else: - raise excp.InvalidFormat("%s message sender data not of the" - " expected format: %s" - % (cls_name, e.message), e) + excp.raise_with_cause(excp.InvalidFormat, + "%s message sender data not of the" + " expected format: %s" % (cls_name, + e.message), + cause=e) _WorkUnit = collections.namedtuple('_WorkUnit', ['task_cls', 'task_name', @@ -361,9 +365,11 @@ class Request(Message): su.schema_validate(data, cls.SCHEMA) except su.ValidationError as e: cls_name = reflection.get_class_name(cls, fully_qualified=False) - raise excp.InvalidFormat("%s message response data not of the" - " expected format: %s" - % (cls_name, e.message), e) + excp.raise_with_cause(excp.InvalidFormat, + "%s message response data not of the" + " expected format: %s" % (cls_name, + e.message), + cause=e) else: # Validate all failure dictionaries that *may* be present... failures = [] @@ -505,9 +511,11 @@ class Response(Message): su.schema_validate(data, cls.SCHEMA) except su.ValidationError as e: cls_name = reflection.get_class_name(cls, fully_qualified=False) - raise excp.InvalidFormat("%s message response data not of the" - " expected format: %s" - % (cls_name, e.message), e) + excp.raise_with_cause(excp.InvalidFormat, + "%s message response data not of the" + " expected format: %s" % (cls_name, + e.message), + cause=e) else: state = data['state'] if state == FAILURE and 'result' in data: diff --git a/taskflow/jobs/backends/impl_zookeeper.py b/taskflow/jobs/backends/impl_zookeeper.py index 55c7b8eb..2346b4b2 100644 --- a/taskflow/jobs/backends/impl_zookeeper.py +++ b/taskflow/jobs/backends/impl_zookeeper.py @@ -17,6 +17,7 @@ import collections import contextlib import functools +import sys import threading from concurrent import futures @@ -113,22 +114,27 @@ class ZookeeperJob(base.Job): return trans_func(attr) else: return attr - except k_exceptions.NoNodeError as e: - raise excp.NotFound("Can not fetch the %r attribute" - " of job %s (%s), path %s not found" - % (attr_name, self.uuid, self.path, path), e) - except self._client.handler.timeout_exception as e: - raise excp.JobFailure("Can not fetch the %r attribute" - " of job %s (%s), operation timed out" - % (attr_name, self.uuid, self.path), e) - except k_exceptions.SessionExpiredError as e: - raise excp.JobFailure("Can not fetch the %r attribute" - " of job %s (%s), session expired" - % (attr_name, self.uuid, self.path), e) - except (AttributeError, k_exceptions.KazooException) as e: - raise excp.JobFailure("Can not fetch the %r attribute" - " of job %s (%s), internal error" % - (attr_name, self.uuid, self.path), e) + except k_exceptions.NoNodeError: + excp.raise_with_cause( + excp.NotFound, + "Can not fetch the %r attribute of job %s (%s)," + " path %s not found" % (attr_name, self.uuid, + self.path, path)) + except self._client.handler.timeout_exception: + excp.raise_with_cause( + excp.JobFailure, + "Can not fetch the %r attribute of job %s (%s)," + " operation timed out" % (attr_name, self.uuid, self.path)) + except k_exceptions.SessionExpiredError: + excp.raise_with_cause( + excp.JobFailure, + "Can not fetch the %r attribute of job %s (%s)," + " session expired" % (attr_name, self.uuid, self.path)) + except (AttributeError, k_exceptions.KazooException): + excp.raise_with_cause( + excp.JobFailure, + "Can not fetch the %r attribute of job %s (%s)," + " internal error" % (attr_name, self.uuid, self.path)) @property def last_modified(self): @@ -182,15 +188,21 @@ class ZookeeperJob(base.Job): job_data = misc.decode_json(raw_data) except k_exceptions.NoNodeError: pass - except k_exceptions.SessionExpiredError as e: - raise excp.JobFailure("Can not fetch the state of %s," - " session expired" % (self.uuid), e) - except self._client.handler.timeout_exception as e: - raise excp.JobFailure("Can not fetch the state of %s," - " operation timed out" % (self.uuid), e) - except k_exceptions.KazooException as e: - raise excp.JobFailure("Can not fetch the state of %s, internal" - " error" % (self.uuid), e) + except k_exceptions.SessionExpiredError: + excp.raise_with_cause( + excp.JobFailure, + "Can not fetch the state of %s," + " session expired" % (self.uuid)) + except self._client.handler.timeout_exception: + excp.raise_with_cause( + excp.JobFailure, + "Can not fetch the state of %s," + " operation timed out" % (self.uuid)) + except k_exceptions.KazooException: + excp.raise_with_cause( + excp.JobFailure, + "Can not fetch the state of %s," + " internal error" % (self.uuid)) if not job_data: # No data this job has been completed (the owner that we might have # fetched will not be able to be fetched again, since the job node @@ -383,15 +395,17 @@ class ZookeeperJobBoard(base.NotifyingJobBoard): def _force_refresh(self): try: children = self._client.get_children(self.path) - except self._client.handler.timeout_exception as e: - raise excp.JobFailure("Refreshing failure, operation timed out", - e) - except k_exceptions.SessionExpiredError as e: - raise excp.JobFailure("Refreshing failure, session expired", e) + except self._client.handler.timeout_exception: + excp.raise_with_cause(excp.JobFailure, + "Refreshing failure, operation timed out") + except k_exceptions.SessionExpiredError: + excp.raise_with_cause(excp.JobFailure, + "Refreshing failure, session expired") except k_exceptions.NoNodeError: pass - except k_exceptions.KazooException as e: - raise excp.JobFailure("Refreshing failure, internal error", e) + except k_exceptions.KazooException: + excp.raise_with_cause(excp.JobFailure, + "Refreshing failure, internal error") else: self._on_job_posting(children, delayed=False) @@ -542,10 +556,11 @@ class ZookeeperJobBoard(base.NotifyingJobBoard): except Exception: owner = None if owner: - msg = "Job %s already claimed by '%s'" % (job.uuid, owner) + message = "Job %s already claimed by '%s'" % (job.uuid, owner) else: - msg = "Job %s already claimed" % (job.uuid) - return excp.UnclaimableJob(msg, cause) + message = "Job %s already claimed" % (job.uuid) + excp.raise_with_cause(excp.UnclaimableJob, + message, cause=cause) _check_who(who) with self._wrap(job.uuid, job.path, "Claiming failure: %s"): @@ -566,21 +581,23 @@ class ZookeeperJobBoard(base.NotifyingJobBoard): try: kazoo_utils.checked_commit(txn) except k_exceptions.NodeExistsError as e: - raise _unclaimable_try_find_owner(e) + _unclaimable_try_find_owner(e) except kazoo_utils.KazooTransactionException as e: if len(e.failures) < 2: raise else: if isinstance(e.failures[0], k_exceptions.NoNodeError): - raise excp.NotFound( + excp.raise_with_cause( + excp.NotFound, "Job %s not found to be claimed" % job.uuid, - e.failures[0]) + cause=e.failures[0]) if isinstance(e.failures[1], k_exceptions.NodeExistsError): - raise _unclaimable_try_find_owner(e.failures[1]) + _unclaimable_try_find_owner(e.failures[1]) else: - raise excp.UnclaimableJob( + excp.raise_with_cause( + excp.UnclaimableJob, "Job %s claim failed due to transaction" - " not succeeding" % (job.uuid), e) + " not succeeding" % (job.uuid), cause=e) @contextlib.contextmanager def _wrap(self, job_uuid, job_path, @@ -596,18 +613,18 @@ class ZookeeperJobBoard(base.NotifyingJobBoard): raise excp.NotFound(fail_msg_tpl % (job_uuid)) try: yield - except self._client.handler.timeout_exception as e: + except self._client.handler.timeout_exception: fail_msg_tpl += ", operation timed out" - raise excp.JobFailure(fail_msg_tpl % (job_uuid), e) - except k_exceptions.SessionExpiredError as e: + excp.raise_with_cause(excp.JobFailure, fail_msg_tpl % (job_uuid)) + except k_exceptions.SessionExpiredError: fail_msg_tpl += ", session expired" - raise excp.JobFailure(fail_msg_tpl % (job_uuid), e) + excp.raise_with_cause(excp.JobFailure, fail_msg_tpl % (job_uuid)) except k_exceptions.NoNodeError: fail_msg_tpl += ", unknown job" - raise excp.NotFound(fail_msg_tpl % (job_uuid)) - except k_exceptions.KazooException as e: + excp.raise_with_cause(excp.NotFound, fail_msg_tpl % (job_uuid)) + except k_exceptions.KazooException: fail_msg_tpl += ", internal error" - raise excp.JobFailure(fail_msg_tpl % (job_uuid), e) + excp.raise_with_cause(excp.JobFailure, fail_msg_tpl % (job_uuid)) def find_owner(self, job): with self._wrap(job.uuid, job.path, "Owner query failure: %s"): @@ -750,8 +767,9 @@ class ZookeeperJobBoard(base.NotifyingJobBoard): timeout = float(timeout) self._client.start(timeout=timeout) except (self._client.handler.timeout_exception, - k_exceptions.KazooException) as e: - raise excp.JobFailure("Failed to connect to zookeeper", e) + k_exceptions.KazooException): + excp.raise_with_cause(excp.JobFailure, + "Failed to connect to zookeeper") try: if self._conf.get('check_compatible', True): kazoo_utils.check_compatible(self._client, MIN_ZK_VERSION) @@ -770,7 +788,12 @@ class ZookeeperJobBoard(base.NotifyingJobBoard): with excutils.save_and_reraise_exception(): try_clean() except (self._client.handler.timeout_exception, - k_exceptions.KazooException) as e: - try_clean() - raise excp.JobFailure("Failed to do post-connection" - " initialization", e) + k_exceptions.KazooException): + exc_type, exc, exc_tb = sys.exc_info() + try: + try_clean() + excp.raise_with_cause(excp.JobFailure, + "Failed to do post-connection" + " initialization", cause=exc) + finally: + del(exc_type, exc, exc_tb) diff --git a/taskflow/persistence/backends/impl_dir.py b/taskflow/persistence/backends/impl_dir.py index 5185b2b8..e71d5b9c 100644 --- a/taskflow/persistence/backends/impl_dir.py +++ b/taskflow/persistence/backends/impl_dir.py @@ -36,8 +36,12 @@ def _storagefailure_wrapper(): raise except Exception as e: if isinstance(e, (IOError, OSError)) and e.errno == errno.ENOENT: - raise exc.NotFound('Item not found: %s' % e.filename, e) - raise exc.StorageFailure("Storage backend internal error", e) + exc.raise_with_cause(exc.NotFound, + 'Item not found: %s' % e.filename, + cause=e) + else: + exc.raise_with_cause(exc.StorageFailure, + "Storage backend internal error", cause=e) class DirBackend(path_based.PathBasedBackend): diff --git a/taskflow/persistence/backends/impl_memory.py b/taskflow/persistence/backends/impl_memory.py index 7efe6ec0..0157fcfc 100644 --- a/taskflow/persistence/backends/impl_memory.py +++ b/taskflow/persistence/backends/impl_memory.py @@ -209,10 +209,11 @@ class Connection(path_based.PathBasedConnection): with lock(): try: yield - except exc.TaskFlowException as e: + except exc.TaskFlowException: raise - except Exception as e: - raise exc.StorageFailure("Storage backend internal error", e) + except Exception: + exc.raise_with_cause(exc.StorageFailure, + "Storage backend internal error") def _join_path(self, *parts): return pp.join(*parts) diff --git a/taskflow/persistence/backends/impl_sqlalchemy.py b/taskflow/persistence/backends/impl_sqlalchemy.py index d6eeaaba..d9fdf732 100644 --- a/taskflow/persistence/backends/impl_sqlalchemy.py +++ b/taskflow/persistence/backends/impl_sqlalchemy.py @@ -405,16 +405,18 @@ class Connection(base.Connection): self._metadata.create_all(bind=conn) else: migration.db_sync(conn) - except sa_exc.SQLAlchemyError as e: - raise exc.StorageFailure("Failed upgrading database version", e) + except sa_exc.SQLAlchemyError: + exc.raise_with_cause(exc.StorageFailure, + "Failed upgrading database version") def clear_all(self): try: logbooks = self._tables.logbooks with self._engine.begin() as conn: conn.execute(logbooks.delete()) - except sa_exc.DBAPIError as e: - raise exc.StorageFailure("Failed clearing all entries", e) + except sa_exc.DBAPIError: + exc.raise_with_cause(exc.StorageFailure, + "Failed clearing all entries") def update_atom_details(self, atom_detail): try: @@ -429,9 +431,10 @@ class Connection(base.Connection): e_ad = self._converter.convert_atom_detail(row) self._update_atom_details(conn, atom_detail, e_ad) return e_ad - except sa_exc.SQLAlchemyError as e: - raise exc.StorageFailure("Failed updating atom details with" - " uuid '%s'" % atom_detail.uuid, e) + except sa_exc.SQLAlchemyError: + exc.raise_with_cause(exc.StorageFailure, + "Failed updating atom details" + " with uuid '%s'" % atom_detail.uuid) def _insert_flow_details(self, conn, fd, parent_uuid): value = fd.to_dict() @@ -479,9 +482,10 @@ class Connection(base.Connection): self._converter.populate_flow_detail(conn, e_fd) self._update_flow_details(conn, flow_detail, e_fd) return e_fd - except sa_exc.SQLAlchemyError as e: - raise exc.StorageFailure("Failed updating flow details with" - " uuid '%s'" % flow_detail.uuid, e) + except sa_exc.SQLAlchemyError: + exc.raise_with_cause(exc.StorageFailure, + "Failed updating flow details with" + " uuid '%s'" % flow_detail.uuid) def destroy_logbook(self, book_uuid): try: @@ -492,9 +496,9 @@ class Connection(base.Connection): if r.rowcount == 0: raise exc.NotFound("No logbook found with" " uuid '%s'" % book_uuid) - except sa_exc.DBAPIError as e: - raise exc.StorageFailure("Failed destroying" - " logbook '%s'" % book_uuid, e) + except sa_exc.DBAPIError: + exc.raise_with_cause(exc.StorageFailure, + "Failed destroying logbook '%s'" % book_uuid) def save_logbook(self, book): try: @@ -523,9 +527,10 @@ class Connection(base.Connection): for fd in book: self._insert_flow_details(conn, fd, book.uuid) return book - except sa_exc.DBAPIError as e: - raise exc.StorageFailure("Failed saving logbook" - " '%s'" % book.uuid, e) + except sa_exc.DBAPIError: + exc.raise_with_cause( + exc.StorageFailure, + "Failed saving logbook '%s'" % book.uuid) def get_logbook(self, book_uuid, lazy=False): try: @@ -541,9 +546,9 @@ class Connection(base.Connection): if not lazy: self._converter.populate_book(conn, book) return book - except sa_exc.DBAPIError as e: - raise exc.StorageFailure( - "Failed getting logbook '%s'" % book_uuid, e) + except sa_exc.DBAPIError: + exc.raise_with_cause(exc.StorageFailure, + "Failed getting logbook '%s'" % book_uuid) def get_logbooks(self, lazy=False): gathered = [] @@ -555,8 +560,9 @@ class Connection(base.Connection): if not lazy: self._converter.populate_book(conn, book) gathered.append(book) - except sa_exc.DBAPIError as e: - raise exc.StorageFailure("Failed getting logbooks", e) + except sa_exc.DBAPIError: + exc.raise_with_cause(exc.StorageFailure, + "Failed getting logbooks") for book in gathered: yield book @@ -568,8 +574,10 @@ class Connection(base.Connection): if not lazy: self._converter.populate_flow_detail(conn, fd) gathered.append(fd) - except sa_exc.DBAPIError as e: - raise exc.StorageFailure("Failed getting flow details", e) + except sa_exc.DBAPIError: + exc.raise_with_cause(exc.StorageFailure, + "Failed getting flow details in" + " logbook '%s'" % book_uuid) for flow_details in gathered: yield flow_details @@ -587,9 +595,10 @@ class Connection(base.Connection): if not lazy: self._converter.populate_flow_detail(conn, fd) return fd - except sa_exc.SQLAlchemyError as e: - raise exc.StorageFailure("Failed getting flow details with" - " uuid '%s'" % fd_uuid, e) + except sa_exc.SQLAlchemyError: + exc.raise_with_cause(exc.StorageFailure, + "Failed getting flow details with" + " uuid '%s'" % fd_uuid) def get_atom_details(self, ad_uuid): try: @@ -602,9 +611,10 @@ class Connection(base.Connection): raise exc.NotFound("No atom details found with uuid" " '%s'" % ad_uuid) return self._converter.convert_atom_detail(row) - except sa_exc.SQLAlchemyError as e: - raise exc.StorageFailure("Failed getting atom details with" - " uuid '%s'" % ad_uuid, e) + except sa_exc.SQLAlchemyError: + exc.raise_with_cause(exc.StorageFailure, + "Failed getting atom details with" + " uuid '%s'" % ad_uuid) def get_atoms_for_flow(self, fd_uuid): gathered = [] @@ -612,8 +622,10 @@ class Connection(base.Connection): with contextlib.closing(self._engine.connect()) as conn: for ad in self._converter.atom_query_iter(conn, fd_uuid): gathered.append(ad) - except sa_exc.DBAPIError as e: - raise exc.StorageFailure("Failed getting atom details", e) + except sa_exc.DBAPIError: + exc.raise_with_cause(exc.StorageFailure, + "Failed getting atom details in flow" + " detail '%s'" % fd_uuid) for atom_details in gathered: yield atom_details diff --git a/taskflow/persistence/backends/impl_zookeeper.py b/taskflow/persistence/backends/impl_zookeeper.py index 5626b289..0d7c00ee 100644 --- a/taskflow/persistence/backends/impl_zookeeper.py +++ b/taskflow/persistence/backends/impl_zookeeper.py @@ -67,8 +67,9 @@ class ZkBackend(path_based.PathBasedBackend): return try: k_utils.finalize_client(self._client) - except (k_exc.KazooException, k_exc.ZookeeperError) as e: - raise exc.StorageFailure("Unable to finalize client", e) + except (k_exc.KazooException, k_exc.ZookeeperError): + exc.raise_with_cause(exc.StorageFailure, + "Unable to finalize client") class ZkConnection(path_based.PathBasedConnection): @@ -90,16 +91,21 @@ class ZkConnection(path_based.PathBasedConnection): """ try: yield - except self._client.handler.timeout_exception as e: - raise exc.StorageFailure("Storage backend timeout", e) - except k_exc.SessionExpiredError as e: - raise exc.StorageFailure("Storage backend session has expired", e) + except self._client.handler.timeout_exception: + exc.raise_with_cause(exc.StorageFailure, + "Storage backend timeout") + except k_exc.SessionExpiredError: + exc.raise_with_cause(exc.StorageFailure, + "Storage backend session has expired") except k_exc.NoNodeError as e: - raise exc.NotFound("Storage backend node not found: %s" % e) + exc.raise_with_cause(exc.NotFound, + "Storage backend node not found: %s" % e) except k_exc.NodeExistsError as e: - raise exc.Duplicate("Storage backend duplicate node: %s" % e) - except (k_exc.KazooException, k_exc.ZookeeperError) as e: - raise exc.StorageFailure("Storage backend internal error", e) + exc.raise_with_cause(exc.Duplicate, + "Storage backend duplicate node: %s" % e) + except (k_exc.KazooException, k_exc.ZookeeperError): + exc.raise_with_cause(exc.StorageFailure, + "Storage backend internal error") def _join_path(self, *parts): return paths.join(*parts) @@ -145,6 +151,6 @@ class ZkConnection(path_based.PathBasedConnection): try: if self._conf.get('check_compatible', True): k_utils.check_compatible(self._client, MIN_ZK_VERSION) - except exc.IncompatibleVersion as e: - raise exc.StorageFailure("Backend storage is not a" - " compatible version", e) + except exc.IncompatibleVersion: + exc.raise_with_cause(exc.StorageFailure, "Backend storage is" + " not a compatible version") From 7e8981a8914e487f834ad9853365c935bfd1d25b Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Fri, 3 Apr 2015 16:10:41 -0700 Subject: [PATCH 121/246] Avoid going into causes of non-taskflow exceptions Change-Id: If043bf27f3b025a279f3747841e9a88fcc5fab85 --- taskflow/exceptions.py | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/taskflow/exceptions.py b/taskflow/exceptions.py index 069a2f2a..5d15355a 100644 --- a/taskflow/exceptions.py +++ b/taskflow/exceptions.py @@ -116,11 +116,13 @@ class TaskFlowException(Exception): buf.write(line) if i + 1 != len(lines): buf.write(os.linesep) + if not isinstance(next_up, TaskFlowException): + # Don't go deeper into non-taskflow exceptions... as we + # don't know if there exception 'cause' attributes are even + # useable objects... + break active_indent += indent - try: - next_up = next_up.cause - except AttributeError: - next_up = None + next_up = getattr(next_up, 'cause', None) return buf.getvalue() From 2ad837c9b98a60110c04aa4d2859fc4a71ecccf8 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Fri, 3 Apr 2015 17:37:09 -0700 Subject: [PATCH 122/246] Note that the traditional mode also avoids this truncation issue Change-Id: I03bcd2b6971ac14551ed9410d56e4e9dabc39a80 --- doc/source/persistence.rst | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/doc/source/persistence.rst b/doc/source/persistence.rst index a0731a30..6f8256bf 100644 --- a/doc/source/persistence.rst +++ b/doc/source/persistence.rst @@ -249,9 +249,13 @@ parent_uuid VARCHAR False ``results`` will contain. This size limit will restrict how many prior failures a retry atom can contain. More information and a future fix will be posted to bug `1416088`_ (for the meantime try to ensure that - your retry units history does not grow beyond ~80 prior results). + your retry units history does not grow beyond ~80 prior results). This + truncation can also be avoided by providing ``mysql_sql_mode`` as + ``traditional`` when selecting your mysql + sqlalchemy based + backend (see the `mysql modes`_ documentation for what this implies). .. _1416088: http://bugs.launchpad.net/taskflow/+bug/1416088 +.. _mysql modes: http://dev.mysql.com/doc/refman/5.0/en/sql-mode.html Zookeeper --------- From ebc07828684ef12c55fe1d88b836926ff06713e5 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Sat, 4 Apr 2015 19:45:33 -0700 Subject: [PATCH 123/246] Put semantics preservation section into note block Change-Id: I2fceb1628ef384f6d5db1f292cabe48195efbf9c --- doc/source/engines.rst | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/doc/source/engines.rst b/doc/source/engines.rst index 5225f96f..9563cb45 100644 --- a/doc/source/engines.rst +++ b/doc/source/engines.rst @@ -17,11 +17,13 @@ and *ideal* is that deployers or developers of a service that use TaskFlow can select an engine that suites their setup best without modifying the code of said service. -Engines usually have different capabilities and configuration, but all of them -**must** implement the same interface and preserve the semantics of patterns -(e.g. parts of a :py:class:`.linear_flow.Flow` -are run one after another, in order, even if the selected engine is *capable* -of running tasks in parallel). +.. note:: + + Engines usually have different capabilities and configuration, but all of + them **must** implement the same interface and preserve the semantics of + patterns (e.g. parts of a :py:class:`.linear_flow.Flow` + are run one after another, in order, even if the selected + engine is *capable* of running tasks in parallel). Why they exist -------------- From 647b69e509e773cc28e2bb364435cc0f2943e75d Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Sun, 5 Apr 2015 09:40:52 -0700 Subject: [PATCH 124/246] Have the notification/listener docs match other sections This doc has a structure which doesn't currently match the other sections (no hierarchy, no impls...) so adjust it so that it does match the other sections/docs. Change-Id: I48599f6742752dd654bdae90991a44c8ca713e7a --- doc/source/notifications.rst | 26 ++++++++++++-------------- 1 file changed, 12 insertions(+), 14 deletions(-) diff --git a/doc/source/notifications.rst b/doc/source/notifications.rst index 816393ed..aa42d6b9 100644 --- a/doc/source/notifications.rst +++ b/doc/source/notifications.rst @@ -1,6 +1,6 @@ -=========================== +--------------------------- Notifications and listeners -=========================== +--------------------------- .. testsetup:: @@ -10,9 +10,8 @@ Notifications and listeners from taskflow.types import notifier ANY = notifier.Notifier.ANY --------- Overview --------- +======== Engines provide a way to receive notification on task and flow state transitions, which is useful for monitoring, logging, metrics, debugging @@ -27,9 +26,8 @@ TaskFlow also comes with a set of predefined :ref:`listeners `, and provides means to write your own listeners, which can be more convenient than using raw callbacks. --------------------------------------- Receiving notifications with callbacks --------------------------------------- +====================================== Flow notifications ------------------ @@ -106,9 +104,8 @@ A basic example is: .. _listeners: ---------- Listeners ---------- +========= TaskFlow comes with a set of predefined listeners -- helper classes that can be used to do various actions on flow and/or tasks transitions. You can also @@ -147,16 +144,17 @@ For example, this is how you can use has moved task 'DogTalk' (...) into state 'SUCCESS' from state 'RUNNING' with result 'dog' (failure=False) has moved flow 'cat-dog' (...) into state 'SUCCESS' from state 'RUNNING' -Basic listener --------------- +Interfaces +========== -.. autoclass:: taskflow.listeners.base.Listener +.. automodule:: taskflow.listeners.base + +Implementations +=============== Printing and logging listeners ------------------------------ -.. autoclass:: taskflow.listeners.base.DumpingListener - .. autoclass:: taskflow.listeners.logging.LoggingListener .. autoclass:: taskflow.listeners.logging.DynamicLoggingListener @@ -183,7 +181,7 @@ Capturing listener .. autoclass:: taskflow.listeners.capturing.CaptureListener Hierarchy ---------- +========= .. inheritance-diagram:: taskflow.listeners.base.DumpingListener From c64ca27fb334d40f245cb13b4158e1e1af1ba26f Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 8 Apr 2015 14:04:38 -0700 Subject: [PATCH 125/246] Add note about thread safety of fake filesystem Change-Id: Ic9851455b559a94bb011c27ba55ad050e6bd6d55 --- taskflow/persistence/backends/impl_memory.py | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/taskflow/persistence/backends/impl_memory.py b/taskflow/persistence/backends/impl_memory.py index 4cb90f00..e4d135fa 100644 --- a/taskflow/persistence/backends/impl_memory.py +++ b/taskflow/persistence/backends/impl_memory.py @@ -37,6 +37,15 @@ class FakeFilesystem(object): which are not relevant in an implementation of a in-memory fake filesystem). + **Not** thread-safe when a single filesystem is mutated at the same + time by multiple threads. For example having multiple threads call into + :meth:`~taskflow.persistence.backends.impl_memory.FakeFilesystem.clear` + at the same time could potentially end badly. It is thread-safe when only + :meth:`~taskflow.persistence.backends.impl_memory.FakeFilesystem.get` + or other read-only actions (like calling into + :meth:`~taskflow.persistence.backends.impl_memory.FakeFilesystem.ls`) + are occuring at the same time. + Example usage: >>> from taskflow.persistence.backends import impl_memory @@ -142,6 +151,7 @@ class FakeFilesystem(object): return paths def clear(self): + """Remove all nodes (except the root) from this filesystem.""" for node in list(self._root.reverse_iter()): node.disassociate() From 0ad2fd92ee674c06515c68bb685798006a4d8ce9 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 8 Apr 2015 14:29:29 -0700 Subject: [PATCH 126/246] Ensure we don't get stuck in formatting loops Change-Id: I516dc2aca05823add9daf8c6d4c4af7ef7d2a313 --- taskflow/exceptions.py | 4 +++- taskflow/tests/unit/test_exceptions.py | 15 +++++++++++++++ 2 files changed, 18 insertions(+), 1 deletion(-) diff --git a/taskflow/exceptions.py b/taskflow/exceptions.py index 5d15355a..15a73b3b 100644 --- a/taskflow/exceptions.py +++ b/taskflow/exceptions.py @@ -98,7 +98,9 @@ class TaskFlowException(Exception): buf.write(self._get_message()) active_indent = indent next_up = self.cause - while next_up is not None: + seen = [] + while next_up is not None and next_up not in seen: + seen.append(next_up) buf.write(os.linesep) if isinstance(next_up, TaskFlowException): buf.write(indent_text * active_indent) diff --git a/taskflow/tests/unit/test_exceptions.py b/taskflow/tests/unit/test_exceptions.py index b3590653..c542ae83 100644 --- a/taskflow/tests/unit/test_exceptions.py +++ b/taskflow/tests/unit/test_exceptions.py @@ -14,6 +14,8 @@ # License for the specific language governing permissions and limitations # under the License. +import string + import six import testtools @@ -56,6 +58,19 @@ class TestExceptions(test.TestCase): self.assertIsNotNone(capture.cause) self.assertIsInstance(capture.cause, IOError) + def test_no_looping(self): + causes = [] + for a in string.ascii_lowercase: + try: + cause = causes[-1] + except IndexError: + cause = None + causes.append(exc.TaskFlowException('%s broken' % a, cause=cause)) + e = causes[0] + last_e = causes[-1] + e._cause = last_e + self.assertIsNotNone(e.pformat()) + def test_pformat_str(self): ex = None try: From ee7f07f81ec442fa5c1df91bc3155c963619c7c8 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 8 Apr 2015 15:28:19 -0700 Subject: [PATCH 127/246] Mention link to states doc in notify state transitions Change-Id: I4d1c12421a48c72431dad57c5a9e42094123ec2a --- doc/source/notifications.rst | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/doc/source/notifications.rst b/doc/source/notifications.rst index aa42d6b9..a8924b6c 100644 --- a/doc/source/notifications.rst +++ b/doc/source/notifications.rst @@ -14,8 +14,8 @@ Overview ======== Engines provide a way to receive notification on task and flow state -transitions, which is useful for monitoring, logging, metrics, debugging -and plenty of other tasks. +transitions (see :doc:`states `), which is useful for +monitoring, logging, metrics, debugging and plenty of other tasks. To receive these notifications you should register a callback with an instance of the :py:class:`~taskflow.types.notifier.Notifier` From 4ef79bcdaa2c7ac355efded37f9c1ff63cb720ba Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 8 Apr 2015 15:52:05 -0700 Subject: [PATCH 128/246] Rename notifier 'listeners' to 'topics' This variable is better renamed to 'topics' to match the typical message queue kind of syntax/usage since each entry in this dictionary is nearly equivalent to a specific topic channel about a event (with the dictionary value being the set of targets/listeners that are registered on that channel who will get notified about any channel/topic activity). Change-Id: Ie76e9cc4e29f4e542be987561cd6872e021cfa58 --- taskflow/types/notifier.py | 34 ++++++++++++++++++---------------- 1 file changed, 18 insertions(+), 16 deletions(-) diff --git a/taskflow/types/notifier.py b/taskflow/types/notifier.py index eaf7b02c..2257f52d 100644 --- a/taskflow/types/notifier.py +++ b/taskflow/types/notifier.py @@ -104,7 +104,7 @@ class _Listener(object): class Notifier(object): - """A notification helper class. + """A notification (`pub/sub`_ *like*) helper class. It is intended to be used to subscribe to notifications of events occurring as well as allow a entity to post said notifications to any @@ -117,6 +117,8 @@ class Notifier(object): potentially end badly. It is thread-safe when only :py:meth:`.notify` calls or other read-only actions (like calling into :py:meth:`.is_registered`) are occuring at the same time. + + .. _pub/sub: http://en.wikipedia.org/wiki/Publish%E2%80%93subscribe_pattern """ #: Keys that can *not* be used in callbacks arguments @@ -129,7 +131,7 @@ class Notifier(object): _DISALLOWED_NOTIFICATION_EVENTS = set([ANY]) def __init__(self): - self._listeners = collections.defaultdict(list) + self._topics = collections.defaultdict(list) def __len__(self): """Returns how many callbacks are registered. @@ -138,7 +140,7 @@ class Notifier(object): :rtype: number """ count = 0 - for (_event_type, listeners) in six.iteritems(self._listeners): + for (_event_type, listeners) in six.iteritems(self._topics): count += len(listeners) return count @@ -148,14 +150,14 @@ class Notifier(object): :returns: checks if the callback is registered :rtype: boolean """ - for listener in self._listeners.get(event_type, []): + for listener in self._topics.get(event_type, []): if listener.is_equivalent(callback, details_filter=details_filter): return True return False def reset(self): """Forget all previously registered callbacks.""" - self._listeners.clear() + self._topics.clear() def notify(self, event_type, details): """Notify about event occurrence. @@ -176,8 +178,8 @@ class Notifier(object): LOG.debug("Event type '%s' is not allowed to trigger" " notifications", event_type) return - listeners = list(self._listeners.get(self.ANY, [])) - listeners.extend(self._listeners.get(event_type, [])) + listeners = list(self._topics.get(self.ANY, [])) + listeners.extend(self._topics.get(event_type, [])) if not listeners: return if not details: @@ -226,7 +228,7 @@ class Notifier(object): if k in kwargs: raise KeyError("Reserved key '%s' not allowed in " "kwargs" % k) - self._listeners[event_type].append( + self._topics[event_type].append( _Listener(callback, args=args, kwargs=kwargs, details_filter=details_filter)) @@ -236,11 +238,11 @@ class Notifier(object): :param event_type: deregister listener bound to event_type """ - if event_type not in self._listeners: + if event_type not in self._topics: return False - for i, listener in enumerate(self._listeners.get(event_type, [])): + for i, listener in enumerate(self._topics.get(event_type, [])): if listener.is_equivalent(callback, details_filter=details_filter): - self._listeners[event_type].pop(i) + self._topics[event_type].pop(i) return True return False @@ -249,18 +251,18 @@ class Notifier(object): :param event_type: deregister listeners bound to event_type """ - return len(self._listeners.pop(event_type, [])) + return len(self._topics.pop(event_type, [])) def copy(self): c = copy.copy(self) - c._listeners = collections.defaultdict(list) - for event_type, listeners in six.iteritems(self._listeners): - c._listeners[event_type] = listeners[:] + c._topics = collections.defaultdict(list) + for (event_type, listeners) in six.iteritems(self._topics): + c._topics[event_type] = listeners[:] return c def listeners_iter(self): """Return an iterator over the mapping of event => listeners bound.""" - for event_type, listeners in six.iteritems(self._listeners): + for event_type, listeners in six.iteritems(self._topics): if listeners: yield (event_type, listeners) From 8910cdbfd00d2876b843e16d766f6b78edf9d53d Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Mon, 6 Apr 2015 11:49:16 -0700 Subject: [PATCH 129/246] Make resolution/retry strategies more clear and better This adds a separate method which is used to locate the action and handler callback that will be used to resolve the failure. This also adjusts the consultation of the parent retry (if any) to ensure that we handle the no parent retry case correctly. Once a decision has been made; return it and add logging that shows what is being activated and how many nodes were affected by the resolution strategy (which can be useful to know during debugging). Change-Id: I28101765fce000dd7c56b7c3a1fbcf1a4315799b --- taskflow/engines/action_engine/analyzer.py | 6 +- taskflow/engines/action_engine/completer.py | 131 ++++++++++++++++---- taskflow/engines/action_engine/runtime.py | 12 +- 3 files changed, 116 insertions(+), 33 deletions(-) diff --git a/taskflow/engines/action_engine/analyzer.py b/taskflow/engines/action_engine/analyzer.py index 9044fd13..a8f20c0d 100644 --- a/taskflow/engines/action_engine/analyzer.py +++ b/taskflow/engines/action_engine/analyzer.py @@ -125,9 +125,9 @@ class Analyzer(object): return all(state in (st.PENDING, st.REVERTED) for state, intention in six.itervalues(task_states)) - def iterate_subgraph(self, retry): - """Iterates a subgraph connected to given retry controller.""" - for _src, dst in traversal.dfs_edges(self._execution_graph, retry): + def iterate_subgraph(self, atom): + """Iterates a subgraph connected to given atom.""" + for _src, dst in traversal.dfs_edges(self._execution_graph, atom): yield dst def iterate_retries(self, state=None): diff --git a/taskflow/engines/action_engine/completer.py b/taskflow/engines/action_engine/completer.py index 90ed4458..7e04862c 100644 --- a/taskflow/engines/action_engine/completer.py +++ b/taskflow/engines/action_engine/completer.py @@ -14,14 +14,81 @@ # License for the specific language governing permissions and limitations # under the License. +import abc import weakref +from oslo_utils import reflection +import six + from taskflow.engines.action_engine import executor as ex +from taskflow import logging from taskflow import retry as retry_atom from taskflow import states as st from taskflow import task as task_atom from taskflow.types import failure +LOG = logging.getLogger(__name__) + + +@six.add_metaclass(abc.ABCMeta) +class _Strategy(object): + """Local/internal helper strategy base object""" + + def __init__(self, runtime): + self._runtime = runtime + + def __str__(self): + base = reflection.get_class_name(self, fully_qualified=False) + return base + "(strategy=%s)" % (self.strategy.name) + + +class _RevertAndRetry(_Strategy): + """Sets the *associated* subflow for revert to be later retried.""" + + strategy = retry_atom.RETRY + + def __init__(self, runtime, retry): + super(_RevertAndRetry, self).__init__(runtime) + self._retry = retry + + def apply(self): + tweaked = self._runtime.reset_nodes([self._retry], state=None, + intention=st.RETRY) + tweaked.extend(self._runtime.reset_subgraph(self._retry, state=None, + intention=st.REVERT)) + return tweaked + + +class _RevertAll(_Strategy): + """Sets *all* nodes/atoms to the ``REVERT`` intention.""" + + strategy = retry_atom.REVERT_ALL + + def __init__(self, runtime): + super(_RevertAll, self).__init__(runtime) + self._analyzer = runtime.analyzer + + def apply(self): + return self._runtime.reset_nodes(self._analyzer.iterate_all_nodes(), + state=None, intention=st.REVERT) + + +class _Revert(_Strategy): + """Sets atom and *associated* nodes to the ``REVERT`` intention.""" + + strategy = retry_atom.REVERT + + def __init__(self, runtime, atom): + super(_Revert, self).__init__(runtime) + self._atom = atom + + def apply(self): + tweaked = self._runtime.reset_nodes([self._atom], state=None, + intention=st.REVERT) + tweaked.extend(self._runtime.reset_subgraph(self._atom, state=None, + intention=st.REVERT)) + return tweaked + class Completer(object): """Completes atoms using actions to complete them.""" @@ -32,6 +99,7 @@ class Completer(object): self._retry_action = runtime.retry_action self._storage = runtime.storage self._task_action = runtime.task_action + self._undefined_resolver = _RevertAll(self._runtime) def _complete_task(self, task, event, result): """Completes the given task, processes task failure.""" @@ -77,6 +145,32 @@ class Completer(object): return True return False + def _determine_resolution(self, atom, failure): + """Determines which resolution strategy to activate/apply.""" + retry = self._analyzer.find_atom_retry(atom) + if retry is not None: + # Ask retry controller what to do in case of failure. + strategy = self._retry_action.on_failure(retry, atom, failure) + if strategy == retry_atom.RETRY: + return _RevertAndRetry(self._runtime, retry) + elif strategy == retry_atom.REVERT: + # Ask parent retry and figure out what to do... + parent_resolver = self._determine_resolution(retry, failure) + # Ok if the parent resolver says something not REVERT, and + # it isn't just using the undefined resolver, assume the + # parent knows best. + if parent_resolver is not self._undefined_resolver: + if parent_resolver.strategy != retry_atom.REVERT: + return parent_resolver + return _Revert(self._runtime, retry) + elif strategy == retry_atom.REVERT_ALL: + return _RevertAll(self._runtime) + else: + raise ValueError("Unknown atom failure resolution" + " action/strategy '%s'" % strategy) + else: + return self._undefined_resolver + def _process_atom_failure(self, atom, failure): """Processes atom failure & applies resolution strategies. @@ -86,30 +180,15 @@ class Completer(object): then adjust the needed other atoms intentions, and states, ... so that the failure can be worked around. """ - retry = self._analyzer.find_atom_retry(atom) - if retry is not None: - # Ask retry controller what to do in case of failure - action = self._retry_action.on_failure(retry, atom, failure) - if action == retry_atom.RETRY: - # Prepare just the surrounding subflow for revert to be later - # retried... - self._storage.set_atom_intention(retry.name, st.RETRY) - self._runtime.reset_subgraph(retry, state=None, - intention=st.REVERT) - elif action == retry_atom.REVERT: - # Ask parent checkpoint. - self._process_atom_failure(retry, failure) - elif action == retry_atom.REVERT_ALL: - # Prepare all flow for revert - self._revert_all() - else: - raise ValueError("Unknown atom failure resolution" - " action '%s'" % action) + resolver = self._determine_resolution(atom, failure) + LOG.debug("Applying resolver '%s' to resolve failure '%s'" + " of atom '%s'", resolver, failure, atom) + tweaked = resolver.apply() + # Only show the tweaked node list when blather is on, otherwise + # just show the amount/count of nodes tweaks... + if LOG.isEnabledFor(logging.BLATHER): + LOG.blather("Modified/tweaked %s nodes while applying" + " resolver '%s'", tweaked, resolver) else: - # Prepare all flow for revert - self._revert_all() - - def _revert_all(self): - """Attempts to set all nodes to the REVERT intention.""" - self._runtime.reset_nodes(self._analyzer.iterate_all_nodes(), - state=None, intention=st.REVERT) + LOG.debug("Modified/tweaked %s nodes while applying" + " resolver '%s'", len(tweaked), resolver) diff --git a/taskflow/engines/action_engine/runtime.py b/taskflow/engines/action_engine/runtime.py index a04808da..74d7cf4e 100644 --- a/taskflow/engines/action_engine/runtime.py +++ b/taskflow/engines/action_engine/runtime.py @@ -97,7 +97,10 @@ class Runtime(object): # consumption... def reset_nodes(self, nodes, state=st.PENDING, intention=st.EXECUTE): + tweaked = [] for node in nodes: + if state or intention: + tweaked.append((node, state, intention)) if state: if self.task_action.handles(node): self.task_action.change_state(node, state, @@ -109,14 +112,15 @@ class Runtime(object): % (node, type(node))) if intention: self.storage.set_atom_intention(node.name, intention) + return tweaked def reset_all(self, state=st.PENDING, intention=st.EXECUTE): - self.reset_nodes(self.analyzer.iterate_all_nodes(), - state=state, intention=intention) + return self.reset_nodes(self.analyzer.iterate_all_nodes(), + state=state, intention=intention) def reset_subgraph(self, node, state=st.PENDING, intention=st.EXECUTE): - self.reset_nodes(self.analyzer.iterate_subgraph(node), - state=state, intention=intention) + return self.reset_nodes(self.analyzer.iterate_subgraph(node), + state=state, intention=intention) def retry_subflow(self, retry): self.storage.set_atom_intention(retry.name, st.EXECUTE) From 624001a2adbb90eac7d9b461c915b701cd1abe4f Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Fri, 10 Apr 2015 17:11:21 -0700 Subject: [PATCH 130/246] Include the 'dump_memory_backend' example in the docs This seems useful to expose to people on the main docs page so include it so that it shows up (versus not showing up and being a hidden example). Change-Id: I4898ee0bff4122e30fb3cbe4af29c323f03ff2bf --- doc/source/examples.rst | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/doc/source/examples.rst b/doc/source/examples.rst index ceed11a8..715a523a 100644 --- a/doc/source/examples.rst +++ b/doc/source/examples.rst @@ -34,6 +34,18 @@ Using listeners :linenos: :lines: 16- +Dumping a in-memory backend +=========================== + +.. note:: + + Full source located at :example:`dump_memory_backend`. + +.. literalinclude:: ../../taskflow/examples/dump_memory_backend.py + :language: python + :linenos: + :lines: 16- + Making phone calls ================== From d8e76fd05a34ab5608869296c64dfe3476e2806b Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 8 Apr 2015 16:32:57 -0700 Subject: [PATCH 131/246] Ensure listener args are always a tuple/immutable Make sure the arguments are not mutable and can not be modified by anyone so that the listener maintains a read-only state after it has been created (immutability is typically always good & preferred). Change-Id: I819ea43e6e7e77179e15df3ecbb7c39e96c0592f --- taskflow/types/notifier.py | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/taskflow/types/notifier.py b/taskflow/types/notifier.py index 2257f52d..f7f339d6 100644 --- a/taskflow/types/notifier.py +++ b/taskflow/types/notifier.py @@ -37,7 +37,7 @@ class _Listener(object): the event (thus avoiding the invocation of the actual callback) :param args: non-keyworded arguments - :type args: list + :type args: list/iterable/tuple :param kwargs: key-value pair arguments :type kwargs: dictionary """ @@ -46,7 +46,10 @@ class _Listener(object): if not args: self._args = () else: - self._args = args[:] + if not isinstance(args, tuple): + self._args = tuple(args) + else: + self._args = args if not kwargs: self._kwargs = {} else: @@ -54,10 +57,12 @@ class _Listener(object): @property def kwargs(self): + """Dictionary of keyword arguments to use in future calls.""" return self._kwargs @property def args(self): + """Tuple of positional arguments to use in future calls.""" return self._args def __call__(self, event_type, details): From 15d73cb6d68da3a0de6c57e06d4e566a5829afed Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Fri, 10 Apr 2015 20:19:53 -0700 Subject: [PATCH 132/246] Expose listener notification type + docs Since this type is returned via the `listeners_iter` method we should ensure that its docs are published and that it's methods and attributes/properties are exposed so that people can know what it is and what can be used. Change-Id: I15e72df2822455be24921a4413aed3fa4f715443 --- doc/source/types.rst | 1 + taskflow/types/notifier.py | 26 +++++++++++++++++++------- 2 files changed, 20 insertions(+), 7 deletions(-) diff --git a/doc/source/types.rst b/doc/source/types.rst index 57e10986..57cebaea 100644 --- a/doc/source/types.rst +++ b/doc/source/types.rst @@ -43,6 +43,7 @@ Notifier ======== .. automodule:: taskflow.types.notifier + :special-members: __call__ Periodic ======== diff --git a/taskflow/types/notifier.py b/taskflow/types/notifier.py index f7f339d6..657ac479 100644 --- a/taskflow/types/notifier.py +++ b/taskflow/types/notifier.py @@ -25,8 +25,8 @@ import six LOG = logging.getLogger(__name__) -class _Listener(object): - """Internal helper that represents a notification listener/target.""" +class Listener(object): + """Immutable helper that represents a notification listener/target.""" def __init__(self, callback, args=None, kwargs=None, details_filter=None): """Initialize members @@ -66,6 +66,12 @@ class _Listener(object): return self._args def __call__(self, event_type, details): + """Activate the target callback with the given event + details. + + NOTE(harlowja): if a details filter callback exists and it returns + a falsey value when called with the provided ``details``, then the + target callback will **not** be called. + """ if self._details_filter is not None: if not self._details_filter(details): return @@ -101,7 +107,7 @@ class _Listener(object): return self._details_filter is None def __eq__(self, other): - if isinstance(other, _Listener): + if isinstance(other, Listener): return self.is_equivalent(other._callback, details_filter=other._details_filter) else: @@ -234,9 +240,9 @@ class Notifier(object): raise KeyError("Reserved key '%s' not allowed in " "kwargs" % k) self._topics[event_type].append( - _Listener(callback, - args=args, kwargs=kwargs, - details_filter=details_filter)) + Listener(callback, + args=args, kwargs=kwargs, + details_filter=details_filter)) def deregister(self, event_type, callback, details_filter=None): """Remove a single listener bound to event ``event_type``. @@ -266,7 +272,13 @@ class Notifier(object): return c def listeners_iter(self): - """Return an iterator over the mapping of event => listeners bound.""" + """Return an iterator over the mapping of event => listeners bound. + + NOTE(harlowja): Each listener in the yielded (event, listeners) + tuple is an instance of the :py:class:`~.Listener` type, which + itself wraps a provided callback (and its details filter + callback, if any). + """ for event_type, listeners in six.iteritems(self._topics): if listeners: yield (event_type, listeners) From 661f3b0f413c2f83f9b70cbf5222825285656d47 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Sun, 12 Apr 2015 22:55:24 -0700 Subject: [PATCH 133/246] Expose r/o listener callback + details filter callback Also makes sure that the kwargs that are returned are not mutable by returning a copy (which if modified will have no effect). Change-Id: I1d04e2084cbf93d876e946b2bd6466a4b700faad --- taskflow/types/notifier.py | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/taskflow/types/notifier.py b/taskflow/types/notifier.py index 657ac479..eacdaacc 100644 --- a/taskflow/types/notifier.py +++ b/taskflow/types/notifier.py @@ -55,10 +55,20 @@ class Listener(object): else: self._kwargs = kwargs.copy() + @property + def callback(self): + """Callback (can not be none) to call with event + details.""" + return self._callback + + @property + def details_filter(self): + """Callback (may be none) to call to discard events + details.""" + return self._details_filter + @property def kwargs(self): """Dictionary of keyword arguments to use in future calls.""" - return self._kwargs + return self._kwargs.copy() @property def args(self): From 0eee98d0d44136e13615fae479680c97e2ca368a Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 15 Apr 2015 15:25:35 -0700 Subject: [PATCH 134/246] Add job states in docs + states in python Change-Id: I25af798974dbfe61e144b15056090084aa8ef6dd --- doc/source/img/job_states.svg | 8 ++++++++ doc/source/states.rst | 23 +++++++++++++++++++++++ taskflow/states.py | 15 +++++++++++++++ tools/generate_states.sh | 4 ++++ tools/state_graph.py | 13 +++++++++++-- 5 files changed, 61 insertions(+), 2 deletions(-) create mode 100644 doc/source/img/job_states.svg diff --git a/doc/source/img/job_states.svg b/doc/source/img/job_states.svg new file mode 100644 index 00000000..17fd4e71 --- /dev/null +++ b/doc/source/img/job_states.svg @@ -0,0 +1,8 @@ + + + + + +Jobs statesUNCLAIMEDCLAIMEDCOMPLETEstart + diff --git a/doc/source/states.rst b/doc/source/states.rst index 9cc7a50a..36035480 100644 --- a/doc/source/states.rst +++ b/doc/source/states.rst @@ -194,3 +194,26 @@ already in the ``FAILURE`` state then this is a no-op). **RETRYING** - If flow that is associated with the current retry was failed and reverted, the engine prepares the flow for the next run and transitions the retry to the ``RETRYING`` state. + +Jobs +==== + +.. image:: img/job_states.svg + :width: 500px + :align: center + :alt: Job state transitions + +**UNCLAIMED** - A job (with details about what work is to be completed) has +been initially posted (by some posting entity) for work on by some other +entity (for example a :doc:`conductor `). This can also be a state +that is entered when some owning entity has manually abandoned (or +lost ownership of) a previously claimed job. + +**CLAIMED** - A job that is *actively* owned by some entity; typically that +ownership is tied to jobs persistent data via some ephemeral connection so +that the job ownership is lost (typically automatically or after some +timeout) if that ephemeral connection is lost. + +**COMPLETE** - The work defined in the job has been finished by its owning +entity and the job can no longer be processed (and it *may* be removed at +some/any point in the future). diff --git a/taskflow/states.py b/taskflow/states.py index 28272cd3..85cda2d0 100644 --- a/taskflow/states.py +++ b/taskflow/states.py @@ -53,6 +53,21 @@ SCHEDULING = 'SCHEDULING' WAITING = 'WAITING' ANALYZING = 'ANALYZING' +# Job state transitions +# See: http://docs.openstack.org/developer/taskflow/states.html + +_ALLOWED_JOB_TRANSITIONS = frozenset(( + # Job is being claimed. + (UNCLAIMED, CLAIMED), + + # Job has been lost (or manually unclaimed/abandoned). + (CLAIMED, UNCLAIMED), + + # Job has been finished. + (CLAIMED, COMPLETE), +)) + + # Flow state transitions # See: http://docs.openstack.org/developer/taskflow/states.html diff --git a/tools/generate_states.sh b/tools/generate_states.sh index 308c6400..60ca3d4b 100755 --- a/tools/generate_states.sh +++ b/tools/generate_states.sh @@ -34,3 +34,7 @@ $xsltproc $PWD/.diagram-tools/notugly.xsl /tmp/states.svg > $img_dir/retry_state echo "---- Updating wbe request state diagram ----" python $script_dir/state_graph.py -w -f /tmp/states.svg $xsltproc $PWD/.diagram-tools/notugly.xsl /tmp/states.svg > $img_dir/wbe_request_states.svg + +echo "---- Updating job state diagram ----" +python $script_dir/state_graph.py -j -f /tmp/states.svg +$xsltproc $PWD/.diagram-tools/notugly.xsl /tmp/states.svg > $img_dir/job_states.svg diff --git a/tools/state_graph.py b/tools/state_graph.py index 5ba9da7f..7711826e 100755 --- a/tools/state_graph.py +++ b/tools/state_graph.py @@ -71,7 +71,7 @@ def map_color(internal_states, state): return 'red' if state == states.REVERTED: return 'darkorange' - if state == states.SUCCESS: + if state in (states.SUCCESS, states.COMPLETE): return 'green' return None @@ -96,6 +96,10 @@ def main(): action='store_true', help="use wbe request transitions", default=False) + parser.add_option("-j", "--jobs", dest="jobs", + action='store_true', + help="use job transitions", + default=False) parser.add_option("-T", "--format", dest="format", help="output in given format", default='svg') @@ -109,9 +113,10 @@ def main(): options.retries, options.tasks, options.wbe_requests, + options.jobs, ] if sum([int(i) for i in types]) > 1: - parser.error("Only one of task/retry/engines/wbe requests" + parser.error("Only one of task/retry/engines/wbe requests/jobs" " may be specified.") internal_states = list() @@ -135,6 +140,10 @@ def main(): source_type = "WBE requests" source = make_machine(protocol.WAITING, list(protocol._ALLOWED_TRANSITIONS), []) + elif options.jobs: + source_type = "Jobs" + source = make_machine(states.UNCLAIMED, + list(states._ALLOWED_JOB_TRANSITIONS), []) else: source_type = "Flow" source = make_machine(states.PENDING, From 018dbf628d472222f6b89d9bf8f227e8f8cd68f9 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Tue, 24 Mar 2015 13:21:13 -0700 Subject: [PATCH 135/246] Have reset state handlers go through a shared list To make it hopefully easier to understand the commonality of this function and its handlers have the handlers be defined at the top of the function and use the common list in the node iteration routine when resetting/matching. Change-Id: I288eb548172b2f6f35e2bd90683f2065d3aa02dc --- taskflow/engines/action_engine/runtime.py | 21 +++++++++++++-------- 1 file changed, 13 insertions(+), 8 deletions(-) diff --git a/taskflow/engines/action_engine/runtime.py b/taskflow/engines/action_engine/runtime.py index 74d7cf4e..fc16fd9d 100644 --- a/taskflow/engines/action_engine/runtime.py +++ b/taskflow/engines/action_engine/runtime.py @@ -98,18 +98,23 @@ class Runtime(object): def reset_nodes(self, nodes, state=st.PENDING, intention=st.EXECUTE): tweaked = [] + node_state_handlers = [ + (self.task_action, {'progress': 0.0}), + (self.retry_action, {}), + ] for node in nodes: if state or intention: tweaked.append((node, state, intention)) if state: - if self.task_action.handles(node): - self.task_action.change_state(node, state, - progress=0.0) - elif self.retry_action.handles(node): - self.retry_action.change_state(node, state) - else: - raise TypeError("Unknown how to reset atom '%s' (%s)" - % (node, type(node))) + handled = False + for h, kwargs in node_state_handlers: + if h.handles(node): + h.change_state(node, state, **kwargs) + handled = True + break + if not handled: + raise TypeError("Unknown how to reset state of" + " node '%s' (%s)" % (node, type(node))) if intention: self.storage.set_atom_intention(node.name, intention) return tweaked From 0a94d68e141599e291d92d7beb3d2f7cda0c34b4 Mon Sep 17 00:00:00 2001 From: Doug Hellmann Date: Thu, 16 Apr 2015 18:15:17 +0000 Subject: [PATCH 136/246] Uncap library requirements for liberty Change-Id: I86914b57e7bc8f7e525cc9f67a243c476d4fcd21 Depends-On: Ib948b756b8e6ca47a4c9c44c48031e54b7386a06 --- requirements.txt | 8 ++++---- test-requirements.txt | 4 ++-- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/requirements.txt b/requirements.txt index 23c0a2d0..8e0c5f3d 100644 --- a/requirements.txt +++ b/requirements.txt @@ -20,7 +20,7 @@ enum34 networkx>=1.8 # Used for backend storage engine loading. -stevedore>=1.3.0,<1.4.0 # Apache-2.0 +stevedore>=1.3.0 # Apache-2.0 # Backport for concurrent.futures which exists in 3.2+ futures>=2.1.6 @@ -29,8 +29,8 @@ futures>=2.1.6 jsonschema>=2.0.0,<3.0.0 # For common utilities -oslo.utils>=1.4.0,<1.5.0 # Apache-2.0 -oslo.serialization>=1.4.0,<1.5.0 # Apache-2.0 +oslo.utils>=1.4.0 # Apache-2.0 +oslo.serialization>=1.4.0 # Apache-2.0 # For deprecation of things -debtcollector>=0.3.0,<0.4.0 # Apache-2.0 +debtcollector>=0.3.0 # Apache-2.0 diff --git a/test-requirements.txt b/test-requirements.txt index 23d9ac3a..939b2ffd 100644 --- a/test-requirements.txt +++ b/test-requirements.txt @@ -3,7 +3,7 @@ # process, which may cause wedges in the gate later. hacking<0.11,>=0.10.0 -oslotest>=1.5.1,<1.6.0 # Apache-2.0 +oslotest>=1.5.1 # Apache-2.0 mock>=1.0 testtools>=0.9.36,!=1.2.0 testscenarios>=0.4 @@ -30,4 +30,4 @@ psycopg2 # Docs build jobs need these packages. sphinx>=1.1.2,!=1.2.0,!=1.3b1,<1.3 -oslosphinx>=2.5.0,<2.6.0 # Apache-2.0 +oslosphinx>=2.5.0 # Apache-2.0 From 50eb38707a03473351aa38608a88df079fb18e1a Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Thu, 16 Apr 2015 11:42:42 -0700 Subject: [PATCH 137/246] Add strict job state transition checking Also adds basic unit tests to test that the states module functions have some level of testing to ensure they don't break. Change-Id: Icedd6e70820bcf484c0dace76f2acf01d9bc967e --- taskflow/jobs/backends/impl_zookeeper.py | 8 +++ taskflow/states.py | 18 ++++- taskflow/tests/unit/test_states.py | 87 ++++++++++++++++++++++++ 3 files changed, 112 insertions(+), 1 deletion(-) create mode 100644 taskflow/tests/unit/test_states.py diff --git a/taskflow/jobs/backends/impl_zookeeper.py b/taskflow/jobs/backends/impl_zookeeper.py index 34ec5b42..b587925e 100644 --- a/taskflow/jobs/backends/impl_zookeeper.py +++ b/taskflow/jobs/backends/impl_zookeeper.py @@ -93,6 +93,7 @@ class ZookeeperJob(base.Job): basename = k_paths.basename(self._path) self._root = self._path[0:-len(basename)] self._sequence = int(basename[len(JOB_PREFIX):]) + self._last_state = None @property def lock_path(self): @@ -182,6 +183,13 @@ class ZookeeperJob(base.Job): @property def state(self): + current_state = self._fetch_state() + if self._last_state is not None: + states.check_job_transition(self._last_state, current_state) + self._last_state = current_state + return current_state + + def _fetch_state(self): owner = self.board.find_owner(self) job_data = {} try: diff --git a/taskflow/states.py b/taskflow/states.py index 85cda2d0..c5ea579e 100644 --- a/taskflow/states.py +++ b/taskflow/states.py @@ -68,6 +68,22 @@ _ALLOWED_JOB_TRANSITIONS = frozenset(( )) +def check_job_transition(old_state, new_state): + """Check that job can transition from old_state to new_state. + + If transition can be performed, it returns True. If transition + should be ignored, it returns False. If transition is not + valid, it raises an InvalidState exception. + """ + if old_state == new_state: + return False + pair = (old_state, new_state) + if pair in _ALLOWED_JOB_TRANSITIONS: + return True + raise exc.InvalidState("Job transition from '%s' to '%s' is not allowed" + % pair) + + # Flow state transitions # See: http://docs.openstack.org/developer/taskflow/states.html @@ -135,7 +151,7 @@ def check_flow_transition(old_state, new_state): return True if pair in _IGNORED_FLOW_TRANSITIONS: return False - raise exc.InvalidState("Flow transition from %s to %s is not allowed" + raise exc.InvalidState("Flow transition from '%s' to '%s' is not allowed" % pair) diff --git a/taskflow/tests/unit/test_states.py b/taskflow/tests/unit/test_states.py new file mode 100644 index 00000000..701a6888 --- /dev/null +++ b/taskflow/tests/unit/test_states.py @@ -0,0 +1,87 @@ +# -*- coding: utf-8 -*- + +# Copyright (C) 2015 Yahoo! Inc. 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. + +from taskflow import exceptions as excp +from taskflow import states +from taskflow import test + + +class TestStates(test.TestCase): + def test_valid_flow_states(self): + for start_state, end_state in states._ALLOWED_FLOW_TRANSITIONS: + self.assertTrue(states.check_flow_transition(start_state, + end_state)) + + def test_ignored_flow_states(self): + for start_state, end_state in states._IGNORED_FLOW_TRANSITIONS: + self.assertFalse(states.check_flow_transition(start_state, + end_state)) + + def test_invalid_flow_states(self): + invalids = [ + # Not a comprhensive set/listing... + (states.RUNNING, states.PENDING), + (states.REVERTED, states.RUNNING), + (states.RESUMING, states.RUNNING), + ] + for start_state, end_state in invalids: + self.assertRaises(excp.InvalidState, + states.check_flow_transition, + start_state, end_state) + + def test_valid_job_states(self): + for start_state, end_state in states._ALLOWED_JOB_TRANSITIONS: + self.assertTrue(states.check_job_transition(start_state, + end_state)) + + def test_ignored_job_states(self): + ignored = [] + for start_state, end_state in states._ALLOWED_JOB_TRANSITIONS: + ignored.append((start_state, start_state)) + ignored.append((end_state, end_state)) + for start_state, end_state in ignored: + self.assertFalse(states.check_job_transition(start_state, + end_state)) + + def test_invalid_job_states(self): + invalids = [ + (states.COMPLETE, states.UNCLAIMED), + (states.UNCLAIMED, states.COMPLETE), + ] + for start_state, end_state in invalids: + self.assertRaises(excp.InvalidState, + states.check_job_transition, + start_state, end_state) + + def test_valid_task_states(self): + for start_state, end_state in states._ALLOWED_TASK_TRANSITIONS: + self.assertTrue(states.check_task_transition(start_state, + end_state)) + + def test_invalid_task_states(self): + invalids = [ + # Not a comprhensive set/listing... + (states.RUNNING, states.PENDING), + (states.PENDING, states.REVERTED), + (states.PENDING, states.SUCCESS), + (states.PENDING, states.FAILURE), + (states.RETRYING, states.PENDING), + ] + for start_state, end_state in invalids: + # TODO(harlowja): fix this so that it raises instead of + # returning false... + self.assertFalse( + states.check_task_transition(start_state, end_state)) From 672e1f44c567c9322acd763415952e7813ef6407 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Thu, 16 Apr 2015 12:14:07 -0700 Subject: [PATCH 138/246] Add states generating venv and use pydot2 Make it easier for others to create the states diagrams and use a non-broken pydot fork that is actually being maintained (pydot2). Change-Id: Ia16c53a513159a8f15c5d7c75e22553dc1273b37 --- tools/state_graph.py | 5 +++-- tools/{generate_states.sh => update_states.sh} | 0 tox.ini | 7 +++++++ 3 files changed, 10 insertions(+), 2 deletions(-) rename tools/{generate_states.sh => update_states.sh} (100%) diff --git a/tools/state_graph.py b/tools/state_graph.py index 7711826e..31961404 100755 --- a/tools/state_graph.py +++ b/tools/state_graph.py @@ -22,8 +22,9 @@ top_dir = os.path.abspath(os.path.join(os.path.dirname(__file__), os.pardir)) sys.path.insert(0, top_dir) -# To get this installed you may have to follow: -# https://code.google.com/p/pydot/issues/detail?id=93 (until fixed). +# To get this installed you have to do the following: +# +# $ pip install pydot2 import pydot from taskflow.engines.action_engine import runner diff --git a/tools/generate_states.sh b/tools/update_states.sh similarity index 100% rename from tools/generate_states.sh rename to tools/update_states.sh diff --git a/tox.ini b/tox.ini index d87ed9a8..85545cac 100644 --- a/tox.ini +++ b/tox.ini @@ -11,6 +11,7 @@ envlist = cover, py33, py34, pylint, + update-states [testenv] usedevelop = True @@ -26,6 +27,12 @@ deps = {[testenv:py27]deps} commands = python setup.py build_sphinx doc8 doc/source +[testenv:update-states] +basepython = python2.7 +deps = {[testenv:py27]deps} + pydot2 +commands = {toxinidir}/tools/update_states.sh + [tox:jenkins] downloadcache = ~/cache/pip From 18d3cd15672416f5c6324b50774ea8ac87aecfad Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Fri, 17 Apr 2015 21:04:24 -0700 Subject: [PATCH 139/246] Remove link to kazoo eventlet handler This special pypi package and link is no longer needed since kazoo itself has a in-tree supported eventlet handler that can now be used instead. Change-Id: I8587e22bfc13c9c5d76971cd5ff94c0cf93b6d93 --- doc/source/jobs.rst | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/doc/source/jobs.rst b/doc/source/jobs.rst index d25436ce..0374475a 100644 --- a/doc/source/jobs.rst +++ b/doc/source/jobs.rst @@ -200,7 +200,7 @@ Additional *configuration* parameters: * ``handler``: a class that provides ``kazoo.handlers``-like interface; it will be used internally by `kazoo`_ to perform asynchronous operations, useful when your program uses eventlet and you want to instruct kazoo to use an - eventlet compatible handler (such as the `eventlet handler`_). + eventlet compatible handler. .. note:: @@ -278,5 +278,4 @@ Hierarchy .. _paradigm shift: https://wiki.openstack.org/wiki/TaskFlow/Paradigm_shifts#Workflow_ownership_transfer .. _zookeeper: http://zookeeper.apache.org/ .. _kazoo: http://kazoo.readthedocs.org/ -.. _eventlet handler: https://pypi.python.org/pypi/kazoo-eventlet-handler/ .. _stevedore: http://stevedore.readthedocs.org/ From c76ee2557dd6968f6e1ea0fc6de2dae05c1889a3 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Fri, 17 Apr 2015 17:55:31 -0700 Subject: [PATCH 140/246] Validate correct exception subclass in 'raise_with_cause' Change-Id: Ia1a8b692eda2f3856bb4e23a9baa34ac06096d3a --- taskflow/exceptions.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/taskflow/exceptions.py b/taskflow/exceptions.py index 15a73b3b..3d153ffc 100644 --- a/taskflow/exceptions.py +++ b/taskflow/exceptions.py @@ -46,6 +46,8 @@ def raise_with_cause(exc_cls, message, *args, **kwargs): :param kwargs: any additional keyword arguments to pass to the exceptions constructor. """ + if not issubclass(exc_cls, TaskFlowException): + raise ValueError("Subclass of taskflow exception is required") if 'cause' not in kwargs: exc_type, exc, exc_tb = sys.exc_info() if exc is not None: From 4b0091a82fb508037495c418d98e2be52ecd47f5 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Mon, 20 Apr 2015 14:48:20 -0700 Subject: [PATCH 141/246] Avoid attribute error by checking executor for being non-none Change-Id: I47e7a14a93026918f4a469139ca6a53dbaa1f702 --- taskflow/jobs/backends/impl_zookeeper.py | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/taskflow/jobs/backends/impl_zookeeper.py b/taskflow/jobs/backends/impl_zookeeper.py index 7aea3517..c73c1598 100644 --- a/taskflow/jobs/backends/impl_zookeeper.py +++ b/taskflow/jobs/backends/impl_zookeeper.py @@ -381,12 +381,16 @@ class ZookeeperJobBoard(base.NotifyingJobBoard): self._connected = False def _emit(self, state, details): - # Submit the work to the executor to avoid blocking the kazoo queue. + # Submit the work to the executor to avoid blocking the kazoo threads + # and queue(s)... + worker = self._worker + if worker is None: + return try: - self._worker.submit(self.notifier.notify, state, details) - except (AttributeError, RuntimeError): - # Notification thread is shutdown or non-existent, either case we - # just want to skip submitting a notification... + worker.submit(self.notifier.notify, state, details) + except RuntimeError: + # Notification thread is shutdown just skip submitting a + # notification... pass @property From 46166701f4923920bff901993d6a0ab421fecf9d Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Mon, 20 Apr 2015 14:51:39 -0700 Subject: [PATCH 142/246] Replace more instance(s) of exception chaining with helper Change-Id: I5a35194022656786c329ea3586366d3b00b461ba --- taskflow/jobs/backends/impl_zookeeper.py | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/taskflow/jobs/backends/impl_zookeeper.py b/taskflow/jobs/backends/impl_zookeeper.py index 7aea3517..ce8363c2 100644 --- a/taskflow/jobs/backends/impl_zookeeper.py +++ b/taskflow/jobs/backends/impl_zookeeper.py @@ -665,7 +665,8 @@ class ZookeeperJobBoard(base.NotifyingJobBoard): owner_data = self._get_owner_and_data(job) lock_data, lock_stat, data, data_stat = owner_data except k_exceptions.NoNodeError: - raise excp.JobFailure("Can not consume a job %s" + excp.raise_with_cause(excp.JobFailure, + "Can not consume a job %s" " which we can not determine" " the owner of" % (job.uuid)) if lock_data.get("owner") != who: @@ -685,7 +686,8 @@ class ZookeeperJobBoard(base.NotifyingJobBoard): owner_data = self._get_owner_and_data(job) lock_data, lock_stat, data, data_stat = owner_data except k_exceptions.NoNodeError: - raise excp.JobFailure("Can not abandon a job %s" + excp.raise_with_cause(excp.JobFailure, + "Can not abandon a job %s" " which we can not determine" " the owner of" % (job.uuid)) if lock_data.get("owner") != who: @@ -703,17 +705,16 @@ class ZookeeperJobBoard(base.NotifyingJobBoard): owner_data = self._get_owner_and_data(job) lock_data, lock_stat, data, data_stat = owner_data except k_exceptions.NoNodeError: - raise excp.JobFailure("Can not trash a job %s" + excp.raise_with_cause(excp.JobFailure, + "Can not trash a job %s" " which we can not determine" " the owner of" % (job.uuid)) if lock_data.get("owner") != who: raise excp.JobFailure("Can not trash a job %s" " which is not owned by %s" % (job.uuid, who)) - trash_path = job.path.replace(self.path, self.trash_path) value = misc.binary_encode(jsonutils.dumps(data)) - txn = self._client.transaction() txn.create(trash_path, value=value) txn.delete(job.lock_path, version=lock_stat.version) From 3bbbcc6842df5e12e10a17914118d9278738c465 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Fri, 17 Oct 2014 19:27:19 -0700 Subject: [PATCH 143/246] Add a conductor running example Create an example which can be extended to create a simplistic review checkout, tox running system which can be used to run some set of actions on every review posted. This could be expanded and connected into a gerrit pipeline to create a mini-jenkins like trigger/build/result system. Part of ongoing blueprint more-examples Change-Id: I5cf1bf02eeddf897ac7f098f1d73377f262a267b --- doc/source/examples.rst | 12 ++ taskflow/examples/tox_conductor.py | 243 +++++++++++++++++++++++++++++ 2 files changed, 255 insertions(+) create mode 100644 taskflow/examples/tox_conductor.py diff --git a/doc/source/examples.rst b/doc/source/examples.rst index 715a523a..64229f1b 100644 --- a/doc/source/examples.rst +++ b/doc/source/examples.rst @@ -330,3 +330,15 @@ Jobboard producer/consumer (simple) :language: python :linenos: :lines: 16- + +Conductor simulating a CI pipeline +================================== + +.. note:: + + Full source located at :example:`tox_conductor` + +.. literalinclude:: ../../taskflow/examples/tox_conductor.py + :language: python + :linenos: + :lines: 16- diff --git a/taskflow/examples/tox_conductor.py b/taskflow/examples/tox_conductor.py new file mode 100644 index 00000000..feff4245 --- /dev/null +++ b/taskflow/examples/tox_conductor.py @@ -0,0 +1,243 @@ +# -*- coding: utf-8 -*- + +# Copyright (C) 2014 Yahoo! Inc. 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. + +import contextlib +import itertools +import logging +import os +import shutil +import socket +import sys +import tempfile +import threading +import time + +logging.basicConfig(level=logging.ERROR) + +top_dir = os.path.abspath(os.path.join(os.path.dirname(__file__), + os.pardir, + os.pardir)) +sys.path.insert(0, top_dir) + +from oslo_utils import timeutils +from oslo_utils import uuidutils +import six +from zake import fake_client + +from taskflow.conductors import backends as conductors +from taskflow import engines +from taskflow.jobs import backends as boards +from taskflow.patterns import linear_flow +from taskflow.persistence import backends as persistence +from taskflow.persistence import logbook +from taskflow import task +from taskflow.utils import threading_utils + +# INTRO: This examples shows how a worker/producer can post desired work (jobs) +# to a jobboard and a conductor can consume that work (jobs) from that jobboard +# and execute those jobs in a reliable & async manner (for example, if the +# conductor were to crash then the job will be released back onto the jobboard +# and another conductor can attempt to finish it, from wherever that job last +# left off). +# +# In this example a in-memory jobboard (and in-memory storage) is created and +# used that simulates how this would be done at a larger scale (it is an +# example after all). + +# Restrict how long this example runs for... +RUN_TIME = 5 +REVIEW_CREATION_DELAY = 0.5 +SCAN_DELAY = 0.1 +NAME = "%s_%s" % (socket.getfqdn(), os.getpid()) + +# This won't really use zookeeper but will use a local version of it using +# the zake library that mimics an actual zookeeper cluster using threads and +# an in-memory data structure. +JOBBOARD_CONF = { + 'board': 'zookeeper://localhost?path=/taskflow/tox/jobs', +} + + +class RunReview(task.Task): + # A dummy task that clones the review and runs tox... + + def _clone_review(self, review, temp_dir): + print("Cloning review '%s' into %s" % (review['id'], temp_dir)) + + def _run_tox(self, temp_dir): + print("Running tox in %s" % temp_dir) + + def execute(self, review, temp_dir): + self._clone_review(review, temp_dir) + self._run_tox(temp_dir) + + +class MakeTempDir(task.Task): + # A task that creates and destroys a temporary dir (on failure). + # + # It provides the location of the temporary dir for other tasks to use + # as they see fit. + + default_provides = 'temp_dir' + + def execute(self): + return tempfile.mkdtemp() + + def revert(self, *args, **kwargs): + temp_dir = kwargs.get(task.REVERT_RESULT) + if temp_dir: + shutil.rmtree(temp_dir) + + +class CleanResources(task.Task): + # A task that cleans up any workflow resources. + + def execute(self, temp_dir): + print("Removing %s" % temp_dir) + shutil.rmtree(temp_dir) + + +def review_iter(): + """Makes reviews (never-ending iterator/generator).""" + review_id_gen = itertools.count(0) + while True: + review_id = six.next(review_id_gen) + review = { + 'id': review_id, + } + yield review + + +# The reason this is at the module namespace level is important, since it must +# be accessible from a conductor dispatching an engine, if it was a lambda +# function for example, it would not be reimportable and the conductor would +# be unable to reference it when creating the workflow to run. +def create_review_workflow(): + """Factory method used to create a review workflow to run.""" + f = linear_flow.Flow("tester") + f.add( + MakeTempDir(name="maker"), + RunReview(name="runner"), + CleanResources(name="cleaner") + ) + return f + + +def generate_reviewer(client, saver, name=NAME): + """Creates a review producer thread with the given name prefix.""" + real_name = "%s_reviewer" % name + no_more = threading.Event() + jb = boards.fetch(real_name, JOBBOARD_CONF, + client=client, persistence=saver) + + def make_save_book(saver, review_id): + # Record what we want to happen (sometime in the future). + book = logbook.LogBook("book_%s" % review_id) + detail = logbook.FlowDetail("flow_%s" % review_id, + uuidutils.generate_uuid()) + book.add(detail) + # Associate the factory method we want to be called (in the future) + # with the book, so that the conductor will be able to call into + # that factory to retrieve the workflow objects that represent the + # work. + # + # These args and kwargs *can* be used to save any specific parameters + # into the factory when it is being called to create the workflow + # objects (typically used to tell a factory how to create a unique + # workflow that represents this review). + factory_args = () + factory_kwargs = {} + engines.save_factory_details(detail, create_review_workflow, + factory_args, factory_kwargs) + with contextlib.closing(saver.get_connection()) as conn: + conn.save_logbook(book) + return book + + def run(): + """Periodically publishes 'fake' reviews to analyze.""" + jb.connect() + review_generator = review_iter() + with contextlib.closing(jb): + while not no_more.is_set(): + review = six.next(review_generator) + details = { + 'store': { + 'review': review, + }, + } + job_name = "%s_%s" % (real_name, review['id']) + print("Posting review '%s'" % review['id']) + jb.post(job_name, + book=make_save_book(saver, review['id']), + details=details) + time.sleep(REVIEW_CREATION_DELAY) + + # Return the unstarted thread, and a callback that can be used + # shutdown that thread (to avoid running forever). + return (threading_utils.daemon_thread(target=run), no_more.set) + + +def generate_conductor(client, saver, name=NAME): + """Creates a conductor thread with the given name prefix.""" + real_name = "%s_conductor" % name + jb = boards.fetch(name, JOBBOARD_CONF, + client=client, persistence=saver) + conductor = conductors.fetch("blocking", real_name, jb, + engine='parallel', wait_timeout=SCAN_DELAY) + + def run(): + jb.connect() + with contextlib.closing(jb): + conductor.run() + + # Return the unstarted thread, and a callback that can be used + # shutdown that thread (to avoid running forever). + return (threading_utils.daemon_thread(target=run), conductor.stop) + + +def main(): + # Need to share the same backend, so that data can be shared... + persistence_conf = { + 'connection': 'memory', + } + saver = persistence.fetch(persistence_conf) + with contextlib.closing(saver.get_connection()) as conn: + # This ensures that the needed backend setup/data directories/schema + # upgrades and so on... exist before they are attempted to be used... + conn.upgrade() + fc1 = fake_client.FakeClient() + # Done like this to share the same client storage location so the correct + # zookeeper features work across clients... + fc2 = fake_client.FakeClient(storage=fc1.storage) + entities = [ + generate_reviewer(fc1, saver), + generate_conductor(fc2, saver), + ] + for t, stopper in entities: + t.start() + try: + watch = timeutils.StopWatch(duration=RUN_TIME) + watch.start() + while not watch.expired(): + time.sleep(0.1) + finally: + for t, stopper in reversed(entities): + stopper() + t.join() + + +if __name__ == '__main__': + main() From 43194f554e4951ca400d1bea3cf61adfa19292ca Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 22 Apr 2015 14:38:16 -0700 Subject: [PATCH 144/246] Test more engine types in argument passing unit test Change-Id: I8cdca39b488fecef495a501640a1b2c7efaf6b79 --- taskflow/tests/unit/test_arguments_passing.py | 47 +++++++++++++++++-- 1 file changed, 42 insertions(+), 5 deletions(-) diff --git a/taskflow/tests/unit/test_arguments_passing.py b/taskflow/tests/unit/test_arguments_passing.py index c84d8534..a1ba2ac8 100644 --- a/taskflow/tests/unit/test_arguments_passing.py +++ b/taskflow/tests/unit/test_arguments_passing.py @@ -14,10 +14,14 @@ # License for the specific language governing permissions and limitations # under the License. +import testtools + import taskflow.engines from taskflow import exceptions as exc from taskflow import test from taskflow.tests import utils +from taskflow.types import futures +from taskflow.utils import eventlet_utils as eu class ArgumentsPassingTest(utils.EngineTestBase): @@ -160,8 +164,8 @@ class ArgumentsPassingTest(utils.EngineTestBase): }) -class SingleThreadedEngineTest(ArgumentsPassingTest, - test.TestCase): +class SerialEngineTest(ArgumentsPassingTest, test.TestCase): + def _make_engine(self, flow, flow_detail=None): return taskflow.engines.load(flow, flow_detail=flow_detail, @@ -169,10 +173,43 @@ class SingleThreadedEngineTest(ArgumentsPassingTest, backend=self.backend) -class MultiThreadedEngineTest(ArgumentsPassingTest, - test.TestCase): +class ParallelEngineWithThreadsTest(ArgumentsPassingTest, test.TestCase): + _EXECUTOR_WORKERS = 2 + def _make_engine(self, flow, flow_detail=None, executor=None): - return taskflow.engines.load(flow, flow_detail=flow_detail, + if executor is None: + executor = 'threads' + return taskflow.engines.load(flow, + flow_detail=flow_detail, engine='parallel', backend=self.backend, + executor=executor, + max_workers=self._EXECUTOR_WORKERS) + + +@testtools.skipIf(not eu.EVENTLET_AVAILABLE, 'eventlet is not available') +class ParallelEngineWithEventletTest(ArgumentsPassingTest, test.TestCase): + + def _make_engine(self, flow, flow_detail=None, executor=None): + if executor is None: + executor = futures.GreenThreadPoolExecutor() + self.addCleanup(executor.shutdown) + return taskflow.engines.load(flow, + flow_detail=flow_detail, + backend=self.backend, + engine='parallel', executor=executor) + + +class ParallelEngineWithProcessTest(ArgumentsPassingTest, test.TestCase): + _EXECUTOR_WORKERS = 2 + + def _make_engine(self, flow, flow_detail=None, executor=None): + if executor is None: + executor = 'processes' + return taskflow.engines.load(flow, + flow_detail=flow_detail, + backend=self.backend, + engine='parallel', + executor=executor, + max_workers=self._EXECUTOR_WORKERS) From d80ee56ea6259265a534231a52146f9fd04c9689 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 22 Apr 2015 17:13:20 -0700 Subject: [PATCH 145/246] Use oslo_utils eventletutils to warn about eventlet patching Change-Id: I86ba0de51b5c5789efae187ebc1c46ae32ff8b8b --- taskflow/engines/__init__.py | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/taskflow/engines/__init__.py b/taskflow/engines/__init__.py index 942afd66..24669cbb 100644 --- a/taskflow/engines/__init__.py +++ b/taskflow/engines/__init__.py @@ -14,8 +14,16 @@ # License for the specific language governing permissions and limitations # under the License. +from oslo_utils import eventletutils as _eventletutils -# promote helpers to this module namespace +# Give a nice warning that if eventlet is being used these modules +# are highly recommended to be patched (or otherwise bad things could +# happen). +_eventletutils.warn_eventlet_not_patched( + expected_patched_modules=['time', 'thread']) + + +# Promote helpers to this module namespace (for easy access). from taskflow.engines.helpers import flow_from_detail # noqa from taskflow.engines.helpers import load # noqa from taskflow.engines.helpers import load_from_detail # noqa From 518634101d8f43715fb0911b90a3346f2cf0f6fd Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 22 Apr 2015 17:40:07 -0700 Subject: [PATCH 146/246] Add openclipart.org conductor image to conductor docs Change-Id: Ic9adb307cdeaded768e84ba3b09b298a7e8d61e5 --- doc/source/conductors.rst | 11 ++++++++--- doc/source/img/conductor.png | Bin 0 -> 9664 bytes 2 files changed, 8 insertions(+), 3 deletions(-) create mode 100644 doc/source/img/conductor.png diff --git a/doc/source/conductors.rst b/doc/source/conductors.rst index a819c8a0..efd9bf7e 100644 --- a/doc/source/conductors.rst +++ b/doc/source/conductors.rst @@ -2,6 +2,10 @@ Conductors ---------- +.. image:: img/conductor.png + :width: 97px + :alt: Conductor + Overview ======== @@ -18,14 +22,14 @@ They are responsible for the following: tasks and flows to be executed). * Dispatching the engine using the provided :doc:`persistence ` layer and engine configuration. -* Completing or abandoning the claimed job (depending on dispatching and - execution outcome). +* Completing or abandoning the claimed :doc:`job ` (depending on + dispatching and execution outcome). * *Rinse and repeat*. .. note:: They are inspired by and have similar responsibilities - as `railroad conductors`_. + as `railroad conductors`_ or `musical conductors`_. Considerations ============== @@ -76,4 +80,5 @@ Hierarchy taskflow.conductors.backends.impl_blocking :parts: 1 +.. _musical conductors: http://en.wikipedia.org/wiki/Conducting .. _railroad conductors: http://en.wikipedia.org/wiki/Conductor_%28transportation%29 diff --git a/doc/source/img/conductor.png b/doc/source/img/conductor.png new file mode 100644 index 0000000000000000000000000000000000000000..98178c081d1e1670ab74dc055ea24ff98b8b6871 GIT binary patch literal 9664 zcmYj%2UwHI_P$;hy`q3w2p|e-Sd=1Q1OkG9N&sn6LTI5#kQxF~LKSpTFodQe5UM~R zKtcimLJ4ir1QbDO2^iWMmM$RDr231y_ul{iJWpoke9xT8%$b>Y-Z|fwXm!U}NKi`f z(4j*@CbywB{QUKyLx(T_`xkymr@v&6pB=w{+cD(Op%bD%e}@m{cPk74pd};S?C9?p0|MJv2eyEVrqZH${D{Hxi@M zLiY>#qR2xHNUYZet&(+TQOVt zX!DpbgkH3O4OXYnt4jFhr}Yok4W@yX{zy5XLSh+J(igou@NKf^CL=>Y)Y3xs=+ z`OjB@AS15zJOA2(Ktm=yN-^8QPW=|Ng_`!_)4iaTM63vfau1R{nFK=S9M+c4R%S#x zK$-6|AzKSRLhYf&u>B9h+IT|feKHoS@PWe~D}5}Kw;Ysi2!*yRsgF82i?psoj`>W1 z0D#q0&mQT_;Fe<+Vt{`{)O=mSEceJ+(k&>{VRX9)(V0R_#A31UCf3WNlhs|Jd}*2U zIf3tme9gd?t9Psl#+GYeL&RgT;D9i^g)%WEHT8<~&uUPp?f0@*ITZ2LIBdnTsQ0eB z)4OI6vQ7z2Z1r1`UV#1QF9CqGPYxaagZ&r~^2v8O&i(B6PiC6>zqsW2*Oae68V^+; z&(!;G{{YrS&dj?yC~t^@mQg9T{K_Bq0N{?pf^!R!PZ#grsl$mFp4=40NSE6D(*=HN zUYiJuQA9qOT=0i{>XZP1c)HP_I`0NiVkZErPbOy6+tp2ytJ|;AG53Ft^uID3I>1Ri zn)H1j5CcK008X&72N=VeDv~=U@Cv8Dj zb(?SuW;IpPPxc%(Sd@jh(?947`bUr@kLmDfO}@g@BrftWt0@L~_aJit)rNZCdZ#(7qZ7P`{q~y=g}U=<#I*L68pAGsa<$|EI0C;7l1mu{{J7%AA5! z=f(FDPMYo99t(hc>ebO+oQ_@1_Jr$j{!K=jh%R{t0aFKRAfl~#pCKC#vc$cTuC6fNSM8g`^;-5!Etnw_&ZrpR+*n#y@6@jZ#a2uMhZ+h@)6oHVJyOpUDu|kE zXdYG>@*?>(0vZm4$FY~}3zaL<( z+9t13+{(ie@OR^|SWk^!2#BWwOIc42UZN#^&#z^rdp(X!v8{Mb2%x8h7&33`vwnrZ z`-W|eM{O=o?S|p7k|pLD1)Zn80J;{UfW>lJdAjZwgL9E@?us;zrl^GB48Gj6SW_qu zF2`ckFbxe3CKvgq&!qiVQRe-0D70yx-N?@6ZVx%Oxt+kDFCJJR?Yp+1 zHQRp$(Bk}z75IzlRYbo|Cx4GfP6=p!zYT?|emTDh_dG#s`+?i-Ia-zUu2bio2Sckf z3xvc<=_Gl$%&NWL)yr1^QSB)CpIyI+h#Eqfw@5;vO_tE)pg0IgY`w1LVGq2=_Z<<9`^xi=7OZsU!-vXkh_IZaTv^8D`o zENm`Y%R2A(Z3}TgP*L{0>>b!7k-yH%tUO5v;N6pcGi^Id}bE}(sA91I|*fWlcdilH1|UR0I;LI zS}l3RHO7;_`3f-4by~>Wc0P#N?^hP3&qp=dm7?ffq=~N;sZP!vb_tPxkcf!4gT%QN z*oSpMi}|kWn2lwlTm9Rd?%c?_!Y2P^RI(xA@&4+PT$K*Oyv_(<{|dW z!SL<}_ij!mQ90k*@-6JEe}8Hblopo>yH5sy^x%}cjS=j3oZxmW_DJv8Yoi>mGaE6= zARa8b$*RfT*4LZFUtSRL6OC4TW)g!-J~(!Ds&)5+z$iw4DI+`J_#39H)4)v-veM5C zS2ezD003C~9!~jm=Rt9sCI9RY2Z)r{v|zE{3FC&xz7!2?_(gZb&-nyR)=qBsRMo)= z1mooTy3D~hWuRiR;xVjSc})ry>pkS-eL+q8*7d5k21DrC`LY1fSyC_`%b!Q-+^&Op zJ{leh%TDhR_mmM2oU1a@Pfw<|O64n;C(VSmVn=K<7Fj1auPynO#azo)_r@@i@`CoP z)H|uHauP;)QUhT0`X*X$+TC&?@7V0uvm8QtXcm# zvm~tGC^;DKa64mC%0D$YUzRQ3c#A2Q`K=M#rO_Z3^l+2|cs0QIhpnwJNN=^0YeO zy;;N%>=cA&WnX9DhpU(Mb*f(d=B4UsKD%ZL!YqoKT6HXJ^Hpl4v;&t{IkReS|}sBf9@Z)VWoP#p-G_xLBNe zrKI^b1cZFF6V*8K+iIqe?RLCmVSVb1eRV7BIw*GO97yFWUdz;F6nnzji`3flJNH_z zy6W6LhRe$&Z15SM0JEkc2mbN>v|>dZ@NmGDJ`LY-%4T#YuRg=yJD!{5d!RB=-N@Jk zYPuh$#a|oxm>irRHxE&Ff+p9}W`^0Z1wg>GovnR%x_L+;_dwascKF5=T7`E!EwS0+ zYfHewZbhS*KHx;Ihw`G?7XD-C8LXgc#DrKg>a;%O#-3BmZlVF3p7Fhu%K?;eEt0KN zkU1B`Ap`+d)5x9BtH%}t@oCD`&NP;R>tII)&p|-hU5-ZGp{~&&3tX!&#QI|oJYIxeCfKh9{jR_zURq}A`ZUSWB1suPx^d7%$A zN3>|q=Q*d!NLSm{l7UaNn?B#^JZG4F&H2&~@W40HI+$%!{7R90+N&f6s9qJ%vRhD8 z`?kqJXVujOmAe@X@J9-f;6yityePqz6=ae*(+Px4IOqwb(>N8DVb z`2fi0A?FA8p?MV%Tg@l8w1-7rx9EYPn);i1G9j5#0yvBTMdAmQjPlKIph0bhwg|UH8rzAP0?)5#Pc@rgfGKS#&V}&=i*N)hJ7~!_b>}8 zAvTV=d>?{NC(JD!`*WBF5c%_$s`9F`HH+1FH`MII@S;R*@*?Ud84k*qT59nu-^}4z z%Gi0~_A%cIboLItjC68NYkg33!EC22Vsg2AFOVZTN?H{CX+c=Gp87W}op_CtGz0AV z(oSBbCqsGn&@0n=9 z*h8)~rFuYCZg!l@1Nj4#=j1s2Vzxbc_9q$E0otObPSfg%4s19Gq_{sk3)+Nqp*&v1 zj<_;~y{L~p9*_raocJEF4Co?Lnv3`+ov#a9%Vw-1VrlHRiQv40%-Wp&*7I#)kAq_U zg&UFqf?Ce;Fk3Avj~UGK3L_l{gay3?`Lcas(f6!bU<14h)12gqUvx52ZECvwX-Uq*0b6j?yt?S|$q5%cW#EW>Uw4^0o| zMwE|3_JvN#=`$jU*zbR+Fi7O8Eanmj3ATWFVNEaHRWo2djTk+Eh&DS7H;8W@fqgv| zL>9);o}eClvCy~nG8)Cb;j=ae0GK5!tk$l`zqp0SqP?R5v^wHWD|os3iSGD3V6ZMc zQCeDFeDKXB@vlchp9xwzSOp#YhI}?X6y84aMshV38w>zU;siw)!xG*5z29aw&ZR#% zKfwuY*d~|RUNg8nBWJk_^`F)Fv3Z1^-QQNzguMLOYBtvo_qf;t(^BZkU#Dg4 zE)a28{9Lds3o3zl9W$YE=gYjpYB4eT!z#yaI*WIYz~?q4S?4NCTdD?VIP2>}}WW$nE7?k(myD05qnPi6psLFRo~KC|Rif!cyfT`;yS=+@GI~Z2~dnpDsrAVu2Vs7%J z)amAj(LstpA(q|9i)IW*ok<*TRGdlxs(SWwN zR|kW8oBy^o;yZFNS)xp-+-Alc5R}+?S#}~k%Qv7VE=~$PcFE|>vR+8a!Nq37Rq82O z!eXM>22pL(P(Q8%7Ry5mJ$kRkb2Uu&?9ubuz$p%Hju5Bwj6 z@W*ej@Q)Qw?FdUlaDm$=ze<(ynt4UFn$bTXno2y4;QY3A&E|`{tjAeD6`L7%H@q7d z0{HRu#s6G_^=77XbNhJ-xE~AU?gHx?Y#b56ePvilCah*^*gKB+ST5eRV|-_d@$W+X z<^CTrSnsQj1r_wY@5!4BzyOm)J@&hd1!t~~l0qqQofOnF)=Nibtc(QTe}gfyfA>;v zXt<|N6cCu}X3+iMt~5sZqHtJT*;0T?XtI{2Wv&8eKnCEpm?#`}pFtk~OMySrZF0TN z9kHFbi0&6&b+iWH5WTfuI|l`HmP?4dwJI1^M`grb1o0GFdrcos;_x$kk~99rVPn4o zGvl`;B>rl5GVeD~K>tlJ1m@eQ!%AaAs*dlJuK;wi-($jC@LfN6H^H5=Tt&_`MSo9E z>7lSET|>hhN!^}}oy3fo@THXShAfw|h^3Pd&bNiA4)+BFJH+YtW#pyFaW7=x>KO}K zBlxHSKEOA9-DdLhhkijZ^RTl!(LGh_2w7+ed&cIMML?>6k*YoqMb$_V=jKx7TyLq-n)4p~2An)|Dfzw%Zk3 z{2O((Jza{AIWeGEg0IR}V%(bUO!LT%leW{UD(VCm7Ydsn>>AA+R9{L7{*@a1cxC1) z?RRQhecNf5zfMtZH~B{SlBARSfDPANJ%oc=5-=O2hJ2o~fLc_?f>TNu4`{LVXw)|H zj}EEY=YlBTk+=zfagMcwD5lTv@^2~wwr03=+tYuYPnnlZn5Vj*!B=LTM$dUt-g1za zEHwVGJwf_LJfcCQU>7yM93A$3d;EY1yyjRskwZ+k{s2ce9#N;hrYoX7jRYng3=IOp zv37rjRiQ}x7PVNv$(@YOl;KIkiQ(Sl%;9R0%2HtD$CMo6pYU+9ot3+O$}ZP3^eR&b z$A*;#IHlZClrHnFKb1@3nC#h1>Cq2oYJ6J8t*Nb`4DEoUemR%+;MsnytTrjlAr8se z|Am`;XP|B&a68_?6jnJ6L$mR2d$*=s8*&xbwqKvY1rby+9KSq|8b}ULoPe<-q>UI) z6@)+XYsTt%Eh*~{wCK6ydO@*>3R4#N#V{F*K>ot zs+$9ixq0MHWLtfCCnXSTS3(fHq%=`{QOf|W%aoDQ`~ySO>6MlWPe#>&^ntlb6DdXu z#QG0Nk(Aw(3&h1QqRl=%6ybi{S^elNNiF_6MM2B;Y&kVQgD8(rF+1tKIJm zHj1*W)D{Bc+{h?d4|=~}s0Payw-!RHOlV!Slxir5Ff^zKPt^G7j6ZgIp3maqLYgml z&0uDIoxOntebuPel9YLwpd7ou>SQ@TM*~rs$lC%&wJ*kl9r#;*omcBi!ina_GIf%B(Q2HRCH`JJ`>%lqrOD#pA z7T0jLG{a%!mxs2M7XpSFe|J?wJ;mLTvhOnuT}_@NYbhngXe>*e!_(2M8q+w#u~1m#k%+G}zPqu9)Je zAh~!^jvIBe(%UYqnyT1^JMfym-T$Vtbxuf5SRxCbvCX;-UtTYYbNu9X*^l!<4StE@ zeCAw)n|JMiaZ3ussJX9l3AB?-w_tWUHBng0UX(v&%a9maf#axir3Z(RJ#(2db|gU_ zmRwKDn>m5*;rj8ZYXp1Ei9XE~&f&?~D|RB0nykh#b=^AT4=^;Fr+gU#Oa8#S!5Xpo zQqFdA`%)r_w$ z$@hs`E~-F2i=EA{(mF=!GgKOScg0d(-ssG^BY_&Ut*&%4!GUKf=n65H5DNYbiXUjP6FX0 z(0v)DvC?>J^^QbCl9uye_pS|8Ws z6uo5o>7AI!d~t+yN;salpcGr85G%_WG_71geO#z;KqSJ|;i|MJ#5pwXp^rx9*ZK=P ziXRU3-+}ciWe%u^;M+FCQ+_nQ z%eYh1U}(E2FWeV;2}hy(nQixJa%`uzil43^6=+RFzp1;cA{1x0l^h>?b^v%Ryp}LW zjaX}BhIA%I=asS%Lzj5wo{|gtZMm=(f`fok6f}7&oUE|@Fib3n&SF)xmdfbGI^ z(+rznq3~}%Mh#8^R&bODny;FNde-SlF4BvN28J!X>-#T<`iJ4dI5&`@^H}_e`bME!_~^D5UCS@9Tn7=bFZ6Rl+hPGNRp}$yfRgGOerz zux*{uP&b3I>1IWT_vglVL62K`K-F#YlMNLRQ(SPdmK(y%MYGLAI7?ot_t$eixqxAz&tFkzAdTsByhE0 zrDK}I zdZ1#r9J(R&twMHhVz+a&FIoI5N7*!|cYo7vvb*inYO$xrUpBP#7hWOxleyM`4Y}p5 z39TM4KJcqtqYgcLd~%?g#f&k{=|Ao|^lZw7?0zhXkfBZ>2n|15%PV!)#AUoxCm6(( zUKrN8^+s>?c^sdtz5+`HXXR_$0|z~S;YDv%kos1F(YO~*OMT(RcB|7MXjT?pkdP`e zH>q{9URNi4RHk0|$hc{uhPYi>(iRuImTRq(zVBpGyF0MCfRbh4`$@&^FTA#D2t&ix zFSGk^w(-6GWjW~i4kM@J!b42j3-K#|v=0rx(GrU6fAzBNk;_Tz$p@X;*3r{D3ciP% zCCP=K|T8cP^{7nd>GV;?++-j9sp%Z=vxo-_*!;=)c{Dat-hAAv&~6fzR!3e zewDd0#Qyk;TGS}V)amVGs?~6DaUpe+xIih1Ke=AW*ukbQI88h?b?TYdJzVJYp-17WMtxC!_`aq8D+jOA}<=o?#=FU0B15o+evB?x|x$~Gyr9FKi z9@ZW`v`HFdhn42PtgC()VU0egZHPATFD;4RXI-JkEJ5i)wKRUcJQ{KFvas#w?L|dj z_fhPo-@3$N4z+O1&(vu|8>y!eCe~^`Y_e`Y@Y+t?tRg5bYpA|(u8iM`!yYuHIge65 z%>wbO-a>k?1>33%xEx?>MqRZG@)BFzzNnR3FPhI+Q+X1krO&z;X_an%Z-jYP?9{a< zjCzQ*NfPQ(Y#H}ZS_KL|DeoJEX3dja(8Z~KI`951r%W|>F62^>a&+C#Xuj6kXhPqG zYSL3S@h`+VY#Nd@i0Mf0EDuUZ$B1h3Ckee27>9Kb4%{_QnTT3YIVdg>p!k&oWHzeF z=+rn96>CbqE8hMrf&bY13|bi#{xCF2ZXM7?3d=2`ers%;Jj{U&fVkW7xiJcDn}zyw z_k?IPKWWOJcY&abNgf$>k23b&J+G%F#Q<$TS@e)Q3kozG_ujNX8u*?Z?A=Z9E9|U86Pw`6ER!CrJApwF^5{vwRKBddek*#p?`V(krx;d~Yr#UwMc0+Z!xh(021L zFA+drR?!&uxMMR+<7vq$6LZUA+;0vF(OxkifGh=H@I&!MGWv-!o6QpYVK;C~dxvW! zFbFhXg>C4rSjZ{g3k-U}odM|EP3pf~EGZEXYq^VS^FuhV+fANE!sL|iS_V-Vb`ogz zRtWfpN$n!CQDLV7W#-ZA173ah!YjPI(JE&5`$)2&kyKGhc1Gl=WGj0MQRVmcad8$O zL(;Y0Cx6()C!9Q$tK@6s=?e1&t1-IsrE0RY%uLj{j$>69!kdi@GJpLj?+fvqIHlwJ z_I#Po?GF2kzT}eZFG@1@eb1`C6uJNSRY}oT+~vVV&cL$*q=t~roiAl2j(HajsP3rp zK8rWP60`EYM`J9k0^Egk#?A}P&pHh<;UtWml}$;UYbLGDZKpdUip}yJ_e~c=V82!p^`yA2_V0rhi!w zOT+&L0v)Y51ptpRxSyE0Af5~Gp8&}JpI+4BirAbR)#-K{n|8O1?nnF+Hv)5x6@;Xn zaru0`Ooww}vz|^@_P#Jq`RjZ``6&P(Q2PTzPgKXDFIcCYy%kS1VgFLuWHFPw`)lzTfU( ze6$z0A{0KJfiGLLslZ%Tj zj3TbjTz{XGra5-Wah{X-a))0*)Jjw~WR9)-{{mX?sJyE?BcRPIxnucPy@rd4PD1ks*IUBV`9 zsD~_SeI#7O;TI0?e(E}2U;R?1`=uBqWqN55`LRN5MWLLQ@tlWfi0FEnu>JOmYV;o) zN3{k{y8^*1R(){2T!#ANiOsr7uAn$QURQeAR-%b}qalj_>cTvvx%B0O6>&)SvE`7q frBKZ?dk;&NI1<7Ra?bp(uMU|Q-honY-HrP{QrI95 literal 0 HcmV?d00001 From c1127efab01c12d8918b2d6c8b4b41b822fda697 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 22 Apr 2015 14:21:44 -0700 Subject: [PATCH 147/246] Add + use diagram explaining retry controller area of influence Change-Id: If2b5bb1460ad98a3ba65705727b2482e8428e8d8 --- doc/diagrams/area_of_influence.graffle.tgz | Bin 0 -> 2710 bytes doc/source/atoms.rst | 27 +++++++++++++++++++-- doc/source/img/area_of_influence.svg | 3 +++ 3 files changed, 28 insertions(+), 2 deletions(-) create mode 100644 doc/diagrams/area_of_influence.graffle.tgz create mode 100644 doc/source/img/area_of_influence.svg diff --git a/doc/diagrams/area_of_influence.graffle.tgz b/doc/diagrams/area_of_influence.graffle.tgz new file mode 100644 index 0000000000000000000000000000000000000000..b5aa4648883326085b631426845123b6ee97e985 GIT binary patch literal 2710 zcmV;H3TgEpiwFRH5I9u;1MOS?PuofoM<+d<%1?K{+|NFKy3>KPyZ+t>JcR&-9t9o= z@LENBTAWR?564bzr$8&D`%~{vxmi1jzo(%E+UmR#*zwNHj(0yZJ7bUK#woo$V6?~l zhWtDA8U5Ys|IXdMGhwdH4s{`B9o#Qssv{xQMEii93%{DtlR z-2PX7`rDsZ{6}eZZwp*?`>xtk z+rDUd`xgd6)S0QkHABnVKR5)Wd}sC#Wc~gEIX|fVZ~k&Sa-CzPS1Ca4qMXNMx)Ujj~&)%>~?mg)1$+7 zM*bWGtV+*{j$dK;zBElqf}y6-&k@+yFKWQ!!H{n~j<9 zScysKHG6*m+~$La{)9DV{X5jY7K6Gy7;rq~_MB{b_fs-K5zPALYJB>H)+KbmE`8Dr zeO)pX<^DdSuApx@G$^vRJ%=Y_=2#AQ6Z+Y#lJo?}WbN?IVF~3-z(w=uS?@e{b z{kuNBNz_UdJ3dLP1M`^^8~WZgdqlDGv*aXRn1H?WywMUsYf<+D4NiPM;0eTLD-~ML=wwC;KLp@+X~^=b5QMc2L^d#_ zUDiuQd#PdRU&3n@v=c%+KhOW#3x*Ra9Yh{8&$G*eu|x0LHgyUEV+l13B0&(fCA@_2 zlVYN>0wO5xdiG?*+%Rvivk5T^b2CM7AUsNmd1tYcwS>8TN8Dv4mJsuZB)J-GDMLw_ zij^^y0)*2{iFu(Mlb|^D;8I%S`JJVF0I_u0BCxe;s%~0X)r@sXF_Ekqx@Exy-GqHr z&-jH5H@Gpwg5-`slBa4&TbER<$QoqhurXT%Bv{&g&Ug|%N*qC9mhv{6CP-$v>HBGa zk?6S39f!@j(>D?G95TT^AQ0o5cx z3T9vu!L7?G(oC$G0_&zyBl|g#t$nw^uGDN1u=CciG_bKkWL33bM>Z{DDLRJejUYod z0OLAjO%Toz6;}ZpYe?2KWLXw6G(;?H0t zZX|ijiohq7EE8;p)Ezh_vOfdYKHzN;So3K?RbG(BLU|tZsIot6WM3k?0O#y_=X5wt z`^t(mr(9Bk*sdS;ahJLQkN(-?0VW#h+MexE(M5DE{t-;B;5R{Nb^6=(8FTv{cuf4M zD~{on)9s^fKe*vR(Cynp>IW=D(gAZmun_Y3pYK;c0kh-zK>BmnXFZ~6-GPuJPOh8@ zgCkIN$JFn2!&^;9@E^j5=%Ab8K!_S0Oe7A95|JoYM53w@iH0H);1&n)(9yvF5+RrS zE)6F>b-M5E)zw&IqDo!6kEAme`gh&c7Tk^RfgALtf8c-SeYu!KNOQy;Im<0UepxTU z#ET}B#l<(@R;xqVMJ_}A@_-S7L-B3_qCQ#PXI0lH6;*HgWT*5?;x3QKYxhz1e*!x=C zx0l`c8p}HO+F>u_usbxkmI|5R*934X#Pe9Oo(*RYh_Py%eOsK7N5q+?iYFSzv%xH% za2Y6B|AV z5YpwK&cM1Hx|ta&r>Nn*0N$+vLejZ)`ut%?6!`)|&yK`=np8va$ARL`lRy!x=o>+? zJO!+w_ya((`7}@z&q0W$2q*#~#q((VTwo+M7=J)8mOKF^`6r-N*epCyE6qf*uav(a zb=tsWU9vQRY1I_*r)52H@?<!(J!UDcaHo^4Sb6S4;TH=edha(_tf!T%3UxNJK zjEy^9wrs3Vogk@@jA}lUk*jLE#EpOMZAeUcOh+Xq7Y!rfBKax&T=sS81!pC%OLL@E z%L^KKld!PB5sUcUbonS%)>rk$NA%j@KfE{G-Az@+O)CaPG0wDNFh$#DfY_*VFh`DL zl%sbdHTT>BcZ<68ys)5*E2;?}FhF?Yjrmga*^130?p{|Rx@Z7z=)h&+L+ArQbbH8H zm;>V+sc*aUt;B@YL*vZ#)8*;h5{kJ~$57z0&{?T*Ai&z*Vhq~fGw$~Fbwq&54X85sp@Y;EMi4ps4$9Z>n;5F4>cYZx=?+u&7 zlXIlutIOApJ!rx2-pM(BH@rHyaGKEH+S586^7{R|%VX3$yxeQH-viD5t#i|QwOIWl z)(^jCda$`z{wc&Lemko8s1{$W2$}8tx`?z4EBoBLWE6peGbTO)U?u1*8#8juB}1ac zBc1WV@JT1Eyv3e*H_KL|1L;zw=#i`J@UZAG z;(GbaUK?|aAQFt!5H;X&gc5)Ev4k6mn8kiaU7J0S_uS?0CoBot7u@}M!1mmb`-`Ui zV%b>flDiPESNp!!#P?w)qM1yfR9+quG3?!rsSD02p5a7?4E}NfyPFAz7A;aXM$LkL z-%pE4(k#Uk_2gPy$%fV?v-A@cQN5puosz1Bc>kG%4ul(*ycZ6O6M@x|Z&M;8Zvzaj zsGqRR7ixx4@8)dcFu!Wxf`HQQD!oX_5=}SJGd9RLw`^V-f6}AlbxAj@lKD!1m@ + +2015-04-22 21:13ZArea of influenceLayer 1RetryRetryController (2)Task ATask BTask CTask 1Task 2Task 3RetryController (1) From da61f15d89ec8642150d9a64f31c1a7efe234349 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 22 Apr 2015 14:52:13 -0700 Subject: [PATCH 148/246] Expose fake filesystem 'join' and 'normpath' Change-Id: I9572b2b739c2a1743df1bd6222a9d8dcff77cb4d --- taskflow/persistence/backends/impl_memory.py | 25 +++++++++++++------- 1 file changed, 16 insertions(+), 9 deletions(-) diff --git a/taskflow/persistence/backends/impl_memory.py b/taskflow/persistence/backends/impl_memory.py index a8a9b6e4..b9489835 100644 --- a/taskflow/persistence/backends/impl_memory.py +++ b/taskflow/persistence/backends/impl_memory.py @@ -65,12 +65,19 @@ class FakeFilesystem(object): root_path = pp.sep @classmethod - def _normpath(cls, path): + def normpath(cls, path): + """Return a normalized absolutized version of the pathname path.""" if not path.startswith(cls.root_path): - raise ValueError("This filesystem can only normalize absolute" - " paths: '%s' is not valid" % path) + raise ValueError("This filesystem can only normalize" + " paths that start with %s: '%s' is not" + " valid" % (cls.root_path, path)) return pp.normpath(path) + @staticmethod + def join(*pieces): + """Join many path segments together.""" + return pp.sep.join(pieces) + def __init__(self, deep_copy=True): self._root = tree.Node(self.root_path, value=None) if deep_copy: @@ -80,7 +87,7 @@ class FakeFilesystem(object): def ensure_path(self, path): """Ensure the path (and parents) exists.""" - path = self._normpath(path) + path = self.normpath(path) # Ignore the root path as we already checked for that; and it # will always exist/can't be removed anyway... if path == self._root.item: @@ -96,7 +103,7 @@ class FakeFilesystem(object): def _fetch_node(self, path): node = self._root - path = self._normpath(path) + path = self.normpath(path) if path == self._root.item: return node for piece in self._iter_pieces(path): @@ -144,7 +151,7 @@ class FakeFilesystem(object): hops.append(parent.item) hops.reverse() # This avoids getting '//a/b' (duplicated sep at start)... - child_path = pp.sep.join(hops) + child_path = self.join(*hops) if child_path.startswith("//"): child_path = child_path[1:] paths.append(child_path) @@ -190,8 +197,8 @@ class FakeFilesystem(object): def symlink(self, src_path, dest_path): """Link the destionation path to the source path.""" - dest_path = self._normpath(dest_path) - src_path = self._normpath(src_path) + dest_path = self.normpath(dest_path) + src_path = self.normpath(src_path) dirname, basename = pp.split(dest_path) parent_node = self._fetch_node(dirname) child_node = parent_node.find(basename, @@ -206,7 +213,7 @@ class FakeFilesystem(object): return self._get_item(path) def __setitem__(self, path, value): - path = self._normpath(path) + path = self.normpath(path) value = self._copier(value) try: item_node = self._fetch_node(path) From 1764e3efc9042819d127d26ddff0c1bbed698d83 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Tue, 21 Apr 2015 18:01:59 -0700 Subject: [PATCH 149/246] Retain chain of missing dependencies Change-Id: I170bedb5c667aa30764ad29634910dda40c6cd49 --- taskflow/engines/action_engine/engine.py | 30 ++++++++++++++++++++---- taskflow/tests/unit/test_engines.py | 25 ++++++++++++++++++++ 2 files changed, 51 insertions(+), 4 deletions(-) diff --git a/taskflow/engines/action_engine/engine.py b/taskflow/engines/action_engine/engine.py index e7763068..bed43e8e 100644 --- a/taskflow/engines/action_engine/engine.py +++ b/taskflow/engines/action_engine/engine.py @@ -225,12 +225,34 @@ class ActionEngine(base.Engine): execution_graph.number_of_edges(), nx.density(execution_graph)) missing = set() - fetch = self.storage.fetch_unsatisfied_args + # Attempt to retain a chain of what was missing (so that the final + # raised exception for the flow has the nodes that had missing + # dependencies). + last_cause = None + last_node = None + missing_nodes = 0 + fetch_func = self.storage.fetch_unsatisfied_args for node in execution_graph.nodes_iter(): - missing.update(fetch(node.name, node.rebind, - optional_args=node.optional)) + node_missing = fetch_func(node.name, node.rebind, + optional_args=node.optional) + if node_missing: + cause = exc.MissingDependencies(node, + sorted(node_missing), + cause=last_cause) + last_cause = cause + last_node = node + missing_nodes += 1 + missing.update(node_missing) if missing: - raise exc.MissingDependencies(self._flow, sorted(missing)) + # For when a task is provided (instead of a flow) and that + # task is the only item in the graph and its missing deps, avoid + # re-wrapping it in yet another exception... + if missing_nodes == 1 and last_node is self._flow: + raise last_cause + else: + raise exc.MissingDependencies(self._flow, + sorted(missing), + cause=last_cause) @lock_utils.locked def prepare(self): diff --git a/taskflow/tests/unit/test_engines.py b/taskflow/tests/unit/test_engines.py index 04b8fa3a..c6a2af95 100644 --- a/taskflow/tests/unit/test_engines.py +++ b/taskflow/tests/unit/test_engines.py @@ -651,6 +651,26 @@ class EngineGraphFlowTest(utils.EngineTestBase): self.assertIsInstance(graph, gr.DiGraph) +class EngineMissingDepsTest(utils.EngineTestBase): + def test_missing_deps_deep(self): + flow = gf.Flow('missing-many').add( + utils.TaskOneReturn(name='task1', + requires=['a', 'b', 'c']), + utils.TaskMultiArgOneReturn(name='task2', + rebind=['e', 'f', 'g'])) + engine = self._make_engine(flow) + engine.compile() + engine.prepare() + self.assertRaises(exc.MissingDependencies, engine.validate) + c_e = None + try: + engine.validate() + except exc.MissingDependencies as e: + c_e = e + self.assertIsNotNone(c_e) + self.assertIsNotNone(c_e.cause) + + class EngineCheckingTaskTest(utils.EngineTestBase): # FIXME: this test uses a inner class that workers/process engines can't # get to, so we need to do something better to make this test useful for @@ -682,6 +702,7 @@ class SerialEngineTest(EngineTaskTest, EngineLinearAndUnorderedExceptionsTest, EngineOptionalRequirementsTest, EngineGraphFlowTest, + EngineMissingDepsTest, EngineCheckingTaskTest, test.TestCase): def _make_engine(self, flow, @@ -707,6 +728,7 @@ class ParallelEngineWithThreadsTest(EngineTaskTest, EngineLinearAndUnorderedExceptionsTest, EngineOptionalRequirementsTest, EngineGraphFlowTest, + EngineMissingDepsTest, EngineCheckingTaskTest, test.TestCase): _EXECUTOR_WORKERS = 2 @@ -744,6 +766,7 @@ class ParallelEngineWithEventletTest(EngineTaskTest, EngineLinearAndUnorderedExceptionsTest, EngineOptionalRequirementsTest, EngineGraphFlowTest, + EngineMissingDepsTest, EngineCheckingTaskTest, test.TestCase): @@ -764,6 +787,7 @@ class ParallelEngineWithProcessTest(EngineTaskTest, EngineLinearAndUnorderedExceptionsTest, EngineOptionalRequirementsTest, EngineGraphFlowTest, + EngineMissingDepsTest, test.TestCase): _EXECUTOR_WORKERS = 2 @@ -789,6 +813,7 @@ class WorkerBasedEngineTest(EngineTaskTest, EngineLinearAndUnorderedExceptionsTest, EngineOptionalRequirementsTest, EngineGraphFlowTest, + EngineMissingDepsTest, test.TestCase): def setUp(self): super(WorkerBasedEngineTest, self).setUp() From 3097e52be82ba4ce4f8ca4e916edf039bd7f0a31 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Mon, 27 Apr 2015 15:45:22 -0700 Subject: [PATCH 150/246] Move zookeeper jobboard constants to class level To make it easy to document and inquire on the zookeeper job constants expose them as class level instead of module level constants (and use them where needed). Change-Id: Ice53db104c55ba4a7794a358d56922074d44492b --- taskflow/jobs/backends/impl_zookeeper.py | 54 ++++++++++++++---------- taskflow/tests/unit/jobs/test_zk_job.py | 12 +++--- 2 files changed, 38 insertions(+), 28 deletions(-) diff --git a/taskflow/jobs/backends/impl_zookeeper.py b/taskflow/jobs/backends/impl_zookeeper.py index ce8363c2..a4e512be 100644 --- a/taskflow/jobs/backends/impl_zookeeper.py +++ b/taskflow/jobs/backends/impl_zookeeper.py @@ -40,21 +40,6 @@ from taskflow.utils import misc LOG = logging.getLogger(__name__) -UNCLAIMED_JOB_STATES = ( - states.UNCLAIMED, -) -ALL_JOB_STATES = ( - states.UNCLAIMED, - states.COMPLETE, - states.CLAIMED, -) - -# Transaction support was added in 3.4.0 -MIN_ZK_VERSION = (3, 4, 0) -LOCK_POSTFIX = ".lock" -TRASH_FOLDER = ".trash" -JOB_PREFIX = 'job' - def check_who(meth): """Decorator that checks the expected owner type & value restrictions.""" @@ -88,12 +73,12 @@ class ZookeeperJob(base.Job): raise ValueError("Only one of 'book_data' or 'book'" " can be provided") self._path = k_paths.normpath(path) - self._lock_path = self._path + LOCK_POSTFIX + self._lock_path = self._path + board.LOCK_POSTFIX self._created_on = created_on self._node_not_found = False basename = k_paths.basename(self._path) self._root = self._path[0:-len(basename)] - self._sequence = int(basename[len(JOB_PREFIX):]) + self._sequence = int(basename[len(board.JOB_PREFIX):]) self._last_state = None @property @@ -274,6 +259,16 @@ class ZookeeperJobBoardIterator(six.Iterator): over unclaimed jobs. """ + _UNCLAIMED_JOB_STATES = ( + states.UNCLAIMED, + ) + + _JOB_STATES = ( + states.UNCLAIMED, + states.COMPLETE, + states.CLAIMED, + ) + def __init__(self, board, only_unclaimed=False, ensure_fresh=False): self._board = board self._jobs = collections.deque() @@ -290,9 +285,9 @@ class ZookeeperJobBoardIterator(six.Iterator): def _next_job(self): if self.only_unclaimed: - allowed_states = UNCLAIMED_JOB_STATES + allowed_states = self._UNCLAIMED_JOB_STATES else: - allowed_states = ALL_JOB_STATES + allowed_states = self._JOB_STATES job = None while self._jobs and job is None: maybe_job = self._jobs.popleft() @@ -343,6 +338,18 @@ class ZookeeperJobBoard(base.NotifyingJobBoard): automatically by zookeeper the ephemeral is deemed to be lost). """ + #: Transaction support was added in 3.4.0 so we need at least that version. + MIN_ZK_VERSION = (3, 4, 0) + + #: Znode **postfix** that lock entries have. + LOCK_POSTFIX = ".lock" + + #: Znode child path created under root path that contains trashed jobs. + TRASH_FOLDER = ".trash" + + #: Znode **prefix** that job entries have. + JOB_PREFIX = 'job' + def __init__(self, name, conf, client=None, persistence=None, emit_notifications=True): super(ZookeeperJobBoard, self).__init__(name, conf) @@ -359,7 +366,7 @@ class ZookeeperJobBoard(base.NotifyingJobBoard): raise ValueError("Zookeeper path must be absolute") self._path = path self._trash_path = self._path.replace(k_paths.basename(self._path), - TRASH_FOLDER) + self.TRASH_FOLDER) # The backend to load the full logbooks from, since whats sent over # the zookeeper data connection is only the logbook uuid and name, and # not currently the full logbook (later when a zookeeper backend @@ -375,7 +382,7 @@ class ZookeeperJobBoard(base.NotifyingJobBoard): self._job_watcher = None # Since we use sequenced ids this will be the path that the sequences # are prefixed with, for example, job0000000001, job0000000002, ... - self._job_base = k_paths.join(path, JOB_PREFIX) + self._job_base = k_paths.join(path, self.JOB_PREFIX) self._worker = None self._emit_notifications = bool(emit_notifications) self._connected = False @@ -481,7 +488,8 @@ class ZookeeperJobBoard(base.NotifyingJobBoard): LOG.debug("Got children %s under path %s", children, self.path) child_paths = [] for c in children: - if c.endswith(LOCK_POSTFIX) or not c.startswith(JOB_PREFIX): + if (c.endswith(self.LOCK_POSTFIX) or + not c.startswith(self.JOB_PREFIX)): # Skip lock paths or non-job-paths (these are not valid jobs) continue child_paths.append(k_paths.join(self.path, c)) @@ -788,7 +796,7 @@ class ZookeeperJobBoard(base.NotifyingJobBoard): "Failed to connect to zookeeper") try: if self._conf.get('check_compatible', True): - kazoo_utils.check_compatible(self._client, MIN_ZK_VERSION) + kazoo_utils.check_compatible(self._client, self.MIN_ZK_VERSION) if self._worker is None and self._emit_notifications: self._worker = futures.ThreadPoolExecutor(max_workers=1) self._client.ensure_path(self.path) diff --git a/taskflow/tests/unit/jobs/test_zk_job.py b/taskflow/tests/unit/jobs/test_zk_job.py index 17385306..a22a41b9 100644 --- a/taskflow/tests/unit/jobs/test_zk_job.py +++ b/taskflow/tests/unit/jobs/test_zk_job.py @@ -32,11 +32,13 @@ from taskflow.utils import persistence_utils as p_utils TEST_PATH_TPL = '/taskflow/board-test/%s' -_ZOOKEEPER_AVAILABLE = test_utils.zookeeper_available( - impl_zookeeper.MIN_ZK_VERSION) +ZOOKEEPER_AVAILABLE = test_utils.zookeeper_available( + impl_zookeeper.ZookeeperJobBoard.MIN_ZK_VERSION) +TRASH_FOLDER = impl_zookeeper.ZookeeperJobBoard.TRASH_FOLDER +LOCK_POSTFIX = impl_zookeeper.ZookeeperJobBoard.LOCK_POSTFIX -@testtools.skipIf(not _ZOOKEEPER_AVAILABLE, 'zookeeper is not available') +@testtools.skipIf(not ZOOKEEPER_AVAILABLE, 'zookeeper is not available') class ZookeeperJobboardTest(test.TestCase, base.BoardTestMixin): def _create_board(self, persistence=None): @@ -137,10 +139,10 @@ class ZakeJobboardTest(test.TestCase, base.BoardTestMixin): for (path, value) in paths: if path in self.bad_paths: continue - if path.find(impl_zookeeper.TRASH_FOLDER) > -1: + if path.find(TRASH_FOLDER) > -1: trashed.append(path) elif (path.find(self.board._job_base) > -1 - and not path.endswith(impl_zookeeper.LOCK_POSTFIX)): + and not path.endswith(LOCK_POSTFIX)): jobs.append(path) self.assertEqual(len(trashed), 1) From f5a276d3cd48c936b3fad04289735572bda528d0 Mon Sep 17 00:00:00 2001 From: Doug Hellmann Date: Fri, 1 May 2015 19:40:16 +0000 Subject: [PATCH 151/246] Remove run_cross_tests.sh This script is part of the oslotest package now. Change-Id: I4be1461329f0b6661217759245c2346b83e80135 --- tools/run_cross_tests.sh | 91 ---------------------------------------- 1 file changed, 91 deletions(-) delete mode 100755 tools/run_cross_tests.sh diff --git a/tools/run_cross_tests.sh b/tools/run_cross_tests.sh deleted file mode 100755 index 5e7bc118..00000000 --- a/tools/run_cross_tests.sh +++ /dev/null @@ -1,91 +0,0 @@ -#!/bin/bash -# -# Run cross-project tests -# -# Usage: -# -# run_cross_tests.sh project_dir venv - -# Fail the build if any command fails -set -e - -project_dir="$1" -venv="$2" - -if [ -z "$project_dir" -o -z "$venv" ] -then - cat - < ./subunit_log.txt - fi - .tox/$venv/bin/python /usr/local/jenkins/slave_scripts/subunit2html.py ./subunit_log.txt testr_results.html - gzip -9 ./subunit_log.txt - gzip -9 ./testr_results.html - - export PYTHON=.tox/$venv/bin/python - set -e - rancount=$(.tox/$venv/bin/testr last | sed -ne 's/Ran \([0-9]\+\).*tests in.*/\1/p') - if [ "$rancount" -eq "0" ] ; then - echo - echo "Zero tests were run. At least one test should have been run." - echo "Failing this test as a result" - echo - exit 1 - fi -fi - -# If we make it this far, report status based on the tests that were -# run. -exit $result From 46d30eeb29599f698ef95ddc343776d1ebb6b989 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Fri, 1 May 2015 17:05:55 -0700 Subject: [PATCH 152/246] Move implementations into there own sub-sections Also fixes up some inline-code/examples docs to correctly display in the generated docs (and tweaks some URI capitalization). Change-Id: I001ef2460eb5e9a884ca6db6e8d6f72864191fe7 --- doc/source/conductors.rst | 5 ++++- doc/source/engines.rst | 6 +++++- doc/source/jobs.rst | 3 +++ doc/source/persistence.rst | 19 +++++++++++++++++-- doc/source/workers.rst | 8 ++++++-- taskflow/jobs/backends/__init__.py | 12 ++++++------ taskflow/persistence/backends/__init__.py | 10 +++++----- .../persistence/backends/impl_sqlalchemy.py | 2 -- 8 files changed, 46 insertions(+), 19 deletions(-) diff --git a/doc/source/conductors.rst b/doc/source/conductors.rst index efd9bf7e..d6d99a2c 100644 --- a/doc/source/conductors.rst +++ b/doc/source/conductors.rst @@ -65,11 +65,14 @@ Interfaces ========== .. automodule:: taskflow.conductors.base +.. automodule:: taskflow.conductors.backends Implementations =============== -.. automodule:: taskflow.conductors.backends +Blocking +-------- + .. automodule:: taskflow.conductors.backends.impl_blocking Hierarchy diff --git a/doc/source/engines.rst b/doc/source/engines.rst index 9563cb45..abdbdb62 100644 --- a/doc/source/engines.rst +++ b/doc/source/engines.rst @@ -436,10 +436,14 @@ Interfaces Implementations =============== +.. automodule:: taskflow.engines.action_engine.engine + +Components +---------- + .. automodule:: taskflow.engines.action_engine.analyzer .. automodule:: taskflow.engines.action_engine.compiler .. automodule:: taskflow.engines.action_engine.completer -.. automodule:: taskflow.engines.action_engine.engine .. automodule:: taskflow.engines.action_engine.executor .. automodule:: taskflow.engines.action_engine.runner .. automodule:: taskflow.engines.action_engine.runtime diff --git a/doc/source/jobs.rst b/doc/source/jobs.rst index 0374475a..cc7f6f10 100644 --- a/doc/source/jobs.rst +++ b/doc/source/jobs.rst @@ -265,6 +265,9 @@ Interfaces Implementations =============== +Zookeeper +--------- + .. automodule:: taskflow.jobs.backends.impl_zookeeper Hierarchy diff --git a/doc/source/persistence.rst b/doc/source/persistence.rst index 6f8256bf..0bab644c 100644 --- a/doc/source/persistence.rst +++ b/doc/source/persistence.rst @@ -176,7 +176,7 @@ concept everyone is familiar with). See :py:class:`~taskflow.persistence.backends.impl_dir.DirBackend` for implementation details. -Sqlalchemy +SQLAlchemy ---------- **Connection**: ``'mysql'`` or ``'postgres'`` or ``'sqlite'`` @@ -288,9 +288,24 @@ Interfaces Implementations =============== -.. automodule:: taskflow.persistence.backends.impl_dir +Memory +------ + .. automodule:: taskflow.persistence.backends.impl_memory + +Files +----- + +.. automodule:: taskflow.persistence.backends.impl_dir + +SQLAlchemy +---------- + .. automodule:: taskflow.persistence.backends.impl_sqlalchemy + +Zookeeper +--------- + .. automodule:: taskflow.persistence.backends.impl_zookeeper Storage diff --git a/doc/source/workers.rst b/doc/source/workers.rst index 38212405..95c15598 100644 --- a/doc/source/workers.rst +++ b/doc/source/workers.rst @@ -413,10 +413,14 @@ Limitations possibly spawn the task on a secondary worker if a timeout is reached (aka the first worker has died or has stopped responding). -Interfaces -========== +Implementations +=============== .. automodule:: taskflow.engines.worker_based.engine + +Components +---------- + .. automodule:: taskflow.engines.worker_based.proxy .. automodule:: taskflow.engines.worker_based.worker diff --git a/taskflow/jobs/backends/__init__.py b/taskflow/jobs/backends/__init__.py index e8bd6daf..f818905f 100644 --- a/taskflow/jobs/backends/__init__.py +++ b/taskflow/jobs/backends/__init__.py @@ -39,17 +39,17 @@ def fetch(name, conf, namespace=BACKEND_NAMESPACE, **kwargs): NOTE(harlowja): to aid in making it easy to specify configuration and options to a board the configuration (which is typical just a dictionary) - can also be a uri string that identifies the entrypoint name and any + can also be a URI string that identifies the entrypoint name and any configuration specific to that board. - For example, given the following configuration uri: + For example, given the following configuration URI:: - zookeeper:///?a=b&c=d + zookeeper:///?a=b&c=d This will look for the entrypoint named 'zookeeper' and will provide - a configuration object composed of the uris parameters, in this case that - is {'a': 'b', 'c': 'd'} to the constructor of that board instance (also - including the name specified). + a configuration object composed of the URI's components, in this case that + is ``{'a': 'b', 'c': 'd'}`` to the constructor of that board + instance (also including the name specified). """ if isinstance(conf, six.string_types): conf = {'board': conf} diff --git a/taskflow/persistence/backends/__init__.py b/taskflow/persistence/backends/__init__.py index ec0b6238..50f24167 100644 --- a/taskflow/persistence/backends/__init__.py +++ b/taskflow/persistence/backends/__init__.py @@ -39,16 +39,16 @@ def fetch(conf, namespace=BACKEND_NAMESPACE, **kwargs): NOTE(harlowja): to aid in making it easy to specify configuration and options to a backend the configuration (which is typical just a dictionary) - can also be a uri string that identifies the entrypoint name and any + can also be a URI string that identifies the entrypoint name and any configuration specific to that backend. - For example, given the following configuration uri: + For example, given the following configuration URI:: - mysql:///?a=b&c=d + mysql:///?a=b&c=d This will look for the entrypoint named 'mysql' and will provide - a configuration object composed of the uris parameters, in this case that - is {'a': 'b', 'c': 'd'} to the constructor of that persistence backend + a configuration object composed of the URI's components, in this case that + is ``{'a': 'b', 'c': 'd'}`` to the constructor of that persistence backend instance. """ if isinstance(conf, six.string_types): diff --git a/taskflow/persistence/backends/impl_sqlalchemy.py b/taskflow/persistence/backends/impl_sqlalchemy.py index d9fdf732..483dd445 100644 --- a/taskflow/persistence/backends/impl_sqlalchemy.py +++ b/taskflow/persistence/backends/impl_sqlalchemy.py @@ -15,8 +15,6 @@ # License for the specific language governing permissions and limitations # under the License. -"""Implementation of a SQLAlchemy storage backend.""" - from __future__ import absolute_import import contextlib From 530328a86c3fe7be7235c602332d0bd823e29dbe Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Thu, 2 Apr 2015 11:15:06 -0700 Subject: [PATCH 153/246] Refactor/reduce shared 'ensure(task/retry)' code These methods are nearly identical so we should refactor them to use the same code for sanity and understanding purposes. Change-Id: Ibaf270bd451b6d02d7782901bc2327afe04d3847 --- taskflow/storage.py | 86 +++++++++++++++--------------------------- taskflow/utils/misc.py | 16 +++++--- 2 files changed, 40 insertions(+), 62 deletions(-) diff --git a/taskflow/storage.py b/taskflow/storage.py index 83311e92..d40c5870 100644 --- a/taskflow/storage.py +++ b/taskflow/storage.py @@ -152,8 +152,8 @@ class Storage(object): self._injected_args = {} self._lock = lock_utils.ReaderWriterLock() self._ensure_matchers = [ - ((task.BaseTask,), self._ensure_task), - ((retry.Retry,), self._ensure_retry), + ((task.BaseTask,), (logbook.TaskDetail, 'Task')), + ((retry.Retry,), (logbook.RetryDetail, 'Retry')), ] if scope_fetcher is None: scope_fetcher = lambda atom_name: None @@ -191,73 +191,47 @@ class Storage(object): Returns uuid for the atomdetail that is/was created. """ - functor = misc.match_type_handler(atom, self._ensure_matchers) - if not functor: - raise TypeError("Unknown item '%s' (%s) requested to ensure" + match = misc.match_type(atom, self._ensure_matchers) + if not match: + raise TypeError("Unknown atom '%s' (%s) requested to ensure" % (atom, type(atom))) else: - return functor(atom.name, - misc.get_version_string(atom), - atom.save_as) + detail_cls, kind = match + atom_id = self._ensure_atom_detail(kind, detail_cls, atom.name, + misc.get_version_string(atom), + atom.save_as) + return atom_id - def _ensure_task(self, task_name, task_version, result_mapping): - """Ensures there is a taskdetail that corresponds to the task info. + def _ensure_atom_detail(self, kind, detail_cls, + atom_name, atom_version, result_mapping): + """Ensures there is a atomdetail that corresponds to the given atom. - If task does not exist, adds a record for it. Added task will have - PENDING state. Sets result mapping for the task from result_mapping + If atom does not exist, adds a record for it. Added atom will have + PENDING state. Sets result mapping for the atom from result_mapping argument. - Returns uuid for the task details corresponding to the task with + Returns uuid for the atomdetails corresponding to the atom with given name. """ - if not task_name: - raise ValueError("Task name must be non-empty") + if not atom_name: + raise ValueError("%s name must be non-empty" % (kind)) with self._lock.write_lock(): try: - task_id = self._atom_name_to_uuid[task_name] + atom_id = self._atom_name_to_uuid[atom_name] except KeyError: - task_id = uuidutils.generate_uuid() - self._create_atom_detail(logbook.TaskDetail, task_name, - task_id, task_version) + atom_id = uuidutils.generate_uuid() + self._create_atom_detail(detail_cls, atom_name, + atom_id, atom_version=atom_version) else: - ad = self._flowdetail.find(task_id) - if not isinstance(ad, logbook.TaskDetail): + ad = self._flowdetail.find(atom_id) + if not isinstance(ad, detail_cls): raise exceptions.Duplicate( - "Atom detail %s already exists in flow detail %s." % - (task_name, self._flowdetail.name)) - self._set_result_mapping(task_name, result_mapping) - return task_id + "Atom detail '%s' already exists in flow" + " detail '%s'" % (atom_name, self._flowdetail.name)) + self._set_result_mapping(atom_name, result_mapping) + return atom_id - def _ensure_retry(self, retry_name, retry_version, result_mapping): - """Ensures there is a retrydetail that corresponds to the retry info. - - If retry does not exist, adds a record for it. Added retry - will have PENDING state. Sets result mapping for the retry from - result_mapping argument. Initializes retry result as an empty - collections of results and failures history. - - Returns uuid for the retry details corresponding to the retry - with given name. - """ - if not retry_name: - raise ValueError("Retry name must be non-empty") - with self._lock.write_lock(): - try: - retry_id = self._atom_name_to_uuid[retry_name] - except KeyError: - retry_id = uuidutils.generate_uuid() - self._create_atom_detail(logbook.RetryDetail, retry_name, - retry_id, retry_version) - else: - ad = self._flowdetail.find(retry_id) - if not isinstance(ad, logbook.RetryDetail): - raise exceptions.Duplicate( - "Atom detail %s already exists in flow detail %s." % - (retry_name, self._flowdetail.name)) - self._set_result_mapping(retry_name, result_mapping) - return retry_id - - def _create_atom_detail(self, detail_cls, name, uuid, task_version=None): + def _create_atom_detail(self, detail_cls, name, uuid, atom_version=None): """Add the atom detail to flow detail. Atom becomes known to storage by that name and uuid. @@ -265,7 +239,7 @@ class Storage(object): """ ad = detail_cls(name, uuid) ad.state = states.PENDING - ad.version = task_version + ad.version = atom_version # Add the atom detail to the clone, which upon success will be # updated into the contained flow detail; if it does not get saved # then no update will happen. diff --git a/taskflow/utils/misc.py b/taskflow/utils/misc.py index ee5fb393..d79fe8f0 100644 --- a/taskflow/utils/misc.py +++ b/taskflow/utils/misc.py @@ -88,14 +88,18 @@ def find_monotonic(allow_time_time=False): return None -def match_type_handler(item, type_handlers): - """Matches a given items type using the given match types + handlers. +def match_type(obj, matchers): + """Matches a given object using the given matchers list/iterable. - Returns the handler if a type match occurs, otherwise none. + NOTE(harlowja): each element of the provided list/iterable must be + tuple of (valid types, result). + + Returns the result (the second element of the provided tuple) if a type + match occurs, otherwise none if no matches are found. """ - for (match_types, handler_func) in type_handlers: - if isinstance(item, match_types): - return handler_func + for (match_types, match_result) in matchers: + if isinstance(obj, match_types): + return match_result else: return None From b8f1447a083dfb15b042ac3bf258c04deddbf16d Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Fri, 1 May 2015 23:18:52 -0700 Subject: [PATCH 154/246] Fix post coverage job option not recognized Remove option --coverage-package-name as it does not appear to be a currently valid option. Example broken jenkins job: https://jenkins05.openstack.org/job/taskflow-coverage/68 Change-Id: I63c501c2e4b1f83c5ca0efe192d9534cc156903d --- tox.ini | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tox.ini b/tox.ini index 85545cac..705af0b5 100644 --- a/tox.ini +++ b/tox.ini @@ -49,7 +49,7 @@ commands = pylint --rcfile=pylintrc taskflow basepython = python2.7 deps = {[testenv:py27]deps} coverage>=3.6 -commands = python setup.py testr --coverage --coverage-package-name=taskflow --testr-args='{posargs}' +commands = python setup.py testr --coverage --testr-args='{posargs}' [testenv:venv] basepython = python2.7 From 7c3fdcc8c4bdfd156cca83e9f840aed0c8eb37ad Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Fri, 1 May 2015 12:44:17 -0700 Subject: [PATCH 155/246] Small refactoring of 'merge_uri' utility function Perform some small adjustments/cleanups and add some unit tests to ensure this function keeps operating as expected. Change-Id: I496bd6844072f57624de31fc7ddb0362f163cc53 --- taskflow/tests/unit/test_utils.py | 36 +++++++++++++++++++++++++++++++ taskflow/utils/misc.py | 32 ++++++++++++++++----------- 2 files changed, 56 insertions(+), 12 deletions(-) diff --git a/taskflow/tests/unit/test_utils.py b/taskflow/tests/unit/test_utils.py index 2465f9fa..c136cb7c 100644 --- a/taskflow/tests/unit/test_utils.py +++ b/taskflow/tests/unit/test_utils.py @@ -262,6 +262,42 @@ class TestLookFor(test.TestCase): self.assertEqual([10, 44], misc.look_for(hay, [44, 10])) +class TestMergeUri(test.TestCase): + def test_merge(self): + url = "http://www.yahoo.com/?a=b&c=d" + parsed = misc.parse_uri(url) + joined = misc.merge_uri(parsed, {}) + self.assertEqual('b', joined.get('a')) + self.assertEqual('d', joined.get('c')) + self.assertEqual('www.yahoo.com', joined.get('hostname')) + + def test_merge_existing_hostname(self): + url = "http://www.yahoo.com/" + parsed = misc.parse_uri(url) + joined = misc.merge_uri(parsed, {'hostname': 'b.com'}) + self.assertEqual('b.com', joined.get('hostname')) + + def test_merge_user_password(self): + url = "http://josh:harlow@www.yahoo.com/" + parsed = misc.parse_uri(url) + joined = misc.merge_uri(parsed, {}) + self.assertEqual('www.yahoo.com', joined.get('hostname')) + self.assertEqual('josh', joined.get('username')) + self.assertEqual('harlow', joined.get('password')) + + def test_merge_user_password_existing(self): + url = "http://josh:harlow@www.yahoo.com/" + parsed = misc.parse_uri(url) + existing = { + 'username': 'joe', + 'password': 'biggie', + } + joined = misc.merge_uri(parsed, existing) + self.assertEqual('www.yahoo.com', joined.get('hostname')) + self.assertEqual('joe', joined.get('username')) + self.assertEqual('biggie', joined.get('password')) + + class TestClamping(test.TestCase): def test_simple_clamp(self): result = misc.clamp(1.0, 2.0, 3.0) diff --git a/taskflow/utils/misc.py b/taskflow/utils/misc.py index ee5fb393..ffd487ef 100644 --- a/taskflow/utils/misc.py +++ b/taskflow/utils/misc.py @@ -124,21 +124,29 @@ def reverse_enumerate(items): def merge_uri(uri, conf): """Merges a parsed uri into the given configuration dictionary. - Merges the username, password, hostname, and query params of a uri into - the given configuration (it does not overwrite the configuration keys if - they already exist) and returns the adjusted configuration. + Merges the username, password, hostname, port, and query parameters of + a URI into the given configuration dictionary (it does **not** overwrite + existing configuration keys if they already exist) and returns the merged + configuration. NOTE(harlowja): does not merge the path, scheme or fragment. """ - for (k, v) in [('username', uri.username), ('password', uri.password)]: - if not v: - continue - conf.setdefault(k, v) - if uri.hostname: - hostname = uri.hostname - if uri.port is not None: - hostname += ":%s" % (uri.port) - conf.setdefault('hostname', hostname) + uri_port = uri.port + specials = [ + ('username', uri.username, lambda v: bool(v)), + ('password', uri.password, lambda v: bool(v)), + # NOTE(harlowja): A different check function is used since 0 is + # false (when bool(v) is applied), and that is a valid port... + ('port', uri_port, lambda v: v is not None), + ] + hostname = uri.hostname + if hostname: + if uri_port is not None: + hostname += ":%s" % (uri_port) + specials.append(('hostname', hostname, lambda v: bool(v))) + for (k, v, is_not_empty_value_func) in specials: + if is_not_empty_value_func(v): + conf.setdefault(k, v) for (k, v) in six.iteritems(uri.params()): conf.setdefault(k, v) return conf From 21f49858e37483185a8db7c28f1542168b81e8eb Mon Sep 17 00:00:00 2001 From: leizhang Date: Mon, 4 May 2015 17:21:51 +0800 Subject: [PATCH 156/246] Fix a typo in taskflow docs Fix a misspelling, I think it should other than instead of other then. Change-Id: I133652f4ee8c9edc1c12cf71cd7372e730162df5 --- doc/source/arguments_and_results.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/doc/source/arguments_and_results.rst b/doc/source/arguments_and_results.rst index 009086bf..a8cc2e57 100644 --- a/doc/source/arguments_and_results.rst +++ b/doc/source/arguments_and_results.rst @@ -84,7 +84,7 @@ Rebinding --------- **Why:** There are cases when the value you want to pass to a task/retry is -stored with a name other then the corresponding arguments name. That's when the +stored with a name other than the corresponding arguments name. That's when the ``rebind`` constructor parameter comes in handy. Using it the flow author can instruct the engine to fetch a value from storage by one name, but pass it to a tasks/retrys ``execute`` method with another name. There are two possible From a37a881764cec2ed3e35ad0a97b665b24c687803 Mon Sep 17 00:00:00 2001 From: OpenStack Proposal Bot Date: Mon, 4 May 2015 18:51:54 +0000 Subject: [PATCH 157/246] Updated from global requirements Change-Id: I04e09650f8018bb1cde6072865b6c38fac9c17a0 --- test-requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test-requirements.txt b/test-requirements.txt index 939b2ffd..a6a7a19d 100644 --- a/test-requirements.txt +++ b/test-requirements.txt @@ -9,7 +9,7 @@ testtools>=0.9.36,!=1.2.0 testscenarios>=0.4 # Used for testing the WBE engine. -kombu>=2.5.0 +kombu>=3.0.7 # Used for testing zookeeper & backends. zake>=0.1.6 # Apache-2.0 From a0cc12db182a9b8d57bbfbee223286fc85638738 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Tue, 5 May 2015 18:16:46 -0700 Subject: [PATCH 158/246] Speed up memory backend via a path -> node reverse mapping A simple reverse mapping/dict is added and used to avoid repeated linear finding of paths, and metadata (and a little tiny helper inode class) is added so that each node knows its full path (avoiding repeated traversal to determine this same info). For linear flow with 500 'dummy' nodes on an old machine running python 2.6 (your numbers will vary): Old --- Took 0.255 seconds to run building Took 0.456 seconds to run compiling Took 50.039 seconds to run preparing Took 0.796 seconds to run validating Took 2.853 seconds to run running New --- Took 0.254 seconds to run building Took 0.457 seconds to run compiling Took 30.780 seconds to run preparing Took 0.776 seconds to run validating Took 2.222 seconds to run running Change-Id: Idc9ce88865b58a2dffd2e8955f0fab7f056d92b3 --- taskflow/persistence/backends/impl_memory.py | 85 ++++++++++++-------- 1 file changed, 51 insertions(+), 34 deletions(-) diff --git a/taskflow/persistence/backends/impl_memory.py b/taskflow/persistence/backends/impl_memory.py index b9489835..31913a6b 100644 --- a/taskflow/persistence/backends/impl_memory.py +++ b/taskflow/persistence/backends/impl_memory.py @@ -27,6 +27,13 @@ from taskflow.types import tree from taskflow.utils import lock_utils +class FakeInode(tree.Node): + """A in-memory filesystem inode-like object.""" + + def __init__(self, item, path, value=None): + super(FakeInode, self).__init__(item, path=path, value=value) + + class FakeFilesystem(object): """An in-memory filesystem-like structure. @@ -79,7 +86,10 @@ class FakeFilesystem(object): return pp.sep.join(pieces) def __init__(self, deep_copy=True): - self._root = tree.Node(self.root_path, value=None) + self._root = FakeInode(self.root_path, self.root_path) + self._reverse_mapping = { + self.root_path: self._root, + } if deep_copy: self._copier = copy.deepcopy else: @@ -97,21 +107,33 @@ class FakeFilesystem(object): child_node = node.find(piece, only_direct=True, include_self=False) if child_node is None: - child_node = tree.Node(piece, value=None) - node.add(child_node) + child_node = self._insert_child(node, piece) node = child_node - def _fetch_node(self, path): - node = self._root - path = self.normpath(path) - if path == self._root.item: - return node - for piece in self._iter_pieces(path): - node = node.find(piece, only_direct=True, - include_self=False) - if node is None: - raise exc.NotFound("Path '%s' not found" % path) - return node + def _insert_child(self, parent_node, basename, value=None): + child_path = self.join(parent_node.metadata['path'], basename) + # This avoids getting '//a/b' (duplicated sep at start)... + # + # Which can happen easily if something like the following is given. + # >>> x = ['/', 'b'] + # >>> pp.sep.join(x) + # '//b' + if child_path.startswith(pp.sep * 2): + child_path = child_path[1:] + child_node = FakeInode(basename, child_path, value=value) + parent_node.add(child_node) + self._reverse_mapping[child_path] = child_node + return child_node + + def _fetch_node(self, path, normalized=False): + if not normalized: + normed_path = self.normpath(path) + else: + normed_path = path + try: + return self._reverse_mapping[normed_path] + except KeyError: + raise exc.NotFound("Path '%s' not found" % path) def get(self, path, default=None): """Fetch the value of given path (and return default if not found).""" @@ -142,23 +164,14 @@ class FakeFilesystem(object): if not recursive: return [node.item for node in self._fetch_node(path)] else: - paths = [] node = self._fetch_node(path) - for child in node.bfs_iter(): - # Reconstruct the child's path... - hops = [child.item] - for parent in child.path_iter(include_self=False): - hops.append(parent.item) - hops.reverse() - # This avoids getting '//a/b' (duplicated sep at start)... - child_path = self.join(*hops) - if child_path.startswith("//"): - child_path = child_path[1:] - paths.append(child_path) - return paths + return [child.metadata['path'] for child in node.bfs_iter()] def clear(self): """Remove all nodes (except the root) from this filesystem.""" + self._reverse_mapping = { + self.root_path: self._root, + } for node in list(self._root.reverse_iter()): node.disassociate() @@ -179,9 +192,14 @@ class FakeFilesystem(object): yield piece def __delitem__(self, path): - node = self._fetch_node(path) + path = self.normpath(path) + node = self._fetch_node(path, normalized=True) if node is self._root: raise ValueError("Can not delete '%s'" % self._root.item) + removals = [path] + removals.extend(child.metadata['path'] for child in node.bfs_iter()) + for path in removals: + self._reverse_mapping.pop(path, None) node.disassociate() @staticmethod @@ -200,13 +218,12 @@ class FakeFilesystem(object): dest_path = self.normpath(dest_path) src_path = self.normpath(src_path) dirname, basename = pp.split(dest_path) - parent_node = self._fetch_node(dirname) + parent_node = self._fetch_node(dirname, normalized=True) child_node = parent_node.find(basename, only_direct=True, include_self=False) if child_node is None: - child_node = tree.Node(basename, value=None) - parent_node.add(child_node) + child_node = self._insert_child(parent_node, basename) child_node.metadata['target'] = src_path def __getitem__(self, path): @@ -216,12 +233,12 @@ class FakeFilesystem(object): path = self.normpath(path) value = self._copier(value) try: - item_node = self._fetch_node(path) + item_node = self._fetch_node(path, normalized=True) item_node.metadata.update(value=value) except exc.NotFound: dirname, basename = pp.split(path) - parent_node = self._fetch_node(dirname) - parent_node.add(tree.Node(basename, value=value)) + parent_node = self._fetch_node(dirname, normalized=True) + self._insert_child(parent_node, basename, value=value) class MemoryBackend(path_based.PathBasedBackend): From 5efcbf0b4c99bf7e23d875ae4309e78e64f59854 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 6 May 2015 11:56:14 -0700 Subject: [PATCH 159/246] Add speed-test tools script This little helper/tool script can be used to profile (at a very high level) the various stages that a taskflow user would typically perform and can be used to spot obvious issues with the various stages (which can then be used to guide where changes should be made to improve the overall stage speed). Change-Id: Ib3a69801280bcacc56e3a4a88c4d24e4fae4d353 --- tools/speed_test.py | 61 +++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 61 insertions(+) create mode 100644 tools/speed_test.py diff --git a/tools/speed_test.py b/tools/speed_test.py new file mode 100644 index 00000000..90385940 --- /dev/null +++ b/tools/speed_test.py @@ -0,0 +1,61 @@ +# Copyright (C) 2015 Yahoo! Inc. 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. + +from six.moves import range as compat_range + +from taskflow import engines +from taskflow.patterns import linear_flow as lf +from taskflow import task +from taskflow.types import timing + + +class TimeIt(object): + def __init__(self, name): + self.watch = timing.StopWatch() + self.name = name + + def __enter__(self): + self.watch.restart() + + def __exit__(self, exc_tp, exc_v, exc_tb): + self.watch.stop() + duration = self.watch.elapsed() + print("Took %0.3f seconds to run '%s'" % (duration, self.name)) + + +class DummyTask(task.Task): + def execute(self): + pass + + +def main(): + dummy_am = 100 + with TimeIt("building"): + f = lf.Flow("root") + for i in compat_range(0, dummy_am): + f.add(DummyTask(name="dummy_%s" % i)) + with TimeIt("loading"): + e = engines.load(f) + with TimeIt("compiling"): + e.compile() + with TimeIt("preparing"): + e.prepare() + with TimeIt("validating"): + e.validate() + with TimeIt("running"): + e.run() + + +if __name__ == "__main__": + main() From 73b98de8b4a7e6871fe8174936e39de9e85efc17 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 6 May 2015 18:41:39 -0700 Subject: [PATCH 160/246] Avoid duplicating exception message Now that we have exception chaining working nicely we do not need to duplicate the original exceptions message in followup exceptions (as the chain will by its very nature have this same information/message); so let's stop duplicating it. Change-Id: I9b947c0d8c1df894de346835f92c12e8773349c4 --- taskflow/persistence/backends/impl_zookeeper.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/taskflow/persistence/backends/impl_zookeeper.py b/taskflow/persistence/backends/impl_zookeeper.py index 0d7c00ee..279e0c66 100644 --- a/taskflow/persistence/backends/impl_zookeeper.py +++ b/taskflow/persistence/backends/impl_zookeeper.py @@ -97,12 +97,12 @@ class ZkConnection(path_based.PathBasedConnection): except k_exc.SessionExpiredError: exc.raise_with_cause(exc.StorageFailure, "Storage backend session has expired") - except k_exc.NoNodeError as e: + except k_exc.NoNodeError: exc.raise_with_cause(exc.NotFound, - "Storage backend node not found: %s" % e) - except k_exc.NodeExistsError as e: + "Storage backend node not found") + except k_exc.NodeExistsError: exc.raise_with_cause(exc.Duplicate, - "Storage backend duplicate node: %s" % e) + "Storage backend duplicate node") except (k_exc.KazooException, k_exc.ZookeeperError): exc.raise_with_cause(exc.StorageFailure, "Storage backend internal error") From 32ccdf1340af8454c02e07b6157786374a04cd37 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Tue, 5 May 2015 15:45:06 -0700 Subject: [PATCH 161/246] Make the default path a constant and tweak class docstring Have the default path used be a class level constant (that is picked up by sphinx) and tweak the docstring to be formatted better and to read better (with less grammatical issues). Change-Id: I43b78a2b36ed92dbd6dfb4d7376535ad361dc03d --- taskflow/jobs/backends/impl_zookeeper.py | 36 +++++++++++++----------- 1 file changed, 20 insertions(+), 16 deletions(-) diff --git a/taskflow/jobs/backends/impl_zookeeper.py b/taskflow/jobs/backends/impl_zookeeper.py index de731423..252ae81a 100644 --- a/taskflow/jobs/backends/impl_zookeeper.py +++ b/taskflow/jobs/backends/impl_zookeeper.py @@ -320,22 +320,23 @@ class ZookeeperJobBoard(base.NotifyingJobBoard): Powered by the `kazoo `_ library. This jobboard creates *sequenced* persistent znodes in a directory in - zookeeper (that directory defaults ``/taskflow/jobs``) and uses zookeeper - watches to notify other jobboards that the job which was posted using the - :meth:`.post` method (this creates a znode with contents/details in json) - The users of those jobboard(s) (potentially on disjoint sets of machines) - can then iterate over the available jobs and decide if they want to attempt - to claim one of the jobs they have iterated over. If so they will then - attempt to contact zookeeper and will attempt to create a ephemeral znode - using the name of the persistent znode + ".lock" as a postfix. If the - entity trying to use the jobboard to :meth:`.claim` the job is able to - create a ephemeral znode with that name then it will be allowed (and - expected) to perform whatever *work* the contents of that job that it - locked described. Once finished the ephemeral znode and persistent znode - may be deleted (if successfully completed) in a single transcation or if - not successfull (or the entity that claimed the znode dies) the ephemeral + zookeeper (that directory defaults to ``/taskflow/jobs``) and uses + zookeeper watches to notify other jobboards that the job which was posted + using the :meth:`.post` method (this creates a znode with contents/details + in json). The users of those jobboard(s) (potentially on disjoint sets of + machines) can then iterate over the available jobs and decide if they want + to attempt to claim one of the jobs they have iterated over. If so they + will then attempt to contact zookeeper and they will attempt to create a + ephemeral znode using the name of the persistent znode + ".lock" as a + postfix. If the entity trying to use the jobboard to :meth:`.claim` the + job is able to create a ephemeral znode with that name then it will be + allowed (and expected) to perform whatever *work* the contents of that + job described. Once finished the ephemeral znode and persistent znode may + be deleted (if successfully completed) in a single transaction or if not + successful (or the entity that claimed the znode dies) the ephemeral znode will be released (either manually by using :meth:`.abandon` or - automatically by zookeeper the ephemeral is deemed to be lost). + automatically by zookeeper when the ephemeral node and associated session + is deemed to have been lost). """ #: Transaction support was added in 3.4.0 so we need at least that version. @@ -350,6 +351,9 @@ class ZookeeperJobBoard(base.NotifyingJobBoard): #: Znode **prefix** that job entries have. JOB_PREFIX = 'job' + #: Default znode path used for jobs (data, locks...). + DEFAULT_PATH = "/taskflow/jobs" + def __init__(self, name, conf, client=None, persistence=None, emit_notifications=True): super(ZookeeperJobBoard, self).__init__(name, conf) @@ -359,7 +363,7 @@ class ZookeeperJobBoard(base.NotifyingJobBoard): else: self._client = kazoo_utils.make_client(self._conf) self._owned = True - path = str(conf.get("path", "/taskflow/jobs")) + path = str(conf.get("path", self.DEFAULT_PATH)) if not path: raise ValueError("Empty zookeeper path is disallowed") if not k_paths.isabs(path): From 51a82bbc211d2f4d15fb200aae4c7d8c6de53a9a Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Thu, 7 May 2015 10:01:46 -0700 Subject: [PATCH 162/246] Remove validation of state on state read property access This should be done (and are done, but not with the best exception messages) in a more appropriate place (in the methods of the jobboard that mutate a job via some manner, aka, in consume, claim, abandon, trash...). Part of fix for bug 1452388 Change-Id: If29bc73cc6e9282ad35a50ce0041efc3ff8f71d9 --- taskflow/jobs/backends/impl_zookeeper.py | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/taskflow/jobs/backends/impl_zookeeper.py b/taskflow/jobs/backends/impl_zookeeper.py index 252ae81a..246416fa 100644 --- a/taskflow/jobs/backends/impl_zookeeper.py +++ b/taskflow/jobs/backends/impl_zookeeper.py @@ -79,7 +79,6 @@ class ZookeeperJob(base.Job): basename = k_paths.basename(self._path) self._root = self._path[0:-len(basename)] self._sequence = int(basename[len(board.JOB_PREFIX):]) - self._last_state = None @property def lock_path(self): @@ -174,11 +173,7 @@ class ZookeeperJob(base.Job): @property def state(self): - current_state = self._fetch_state() - if self._last_state is not None: - states.check_job_transition(self._last_state, current_state) - self._last_state = current_state - return current_state + return self._fetch_state() def _fetch_state(self): owner = self.board.find_owner(self) From 562884d551049f1074b9cdcd430bd5133402cfa2 Mon Sep 17 00:00:00 2001 From: OpenStack Proposal Bot Date: Thu, 7 May 2015 23:38:08 +0000 Subject: [PATCH 163/246] Updated from global requirements Change-Id: I65c93be5a9163501069961e632bc3bd313695ff9 --- requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/requirements.txt b/requirements.txt index 8e0c5f3d..f0ec27ed 100644 --- a/requirements.txt +++ b/requirements.txt @@ -3,7 +3,7 @@ # process, which may cause wedges in the gate later. # See: https://bugs.launchpad.net/pbr/+bug/1384919 for why this is here... -pbr>=0.6,!=0.7,<1.0 +pbr>=0.11,<2.0 # Packages needed for using this library. From e247e07df9426ca1ab1cc19a8e105d559b1d7b90 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Thu, 7 May 2015 16:55:39 -0700 Subject: [PATCH 164/246] Add a profiling context manager that can be easily enabled Change-Id: Icd27abe032b7e4daf78dc9a9e80f5daeee8f078e --- tools/speed_test.py | 87 ++++++++++++++++++++++++++++++++++++++++----- 1 file changed, 78 insertions(+), 9 deletions(-) diff --git a/tools/speed_test.py b/tools/speed_test.py index 90385940..45bca783 100644 --- a/tools/speed_test.py +++ b/tools/speed_test.py @@ -12,6 +12,15 @@ # License for the specific language governing permissions and limitations # under the License. +""" +Profile a simple engine build/load/compile/prepare/validate/run. +""" + +import argparse +import cProfile as profiler +import pstats + +import six from six.moves import range as compat_range from taskflow import engines @@ -20,10 +29,48 @@ from taskflow import task from taskflow.types import timing +def print_header(name): + if name: + header_footer = "-" * len(name) + print(header_footer) + print(name) + print(header_footer) + + +class ProfileIt(object): + stats_ordering = ('cumulative', 'calls',) + + def __init__(self, name, args): + self.name = name + self.profile = profiler.Profile() + self.args = args + + def __enter__(self): + self.profile.enable() + + def __exit__(self, exc_tp, exc_v, exc_tb): + self.profile.disable() + buf = six.StringIO() + ps = pstats.Stats(self.profile, stream=buf) + ps = ps.sort_stats(*self.stats_ordering) + percent_limit = max(0.0, max(1.0, self.args.limit / 100.0)) + ps.print_stats(percent_limit) + print_header(self.name) + needs_newline = False + for line in buf.getvalue().splitlines(): + line = line.lstrip() + if line: + print(line) + needs_newline = True + if needs_newline: + print("") + + class TimeIt(object): - def __init__(self, name): + def __init__(self, name, args): self.watch = timing.StopWatch() self.name = name + self.args = args def __enter__(self): self.watch.restart() @@ -31,7 +78,8 @@ class TimeIt(object): def __exit__(self, exc_tp, exc_v, exc_tb): self.watch.stop() duration = self.watch.elapsed() - print("Took %0.3f seconds to run '%s'" % (duration, self.name)) + print_header(self.name) + print("- Took %0.3f seconds to run" % (duration)) class DummyTask(task.Task): @@ -40,20 +88,41 @@ class DummyTask(task.Task): def main(): - dummy_am = 100 - with TimeIt("building"): + parser = argparse.ArgumentParser(description=__doc__) + parser.add_argument('--profile', "-p", + dest='profile', action='store_true', + default=False, + help='profile instead of gather timing' + ' (default: False)') + parser.add_argument('--dummies', "-d", + dest='dummies', action='store', type=int, + default=100, metavar="", + help='how many dummy/no-op tasks to inject' + ' (default: 100)') + parser.add_argument('--limit', '-l', + dest='limit', action='store', type=float, + default=100.0, metavar="", + help='percentage of profiling output to show' + ' (default: 100%%)') + args = parser.parse_args() + if args.profile: + ctx_manager = ProfileIt + else: + ctx_manager = TimeIt + dummy_am = max(0, args.dummies) + with ctx_manager("Building linear flow with %s tasks" % dummy_am, args): f = lf.Flow("root") for i in compat_range(0, dummy_am): f.add(DummyTask(name="dummy_%s" % i)) - with TimeIt("loading"): + with ctx_manager("Loading", args): e = engines.load(f) - with TimeIt("compiling"): + with ctx_manager("Compiling", args): e.compile() - with TimeIt("preparing"): + with ctx_manager("Preparing", args): e.prepare() - with TimeIt("validating"): + with ctx_manager("Validating", args): e.validate() - with TimeIt("running"): + with ctx_manager("Running", args): e.run() From 115bf989bf10dce55e15cdaad0253a3034732919 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Thu, 7 May 2015 18:05:01 -0700 Subject: [PATCH 165/246] Avoid re-normalizing paths when following links There is not any need to re-normalize internal paths once the initial get() or __get_item__() have normalized there path arguments so in _get_item() just tell the fetching function to avoid re-normalizing the paths its provided. This also works for link following since the target placed in the fake inodes metadata is already normalized when it is inserted. Part of blueprint make-things-speedy Change-Id: If31d61b744f8df618c7db3afffc4a118cb17003a --- taskflow/persistence/backends/impl_memory.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/taskflow/persistence/backends/impl_memory.py b/taskflow/persistence/backends/impl_memory.py index 31913a6b..5a90996c 100644 --- a/taskflow/persistence/backends/impl_memory.py +++ b/taskflow/persistence/backends/impl_memory.py @@ -138,12 +138,12 @@ class FakeFilesystem(object): def get(self, path, default=None): """Fetch the value of given path (and return default if not found).""" try: - return self._get_item(path) + return self._get_item(self.normpath(path)) except exc.NotFound: return default def _get_item(self, path, links=None): - node = self._fetch_node(path) + node = self._fetch_node(path, normalized=True) if 'target' in node.metadata: # Follow the link (and watch out for loops)... path = node.metadata['target'] @@ -227,7 +227,7 @@ class FakeFilesystem(object): child_node.metadata['target'] = src_path def __getitem__(self, path): - return self._get_item(path) + return self._get_item(self.normpath(path)) def __setitem__(self, path, value): path = self.normpath(path) From e6a04193e73bd44450eca9859b035cf91b4fcf4b Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Thu, 7 May 2015 20:49:46 -0700 Subject: [PATCH 166/246] Avoid trying to copy tasks results when cloning/copying It is not always safe to copy task results so instad of trying to copy.copy them just retain the existing results in the clone (and avoid copying any users results if they exist). This appeared to be causing sqlalchemy weak references to get garbage collected prematurely in cinder. Fixes bug 1452978 Change-Id: I75e4ad0e9eb50284ff09179565b6d146a8da3d92 --- taskflow/persistence/logbook.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/taskflow/persistence/logbook.py b/taskflow/persistence/logbook.py index fcd777da..db830b95 100644 --- a/taskflow/persistence/logbook.py +++ b/taskflow/persistence/logbook.py @@ -448,7 +448,12 @@ class TaskDetail(AtomDetail): def copy(self): """Copies/clones this task detail.""" clone = copy.copy(self) - clone.results = copy.copy(self.results) + # Just directly assign to the clone (do **not** copy). + # + # See: https://bugs.launchpad.net/taskflow/+bug/1452978 for + # what happens if this is cloned/copied (even using copy.copy to + # try to do a shallow copy). + clone.results = self.results if self.meta: clone.meta = self.meta.copy() if self.version: From 84c1ad65fc1bc2a43b446f4e5fe84e9e21f87001 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Fri, 8 May 2015 16:25:38 -0700 Subject: [PATCH 167/246] Avoid creating temporary removal lists Instead of creating a temporary list, only to then iterate over it (and then never again use that temporary list) just use itertools and create an iterator that we use for iterating (therefore avoiding any need to create a temporary list). Part of ongoing blueprint make-things-speedy Change-Id: I5322e5bdf613d485fbc8851c1319a907b425e2dd --- taskflow/persistence/backends/impl_memory.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/taskflow/persistence/backends/impl_memory.py b/taskflow/persistence/backends/impl_memory.py index 31913a6b..ab3bf25c 100644 --- a/taskflow/persistence/backends/impl_memory.py +++ b/taskflow/persistence/backends/impl_memory.py @@ -17,6 +17,7 @@ import contextlib import copy +import itertools import posixpath as pp import six @@ -196,9 +197,8 @@ class FakeFilesystem(object): node = self._fetch_node(path, normalized=True) if node is self._root: raise ValueError("Can not delete '%s'" % self._root.item) - removals = [path] - removals.extend(child.metadata['path'] for child in node.bfs_iter()) - for path in removals: + child_gen = (child.metadata['path'] for child in node.bfs_iter()) + for path in itertools.chain([path], child_gen): self._reverse_mapping.pop(path, None) node.disassociate() From 98cb8418d531115516047a1429435fd1c16acaf5 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Sat, 9 May 2015 14:35:28 -0700 Subject: [PATCH 168/246] Remove tox py33 environment no longer used This py33 environment is no longer being tested with in an automated fashion (since the py34 environment replaces it) so we should just remove it from the tox ini file. Change-Id: I3a4040b8a3c656b93b7e0ff2c20bed4119ac7c74 --- tox.ini | 6 ------ 1 file changed, 6 deletions(-) diff --git a/tox.ini b/tox.ini index 705af0b5..f3ba27cf 100644 --- a/tox.ini +++ b/tox.ini @@ -8,7 +8,6 @@ envlist = cover, py26-sa7-mysql, py27, py27-sa8-mysql, - py33, py34, pylint, update-states @@ -85,11 +84,6 @@ commands = sphinx-build -b doctest doc/source doc/build doc8 doc/source -[testenv:py33] -deps = {[testenv]deps} - SQLAlchemy>=0.7.8,<=0.9.99 - PyMySQL>=0.6.2 - [testenv:py34] deps = {[testenv]deps} SQLAlchemy>=0.7.8,<=0.9.99 From 4e550ba48ca76c93d58dcdeecf96d774a5ee62f0 Mon Sep 17 00:00:00 2001 From: OpenStack Proposal Bot Date: Mon, 11 May 2015 14:04:44 +0000 Subject: [PATCH 169/246] Updated from global requirements Change-Id: Icdc0089935838811142d83da6d33065e9901139b --- requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/requirements.txt b/requirements.txt index f0ec27ed..b0e0eb22 100644 --- a/requirements.txt +++ b/requirements.txt @@ -23,7 +23,7 @@ networkx>=1.8 stevedore>=1.3.0 # Apache-2.0 # Backport for concurrent.futures which exists in 3.2+ -futures>=2.1.6 +futures>=3.0 # Used for structured input validation jsonschema>=2.0.0,<3.0.0 From d76bd26077dbd18064ca2ce61f4a749dc242eada Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Fri, 8 May 2015 16:06:39 -0700 Subject: [PATCH 170/246] Expose in memory backend split staticmethod Just like we expose join we should also expose split to ensure that both of these operations can easily performed by users of this backend. This also refactors the internal usage to call into that static method vs calling into the posix path module specific one. Change-Id: I496c730b86f0af6d4b637862a92482c0df6a63b3 --- taskflow/persistence/backends/impl_memory.py | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/taskflow/persistence/backends/impl_memory.py b/taskflow/persistence/backends/impl_memory.py index 31913a6b..0b1b1d5d 100644 --- a/taskflow/persistence/backends/impl_memory.py +++ b/taskflow/persistence/backends/impl_memory.py @@ -80,6 +80,9 @@ class FakeFilesystem(object): " valid" % (cls.root_path, path)) return pp.normpath(path) + #: Split a pathname into a tuple of ``(head, tail)``. + split = staticmethod(pp.split) + @staticmethod def join(*pieces): """Join many path segments together.""" @@ -217,7 +220,7 @@ class FakeFilesystem(object): """Link the destionation path to the source path.""" dest_path = self.normpath(dest_path) src_path = self.normpath(src_path) - dirname, basename = pp.split(dest_path) + dirname, basename = self.split(dest_path) parent_node = self._fetch_node(dirname, normalized=True) child_node = parent_node.find(basename, only_direct=True, @@ -236,7 +239,7 @@ class FakeFilesystem(object): item_node = self._fetch_node(path, normalized=True) item_node.metadata.update(value=value) except exc.NotFound: - dirname, basename = pp.split(path) + dirname, basename = self.split(path) parent_node = self._fetch_node(dirname, normalized=True) self._insert_child(parent_node, basename, value=value) From 5cfeeb6b0264eb476368819980a6b5cae47ab62e Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Fri, 8 May 2015 16:49:19 -0700 Subject: [PATCH 171/246] Chain a few more exception raises that were previously missed Change-Id: I959b65ba9effbe3040fb0d010f1f8516a80e7cb9 --- taskflow/persistence/logbook.py | 8 ++++---- taskflow/storage.py | 33 +++++++++++++++++++-------------- 2 files changed, 23 insertions(+), 18 deletions(-) diff --git a/taskflow/persistence/logbook.py b/taskflow/persistence/logbook.py index db830b95..b3385b88 100644 --- a/taskflow/persistence/logbook.py +++ b/taskflow/persistence/logbook.py @@ -495,15 +495,15 @@ class RetryDetail(AtomDetail): def last_results(self): try: return self.results[-1][0] - except IndexError as e: - raise exc.NotFound("Last results not found", e) + except IndexError: + exc.raise_with_cause(exc.NotFound, "Last results not found") @property def last_failures(self): try: return self.results[-1][1] - except IndexError as e: - raise exc.NotFound("Last failures not found", e) + except IndexError: + exc.raise_with_cause(exc.NotFound, "Last failures not found") def put(self, state, result): # Do not clean retry history (only on reset does this happen). diff --git a/taskflow/storage.py b/taskflow/storage.py index 83311e92..22223f48 100644 --- a/taskflow/storage.py +++ b/taskflow/storage.py @@ -94,7 +94,8 @@ def _item_from_single(provider, container, looking_for): try: return _item_from(container, provider.index) except _EXTRACTION_EXCEPTIONS: - raise exceptions.NotFound( + exceptions.raise_with_cause( + exceptions.NotFound, "Unable to find result %r, expected to be able to find it" " created by %s but was unable to perform successful" " extraction" % (looking_for, provider)) @@ -311,7 +312,8 @@ class Storage(object): try: ad = self._flowdetail.find(self._atom_name_to_uuid[atom_name]) except KeyError: - raise exceptions.NotFound("Unknown atom name: %s" % atom_name) + exceptions.raise_with_cause(exceptions.NotFound, + "Unknown atom name: %s" % atom_name) else: # TODO(harlowja): we need to figure out how to get away from doing # these kinds of type checks in general (since they likely mean @@ -484,10 +486,11 @@ class Storage(object): retry_name, expected_type=logbook.RetryDetail, clone=True) try: failures = clone.last_failures - except exceptions.NotFound as e: - raise exceptions.StorageFailure("Unable to fetch most recent" - " retry failures so new retry" - " failure can be inserted", e) + except exceptions.NotFound: + exceptions.raise_with_cause(exceptions.StorageFailure, + "Unable to fetch most recent retry" + " failures so new retry failure can" + " be inserted") else: if failed_atom_name not in failures: failures[failed_atom_name] = failure @@ -695,9 +698,9 @@ class Storage(object): try: providers = self._reverse_mapping[name] except KeyError: - raise exceptions.NotFound("Name %r is not mapped as a" - " produced output by any" - " providers" % name) + exceptions.raise_with_cause(exceptions.NotFound, + "Name %r is not mapped as a produced" + " output by any providers" % name) values = [] for provider in providers: if provider.name is _TRANSIENT_PROVIDER: @@ -839,11 +842,13 @@ class Storage(object): """Gets the results saved for a given provider.""" try: return self._get(provider.name, only_last=True) - except exceptions.NotFound as e: - raise exceptions.NotFound( - "Expected to be able to find output %r produced" - " by %s but was unable to get at that providers" - " results" % (looking_for, provider), e) + except exceptions.NotFound: + exceptions.raise_with_cause(exceptions.NotFound, + "Expected to be able to find" + " output %r produced by %s but was" + " unable to get at that providers" + " results" % (looking_for, + provider)) def _locate_providers(looking_for, possible_providers, scope_walker=None): From 24a11aa5235782d4c9aab91e977d6df73d011bea Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Mon, 11 May 2015 15:06:25 -0700 Subject: [PATCH 172/246] Expose action engine no reraising states constants Change-Id: I65746fc7bc27f0b03e416eb6584df18bbc4a3053 --- taskflow/engines/action_engine/engine.py | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/taskflow/engines/action_engine/engine.py b/taskflow/engines/action_engine/engine.py index bed43e8e..12f5feac 100644 --- a/taskflow/engines/action_engine/engine.py +++ b/taskflow/engines/action_engine/engine.py @@ -66,6 +66,13 @@ class ActionEngine(base.Engine): """ _compiler_factory = compiler.PatternCompiler + NO_RERAISING_STATES = frozenset([states.SUSPENDED, states.SUCCESS]) + """ + States that if the engine stops in will **not** cause any potential + failures to be reraised. States **not** in this list will cause any + failure/s that were captured (if any) to get reraised. + """ + def __init__(self, flow, flow_detail, backend, options): super(ActionEngine, self).__init__(flow, flow_detail, backend, options) self._runtime = None @@ -178,7 +185,7 @@ class ActionEngine(base.Engine): ignorable_states = getattr(runner, 'ignorable_states', []) if last_state and last_state not in ignorable_states: self._change_state(last_state) - if last_state not in [states.SUSPENDED, states.SUCCESS]: + if last_state not in self.NO_RERAISING_STATES: failures = self.storage.get_failures() failure.Failure.reraise_if_any(failures.values()) From 4fc6b612e00cfe549419ede223b8711e2be4d653 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Mon, 11 May 2015 21:04:46 -0700 Subject: [PATCH 173/246] Remove listener stack and replace with exit stack Change-Id: Ic9150b8208420d29865af297ae8d0c656becb81e --- requirements.txt | 3 ++ taskflow/conductors/backends/impl_blocking.py | 11 +++++-- taskflow/utils/misc.py | 32 ------------------- 3 files changed, 11 insertions(+), 35 deletions(-) diff --git a/requirements.txt b/requirements.txt index b0e0eb22..6abcb19f 100644 --- a/requirements.txt +++ b/requirements.txt @@ -19,6 +19,9 @@ enum34 # Very nice graph library networkx>=1.8 +# For contextlib new additions/compatibility for <= python 3.3 +contextlib2>=0.4.0 + # Used for backend storage engine loading. stevedore>=1.3.0 # Apache-2.0 diff --git a/taskflow/conductors/backends/impl_blocking.py b/taskflow/conductors/backends/impl_blocking.py index 5156da10..fb8a3c3a 100644 --- a/taskflow/conductors/backends/impl_blocking.py +++ b/taskflow/conductors/backends/impl_blocking.py @@ -12,6 +12,11 @@ # License for the specific language governing permissions and limitations # under the License. +try: + from contextlib import ExitStack # noqa +except ImportError: + from contextlib2 import ExitStack # noqa + from debtcollector import removals import six @@ -21,7 +26,6 @@ from taskflow.listeners import logging as logging_listener from taskflow import logging from taskflow.types import timing as tt from taskflow.utils import async_utils -from taskflow.utils import misc from taskflow.utils import threading_utils LOG = logging.getLogger(__name__) @@ -97,8 +101,9 @@ class BlockingConductor(base.Conductor): def _dispatch_job(self, job): engine = self._engine_from_job(job) listeners = self._listeners_from_job(job, engine) - with misc.ListenerStack(LOG) as stack: - stack.register(listeners) + with ExitStack() as stack: + for listener in listeners: + stack.enter_context(listener) LOG.debug("Dispatching engine %s for job: %s", engine, job) consume = True try: diff --git a/taskflow/utils/misc.py b/taskflow/utils/misc.py index ee5fb393..8624d7cd 100644 --- a/taskflow/utils/misc.py +++ b/taskflow/utils/misc.py @@ -437,38 +437,6 @@ def get_duplicate_keys(iterable, key=None): return duplicates -class ListenerStack(object): - """Listeners that are deregistered on context manager exit. - - TODO(harlowja): replace this with ``contextlib.ExitStack`` or equivalent - in the future (that code is in python3.2+ and in a few backports that - provide nearly equivalent functionality). When/if - https://review.openstack.org/#/c/164222/ merges we should be able to - remove this since listeners are already context managers. - """ - - def __init__(self, log): - self._registered = [] - self._log = log - - def register(self, listeners): - for listener in listeners: - listener.register() - self._registered.append(listener) - - def __enter__(self): - return self - - def __exit__(self, type, value, tb): - while self._registered: - listener = self._registered.pop() - try: - listener.deregister() - except Exception: - self._log.warn("Failed deregistering listener '%s'", - listener, exc_info=True) - - class ExponentialBackoff(object): """An iterable object that will yield back an exponential delay sequence. From 5d4dfe055efe5deab537ab77f78a689186508545 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Mon, 11 May 2015 22:31:38 -0700 Subject: [PATCH 174/246] Ensure empty paths raise a value error Empty paths should not be allowed to be set or fetched or normalized so check for those and raise an error when one is encountered. Also add some basic test conditions that ensure the normpath method raises when it should. Change-Id: I7f7e6600f03c67376ba310ab231b2e33cd7528db --- taskflow/persistence/backends/impl_memory.py | 3 +++ taskflow/tests/unit/persistence/test_memory_persistence.py | 6 ++++++ 2 files changed, 9 insertions(+) diff --git a/taskflow/persistence/backends/impl_memory.py b/taskflow/persistence/backends/impl_memory.py index 31913a6b..e341f18f 100644 --- a/taskflow/persistence/backends/impl_memory.py +++ b/taskflow/persistence/backends/impl_memory.py @@ -74,6 +74,9 @@ class FakeFilesystem(object): @classmethod def normpath(cls, path): """Return a normalized absolutized version of the pathname path.""" + if not path: + raise ValueError("This filesystem can only normalize paths" + " that are not empty") if not path.startswith(cls.root_path): raise ValueError("This filesystem can only normalize" " paths that start with %s: '%s' is not" diff --git a/taskflow/tests/unit/persistence/test_memory_persistence.py b/taskflow/tests/unit/persistence/test_memory_persistence.py index 2b3599e4..24f76aa3 100644 --- a/taskflow/tests/unit/persistence/test_memory_persistence.py +++ b/taskflow/tests/unit/persistence/test_memory_persistence.py @@ -128,6 +128,12 @@ class MemoryFilesystemTest(test.TestCase): fs = impl_memory.FakeFilesystem() self.assertRaises(exc.NotFound, self._get_item_path, fs, '/c') + def test_bad_norms(self): + fs = impl_memory.FakeFilesystem() + self.assertRaises(ValueError, fs.normpath, '') + self.assertRaises(ValueError, fs.normpath, 'abc/c') + self.assertRaises(ValueError, fs.normpath, '../c') + def test_del_root_not_allowed(self): fs = impl_memory.FakeFilesystem() self.assertRaises(ValueError, self._del_item_path, fs, '/') From 6990fb1d9ad092df0e3c1dcb1fa5bf775d3e5e84 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Mon, 4 May 2015 16:28:19 -0700 Subject: [PATCH 175/246] Use newer versions of futures that adds exception tracebacks Depends-On: I36f1241b983f6552d8bd0471e6a7485532a95a14 Change-Id: I49899019c4a1683bc6664cf0a52dcd5151b6e282 --- taskflow/tests/unit/test_utils_async_utils.py | 7 ------- taskflow/types/futures.py | 13 +++++++++++-- taskflow/utils/async_utils.py | 9 +++------ 3 files changed, 14 insertions(+), 15 deletions(-) diff --git a/taskflow/tests/unit/test_utils_async_utils.py b/taskflow/tests/unit/test_utils_async_utils.py index b538c2ee..642be96e 100644 --- a/taskflow/tests/unit/test_utils_async_utils.py +++ b/taskflow/tests/unit/test_utils_async_utils.py @@ -74,13 +74,6 @@ class MakeCompletedFutureTest(test.TestCase): self.assertTrue(future.done()) self.assertIs(future.result(), result) - def test_make_completed_future_exception(self): - result = IOError("broken") - future = au.make_completed_future(result, exception=True) - self.assertTrue(future.done()) - self.assertRaises(IOError, future.result) - self.assertIsNotNone(future.exception()) - class AsyncUtilsSynchronousTest(test.TestCase, WaitForAnyTestsMixin): diff --git a/taskflow/types/futures.py b/taskflow/types/futures.py index 2a6f7b6f..f14e4015 100644 --- a/taskflow/types/futures.py +++ b/taskflow/types/futures.py @@ -15,6 +15,7 @@ # under the License. import functools +import sys import threading from concurrent import futures as _futures @@ -22,6 +23,7 @@ from concurrent.futures import process as _process from concurrent.futures import thread as _thread from oslo_utils import importutils from oslo_utils import reflection +import six greenpatcher = importutils.try_import('eventlet.patcher') greenpool = importutils.try_import('eventlet.greenpool') @@ -175,8 +177,15 @@ class _WorkItem(object): return try: result = self.fn(*self.args, **self.kwargs) - except BaseException as e: - self.future.set_exception(e) + except BaseException: + exc_type, exc_value, exc_tb = sys.exc_info() + try: + if six.PY2: + self.future.set_exception_info(exc_value, exc_tb) + else: + self.future.set_exception(exc_value) + finally: + del(exc_type, exc_value, exc_tb) else: self.future.set_result(result) diff --git a/taskflow/utils/async_utils.py b/taskflow/utils/async_utils.py index e04c44e7..aec62abf 100644 --- a/taskflow/utils/async_utils.py +++ b/taskflow/utils/async_utils.py @@ -30,13 +30,10 @@ _DONE_STATES = frozenset([ ]) -def make_completed_future(result, exception=False): - """Make a future completed with a given result.""" +def make_completed_future(result): + """Make and return a future completed with a given result.""" future = futures.Future() - if exception: - future.set_exception(result) - else: - future.set_result(result) + future.set_result(result) return future From efd6d53aceb7de8723fe077c4a9f05a75f2875d7 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Mon, 11 May 2015 23:00:55 -0700 Subject: [PATCH 176/246] Executors come in via options config, not keyword arguments Change-Id: I5654b5db0db6b8e9884dae5cc285400edce2a698 --- taskflow/engines/action_engine/engine.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/taskflow/engines/action_engine/engine.py b/taskflow/engines/action_engine/engine.py index bed43e8e..5eb4fee6 100644 --- a/taskflow/engines/action_engine/engine.py +++ b/taskflow/engines/action_engine/engine.py @@ -312,7 +312,7 @@ class _ExecutorTextMatch(collections.namedtuple('_ExecutorTextMatch', class ParallelActionEngine(ActionEngine): """Engine that runs tasks in parallel manner. - Supported keyword arguments: + Supported option keys: * ``executor``: a object that implements a :pep:`3148` compatible executor interface; it will be used for scheduling tasks. The following From fb2b813e38504f17b5231435e8913a69530753ad Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Tue, 12 May 2015 17:56:39 -0700 Subject: [PATCH 177/246] Remove custom py26/py27 tox venvs no longer used Change-Id: I522ce82923837e9fd9d0dcf90e70916e85bc65ad --- tox.ini | 13 ------------- 1 file changed, 13 deletions(-) diff --git a/tox.ini b/tox.ini index f3ba27cf..3581facc 100644 --- a/tox.ini +++ b/tox.ini @@ -5,9 +5,7 @@ envlist = cover, docs, pep8, py26, - py26-sa7-mysql, py27, - py27-sa8-mysql, py34, pylint, update-states @@ -89,14 +87,3 @@ deps = {[testenv]deps} SQLAlchemy>=0.7.8,<=0.9.99 PyMySQL>=0.6.2 -[testenv:py26-sa7-mysql] -basepython = python2.6 -deps = {[testenv]deps} - SQLAlchemy>=0.7.8,<=0.7.99 - MySQL-python - -[testenv:py27-sa8-mysql] -basepython = python2.7 -deps = {[testenv]deps} - SQLAlchemy>=0.8,<=0.8.99 - MySQL-python From ad5b556add17fe763d2d739ed9cfc0f0450fcd79 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Fri, 8 May 2015 10:29:03 -0700 Subject: [PATCH 178/246] Beef up docs on the logbook/flow detail/atom details models Make it easier to understand what these objects do and what the methods on those objects perform and return (and the limitations certain methods have). Also fixes one more instance of copying users results (where on task detail merge the incoming task details results attribute was copied into the current objects). This has the same problem as mentioned in bug 1452978. Change-Id: I265f06ccea751a70ee428e8a690805a5037a5a9d --- doc/source/persistence.rst | 4 + taskflow/persistence/logbook.py | 463 +++++++++++++++++++++++++------- 2 files changed, 373 insertions(+), 94 deletions(-) diff --git a/doc/source/persistence.rst b/doc/source/persistence.rst index 0bab644c..7db76341 100644 --- a/doc/source/persistence.rst +++ b/doc/source/persistence.rst @@ -283,6 +283,10 @@ Interfaces .. automodule:: taskflow.persistence.backends .. automodule:: taskflow.persistence.base + +Models +====== + .. automodule:: taskflow.persistence.logbook Implementations diff --git a/taskflow/persistence/logbook.py b/taskflow/persistence/logbook.py index db830b95..c7770feb 100644 --- a/taskflow/persistence/logbook.py +++ b/taskflow/persistence/logbook.py @@ -30,6 +30,9 @@ from taskflow.types import failure as ft LOG = logging.getLogger(__name__) +# Internal helpers... + + def _copy_function(deep_copy): if deep_copy: return copy.deepcopy @@ -64,20 +67,27 @@ def _fix_meta(data): class LogBook(object): - """A container of flow details, a name and associated metadata. + """A collection of flow details and associated metadata. Typically this class contains a collection of flow detail entries for a given engine (or job) so that those entities can track what 'work' has been completed for resumption, reverting and miscellaneous tracking purposes. - The data contained within this class need *not* be backed by the backend - storage in real time. The data in this class will only be guaranteed to be - persisted when a save occurs via some backend connection. + The data contained within this class need **not** be persisted to the + backend storage in real time. The data in this class will only be + guaranteed to be persisted when a save occurs via some backend + connection. - NOTE(harlowja): the naming of this class is analogous to a ships log or a - similar type of record used in detailing work that been completed (or work - that has not been completed). + NOTE(harlowja): the naming of this class is analogous to a ship's log or a + similar type of record used in detailing work that has been completed (or + work that has not been completed). + + :ivar created_at: A ``datetime.datetime`` object of when this logbook + was created. + :ivar updated_at: A ``datetime.datetime`` object of when this logbook + was last updated at. + :ivar meta: A dictionary of meta-data associated with this logbook. """ def __init__(self, name, uuid=None): if uuid: @@ -91,7 +101,11 @@ class LogBook(object): self.meta = {} def add(self, fd): - """Adds a new entry to the underlying logbook. + """Adds a new flow detail into this logbook. + + NOTE(harlowja): if an existing flow detail exists with the same + uuid the existing one will be overwritten with the newly provided + one. Does not *guarantee* that the details will be immediately saved. """ @@ -99,12 +113,29 @@ class LogBook(object): self.updated_at = timeutils.utcnow() def find(self, flow_uuid): + """Locate the flow detail corresponding to the given uuid. + + :returns: the flow detail with that uuid + :rtype: :py:class:`.FlowDetail` (or ``None`` if not found) + """ return self._flowdetails_by_id.get(flow_uuid, None) def merge(self, lb, deep_copy=False): """Merges the current object state with the given ones state. - NOTE(harlowja): Does not merge the flow details contained in either. + If ``deep_copy`` is provided as truthy then the + local object will use ``copy.deepcopy`` to replace this objects + local attributes with the provided objects attributes (**only** if + there is a difference between this objects attributes and the + provided attributes). If ``deep_copy`` is falsey (the default) then a + reference copy will occur instead when a difference is detected. + + NOTE(harlowja): If the provided object is this object itself + then **no** merging is done. Also note that this does **not** merge + the flow details contained in either. + + :returns: this logbook (freshly merged with the incoming object) + :rtype: :py:class:`.LogBook` """ if lb is self: return self @@ -118,26 +149,35 @@ class LogBook(object): return self def to_dict(self, marshal_time=False): - """Translates the internal state of this object to a dictionary. + """Translates the internal state of this object to a ``dict``. - NOTE(harlowja): Does not include the contained flow details. + NOTE(harlowja): The returned ``dict`` does **not** include any + contained flow details. + + :returns: this logbook in ``dict`` form """ if not marshal_time: marshal_fn = lambda x: x else: marshal_fn = _safe_marshal_time - data = { + return { 'name': self.name, 'meta': self.meta, 'uuid': self.uuid, 'updated_at': marshal_fn(self.updated_at), 'created_at': marshal_fn(self.created_at), } - return data @classmethod def from_dict(cls, data, unmarshal_time=False): - """Translates the given dictionary into an instance of this class.""" + """Translates the given ``dict`` into an instance of this class. + + NOTE(harlowja): the ``dict`` provided should come from a prior + call to :meth:`.to_dict`. + + :returns: a new logbook + :rtype: :py:class:`.LogBook` + """ if not unmarshal_time: unmarshal_fn = lambda x: x else: @@ -150,10 +190,12 @@ class LogBook(object): @property def uuid(self): + """The unique identifer of this logbook.""" return self._uuid @property def name(self): + """The name of this logbook.""" return self._name def __iter__(self): @@ -164,7 +206,19 @@ class LogBook(object): return len(self._flowdetails_by_id) def copy(self, retain_contents=True): - """Copies/clones this log book.""" + """Copies this logbook. + + Creates a shallow copy of this logbook. If this logbook contains + flow details and ``retain_contents`` is truthy (the default) then + the flow details container will be shallow copied (the flow details + contained there-in will **not** be copied). If ``retain_contents`` is + falsey then the copied logbook will have **no** contained flow + details (but it will have the rest of the local objects attributes + copied). + + :returns: a new logbook + :rtype: :py:class:`.LogBook` + """ clone = copy.copy(self) if not retain_contents: clone._flowdetails_by_id = {} @@ -176,15 +230,19 @@ class LogBook(object): class FlowDetail(object): - """A container of atom details, a name and associated metadata. + """A collection of atom details and associated metadata. Typically this class contains a collection of atom detail entries that represent the atoms in a given flow structure (along with any other needed metadata relevant to that flow). - The data contained within this class need *not* be backed by the backend - storage in real time. The data in this class will only be guaranteed to be - persisted when a save/update occurs via some backend connection. + The data contained within this class need **not** be persisted to the + backend storage in real time. The data in this class will only be + guaranteed to be persisted when a save (or update) occurs via some backend + connection. + + :ivar state: The state of the flow associated with this flow detail. + :ivar meta: A dictionary of meta-data associated with this flow detail. """ def __init__(self, name, uuid): self._uuid = uuid @@ -194,7 +252,18 @@ class FlowDetail(object): self.meta = {} def update(self, fd): - """Updates the objects state to be the same as the given one.""" + """Updates the objects state to be the same as the given one. + + This will assign the private and public attributes of the given + flow detail directly to this object (replacing any existing + attributes in this object; even if they are the **same**). + + NOTE(harlowja): If the provided object is this object itself + then **no** update is done. + + :returns: this flow detail + :rtype: :py:class:`.FlowDetail` + """ if fd is self: return self self._atomdetails_by_id = fd._atomdetails_by_id @@ -203,9 +272,21 @@ class FlowDetail(object): return self def merge(self, fd, deep_copy=False): - """Merges the current object state with the given ones state. + """Merges the current object state with the given one's state. - NOTE(harlowja): Does not merge the atom details contained in either. + If ``deep_copy`` is provided as truthy then the + local object will use ``copy.deepcopy`` to replace this objects + local attributes with the provided objects attributes (**only** if + there is a difference between this objects attributes and the + provided attributes). If ``deep_copy`` is falsey (the default) then a + reference copy will occur instead when a difference is detected. + + NOTE(harlowja): If the provided object is this object itself + then **no** merging is done. Also this does **not** merge the atom + details contained in either. + + :returns: this flow detail (freshly merged with the incoming object) + :rtype: :py:class:`.FlowDetail` """ if fd is self: return self @@ -218,7 +299,19 @@ class FlowDetail(object): return self def copy(self, retain_contents=True): - """Copies/clones this flow detail.""" + """Copies this flow detail. + + Creates a shallow copy of this flow detail. If this detail contains + flow details and ``retain_contents`` is truthy (the default) then + the atom details container will be shallow copied (the atom details + contained there-in will **not** be copied). If ``retain_contents`` is + falsey then the copied flow detail will have **no** contained atom + details (but it will have the rest of the local objects attributes + copied). + + :returns: a new flow detail + :rtype: :py:class:`.FlowDetail` + """ clone = copy.copy(self) if not retain_contents: clone._atomdetails_by_id = {} @@ -229,9 +322,12 @@ class FlowDetail(object): return clone def to_dict(self): - """Translates the internal state of this object to a dictionary. + """Translates the internal state of this object to a ``dict``. - NOTE(harlowja): Does not include the contained atom details. + NOTE(harlowja): The returned ``dict`` does **not** include any + contained atom details. + + :returns: this flow detail in ``dict`` form """ return { 'name': self.name, @@ -242,24 +338,46 @@ class FlowDetail(object): @classmethod def from_dict(cls, data): - """Translates the given data into an instance of this class.""" + """Translates the given ``dict`` into an instance of this class. + + NOTE(harlowja): the ``dict`` provided should come from a prior + call to :meth:`.to_dict`. + + :returns: a new flow detail + :rtype: :py:class:`.FlowDetail` + """ obj = cls(data['name'], data['uuid']) obj.state = data.get('state') obj.meta = _fix_meta(data) return obj def add(self, ad): + """Adds a new atom detail into this flow detail. + + NOTE(harlowja): if an existing atom detail exists with the same + uuid the existing one will be overwritten with the newly provided + one. + + Does not *guarantee* that the details will be immediately saved. + """ self._atomdetails_by_id[ad.uuid] = ad def find(self, ad_uuid): + """Locate the atom detail corresponding to the given uuid. + + :returns: the atom detail with that uuid + :rtype: :py:class:`.AtomDetail` (or ``None`` if not found) + """ return self._atomdetails_by_id.get(ad_uuid) @property def uuid(self): + """The unique identifer of this flow detail.""" return self._uuid @property def name(self): + """The name of this flow detail.""" return self._name def __iter__(self): @@ -272,39 +390,48 @@ class FlowDetail(object): @six.add_metaclass(abc.ABCMeta) class AtomDetail(object): - """A base container of atom specific runtime information and metadata. + """A collection of atom specific runtime information and metadata. - This is a base class that contains attributes that are used to connect - a atom to the persistence layer during, after, or before it is running - including any results it may have produced, any state that it may be - in (failed for example), any exception that occurred when running and any - associated stacktrace that may have occurring during that exception being - thrown and any other metadata that should be stored along-side the details - about the connected atom. + This is a base **abstract** class that contains attributes that are used + to connect a atom to the persistence layer before, during, or after it is + running. It includes any results it may have produced, any state that it + may be in (for example ``FAILURE``), any exception that occurred when + running, and any associated stacktrace that may have occurring during an + exception being thrown. It may also contain any other metadata that + should also be stored along-side the details about the connected atom. - The data contained within this class need *not* backed by the backend - storage in real time. The data in this class will only be guaranteed to be - persisted when a save/update occurs via some backend connection. + The data contained within this class need **not** be persisted to the + backend storage in real time. The data in this class will only be + guaranteed to be persisted when a save (or update) occurs via some backend + connection. + + :ivar state: The state of the atom associated with this atom detail. + :ivar intention: The execution strategy of the atom associated + with this atom detail (used by an engine/others to + determine if the associated atom needs to be + executed, reverted, retried and so-on). + :ivar meta: A dictionary of meta-data associated with this atom detail. + :ivar version: A version tuple or string that represents the + atom version this atom detail is associated with (typically + used for introspection and any data migration + strategies). + :ivar results: Any results the atom produced from either its + ``execute`` method or from other sources. + :ivar failure: If the atom failed (possibly due to its ``execute`` + method raising) this will be a + :py:class:`~taskflow.types.failure.Failure` object that + represents that failure (if there was no failure this + will be set to none). """ + def __init__(self, name, uuid): self._uuid = uuid self._name = name - # TODO(harlowja): decide if these should be passed in and therefore - # immutable or let them be assigned? - # - # The state the atom was last in. self.state = None - # The intention of action that would be applied to the atom. self.intention = states.EXECUTE - # The results it may have produced (useful for reverting). self.results = None - # An Failure object that holds exception the atom may have thrown - # (or part of it), useful for knowing what failed. self.failure = None self.meta = {} - # The version of the atom this atom details was associated with which - # is quite useful for determining what versions of atoms this detail - # information can be associated with. self.version = None @property @@ -318,7 +445,18 @@ class AtomDetail(object): return self.results def update(self, ad): - """Updates the objects state to be the same as the given one.""" + """Updates the object's state to be the same as the given one. + + This will assign the private and public attributes of the given + atom detail directly to this object (replacing any existing + attributes in this object; even if they are the **same**). + + NOTE(harlowja): If the provided object is this object itself + then **no** update is done. + + :returns: this atom detail + :rtype: :py:class:`.AtomDetail` + """ if ad is self: return self self.state = ad.state @@ -331,7 +469,24 @@ class AtomDetail(object): @abc.abstractmethod def merge(self, other, deep_copy=False): - """Merges the current object state with the given ones state.""" + """Merges the current object state with the given ones state. + + If ``deep_copy`` is provided as truthy then the + local object will use ``copy.deepcopy`` to replace this objects + local attributes with the provided objects attributes (**only** if + there is a difference between this objects attributes and the + provided attributes). If ``deep_copy`` is falsey (the default) then a + reference copy will occur instead when a difference is detected. + + NOTE(harlowja): If the provided object is this object itself + then **no** merging is done. Do note that **no** results are merged + in this method. That operation **must** to be the responsibilty of + subclasses to implement and override this abstract method + and provide that merging themselves as they see fit. + + :returns: this atom detail (freshly merged with the incoming object) + :rtype: :py:class:`.AtomDetail` + """ copy_fn = _copy_function(deep_copy) # NOTE(imelnikov): states and intentions are just strings, # so there is no need to copy them (strings are immutable in python). @@ -353,15 +508,15 @@ class AtomDetail(object): self.version = copy_fn(other.version) return self - @abc.abstractmethod - def to_dict(self): - """Translates the internal state of this object to a dictionary.""" - @abc.abstractmethod def put(self, state, result): """Puts a result (acquired in the given state) into this detail.""" - def _to_dict_shared(self): + def to_dict(self): + """Translates the internal state of this object to a ``dict``. + + :returns: this atom detail in ``dict`` form + """ if self.failure: failure = self.failure.to_dict() else: @@ -377,42 +532,75 @@ class AtomDetail(object): 'uuid': self.uuid, } - def _from_dict_shared(self, data): - self.state = data.get('state') - self.intention = data.get('intention') - self.results = data.get('results') - self.version = data.get('version') - self.meta = _fix_meta(data) + @classmethod + def from_dict(cls, data): + """Translates the given ``dict`` into an instance of this class. + + NOTE(harlowja): the ``dict`` provided should come from a prior + call to :meth:`.to_dict`. + + :returns: a new atom detail + :rtype: :py:class:`.AtomDetail` + """ + obj = cls(data['name'], data['uuid']) + obj.state = data.get('state') + obj.intention = data.get('intention') + obj.results = data.get('results') + obj.version = data.get('version') + obj.meta = _fix_meta(data) failure = data.get('failure') if failure: - self.failure = ft.Failure.from_dict(failure) + obj.failure = ft.Failure.from_dict(failure) + return obj @property def uuid(self): + """The unique identifer of this atom detail.""" return self._uuid @property def name(self): + """The name of this atom detail.""" return self._name @abc.abstractmethod def reset(self, state): - """Resets detail results and failures.""" + """Resets this atom detail and sets ``state`` attribute value.""" + + @abc.abstractmethod + def copy(self): + """Copies this atom detail.""" class TaskDetail(AtomDetail): - """This class represents a task detail for flow task object.""" + """A task detail (an atom detail typically associated with a |tt| atom). - def __init__(self, name, uuid): - super(TaskDetail, self).__init__(name, uuid) + .. |tt| replace:: :py:class:`~taskflow.task.BaseTask` + """ def reset(self, state): + """Resets this task detail and sets ``state`` attribute value. + + This sets any previously set ``results`` and ``failure`` attributes + back to ``None`` and sets the state to the provided one, as well as + setting this task details ``intention`` attribute to ``EXECUTE``. + """ self.results = None self.failure = None self.state = state self.intention = states.EXECUTE def put(self, state, result): + """Puts a result (acquired in the given state) into this detail. + + If the result is a :py:class:`~taskflow.types.failure.Failure` object + then the ``failure`` attribute will be set (and the ``results`` + attribute will be set to ``None``); if the result is not a + :py:class:`~taskflow.types.failure.Failure` object then the + ``results`` attribute will be set (and the ``failure`` attribute + will be set to ``None``). In either case the ``state`` + attribute will be set to the provided state. + """ self.state = state if _was_failure(state, result): self.failure = result @@ -421,38 +609,52 @@ class TaskDetail(AtomDetail): self.results = result self.failure = None - @classmethod - def from_dict(cls, data): - """Translates the given data into an instance of this class.""" - obj = cls(data['name'], data['uuid']) - obj._from_dict_shared(data) - return obj - - def to_dict(self): - """Translates the internal state of this object to a dictionary.""" - return self._to_dict_shared() - def merge(self, other, deep_copy=False): - """Merges the current object state with the given ones state.""" + """Merges the current task detail with the given one. + + NOTE(harlowja): This merge does **not** copy and replace + the ``results`` attribute if it differs. Instead the current + objects ``results`` attribute directly becomes (via assignment) the + other objects ``results`` attribute. Also note that if the provided + object is this object itself then **no** merging is done. + + See: https://bugs.launchpad.net/taskflow/+bug/1452978 for + what happens if this is copied at a deeper level (for example by + using ``copy.deepcopy`` or by using ``copy.copy``). + + :returns: this task detail (freshly merged with the incoming object) + :rtype: :py:class:`.TaskDetail` + """ if not isinstance(other, TaskDetail): raise exc.NotImplementedError("Can only merge with other" " task details") if other is self: return self super(TaskDetail, self).merge(other, deep_copy=deep_copy) - copy_fn = _copy_function(deep_copy) if self.results != other.results: - self.results = copy_fn(other.results) + self.results = other.results return self def copy(self): - """Copies/clones this task detail.""" + """Copies this task detail. + + Creates a shallow copy of this task detail (any meta-data and + version information that this object maintains is shallow + copied via ``copy.copy``). + + NOTE(harlowja): This copy does **not** perform ``copy.copy`` on + the ``results`` attribute of this object (before assigning to the + copy). Instead the current objects ``results`` attribute directly + becomes (via assignment) the copied objects ``results`` attribute. + + See: https://bugs.launchpad.net/taskflow/+bug/1452978 for + what happens if this is copied at a deeper level (for example by + using ``copy.deepcopy`` or by using ``copy.copy``). + + :returns: a new task detail + :rtype: :py:class:`.TaskDetail` + """ clone = copy.copy(self) - # Just directly assign to the clone (do **not** copy). - # - # See: https://bugs.launchpad.net/taskflow/+bug/1452978 for - # what happens if this is cloned/copied (even using copy.copy to - # try to do a shallow copy). clone.results = self.results if self.meta: clone.meta = self.meta.copy() @@ -462,19 +664,53 @@ class TaskDetail(AtomDetail): class RetryDetail(AtomDetail): - """This class represents a retry detail for retry controller object.""" + """A retry detail (an atom detail typically associated with a |rt| atom). + + .. |rt| replace:: :py:class:`~taskflow.retry.Retry` + """ + def __init__(self, name, uuid): super(RetryDetail, self).__init__(name, uuid) self.results = [] def reset(self, state): + """Resets this retry detail and sets ``state`` attribute value. + + This sets any previously added ``results`` back to an empty list + and resets the ``failure`` attribute back to ``None`` and sets the + state to the provided one, as well as setting this atom + details ``intention`` attribute to ``EXECUTE``. + """ self.results = [] self.failure = None self.state = state self.intention = states.EXECUTE def copy(self): - """Copies/clones this retry detail.""" + """Copies this retry detail. + + Creates a shallow copy of this retry detail (any meta-data and + version information that this object maintains is shallow + copied via ``copy.copy``). + + NOTE(harlowja): This copy does **not** copy + the incoming objects ``results`` attribute. Instead this + objects ``results`` attribute list is iterated over and a new list + is constructed with each ``(data, failures)`` element in that list + having its ``failures`` (a dictionary of each named + :py:class:`~taskflow.types.failure.Failure` object that + occured) copied but its ``data`` is left untouched. After + this is done that new list becomes (via assignment) the cloned + objects ``results`` attribute. + + See: https://bugs.launchpad.net/taskflow/+bug/1452978 for + what happens if the ``data`` in ``results`` is copied at a + deeper level (for example by using ``copy.deepcopy`` or by + using ``copy.copy``). + + :returns: a new retry detail + :rtype: :py:class:`.RetryDetail` + """ clone = copy.copy(self) results = [] # NOTE(imelnikov): we can't just deep copy Failures, as they @@ -493,6 +729,7 @@ class RetryDetail(AtomDetail): @property def last_results(self): + """The last result that was produced.""" try: return self.results[-1][0] except IndexError as e: @@ -500,12 +737,30 @@ class RetryDetail(AtomDetail): @property def last_failures(self): + """The last failure dictionary that was produced. + + NOTE(harlowja): This is **not** the same as the + local ``failure`` attribute as the obtained failure dictionary in + the ``results`` attribute (which is what this returns) is from + associated atom failures (which is different from the directly + related failure of the retry unit associated with this + atom detail). + """ try: return self.results[-1][1] except IndexError as e: raise exc.NotFound("Last failures not found", e) def put(self, state, result): + """Puts a result (acquired in the given state) into this detail. + + If the result is a :py:class:`~taskflow.types.failure.Failure` object + then the ``failure`` attribute will be set; if the result is not a + :py:class:`~taskflow.types.failure.Failure` object then the + ``results`` attribute will be appended to (and the ``failure`` + attribute will be set to ``None``). In either case the ``state`` + attribute will be set to the provided state. + """ # Do not clean retry history (only on reset does this happen). self.state = state if _was_failure(state, result): @@ -516,7 +771,7 @@ class RetryDetail(AtomDetail): @classmethod def from_dict(cls, data): - """Translates the given data into an instance of this class.""" + """Translates the given ``dict`` into an instance of this class.""" def decode_results(results): if not results: @@ -529,13 +784,12 @@ class RetryDetail(AtomDetail): new_results.append((data, new_failures)) return new_results - obj = cls(data['name'], data['uuid']) - obj._from_dict_shared(data) + obj = super(RetryDetail, cls).from_dict(data) obj.results = decode_results(obj.results) return obj def to_dict(self): - """Translates the internal state of this object to a dictionary.""" + """Translates the internal state of this object to a ``dict``.""" def encode_results(results): if not results: @@ -548,12 +802,33 @@ class RetryDetail(AtomDetail): new_results.append((data, new_failures)) return new_results - base = self._to_dict_shared() + base = super(RetryDetail, self).to_dict() base['results'] = encode_results(base.get('results')) return base def merge(self, other, deep_copy=False): - """Merges the current object state with the given ones state.""" + """Merges the current retry detail with the given one. + + NOTE(harlowja): This merge does **not** deep copy + the incoming objects ``results`` attribute (if it differs). Instead + the incoming objects ``results`` attribute list is **always** iterated + over and a new list is constructed with + each ``(data, failures)`` element in that list having + its ``failures`` (a dictionary of each named + :py:class:`~taskflow.types.failure.Failure` objects that + occurred) copied but its ``data`` is left untouched. After + this is done that new list becomes (via assignment) this + objects ``results`` attribute. Also note that if the provided object + is this object itself then **no** merging is done. + + See: https://bugs.launchpad.net/taskflow/+bug/1452978 for + what happens if the ``data`` in ``results`` is copied at a + deeper level (for example by using ``copy.deepcopy`` or by + using ``copy.copy``). + + :returns: this retry detail (freshly merged with the incoming object) + :rtype: :py:class:`.RetryDetail` + """ if not isinstance(other, RetryDetail): raise exc.NotImplementedError("Can only merge with other" " retry details") From 58f43d240cfcf664f5908d962968a383cd08e123 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Fri, 15 May 2015 09:47:14 -0700 Subject: [PATCH 179/246] Ensure path_based abstract base class is included in docs Adding this ensures that subclasses have there links setup to the right parent (vs having no link that works at all). Change-Id: I69409c09076f41fbcd4e38fe066e48e5b460e617 --- doc/source/persistence.rst | 1 + 1 file changed, 1 insertion(+) diff --git a/doc/source/persistence.rst b/doc/source/persistence.rst index 7db76341..70d6b7d7 100644 --- a/doc/source/persistence.rst +++ b/doc/source/persistence.rst @@ -283,6 +283,7 @@ Interfaces .. automodule:: taskflow.persistence.backends .. automodule:: taskflow.persistence.base +.. automodule:: taskflow.persistence.path_based Models ====== From 9c4520e989127fa89496ef1d8e9e71edb21ed187 Mon Sep 17 00:00:00 2001 From: Davanum Srinivas Date: Sat, 16 May 2015 16:03:31 -0400 Subject: [PATCH 180/246] Remove script already nuked from oslo-incubator Change-Id: I7953621f925fadfa18efb6c9b1bbba24d9ba496c --- openstack-common.conf | 3 --- 1 file changed, 3 deletions(-) diff --git a/openstack-common.conf b/openstack-common.conf index 127bc839..5eb87cc9 100644 --- a/openstack-common.conf +++ b/openstack-common.conf @@ -1,7 +1,4 @@ [DEFAULT] -# The list of modules to copy from oslo-incubator.git -script=tools/run_cross_tests.sh - # The base module to hold the copy of openstack.common base=taskflow From e183fc9dbc101d10491a8b04bb73cfb789b8826c Mon Sep 17 00:00:00 2001 From: Fredrik Bergroth Date: Wed, 20 May 2015 20:24:01 +0200 Subject: [PATCH 181/246] Fix updated_at column of sqlalchemy tables The column updated_at does not update its value after the refactoring made in 687ec913790653f79badc8f5d656c86792e94271. Closes-Bug: #1457309 Change-Id: Iff45f386b5dc8efc3fe82ca3b1e961a0c23d7ac7 --- taskflow/persistence/backends/sqlalchemy/tables.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/taskflow/persistence/backends/sqlalchemy/tables.py b/taskflow/persistence/backends/sqlalchemy/tables.py index 81f0b964..47306b9c 100644 --- a/taskflow/persistence/backends/sqlalchemy/tables.py +++ b/taskflow/persistence/backends/sqlalchemy/tables.py @@ -55,7 +55,7 @@ def fetch(metadata): Column('created_at', DateTime, default=timeutils.utcnow), Column('updated_at', DateTime, - default=timeutils.utcnow), + onupdate=timeutils.utcnow), Column('meta', Json), Column('name', String(length=NAME_LENGTH)), Column('uuid', String(length=UUID_LENGTH), @@ -65,7 +65,7 @@ def fetch(metadata): Column('created_at', DateTime, default=timeutils.utcnow), Column('updated_at', DateTime, - default=timeutils.utcnow), + onupdate=timeutils.utcnow), Column('parent_uuid', String(length=UUID_LENGTH), ForeignKey('logbooks.uuid', ondelete='CASCADE')), @@ -79,7 +79,7 @@ def fetch(metadata): Column('created_at', DateTime, default=timeutils.utcnow), Column('updated_at', DateTime, - default=timeutils.utcnow), + onupdate=timeutils.utcnow), Column('meta', Json), Column('parent_uuid', String(length=UUID_LENGTH), ForeignKey('flowdetails.uuid', From 712eed5b376804cd1792577d517f7f56a36b6ba4 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Thu, 21 May 2015 09:22:12 -0700 Subject: [PATCH 182/246] Only show state transitions to logging when in BLATHER mode To help reduce the amount of log pollution for various consuming projects, disable showing the runtime state transitions when in DEBUG mode and only show them when a lower level BLATHER mode is active instead. Change-Id: I218a380294f9b6e06cb0740bdb3a2a7ea0473299 --- taskflow/engines/action_engine/runner.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/taskflow/engines/action_engine/runner.py b/taskflow/engines/action_engine/runner.py index f1f880ce..d50de157 100644 --- a/taskflow/engines/action_engine/runner.py +++ b/taskflow/engines/action_engine/runner.py @@ -204,10 +204,10 @@ class _MachineBuilder(object): LOG.debug("Entering new state '%s' in response to event '%s'", new_state, event) - # NOTE(harlowja): when ran in debugging mode it is quite useful + # NOTE(harlowja): when ran in blather mode it is quite useful # to track the various state transitions as they happen... watchers = {} - if LOG.isEnabledFor(logging.DEBUG): + if LOG.isEnabledFor(logging.BLATHER): watchers['on_exit'] = on_exit watchers['on_enter'] = on_enter From b9da5e1675fd8e2a7db10704aa5d7303d0c9d309 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Sun, 24 May 2015 09:41:42 -0700 Subject: [PATCH 183/246] Remove 'pass' usage not needed These usages are not really needed and can just be eliminated since they offer no benefit (and coverage may classify them as dead/useless code). Change-Id: I951c4cb9216a19ce37dc8fc390ae905fd525a3c7 --- taskflow/jobs/base.py | 2 -- taskflow/persistence/base.py | 17 ----------------- 2 files changed, 19 deletions(-) diff --git a/taskflow/jobs/base.py b/taskflow/jobs/base.py index 1a5bcf2f..0f54b8f8 100644 --- a/taskflow/jobs/base.py +++ b/taskflow/jobs/base.py @@ -56,12 +56,10 @@ class Job(object): @abc.abstractproperty def last_modified(self): """The datetime the job was last modified.""" - pass @abc.abstractproperty def created_on(self): """The datetime the job was created on.""" - pass @abc.abstractproperty def board(self): diff --git a/taskflow/persistence/base.py b/taskflow/persistence/base.py index aebb7dd9..91fd095e 100644 --- a/taskflow/persistence/base.py +++ b/taskflow/persistence/base.py @@ -36,12 +36,10 @@ class Backend(object): @abc.abstractmethod def get_connection(self): """Return a Connection instance based on the configuration settings.""" - pass @abc.abstractmethod def close(self): """Closes any resources this backend has open.""" - pass @six.add_metaclass(abc.ABCMeta) @@ -51,22 +49,18 @@ class Connection(object): @abc.abstractproperty def backend(self): """Returns the backend this connection is associated with.""" - pass @abc.abstractmethod def close(self): """Closes any resources this connection has open.""" - pass @abc.abstractmethod def upgrade(self): """Migrate the persistence backend to the most recent version.""" - pass @abc.abstractmethod def clear_all(self): """Clear all entries from this backend.""" - pass @abc.abstractmethod def validate(self): @@ -76,7 +70,6 @@ class Connection(object): backend specific exception should be raised that will indicate why the failure occurred. """ - pass @abc.abstractmethod def update_atom_details(self, atom_detail): @@ -86,7 +79,6 @@ class Connection(object): been created by saving a flow details with the given atom detail inside of it. """ - pass @abc.abstractmethod def update_flow_details(self, flow_detail): @@ -96,47 +88,38 @@ class Connection(object): been created by saving a logbook with the given flow detail inside of it. """ - pass @abc.abstractmethod def save_logbook(self, book): """Saves a logbook, and all its contained information.""" - pass @abc.abstractmethod def destroy_logbook(self, book_uuid): """Deletes/destroys a logbook matching the given uuid.""" - pass @abc.abstractmethod def get_logbook(self, book_uuid, lazy=False): """Fetches a logbook object matching the given uuid.""" - pass @abc.abstractmethod def get_logbooks(self, lazy=False): """Return an iterable of logbook objects.""" - pass @abc.abstractmethod def get_flows_for_book(self, book_uuid): """Return an iterable of flowdetails for a given logbook uuid.""" - pass @abc.abstractmethod def get_flow_details(self, fd_uuid, lazy=False): """Fetches a flowdetails object matching the given uuid.""" - pass @abc.abstractmethod def get_atom_details(self, ad_uuid): """Fetches a atomdetails object matching the given uuid.""" - pass @abc.abstractmethod def get_atoms_for_flow(self, fd_uuid): """Return an iterable of atomdetails for a given flowdetails uuid.""" - pass def _format_atom(atom_detail): From c7e8c868cd85c878ab9896c868f9989adc7dfc09 Mon Sep 17 00:00:00 2001 From: Rick van de Loo Date: Wed, 6 May 2015 09:56:25 +0200 Subject: [PATCH 184/246] Allow same deps for requires and provides in task This change removes the DependencyFailure that is raised when a task requires the same dependency as it provides. Taskflow returns a frozenset([]) instead of a frozenset() when more than one value with the same name is in the store. This prevents the need for an inconvenient rename function when you want to update a store variable with a new value. Example case: class Inc(task.Task): def execute(self, a): return a + 1 class AwkwardRename(task.Task): def execute(self, b): return b store = { 'a': 1 } f = linear_flow.Flow('inc-flow') f.add( Inc('t1', provides='b', requires='a', ), AwkwardRename('t2', provides='a', requires='b')) e = engines.load(f, store=store) e.run() print e.storage.fetch('a', many_handler=lambda x: x[-1]) Now with ability to have the same provides as requires: class Inc(task.Task): def execute(self, a): return a + 1 store = { 'a': 1 } f = linear_flow.Flow('inc-flow') f.add( Inc('t3', provides='a', requires='a'), ) e = engines.load(f, store=store) e.run() print e.storage.fetch('a', many_handler=lambda x: x[-1]) Change-Id: I421e1ab33508c25baf78bd76df158bb6116d6fb0 --- taskflow/atom.py | 11 ---- taskflow/tests/unit/test_engines.py | 52 +++++++++++++++++++ taskflow/tests/unit/test_flow_dependencies.py | 16 +++--- 3 files changed, 61 insertions(+), 18 deletions(-) diff --git a/taskflow/atom.py b/taskflow/atom.py index ebc5bad3..82671df7 100644 --- a/taskflow/atom.py +++ b/taskflow/atom.py @@ -28,7 +28,6 @@ from oslo_utils import reflection import six from six.moves import zip as compat_zip -from taskflow import exceptions from taskflow.types import sets from taskflow.utils import misc @@ -162,10 +161,6 @@ class Atom(object): some action that furthers the overall flows progress. It usually also produces some of its own named output as a result of this process. - NOTE(harlowja): there can be no intersection between what this atom - requires and what it produces (since this would be an impossible - dependency to satisfy). - :param name: Meaningful name for this atom, should be something that is distinguishable and understandable for notification, debugging, storing and any other similar purposes. @@ -229,12 +224,6 @@ class Atom(object): inject_keys = frozenset(six.iterkeys(self.inject)) self.requires -= inject_keys self.optional -= inject_keys - out_of_order = self.provides.intersection(self.requires) - if out_of_order: - raise exceptions.DependencyFailure( - "Atom %(item)s provides %(oo)s that are required " - "by this atom" - % dict(item=self.name, oo=sorted(out_of_order))) @abc.abstractmethod def execute(self, *args, **kwargs): diff --git a/taskflow/tests/unit/test_engines.py b/taskflow/tests/unit/test_engines.py index c6a2af95..cec68f32 100644 --- a/taskflow/tests/unit/test_engines.py +++ b/taskflow/tests/unit/test_engines.py @@ -149,6 +149,53 @@ class EngineOptionalRequirementsTest(utils.EngineTestBase): self.assertEqual(result, {'a': 3, 'b': 7, 'result': 3000}) +class EngineMultipleResultsTest(utils.EngineTestBase): + def test_fetch_with_a_single_result(self): + flow = lf.Flow("flow") + flow.add(utils.TaskOneReturn(provides='x')) + + engine = self._make_engine(flow) + engine.run() + result = engine.storage.fetch('x') + self.assertEqual(result, 1) + + def test_fetch_with_two_results(self): + flow = lf.Flow("flow") + flow.add(utils.TaskOneReturn(provides='x')) + + engine = self._make_engine(flow, store={'x': 0}) + engine.run() + result = engine.storage.fetch('x') + self.assertEqual(result, 0) + + def test_fetch_all_with_a_single_result(self): + flow = lf.Flow("flow") + flow.add(utils.TaskOneReturn(provides='x')) + + engine = self._make_engine(flow) + engine.run() + result = engine.storage.fetch_all() + self.assertEqual(result, {'x': 1}) + + def test_fetch_all_with_two_results(self): + flow = lf.Flow("flow") + flow.add(utils.TaskOneReturn(provides='x')) + + engine = self._make_engine(flow, store={'x': 0}) + engine.run() + result = engine.storage.fetch_all() + self.assertEqual(result, {'x': [0, 1]}) + + def test_task_can_update_value(self): + flow = lf.Flow("flow") + flow.add(utils.TaskOneArgOneReturn(requires='x', provides='x')) + + engine = self._make_engine(flow, store={'x': 0}) + engine.run() + result = engine.storage.fetch_all() + self.assertEqual(result, {'x': [0, 1]}) + + class EngineLinearFlowTest(utils.EngineTestBase): def test_run_empty_flow(self): @@ -697,6 +744,7 @@ class EngineCheckingTaskTest(utils.EngineTestBase): class SerialEngineTest(EngineTaskTest, + EngineMultipleResultsTest, EngineLinearFlowTest, EngineParallelFlowTest, EngineLinearAndUnorderedExceptionsTest, @@ -723,6 +771,7 @@ class SerialEngineTest(EngineTaskTest, class ParallelEngineWithThreadsTest(EngineTaskTest, + EngineMultipleResultsTest, EngineLinearFlowTest, EngineParallelFlowTest, EngineLinearAndUnorderedExceptionsTest, @@ -761,6 +810,7 @@ class ParallelEngineWithThreadsTest(EngineTaskTest, @testtools.skipIf(not eu.EVENTLET_AVAILABLE, 'eventlet is not available') class ParallelEngineWithEventletTest(EngineTaskTest, + EngineMultipleResultsTest, EngineLinearFlowTest, EngineParallelFlowTest, EngineLinearAndUnorderedExceptionsTest, @@ -782,6 +832,7 @@ class ParallelEngineWithEventletTest(EngineTaskTest, class ParallelEngineWithProcessTest(EngineTaskTest, + EngineMultipleResultsTest, EngineLinearFlowTest, EngineParallelFlowTest, EngineLinearAndUnorderedExceptionsTest, @@ -808,6 +859,7 @@ class ParallelEngineWithProcessTest(EngineTaskTest, class WorkerBasedEngineTest(EngineTaskTest, + EngineMultipleResultsTest, EngineLinearFlowTest, EngineParallelFlowTest, EngineLinearAndUnorderedExceptionsTest, diff --git a/taskflow/tests/unit/test_flow_dependencies.py b/taskflow/tests/unit/test_flow_dependencies.py index 9627a696..bdb427d9 100644 --- a/taskflow/tests/unit/test_flow_dependencies.py +++ b/taskflow/tests/unit/test_flow_dependencies.py @@ -243,10 +243,10 @@ class FlowDependenciesTest(test.TestCase): requires=['a'])) def test_task_requires_and_provides_same_values(self): - self.assertRaises(exceptions.DependencyFailure, - utils.TaskOneArgOneReturn, - requires='a', - provides='a') + flow = lf.Flow('lf', utils.TaskOneArgOneReturn('rt', requires='x', + provides='x')) + self.assertEqual(flow.requires, set('x')) + self.assertEqual(flow.provides, set('x')) def test_retry_in_linear_flow_no_requirements_no_provides(self): flow = lf.Flow('lf', retry.AlwaysRevert('rt')) @@ -271,9 +271,11 @@ class FlowDependenciesTest(test.TestCase): self.assertEqual(flow.provides, set(['a', 'b'])) def test_retry_requires_and_provides_same_value(self): - self.assertRaises(exceptions.DependencyFailure, - retry.AlwaysRevert, - 'rt', requires=['x', 'y'], provides=['x', 'y']) + flow = lf.Flow('lf', retry.AlwaysRevert('rt', + requires=['x', 'y'], + provides=['x', 'y'])) + self.assertEqual(flow.requires, set(['x', 'y'])) + self.assertEqual(flow.provides, set(['x', 'y'])) def test_retry_in_unordered_flow_no_requirements_no_provides(self): flow = uf.Flow('uf', retry.AlwaysRevert('rt')) From 553b6b33751114eaee0dcda03a613481b936c282 Mon Sep 17 00:00:00 2001 From: OpenStack Proposal Bot Date: Wed, 27 May 2015 01:14:09 +0000 Subject: [PATCH 185/246] Updated from global requirements Change-Id: Ic4792c08942b51bf019f62238a609a9b66947983 --- test-requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test-requirements.txt b/test-requirements.txt index a6a7a19d..d4d99f68 100644 --- a/test-requirements.txt +++ b/test-requirements.txt @@ -13,7 +13,7 @@ kombu>=3.0.7 # Used for testing zookeeper & backends. zake>=0.1.6 # Apache-2.0 -kazoo>=1.3.1 +kazoo>=1.3.1,!=2.1 # Used for testing database persistence backends. # From 2e0eac09c4212ae6a25b2e02d7e14ee203fdd4eb Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Tue, 12 May 2015 12:13:00 -0700 Subject: [PATCH 186/246] Remove customized pyX.Y tox requirements Instead of having custom pyX.Y that test a specific set of requirements and similar with other tox envs, just enable more test-requirements directly, in a later change we will use environment markers to denote the differences. Also tweaks the jobboard example to bypass/hack a needed eventlet fix into it so that it actually runs without breaking/never finishing. A TODO note has been left to eventually (someday?) remove this hack when it is no longer needed. Part of fix for bug 1445827 Change-Id: I0fb669f0bdfbe4fe81d89234173f2c3ac4d06de6 --- .../jobboard_produce_consume_colors.py | 9 +++++++++ test-requirements.txt | 14 +++++--------- tox.ini | 18 ------------------ 3 files changed, 14 insertions(+), 27 deletions(-) diff --git a/taskflow/examples/jobboard_produce_consume_colors.py b/taskflow/examples/jobboard_produce_consume_colors.py index 6f586030..54983c47 100644 --- a/taskflow/examples/jobboard_produce_consume_colors.py +++ b/taskflow/examples/jobboard_produce_consume_colors.py @@ -30,6 +30,7 @@ top_dir = os.path.abspath(os.path.join(os.path.dirname(__file__), os.pardir)) sys.path.insert(0, top_dir) +import six from six.moves import range as compat_range from zake import fake_client @@ -150,6 +151,14 @@ def producer(ident, client): def main(): + if six.PY3: + # TODO(harlowja): Hack to make eventlet work right, remove when the + # following is fixed: https://github.com/eventlet/eventlet/issues/230 + from taskflow.utils import eventlet_utils as _eu # noqa + try: + import eventlet as _eventlet # noqa + except ImportError: + pass with contextlib.closing(fake_client.FakeClient()) as c: created = [] for i in compat_range(0, PRODUCERS): diff --git a/test-requirements.txt b/test-requirements.txt index d4d99f68..d2b601c6 100644 --- a/test-requirements.txt +++ b/test-requirements.txt @@ -16,17 +16,13 @@ zake>=0.1.6 # Apache-2.0 kazoo>=1.3.1,!=2.1 # Used for testing database persistence backends. -# -# NOTE(harlowja): SQLAlchemy isn't listed here currently but is -# listed in our tox.ini files so that we can test multiple varying SQLAlchemy -# versions to ensure a wider range of compatibility. -# -# Explict mysql drivers are also not listed here so that we can test against -# PyMySQL or MySQL-python depending on the python version the tests are being -# ran in (MySQL-python is currently preferred for 2.x environments, since -# it has been used in openstack for the longest). +SQLAlchemy>=0.9.7,<=0.9.99 alembic>=0.7.2 psycopg2 +PyMySQL>=0.6.2 + +# Used for making sure we still work with eventlet. +eventlet>=0.17.3 # Docs build jobs need these packages. sphinx>=1.1.2,!=1.2.0,!=1.3b1,<1.3 diff --git a/tox.ini b/tox.ini index 3581facc..535e611f 100644 --- a/tox.ini +++ b/tox.ini @@ -62,28 +62,10 @@ import_exceptions = six.moves taskflow.test.mock unittest.mock -# NOTE(imelnikov): pyXY envs are considered to be default, so they must have -# richest set of test requirements -[testenv:py26] -basepython = python2.6 -deps = {[testenv]deps} - MySQL-python - eventlet>=0.15.1 - SQLAlchemy>=0.7.8,<=0.8.99 - [testenv:py27] deps = {[testenv]deps} - MySQL-python - eventlet>=0.15.1 - SQLAlchemy>=0.7.8,<=0.9.99 doc8 commands = python setup.py testr --slowest --testr-args='{posargs}' sphinx-build -b doctest doc/source doc/build doc8 doc/source - -[testenv:py34] -deps = {[testenv]deps} - SQLAlchemy>=0.7.8,<=0.9.99 - PyMySQL>=0.6.2 - From 6b5b9dc185f8c60392a6f0ae1715b31b5a468c82 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Fri, 29 May 2015 15:57:49 -0700 Subject: [PATCH 187/246] Remove testing using persistence sqlalchemy backend with 'mysqldb' It appears no longer recommended to test against and/or use mysqldb as a backend that is tested against (since it lacks real support and development) and it is now better to just test against pymysql which is under development (and has better eventlet integration and py3.x support). Change-Id: I08f7ea67fbeb2e8b2976aef442ed442e33c77e09 --- .../unit/persistence/test_sql_persistence.py | 111 ++++++------------ 1 file changed, 37 insertions(+), 74 deletions(-) diff --git a/taskflow/tests/unit/persistence/test_sql_persistence.py b/taskflow/tests/unit/persistence/test_sql_persistence.py index 8489160d..8a8e22c1 100644 --- a/taskflow/tests/unit/persistence/test_sql_persistence.py +++ b/taskflow/tests/unit/persistence/test_sql_persistence.py @@ -40,18 +40,10 @@ PASSWD = "openstack_citest" DATABASE = "tftest_" + ''.join(random.choice('0123456789') for _ in range(12)) -try: - from taskflow.persistence.backends import impl_sqlalchemy - - import sqlalchemy as sa - SQLALCHEMY_AVAILABLE = True -except Exception: - SQLALCHEMY_AVAILABLE = False - -# Testing will try to run against these two mysql library variants. -MYSQL_VARIANTS = ('mysqldb', 'pymysql') +import sqlalchemy as sa from taskflow.persistence import backends +from taskflow.persistence.backends import impl_sqlalchemy from taskflow import test from taskflow.tests.unit.persistence import base @@ -64,7 +56,7 @@ def _get_connect_string(backend, user, passwd, database=None, variant=None): backend = "postgresql+%s" % (variant) elif backend == "mysql": if not variant: - variant = 'mysqldb' + variant = 'pymysql' backend = "mysql+%s" % (variant) else: raise Exception("Unrecognized backend: '%s'" % backend) @@ -74,30 +66,24 @@ def _get_connect_string(backend, user, passwd, database=None, variant=None): def _mysql_exists(): - if not SQLALCHEMY_AVAILABLE: - return False - for variant in MYSQL_VARIANTS: - engine = None - try: - db_uri = _get_connect_string('mysql', USER, PASSWD, - variant=variant) - engine = sa.create_engine(db_uri) - with contextlib.closing(engine.connect()): - return True - except Exception: - pass - finally: - if engine is not None: - try: - engine.dispose() - except Exception: - pass + engine = None + try: + db_uri = _get_connect_string('mysql', USER, PASSWD) + engine = sa.create_engine(db_uri) + with contextlib.closing(engine.connect()): + return True + except Exception: + pass + finally: + if engine is not None: + try: + engine.dispose() + except Exception: + pass return False def _postgres_exists(): - if not SQLALCHEMY_AVAILABLE: - return False engine = None try: db_uri = _get_connect_string('postgres', USER, PASSWD, 'postgres') @@ -114,7 +100,6 @@ def _postgres_exists(): pass -@testtools.skipIf(not SQLALCHEMY_AVAILABLE, 'sqlalchemy is not available') class SqlitePersistenceTest(test.TestCase, base.PersistenceTestMixin): """Inherits from the base test and sets up a sqlite temporary db.""" def _get_connection(self): @@ -185,43 +170,26 @@ class BackendPersistenceTestMixin(base.PersistenceTestMixin): " testing being skipped due to: %s" % (e)) -@testtools.skipIf(not SQLALCHEMY_AVAILABLE, 'sqlalchemy is not available') @testtools.skipIf(not _mysql_exists(), 'mysql is not available') class MysqlPersistenceTest(BackendPersistenceTestMixin, test.TestCase): - def __init__(self, *args, **kwargs): - test.TestCase.__init__(self, *args, **kwargs) - def _init_db(self): - working_variant = None - for variant in MYSQL_VARIANTS: - engine = None - try: - db_uri = _get_connect_string('mysql', USER, PASSWD, - variant=variant) - engine = sa.create_engine(db_uri) - with contextlib.closing(engine.connect()) as conn: - conn.execute("CREATE DATABASE %s" % DATABASE) - working_variant = variant - except Exception: - pass - finally: - if engine is not None: - try: - engine.dispose() - except Exception: - pass - if working_variant: - break - if not working_variant: - variants = ", ".join(MYSQL_VARIANTS) - raise Exception("Failed to initialize MySQL db." - " Tried these variants: %s; MySQL testing" - " being skipped" % (variants)) - else: - return _get_connect_string('mysql', USER, PASSWD, - database=DATABASE, - variant=working_variant) + engine = None + try: + db_uri = _get_connect_string('mysql', USER, PASSWD) + engine = sa.create_engine(db_uri) + with contextlib.closing(engine.connect()) as conn: + conn.execute("CREATE DATABASE %s" % DATABASE) + except Exception as e: + raise Exception('Failed to initialize MySQL db: %s' % (e)) + finally: + if engine is not None: + try: + engine.dispose() + except Exception: + pass + return _get_connect_string('mysql', USER, PASSWD, + database=DATABASE) def _remove_db(self): engine = None @@ -239,13 +207,9 @@ class MysqlPersistenceTest(BackendPersistenceTestMixin, test.TestCase): pass -@testtools.skipIf(not SQLALCHEMY_AVAILABLE, 'sqlalchemy is not available') @testtools.skipIf(not _postgres_exists(), 'postgres is not available') class PostgresPersistenceTest(BackendPersistenceTestMixin, test.TestCase): - def __init__(self, *args, **kwargs): - test.TestCase.__init__(self, *args, **kwargs) - def _init_db(self): engine = None try: @@ -293,7 +257,6 @@ class PostgresPersistenceTest(BackendPersistenceTestMixin, test.TestCase): pass -@testtools.skipIf(not SQLALCHEMY_AVAILABLE, 'sqlalchemy is not available') class SQLBackendFetchingTest(test.TestCase): def test_sqlite_persistence_entry_point(self): @@ -301,16 +264,16 @@ class SQLBackendFetchingTest(test.TestCase): with contextlib.closing(backends.fetch(conf)) as be: self.assertIsInstance(be, impl_sqlalchemy.SQLAlchemyBackend) - @testtools.skipIf(not _postgres_exists(), 'postgres is not available') + @testtools.skipIf(not _mysql_exists(), 'mysql is not available') def test_mysql_persistence_entry_point(self): - uri = "mysql://%s:%s@localhost/%s" % (USER, PASSWD, DATABASE) + uri = _get_connect_string('mysql', USER, PASSWD, database=DATABASE) conf = {'connection': uri} with contextlib.closing(backends.fetch(conf)) as be: self.assertIsInstance(be, impl_sqlalchemy.SQLAlchemyBackend) - @testtools.skipIf(not _mysql_exists(), 'mysql is not available') + @testtools.skipIf(not _postgres_exists(), 'postgres is not available') def test_postgres_persistence_entry_point(self): - uri = "postgresql://%s:%s@localhost/%s" % (USER, PASSWD, DATABASE) + uri = _get_connect_string('postgres', USER, PASSWD, database=DATABASE) conf = {'connection': uri} with contextlib.closing(backends.fetch(conf)) as be: self.assertIsInstance(be, impl_sqlalchemy.SQLAlchemyBackend) From 16a2c791f0c3fdca8250a0e133bb1a8923d66d3c Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Thu, 28 May 2015 16:17:12 -0700 Subject: [PATCH 188/246] Add a test that checks for task result visibility Now that a task can provide the same thing that it requires, having a nice little test that ensures what is gotten is expected in a nice manner should be done to make sure what a task will be getting for its arguments respect the scoping that is defined by the flow ordering. Change-Id: I1cdc5c3bd3f56b39f20f28f51c36017405150cea --- taskflow/tests/unit/test_engines.py | 54 +++++++++++++++++++ .../tests/unit/worker_based/test_worker.py | 2 +- taskflow/tests/utils.py | 7 +++ 3 files changed, 62 insertions(+), 1 deletion(-) diff --git a/taskflow/tests/unit/test_engines.py b/taskflow/tests/unit/test_engines.py index cec68f32..4e38dfa5 100644 --- a/taskflow/tests/unit/test_engines.py +++ b/taskflow/tests/unit/test_engines.py @@ -159,6 +159,60 @@ class EngineMultipleResultsTest(utils.EngineTestBase): result = engine.storage.fetch('x') self.assertEqual(result, 1) + def test_many_results_visible_to(self): + flow = lf.Flow("flow") + flow.add(utils.AddOneSameProvidesRequires( + 'a', rebind={'value': 'source'})) + flow.add(utils.AddOneSameProvidesRequires('b')) + flow.add(utils.AddOneSameProvidesRequires('c')) + engine = self._make_engine(flow, store={'source': 0}) + engine.run() + + # Check what each task in the prior should be seeing... + atoms = list(flow) + a = atoms[0] + a_kwargs = engine.storage.fetch_mapped_args(a.rebind, + atom_name='a') + self.assertEqual({'value': 0}, a_kwargs) + + b = atoms[1] + b_kwargs = engine.storage.fetch_mapped_args(b.rebind, + atom_name='b') + self.assertEqual({'value': 1}, b_kwargs) + + c = atoms[2] + c_kwargs = engine.storage.fetch_mapped_args(c.rebind, + atom_name='c') + self.assertEqual({'value': 2}, c_kwargs) + + def test_many_results_storage_provided_visible_to(self): + # This works as expected due to docs listed at + # + # http://docs.openstack.org/developer/taskflow/engines.html#scoping + flow = lf.Flow("flow") + flow.add(utils.AddOneSameProvidesRequires('a')) + flow.add(utils.AddOneSameProvidesRequires('b')) + flow.add(utils.AddOneSameProvidesRequires('c')) + engine = self._make_engine(flow, store={'value': 0}) + engine.run() + + # Check what each task in the prior should be seeing... + atoms = list(flow) + a = atoms[0] + a_kwargs = engine.storage.fetch_mapped_args(a.rebind, + atom_name='a') + self.assertEqual({'value': 0}, a_kwargs) + + b = atoms[1] + b_kwargs = engine.storage.fetch_mapped_args(b.rebind, + atom_name='b') + self.assertEqual({'value': 0}, b_kwargs) + + c = atoms[2] + c_kwargs = engine.storage.fetch_mapped_args(c.rebind, + atom_name='c') + self.assertEqual({'value': 0}, c_kwargs) + def test_fetch_with_two_results(self): flow = lf.Flow("flow") flow.add(utils.TaskOneReturn(provides='x')) diff --git a/taskflow/tests/unit/worker_based/test_worker.py b/taskflow/tests/unit/worker_based/test_worker.py index a475c51d..3acf245b 100644 --- a/taskflow/tests/unit/worker_based/test_worker.py +++ b/taskflow/tests/unit/worker_based/test_worker.py @@ -33,7 +33,7 @@ class TestWorker(test.MockTestCase): self.broker_url = 'test-url' self.exchange = 'test-exchange' self.topic = 'test-topic' - self.endpoint_count = 24 + self.endpoint_count = 25 # patch classes self.executor_mock, self.executor_inst_mock = self.patchClass( diff --git a/taskflow/tests/utils.py b/taskflow/tests/utils.py index 031dd706..43f208bb 100644 --- a/taskflow/tests/utils.py +++ b/taskflow/tests/utils.py @@ -89,6 +89,13 @@ class DummyTask(task.Task): pass +class AddOneSameProvidesRequires(task.Task): + default_provides = 'value' + + def execute(self, value): + return value + 1 + + class AddOne(task.Task): default_provides = 'result' From b8da7f70a164a5c78d5451c5bd670e3ae0208252 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Fri, 29 May 2015 17:44:14 -0700 Subject: [PATCH 189/246] Remove a couple more useless 'pass' keywords found Change-Id: I01e444a766836ef6867e3312c18043cae6ac6c4b --- taskflow/engines/action_engine/executor.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/taskflow/engines/action_engine/executor.py b/taskflow/engines/action_engine/executor.py index bd222a55..c630c161 100644 --- a/taskflow/engines/action_engine/executor.py +++ b/taskflow/engines/action_engine/executor.py @@ -347,11 +347,9 @@ class TaskExecutor(object): def start(self): """Prepare to execute tasks.""" - pass def stop(self): """Finalize task executor.""" - pass class SerialTaskExecutor(TaskExecutor): From ae6e48fed6ece121ef9294a2250428201fd7b654 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Sat, 30 May 2015 12:41:11 -0700 Subject: [PATCH 190/246] Use shared '_check' function to check engine stages Change-Id: I48e5efdb28ee834666ec1eca18f35f245de340d3 --- taskflow/engines/action_engine/engine.py | 23 +++++++++++++---------- 1 file changed, 13 insertions(+), 10 deletions(-) diff --git a/taskflow/engines/action_engine/engine.py b/taskflow/engines/action_engine/engine.py index 12f5feac..3dda9125 100644 --- a/taskflow/engines/action_engine/engine.py +++ b/taskflow/engines/action_engine/engine.py @@ -82,10 +82,18 @@ class ActionEngine(base.Engine): self._state_lock = threading.RLock() self._storage_ensured = False + def _check(self, name, check_compiled, check_storage_ensured): + """Check (and raise) if the engine has not reached a certain stage.""" + if check_compiled and not self._compiled: + raise exc.InvalidState("Can not %s an engine which" + " has not been compiled" % name) + if check_storage_ensured and not self._storage_ensured: + raise exc.InvalidState("Can not %s an engine" + " which has not has its storage" + " populated" % name) + def suspend(self): - if not self._compiled: - raise exc.InvalidState("Can not suspend an engine" - " which has not been compiled") + self._check('suspend', True, False) self._change_state(states.SUSPENDING) @property @@ -216,10 +224,7 @@ class ActionEngine(base.Engine): @lock_utils.locked def validate(self): - if not self._storage_ensured: - raise exc.InvalidState("Can not validate an engine" - " which has not has its storage" - " populated") + self._check('validate', True, True) # At this point we can check to ensure all dependencies are either # flow/task provided or storage provided, if there are still missing # dependencies then this flow will fail at runtime (which we can avoid @@ -263,9 +268,7 @@ class ActionEngine(base.Engine): @lock_utils.locked def prepare(self): - if not self._compiled: - raise exc.InvalidState("Can not prepare an engine" - " which has not been compiled") + self._check('prepare', True, False) if not self._storage_ensured: # Set our own state to resuming -> (ensure atoms exist # in storage) -> suspended in the storage unit and notify any From 1e6b991b23c5e9f14562491915f0fa09b3ce8533 Mon Sep 17 00:00:00 2001 From: OpenStack Proposal Bot Date: Tue, 2 Jun 2015 16:12:28 +0000 Subject: [PATCH 191/246] Updated from global requirements Change-Id: I61949b8a4d566b135a4dab23f7a448dae32398e5 --- test-requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test-requirements.txt b/test-requirements.txt index d2b601c6..16fc8b2b 100644 --- a/test-requirements.txt +++ b/test-requirements.txt @@ -19,7 +19,7 @@ kazoo>=1.3.1,!=2.1 SQLAlchemy>=0.9.7,<=0.9.99 alembic>=0.7.2 psycopg2 -PyMySQL>=0.6.2 +PyMySQL>=0.6.2 # MIT License # Used for making sure we still work with eventlet. eventlet>=0.17.3 From c3afeb52801c40f96ebb6bb358efa0b16fe15219 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Sat, 23 May 2015 15:07:02 -0700 Subject: [PATCH 192/246] Replace lock_utils lock(s) with fasteners package The usage of this package allows us to get rid of the reader/writer lock (since that package provides that) as well as the interprocess lock as all of these are now provided by that package instead. Change-Id: I87990b46d397f6df779de7028bcc40e28621e1ba --- requirements.txt | 3 + taskflow/persistence/backends/impl_dir.py | 6 +- taskflow/persistence/backends/impl_memory.py | 4 +- taskflow/storage.py | 54 +-- taskflow/tests/unit/test_utils_lock_utils.py | 353 ------------------- taskflow/utils/lock_utils.py | 310 ---------------- 6 files changed, 35 insertions(+), 695 deletions(-) diff --git a/requirements.txt b/requirements.txt index b0e0eb22..0b45b8c5 100644 --- a/requirements.txt +++ b/requirements.txt @@ -16,6 +16,9 @@ six>=1.9.0 # Enum library made for <= python 3.3 enum34 +# For reader/writer + interprocess locks. +fasteners>=0.5 # Apache-2.0 + # Very nice graph library networkx>=1.8 diff --git a/taskflow/persistence/backends/impl_dir.py b/taskflow/persistence/backends/impl_dir.py index e71d5b9c..940b9c41 100644 --- a/taskflow/persistence/backends/impl_dir.py +++ b/taskflow/persistence/backends/impl_dir.py @@ -20,11 +20,11 @@ import errno import os import shutil +import fasteners from oslo_serialization import jsonutils from taskflow import exceptions as exc from taskflow.persistence import path_based -from taskflow.utils import lock_utils from taskflow.utils import misc @@ -64,7 +64,7 @@ class DirBackend(path_based.PathBasedBackend): if not self._path: raise ValueError("Empty path is disallowed") self._path = os.path.abspath(self._path) - self.lock = lock_utils.ReaderWriterLock() + self.lock = fasteners.ReaderWriterLock() def get_connection(self): return Connection(self) @@ -97,7 +97,7 @@ class Connection(path_based.PathBasedConnection): @contextlib.contextmanager def _path_lock(self, path): lockfile = self._join_path(path, 'lock') - with lock_utils.InterProcessLock(lockfile) as lock: + with fasteners.InterProcessLock(lockfile) as lock: with _storagefailure_wrapper(): yield lock diff --git a/taskflow/persistence/backends/impl_memory.py b/taskflow/persistence/backends/impl_memory.py index e614247d..008a9134 100644 --- a/taskflow/persistence/backends/impl_memory.py +++ b/taskflow/persistence/backends/impl_memory.py @@ -20,12 +20,12 @@ import copy import itertools import posixpath as pp +import fasteners import six from taskflow import exceptions as exc from taskflow.persistence import path_based from taskflow.types import tree -from taskflow.utils import lock_utils class FakeInode(tree.Node): @@ -261,7 +261,7 @@ class MemoryBackend(path_based.PathBasedBackend): self._path = pp.sep self.memory = FakeFilesystem(deep_copy=self._conf.get('deep_copy', True)) - self.lock = lock_utils.ReaderWriterLock() + self.lock = fasteners.ReaderWriterLock() def get_connection(self): return Connection(self) diff --git a/taskflow/storage.py b/taskflow/storage.py index 83311e92..18957b77 100644 --- a/taskflow/storage.py +++ b/taskflow/storage.py @@ -16,6 +16,7 @@ import contextlib +import fasteners from oslo_utils import reflection from oslo_utils import uuidutils import six @@ -28,7 +29,6 @@ from taskflow import retry from taskflow import states from taskflow import task from taskflow.types import failure -from taskflow.utils import lock_utils from taskflow.utils import misc LOG = logging.getLogger(__name__) @@ -150,7 +150,7 @@ class Storage(object): self._flowdetail = flow_detail self._transients = {} self._injected_args = {} - self._lock = lock_utils.ReaderWriterLock() + self._lock = fasteners.ReaderWriterLock() self._ensure_matchers = [ ((task.BaseTask,), self._ensure_task), ((retry.Retry,), self._ensure_retry), @@ -334,46 +334,46 @@ class Storage(object): original_atom_detail.update(conn.update_atom_details(atom_detail)) return original_atom_detail - @lock_utils.read_locked + @fasteners.read_locked def get_atom_uuid(self, atom_name): """Gets an atoms uuid given a atoms name.""" source, _clone = self._atomdetail_by_name(atom_name) return source.uuid - @lock_utils.write_locked + @fasteners.write_locked def set_atom_state(self, atom_name, state): """Sets an atoms state.""" source, clone = self._atomdetail_by_name(atom_name, clone=True) clone.state = state self._with_connection(self._save_atom_detail, source, clone) - @lock_utils.read_locked + @fasteners.read_locked def get_atom_state(self, atom_name): """Gets the state of an atom given an atoms name.""" source, _clone = self._atomdetail_by_name(atom_name) return source.state - @lock_utils.write_locked + @fasteners.write_locked def set_atom_intention(self, atom_name, intention): """Sets the intention of an atom given an atoms name.""" source, clone = self._atomdetail_by_name(atom_name, clone=True) clone.intention = intention self._with_connection(self._save_atom_detail, source, clone) - @lock_utils.read_locked + @fasteners.read_locked def get_atom_intention(self, atom_name): """Gets the intention of an atom given an atoms name.""" source, _clone = self._atomdetail_by_name(atom_name) return source.intention - @lock_utils.read_locked + @fasteners.read_locked def get_atoms_states(self, atom_names): """Gets all atoms states given a set of names.""" return dict((name, (self.get_atom_state(name), self.get_atom_intention(name))) for name in atom_names) - @lock_utils.write_locked + @fasteners.write_locked def _update_atom_metadata(self, atom_name, update_with, expected_type=None): source, clone = self._atomdetail_by_name(atom_name, @@ -417,7 +417,7 @@ class Storage(object): self._update_atom_metadata(task_name, update_with, expected_type=logbook.TaskDetail) - @lock_utils.read_locked + @fasteners.read_locked def get_task_progress(self, task_name): """Get the progress of a task given a tasks name. @@ -431,7 +431,7 @@ class Storage(object): except KeyError: return 0.0 - @lock_utils.read_locked + @fasteners.read_locked def get_task_progress_details(self, task_name): """Get the progress details of a task given a tasks name. @@ -463,7 +463,7 @@ class Storage(object): LOG.warning("Atom %s did not supply result " "with index %r (name %s)", atom_name, index, name) - @lock_utils.write_locked + @fasteners.write_locked def save(self, atom_name, data, state=states.SUCCESS): """Put result for atom with id 'uuid' to storage.""" source, clone = self._atomdetail_by_name(atom_name, clone=True) @@ -477,7 +477,7 @@ class Storage(object): else: self._check_all_results_provided(result.name, data) - @lock_utils.write_locked + @fasteners.write_locked def save_retry_failure(self, retry_name, failed_atom_name, failure): """Save subflow failure to retry controller history.""" source, clone = self._atomdetail_by_name( @@ -493,7 +493,7 @@ class Storage(object): failures[failed_atom_name] = failure self._with_connection(self._save_atom_detail, source, clone) - @lock_utils.write_locked + @fasteners.write_locked def cleanup_retry_history(self, retry_name, state): """Cleanup history of retry atom with given name.""" source, clone = self._atomdetail_by_name( @@ -502,7 +502,7 @@ class Storage(object): clone.results = [] self._with_connection(self._save_atom_detail, source, clone) - @lock_utils.read_locked + @fasteners.read_locked def _get(self, atom_name, only_last=False): source, _clone = self._atomdetail_by_name(atom_name) if source.failure is not None: @@ -525,7 +525,7 @@ class Storage(object): """Gets the results for an atom with a given name from storage.""" return self._get(atom_name) - @lock_utils.read_locked + @fasteners.read_locked def get_failures(self): """Get list of failures that happened with this flow. @@ -537,7 +537,7 @@ class Storage(object): """Returns True if there are failed tasks in the storage.""" return bool(self._failures) - @lock_utils.write_locked + @fasteners.write_locked def reset(self, atom_name, state=states.PENDING): """Reset atom with given name (if the atom is not in a given state).""" if atom_name == self.injector_name: @@ -605,7 +605,7 @@ class Storage(object): else: save_persistent() - @lock_utils.write_locked + @fasteners.write_locked def inject(self, pairs, transient=False): """Add values into storage. @@ -682,7 +682,7 @@ class Storage(object): if provider not in entries: entries.append(provider) - @lock_utils.read_locked + @fasteners.read_locked def fetch(self, name, many_handler=None): """Fetch a named result.""" def _many_handler(values): @@ -717,7 +717,7 @@ class Storage(object): else: return many_handler(values) - @lock_utils.read_locked + @fasteners.read_locked def fetch_unsatisfied_args(self, atom_name, args_mapping, scope_walker=None, optional_args=None): """Fetch unsatisfied atom arguments using an atoms argument mapping. @@ -803,7 +803,7 @@ class Storage(object): missing.discard(bound_name) return missing - @lock_utils.read_locked + @fasteners.read_locked def fetch_all(self, many_handler=None): """Fetch all named results known so far.""" def _many_handler(values): @@ -820,7 +820,7 @@ class Storage(object): pass return results - @lock_utils.read_locked + @fasteners.read_locked def fetch_mapped_args(self, args_mapping, atom_name=None, scope_walker=None, optional_args=None): @@ -934,14 +934,14 @@ class Storage(object): bound_name, name, value, provider) return mapped_args - @lock_utils.write_locked + @fasteners.write_locked def set_flow_state(self, state): """Set flow details state and save it.""" source, clone = self._fetch_flowdetail(clone=True) clone.state = state self._with_connection(self._save_flow_detail, source, clone) - @lock_utils.write_locked + @fasteners.write_locked def update_flow_metadata(self, update_with): """Update flowdetails metadata and save it.""" if update_with: @@ -949,7 +949,7 @@ class Storage(object): clone.meta.update(update_with) self._with_connection(self._save_flow_detail, source, clone) - @lock_utils.read_locked + @fasteners.read_locked def get_flow_state(self): """Get state from flow details.""" source = self._flowdetail @@ -971,14 +971,14 @@ class Storage(object): failure = ad.failure return retry.History(ad.results, failure=failure) - @lock_utils.read_locked + @fasteners.read_locked def get_retry_history(self, retry_name): """Fetch a single retrys history.""" source, _clone = self._atomdetail_by_name( retry_name, expected_type=logbook.RetryDetail) return self._translate_into_history(source) - @lock_utils.read_locked + @fasteners.read_locked def get_retry_histories(self): """Fetch all retrys histories.""" histories = [] diff --git a/taskflow/tests/unit/test_utils_lock_utils.py b/taskflow/tests/unit/test_utils_lock_utils.py index 3d40ed24..0c8213e9 100644 --- a/taskflow/tests/unit/test_utils_lock_utils.py +++ b/taskflow/tests/unit/test_utils_lock_utils.py @@ -15,16 +15,11 @@ # under the License. import collections -import random import threading import time -from concurrent import futures - from taskflow import test from taskflow.test import mock -from taskflow.tests import utils as test_utils -from taskflow.types import timing from taskflow.utils import lock_utils from taskflow.utils import misc from taskflow.utils import threading_utils @@ -50,51 +45,6 @@ def _find_overlaps(times, start, end): return overlaps -def _spawn_variation(readers, writers, max_workers=None): - start_stops = collections.deque() - lock = lock_utils.ReaderWriterLock() - - def read_func(ident): - with lock.read_lock(): - # TODO(harlowja): sometime in the future use a monotonic clock here - # to avoid problems that can be caused by ntpd resyncing the clock - # while we are actively running. - enter_time = now() - time.sleep(WORK_TIMES[ident % len(WORK_TIMES)]) - exit_time = now() - start_stops.append((lock.READER, enter_time, exit_time)) - time.sleep(NAPPY_TIME) - - def write_func(ident): - with lock.write_lock(): - enter_time = now() - time.sleep(WORK_TIMES[ident % len(WORK_TIMES)]) - exit_time = now() - start_stops.append((lock.WRITER, enter_time, exit_time)) - time.sleep(NAPPY_TIME) - - if max_workers is None: - max_workers = max(0, readers) + max(0, writers) - if max_workers > 0: - with futures.ThreadPoolExecutor(max_workers=max_workers) as e: - count = 0 - for _i in range(0, readers): - e.submit(read_func, count) - count += 1 - for _i in range(0, writers): - e.submit(write_func, count) - count += 1 - - writer_times = [] - reader_times = [] - for (lock_type, start, stop) in list(start_stops): - if lock_type == lock.WRITER: - writer_times.append((start, stop)) - else: - reader_times.append((start, stop)) - return (writer_times, reader_times) - - class MultilockTest(test.TestCase): THREAD_COUNT = 20 @@ -329,306 +279,3 @@ class MultilockTest(test.TestCase): lock2 = threading.Lock() n_lock = lock_utils.MultiLock((lock1, lock2)) self.assertRaises(threading.ThreadError, n_lock.release) - - -class ReadWriteLockTest(test.TestCase): - THREAD_COUNT = 20 - - def test_no_double_writers(self): - lock = lock_utils.ReaderWriterLock() - watch = timing.StopWatch(duration=5) - watch.start() - dups = collections.deque() - active = collections.deque() - - def acquire_check(me): - with lock.write_lock(): - if len(active) >= 1: - dups.append(me) - dups.extend(active) - active.append(me) - try: - time.sleep(random.random() / 100) - finally: - active.remove(me) - - def run(): - me = threading.current_thread() - while not watch.expired(): - acquire_check(me) - - threads = [] - for i in range(0, self.THREAD_COUNT): - t = threading_utils.daemon_thread(run) - threads.append(t) - t.start() - while threads: - t = threads.pop() - t.join() - - self.assertEqual([], list(dups)) - self.assertEqual([], list(active)) - - def test_no_concurrent_readers_writers(self): - lock = lock_utils.ReaderWriterLock() - watch = timing.StopWatch(duration=5) - watch.start() - dups = collections.deque() - active = collections.deque() - - def acquire_check(me, reader): - if reader: - lock_func = lock.read_lock - else: - lock_func = lock.write_lock - with lock_func(): - if not reader: - # There should be no-one else currently active, if there - # is ensure we capture them so that we can later blow-up - # the test. - if len(active) >= 1: - dups.append(me) - dups.extend(active) - active.append(me) - try: - time.sleep(random.random() / 100) - finally: - active.remove(me) - - def run(): - me = threading.current_thread() - while not watch.expired(): - acquire_check(me, random.choice([True, False])) - - threads = [] - for i in range(0, self.THREAD_COUNT): - t = threading_utils.daemon_thread(run) - threads.append(t) - t.start() - while threads: - t = threads.pop() - t.join() - - self.assertEqual([], list(dups)) - self.assertEqual([], list(active)) - - def test_writer_abort(self): - lock = lock_utils.ReaderWriterLock() - self.assertFalse(lock.owner) - - def blow_up(): - with lock.write_lock(): - self.assertEqual(lock.WRITER, lock.owner) - raise RuntimeError("Broken") - - self.assertRaises(RuntimeError, blow_up) - self.assertFalse(lock.owner) - - def test_reader_abort(self): - lock = lock_utils.ReaderWriterLock() - self.assertFalse(lock.owner) - - def blow_up(): - with lock.read_lock(): - self.assertEqual(lock.READER, lock.owner) - raise RuntimeError("Broken") - - self.assertRaises(RuntimeError, blow_up) - self.assertFalse(lock.owner) - - def test_double_reader_abort(self): - lock = lock_utils.ReaderWriterLock() - activated = collections.deque() - - def double_bad_reader(): - with lock.read_lock(): - with lock.read_lock(): - raise RuntimeError("Broken") - - def happy_writer(): - with lock.write_lock(): - activated.append(lock.owner) - - with futures.ThreadPoolExecutor(max_workers=20) as e: - for i in range(0, 20): - if i % 2 == 0: - e.submit(double_bad_reader) - else: - e.submit(happy_writer) - - self.assertEqual(10, len([a for a in activated if a == 'w'])) - - def test_double_reader_writer(self): - lock = lock_utils.ReaderWriterLock() - activated = collections.deque() - active = threading_utils.Event() - - def double_reader(): - with lock.read_lock(): - active.set() - while not lock.has_pending_writers: - time.sleep(0.001) - with lock.read_lock(): - activated.append(lock.owner) - - def happy_writer(): - with lock.write_lock(): - activated.append(lock.owner) - - reader = threading_utils.daemon_thread(double_reader) - reader.start() - self.assertTrue(active.wait(test_utils.WAIT_TIMEOUT)) - - writer = threading_utils.daemon_thread(happy_writer) - writer.start() - - reader.join() - writer.join() - self.assertEqual(2, len(activated)) - self.assertEqual(['r', 'w'], list(activated)) - - def test_reader_chaotic(self): - lock = lock_utils.ReaderWriterLock() - activated = collections.deque() - - def chaotic_reader(blow_up): - with lock.read_lock(): - if blow_up: - raise RuntimeError("Broken") - else: - activated.append(lock.owner) - - def happy_writer(): - with lock.write_lock(): - activated.append(lock.owner) - - with futures.ThreadPoolExecutor(max_workers=20) as e: - for i in range(0, 20): - if i % 2 == 0: - e.submit(chaotic_reader, blow_up=bool(i % 4 == 0)) - else: - e.submit(happy_writer) - - writers = [a for a in activated if a == 'w'] - readers = [a for a in activated if a == 'r'] - self.assertEqual(10, len(writers)) - self.assertEqual(5, len(readers)) - - def test_writer_chaotic(self): - lock = lock_utils.ReaderWriterLock() - activated = collections.deque() - - def chaotic_writer(blow_up): - with lock.write_lock(): - if blow_up: - raise RuntimeError("Broken") - else: - activated.append(lock.owner) - - def happy_reader(): - with lock.read_lock(): - activated.append(lock.owner) - - with futures.ThreadPoolExecutor(max_workers=20) as e: - for i in range(0, 20): - if i % 2 == 0: - e.submit(chaotic_writer, blow_up=bool(i % 4 == 0)) - else: - e.submit(happy_reader) - - writers = [a for a in activated if a == 'w'] - readers = [a for a in activated if a == 'r'] - self.assertEqual(5, len(writers)) - self.assertEqual(10, len(readers)) - - def test_single_reader_writer(self): - results = [] - lock = lock_utils.ReaderWriterLock() - with lock.read_lock(): - self.assertTrue(lock.is_reader()) - self.assertEqual(0, len(results)) - with lock.write_lock(): - results.append(1) - self.assertTrue(lock.is_writer()) - with lock.read_lock(): - self.assertTrue(lock.is_reader()) - self.assertEqual(1, len(results)) - self.assertFalse(lock.is_reader()) - self.assertFalse(lock.is_writer()) - - def test_reader_to_writer(self): - lock = lock_utils.ReaderWriterLock() - - def writer_func(): - with lock.write_lock(): - pass - - with lock.read_lock(): - self.assertRaises(RuntimeError, writer_func) - self.assertFalse(lock.is_writer()) - - self.assertFalse(lock.is_reader()) - self.assertFalse(lock.is_writer()) - - def test_writer_to_reader(self): - lock = lock_utils.ReaderWriterLock() - - def reader_func(): - with lock.read_lock(): - self.assertTrue(lock.is_writer()) - self.assertTrue(lock.is_reader()) - - with lock.write_lock(): - self.assertIsNone(reader_func()) - self.assertFalse(lock.is_reader()) - - self.assertFalse(lock.is_reader()) - self.assertFalse(lock.is_writer()) - - def test_double_writer(self): - lock = lock_utils.ReaderWriterLock() - with lock.write_lock(): - self.assertFalse(lock.is_reader()) - self.assertTrue(lock.is_writer()) - with lock.write_lock(): - self.assertTrue(lock.is_writer()) - self.assertTrue(lock.is_writer()) - - self.assertFalse(lock.is_reader()) - self.assertFalse(lock.is_writer()) - - def test_double_reader(self): - lock = lock_utils.ReaderWriterLock() - with lock.read_lock(): - self.assertTrue(lock.is_reader()) - self.assertFalse(lock.is_writer()) - with lock.read_lock(): - self.assertTrue(lock.is_reader()) - self.assertTrue(lock.is_reader()) - - self.assertFalse(lock.is_reader()) - self.assertFalse(lock.is_writer()) - - def test_multi_reader_multi_writer(self): - writer_times, reader_times = _spawn_variation(10, 10) - self.assertEqual(10, len(writer_times)) - self.assertEqual(10, len(reader_times)) - for (start, stop) in writer_times: - self.assertEqual(0, _find_overlaps(reader_times, start, stop)) - self.assertEqual(1, _find_overlaps(writer_times, start, stop)) - for (start, stop) in reader_times: - self.assertEqual(0, _find_overlaps(writer_times, start, stop)) - - def test_multi_reader_single_writer(self): - writer_times, reader_times = _spawn_variation(9, 1) - self.assertEqual(1, len(writer_times)) - self.assertEqual(9, len(reader_times)) - start, stop = writer_times[0] - self.assertEqual(0, _find_overlaps(reader_times, start, stop)) - - def test_multi_writer(self): - writer_times, reader_times = _spawn_variation(0, 10) - self.assertEqual(10, len(writer_times)) - self.assertEqual(0, len(reader_times)) - for (start, stop) in writer_times: - self.assertEqual(1, _find_overlaps(writer_times, start, stop)) diff --git a/taskflow/utils/lock_utils.py b/taskflow/utils/lock_utils.py index 58945c0f..7b1b026f 100644 --- a/taskflow/utils/lock_utils.py +++ b/taskflow/utils/lock_utils.py @@ -19,23 +19,14 @@ # pulls in oslo.cfg) and is reduced to only what taskflow currently wants to # use from that code. -import collections import contextlib -import errno -import os import threading -import time -from oslo_utils import importutils import six from taskflow import logging from taskflow.utils import misc -# Used for the reader-writer lock get the right thread 'hack' (needed below). -eventlet = importutils.try_import('eventlet') -eventlet_patcher = importutils.try_import('eventlet.patcher') - LOG = logging.getLogger(__name__) @@ -96,203 +87,6 @@ def locked(*args, **kwargs): return decorator -def read_locked(*args, **kwargs): - """Acquires & releases a read lock around call into decorated method. - - NOTE(harlowja): if no attribute name is provided then by default the - attribute named '_lock' is looked for (this attribute is expected to be - the rw-lock object) in the instance object this decorator is attached to. - """ - - def decorator(f): - attr_name = kwargs.get('lock', '_lock') - - @six.wraps(f) - def wrapper(self, *args, **kwargs): - rw_lock = getattr(self, attr_name) - with rw_lock.read_lock(): - return f(self, *args, **kwargs) - - return wrapper - - # This is needed to handle when the decorator has args or the decorator - # doesn't have args, python is rather weird here... - if kwargs or not args: - return decorator - else: - if len(args) == 1: - return decorator(args[0]) - else: - return decorator - - -def write_locked(*args, **kwargs): - """Acquires & releases a write lock around call into decorated method. - - NOTE(harlowja): if no attribute name is provided then by default the - attribute named '_lock' is looked for (this attribute is expected to be - the rw-lock object) in the instance object this decorator is attached to. - """ - - def decorator(f): - attr_name = kwargs.get('lock', '_lock') - - @six.wraps(f) - def wrapper(self, *args, **kwargs): - rw_lock = getattr(self, attr_name) - with rw_lock.write_lock(): - return f(self, *args, **kwargs) - - return wrapper - - # This is needed to handle when the decorator has args or the decorator - # doesn't have args, python is rather weird here... - if kwargs or not args: - return decorator - else: - if len(args) == 1: - return decorator(args[0]) - else: - return decorator - - -class ReaderWriterLock(object): - """A reader/writer lock. - - This lock allows for simultaneous readers to exist but only one writer - to exist for use-cases where it is useful to have such types of locks. - - Currently a reader can not escalate its read lock to a write lock and - a writer can not acquire a read lock while it is waiting on the write - lock. - - In the future these restrictions may be relaxed. - - This can be eventually removed if http://bugs.python.org/issue8800 ever - gets accepted into the python standard threading library... - """ - WRITER = 'w' - READER = 'r' - - @staticmethod - def _fetch_current_thread_functor(): - # Until https://github.com/eventlet/eventlet/issues/172 is resolved - # or addressed we have to use complicated workaround to get a object - # that will not be recycled; the usage of threading.current_thread() - # doesn't appear to currently be monkey patched and therefore isn't - # reliable to use (and breaks badly when used as all threads share - # the same current_thread() object)... - if eventlet is not None and eventlet_patcher is not None: - if eventlet_patcher.is_monkey_patched('thread'): - return lambda: eventlet.getcurrent() - return lambda: threading.current_thread() - - def __init__(self): - self._writer = None - self._pending_writers = collections.deque() - self._readers = collections.deque() - self._cond = threading.Condition() - self._current_thread = self._fetch_current_thread_functor() - - @property - def has_pending_writers(self): - """Returns if there are writers waiting to become the *one* writer.""" - return bool(self._pending_writers) - - def is_writer(self, check_pending=True): - """Returns if the caller is the active writer or a pending writer.""" - me = self._current_thread() - if self._writer == me: - return True - if check_pending: - return me in self._pending_writers - else: - return False - - @property - def owner(self): - """Returns whether the lock is locked by a writer or reader.""" - with self._cond: - # Obtain the lock to ensure we get a accurate view of the actual - # owner that isn't likely to change when we are reading it... - if self._writer is not None: - return self.WRITER - if self._readers: - return self.READER - return None - - def is_reader(self): - """Returns if the caller is one of the readers.""" - me = self._current_thread() - return me in self._readers - - @contextlib.contextmanager - def read_lock(self): - """Context manager that grants a read lock. - - Will wait until no active or pending writers. - - Raises a RuntimeError if a pending writer tries to acquire - a read lock. - """ - me = self._current_thread() - if me in self._pending_writers: - raise RuntimeError("Writer %s can not acquire a read lock" - " while waiting for the write lock" - % me) - with self._cond: - while True: - # No active writer, or we are the writer; - # we are good to become a reader. - if self._writer is None or self._writer == me: - self._readers.append(me) - break - # An active writer; guess we have to wait. - self._cond.wait() - try: - yield self - finally: - # I am no longer a reader, remove *one* occurrence of myself. - # If the current thread acquired two read locks, then it will - # still have to remove that other read lock; this allows for - # basic reentrancy to be possible. - with self._cond: - self._readers.remove(me) - self._cond.notify_all() - - @contextlib.contextmanager - def write_lock(self): - """Context manager that grants a write lock. - - Will wait until no active readers. Blocks readers after acquiring. - - Raises a RuntimeError if an active reader attempts to acquire a lock. - """ - me = self._current_thread() - if self.is_reader(): - raise RuntimeError("Reader %s to writer privilege" - " escalation not allowed" % me) - if self.is_writer(check_pending=False): - # Already the writer; this allows for basic reentrancy. - yield self - else: - with self._cond: - self._pending_writers.append(me) - while True: - # No readers, and no active writer, am I next?? - if len(self._readers) == 0 and self._writer is None: - if self._pending_writers[0] == me: - self._writer = self._pending_writers.popleft() - break - self._cond.wait() - try: - yield self - finally: - with self._cond: - self._writer = None - self._cond.notify_all() - - class MultiLock(object): """A class which attempts to obtain & release many locks at once. @@ -411,107 +205,3 @@ class MultiLock(object): # At the end only clear it off, so that under partial failure we don't # lose any locks... self._lock_stacks.pop() - - -class _InterProcessLock(object): - """An interprocess locking implementation. - - This is a lock implementation which allows multiple locks, working around - issues like bugs.debian.org/cgi-bin/bugreport.cgi?bug=632857 and does - not require any cleanup. Since the lock is always held on a file - descriptor rather than outside of the process, the lock gets dropped - automatically if the process crashes, even if __exit__ is not executed. - - There are no guarantees regarding usage by multiple green threads in a - single process here. This lock works only between processes. - - Note these locks are released when the descriptor is closed, so it's not - safe to close the file descriptor while another green thread holds the - lock. Just opening and closing the lock file can break synchronisation, - so lock files must be accessed only using this abstraction. - """ - - def __init__(self, name): - self.lockfile = None - self.fname = name - - def acquire(self): - basedir = os.path.dirname(self.fname) - - if not os.path.exists(basedir): - misc.ensure_tree(basedir) - LOG.debug('Created lock path: %s', basedir) - - self.lockfile = open(self.fname, 'w') - - while True: - try: - # Using non-blocking locks since green threads are not - # patched to deal with blocking locking calls. - # Also upon reading the MSDN docs for locking(), it seems - # to have a laughable 10 attempts "blocking" mechanism. - self.trylock() - LOG.debug('Got file lock "%s"', self.fname) - return True - except IOError as e: - if e.errno in (errno.EACCES, errno.EAGAIN): - # external locks synchronise things like iptables - # updates - give it some time to prevent busy spinning - time.sleep(0.01) - else: - raise threading.ThreadError("Unable to acquire lock on" - " `%(filename)s` due to" - " %(exception)s" % - { - 'filename': self.fname, - 'exception': e, - }) - - def __enter__(self): - self.acquire() - return self - - def release(self): - try: - self.unlock() - self.lockfile.close() - LOG.debug('Released file lock "%s"', self.fname) - except IOError: - LOG.exception("Could not release the acquired lock `%s`", - self.fname) - - def __exit__(self, exc_type, exc_val, exc_tb): - self.release() - - def exists(self): - return os.path.exists(self.fname) - - def trylock(self): - raise NotImplementedError() - - def unlock(self): - raise NotImplementedError() - - -class _WindowsLock(_InterProcessLock): - def trylock(self): - msvcrt.locking(self.lockfile.fileno(), msvcrt.LK_NBLCK, 1) - - def unlock(self): - msvcrt.locking(self.lockfile.fileno(), msvcrt.LK_UNLCK, 1) - - -class _PosixLock(_InterProcessLock): - def trylock(self): - fcntl.lockf(self.lockfile, fcntl.LOCK_EX | fcntl.LOCK_NB) - - def unlock(self): - fcntl.lockf(self.lockfile, fcntl.LOCK_UN) - - -if os.name == 'nt': - import msvcrt - InterProcessLock = _WindowsLock -else: - import fcntl - InterProcessLock = _PosixLock From abf21c728914ac2c2390efcce7dec4f19eb926a5 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 3 Jun 2015 16:52:16 -0700 Subject: [PATCH 193/246] Refactor machine builder + runner into single unit There is really no need to seperate off the runner into a unit that builds a state-machine and then provides a tiny utility function, both of these can just be in the same class and code so that it is easier to understand/read. Change-Id: I18b97514e230451ef804a878a0edcea4d0b2ad20 --- taskflow/engines/action_engine/runner.py | 90 ++++++++----------- .../tests/unit/action_engine/test_runner.py | 20 ++--- tools/state_graph.py | 2 +- 3 files changed, 48 insertions(+), 64 deletions(-) diff --git a/taskflow/engines/action_engine/runner.py b/taskflow/engines/action_engine/runner.py index d50de157..8d637c1c 100644 --- a/taskflow/engines/action_engine/runner.py +++ b/taskflow/engines/action_engine/runner.py @@ -51,38 +51,48 @@ class _MachineMemory(object): self.done = set() -class _MachineBuilder(object): - """State machine *builder* that the runner uses. +class Runner(object): + """State machine *builder* + *runner* that powers the engine components. - NOTE(harlowja): the machine states that this build will for are:: + NOTE(harlowja): the machine (states and events that will trigger + transitions) that this builds is represented by the following + table:: - +--------------+------------------+------------+----------+---------+ - Start | Event | End | On Enter | On Exit - +--------------+------------------+------------+----------+---------+ - ANALYZING | completed | GAME_OVER | | - ANALYZING | schedule_next | SCHEDULING | | - ANALYZING | wait_finished | WAITING | | - FAILURE[$] | | | | - GAME_OVER | failed | FAILURE | | - GAME_OVER | reverted | REVERTED | | - GAME_OVER | success | SUCCESS | | - GAME_OVER | suspended | SUSPENDED | | - RESUMING | schedule_next | SCHEDULING | | - REVERTED[$] | | | | - SCHEDULING | wait_finished | WAITING | | - SUCCESS[$] | | | | - SUSPENDED[$] | | | | - UNDEFINED[^] | start | RESUMING | | - WAITING | examine_finished | ANALYZING | | - +--------------+------------------+------------+----------+---------+ + +--------------+------------------+------------+----------+---------+ + Start | Event | End | On Enter | On Exit + +--------------+------------------+------------+----------+---------+ + ANALYZING | completed | GAME_OVER | | + ANALYZING | schedule_next | SCHEDULING | | + ANALYZING | wait_finished | WAITING | | + FAILURE[$] | | | | + GAME_OVER | failed | FAILURE | | + GAME_OVER | reverted | REVERTED | | + GAME_OVER | success | SUCCESS | | + GAME_OVER | suspended | SUSPENDED | | + RESUMING | schedule_next | SCHEDULING | | + REVERTED[$] | | | | + SCHEDULING | wait_finished | WAITING | | + SUCCESS[$] | | | | + SUSPENDED[$] | | | | + UNDEFINED[^] | start | RESUMING | | + WAITING | examine_finished | ANALYZING | | + +--------------+------------------+------------+----------+---------+ Between any of these yielded states (minus ``GAME_OVER`` and ``UNDEFINED``) if the engine has been suspended or the engine has failed (due to a non-resolveable task failure or scheduling failure) the machine will stop executing new tasks (currently running tasks will be allowed to complete) and this machines run loop will be broken. + + NOTE(harlowja): If the runtimes scheduler component is able to schedule + tasks in parallel, this enables parallel running and/or reversion. """ + # Informational states this action yields while running, not useful to + # have the engine record but useful to provide to end-users when doing + # execution iterations. + ignorable_states = (st.SCHEDULING, st.WAITING, st.RESUMING, st.ANALYZING) + def __init__(self, runtime, waiter): self._analyzer = runtime.analyzer self._completer = runtime.completer @@ -91,9 +101,12 @@ class _MachineBuilder(object): self._waiter = waiter def runnable(self): + """Checks if the storage says the flow is still runnable/running.""" return self._storage.get_flow_state() == st.RUNNING def build(self, timeout=None): + """Builds a state-machine (that can be/is used during running).""" + memory = _MachineMemory() if timeout is None: timeout = _WAITING_TIMEOUT @@ -244,38 +257,9 @@ class _MachineBuilder(object): m.freeze() return (m, memory) - -class Runner(object): - """Runner that iterates while executing nodes using the given runtime. - - This runner acts as the action engine run loop/state-machine, it resumes - the workflow, schedules all task it can for execution using the runtimes - scheduler and analyzer components, and than waits on returned futures and - then activates the runtimes completion component to finish up those tasks - and so on... - - NOTE(harlowja): If the runtimes scheduler component is able to schedule - tasks in parallel, this enables parallel running and/or reversion. - """ - - # Informational states this action yields while running, not useful to - # have the engine record but useful to provide to end-users when doing - # execution iterations. - ignorable_states = (st.SCHEDULING, st.WAITING, st.RESUMING, st.ANALYZING) - - def __init__(self, runtime, waiter): - self._builder = _MachineBuilder(runtime, waiter) - - @property - def builder(self): - return self._builder - - def runnable(self): - return self._builder.runnable() - def run_iter(self, timeout=None): - """Runs the nodes using a built state machine.""" - machine, memory = self.builder.build(timeout=timeout) + """Runs iteratively using a locally built state machine.""" + machine, memory = self.build(timeout=timeout) for (_prior_state, new_state) in machine.run_iter(_START): # NOTE(harlowja): skip over meta-states. if new_state not in _META_STATES: diff --git a/taskflow/tests/unit/action_engine/test_runner.py b/taskflow/tests/unit/action_engine/test_runner.py index 98ae0e28..cbc1e2e9 100644 --- a/taskflow/tests/unit/action_engine/test_runner.py +++ b/taskflow/tests/unit/action_engine/test_runner.py @@ -174,7 +174,7 @@ class RunnerTest(test.TestCase, _RunnerTestMixin): rt.storage.get_atom_state(sad_tasks[0].name)) -class RunnerBuilderTest(test.TestCase, _RunnerTestMixin): +class RunnerBuildTest(test.TestCase, _RunnerTestMixin): def test_builder_manual_process(self): flow = lf.Flow("root") tasks = test_utils.make_many( @@ -182,8 +182,8 @@ class RunnerBuilderTest(test.TestCase, _RunnerTestMixin): flow.add(*tasks) rt = self._make_runtime(flow, initial_state=st.RUNNING) - machine, memory = rt.runner.builder.build() - self.assertTrue(rt.runner.builder.runnable()) + machine, memory = rt.runner.build() + self.assertTrue(rt.runner.runnable()) self.assertRaises(fsm.NotInitialized, machine.process_event, 'poke') # Should now be pending... @@ -251,8 +251,8 @@ class RunnerBuilderTest(test.TestCase, _RunnerTestMixin): flow.add(*tasks) rt = self._make_runtime(flow, initial_state=st.RUNNING) - machine, memory = rt.runner.builder.build() - self.assertTrue(rt.runner.builder.runnable()) + machine, memory = rt.runner.build() + self.assertTrue(rt.runner.runnable()) transitions = list(machine.run_iter('start')) self.assertEqual((runner._UNDEFINED, st.RESUMING), transitions[0]) @@ -265,8 +265,8 @@ class RunnerBuilderTest(test.TestCase, _RunnerTestMixin): flow.add(*tasks) rt = self._make_runtime(flow, initial_state=st.RUNNING) - machine, memory = rt.runner.builder.build() - self.assertTrue(rt.runner.builder.runnable()) + machine, memory = rt.runner.build() + self.assertTrue(rt.runner.runnable()) transitions = list(machine.run_iter('start')) self.assertEqual((runner._GAME_OVER, st.FAILURE), transitions[-1]) @@ -278,8 +278,8 @@ class RunnerBuilderTest(test.TestCase, _RunnerTestMixin): flow.add(*tasks) rt = self._make_runtime(flow, initial_state=st.RUNNING) - machine, memory = rt.runner.builder.build() - self.assertTrue(rt.runner.builder.runnable()) + machine, memory = rt.runner.build() + self.assertTrue(rt.runner.runnable()) transitions = list(machine.run_iter('start')) self.assertEqual((runner._GAME_OVER, st.REVERTED), transitions[-1]) @@ -292,7 +292,7 @@ class RunnerBuilderTest(test.TestCase, _RunnerTestMixin): flow.add(*tasks) rt = self._make_runtime(flow, initial_state=st.RUNNING) - machine, memory = rt.runner.builder.build() + machine, memory = rt.runner.build() transitions = list(machine.run_iter('start')) occurrences = dict((t, transitions.count(t)) for t in transitions) diff --git a/tools/state_graph.py b/tools/state_graph.py index 31961404..38bd6d91 100755 --- a/tools/state_graph.py +++ b/tools/state_graph.py @@ -134,7 +134,7 @@ def main(): elif options.engines: source_type = "Engines" r = runner.Runner(DummyRuntime(), None) - source, memory = r.builder.build() + source, memory = r.build() internal_states.extend(runner._META_STATES) ordering = 'out' elif options.wbe_requests: From 9091a9ce96ab29182e97f975dcbf73978b5a5682 Mon Sep 17 00:00:00 2001 From: OpenStack Proposal Bot Date: Thu, 4 Jun 2015 14:05:59 +0000 Subject: [PATCH 194/246] Updated from global requirements Change-Id: I9a8f403a24dbf694bbd25150106671bfaeea39cc --- requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/requirements.txt b/requirements.txt index 0b45b8c5..109bcff3 100644 --- a/requirements.txt +++ b/requirements.txt @@ -17,7 +17,7 @@ six>=1.9.0 enum34 # For reader/writer + interprocess locks. -fasteners>=0.5 # Apache-2.0 +fasteners>=0.5 # Apache-2.0 # Very nice graph library networkx>=1.8 From 4ee5ffd0871fcab8e814c0eaade0f184570269b1 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Thu, 14 May 2015 13:04:24 -0700 Subject: [PATCH 195/246] Add a new `ls_r` method Instead of having a `ls` method that when used recursively *always* returns the absolute path of items in the fake in memory storage tree and *relative* paths (when used in non-recursive mode) add a new `ls_r` method that can return absolute *or* relative paths. In the future it is highly likely that the the `ls` recursive keyword argument will be removed (so preferring and moving to the `ls_r` should occur earlier rather than later), so this also adds a debtcollector removed keyword argument decorator over the existing `ls` to ensure that users are aware of this change (as well as a adjusted docstring). Fixes bug 1458114 Change-Id: Id2a5869e94ac44679020a14297d1073d1dc2718f --- taskflow/examples/dump_memory_backend.py | 2 +- taskflow/persistence/backends/impl_memory.py | 63 +++++++++++++++++-- .../persistence/test_memory_persistence.py | 60 +++++++++++++++++- 3 files changed, 117 insertions(+), 8 deletions(-) diff --git a/taskflow/examples/dump_memory_backend.py b/taskflow/examples/dump_memory_backend.py index 2e3aee71..6c6d5488 100644 --- a/taskflow/examples/dump_memory_backend.py +++ b/taskflow/examples/dump_memory_backend.py @@ -70,7 +70,7 @@ e.run() print("---------") print("After run") print("---------") -for path in backend.memory.ls(backend.memory.root_path, recursive=True): +for path in backend.memory.ls_r(backend.memory.root_path, absolute=True): value = backend.memory[path] if value: print("%s -> %s" % (path, value)) diff --git a/taskflow/persistence/backends/impl_memory.py b/taskflow/persistence/backends/impl_memory.py index 0aac58eb..43207b81 100644 --- a/taskflow/persistence/backends/impl_memory.py +++ b/taskflow/persistence/backends/impl_memory.py @@ -20,6 +20,7 @@ import copy import itertools import posixpath as pp +from debtcollector import removals import fasteners import six @@ -166,13 +167,63 @@ class FakeFilesystem(object): else: return self._copier(node.metadata['value']) - def ls(self, path, recursive=False): - """Return list of all children of the given path.""" - if not recursive: - return [node.item for node in self._fetch_node(path)] + def _up_to_root_selector(self, root_node, child_node): + # Build the path from the child to the root and stop at the + # root, and then form a path string... + path_pieces = [child_node.item] + for parent_node in child_node.path_iter(include_self=False): + if parent_node is root_node: + break + path_pieces.append(parent_node.item) + if len(path_pieces) > 1: + path_pieces.reverse() + return self.join(*path_pieces) + + @staticmethod + def _metadata_path_selector(root_node, child_node): + return child_node.metadata['path'] + + def ls_r(self, path, absolute=False): + """Return list of all children of the given path (recursively).""" + node = self._fetch_node(path) + if absolute: + selector_func = self._metadata_path_selector else: - node = self._fetch_node(path) - return [child.metadata['path'] for child in node.bfs_iter()] + selector_func = self._up_to_root_selector + return [selector_func(node, child_node) + for child_node in node.bfs_iter()] + + @removals.removed_kwarg('recursive', version="0.11", removal_version="?") + def ls(self, path, recursive=False): + """Return list of all children of the given path. + + NOTE(harlowja): if ``recursive`` is passed in as truthy then the + absolute path is **always** returned (not the relative path). If + ``recursive`` is left as the default or falsey then the + relative path is **always** returned. + + This is documented in bug `1458114`_ and the existing behavior is + being maintained, to get a recursive version that is absolute (or is + not absolute) it is recommended to use the :py:meth:`.ls_r` method + instead. + + .. deprecated:: 0.11 + + In a future release the ``recursive`` keyword argument will + be removed (so preferring and moving to the :py:meth:`.ls_r` should + occur earlier rather than later). + + .. _1458114: https://bugs.launchpad.net/taskflow/+bug/1458114 + """ + node = self._fetch_node(path) + if recursive: + selector_func = self._metadata_path_selector + child_node_it = node.bfs_iter() + else: + selector_func = self._up_to_root_selector + child_node_it = iter(node) + return [selector_func(node, child_node) + for child_node in child_node_it] def clear(self): """Remove all nodes (except the root) from this filesystem.""" diff --git a/taskflow/tests/unit/persistence/test_memory_persistence.py b/taskflow/tests/unit/persistence/test_memory_persistence.py index 24f76aa3..80686396 100644 --- a/taskflow/tests/unit/persistence/test_memory_persistence.py +++ b/taskflow/tests/unit/persistence/test_memory_persistence.py @@ -71,7 +71,7 @@ class MemoryFilesystemTest(test.TestCase): self.assertEqual('c', fs['/c']) self.assertEqual('db', fs['/d/b']) - def test_ls_recursive(self): + def test_old_ls_recursive(self): fs = impl_memory.FakeFilesystem() fs.ensure_path("/d") fs.ensure_path("/c/d") @@ -91,7 +91,65 @@ class MemoryFilesystemTest(test.TestCase): '/a/b/c/d', ], contents) + def test_ls_recursive(self): + fs = impl_memory.FakeFilesystem() + fs.ensure_path("/d") + fs.ensure_path("/c/d") + fs.ensure_path("/b/c/d") + fs.ensure_path("/a/b/c/d") + contents = fs.ls_r("/", absolute=False) + self.assertEqual([ + 'a', + 'b', + 'c', + 'd', + 'a/b', + 'b/c', + 'c/d', + 'a/b/c', + 'b/c/d', + 'a/b/c/d', + ], contents) + + def test_ls_recursive_absolute(self): + fs = impl_memory.FakeFilesystem() + fs.ensure_path("/d") + fs.ensure_path("/c/d") + fs.ensure_path("/b/c/d") + fs.ensure_path("/a/b/c/d") + contents = fs.ls_r("/", absolute=True) + self.assertEqual([ + '/a', + '/b', + '/c', + '/d', + '/a/b', + '/b/c', + '/c/d', + '/a/b/c', + '/b/c/d', + '/a/b/c/d', + ], contents) + def test_ls_recursive_targeted(self): + fs = impl_memory.FakeFilesystem() + fs.ensure_path("/d") + fs.ensure_path("/c/d") + fs.ensure_path("/b/c/d") + fs.ensure_path("/a/b/c/d") + contents = fs.ls_r("/a/b", absolute=False) + self.assertEqual(['c', 'c/d'], contents) + + def test_ls_recursive_targeted_absolute(self): + fs = impl_memory.FakeFilesystem() + fs.ensure_path("/d") + fs.ensure_path("/c/d") + fs.ensure_path("/b/c/d") + fs.ensure_path("/a/b/c/d") + contents = fs.ls_r("/a/b", absolute=True) + self.assertEqual(['/a/b/c', '/a/b/c/d'], contents) + + def test_old_ls_recursive_targeted_absolute(self): fs = impl_memory.FakeFilesystem() fs.ensure_path("/d") fs.ensure_path("/c/d") From f1bd24fbe47cc0d8103ce5652b6952108de372f2 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Thu, 4 Jun 2015 16:21:35 -0700 Subject: [PATCH 196/246] Remove all 'lock_utils' now that fasteners provides equivalents Depends-On: I442249783da4a6ae10c78b95e0b279409c95d2e6 Change-Id: I877928c858e8d6176d3f01ad9de2765104acf5c3 --- doc/source/utils.rst | 5 - requirements.txt | 2 +- taskflow/conductors/base.py | 6 +- taskflow/engines/action_engine/compiler.py | 5 +- taskflow/engines/action_engine/engine.py | 10 +- taskflow/engines/worker_based/protocol.py | 4 +- taskflow/jobs/backends/impl_zookeeper.py | 6 +- taskflow/tests/unit/test_utils_lock_utils.py | 281 ------------------- taskflow/utils/lock_utils.py | 207 -------------- 9 files changed, 17 insertions(+), 509 deletions(-) delete mode 100644 taskflow/tests/unit/test_utils_lock_utils.py delete mode 100644 taskflow/utils/lock_utils.py diff --git a/doc/source/utils.rst b/doc/source/utils.rst index 6949ccf0..ac0dd5c4 100644 --- a/doc/source/utils.rst +++ b/doc/source/utils.rst @@ -33,11 +33,6 @@ Kombu .. automodule:: taskflow.utils.kombu_utils -Locks -~~~~~ - -.. automodule:: taskflow.utils.lock_utils - Miscellaneous ~~~~~~~~~~~~~ diff --git a/requirements.txt b/requirements.txt index 109bcff3..e9d0550b 100644 --- a/requirements.txt +++ b/requirements.txt @@ -17,7 +17,7 @@ six>=1.9.0 enum34 # For reader/writer + interprocess locks. -fasteners>=0.5 # Apache-2.0 +fasteners>=0.7 # Apache-2.0 # Very nice graph library networkx>=1.8 diff --git a/taskflow/conductors/base.py b/taskflow/conductors/base.py index 7a6b8ce8..6e46fff8 100644 --- a/taskflow/conductors/base.py +++ b/taskflow/conductors/base.py @@ -15,11 +15,11 @@ import abc import threading +import fasteners import six from taskflow import engines from taskflow import exceptions as excp -from taskflow.utils import lock_utils @six.add_metaclass(abc.ABCMeta) @@ -109,13 +109,13 @@ class Conductor(object): # listener factories over the jobboard return [] - @lock_utils.locked + @fasteners.locked def connect(self): """Ensures the jobboard is connected (noop if it is already).""" if not self._jobboard.connected: self._jobboard.connect() - @lock_utils.locked + @fasteners.locked def close(self): """Closes the contained jobboard, disallowing further use.""" self._jobboard.close() diff --git a/taskflow/engines/action_engine/compiler.py b/taskflow/engines/action_engine/compiler.py index ff86de27..dc6c24e1 100644 --- a/taskflow/engines/action_engine/compiler.py +++ b/taskflow/engines/action_engine/compiler.py @@ -17,13 +17,14 @@ import collections import threading +import fasteners + from taskflow import exceptions as exc from taskflow import flow from taskflow import logging from taskflow import task from taskflow.types import graph as gr from taskflow.types import tree as tr -from taskflow.utils import lock_utils from taskflow.utils import misc LOG = logging.getLogger(__name__) @@ -423,7 +424,7 @@ class PatternCompiler(object): # Indent it so that it's slightly offset from the above line. LOG.blather(" %s", line) - @lock_utils.locked + @fasteners.locked def compile(self): """Compiles the contained item into a compiled equivalent.""" if self._compilation is None: diff --git a/taskflow/engines/action_engine/engine.py b/taskflow/engines/action_engine/engine.py index 12f5feac..716279e9 100644 --- a/taskflow/engines/action_engine/engine.py +++ b/taskflow/engines/action_engine/engine.py @@ -19,6 +19,7 @@ import contextlib import threading from concurrent import futures +import fasteners import networkx as nx from oslo_utils import excutils from oslo_utils import strutils @@ -33,7 +34,6 @@ from taskflow import logging from taskflow import states from taskflow import storage from taskflow.types import failure -from taskflow.utils import lock_utils from taskflow.utils import misc LOG = logging.getLogger(__name__) @@ -125,7 +125,7 @@ class ActionEngine(base.Engine): scope_fetcher=_scope_fetcher) def run(self): - with lock_utils.try_lock(self._lock) as was_locked: + with fasteners.try_lock(self._lock) as was_locked: if not was_locked: raise exc.ExecutionFailure("Engine currently locked, please" " try again later") @@ -214,7 +214,7 @@ class ActionEngine(base.Engine): node.inject, transient=transient) - @lock_utils.locked + @fasteners.locked def validate(self): if not self._storage_ensured: raise exc.InvalidState("Can not validate an engine" @@ -261,7 +261,7 @@ class ActionEngine(base.Engine): sorted(missing), cause=last_cause) - @lock_utils.locked + @fasteners.locked def prepare(self): if not self._compiled: raise exc.InvalidState("Can not prepare an engine" @@ -283,7 +283,7 @@ class ActionEngine(base.Engine): def _compiler(self): return self._compiler_factory(self._flow) - @lock_utils.locked + @fasteners.locked def compile(self): if self._compiled: return diff --git a/taskflow/engines/worker_based/protocol.py b/taskflow/engines/worker_based/protocol.py index 867f5369..cbb61ebe 100644 --- a/taskflow/engines/worker_based/protocol.py +++ b/taskflow/engines/worker_based/protocol.py @@ -19,6 +19,7 @@ import collections import threading from concurrent import futures +import fasteners from oslo_utils import reflection from oslo_utils import timeutils import six @@ -28,7 +29,6 @@ from taskflow import exceptions as excp from taskflow import logging from taskflow.types import failure as ft from taskflow.types import timing as tt -from taskflow.utils import lock_utils from taskflow.utils import schema_utils as su # NOTE(skudriashev): This is protocol states and events, which are not @@ -336,7 +336,7 @@ class Request(Message): new_state, exc_info=True) return moved - @lock_utils.locked + @fasteners.locked def transition(self, new_state): """Transitions the request to a new state. diff --git a/taskflow/jobs/backends/impl_zookeeper.py b/taskflow/jobs/backends/impl_zookeeper.py index 246416fa..d92c2ba5 100644 --- a/taskflow/jobs/backends/impl_zookeeper.py +++ b/taskflow/jobs/backends/impl_zookeeper.py @@ -21,6 +21,7 @@ import sys import threading from concurrent import futures +import fasteners from kazoo import exceptions as k_exceptions from kazoo.protocol import paths as k_paths from kazoo.recipe import watchers @@ -35,7 +36,6 @@ from taskflow import logging from taskflow import states from taskflow.types import timing as tt from taskflow.utils import kazoo_utils -from taskflow.utils import lock_utils from taskflow.utils import misc LOG = logging.getLogger(__name__) @@ -762,7 +762,7 @@ class ZookeeperJobBoard(base.NotifyingJobBoard): def connected(self): return self._connected and self._client.connected - @lock_utils.locked(lock='_open_close_lock') + @fasteners.locked(lock='_open_close_lock') def close(self): if self._owned: LOG.debug("Stopping client") @@ -776,7 +776,7 @@ class ZookeeperJobBoard(base.NotifyingJobBoard): LOG.debug("Stopped & cleared local state") self._connected = False - @lock_utils.locked(lock='_open_close_lock') + @fasteners.locked(lock='_open_close_lock') def connect(self, timeout=10.0): def try_clean(): diff --git a/taskflow/tests/unit/test_utils_lock_utils.py b/taskflow/tests/unit/test_utils_lock_utils.py deleted file mode 100644 index 0c8213e9..00000000 --- a/taskflow/tests/unit/test_utils_lock_utils.py +++ /dev/null @@ -1,281 +0,0 @@ -# -*- coding: utf-8 -*- - -# Copyright (C) 2014 Yahoo! Inc. 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. - -import collections -import threading -import time - -from taskflow import test -from taskflow.test import mock -from taskflow.utils import lock_utils -from taskflow.utils import misc -from taskflow.utils import threading_utils - -# NOTE(harlowja): Sleep a little so now() can not be the same (which will -# cause false positives when our overlap detection code runs). If there are -# real overlaps then they will still exist. -NAPPY_TIME = 0.05 - -# We will spend this amount of time doing some "fake" work. -WORK_TIMES = [(0.01 + x / 100.0) for x in range(0, 5)] - -# Try to use a more accurate time for overlap detection (one that should -# never go backwards and cause false positives during overlap detection...). -now = misc.find_monotonic(allow_time_time=True) - - -def _find_overlaps(times, start, end): - overlaps = 0 - for (s, e) in times: - if s >= start and e <= end: - overlaps += 1 - return overlaps - - -class MultilockTest(test.TestCase): - THREAD_COUNT = 20 - - def test_empty_error(self): - self.assertRaises(ValueError, - lock_utils.MultiLock, []) - self.assertRaises(ValueError, - lock_utils.MultiLock, ()) - self.assertRaises(ValueError, - lock_utils.MultiLock, iter([])) - - def test_creation(self): - locks = [] - for _i in range(0, 10): - locks.append(threading.Lock()) - n_lock = lock_utils.MultiLock(locks) - self.assertEqual(0, n_lock.obtained) - self.assertEqual(len(locks), len(n_lock)) - - def test_acquired(self): - lock1 = threading.Lock() - lock2 = threading.Lock() - n_lock = lock_utils.MultiLock((lock1, lock2)) - self.assertTrue(n_lock.acquire()) - try: - self.assertTrue(lock1.locked()) - self.assertTrue(lock2.locked()) - finally: - n_lock.release() - self.assertFalse(lock1.locked()) - self.assertFalse(lock2.locked()) - - def test_acquired_context_manager(self): - lock1 = threading.Lock() - n_lock = lock_utils.MultiLock([lock1]) - with n_lock as gotten: - self.assertTrue(gotten) - self.assertTrue(lock1.locked()) - self.assertFalse(lock1.locked()) - self.assertEqual(0, n_lock.obtained) - - def test_partial_acquired(self): - lock1 = threading.Lock() - lock2 = mock.create_autospec(threading.Lock()) - lock2.acquire.return_value = False - n_lock = lock_utils.MultiLock((lock1, lock2)) - with n_lock as gotten: - self.assertFalse(gotten) - self.assertTrue(lock1.locked()) - self.assertEqual(1, n_lock.obtained) - self.assertEqual(2, len(n_lock)) - self.assertEqual(0, n_lock.obtained) - - def test_partial_acquired_failure(self): - lock1 = threading.Lock() - lock2 = mock.create_autospec(threading.Lock()) - lock2.acquire.side_effect = RuntimeError("Broke") - n_lock = lock_utils.MultiLock((lock1, lock2)) - self.assertRaises(threading.ThreadError, n_lock.acquire) - self.assertEqual(1, n_lock.obtained) - n_lock.release() - - def test_release_failure(self): - lock1 = threading.Lock() - lock2 = mock.create_autospec(threading.Lock()) - lock2.acquire.return_value = True - lock2.release.side_effect = RuntimeError("Broke") - n_lock = lock_utils.MultiLock((lock1, lock2)) - self.assertTrue(n_lock.acquire()) - self.assertEqual(2, n_lock.obtained) - self.assertRaises(threading.ThreadError, n_lock.release) - self.assertEqual(2, n_lock.obtained) - lock2.release.side_effect = None - n_lock.release() - self.assertEqual(0, n_lock.obtained) - - def test_release_partial_failure(self): - lock1 = threading.Lock() - lock2 = mock.create_autospec(threading.Lock()) - lock2.acquire.return_value = True - lock2.release.side_effect = RuntimeError("Broke") - lock3 = threading.Lock() - n_lock = lock_utils.MultiLock((lock1, lock2, lock3)) - self.assertTrue(n_lock.acquire()) - self.assertEqual(3, n_lock.obtained) - self.assertRaises(threading.ThreadError, n_lock.release) - self.assertEqual(2, n_lock.obtained) - lock2.release.side_effect = None - n_lock.release() - self.assertEqual(0, n_lock.obtained) - - def test_acquired_pass(self): - activated = collections.deque() - acquires = collections.deque() - lock1 = threading.Lock() - lock2 = threading.Lock() - n_lock = lock_utils.MultiLock((lock1, lock2)) - - def critical_section(): - start = now() - time.sleep(NAPPY_TIME) - end = now() - activated.append((start, end)) - - def run(): - with n_lock as gotten: - acquires.append(gotten) - critical_section() - - threads = [] - for _i in range(0, self.THREAD_COUNT): - t = threading_utils.daemon_thread(run) - threads.append(t) - t.start() - while threads: - t = threads.pop() - t.join() - - self.assertEqual(self.THREAD_COUNT, len(acquires)) - self.assertTrue(all(acquires)) - for (start, end) in activated: - self.assertEqual(1, _find_overlaps(activated, start, end)) - self.assertFalse(lock1.locked()) - self.assertFalse(lock2.locked()) - - def test_acquired_fail(self): - activated = collections.deque() - acquires = collections.deque() - lock1 = threading.Lock() - lock2 = threading.Lock() - n_lock = lock_utils.MultiLock((lock1, lock2)) - - def run(): - with n_lock as gotten: - acquires.append(gotten) - start = now() - time.sleep(NAPPY_TIME) - end = now() - activated.append((start, end)) - - def run_fail(): - try: - with n_lock as gotten: - acquires.append(gotten) - raise RuntimeError() - except RuntimeError: - pass - - threads = [] - for i in range(0, self.THREAD_COUNT): - if i % 2 == 1: - target = run_fail - else: - target = run - t = threading_utils.daemon_thread(target) - threads.append(t) - t.start() - while threads: - t = threads.pop() - t.join() - - self.assertEqual(self.THREAD_COUNT, len(acquires)) - self.assertTrue(all(acquires)) - for (start, end) in activated: - self.assertEqual(1, _find_overlaps(activated, start, end)) - self.assertFalse(lock1.locked()) - self.assertFalse(lock2.locked()) - - def test_double_acquire_single(self): - activated = collections.deque() - acquires = [] - - def run(): - start = now() - time.sleep(NAPPY_TIME) - end = now() - activated.append((start, end)) - - lock1 = threading.RLock() - lock2 = threading.RLock() - n_lock = lock_utils.MultiLock((lock1, lock2)) - with n_lock as gotten: - acquires.append(gotten) - run() - with n_lock as gotten: - acquires.append(gotten) - run() - run() - - self.assertTrue(all(acquires)) - self.assertEqual(2, len(acquires)) - for (start, end) in activated: - self.assertEqual(1, _find_overlaps(activated, start, end)) - - def test_double_acquire_many(self): - activated = collections.deque() - acquires = collections.deque() - n_lock = lock_utils.MultiLock((threading.RLock(), threading.RLock())) - - def critical_section(): - start = now() - time.sleep(NAPPY_TIME) - end = now() - activated.append((start, end)) - - def run(): - with n_lock as gotten: - acquires.append(gotten) - critical_section() - with n_lock as gotten: - acquires.append(gotten) - critical_section() - critical_section() - - threads = [] - for i in range(0, self.THREAD_COUNT): - t = threading_utils.daemon_thread(run) - threads.append(t) - t.start() - while threads: - t = threads.pop() - t.join() - - self.assertTrue(all(acquires)) - self.assertEqual(self.THREAD_COUNT * 2, len(acquires)) - self.assertEqual(self.THREAD_COUNT * 3, len(activated)) - for (start, end) in activated: - self.assertEqual(1, _find_overlaps(activated, start, end)) - - def test_no_acquire_release(self): - lock1 = threading.Lock() - lock2 = threading.Lock() - n_lock = lock_utils.MultiLock((lock1, lock2)) - self.assertRaises(threading.ThreadError, n_lock.release) diff --git a/taskflow/utils/lock_utils.py b/taskflow/utils/lock_utils.py deleted file mode 100644 index 7b1b026f..00000000 --- a/taskflow/utils/lock_utils.py +++ /dev/null @@ -1,207 +0,0 @@ -# Copyright 2011 OpenStack Foundation. -# 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. - -# This is a modified version of what was in oslo-incubator lockutils.py from -# commit 5039a610355e5265fb9fbd1f4023e8160750f32e but this one does not depend -# on oslo.cfg or the very large oslo-incubator oslo logging module (which also -# pulls in oslo.cfg) and is reduced to only what taskflow currently wants to -# use from that code. - -import contextlib -import threading - -import six - -from taskflow import logging -from taskflow.utils import misc - -LOG = logging.getLogger(__name__) - - -@contextlib.contextmanager -def try_lock(lock): - """Attempts to acquire a lock, and auto releases if acquired (on exit).""" - # NOTE(harlowja): the keyword argument for 'blocking' does not work - # in py2.x and only is fixed in py3.x (this adjustment is documented - # and/or debated in http://bugs.python.org/issue10789); so we'll just - # stick to the format that works in both (oddly the keyword argument - # works in py2.x but only with reentrant locks). - was_locked = lock.acquire(False) - try: - yield was_locked - finally: - if was_locked: - lock.release() - - -def locked(*args, **kwargs): - """A locking decorator. - - It will look for a provided attribute (typically a lock or a list - of locks) on the first argument of the function decorated (typically this - is the 'self' object) and before executing the decorated function it - activates the given lock or list of locks as a context manager, - automatically releasing that lock on exit. - - NOTE(harlowja): if no attribute name is provided then by default the - attribute named '_lock' is looked for (this attribute is expected to be - the lock/list of locks object/s) in the instance object this decorator - is attached to. - """ - - def decorator(f): - attr_name = kwargs.get('lock', '_lock') - - @six.wraps(f) - def wrapper(self, *args, **kwargs): - attr_value = getattr(self, attr_name) - if isinstance(attr_value, (tuple, list)): - lock = MultiLock(attr_value) - else: - lock = attr_value - with lock: - return f(self, *args, **kwargs) - - return wrapper - - # This is needed to handle when the decorator has args or the decorator - # doesn't have args, python is rather weird here... - if kwargs or not args: - return decorator - else: - if len(args) == 1: - return decorator(args[0]) - else: - return decorator - - -class MultiLock(object): - """A class which attempts to obtain & release many locks at once. - - It is typically useful as a context manager around many locks (instead of - having to nest individual lock context managers, which can become pretty - awkward looking). - - NOTE(harlowja): The locks that will be obtained will be in the order the - locks are given in the constructor, they will be acquired in order and - released in reverse order (so ordering matters). - """ - - def __init__(self, locks): - if not isinstance(locks, tuple): - locks = tuple(locks) - if len(locks) <= 0: - raise ValueError("Zero locks requested") - self._locks = locks - self._local = threading.local() - - @property - def _lock_stacks(self): - # This is weird, but this is how thread locals work (in that each - # thread will need to check if it has already created the attribute and - # if not then create it and set it to the thread local variable...) - # - # This isn't done in the constructor since the constructor is only - # activated by one of the many threads that could use this object, - # and that means that the attribute will only exist for that one - # thread. - try: - return self._local.stacks - except AttributeError: - self._local.stacks = [] - return self._local.stacks - - def __enter__(self): - return self.acquire() - - @property - def obtained(self): - """Returns how many locks were last acquired/obtained.""" - try: - return self._lock_stacks[-1] - except IndexError: - return 0 - - def __len__(self): - return len(self._locks) - - def acquire(self): - """This will attempt to acquire all the locks given in the constructor. - - If all the locks can not be acquired (and say only X of Y locks could - be acquired then this will return false to signify that not all the - locks were able to be acquired, you can later use the :attr:`.obtained` - property to determine how many were obtained during the last - acquisition attempt). - - NOTE(harlowja): When not all locks were acquired it is still required - to release since under partial acquisition the acquired locks - must still be released. For example if 4 out of 5 locks were acquired - this will return false, but the user **must** still release those - other 4 to avoid causing locking issues... - """ - gotten = 0 - for lock in self._locks: - try: - acked = lock.acquire() - except (threading.ThreadError, RuntimeError) as e: - # If we have already gotten some set of the desired locks - # make sure we track that and ensure that we later release them - # instead of losing them. - if gotten: - self._lock_stacks.append(gotten) - raise threading.ThreadError( - "Unable to acquire lock %s/%s due to '%s'" - % (gotten + 1, len(self._locks), e)) - else: - if not acked: - break - else: - gotten += 1 - if gotten: - self._lock_stacks.append(gotten) - return gotten == len(self._locks) - - def __exit__(self, type, value, traceback): - self.release() - - def release(self): - """Releases any past acquired locks (partial or otherwise).""" - height = len(self._lock_stacks) - if not height: - # Raise the same error type as the threading.Lock raises so that - # it matches the behavior of the built-in class (it's odd though - # that the threading.RLock raises a runtime error on this same - # method instead...) - raise threading.ThreadError('Release attempted on unlocked lock') - # Cleans off one level of the stack (this is done so that if there - # are multiple __enter__() and __exit__() pairs active that this will - # only remove one level (the last one), and not all levels... - for left in misc.countdown_iter(self._lock_stacks[-1]): - lock_idx = left - 1 - lock = self._locks[lock_idx] - try: - lock.release() - except (threading.ThreadError, RuntimeError) as e: - # Ensure that we adjust the lock stack under failure so that - # if release is attempted again that we do not try to release - # the locks we already released... - self._lock_stacks[-1] = left - raise threading.ThreadError( - "Unable to release lock %s/%s due to '%s'" - % (left, len(self._locks), e)) - # At the end only clear it off, so that under partial failure we don't - # lose any locks... - self._lock_stacks.pop() From 24752c204b418014e72c4ad9909200085155623d Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Mon, 11 May 2015 17:11:43 -0700 Subject: [PATCH 197/246] Use hash path lookup vs path finding When creating a fake symlink to a path just use hash lookup via `fetch_node` and handle that failing if the destination nodes does not exist vs. fetching the parent and linear searching for the existing target node (which gets slower as the parent node gets more children). This makes the preparing code using the little speed test helper on my not-very-fast box change to be the following: Old (preparing) - Took 29.724 seconds to run New (preparing) - Took 21.343 seconds to run Part of ongoing blueprint make-things-speedy Change-Id: I608b90ae58b4e4b6724b7f1bb8faebd118a1ec79 --- taskflow/persistence/backends/impl_memory.py | 23 ++++++++++---------- 1 file changed, 11 insertions(+), 12 deletions(-) diff --git a/taskflow/persistence/backends/impl_memory.py b/taskflow/persistence/backends/impl_memory.py index 55ea1455..2bc21662 100644 --- a/taskflow/persistence/backends/impl_memory.py +++ b/taskflow/persistence/backends/impl_memory.py @@ -220,14 +220,13 @@ class FakeFilesystem(object): """Link the destionation path to the source path.""" dest_path = self.normpath(dest_path) src_path = self.normpath(src_path) - dirname, basename = self.split(dest_path) - parent_node = self._fetch_node(dirname, normalized=True) - child_node = parent_node.find(basename, - only_direct=True, - include_self=False) - if child_node is None: - child_node = self._insert_child(parent_node, basename) - child_node.metadata['target'] = src_path + try: + dest_node = self._fetch_node(dest_path, normalized=True) + except exc.NotFound: + parent_path, basename = self.split(dest_path) + parent_node = self._fetch_node(parent_path, normalized=True) + dest_node = self._insert_child(parent_node, basename) + dest_node.metadata['target'] = src_path def __getitem__(self, path): return self._get_item(self.normpath(path)) @@ -236,11 +235,11 @@ class FakeFilesystem(object): path = self.normpath(path) value = self._copier(value) try: - item_node = self._fetch_node(path, normalized=True) - item_node.metadata.update(value=value) + node = self._fetch_node(path, normalized=True) + node.metadata.update(value=value) except exc.NotFound: - dirname, basename = self.split(path) - parent_node = self._fetch_node(dirname, normalized=True) + parent_path, basename = self.split(path) + parent_node = self._fetch_node(parent_path, normalized=True) self._insert_child(parent_node, basename, value=value) From 14f5d0a39596b712c43056e89df18590c83dea2f Mon Sep 17 00:00:00 2001 From: OpenStack Proposal Bot Date: Mon, 8 Jun 2015 21:20:50 +0000 Subject: [PATCH 198/246] Updated from global requirements Change-Id: I9da913a91c58b865518e446c9c4751ef128f7a3e --- requirements.txt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/requirements.txt b/requirements.txt index 11e96494..ad9b2027 100644 --- a/requirements.txt +++ b/requirements.txt @@ -23,7 +23,7 @@ fasteners>=0.5 # Apache-2.0 networkx>=1.8 # For contextlib new additions/compatibility for <= python 3.3 -contextlib2>=0.4.0 +contextlib2>=0.4.0 # PSF License # Used for backend storage engine loading. stevedore>=1.3.0 # Apache-2.0 @@ -32,7 +32,7 @@ stevedore>=1.3.0 # Apache-2.0 futures>=3.0 # Used for structured input validation -jsonschema>=2.0.0,<3.0.0 +jsonschema>=2.0.0,<3.0.0,!=2.5.0 # For common utilities oslo.utils>=1.4.0 # Apache-2.0 From 33e9ccc42566f4b666e04c1291bf01539288ffaf Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Tue, 21 Apr 2015 12:49:06 -0700 Subject: [PATCH 199/246] Use a lru cache to limit the size of the internal file cache Instead of having an unbounded internal file cache which will eventually absorb all memory of the running/containing python process have the size of that cache be limited by an optionally provided size (and have eviction be based on how recent a cached entry was used). Fixes bug 1458248 Change-Id: I5e59efc4edd51b05cfb1e67d3e7014e378e352aa --- requirements.txt | 3 + taskflow/persistence/backends/impl_dir.py | 15 ++++- .../unit/persistence/test_dir_persistence.py | 61 ++++++++++++++++--- 3 files changed, 68 insertions(+), 11 deletions(-) diff --git a/requirements.txt b/requirements.txt index 109bcff3..58e52959 100644 --- a/requirements.txt +++ b/requirements.txt @@ -35,5 +35,8 @@ jsonschema>=2.0.0,<3.0.0 oslo.utils>=1.4.0 # Apache-2.0 oslo.serialization>=1.4.0 # Apache-2.0 +# For lru caches and such +cachetools>=1.0.0 # MIT License + # For deprecation of things debtcollector>=0.3.0 # Apache-2.0 diff --git a/taskflow/persistence/backends/impl_dir.py b/taskflow/persistence/backends/impl_dir.py index 940b9c41..b6d1a27b 100644 --- a/taskflow/persistence/backends/impl_dir.py +++ b/taskflow/persistence/backends/impl_dir.py @@ -20,6 +20,7 @@ import errno import os import shutil +import cachetools import fasteners from oslo_serialization import jsonutils @@ -54,12 +55,22 @@ class DirBackend(path_based.PathBasedBackend): Example configuration:: conf = { - "path": "/tmp/taskflow", + "path": "/tmp/taskflow", # save data to this root directory + "max_cache_size": 1024, # keep up-to 1024 entries in memory } """ + def __init__(self, conf): super(DirBackend, self).__init__(conf) - self.file_cache = {} + max_cache_size = self._conf.get('max_cache_size') + if max_cache_size is not None: + max_cache_size = int(max_cache_size) + if max_cache_size < 1: + raise ValueError("Maximum cache size must be greater than" + " or equal to one") + self.file_cache = cachetools.LRUCache(max_cache_size) + else: + self.file_cache = {} self.encoding = self._conf.get('encoding', 'utf-8') if not self._path: raise ValueError("Empty path is disallowed") diff --git a/taskflow/tests/unit/persistence/test_dir_persistence.py b/taskflow/tests/unit/persistence/test_dir_persistence.py index 8c1171cf..7445145a 100644 --- a/taskflow/tests/unit/persistence/test_dir_persistence.py +++ b/taskflow/tests/unit/persistence/test_dir_persistence.py @@ -19,37 +19,80 @@ import os import shutil import tempfile +from oslo_utils import uuidutils +import testscenarios + +from taskflow import exceptions as exc from taskflow.persistence import backends from taskflow.persistence.backends import impl_dir +from taskflow.persistence import logbook from taskflow import test from taskflow.tests.unit.persistence import base -class DirPersistenceTest(test.TestCase, base.PersistenceTestMixin): +class DirPersistenceTest(testscenarios.TestWithScenarios, + test.TestCase, base.PersistenceTestMixin): + + scenarios = [ + ('no_cache', {'max_cache_size': None}), + ('one', {'max_cache_size': 1}), + ('tiny', {'max_cache_size': 256}), + ('medimum', {'max_cache_size': 512}), + ('large', {'max_cache_size': 1024}), + ] + def _get_connection(self): - conf = { - 'path': self.path, - } - return impl_dir.DirBackend(conf).get_connection() + return self.backend.get_connection() def setUp(self): super(DirPersistenceTest, self).setUp() self.path = tempfile.mkdtemp() - conn = self._get_connection() - conn.upgrade() + self.backend = impl_dir.DirBackend({ + 'path': self.path, + 'max_cache_size': self.max_cache_size, + }) + with contextlib.closing(self._get_connection()) as conn: + conn.upgrade() def tearDown(self): super(DirPersistenceTest, self).tearDown() - conn = self._get_connection() - conn.clear_all() if self.path and os.path.isdir(self.path): shutil.rmtree(self.path) self.path = None + self.backend = None def _check_backend(self, conf): with contextlib.closing(backends.fetch(conf)) as be: self.assertIsInstance(be, impl_dir.DirBackend) + def test_dir_backend_invalid_cache_size(self): + for invalid_size in [-1024, 0, -1]: + conf = { + 'path': self.path, + 'max_cache_size': invalid_size, + } + self.assertRaises(ValueError, impl_dir.DirBackend, conf) + + def test_dir_backend_cache_overfill(self): + if self.max_cache_size is not None: + # Ensure cache never goes past the desired max size... + books_ids_made = [] + with contextlib.closing(self._get_connection()) as conn: + for i in range(0, int(1.5 * self.max_cache_size)): + lb_name = 'book-%s' % (i) + lb_id = uuidutils.generate_uuid() + lb = logbook.LogBook(name=lb_name, uuid=lb_id) + self.assertRaises(exc.NotFound, conn.get_logbook, lb_id) + conn.save_logbook(lb) + books_ids_made.append(lb_id) + self.assertLessEqual(self.backend.file_cache.currsize, + self.max_cache_size) + # Also ensure that we can still read all created books... + with contextlib.closing(self._get_connection()) as conn: + for lb_id in books_ids_made: + lb = conn.get_logbook(lb_id) + self.assertIsNotNone(lb) + def test_dir_backend_entry_point(self): self._check_backend(dict(connection='dir:', path=self.path)) From deaf61caac32961266e2d0644ad4312ea2dcd867 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Sat, 23 May 2015 13:50:43 -0700 Subject: [PATCH 200/246] Make the default file encoding a class constant with a docstring Having this exposed makes it easier to know what it is being used for and what the default is (and also makes it show up in generated docs, so people can read all about it). Change-Id: I2e0f85d9c087d220671e2dbf82d497677c462a9e --- taskflow/persistence/backends/impl_dir.py | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/taskflow/persistence/backends/impl_dir.py b/taskflow/persistence/backends/impl_dir.py index b6d1a27b..1047d671 100644 --- a/taskflow/persistence/backends/impl_dir.py +++ b/taskflow/persistence/backends/impl_dir.py @@ -60,6 +60,12 @@ class DirBackend(path_based.PathBasedBackend): } """ + DEFAULT_FILE_ENCODING = 'utf-8' + """ + Default encoding used when decoding or encoding files into or from + text/unicode into binary or binary into text/unicode. + """ + def __init__(self, conf): super(DirBackend, self).__init__(conf) max_cache_size = self._conf.get('max_cache_size') @@ -71,7 +77,7 @@ class DirBackend(path_based.PathBasedBackend): self.file_cache = cachetools.LRUCache(max_cache_size) else: self.file_cache = {} - self.encoding = self._conf.get('encoding', 'utf-8') + self.encoding = self._conf.get('encoding', self.DEFAULT_FILE_ENCODING) if not self._path: raise ValueError("Empty path is disallowed") self._path = os.path.abspath(self._path) From 61a4f978b37263472bce3d087dc591c46d17751c Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 10 Jun 2015 14:30:46 -0700 Subject: [PATCH 201/246] Remove example not tested This example is not automatically tested and is better served by other examples involving conductors, jobs, and such so it doesn't seem to have much value to retain it. Change-Id: Idc622bb6e6098507c99758614a96feb6a8a9b0c8 --- taskflow/examples/job_board_no_test.py | 171 ------------------------- taskflow/tests/test_examples.py | 2 +- 2 files changed, 1 insertion(+), 172 deletions(-) delete mode 100644 taskflow/examples/job_board_no_test.py diff --git a/taskflow/examples/job_board_no_test.py b/taskflow/examples/job_board_no_test.py deleted file mode 100644 index d37c96a7..00000000 --- a/taskflow/examples/job_board_no_test.py +++ /dev/null @@ -1,171 +0,0 @@ -# -*- encoding: utf-8 -*- -# -# Copyright © 2013 eNovance -# -# Authors: Dan Krause -# Cyril Roelandt -# -# 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. - -# This example shows how to use the job board feature. -# -# Let's start by creating some jobs: -# $ python job_board_no_test.py create my-board my-job '{}' -# $ python job_board_no_test.py create my-board my-job '{"foo": "bar"}' -# $ python job_board_no_test.py create my-board my-job '{"foo": "baz"}' -# $ python job_board_no_test.py create my-board my-job '{"foo": "barbaz"}' -# -# Make sure they were registered: -# $ python job_board_no_test.py list my-board -# 7277181a-1f83-473d-8233-f361615bae9e - {} -# 84a396e8-d02e-450d-8566-d93cb68550c0 - {u'foo': u'bar'} -# 4d355d6a-2c72-44a2-a558-19ae52e8ae2c - {u'foo': u'baz'} -# cd9aae2c-fd64-416d-8ba0-426fa8e3d59c - {u'foo': u'barbaz'} -# -# Perform one job: -# $ python job_board_no_test.py consume my-board \ -# 84a396e8-d02e-450d-8566-d93cb68550c0 -# Performing job 84a396e8-d02e-450d-8566-d93cb68550c0 with args \ -# {u'foo': u'bar'} -# $ python job_board_no_test.py list my-board -# 7277181a-1f83-473d-8233-f361615bae9e - {} -# 4d355d6a-2c72-44a2-a558-19ae52e8ae2c - {u'foo': u'baz'} -# cd9aae2c-fd64-416d-8ba0-426fa8e3d59c - {u'foo': u'barbaz'} -# -# Delete a job: -# $ python job_board_no_test.py delete my-board \ -# cd9aae2c-fd64-416d-8ba0-426fa8e3d59c -# $ python job_board_no_test.py list my-board -# 7277181a-1f83-473d-8233-f361615bae9e - {} -# 4d355d6a-2c72-44a2-a558-19ae52e8ae2c - {u'foo': u'baz'} -# -# Delete all the remaining jobs -# $ python job_board_no_test.py clear my-board -# $ python job_board_no_test.py list my-board -# $ - -import argparse -import contextlib -import json -import os -import sys -import tempfile - -import taskflow.jobs.backends as job_backends -from taskflow.persistence import logbook - -import example_utils # noqa - - -@contextlib.contextmanager -def jobboard(*args, **kwargs): - jb = job_backends.fetch(*args, **kwargs) - jb.connect() - yield jb - jb.close() - - -conf = { - 'board': 'zookeeper', - 'hosts': ['127.0.0.1:2181'] -} - - -def consume_job(args): - def perform_job(job): - print("Performing job %s with args %s" % (job.uuid, job.details)) - - with jobboard(args.board_name, conf) as jb: - for job in jb.iterjobs(ensure_fresh=True): - if job.uuid == args.job_uuid: - jb.claim(job, "test-client") - perform_job(job) - jb.consume(job, "test-client") - - -def clear_jobs(args): - with jobboard(args.board_name, conf) as jb: - for job in jb.iterjobs(ensure_fresh=True): - jb.claim(job, "test-client") - jb.consume(job, "test-client") - - -def create_job(args): - store = json.loads(args.details) - book = logbook.LogBook(args.job_name) - if example_utils.SQLALCHEMY_AVAILABLE: - persist_path = os.path.join(tempfile.gettempdir(), "persisting.db") - backend_uri = "sqlite:///%s" % (persist_path) - else: - persist_path = os.path.join(tempfile.gettempdir(), "persisting") - backend_uri = "file:///%s" % (persist_path) - with example_utils.get_backend(backend_uri) as backend: - backend.get_connection().save_logbook(book) - with jobboard(args.board_name, conf, persistence=backend) as jb: - jb.post(args.job_name, book, details=store) - - -def list_jobs(args): - with jobboard(args.board_name, conf) as jb: - for job in jb.iterjobs(ensure_fresh=True): - print("%s - %s" % (job.uuid, job.details)) - - -def delete_job(args): - with jobboard(args.board_name, conf) as jb: - for job in jb.iterjobs(ensure_fresh=True): - if job.uuid == args.job_uuid: - jb.claim(job, "test-client") - jb.consume(job, "test-client") - - -def main(argv): - parser = argparse.ArgumentParser() - subparsers = parser.add_subparsers(title='subcommands', - description='valid subcommands', - help='additional help') - - # Consume command - parser_consume = subparsers.add_parser('consume') - parser_consume.add_argument('board_name') - parser_consume.add_argument('job_uuid') - parser_consume.set_defaults(func=consume_job) - - # Clear command - parser_consume = subparsers.add_parser('clear') - parser_consume.add_argument('board_name') - parser_consume.set_defaults(func=clear_jobs) - - # Create command - parser_create = subparsers.add_parser('create') - parser_create.add_argument('board_name') - parser_create.add_argument('job_name') - parser_create.add_argument('details') - parser_create.set_defaults(func=create_job) - - # Delete command - parser_delete = subparsers.add_parser('delete') - parser_delete.add_argument('board_name') - parser_delete.add_argument('job_uuid') - parser_delete.set_defaults(func=delete_job) - - # List command - parser_list = subparsers.add_parser('list') - parser_list.add_argument('board_name') - parser_list.set_defaults(func=list_jobs) - - args = parser.parse_args(argv) - args.func(args) - -if __name__ == '__main__': - main(sys.argv[1:]) diff --git a/taskflow/tests/test_examples.py b/taskflow/tests/test_examples.py index a7a297c3..ce795dd0 100644 --- a/taskflow/tests/test_examples.py +++ b/taskflow/tests/test_examples.py @@ -95,7 +95,7 @@ def iter_examples(): name, ext = os.path.splitext(filename) if ext != ".py": continue - if not any(name.endswith(i) for i in ("utils", "no_test")): + if not name.endswith('utils'): safe_name = safe_filename(name) if safe_name: yield name, safe_name From 4b28b32778de58ff6b893bba1438c54c2092e5dc Mon Sep 17 00:00:00 2001 From: OpenStack Proposal Bot Date: Thu, 11 Jun 2015 00:49:02 +0000 Subject: [PATCH 202/246] Updated from global requirements Change-Id: Ib8345d536a1d4895d7f91ad415545c3573b09e10 --- requirements.txt | 2 +- test-requirements.txt | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/requirements.txt b/requirements.txt index 43efdec5..6bfcef79 100644 --- a/requirements.txt +++ b/requirements.txt @@ -26,7 +26,7 @@ networkx>=1.8 contextlib2>=0.4.0 # PSF License # Used for backend storage engine loading. -stevedore>=1.3.0 # Apache-2.0 +stevedore>=1.5.0 # Apache-2.0 # Backport for concurrent.futures which exists in 3.2+ futures>=3.0 diff --git a/test-requirements.txt b/test-requirements.txt index 16fc8b2b..f25f6d40 100644 --- a/test-requirements.txt +++ b/test-requirements.txt @@ -16,7 +16,7 @@ zake>=0.1.6 # Apache-2.0 kazoo>=1.3.1,!=2.1 # Used for testing database persistence backends. -SQLAlchemy>=0.9.7,<=0.9.99 +SQLAlchemy>=0.9.7,<1.1.0 alembic>=0.7.2 psycopg2 PyMySQL>=0.6.2 # MIT License From ad296327136015b03f5e3d95234e79972f99d9f7 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 10 Jun 2015 18:10:57 -0700 Subject: [PATCH 203/246] Use a class constant for the default path based backend path When no path is provided to a path based backend via configuration use a class constant to provide the default, and override this in backends that support providing defaults. Change-Id: I0a6c88398403a162b113e34abe7e56821d1f02bc --- taskflow/persistence/backends/impl_memory.py | 6 ++++-- taskflow/persistence/backends/impl_zookeeper.py | 6 ++++-- taskflow/persistence/path_based.py | 5 +++++ 3 files changed, 13 insertions(+), 4 deletions(-) diff --git a/taskflow/persistence/backends/impl_memory.py b/taskflow/persistence/backends/impl_memory.py index 43207b81..c18b4715 100644 --- a/taskflow/persistence/backends/impl_memory.py +++ b/taskflow/persistence/backends/impl_memory.py @@ -309,10 +309,12 @@ class MemoryBackend(path_based.PathBasedBackend): guarantee that there will be no inter-thread race conditions when writing and reading by using a read/write locks. """ + + #: Default path used when none is provided. + DEFAULT_PATH = pp.sep + def __init__(self, conf=None): super(MemoryBackend, self).__init__(conf) - if self._path is None: - self._path = pp.sep self.memory = FakeFilesystem(deep_copy=self._conf.get('deep_copy', True)) self.lock = fasteners.ReaderWriterLock() diff --git a/taskflow/persistence/backends/impl_zookeeper.py b/taskflow/persistence/backends/impl_zookeeper.py index 279e0c66..687f103d 100644 --- a/taskflow/persistence/backends/impl_zookeeper.py +++ b/taskflow/persistence/backends/impl_zookeeper.py @@ -40,10 +40,12 @@ class ZkBackend(path_based.PathBasedBackend): "path": "/taskflow", } """ + + #: Default path used when none is provided. + DEFAULT_PATH = '/taskflow' + def __init__(self, conf, client=None): super(ZkBackend, self).__init__(conf) - if not self._path: - self._path = '/taskflow' if not paths.isabs(self._path): raise ValueError("Zookeeper path must be absolute") if client is not None: diff --git a/taskflow/persistence/path_based.py b/taskflow/persistence/path_based.py index 6c065df0..0d729978 100644 --- a/taskflow/persistence/path_based.py +++ b/taskflow/persistence/path_based.py @@ -34,9 +34,14 @@ class PathBasedBackend(base.Backend): the contents of those objects for later reading and writing. """ + #: Default path used when none is provided. + DEFAULT_PATH = None + def __init__(self, conf): super(PathBasedBackend, self).__init__(conf) self._path = self._conf.get('path', None) + if not self._path: + self._path = self.DEFAULT_PATH @property def path(self): From 4707bd91b64d9f7cbee89db3fca4aa00a453419c Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 10 Jun 2015 20:00:49 -0700 Subject: [PATCH 204/246] Use sphinx deprecated docstring markup Along with using the `debtcollector` library we should also be marking the docstrings of deprecated functions/methods/properties with the appropriate deprecated information so that the generated docs also show what is deprecated. Change-Id: I8a1960731fd53434bf04d22cc6ff208998adb4f3 --- taskflow/conductors/backends/impl_blocking.py | 10 ++++--- taskflow/engines/base.py | 10 ++++++- taskflow/engines/helpers.py | 28 +++++++++++++++++++ taskflow/listeners/base.py | 9 ++++++ 4 files changed, 52 insertions(+), 5 deletions(-) diff --git a/taskflow/conductors/backends/impl_blocking.py b/taskflow/conductors/backends/impl_blocking.py index fb8a3c3a..895f7d31 100644 --- a/taskflow/conductors/backends/impl_blocking.py +++ b/taskflow/conductors/backends/impl_blocking.py @@ -81,10 +81,12 @@ class BlockingConductor(base.Conductor): The method returns immediately regardless of whether the conductor has been stopped. - :param timeout: This parameter is **deprecated** and is present for - backward compatibility **only**. In order to wait for - the conductor to gracefully shut down, :meth:`wait` - should be used instead. + .. deprecated:: 0.8 + + The ``timeout`` parameter is **deprecated** and is present for + backward compatibility **only**. In order to wait for the + conductor to gracefully shut down, :py:meth:`wait` should be used + instead. """ self._wait_timeout.interrupt() diff --git a/taskflow/engines/base.py b/taskflow/engines/base.py index 824d9087..f41ae583 100644 --- a/taskflow/engines/base.py +++ b/taskflow/engines/base.py @@ -57,7 +57,15 @@ class Engine(object): @moves.moved_property('atom_notifier', version="0.6", removal_version="?") def task_notifier(self): - """The task notifier.""" + """The task notifier. + + .. deprecated:: 0.6 + + The property is **deprecated** and is present for + backward compatibility **only**. In order to access this + property going forward the :py:attr:`.atom_notifier` should + be used instead. + """ return self._atom_notifier @property diff --git a/taskflow/engines/helpers.py b/taskflow/engines/helpers.py index 2646eea1..a9171e46 100644 --- a/taskflow/engines/helpers.py +++ b/taskflow/engines/helpers.py @@ -146,6 +146,13 @@ def load(flow, store=None, flow_detail=None, book=None, :py:func:`~taskflow.persistence.backends.fetch` to obtain a viable backend. + .. deprecated:: 0.6 + + The ``engine_conf`` argument is **deprecated** and is present + for backward compatibility **only**. In order to provide this + argument going forward the ``engine`` string (or URI) argument + should be used instead. + :param flow: flow to load :param store: dict -- data to put to storage to satisfy flow requirements :param flow_detail: FlowDetail that holds the state of the flow (if one is @@ -199,6 +206,13 @@ def run(flow, store=None, flow_detail=None, book=None, The arguments are interpreted as for :func:`load() `. + .. deprecated:: 0.6 + + The ``engine_conf`` argument is **deprecated** and is present + for backward compatibility **only**. In order to provide this + argument going forward the ``engine`` string (or URI) argument + should be used instead. + :returns: dictionary of all named results (see :py:meth:`~.taskflow.storage.Storage.fetch_all`) """ @@ -264,6 +278,13 @@ def load_from_factory(flow_factory, factory_args=None, factory_kwargs=None, Further arguments are interpreted as for :func:`load() `. + .. deprecated:: 0.6 + + The ``engine_conf`` argument is **deprecated** and is present + for backward compatibility **only**. In order to provide this + argument going forward the ``engine`` string (or URI) argument + should be used instead. + :returns: engine """ @@ -324,6 +345,13 @@ def load_from_detail(flow_detail, store=None, engine_conf=None, backend=None, Further arguments are interpreted as for :func:`load() `. + .. deprecated:: 0.6 + + The ``engine_conf`` argument is **deprecated** and is present + for backward compatibility **only**. In order to provide this + argument going forward the ``engine`` string (or URI) argument + should be used instead. + :returns: engine """ flow = flow_from_detail(flow_detail) diff --git a/taskflow/listeners/base.py b/taskflow/listeners/base.py index 75b0db09..a83cd79b 100644 --- a/taskflow/listeners/base.py +++ b/taskflow/listeners/base.py @@ -215,6 +215,15 @@ class LoggingBase(moves.moved_class(DumpingListener, 'LoggingBase', __name__, version="0.6", removal_version="?")): + """Legacy logging base. + + .. deprecated:: 0.6 + + This class is **deprecated** and is present for backward + compatibility **only**, its replacement + :py:class:`.DumpingListener` should be used going forward. + """ + def _dump(self, message, *args, **kwargs): self._log(message, *args, **kwargs) From 2c9e2146bcf88349a54057d3d8bb8a732a0d4883 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 10 Jun 2015 21:50:41 -0700 Subject: [PATCH 205/246] A few jobboard documentation tweaks Change-Id: I67fcfcb9e721d3293b60f04f7bebac3723b1a3ae --- doc/source/jobs.rst | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/doc/source/jobs.rst b/doc/source/jobs.rst index cc7f6f10..7dd73063 100644 --- a/doc/source/jobs.rst +++ b/doc/source/jobs.rst @@ -62,7 +62,8 @@ Features the previously partially completed work or begin initial work to ensure that the workflow as a whole progresses (where progressing implies transitioning through the workflow :doc:`patterns ` and - :doc:`atoms ` and completing their associated state transitions). + :doc:`atoms ` and completing their associated + :doc:`states ` transitions). - Atomic transfer and single ownership @@ -94,11 +95,12 @@ Features Usage ===== -All engines are mere classes that implement same interface, and of course it is -possible to import them and create their instances just like with any classes -in Python. But the easier (and recommended) way for creating jobboards is by -using the :py:meth:`fetch() ` function which uses -entrypoints (internally using `stevedore`_) to fetch and configure your backend +All jobboards are mere classes that implement same interface, and of course +it is possible to import them and create instances of them just like with any +other class in Python. But the easier (and recommended) way for creating +jobboards is by using the :py:meth:`fetch() ` +function which uses entrypoints (internally using `stevedore`_) to fetch and +configure your backend. Using this function the typical creation of a jobboard (and an example posting of a job) might look like: From 3d4bbb92d89b080b8accf026751834506068f375 Mon Sep 17 00:00:00 2001 From: Min Pae Date: Wed, 10 Jun 2015 02:04:05 -0700 Subject: [PATCH 206/246] Adding a revert_all option to retry controllers When a retry controller is added to a nested flow, with the exception of AlwaysRetryAll, existing retry controllers do not cause a revert of predecessor tasks in the surrounding flow. This is due to the fact that these retry controllers always return RETRY. The revert_all option, when set True, will result in the retry controller returning REVERT_ALL rather than REVERT, which will cause predecessor tasks in the surrounding flow to also be REVERTed. Change-Id: I2d867b2d05e8559121ec48fd7249f15078450532 --- taskflow/retry.py | 31 +++- taskflow/tests/unit/test_retries.py | 251 ++++++++++++++++++++++++++++ 2 files changed, 277 insertions(+), 5 deletions(-) diff --git a/taskflow/retry.py b/taskflow/retry.py index b7135a92..3015c79d 100644 --- a/taskflow/retry.py +++ b/taskflow/retry.py @@ -241,15 +241,20 @@ class Times(Retry): """Retries subflow given number of times. Returns attempt number.""" def __init__(self, attempts=1, name=None, provides=None, requires=None, - auto_extract=True, rebind=None): + auto_extract=True, rebind=None, revert_all=False): super(Times, self).__init__(name, provides, requires, auto_extract, rebind) self._attempts = attempts + if revert_all: + self._revert_action = REVERT_ALL + else: + self._revert_action = REVERT + def on_failure(self, history, *args, **kwargs): if len(history) < self._attempts: return RETRY - return REVERT + return self._revert_action def execute(self, history, *args, **kwargs): return len(history) + 1 @@ -258,6 +263,16 @@ class Times(Retry): class ForEachBase(Retry): """Base class for retries that iterate over a given collection.""" + def __init__(self, name=None, provides=None, requires=None, + auto_extract=True, rebind=None, revert_all=False): + super(ForEachBase, self).__init__(name, provides, requires, + auto_extract, rebind) + + if revert_all: + self._revert_action = REVERT_ALL + else: + self._revert_action = REVERT + def _get_next_value(self, values, history): # Fetches the next resolution result to try, removes overlapping # entries with what has already been tried and then returns the first @@ -272,7 +287,7 @@ class ForEachBase(Retry): try: self._get_next_value(values, history) except exc.NotFound: - return REVERT + return self._revert_action else: return RETRY @@ -285,9 +300,9 @@ class ForEach(ForEachBase): """ def __init__(self, values, name=None, provides=None, requires=None, - auto_extract=True, rebind=None): + auto_extract=True, rebind=None, revert_all=False): super(ForEach, self).__init__(name, provides, requires, - auto_extract, rebind) + auto_extract, rebind, revert_all) self._values = values def on_failure(self, history, *args, **kwargs): @@ -307,6 +322,12 @@ class ParameterizedForEach(ForEachBase): each try. """ + def __init__(self, name=None, provides=None, requires=None, + auto_extract=True, rebind=None, revert_all=False): + super(ParameterizedForEach, self).__init__(name, provides, requires, + auto_extract, rebind, + revert_all) + def on_failure(self, values, history, *args, **kwargs): return self._on_failure(values, history) diff --git a/taskflow/tests/unit/test_retries.py b/taskflow/tests/unit/test_retries.py index edcc6d8b..ddb256b0 100644 --- a/taskflow/tests/unit/test_retries.py +++ b/taskflow/tests/unit/test_retries.py @@ -336,6 +336,66 @@ class RetryTest(utils.EngineTestBase): 'flow-1.f SUCCESS'] self.assertEqual(expected, capturer.values) + def test_nested_flow_with_retry_revert(self): + retry1 = retry.Times(0, 'r1', provides='x2') + flow = lf.Flow('flow-1').add( + utils.ProgressingTask("task1"), + lf.Flow('flow-2', retry1).add( + utils.ConditionalTask("task2", inject={'x': 1})) + ) + engine = self._make_engine(flow) + engine.storage.inject({'y': 2}) + with utils.CaptureListener(engine) as capturer: + try: + engine.run() + except Exception: + pass + self.assertEqual(engine.storage.fetch_all(), {'y': 2}) + expected = ['flow-1.f RUNNING', + 'task1.t RUNNING', + 'task1.t SUCCESS(5)', + 'r1.r RUNNING', + 'r1.r SUCCESS(1)', + 'task2.t RUNNING', + 'task2.t FAILURE(Failure: RuntimeError: Woot!)', + 'task2.t REVERTING', + 'task2.t REVERTED', + 'r1.r REVERTING', + 'r1.r REVERTED', + 'flow-1.f REVERTED'] + self.assertEqual(expected, capturer.values) + + def test_nested_flow_with_retry_revert_all(self): + retry1 = retry.Times(0, 'r1', provides='x2', revert_all=True) + flow = lf.Flow('flow-1').add( + utils.ProgressingTask("task1"), + lf.Flow('flow-2', retry1).add( + utils.ConditionalTask("task2", inject={'x': 1})) + ) + engine = self._make_engine(flow) + engine.storage.inject({'y': 2}) + with utils.CaptureListener(engine) as capturer: + try: + engine.run() + except Exception: + pass + self.assertEqual(engine.storage.fetch_all(), {'y': 2}) + expected = ['flow-1.f RUNNING', + 'task1.t RUNNING', + 'task1.t SUCCESS(5)', + 'r1.r RUNNING', + 'r1.r SUCCESS(1)', + 'task2.t RUNNING', + 'task2.t FAILURE(Failure: RuntimeError: Woot!)', + 'task2.t REVERTING', + 'task2.t REVERTED', + 'r1.r REVERTING', + 'r1.r REVERTED', + 'task1.t REVERTING', + 'task1.t REVERTED', + 'flow-1.f REVERTED'] + self.assertEqual(expected, capturer.values) + def test_revert_all_retry(self): flow = lf.Flow('flow-1', retry.Times(3, 'r1', provides='x')).add( utils.ProgressingTask("task1"), @@ -594,6 +654,108 @@ class RetryTest(utils.EngineTestBase): 'flow-1.f REVERTED'] self.assertItemsEqual(capturer.values, expected) + def test_nested_for_each_revert(self): + collection = [3, 2, 3, 5] + retry1 = retry.ForEach(collection, 'r1', provides='x') + flow = lf.Flow('flow-1').add( + utils.ProgressingTask("task1"), + lf.Flow('flow-2', retry1).add( + utils.FailingTaskWithOneArg('task2') + ) + ) + engine = self._make_engine(flow) + with utils.CaptureListener(engine) as capturer: + self.assertRaisesRegexp(RuntimeError, '^Woot', engine.run) + expected = ['flow-1.f RUNNING', + 'task1.t RUNNING', + 'task1.t SUCCESS(5)', + 'r1.r RUNNING', + 'r1.r SUCCESS(3)', + 'task2.t RUNNING', + 'task2.t FAILURE(Failure: RuntimeError: Woot with 3)', + 'task2.t REVERTING', + 'task2.t REVERTED', + 'r1.r RETRYING', + 'task2.t PENDING', + 'r1.r RUNNING', + 'r1.r SUCCESS(2)', + 'task2.t RUNNING', + 'task2.t FAILURE(Failure: RuntimeError: Woot with 2)', + 'task2.t REVERTING', + 'task2.t REVERTED', + 'r1.r RETRYING', + 'task2.t PENDING', + 'r1.r RUNNING', + 'r1.r SUCCESS(3)', + 'task2.t RUNNING', + 'task2.t FAILURE(Failure: RuntimeError: Woot with 3)', + 'task2.t REVERTING', + 'task2.t REVERTED', + 'r1.r RETRYING', + 'task2.t PENDING', + 'r1.r RUNNING', + 'r1.r SUCCESS(5)', + 'task2.t RUNNING', + 'task2.t FAILURE(Failure: RuntimeError: Woot with 5)', + 'task2.t REVERTING', + 'task2.t REVERTED', + 'r1.r REVERTING', + 'r1.r REVERTED', + 'flow-1.f REVERTED'] + self.assertEqual(expected, capturer.values) + + def test_nested_for_each_revert_all(self): + collection = [3, 2, 3, 5] + retry1 = retry.ForEach(collection, 'r1', provides='x', revert_all=True) + flow = lf.Flow('flow-1').add( + utils.ProgressingTask("task1"), + lf.Flow('flow-2', retry1).add( + utils.FailingTaskWithOneArg('task2') + ) + ) + engine = self._make_engine(flow) + with utils.CaptureListener(engine) as capturer: + self.assertRaisesRegexp(RuntimeError, '^Woot', engine.run) + expected = ['flow-1.f RUNNING', + 'task1.t RUNNING', + 'task1.t SUCCESS(5)', + 'r1.r RUNNING', + 'r1.r SUCCESS(3)', + 'task2.t RUNNING', + 'task2.t FAILURE(Failure: RuntimeError: Woot with 3)', + 'task2.t REVERTING', + 'task2.t REVERTED', + 'r1.r RETRYING', + 'task2.t PENDING', + 'r1.r RUNNING', + 'r1.r SUCCESS(2)', + 'task2.t RUNNING', + 'task2.t FAILURE(Failure: RuntimeError: Woot with 2)', + 'task2.t REVERTING', + 'task2.t REVERTED', + 'r1.r RETRYING', + 'task2.t PENDING', + 'r1.r RUNNING', + 'r1.r SUCCESS(3)', + 'task2.t RUNNING', + 'task2.t FAILURE(Failure: RuntimeError: Woot with 3)', + 'task2.t REVERTING', + 'task2.t REVERTED', + 'r1.r RETRYING', + 'task2.t PENDING', + 'r1.r RUNNING', + 'r1.r SUCCESS(5)', + 'task2.t RUNNING', + 'task2.t FAILURE(Failure: RuntimeError: Woot with 5)', + 'task2.t REVERTING', + 'task2.t REVERTED', + 'r1.r REVERTING', + 'r1.r REVERTED', + 'task1.t REVERTING', + 'task1.t REVERTED', + 'flow-1.f REVERTED'] + self.assertEqual(expected, capturer.values) + def test_for_each_empty_collection(self): values = [] retry1 = retry.ForEach(values, 'r1', provides='x') @@ -674,6 +836,95 @@ class RetryTest(utils.EngineTestBase): 'flow-1.f REVERTED'] self.assertItemsEqual(capturer.values, expected) + def test_nested_parameterized_for_each_revert(self): + values = [3, 2, 5] + retry1 = retry.ParameterizedForEach('r1', provides='x') + flow = lf.Flow('flow-1').add( + utils.ProgressingTask('task-1'), + lf.Flow('flow-2', retry1).add( + utils.FailingTaskWithOneArg('task-2') + ) + ) + engine = self._make_engine(flow) + engine.storage.inject({'values': values, 'y': 1}) + with utils.CaptureListener(engine) as capturer: + self.assertRaisesRegexp(RuntimeError, '^Woot', engine.run) + expected = ['flow-1.f RUNNING', + 'task-1.t RUNNING', + 'task-1.t SUCCESS(5)', + 'r1.r RUNNING', + 'r1.r SUCCESS(3)', + 'task-2.t RUNNING', + 'task-2.t FAILURE(Failure: RuntimeError: Woot with 3)', + 'task-2.t REVERTING', + 'task-2.t REVERTED', + 'r1.r RETRYING', + 'task-2.t PENDING', + 'r1.r RUNNING', + 'r1.r SUCCESS(2)', + 'task-2.t RUNNING', + 'task-2.t FAILURE(Failure: RuntimeError: Woot with 2)', + 'task-2.t REVERTING', + 'task-2.t REVERTED', + 'r1.r RETRYING', + 'task-2.t PENDING', + 'r1.r RUNNING', + 'r1.r SUCCESS(5)', + 'task-2.t RUNNING', + 'task-2.t FAILURE(Failure: RuntimeError: Woot with 5)', + 'task-2.t REVERTING', + 'task-2.t REVERTED', + 'r1.r REVERTING', + 'r1.r REVERTED', + 'flow-1.f REVERTED'] + self.assertEqual(expected, capturer.values) + + def test_nested_parameterized_for_each_revert_all(self): + values = [3, 2, 5] + retry1 = retry.ParameterizedForEach('r1', provides='x', + revert_all=True) + flow = lf.Flow('flow-1').add( + utils.ProgressingTask('task-1'), + lf.Flow('flow-2', retry1).add( + utils.FailingTaskWithOneArg('task-2') + ) + ) + engine = self._make_engine(flow) + engine.storage.inject({'values': values, 'y': 1}) + with utils.CaptureListener(engine) as capturer: + self.assertRaisesRegexp(RuntimeError, '^Woot', engine.run) + expected = ['flow-1.f RUNNING', + 'task-1.t RUNNING', + 'task-1.t SUCCESS(5)', + 'r1.r RUNNING', + 'r1.r SUCCESS(3)', + 'task-2.t RUNNING', + 'task-2.t FAILURE(Failure: RuntimeError: Woot with 3)', + 'task-2.t REVERTING', + 'task-2.t REVERTED', + 'r1.r RETRYING', + 'task-2.t PENDING', + 'r1.r RUNNING', + 'r1.r SUCCESS(2)', + 'task-2.t RUNNING', + 'task-2.t FAILURE(Failure: RuntimeError: Woot with 2)', + 'task-2.t REVERTING', + 'task-2.t REVERTED', + 'r1.r RETRYING', + 'task-2.t PENDING', + 'r1.r RUNNING', + 'r1.r SUCCESS(5)', + 'task-2.t RUNNING', + 'task-2.t FAILURE(Failure: RuntimeError: Woot with 5)', + 'task-2.t REVERTING', + 'task-2.t REVERTED', + 'r1.r REVERTING', + 'r1.r REVERTED', + 'task-1.t REVERTING', + 'task-1.t REVERTED', + 'flow-1.f REVERTED'] + self.assertEqual(expected, capturer.values) + def test_parameterized_for_each_empty_collection(self): values = [] retry1 = retry.ParameterizedForEach('r1', provides='x') From 28eafa6d69f811c146af32b93d9f4f666a0faa38 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Thu, 11 Jun 2015 20:45:34 -0700 Subject: [PATCH 207/246] Switch badges from 'pypip.in' to 'shields.io' The badge hosting at 'pypip.in' has been broken for a while now, it was supposed to be fixed in the following PR(s): - https://github.com/badges/pypipins/issues/39 - https://github.com/badges/pypipins/issues/38 - https://github.com/badges/pypipins/issues/37 That does not appear to have actually happened and/or is not getting addressed, so switch to a more reliable badge hosting site that provides the same badge information. Change-Id: I352fbb83af64becc6b8632a88de9af1fa66ee4a3 --- README.rst | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/README.rst b/README.rst index 70ff715a..9023dd67 100644 --- a/README.rst +++ b/README.rst @@ -1,11 +1,11 @@ TaskFlow ======== -.. image:: https://pypip.in/version/taskflow/badge.svg +.. image:: https://img.shields.io/pypi/v/taskflow.svg :target: https://pypi.python.org/pypi/taskflow/ :alt: Latest Version -.. image:: https://pypip.in/download/taskflow/badge.svg?period=month +.. image:: https://img.shields.io/pypi/dm/taskflow.svg :target: https://pypi.python.org/pypi/taskflow/ :alt: Downloads From 56ca3541236ee71d78d27ff4cefade7d05c21be0 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Fri, 12 Jun 2015 09:04:47 -0700 Subject: [PATCH 208/246] Handy access to INFO level It appears that the INFO log level does not have itself exposed for handy access, so fix that. Change-Id: I74582e645ad883f33b08a460756b84a0a2d25b51 --- taskflow/logging.py | 1 + 1 file changed, 1 insertion(+) diff --git a/taskflow/logging.py b/taskflow/logging.py index 0ce457eb..0ededf7f 100644 --- a/taskflow/logging.py +++ b/taskflow/logging.py @@ -32,6 +32,7 @@ CRITICAL = logging.CRITICAL DEBUG = logging.DEBUG ERROR = logging.ERROR FATAL = logging.FATAL +INFO = logging.INFO NOTSET = logging.NOTSET WARN = logging.WARN WARNING = logging.WARNING From a26efda03915c272ce7bef57774d548cbd289f6e Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Fri, 12 Jun 2015 11:27:08 -0700 Subject: [PATCH 209/246] Add `simple_linear_listening` example to generated docs This one is a nice to have in the examples, and should also be updated to not use deprecated properties. Change-Id: I5ee754525caccf0a9ea2a2b00aaa4e0d305e55b7 --- doc/source/examples.rst | 12 ++++++++++++ taskflow/examples/simple_linear_listening.py | 4 ++-- 2 files changed, 14 insertions(+), 2 deletions(-) diff --git a/doc/source/examples.rst b/doc/source/examples.rst index 64229f1b..187171fe 100644 --- a/doc/source/examples.rst +++ b/doc/source/examples.rst @@ -34,6 +34,18 @@ Using listeners :linenos: :lines: 16- +Using listeners (to watch a phone call) +======================================= + +.. note:: + + Full source located at :example:`simple_linear_listening`. + +.. literalinclude:: ../../taskflow/examples/simple_linear_listening.py + :language: python + :linenos: + :lines: 16- + Dumping a in-memory backend =========================== diff --git a/taskflow/examples/simple_linear_listening.py b/taskflow/examples/simple_linear_listening.py index deff63c7..850421b0 100644 --- a/taskflow/examples/simple_linear_listening.py +++ b/taskflow/examples/simple_linear_listening.py @@ -92,11 +92,11 @@ engine = taskflow.engines.load(flow, store={ }) # This is where we attach our callback functions to the 2 different -# notification objects that an engine exposes. The usage of a '*' (kleene star) +# notification objects that an engine exposes. The usage of a ANY (kleene star) # here means that we want to be notified on all state changes, if you want to # restrict to a specific state change, just register that instead. engine.notifier.register(ANY, flow_watch) -engine.task_notifier.register(ANY, task_watch) +engine.atom_notifier.register(ANY, task_watch) # And now run! engine.run() From 2e3543e566476a8be29379cf231d07b6c8697713 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Fri, 12 Jun 2015 11:35:19 -0700 Subject: [PATCH 210/246] Remove usage of deprecated 'task_notifier' property in build_car example Change-Id: Ia9d18c09709d550a017c5efbe1075b9a551496f6 --- taskflow/examples/build_a_car.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/taskflow/examples/build_a_car.py b/taskflow/examples/build_a_car.py index a3291817..cb952f39 100644 --- a/taskflow/examples/build_a_car.py +++ b/taskflow/examples/build_a_car.py @@ -167,7 +167,7 @@ engine = taskflow.engines.load(flow, store={'spec': spec.copy()}) # flow_watch function for flow state transitions, and registers the # same all (ANY) state transitions for task state transitions. engine.notifier.register(ANY, flow_watch) -engine.task_notifier.register(ANY, task_watch) +engine.atom_notifier.register(ANY, task_watch) eu.print_wrapped("Building a car") engine.run() @@ -180,7 +180,7 @@ spec['doors'] = 5 engine = taskflow.engines.load(flow, store={'spec': spec.copy()}) engine.notifier.register(ANY, flow_watch) -engine.task_notifier.register(ANY, task_watch) +engine.atom_notifier.register(ANY, task_watch) eu.print_wrapped("Building a wrong car that doesn't match specification") try: From 6e6284252bcbb34682812d79849f73bc578eb477 Mon Sep 17 00:00:00 2001 From: OpenStack Proposal Bot Date: Mon, 15 Jun 2015 03:13:22 +0000 Subject: [PATCH 211/246] Updated from global requirements Change-Id: I8945752e6847f964ec92ebd51e5618d47f4faf17 --- requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/requirements.txt b/requirements.txt index 6bfcef79..f85b92d3 100644 --- a/requirements.txt +++ b/requirements.txt @@ -17,7 +17,7 @@ six>=1.9.0 enum34 # For reader/writer + interprocess locks. -fasteners>=0.5 # Apache-2.0 +fasteners>=0.7 # Apache-2.0 # Very nice graph library networkx>=1.8 From 4f867db055eb74890dd1f46d104ea161a6bc84de Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Mon, 8 Jun 2015 18:33:48 -0700 Subject: [PATCH 212/246] Add indestructible 99 bottles of beer example Change-Id: Ic4edc7e04e8c3b70ce2f5e398db20363fa797cf8 --- doc/source/examples.rst | 13 +++ taskflow/examples/99_bottles.py | 139 ++++++++++++++++++++++++++++++++ 2 files changed, 152 insertions(+) create mode 100644 taskflow/examples/99_bottles.py diff --git a/doc/source/examples.rst b/doc/source/examples.rst index 64229f1b..c7f1140b 100644 --- a/doc/source/examples.rst +++ b/doc/source/examples.rst @@ -342,3 +342,16 @@ Conductor simulating a CI pipeline :language: python :linenos: :lines: 16- + + +Conductor running 99 bottles of beer song requests +================================================== + +.. note:: + + Full source located at :example:`99_bottles` + +.. literalinclude:: ../../taskflow/examples/99_bottles.py + :language: python + :linenos: + :lines: 16- diff --git a/taskflow/examples/99_bottles.py b/taskflow/examples/99_bottles.py new file mode 100644 index 00000000..9959255b --- /dev/null +++ b/taskflow/examples/99_bottles.py @@ -0,0 +1,139 @@ +# -*- coding: utf-8 -*- + +# Copyright (C) 2015 Yahoo! Inc. 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. + +import contextlib +import logging +import os +import sys +import time + +logging.basicConfig(level=logging.ERROR) + +top_dir = os.path.abspath(os.path.join(os.path.dirname(__file__), + os.pardir, + os.pardir)) +sys.path.insert(0, top_dir) + +from taskflow.conductors import backends as conductor_backends +from taskflow import engines +from taskflow.jobs import backends as job_backends +from taskflow.patterns import linear_flow as lf +from taskflow.persistence import backends as persistence_backends +from taskflow.persistence import logbook +from taskflow import task + +from oslo_utils import uuidutils + +# Instructions! +# +# 1. Install zookeeper (or change host listed below) +# 2. Download this example, place in file '99_bottles.py' +# 3. Run `python 99_bottles.py p` to place a song request onto the jobboard +# 4. Run `python 99_bottles.py c` a few times (in different shells) +# 5. On demand kill previously listed processes created in (4) and watch +# the work resume on another process (and repeat) +# 6. Keep enough workers alive to eventually finish the song (if desired). + +ME = os.getpid() +ZK_HOST = "localhost:2181" +JB_CONF = { + 'hosts': ZK_HOST, + 'board': 'zookeeper', + 'path': '/taskflow/99-bottles-demo', +} +DB_URI = r"sqlite:////tmp/bottles.db" +PART_DELAY = 1.0 +HOW_MANY_BOTTLES = 99 + + +class TakeABottleDownPassItAround(task.Task): + def execute(self, bottles_left): + sys.stdout.write('Take one down, ') + time.sleep(PART_DELAY) + sys.stdout.write('pass it around, ') + time.sleep(PART_DELAY) + sys.stdout.write('%s bottles of beer on the wall...\n' % bottles_left) + + +def make_bottles(count): + s = lf.Flow("bottle-song") + for bottle in reversed(list(range(1, count + 1))): + t = TakeABottleDownPassItAround("take-bottle-%s" % bottle, + inject={"bottles_left": bottle - 1}) + s.add(t) + return s + + +def run_conductor(): + print("Starting conductor with pid: %s" % ME) + my_name = "conductor-%s" % ME + persist_backend = persistence_backends.fetch(DB_URI) + with contextlib.closing(persist_backend): + with contextlib.closing(persist_backend.get_connection()) as conn: + conn.upgrade() + job_backend = job_backends.fetch(my_name, JB_CONF, + persistence=persist_backend) + job_backend.connect() + with contextlib.closing(job_backend): + cond = conductor_backends.fetch('blocking', my_name, job_backend, + persistence=persist_backend) + # Run forever, and kill -9 me... + # + # TODO(harlowja): it would be nicer if we could handle + # ctrl-c better... + cond.run() + + +def run_poster(): + print("Starting poster with pid: %s" % ME) + my_name = "poster-%s" % ME + persist_backend = persistence_backends.fetch(DB_URI) + with contextlib.closing(persist_backend): + with contextlib.closing(persist_backend.get_connection()) as conn: + conn.upgrade() + job_backend = job_backends.fetch(my_name, JB_CONF, + persistence=persist_backend) + job_backend.connect() + with contextlib.closing(job_backend): + # Create information in the persistence backend about the + # unit of work we want to complete and the factory that + # can be called to create the tasks that the work unit needs + # to be done. + lb = logbook.LogBook("post-from-%s" % my_name) + fd = logbook.FlowDetail("song-from-%s" % my_name, + uuidutils.generate_uuid()) + lb.add(fd) + with contextlib.closing(persist_backend.get_connection()) as conn: + conn.save_logbook(lb) + engines.save_factory_details(fd, make_bottles, + [HOW_MANY_BOTTLES], {}, + backend=persist_backend) + # Post, and be done with it! + job_backend.post("song-from-%s" % my_name, book=lb) + + +def main(): + if len(sys.argv) == 1: + sys.stderr.write("%s p|c\n" % os.path.basename(sys.argv[0])) + return + if sys.argv[1] == 'p': + run_poster() + if sys.argv[1] == 'c': + run_conductor() + + +if __name__ == '__main__': + main() From b93891dab13a9e910755139d304ac8e18d6889fb Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 10 Jun 2015 15:17:40 -0700 Subject: [PATCH 213/246] Use monotonic lib. to avoid finding monotonic time function That library already does this same/equivalent search and ensures that a monotonically increasing time is made available so we can just avoid looking around for it ourselves. Change-Id: I2b5c69430d75a095f6743c10e2d9281f7d9120e0 --- requirements.txt | 3 +++ taskflow/types/periodic.py | 11 ++++------- taskflow/utils/misc.py | 29 ----------------------------- 3 files changed, 7 insertions(+), 36 deletions(-) diff --git a/requirements.txt b/requirements.txt index e9d0550b..d0e7c91b 100644 --- a/requirements.txt +++ b/requirements.txt @@ -28,6 +28,9 @@ stevedore>=1.3.0 # Apache-2.0 # Backport for concurrent.futures which exists in 3.2+ futures>=3.0 +# Backport for time.monotonic which is in 3.3+ +monotonic>=0.1 + # Used for structured input validation jsonschema>=2.0.0,<3.0.0 diff --git a/taskflow/types/periodic.py b/taskflow/types/periodic.py index 7314988c..b658d494 100644 --- a/taskflow/types/periodic.py +++ b/taskflow/types/periodic.py @@ -18,6 +18,7 @@ import heapq import inspect from debtcollector import removals +import monotonic from oslo_utils import reflection import six @@ -27,10 +28,6 @@ from taskflow.utils import threading_utils as tu LOG = logging.getLogger(__name__) -# Find a monotonic providing time (or fallback to using time.time() -# which isn't *always* accurate but will suffice). -_now = misc.find_monotonic(allow_time_time=True) - def _check_attrs(obj): """Checks that a periodic function/method has all the expected attributes. @@ -81,7 +78,7 @@ class _Schedule(object): def push_next(self, cb, index, now=None): if now is None: - now = _now() + now = monotonic.monotonic() self.push(now + cb._periodic_spacing, index) def __len__(self): @@ -103,7 +100,7 @@ def _build(callables): immediates.append(i) else: if now is None: - now = _now() + now = monotonic.monotonic() schedule.push_next(cb, i, now=now) return immediates, schedule @@ -187,7 +184,7 @@ class PeriodicWorker(object): # minimum item from the heap, where the minimum should be # the callable that needs to run next and has the lowest # next desired run time). - now = _now() + now = monotonic.monotonic() next_run, index = self._schedule.pop() when_next = next_run - now if when_next <= 0: diff --git a/taskflow/utils/misc.py b/taskflow/utils/misc.py index 005f3947..42e2de5b 100644 --- a/taskflow/utils/misc.py +++ b/taskflow/utils/misc.py @@ -24,7 +24,6 @@ import os import re import sys import threading -import time import types import enum @@ -48,17 +47,6 @@ NUMERIC_TYPES = six.integer_types + (float,) # see RFC 3986 section 3.1 _SCHEME_REGEX = re.compile(r"^([A-Za-z][A-Za-z0-9+.-]*):") -_MONOTONIC_LOCATIONS = tuple([ - # The built-in/expected location in python3.3+ - 'time.monotonic', - # NOTE(harlowja): Try to use the pypi module that provides this - # functionality for older versions of python less than 3.3 so that - # they to can benefit from better timing... - # - # See: http://pypi.python.org/pypi/monotonic - 'monotonic.monotonic', -]) - class StrEnum(str, enum.Enum): """An enumeration that is also a string and can be compared to strings.""" @@ -71,23 +59,6 @@ class StrEnum(str, enum.Enum): return super(StrEnum, cls).__new__(cls, *args, **kwargs) -def find_monotonic(allow_time_time=False): - """Tries to find a monotonic time providing function (and returns it).""" - for import_str in _MONOTONIC_LOCATIONS: - mod_str, _sep, attr_str = import_str.rpartition('.') - mod = importutils.try_import(mod_str) - if mod is None: - continue - func = getattr(mod, attr_str, None) - if func is not None: - return func - # Finally give up and use time.time (which isn't monotonic)... - if allow_time_time: - return time.time - else: - return None - - def match_type(obj, matchers): """Matches a given object using the given matchers list/iterable. From caf37be3456dd3a0a03624afc8c25bcd20459e02 Mon Sep 17 00:00:00 2001 From: OpenStack Proposal Bot Date: Tue, 16 Jun 2015 19:23:53 +0000 Subject: [PATCH 214/246] Updated from global requirements Change-Id: Ieae928aa74517a4f70039638fe2a7e337e12539f --- requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/requirements.txt b/requirements.txt index f85b92d3..a2ce9064 100644 --- a/requirements.txt +++ b/requirements.txt @@ -35,7 +35,7 @@ futures>=3.0 jsonschema>=2.0.0,<3.0.0,!=2.5.0 # For common utilities -oslo.utils>=1.4.0 # Apache-2.0 +oslo.utils>=1.6.0 # Apache-2.0 oslo.serialization>=1.4.0 # Apache-2.0 # For lru caches and such From b42b2dff164ca5ca5f40fdc93e823aa37676ff62 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Tue, 16 Jun 2015 15:33:28 -0700 Subject: [PATCH 215/246] Use the `excutils.raise_with_cause` after doing our type check Change-Id: Ica886e67b4a75cb695d3ed51a3a952ca8f60f300 --- taskflow/exceptions.py | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/taskflow/exceptions.py b/taskflow/exceptions.py index 3d153ffc..2f22b4f7 100644 --- a/taskflow/exceptions.py +++ b/taskflow/exceptions.py @@ -15,9 +15,9 @@ # under the License. import os -import sys import traceback +from oslo_utils import excutils from oslo_utils import reflection import six @@ -48,12 +48,7 @@ def raise_with_cause(exc_cls, message, *args, **kwargs): """ if not issubclass(exc_cls, TaskFlowException): raise ValueError("Subclass of taskflow exception is required") - if 'cause' not in kwargs: - exc_type, exc, exc_tb = sys.exc_info() - if exc is not None: - kwargs['cause'] = exc - del(exc_type, exc, exc_tb) - six.raise_from(exc_cls(message, *args, **kwargs), kwargs.get('cause')) + excutils.raise_with_cause(exc_cls, message, *args, **kwargs) class TaskFlowException(Exception): From 2fa4af7a24584b211ca549f2df715c2d126360c8 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 17 Jun 2015 23:34:31 -0700 Subject: [PATCH 216/246] Split-off the additional retry states from the task states Split the states that are not task states (but are retry states) into there own additional set and then use that set and a new function to validate the transition at other locations in the code-base. This makes the transitions that are valid for tasks/retries easily viewable, more easy to read and understand, and more correct (instead of being a mix of task + retry atom transitions and states). Change-Id: I9515c19daf59a21e581f51e757ece2050f348214 --- doc/source/img/retry_states.svg | 6 +- doc/source/img/task_states.svg | 4 +- .../engines/action_engine/actions/base.py | 4 - .../engines/action_engine/actions/retry.py | 4 - .../engines/action_engine/actions/task.py | 4 - taskflow/engines/action_engine/analyzer.py | 49 ++++++----- taskflow/engines/action_engine/engine.py | 1 + taskflow/engines/action_engine/runtime.py | 88 ++++++++++++------- taskflow/states.py | 41 ++++++--- .../tests/unit/action_engine/test_runner.py | 6 +- taskflow/tests/unit/test_check_transition.py | 17 +++- tools/state_graph.py | 15 ++-- 12 files changed, 142 insertions(+), 97 deletions(-) diff --git a/doc/source/img/retry_states.svg b/doc/source/img/retry_states.svg index 8b0c6357..1a25bda3 100644 --- a/doc/source/img/retry_states.svg +++ b/doc/source/img/retry_states.svg @@ -1,8 +1,8 @@ - - -Retries statesPENDINGRUNNINGSUCCESSFAILURERETRYINGREVERTINGREVERTEDstart + +Retries statesPENDINGRUNNINGSUCCESSFAILURERETRYINGREVERTINGREVERTEDstart diff --git a/doc/source/img/task_states.svg b/doc/source/img/task_states.svg index 14a1f098..dbb48c60 100644 --- a/doc/source/img/task_states.svg +++ b/doc/source/img/task_states.svg @@ -1,8 +1,8 @@ - -Tasks statesPENDINGRUNNINGSUCCESSFAILUREREVERTINGREVERTEDstart +Tasks statesPENDINGRUNNINGSUCCESSFAILUREREVERTINGREVERTEDstart diff --git a/taskflow/engines/action_engine/actions/base.py b/taskflow/engines/action_engine/actions/base.py index 869ef228..369a6c66 100644 --- a/taskflow/engines/action_engine/actions/base.py +++ b/taskflow/engines/action_engine/actions/base.py @@ -35,7 +35,3 @@ class Action(object): def __init__(self, storage, notifier): self._storage = storage self._notifier = notifier - - @abc.abstractmethod - def handles(self, atom): - """Checks if this action handles the provided atom.""" diff --git a/taskflow/engines/action_engine/actions/retry.py b/taskflow/engines/action_engine/actions/retry.py index f69d5a5b..c8cad50a 100644 --- a/taskflow/engines/action_engine/actions/retry.py +++ b/taskflow/engines/action_engine/actions/retry.py @@ -48,10 +48,6 @@ class RetryAction(base.Action): super(RetryAction, self).__init__(storage, notifier) self._executor = futures.SynchronousExecutor() - @staticmethod - def handles(atom): - return isinstance(atom, retry_atom.Retry) - def _get_retry_args(self, retry, addons=None): arguments = self._storage.fetch_mapped_args( retry.rebind, diff --git a/taskflow/engines/action_engine/actions/task.py b/taskflow/engines/action_engine/actions/task.py index 2a11bf8d..ab4b50d9 100644 --- a/taskflow/engines/action_engine/actions/task.py +++ b/taskflow/engines/action_engine/actions/task.py @@ -32,10 +32,6 @@ class TaskAction(base.Action): super(TaskAction, self).__init__(storage, notifier) self._task_executor = task_executor - @staticmethod - def handles(atom): - return isinstance(atom, task_atom.BaseTask) - def _is_identity_transition(self, old_state, state, task, progress): if state in base.SAVE_RESULT_STATES: # saving result is never identity transition diff --git a/taskflow/engines/action_engine/analyzer.py b/taskflow/engines/action_engine/analyzer.py index a8f20c0d..bef7b8b5 100644 --- a/taskflow/engines/action_engine/analyzer.py +++ b/taskflow/engines/action_engine/analyzer.py @@ -34,6 +34,7 @@ class Analyzer(object): def __init__(self, runtime): self._storage = runtime.storage self._execution_graph = runtime.compilation.execution_graph + self._check_atom_transition = runtime.check_atom_transition def get_next_nodes(self, node=None): if node is None: @@ -93,37 +94,37 @@ class Analyzer(object): available_nodes.append(node) return available_nodes - def _is_ready_for_execute(self, task): - """Checks if task is ready to be executed.""" - state = self.get_state(task) - intention = self._storage.get_atom_intention(task.name) - transition = st.check_task_transition(state, st.RUNNING) + def _is_ready_for_execute(self, atom): + """Checks if atom is ready to be executed.""" + state = self.get_state(atom) + intention = self._storage.get_atom_intention(atom.name) + transition = self._check_atom_transition(atom, state, st.RUNNING) if not transition or intention != st.EXECUTE: return False - task_names = [] - for prev_task in self._execution_graph.predecessors(task): - task_names.append(prev_task.name) + atom_names = [] + for prev_atom in self._execution_graph.predecessors(atom): + atom_names.append(prev_atom.name) - task_states = self._storage.get_atoms_states(task_names) + atom_states = self._storage.get_atoms_states(atom_names) return all(state == st.SUCCESS and intention == st.EXECUTE - for state, intention in six.itervalues(task_states)) + for state, intention in six.itervalues(atom_states)) - def _is_ready_for_revert(self, task): - """Checks if task is ready to be reverted.""" - state = self.get_state(task) - intention = self._storage.get_atom_intention(task.name) - transition = st.check_task_transition(state, st.REVERTING) + def _is_ready_for_revert(self, atom): + """Checks if atom is ready to be reverted.""" + state = self.get_state(atom) + intention = self._storage.get_atom_intention(atom.name) + transition = self._check_atom_transition(atom, state, st.REVERTING) if not transition or intention not in (st.REVERT, st.RETRY): return False - task_names = [] - for prev_task in self._execution_graph.successors(task): - task_names.append(prev_task.name) + atom_names = [] + for prev_atom in self._execution_graph.successors(atom): + atom_names.append(prev_atom.name) - task_states = self._storage.get_atoms_states(task_names) + atom_states = self._storage.get_atoms_states(atom_names) return all(state in (st.PENDING, st.REVERTED) - for state, intention in six.itervalues(task_states)) + for state, intention in six.itervalues(atom_states)) def iterate_subgraph(self, atom): """Iterates a subgraph connected to given atom.""" @@ -148,10 +149,10 @@ class Analyzer(object): return self._execution_graph.node[atom].get('retry') def is_success(self): - for node in self._execution_graph.nodes_iter(): - if self.get_state(node) != st.SUCCESS: + for atom in self.iterate_all_nodes(): + if self.get_state(atom) != st.SUCCESS: return False return True - def get_state(self, node): - return self._storage.get_atom_state(node.name) + def get_state(self, atom): + return self._storage.get_atom_state(atom.name) diff --git a/taskflow/engines/action_engine/engine.py b/taskflow/engines/action_engine/engine.py index 124b8a56..df8d1d3d 100644 --- a/taskflow/engines/action_engine/engine.py +++ b/taskflow/engines/action_engine/engine.py @@ -295,6 +295,7 @@ class ActionEngine(base.Engine): self.storage, self.atom_notifier, self._task_executor) + self._runtime.compile() self._compiled = True diff --git a/taskflow/engines/action_engine/runtime.py b/taskflow/engines/action_engine/runtime.py index fc16fd9d..0fba861d 100644 --- a/taskflow/engines/action_engine/runtime.py +++ b/taskflow/engines/action_engine/runtime.py @@ -14,6 +14,8 @@ # License for the specific language governing permissions and limitations # under the License. +import functools + from taskflow.engines.action_engine.actions import retry as ra from taskflow.engines.action_engine.actions import task as ta from taskflow.engines.action_engine import analyzer as an @@ -22,6 +24,7 @@ from taskflow.engines.action_engine import runner as ru from taskflow.engines.action_engine import scheduler as sched from taskflow.engines.action_engine import scopes as sc from taskflow import states as st +from taskflow import task from taskflow.utils import misc @@ -38,7 +41,30 @@ class Runtime(object): self._task_executor = task_executor self._storage = storage self._compilation = compilation - self._walkers_to_names = {} + self._atom_cache = {} + + def compile(self): + # Build out a cache of commonly used item that are associated + # with the contained atoms (by name), and are useful to have for + # quick lookup on... + change_state_handlers = { + 'task': functools.partial(self.task_action.change_state, + progress=0.0), + 'retry': self.retry_action.change_state, + } + for atom in self.analyzer.iterate_all_nodes(): + metadata = {} + walker = sc.ScopeWalker(self.compilation, atom, names_only=True) + if isinstance(atom, task.BaseTask): + check_transition_handler = st.check_task_transition + change_state_handler = change_state_handlers['task'] + else: + check_transition_handler = st.check_retry_transition + change_state_handler = change_state_handlers['retry'] + metadata['scope_walker'] = walker + metadata['check_transition_handler'] = check_transition_handler + metadata['change_state_handler'] = change_state_handler + self._atom_cache[atom.name] = metadata @property def compilation(self): @@ -75,56 +101,52 @@ class Runtime(object): self._atom_notifier, self._task_executor) + def check_atom_transition(self, atom, current_state, target_state): + """Checks if the atom can transition to the provided target state.""" + # This does not check if the name exists (since this is only used + # internally to the engine, and is not exposed to atoms that will + # not exist and therefore doesn't need to handle that case). + metadata = self._atom_cache[atom.name] + check_transition_handler = metadata['check_transition_handler'] + return check_transition_handler(current_state, target_state) + def fetch_scopes_for(self, atom_name): """Fetches a walker of the visible scopes for the given atom.""" try: - return self._walkers_to_names[atom_name] + metadata = self._atom_cache[atom_name] except KeyError: - atom = None - for node in self.analyzer.iterate_all_nodes(): - if node.name == atom_name: - atom = node - break - if atom is not None: - walker = sc.ScopeWalker(self.compilation, atom, - names_only=True) - self._walkers_to_names[atom_name] = walker - else: - walker = None - return walker + # This signals to the caller that there is no walker for whatever + # atom name was given that doesn't really have any associated atom + # known to be named with that name; this is done since the storage + # layer will call into this layer to fetch a scope for a named + # atom and users can provide random names that do not actually + # exist... + return None + else: + return metadata['scope_walker'] # Various helper methods used by the runtime components; not for public # consumption... - def reset_nodes(self, nodes, state=st.PENDING, intention=st.EXECUTE): + def reset_nodes(self, atoms, state=st.PENDING, intention=st.EXECUTE): tweaked = [] - node_state_handlers = [ - (self.task_action, {'progress': 0.0}), - (self.retry_action, {}), - ] - for node in nodes: + for atom in atoms: + metadata = self._atom_cache[atom.name] if state or intention: - tweaked.append((node, state, intention)) + tweaked.append((atom, state, intention)) if state: - handled = False - for h, kwargs in node_state_handlers: - if h.handles(node): - h.change_state(node, state, **kwargs) - handled = True - break - if not handled: - raise TypeError("Unknown how to reset state of" - " node '%s' (%s)" % (node, type(node))) + change_state_handler = metadata['change_state_handler'] + change_state_handler(atom, state) if intention: - self.storage.set_atom_intention(node.name, intention) + self.storage.set_atom_intention(atom.name, intention) return tweaked def reset_all(self, state=st.PENDING, intention=st.EXECUTE): return self.reset_nodes(self.analyzer.iterate_all_nodes(), state=state, intention=intention) - def reset_subgraph(self, node, state=st.PENDING, intention=st.EXECUTE): - return self.reset_nodes(self.analyzer.iterate_subgraph(node), + def reset_subgraph(self, atom, state=st.PENDING, intention=st.EXECUTE): + return self.reset_nodes(self.analyzer.iterate_subgraph(atom), state=state, intention=intention) def retry_subflow(self, retry): diff --git a/taskflow/states.py b/taskflow/states.py index c5ea579e..265d6b2c 100644 --- a/taskflow/states.py +++ b/taskflow/states.py @@ -69,10 +69,10 @@ _ALLOWED_JOB_TRANSITIONS = frozenset(( def check_job_transition(old_state, new_state): - """Check that job can transition from old_state to new_state. + """Check that job can transition from from ``old_state`` to ``new_state``. - If transition can be performed, it returns True. If transition - should be ignored, it returns False. If transition is not + If transition can be performed, it returns true. If transition + should be ignored, it returns false. If transition is not valid, it raises an InvalidState exception. """ if old_state == new_state: @@ -138,10 +138,10 @@ _IGNORED_FLOW_TRANSITIONS = frozenset( def check_flow_transition(old_state, new_state): - """Check that flow can transition from old_state to new_state. + """Check that flow can transition from ``old_state`` to ``new_state``. - If transition can be performed, it returns True. If transition - should be ignored, it returns False. If transition is not + If transition can be performed, it returns true. If transition + should be ignored, it returns false. If transition is not valid, it raises an InvalidState exception. """ if old_state == new_state: @@ -171,18 +171,37 @@ _ALLOWED_TASK_TRANSITIONS = frozenset(( (REVERTING, FAILURE), # revert failed (REVERTED, PENDING), # try again - - (SUCCESS, RETRYING), # retrying retry controller - (RETRYING, RUNNING), # run retry controller that has been retrying )) def check_task_transition(old_state, new_state): - """Check that task can transition from old_state to new_state. + """Check that task can transition from ``old_state`` to ``new_state``. - If transition can be performed, it returns True, False otherwise. + If transition can be performed, it returns true, false otherwise. """ pair = (old_state, new_state) if pair in _ALLOWED_TASK_TRANSITIONS: return True return False + + +# Retry state transitions +# See: http://docs.openstack.org/developer/taskflow/states.html#retry + +_ALLOWED_RETRY_TRANSITIONS = list(_ALLOWED_TASK_TRANSITIONS) +_ALLOWED_RETRY_TRANSITIONS.extend([ + (SUCCESS, RETRYING), # retrying retry controller + (RETRYING, RUNNING), # run retry controller that has been retrying +]) +_ALLOWED_RETRY_TRANSITIONS = frozenset(_ALLOWED_RETRY_TRANSITIONS) + + +def check_retry_transition(old_state, new_state): + """Check that retry can transition from ``old_state`` to ``new_state``. + + If transition can be performed, it returns true, false otherwise. + """ + pair = (old_state, new_state) + if pair in _ALLOWED_RETRY_TRANSITIONS: + return True + return False diff --git a/taskflow/tests/unit/action_engine/test_runner.py b/taskflow/tests/unit/action_engine/test_runner.py index 98ae0e28..eb0f0a2c 100644 --- a/taskflow/tests/unit/action_engine/test_runner.py +++ b/taskflow/tests/unit/action_engine/test_runner.py @@ -45,8 +45,10 @@ class _RunnerTestMixin(object): task_executor = executor.SerialTaskExecutor() task_executor.start() self.addCleanup(task_executor.stop) - return runtime.Runtime(compilation, store, - task_notifier, task_executor) + r = runtime.Runtime(compilation, store, + task_notifier, task_executor) + r.compile() + return r class RunnerTest(test.TestCase, _RunnerTestMixin): diff --git a/taskflow/tests/unit/test_check_transition.py b/taskflow/tests/unit/test_check_transition.py index bed7bc99..7c820fd9 100644 --- a/taskflow/tests/unit/test_check_transition.py +++ b/taskflow/tests/unit/test_check_transition.py @@ -87,7 +87,7 @@ class CheckTaskTransitionTest(TransitionTest): def test_from_success_state(self): self.assertTransitions(from_state=states.SUCCESS, - allowed=(states.REVERTING, states.RETRYING), + allowed=(states.REVERTING,), ignored=(states.RUNNING, states.SUCCESS, states.PENDING, states.FAILURE, states.REVERTED)) @@ -112,6 +112,21 @@ class CheckTaskTransitionTest(TransitionTest): states.RUNNING, states.SUCCESS, states.FAILURE)) + +class CheckRetryTransitionTest(CheckTaskTransitionTest): + + def setUp(self): + super(CheckRetryTransitionTest, self).setUp() + self.check_transition = states.check_retry_transition + self.transition_exc_regexp = '^Retry transition.*not allowed' + + def test_from_success_state(self): + self.assertTransitions(from_state=states.SUCCESS, + allowed=(states.REVERTING, states.RETRYING), + ignored=(states.RUNNING, states.SUCCESS, + states.PENDING, states.FAILURE, + states.REVERTED)) + def test_from_retrying_state(self): self.assertTransitions(from_state=states.RETRYING, allowed=(states.RUNNING,), diff --git a/tools/state_graph.py b/tools/state_graph.py index 31961404..253ce606 100755 --- a/tools/state_graph.py +++ b/tools/state_graph.py @@ -49,12 +49,10 @@ def clean_event(name): return name -def make_machine(start_state, transitions, disallowed): +def make_machine(start_state, transitions): machine = fsm.FSM(start_state) machine.add_state(start_state) for (start_state, end_state) in transitions: - if start_state in disallowed or end_state in disallowed: - continue if start_state not in machine: machine.add_state(start_state) if end_state not in machine: @@ -125,12 +123,11 @@ def main(): if options.tasks: source_type = "Tasks" source = make_machine(states.PENDING, - list(states._ALLOWED_TASK_TRANSITIONS), - [states.RETRYING]) + list(states._ALLOWED_TASK_TRANSITIONS)) elif options.retries: source_type = "Retries" source = make_machine(states.PENDING, - list(states._ALLOWED_TASK_TRANSITIONS), []) + list(states._ALLOWED_RETRY_TRANSITIONS)) elif options.engines: source_type = "Engines" r = runner.Runner(DummyRuntime(), None) @@ -140,15 +137,15 @@ def main(): elif options.wbe_requests: source_type = "WBE requests" source = make_machine(protocol.WAITING, - list(protocol._ALLOWED_TRANSITIONS), []) + list(protocol._ALLOWED_TRANSITIONS)) elif options.jobs: source_type = "Jobs" source = make_machine(states.UNCLAIMED, - list(states._ALLOWED_JOB_TRANSITIONS), []) + list(states._ALLOWED_JOB_TRANSITIONS)) else: source_type = "Flow" source = make_machine(states.PENDING, - list(states._ALLOWED_FLOW_TRANSITIONS), []) + list(states._ALLOWED_FLOW_TRANSITIONS)) graph_name = "%s states" % source_type g = pydot.Dot(graph_name=graph_name, rankdir='LR', From 08c0bbc3e3a49303302140fc3812daa3c3fdae15 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Thu, 18 Jun 2015 10:17:31 -0700 Subject: [PATCH 217/246] Cache the individual atom schedulers at compile time These do not need to be re-fetched/re-examined each time. Change-Id: Ie48100caa12575c725530911ad3d1dc9046e9d26 --- taskflow/engines/action_engine/runtime.py | 23 ++++++++++ taskflow/engines/action_engine/scheduler.py | 47 +++++++-------------- 2 files changed, 38 insertions(+), 32 deletions(-) diff --git a/taskflow/engines/action_engine/runtime.py b/taskflow/engines/action_engine/runtime.py index 0fba861d..061cca4c 100644 --- a/taskflow/engines/action_engine/runtime.py +++ b/taskflow/engines/action_engine/runtime.py @@ -52,18 +52,25 @@ class Runtime(object): progress=0.0), 'retry': self.retry_action.change_state, } + schedulers = { + 'retry': self.retry_scheduler, + 'task': self.task_scheduler, + } for atom in self.analyzer.iterate_all_nodes(): metadata = {} walker = sc.ScopeWalker(self.compilation, atom, names_only=True) if isinstance(atom, task.BaseTask): check_transition_handler = st.check_task_transition change_state_handler = change_state_handlers['task'] + scheduler = schedulers['task'] else: check_transition_handler = st.check_retry_transition change_state_handler = change_state_handlers['retry'] + scheduler = schedulers['retry'] metadata['scope_walker'] = walker metadata['check_transition_handler'] = check_transition_handler metadata['change_state_handler'] = change_state_handler + metadata['scheduler'] = scheduler self._atom_cache[atom.name] = metadata @property @@ -90,6 +97,14 @@ class Runtime(object): def scheduler(self): return sched.Scheduler(self) + @misc.cachedproperty + def task_scheduler(self): + return sched.TaskScheduler(self) + + @misc.cachedproperty + def retry_scheduler(self): + return sched.RetryScheduler(self) + @misc.cachedproperty def retry_action(self): return ra.RetryAction(self._storage, @@ -110,6 +125,14 @@ class Runtime(object): check_transition_handler = metadata['check_transition_handler'] return check_transition_handler(current_state, target_state) + def fetch_scheduler(self, atom): + """Fetches the cached specific scheduler for the given atom.""" + # This does not check if the name exists (since this is only used + # internally to the engine, and is not exposed to atoms that will + # not exist and therefore doesn't need to handle that case). + metadata = self._atom_cache[atom.name] + return metadata['scheduler'] + def fetch_scopes_for(self, atom_name): """Fetches a walker of the visible scopes for the given atom.""" try: diff --git a/taskflow/engines/action_engine/scheduler.py b/taskflow/engines/action_engine/scheduler.py index 20221830..4ab0b0e1 100644 --- a/taskflow/engines/action_engine/scheduler.py +++ b/taskflow/engines/action_engine/scheduler.py @@ -17,22 +17,18 @@ import weakref from taskflow import exceptions as excp -from taskflow import retry as retry_atom from taskflow import states as st -from taskflow import task as task_atom from taskflow.types import failure -class _RetryScheduler(object): +class RetryScheduler(object): + """Schedules retry atoms.""" + def __init__(self, runtime): self._runtime = weakref.proxy(runtime) self._retry_action = runtime.retry_action self._storage = runtime.storage - @staticmethod - def handles(atom): - return isinstance(atom, retry_atom.Retry) - def schedule(self, retry): """Schedules the given retry atom for *future* completion. @@ -53,15 +49,13 @@ class _RetryScheduler(object): " intention: %s" % intention) -class _TaskScheduler(object): +class TaskScheduler(object): + """Schedules task atoms.""" + def __init__(self, runtime): self._storage = runtime.storage self._task_action = runtime.task_action - @staticmethod - def handles(atom): - return isinstance(atom, task_atom.BaseTask) - def schedule(self, task): """Schedules the given task atom for *future* completion. @@ -79,39 +73,28 @@ class _TaskScheduler(object): class Scheduler(object): - """Schedules atoms using actions to schedule.""" + """Safely schedules atoms using a runtime ``fetch_scheduler`` routine.""" def __init__(self, runtime): - self._schedulers = [ - _RetryScheduler(runtime), - _TaskScheduler(runtime), - ] + self._fetch_scheduler = runtime.fetch_scheduler - def _schedule_node(self, node): - """Schedule a single node for execution.""" - for sched in self._schedulers: - if sched.handles(node): - return sched.schedule(node) - else: - raise TypeError("Unknown how to schedule '%s' (%s)" - % (node, type(node))) + def schedule(self, atoms): + """Schedules the provided atoms for *future* completion. - def schedule(self, nodes): - """Schedules the provided nodes for *future* completion. - - This method should schedule a future for each node provided and return + This method should schedule a future for each atom provided and return a set of those futures to be waited on (or used for other similar purposes). It should also return any failure objects that represented scheduling failures that may have occurred during this scheduling process. """ futures = set() - for node in nodes: + for atom in atoms: + scheduler = self._fetch_scheduler(atom) try: - futures.add(self._schedule_node(node)) + futures.add(scheduler.schedule(atom)) except Exception: # Immediately stop scheduling future work so that we can - # exit execution early (rather than later) if a single task + # exit execution early (rather than later) if a single atom # fails to schedule correctly. return (futures, [failure.Failure()]) return (futures, []) From 40d19c7696f1e0b7d75eacbd271974ee9155c019 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Mon, 8 Jun 2015 19:16:16 -0700 Subject: [PATCH 218/246] Handle conductor ctrl-c more appropriately When a conductor program is interrupted via ctrl-c or equivalent it is much nicer log that that has happened and to reraise that exception. This also slightly tweaks the 99 bottles song to make it even better, by having more pieces/tasks, which makes it possible to kill the program during each task and see how the resumption works when a flow is composed of segments. Change-Id: I5d242eba9a043ef96646ba74ea5928daa0691ed0 --- taskflow/conductors/backends/impl_blocking.py | 4 ++ taskflow/examples/99_bottles.py | 54 ++++++++++++------- 2 files changed, 39 insertions(+), 19 deletions(-) diff --git a/taskflow/conductors/backends/impl_blocking.py b/taskflow/conductors/backends/impl_blocking.py index fb8a3c3a..945f591b 100644 --- a/taskflow/conductors/backends/impl_blocking.py +++ b/taskflow/conductors/backends/impl_blocking.py @@ -18,6 +18,7 @@ except ImportError: from contextlib2 import ExitStack # noqa from debtcollector import removals +from oslo_utils import excutils import six from taskflow.conductors import base @@ -151,6 +152,9 @@ class BlockingConductor(base.Conductor): consume = False try: f = self._dispatch_job(job) + except KeyboardInterrupt: + with excutils.save_and_reraise_exception(): + LOG.warn("Job dispatching interrupted: %s", job) except Exception: LOG.warn("Job dispatching failed: %s", job, exc_info=True) diff --git a/taskflow/examples/99_bottles.py b/taskflow/examples/99_bottles.py index 9959255b..90894e9c 100644 --- a/taskflow/examples/99_bottles.py +++ b/taskflow/examples/99_bottles.py @@ -54,33 +54,47 @@ JB_CONF = { 'board': 'zookeeper', 'path': '/taskflow/99-bottles-demo', } -DB_URI = r"sqlite:////tmp/bottles.db" -PART_DELAY = 1.0 +PERSISTENCE_URI = r"sqlite:////tmp/bottles.db" +TAKE_DOWN_DELAY = 1.0 +PASS_AROUND_DELAY = 3.0 HOW_MANY_BOTTLES = 99 -class TakeABottleDownPassItAround(task.Task): - def execute(self, bottles_left): +class TakeABottleDown(task.Task): + def execute(self): sys.stdout.write('Take one down, ') - time.sleep(PART_DELAY) + sys.stdout.flush() + time.sleep(TAKE_DOWN_DELAY) + + +class PassItAround(task.Task): + def execute(self): sys.stdout.write('pass it around, ') - time.sleep(PART_DELAY) + sys.stdout.flush() + time.sleep(PASS_AROUND_DELAY) + + +class Conclusion(task.Task): + def execute(self, bottles_left): sys.stdout.write('%s bottles of beer on the wall...\n' % bottles_left) + sys.stdout.flush() def make_bottles(count): s = lf.Flow("bottle-song") for bottle in reversed(list(range(1, count + 1))): - t = TakeABottleDownPassItAround("take-bottle-%s" % bottle, - inject={"bottles_left": bottle - 1}) - s.add(t) + take_bottle = TakeABottleDown("take-bottle-%s" % bottle) + pass_it = PassItAround("pass-%s-around" % bottle) + next_bottles = Conclusion("next-bottles-%s" % (bottle - 1), + inject={"bottles_left": bottle - 1}) + s.add(take_bottle, pass_it, next_bottles) return s def run_conductor(): print("Starting conductor with pid: %s" % ME) my_name = "conductor-%s" % ME - persist_backend = persistence_backends.fetch(DB_URI) + persist_backend = persistence_backends.fetch(PERSISTENCE_URI) with contextlib.closing(persist_backend): with contextlib.closing(persist_backend.get_connection()) as conn: conn.upgrade() @@ -90,17 +104,18 @@ def run_conductor(): with contextlib.closing(job_backend): cond = conductor_backends.fetch('blocking', my_name, job_backend, persistence=persist_backend) - # Run forever, and kill -9 me... - # - # TODO(harlowja): it would be nicer if we could handle - # ctrl-c better... - cond.run() + # Run forever, and kill -9 or ctrl-c me... + try: + cond.run() + finally: + cond.stop() + cond.wait() def run_poster(): print("Starting poster with pid: %s" % ME) my_name = "poster-%s" % ME - persist_backend = persistence_backends.fetch(DB_URI) + persist_backend = persistence_backends.fetch(PERSISTENCE_URI) with contextlib.closing(persist_backend): with contextlib.closing(persist_backend.get_connection()) as conn: conn.upgrade() @@ -128,11 +143,12 @@ def run_poster(): def main(): if len(sys.argv) == 1: sys.stderr.write("%s p|c\n" % os.path.basename(sys.argv[0])) - return - if sys.argv[1] == 'p': + elif sys.argv[1] == 'p': run_poster() - if sys.argv[1] == 'c': + elif sys.argv[1] == 'c': run_conductor() + else: + sys.stderr.write("%s p|c\n" % os.path.basename(sys.argv[0])) if __name__ == '__main__': From 23a62fef9fb7f67467aaa714257be7fa451364c4 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Thu, 11 Jun 2015 14:38:34 -0700 Subject: [PATCH 219/246] Perform a few optimizations to decrease persistence interactions To reduce the amount of *unneeded* persistence backend interactions we can optimize a few cases to avoid saving anything when nothing has changed; this should help in a few cases, and is easy low hanging fruit. Part of blueprint make-things-speedy Change-Id: I4fe958c94ef308919395345fd5c0d85f181446fb --- taskflow/persistence/logbook.py | 40 ++++++++++++++++++++++++--------- taskflow/storage.py | 20 ++++++++++------- 2 files changed, 41 insertions(+), 19 deletions(-) diff --git a/taskflow/persistence/logbook.py b/taskflow/persistence/logbook.py index be254ea1..c7a6eae5 100644 --- a/taskflow/persistence/logbook.py +++ b/taskflow/persistence/logbook.py @@ -52,10 +52,6 @@ def _safe_unmarshal_time(when): return timeutils.unmarshall_time(when) -def _was_failure(state, result): - return state == states.FAILURE and isinstance(result, ft.Failure) - - def _fix_meta(data): # Handle the case where older schemas allowed this to be non-dict by # correcting this case by replacing it with a dictionary when a non-dict @@ -434,6 +430,11 @@ class AtomDetail(object): self.meta = {} self.version = None + @staticmethod + def _was_failure(state, result): + # Internal helper method... + return state == states.FAILURE and isinstance(result, ft.Failure) + @property def last_results(self): """Gets the atoms last result. @@ -601,13 +602,29 @@ class TaskDetail(AtomDetail): will be set to ``None``). In either case the ``state`` attribute will be set to the provided state. """ - self.state = state - if _was_failure(state, result): - self.failure = result - self.results = None + was_altered = False + if self.state != state: + self.state = state + was_altered = True + if self._was_failure(state, result): + if self.failure != result: + self.failure = result + was_altered = True + if self.results is not None: + self.results = None + was_altered = True else: - self.results = result - self.failure = None + # We don't really have the ability to determine equality of + # task (user) results at the current time, without making + # potentially bad guesses, so assume the task detail always needs + # to be saved if they are not exactly equivalent... + if self.results is not result: + self.results = result + was_altered = True + if self.failure is not None: + self.failure = None + was_altered = True + return was_altered def merge(self, other, deep_copy=False): """Merges the current task detail with the given one. @@ -763,11 +780,12 @@ class RetryDetail(AtomDetail): """ # Do not clean retry history (only on reset does this happen). self.state = state - if _was_failure(state, result): + if self._was_failure(state, result): self.failure = result else: self.results.append((result, {})) self.failure = None + return True @classmethod def from_dict(cls, data): diff --git a/taskflow/storage.py b/taskflow/storage.py index 6be4b26a..d50d8771 100644 --- a/taskflow/storage.py +++ b/taskflow/storage.py @@ -169,7 +169,6 @@ class Storage(object): self._atom_name_to_uuid = dict((ad.name, ad.uuid) for ad in self._flowdetail) - try: source, _clone = self._atomdetail_by_name( self.injector_name, expected_type=logbook.TaskDetail) @@ -320,8 +319,9 @@ class Storage(object): def set_atom_state(self, atom_name, state): """Sets an atoms state.""" source, clone = self._atomdetail_by_name(atom_name, clone=True) - clone.state = state - self._with_connection(self._save_atom_detail, source, clone) + if source.state != state: + clone.state = state + self._with_connection(self._save_atom_detail, source, clone) @fasteners.read_locked def get_atom_state(self, atom_name): @@ -333,8 +333,9 @@ class Storage(object): def set_atom_intention(self, atom_name, intention): """Sets the intention of an atom given an atoms name.""" source, clone = self._atomdetail_by_name(atom_name, clone=True) - clone.intention = intention - self._with_connection(self._save_atom_detail, source, clone) + if source.intention != intention: + clone.intention = intention + self._with_connection(self._save_atom_detail, source, clone) @fasteners.read_locked def get_atom_intention(self, atom_name): @@ -441,10 +442,13 @@ class Storage(object): @fasteners.write_locked def save(self, atom_name, data, state=states.SUCCESS): - """Put result for atom with id 'uuid' to storage.""" + """Save result for named atom into storage with given state.""" source, clone = self._atomdetail_by_name(atom_name, clone=True) - clone.put(state, data) - result = self._with_connection(self._save_atom_detail, source, clone) + if clone.put(state, data): + result = self._with_connection(self._save_atom_detail, + source, clone) + else: + result = clone if state == states.FAILURE and isinstance(data, failure.Failure): # NOTE(imelnikov): failure serialization looses information, # so we cache failures here, in atom name -> failure mapping so From 2b7b5334116d4bf8d7a62272cdd945aa6a106981 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Fri, 19 Jun 2015 21:44:43 -0700 Subject: [PATCH 220/246] Add doc warning to engine components These components are internal to an engine and should not be used externally so make sure we add a doc note that states this (or at least warns about using them minimally). There are still useful to document as they are used in docs that describe how engines work and what they do. Change-Id: I9143f5d7114ccedddee602dabff4b48446bd99ff --- doc/source/engines.rst | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/doc/source/engines.rst b/doc/source/engines.rst index abdbdb62..f119f50a 100644 --- a/doc/source/engines.rst +++ b/doc/source/engines.rst @@ -441,6 +441,13 @@ Implementations Components ---------- +.. warning:: + + External usage of internal engine functions, components and modules should + be kept to a **minimum** as they may be altered, refactored or moved to + other locations **without** notice (and without the typical deprecation + cycle). + .. automodule:: taskflow.engines.action_engine.analyzer .. automodule:: taskflow.engines.action_engine.compiler .. automodule:: taskflow.engines.action_engine.completer From 9604703c5996063354c665af8b71c2d413faa41e Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Thu, 11 Jun 2015 15:38:14 -0700 Subject: [PATCH 221/246] Make it possible to see the queries executed (in BLATHER mode) Part of blueprint make-things-speedy Change-Id: I3ac092e98a492d3a70c7780efb4647d593fa70d1 --- taskflow/persistence/backends/impl_sqlalchemy.py | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/taskflow/persistence/backends/impl_sqlalchemy.py b/taskflow/persistence/backends/impl_sqlalchemy.py index 483dd445..d5342018 100644 --- a/taskflow/persistence/backends/impl_sqlalchemy.py +++ b/taskflow/persistence/backends/impl_sqlalchemy.py @@ -108,6 +108,12 @@ DEFAULT_TXN_ISOLATION_LEVELS = { } +def _log_statements(log_level, conn, cursor, statement, parameters, *args): + if LOG.isEnabledFor(log_level): + LOG.log(log_level, "Running statement '%s' with parameters %s", + statement, parameters) + + def _in_any(reason, err_haystack): """Checks if any elements of the haystack are in the given reason.""" for err in err_haystack: @@ -290,6 +296,13 @@ class SQLAlchemyBackend(base.Backend): # or engine arg overrides make sure we merge them in. engine_args.update(conf.pop('engine_args', {})) engine = sa.create_engine(sql_connection, **engine_args) + log_statements = conf.pop('log_statements', False) + if _as_bool(log_statements): + log_statements_level = conf.pop("log_statements_level", + logging.BLATHER) + sa.event.listen(engine, "before_cursor_execute", + functools.partial(_log_statements, + log_statements_level)) checkin_yield = conf.pop('checkin_yield', eventlet_utils.EVENTLET_AVAILABLE) if _as_bool(checkin_yield): From b7bb295fb070eed580edbd91f286aca585163b26 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Thu, 11 Jun 2015 16:10:05 -0700 Subject: [PATCH 222/246] Add bulk `ensure_atoms` method to storage The addition of this method will make it possible to optimize the many queries and updates that are performed when a engine is initially having its storage unit prepared. Part of blueprint make-things-speedy Change-Id: Ie812abfbeca97372be05bc13e5c1059278fbbf4c --- taskflow/engines/action_engine/engine.py | 3 ++- taskflow/storage.py | 16 +++++++++++++--- 2 files changed, 15 insertions(+), 4 deletions(-) diff --git a/taskflow/engines/action_engine/engine.py b/taskflow/engines/action_engine/engine.py index 3dda9125..7c275ba6 100644 --- a/taskflow/engines/action_engine/engine.py +++ b/taskflow/engines/action_engine/engine.py @@ -215,8 +215,9 @@ class ActionEngine(base.Engine): """Ensure all contained atoms exist in the storage unit.""" transient = strutils.bool_from_string( self._options.get('inject_transient', True)) + self.storage.ensure_atoms( + self._compilation.execution_graph.nodes_iter()) for node in self._compilation.execution_graph.nodes_iter(): - self.storage.ensure_atom(node) if node.inject: self.storage.inject_atom_args(node.name, node.inject, diff --git a/taskflow/storage.py b/taskflow/storage.py index d50d8771..8eb19c09 100644 --- a/taskflow/storage.py +++ b/taskflow/storage.py @@ -186,10 +186,20 @@ class Storage(object): with contextlib.closing(self._backend.get_connection()) as conn: return functor(conn, *args, **kwargs) - def ensure_atom(self, atom): - """Ensure that there is an atomdetail in storage for the given atom. + def ensure_atoms(self, atoms_iter): + """Ensure there is an atomdetail for **each** of the given atoms. - Returns uuid for the atomdetail that is/was created. + Returns list of atomdetail uuids for each atom processed. + """ + atom_ids = [] + for atom in atoms_iter: + atom_ids.append(self.ensure_atom(atom)) + return atom_ids + + def ensure_atom(self, atom): + """Ensure there is an atomdetail for the **given** atom. + + Returns the uuid for the atomdetail that corresponds to the given atom. """ match = misc.match_type(atom, self._ensure_matchers) if not match: From 4d0200f92e7cb19591c13a1eafbe8eb7ab2360f9 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Thu, 11 Jun 2015 16:46:06 -0700 Subject: [PATCH 223/246] Add smarter/better/faster impl. of `ensure_atoms` Instead of ensuring an atom at a time we can examine at bulk what atoms details need to be created, and then only do any persistence operations if any atom details are determined to not exist. This saves on unneeded reads and writes and helps improve storage and prepare times. Part of blueprint make-things-speedy Change-Id: I2e7e3cc60cc97cbbe4e66c69e22c34ee31ebfeb3 --- taskflow/storage.py | 132 +++++++++++++++++++++++--------------------- 1 file changed, 69 insertions(+), 63 deletions(-) diff --git a/taskflow/storage.py b/taskflow/storage.py index 8eb19c09..cb1fbaad 100644 --- a/taskflow/storage.py +++ b/taskflow/storage.py @@ -186,14 +186,67 @@ class Storage(object): with contextlib.closing(self._backend.get_connection()) as conn: return functor(conn, *args, **kwargs) - def ensure_atoms(self, atoms_iter): + @staticmethod + def _create_atom_detail(atom_name, atom_detail_cls, + atom_version=None, atom_state=states.PENDING): + ad = atom_detail_cls(atom_name, uuidutils.generate_uuid()) + ad.state = atom_state + if atom_version is not None: + ad.version = atom_version + return ad + + @fasteners.write_locked + def ensure_atoms(self, atoms): """Ensure there is an atomdetail for **each** of the given atoms. Returns list of atomdetail uuids for each atom processed. """ atom_ids = [] - for atom in atoms_iter: - atom_ids.append(self.ensure_atom(atom)) + missing_ads = [] + for i, atom in enumerate(atoms): + match = misc.match_type(atom, self._ensure_matchers) + if not match: + raise TypeError("Unknown atom '%s' (%s) requested to ensure" + % (atom, type(atom))) + atom_detail_cls, kind = match + atom_name = atom.name + if not atom_name: + raise ValueError("%s name must be non-empty" % (kind)) + try: + atom_id = self._atom_name_to_uuid[atom_name] + except KeyError: + missing_ads.append((i, atom, atom_detail_cls)) + # This will be later replaced with the uuid that is created... + atom_ids.append(None) + else: + ad = self._flowdetail.find(atom_id) + if not isinstance(ad, atom_detail_cls): + raise exceptions.Duplicate( + "Atom detail '%s' already exists in flow" + " detail '%s'" % (atom_name, self._flowdetail.name)) + else: + atom_ids.append(ad.uuid) + self._set_result_mapping(atom_name, atom.save_as) + if missing_ads: + needs_to_be_created_ads = [] + for (i, atom, atom_detail_cls) in missing_ads: + ad = self._create_atom_detail( + atom.name, atom_detail_cls, + atom_version=misc.get_version_string(atom)) + needs_to_be_created_ads.append((i, atom, ad)) + # Add the atom detail(s) to a clone, which upon success will be + # updated into the contained flow detail; if it does not get saved + # then no update will happen. + source, clone = self._fetch_flowdetail(clone=True) + for (_i, _atom, ad) in needs_to_be_created_ads: + clone.add(ad) + self._with_connection(self._save_flow_detail, source, clone) + # Insert the needed data, and get outta here... + for (i, atom, ad) in needs_to_be_created_ads: + atom_name = atom.name + atom_ids[i] = ad.uuid + self._atom_name_to_uuid[atom_name] = ad.uuid + self._set_result_mapping(atom_name, atom.save_as) return atom_ids def ensure_atom(self, atom): @@ -201,63 +254,7 @@ class Storage(object): Returns the uuid for the atomdetail that corresponds to the given atom. """ - match = misc.match_type(atom, self._ensure_matchers) - if not match: - raise TypeError("Unknown atom '%s' (%s) requested to ensure" - % (atom, type(atom))) - else: - detail_cls, kind = match - atom_id = self._ensure_atom_detail(kind, detail_cls, atom.name, - misc.get_version_string(atom), - atom.save_as) - return atom_id - - def _ensure_atom_detail(self, kind, detail_cls, - atom_name, atom_version, result_mapping): - """Ensures there is a atomdetail that corresponds to the given atom. - - If atom does not exist, adds a record for it. Added atom will have - PENDING state. Sets result mapping for the atom from result_mapping - argument. - - Returns uuid for the atomdetails corresponding to the atom with - given name. - """ - if not atom_name: - raise ValueError("%s name must be non-empty" % (kind)) - with self._lock.write_lock(): - try: - atom_id = self._atom_name_to_uuid[atom_name] - except KeyError: - atom_id = uuidutils.generate_uuid() - self._create_atom_detail(detail_cls, atom_name, - atom_id, atom_version=atom_version) - else: - ad = self._flowdetail.find(atom_id) - if not isinstance(ad, detail_cls): - raise exceptions.Duplicate( - "Atom detail '%s' already exists in flow" - " detail '%s'" % (atom_name, self._flowdetail.name)) - self._set_result_mapping(atom_name, result_mapping) - return atom_id - - def _create_atom_detail(self, detail_cls, name, uuid, atom_version=None): - """Add the atom detail to flow detail. - - Atom becomes known to storage by that name and uuid. - Atom state is set to PENDING. - """ - ad = detail_cls(name, uuid) - ad.state = states.PENDING - ad.version = atom_version - # Add the atom detail to the clone, which upon success will be - # updated into the contained flow detail; if it does not get saved - # then no update will happen. - source, clone = self._fetch_flowdetail(clone=True) - clone.add(ad) - self._with_connection(self._save_flow_detail, source, clone) - self._atom_name_to_uuid[ad.name] = ad.uuid - return ad + return self.ensure_atoms([atom])[0] @property def flow_name(self): @@ -631,9 +628,18 @@ class Storage(object): expected_type=logbook.TaskDetail, clone=True) except exceptions.NotFound: - source = self._create_atom_detail(logbook.TaskDetail, - self.injector_name, - uuidutils.generate_uuid()) + # Ensure we have our special task detail... + # + # TODO(harlowja): get this removed when + # https://review.openstack.org/#/c/165645/ merges. + source = self._create_atom_detail(self.injector_name, + logbook.TaskDetail, + atom_state=None) + fd_source, fd_clone = self._fetch_flowdetail(clone=True) + fd_clone.add(source) + self._with_connection(self._save_flow_detail, fd_source, + fd_clone) + self._atom_name_to_uuid[source.name] = source.uuid clone = source clone.results = dict(pairs) clone.state = states.SUCCESS From 61e0659bbd8ae00cfae32ca0e9fbac105b3b7563 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Sat, 20 Jun 2015 09:11:12 -0700 Subject: [PATCH 224/246] Remove reference to 'requirements-pyN.txt' files These files no longer exist and therefore the README.rst should not reference them. Change-Id: I9317cf89738eff0def877e63591faafa090fbdfe --- README.rst | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/README.rst b/README.rst index 9023dd67..3a15f78f 100644 --- a/README.rst +++ b/README.rst @@ -30,18 +30,16 @@ Requirements ~~~~~~~~~~~~ Because this project has many optional (pluggable) parts like persistence -backends and engines, we decided to split our requirements into three +backends and engines, we decided to split our requirements into two parts: - things that are absolutely required (you can't use the project -without them) are put into ``requirements-pyN.txt`` (``N`` being the -Python *major* version number used to install the package). The requirements +without them) are put into ``requirements.txt``. The requirements that are required by some optional part of this project (you can use the -project without them) are put into our ``tox.ini`` file (so that we can still -test the optional functionality works as expected). If you want to use the -feature in question (`eventlet`_ or the worker based engine that -uses `kombu`_ or the `sqlalchemy`_ persistence backend or jobboards which +project without them) are put into our ``test-requirements.txt`` file (so +that we can still test the optional functionality works as expected). If +you want to use the feature in question (`eventlet`_ or the worker based engine +that uses `kombu`_ or the `sqlalchemy`_ persistence backend or jobboards which have an implementation built using `kazoo`_ ...), you should add -that requirement(s) to your project or environment; - as usual, things that -required only for running tests are put into ``test-requirements.txt``. +that requirement(s) to your project or environment. Tox.ini ~~~~~~~ From 543cf78a6f26fe4929b53a7bfb9cbdc5f2624fbd Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Thu, 18 Jun 2015 15:48:11 -0700 Subject: [PATCH 225/246] Remove 2.6 classifier + 2.6 compatibility code Fixes bug 1445827 Depends-On: I02e3c9aacef0b295a2f823a5cbaf11768a90cb82 Change-Id: I1db681803598ac1bc917fd74a99458bc61edf3f1 --- requirements.txt | 3 -- setup.cfg | 1 - taskflow/atom.py | 28 ++++++++----------- taskflow/conductors/backends/impl_blocking.py | 5 ++-- taskflow/engines/action_engine/executor.py | 3 +- taskflow/engines/worker_based/proxy.py | 4 +-- .../tests/unit/conductor/test_blocking.py | 5 ++-- taskflow/tests/unit/jobs/base.py | 7 +++-- taskflow/tests/unit/test_listeners.py | 4 +-- .../tests/unit/test_utils_threading_utils.py | 12 +++----- .../tests/unit/worker_based/test_executor.py | 4 +-- .../unit/worker_based/test_message_pump.py | 6 ++-- taskflow/tests/utils.py | 4 +-- taskflow/types/fsm.py | 9 ++---- taskflow/types/periodic.py | 4 +-- taskflow/types/sets.py | 7 +---- taskflow/types/timing.py | 6 ++-- taskflow/utils/threading_utils.py | 20 ------------- 18 files changed, 49 insertions(+), 83 deletions(-) diff --git a/requirements.txt b/requirements.txt index a2ce9064..f69bfa5e 100644 --- a/requirements.txt +++ b/requirements.txt @@ -7,9 +7,6 @@ pbr>=0.11,<2.0 # Packages needed for using this library. -# Only needed on python 2.6 -ordereddict - # Python 2->3 compatibility library. six>=1.9.0 diff --git a/setup.cfg b/setup.cfg index a45b6dd8..ac5b566a 100644 --- a/setup.cfg +++ b/setup.cfg @@ -17,7 +17,6 @@ classifier = Operating System :: POSIX :: Linux Programming Language :: Python Programming Language :: Python :: 2 - Programming Language :: Python :: 2.6 Programming Language :: Python :: 2.7 Programming Language :: Python :: 3 Programming Language :: Python :: 3.4 diff --git a/taskflow/atom.py b/taskflow/atom.py index 82671df7..fba04a97 100644 --- a/taskflow/atom.py +++ b/taskflow/atom.py @@ -19,11 +19,6 @@ import abc import collections import itertools -try: - from collections import OrderedDict # noqa -except ImportError: - from ordereddict import OrderedDict # noqa - from oslo_utils import reflection import six from six.moves import zip as compat_zip @@ -46,23 +41,24 @@ def _save_as_to_mapping(save_as): # outside of code so that it's more easily understandable, since what an # atom returns is pretty crucial for other later operations. if save_as is None: - return OrderedDict() + return collections.OrderedDict() if isinstance(save_as, six.string_types): # NOTE(harlowja): this means that your atom will only return one item # instead of a dictionary-like object or a indexable object (like a # list or tuple). - return OrderedDict([(save_as, None)]) + return collections.OrderedDict([(save_as, None)]) elif isinstance(save_as, _sequence_types): # NOTE(harlowja): this means that your atom will return a indexable # object, like a list or tuple and the results can be mapped by index # to that tuple/list that is returned for others to use. - return OrderedDict((key, num) for num, key in enumerate(save_as)) + return collections.OrderedDict((key, num) + for num, key in enumerate(save_as)) elif isinstance(save_as, _set_types): # NOTE(harlowja): in the case where a set is given we will not be # able to determine the numeric ordering in a reliable way (since it # may be an unordered set) so the only way for us to easily map the # result of the atom will be via the key itself. - return OrderedDict((key, key) for key in save_as) + return collections.OrderedDict((key, key) for key in save_as) else: raise TypeError('Atom provides parameter ' 'should be str, set or tuple/list, not %r' % save_as) @@ -76,9 +72,9 @@ def _build_rebind_dict(args, rebind_args): new name onto the required name). """ if rebind_args is None: - return OrderedDict() + return collections.OrderedDict() elif isinstance(rebind_args, (list, tuple)): - rebind = OrderedDict(compat_zip(args, rebind_args)) + rebind = collections.OrderedDict(compat_zip(args, rebind_args)) if len(args) < len(rebind_args): rebind.update((a, a) for a in rebind_args[len(args):]) return rebind @@ -112,7 +108,7 @@ def _build_arg_mapping(atom_name, reqs, rebind_args, function, do_infer, ignore_list = [] # Build the required names. - required = OrderedDict() + required = collections.OrderedDict() # Add required arguments to required mappings if inference is enabled. if do_infer: @@ -133,9 +129,9 @@ def _build_arg_mapping(atom_name, reqs, rebind_args, function, do_infer, opt_args = sets.OrderedSet(all_args) opt_args = opt_args - set(itertools.chain(six.iterkeys(required), iter(ignore_list))) - optional = OrderedDict((a, a) for a in opt_args) + optional = collections.OrderedDict((a, a) for a in opt_args) else: - optional = OrderedDict() + optional = collections.OrderedDict() # Check if we are given some extra arguments that we aren't able to accept. if not reflection.accepts_kwargs(function): @@ -206,14 +202,14 @@ class Atom(object): self.requires = sets.OrderedSet() self.optional = sets.OrderedSet() self.provides = sets.OrderedSet(self.save_as) - self.rebind = OrderedDict() + self.rebind = collections.OrderedDict() def _build_arg_mapping(self, executor, requires=None, rebind=None, auto_extract=True, ignore_list=None): required, optional = _build_arg_mapping(self.name, requires, rebind, executor, auto_extract, ignore_list=ignore_list) - rebind = OrderedDict() + rebind = collections.OrderedDict() for (arg_name, bound_name) in itertools.chain(six.iteritems(required), six.iteritems(optional)): rebind.setdefault(arg_name, bound_name) diff --git a/taskflow/conductors/backends/impl_blocking.py b/taskflow/conductors/backends/impl_blocking.py index fb8a3c3a..1546c0ad 100644 --- a/taskflow/conductors/backends/impl_blocking.py +++ b/taskflow/conductors/backends/impl_blocking.py @@ -12,6 +12,8 @@ # License for the specific language governing permissions and limitations # under the License. +import threading + try: from contextlib import ExitStack # noqa except ImportError: @@ -26,7 +28,6 @@ from taskflow.listeners import logging as logging_listener from taskflow import logging from taskflow.types import timing as tt from taskflow.utils import async_utils -from taskflow.utils import threading_utils LOG = logging.getLogger(__name__) WAIT_TIMEOUT = 0.5 @@ -69,7 +70,7 @@ class BlockingConductor(base.Conductor): self._wait_timeout = wait_timeout else: raise ValueError("Invalid timeout literal: %s" % (wait_timeout)) - self._dead = threading_utils.Event() + self._dead = threading.Event() @removals.removed_kwarg('timeout', version="0.8", removal_version="?") def stop(self, timeout=None): diff --git a/taskflow/engines/action_engine/executor.py b/taskflow/engines/action_engine/executor.py index c630c161..2f8ddb09 100644 --- a/taskflow/engines/action_engine/executor.py +++ b/taskflow/engines/action_engine/executor.py @@ -19,6 +19,7 @@ import collections from multiprocessing import managers import os import pickle +import threading from oslo_utils import excutils from oslo_utils import reflection @@ -240,7 +241,7 @@ class _Dispatcher(object): raise ValueError("Provided dispatch periodicity must be greater" " than zero and not '%s'" % dispatch_periodicity) self._targets = {} - self._dead = threading_utils.Event() + self._dead = threading.Event() self._dispatch_periodicity = dispatch_periodicity self._stop_when_empty = False diff --git a/taskflow/engines/worker_based/proxy.py b/taskflow/engines/worker_based/proxy.py index e9d2ec22..386e3779 100644 --- a/taskflow/engines/worker_based/proxy.py +++ b/taskflow/engines/worker_based/proxy.py @@ -15,6 +15,7 @@ # under the License. import collections +import threading import kombu from kombu import exceptions as kombu_exceptions @@ -22,7 +23,6 @@ import six from taskflow.engines.worker_based import dispatcher from taskflow import logging -from taskflow.utils import threading_utils LOG = logging.getLogger(__name__) @@ -75,7 +75,7 @@ class Proxy(object): self._topic = topic self._exchange_name = exchange self._on_wait = on_wait - self._running = threading_utils.Event() + self._running = threading.Event() self._dispatcher = dispatcher.TypeDispatcher( # NOTE(skudriashev): Process all incoming messages only if proxy is # running, otherwise requeue them. diff --git a/taskflow/tests/unit/conductor/test_blocking.py b/taskflow/tests/unit/conductor/test_blocking.py index 33de7807..caab904f 100644 --- a/taskflow/tests/unit/conductor/test_blocking.py +++ b/taskflow/tests/unit/conductor/test_blocking.py @@ -16,6 +16,7 @@ import collections import contextlib +import threading from zake import fake_client @@ -93,7 +94,7 @@ class BlockingConductorTest(test_utils.EngineTestBase, test.TestCase): def test_run(self): components = self.make_components() components.conductor.connect() - consumed_event = threading_utils.Event() + consumed_event = threading.Event() def on_consume(state, details): consumed_event.set() @@ -123,7 +124,7 @@ class BlockingConductorTest(test_utils.EngineTestBase, test.TestCase): def test_fail_run(self): components = self.make_components() components.conductor.connect() - consumed_event = threading_utils.Event() + consumed_event = threading.Event() def on_consume(state, details): consumed_event.set() diff --git a/taskflow/tests/unit/jobs/base.py b/taskflow/tests/unit/jobs/base.py index 8a8bee22..654702ae 100644 --- a/taskflow/tests/unit/jobs/base.py +++ b/taskflow/tests/unit/jobs/base.py @@ -15,6 +15,7 @@ # under the License. import contextlib +import threading import time from kazoo.recipe import watchers @@ -53,8 +54,8 @@ def flush(client, path=None): # before this context manager exits. if not path: path = FLUSH_PATH_TPL % uuidutils.generate_uuid() - created = threading_utils.Event() - deleted = threading_utils.Event() + created = threading.Event() + deleted = threading.Event() def on_created(data, stat): if stat is not None: @@ -126,7 +127,7 @@ class BoardTestMixin(object): self.assertRaises(excp.NotFound, self.board.wait, timeout=0.1) def test_wait_arrival(self): - ev = threading_utils.Event() + ev = threading.Event() jobs = [] def poster(wait_post=0.2): diff --git a/taskflow/tests/unit/test_listeners.py b/taskflow/tests/unit/test_listeners.py index 13aa9503..80bac47f 100644 --- a/taskflow/tests/unit/test_listeners.py +++ b/taskflow/tests/unit/test_listeners.py @@ -16,6 +16,7 @@ import contextlib import logging +import threading import time from oslo_serialization import jsonutils @@ -38,7 +39,6 @@ from taskflow.test import mock from taskflow.tests import utils as test_utils from taskflow.utils import misc from taskflow.utils import persistence_utils -from taskflow.utils import threading_utils _LOG_LEVELS = frozenset([ @@ -89,7 +89,7 @@ class TestClaimListener(test.TestCase, EngineMakerMixin): self.board.connect() def _post_claim_job(self, job_name, book=None, details=None): - arrived = threading_utils.Event() + arrived = threading.Event() def set_on_children(children): if children: diff --git a/taskflow/tests/unit/test_utils_threading_utils.py b/taskflow/tests/unit/test_utils_threading_utils.py index 66ef2d09..f354a989 100644 --- a/taskflow/tests/unit/test_utils_threading_utils.py +++ b/taskflow/tests/unit/test_utils_threading_utils.py @@ -16,6 +16,7 @@ import collections import functools +import threading import time from taskflow import test @@ -28,17 +29,12 @@ def _spinner(death): class TestThreadHelpers(test.TestCase): - def test_event_wait(self): - e = tu.Event() - e.set() - self.assertTrue(e.wait()) - def test_alive_thread_falsey(self): for v in [False, 0, None, ""]: self.assertFalse(tu.is_alive(v)) def test_alive_thread(self): - death = tu.Event() + death = threading.Event() t = tu.daemon_thread(_spinner, death) self.assertFalse(tu.is_alive(t)) t.start() @@ -48,7 +44,7 @@ class TestThreadHelpers(test.TestCase): self.assertFalse(tu.is_alive(t)) def test_daemon_thread(self): - death = tu.Event() + death = threading.Event() t = tu.daemon_thread(_spinner, death) self.assertTrue(t.daemon) @@ -59,7 +55,7 @@ class TestThreadBundle(test.TestCase): def setUp(self): super(TestThreadBundle, self).setUp() self.bundle = tu.ThreadBundle() - self.death = tu.Event() + self.death = threading.Event() self.addCleanup(self.bundle.stop) self.addCleanup(self.death.set) diff --git a/taskflow/tests/unit/worker_based/test_executor.py b/taskflow/tests/unit/worker_based/test_executor.py index e7831783..d944b64b 100644 --- a/taskflow/tests/unit/worker_based/test_executor.py +++ b/taskflow/tests/unit/worker_based/test_executor.py @@ -14,6 +14,7 @@ # License for the specific language governing permissions and limitations # under the License. +import threading import time from concurrent import futures @@ -26,7 +27,6 @@ from taskflow import test from taskflow.test import mock from taskflow.tests import utils as test_utils from taskflow.types import failure -from taskflow.utils import threading_utils class TestWorkerTaskExecutor(test.MockTestCase): @@ -43,7 +43,7 @@ class TestWorkerTaskExecutor(test.MockTestCase): self.executor_uuid = 'executor-uuid' self.executor_exchange = 'executor-exchange' self.executor_topic = 'test-topic1' - self.proxy_started_event = threading_utils.Event() + self.proxy_started_event = threading.Event() # patch classes self.proxy_mock, self.proxy_inst_mock = self.patchClass( diff --git a/taskflow/tests/unit/worker_based/test_message_pump.py b/taskflow/tests/unit/worker_based/test_message_pump.py index 59bddbda..c1a16f58 100644 --- a/taskflow/tests/unit/worker_based/test_message_pump.py +++ b/taskflow/tests/unit/worker_based/test_message_pump.py @@ -14,6 +14,8 @@ # License for the specific language governing permissions and limitations # under the License. +import threading + from oslo_utils import uuidutils from taskflow.engines.worker_based import dispatcher @@ -31,7 +33,7 @@ POLLING_INTERVAL = 0.01 class TestMessagePump(test.TestCase): def test_notify(self): - barrier = threading_utils.Event() + barrier = threading.Event() on_notify = mock.MagicMock() on_notify.side_effect = lambda *args, **kwargs: barrier.set() @@ -56,7 +58,7 @@ class TestMessagePump(test.TestCase): on_notify.assert_called_with({}, mock.ANY) def test_response(self): - barrier = threading_utils.Event() + barrier = threading.Event() on_response = mock.MagicMock() on_response.side_effect = lambda *args, **kwargs: barrier.set() diff --git a/taskflow/tests/utils.py b/taskflow/tests/utils.py index 43f208bb..6148e0a6 100644 --- a/taskflow/tests/utils.py +++ b/taskflow/tests/utils.py @@ -16,6 +16,7 @@ import contextlib import string +import threading import six @@ -26,7 +27,6 @@ from taskflow import retry from taskflow import task from taskflow.types import failure from taskflow.utils import kazoo_utils -from taskflow.utils import threading_utils ARGS_KEY = '__args__' KWARGS_KEY = '__kwargs__' @@ -365,7 +365,7 @@ class WaitForOneFromTask(ProgressingTask): self.wait_states = [wait_states] else: self.wait_states = wait_states - self.event = threading_utils.Event() + self.event = threading.Event() def execute(self): if not self.event.wait(WAIT_TIMEOUT): diff --git a/taskflow/types/fsm.py b/taskflow/types/fsm.py index afb4eb1d..1ed3193f 100644 --- a/taskflow/types/fsm.py +++ b/taskflow/types/fsm.py @@ -14,10 +14,7 @@ # License for the specific language governing permissions and limitations # under the License. -try: - from collections import OrderedDict # noqa -except ImportError: - from ordereddict import OrderedDict # noqa +import collections import six @@ -66,7 +63,7 @@ class FSM(object): """ def __init__(self, start_state): self._transitions = {} - self._states = OrderedDict() + self._states = collections.OrderedDict() self._start_state = start_state self._current = None self.frozen = False @@ -127,7 +124,7 @@ class FSM(object): 'on_enter': on_enter, 'on_exit': on_exit, } - self._transitions[state] = OrderedDict() + self._transitions[state] = collections.OrderedDict() @misc.disallow_when_frozen(FrozenMachine) def add_reaction(self, state, event, reaction, *args, **kwargs): diff --git a/taskflow/types/periodic.py b/taskflow/types/periodic.py index 7314988c..20d9a52e 100644 --- a/taskflow/types/periodic.py +++ b/taskflow/types/periodic.py @@ -16,6 +16,7 @@ import heapq import inspect +import threading from debtcollector import removals from oslo_utils import reflection @@ -23,7 +24,6 @@ import six from taskflow import logging from taskflow.utils import misc -from taskflow.utils import threading_utils as tu LOG = logging.getLogger(__name__) @@ -146,7 +146,7 @@ class PeriodicWorker(object): @removals.removed_kwarg('tombstone', version="0.8", removal_version="?") def __init__(self, callables, tombstone=None): if tombstone is None: - self._tombstone = tu.Event() + self._tombstone = threading.Event() else: self._tombstone = tombstone self._callables = [] diff --git a/taskflow/types/sets.py b/taskflow/types/sets.py index 527ad2a7..a462189f 100644 --- a/taskflow/types/sets.py +++ b/taskflow/types/sets.py @@ -17,11 +17,6 @@ import collections import itertools -try: - from collections import OrderedDict # noqa -except ImportError: - from ordereddict import OrderedDict # noqa - import six @@ -51,7 +46,7 @@ class OrderedSet(collections.Set, collections.Hashable): __slots__ = ['_data'] def __init__(self, iterable=None): - self._data = _merge_in(OrderedDict(), iterable) + self._data = _merge_in(collections.OrderedDict(), iterable) def __hash__(self): return self._hash() diff --git a/taskflow/types/timing.py b/taskflow/types/timing.py index 791bce3e..57fe1287 100644 --- a/taskflow/types/timing.py +++ b/taskflow/types/timing.py @@ -14,9 +14,9 @@ # License for the specific language governing permissions and limitations # under the License. -from oslo_utils import timeutils +import threading -from taskflow.utils import threading_utils +from oslo_utils import timeutils #: Moved to oslo.utils (just reference them from there until a later time). Split = timeutils.Split @@ -35,7 +35,7 @@ class Timeout(object): if timeout < 0: raise ValueError("Timeout must be >= 0 and not %s" % (timeout)) self._timeout = timeout - self._event = threading_utils.Event() + self._event = threading.Event() def interrupt(self): self._event.set() diff --git a/taskflow/utils/threading_utils.py b/taskflow/utils/threading_utils.py index b2790657..e859fffa 100644 --- a/taskflow/utils/threading_utils.py +++ b/taskflow/utils/threading_utils.py @@ -16,7 +16,6 @@ import collections import multiprocessing -import sys import threading import six @@ -25,25 +24,6 @@ from six.moves import _thread from taskflow.utils import misc -if sys.version_info[0:2] == (2, 6): - # This didn't return that was/wasn't set in 2.6, since we actually care - # whether it did or didn't add that feature by taking the code from 2.7 - # that added this functionality... - # - # TODO(harlowja): remove when we can drop 2.6 support. - class Event(threading._Event): - def wait(self, timeout=None): - self.__cond.acquire() - try: - if not self.__flag: - self.__cond.wait(timeout) - return self.__flag - finally: - self.__cond.release() -else: - Event = threading.Event - - def is_alive(thread): """Helper to determine if a thread is alive (handles none safely).""" if not thread: From 211cf2f084a9772d9c9c731ec12cc9012588d2ee Mon Sep 17 00:00:00 2001 From: OpenStack Proposal Bot Date: Mon, 22 Jun 2015 08:28:48 +0000 Subject: [PATCH 226/246] Updated from global requirements Change-Id: I4d8e9524d47977e3d985f48988d1a5b1b9ae6cf0 --- requirements.txt | 14 +++++++------- setup.py | 1 - test-requirements.txt | 14 +++++++------- 3 files changed, 14 insertions(+), 15 deletions(-) diff --git a/requirements.txt b/requirements.txt index a2ce9064..f4511441 100644 --- a/requirements.txt +++ b/requirements.txt @@ -3,7 +3,7 @@ # process, which may cause wedges in the gate later. # See: https://bugs.launchpad.net/pbr/+bug/1384919 for why this is here... -pbr>=0.11,<2.0 +pbr<2.0,>=0.11 # Packages needed for using this library. @@ -17,7 +17,7 @@ six>=1.9.0 enum34 # For reader/writer + interprocess locks. -fasteners>=0.7 # Apache-2.0 +fasteners>=0.7 # Apache-2.0 # Very nice graph library networkx>=1.8 @@ -26,20 +26,20 @@ networkx>=1.8 contextlib2>=0.4.0 # PSF License # Used for backend storage engine loading. -stevedore>=1.5.0 # Apache-2.0 +stevedore>=1.5.0 # Apache-2.0 # Backport for concurrent.futures which exists in 3.2+ futures>=3.0 # Used for structured input validation -jsonschema>=2.0.0,<3.0.0,!=2.5.0 +jsonschema!=2.5.0,<3.0.0,>=2.0.0 # For common utilities -oslo.utils>=1.6.0 # Apache-2.0 -oslo.serialization>=1.4.0 # Apache-2.0 +oslo.utils>=1.6.0 # Apache-2.0 +oslo.serialization>=1.4.0 # Apache-2.0 # For lru caches and such cachetools>=1.0.0 # MIT License # For deprecation of things -debtcollector>=0.3.0 # Apache-2.0 +debtcollector>=0.3.0 # Apache-2.0 diff --git a/setup.py b/setup.py index 73637574..056c16c2 100644 --- a/setup.py +++ b/setup.py @@ -1,4 +1,3 @@ -#!/usr/bin/env python # Copyright (c) 2013 Hewlett-Packard Development Company, L.P. # # Licensed under the Apache License, Version 2.0 (the "License"); diff --git a/test-requirements.txt b/test-requirements.txt index f25f6d40..62711500 100644 --- a/test-requirements.txt +++ b/test-requirements.txt @@ -3,9 +3,9 @@ # process, which may cause wedges in the gate later. hacking<0.11,>=0.10.0 -oslotest>=1.5.1 # Apache-2.0 +oslotest>=1.5.1 # Apache-2.0 mock>=1.0 -testtools>=0.9.36,!=1.2.0 +testtools>=1.4.0 testscenarios>=0.4 # Used for testing the WBE engine. @@ -13,17 +13,17 @@ kombu>=3.0.7 # Used for testing zookeeper & backends. zake>=0.1.6 # Apache-2.0 -kazoo>=1.3.1,!=2.1 +kazoo!=2.1,>=1.3.1 # Used for testing database persistence backends. -SQLAlchemy>=0.9.7,<1.1.0 +SQLAlchemy<1.1.0,>=0.9.7 alembic>=0.7.2 psycopg2 -PyMySQL>=0.6.2 # MIT License +PyMySQL>=0.6.2 # MIT License # Used for making sure we still work with eventlet. eventlet>=0.17.3 # Docs build jobs need these packages. -sphinx>=1.1.2,!=1.2.0,!=1.3b1,<1.3 -oslosphinx>=2.5.0 # Apache-2.0 +sphinx!=1.2.0,!=1.3b1,<1.3,>=1.1.2 +oslosphinx>=2.5.0 # Apache-2.0 From 12361c248e76d82735b239efe52af14e3549b6d7 Mon Sep 17 00:00:00 2001 From: OpenStack Proposal Bot Date: Mon, 22 Jun 2015 21:54:51 +0000 Subject: [PATCH 227/246] Updated from global requirements Change-Id: Ib12b6f769f2f56e176de426b7cbc05c02cac56d6 --- test-requirements.txt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/test-requirements.txt b/test-requirements.txt index 62711500..275836d9 100644 --- a/test-requirements.txt +++ b/test-requirements.txt @@ -13,7 +13,7 @@ kombu>=3.0.7 # Used for testing zookeeper & backends. zake>=0.1.6 # Apache-2.0 -kazoo!=2.1,>=1.3.1 +kazoo>=2.2 # Used for testing database persistence backends. SQLAlchemy<1.1.0,>=0.9.7 @@ -22,7 +22,7 @@ psycopg2 PyMySQL>=0.6.2 # MIT License # Used for making sure we still work with eventlet. -eventlet>=0.17.3 +eventlet>=0.17.4 # Docs build jobs need these packages. sphinx!=1.2.0,!=1.3b1,<1.3,>=1.1.2 From 153febce9ad827034f9aa70986f296c4292e6d0a Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Mon, 22 Jun 2015 17:15:20 -0700 Subject: [PATCH 228/246] Add docstrings on runtime objects methods and link to them in docs Change-Id: Ia85fe4b7849068271a5452d622fbed163dc9847d --- doc/source/engines.rst | 4 +++- taskflow/engines/action_engine/runtime.py | 23 ++++++++++++++++++++--- 2 files changed, 23 insertions(+), 4 deletions(-) diff --git a/doc/source/engines.rst b/doc/source/engines.rst index abdbdb62..24864b3a 100644 --- a/doc/source/engines.rst +++ b/doc/source/engines.rst @@ -237,7 +237,9 @@ saved to internal engine variables (these object help in execution of atoms, analyzing the graph and performing other internal engine activities). At the finishing of this stage a :py:class:`~taskflow.engines.action_engine.runtime.Runtime` object is created -which contains references to all needed runtime components. +which contains references to all needed runtime components and its +:py:func:`~taskflow.engines.action_engine.runtime.Runtime.compile` is called +to compile a cache of frequently used execution helper objects. Preparation ----------- diff --git a/taskflow/engines/action_engine/runtime.py b/taskflow/engines/action_engine/runtime.py index 061cca4c..0439da17 100644 --- a/taskflow/engines/action_engine/runtime.py +++ b/taskflow/engines/action_engine/runtime.py @@ -44,9 +44,14 @@ class Runtime(object): self._atom_cache = {} def compile(self): - # Build out a cache of commonly used item that are associated - # with the contained atoms (by name), and are useful to have for - # quick lookup on... + """Compiles & caches frequently used execution helper objects. + + Build out a cache of commonly used item that are associated + with the contained atoms (by name), and are useful to have for + quick lookup on (for example, the change state handler function for + each atom, the scope walker object for each atom, the task or retry + specific scheduler and so-on). + """ change_state_handlers = { 'task': functools.partial(self.task_action.change_state, progress=0.0), @@ -152,6 +157,7 @@ class Runtime(object): # consumption... def reset_nodes(self, atoms, state=st.PENDING, intention=st.EXECUTE): + """Resets all the provided atoms to the given state and intention.""" tweaked = [] for atom in atoms: metadata = self._atom_cache[atom.name] @@ -165,13 +171,24 @@ class Runtime(object): return tweaked def reset_all(self, state=st.PENDING, intention=st.EXECUTE): + """Resets all atoms to the given state and intention.""" return self.reset_nodes(self.analyzer.iterate_all_nodes(), state=state, intention=intention) def reset_subgraph(self, atom, state=st.PENDING, intention=st.EXECUTE): + """Resets a atoms subgraph to the given state and intention. + + The subgraph is contained of all of the atoms successors. + """ return self.reset_nodes(self.analyzer.iterate_subgraph(atom), state=state, intention=intention) def retry_subflow(self, retry): + """Prepares a retrys + its subgraph for execution. + + This sets the retrys intention to ``EXECUTE`` and resets all of its + subgraph (its successors) to the ``PENDING`` state with an ``EXECUTE`` + intention. + """ self.storage.set_atom_intention(retry.name, st.EXECUTE) self.reset_subgraph(retry) From a629f3fc9a66214317a0fadf4db9a26481951b8b Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Tue, 23 Jun 2015 16:02:41 -0700 Subject: [PATCH 229/246] Add history.rst that uses generated 'ChangeLog' file Change-Id: Iba1947128b7e2da7dbff5669e47883fd388e171d --- ChangeLog | 1 + doc/source/history.rst | 2 ++ doc/source/index.rst | 8 ++++++++ 3 files changed, 11 insertions(+) create mode 100644 ChangeLog create mode 100644 doc/source/history.rst diff --git a/ChangeLog b/ChangeLog new file mode 100644 index 00000000..ba4c69b7 --- /dev/null +++ b/ChangeLog @@ -0,0 +1 @@ +.. This is a generated file! Do not edit. diff --git a/doc/source/history.rst b/doc/source/history.rst new file mode 100644 index 00000000..db8340b5 --- /dev/null +++ b/doc/source/history.rst @@ -0,0 +1,2 @@ +.. include:: ../../ChangeLog + diff --git a/doc/source/index.rst b/doc/source/index.rst index 90e3bcfa..acea31cb 100644 --- a/doc/source/index.rst +++ b/doc/source/index.rst @@ -103,6 +103,14 @@ projects, frameworks and libraries. shelf +History +------- + +.. toctree:: + :maxdepth: 2 + + history + Indices and tables ================== From 934b2bc0e9a614bd07ab3fbd4275cb640cb61ea5 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Thu, 23 Apr 2015 12:36:10 -0700 Subject: [PATCH 230/246] Build-out + test a redis backed jobboard Part of blueprint taskflow-redis-jobs Change-Id: I7c94e2201c5d933c8a1ec73fc0cf705962e5eef6 --- doc/source/jobs.rst | 21 + doc/source/utils.rst | 5 + setup.cfg | 1 + taskflow/jobs/backends/impl_redis.py | 951 ++++++++++++++++++++ taskflow/jobs/backends/impl_zookeeper.py | 187 ++-- taskflow/jobs/base.py | 89 +- taskflow/tests/unit/jobs/base.py | 117 +-- taskflow/tests/unit/jobs/test_entrypoint.py | 13 + taskflow/tests/unit/jobs/test_redis_job.py | 81 ++ taskflow/tests/unit/jobs/test_zk_job.py | 109 ++- taskflow/tests/utils.py | 14 + taskflow/utils/misc.py | 49 +- taskflow/utils/redis_utils.py | 133 +++ test-requirements.txt | 3 + 14 files changed, 1518 insertions(+), 255 deletions(-) create mode 100644 taskflow/jobs/backends/impl_redis.py create mode 100644 taskflow/tests/unit/jobs/test_redis_job.py create mode 100644 taskflow/utils/redis_utils.py diff --git a/doc/source/jobs.rst b/doc/source/jobs.rst index 7dd73063..2b826da4 100644 --- a/doc/source/jobs.rst +++ b/doc/source/jobs.rst @@ -209,6 +209,20 @@ Additional *configuration* parameters: See :py:class:`~taskflow.jobs.backends.impl_zookeeper.ZookeeperJobBoard` for implementation details. +Redis +----- + +**Board type**: ``'redis'`` + +Uses `redis`_ to provide the jobboard capabilities and semantics by using +a redis hash datastructure and individual job ownership keys (that can +optionally expire after a given amount of time). + +.. note:: + + See :py:class:`~taskflow.jobs.backends.impl_redis.RedisJobBoard` + for implementation details. + Considerations ============== @@ -272,11 +286,17 @@ Zookeeper .. automodule:: taskflow.jobs.backends.impl_zookeeper +Redis +----- + +.. automodule:: taskflow.jobs.backends.impl_redis + Hierarchy ========= .. inheritance-diagram:: taskflow.jobs.base + taskflow.jobs.backends.impl_redis taskflow.jobs.backends.impl_zookeeper :parts: 1 @@ -284,3 +304,4 @@ Hierarchy .. _zookeeper: http://zookeeper.apache.org/ .. _kazoo: http://kazoo.readthedocs.org/ .. _stevedore: http://stevedore.readthedocs.org/ +.. _redis: http://redis.io/ diff --git a/doc/source/utils.rst b/doc/source/utils.rst index ac0dd5c4..22a27089 100644 --- a/doc/source/utils.rst +++ b/doc/source/utils.rst @@ -43,6 +43,11 @@ Persistence .. automodule:: taskflow.utils.persistence_utils +Redis +~~~~~ + +.. automodule:: taskflow.utils.redis_utils + Schema ~~~~~~ diff --git a/setup.cfg b/setup.cfg index ac5b566a..de69b76a 100644 --- a/setup.cfg +++ b/setup.cfg @@ -34,6 +34,7 @@ packages = [entry_points] taskflow.jobboards = zookeeper = taskflow.jobs.backends.impl_zookeeper:ZookeeperJobBoard + redis = taskflow.jobs.backends.impl_redis:RedisJobBoard taskflow.conductors = blocking = taskflow.conductors.backends.impl_blocking:BlockingConductor diff --git a/taskflow/jobs/backends/impl_redis.py b/taskflow/jobs/backends/impl_redis.py new file mode 100644 index 00000000..4d61dc01 --- /dev/null +++ b/taskflow/jobs/backends/impl_redis.py @@ -0,0 +1,951 @@ +# -*- coding: utf-8 -*- + +# Copyright (C) 2015 Yahoo! Inc. 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. + +import contextlib +import datetime +import string +import threading +import time + +import fasteners +import msgpack +from oslo_serialization import msgpackutils +from oslo_utils import strutils +from oslo_utils import timeutils +from oslo_utils import uuidutils +from redis import exceptions as redis_exceptions +import six +from six.moves import range as compat_range + +from taskflow import exceptions as exc +from taskflow.jobs import base +from taskflow import logging +from taskflow import states +from taskflow.types import timing +from taskflow.utils import misc +from taskflow.utils import redis_utils as ru + + +LOG = logging.getLogger(__name__) + + +@contextlib.contextmanager +def _translate_failures(): + """Translates common redis exceptions into taskflow exceptions.""" + try: + yield + except redis_exceptions.ConnectionError: + exc.raise_with_cause(exc.JobFailure, "Failed to connect to redis") + except redis_exceptions.TimeoutError: + exc.raise_with_cause(exc.JobFailure, + "Failed to communicate with redis, connection" + " timed out") + except redis_exceptions.RedisError: + exc.raise_with_cause(exc.JobFailure, + "Failed to communicate with redis," + " internal error") + + +class RedisJob(base.Job): + """A redis job.""" + + def __init__(self, board, name, sequence, key, + uuid=None, details=None, + created_on=None, backend=None, + book=None, book_data=None): + super(RedisJob, self).__init__(board, name, + uuid=uuid, details=details, + backend=backend, + book=book, book_data=book_data) + self._created_on = created_on + self._client = board._client + self._redis_version = board._redis_version + self._sequence = sequence + self._key = key + self._last_modified_key = board.join(key + board.LAST_MODIFIED_POSTFIX) + self._owner_key = board.join(key + board.OWNED_POSTFIX) + + @property + def key(self): + """Key (in board listings/trash hash) the job data is stored under.""" + return self._key + + @property + def last_modified_key(self): + """Key the job last modified data is stored under.""" + return self._last_modified_key + + @property + def owner_key(self): + """Key the job claim + data of the owner is stored under.""" + return self._owner_key + + @property + def sequence(self): + """Sequence number of the current job.""" + return self._sequence + + def expires_in(self): + """How many seconds until the claim expires. + + Returns the number of seconds until the ownership entry expires or + :attr:`~taskflow.utils.redis_utils.UnknownExpire.DOES_NOT_EXPIRE` or + :attr:`~taskflow.utils.redis_utils.UnknownExpire.KEY_NOT_FOUND` if it + does not expire or if the expiry can not be determined (perhaps the + :attr:`.owner_key` expired at/before time of inquiry?). + """ + with _translate_failures(): + return ru.get_expiry(self._client, self._owner_key, + prior_version=self._redis_version) + + def extend_expiry(self, expiry): + """Extends the owner key (aka the claim) expiry for this job. + + NOTE(harlowja): if the claim for this job did **not** previously + have an expiry associated with it, calling this method will create + one (and after that time elapses the claim on this job will cease + to exist). + + Returns ``True`` if the expiry request was performed + otherwise ``False``. + """ + with _translate_failures(): + return ru.apply_expiry(self._client, self._owner_key, expiry, + prior_version=self._redis_version) + + def __lt__(self, other): + if self.created_on == other.created_on: + return self.sequence < other.sequence + else: + return self.created_on < other.created_on + + @property + def created_on(self): + return self._created_on + + @property + def last_modified(self): + with _translate_failures(): + raw_last_modified = self._client.get(self._last_modified_key) + last_modified = None + if raw_last_modified: + last_modified = self._board._loads( + raw_last_modified, root_types=(datetime.datetime,)) + # NOTE(harlowja): just incase this is somehow busted (due to time + # sync issues/other), give back the most recent one (since redis + # does not maintain clock information; we could have this happen + # due to now clients who mutate jobs also send the time in). + last_modified = max(last_modified, self._created_on) + return last_modified + + @property + def state(self): + listings_key = self._board.listings_key + owner_key = self._owner_key + listings_sub_key = self._key + + def _do_fetch(p): + # NOTE(harlowja): state of a job in redis is not set into any + # explicit 'state' field, but is maintained by what nodes exist in + # redis instead (ie if a owner key exists, then we know a owner + # is active, if no job data exists and no owner, then we know that + # the job is unclaimed, and so-on)... + p.multi() + p.hexists(listings_key, listings_sub_key) + p.exists(owner_key) + job_exists, owner_exists = p.execute() + if not job_exists: + if owner_exists: + # This should **not** be possible due to lua code ordering + # but let's log an INFO statement if it does happen (so + # that it can be investigated)... + LOG.info("Unexpected owner key found at '%s' when job" + " key '%s[%s]' was not found", owner_key, + listings_key, listings_sub_key) + return states.COMPLETE + else: + if owner_exists: + return states.CLAIMED + else: + return states.UNCLAIMED + + with _translate_failures(): + return self._client.transaction(_do_fetch, + listings_key, owner_key, + value_from_callable=True) + + +class RedisJobBoard(base.JobBoard): + """A jobboard backed by `redis`_. + + Powered by the `redis-py `_ library. + + This jobboard creates job entries by listing jobs in a redis `hash`_. This + hash contains jobs that can be actively worked on by (and examined/claimed + by) some set of eligible consumers. Job posting is typically performed + using the :meth:`.post` method (this creates a hash entry with job + contents/details encoded in `msgpack`_). The users of these + jobboard(s) (potentially on disjoint sets of machines) can then + iterate over the available jobs and decide if they want to attempt to + claim one of the jobs they have iterated over. If so they will then + attempt to contact redis and they will attempt to create a key in + redis (using a embedded lua script to perform this atomically) to claim a + desired job. If the entity trying to use the jobboard to :meth:`.claim` + the job is able to create that lock/owner key then it will be + allowed (and expected) to perform whatever *work* the contents of that + job described. Once the claiming entity is finished the lock/owner key + and the `hash`_ entry will be deleted (if successfully completed) in a + single request (also using a embedded lua script to perform this + atomically). If the claiming entity is not successful (or the entity + that claimed the job dies) the lock/owner key can be released + automatically (by **optional** usage of a claim expiry) or by + using :meth:`.abandon` to manually abandon the job so that it can be + consumed/worked on by others. + + NOTE(harlowja): by default the :meth:`.claim` has no expiry (which + means claims will be persistent, even under claiming entity failure). To + ensure a expiry occurs pass a numeric value for the ``expiry`` keyword + argument to the :meth:`.claim` method that defines how many seconds the + claim should be retained for. When an expiry is used ensure that that + claim is kept alive while it is being worked on by using + the :py:meth:`~.RedisJob.extend_expiry` method periodically. + + .. _msgpack: http://msgpack.org/ + .. _redis: http://redis.io/ + .. _hash: http://redis.io/topics/data-types#hashes + """ + + CLIENT_CONF_TRANSFERS = tuple([ + # Host config... + ('host', str), + ('port', int), + + # See: http://redis.io/commands/auth + ('password', str), + + # Data encoding/decoding + error handling + ('encoding', str), + ('encoding_errors', str), + + # Connection settings. + ('socket_timeout', float), + ('socket_connect_timeout', float), + + # This one negates the usage of host, port, socket connection + # settings as it doesn't use the same kind of underlying socket... + ('unix_socket_path', str), + + # Do u want ssl??? + ('ssl', strutils.bool_from_string), + ('ssl_keyfile', str), + ('ssl_certfile', str), + ('ssl_cert_reqs', str), + ('ssl_ca_certs', str), + + # See: http://www.rediscookbook.org/multiple_databases.html + ('db', int), + ]) + """ + Keys (and value type converters) that we allow to proxy from the jobboard + configuration into the redis client (used to configure the redis client + internals if no explicit client is provided via the ``client`` keyword + argument). + + See: http://redis-py.readthedocs.org/en/latest/#redis.Redis + + See: https://github.com/andymccurdy/redis-py/blob/2.10.3/redis/client.py + """ + + #: Postfix (combined with job key) used to make a jobs owner key. + OWNED_POSTFIX = b".owned" + + #: Postfix (combined with job key) used to make a jobs last modified key. + LAST_MODIFIED_POSTFIX = b".last_modified" + + #: Default namespace for keys when none is provided. + DEFAULT_NAMESPACE = b'taskflow' + + MIN_REDIS_VERSION = (2, 6) + """ + Minimum redis version this backend requires. + + This version is required since we need the built-in server-side lua + scripting support that is included in 2.6 and newer. + """ + + NAMESPACE_SEP = b':' + """ + Separator that is used to combine a key with the namespace (to get + the **actual** key that will be used). + """ + + KEY_PIECE_SEP = b'.' + """ + Separator that is used to combine a bunch of key pieces together (to get + the **actual** key that will be used). + """ + + #: Expected lua response status field when call is ok. + SCRIPT_STATUS_OK = "ok" + + #: Expected lua response status field when call is **not** ok. + SCRIPT_STATUS_ERROR = "error" + + #: Expected lua script error response when the owner is not as expected. + SCRIPT_NOT_EXPECTED_OWNER = "Not expected owner!" + + #: Expected lua script error response when the owner is not findable. + SCRIPT_UNKNOWN_OWNER = "Unknown owner!" + + #: Expected lua script error response when the job is not findable. + SCRIPT_UNKNOWN_JOB = "Unknown job!" + + #: Expected lua script error response when the job is already claimed. + SCRIPT_ALREADY_CLAIMED = "Job already claimed!" + + SCRIPT_TEMPLATES = { + 'consume': """ +-- Extract *all* the variables (so we can easily know what they are)... +local owner_key = KEYS[1] +local listings_key = KEYS[2] +local last_modified_key = KEYS[3] + +local expected_owner = ARGV[1] +local job_key = ARGV[2] +local result = {} +if redis.call("hexists", listings_key, job_key) == 1 then + if redis.call("exists", owner_key) == 1 then + local owner = redis.call("get", owner_key) + if owner ~= expected_owner then + result["status"] = "${error}" + result["reason"] = "${not_expected_owner}" + result["owner"] = owner + else + -- The order is important here, delete the owner first (and if + -- that blows up, the job data will still exist so it can be + -- worked on again, instead of the reverse)... + redis.call("del", owner_key, last_modified_key) + redis.call("hdel", listings_key, job_key) + result["status"] = "${ok}" + end + else + result["status"] = "${error}" + result["reason"] = "${unknown_owner}" + end +else + result["status"] = "${error}" + result["reason"] = "${unknown_job}" +end +return cmsgpack.pack(result) +""", + 'claim': """ +local function apply_ttl(key, ms_expiry) + if ms_expiry ~= nil then + redis.call("pexpire", key, ms_expiry) + end +end + +-- Extract *all* the variables (so we can easily know what they are)... +local owner_key = KEYS[1] +local listings_key = KEYS[2] +local last_modified_key = KEYS[3] + +local expected_owner = ARGV[1] +local job_key = ARGV[2] +local last_modified_blob = ARGV[3] + +-- If this is non-numeric (which it may be) this becomes nil +local ms_expiry = nil +if ARGV[4] ~= "none" then + ms_expiry = tonumber(ARGV[4]) +end +local result = {} +if redis.call("hexists", listings_key, job_key) == 1 then + if redis.call("exists", owner_key) == 1 then + local owner = redis.call("get", owner_key) + if owner == expected_owner then + -- Owner is the same, leave it alone... + redis.call("set", last_modified_key, last_modified_blob) + apply_ttl(owner_key, ms_expiry) + result["status"] = "${ok}" + else + result["status"] = "${error}" + result["reason"] = "${already_claimed}" + result["owner"] = owner + end + else + redis.call("set", owner_key, expected_owner) + redis.call("set", last_modified_key, last_modified_blob) + apply_ttl(owner_key, ms_expiry) + result["status"] = "${ok}" + end +else + result["status"] = "${error}" + result["reason"] = "${unknown_job}" +end +return cmsgpack.pack(result) +""", + 'abandon': """ +-- Extract *all* the variables (so we can easily know what they are)... +local owner_key = KEYS[1] +local listings_key = KEYS[2] +local last_modified_key = KEYS[3] + +local expected_owner = ARGV[1] +local job_key = ARGV[2] +local last_modified_blob = ARGV[3] +local result = {} +if redis.call("hexists", listings_key, job_key) == 1 then + if redis.call("exists", owner_key) == 1 then + local owner = redis.call("get", owner_key) + if owner ~= expected_owner then + result["status"] = "${error}" + result["reason"] = "${not_expected_owner}" + result["owner"] = owner + else + redis.call("del", owner_key) + redis.call("set", last_modified_key, last_modified_blob) + result["status"] = "${ok}" + end + else + result["status"] = "${error}" + result["reason"] = "${unknown_owner}" + end +else + result["status"] = "${error}" + result["reason"] = "${unknown_job}" +end +return cmsgpack.pack(result) +""", + 'trash': """ +-- Extract *all* the variables (so we can easily know what they are)... +local owner_key = KEYS[1] +local listings_key = KEYS[2] +local last_modified_key = KEYS[3] +local trash_listings_key = KEYS[4] + +local expected_owner = ARGV[1] +local job_key = ARGV[2] +local last_modified_blob = ARGV[3] +local result = {} +if redis.call("hexists", listings_key, job_key) == 1 then + local raw_posting = redis.call("hget", listings_key, job_key) + if redis.call("exists", owner_key) == 1 then + local owner = redis.call("get", owner_key) + if owner ~= expected_owner then + result["status"] = "${error}" + result["reason"] = "${not_expected_owner}" + result["owner"] = owner + else + -- This ordering is important (try to first move the value + -- and only if that works do we try to do any deletions)... + redis.call("hset", trash_listings_key, job_key, raw_posting) + redis.call("set", last_modified_key, last_modified_blob) + redis.call("del", owner_key) + redis.call("hdel", listings_key, job_key) + result["status"] = "${ok}" + end + else + result["status"] = "${error}" + result["reason"] = "${unknown_owner}" + end +else + result["status"] = "${error}" + result["reason"] = "${unknown_job}" +end +return cmsgpack.pack(result) +""", + } + """`Lua`_ **template** scripts that will be used by various methods (they + are turned into real scripts and loaded on call into the :func:`.connect` + method). + + Some things to note: + + - The lua script is ran serially, so when this runs no other command will + be mutating the backend (and redis also ensures that no other script + will be running) so atomicity of these scripts are guaranteed by redis. + + - Transactions were considered (and even mostly implemented) but + ultimately rejected since redis does not support rollbacks and + transactions can **not** be interdependent (later operations can **not** + depend on the results of earlier operations). Both of these issues limit + our ability to correctly report errors (with useful messages) and to + maintain consistency under failure/contention (due to the inability to + rollback). A third and final blow to using transactions was to + correctly use them we would have to set a watch on a *very* contentious + key (the listings key) which would under load cause clients to retry more + often then would be desired (this also increases network load, CPU + cycles used, transactions failures triggered and so on). + + - Partial transaction execution is possible due to pre/post ``EXEC`` + failures (and the lack of rollback makes this worse). + + So overall after thinking, it seemed like having little lua scripts + was not that bad (even if it is somewhat convoluted) due to the above and + public mentioned issues with transactions. In general using lua scripts + for this purpose seems to be somewhat common practice and it solves the + issues that came up when transactions were considered & implemented. + + Some links about redis (and redis + lua) that may be useful to look over: + + - `Atomicity of scripts`_ + - `Scripting and transactions`_ + - `Why redis does not support rollbacks`_ + - `Intro to lua for redis programmers`_ + - `Five key takeaways for developing with redis`_ + - `Everything you always wanted to know about redis`_ (slides) + + .. _Lua: http://www.lua.org/ + .. _Atomicity of scripts: http://redis.io/commands/eval#atomicity-of-\ + scripts + .. _Scripting and transactions: http://redis.io/topics/transactions#redis-\ + scripting-and-transactions + .. _Why redis does not support rollbacks: http://redis.io/topics/transa\ + ctions#why-redis-does-not-suppo\ + rt-roll-backs + .. _Intro to lua for redis programmers: http://www.redisgreen.net/blog/int\ + ro-to-lua-for-redis-programmers + .. _Five key takeaways for developing with redis: https://redislabs.com/bl\ + og/5-key-takeaways-fo\ + r-developing-with-redis + .. _Everything you always wanted to know about redis: http://www.slidesh + are.net/carlosabal\ + de/everything-you-a\ + lways-wanted-to-\ + know-about-redis-b\ + ut-were-afraid-to-ask + """ + + @classmethod + def _make_client(cls, conf): + client_conf = {} + for key, value_type_converter in cls.CLIENT_CONF_TRANSFERS: + if key in conf: + if value_type_converter is not None: + client_conf[key] = value_type_converter(conf[key]) + else: + client_conf[key] = conf[key] + return ru.RedisClient(**client_conf) + + def __init__(self, name, conf, + client=None, persistence=None): + super(RedisJobBoard, self).__init__(name, conf) + self._closed = True + if client is not None: + self._client = client + self._owns_client = False + else: + self._client = self._make_client(self._conf) + # NOTE(harlowja): This client should not work until connected... + self._client.close() + self._owns_client = True + self._namespace = self._conf.get('namespace', self.DEFAULT_NAMESPACE) + self._open_close_lock = threading.RLock() + # Redis server version connected to + scripts (populated on connect). + self._redis_version = None + self._scripts = {} + # The backend to load the full logbooks from, since what is sent over + # the data connection is only the logbook uuid and name, and not the + # full logbook. + self._persistence = persistence + + def join(self, key_piece, *more_key_pieces): + """Create and return a namespaced key from many segments. + + NOTE(harlowja): all pieces that are text/unicode are converted into + their binary equivalent (if they are already binary no conversion + takes place) before being joined (as redis expects binary keys and not + unicode/text ones). + """ + namespace_pieces = [] + if self._namespace is not None: + namespace_pieces = [self._namespace, self.NAMESPACE_SEP] + else: + namespace_pieces = [] + key_pieces = [key_piece] + if more_key_pieces: + key_pieces.extend(more_key_pieces) + for i in compat_range(0, len(namespace_pieces)): + namespace_pieces[i] = misc.binary_encode(namespace_pieces[i]) + for i in compat_range(0, len(key_pieces)): + key_pieces[i] = misc.binary_encode(key_pieces[i]) + namespace = b"".join(namespace_pieces) + key = self.KEY_PIECE_SEP.join(key_pieces) + return namespace + key + + @property + def namespace(self): + """The namespace all keys will be prefixed with (or none).""" + return self._namespace + + @misc.cachedproperty + def trash_key(self): + """Key where a hash will be stored with trashed jobs in it.""" + return self.join(b"trash") + + @misc.cachedproperty + def sequence_key(self): + """Key where a integer will be stored (used to sequence jobs).""" + return self.join(b"sequence") + + @misc.cachedproperty + def listings_key(self): + """Key where a hash will be stored with active jobs in it.""" + return self.join(b"listings") + + @property + def job_count(self): + with _translate_failures(): + return self._client.hlen(self.listings_key) + + @property + def connected(self): + return not self._closed + + @fasteners.locked(lock='_open_close_lock') + def connect(self): + self.close() + if self._owns_client: + self._client = self._make_client(self._conf) + with _translate_failures(): + # The client maintains a connection pool, so do a ping and + # if that works then assume the connection works, which may or + # may not be continuously maintained (if the server dies + # at a later time, we will become aware of that when the next + # op occurs). + self._client.ping() + is_new_enough, redis_version = ru.is_server_new_enough( + self._client, self.MIN_REDIS_VERSION) + if not is_new_enough: + wanted_version = ".".join([str(p) + for p in self.MIN_REDIS_VERSION]) + if redis_version: + raise exc.JobFailure("Redis version %s or greater is" + " required (version %s is to" + " old)" % (wanted_version, + redis_version)) + else: + raise exc.JobFailure("Redis version %s or greater is" + " required" % (wanted_version)) + else: + self._redis_version = redis_version + script_params = { + # Status field values. + 'ok': self.SCRIPT_STATUS_OK, + 'error': self.SCRIPT_STATUS_ERROR, + + # Known error reasons (when status field is error). + 'not_expected_owner': self.SCRIPT_NOT_EXPECTED_OWNER, + 'unknown_owner': self.SCRIPT_UNKNOWN_OWNER, + 'unknown_job': self.SCRIPT_UNKNOWN_JOB, + 'already_claimed': self.SCRIPT_ALREADY_CLAIMED, + } + prepared_scripts = {} + for n, raw_script_tpl in six.iteritems(self.SCRIPT_TEMPLATES): + script_tpl = string.Template(raw_script_tpl) + script_blob = script_tpl.substitute(**script_params) + script = self._client.register_script(script_blob) + prepared_scripts[n] = script + self._scripts.update(prepared_scripts) + self._closed = False + + @fasteners.locked(lock='_open_close_lock') + def close(self): + if self._owns_client: + self._client.close() + self._scripts.clear() + self._redis_version = None + self._closed = True + + @staticmethod + def _dumps(obj): + try: + return msgpackutils.dumps(obj) + except (msgpack.PackException, ValueError): + # TODO(harlowja): remove direct msgpack exception access when + # oslo.utils provides easy access to the underlying msgpack + # pack/unpack exceptions.. + exc.raise_with_cause(exc.JobFailure, + "Failed to serialize object to" + " msgpack blob") + + @staticmethod + def _loads(blob, root_types=(dict,)): + try: + return misc.decode_msgpack(blob, root_types=root_types) + except (msgpack.UnpackException, ValueError): + # TODO(harlowja): remove direct msgpack exception access when + # oslo.utils provides easy access to the underlying msgpack + # pack/unpack exceptions.. + exc.raise_with_cause(exc.JobFailure, + "Failed to deserialize object from" + " msgpack blob (of length %s)" % len(blob)) + + _decode_owner = staticmethod(misc.binary_decode) + + _encode_owner = staticmethod(misc.binary_encode) + + def find_owner(self, job): + owner_key = self.join(job.key + self.OWNED_POSTFIX) + with _translate_failures(): + raw_owner = self._client.get(owner_key) + return self._decode_owner(raw_owner) + + def post(self, name, book=None, details=None): + job_uuid = uuidutils.generate_uuid() + posting = base.format_posting(job_uuid, name, + created_on=timeutils.utcnow(), + book=book, details=details) + with _translate_failures(): + sequence = self._client.incr(self.sequence_key) + posting.update({ + 'sequence': sequence, + }) + with _translate_failures(): + raw_posting = self._dumps(posting) + raw_job_uuid = six.b(job_uuid) + was_posted = bool(self._client.hsetnx(self.listings_key, + raw_job_uuid, raw_posting)) + if not was_posted: + raise exc.JobFailure("New job located at '%s[%s]' could not" + " be posted" % (self.listings_key, + raw_job_uuid)) + else: + return RedisJob(self, name, sequence, raw_job_uuid, + uuid=job_uuid, details=details, + created_on=posting['created_on'], + book=book, book_data=posting.get('book'), + backend=self._persistence) + + def wait(self, timeout=None, initial_delay=0.005, + max_delay=1.0, sleep_func=time.sleep): + if initial_delay > max_delay: + raise ValueError("Initial delay %s must be less than or equal" + " to the provided max delay %s" + % (initial_delay, max_delay)) + # This does a spin-loop that backs off by doubling the delay + # up to the provided max-delay. In the future we could try having + # a secondary client connected into redis pubsub and use that + # instead, but for now this is simpler. + w = timing.StopWatch(duration=timeout) + w.start() + delay = initial_delay + while True: + jc = self.job_count + if jc > 0: + it = self.iterjobs() + return it + else: + if w.expired(): + raise exc.NotFound("Expired waiting for jobs to" + " arrive; waited %s seconds" + % w.elapsed()) + else: + remaining = w.leftover(return_none=True) + if remaining is not None: + delay = min(delay * 2, remaining, max_delay) + else: + delay = min(delay * 2, max_delay) + sleep_func(delay) + + def iterjobs(self, only_unclaimed=False, ensure_fresh=False): + with _translate_failures(): + raw_postings = self._client.hgetall(self.listings_key) + postings = [] + for raw_job_key, raw_posting in six.iteritems(raw_postings): + posting = self._loads(raw_posting) + details = posting.get('details', {}) + job_uuid = posting['uuid'] + job = RedisJob(self, posting['name'], posting['sequence'], + raw_job_key, uuid=job_uuid, details=details, + created_on=posting['created_on'], + book_data=posting.get('book'), + backend=self._persistence) + postings.append(job) + postings = sorted(postings) + for job in postings: + if only_unclaimed: + if job.state == states.UNCLAIMED: + yield job + else: + yield job + + @base.check_who + def consume(self, job, who): + script = self._get_script('consume') + with _translate_failures(): + raw_who = self._encode_owner(who) + raw_result = script(keys=[job.owner_key, self.listings_key, + job.last_modified_key], + args=[raw_who, job.key]) + result = self._loads(raw_result) + status = result['status'] + if status != self.SCRIPT_STATUS_OK: + reason = result.get('reason') + if reason == self.SCRIPT_UNKNOWN_JOB: + raise exc.NotFound("Job %s not found to be" + " consumed" % (job.uuid)) + elif reason == self.SCRIPT_UNKNOWN_OWNER: + raise exc.NotFound("Can not consume job %s" + " which we can not determine" + " the owner of" % (job.uuid)) + elif reason == self.SCRIPT_NOT_EXPECTED_OWNER: + raw_owner = result.get('owner') + if raw_owner: + owner = self._decode_owner(raw_owner) + raise exc.JobFailure("Can not consume job %s" + " which is not owned by %s (it is" + " actively owned by %s)" + % (job.uuid, who, owner)) + else: + raise exc.JobFailure("Can not consume job %s" + " which is not owned by %s" + % (job.uuid, who)) + else: + raise exc.JobFailure("Failure to consume job %s," + " unknown internal error (reason=%s)" + % (job.uuid, reason)) + + @base.check_who + def claim(self, job, who, expiry=None): + if expiry is None: + # On the lua side none doesn't translate to nil so we have + # do to this string conversion to make sure that we can tell + # the difference. + ms_expiry = "none" + else: + ms_expiry = int(expiry * 1000.0) + if ms_expiry <= 0: + raise ValueError("Provided expiry (when converted to" + " milliseconds) must be greater" + " than zero instead of %s" % (expiry)) + script = self._get_script('claim') + with _translate_failures(): + raw_who = self._encode_owner(who) + raw_result = script(keys=[job.owner_key, self.listings_key, + job.last_modified_key], + args=[raw_who, job.key, + # NOTE(harlowja): we need to send this + # in as a blob (even if it's not + # set/used), since the format can not + # currently be created in lua... + self._dumps(timeutils.utcnow()), + ms_expiry]) + result = self._loads(raw_result) + status = result['status'] + if status != self.SCRIPT_STATUS_OK: + reason = result.get('reason') + if reason == self.SCRIPT_UNKNOWN_JOB: + raise exc.NotFound("Job %s not found to be" + " claimed" % (job.uuid)) + elif reason == self.SCRIPT_ALREADY_CLAIMED: + raw_owner = result.get('owner') + if raw_owner: + owner = self._decode_owner(raw_owner) + raise exc.UnclaimableJob("Job %s already" + " claimed by %s" + % (job.uuid, owner)) + else: + raise exc.UnclaimableJob("Job %s already" + " claimed" % (job.uuid)) + else: + raise exc.JobFailure("Failure to claim job %s," + " unknown internal error (reason=%s)" + % (job.uuid, reason)) + + @base.check_who + def abandon(self, job, who): + script = self._get_script('abandon') + with _translate_failures(): + raw_who = self._encode_owner(who) + raw_result = script(keys=[job.owner_key, self.listings_key, + job.last_modified_key], + args=[raw_who, job.key, + self._dumps(timeutils.utcnow())]) + result = self._loads(raw_result) + status = result.get('status') + if status != self.SCRIPT_STATUS_OK: + reason = result.get('reason') + if reason == self.SCRIPT_UNKNOWN_JOB: + raise exc.NotFound("Job %s not found to be" + " abandoned" % (job.uuid)) + elif reason == self.SCRIPT_UNKNOWN_OWNER: + raise exc.NotFound("Can not abandon job %s" + " which we can not determine" + " the owner of" % (job.uuid)) + elif reason == self.SCRIPT_NOT_EXPECTED_OWNER: + raw_owner = result.get('owner') + if raw_owner: + owner = self._decode_owner(raw_owner) + raise exc.JobFailure("Can not abandon job %s" + " which is not owned by %s (it is" + " actively owned by %s)" + % (job.uuid, who, owner)) + else: + raise exc.JobFailure("Can not abandon job %s" + " which is not owned by %s" + % (job.uuid, who)) + else: + raise exc.JobFailure("Failure to abandon job %s," + " unknown internal" + " error (status=%s, reason=%s)" + % (job.uuid, status, reason)) + + def _get_script(self, name): + try: + return self._scripts[name] + except KeyError: + exc.raise_with_cause(exc.NotFound, + "Can not access %s script (has this" + " board been connected?)" % name) + + @base.check_who + def trash(self, job, who): + script = self._get_script('trash') + with _translate_failures(): + raw_who = self._encode_owner(who) + raw_result = script(keys=[job.owner_key, self.listings_key, + job.last_modified_key, self.trash_key], + args=[raw_who, job.key, + self._dumps(timeutils.utcnow())]) + result = self._loads(raw_result) + status = result['status'] + if status != self.SCRIPT_STATUS_OK: + reason = result.get('reason') + if reason == self.SCRIPT_UNKNOWN_JOB: + raise exc.NotFound("Job %s not found to be" + " trashed" % (job.uuid)) + elif reason == self.SCRIPT_UNKNOWN_OWNER: + raise exc.NotFound("Can not trash job %s" + " which we can not determine" + " the owner of" % (job.uuid)) + elif reason == self.SCRIPT_NOT_EXPECTED_OWNER: + raw_owner = result.get('owner') + if raw_owner: + owner = self._decode_owner(raw_owner) + raise exc.JobFailure("Can not trash job %s" + " which is not owned by %s (it is" + " actively owned by %s)" + % (job.uuid, who, owner)) + else: + raise exc.JobFailure("Can not trash job %s" + " which is not owned by %s" + % (job.uuid, who)) + else: + raise exc.JobFailure("Failure to trash job %s," + " unknown internal error (reason=%s)" + % (job.uuid, reason)) diff --git a/taskflow/jobs/backends/impl_zookeeper.py b/taskflow/jobs/backends/impl_zookeeper.py index d92c2ba5..15b31034 100644 --- a/taskflow/jobs/backends/impl_zookeeper.py +++ b/taskflow/jobs/backends/impl_zookeeper.py @@ -41,37 +41,17 @@ from taskflow.utils import misc LOG = logging.getLogger(__name__) -def check_who(meth): - """Decorator that checks the expected owner type & value restrictions.""" - - @six.wraps(meth) - def wrapper(self, job, who, *args, **kwargs): - if not isinstance(who, six.string_types): - raise TypeError("Job applicant must be a string type") - if len(who) == 0: - raise ValueError("Job applicant must be non-empty") - return meth(self, job, who, *args, **kwargs) - - return wrapper - - class ZookeeperJob(base.Job): """A zookeeper job.""" - def __init__(self, name, board, client, backend, path, + def __init__(self, board, name, client, path, uuid=None, details=None, book=None, book_data=None, - created_on=None): - super(ZookeeperJob, self).__init__(name, uuid=uuid, details=details) - self._board = board - self._book = book - if not book_data: - book_data = {} - self._book_data = book_data + created_on=None, backend=None): + super(ZookeeperJob, self).__init__(board, name, + uuid=uuid, details=details, + backend=backend, + book=book, book_data=book_data) self._client = client - self._backend = backend - if all((self._book, self._book_data)): - raise ValueError("Only one of 'book_data' or 'book'" - " can be provided") self._path = k_paths.normpath(path) self._lock_path = self._path + board.LOCK_POSTFIX self._created_on = created_on @@ -82,10 +62,12 @@ class ZookeeperJob(base.Job): @property def lock_path(self): + """Path the job lock/claim and owner znode is stored.""" return self._lock_path @property def path(self): + """Path the job data znode is stored.""" return self._path @property @@ -154,28 +136,8 @@ class ZookeeperJob(base.Job): self._node_not_found = True return self._created_on - @property - def board(self): - return self._board - - def _load_book(self): - book_uuid = self.book_uuid - if self._backend is not None and book_uuid is not None: - # TODO(harlowja): we are currently limited by assuming that the - # job posted has the same backend as this loader (to start this - # seems to be a ok assumption, and can be adjusted in the future - # if we determine there is a use-case for multi-backend loaders, - # aka a registry of loaders). - with contextlib.closing(self._backend.get_connection()) as conn: - return conn.get_logbook(book_uuid) - # No backend to fetch from or no uuid specified - return None - @property def state(self): - return self._fetch_state() - - def _fetch_state(self): owner = self.board.find_owner(self) job_data = {} try: @@ -217,30 +179,6 @@ class ZookeeperJob(base.Job): def __hash__(self): return hash(self.path) - @property - def book(self): - if self._book is None: - self._book = self._load_book() - return self._book - - @property - def book_uuid(self): - if self._book: - return self._book.uuid - if self._book_data: - return self._book_data.get('uuid') - else: - return None - - @property - def book_name(self): - if self._book: - return self._book.name - if self._book_data: - return self._book_data.get('name') - else: - return None - class ZookeeperJobBoardIterator(six.Iterator): """Iterator over a zookeeper jobboard that iterates over potential jobs. @@ -273,6 +211,7 @@ class ZookeeperJobBoardIterator(six.Iterator): @property def board(self): + """The board this iterator was created from.""" return self._board def __iter__(self): @@ -310,28 +249,32 @@ class ZookeeperJobBoardIterator(six.Iterator): class ZookeeperJobBoard(base.NotifyingJobBoard): - """A jobboard backend by zookeeper. + """A jobboard backed by `zookeeper`_. Powered by the `kazoo `_ library. This jobboard creates *sequenced* persistent znodes in a directory in - zookeeper (that directory defaults to ``/taskflow/jobs``) and uses - zookeeper watches to notify other jobboards that the job which was posted - using the :meth:`.post` method (this creates a znode with contents/details - in json). The users of those jobboard(s) (potentially on disjoint sets of - machines) can then iterate over the available jobs and decide if they want + zookeeper and uses zookeeper watches to notify other jobboards of + jobs which were posted using the :meth:`.post` method (this creates a + znode with job contents/details encoded in `json`_). The users of these + jobboard(s) (potentially on disjoint sets of machines) can then iterate + over the available jobs and decide if they want to attempt to claim one of the jobs they have iterated over. If so they will then attempt to contact zookeeper and they will attempt to create a ephemeral znode using the name of the persistent znode + ".lock" as a postfix. If the entity trying to use the jobboard to :meth:`.claim` the job is able to create a ephemeral znode with that name then it will be allowed (and expected) to perform whatever *work* the contents of that - job described. Once finished the ephemeral znode and persistent znode may - be deleted (if successfully completed) in a single transaction or if not - successful (or the entity that claimed the znode dies) the ephemeral - znode will be released (either manually by using :meth:`.abandon` or - automatically by zookeeper when the ephemeral node and associated session - is deemed to have been lost). + job described. Once the claiming entity is finished the ephemeral znode + and persistent znode will be deleted (if successfully completed) in a + single transaction. If the claiming entity is not successful (or the + entity that claimed the znode dies) the ephemeral znode will be + released (either manually by using :meth:`.abandon` or automatically by + zookeeper when the ephemeral node and associated session is deemed to + have been lost). + + .. _zookeeper: http://zookeeper.apache.org/ + .. _json: http://json.org/ """ #: Transaction support was added in 3.4.0 so we need at least that version. @@ -366,11 +309,9 @@ class ZookeeperJobBoard(base.NotifyingJobBoard): self._path = path self._trash_path = self._path.replace(k_paths.basename(self._path), self.TRASH_FOLDER) - # The backend to load the full logbooks from, since whats sent over - # the zookeeper data connection is only the logbook uuid and name, and - # not currently the full logbook (later when a zookeeper backend - # appears we can likely optimize for that backend usage by directly - # reading from the path where the data is stored, if we want). + # The backend to load the full logbooks from, since what is sent over + # the data connection is only the logbook uuid and name, and not the + # full logbook. self._persistence = persistence # Misc. internal details self._known_jobs = {} @@ -401,10 +342,12 @@ class ZookeeperJobBoard(base.NotifyingJobBoard): @property def path(self): + """Path where all job znodes will be stored.""" return self._path @property def trash_path(self): + """Path where all trashed job znodes will be stored.""" return self._trash_path @property @@ -476,8 +419,9 @@ class ZookeeperJobBoard(base.NotifyingJobBoard): # jobs information into the known job set (if it's already # existing then just leave it alone). if path not in self._known_jobs: - job = ZookeeperJob(job_data['name'], self, - self._client, self._persistence, path, + job = ZookeeperJob(self, job_data['name'], + self._client, path, + backend=self._persistence, uuid=job_data['uuid'], book_data=job_data.get("book"), details=job_data.get("details", {}), @@ -536,46 +480,31 @@ class ZookeeperJobBoard(base.NotifyingJobBoard): self._process_child(path, request) def post(self, name, book=None, details=None): - - def format_posting(job_uuid): - posting = { - 'uuid': job_uuid, - 'name': name, - } - if details: - posting['details'] = details - else: - posting['details'] = {} - if book is not None: - posting['book'] = { - 'name': book.name, - 'uuid': book.uuid, - } - return posting - # NOTE(harlowja): Jobs are not ephemeral, they will persist until they # are consumed (this may change later, but seems safer to do this until # further notice). job_uuid = uuidutils.generate_uuid() + job_posting = base.format_posting(job_uuid, name, + book=book, details=details) + raw_job_posting = misc.binary_encode(jsonutils.dumps(job_posting)) with self._wrap(job_uuid, None, - "Posting failure: %s", ensure_known=False): - job_posting = format_posting(job_uuid) - job_posting = misc.binary_encode(jsonutils.dumps(job_posting)) + fail_msg_tpl="Posting failure: %s", + ensure_known=False): job_path = self._client.create(self._job_base, - value=job_posting, + value=raw_job_posting, sequence=True, ephemeral=False) - job = ZookeeperJob(name, self, self._client, - self._persistence, job_path, - book=book, details=details, - uuid=job_uuid) + job = ZookeeperJob(self, name, self._client, job_path, + backend=self._persistence, + book=book, details=details, uuid=job_uuid, + book_data=job_posting.get('book')) with self._job_cond: self._known_jobs[job_path] = job self._job_cond.notify_all() self._emit(base.POSTED, details={'job': job}) return job - @check_who + @base.check_who def claim(self, job, who): def _unclaimable_try_find_owner(cause): try: @@ -589,7 +518,8 @@ class ZookeeperJobBoard(base.NotifyingJobBoard): excp.raise_with_cause(excp.UnclaimableJob, message, cause=cause) - with self._wrap(job.uuid, job.path, "Claiming failure: %s"): + with self._wrap(job.uuid, job.path, + fail_msg_tpl="Claiming failure: %s"): # NOTE(harlowja): post as json which will allow for future changes # more easily than a raw string/text. value = jsonutils.dumps({ @@ -653,7 +583,9 @@ class ZookeeperJobBoard(base.NotifyingJobBoard): excp.raise_with_cause(excp.JobFailure, fail_msg_tpl % (job_uuid)) def find_owner(self, job): - with self._wrap(job.uuid, job.path, "Owner query failure: %s"): + with self._wrap(job.uuid, job.path, + fail_msg_tpl="Owner query failure: %s", + ensure_known=False): try: self._client.sync(job.lock_path) raw_data, _lock_stat = self._client.get(job.lock_path) @@ -669,14 +601,15 @@ class ZookeeperJobBoard(base.NotifyingJobBoard): return (misc.decode_json(lock_data), lock_stat, misc.decode_json(job_data), job_stat) - @check_who + @base.check_who def consume(self, job, who): - with self._wrap(job.uuid, job.path, "Consumption failure: %s"): + with self._wrap(job.uuid, job.path, + fail_msg_tpl="Consumption failure: %s"): try: owner_data = self._get_owner_and_data(job) lock_data, lock_stat, data, data_stat = owner_data except k_exceptions.NoNodeError: - excp.raise_with_cause(excp.JobFailure, + excp.raise_with_cause(excp.NotFound, "Can not consume a job %s" " which we can not determine" " the owner of" % (job.uuid)) @@ -690,14 +623,15 @@ class ZookeeperJobBoard(base.NotifyingJobBoard): kazoo_utils.checked_commit(txn) self._remove_job(job.path) - @check_who + @base.check_who def abandon(self, job, who): - with self._wrap(job.uuid, job.path, "Abandonment failure: %s"): + with self._wrap(job.uuid, job.path, + fail_msg_tpl="Abandonment failure: %s"): try: owner_data = self._get_owner_and_data(job) lock_data, lock_stat, data, data_stat = owner_data except k_exceptions.NoNodeError: - excp.raise_with_cause(excp.JobFailure, + excp.raise_with_cause(excp.NotFound, "Can not abandon a job %s" " which we can not determine" " the owner of" % (job.uuid)) @@ -709,14 +643,15 @@ class ZookeeperJobBoard(base.NotifyingJobBoard): txn.delete(job.lock_path, version=lock_stat.version) kazoo_utils.checked_commit(txn) - @check_who + @base.check_who def trash(self, job, who): - with self._wrap(job.uuid, job.path, "Trash failure: %s"): + with self._wrap(job.uuid, job.path, + fail_msg_tpl="Trash failure: %s"): try: owner_data = self._get_owner_and_data(job) lock_data, lock_stat, data, data_stat = owner_data except k_exceptions.NoNodeError: - excp.raise_with_cause(excp.JobFailure, + excp.raise_with_cause(excp.NotFound, "Can not trash a job %s" " which we can not determine" " the owner of" % (job.uuid)) diff --git a/taskflow/jobs/base.py b/taskflow/jobs/base.py index 0f54b8f8..a9ff0274 100644 --- a/taskflow/jobs/base.py +++ b/taskflow/jobs/base.py @@ -16,6 +16,7 @@ # under the License. import abc +import contextlib from oslo_utils import uuidutils import six @@ -43,7 +44,9 @@ class Job(object): reverting... """ - def __init__(self, name, uuid=None, details=None): + def __init__(self, board, name, + uuid=None, details=None, backend=None, + book=None, book_data=None): if uuid: self._uuid = uuid else: @@ -52,6 +55,12 @@ class Job(object): if not details: details = {} self._details = details + self._backend = backend + self._board = board + self._book = book + if not book_data: + book_data = {} + self._book_data = book_data @abc.abstractproperty def last_modified(self): @@ -61,34 +70,47 @@ class Job(object): def created_on(self): """The datetime the job was created on.""" - @abc.abstractproperty + @property def board(self): """The board this job was posted on or was created from.""" + return self._board @abc.abstractproperty def state(self): - """The current state of this job.""" + """Access the current state of this job.""" + pass - @abc.abstractproperty + @property def book(self): """Logbook associated with this job. If no logbook is associated with this job, this property is None. """ + if self._book is None: + self._book = self._load_book() + return self._book - @abc.abstractproperty + @property def book_uuid(self): """UUID of logbook associated with this job. If no logbook is associated with this job, this property is None. """ + if self._book is not None: + return self._book.uuid + else: + return self._book_data.get('uuid') - @abc.abstractproperty + @property def book_name(self): """Name of logbook associated with this job. If no logbook is associated with this job, this property is None. """ + if self._book is not None: + return self._book.name + else: + return self._book_data.get('name') @property def uuid(self): @@ -105,10 +127,24 @@ class Job(object): """The non-uniquely identifying name of this job.""" return self._name + def _load_book(self): + book_uuid = self.book_uuid + if self._backend is not None and book_uuid is not None: + # TODO(harlowja): we are currently limited by assuming that the + # job posted has the same backend as this loader (to start this + # seems to be a ok assumption, and can be adjusted in the future + # if we determine there is a use-case for multi-backend loaders, + # aka a registry of loaders). + with contextlib.closing(self._backend.get_connection()) as conn: + return conn.get_logbook(book_uuid) + # No backend to fetch from or no uuid specified + return None + def __str__(self): """Pretty formats the job into something *more* meaningful.""" - return "%s %s (%s): %s" % (type(self).__name__, - self.name, self.uuid, self.details) + return "%s: %s (uuid=%s, details=%s)" % (type(self).__name__, + self.name, self.uuid, + self.details) @six.add_metaclass(abc.ABCMeta) @@ -312,3 +348,40 @@ class NotifyingJobBoard(JobBoard): def __init__(self, name, conf): super(NotifyingJobBoard, self).__init__(name, conf) self.notifier = notifier.Notifier() + + +# Internal helpers for usage by board implementations... + +def check_who(meth): + + @six.wraps(meth) + def wrapper(self, job, who, *args, **kwargs): + if not isinstance(who, six.string_types): + raise TypeError("Job applicant must be a string type") + if len(who) == 0: + raise ValueError("Job applicant must be non-empty") + return meth(self, job, who, *args, **kwargs) + + return wrapper + + +def format_posting(uuid, name, created_on=None, last_modified=None, + details=None, book=None): + posting = { + 'uuid': uuid, + 'name': name, + } + if created_on is not None: + posting['created_on'] = created_on + if last_modified is not None: + posting['last_modified'] = last_modified + if details: + posting['details'] = details + else: + posting['details'] = {} + if book is not None: + posting['book'] = { + 'name': book.name, + 'uuid': book.uuid, + } + return posting diff --git a/taskflow/tests/unit/jobs/base.py b/taskflow/tests/unit/jobs/base.py index 654702ae..46c78dfe 100644 --- a/taskflow/tests/unit/jobs/base.py +++ b/taskflow/tests/unit/jobs/base.py @@ -18,20 +18,13 @@ import contextlib import threading import time -from kazoo.recipe import watchers -from oslo_utils import uuidutils - from taskflow import exceptions as excp from taskflow.persistence.backends import impl_dir from taskflow import states -from taskflow.test import mock from taskflow.tests import utils as test_utils -from taskflow.utils import misc from taskflow.utils import persistence_utils as p_utils from taskflow.utils import threading_utils -FLUSH_PATH_TPL = '/taskflow/flush-test/%s' - @contextlib.contextmanager def connect_close(*args): @@ -44,72 +37,20 @@ def connect_close(*args): a.close() -@contextlib.contextmanager -def flush(client, path=None): - # This uses the linearity guarantee of zookeeper (and associated libraries) - # to create a temporary node, wait until a watcher notifies it's created, - # then yield back for more work, and then at the end of that work delete - # the created node. This ensures that the operations done in the yield - # of this context manager will be applied and all watchers will have fired - # before this context manager exits. - if not path: - path = FLUSH_PATH_TPL % uuidutils.generate_uuid() - created = threading.Event() - deleted = threading.Event() - - def on_created(data, stat): - if stat is not None: - created.set() - return False # cause this watcher to cease to exist - - def on_deleted(data, stat): - if stat is None: - deleted.set() - return False # cause this watcher to cease to exist - - watchers.DataWatch(client, path, func=on_created) - client.create(path, makepath=True) - if not created.wait(test_utils.WAIT_TIMEOUT): - raise RuntimeError("Could not receive creation of %s in" - " the alloted timeout of %s seconds" - % (path, test_utils.WAIT_TIMEOUT)) - try: - yield - finally: - watchers.DataWatch(client, path, func=on_deleted) - client.delete(path, recursive=True) - if not deleted.wait(test_utils.WAIT_TIMEOUT): - raise RuntimeError("Could not receive deletion of %s in" - " the alloted timeout of %s seconds" - % (path, test_utils.WAIT_TIMEOUT)) - - class BoardTestMixin(object): + + @contextlib.contextmanager + def flush(self, client): + yield + + def close_client(self, client): + pass + def test_connect(self): self.assertFalse(self.board.connected) with connect_close(self.board): self.assertTrue(self.board.connected) - @mock.patch("taskflow.jobs.backends.impl_zookeeper.misc." - "millis_to_datetime") - def test_posting_dates(self, mock_dt): - epoch = misc.millis_to_datetime(0) - mock_dt.return_value = epoch - - with connect_close(self.board): - j = self.board.post('test', p_utils.temporary_log_book()) - self.assertEqual(epoch, j.created_on) - self.assertEqual(epoch, j.last_modified) - - self.assertTrue(mock_dt.called) - - def test_board_iter(self): - with connect_close(self.board): - it = self.board.iterjobs() - self.assertEqual(it.board, self.board) - self.assertFalse(it.only_unclaimed) - self.assertFalse(it.ensure_fresh) - def test_board_iter_empty(self): with connect_close(self.board): jobs_found = list(self.board.iterjobs()) @@ -155,7 +96,7 @@ class BoardTestMixin(object): def test_posting_claim(self): with connect_close(self.board): - with flush(self.client): + with self.flush(self.client): self.board.post('test', p_utils.temporary_log_book()) self.assertEqual(1, self.board.job_count) @@ -164,7 +105,7 @@ class BoardTestMixin(object): j = possible_jobs[0] self.assertEqual(states.UNCLAIMED, j.state) - with flush(self.client): + with self.flush(self.client): self.board.claim(j, self.board.name) self.assertEqual(self.board.name, self.board.find_owner(j)) @@ -173,25 +114,24 @@ class BoardTestMixin(object): possible_jobs = list(self.board.iterjobs(only_unclaimed=True)) self.assertEqual(0, len(possible_jobs)) - self.assertRaisesAttrAccess(excp.NotFound, j, 'state') - self.assertRaises(excp.NotFound, - self.board.consume, j, self.board.name) + self.close_client(self.client) + self.assertRaisesAttrAccess(excp.JobFailure, j, 'state') def test_posting_claim_consume(self): with connect_close(self.board): - with flush(self.client): + with self.flush(self.client): self.board.post('test', p_utils.temporary_log_book()) possible_jobs = list(self.board.iterjobs(only_unclaimed=True)) self.assertEqual(1, len(possible_jobs)) j = possible_jobs[0] - with flush(self.client): + with self.flush(self.client): self.board.claim(j, self.board.name) possible_jobs = list(self.board.iterjobs(only_unclaimed=True)) self.assertEqual(0, len(possible_jobs)) - with flush(self.client): + with self.flush(self.client): self.board.consume(j, self.board.name) self.assertEqual(0, len(list(self.board.iterjobs()))) @@ -201,18 +141,18 @@ class BoardTestMixin(object): def test_posting_claim_abandon(self): with connect_close(self.board): - with flush(self.client): + with self.flush(self.client): self.board.post('test', p_utils.temporary_log_book()) possible_jobs = list(self.board.iterjobs(only_unclaimed=True)) self.assertEqual(1, len(possible_jobs)) j = possible_jobs[0] - with flush(self.client): + with self.flush(self.client): self.board.claim(j, self.board.name) possible_jobs = list(self.board.iterjobs(only_unclaimed=True)) self.assertEqual(0, len(possible_jobs)) - with flush(self.client): + with self.flush(self.client): self.board.abandon(j, self.board.name) possible_jobs = list(self.board.iterjobs(only_unclaimed=True)) @@ -221,12 +161,12 @@ class BoardTestMixin(object): def test_posting_claim_diff_owner(self): with connect_close(self.board): - with flush(self.client): + with self.flush(self.client): self.board.post('test', p_utils.temporary_log_book()) possible_jobs = list(self.board.iterjobs(only_unclaimed=True)) self.assertEqual(1, len(possible_jobs)) - with flush(self.client): + with self.flush(self.client): self.board.claim(possible_jobs[0], self.board.name) possible_jobs = list(self.board.iterjobs()) @@ -236,14 +176,6 @@ class BoardTestMixin(object): possible_jobs = list(self.board.iterjobs(only_unclaimed=True)) self.assertEqual(0, len(possible_jobs)) - def test_posting_no_post(self): - with connect_close(self.board): - with mock.patch.object(self.client, 'create') as create_func: - create_func.side_effect = IOError("Unable to post") - self.assertRaises(IOError, self.board.post, - 'test', p_utils.temporary_log_book()) - self.assertEqual(0, self.board.job_count) - def test_posting_with_book(self): backend = impl_dir.DirBackend(conf={ 'path': self.makeTmpDir(), @@ -252,9 +184,9 @@ class BoardTestMixin(object): book, flow_detail = p_utils.temporary_flow_detail(backend) self.assertEqual(1, len(book)) - client, board = self._create_board(persistence=backend) + client, board = self.create_board(persistence=backend) with connect_close(board): - with flush(client): + with self.flush(client): board.post('test', book) possible_jobs = list(board.iterjobs(only_unclaimed=True)) @@ -273,11 +205,12 @@ class BoardTestMixin(object): def test_posting_abandon_no_owner(self): with connect_close(self.board): - with flush(self.client): + with self.flush(self.client): self.board.post('test', p_utils.temporary_log_book()) self.assertEqual(1, self.board.job_count) possible_jobs = list(self.board.iterjobs(only_unclaimed=True)) self.assertEqual(1, len(possible_jobs)) j = possible_jobs[0] - self.assertRaises(excp.JobFailure, self.board.abandon, j, j.name) + self.assertRaises(excp.NotFound, self.board.abandon, + j, j.name) diff --git a/taskflow/tests/unit/jobs/test_entrypoint.py b/taskflow/tests/unit/jobs/test_entrypoint.py index 17dfa02e..ae6789b9 100644 --- a/taskflow/tests/unit/jobs/test_entrypoint.py +++ b/taskflow/tests/unit/jobs/test_entrypoint.py @@ -19,6 +19,7 @@ import contextlib from zake import fake_client from taskflow.jobs import backends +from taskflow.jobs.backends import impl_redis from taskflow.jobs.backends import impl_zookeeper from taskflow import test @@ -47,3 +48,15 @@ class BackendFetchingTest(test.TestCase): with contextlib.closing(backends.fetch('test', conf, **kwargs)) as be: self.assertIsInstance(be, impl_zookeeper.ZookeeperJobBoard) self.assertIs(existing_client, be._client) + + def test_redis_entry_point_text(self): + conf = 'redis' + with contextlib.closing(backends.fetch('test', conf)) as be: + self.assertIsInstance(be, impl_redis.RedisJobBoard) + + def test_redis_entry_point(self): + conf = { + 'board': 'redis', + } + with contextlib.closing(backends.fetch('test', conf)) as be: + self.assertIsInstance(be, impl_redis.RedisJobBoard) diff --git a/taskflow/tests/unit/jobs/test_redis_job.py b/taskflow/tests/unit/jobs/test_redis_job.py new file mode 100644 index 00000000..f988788d --- /dev/null +++ b/taskflow/tests/unit/jobs/test_redis_job.py @@ -0,0 +1,81 @@ +# -*- coding: utf-8 -*- + +# Copyright (C) 2013 Yahoo! Inc. 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. + +import time + +from oslo_utils import uuidutils +import six +import testtools + +from taskflow.jobs.backends import impl_redis +from taskflow import states +from taskflow import test +from taskflow.tests.unit.jobs import base +from taskflow.tests import utils as test_utils +from taskflow.utils import persistence_utils as p_utils +from taskflow.utils import redis_utils as ru + + +REDIS_AVAILABLE = test_utils.redis_available( + impl_redis.RedisJobBoard.MIN_REDIS_VERSION) + + +@testtools.skipIf(not REDIS_AVAILABLE, 'redis is not available') +class RedisJobboardTest(test.TestCase, base.BoardTestMixin): + def close_client(self, client): + client.close() + + def create_board(self, persistence=None): + namespace = uuidutils.generate_uuid() + client = ru.RedisClient() + config = { + 'namespace': six.b("taskflow-%s" % namespace), + } + kwargs = { + 'client': client, + 'persistence': persistence, + } + board = impl_redis.RedisJobBoard('test-board', config, **kwargs) + self.addCleanup(board.close) + self.addCleanup(self.close_client, client) + return (client, board) + + def test_posting_claim_expiry(self): + + with base.connect_close(self.board): + with self.flush(self.client): + self.board.post('test', p_utils.temporary_log_book()) + + self.assertEqual(1, self.board.job_count) + possible_jobs = list(self.board.iterjobs(only_unclaimed=True)) + self.assertEqual(1, len(possible_jobs)) + j = possible_jobs[0] + self.assertEqual(states.UNCLAIMED, j.state) + + with self.flush(self.client): + self.board.claim(j, self.board.name, expiry=0.5) + + self.assertEqual(self.board.name, self.board.find_owner(j)) + self.assertEqual(states.CLAIMED, j.state) + + time.sleep(0.6) + self.assertEqual(states.UNCLAIMED, j.state) + possible_jobs = list(self.board.iterjobs(only_unclaimed=True)) + self.assertEqual(1, len(possible_jobs)) + + def setUp(self): + super(RedisJobboardTest, self).setUp() + self.client, self.board = self.create_board() diff --git a/taskflow/tests/unit/jobs/test_zk_job.py b/taskflow/tests/unit/jobs/test_zk_job.py index a22a41b9..e42bed66 100644 --- a/taskflow/tests/unit/jobs/test_zk_job.py +++ b/taskflow/tests/unit/jobs/test_zk_job.py @@ -14,6 +14,10 @@ # License for the specific language governing permissions and limitations # under the License. +import contextlib +import threading + +from kazoo.recipe import watchers from oslo_serialization import jsonutils from oslo_utils import uuidutils import six @@ -24,13 +28,14 @@ from zake import utils as zake_utils from taskflow.jobs.backends import impl_zookeeper from taskflow import states from taskflow import test +from taskflow.test import mock from taskflow.tests.unit.jobs import base from taskflow.tests import utils as test_utils from taskflow.utils import kazoo_utils from taskflow.utils import misc from taskflow.utils import persistence_utils as p_utils - +FLUSH_PATH_TPL = '/taskflow/flush-test/%s' TEST_PATH_TPL = '/taskflow/board-test/%s' ZOOKEEPER_AVAILABLE = test_utils.zookeeper_available( impl_zookeeper.ZookeeperJobBoard.MIN_ZK_VERSION) @@ -38,9 +43,81 @@ TRASH_FOLDER = impl_zookeeper.ZookeeperJobBoard.TRASH_FOLDER LOCK_POSTFIX = impl_zookeeper.ZookeeperJobBoard.LOCK_POSTFIX +class ZookeeperBoardTestMixin(base.BoardTestMixin): + def close_client(self, client): + kazoo_utils.finalize_client(client) + + @contextlib.contextmanager + def flush(self, client, path=None): + # This uses the linearity guarantee of zookeeper (and associated + # libraries) to create a temporary node, wait until a watcher notifies + # it's created, then yield back for more work, and then at the end of + # that work delete the created node. This ensures that the operations + # done in the yield of this context manager will be applied and all + # watchers will have fired before this context manager exits. + if not path: + path = FLUSH_PATH_TPL % uuidutils.generate_uuid() + created = threading.Event() + deleted = threading.Event() + + def on_created(data, stat): + if stat is not None: + created.set() + return False # cause this watcher to cease to exist + + def on_deleted(data, stat): + if stat is None: + deleted.set() + return False # cause this watcher to cease to exist + + watchers.DataWatch(client, path, func=on_created) + client.create(path, makepath=True) + if not created.wait(test_utils.WAIT_TIMEOUT): + raise RuntimeError("Could not receive creation of %s in" + " the alloted timeout of %s seconds" + % (path, test_utils.WAIT_TIMEOUT)) + try: + yield + finally: + watchers.DataWatch(client, path, func=on_deleted) + client.delete(path, recursive=True) + if not deleted.wait(test_utils.WAIT_TIMEOUT): + raise RuntimeError("Could not receive deletion of %s in" + " the alloted timeout of %s seconds" + % (path, test_utils.WAIT_TIMEOUT)) + + def test_posting_no_post(self): + with base.connect_close(self.board): + with mock.patch.object(self.client, 'create') as create_func: + create_func.side_effect = IOError("Unable to post") + self.assertRaises(IOError, self.board.post, + 'test', p_utils.temporary_log_book()) + self.assertEqual(0, self.board.job_count) + + def test_board_iter(self): + with base.connect_close(self.board): + it = self.board.iterjobs() + self.assertEqual(it.board, self.board) + self.assertFalse(it.only_unclaimed) + self.assertFalse(it.ensure_fresh) + + @mock.patch("taskflow.jobs.backends.impl_zookeeper.misc." + "millis_to_datetime") + def test_posting_dates(self, mock_dt): + epoch = misc.millis_to_datetime(0) + mock_dt.return_value = epoch + + with base.connect_close(self.board): + j = self.board.post('test', p_utils.temporary_log_book()) + self.assertEqual(epoch, j.created_on) + self.assertEqual(epoch, j.last_modified) + + self.assertTrue(mock_dt.called) + + @testtools.skipIf(not ZOOKEEPER_AVAILABLE, 'zookeeper is not available') -class ZookeeperJobboardTest(test.TestCase, base.BoardTestMixin): - def _create_board(self, persistence=None): +class ZookeeperJobboardTest(test.TestCase, ZookeeperBoardTestMixin): + def create_board(self, persistence=None): def cleanup_path(client, path): if not client.connected: @@ -52,39 +129,39 @@ class ZookeeperJobboardTest(test.TestCase, base.BoardTestMixin): board = impl_zookeeper.ZookeeperJobBoard('test-board', {'path': path}, client=client, persistence=persistence) - self.addCleanup(kazoo_utils.finalize_client, client) + self.addCleanup(self.close_client, client) self.addCleanup(cleanup_path, client, path) self.addCleanup(board.close) return (client, board) def setUp(self): super(ZookeeperJobboardTest, self).setUp() - self.client, self.board = self._create_board() + self.client, self.board = self.create_board() -class ZakeJobboardTest(test.TestCase, base.BoardTestMixin): - def _create_board(self, persistence=None): +class ZakeJobboardTest(test.TestCase, ZookeeperBoardTestMixin): + def create_board(self, persistence=None): client = fake_client.FakeClient() board = impl_zookeeper.ZookeeperJobBoard('test-board', {}, client=client, persistence=persistence) self.addCleanup(board.close) - self.addCleanup(kazoo_utils.finalize_client, client) + self.addCleanup(self.close_client, client) return (client, board) def setUp(self): super(ZakeJobboardTest, self).setUp() - self.client, self.board = self._create_board() + self.client, self.board = self.create_board() self.bad_paths = [self.board.path, self.board.trash_path] self.bad_paths.extend(zake_utils.partition_path(self.board.path)) def test_posting_owner_lost(self): with base.connect_close(self.board): - with base.flush(self.client): + with self.flush(self.client): j = self.board.post('test', p_utils.temporary_log_book()) self.assertEqual(states.UNCLAIMED, j.state) - with base.flush(self.client): + with self.flush(self.client): self.board.claim(j, self.board.name) self.assertEqual(states.CLAIMED, j.state) @@ -102,10 +179,10 @@ class ZakeJobboardTest(test.TestCase, base.BoardTestMixin): def test_posting_state_lock_lost(self): with base.connect_close(self.board): - with base.flush(self.client): + with self.flush(self.client): j = self.board.post('test', p_utils.temporary_log_book()) self.assertEqual(states.UNCLAIMED, j.state) - with base.flush(self.client): + with self.flush(self.client): self.board.claim(j, self.board.name) self.assertEqual(states.CLAIMED, j.state) @@ -123,14 +200,14 @@ class ZakeJobboardTest(test.TestCase, base.BoardTestMixin): def test_trashing_claimed_job(self): with base.connect_close(self.board): - with base.flush(self.client): + with self.flush(self.client): j = self.board.post('test', p_utils.temporary_log_book()) self.assertEqual(states.UNCLAIMED, j.state) - with base.flush(self.client): + with self.flush(self.client): self.board.claim(j, self.board.name) self.assertEqual(states.CLAIMED, j.state) - with base.flush(self.client): + with self.flush(self.client): self.board.trash(j, self.board.name) trashed = [] diff --git a/taskflow/tests/utils.py b/taskflow/tests/utils.py index 6148e0a6..b295fc2a 100644 --- a/taskflow/tests/utils.py +++ b/taskflow/tests/utils.py @@ -18,6 +18,7 @@ import contextlib import string import threading +import redis import six from taskflow import exceptions @@ -27,6 +28,7 @@ from taskflow import retry from taskflow import task from taskflow.types import failure from taskflow.utils import kazoo_utils +from taskflow.utils import redis_utils ARGS_KEY = '__args__' KWARGS_KEY = '__kwargs__' @@ -73,6 +75,18 @@ def zookeeper_available(min_version, timeout=3): kazoo_utils.finalize_client(client) +def redis_available(min_version): + client = redis.StrictRedis() + try: + client.ping() + except Exception: + return False + else: + ok, redis_version = redis_utils.is_server_new_enough(client, + min_version) + return ok + + class NoopRetry(retry.AlwaysRevert): pass diff --git a/taskflow/utils/misc.py b/taskflow/utils/misc.py index d89feea2..87f68355 100644 --- a/taskflow/utils/misc.py +++ b/taskflow/utils/misc.py @@ -28,6 +28,7 @@ import types import enum from oslo_serialization import jsonutils +from oslo_serialization import msgpackutils from oslo_utils import encodeutils from oslo_utils import importutils from oslo_utils import netutils @@ -285,19 +286,7 @@ def binary_decode(data, encoding='utf-8', errors='strict'): errors=errors) -def decode_json(raw_data, root_types=(dict,)): - """Parse raw data to get JSON object. - - Decodes a JSON from a given raw data binary and checks that the root - type of that decoded object is in the allowed set of types (by - default a JSON object/dict should be the root type). - """ - try: - data = jsonutils.loads(binary_decode(raw_data)) - except UnicodeDecodeError as e: - raise ValueError("Expected UTF-8 decodable data: %s" % e) - except ValueError as e: - raise ValueError("Expected JSON decodable data: %s" % e) +def _check_decoded_type(data, root_types=(dict,)): if root_types: if not isinstance(root_types, tuple): root_types = tuple(root_types) @@ -312,6 +301,40 @@ def decode_json(raw_data, root_types=(dict,)): return data +def decode_msgpack(raw_data, root_types=(dict,)): + """Parse raw data to get decoded object. + + Decodes a msgback encoded 'blob' from a given raw data binary string and + checks that the root type of that decoded object is in the allowed set of + types (by default a dict should be the root type). + """ + try: + data = msgpackutils.loads(raw_data) + except Exception as e: + # TODO(harlowja): fix this when msgpackutils exposes the msgpack + # exceptions so that we can avoid catching just exception... + raise ValueError("Expected msgpack decodable data: %s" % e) + else: + return _check_decoded_type(data, root_types=root_types) + + +def decode_json(raw_data, root_types=(dict,)): + """Parse raw data to get decoded object. + + Decodes a JSON encoded 'blob' from a given raw data binary string and + checks that the root type of that decoded object is in the allowed set of + types (by default a dict should be the root type). + """ + try: + data = jsonutils.loads(binary_decode(raw_data)) + except UnicodeDecodeError as e: + raise ValueError("Expected UTF-8 decodable data: %s" % e) + except ValueError as e: + raise ValueError("Expected JSON decodable data: %s" % e) + else: + return _check_decoded_type(data, root_types=root_types) + + class cachedproperty(object): """A *thread-safe* descriptor property that is only evaluated once. diff --git a/taskflow/utils/redis_utils.py b/taskflow/utils/redis_utils.py new file mode 100644 index 00000000..0d040738 --- /dev/null +++ b/taskflow/utils/redis_utils.py @@ -0,0 +1,133 @@ +# -*- coding: utf-8 -*- + +# Copyright (C) 2015 Yahoo! Inc. 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. + +import enum + +import redis +from redis import exceptions as redis_exceptions +import six + + +def _raise_on_closed(meth): + + @six.wraps(meth) + def wrapper(self, *args, **kwargs): + if self.closed: + raise redis_exceptions.ConnectionError("Connection has been" + " closed") + return meth(self, *args, **kwargs) + + return wrapper + + +class RedisClient(redis.StrictRedis): + """A redis client that can be closed (and raises on-usage after closed). + + TODO(harlowja): if https://github.com/andymccurdy/redis-py/issues/613 ever + gets resolved or merged or other then we can likely remove this. + """ + + def __init__(self, *args, **kwargs): + super(RedisClient, self).__init__(*args, **kwargs) + self.closed = False + + def close(self): + self.closed = True + self.connection_pool.disconnect() + + execute_command = _raise_on_closed(redis.StrictRedis.execute_command) + transaction = _raise_on_closed(redis.StrictRedis.transaction) + pubsub = _raise_on_closed(redis.StrictRedis.pubsub) + + +class UnknownExpire(enum.IntEnum): + """Non-expiry (not ttls) results return from :func:`.get_expiry`. + + See: http://redis.io/commands/ttl or http://redis.io/commands/pttl + """ + + DOES_NOT_EXPIRE = -1 + """ + The command returns ``-1`` if the key exists but has no associated expire. + """ + + #: The command returns ``-2`` if the key does not exist. + KEY_NOT_FOUND = -2 + + +DOES_NOT_EXPIRE = UnknownExpire.DOES_NOT_EXPIRE +KEY_NOT_FOUND = UnknownExpire.KEY_NOT_FOUND + +_UNKNOWN_EXPIRE_MAPPING = dict((e.value, e) for e in list(UnknownExpire)) + + +def get_expiry(client, key, prior_version=None): + """Gets an expiry for a key (using **best** determined ttl method).""" + is_new_enough, _prior_version = is_server_new_enough( + client, (2, 6), prior_version=prior_version) + if is_new_enough: + result = client.pttl(key) + try: + return _UNKNOWN_EXPIRE_MAPPING[result] + except KeyError: + return result / 1000.0 + else: + result = client.ttl(key) + try: + return _UNKNOWN_EXPIRE_MAPPING[result] + except KeyError: + return float(result) + + +def apply_expiry(client, key, expiry, prior_version=None): + """Applies an expiry to a key (using **best** determined expiry method).""" + is_new_enough, _prior_version = is_server_new_enough( + client, (2, 6), prior_version=prior_version) + if is_new_enough: + # Use milliseconds (as that is what pexpire uses/expects...) + ms_expiry = expiry * 1000.0 + ms_expiry = max(0, int(ms_expiry)) + result = client.pexpire(key, ms_expiry) + else: + # Only supports seconds (not subseconds...) + sec_expiry = int(expiry) + sec_expiry = max(0, sec_expiry) + result = client.expire(key, sec_expiry) + return bool(result) + + +def is_server_new_enough(client, min_version, + default=False, prior_version=None): + """Checks if a client is attached to a new enough redis server.""" + if not prior_version: + try: + server_info = client.info() + except redis_exceptions.ResponseError: + server_info = {} + version_text = server_info.get('redis_version', '') + else: + version_text = prior_version + version_pieces = [] + for p in version_text.split("."): + try: + version_pieces.append(int(p)) + except ValueError: + break + if not version_pieces: + return (default, version_text) + else: + version_pieces = tuple(version_pieces) + return (version_pieces >= min_version, version_text) diff --git a/test-requirements.txt b/test-requirements.txt index 275836d9..805eeb1f 100644 --- a/test-requirements.txt +++ b/test-requirements.txt @@ -15,6 +15,9 @@ kombu>=3.0.7 zake>=0.1.6 # Apache-2.0 kazoo>=2.2 +# Used for testing redis jobboards +redis>=2.10.0 + # Used for testing database persistence backends. SQLAlchemy<1.1.0,>=0.9.7 alembic>=0.7.2 From e012d8de0c828c9f9037752dd0377012a638bca3 Mon Sep 17 00:00:00 2001 From: OpenStack Proposal Bot Date: Mon, 29 Jun 2015 20:34:44 +0000 Subject: [PATCH 231/246] Updated from global requirements Change-Id: I4473230267674ad1f85307b31b74380ada24454f --- requirements.txt | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/requirements.txt b/requirements.txt index bb4903b4..d4abcdd0 100644 --- a/requirements.txt +++ b/requirements.txt @@ -11,7 +11,7 @@ pbr<2.0,>=0.11 six>=1.9.0 # Enum library made for <= python 3.3 -enum34 +enum34;python_version=='2.7' or python_version=='2.6' # For reader/writer + interprocess locks. fasteners>=0.7 # Apache-2.0 @@ -26,10 +26,10 @@ contextlib2>=0.4.0 # PSF License stevedore>=1.5.0 # Apache-2.0 # Backport for concurrent.futures which exists in 3.2+ -futures>=3.0 +futures>=3.0;python_version=='2.7' or python_version=='2.6' # Backport for time.monotonic which is in 3.3+ -monotonic>=0.1 +monotonic>=0.1 # Apache-2.0 # Used for structured input validation jsonschema!=2.5.0,<3.0.0,>=2.0.0 From 2b827e1e363a90e4037172ccb0d57ac0873497fb Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Mon, 16 Mar 2015 18:57:23 -0700 Subject: [PATCH 232/246] Add support for conditional execution To make it possible to alter the runtime flow via a simple conditional like structure make it possible to have the graph flow link function take a decider that is expected to be some callable that will decide (via a boolean return) whether the edge should actually be traversed when running. When a decider returns false; the affected + successors will be set into the IGNORE state and they will be exempt from future runtime and scheduling decisions. Part of blueprint taskflow-conditional-execution Change-Id: Iab0ee46f86d6b8e747911174d54a7295b3fa404d --- doc/source/img/retry_states.svg | 4 +- doc/source/img/task_states.svg | 4 +- doc/source/states.rst | 9 ++ taskflow/engines/action_engine/analyzer.py | 160 ++++++++++++++++----- taskflow/engines/action_engine/runner.py | 28 ++-- taskflow/engines/action_engine/runtime.py | 20 +++ taskflow/examples/switch_graph_flow.py | 75 ++++++++++ taskflow/flow.py | 3 + taskflow/patterns/graph_flow.py | 18 ++- taskflow/states.py | 5 +- taskflow/tests/unit/test_engines.py | 127 ++++++++++++++++ tools/state_graph.py | 12 +- 12 files changed, 403 insertions(+), 62 deletions(-) create mode 100644 taskflow/examples/switch_graph_flow.py diff --git a/doc/source/img/retry_states.svg b/doc/source/img/retry_states.svg index 1a25bda3..d6801b19 100644 --- a/doc/source/img/retry_states.svg +++ b/doc/source/img/retry_states.svg @@ -3,6 +3,6 @@ - -Retries statesPENDINGRUNNINGSUCCESSFAILURERETRYINGREVERTINGREVERTEDstart + +Retries statesPENDINGIGNORERUNNINGSUCCESSFAILURERETRYINGREVERTINGREVERTEDstart diff --git a/doc/source/img/task_states.svg b/doc/source/img/task_states.svg index dbb48c60..9c27c843 100644 --- a/doc/source/img/task_states.svg +++ b/doc/source/img/task_states.svg @@ -3,6 +3,6 @@ - -Tasks statesPENDINGRUNNINGSUCCESSFAILUREREVERTINGREVERTEDstart + +Tasks statesPENDINGIGNORERUNNINGFAILURESUCCESSREVERTINGREVERTEDstart diff --git a/doc/source/states.rst b/doc/source/states.rst index 36035480..01e9da59 100644 --- a/doc/source/states.rst +++ b/doc/source/states.rst @@ -124,6 +124,11 @@ or if needed will wait for all of the atoms it depends on to complete. An engine running a task also transitions the task to the ``PENDING`` state after it was reverted and its containing flow was restarted or retried. + +**IGNORE** - When a conditional decision has been made to skip (not +execute) the task the engine will transition the task to +the ``IGNORE`` state. + **RUNNING** - When an engine running the task starts to execute the task, the engine will transition the task to the ``RUNNING`` state, and the task will stay in this state until the tasks :py:meth:`~taskflow.task.BaseTask.execute` @@ -171,6 +176,10 @@ flow that the retry is associated with by consulting its An engine running a retry also transitions the retry to the ``PENDING`` state after it was reverted and its associated flow was restarted or retried. +**IGNORE** - When a conditional decision has been made to skip (not +execute) the retry the engine will transition the retry to +the ``IGNORE`` state. + **RUNNING** - When an engine starts to execute the retry, the engine transitions the retry to the ``RUNNING`` state, and the retry stays in this state until its :py:meth:`~taskflow.retry.Retry.execute` method returns. diff --git a/taskflow/engines/action_engine/analyzer.py b/taskflow/engines/action_engine/analyzer.py index bef7b8b5..909d6751 100644 --- a/taskflow/engines/action_engine/analyzer.py +++ b/taskflow/engines/action_engine/analyzer.py @@ -14,6 +14,8 @@ # License for the specific language governing permissions and limitations # under the License. +import itertools + from networkx.algorithms import traversal import six @@ -21,6 +23,60 @@ from taskflow import retry as retry_atom from taskflow import states as st +class IgnoreDecider(object): + """Checks any provided edge-deciders and determines if ok to run.""" + + def __init__(self, atom, edge_deciders): + self._atom = atom + self._edge_deciders = edge_deciders + + def check(self, runtime): + """Returns bool of whether this decider should allow running.""" + results = {} + for name in six.iterkeys(self._edge_deciders): + results[name] = runtime.storage.get(name) + for local_decider in six.itervalues(self._edge_deciders): + if not local_decider(history=results): + return False + return True + + def affect(self, runtime): + """If the :py:func:`~.check` returns false, affects associated atoms. + + This will alter the associated atom + successor atoms by setting there + state to ``IGNORE`` so that they are ignored in future runtime + activities. + """ + successors_iter = runtime.analyzer.iterate_subgraph(self._atom) + runtime.reset_nodes(itertools.chain([self._atom], successors_iter), + state=st.IGNORE, intention=st.IGNORE) + + def check_and_affect(self, runtime): + """Handles :py:func:`~.check` + :py:func:`~.affect` in right order.""" + proceed = self.check(runtime) + if not proceed: + self.affect(runtime) + return proceed + + +class NoOpDecider(object): + """No-op decider that says it is always ok to run & has no effect(s).""" + + def check(self, runtime): + """Always good to go.""" + return True + + def affect(self, runtime): + """Does nothing.""" + + def check_and_affect(self, runtime): + """Handles :py:func:`~.check` + :py:func:`~.affect` in right order. + + Does nothing. + """ + return self.check(runtime) + + class Analyzer(object): """Analyzes a compilation and aids in execution processes. @@ -35,18 +91,21 @@ class Analyzer(object): self._storage = runtime.storage self._execution_graph = runtime.compilation.execution_graph self._check_atom_transition = runtime.check_atom_transition + self._fetch_edge_deciders = runtime.fetch_edge_deciders def get_next_nodes(self, node=None): + """Get next nodes to run (originating from node or all nodes).""" if node is None: execute = self.browse_nodes_for_execute() revert = self.browse_nodes_for_revert() return execute + revert - state = self.get_state(node) intention = self._storage.get_atom_intention(node.name) if state == st.SUCCESS: if intention == st.REVERT: - return [node] + return [ + (node, NoOpDecider()), + ] elif intention == st.EXECUTE: return self.browse_nodes_for_execute(node) else: @@ -61,70 +120,86 @@ class Analyzer(object): def browse_nodes_for_execute(self, node=None): """Browse next nodes to execute. - This returns a collection of nodes that are ready to be executed, if - given a specific node it will only examine the successors of that node, - otherwise it will examine the whole graph. + This returns a collection of nodes that *may* be ready to be + executed, if given a specific node it will only examine the successors + of that node, otherwise it will examine the whole graph. """ - if node: + if node is not None: nodes = self._execution_graph.successors(node) else: nodes = self._execution_graph.nodes_iter() - - available_nodes = [] + ready_nodes = [] for node in nodes: - if self._is_ready_for_execute(node): - available_nodes.append(node) - return available_nodes + is_ready, late_decider = self._get_maybe_ready_for_execute(node) + if is_ready: + ready_nodes.append((node, late_decider)) + return ready_nodes def browse_nodes_for_revert(self, node=None): """Browse next nodes to revert. - This returns a collection of nodes that are ready to be be reverted, if - given a specific node it will only examine the predecessors of that - node, otherwise it will examine the whole graph. + This returns a collection of nodes that *may* be ready to be be + reverted, if given a specific node it will only examine the + predecessors of that node, otherwise it will examine the whole + graph. """ - if node: + if node is not None: nodes = self._execution_graph.predecessors(node) else: nodes = self._execution_graph.nodes_iter() - - available_nodes = [] + ready_nodes = [] for node in nodes: - if self._is_ready_for_revert(node): - available_nodes.append(node) - return available_nodes + is_ready, late_decider = self._get_maybe_ready_for_revert(node) + if is_ready: + ready_nodes.append((node, late_decider)) + return ready_nodes + + def _get_maybe_ready_for_execute(self, atom): + """Returns if an atom is *likely* ready to be executed.""" - def _is_ready_for_execute(self, atom): - """Checks if atom is ready to be executed.""" state = self.get_state(atom) intention = self._storage.get_atom_intention(atom.name) transition = self._check_atom_transition(atom, state, st.RUNNING) if not transition or intention != st.EXECUTE: - return False + return (False, None) - atom_names = [] - for prev_atom in self._execution_graph.predecessors(atom): - atom_names.append(prev_atom.name) + predecessor_names = [] + for previous_atom in self._execution_graph.predecessors(atom): + predecessor_names.append(previous_atom.name) - atom_states = self._storage.get_atoms_states(atom_names) - return all(state == st.SUCCESS and intention == st.EXECUTE - for state, intention in six.itervalues(atom_states)) + predecessor_states = self._storage.get_atoms_states(predecessor_names) + predecessor_states_iter = six.itervalues(predecessor_states) + ok_to_run = all(state == st.SUCCESS and intention == st.EXECUTE + for state, intention in predecessor_states_iter) + + if not ok_to_run: + return (False, None) + else: + edge_deciders = self._fetch_edge_deciders(atom) + return (True, IgnoreDecider(atom, edge_deciders)) + + def _get_maybe_ready_for_revert(self, atom): + """Returns if an atom is *likely* ready to be reverted.""" - def _is_ready_for_revert(self, atom): - """Checks if atom is ready to be reverted.""" state = self.get_state(atom) intention = self._storage.get_atom_intention(atom.name) transition = self._check_atom_transition(atom, state, st.REVERTING) if not transition or intention not in (st.REVERT, st.RETRY): - return False + return (False, None) - atom_names = [] - for prev_atom in self._execution_graph.successors(atom): - atom_names.append(prev_atom.name) + predecessor_names = [] + for previous_atom in self._execution_graph.successors(atom): + predecessor_names.append(previous_atom.name) - atom_states = self._storage.get_atoms_states(atom_names) - return all(state in (st.PENDING, st.REVERTED) - for state, intention in six.itervalues(atom_states)) + predecessor_states = self._storage.get_atoms_states(predecessor_names) + predecessor_states_iter = six.itervalues(predecessor_states) + ok_to_run = all(state in (st.PENDING, st.REVERTED) + for state, intention in predecessor_states_iter) + + if not ok_to_run: + return (False, None) + else: + return (True, NoOpDecider()) def iterate_subgraph(self, atom): """Iterates a subgraph connected to given atom.""" @@ -142,17 +217,24 @@ class Analyzer(object): yield node def iterate_all_nodes(self): + """Yields back all nodes in the execution graph.""" for node in self._execution_graph.nodes_iter(): yield node def find_atom_retry(self, atom): + """Returns the retry atom associated to the given atom (or none).""" return self._execution_graph.node[atom].get('retry') def is_success(self): + """Checks if all nodes in the execution graph are in 'happy' state.""" for atom in self.iterate_all_nodes(): - if self.get_state(atom) != st.SUCCESS: + atom_state = self.get_state(atom) + if atom_state == st.IGNORE: + continue + if atom_state != st.SUCCESS: return False return True def get_state(self, atom): + """Gets the state of a given atom (from the backend storage unit).""" return self._storage.get_atom_state(atom.name) diff --git a/taskflow/engines/action_engine/runner.py b/taskflow/engines/action_engine/runner.py index 8d637c1c..9b6043a3 100644 --- a/taskflow/engines/action_engine/runner.py +++ b/taskflow/engines/action_engine/runner.py @@ -94,6 +94,7 @@ class Runner(object): ignorable_states = (st.SCHEDULING, st.WAITING, st.RESUMING, st.ANALYZING) def __init__(self, runtime, waiter): + self._runtime = runtime self._analyzer = runtime.analyzer self._completer = runtime.completer self._scheduler = runtime.scheduler @@ -111,13 +112,26 @@ class Runner(object): if timeout is None: timeout = _WAITING_TIMEOUT + # Cache some local functions/methods... + do_schedule = self._scheduler.schedule + wait_for_any = self._waiter.wait_for_any + do_complete = self._completer.complete + + def iter_next_nodes(target_node=None): + # Yields and filters and tweaks the next nodes to execute... + maybe_nodes = self._analyzer.get_next_nodes(node=target_node) + for node, late_decider in maybe_nodes: + proceed = late_decider.check_and_affect(self._runtime) + if proceed: + yield node + def resume(old_state, new_state, event): # This reaction function just updates the state machines memory # to include any nodes that need to be executed (from a previous # attempt, which may be empty if never ran before) and any nodes # that are now ready to be ran. memory.next_nodes.update(self._completer.resume()) - memory.next_nodes.update(self._analyzer.get_next_nodes()) + memory.next_nodes.update(iter_next_nodes()) return _SCHEDULE def game_over(old_state, new_state, event): @@ -127,7 +141,7 @@ class Runner(object): # it is *always* called before the final state is entered. if memory.failures: return _FAILED - if self._analyzer.get_next_nodes(): + if any(1 for node in iter_next_nodes()): return _SUSPENDED elif self._analyzer.is_success(): return _SUCCESS @@ -141,8 +155,7 @@ class Runner(object): # that holds this information to stop or suspend); handles failures # that occur during this process safely... if self.runnable() and memory.next_nodes: - not_done, failures = self._scheduler.schedule( - memory.next_nodes) + not_done, failures = do_schedule(memory.next_nodes) if not_done: memory.not_done.update(not_done) if failures: @@ -155,8 +168,7 @@ class Runner(object): # call sometime in the future, or equivalent that will work in # py2 and py3. if memory.not_done: - done, not_done = self._waiter.wait_for_any(memory.not_done, - timeout) + done, not_done = wait_for_any(memory.not_done, timeout) memory.done.update(done) memory.not_done = not_done return _ANALYZE @@ -173,7 +185,7 @@ class Runner(object): node = fut.atom try: event, result = fut.result() - retain = self._completer.complete(node, event, result) + retain = do_complete(node, event, result) if isinstance(result, failure.Failure): if retain: memory.failures.append(result) @@ -196,7 +208,7 @@ class Runner(object): memory.failures.append(failure.Failure()) else: try: - more_nodes = self._analyzer.get_next_nodes(node) + more_nodes = set(iter_next_nodes(target_node=node)) except Exception: memory.failures.append(failure.Failure()) else: diff --git a/taskflow/engines/action_engine/runtime.py b/taskflow/engines/action_engine/runtime.py index 0439da17..0cf303c6 100644 --- a/taskflow/engines/action_engine/runtime.py +++ b/taskflow/engines/action_engine/runtime.py @@ -23,6 +23,7 @@ from taskflow.engines.action_engine import completer as co from taskflow.engines.action_engine import runner as ru from taskflow.engines.action_engine import scheduler as sched from taskflow.engines.action_engine import scopes as sc +from taskflow import flow as flow_type from taskflow import states as st from taskflow import task from taskflow.utils import misc @@ -61,6 +62,7 @@ class Runtime(object): 'retry': self.retry_scheduler, 'task': self.task_scheduler, } + execution_graph = self._compilation.execution_graph for atom in self.analyzer.iterate_all_nodes(): metadata = {} walker = sc.ScopeWalker(self.compilation, atom, names_only=True) @@ -72,10 +74,20 @@ class Runtime(object): check_transition_handler = st.check_retry_transition change_state_handler = change_state_handlers['retry'] scheduler = schedulers['retry'] + edge_deciders = {} + for previous_atom in execution_graph.predecessors(atom): + # If there is any link function that says if this connection + # is able to run (or should not) ensure we retain it and use + # it later as needed. + u_v_data = execution_graph.adj[previous_atom][atom] + u_v_decider = u_v_data.get(flow_type.LINK_DECIDER) + if u_v_decider is not None: + edge_deciders[previous_atom.name] = u_v_decider metadata['scope_walker'] = walker metadata['check_transition_handler'] = check_transition_handler metadata['change_state_handler'] = change_state_handler metadata['scheduler'] = scheduler + metadata['edge_deciders'] = edge_deciders self._atom_cache[atom.name] = metadata @property @@ -130,6 +142,14 @@ class Runtime(object): check_transition_handler = metadata['check_transition_handler'] return check_transition_handler(current_state, target_state) + def fetch_edge_deciders(self, atom): + """Fetches the edge deciders for the given atom.""" + # This does not check if the name exists (since this is only used + # internally to the engine, and is not exposed to atoms that will + # not exist and therefore doesn't need to handle that case). + metadata = self._atom_cache[atom.name] + return metadata['edge_deciders'] + def fetch_scheduler(self, atom): """Fetches the cached specific scheduler for the given atom.""" # This does not check if the name exists (since this is only used diff --git a/taskflow/examples/switch_graph_flow.py b/taskflow/examples/switch_graph_flow.py new file mode 100644 index 00000000..273763cd --- /dev/null +++ b/taskflow/examples/switch_graph_flow.py @@ -0,0 +1,75 @@ +# -*- coding: utf-8 -*- + +# Copyright (C) 2014 Yahoo! Inc. 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. + +import logging +import os +import sys + +logging.basicConfig(level=logging.ERROR) + +top_dir = os.path.abspath(os.path.join(os.path.dirname(__file__), + os.pardir, + os.pardir)) +sys.path.insert(0, top_dir) + +from taskflow import engines +from taskflow.patterns import graph_flow as gf +from taskflow.persistence import backends +from taskflow import task +from taskflow.utils import persistence_utils as pu + + +class DummyTask(task.Task): + def execute(self): + print("Running %s" % self.name) + + +def allow(history): + print(history) + return False + + +r = gf.Flow("root") +r_a = DummyTask('r-a') +r_b = DummyTask('r-b') +r.add(r_a, r_b) +r.link(r_a, r_b, decider=allow) + +backend = backends.fetch({ + 'connection': 'memory://', +}) +book, flow_detail = pu.temporary_flow_detail(backend=backend) + +e = engines.load(r, flow_detail=flow_detail, book=book, backend=backend) +e.compile() +e.prepare() +e.run() + + +print("---------") +print("After run") +print("---------") +entries = [os.path.join(backend.memory.root_path, child) + for child in backend.memory.ls(backend.memory.root_path)] +while entries: + path = entries.pop() + value = backend.memory[path] + if value: + print("%s -> %s" % (path, value)) + else: + print("%s" % (path)) + entries.extend(os.path.join(path, child) + for child in backend.memory.ls(path)) diff --git a/taskflow/flow.py b/taskflow/flow.py index 5eb05825..56786d4d 100644 --- a/taskflow/flow.py +++ b/taskflow/flow.py @@ -31,6 +31,9 @@ LINK_RETRY = 'retry' # This key denotes the link was created due to symbol constraints and the # value will be a set of names that the constraint ensures are satisfied. LINK_REASONS = 'reasons' +# +# This key denotes a callable that will determine if the target is visited. +LINK_DECIDER = 'decider' @six.add_metaclass(abc.ABCMeta) diff --git a/taskflow/patterns/graph_flow.py b/taskflow/patterns/graph_flow.py index 7d407c8c..50a4d61d 100644 --- a/taskflow/patterns/graph_flow.py +++ b/taskflow/patterns/graph_flow.py @@ -16,6 +16,8 @@ import collections +import six + from taskflow import exceptions as exc from taskflow import flow from taskflow.types import graph as gr @@ -66,16 +68,20 @@ class Flow(flow.Flow): #: Extracts the unsatisified symbol requirements of a single node. _unsatisfied_requires = staticmethod(_unsatisfied_requires) - def link(self, u, v): + def link(self, u, v, decider=None): """Link existing node u as a runtime dependency of existing node v.""" if not self._graph.has_node(u): raise ValueError("Node '%s' not found to link from" % (u)) if not self._graph.has_node(v): raise ValueError("Node '%s' not found to link to" % (v)) - self._swap(self._link(u, v, manual=True)) + if decider is not None: + if not six.callable(decider): + raise ValueError("Decider boolean callback must be callable") + self._swap(self._link(u, v, manual=True, decider=decider)) return self - def _link(self, u, v, graph=None, reason=None, manual=False): + def _link(self, u, v, graph=None, + reason=None, manual=False, decider=None): mutable_graph = True if graph is None: graph = self._graph @@ -85,6 +91,8 @@ class Flow(flow.Flow): attrs = graph.get_edge_data(u, v) if not attrs: attrs = {} + if decider is not None: + attrs[flow.LINK_DECIDER] = decider if manual: attrs[flow.LINK_MANUAL] = True if reason is not None: @@ -281,9 +289,9 @@ class TargetedFlow(Flow): self._subgraph = None return self - def link(self, u, v): + def link(self, u, v, decider=None): """Link existing node u as a runtime dependency of existing node v.""" - super(TargetedFlow, self).link(u, v) + super(TargetedFlow, self).link(u, v, decider=decider) # reset cached subgraph, in case it was affected self._subgraph = None return self diff --git a/taskflow/states.py b/taskflow/states.py index 265d6b2c..cbef58c7 100644 --- a/taskflow/states.py +++ b/taskflow/states.py @@ -40,10 +40,11 @@ REVERTING = REVERTING SUCCESS = SUCCESS RUNNING = RUNNING RETRYING = 'RETRYING' +IGNORE = 'IGNORE' # Atom intentions. EXECUTE = 'EXECUTE' -IGNORE = 'IGNORE' +IGNORE = IGNORE REVERT = 'REVERT' RETRY = 'RETRY' INTENTIONS = (EXECUTE, IGNORE, REVERT, RETRY) @@ -160,6 +161,7 @@ def check_flow_transition(old_state, new_state): _ALLOWED_TASK_TRANSITIONS = frozenset(( (PENDING, RUNNING), # run it! + (PENDING, IGNORE), # skip it! (RUNNING, SUCCESS), # the task finished successfully (RUNNING, FAILURE), # the task failed @@ -171,6 +173,7 @@ _ALLOWED_TASK_TRANSITIONS = frozenset(( (REVERTING, FAILURE), # revert failed (REVERTED, PENDING), # try again + (IGNORE, PENDING), # try again )) diff --git a/taskflow/tests/unit/test_engines.py b/taskflow/tests/unit/test_engines.py index 4e38dfa5..ed073e6a 100644 --- a/taskflow/tests/unit/test_engines.py +++ b/taskflow/tests/unit/test_engines.py @@ -15,7 +15,9 @@ # under the License. import contextlib +import functools +import six import testtools import taskflow.engines @@ -772,6 +774,126 @@ class EngineMissingDepsTest(utils.EngineTestBase): self.assertIsNotNone(c_e.cause) +class EngineGraphConditionalFlowTest(utils.EngineTestBase): + + def test_graph_flow_conditional(self): + flow = gf.Flow('root') + + task1 = utils.ProgressingTask(name='task1') + task2 = utils.ProgressingTask(name='task2') + task2_2 = utils.ProgressingTask(name='task2_2') + task3 = utils.ProgressingTask(name='task3') + + flow.add(task1, task2, task2_2, task3) + flow.link(task1, task2, decider=lambda history: False) + flow.link(task2, task2_2) + flow.link(task1, task3, decider=lambda history: True) + + engine = self._make_engine(flow) + with utils.CaptureListener(engine, capture_flow=False) as capturer: + engine.run() + + expected = set([ + 'task1.t RUNNING', + 'task1.t SUCCESS(5)', + + 'task2.t IGNORE', + 'task2_2.t IGNORE', + + 'task3.t RUNNING', + 'task3.t SUCCESS(5)', + ]) + self.assertEqual(expected, set(capturer.values)) + + def test_graph_flow_diamond_ignored(self): + flow = gf.Flow('root') + + task1 = utils.ProgressingTask(name='task1') + task2 = utils.ProgressingTask(name='task2') + task3 = utils.ProgressingTask(name='task3') + task4 = utils.ProgressingTask(name='task4') + + flow.add(task1, task2, task3, task4) + flow.link(task1, task2) + flow.link(task2, task4, decider=lambda history: False) + flow.link(task1, task3) + flow.link(task3, task4, decider=lambda history: True) + + engine = self._make_engine(flow) + with utils.CaptureListener(engine, capture_flow=False) as capturer: + engine.run() + + expected = set([ + 'task1.t RUNNING', + 'task1.t SUCCESS(5)', + + 'task2.t RUNNING', + 'task2.t SUCCESS(5)', + + 'task3.t RUNNING', + 'task3.t SUCCESS(5)', + + 'task4.t IGNORE', + ]) + self.assertEqual(expected, set(capturer.values)) + self.assertEqual(states.IGNORE, + engine.storage.get_atom_state('task4')) + self.assertEqual(states.IGNORE, + engine.storage.get_atom_intention('task4')) + + def test_graph_flow_conditional_history(self): + + def even_odd_decider(history, allowed): + total = sum(six.itervalues(history)) + if total == allowed: + return True + return False + + flow = gf.Flow('root') + + task1 = utils.TaskMultiArgOneReturn(name='task1') + task2 = utils.ProgressingTask(name='task2') + task2_2 = utils.ProgressingTask(name='task2_2') + task3 = utils.ProgressingTask(name='task3') + task3_3 = utils.ProgressingTask(name='task3_3') + + flow.add(task1, task2, task2_2, task3, task3_3) + flow.link(task1, task2, + decider=functools.partial(even_odd_decider, allowed=2)) + flow.link(task2, task2_2) + + flow.link(task1, task3, + decider=functools.partial(even_odd_decider, allowed=1)) + flow.link(task3, task3_3) + + engine = self._make_engine(flow) + engine.storage.inject({'x': 0, 'y': 1, 'z': 1}) + + with utils.CaptureListener(engine, capture_flow=False) as capturer: + engine.run() + + expected = set([ + 'task1.t RUNNING', 'task1.t SUCCESS(2)', + 'task3.t IGNORE', 'task3_3.t IGNORE', + 'task2.t RUNNING', 'task2.t SUCCESS(5)', + 'task2_2.t RUNNING', 'task2_2.t SUCCESS(5)', + ]) + self.assertEqual(expected, set(capturer.values)) + + engine = self._make_engine(flow) + engine.storage.inject({'x': 0, 'y': 0, 'z': 1}) + with utils.CaptureListener(engine, capture_flow=False) as capturer: + engine.run() + + expected = set([ + 'task1.t RUNNING', 'task1.t SUCCESS(1)', + 'task2.t IGNORE', 'task2_2.t IGNORE', + 'task3.t RUNNING', 'task3.t SUCCESS(5)', + 'task3_3.t RUNNING', 'task3_3.t SUCCESS(5)', + ]) + self.assertEqual(expected, set(capturer.values)) + + class EngineCheckingTaskTest(utils.EngineTestBase): # FIXME: this test uses a inner class that workers/process engines can't # get to, so we need to do something better to make this test useful for @@ -805,6 +927,7 @@ class SerialEngineTest(EngineTaskTest, EngineOptionalRequirementsTest, EngineGraphFlowTest, EngineMissingDepsTest, + EngineGraphConditionalFlowTest, EngineCheckingTaskTest, test.TestCase): def _make_engine(self, flow, @@ -832,6 +955,7 @@ class ParallelEngineWithThreadsTest(EngineTaskTest, EngineOptionalRequirementsTest, EngineGraphFlowTest, EngineMissingDepsTest, + EngineGraphConditionalFlowTest, EngineCheckingTaskTest, test.TestCase): _EXECUTOR_WORKERS = 2 @@ -871,6 +995,7 @@ class ParallelEngineWithEventletTest(EngineTaskTest, EngineOptionalRequirementsTest, EngineGraphFlowTest, EngineMissingDepsTest, + EngineGraphConditionalFlowTest, EngineCheckingTaskTest, test.TestCase): @@ -893,6 +1018,7 @@ class ParallelEngineWithProcessTest(EngineTaskTest, EngineOptionalRequirementsTest, EngineGraphFlowTest, EngineMissingDepsTest, + EngineGraphConditionalFlowTest, test.TestCase): _EXECUTOR_WORKERS = 2 @@ -920,6 +1046,7 @@ class WorkerBasedEngineTest(EngineTaskTest, EngineOptionalRequirementsTest, EngineGraphFlowTest, EngineMissingDepsTest, + EngineGraphConditionalFlowTest, test.TestCase): def setUp(self): super(WorkerBasedEngineTest, self).setUp() diff --git a/tools/state_graph.py b/tools/state_graph.py index 997920da..c37cd703 100755 --- a/tools/state_graph.py +++ b/tools/state_graph.py @@ -14,6 +14,8 @@ # License for the specific language governing permissions and limitations # under the License. +import mock + import optparse import os import sys @@ -37,10 +39,10 @@ from taskflow.types import fsm # actually be running it...). class DummyRuntime(object): def __init__(self): - self.analyzer = None - self.completer = None - self.scheduler = None - self.storage = None + self.analyzer = mock.MagicMock() + self.completer = mock.MagicMock() + self.scheduler = mock.MagicMock() + self.storage = mock.MagicMock() def clean_event(name): @@ -130,7 +132,7 @@ def main(): list(states._ALLOWED_RETRY_TRANSITIONS)) elif options.engines: source_type = "Engines" - r = runner.Runner(DummyRuntime(), None) + r = runner.Runner(DummyRuntime(), mock.MagicMock()) source, memory = r.build() internal_states.extend(runner._META_STATES) ordering = 'out' From 9800c2514675cc7a08d896978ee55e02c0d59167 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 1 Jul 2015 18:55:36 -0700 Subject: [PATCH 233/246] Denote mail subject should be '[Oslo][TaskFlow]' Change-Id: I58370066d915260cf940088770f5af0096f057a8 --- doc/source/index.rst | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/doc/source/index.rst b/doc/source/index.rst index acea31cb..4c0ab593 100644 --- a/doc/source/index.rst +++ b/doc/source/index.rst @@ -65,7 +65,8 @@ TaskFlow into your project: * Read over the `paradigm shifts`_ and engage the team in `IRC`_ (or via the `openstack-dev`_ mailing list) if these need more explanation (prefix - ``[TaskFlow]`` to your emails subject to get an even faster response). + ``[Oslo][TaskFlow]`` to your emails subject to get an even faster + response). * Follow (or at least attempt to follow) some of the established `best practices`_ (feel free to add your own suggested best practices). * Keep in touch with the team (see above); we are all friendly and enjoy From 40cf88e55ceffedfa96865696ba439312a6c66a4 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 24 Jun 2015 20:47:40 -0700 Subject: [PATCH 234/246] Expose strategies so doc generation can easily pick them up Since we have put a warning on the engine components docs saying they should not be used externally it seems ok to make these more visible so that people reading the docs can follow more of how it works (and the components involved there-in). Depends-On: I0f07858e96ea3baf46f8a453e253b9ed29c7f7e2 Depends-On: I33bd2d9dff9cb7dc1a50177db7286b7317966784 Change-Id: I24d5cbfd47aa786ffab923243b97c7ae43b2f21f --- taskflow/engines/action_engine/completer.py | 36 +++++++++++++-------- 1 file changed, 23 insertions(+), 13 deletions(-) diff --git a/taskflow/engines/action_engine/completer.py b/taskflow/engines/action_engine/completer.py index 7e04862c..318e3bc0 100644 --- a/taskflow/engines/action_engine/completer.py +++ b/taskflow/engines/action_engine/completer.py @@ -31,24 +31,34 @@ LOG = logging.getLogger(__name__) @six.add_metaclass(abc.ABCMeta) -class _Strategy(object): - """Local/internal helper strategy base object""" +class Strategy(object): + """Failure resolution strategy base class.""" + + strategy = None def __init__(self, runtime): self._runtime = runtime + @abc.abstractmethod + def apply(self): + """Applies some algorithm to resolve some detected failure.""" + def __str__(self): base = reflection.get_class_name(self, fully_qualified=False) - return base + "(strategy=%s)" % (self.strategy.name) + if self.strategy is not None: + strategy_name = self.strategy.name + else: + strategy_name = "???" + return base + "(strategy=%s)" % (strategy_name) -class _RevertAndRetry(_Strategy): +class RevertAndRetry(Strategy): """Sets the *associated* subflow for revert to be later retried.""" strategy = retry_atom.RETRY def __init__(self, runtime, retry): - super(_RevertAndRetry, self).__init__(runtime) + super(RevertAndRetry, self).__init__(runtime) self._retry = retry def apply(self): @@ -59,13 +69,13 @@ class _RevertAndRetry(_Strategy): return tweaked -class _RevertAll(_Strategy): +class RevertAll(Strategy): """Sets *all* nodes/atoms to the ``REVERT`` intention.""" strategy = retry_atom.REVERT_ALL def __init__(self, runtime): - super(_RevertAll, self).__init__(runtime) + super(RevertAll, self).__init__(runtime) self._analyzer = runtime.analyzer def apply(self): @@ -73,13 +83,13 @@ class _RevertAll(_Strategy): state=None, intention=st.REVERT) -class _Revert(_Strategy): +class Revert(Strategy): """Sets atom and *associated* nodes to the ``REVERT`` intention.""" strategy = retry_atom.REVERT def __init__(self, runtime, atom): - super(_Revert, self).__init__(runtime) + super(Revert, self).__init__(runtime) self._atom = atom def apply(self): @@ -99,7 +109,7 @@ class Completer(object): self._retry_action = runtime.retry_action self._storage = runtime.storage self._task_action = runtime.task_action - self._undefined_resolver = _RevertAll(self._runtime) + self._undefined_resolver = RevertAll(self._runtime) def _complete_task(self, task, event, result): """Completes the given task, processes task failure.""" @@ -152,7 +162,7 @@ class Completer(object): # Ask retry controller what to do in case of failure. strategy = self._retry_action.on_failure(retry, atom, failure) if strategy == retry_atom.RETRY: - return _RevertAndRetry(self._runtime, retry) + return RevertAndRetry(self._runtime, retry) elif strategy == retry_atom.REVERT: # Ask parent retry and figure out what to do... parent_resolver = self._determine_resolution(retry, failure) @@ -162,9 +172,9 @@ class Completer(object): if parent_resolver is not self._undefined_resolver: if parent_resolver.strategy != retry_atom.REVERT: return parent_resolver - return _Revert(self._runtime, retry) + return Revert(self._runtime, retry) elif strategy == retry_atom.REVERT_ALL: - return _RevertAll(self._runtime) + return RevertAll(self._runtime) else: raise ValueError("Unknown atom failure resolution" " action/strategy '%s'" % strategy) From 63c67302488bea87e2e2870dc83e73aacddeaa34 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Mon, 8 Jun 2015 22:16:47 -0700 Subject: [PATCH 235/246] Notify on the individual engine steps When a conductor is running it is quite useful to be able to how long each engine step takes. To enable this information being output, add a notifier to the base conductor and use it in the blocking conductor to emit events around engine activities. This makes it possible to track the timing (or other information that can be gathered from these events) in a non-intrusive manner. In the `99_bottles.py` demo we also now use this to be able to easily see what the conductor is actively doing (without having to enable the more verbose DEBUG level logging). Change-Id: Ifd8ff38f82fc8135fe5fec4c8e41f0e06f4fdee3 --- taskflow/conductors/backends/impl_blocking.py | 28 ++++++++++- taskflow/conductors/base.py | 13 ++++++ taskflow/examples/99_bottles.py | 46 +++++++++++++++++-- 3 files changed, 80 insertions(+), 7 deletions(-) diff --git a/taskflow/conductors/backends/impl_blocking.py b/taskflow/conductors/backends/impl_blocking.py index 945f591b..a87eacdf 100644 --- a/taskflow/conductors/backends/impl_blocking.py +++ b/taskflow/conductors/backends/impl_blocking.py @@ -56,6 +56,16 @@ class BlockingConductor(base.Conductor): upon the jobboard capabilities to automatically abandon these jobs. """ + START_FINISH_EVENTS_EMITTED = tuple([ + 'compilation', 'preparation', + 'validation', 'running', + ]) + """Events will be emitted for the start and finish of each engine + activity defined above, the actual event name that can be registered + to subscribe to will be ``${event}_start`` and ``${event}_end`` where + the ``${event}`` in this pseudo-variable will be one of these events. + """ + def __init__(self, name, jobboard, persistence=None, engine=None, engine_options=None, wait_timeout=None): @@ -105,10 +115,24 @@ class BlockingConductor(base.Conductor): with ExitStack() as stack: for listener in listeners: stack.enter_context(listener) - LOG.debug("Dispatching engine %s for job: %s", engine, job) + LOG.debug("Dispatching engine for job '%s'", job) consume = True try: - engine.run() + for stage_func, event_name in [(engine.compile, 'compilation'), + (engine.prepare, 'preparation'), + (engine.validate, 'validation'), + (engine.run, 'running')]: + self._notifier.notify("%s_start" % event_name, { + 'job': job, + 'engine': engine, + 'conductor': self, + }) + stage_func() + self._notifier.notify("%s_end" % event_name, { + 'job': job, + 'engine': engine, + 'conductor': self, + }) except excp.WrappedFailure as e: if all((f.check(*NO_CONSUME_EXCEPTIONS) for f in e)): consume = False diff --git a/taskflow/conductors/base.py b/taskflow/conductors/base.py index 6e46fff8..69424232 100644 --- a/taskflow/conductors/base.py +++ b/taskflow/conductors/base.py @@ -20,6 +20,7 @@ import six from taskflow import engines from taskflow import exceptions as excp +from taskflow.types import notifier @six.add_metaclass(abc.ABCMeta) @@ -45,6 +46,18 @@ class Conductor(object): self._engine_options = engine_options.copy() self._persistence = persistence self._lock = threading.RLock() + self._notifier = notifier.Notifier() + + @property + def notifier(self): + """The conductor actions (or other state changes) notifier. + + NOTE(harlowja): different conductor implementations may emit + different events + event details at different times, so refer to your + conductor documentation to know exactly what can and what can not be + subscribed to. + """ + return self._notifier def _flow_detail_from_job(self, job): """Extracts a flow detail from a job (via some manner). diff --git a/taskflow/examples/99_bottles.py b/taskflow/examples/99_bottles.py index 90894e9c..cbcf54ce 100644 --- a/taskflow/examples/99_bottles.py +++ b/taskflow/examples/99_bottles.py @@ -34,6 +34,7 @@ from taskflow.patterns import linear_flow as lf from taskflow.persistence import backends as persistence_backends from taskflow.persistence import logbook from taskflow import task +from taskflow.types import timing from oslo_utils import uuidutils @@ -61,10 +62,11 @@ HOW_MANY_BOTTLES = 99 class TakeABottleDown(task.Task): - def execute(self): + def execute(self, bottles_left): sys.stdout.write('Take one down, ') sys.stdout.flush() time.sleep(TAKE_DOWN_DELAY) + return bottles_left - 1 class PassItAround(task.Task): @@ -82,16 +84,49 @@ class Conclusion(task.Task): def make_bottles(count): s = lf.Flow("bottle-song") - for bottle in reversed(list(range(1, count + 1))): - take_bottle = TakeABottleDown("take-bottle-%s" % bottle) + + take_bottle = TakeABottleDown("take-bottle-%s" % count, + inject={'bottles_left': count}, + provides='bottles_left') + pass_it = PassItAround("pass-%s-around" % count) + next_bottles = Conclusion("next-bottles-%s" % (count - 1)) + s.add(take_bottle, pass_it, next_bottles) + + for bottle in reversed(list(range(1, count))): + take_bottle = TakeABottleDown("take-bottle-%s" % bottle, + provides='bottles_left') pass_it = PassItAround("pass-%s-around" % bottle) - next_bottles = Conclusion("next-bottles-%s" % (bottle - 1), - inject={"bottles_left": bottle - 1}) + next_bottles = Conclusion("next-bottles-%s" % (bottle - 1)) s.add(take_bottle, pass_it, next_bottles) + return s def run_conductor(): + event_watches = {} + + # This will be triggered by the conductor doing various activities + # with engines, and is quite nice to be able to see the various timing + # segments (which is useful for debugging, or watching, or figuring out + # where to optimize). + def on_conductor_event(event, details): + print("Event '%s' has been received..." % event) + print("Details = %s" % details) + if event.endswith("_start"): + w = timing.StopWatch() + w.start() + base_event = event[0:-len("_start")] + event_watches[base_event] = w + if event.endswith("_end"): + base_event = event[0:-len("_end")] + try: + w = event_watches.pop(base_event) + w.stop() + print("It took %0.3f seconds for event '%s' to finish" + % (w.elapsed(), base_event)) + except KeyError: + pass + print("Starting conductor with pid: %s" % ME) my_name = "conductor-%s" % ME persist_backend = persistence_backends.fetch(PERSISTENCE_URI) @@ -104,6 +139,7 @@ def run_conductor(): with contextlib.closing(job_backend): cond = conductor_backends.fetch('blocking', my_name, job_backend, persistence=persist_backend) + cond.notifier.register(cond.notifier.ANY, on_conductor_event) # Run forever, and kill -9 or ctrl-c me... try: cond.run() From 1e3dc09453e7e179fba613b61317ebad1556b77b Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Tue, 9 Jun 2015 18:19:11 -0700 Subject: [PATCH 236/246] Rename logbook module -> models module Since this module contains more than the logbook class and really is a our generic models that are used to hold the runtime structure it is more appropriate to place it under a models module and deprecate the usage of the old module by placing a warning there (so that when it is imported that warning is triggered). Change-Id: I79def5ee08f560d38f2c9dcefd0b33becc2a4d36 --- doc/source/engines.rst | 8 +- doc/source/jobs.rst | 2 +- doc/source/persistence.rst | 22 +- doc/source/resumption.rst | 14 +- taskflow/examples/persistence_example.py | 4 +- taskflow/examples/tox_conductor.py | 8 +- .../persistence/backends/impl_sqlalchemy.py | 10 +- .../84d6e888850_add_task_detail_type.py | 4 +- .../persistence/backends/sqlalchemy/tables.py | 4 +- taskflow/persistence/base.py | 4 +- taskflow/persistence/logbook.py | 889 +---------------- taskflow/persistence/models.py | 892 ++++++++++++++++++ taskflow/persistence/path_based.py | 28 +- taskflow/storage.py | 26 +- taskflow/tests/unit/persistence/base.py | 102 +- taskflow/tests/unit/test_engines.py | 4 +- taskflow/tests/unit/test_storage.py | 10 +- taskflow/utils/persistence_utils.py | 10 +- 18 files changed, 1039 insertions(+), 1002 deletions(-) create mode 100644 taskflow/persistence/models.py diff --git a/doc/source/engines.rst b/doc/source/engines.rst index 2f162c89..fa37274e 100644 --- a/doc/source/engines.rst +++ b/doc/source/engines.rst @@ -246,7 +246,7 @@ Preparation This stage (see :py:func:`~taskflow.engines.base.Engine.prepare`) starts by setting up the storage needed for all atoms in the compiled graph, ensuring -that corresponding :py:class:`~taskflow.persistence.logbook.AtomDetail` (or +that corresponding :py:class:`~taskflow.persistence.models.AtomDetail` (or subclass of) objects are created for each node in the graph. Validation @@ -293,7 +293,7 @@ for things like retry atom which can influence what a tasks intention should be :py:class:`~taskflow.engines.action_engine.analyzer.Analyzer` helper object which was designed to provide helper methods for this analysis). Once these intentions are determined and associated with each task (the intention is -also stored in the :py:class:`~taskflow.persistence.logbook.AtomDetail` object) +also stored in the :py:class:`~taskflow.persistence.models.AtomDetail` object) the :ref:`scheduling ` stage starts. .. _scheduling: @@ -323,8 +323,8 @@ submitted to complete. Once one of the future objects completes (or fails) that atoms result will be examined and finalized using a :py:class:`~taskflow.engines.action_engine.completer.Completer` implementation. It typically will persist results to a provided persistence backend (saved -into the corresponding :py:class:`~taskflow.persistence.logbook.AtomDetail` -and :py:class:`~taskflow.persistence.logbook.FlowDetail` objects via the +into the corresponding :py:class:`~taskflow.persistence.models.AtomDetail` +and :py:class:`~taskflow.persistence.models.FlowDetail` objects via the :py:class:`~taskflow.storage.Storage` helper) and reflect the new state of the atom. At this point what typically happens falls into two categories, one for if that atom failed and one for if it did not. If the atom diff --git a/doc/source/jobs.rst b/doc/source/jobs.rst index 2b826da4..3cd7f95f 100644 --- a/doc/source/jobs.rst +++ b/doc/source/jobs.rst @@ -30,7 +30,7 @@ Definitions Jobs A :py:class:`job ` consists of a unique identifier, name, and a reference to a :py:class:`logbook - ` which contains the details of the + ` which contains the details of the work that has been or should be/will be completed to finish the work that has been created for that job. diff --git a/doc/source/persistence.rst b/doc/source/persistence.rst index 70d6b7d7..8b451d42 100644 --- a/doc/source/persistence.rst +++ b/doc/source/persistence.rst @@ -40,38 +40,38 @@ On :doc:`engine ` construction typically a backend (it can be optional) will be provided which satisfies the :py:class:`~taskflow.persistence.base.Backend` abstraction. Along with providing a backend object a -:py:class:`~taskflow.persistence.logbook.FlowDetail` object will also be +:py:class:`~taskflow.persistence.models.FlowDetail` object will also be created and provided (this object will contain the details about the flow to be ran) to the engine constructor (or associated :py:meth:`load() ` helper functions). Typically a -:py:class:`~taskflow.persistence.logbook.FlowDetail` object is created from a -:py:class:`~taskflow.persistence.logbook.LogBook` object (the book object acts -as a type of container for :py:class:`~taskflow.persistence.logbook.FlowDetail` -and :py:class:`~taskflow.persistence.logbook.AtomDetail` objects). +:py:class:`~taskflow.persistence.models.FlowDetail` object is created from a +:py:class:`~taskflow.persistence.models.LogBook` object (the book object acts +as a type of container for :py:class:`~taskflow.persistence.models.FlowDetail` +and :py:class:`~taskflow.persistence.models.AtomDetail` objects). **Preparation**: Once an engine starts to run it will create a :py:class:`~taskflow.storage.Storage` object which will act as the engines interface to the underlying backend storage objects (it provides helper functions that are commonly used by the engine, avoiding repeating code when interacting with the provided -:py:class:`~taskflow.persistence.logbook.FlowDetail` and +:py:class:`~taskflow.persistence.models.FlowDetail` and :py:class:`~taskflow.persistence.base.Backend` objects). As an engine initializes it will extract (or create) -:py:class:`~taskflow.persistence.logbook.AtomDetail` objects for each atom in +:py:class:`~taskflow.persistence.models.AtomDetail` objects for each atom in the workflow the engine will be executing. **Execution:** When an engine beings to execute (see :doc:`engine ` for more of the details about how an engine goes about this process) it will examine any previously existing -:py:class:`~taskflow.persistence.logbook.AtomDetail` objects to see if they can +:py:class:`~taskflow.persistence.models.AtomDetail` objects to see if they can be used for resuming; see :doc:`resumption ` for more details on this subject. For atoms which have not finished (or did not finish correctly from a previous run) they will begin executing only after any dependent inputs are ready. This is done by analyzing the execution graph and looking at -predecessor :py:class:`~taskflow.persistence.logbook.AtomDetail` outputs and +predecessor :py:class:`~taskflow.persistence.models.AtomDetail` outputs and states (which may have been persisted in a past run). This will result in either using their previous information or by running those predecessors and -saving their output to the :py:class:`~taskflow.persistence.logbook.FlowDetail` +saving their output to the :py:class:`~taskflow.persistence.models.FlowDetail` and :py:class:`~taskflow.persistence.base.Backend` objects. This execution, analysis and interaction with the storage objects continues (what is described here is a simplification of what really happens; which is quite a bit @@ -288,7 +288,7 @@ Interfaces Models ====== -.. automodule:: taskflow.persistence.logbook +.. automodule:: taskflow.persistence.models Implementations =============== diff --git a/doc/source/resumption.rst b/doc/source/resumption.rst index 3be864f6..4a85ab6a 100644 --- a/doc/source/resumption.rst +++ b/doc/source/resumption.rst @@ -46,7 +46,7 @@ name serves a special purpose in the resumption process (as well as serving a useful purpose when running, allowing for atom identification in the :doc:`notification ` process). The reason for having names is that an atom in a flow needs to be somehow matched with (a potentially) -existing :py:class:`~taskflow.persistence.logbook.AtomDetail` during engine +existing :py:class:`~taskflow.persistence.models.AtomDetail` during engine resumption & subsequent running. The match should be: @@ -71,9 +71,9 @@ Scenarios ========= When new flow is loaded into engine, there is no persisted data for it yet, so -a corresponding :py:class:`~taskflow.persistence.logbook.FlowDetail` object +a corresponding :py:class:`~taskflow.persistence.models.FlowDetail` object will be created, as well as a -:py:class:`~taskflow.persistence.logbook.AtomDetail` object for each atom that +:py:class:`~taskflow.persistence.models.AtomDetail` object for each atom that is contained in it. These will be immediately saved into the persistence backend that is configured. If no persistence backend is configured, then as expected nothing will be saved and the atoms and flow will be ran in a @@ -94,7 +94,7 @@ When the factory function mentioned above returns the exact same the flow and atoms (no changes are performed). **Runtime change:** Nothing should be done -- the engine will re-associate -atoms with :py:class:`~taskflow.persistence.logbook.AtomDetail` objects by name +atoms with :py:class:`~taskflow.persistence.models.AtomDetail` objects by name and then the engine resumes. Atom was added @@ -105,7 +105,7 @@ in (for example for changing the runtime structure of what was previously ran in the first run). **Runtime change:** By default when the engine resumes it will notice that a -corresponding :py:class:`~taskflow.persistence.logbook.AtomDetail` does not +corresponding :py:class:`~taskflow.persistence.models.AtomDetail` does not exist and one will be created and associated. Atom was removed @@ -134,7 +134,7 @@ factory should replace this name where it was being used previously. exist when a new atom is added. In the future TaskFlow could make this easier by providing a ``upgrade()`` function that can be used to give users the ability to upgrade atoms before running (manual introspection & modification of -a :py:class:`~taskflow.persistence.logbook.LogBook` can be done before engine +a :py:class:`~taskflow.persistence.models.LogBook` can be done before engine loading and running to accomplish this in the meantime). Atom was split in two atoms or merged @@ -150,7 +150,7 @@ exist when a new atom is added or removed. In the future TaskFlow could make this easier by providing a ``migrate()`` function that can be used to give users the ability to migrate atoms previous data before running (manual introspection & modification of a -:py:class:`~taskflow.persistence.logbook.LogBook` can be done before engine +:py:class:`~taskflow.persistence.models.LogBook` can be done before engine loading and running to accomplish this in the meantime). Flow structure was changed diff --git a/taskflow/examples/persistence_example.py b/taskflow/examples/persistence_example.py index c911c2f1..de9b4274 100644 --- a/taskflow/examples/persistence_example.py +++ b/taskflow/examples/persistence_example.py @@ -31,7 +31,7 @@ sys.path.insert(0, self_dir) from taskflow import engines from taskflow.patterns import linear_flow as lf -from taskflow.persistence import logbook +from taskflow.persistence import models from taskflow import task from taskflow.utils import persistence_utils as p_utils @@ -94,7 +94,7 @@ with eu.get_backend(backend_uri) as backend: # Make a flow that will blow up if the file didn't exist previously, if it # did exist, assume we won't blow up (and therefore this shows the undo # and redo that a flow will go through). - book = logbook.LogBook("my-test") + book = models.LogBook("my-test") flow = make_flow(blowup=blowup) eu.print_wrapped("Running") try: diff --git a/taskflow/examples/tox_conductor.py b/taskflow/examples/tox_conductor.py index feff4245..66e575b5 100644 --- a/taskflow/examples/tox_conductor.py +++ b/taskflow/examples/tox_conductor.py @@ -42,7 +42,7 @@ from taskflow import engines from taskflow.jobs import backends as boards from taskflow.patterns import linear_flow from taskflow.persistence import backends as persistence -from taskflow.persistence import logbook +from taskflow.persistence import models from taskflow import task from taskflow.utils import threading_utils @@ -145,9 +145,9 @@ def generate_reviewer(client, saver, name=NAME): def make_save_book(saver, review_id): # Record what we want to happen (sometime in the future). - book = logbook.LogBook("book_%s" % review_id) - detail = logbook.FlowDetail("flow_%s" % review_id, - uuidutils.generate_uuid()) + book = models.LogBook("book_%s" % review_id) + detail = models.FlowDetail("flow_%s" % review_id, + uuidutils.generate_uuid()) book.add(detail) # Associate the factory method we want to be called (in the future) # with the book, so that the conductor will be able to call into diff --git a/taskflow/persistence/backends/impl_sqlalchemy.py b/taskflow/persistence/backends/impl_sqlalchemy.py index d5342018..b9aef44c 100644 --- a/taskflow/persistence/backends/impl_sqlalchemy.py +++ b/taskflow/persistence/backends/impl_sqlalchemy.py @@ -34,7 +34,7 @@ from taskflow import logging from taskflow.persistence.backends.sqlalchemy import migration from taskflow.persistence.backends.sqlalchemy import tables from taskflow.persistence import base -from taskflow.persistence import logbook +from taskflow.persistence import models from taskflow.types import failure from taskflow.utils import eventlet_utils from taskflow.utils import misc @@ -194,16 +194,16 @@ class _Alchemist(object): @staticmethod def convert_flow_detail(row): - return logbook.FlowDetail.from_dict(dict(row.items())) + return models.FlowDetail.from_dict(dict(row.items())) @staticmethod def convert_book(row): - return logbook.LogBook.from_dict(dict(row.items())) + return models.LogBook.from_dict(dict(row.items())) @staticmethod def convert_atom_detail(row): row = dict(row.items()) - atom_cls = logbook.atom_detail_class(row.pop('atom_type')) + atom_cls = models.atom_detail_class(row.pop('atom_type')) return atom_cls.from_dict(row) def atom_query_iter(self, conn, parent_uuid): @@ -457,7 +457,7 @@ class Connection(base.Connection): def _insert_atom_details(self, conn, ad, parent_uuid): value = ad.to_dict() value['parent_uuid'] = parent_uuid - value['atom_type'] = logbook.atom_detail_type(ad) + value['atom_type'] = models.atom_detail_type(ad) conn.execute(sql.insert(self._tables.atomdetails, value)) def _update_atom_details(self, conn, ad, e_ad): diff --git a/taskflow/persistence/backends/sqlalchemy/alembic/versions/84d6e888850_add_task_detail_type.py b/taskflow/persistence/backends/sqlalchemy/alembic/versions/84d6e888850_add_task_detail_type.py index 756cf93a..47441dcf 100644 --- a/taskflow/persistence/backends/sqlalchemy/alembic/versions/84d6e888850_add_task_detail_type.py +++ b/taskflow/persistence/backends/sqlalchemy/alembic/versions/84d6e888850_add_task_detail_type.py @@ -29,11 +29,11 @@ down_revision = '1c783c0c2875' from alembic import op import sqlalchemy as sa -from taskflow.persistence import logbook +from taskflow.persistence import models def upgrade(): - atom_types = sa.Enum(*logbook.ATOM_TYPES, name='atom_types') + atom_types = sa.Enum(*models.ATOM_TYPES, name='atom_types') column = sa.Column('atom_type', atom_types) bind = op.get_bind() impl = atom_types.dialect_impl(bind.dialect) diff --git a/taskflow/persistence/backends/sqlalchemy/tables.py b/taskflow/persistence/backends/sqlalchemy/tables.py index 47306b9c..28acca1a 100644 --- a/taskflow/persistence/backends/sqlalchemy/tables.py +++ b/taskflow/persistence/backends/sqlalchemy/tables.py @@ -22,7 +22,7 @@ from oslo_utils import uuidutils from sqlalchemy import Table, Column, String, ForeignKey, DateTime, Enum from sqlalchemy import types -from taskflow.persistence import logbook +from taskflow.persistence import models from taskflow import states Tables = collections.namedtuple('Tables', @@ -92,7 +92,7 @@ def fetch(metadata): default=uuidutils.generate_uuid), Column('failure', Json), Column('results', Json), - Column('atom_type', Enum(*logbook.ATOM_TYPES, + Column('atom_type', Enum(*models.ATOM_TYPES, name='atom_types')), Column('intention', Enum(*states.INTENTIONS, name='intentions'))) diff --git a/taskflow/persistence/base.py b/taskflow/persistence/base.py index 91fd095e..7f08c925 100644 --- a/taskflow/persistence/base.py +++ b/taskflow/persistence/base.py @@ -18,7 +18,7 @@ import abc import six -from taskflow.persistence import logbook +from taskflow.persistence import models @six.add_metaclass(abc.ABCMeta) @@ -125,5 +125,5 @@ class Connection(object): def _format_atom(atom_detail): return { 'atom': atom_detail.to_dict(), - 'type': logbook.atom_detail_type(atom_detail), + 'type': models.atom_detail_type(atom_detail), } diff --git a/taskflow/persistence/logbook.py b/taskflow/persistence/logbook.py index c7a6eae5..16273031 100644 --- a/taskflow/persistence/logbook.py +++ b/taskflow/persistence/logbook.py @@ -1,7 +1,6 @@ # -*- coding: utf-8 -*- -# Copyright (C) 2012 Yahoo! Inc. All Rights Reserved. -# Copyright (C) 2013 Rackspace Hosting All Rights Reserved. +# Copyright (C) 2015 Yahoo! Inc. 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 @@ -15,878 +14,24 @@ # License for the specific language governing permissions and limitations # under the License. -import abc -import copy +from debtcollector import removals -from oslo_utils import timeutils -from oslo_utils import uuidutils -import six +from taskflow.persistence import models -from taskflow import exceptions as exc -from taskflow import logging -from taskflow import states -from taskflow.types import failure as ft -LOG = logging.getLogger(__name__) +# TODO(harlowja): remove me in a future version, since the models +# module is more appropriately named to what the objects in it are used for... +removals.removed_module(__name__, replacement="'%s'" % models.__name__, + version="0.11", removal_version='?', + stacklevel=4) -# Internal helpers... - - -def _copy_function(deep_copy): - if deep_copy: - return copy.deepcopy - else: - return lambda x: x - - -def _safe_marshal_time(when): - if not when: - return None - return timeutils.marshall_now(now=when) - - -def _safe_unmarshal_time(when): - if not when: - return None - return timeutils.unmarshall_time(when) - - -def _fix_meta(data): - # Handle the case where older schemas allowed this to be non-dict by - # correcting this case by replacing it with a dictionary when a non-dict - # is found. - meta = data.get('meta') - if not isinstance(meta, dict): - meta = {} - return meta - - -class LogBook(object): - """A collection of flow details and associated metadata. - - Typically this class contains a collection of flow detail entries - for a given engine (or job) so that those entities can track what 'work' - has been completed for resumption, reverting and miscellaneous tracking - purposes. - - The data contained within this class need **not** be persisted to the - backend storage in real time. The data in this class will only be - guaranteed to be persisted when a save occurs via some backend - connection. - - NOTE(harlowja): the naming of this class is analogous to a ship's log or a - similar type of record used in detailing work that has been completed (or - work that has not been completed). - - :ivar created_at: A ``datetime.datetime`` object of when this logbook - was created. - :ivar updated_at: A ``datetime.datetime`` object of when this logbook - was last updated at. - :ivar meta: A dictionary of meta-data associated with this logbook. - """ - def __init__(self, name, uuid=None): - if uuid: - self._uuid = uuid - else: - self._uuid = uuidutils.generate_uuid() - self._name = name - self._flowdetails_by_id = {} - self.created_at = timeutils.utcnow() - self.updated_at = None - self.meta = {} - - def add(self, fd): - """Adds a new flow detail into this logbook. - - NOTE(harlowja): if an existing flow detail exists with the same - uuid the existing one will be overwritten with the newly provided - one. - - Does not *guarantee* that the details will be immediately saved. - """ - self._flowdetails_by_id[fd.uuid] = fd - self.updated_at = timeutils.utcnow() - - def find(self, flow_uuid): - """Locate the flow detail corresponding to the given uuid. - - :returns: the flow detail with that uuid - :rtype: :py:class:`.FlowDetail` (or ``None`` if not found) - """ - return self._flowdetails_by_id.get(flow_uuid, None) - - def merge(self, lb, deep_copy=False): - """Merges the current object state with the given ones state. - - If ``deep_copy`` is provided as truthy then the - local object will use ``copy.deepcopy`` to replace this objects - local attributes with the provided objects attributes (**only** if - there is a difference between this objects attributes and the - provided attributes). If ``deep_copy`` is falsey (the default) then a - reference copy will occur instead when a difference is detected. - - NOTE(harlowja): If the provided object is this object itself - then **no** merging is done. Also note that this does **not** merge - the flow details contained in either. - - :returns: this logbook (freshly merged with the incoming object) - :rtype: :py:class:`.LogBook` - """ - if lb is self: - return self - copy_fn = _copy_function(deep_copy) - if self.meta != lb.meta: - self.meta = copy_fn(lb.meta) - if lb.created_at != self.created_at: - self.created_at = copy_fn(lb.created_at) - if lb.updated_at != self.updated_at: - self.updated_at = copy_fn(lb.updated_at) - return self - - def to_dict(self, marshal_time=False): - """Translates the internal state of this object to a ``dict``. - - NOTE(harlowja): The returned ``dict`` does **not** include any - contained flow details. - - :returns: this logbook in ``dict`` form - """ - if not marshal_time: - marshal_fn = lambda x: x - else: - marshal_fn = _safe_marshal_time - return { - 'name': self.name, - 'meta': self.meta, - 'uuid': self.uuid, - 'updated_at': marshal_fn(self.updated_at), - 'created_at': marshal_fn(self.created_at), - } - - @classmethod - def from_dict(cls, data, unmarshal_time=False): - """Translates the given ``dict`` into an instance of this class. - - NOTE(harlowja): the ``dict`` provided should come from a prior - call to :meth:`.to_dict`. - - :returns: a new logbook - :rtype: :py:class:`.LogBook` - """ - if not unmarshal_time: - unmarshal_fn = lambda x: x - else: - unmarshal_fn = _safe_unmarshal_time - obj = cls(data['name'], uuid=data['uuid']) - obj.updated_at = unmarshal_fn(data['updated_at']) - obj.created_at = unmarshal_fn(data['created_at']) - obj.meta = _fix_meta(data) - return obj - - @property - def uuid(self): - """The unique identifer of this logbook.""" - return self._uuid - - @property - def name(self): - """The name of this logbook.""" - return self._name - - def __iter__(self): - for fd in six.itervalues(self._flowdetails_by_id): - yield fd - - def __len__(self): - return len(self._flowdetails_by_id) - - def copy(self, retain_contents=True): - """Copies this logbook. - - Creates a shallow copy of this logbook. If this logbook contains - flow details and ``retain_contents`` is truthy (the default) then - the flow details container will be shallow copied (the flow details - contained there-in will **not** be copied). If ``retain_contents`` is - falsey then the copied logbook will have **no** contained flow - details (but it will have the rest of the local objects attributes - copied). - - :returns: a new logbook - :rtype: :py:class:`.LogBook` - """ - clone = copy.copy(self) - if not retain_contents: - clone._flowdetails_by_id = {} - else: - clone._flowdetails_by_id = self._flowdetails_by_id.copy() - if self.meta: - clone.meta = self.meta.copy() - return clone - - -class FlowDetail(object): - """A collection of atom details and associated metadata. - - Typically this class contains a collection of atom detail entries that - represent the atoms in a given flow structure (along with any other needed - metadata relevant to that flow). - - The data contained within this class need **not** be persisted to the - backend storage in real time. The data in this class will only be - guaranteed to be persisted when a save (or update) occurs via some backend - connection. - - :ivar state: The state of the flow associated with this flow detail. - :ivar meta: A dictionary of meta-data associated with this flow detail. - """ - def __init__(self, name, uuid): - self._uuid = uuid - self._name = name - self._atomdetails_by_id = {} - self.state = None - self.meta = {} - - def update(self, fd): - """Updates the objects state to be the same as the given one. - - This will assign the private and public attributes of the given - flow detail directly to this object (replacing any existing - attributes in this object; even if they are the **same**). - - NOTE(harlowja): If the provided object is this object itself - then **no** update is done. - - :returns: this flow detail - :rtype: :py:class:`.FlowDetail` - """ - if fd is self: - return self - self._atomdetails_by_id = fd._atomdetails_by_id - self.state = fd.state - self.meta = fd.meta - return self - - def merge(self, fd, deep_copy=False): - """Merges the current object state with the given one's state. - - If ``deep_copy`` is provided as truthy then the - local object will use ``copy.deepcopy`` to replace this objects - local attributes with the provided objects attributes (**only** if - there is a difference between this objects attributes and the - provided attributes). If ``deep_copy`` is falsey (the default) then a - reference copy will occur instead when a difference is detected. - - NOTE(harlowja): If the provided object is this object itself - then **no** merging is done. Also this does **not** merge the atom - details contained in either. - - :returns: this flow detail (freshly merged with the incoming object) - :rtype: :py:class:`.FlowDetail` - """ - if fd is self: - return self - copy_fn = _copy_function(deep_copy) - if self.meta != fd.meta: - self.meta = copy_fn(fd.meta) - if self.state != fd.state: - # NOTE(imelnikov): states are just strings, no need to copy. - self.state = fd.state - return self - - def copy(self, retain_contents=True): - """Copies this flow detail. - - Creates a shallow copy of this flow detail. If this detail contains - flow details and ``retain_contents`` is truthy (the default) then - the atom details container will be shallow copied (the atom details - contained there-in will **not** be copied). If ``retain_contents`` is - falsey then the copied flow detail will have **no** contained atom - details (but it will have the rest of the local objects attributes - copied). - - :returns: a new flow detail - :rtype: :py:class:`.FlowDetail` - """ - clone = copy.copy(self) - if not retain_contents: - clone._atomdetails_by_id = {} - else: - clone._atomdetails_by_id = self._atomdetails_by_id.copy() - if self.meta: - clone.meta = self.meta.copy() - return clone - - def to_dict(self): - """Translates the internal state of this object to a ``dict``. - - NOTE(harlowja): The returned ``dict`` does **not** include any - contained atom details. - - :returns: this flow detail in ``dict`` form - """ - return { - 'name': self.name, - 'meta': self.meta, - 'state': self.state, - 'uuid': self.uuid, - } - - @classmethod - def from_dict(cls, data): - """Translates the given ``dict`` into an instance of this class. - - NOTE(harlowja): the ``dict`` provided should come from a prior - call to :meth:`.to_dict`. - - :returns: a new flow detail - :rtype: :py:class:`.FlowDetail` - """ - obj = cls(data['name'], data['uuid']) - obj.state = data.get('state') - obj.meta = _fix_meta(data) - return obj - - def add(self, ad): - """Adds a new atom detail into this flow detail. - - NOTE(harlowja): if an existing atom detail exists with the same - uuid the existing one will be overwritten with the newly provided - one. - - Does not *guarantee* that the details will be immediately saved. - """ - self._atomdetails_by_id[ad.uuid] = ad - - def find(self, ad_uuid): - """Locate the atom detail corresponding to the given uuid. - - :returns: the atom detail with that uuid - :rtype: :py:class:`.AtomDetail` (or ``None`` if not found) - """ - return self._atomdetails_by_id.get(ad_uuid) - - @property - def uuid(self): - """The unique identifer of this flow detail.""" - return self._uuid - - @property - def name(self): - """The name of this flow detail.""" - return self._name - - def __iter__(self): - for ad in six.itervalues(self._atomdetails_by_id): - yield ad - - def __len__(self): - return len(self._atomdetails_by_id) - - -@six.add_metaclass(abc.ABCMeta) -class AtomDetail(object): - """A collection of atom specific runtime information and metadata. - - This is a base **abstract** class that contains attributes that are used - to connect a atom to the persistence layer before, during, or after it is - running. It includes any results it may have produced, any state that it - may be in (for example ``FAILURE``), any exception that occurred when - running, and any associated stacktrace that may have occurring during an - exception being thrown. It may also contain any other metadata that - should also be stored along-side the details about the connected atom. - - The data contained within this class need **not** be persisted to the - backend storage in real time. The data in this class will only be - guaranteed to be persisted when a save (or update) occurs via some backend - connection. - - :ivar state: The state of the atom associated with this atom detail. - :ivar intention: The execution strategy of the atom associated - with this atom detail (used by an engine/others to - determine if the associated atom needs to be - executed, reverted, retried and so-on). - :ivar meta: A dictionary of meta-data associated with this atom detail. - :ivar version: A version tuple or string that represents the - atom version this atom detail is associated with (typically - used for introspection and any data migration - strategies). - :ivar results: Any results the atom produced from either its - ``execute`` method or from other sources. - :ivar failure: If the atom failed (possibly due to its ``execute`` - method raising) this will be a - :py:class:`~taskflow.types.failure.Failure` object that - represents that failure (if there was no failure this - will be set to none). - """ - - def __init__(self, name, uuid): - self._uuid = uuid - self._name = name - self.state = None - self.intention = states.EXECUTE - self.results = None - self.failure = None - self.meta = {} - self.version = None - - @staticmethod - def _was_failure(state, result): - # Internal helper method... - return state == states.FAILURE and isinstance(result, ft.Failure) - - @property - def last_results(self): - """Gets the atoms last result. - - If the atom has produced many results (for example if it has been - retried, reverted, executed and ...) this returns the last one of - many results. - """ - return self.results - - def update(self, ad): - """Updates the object's state to be the same as the given one. - - This will assign the private and public attributes of the given - atom detail directly to this object (replacing any existing - attributes in this object; even if they are the **same**). - - NOTE(harlowja): If the provided object is this object itself - then **no** update is done. - - :returns: this atom detail - :rtype: :py:class:`.AtomDetail` - """ - if ad is self: - return self - self.state = ad.state - self.intention = ad.intention - self.meta = ad.meta - self.failure = ad.failure - self.results = ad.results - self.version = ad.version - return self - - @abc.abstractmethod - def merge(self, other, deep_copy=False): - """Merges the current object state with the given ones state. - - If ``deep_copy`` is provided as truthy then the - local object will use ``copy.deepcopy`` to replace this objects - local attributes with the provided objects attributes (**only** if - there is a difference between this objects attributes and the - provided attributes). If ``deep_copy`` is falsey (the default) then a - reference copy will occur instead when a difference is detected. - - NOTE(harlowja): If the provided object is this object itself - then **no** merging is done. Do note that **no** results are merged - in this method. That operation **must** to be the responsibilty of - subclasses to implement and override this abstract method - and provide that merging themselves as they see fit. - - :returns: this atom detail (freshly merged with the incoming object) - :rtype: :py:class:`.AtomDetail` - """ - copy_fn = _copy_function(deep_copy) - # NOTE(imelnikov): states and intentions are just strings, - # so there is no need to copy them (strings are immutable in python). - self.state = other.state - self.intention = other.intention - if self.failure != other.failure: - # NOTE(imelnikov): we can't just deep copy Failures, as they - # contain tracebacks, which are not copyable. - if other.failure: - if deep_copy: - self.failure = other.failure.copy() - else: - self.failure = other.failure - else: - self.failure = None - if self.meta != other.meta: - self.meta = copy_fn(other.meta) - if self.version != other.version: - self.version = copy_fn(other.version) - return self - - @abc.abstractmethod - def put(self, state, result): - """Puts a result (acquired in the given state) into this detail.""" - - def to_dict(self): - """Translates the internal state of this object to a ``dict``. - - :returns: this atom detail in ``dict`` form - """ - if self.failure: - failure = self.failure.to_dict() - else: - failure = None - return { - 'failure': failure, - 'meta': self.meta, - 'name': self.name, - 'results': self.results, - 'state': self.state, - 'version': self.version, - 'intention': self.intention, - 'uuid': self.uuid, - } - - @classmethod - def from_dict(cls, data): - """Translates the given ``dict`` into an instance of this class. - - NOTE(harlowja): the ``dict`` provided should come from a prior - call to :meth:`.to_dict`. - - :returns: a new atom detail - :rtype: :py:class:`.AtomDetail` - """ - obj = cls(data['name'], data['uuid']) - obj.state = data.get('state') - obj.intention = data.get('intention') - obj.results = data.get('results') - obj.version = data.get('version') - obj.meta = _fix_meta(data) - failure = data.get('failure') - if failure: - obj.failure = ft.Failure.from_dict(failure) - return obj - - @property - def uuid(self): - """The unique identifer of this atom detail.""" - return self._uuid - - @property - def name(self): - """The name of this atom detail.""" - return self._name - - @abc.abstractmethod - def reset(self, state): - """Resets this atom detail and sets ``state`` attribute value.""" - - @abc.abstractmethod - def copy(self): - """Copies this atom detail.""" - - -class TaskDetail(AtomDetail): - """A task detail (an atom detail typically associated with a |tt| atom). - - .. |tt| replace:: :py:class:`~taskflow.task.BaseTask` - """ - - def reset(self, state): - """Resets this task detail and sets ``state`` attribute value. - - This sets any previously set ``results`` and ``failure`` attributes - back to ``None`` and sets the state to the provided one, as well as - setting this task details ``intention`` attribute to ``EXECUTE``. - """ - self.results = None - self.failure = None - self.state = state - self.intention = states.EXECUTE - - def put(self, state, result): - """Puts a result (acquired in the given state) into this detail. - - If the result is a :py:class:`~taskflow.types.failure.Failure` object - then the ``failure`` attribute will be set (and the ``results`` - attribute will be set to ``None``); if the result is not a - :py:class:`~taskflow.types.failure.Failure` object then the - ``results`` attribute will be set (and the ``failure`` attribute - will be set to ``None``). In either case the ``state`` - attribute will be set to the provided state. - """ - was_altered = False - if self.state != state: - self.state = state - was_altered = True - if self._was_failure(state, result): - if self.failure != result: - self.failure = result - was_altered = True - if self.results is not None: - self.results = None - was_altered = True - else: - # We don't really have the ability to determine equality of - # task (user) results at the current time, without making - # potentially bad guesses, so assume the task detail always needs - # to be saved if they are not exactly equivalent... - if self.results is not result: - self.results = result - was_altered = True - if self.failure is not None: - self.failure = None - was_altered = True - return was_altered - - def merge(self, other, deep_copy=False): - """Merges the current task detail with the given one. - - NOTE(harlowja): This merge does **not** copy and replace - the ``results`` attribute if it differs. Instead the current - objects ``results`` attribute directly becomes (via assignment) the - other objects ``results`` attribute. Also note that if the provided - object is this object itself then **no** merging is done. - - See: https://bugs.launchpad.net/taskflow/+bug/1452978 for - what happens if this is copied at a deeper level (for example by - using ``copy.deepcopy`` or by using ``copy.copy``). - - :returns: this task detail (freshly merged with the incoming object) - :rtype: :py:class:`.TaskDetail` - """ - if not isinstance(other, TaskDetail): - raise exc.NotImplementedError("Can only merge with other" - " task details") - if other is self: - return self - super(TaskDetail, self).merge(other, deep_copy=deep_copy) - if self.results != other.results: - self.results = other.results - return self - - def copy(self): - """Copies this task detail. - - Creates a shallow copy of this task detail (any meta-data and - version information that this object maintains is shallow - copied via ``copy.copy``). - - NOTE(harlowja): This copy does **not** perform ``copy.copy`` on - the ``results`` attribute of this object (before assigning to the - copy). Instead the current objects ``results`` attribute directly - becomes (via assignment) the copied objects ``results`` attribute. - - See: https://bugs.launchpad.net/taskflow/+bug/1452978 for - what happens if this is copied at a deeper level (for example by - using ``copy.deepcopy`` or by using ``copy.copy``). - - :returns: a new task detail - :rtype: :py:class:`.TaskDetail` - """ - clone = copy.copy(self) - clone.results = self.results - if self.meta: - clone.meta = self.meta.copy() - if self.version: - clone.version = copy.copy(self.version) - return clone - - -class RetryDetail(AtomDetail): - """A retry detail (an atom detail typically associated with a |rt| atom). - - .. |rt| replace:: :py:class:`~taskflow.retry.Retry` - """ - - def __init__(self, name, uuid): - super(RetryDetail, self).__init__(name, uuid) - self.results = [] - - def reset(self, state): - """Resets this retry detail and sets ``state`` attribute value. - - This sets any previously added ``results`` back to an empty list - and resets the ``failure`` attribute back to ``None`` and sets the - state to the provided one, as well as setting this atom - details ``intention`` attribute to ``EXECUTE``. - """ - self.results = [] - self.failure = None - self.state = state - self.intention = states.EXECUTE - - def copy(self): - """Copies this retry detail. - - Creates a shallow copy of this retry detail (any meta-data and - version information that this object maintains is shallow - copied via ``copy.copy``). - - NOTE(harlowja): This copy does **not** copy - the incoming objects ``results`` attribute. Instead this - objects ``results`` attribute list is iterated over and a new list - is constructed with each ``(data, failures)`` element in that list - having its ``failures`` (a dictionary of each named - :py:class:`~taskflow.types.failure.Failure` object that - occured) copied but its ``data`` is left untouched. After - this is done that new list becomes (via assignment) the cloned - objects ``results`` attribute. - - See: https://bugs.launchpad.net/taskflow/+bug/1452978 for - what happens if the ``data`` in ``results`` is copied at a - deeper level (for example by using ``copy.deepcopy`` or by - using ``copy.copy``). - - :returns: a new retry detail - :rtype: :py:class:`.RetryDetail` - """ - clone = copy.copy(self) - results = [] - # NOTE(imelnikov): we can't just deep copy Failures, as they - # contain tracebacks, which are not copyable. - for (data, failures) in self.results: - copied_failures = {} - for (key, failure) in six.iteritems(failures): - copied_failures[key] = failure - results.append((data, copied_failures)) - clone.results = results - if self.meta: - clone.meta = self.meta.copy() - if self.version: - clone.version = copy.copy(self.version) - return clone - - @property - def last_results(self): - """The last result that was produced.""" - try: - return self.results[-1][0] - except IndexError: - exc.raise_with_cause(exc.NotFound, "Last results not found") - - @property - def last_failures(self): - """The last failure dictionary that was produced. - - NOTE(harlowja): This is **not** the same as the - local ``failure`` attribute as the obtained failure dictionary in - the ``results`` attribute (which is what this returns) is from - associated atom failures (which is different from the directly - related failure of the retry unit associated with this - atom detail). - """ - try: - return self.results[-1][1] - except IndexError: - exc.raise_with_cause(exc.NotFound, "Last failures not found") - - def put(self, state, result): - """Puts a result (acquired in the given state) into this detail. - - If the result is a :py:class:`~taskflow.types.failure.Failure` object - then the ``failure`` attribute will be set; if the result is not a - :py:class:`~taskflow.types.failure.Failure` object then the - ``results`` attribute will be appended to (and the ``failure`` - attribute will be set to ``None``). In either case the ``state`` - attribute will be set to the provided state. - """ - # Do not clean retry history (only on reset does this happen). - self.state = state - if self._was_failure(state, result): - self.failure = result - else: - self.results.append((result, {})) - self.failure = None - return True - - @classmethod - def from_dict(cls, data): - """Translates the given ``dict`` into an instance of this class.""" - - def decode_results(results): - if not results: - return [] - new_results = [] - for (data, failures) in results: - new_failures = {} - for (key, data) in six.iteritems(failures): - new_failures[key] = ft.Failure.from_dict(data) - new_results.append((data, new_failures)) - return new_results - - obj = super(RetryDetail, cls).from_dict(data) - obj.results = decode_results(obj.results) - return obj - - def to_dict(self): - """Translates the internal state of this object to a ``dict``.""" - - def encode_results(results): - if not results: - return [] - new_results = [] - for (data, failures) in results: - new_failures = {} - for (key, failure) in six.iteritems(failures): - new_failures[key] = failure.to_dict() - new_results.append((data, new_failures)) - return new_results - - base = super(RetryDetail, self).to_dict() - base['results'] = encode_results(base.get('results')) - return base - - def merge(self, other, deep_copy=False): - """Merges the current retry detail with the given one. - - NOTE(harlowja): This merge does **not** deep copy - the incoming objects ``results`` attribute (if it differs). Instead - the incoming objects ``results`` attribute list is **always** iterated - over and a new list is constructed with - each ``(data, failures)`` element in that list having - its ``failures`` (a dictionary of each named - :py:class:`~taskflow.types.failure.Failure` objects that - occurred) copied but its ``data`` is left untouched. After - this is done that new list becomes (via assignment) this - objects ``results`` attribute. Also note that if the provided object - is this object itself then **no** merging is done. - - See: https://bugs.launchpad.net/taskflow/+bug/1452978 for - what happens if the ``data`` in ``results`` is copied at a - deeper level (for example by using ``copy.deepcopy`` or by - using ``copy.copy``). - - :returns: this retry detail (freshly merged with the incoming object) - :rtype: :py:class:`.RetryDetail` - """ - if not isinstance(other, RetryDetail): - raise exc.NotImplementedError("Can only merge with other" - " retry details") - if other is self: - return self - super(RetryDetail, self).merge(other, deep_copy=deep_copy) - results = [] - # NOTE(imelnikov): we can't just deep copy Failures, as they - # contain tracebacks, which are not copyable. - for (data, failures) in other.results: - copied_failures = {} - for (key, failure) in six.iteritems(failures): - if deep_copy: - copied_failures[key] = failure.copy() - else: - copied_failures[key] = failure - results.append((data, copied_failures)) - self.results = results - return self - - -_DETAIL_TO_NAME = { - RetryDetail: 'RETRY_DETAIL', - TaskDetail: 'TASK_DETAIL', -} -_NAME_TO_DETAIL = dict((name, cls) - for (cls, name) in six.iteritems(_DETAIL_TO_NAME)) -ATOM_TYPES = list(six.iterkeys(_NAME_TO_DETAIL)) - - -def atom_detail_class(atom_type): - try: - return _NAME_TO_DETAIL[atom_type] - except KeyError: - raise TypeError("Unknown atom type '%s'" % (atom_type)) - - -def atom_detail_type(atom_detail): - try: - return _DETAIL_TO_NAME[type(atom_detail)] - except KeyError: - raise TypeError("Unknown atom '%s' (%s)" - % (atom_detail, type(atom_detail))) +# Keep alias classes/functions... around until this module is removed. +LogBook = models.LogBook +FlowDetail = models.FlowDetail +AtomDetail = models.AtomDetail +TaskDetail = models.TaskDetail +RetryDetail = models.RetryDetail +atom_detail_type = models.atom_detail_type +atom_detail_class = models.atom_detail_class +ATOM_TYPES = models.ATOM_TYPES diff --git a/taskflow/persistence/models.py b/taskflow/persistence/models.py new file mode 100644 index 00000000..c7a6eae5 --- /dev/null +++ b/taskflow/persistence/models.py @@ -0,0 +1,892 @@ +# -*- coding: utf-8 -*- + +# Copyright (C) 2012 Yahoo! Inc. All Rights Reserved. +# Copyright (C) 2013 Rackspace Hosting 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. + +import abc +import copy + +from oslo_utils import timeutils +from oslo_utils import uuidutils +import six + +from taskflow import exceptions as exc +from taskflow import logging +from taskflow import states +from taskflow.types import failure as ft + +LOG = logging.getLogger(__name__) + + +# Internal helpers... + + +def _copy_function(deep_copy): + if deep_copy: + return copy.deepcopy + else: + return lambda x: x + + +def _safe_marshal_time(when): + if not when: + return None + return timeutils.marshall_now(now=when) + + +def _safe_unmarshal_time(when): + if not when: + return None + return timeutils.unmarshall_time(when) + + +def _fix_meta(data): + # Handle the case where older schemas allowed this to be non-dict by + # correcting this case by replacing it with a dictionary when a non-dict + # is found. + meta = data.get('meta') + if not isinstance(meta, dict): + meta = {} + return meta + + +class LogBook(object): + """A collection of flow details and associated metadata. + + Typically this class contains a collection of flow detail entries + for a given engine (or job) so that those entities can track what 'work' + has been completed for resumption, reverting and miscellaneous tracking + purposes. + + The data contained within this class need **not** be persisted to the + backend storage in real time. The data in this class will only be + guaranteed to be persisted when a save occurs via some backend + connection. + + NOTE(harlowja): the naming of this class is analogous to a ship's log or a + similar type of record used in detailing work that has been completed (or + work that has not been completed). + + :ivar created_at: A ``datetime.datetime`` object of when this logbook + was created. + :ivar updated_at: A ``datetime.datetime`` object of when this logbook + was last updated at. + :ivar meta: A dictionary of meta-data associated with this logbook. + """ + def __init__(self, name, uuid=None): + if uuid: + self._uuid = uuid + else: + self._uuid = uuidutils.generate_uuid() + self._name = name + self._flowdetails_by_id = {} + self.created_at = timeutils.utcnow() + self.updated_at = None + self.meta = {} + + def add(self, fd): + """Adds a new flow detail into this logbook. + + NOTE(harlowja): if an existing flow detail exists with the same + uuid the existing one will be overwritten with the newly provided + one. + + Does not *guarantee* that the details will be immediately saved. + """ + self._flowdetails_by_id[fd.uuid] = fd + self.updated_at = timeutils.utcnow() + + def find(self, flow_uuid): + """Locate the flow detail corresponding to the given uuid. + + :returns: the flow detail with that uuid + :rtype: :py:class:`.FlowDetail` (or ``None`` if not found) + """ + return self._flowdetails_by_id.get(flow_uuid, None) + + def merge(self, lb, deep_copy=False): + """Merges the current object state with the given ones state. + + If ``deep_copy`` is provided as truthy then the + local object will use ``copy.deepcopy`` to replace this objects + local attributes with the provided objects attributes (**only** if + there is a difference between this objects attributes and the + provided attributes). If ``deep_copy`` is falsey (the default) then a + reference copy will occur instead when a difference is detected. + + NOTE(harlowja): If the provided object is this object itself + then **no** merging is done. Also note that this does **not** merge + the flow details contained in either. + + :returns: this logbook (freshly merged with the incoming object) + :rtype: :py:class:`.LogBook` + """ + if lb is self: + return self + copy_fn = _copy_function(deep_copy) + if self.meta != lb.meta: + self.meta = copy_fn(lb.meta) + if lb.created_at != self.created_at: + self.created_at = copy_fn(lb.created_at) + if lb.updated_at != self.updated_at: + self.updated_at = copy_fn(lb.updated_at) + return self + + def to_dict(self, marshal_time=False): + """Translates the internal state of this object to a ``dict``. + + NOTE(harlowja): The returned ``dict`` does **not** include any + contained flow details. + + :returns: this logbook in ``dict`` form + """ + if not marshal_time: + marshal_fn = lambda x: x + else: + marshal_fn = _safe_marshal_time + return { + 'name': self.name, + 'meta': self.meta, + 'uuid': self.uuid, + 'updated_at': marshal_fn(self.updated_at), + 'created_at': marshal_fn(self.created_at), + } + + @classmethod + def from_dict(cls, data, unmarshal_time=False): + """Translates the given ``dict`` into an instance of this class. + + NOTE(harlowja): the ``dict`` provided should come from a prior + call to :meth:`.to_dict`. + + :returns: a new logbook + :rtype: :py:class:`.LogBook` + """ + if not unmarshal_time: + unmarshal_fn = lambda x: x + else: + unmarshal_fn = _safe_unmarshal_time + obj = cls(data['name'], uuid=data['uuid']) + obj.updated_at = unmarshal_fn(data['updated_at']) + obj.created_at = unmarshal_fn(data['created_at']) + obj.meta = _fix_meta(data) + return obj + + @property + def uuid(self): + """The unique identifer of this logbook.""" + return self._uuid + + @property + def name(self): + """The name of this logbook.""" + return self._name + + def __iter__(self): + for fd in six.itervalues(self._flowdetails_by_id): + yield fd + + def __len__(self): + return len(self._flowdetails_by_id) + + def copy(self, retain_contents=True): + """Copies this logbook. + + Creates a shallow copy of this logbook. If this logbook contains + flow details and ``retain_contents`` is truthy (the default) then + the flow details container will be shallow copied (the flow details + contained there-in will **not** be copied). If ``retain_contents`` is + falsey then the copied logbook will have **no** contained flow + details (but it will have the rest of the local objects attributes + copied). + + :returns: a new logbook + :rtype: :py:class:`.LogBook` + """ + clone = copy.copy(self) + if not retain_contents: + clone._flowdetails_by_id = {} + else: + clone._flowdetails_by_id = self._flowdetails_by_id.copy() + if self.meta: + clone.meta = self.meta.copy() + return clone + + +class FlowDetail(object): + """A collection of atom details and associated metadata. + + Typically this class contains a collection of atom detail entries that + represent the atoms in a given flow structure (along with any other needed + metadata relevant to that flow). + + The data contained within this class need **not** be persisted to the + backend storage in real time. The data in this class will only be + guaranteed to be persisted when a save (or update) occurs via some backend + connection. + + :ivar state: The state of the flow associated with this flow detail. + :ivar meta: A dictionary of meta-data associated with this flow detail. + """ + def __init__(self, name, uuid): + self._uuid = uuid + self._name = name + self._atomdetails_by_id = {} + self.state = None + self.meta = {} + + def update(self, fd): + """Updates the objects state to be the same as the given one. + + This will assign the private and public attributes of the given + flow detail directly to this object (replacing any existing + attributes in this object; even if they are the **same**). + + NOTE(harlowja): If the provided object is this object itself + then **no** update is done. + + :returns: this flow detail + :rtype: :py:class:`.FlowDetail` + """ + if fd is self: + return self + self._atomdetails_by_id = fd._atomdetails_by_id + self.state = fd.state + self.meta = fd.meta + return self + + def merge(self, fd, deep_copy=False): + """Merges the current object state with the given one's state. + + If ``deep_copy`` is provided as truthy then the + local object will use ``copy.deepcopy`` to replace this objects + local attributes with the provided objects attributes (**only** if + there is a difference between this objects attributes and the + provided attributes). If ``deep_copy`` is falsey (the default) then a + reference copy will occur instead when a difference is detected. + + NOTE(harlowja): If the provided object is this object itself + then **no** merging is done. Also this does **not** merge the atom + details contained in either. + + :returns: this flow detail (freshly merged with the incoming object) + :rtype: :py:class:`.FlowDetail` + """ + if fd is self: + return self + copy_fn = _copy_function(deep_copy) + if self.meta != fd.meta: + self.meta = copy_fn(fd.meta) + if self.state != fd.state: + # NOTE(imelnikov): states are just strings, no need to copy. + self.state = fd.state + return self + + def copy(self, retain_contents=True): + """Copies this flow detail. + + Creates a shallow copy of this flow detail. If this detail contains + flow details and ``retain_contents`` is truthy (the default) then + the atom details container will be shallow copied (the atom details + contained there-in will **not** be copied). If ``retain_contents`` is + falsey then the copied flow detail will have **no** contained atom + details (but it will have the rest of the local objects attributes + copied). + + :returns: a new flow detail + :rtype: :py:class:`.FlowDetail` + """ + clone = copy.copy(self) + if not retain_contents: + clone._atomdetails_by_id = {} + else: + clone._atomdetails_by_id = self._atomdetails_by_id.copy() + if self.meta: + clone.meta = self.meta.copy() + return clone + + def to_dict(self): + """Translates the internal state of this object to a ``dict``. + + NOTE(harlowja): The returned ``dict`` does **not** include any + contained atom details. + + :returns: this flow detail in ``dict`` form + """ + return { + 'name': self.name, + 'meta': self.meta, + 'state': self.state, + 'uuid': self.uuid, + } + + @classmethod + def from_dict(cls, data): + """Translates the given ``dict`` into an instance of this class. + + NOTE(harlowja): the ``dict`` provided should come from a prior + call to :meth:`.to_dict`. + + :returns: a new flow detail + :rtype: :py:class:`.FlowDetail` + """ + obj = cls(data['name'], data['uuid']) + obj.state = data.get('state') + obj.meta = _fix_meta(data) + return obj + + def add(self, ad): + """Adds a new atom detail into this flow detail. + + NOTE(harlowja): if an existing atom detail exists with the same + uuid the existing one will be overwritten with the newly provided + one. + + Does not *guarantee* that the details will be immediately saved. + """ + self._atomdetails_by_id[ad.uuid] = ad + + def find(self, ad_uuid): + """Locate the atom detail corresponding to the given uuid. + + :returns: the atom detail with that uuid + :rtype: :py:class:`.AtomDetail` (or ``None`` if not found) + """ + return self._atomdetails_by_id.get(ad_uuid) + + @property + def uuid(self): + """The unique identifer of this flow detail.""" + return self._uuid + + @property + def name(self): + """The name of this flow detail.""" + return self._name + + def __iter__(self): + for ad in six.itervalues(self._atomdetails_by_id): + yield ad + + def __len__(self): + return len(self._atomdetails_by_id) + + +@six.add_metaclass(abc.ABCMeta) +class AtomDetail(object): + """A collection of atom specific runtime information and metadata. + + This is a base **abstract** class that contains attributes that are used + to connect a atom to the persistence layer before, during, or after it is + running. It includes any results it may have produced, any state that it + may be in (for example ``FAILURE``), any exception that occurred when + running, and any associated stacktrace that may have occurring during an + exception being thrown. It may also contain any other metadata that + should also be stored along-side the details about the connected atom. + + The data contained within this class need **not** be persisted to the + backend storage in real time. The data in this class will only be + guaranteed to be persisted when a save (or update) occurs via some backend + connection. + + :ivar state: The state of the atom associated with this atom detail. + :ivar intention: The execution strategy of the atom associated + with this atom detail (used by an engine/others to + determine if the associated atom needs to be + executed, reverted, retried and so-on). + :ivar meta: A dictionary of meta-data associated with this atom detail. + :ivar version: A version tuple or string that represents the + atom version this atom detail is associated with (typically + used for introspection and any data migration + strategies). + :ivar results: Any results the atom produced from either its + ``execute`` method or from other sources. + :ivar failure: If the atom failed (possibly due to its ``execute`` + method raising) this will be a + :py:class:`~taskflow.types.failure.Failure` object that + represents that failure (if there was no failure this + will be set to none). + """ + + def __init__(self, name, uuid): + self._uuid = uuid + self._name = name + self.state = None + self.intention = states.EXECUTE + self.results = None + self.failure = None + self.meta = {} + self.version = None + + @staticmethod + def _was_failure(state, result): + # Internal helper method... + return state == states.FAILURE and isinstance(result, ft.Failure) + + @property + def last_results(self): + """Gets the atoms last result. + + If the atom has produced many results (for example if it has been + retried, reverted, executed and ...) this returns the last one of + many results. + """ + return self.results + + def update(self, ad): + """Updates the object's state to be the same as the given one. + + This will assign the private and public attributes of the given + atom detail directly to this object (replacing any existing + attributes in this object; even if they are the **same**). + + NOTE(harlowja): If the provided object is this object itself + then **no** update is done. + + :returns: this atom detail + :rtype: :py:class:`.AtomDetail` + """ + if ad is self: + return self + self.state = ad.state + self.intention = ad.intention + self.meta = ad.meta + self.failure = ad.failure + self.results = ad.results + self.version = ad.version + return self + + @abc.abstractmethod + def merge(self, other, deep_copy=False): + """Merges the current object state with the given ones state. + + If ``deep_copy`` is provided as truthy then the + local object will use ``copy.deepcopy`` to replace this objects + local attributes with the provided objects attributes (**only** if + there is a difference between this objects attributes and the + provided attributes). If ``deep_copy`` is falsey (the default) then a + reference copy will occur instead when a difference is detected. + + NOTE(harlowja): If the provided object is this object itself + then **no** merging is done. Do note that **no** results are merged + in this method. That operation **must** to be the responsibilty of + subclasses to implement and override this abstract method + and provide that merging themselves as they see fit. + + :returns: this atom detail (freshly merged with the incoming object) + :rtype: :py:class:`.AtomDetail` + """ + copy_fn = _copy_function(deep_copy) + # NOTE(imelnikov): states and intentions are just strings, + # so there is no need to copy them (strings are immutable in python). + self.state = other.state + self.intention = other.intention + if self.failure != other.failure: + # NOTE(imelnikov): we can't just deep copy Failures, as they + # contain tracebacks, which are not copyable. + if other.failure: + if deep_copy: + self.failure = other.failure.copy() + else: + self.failure = other.failure + else: + self.failure = None + if self.meta != other.meta: + self.meta = copy_fn(other.meta) + if self.version != other.version: + self.version = copy_fn(other.version) + return self + + @abc.abstractmethod + def put(self, state, result): + """Puts a result (acquired in the given state) into this detail.""" + + def to_dict(self): + """Translates the internal state of this object to a ``dict``. + + :returns: this atom detail in ``dict`` form + """ + if self.failure: + failure = self.failure.to_dict() + else: + failure = None + return { + 'failure': failure, + 'meta': self.meta, + 'name': self.name, + 'results': self.results, + 'state': self.state, + 'version': self.version, + 'intention': self.intention, + 'uuid': self.uuid, + } + + @classmethod + def from_dict(cls, data): + """Translates the given ``dict`` into an instance of this class. + + NOTE(harlowja): the ``dict`` provided should come from a prior + call to :meth:`.to_dict`. + + :returns: a new atom detail + :rtype: :py:class:`.AtomDetail` + """ + obj = cls(data['name'], data['uuid']) + obj.state = data.get('state') + obj.intention = data.get('intention') + obj.results = data.get('results') + obj.version = data.get('version') + obj.meta = _fix_meta(data) + failure = data.get('failure') + if failure: + obj.failure = ft.Failure.from_dict(failure) + return obj + + @property + def uuid(self): + """The unique identifer of this atom detail.""" + return self._uuid + + @property + def name(self): + """The name of this atom detail.""" + return self._name + + @abc.abstractmethod + def reset(self, state): + """Resets this atom detail and sets ``state`` attribute value.""" + + @abc.abstractmethod + def copy(self): + """Copies this atom detail.""" + + +class TaskDetail(AtomDetail): + """A task detail (an atom detail typically associated with a |tt| atom). + + .. |tt| replace:: :py:class:`~taskflow.task.BaseTask` + """ + + def reset(self, state): + """Resets this task detail and sets ``state`` attribute value. + + This sets any previously set ``results`` and ``failure`` attributes + back to ``None`` and sets the state to the provided one, as well as + setting this task details ``intention`` attribute to ``EXECUTE``. + """ + self.results = None + self.failure = None + self.state = state + self.intention = states.EXECUTE + + def put(self, state, result): + """Puts a result (acquired in the given state) into this detail. + + If the result is a :py:class:`~taskflow.types.failure.Failure` object + then the ``failure`` attribute will be set (and the ``results`` + attribute will be set to ``None``); if the result is not a + :py:class:`~taskflow.types.failure.Failure` object then the + ``results`` attribute will be set (and the ``failure`` attribute + will be set to ``None``). In either case the ``state`` + attribute will be set to the provided state. + """ + was_altered = False + if self.state != state: + self.state = state + was_altered = True + if self._was_failure(state, result): + if self.failure != result: + self.failure = result + was_altered = True + if self.results is not None: + self.results = None + was_altered = True + else: + # We don't really have the ability to determine equality of + # task (user) results at the current time, without making + # potentially bad guesses, so assume the task detail always needs + # to be saved if they are not exactly equivalent... + if self.results is not result: + self.results = result + was_altered = True + if self.failure is not None: + self.failure = None + was_altered = True + return was_altered + + def merge(self, other, deep_copy=False): + """Merges the current task detail with the given one. + + NOTE(harlowja): This merge does **not** copy and replace + the ``results`` attribute if it differs. Instead the current + objects ``results`` attribute directly becomes (via assignment) the + other objects ``results`` attribute. Also note that if the provided + object is this object itself then **no** merging is done. + + See: https://bugs.launchpad.net/taskflow/+bug/1452978 for + what happens if this is copied at a deeper level (for example by + using ``copy.deepcopy`` or by using ``copy.copy``). + + :returns: this task detail (freshly merged with the incoming object) + :rtype: :py:class:`.TaskDetail` + """ + if not isinstance(other, TaskDetail): + raise exc.NotImplementedError("Can only merge with other" + " task details") + if other is self: + return self + super(TaskDetail, self).merge(other, deep_copy=deep_copy) + if self.results != other.results: + self.results = other.results + return self + + def copy(self): + """Copies this task detail. + + Creates a shallow copy of this task detail (any meta-data and + version information that this object maintains is shallow + copied via ``copy.copy``). + + NOTE(harlowja): This copy does **not** perform ``copy.copy`` on + the ``results`` attribute of this object (before assigning to the + copy). Instead the current objects ``results`` attribute directly + becomes (via assignment) the copied objects ``results`` attribute. + + See: https://bugs.launchpad.net/taskflow/+bug/1452978 for + what happens if this is copied at a deeper level (for example by + using ``copy.deepcopy`` or by using ``copy.copy``). + + :returns: a new task detail + :rtype: :py:class:`.TaskDetail` + """ + clone = copy.copy(self) + clone.results = self.results + if self.meta: + clone.meta = self.meta.copy() + if self.version: + clone.version = copy.copy(self.version) + return clone + + +class RetryDetail(AtomDetail): + """A retry detail (an atom detail typically associated with a |rt| atom). + + .. |rt| replace:: :py:class:`~taskflow.retry.Retry` + """ + + def __init__(self, name, uuid): + super(RetryDetail, self).__init__(name, uuid) + self.results = [] + + def reset(self, state): + """Resets this retry detail and sets ``state`` attribute value. + + This sets any previously added ``results`` back to an empty list + and resets the ``failure`` attribute back to ``None`` and sets the + state to the provided one, as well as setting this atom + details ``intention`` attribute to ``EXECUTE``. + """ + self.results = [] + self.failure = None + self.state = state + self.intention = states.EXECUTE + + def copy(self): + """Copies this retry detail. + + Creates a shallow copy of this retry detail (any meta-data and + version information that this object maintains is shallow + copied via ``copy.copy``). + + NOTE(harlowja): This copy does **not** copy + the incoming objects ``results`` attribute. Instead this + objects ``results`` attribute list is iterated over and a new list + is constructed with each ``(data, failures)`` element in that list + having its ``failures`` (a dictionary of each named + :py:class:`~taskflow.types.failure.Failure` object that + occured) copied but its ``data`` is left untouched. After + this is done that new list becomes (via assignment) the cloned + objects ``results`` attribute. + + See: https://bugs.launchpad.net/taskflow/+bug/1452978 for + what happens if the ``data`` in ``results`` is copied at a + deeper level (for example by using ``copy.deepcopy`` or by + using ``copy.copy``). + + :returns: a new retry detail + :rtype: :py:class:`.RetryDetail` + """ + clone = copy.copy(self) + results = [] + # NOTE(imelnikov): we can't just deep copy Failures, as they + # contain tracebacks, which are not copyable. + for (data, failures) in self.results: + copied_failures = {} + for (key, failure) in six.iteritems(failures): + copied_failures[key] = failure + results.append((data, copied_failures)) + clone.results = results + if self.meta: + clone.meta = self.meta.copy() + if self.version: + clone.version = copy.copy(self.version) + return clone + + @property + def last_results(self): + """The last result that was produced.""" + try: + return self.results[-1][0] + except IndexError: + exc.raise_with_cause(exc.NotFound, "Last results not found") + + @property + def last_failures(self): + """The last failure dictionary that was produced. + + NOTE(harlowja): This is **not** the same as the + local ``failure`` attribute as the obtained failure dictionary in + the ``results`` attribute (which is what this returns) is from + associated atom failures (which is different from the directly + related failure of the retry unit associated with this + atom detail). + """ + try: + return self.results[-1][1] + except IndexError: + exc.raise_with_cause(exc.NotFound, "Last failures not found") + + def put(self, state, result): + """Puts a result (acquired in the given state) into this detail. + + If the result is a :py:class:`~taskflow.types.failure.Failure` object + then the ``failure`` attribute will be set; if the result is not a + :py:class:`~taskflow.types.failure.Failure` object then the + ``results`` attribute will be appended to (and the ``failure`` + attribute will be set to ``None``). In either case the ``state`` + attribute will be set to the provided state. + """ + # Do not clean retry history (only on reset does this happen). + self.state = state + if self._was_failure(state, result): + self.failure = result + else: + self.results.append((result, {})) + self.failure = None + return True + + @classmethod + def from_dict(cls, data): + """Translates the given ``dict`` into an instance of this class.""" + + def decode_results(results): + if not results: + return [] + new_results = [] + for (data, failures) in results: + new_failures = {} + for (key, data) in six.iteritems(failures): + new_failures[key] = ft.Failure.from_dict(data) + new_results.append((data, new_failures)) + return new_results + + obj = super(RetryDetail, cls).from_dict(data) + obj.results = decode_results(obj.results) + return obj + + def to_dict(self): + """Translates the internal state of this object to a ``dict``.""" + + def encode_results(results): + if not results: + return [] + new_results = [] + for (data, failures) in results: + new_failures = {} + for (key, failure) in six.iteritems(failures): + new_failures[key] = failure.to_dict() + new_results.append((data, new_failures)) + return new_results + + base = super(RetryDetail, self).to_dict() + base['results'] = encode_results(base.get('results')) + return base + + def merge(self, other, deep_copy=False): + """Merges the current retry detail with the given one. + + NOTE(harlowja): This merge does **not** deep copy + the incoming objects ``results`` attribute (if it differs). Instead + the incoming objects ``results`` attribute list is **always** iterated + over and a new list is constructed with + each ``(data, failures)`` element in that list having + its ``failures`` (a dictionary of each named + :py:class:`~taskflow.types.failure.Failure` objects that + occurred) copied but its ``data`` is left untouched. After + this is done that new list becomes (via assignment) this + objects ``results`` attribute. Also note that if the provided object + is this object itself then **no** merging is done. + + See: https://bugs.launchpad.net/taskflow/+bug/1452978 for + what happens if the ``data`` in ``results`` is copied at a + deeper level (for example by using ``copy.deepcopy`` or by + using ``copy.copy``). + + :returns: this retry detail (freshly merged with the incoming object) + :rtype: :py:class:`.RetryDetail` + """ + if not isinstance(other, RetryDetail): + raise exc.NotImplementedError("Can only merge with other" + " retry details") + if other is self: + return self + super(RetryDetail, self).merge(other, deep_copy=deep_copy) + results = [] + # NOTE(imelnikov): we can't just deep copy Failures, as they + # contain tracebacks, which are not copyable. + for (data, failures) in other.results: + copied_failures = {} + for (key, failure) in six.iteritems(failures): + if deep_copy: + copied_failures[key] = failure.copy() + else: + copied_failures[key] = failure + results.append((data, copied_failures)) + self.results = results + return self + + +_DETAIL_TO_NAME = { + RetryDetail: 'RETRY_DETAIL', + TaskDetail: 'TASK_DETAIL', +} +_NAME_TO_DETAIL = dict((name, cls) + for (cls, name) in six.iteritems(_DETAIL_TO_NAME)) +ATOM_TYPES = list(six.iterkeys(_NAME_TO_DETAIL)) + + +def atom_detail_class(atom_type): + try: + return _NAME_TO_DETAIL[atom_type] + except KeyError: + raise TypeError("Unknown atom type '%s'" % (atom_type)) + + +def atom_detail_type(atom_detail): + try: + return _DETAIL_TO_NAME[type(atom_detail)] + except KeyError: + raise TypeError("Unknown atom '%s' (%s)" + % (atom_detail, type(atom_detail))) diff --git a/taskflow/persistence/path_based.py b/taskflow/persistence/path_based.py index 0d729978..f2d411b2 100644 --- a/taskflow/persistence/path_based.py +++ b/taskflow/persistence/path_based.py @@ -19,7 +19,7 @@ import six from taskflow import exceptions as exc from taskflow.persistence import base -from taskflow.persistence import logbook +from taskflow.persistence import models @six.add_metaclass(abc.ABCMeta) @@ -60,23 +60,23 @@ class PathBasedConnection(base.Connection): @staticmethod def _serialize(obj): - if isinstance(obj, logbook.LogBook): + if isinstance(obj, models.LogBook): return obj.to_dict(marshal_time=True) - elif isinstance(obj, logbook.FlowDetail): + elif isinstance(obj, models.FlowDetail): return obj.to_dict() - elif isinstance(obj, logbook.AtomDetail): + elif isinstance(obj, models.AtomDetail): return base._format_atom(obj) else: raise exc.StorageFailure("Invalid storage class %s" % type(obj)) @staticmethod def _deserialize(cls, data): - if issubclass(cls, logbook.LogBook): + if issubclass(cls, models.LogBook): return cls.from_dict(data, unmarshal_time=True) - elif issubclass(cls, logbook.FlowDetail): + elif issubclass(cls, models.FlowDetail): return cls.from_dict(data) - elif issubclass(cls, logbook.AtomDetail): - atom_class = logbook.atom_detail_class(data['type']) + elif issubclass(cls, models.AtomDetail): + atom_class = models.atom_detail_class(data['type']) return atom_class.from_dict(data['atom']) else: raise exc.StorageFailure("Invalid storage class %s" % cls) @@ -130,11 +130,11 @@ class PathBasedConnection(base.Connection): """Context manager that yields a transaction""" def _get_obj_path(self, obj): - if isinstance(obj, logbook.LogBook): + if isinstance(obj, models.LogBook): path = self.book_path - elif isinstance(obj, logbook.FlowDetail): + elif isinstance(obj, models.FlowDetail): path = self.flow_path - elif isinstance(obj, logbook.AtomDetail): + elif isinstance(obj, models.AtomDetail): path = self.atom_path else: raise exc.StorageFailure("Invalid storage class %s" % type(obj)) @@ -159,7 +159,7 @@ class PathBasedConnection(base.Connection): def get_logbook(self, book_uuid, lazy=False): book_path = self._join_path(self.book_path, book_uuid) book_data = self._get_item(book_path) - book = self._deserialize(logbook.LogBook, book_data) + book = self._deserialize(models.LogBook, book_data) if not lazy: for flow_details in self.get_flows_for_book(book_uuid): book.add(flow_details) @@ -185,7 +185,7 @@ class PathBasedConnection(base.Connection): def get_flow_details(self, flow_uuid, lazy=False): flow_path = self._join_path(self.flow_path, flow_uuid) flow_data = self._get_item(flow_path) - flow_details = self._deserialize(logbook.FlowDetail, flow_data) + flow_details = self._deserialize(models.FlowDetail, flow_data) if not lazy: for atom_details in self.get_atoms_for_flow(flow_uuid): flow_details.add(atom_details) @@ -216,7 +216,7 @@ class PathBasedConnection(base.Connection): def get_atom_details(self, atom_uuid): atom_path = self._join_path(self.atom_path, atom_uuid) atom_data = self._get_item(atom_path) - return self._deserialize(logbook.AtomDetail, atom_data) + return self._deserialize(models.AtomDetail, atom_data) def update_atom_details(self, atom_detail, ignore_missing=False): with self._transaction() as transaction: diff --git a/taskflow/storage.py b/taskflow/storage.py index cb1fbaad..05b48999 100644 --- a/taskflow/storage.py +++ b/taskflow/storage.py @@ -24,7 +24,7 @@ import six from taskflow import exceptions from taskflow import logging from taskflow.persistence.backends import impl_memory -from taskflow.persistence import logbook +from taskflow.persistence import models from taskflow import retry from taskflow import states from taskflow import task @@ -153,8 +153,8 @@ class Storage(object): self._injected_args = {} self._lock = fasteners.ReaderWriterLock() self._ensure_matchers = [ - ((task.BaseTask,), (logbook.TaskDetail, 'Task')), - ((retry.Retry,), (logbook.RetryDetail, 'Retry')), + ((task.BaseTask,), (models.TaskDetail, 'Task')), + ((retry.Retry,), (models.RetryDetail, 'Retry')), ] if scope_fetcher is None: scope_fetcher = lambda atom_name: None @@ -171,7 +171,7 @@ class Storage(object): for ad in self._flowdetail) try: source, _clone = self._atomdetail_by_name( - self.injector_name, expected_type=logbook.TaskDetail) + self.injector_name, expected_type=models.TaskDetail) except exceptions.NotFound: pass else: @@ -399,7 +399,7 @@ class Storage(object): else: update_with[META_PROGRESS_DETAILS] = None self._update_atom_metadata(task_name, update_with, - expected_type=logbook.TaskDetail) + expected_type=models.TaskDetail) @fasteners.read_locked def get_task_progress(self, task_name): @@ -409,7 +409,7 @@ class Storage(object): :returns: current task progress value """ source, _clone = self._atomdetail_by_name( - task_name, expected_type=logbook.TaskDetail) + task_name, expected_type=models.TaskDetail) try: return source.meta[META_PROGRESS] except KeyError: @@ -424,7 +424,7 @@ class Storage(object): dict """ source, _clone = self._atomdetail_by_name( - task_name, expected_type=logbook.TaskDetail) + task_name, expected_type=models.TaskDetail) try: return source.meta[META_PROGRESS_DETAILS] except KeyError: @@ -468,7 +468,7 @@ class Storage(object): def save_retry_failure(self, retry_name, failed_atom_name, failure): """Save subflow failure to retry controller history.""" source, clone = self._atomdetail_by_name( - retry_name, expected_type=logbook.RetryDetail, clone=True) + retry_name, expected_type=models.RetryDetail, clone=True) try: failures = clone.last_failures except exceptions.NotFound: @@ -485,7 +485,7 @@ class Storage(object): def cleanup_retry_history(self, retry_name, state): """Cleanup history of retry atom with given name.""" source, clone = self._atomdetail_by_name( - retry_name, expected_type=logbook.RetryDetail, clone=True) + retry_name, expected_type=models.RetryDetail, clone=True) clone.state = state clone.results = [] self._with_connection(self._save_atom_detail, source, clone) @@ -625,7 +625,7 @@ class Storage(object): try: source, clone = self._atomdetail_by_name( self.injector_name, - expected_type=logbook.TaskDetail, + expected_type=models.TaskDetail, clone=True) except exceptions.NotFound: # Ensure we have our special task detail... @@ -633,7 +633,7 @@ class Storage(object): # TODO(harlowja): get this removed when # https://review.openstack.org/#/c/165645/ merges. source = self._create_atom_detail(self.injector_name, - logbook.TaskDetail, + models.TaskDetail, atom_state=None) fd_source, fd_clone = self._fetch_flowdetail(clone=True) fd_clone.add(source) @@ -974,7 +974,7 @@ class Storage(object): def get_retry_history(self, retry_name): """Fetch a single retrys history.""" source, _clone = self._atomdetail_by_name( - retry_name, expected_type=logbook.RetryDetail) + retry_name, expected_type=models.RetryDetail) return self._translate_into_history(source) @fasteners.read_locked @@ -982,7 +982,7 @@ class Storage(object): """Fetch all retrys histories.""" histories = [] for ad in self._flowdetail: - if isinstance(ad, logbook.RetryDetail): + if isinstance(ad, models.RetryDetail): histories.append((ad.name, self._translate_into_history(ad))) return histories diff --git a/taskflow/tests/unit/persistence/base.py b/taskflow/tests/unit/persistence/base.py index 924e62b2..0b56617f 100644 --- a/taskflow/tests/unit/persistence/base.py +++ b/taskflow/tests/unit/persistence/base.py @@ -19,7 +19,7 @@ import contextlib from oslo_utils import uuidutils from taskflow import exceptions as exc -from taskflow.persistence import logbook +from taskflow.persistence import models from taskflow import states from taskflow.types import failure @@ -31,15 +31,15 @@ class PersistenceTestMixin(object): def test_task_detail_update_not_existing(self): lb_id = uuidutils.generate_uuid() lb_name = 'lb-%s' % (lb_id) - lb = logbook.LogBook(name=lb_name, uuid=lb_id) - fd = logbook.FlowDetail('test', uuid=uuidutils.generate_uuid()) + lb = models.LogBook(name=lb_name, uuid=lb_id) + fd = models.FlowDetail('test', uuid=uuidutils.generate_uuid()) lb.add(fd) - td = logbook.TaskDetail("detail-1", uuid=uuidutils.generate_uuid()) + td = models.TaskDetail("detail-1", uuid=uuidutils.generate_uuid()) fd.add(td) with contextlib.closing(self._get_connection()) as conn: conn.save_logbook(lb) - td2 = logbook.TaskDetail("detail-1", uuid=uuidutils.generate_uuid()) + td2 = models.TaskDetail("detail-1", uuid=uuidutils.generate_uuid()) fd.add(td2) with contextlib.closing(self._get_connection()) as conn: conn.update_flow_details(fd) @@ -53,13 +53,13 @@ class PersistenceTestMixin(object): def test_flow_detail_update_not_existing(self): lb_id = uuidutils.generate_uuid() lb_name = 'lb-%s' % (lb_id) - lb = logbook.LogBook(name=lb_name, uuid=lb_id) - fd = logbook.FlowDetail('test', uuid=uuidutils.generate_uuid()) + lb = models.LogBook(name=lb_name, uuid=lb_id) + fd = models.FlowDetail('test', uuid=uuidutils.generate_uuid()) lb.add(fd) with contextlib.closing(self._get_connection()) as conn: conn.save_logbook(lb) - fd2 = logbook.FlowDetail('test-2', uuid=uuidutils.generate_uuid()) + fd2 = models.FlowDetail('test-2', uuid=uuidutils.generate_uuid()) lb.add(fd2) with contextlib.closing(self._get_connection()) as conn: conn.save_logbook(lb) @@ -73,7 +73,7 @@ class PersistenceTestMixin(object): lb_id = uuidutils.generate_uuid() lb_meta = {'1': 2} lb_name = 'lb-%s' % (lb_id) - lb = logbook.LogBook(name=lb_name, uuid=lb_id) + lb = models.LogBook(name=lb_name, uuid=lb_id) lb.meta = lb_meta # Should not already exist @@ -94,8 +94,8 @@ class PersistenceTestMixin(object): def test_flow_detail_save(self): lb_id = uuidutils.generate_uuid() lb_name = 'lb-%s' % (lb_id) - lb = logbook.LogBook(name=lb_name, uuid=lb_id) - fd = logbook.FlowDetail('test', uuid=uuidutils.generate_uuid()) + lb = models.LogBook(name=lb_name, uuid=lb_id) + fd = models.FlowDetail('test', uuid=uuidutils.generate_uuid()) lb.add(fd) # Ensure we can't save it since its owning logbook hasn't been @@ -113,8 +113,8 @@ class PersistenceTestMixin(object): def test_flow_detail_meta_update(self): lb_id = uuidutils.generate_uuid() lb_name = 'lb-%s' % (lb_id) - lb = logbook.LogBook(name=lb_name, uuid=lb_id) - fd = logbook.FlowDetail('test', uuid=uuidutils.generate_uuid()) + lb = models.LogBook(name=lb_name, uuid=lb_id) + fd = models.FlowDetail('test', uuid=uuidutils.generate_uuid()) fd.meta = {'test': 42} lb.add(fd) @@ -133,9 +133,9 @@ class PersistenceTestMixin(object): def test_flow_detail_lazy_fetch(self): lb_id = uuidutils.generate_uuid() lb_name = 'lb-%s' % (lb_id) - lb = logbook.LogBook(name=lb_name, uuid=lb_id) - fd = logbook.FlowDetail('test', uuid=uuidutils.generate_uuid()) - td = logbook.TaskDetail("detail-1", uuid=uuidutils.generate_uuid()) + lb = models.LogBook(name=lb_name, uuid=lb_id) + fd = models.FlowDetail('test', uuid=uuidutils.generate_uuid()) + td = models.TaskDetail("detail-1", uuid=uuidutils.generate_uuid()) td.version = '4.2' fd.add(td) lb.add(fd) @@ -149,10 +149,10 @@ class PersistenceTestMixin(object): def test_task_detail_save(self): lb_id = uuidutils.generate_uuid() lb_name = 'lb-%s' % (lb_id) - lb = logbook.LogBook(name=lb_name, uuid=lb_id) - fd = logbook.FlowDetail('test', uuid=uuidutils.generate_uuid()) + lb = models.LogBook(name=lb_name, uuid=lb_id) + fd = models.FlowDetail('test', uuid=uuidutils.generate_uuid()) lb.add(fd) - td = logbook.TaskDetail("detail-1", uuid=uuidutils.generate_uuid()) + td = models.TaskDetail("detail-1", uuid=uuidutils.generate_uuid()) fd.add(td) # Ensure we can't save it since its owning logbook hasn't been @@ -171,10 +171,10 @@ class PersistenceTestMixin(object): def test_task_detail_meta_update(self): lb_id = uuidutils.generate_uuid() lb_name = 'lb-%s' % (lb_id) - lb = logbook.LogBook(name=lb_name, uuid=lb_id) - fd = logbook.FlowDetail('test', uuid=uuidutils.generate_uuid()) + lb = models.LogBook(name=lb_name, uuid=lb_id) + fd = models.FlowDetail('test', uuid=uuidutils.generate_uuid()) lb.add(fd) - td = logbook.TaskDetail("detail-1", uuid=uuidutils.generate_uuid()) + td = models.TaskDetail("detail-1", uuid=uuidutils.generate_uuid()) td.meta = {'test': 42} fd.add(td) @@ -192,15 +192,15 @@ class PersistenceTestMixin(object): fd2 = lb2.find(fd.uuid) td2 = fd2.find(td.uuid) self.assertEqual(td2.meta.get('test'), 43) - self.assertIsInstance(td2, logbook.TaskDetail) + self.assertIsInstance(td2, models.TaskDetail) def test_task_detail_with_failure(self): lb_id = uuidutils.generate_uuid() lb_name = 'lb-%s' % (lb_id) - lb = logbook.LogBook(name=lb_name, uuid=lb_id) - fd = logbook.FlowDetail('test', uuid=uuidutils.generate_uuid()) + lb = models.LogBook(name=lb_name, uuid=lb_id) + fd = models.FlowDetail('test', uuid=uuidutils.generate_uuid()) lb.add(fd) - td = logbook.TaskDetail("detail-1", uuid=uuidutils.generate_uuid()) + td = models.TaskDetail("detail-1", uuid=uuidutils.generate_uuid()) try: raise RuntimeError('Woot!') @@ -222,18 +222,18 @@ class PersistenceTestMixin(object): self.assertEqual(td2.failure.exception_str, 'Woot!') self.assertIs(td2.failure.check(RuntimeError), RuntimeError) self.assertEqual(td2.failure.traceback_str, td.failure.traceback_str) - self.assertIsInstance(td2, logbook.TaskDetail) + self.assertIsInstance(td2, models.TaskDetail) def test_logbook_merge_flow_detail(self): lb_id = uuidutils.generate_uuid() lb_name = 'lb-%s' % (lb_id) - lb = logbook.LogBook(name=lb_name, uuid=lb_id) - fd = logbook.FlowDetail('test', uuid=uuidutils.generate_uuid()) + lb = models.LogBook(name=lb_name, uuid=lb_id) + fd = models.FlowDetail('test', uuid=uuidutils.generate_uuid()) lb.add(fd) with contextlib.closing(self._get_connection()) as conn: conn.save_logbook(lb) - lb2 = logbook.LogBook(name=lb_name, uuid=lb_id) - fd2 = logbook.FlowDetail('test2', uuid=uuidutils.generate_uuid()) + lb2 = models.LogBook(name=lb_name, uuid=lb_id) + fd2 = models.FlowDetail('test2', uuid=uuidutils.generate_uuid()) lb2.add(fd2) with contextlib.closing(self._get_connection()) as conn: conn.save_logbook(lb2) @@ -244,8 +244,8 @@ class PersistenceTestMixin(object): def test_logbook_add_flow_detail(self): lb_id = uuidutils.generate_uuid() lb_name = 'lb-%s' % (lb_id) - lb = logbook.LogBook(name=lb_name, uuid=lb_id) - fd = logbook.FlowDetail('test', uuid=uuidutils.generate_uuid()) + lb = models.LogBook(name=lb_name, uuid=lb_id) + fd = models.FlowDetail('test', uuid=uuidutils.generate_uuid()) lb.add(fd) with contextlib.closing(self._get_connection()) as conn: conn.save_logbook(lb) @@ -258,8 +258,8 @@ class PersistenceTestMixin(object): def test_logbook_lazy_fetch(self): lb_id = uuidutils.generate_uuid() lb_name = 'lb-%s' % (lb_id) - lb = logbook.LogBook(name=lb_name, uuid=lb_id) - fd = logbook.FlowDetail('test', uuid=uuidutils.generate_uuid()) + lb = models.LogBook(name=lb_name, uuid=lb_id) + fd = models.FlowDetail('test', uuid=uuidutils.generate_uuid()) lb.add(fd) with contextlib.closing(self._get_connection()) as conn: conn.save_logbook(lb) @@ -271,9 +271,9 @@ class PersistenceTestMixin(object): def test_logbook_add_task_detail(self): lb_id = uuidutils.generate_uuid() lb_name = 'lb-%s' % (lb_id) - lb = logbook.LogBook(name=lb_name, uuid=lb_id) - fd = logbook.FlowDetail('test', uuid=uuidutils.generate_uuid()) - td = logbook.TaskDetail("detail-1", uuid=uuidutils.generate_uuid()) + lb = models.LogBook(name=lb_name, uuid=lb_id) + fd = models.FlowDetail('test', uuid=uuidutils.generate_uuid()) + td = models.TaskDetail("detail-1", uuid=uuidutils.generate_uuid()) td.version = '4.2' fd.add(td) lb.add(fd) @@ -298,7 +298,7 @@ class PersistenceTestMixin(object): def test_logbook_delete(self): lb_id = uuidutils.generate_uuid() lb_name = 'lb-%s' % (lb_id) - lb = logbook.LogBook(name=lb_name, uuid=lb_id) + lb = models.LogBook(name=lb_name, uuid=lb_id) with contextlib.closing(self._get_connection()) as conn: self.assertRaises(exc.NotFound, conn.destroy_logbook, lb_id) with contextlib.closing(self._get_connection()) as conn: @@ -313,10 +313,10 @@ class PersistenceTestMixin(object): def test_task_detail_retry_type_(self): lb_id = uuidutils.generate_uuid() lb_name = 'lb-%s' % (lb_id) - lb = logbook.LogBook(name=lb_name, uuid=lb_id) - fd = logbook.FlowDetail('test', uuid=uuidutils.generate_uuid()) + lb = models.LogBook(name=lb_name, uuid=lb_id) + fd = models.FlowDetail('test', uuid=uuidutils.generate_uuid()) lb.add(fd) - rd = logbook.RetryDetail("detail-1", uuid=uuidutils.generate_uuid()) + rd = models.RetryDetail("detail-1", uuid=uuidutils.generate_uuid()) rd.intention = states.REVERT fd.add(rd) @@ -330,15 +330,15 @@ class PersistenceTestMixin(object): fd2 = lb2.find(fd.uuid) rd2 = fd2.find(rd.uuid) self.assertEqual(rd2.intention, states.REVERT) - self.assertIsInstance(rd2, logbook.RetryDetail) + self.assertIsInstance(rd2, models.RetryDetail) def test_retry_detail_save_with_task_failure(self): lb_id = uuidutils.generate_uuid() lb_name = 'lb-%s' % (lb_id) - lb = logbook.LogBook(name=lb_name, uuid=lb_id) - fd = logbook.FlowDetail('test', uuid=uuidutils.generate_uuid()) + lb = models.LogBook(name=lb_name, uuid=lb_id) + fd = models.FlowDetail('test', uuid=uuidutils.generate_uuid()) lb.add(fd) - rd = logbook.RetryDetail("retry-1", uuid=uuidutils.generate_uuid()) + rd = models.RetryDetail("retry-1", uuid=uuidutils.generate_uuid()) fail = failure.Failure.from_exception(RuntimeError('fail')) rd.results.append((42, {'some-task': fail})) fd.add(rd) @@ -354,7 +354,7 @@ class PersistenceTestMixin(object): lb2 = conn.get_logbook(lb_id) fd2 = lb2.find(fd.uuid) rd2 = fd2.find(rd.uuid) - self.assertIsInstance(rd2, logbook.RetryDetail) + self.assertIsInstance(rd2, models.RetryDetail) fail2 = rd2.results[0][1].get('some-task') self.assertIsInstance(fail2, failure.Failure) self.assertTrue(fail.matches(fail2)) @@ -362,10 +362,10 @@ class PersistenceTestMixin(object): def test_retry_detail_save_intention(self): lb_id = uuidutils.generate_uuid() lb_name = 'lb-%s' % (lb_id) - lb = logbook.LogBook(name=lb_name, uuid=lb_id) - fd = logbook.FlowDetail('test', uuid=uuidutils.generate_uuid()) + lb = models.LogBook(name=lb_name, uuid=lb_id) + fd = models.FlowDetail('test', uuid=uuidutils.generate_uuid()) lb.add(fd) - rd = logbook.RetryDetail("retry-1", uuid=uuidutils.generate_uuid()) + rd = models.RetryDetail("retry-1", uuid=uuidutils.generate_uuid()) fd.add(rd) # save it @@ -385,4 +385,4 @@ class PersistenceTestMixin(object): fd2 = lb2.find(fd.uuid) rd2 = fd2.find(rd.uuid) self.assertEqual(rd2.intention, states.REVERT) - self.assertIsInstance(rd2, logbook.RetryDetail) + self.assertIsInstance(rd2, models.RetryDetail) diff --git a/taskflow/tests/unit/test_engines.py b/taskflow/tests/unit/test_engines.py index 4e38dfa5..c49804dd 100644 --- a/taskflow/tests/unit/test_engines.py +++ b/taskflow/tests/unit/test_engines.py @@ -26,7 +26,7 @@ from taskflow import exceptions as exc from taskflow.patterns import graph_flow as gf from taskflow.patterns import linear_flow as lf from taskflow.patterns import unordered_flow as uf -from taskflow.persistence import logbook +from taskflow.persistence import models from taskflow import states from taskflow import task from taskflow import test @@ -493,7 +493,7 @@ class EngineParallelFlowTest(utils.EngineTestBase): # Create FlowDetail as if we already run task1 lb, fd = p_utils.temporary_flow_detail(self.backend) - td = logbook.TaskDetail(name='task1', uuid='42') + td = models.TaskDetail(name='task1', uuid='42') td.state = states.SUCCESS td.results = 17 fd.add(td) diff --git a/taskflow/tests/unit/test_storage.py b/taskflow/tests/unit/test_storage.py index 8793f0ff..958d5a53 100644 --- a/taskflow/tests/unit/test_storage.py +++ b/taskflow/tests/unit/test_storage.py @@ -21,7 +21,7 @@ from oslo_utils import uuidutils from taskflow import exceptions from taskflow.persistence import backends -from taskflow.persistence import logbook +from taskflow.persistence import models from taskflow import states from taskflow import storage from taskflow import test @@ -61,7 +61,7 @@ class StorageTestMixin(object): self.assertTrue(uuidutils.is_uuid_like(s.get_atom_uuid('my_task'))) def test_flow_name_and_uuid(self): - flow_detail = logbook.FlowDetail(name='test-fd', uuid='aaaa') + flow_detail = models.FlowDetail(name='test-fd', uuid='aaaa') s = self._get_storage(flow_detail) self.assertEqual(s.flow_name, 'test-fd') self.assertEqual(s.flow_uuid, 'aaaa') @@ -97,14 +97,14 @@ class StorageTestMixin(object): def test_get_without_save(self): _lb, flow_detail = p_utils.temporary_flow_detail(self.backend) - td = logbook.TaskDetail(name='my_task', uuid='42') + td = models.TaskDetail(name='my_task', uuid='42') flow_detail.add(td) s = self._get_storage(flow_detail) self.assertEqual('42', s.get_atom_uuid('my_task')) def test_ensure_existing_task(self): _lb, flow_detail = p_utils.temporary_flow_detail(self.backend) - td = logbook.TaskDetail(name='my_task', uuid='42') + td = models.TaskDetail(name='my_task', uuid='42') flow_detail.add(td) s = self._get_storage(flow_detail) s.ensure_atom(test_utils.NoopTask('my_task')) @@ -523,7 +523,7 @@ class StorageTestMixin(object): def test_logbook_get_unknown_atom_type(self): self.assertRaisesRegexp(TypeError, 'Unknown atom', - logbook.atom_detail_class, 'some_detail') + models.atom_detail_class, 'some_detail') def test_save_task_intention(self): s = self._get_storage() diff --git a/taskflow/utils/persistence_utils.py b/taskflow/utils/persistence_utils.py index dd304bc6..0837afb9 100644 --- a/taskflow/utils/persistence_utils.py +++ b/taskflow/utils/persistence_utils.py @@ -21,7 +21,7 @@ from oslo_utils import timeutils from oslo_utils import uuidutils from taskflow import logging -from taskflow.persistence import logbook +from taskflow.persistence import models from taskflow.utils import misc LOG = logging.getLogger(__name__) @@ -33,7 +33,7 @@ def temporary_log_book(backend=None): Mainly useful for tests and other use cases where a temporary logbook is needed for a short-period of time. """ - book = logbook.LogBook('tmp') + book = models.LogBook('tmp') if backend is not None: with contextlib.closing(backend.get_connection()) as conn: conn.save_logbook(book) @@ -48,7 +48,7 @@ def temporary_flow_detail(backend=None): """ flow_id = uuidutils.generate_uuid() book = temporary_log_book(backend) - book.add(logbook.FlowDetail(name='tmp-flow-detail', uuid=flow_id)) + book.add(models.FlowDetail(name='tmp-flow-detail', uuid=flow_id)) if backend is not None: with contextlib.closing(backend.get_connection()) as conn: conn.save_logbook(book) @@ -77,7 +77,7 @@ def create_flow_detail(flow, book=None, backend=None, meta=None): LOG.warn("No name provided for flow %s (id %s)", flow, flow_id) flow_name = flow_id - flow_detail = logbook.FlowDetail(name=flow_name, uuid=flow_id) + flow_detail = models.FlowDetail(name=flow_name, uuid=flow_id) if meta is not None: if flow_detail.meta is None: flow_detail.meta = {} @@ -130,7 +130,7 @@ def _format_shared(obj, indent): def pformat_atom_detail(atom_detail, indent=0): """Pretty formats a atom detail.""" - detail_type = logbook.atom_detail_type(atom_detail) + detail_type = models.atom_detail_type(atom_detail) lines = ["%s%s: '%s'" % (" " * (indent), detail_type, atom_detail.name)] lines.extend(_format_shared(atom_detail, indent=indent + 1)) lines.append("%s- version = %s" From c5c2d8482731747ac300ff58cf18849e742331dc Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 8 Jul 2015 17:33:36 -0700 Subject: [PATCH 237/246] Show job posted and goodbye in 99_bottles.py example This makes it more obvious exactly what this job poster has done and why it exited (all it is supposed to do is post some work and die). Change-Id: I7e83f90e0af0edf8836d6950b45905982e32a57f --- taskflow/examples/99_bottles.py | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/taskflow/examples/99_bottles.py b/taskflow/examples/99_bottles.py index cbcf54ce..e134b303 100644 --- a/taskflow/examples/99_bottles.py +++ b/taskflow/examples/99_bottles.py @@ -83,6 +83,10 @@ class Conclusion(task.Task): def make_bottles(count): + # This is the function that will be called to generate the workflow + # and will also be called to regenerate it on resumption so that work + # can continue from where it last left off... + s = lf.Flow("bottle-song") take_bottle = TakeABottleDown("take-bottle-%s" % count, @@ -103,6 +107,9 @@ def make_bottles(count): def run_conductor(): + # This continuously consumers until its stopped via ctrl-c or other + # kill signal... + event_watches = {} # This will be triggered by the conductor doing various activities @@ -149,6 +156,7 @@ def run_conductor(): def run_poster(): + # This just posts a single job and then ends... print("Starting poster with pid: %s" % ME) my_name = "poster-%s" % ME persist_backend = persistence_backends.fetch(PERSISTENCE_URI) @@ -173,7 +181,9 @@ def run_poster(): [HOW_MANY_BOTTLES], {}, backend=persist_backend) # Post, and be done with it! - job_backend.post("song-from-%s" % my_name, book=lb) + jb = job_backend.post("song-from-%s" % my_name, book=lb) + print("Posted: %s" % jb) + print("Goodbye...") def main(): From fcd005f61cab961eb8f89a41756e3bb3e4ebcc0f Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 8 Jul 2015 17:55:23 -0700 Subject: [PATCH 238/246] Add more useful `__str__` to redis job The information about the sequence and the owner_key is useful for debugging and for interacting with redis if the owner key ever needs to be manually removed. Change-Id: Ibe751d15938a2bcb83fb057f4c139f8f0022a60b --- taskflow/jobs/backends/impl_redis.py | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/taskflow/jobs/backends/impl_redis.py b/taskflow/jobs/backends/impl_redis.py index 4d61dc01..1b06b691 100644 --- a/taskflow/jobs/backends/impl_redis.py +++ b/taskflow/jobs/backends/impl_redis.py @@ -187,6 +187,13 @@ class RedisJob(base.Job): listings_key, owner_key, value_from_callable=True) + def __str__(self): + """Pretty formats the job into something *more* meaningful.""" + tpl = "%s: %s (uuid=%s, owner_key=%s, sequence=%s, details=%s)" + return tpl % (type(self).__name__, + self.name, self.uuid, self.owner_key, + self.sequence, self.details) + class RedisJobBoard(base.JobBoard): """A jobboard backed by `redis`_. From 2eb1af31e5a82bfe2dc7bffe3a5a1345f9c438ac Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Wed, 8 Jul 2015 19:13:01 -0700 Subject: [PATCH 239/246] Allow the 99_bottles.py demo to run in BLATHER mode Change-Id: I0889fe109d95ee705ca3b224db135a490b846d53 --- taskflow/examples/99_bottles.py | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/taskflow/examples/99_bottles.py b/taskflow/examples/99_bottles.py index cbcf54ce..1d03490e 100644 --- a/taskflow/examples/99_bottles.py +++ b/taskflow/examples/99_bottles.py @@ -20,8 +20,6 @@ import os import sys import time -logging.basicConfig(level=logging.ERROR) - top_dir = os.path.abspath(os.path.join(os.path.dirname(__file__), os.pardir, os.pardir)) @@ -179,12 +177,17 @@ def run_poster(): def main(): if len(sys.argv) == 1: sys.stderr.write("%s p|c\n" % os.path.basename(sys.argv[0])) - elif sys.argv[1] == 'p': - run_poster() - elif sys.argv[1] == 'c': - run_conductor() + elif sys.argv[1] in ('p', 'c'): + if sys.argv[-1] == "v": + logging.basicConfig(level=5) + else: + logging.basicConfig(level=logging.ERROR) + if sys.argv[1] == 'p': + run_poster() + else: + run_conductor() else: - sys.stderr.write("%s p|c\n" % os.path.basename(sys.argv[0])) + sys.stderr.write("%s p|c (v?)\n" % os.path.basename(sys.argv[0])) if __name__ == '__main__': From 27272a2aa70b56182b67a06d3707430e86a0ccde Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Fri, 26 Jun 2015 14:42:18 -0700 Subject: [PATCH 240/246] Integrate futurist (and **remove** taskflow originating code) Change-Id: If89baa042695f19e42b6368034f3ccf22c2cf0aa --- doc/source/types.rst | 10 - requirements.txt | 3 + .../engines/action_engine/actions/retry.py | 5 +- taskflow/engines/action_engine/engine.py | 2 +- taskflow/engines/action_engine/executor.py | 8 +- taskflow/engines/worker_based/executor.py | 4 +- taskflow/engines/worker_based/protocol.py | 4 +- taskflow/engines/worker_based/types.py | 4 +- taskflow/engines/worker_based/worker.py | 4 +- taskflow/examples/hello_world.py | 9 +- taskflow/examples/parallel_table_multiply.py | 6 +- taskflow/examples/resume_vm_boot.py | 4 +- taskflow/examples/share_engine_thread.py | 4 +- taskflow/jobs/backends/impl_zookeeper.py | 4 +- .../tests/unit/action_engine/test_creation.py | 10 +- taskflow/tests/unit/test_arguments_passing.py | 4 +- taskflow/tests/unit/test_engines.py | 6 +- taskflow/tests/unit/test_futures.py | 229 --------- taskflow/tests/unit/test_retries.py | 4 +- taskflow/tests/unit/test_suspend.py | 4 +- taskflow/tests/unit/test_types.py | 131 ------ taskflow/tests/unit/test_utils_async_utils.py | 14 +- .../tests/unit/worker_based/test_executor.py | 6 +- .../tests/unit/worker_based/test_pipeline.py | 4 +- .../tests/unit/worker_based/test_protocol.py | 4 +- .../tests/unit/worker_based/test_worker.py | 2 +- taskflow/types/futures.py | 444 ------------------ taskflow/types/periodic.py | 209 --------- taskflow/utils/async_utils.py | 9 +- taskflow/utils/threading_utils.py | 12 - 30 files changed, 68 insertions(+), 1095 deletions(-) delete mode 100644 taskflow/tests/unit/test_futures.py delete mode 100644 taskflow/types/futures.py delete mode 100644 taskflow/types/periodic.py diff --git a/doc/source/types.rst b/doc/source/types.rst index 57cebaea..84d446ac 100644 --- a/doc/source/types.rst +++ b/doc/source/types.rst @@ -29,11 +29,6 @@ FSM .. automodule:: taskflow.types.fsm -Futures -======= - -.. automodule:: taskflow.types.futures - Graph ===== @@ -45,11 +40,6 @@ Notifier .. automodule:: taskflow.types.notifier :special-members: __call__ -Periodic -======== - -.. automodule:: taskflow.types.periodic - Sets ==== diff --git a/requirements.txt b/requirements.txt index d4abcdd0..24414c6d 100644 --- a/requirements.txt +++ b/requirements.txt @@ -13,6 +13,9 @@ six>=1.9.0 # Enum library made for <= python 3.3 enum34;python_version=='2.7' or python_version=='2.6' +# For async and/or periodic work +futurist>=0.1.1 # Apache-2.0 + # For reader/writer + interprocess locks. fasteners>=0.7 # Apache-2.0 diff --git a/taskflow/engines/action_engine/actions/retry.py b/taskflow/engines/action_engine/actions/retry.py index c8cad50a..a3834d15 100644 --- a/taskflow/engines/action_engine/actions/retry.py +++ b/taskflow/engines/action_engine/actions/retry.py @@ -14,13 +14,14 @@ # License for the specific language governing permissions and limitations # under the License. +import futurist + from taskflow.engines.action_engine.actions import base from taskflow.engines.action_engine import executor as ex from taskflow import logging from taskflow import retry as retry_atom from taskflow import states from taskflow.types import failure -from taskflow.types import futures LOG = logging.getLogger(__name__) @@ -46,7 +47,7 @@ class RetryAction(base.Action): def __init__(self, storage, notifier): super(RetryAction, self).__init__(storage, notifier) - self._executor = futures.SynchronousExecutor() + self._executor = futurist.SynchronousExecutor() def _get_retry_args(self, retry, addons=None): arguments = self._storage.fetch_mapped_args( diff --git a/taskflow/engines/action_engine/engine.py b/taskflow/engines/action_engine/engine.py index 9da9ae9d..11bc1d38 100644 --- a/taskflow/engines/action_engine/engine.py +++ b/taskflow/engines/action_engine/engine.py @@ -364,7 +364,7 @@ String (case insensitive) Executor used # # NOTE(harlowja): the reason we use the library/built-in futures is to # allow for instances of that to be detected and handled correctly, instead - # of forcing everyone to use our derivatives... + # of forcing everyone to use our derivatives (futurist or other)... _executor_cls_matchers = [ _ExecutorTypeMatch((futures.ThreadPoolExecutor,), executor.ParallelThreadTaskExecutor), diff --git a/taskflow/engines/action_engine/executor.py b/taskflow/engines/action_engine/executor.py index 2f8ddb09..6f2e8b82 100644 --- a/taskflow/engines/action_engine/executor.py +++ b/taskflow/engines/action_engine/executor.py @@ -21,6 +21,7 @@ import os import pickle import threading +import futurist from oslo_utils import excutils from oslo_utils import reflection from oslo_utils import timeutils @@ -31,7 +32,6 @@ from six.moves import queue as compat_queue from taskflow import logging from taskflow import task as task_atom from taskflow.types import failure -from taskflow.types import futures from taskflow.types import notifier from taskflow.types import timing from taskflow.utils import async_utils @@ -357,7 +357,7 @@ class SerialTaskExecutor(TaskExecutor): """Executes tasks one after another.""" def __init__(self): - self._executor = futures.SynchronousExecutor() + self._executor = futurist.SynchronousExecutor() def start(self): self._executor.restart() @@ -429,7 +429,7 @@ class ParallelThreadTaskExecutor(ParallelTaskExecutor): """Executes tasks in parallel using a thread pool executor.""" def _create_executor(self, max_workers=None): - return futures.ThreadPoolExecutor(max_workers=max_workers) + return futurist.ThreadPoolExecutor(max_workers=max_workers) class ParallelProcessTaskExecutor(ParallelTaskExecutor): @@ -459,7 +459,7 @@ class ParallelProcessTaskExecutor(ParallelTaskExecutor): self._queue = None def _create_executor(self, max_workers=None): - return futures.ProcessPoolExecutor(max_workers=max_workers) + return futurist.ProcessPoolExecutor(max_workers=max_workers) def start(self): if threading_utils.is_alive(self._worker): diff --git a/taskflow/engines/worker_based/executor.py b/taskflow/engines/worker_based/executor.py index b5b7d39b..6ad86d7b 100644 --- a/taskflow/engines/worker_based/executor.py +++ b/taskflow/engines/worker_based/executor.py @@ -16,6 +16,7 @@ import functools +from futurist import periodics from oslo_utils import timeutils from taskflow.engines.action_engine import executor @@ -26,7 +27,6 @@ from taskflow.engines.worker_based import types as wt from taskflow import exceptions as exc from taskflow import logging from taskflow import task as task_atom -from taskflow.types import periodic from taskflow.utils import kombu_utils as ku from taskflow.utils import misc from taskflow.utils import threading_utils as tu @@ -67,7 +67,7 @@ class WorkerTaskExecutor(executor.TaskExecutor): self._helpers.bind(lambda: tu.daemon_thread(self._proxy.start), after_start=lambda t: self._proxy.wait(), before_join=lambda t: self._proxy.stop()) - p_worker = periodic.PeriodicWorker.create([self._finder]) + p_worker = periodics.PeriodicWorker.create([self._finder]) if p_worker: self._helpers.bind(lambda: tu.daemon_thread(p_worker.start), before_join=lambda t: p_worker.stop(), diff --git a/taskflow/engines/worker_based/protocol.py b/taskflow/engines/worker_based/protocol.py index cbb61ebe..823f83c9 100644 --- a/taskflow/engines/worker_based/protocol.py +++ b/taskflow/engines/worker_based/protocol.py @@ -18,8 +18,8 @@ import abc import collections import threading -from concurrent import futures import fasteners +import futurist from oslo_utils import reflection from oslo_utils import timeutils import six @@ -243,7 +243,7 @@ class Request(Message): self._state = WAITING self._lock = threading.Lock() self._created_on = timeutils.utcnow() - self._result = futures.Future() + self._result = futurist.Future() self._result.atom = task self._notifier = task.notifier diff --git a/taskflow/engines/worker_based/types.py b/taskflow/engines/worker_based/types.py index 1ee8f4b8..5d212e5c 100644 --- a/taskflow/engines/worker_based/types.py +++ b/taskflow/engines/worker_based/types.py @@ -20,6 +20,7 @@ import itertools import random import threading +from futurist import periodics from oslo_utils import reflection import six @@ -28,7 +29,6 @@ from taskflow.engines.worker_based import protocol as pr from taskflow import logging from taskflow.types import cache as base from taskflow.types import notifier -from taskflow.types import periodic from taskflow.types import timing as tt from taskflow.utils import kombu_utils as ku @@ -180,7 +180,7 @@ class ProxyWorkerFinder(WorkerFinder): else: return TopicWorker(topic, tasks) - @periodic.periodic(pr.NOTIFY_PERIOD) + @periodics.periodic(pr.NOTIFY_PERIOD, run_immediately=True) def beat(self): """Cyclically called to publish notify message to each topic.""" self._proxy.publish(pr.Notify(), self._topics, reply_to=self._uuid) diff --git a/taskflow/engines/worker_based/worker.py b/taskflow/engines/worker_based/worker.py index 8a79133f..a462c7f3 100644 --- a/taskflow/engines/worker_based/worker.py +++ b/taskflow/engines/worker_based/worker.py @@ -20,13 +20,13 @@ import socket import string import sys +import futurist from oslo_utils import reflection from taskflow.engines.worker_based import endpoint from taskflow.engines.worker_based import server from taskflow import logging from taskflow import task as t_task -from taskflow.types import futures from taskflow.utils import misc from taskflow.utils import threading_utils as tu from taskflow import version @@ -99,7 +99,7 @@ System details: self._executor = executor self._owns_executor = False if self._executor is None: - self._executor = futures.ThreadPoolExecutor( + self._executor = futurist.ThreadPoolExecutor( max_workers=threads_count) self._owns_executor = True self._endpoints = self._derive_endpoints(tasks) diff --git a/taskflow/examples/hello_world.py b/taskflow/examples/hello_world.py index caba5275..38a6b387 100644 --- a/taskflow/examples/hello_world.py +++ b/taskflow/examples/hello_world.py @@ -25,11 +25,12 @@ top_dir = os.path.abspath(os.path.join(os.path.dirname(__file__), os.pardir)) sys.path.insert(0, top_dir) +import futurist + from taskflow import engines from taskflow.patterns import linear_flow as lf from taskflow.patterns import unordered_flow as uf from taskflow import task -from taskflow.types import futures from taskflow.utils import eventlet_utils @@ -82,19 +83,19 @@ song.add(PrinterTask("conductor@begin", # Run in parallel using eventlet green threads... if eventlet_utils.EVENTLET_AVAILABLE: - with futures.GreenThreadPoolExecutor() as executor: + with futurist.GreenThreadPoolExecutor() as executor: e = engines.load(song, executor=executor, engine='parallel') e.run() # Run in parallel using real threads... -with futures.ThreadPoolExecutor(max_workers=1) as executor: +with futurist.ThreadPoolExecutor(max_workers=1) as executor: e = engines.load(song, executor=executor, engine='parallel') e.run() # Run in parallel using external processes... -with futures.ProcessPoolExecutor(max_workers=1) as executor: +with futurist.ProcessPoolExecutor(max_workers=1) as executor: e = engines.load(song, executor=executor, engine='parallel') e.run() diff --git a/taskflow/examples/parallel_table_multiply.py b/taskflow/examples/parallel_table_multiply.py index f4550c20..e06e36d8 100644 --- a/taskflow/examples/parallel_table_multiply.py +++ b/taskflow/examples/parallel_table_multiply.py @@ -27,12 +27,12 @@ top_dir = os.path.abspath(os.path.join(os.path.dirname(__file__), os.pardir)) sys.path.insert(0, top_dir) +import futurist from six.moves import range as compat_range from taskflow import engines from taskflow.patterns import unordered_flow as uf from taskflow import task -from taskflow.types import futures from taskflow.utils import eventlet_utils # INTRO: This example walks through a miniature workflow which does a parallel @@ -98,9 +98,9 @@ def main(): # Now run it (using the specified executor)... if eventlet_utils.EVENTLET_AVAILABLE: - executor = futures.GreenThreadPoolExecutor(max_workers=5) + executor = futurist.GreenThreadPoolExecutor(max_workers=5) else: - executor = futures.ThreadPoolExecutor(max_workers=5) + executor = futurist.ThreadPoolExecutor(max_workers=5) try: e = engines.load(f, engine='parallel', executor=executor) for st in e.run_iter(): diff --git a/taskflow/examples/resume_vm_boot.py b/taskflow/examples/resume_vm_boot.py index 8c7d4aea..ec2293bf 100644 --- a/taskflow/examples/resume_vm_boot.py +++ b/taskflow/examples/resume_vm_boot.py @@ -31,6 +31,7 @@ top_dir = os.path.abspath(os.path.join(os.path.dirname(__file__), sys.path.insert(0, top_dir) sys.path.insert(0, self_dir) +import futurist from oslo_utils import uuidutils from taskflow import engines @@ -38,7 +39,6 @@ from taskflow import exceptions as exc from taskflow.patterns import graph_flow as gf from taskflow.patterns import linear_flow as lf from taskflow import task -from taskflow.types import futures from taskflow.utils import eventlet_utils from taskflow.utils import persistence_utils as p_utils @@ -239,7 +239,7 @@ with eu.get_backend() as backend: # Set up how we want our engine to run, serial, parallel... executor = None if eventlet_utils.EVENTLET_AVAILABLE: - executor = futures.GreenThreadPoolExecutor(5) + executor = futurist.GreenThreadPoolExecutor(5) # Create/fetch a logbook that will track the workflows work. book = None diff --git a/taskflow/examples/share_engine_thread.py b/taskflow/examples/share_engine_thread.py index 5654fa06..5223721b 100644 --- a/taskflow/examples/share_engine_thread.py +++ b/taskflow/examples/share_engine_thread.py @@ -27,12 +27,12 @@ top_dir = os.path.abspath(os.path.join(os.path.dirname(__file__), os.pardir)) sys.path.insert(0, top_dir) +import futurist import six from taskflow import engines from taskflow.patterns import unordered_flow as uf from taskflow import task -from taskflow.types import futures from taskflow.utils import threading_utils as tu # INTRO: in this example we create 2 dummy flow(s) with a 2 dummy task(s), and @@ -61,7 +61,7 @@ f2.add(DelayedTask("f2-1")) f2.add(DelayedTask("f2-2")) # Run them all using the same futures (thread-pool based) executor... -with futures.ThreadPoolExecutor() as ex: +with futurist.ThreadPoolExecutor() as ex: e1 = engines.load(f1, engine='parallel', executor=ex) e2 = engines.load(f2, engine='parallel', executor=ex) iters = [e1.run_iter(), e2.run_iter()] diff --git a/taskflow/jobs/backends/impl_zookeeper.py b/taskflow/jobs/backends/impl_zookeeper.py index 15b31034..9df706e7 100644 --- a/taskflow/jobs/backends/impl_zookeeper.py +++ b/taskflow/jobs/backends/impl_zookeeper.py @@ -20,8 +20,8 @@ import functools import sys import threading -from concurrent import futures import fasteners +import futurist from kazoo import exceptions as k_exceptions from kazoo.protocol import paths as k_paths from kazoo.recipe import watchers @@ -736,7 +736,7 @@ class ZookeeperJobBoard(base.NotifyingJobBoard): if self._conf.get('check_compatible', True): kazoo_utils.check_compatible(self._client, self.MIN_ZK_VERSION) if self._worker is None and self._emit_notifications: - self._worker = futures.ThreadPoolExecutor(max_workers=1) + self._worker = futurist.ThreadPoolExecutor(max_workers=1) self._client.ensure_path(self.path) self._client.ensure_path(self.trash_path) if self._job_watcher is None: diff --git a/taskflow/tests/unit/action_engine/test_creation.py b/taskflow/tests/unit/action_engine/test_creation.py index 2c6ed585..a2390993 100644 --- a/taskflow/tests/unit/action_engine/test_creation.py +++ b/taskflow/tests/unit/action_engine/test_creation.py @@ -14,6 +14,7 @@ # License for the specific language governing permissions and limitations # under the License. +import futurist import testtools from taskflow.engines.action_engine import engine @@ -22,7 +23,6 @@ from taskflow.patterns import linear_flow as lf from taskflow.persistence import backends from taskflow import test from taskflow.tests import utils -from taskflow.types import futures as futures from taskflow.utils import eventlet_utils as eu from taskflow.utils import persistence_utils as pu @@ -50,26 +50,26 @@ class ParallelCreationTest(test.TestCase): executor.ParallelProcessTaskExecutor) def test_thread_executor_creation(self): - with futures.ThreadPoolExecutor(1) as e: + with futurist.ThreadPoolExecutor(1) as e: eng = self._create_engine(executor=e) self.assertIsInstance(eng._task_executor, executor.ParallelThreadTaskExecutor) def test_process_executor_creation(self): - with futures.ProcessPoolExecutor(1) as e: + with futurist.ProcessPoolExecutor(1) as e: eng = self._create_engine(executor=e) self.assertIsInstance(eng._task_executor, executor.ParallelProcessTaskExecutor) @testtools.skipIf(not eu.EVENTLET_AVAILABLE, 'eventlet is not available') def test_green_executor_creation(self): - with futures.GreenThreadPoolExecutor(1) as e: + with futurist.GreenThreadPoolExecutor(1) as e: eng = self._create_engine(executor=e) self.assertIsInstance(eng._task_executor, executor.ParallelThreadTaskExecutor) def test_sync_executor_creation(self): - with futures.SynchronousExecutor() as e: + with futurist.SynchronousExecutor() as e: eng = self._create_engine(executor=e) self.assertIsInstance(eng._task_executor, executor.ParallelThreadTaskExecutor) diff --git a/taskflow/tests/unit/test_arguments_passing.py b/taskflow/tests/unit/test_arguments_passing.py index a1ba2ac8..2cde1dea 100644 --- a/taskflow/tests/unit/test_arguments_passing.py +++ b/taskflow/tests/unit/test_arguments_passing.py @@ -14,13 +14,13 @@ # License for the specific language governing permissions and limitations # under the License. +import futurist import testtools import taskflow.engines from taskflow import exceptions as exc from taskflow import test from taskflow.tests import utils -from taskflow.types import futures from taskflow.utils import eventlet_utils as eu @@ -192,7 +192,7 @@ class ParallelEngineWithEventletTest(ArgumentsPassingTest, test.TestCase): def _make_engine(self, flow, flow_detail=None, executor=None): if executor is None: - executor = futures.GreenThreadPoolExecutor() + executor = futurist.GreenThreadPoolExecutor() self.addCleanup(executor.shutdown) return taskflow.engines.load(flow, flow_detail=flow_detail, diff --git a/taskflow/tests/unit/test_engines.py b/taskflow/tests/unit/test_engines.py index ed073e6a..093fe9b9 100644 --- a/taskflow/tests/unit/test_engines.py +++ b/taskflow/tests/unit/test_engines.py @@ -17,6 +17,7 @@ import contextlib import functools +import futurist import six import testtools @@ -34,7 +35,6 @@ from taskflow import task from taskflow import test from taskflow.tests import utils from taskflow.types import failure -from taskflow.types import futures from taskflow.types import graph as gr from taskflow.utils import eventlet_utils as eu from taskflow.utils import persistence_utils as p_utils @@ -977,7 +977,7 @@ class ParallelEngineWithThreadsTest(EngineTaskTest, def test_using_common_executor(self): flow = utils.TaskNoRequiresNoReturns(name='task1') - executor = futures.ThreadPoolExecutor(self._EXECUTOR_WORKERS) + executor = futurist.ThreadPoolExecutor(self._EXECUTOR_WORKERS) try: e1 = self._make_engine(flow, executor=executor) e2 = self._make_engine(flow, executor=executor) @@ -1002,7 +1002,7 @@ class ParallelEngineWithEventletTest(EngineTaskTest, def _make_engine(self, flow, flow_detail=None, executor=None, store=None): if executor is None: - executor = futures.GreenThreadPoolExecutor() + executor = futurist.GreenThreadPoolExecutor() self.addCleanup(executor.shutdown) return taskflow.engines.load(flow, flow_detail=flow_detail, backend=self.backend, engine='parallel', diff --git a/taskflow/tests/unit/test_futures.py b/taskflow/tests/unit/test_futures.py deleted file mode 100644 index ce2c69c1..00000000 --- a/taskflow/tests/unit/test_futures.py +++ /dev/null @@ -1,229 +0,0 @@ -# -*- coding: utf-8 -*- - -# Copyright (C) 2013 Yahoo! Inc. 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. - -import collections -import functools -import threading -import time - -import testtools - -from taskflow import test -from taskflow.types import futures -from taskflow.utils import eventlet_utils as eu - -try: - from eventlet.green import threading as greenthreading - from eventlet.green import time as greentime -except ImportError: - pass - - -def _noop(): - pass - - -def _blowup(): - raise IOError("Broke!") - - -def _return_given(given): - return given - - -def _return_one(): - return 1 - - -def _double(x): - return x * 2 - - -class _SimpleFuturesTestMixin(object): - # This exists to test basic functionality, mainly required to test the - # process executor which has a very restricted set of things it can - # execute (no lambda functions, no instance methods...) - def _make_executor(self, max_workers): - raise NotImplementedError("Not implemented") - - def test_invalid_workers(self): - self.assertRaises(ValueError, self._make_executor, -1) - self.assertRaises(ValueError, self._make_executor, 0) - - def test_exception_transfer(self): - with self._make_executor(2) as e: - f = e.submit(_blowup) - self.assertRaises(IOError, f.result) - self.assertEqual(1, e.statistics.failures) - - def test_accumulator(self): - created = [] - with self._make_executor(5) as e: - for _i in range(0, 10): - created.append(e.submit(_return_one)) - results = [f.result() for f in created] - self.assertEqual(10, sum(results)) - self.assertEqual(10, e.statistics.executed) - - def test_map(self): - count = [i for i in range(0, 100)] - with self._make_executor(5) as e: - results = list(e.map(_double, count)) - initial = sum(count) - self.assertEqual(2 * initial, sum(results)) - - def test_alive(self): - e = self._make_executor(1) - self.assertTrue(e.alive) - e.shutdown() - self.assertFalse(e.alive) - with self._make_executor(1) as e2: - self.assertTrue(e2.alive) - self.assertFalse(e2.alive) - - -class _FuturesTestMixin(_SimpleFuturesTestMixin): - def _delay(self, secs): - raise NotImplementedError("Not implemented") - - def _make_lock(self): - raise NotImplementedError("Not implemented") - - def _make_funcs(self, called, amount): - mutator = self._make_lock() - - def store_call(ident): - with mutator: - called[ident] += 1 - - for i in range(0, amount): - yield functools.partial(store_call, ident=i) - - def test_func_calls(self): - called = collections.defaultdict(int) - - with self._make_executor(2) as e: - for f in self._make_funcs(called, 2): - e.submit(f) - - self.assertEqual(1, called[0]) - self.assertEqual(1, called[1]) - self.assertEqual(2, e.statistics.executed) - - def test_result_callback(self): - called = collections.defaultdict(int) - mutator = self._make_lock() - - def callback(future): - with mutator: - called[future] += 1 - - funcs = list(self._make_funcs(called, 1)) - with self._make_executor(2) as e: - for func in funcs: - f = e.submit(func) - f.add_done_callback(callback) - - self.assertEqual(2, len(called)) - - def test_result_transfer(self): - create_am = 50 - with self._make_executor(2) as e: - fs = [] - for i in range(0, create_am): - fs.append(e.submit(functools.partial(_return_given, i))) - self.assertEqual(create_am, len(fs)) - self.assertEqual(create_am, e.statistics.executed) - for i in range(0, create_am): - result = fs[i].result() - self.assertEqual(i, result) - - def test_called_restricted_size(self): - create_am = 100 - called = collections.defaultdict(int) - - with self._make_executor(1) as e: - for f in self._make_funcs(called, create_am): - e.submit(f) - - self.assertFalse(e.alive) - self.assertEqual(create_am, len(called)) - self.assertEqual(create_am, e.statistics.executed) - - -class ThreadPoolExecutorTest(test.TestCase, _FuturesTestMixin): - def _make_executor(self, max_workers): - return futures.ThreadPoolExecutor(max_workers=max_workers) - - def _delay(self, secs): - time.sleep(secs) - - def _make_lock(self): - return threading.Lock() - - -class ProcessPoolExecutorTest(test.TestCase, _SimpleFuturesTestMixin): - def _make_executor(self, max_workers): - return futures.ProcessPoolExecutor(max_workers=max_workers) - - -class SynchronousExecutorTest(test.TestCase, _FuturesTestMixin): - def _make_executor(self, max_workers): - return futures.SynchronousExecutor() - - def _delay(self, secs): - time.sleep(secs) - - def _make_lock(self): - return threading.Lock() - - def test_invalid_workers(self): - pass - - -@testtools.skipIf(not eu.EVENTLET_AVAILABLE, 'eventlet is not available') -class GreenThreadPoolExecutorTest(test.TestCase, _FuturesTestMixin): - def _make_executor(self, max_workers): - return futures.GreenThreadPoolExecutor(max_workers=max_workers) - - def _delay(self, secs): - greentime.sleep(secs) - - def _make_lock(self): - return greenthreading.Lock() - - def test_cancellation(self): - called = collections.defaultdict(int) - - fs = [] - with self._make_executor(2) as e: - for func in self._make_funcs(called, 2): - fs.append(e.submit(func)) - # Greenthreads don't start executing until we wait for them - # to, since nothing here does IO, this will work out correctly. - # - # If something here did a blocking call, then eventlet could swap - # one of the executors threads in, but nothing in this test does. - for f in fs: - self.assertFalse(f.running()) - f.cancel() - - self.assertEqual(0, len(called)) - self.assertEqual(2, len(fs)) - self.assertEqual(2, e.statistics.cancelled) - for f in fs: - self.assertTrue(f.cancelled()) - self.assertTrue(f.done()) diff --git a/taskflow/tests/unit/test_retries.py b/taskflow/tests/unit/test_retries.py index ddb256b0..330c13a7 100644 --- a/taskflow/tests/unit/test_retries.py +++ b/taskflow/tests/unit/test_retries.py @@ -14,6 +14,7 @@ # License for the specific language governing permissions and limitations # under the License. +import futurist import testtools import taskflow.engines @@ -26,7 +27,6 @@ from taskflow import states as st from taskflow import test from taskflow.tests import utils from taskflow.types import failure -from taskflow.types import futures from taskflow.utils import eventlet_utils as eu @@ -1236,7 +1236,7 @@ class ParallelEngineWithEventletTest(RetryTest, test.TestCase): def _make_engine(self, flow, flow_detail=None, executor=None): if executor is None: - executor = futures.GreenThreadPoolExecutor() + executor = futurist.GreenThreadPoolExecutor() self.addCleanup(executor.shutdown) return taskflow.engines.load(flow, flow_detail=flow_detail, backend=self.backend, engine='parallel', diff --git a/taskflow/tests/unit/test_suspend.py b/taskflow/tests/unit/test_suspend.py index e5d0288f..5d11eddd 100644 --- a/taskflow/tests/unit/test_suspend.py +++ b/taskflow/tests/unit/test_suspend.py @@ -14,6 +14,7 @@ # License for the specific language governing permissions and limitations # under the License. +import futurist import testtools import taskflow.engines @@ -22,7 +23,6 @@ from taskflow.patterns import linear_flow as lf from taskflow import states from taskflow import test from taskflow.tests import utils -from taskflow.types import futures from taskflow.utils import eventlet_utils as eu @@ -217,7 +217,7 @@ class ParallelEngineWithEventletTest(SuspendTest, test.TestCase): def _make_engine(self, flow, flow_detail=None, executor=None): if executor is None: - executor = futures.GreenThreadPoolExecutor() + executor = futurist.GreenThreadPoolExecutor() self.addCleanup(executor.shutdown) return taskflow.engines.load(flow, flow_detail=flow_detail, backend=self.backend, engine='parallel', diff --git a/taskflow/tests/unit/test_types.py b/taskflow/tests/unit/test_types.py index 809c1926..8980aa5c 100644 --- a/taskflow/tests/unit/test_types.py +++ b/taskflow/tests/unit/test_types.py @@ -14,8 +14,6 @@ # License for the specific language governing permissions and limitations # under the License. -import time - import networkx as nx import six from six.moves import cPickle as pickle @@ -24,31 +22,9 @@ from taskflow import exceptions as excp from taskflow import test from taskflow.types import fsm from taskflow.types import graph -from taskflow.types import latch -from taskflow.types import periodic from taskflow.types import sets from taskflow.types import table from taskflow.types import tree -from taskflow.utils import threading_utils as tu - - -class PeriodicThingy(object): - def __init__(self): - self.capture = [] - - @periodic.periodic(0.01) - def a(self): - self.capture.append('a') - - @periodic.periodic(0.02) - def b(self): - self.capture.append('b') - - def c(self): - pass - - def d(self): - pass class GraphTest(test.TestCase): @@ -629,110 +605,3 @@ class OrderedSetTest(test.TestCase): es3 = set(s3) self.assertEqual(es.union(es2, es3), s.union(s2, s3)) - - -class PeriodicTest(test.TestCase): - - def test_invalid_periodic(self): - - def no_op(): - pass - - self.assertRaises(ValueError, periodic.periodic, -1) - - def test_valid_periodic(self): - - @periodic.periodic(2) - def no_op(): - pass - - self.assertTrue(getattr(no_op, '_periodic')) - self.assertEqual(2, getattr(no_op, '_periodic_spacing')) - self.assertEqual(True, getattr(no_op, '_periodic_run_immediately')) - - def test_scanning_periodic(self): - p = PeriodicThingy() - w = periodic.PeriodicWorker.create([p]) - self.assertEqual(2, len(w)) - - t = tu.daemon_thread(target=w.start) - t.start() - time.sleep(0.1) - w.stop() - t.join() - - b_calls = [c for c in p.capture if c == 'b'] - self.assertGreater(0, len(b_calls)) - a_calls = [c for c in p.capture if c == 'a'] - self.assertGreater(0, len(a_calls)) - - def test_periodic_single(self): - barrier = latch.Latch(5) - capture = [] - - @periodic.periodic(0.01) - def callee(): - barrier.countdown() - if barrier.needed == 0: - w.stop() - capture.append(1) - - w = periodic.PeriodicWorker([callee]) - t = tu.daemon_thread(target=w.start) - t.start() - t.join() - - self.assertEqual(0, barrier.needed) - self.assertEqual(5, sum(capture)) - - def test_immediate(self): - capture = [] - - @periodic.periodic(120, run_immediately=True) - def a(): - capture.append('a') - - w = periodic.PeriodicWorker([a]) - t = tu.daemon_thread(target=w.start) - t.start() - time.sleep(0.1) - w.stop() - t.join() - - a_calls = [c for c in capture if c == 'a'] - self.assertGreater(0, len(a_calls)) - - def test_period_double_no_immediate(self): - capture = [] - - @periodic.periodic(0.01, run_immediately=False) - def a(): - capture.append('a') - - @periodic.periodic(0.02, run_immediately=False) - def b(): - capture.append('b') - - w = periodic.PeriodicWorker([a, b]) - t = tu.daemon_thread(target=w.start) - t.start() - time.sleep(0.1) - w.stop() - t.join() - - b_calls = [c for c in capture if c == 'b'] - self.assertGreater(0, len(b_calls)) - a_calls = [c for c in capture if c == 'a'] - self.assertGreater(0, len(a_calls)) - - def test_start_nothing_error(self): - w = periodic.PeriodicWorker([]) - self.assertRaises(RuntimeError, w.start) - - def test_missing_function_attrs(self): - - def fake_periodic(): - pass - - cb = fake_periodic - self.assertRaises(ValueError, periodic.PeriodicWorker, [cb]) diff --git a/taskflow/tests/unit/test_utils_async_utils.py b/taskflow/tests/unit/test_utils_async_utils.py index 642be96e..1f8b0119 100644 --- a/taskflow/tests/unit/test_utils_async_utils.py +++ b/taskflow/tests/unit/test_utils_async_utils.py @@ -14,10 +14,10 @@ # License for the specific language governing permissions and limitations # under the License. +import futurist import testtools from taskflow import test -from taskflow.types import futures from taskflow.utils import async_utils as au from taskflow.utils import eventlet_utils as eu @@ -37,14 +37,14 @@ class WaitForAnyTestsMixin(object): self.assertTrue(any(f in done for f in fs)) def test_not_done_futures(self): - fs = [futures.Future(), futures.Future()] + fs = [futurist.Future(), futurist.Future()] done, not_done = au.wait_for_any(fs, self.timeout) self.assertEqual(len(done), 0) self.assertEqual(len(not_done), 2) def test_mixed_futures(self): - f1 = futures.Future() - f2 = futures.Future() + f1 = futurist.Future() + f2 = futurist.Future() f2.set_result(1) done, not_done = au.wait_for_any([f1, f2], self.timeout) self.assertEqual(len(done), 1) @@ -57,13 +57,13 @@ class WaitForAnyTestsMixin(object): class AsyncUtilsEventletTest(test.TestCase, WaitForAnyTestsMixin): def _make_executor(self, max_workers): - return futures.GreenThreadPoolExecutor(max_workers=max_workers) + return futurist.GreenThreadPoolExecutor(max_workers=max_workers) class AsyncUtilsThreadedTest(test.TestCase, WaitForAnyTestsMixin): def _make_executor(self, max_workers): - return futures.ThreadPoolExecutor(max_workers=max_workers) + return futurist.ThreadPoolExecutor(max_workers=max_workers) class MakeCompletedFutureTest(test.TestCase): @@ -78,4 +78,4 @@ class MakeCompletedFutureTest(test.TestCase): class AsyncUtilsSynchronousTest(test.TestCase, WaitForAnyTestsMixin): def _make_executor(self, max_workers): - return futures.SynchronousExecutor() + return futurist.SynchronousExecutor() diff --git a/taskflow/tests/unit/worker_based/test_executor.py b/taskflow/tests/unit/worker_based/test_executor.py index d944b64b..e6c21eb4 100644 --- a/taskflow/tests/unit/worker_based/test_executor.py +++ b/taskflow/tests/unit/worker_based/test_executor.py @@ -17,7 +17,7 @@ import threading import time -from concurrent import futures +import futurist from oslo_utils import timeutils from taskflow.engines.worker_based import executor @@ -281,7 +281,7 @@ class TestWorkerTaskExecutor(test.MockTestCase): self.assertEqual(expected_calls, self.master_mock.mock_calls) def test_wait_for_any(self): - fs = [futures.Future(), futures.Future()] + fs = [futurist.Future(), futurist.Future()] ex = self.executor() ex.wait_for_any(fs) @@ -292,7 +292,7 @@ class TestWorkerTaskExecutor(test.MockTestCase): def test_wait_for_any_with_timeout(self): timeout = 30 - fs = [futures.Future(), futures.Future()] + fs = [futurist.Future(), futurist.Future()] ex = self.executor() ex.wait_for_any(fs, timeout) diff --git a/taskflow/tests/unit/worker_based/test_pipeline.py b/taskflow/tests/unit/worker_based/test_pipeline.py index 8d4de7f5..3030b831 100644 --- a/taskflow/tests/unit/worker_based/test_pipeline.py +++ b/taskflow/tests/unit/worker_based/test_pipeline.py @@ -14,7 +14,7 @@ # License for the specific language governing permissions and limitations # under the License. -from concurrent import futures +import futurist from oslo_utils import uuidutils from taskflow.engines.action_engine import executor as base_executor @@ -39,7 +39,7 @@ class TestPipeline(test.TestCase): endpoints.append(endpoint.Endpoint(cls)) server = worker_server.Server( TEST_TOPIC, TEST_EXCHANGE, - futures.ThreadPoolExecutor(1), endpoints, + futurist.ThreadPoolExecutor(max_workers=1), endpoints, transport='memory', transport_options={ 'polling_interval': POLLING_INTERVAL, diff --git a/taskflow/tests/unit/worker_based/test_protocol.py b/taskflow/tests/unit/worker_based/test_protocol.py index 73fe9c8c..4647ae8c 100644 --- a/taskflow/tests/unit/worker_based/test_protocol.py +++ b/taskflow/tests/unit/worker_based/test_protocol.py @@ -14,7 +14,7 @@ # License for the specific language governing permissions and limitations # under the License. -from concurrent import futures +import futurist from oslo_utils import uuidutils from taskflow.engines.action_engine import executor @@ -135,7 +135,7 @@ class TestProtocol(test.TestCase): request = self.request() self.assertEqual(request.uuid, self.task_uuid) self.assertEqual(request.task, self.task) - self.assertIsInstance(request.result, futures.Future) + self.assertIsInstance(request.result, futurist.Future) self.assertFalse(request.result.done()) def test_to_dict_default(self): diff --git a/taskflow/tests/unit/worker_based/test_worker.py b/taskflow/tests/unit/worker_based/test_worker.py index 3acf245b..f4a02aea 100644 --- a/taskflow/tests/unit/worker_based/test_worker.py +++ b/taskflow/tests/unit/worker_based/test_worker.py @@ -37,7 +37,7 @@ class TestWorker(test.MockTestCase): # patch classes self.executor_mock, self.executor_inst_mock = self.patchClass( - worker.futures, 'ThreadPoolExecutor', attach_as='executor') + worker.futurist, 'ThreadPoolExecutor', attach_as='executor') self.server_mock, self.server_inst_mock = self.patchClass( worker.server, 'Server') diff --git a/taskflow/types/futures.py b/taskflow/types/futures.py deleted file mode 100644 index da79fc8e..00000000 --- a/taskflow/types/futures.py +++ /dev/null @@ -1,444 +0,0 @@ -# -*- coding: utf-8 -*- - -# Copyright (C) 2014 Yahoo! Inc. 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. - -import functools -import sys -import threading - -from concurrent import futures as _futures -from concurrent.futures import process as _process -from concurrent.futures import thread as _thread -from oslo_utils import importutils -from oslo_utils import reflection -import six - -greenpatcher = importutils.try_import('eventlet.patcher') -greenpool = importutils.try_import('eventlet.greenpool') -greenqueue = importutils.try_import('eventlet.queue') -greenthreading = importutils.try_import('eventlet.green.threading') - -from taskflow.types import timing -from taskflow.utils import eventlet_utils as eu -from taskflow.utils import threading_utils as tu - - -# NOTE(harlowja): Allows for simpler access to this type... -Future = _futures.Future - - -class _Gatherer(object): - def __init__(self, submit_func, - lock_cls=threading.Lock, start_before_submit=False): - self._submit_func = submit_func - self._stats_lock = lock_cls() - self._stats = ExecutorStatistics() - self._start_before_submit = start_before_submit - - @property - def statistics(self): - return self._stats - - def clear(self): - with self._stats_lock: - self._stats = ExecutorStatistics() - - def _capture_stats(self, watch, fut): - """Capture statistics - - :param watch: stopwatch object - :param fut: future object - """ - watch.stop() - with self._stats_lock: - # Use a new collection and lock so that all mutations are seen as - # atomic and not overlapping and corrupting with other - # mutations (the clone ensures that others reading the current - # values will not see a mutated/corrupted one). Since futures may - # be completed by different threads we need to be extra careful to - # gather this data in a way that is thread-safe... - (failures, executed, runtime, cancelled) = (self._stats.failures, - self._stats.executed, - self._stats.runtime, - self._stats.cancelled) - if fut.cancelled(): - cancelled += 1 - else: - executed += 1 - if fut.exception() is not None: - failures += 1 - runtime += watch.elapsed() - self._stats = ExecutorStatistics(failures=failures, - executed=executed, - runtime=runtime, - cancelled=cancelled) - - def submit(self, fn, *args, **kwargs): - """Submit work to be executed and capture statistics.""" - watch = timing.StopWatch() - if self._start_before_submit: - watch.start() - fut = self._submit_func(fn, *args, **kwargs) - if not self._start_before_submit: - watch.start() - fut.add_done_callback(functools.partial(self._capture_stats, watch)) - return fut - - -class ThreadPoolExecutor(_thread.ThreadPoolExecutor): - """Executor that uses a thread pool to execute calls asynchronously. - - It gathers statistics about the submissions executed for post-analysis... - - See: https://docs.python.org/dev/library/concurrent.futures.html - """ - def __init__(self, max_workers=None): - if max_workers is None: - max_workers = tu.get_optimal_thread_count() - super(ThreadPoolExecutor, self).__init__(max_workers=max_workers) - if self._max_workers <= 0: - raise ValueError("Max workers must be greater than zero") - self._gatherer = _Gatherer( - # Since our submit will use this gatherer we have to reference - # the parent submit, bound to this instance (which is what we - # really want to use anyway). - super(ThreadPoolExecutor, self).submit) - - @property - def max_workers(self): - """The max number of workers that this executor will ever have.""" - return self._max_workers - - @property - def statistics(self): - """:class:`.ExecutorStatistics` about the executors executions.""" - return self._gatherer.statistics - - @property - def alive(self): - """Accessor to determine if the executor is alive/active.""" - return not self._shutdown - - def submit(self, fn, *args, **kwargs): - """Submit some work to be executed (and gather statistics).""" - return self._gatherer.submit(fn, *args, **kwargs) - - -class ProcessPoolExecutor(_process.ProcessPoolExecutor): - """Executor that uses a process pool to execute calls asynchronously. - - It gathers statistics about the submissions executed for post-analysis... - - See: https://docs.python.org/dev/library/concurrent.futures.html - """ - def __init__(self, max_workers=None): - if max_workers is None: - max_workers = tu.get_optimal_thread_count() - super(ProcessPoolExecutor, self).__init__(max_workers=max_workers) - if self._max_workers <= 0: - raise ValueError("Max workers must be greater than zero") - self._gatherer = _Gatherer( - # Since our submit will use this gatherer we have to reference - # the parent submit, bound to this instance (which is what we - # really want to use anyway). - super(ProcessPoolExecutor, self).submit) - - @property - def alive(self): - """Accessor to determine if the executor is alive/active.""" - return not self._shutdown_thread - - @property - def max_workers(self): - """The max number of workers that this executor will ever have.""" - return self._max_workers - - @property - def statistics(self): - """:class:`.ExecutorStatistics` about the executors executions.""" - return self._gatherer.statistics - - def submit(self, fn, *args, **kwargs): - """Submit some work to be executed (and gather statistics).""" - return self._gatherer.submit(fn, *args, **kwargs) - - -class _WorkItem(object): - def __init__(self, future, fn, args, kwargs): - self.future = future - self.fn = fn - self.args = args - self.kwargs = kwargs - - def run(self): - if not self.future.set_running_or_notify_cancel(): - return - try: - result = self.fn(*self.args, **self.kwargs) - except BaseException: - exc_type, exc_value, exc_tb = sys.exc_info() - try: - if six.PY2: - self.future.set_exception_info(exc_value, exc_tb) - else: - self.future.set_exception(exc_value) - finally: - del(exc_type, exc_value, exc_tb) - else: - self.future.set_result(result) - - -class SynchronousExecutor(_futures.Executor): - """Executor that uses the caller to execute calls synchronously. - - This provides an interface to a caller that looks like an executor but - will execute the calls inside the caller thread instead of executing it - in a external process/thread for when this type of functionality is - useful to provide... - - It gathers statistics about the submissions executed for post-analysis... - """ - - def __init__(self): - self._shutoff = False - self._gatherer = _Gatherer(self._submit, - start_before_submit=True) - - @property - def alive(self): - """Accessor to determine if the executor is alive/active.""" - return not self._shutoff - - def shutdown(self, wait=True): - self._shutoff = True - - def restart(self): - """Restarts this executor (*iff* previously shutoff/shutdown). - - NOTE(harlowja): clears any previously gathered statistics. - """ - if self._shutoff: - self._shutoff = False - self._gatherer.clear() - - @property - def statistics(self): - """:class:`.ExecutorStatistics` about the executors executions.""" - return self._gatherer.statistics - - def submit(self, fn, *args, **kwargs): - """Submit some work to be executed (and gather statistics).""" - if self._shutoff: - raise RuntimeError('Can not schedule new futures' - ' after being shutdown') - return self._gatherer.submit(fn, *args, **kwargs) - - def _submit(self, fn, *args, **kwargs): - f = Future() - runner = _WorkItem(f, fn, args, kwargs) - runner.run() - return f - - -class _GreenWorker(object): - def __init__(self, executor, work, work_queue): - self.executor = executor - self.work = work - self.work_queue = work_queue - - def __call__(self): - # Run our main piece of work. - try: - self.work.run() - finally: - # Consume any delayed work before finishing (this is how we finish - # work that was to big for the pool size, but needs to be finished - # no matter). - while True: - try: - w = self.work_queue.get_nowait() - except greenqueue.Empty: - break - else: - try: - w.run() - finally: - self.work_queue.task_done() - - -class GreenFuture(Future): - def __init__(self): - super(GreenFuture, self).__init__() - eu.check_for_eventlet(RuntimeError('Eventlet is needed to use a green' - ' future')) - # NOTE(harlowja): replace the built-in condition with a greenthread - # compatible one so that when getting the result of this future the - # functions will correctly yield to eventlet. If this is not done then - # waiting on the future never actually causes the greenthreads to run - # and thus you wait for infinity. - if not greenpatcher.is_monkey_patched('threading'): - self._condition = greenthreading.Condition() - - -class GreenThreadPoolExecutor(_futures.Executor): - """Executor that uses a green thread pool to execute calls asynchronously. - - See: https://docs.python.org/dev/library/concurrent.futures.html - and http://eventlet.net/doc/modules/greenpool.html for information on - how this works. - - It gathers statistics about the submissions executed for post-analysis... - """ - - def __init__(self, max_workers=1000): - eu.check_for_eventlet(RuntimeError('Eventlet is needed to use a green' - ' executor')) - if max_workers <= 0: - raise ValueError("Max workers must be greater than zero") - self._max_workers = max_workers - self._pool = greenpool.GreenPool(self._max_workers) - self._delayed_work = greenqueue.Queue() - self._shutdown_lock = greenthreading.Lock() - self._shutdown = False - self._gatherer = _Gatherer(self._submit, - lock_cls=greenthreading.Lock) - - @property - def alive(self): - """Accessor to determine if the executor is alive/active.""" - return not self._shutdown - - @property - def statistics(self): - """:class:`.ExecutorStatistics` about the executors executions.""" - return self._gatherer.statistics - - def submit(self, fn, *args, **kwargs): - """Submit some work to be executed (and gather statistics). - - :param args: non-keyworded arguments - :type args: list - :param kwargs: key-value arguments - :type kwargs: dictionary - """ - with self._shutdown_lock: - if self._shutdown: - raise RuntimeError('Can not schedule new futures' - ' after being shutdown') - return self._gatherer.submit(fn, *args, **kwargs) - - def _submit(self, fn, *args, **kwargs): - f = GreenFuture() - work = _WorkItem(f, fn, args, kwargs) - if not self._spin_up(work): - self._delayed_work.put(work) - return f - - def _spin_up(self, work): - """Spin up a greenworker if less than max_workers. - - :param work: work to be given to the greenworker - :returns: whether a green worker was spun up or not - :rtype: boolean - """ - alive = self._pool.running() + self._pool.waiting() - if alive < self._max_workers: - self._pool.spawn_n(_GreenWorker(self, work, self._delayed_work)) - return True - return False - - def shutdown(self, wait=True): - with self._shutdown_lock: - if not self._shutdown: - self._shutdown = True - shutoff = True - else: - shutoff = False - if wait and shutoff: - self._pool.waitall() - self._delayed_work.join() - - -class ExecutorStatistics(object): - """Holds *immutable* information about a executors executions.""" - - __slots__ = ['_failures', '_executed', '_runtime', '_cancelled'] - - __repr_format = ("failures=%(failures)s, executed=%(executed)s, " - "runtime=%(runtime)s, cancelled=%(cancelled)s") - - def __init__(self, failures=0, executed=0, runtime=0.0, cancelled=0): - self._failures = failures - self._executed = executed - self._runtime = runtime - self._cancelled = cancelled - - @property - def failures(self): - """How many submissions ended up raising exceptions. - - :returns: how many submissions ended up raising exceptions - :rtype: number - """ - return self._failures - - @property - def executed(self): - """How many submissions were executed (failed or not). - - :returns: how many submissions were executed - :rtype: number - """ - return self._executed - - @property - def runtime(self): - """Total runtime of all submissions executed (failed or not). - - :returns: total runtime of all submissions executed - :rtype: number - """ - return self._runtime - - @property - def cancelled(self): - """How many submissions were cancelled before executing. - - :returns: how many submissions were cancelled before executing - :rtype: number - """ - return self._cancelled - - @property - def average_runtime(self): - """The average runtime of all submissions executed. - - :returns: average runtime of all submissions executed - :rtype: number - :raises: ZeroDivisionError when no executions have occurred. - """ - return self._runtime / self._executed - - def __repr__(self): - r = reflection.get_class_name(self, fully_qualified=False) - r += "(" - r += self.__repr_format % ({ - 'failures': self._failures, - 'executed': self._executed, - 'runtime': self._runtime, - 'cancelled': self._cancelled, - }) - r += ")" - return r diff --git a/taskflow/types/periodic.py b/taskflow/types/periodic.py deleted file mode 100644 index 872326da..00000000 --- a/taskflow/types/periodic.py +++ /dev/null @@ -1,209 +0,0 @@ -# -*- coding: utf-8 -*- - -# Copyright (C) 2015 Yahoo! Inc. 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. - -import heapq -import inspect -import threading - -from debtcollector import removals -import monotonic -from oslo_utils import reflection -import six - -from taskflow import logging -from taskflow.utils import misc - -LOG = logging.getLogger(__name__) - - -def _check_attrs(obj): - """Checks that a periodic function/method has all the expected attributes. - - This will return the expected attributes that were **not** found. - """ - missing_attrs = [] - for a in ('_periodic', '_periodic_spacing', '_periodic_run_immediately'): - if not hasattr(obj, a): - missing_attrs.append(a) - return missing_attrs - - -def periodic(spacing, run_immediately=True): - """Tags a method/function as wanting/able to execute periodically. - - :param run_immediately: option to specify whether to run - immediately or not - :type run_immediately: boolean - """ - - if spacing <= 0: - raise ValueError("Periodicity/spacing must be greater than" - " zero instead of %s" % spacing) - - def wrapper(f): - f._periodic = True - f._periodic_spacing = spacing - f._periodic_run_immediately = run_immediately - - @six.wraps(f) - def decorator(*args, **kwargs): - return f(*args, **kwargs) - - return decorator - - return wrapper - - -class _Schedule(object): - """Internal heap-based structure that maintains the schedule/ordering.""" - - def __init__(self): - self._ordering = [] - - def push(self, next_run, index): - heapq.heappush(self._ordering, (next_run, index)) - - def push_next(self, cb, index, now=None): - if now is None: - now = monotonic.monotonic() - self.push(now + cb._periodic_spacing, index) - - def __len__(self): - return len(self._ordering) - - def pop(self): - return heapq.heappop(self._ordering) - - -def _build(callables): - schedule = _Schedule() - now = None - immediates = [] - # Reverse order is used since these are later popped off (and to - # ensure the popping order is first -> last we need to append them - # in the opposite ordering last -> first). - for i, cb in misc.reverse_enumerate(callables): - if cb._periodic_run_immediately: - immediates.append(i) - else: - if now is None: - now = monotonic.monotonic() - schedule.push_next(cb, i, now=now) - return immediates, schedule - - -def _safe_call(cb, kind): - try: - cb() - except Exception: - LOG.warn("Failed to call %s '%r'", kind, cb, exc_info=True) - - -class PeriodicWorker(object): - """Calls a collection of callables periodically (sleeping as needed...). - - NOTE(harlowja): typically the :py:meth:`.start` method is executed in a - background thread so that the periodic callables are executed in - the background/asynchronously (using the defined periods to determine - when each is called). - """ - - @classmethod - def create(cls, objects, exclude_hidden=True): - """Automatically creates a worker by analyzing object(s) methods. - - Only picks up methods that have been tagged/decorated with - the :py:func:`.periodic` decorator (does not match against private - or protected methods unless explicitly requested to). - """ - callables = [] - for obj in objects: - for (name, member) in inspect.getmembers(obj): - if name.startswith("_") and exclude_hidden: - continue - if reflection.is_bound_method(member): - missing_attrs = _check_attrs(member) - if not missing_attrs: - callables.append(member) - return cls(callables) - - @removals.removed_kwarg('tombstone', version="0.8", removal_version="?") - def __init__(self, callables, tombstone=None): - if tombstone is None: - self._tombstone = threading.Event() - else: - self._tombstone = tombstone - self._callables = [] - for i, cb in enumerate(callables, 1): - if not six.callable(cb): - raise ValueError("Periodic callback %s must be callable" % i) - missing_attrs = _check_attrs(cb) - if missing_attrs: - raise ValueError("Periodic callback %s missing required" - " attributes %s" % (i, missing_attrs)) - if cb._periodic: - self._callables.append(cb) - self._immediates, self._schedule = _build(self._callables) - - def __len__(self): - return len(self._callables) - - def start(self): - """Starts running (will not return until :py:meth:`.stop` is called). - - NOTE(harlowja): If this worker has no contained callables this raises - a runtime error and does not run since it is impossible to periodically - run nothing. - """ - if not self._callables: - raise RuntimeError("A periodic worker can not start" - " without any callables") - while not self._tombstone.is_set(): - if self._immediates: - # Run & schedule its next execution. - index = self._immediates.pop() - cb = self._callables[index] - LOG.blather("Calling immediate '%r'", cb) - _safe_call(cb, 'immediate') - self._schedule.push_next(cb, index) - else: - # Figure out when we should run next (by selecting the - # minimum item from the heap, where the minimum should be - # the callable that needs to run next and has the lowest - # next desired run time). - now = monotonic.monotonic() - next_run, index = self._schedule.pop() - when_next = next_run - now - if when_next <= 0: - # Run & schedule its next execution. - cb = self._callables[index] - LOG.blather("Calling periodic '%r' (it runs every" - " %s seconds)", cb, cb._periodic_spacing) - _safe_call(cb, 'periodic') - self._schedule.push_next(cb, index, now=now) - else: - # Gotta wait... - self._schedule.push(next_run, index) - self._tombstone.wait(when_next) - - def stop(self): - """Sets the tombstone (this stops any further executions).""" - self._tombstone.set() - - def reset(self): - """Resets the tombstone and re-queues up any immediate executions.""" - self._tombstone.clear() - self._immediates, self._schedule = _build(self._callables) diff --git a/taskflow/utils/async_utils.py b/taskflow/utils/async_utils.py index aec62abf..c4b114b8 100644 --- a/taskflow/utils/async_utils.py +++ b/taskflow/utils/async_utils.py @@ -16,11 +16,11 @@ from concurrent import futures as _futures from concurrent.futures import _base +import futurist from oslo_utils import importutils greenthreading = importutils.try_import('eventlet.green.threading') -from taskflow.types import futures from taskflow.utils import eventlet_utils as eu @@ -32,7 +32,7 @@ _DONE_STATES = frozenset([ def make_completed_future(result): """Make and return a future completed with a given result.""" - future = futures.Future() + future = futurist.Future() future.set_result(result) return future @@ -47,7 +47,10 @@ def wait_for_any(fs, timeout=None): Returns pair (done futures, not done futures). """ - green_fs = sum(1 for f in fs if isinstance(f, futures.GreenFuture)) + # TODO(harlowja): remove this when + # https://review.openstack.org/#/c/196269/ is merged and is made + # available. + green_fs = sum(1 for f in fs if isinstance(f, futurist.GreenFuture)) if not green_fs: return _futures.wait(fs, timeout=timeout, diff --git a/taskflow/utils/threading_utils.py b/taskflow/utils/threading_utils.py index e859fffa..7de0151d 100644 --- a/taskflow/utils/threading_utils.py +++ b/taskflow/utils/threading_utils.py @@ -15,7 +15,6 @@ # under the License. import collections -import multiprocessing import threading import six @@ -36,17 +35,6 @@ def get_ident(): return _thread.get_ident() -def get_optimal_thread_count(): - """Try to guess optimal thread count for current system.""" - try: - return multiprocessing.cpu_count() + 1 - except NotImplementedError: - # NOTE(harlowja): apparently may raise so in this case we will - # just setup two threads since it's hard to know what else we - # should do in this situation. - return 2 - - def daemon_thread(target, *args, **kwargs): """Makes a daemon thread that calls the given target when started.""" thread = threading.Thread(target=target, args=args, kwargs=kwargs) From b8d2a5f18e71a870e36719f60cfc0650a8ea686b Mon Sep 17 00:00:00 2001 From: Doug Hellmann Date: Fri, 10 Jul 2015 13:46:00 +0000 Subject: [PATCH 241/246] Fix mock calls Use the proper assertion methods of the mocks. Change-Id: I7b02f3b52012976a3ed0b8d5fc445653c0546547 --- taskflow/tests/unit/test_engine_helpers.py | 4 ++-- taskflow/tests/unit/test_task.py | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/taskflow/tests/unit/test_engine_helpers.py b/taskflow/tests/unit/test_engine_helpers.py index ed40caa0..64be04b6 100644 --- a/taskflow/tests/unit/test_engine_helpers.py +++ b/taskflow/tests/unit/test_engine_helpers.py @@ -84,7 +84,7 @@ class FlowFromDetailTestCase(test.TestCase): with mock.patch('oslo_utils.importutils.import_class', return_value=lambda: 'RESULT') as mock_import: result = taskflow.engines.flow_from_detail(flow_detail) - mock_import.assert_called_onec_with(name) + mock_import.assert_called_once_with(name) self.assertEqual(result, 'RESULT') def test_factory_with_arg(self): @@ -95,7 +95,7 @@ class FlowFromDetailTestCase(test.TestCase): with mock.patch('oslo_utils.importutils.import_class', return_value=lambda x: 'RESULT %s' % x) as mock_import: result = taskflow.engines.flow_from_detail(flow_detail) - mock_import.assert_called_onec_with(name) + mock_import.assert_called_once_with(name) self.assertEqual(result, 'RESULT foo') diff --git a/taskflow/tests/unit/test_task.py b/taskflow/tests/unit/test_task.py index 2f6d4e97..30474af7 100644 --- a/taskflow/tests/unit/test_task.py +++ b/taskflow/tests/unit/test_task.py @@ -274,7 +274,7 @@ class TaskTest(test.TestCase): a_task = ProgressTask() a_task.notifier.register(task.EVENT_UPDATE_PROGRESS, progress_callback) a_task.execute([0.5]) - mocked_warn.assert_called_once() + self.assertEqual(1, mocked_warn.call_count) def test_register_handler_is_none(self): a_task = MyTask() From d4b153d8ec3d3912372661048fb72e2711109fef Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Fri, 10 Jul 2015 18:05:07 -0700 Subject: [PATCH 242/246] Update the version on the old/deprecated logbook module This version is no longer relevant due to the 1.x change that occurred, so update it to the right version. Change-Id: I50db7a8fb0367978c06e3248a0265ffbd3c6c642 --- taskflow/persistence/logbook.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/taskflow/persistence/logbook.py b/taskflow/persistence/logbook.py index 16273031..829f6bac 100644 --- a/taskflow/persistence/logbook.py +++ b/taskflow/persistence/logbook.py @@ -22,7 +22,7 @@ from taskflow.persistence import models # TODO(harlowja): remove me in a future version, since the models # module is more appropriately named to what the objects in it are used for... removals.removed_module(__name__, replacement="'%s'" % models.__name__, - version="0.11", removal_version='?', + version="1.15", removal_version='2.0', stacklevel=4) From e34dde271e11d3cfff751dab3ccb9d3fdf6f9d75 Mon Sep 17 00:00:00 2001 From: OpenStack Proposal Bot Date: Sun, 12 Jul 2015 15:23:03 +0000 Subject: [PATCH 243/246] Updated from global requirements Change-Id: I98483517bb89ad1438ddda6076ae01ba1ea1c49d --- test-requirements.txt | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/test-requirements.txt b/test-requirements.txt index 805eeb1f..028c9536 100644 --- a/test-requirements.txt +++ b/test-requirements.txt @@ -4,7 +4,8 @@ hacking<0.11,>=0.10.0 oslotest>=1.5.1 # Apache-2.0 -mock>=1.0 +mock>=1.1;python_version!='2.6' +mock==1.0.1;python_version=='2.6' testtools>=1.4.0 testscenarios>=0.4 From b03d524633032854c02345ee514de7a1a17b5c3f Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Sun, 12 Jul 2015 18:08:58 -0700 Subject: [PATCH 244/246] Add deprecated and only alias modules for the moved types Provide modules for the moved futurist types that only provide aliases to the old classes/functions and emit a deprecation warning when imported. This allows for users of those classes/functions to get off that code in a way that will be easily do-able (without totally breaking there code-bases, until we do that in the 2.0 release). Change-Id: If25ac34b437e2a575fe89b98abccbc9a56326d53 --- taskflow/types/futures.py | 36 ++++++++++++++++++++++++++++++++++++ taskflow/types/periodic.py | 32 ++++++++++++++++++++++++++++++++ 2 files changed, 68 insertions(+) create mode 100644 taskflow/types/futures.py create mode 100644 taskflow/types/periodic.py diff --git a/taskflow/types/futures.py b/taskflow/types/futures.py new file mode 100644 index 00000000..7d1f9aaa --- /dev/null +++ b/taskflow/types/futures.py @@ -0,0 +1,36 @@ +# -*- coding: utf-8 -*- + +# Copyright (C) 2015 Yahoo! Inc. 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. + +from debtcollector import removals +import futurist + + +# TODO(harlowja): remove me in a future version, since the futurist +# is the replacement for this whole module... +removals.removed_module(__name__, + replacement="the '%s' library" % futurist.__name__, + version="1.15", removal_version='2.0', + stacklevel=4) + + +# Keep alias classes/functions... around until this module is removed. +Future = futurist.Future +ThreadPoolExecutor = futurist.ThreadPoolExecutor +GreenThreadPoolExecutor = futurist.GreenThreadPoolExecutor +ProcessPoolExecutor = futurist.ProcessPoolExecutor +GreenFuture = futurist.GreenFuture +SynchronousExecutor = futurist.SynchronousExecutor +ExecutorStatistics = futurist.ExecutorStatistics diff --git a/taskflow/types/periodic.py b/taskflow/types/periodic.py new file mode 100644 index 00000000..9f2897ad --- /dev/null +++ b/taskflow/types/periodic.py @@ -0,0 +1,32 @@ +# -*- coding: utf-8 -*- + +# Copyright (C) 2015 Yahoo! Inc. 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. + +from debtcollector import removals +import futurist +from futurist import periodics + + +# TODO(harlowja): remove me in a future version, since the futurist +# is the replacement for this whole module... +removals.removed_module(__name__, + replacement="the '%s' library" % futurist.__name__, + version="1.15", removal_version='2.0', + stacklevel=4) + + +# Keep alias classes/functions... around until this module is removed. +periodic = periodics.periodic +PeriodicWorker = periodics.PeriodicWorker From 9633c5b21787cc50538124e8c1ba0f06ec45c247 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Sun, 12 Jul 2015 18:34:10 -0700 Subject: [PATCH 245/246] Update all removal_version from being ? to being 2.0 Before the 2.0 release it would be great to get these removed so let's mark that version as the version that will no longer have these in it. Change-Id: I66a74d270bf95db005e9febfce1a5e211c7a49f6 --- taskflow/conductors/backends/impl_blocking.py | 2 +- taskflow/conductors/single_threaded.py | 2 +- taskflow/engines/base.py | 4 ++-- taskflow/engines/helpers.py | 2 +- taskflow/listeners/base.py | 4 ++-- taskflow/listeners/timing.py | 4 ++-- taskflow/persistence/backends/impl_memory.py | 2 +- taskflow/utils/misc.py | 4 ++-- 8 files changed, 12 insertions(+), 12 deletions(-) diff --git a/taskflow/conductors/backends/impl_blocking.py b/taskflow/conductors/backends/impl_blocking.py index 26a75c8a..e3f2f5ab 100644 --- a/taskflow/conductors/backends/impl_blocking.py +++ b/taskflow/conductors/backends/impl_blocking.py @@ -83,7 +83,7 @@ class BlockingConductor(base.Conductor): raise ValueError("Invalid timeout literal: %s" % (wait_timeout)) self._dead = threading.Event() - @removals.removed_kwarg('timeout', version="0.8", removal_version="?") + @removals.removed_kwarg('timeout', version="0.8", removal_version="2.0") def stop(self, timeout=None): """Requests the conductor to stop dispatching. diff --git a/taskflow/conductors/single_threaded.py b/taskflow/conductors/single_threaded.py index e1fafcce..6f50fe74 100644 --- a/taskflow/conductors/single_threaded.py +++ b/taskflow/conductors/single_threaded.py @@ -22,7 +22,7 @@ from taskflow.conductors.backends import impl_blocking # TODO(harlowja): remove this module soon... removals.removed_module(__name__, replacement="the conductor entrypoints", - version="0.8", removal_version="?", + version="0.8", removal_version="2.0", stacklevel=4) # TODO(harlowja): remove this proxy/legacy class soon... diff --git a/taskflow/engines/base.py b/taskflow/engines/base.py index f41ae583..4b6a648d 100644 --- a/taskflow/engines/base.py +++ b/taskflow/engines/base.py @@ -55,7 +55,7 @@ class Engine(object): @property @moves.moved_property('atom_notifier', version="0.6", - removal_version="?") + removal_version="2.0") def task_notifier(self): """The task notifier. @@ -128,4 +128,4 @@ class Engine(object): # TODO(harlowja): remove in 0.7 or later... EngineBase = moves.moved_class(Engine, 'EngineBase', __name__, - version="0.6", removal_version="?") + version="0.6", removal_version="2.0") diff --git a/taskflow/engines/helpers.py b/taskflow/engines/helpers.py index a9171e46..679b65ea 100644 --- a/taskflow/engines/helpers.py +++ b/taskflow/engines/helpers.py @@ -92,7 +92,7 @@ def _extract_engine(**kwargs): stacklevel = sum(1 for _frame in finder) decorator = renames.renamed_kwarg('engine_conf', 'engine', version="0.6", - removal_version="?", + removal_version="2.0", # Three is added on since the # decorator adds three of its own # stack levels that we need to diff --git a/taskflow/listeners/base.py b/taskflow/listeners/base.py index a83cd79b..8a0badb0 100644 --- a/taskflow/listeners/base.py +++ b/taskflow/listeners/base.py @@ -166,7 +166,7 @@ class Listener(object): # TODO(harlowja): remove in 0.7 or later... ListenerBase = moves.moved_class(Listener, 'ListenerBase', __name__, - version="0.6", removal_version="?") + version="0.6", removal_version="2.0") @six.add_metaclass(abc.ABCMeta) @@ -213,7 +213,7 @@ class DumpingListener(Listener): # TODO(harlowja): remove in 0.7 or later... class LoggingBase(moves.moved_class(DumpingListener, 'LoggingBase', __name__, - version="0.6", removal_version="?")): + version="0.6", removal_version="2.0")): """Legacy logging base. diff --git a/taskflow/listeners/timing.py b/taskflow/listeners/timing.py index e346f08f..be3b74b3 100644 --- a/taskflow/listeners/timing.py +++ b/taskflow/listeners/timing.py @@ -91,7 +91,7 @@ class DurationListener(base.Listener): TimingListener = moves.moved_class(DurationListener, 'TimingListener', __name__, - version="0.8", removal_version="?") + version="0.8", removal_version="2.0") class PrintingDurationListener(DurationListener): @@ -117,7 +117,7 @@ class PrintingDurationListener(DurationListener): PrintingTimingListener = moves.moved_class( PrintingDurationListener, 'PrintingTimingListener', __name__, - version="0.8", removal_version="?") + version="0.8", removal_version="2.0") class EventTimeListener(base.Listener): diff --git a/taskflow/persistence/backends/impl_memory.py b/taskflow/persistence/backends/impl_memory.py index 37a67d06..a056856d 100644 --- a/taskflow/persistence/backends/impl_memory.py +++ b/taskflow/persistence/backends/impl_memory.py @@ -193,7 +193,7 @@ class FakeFilesystem(object): return [selector_func(node, child_node) for child_node in node.bfs_iter()] - @removals.removed_kwarg('recursive', version="0.11", removal_version="?") + @removals.removed_kwarg('recursive', version="0.11", removal_version="2.0") def ls(self, path, recursive=False): """Return list of all children of the given path. diff --git a/taskflow/utils/misc.py b/taskflow/utils/misc.py index 87f68355..bd2e6e38 100644 --- a/taskflow/utils/misc.py +++ b/taskflow/utils/misc.py @@ -505,12 +505,12 @@ def ensure_tree(path): Failure = deprecation.moved_proxy_class(failure.Failure, 'Failure', __name__, - version="0.6", removal_version="?") + version="0.6", removal_version="2.0") Notifier = deprecation.moved_proxy_class(notifier.Notifier, 'Notifier', __name__, - version="0.6", removal_version="?") + version="0.6", removal_version="2.0") @contextlib.contextmanager From 9478226141ad148fe55e40d28de1721f048f6c05 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Fri, 26 Jun 2015 16:08:19 -0700 Subject: [PATCH 246/246] Provide a deprecated alias for the now removed stop watch class This allows those who were using it to still continue using it until 2.0 where it will be removed; this makes it possible for those users to get off that code in a way that will be easily do-able (without totally breaking there code-bases, until we do that in the 2.0 release). This also removes all internal usage of that stop watch class so that the library itself will not reference it anymore. Change-Id: If313d8e7b9bdc8741db2e2e1dfb381aa3260b971 --- taskflow/engines/action_engine/executor.py | 5 ++--- taskflow/engines/worker_based/protocol.py | 3 +-- taskflow/engines/worker_based/server.py | 4 ++-- taskflow/engines/worker_based/types.py | 4 ++-- taskflow/jobs/backends/impl_redis.py | 3 +-- taskflow/jobs/backends/impl_zookeeper.py | 4 ++-- taskflow/listeners/timing.py | 4 ++-- taskflow/types/latch.py | 4 ++-- taskflow/types/timing.py | 8 ++++---- tools/speed_test.py | 4 ++-- 10 files changed, 20 insertions(+), 23 deletions(-) diff --git a/taskflow/engines/action_engine/executor.py b/taskflow/engines/action_engine/executor.py index 6f2e8b82..f03aa8e2 100644 --- a/taskflow/engines/action_engine/executor.py +++ b/taskflow/engines/action_engine/executor.py @@ -33,7 +33,6 @@ from taskflow import logging from taskflow import task as task_atom from taskflow.types import failure from taskflow.types import notifier -from taskflow.types import timing from taskflow.utils import async_utils from taskflow.utils import threading_utils @@ -176,7 +175,7 @@ class _WaitWorkItem(object): 'kind': _KIND_COMPLETE_ME, } if self._channel.put(message): - watch = timing.StopWatch() + watch = timeutils.StopWatch() watch.start() self._barrier.wait() LOG.blather("Waited %s seconds until task '%s' %s emitted" @@ -305,7 +304,7 @@ class _Dispatcher(object): " %s to target '%s'", kind, sender, target) def run(self, queue): - watch = timing.StopWatch(duration=self._dispatch_periodicity) + watch = timeutils.StopWatch(duration=self._dispatch_periodicity) while (not self._dead.is_set() or (self._stop_when_empty and self._targets)): watch.restart() diff --git a/taskflow/engines/worker_based/protocol.py b/taskflow/engines/worker_based/protocol.py index 823f83c9..44913064 100644 --- a/taskflow/engines/worker_based/protocol.py +++ b/taskflow/engines/worker_based/protocol.py @@ -28,7 +28,6 @@ from taskflow.engines.action_engine import executor from taskflow import exceptions as excp from taskflow import logging from taskflow.types import failure as ft -from taskflow.types import timing as tt from taskflow.utils import schema_utils as su # NOTE(skudriashev): This is protocol states and events, which are not @@ -239,7 +238,7 @@ class Request(Message): self._event = ACTION_TO_EVENT[action] self._arguments = arguments self._kwargs = kwargs - self._watch = tt.StopWatch(duration=timeout).start() + self._watch = timeutils.StopWatch(duration=timeout).start() self._state = WAITING self._lock = threading.Lock() self._created_on = timeutils.utcnow() diff --git a/taskflow/engines/worker_based/server.py b/taskflow/engines/worker_based/server.py index 033bd3c7..f1bd0fcd 100644 --- a/taskflow/engines/worker_based/server.py +++ b/taskflow/engines/worker_based/server.py @@ -17,6 +17,7 @@ import functools from oslo_utils import reflection +from oslo_utils import timeutils from taskflow.engines.worker_based import dispatcher from taskflow.engines.worker_based import protocol as pr @@ -24,7 +25,6 @@ from taskflow.engines.worker_based import proxy from taskflow import logging from taskflow.types import failure as ft from taskflow.types import notifier as nt -from taskflow.types import timing as tt from taskflow.utils import kombu_utils as ku from taskflow.utils import misc @@ -76,7 +76,7 @@ class Server(object): def _on_receive(content, message): LOG.debug("Submitting message '%s' for execution in the" " future to '%s'", ku.DelayedPretty(message), func_name) - watch = tt.StopWatch() + watch = timeutils.StopWatch() watch.start() try: self._executor.submit(_on_run, watch, content, message) diff --git a/taskflow/engines/worker_based/types.py b/taskflow/engines/worker_based/types.py index 5d212e5c..09a41ab3 100644 --- a/taskflow/engines/worker_based/types.py +++ b/taskflow/engines/worker_based/types.py @@ -22,6 +22,7 @@ import threading from futurist import periodics from oslo_utils import reflection +from oslo_utils import timeutils import six from taskflow.engines.worker_based import dispatcher @@ -29,7 +30,6 @@ from taskflow.engines.worker_based import protocol as pr from taskflow import logging from taskflow.types import cache as base from taskflow.types import notifier -from taskflow.types import timing as tt from taskflow.utils import kombu_utils as ku LOG = logging.getLogger(__name__) @@ -123,7 +123,7 @@ class WorkerFinder(object): """ if workers <= 0: raise ValueError("Worker amount must be greater than zero") - watch = tt.StopWatch(duration=timeout) + watch = timeutils.StopWatch(duration=timeout) watch.start() with self._cond: while self._total_workers() < workers: diff --git a/taskflow/jobs/backends/impl_redis.py b/taskflow/jobs/backends/impl_redis.py index 4d61dc01..5d5e7255 100644 --- a/taskflow/jobs/backends/impl_redis.py +++ b/taskflow/jobs/backends/impl_redis.py @@ -34,7 +34,6 @@ from taskflow import exceptions as exc from taskflow.jobs import base from taskflow import logging from taskflow import states -from taskflow.types import timing from taskflow.utils import misc from taskflow.utils import redis_utils as ru @@ -741,7 +740,7 @@ return cmsgpack.pack(result) # up to the provided max-delay. In the future we could try having # a secondary client connected into redis pubsub and use that # instead, but for now this is simpler. - w = timing.StopWatch(duration=timeout) + w = timeutils.StopWatch(duration=timeout) w.start() delay = initial_delay while True: diff --git a/taskflow/jobs/backends/impl_zookeeper.py b/taskflow/jobs/backends/impl_zookeeper.py index 9df706e7..34a87d5a 100644 --- a/taskflow/jobs/backends/impl_zookeeper.py +++ b/taskflow/jobs/backends/impl_zookeeper.py @@ -27,6 +27,7 @@ from kazoo.protocol import paths as k_paths from kazoo.recipe import watchers from oslo_serialization import jsonutils from oslo_utils import excutils +from oslo_utils import timeutils from oslo_utils import uuidutils import six @@ -34,7 +35,6 @@ from taskflow import exceptions as excp from taskflow.jobs import base from taskflow import logging from taskflow import states -from taskflow.types import timing as tt from taskflow.utils import kazoo_utils from taskflow.utils import misc @@ -672,7 +672,7 @@ class ZookeeperJobBoard(base.NotifyingJobBoard): def wait(self, timeout=None): # Wait until timeout expires (or forever) for jobs to appear. - watch = tt.StopWatch(duration=timeout) + watch = timeutils.StopWatch(duration=timeout) watch.start() with self._job_cond: while True: diff --git a/taskflow/listeners/timing.py b/taskflow/listeners/timing.py index e346f08f..d91ff007 100644 --- a/taskflow/listeners/timing.py +++ b/taskflow/listeners/timing.py @@ -20,12 +20,12 @@ import itertools import time from debtcollector import moves +from oslo_utils import timeutils from taskflow import exceptions as exc from taskflow.listeners import base from taskflow import logging from taskflow import states -from taskflow.types import timing as tt STARTING_STATES = frozenset((states.RUNNING, states.REVERTING)) FINISHED_STATES = frozenset((base.FINISH_STATES + (states.REVERTED,))) @@ -81,7 +81,7 @@ class DurationListener(base.Listener): if state == states.PENDING: self._timers.pop(task_name, None) elif state in STARTING_STATES: - self._timers[task_name] = tt.StopWatch().start() + self._timers[task_name] = timeutils.StopWatch().start() elif state in FINISHED_STATES: timer = self._timers.pop(task_name, None) if timer is not None: diff --git a/taskflow/types/latch.py b/taskflow/types/latch.py index 07783309..160df511 100644 --- a/taskflow/types/latch.py +++ b/taskflow/types/latch.py @@ -16,7 +16,7 @@ import threading -from taskflow.types import timing as tt +from oslo_utils import timeutils class Latch(object): @@ -55,7 +55,7 @@ class Latch(object): timeout expires otherwise false :rtype: boolean """ - watch = tt.StopWatch(duration=timeout) + watch = timeutils.StopWatch(duration=timeout) watch.start() with self._cond: while self._count > 0: diff --git a/taskflow/types/timing.py b/taskflow/types/timing.py index 57fe1287..2fa7d20a 100644 --- a/taskflow/types/timing.py +++ b/taskflow/types/timing.py @@ -16,13 +16,13 @@ import threading +from debtcollector import moves from oslo_utils import timeutils -#: Moved to oslo.utils (just reference them from there until a later time). -Split = timeutils.Split -#: Moved to oslo.utils (just reference them from there until a later time). -StopWatch = timeutils.StopWatch +# TODO(harlowja): Keep alias class... around until 2.0 is released. +StopWatch = moves.moved_class(timeutils.StopWatch, 'StopWatch', __name__, + version="1.15", removal_version="2.0") class Timeout(object): diff --git a/tools/speed_test.py b/tools/speed_test.py index 45bca783..f9da37ac 100644 --- a/tools/speed_test.py +++ b/tools/speed_test.py @@ -20,13 +20,13 @@ import argparse import cProfile as profiler import pstats +from oslo_utils import timeutils import six from six.moves import range as compat_range from taskflow import engines from taskflow.patterns import linear_flow as lf from taskflow import task -from taskflow.types import timing def print_header(name): @@ -68,7 +68,7 @@ class ProfileIt(object): class TimeIt(object): def __init__(self, name, args): - self.watch = timing.StopWatch() + self.watch = timeutils.StopWatch() self.name = name self.args = args