Create and use a serial retry executor

To make it easily possible to change the retry
atom execution from being in the engine thread this
creates a retry executor (which is similar to the task
executor) and provide that a serial executor (which it will
use to execute with). This makes the retry and task actions
closer to being the same and makes the surrounding code that
much similar (which makes understanding it easier).

Change-Id: I993e938280df3bd97f8076293183ef21989e2dba
This commit is contained in:
Joshua Harlow
2015-06-25 16:02:21 -07:00
parent 592b468416
commit 359cc490bd
12 changed files with 120 additions and 101 deletions

View File

@@ -37,3 +37,19 @@ class Action(object):
def __init__(self, storage, notifier): def __init__(self, storage, notifier):
self._storage = storage self._storage = storage
self._notifier = notifier self._notifier = notifier
@abc.abstractmethod
def schedule_execution(self, atom):
"""Schedules atom execution."""
@abc.abstractmethod
def schedule_reversion(self, atom):
"""Schedules atom reversion."""
@abc.abstractmethod
def complete_reversion(self, atom, result):
"""Completes atom reversion."""
@abc.abstractmethod
def complete_execution(self, atom, result):
"""Completes atom execution."""

View File

@@ -14,10 +14,7 @@
# License for the specific language governing permissions and limitations # License for the specific language governing permissions and limitations
# under the License. # under the License.
import futurist
from taskflow.engines.action_engine.actions import base from taskflow.engines.action_engine.actions import base
from taskflow.engines.action_engine import executor as ex
from taskflow import logging from taskflow import logging
from taskflow import retry as retry_atom from taskflow import retry as retry_atom
from taskflow import states from taskflow import states
@@ -26,28 +23,12 @@ from taskflow.types import failure
LOG = logging.getLogger(__name__) LOG = logging.getLogger(__name__)
def _execute_retry(retry, arguments):
try:
result = retry.execute(**arguments)
except Exception:
result = failure.Failure()
return (ex.EXECUTED, result)
def _revert_retry(retry, arguments):
try:
result = retry.revert(**arguments)
except Exception:
result = failure.Failure()
return (ex.REVERTED, result)
class RetryAction(base.Action): class RetryAction(base.Action):
"""An action that handles executing, state changes, ... of retry atoms.""" """An action that handles executing, state changes, ... of retry atoms."""
def __init__(self, storage, notifier): def __init__(self, storage, notifier, retry_executor):
super(RetryAction, self).__init__(storage, notifier) super(RetryAction, self).__init__(storage, notifier)
self._executor = futurist.SynchronousExecutor() self._retry_executor = retry_executor
def _get_retry_args(self, retry, addons=None): def _get_retry_args(self, retry, addons=None):
arguments = self._storage.fetch_mapped_args( arguments = self._storage.fetch_mapped_args(
@@ -88,41 +69,30 @@ class RetryAction(base.Action):
details['result'] = result details['result'] = result
self._notifier.notify(state, details) self._notifier.notify(state, details)
def execute(self, retry): def schedule_execution(self, retry):
def _on_done_callback(fut):
result = fut.result()[-1]
if isinstance(result, failure.Failure):
self.change_state(retry, states.FAILURE, result=result)
else:
self.change_state(retry, states.SUCCESS, result=result)
self.change_state(retry, states.RUNNING) self.change_state(retry, states.RUNNING)
fut = self._executor.submit(_execute_retry, retry, return self._retry_executor.execute_retry(
self._get_retry_args(retry)) retry, self._get_retry_args(retry))
fut.add_done_callback(_on_done_callback)
fut.atom = retry
return fut
def revert(self, retry): def complete_reversion(self, retry, result):
def _on_done_callback(fut):
result = fut.result()[-1]
if isinstance(result, failure.Failure): if isinstance(result, failure.Failure):
self.change_state(retry, states.REVERT_FAILURE, result=result) self.change_state(retry, states.REVERT_FAILURE, result=result)
else: else:
self.change_state(retry, states.REVERTED, result=result) self.change_state(retry, states.REVERTED, result=result)
def complete_execution(self, retry, result):
if isinstance(result, failure.Failure):
self.change_state(retry, states.FAILURE, result=result)
else:
self.change_state(retry, states.SUCCESS, result=result)
def schedule_reversion(self, retry):
self.change_state(retry, states.REVERTING) self.change_state(retry, states.REVERTING)
arg_addons = { arg_addons = {
retry_atom.REVERT_FLOW_FAILURES: self._storage.get_failures(), retry_atom.REVERT_FLOW_FAILURES: self._storage.get_failures(),
} }
fut = self._executor.submit(_revert_retry, retry, return self._retry_executor.revert_retry(
self._get_retry_args(retry, retry, self._get_retry_args(retry, addons=arg_addons))
addons=arg_addons))
fut.add_done_callback(_on_done_callback)
fut.atom = retry
return fut
def on_failure(self, retry, atom, last_failure): def on_failure(self, retry, atom, last_failure):
self._storage.save_retry_failure(retry.name, atom.name, last_failure) self._storage.save_retry_failure(retry.name, atom.name, last_failure)

View File

@@ -134,10 +134,9 @@ class TaskAction(base.Action):
task) task)
else: else:
progress_callback = None progress_callback = None
future = self._task_executor.revert_task( return self._task_executor.revert_task(
task, task_uuid, arguments, task_result, failures, task, task_uuid, arguments, task_result, failures,
progress_callback=progress_callback) progress_callback=progress_callback)
return future
def complete_reversion(self, task, result): def complete_reversion(self, task, result):
if isinstance(result, failure.Failure): if isinstance(result, failure.Failure):
@@ -145,6 +144,3 @@ class TaskAction(base.Action):
else: else:
self.change_state(task, states.REVERTED, progress=1.0, self.change_state(task, states.REVERTED, progress=1.0,
result=result) result=result)
def wait_for_any(self, fs, timeout):
return self._task_executor.wait_for_any(fs, timeout)

View File

@@ -106,9 +106,9 @@ class Completer(object):
def __init__(self, runtime): def __init__(self, runtime):
self._runtime = weakref.proxy(runtime) self._runtime = weakref.proxy(runtime)
self._analyzer = runtime.analyzer self._analyzer = runtime.analyzer
self._retry_action = runtime.retry_action
self._storage = runtime.storage self._storage = runtime.storage
self._task_action = runtime.task_action self._task_action = runtime.task_action
self._retry_action = runtime.retry_action
self._undefined_resolver = RevertAll(self._runtime) self._undefined_resolver = RevertAll(self._runtime)
def _complete_task(self, task, event, result): def _complete_task(self, task, event, result):
@@ -118,6 +118,13 @@ class Completer(object):
else: else:
self._task_action.complete_reversion(task, result) self._task_action.complete_reversion(task, result)
def _complete_retry(self, retry, event, result):
"""Completes the given retry, processes retry failure."""
if event == ex.EXECUTED:
self._retry_action.complete_execution(retry, result)
else:
self._retry_action.complete_reversion(retry, result)
def resume(self): def resume(self):
"""Resumes nodes in the contained graph. """Resumes nodes in the contained graph.
@@ -148,6 +155,8 @@ class Completer(object):
""" """
if isinstance(node, task_atom.BaseTask): if isinstance(node, task_atom.BaseTask):
self._complete_task(node, event, result) self._complete_task(node, event, result)
else:
self._complete_retry(node, event, result)
if isinstance(result, failure.Failure): if isinstance(result, failure.Failure):
if event == ex.EXECUTED: if event == ex.EXECUTED:
self._process_atom_failure(node, result) self._process_atom_failure(node, result)

View File

@@ -41,13 +41,17 @@ LOG = logging.getLogger(__name__)
@contextlib.contextmanager @contextlib.contextmanager
def _start_stop(executor): def _start_stop(task_executor, retry_executor):
# A teenie helper context manager to safely start/stop a executor... # A teenie helper context manager to safely start/stop engine executors...
executor.start() task_executor.start()
try: try:
yield executor retry_executor.start()
try:
yield (task_executor, retry_executor)
finally: finally:
executor.stop() retry_executor.stop()
finally:
task_executor.stop()
class ActionEngine(base.Engine): class ActionEngine(base.Engine):
@@ -82,6 +86,9 @@ class ActionEngine(base.Engine):
self._lock = threading.RLock() self._lock = threading.RLock()
self._state_lock = threading.RLock() self._state_lock = threading.RLock()
self._storage_ensured = False self._storage_ensured = False
# Retries are not *currently* executed out of the engines process
# or thread (this could change in the future if we desire it to).
self._retry_executor = executor.SerialRetryExecutor()
def _check(self, name, check_compiled, check_storage_ensured): def _check(self, name, check_compiled, check_storage_ensured):
"""Check (and raise) if the engine has not reached a certain stage.""" """Check (and raise) if the engine has not reached a certain stage."""
@@ -167,7 +174,7 @@ class ActionEngine(base.Engine):
self.validate() self.validate()
runner = self._runtime.runner runner = self._runtime.runner
last_state = None last_state = None
with _start_stop(self._task_executor): with _start_stop(self._task_executor, self._retry_executor):
self._change_state(states.RUNNING) self._change_state(states.RUNNING)
try: try:
closed = False closed = False
@@ -294,7 +301,8 @@ class ActionEngine(base.Engine):
self._runtime = runtime.Runtime(self._compilation, self._runtime = runtime.Runtime(self._compilation,
self.storage, self.storage,
self.atom_notifier, self.atom_notifier,
self._task_executor) self._task_executor,
self._retry_executor)
self._runtime.compile() self._runtime.compile()
self._compiled = True self._compiled = True

View File

@@ -33,7 +33,6 @@ from taskflow import logging
from taskflow import task as task_atom from taskflow import task as task_atom
from taskflow.types import failure from taskflow.types import failure
from taskflow.types import notifier from taskflow.types import notifier
from taskflow.utils import async_utils
from taskflow.utils import threading_utils from taskflow.utils import threading_utils
# Execution and reversion events. # Execution and reversion events.
@@ -58,6 +57,22 @@ _KIND_EVENT = 'event'
LOG = logging.getLogger(__name__) LOG = logging.getLogger(__name__)
def _execute_retry(retry, arguments):
try:
result = retry.execute(**arguments)
except Exception:
result = failure.Failure()
return (EXECUTED, result)
def _revert_retry(retry, arguments):
try:
result = retry.revert(**arguments)
except Exception:
result = failure.Failure()
return (REVERTED, result)
def _execute_task(task, arguments, progress_callback=None): def _execute_task(task, arguments, progress_callback=None):
with notifier.register_deregister(task.notifier, with notifier.register_deregister(task.notifier,
_UPDATE_PROGRESS, _UPDATE_PROGRESS,
@@ -322,6 +337,33 @@ class _Dispatcher(object):
self._dead.wait(leftover) self._dead.wait(leftover)
class SerialRetryExecutor(object):
"""Executes and reverts retries."""
def __init__(self):
self._executor = futurist.SynchronousExecutor()
def start(self):
"""Prepare to execute retries."""
self._executor.restart()
def stop(self):
"""Finalize retry executor."""
self._executor.shutdown()
def execute_retry(self, retry, arguments):
"""Schedules retry execution."""
fut = self._executor.submit(_execute_retry, retry, arguments)
fut.atom = retry
return fut
def revert_retry(self, retry, arguments):
"""Schedules retry reversion."""
fut = self._executor.submit(_revert_retry, retry, arguments)
fut.atom = retry
return fut
@six.add_metaclass(abc.ABCMeta) @six.add_metaclass(abc.ABCMeta)
class TaskExecutor(object): class TaskExecutor(object):
"""Executes and reverts tasks. """Executes and reverts tasks.
@@ -341,10 +383,6 @@ class TaskExecutor(object):
progress_callback=None): progress_callback=None):
"""Schedules task reversion.""" """Schedules task reversion."""
def wait_for_any(self, fs, timeout=None):
"""Wait for futures returned by this executor to complete."""
return async_utils.wait_for_any(fs, timeout=timeout)
def start(self): def start(self):
"""Prepare to execute tasks.""" """Prepare to execute tasks."""

View File

@@ -117,7 +117,6 @@ class Runner(object):
# Cache some local functions/methods... # Cache some local functions/methods...
do_schedule = self._scheduler.schedule do_schedule = self._scheduler.schedule
wait_for_any = self._waiter.wait_for_any
do_complete = self._completer.complete do_complete = self._completer.complete
def iter_next_nodes(target_node=None): def iter_next_nodes(target_node=None):
@@ -171,7 +170,7 @@ class Runner(object):
# call sometime in the future, or equivalent that will work in # call sometime in the future, or equivalent that will work in
# py2 and py3. # py2 and py3.
if memory.not_done: if memory.not_done:
done, not_done = wait_for_any(memory.not_done, timeout) done, not_done = self._waiter(memory.not_done, timeout=timeout)
memory.done.update(done) memory.done.update(done)
memory.not_done = not_done memory.not_done = not_done
return _ANALYZE return _ANALYZE

View File

@@ -26,6 +26,7 @@ from taskflow.engines.action_engine import scopes as sc
from taskflow import flow as flow_type from taskflow import flow as flow_type
from taskflow import states as st from taskflow import states as st
from taskflow import task from taskflow import task
from taskflow.utils import async_utils
from taskflow.utils import misc from taskflow.utils import misc
@@ -37,9 +38,11 @@ class Runtime(object):
action engine to run to completion. action engine to run to completion.
""" """
def __init__(self, compilation, storage, atom_notifier, task_executor): def __init__(self, compilation, storage, atom_notifier,
task_executor, retry_executor):
self._atom_notifier = atom_notifier self._atom_notifier = atom_notifier
self._task_executor = task_executor self._task_executor = task_executor
self._retry_executor = retry_executor
self._storage = storage self._storage = storage
self._compilation = compilation self._compilation = compilation
self._atom_cache = {} self._atom_cache = {}
@@ -111,7 +114,7 @@ class Runtime(object):
@misc.cachedproperty @misc.cachedproperty
def runner(self): def runner(self):
return ru.Runner(self, self._task_executor) return ru.Runner(self, async_utils.wait_for_any)
@misc.cachedproperty @misc.cachedproperty
def completer(self): def completer(self):
@@ -132,7 +135,8 @@ class Runtime(object):
@misc.cachedproperty @misc.cachedproperty
def retry_action(self): def retry_action(self):
return ra.RetryAction(self._storage, return ra.RetryAction(self._storage,
self._atom_notifier) self._atom_notifier,
self._retry_executor)
@misc.cachedproperty @misc.cachedproperty
def task_action(self): def task_action(self):

View File

@@ -37,13 +37,13 @@ class RetryScheduler(object):
""" """
intention = self._storage.get_atom_intention(retry.name) intention = self._storage.get_atom_intention(retry.name)
if intention == st.EXECUTE: if intention == st.EXECUTE:
return self._retry_action.execute(retry) return self._retry_action.schedule_execution(retry)
elif intention == st.REVERT: elif intention == st.REVERT:
return self._retry_action.revert(retry) return self._retry_action.schedule_reversion(retry)
elif intention == st.RETRY: elif intention == st.RETRY:
self._retry_action.change_state(retry, st.RETRYING) self._retry_action.change_state(retry, st.RETRYING)
self._runtime.retry_subflow(retry) self._runtime.retry_subflow(retry)
return self._retry_action.execute(retry) return self._retry_action.schedule_execution(retry)
else: else:
raise excp.ExecutionFailure("Unknown how to schedule retry with" raise excp.ExecutionFailure("Unknown how to schedule retry with"
" intention: %s" % intention) " intention: %s" % intention)

View File

@@ -42,10 +42,12 @@ class _RunnerTestMixin(object):
store.set_flow_state(initial_state) store.set_flow_state(initial_state)
task_notifier = notifier.Notifier() task_notifier = notifier.Notifier()
task_executor = executor.SerialTaskExecutor() task_executor = executor.SerialTaskExecutor()
retry_executor = executor.SerialRetryExecutor()
task_executor.start() task_executor.start()
self.addCleanup(task_executor.stop) self.addCleanup(task_executor.stop)
r = runtime.Runtime(compilation, store, r = runtime.Runtime(compilation, store,
task_notifier, task_executor) task_notifier, task_executor,
retry_executor)
r.compile() r.compile()
return r return r

View File

@@ -17,7 +17,6 @@
import threading import threading
import time import time
import futurist
from oslo_utils import fixture from oslo_utils import fixture
from oslo_utils import timeutils from oslo_utils import timeutils
@@ -58,8 +57,6 @@ class TestWorkerTaskExecutor(test.MockTestCase):
self.request_inst_mock.uuid = self.task_uuid self.request_inst_mock.uuid = self.task_uuid
self.request_inst_mock.expired = False self.request_inst_mock.expired = False
self.request_inst_mock.task_cls = self.task.name self.request_inst_mock.task_cls = self.task.name
self.wait_for_any_mock = self.patch(
'taskflow.engines.action_engine.executor.async_utils.wait_for_any')
self.message_mock = mock.MagicMock(name='message') self.message_mock = mock.MagicMock(name='message')
self.message_mock.properties = {'correlation_id': self.task_uuid, self.message_mock.properties = {'correlation_id': self.task_uuid,
'type': pr.RESPONSE} 'type': pr.RESPONSE}
@@ -281,27 +278,6 @@ class TestWorkerTaskExecutor(test.MockTestCase):
] ]
self.assertEqual(expected_calls, self.master_mock.mock_calls) self.assertEqual(expected_calls, self.master_mock.mock_calls)
def test_wait_for_any(self):
fs = [futurist.Future(), futurist.Future()]
ex = self.executor()
ex.wait_for_any(fs)
expected_calls = [
mock.call(fs, timeout=None)
]
self.assertEqual(self.wait_for_any_mock.mock_calls, expected_calls)
def test_wait_for_any_with_timeout(self):
timeout = 30
fs = [futurist.Future(), futurist.Future()]
ex = self.executor()
ex.wait_for_any(fs, timeout)
master_mock_calls = [
mock.call(fs, timeout=timeout)
]
self.assertEqual(self.wait_for_any_mock.mock_calls, master_mock_calls)
def test_start_stop(self): def test_start_stop(self):
ex = self.executor() ex = self.executor()
ex.start() ex.start()

View File

@@ -24,6 +24,7 @@ from taskflow.engines.worker_based import server as worker_server
from taskflow import test from taskflow import test
from taskflow.tests import utils as test_utils from taskflow.tests import utils as test_utils
from taskflow.types import failure from taskflow.types import failure
from taskflow.utils import async_utils
from taskflow.utils import threading_utils from taskflow.utils import threading_utils
@@ -77,7 +78,7 @@ class TestPipeline(test.TestCase):
progress_callback = lambda *args, **kwargs: None progress_callback = lambda *args, **kwargs: None
f = executor.execute_task(t, uuidutils.generate_uuid(), {}, f = executor.execute_task(t, uuidutils.generate_uuid(), {},
progress_callback=progress_callback) progress_callback=progress_callback)
executor.wait_for_any([f]) async_utils.wait_for_any([f])
event, result = f.result() event, result = f.result()
self.assertEqual(1, result) self.assertEqual(1, result)
@@ -93,7 +94,7 @@ class TestPipeline(test.TestCase):
progress_callback = lambda *args, **kwargs: None progress_callback = lambda *args, **kwargs: None
f = executor.execute_task(t, uuidutils.generate_uuid(), {}, f = executor.execute_task(t, uuidutils.generate_uuid(), {},
progress_callback=progress_callback) progress_callback=progress_callback)
executor.wait_for_any([f]) async_utils.wait_for_any([f])
action, result = f.result() action, result = f.result()
self.assertIsInstance(result, failure.Failure) self.assertIsInstance(result, failure.Failure)