Upgrade hacking version and fix some of the issues

Update hacking to the new requirements version and
fix about half of the new reported issues. The other
hacking issues are for now ignored until fixed by
adjusting our tox.ini file.

This commit fixes the following new hacking errors:

H405 - multi line docstring summary not separated
       with an empty line
E265 - block comment should start with '# '
F402 - import 'endpoint' from line 21 shadowed by
       loop variable

Change-Id: I6bae61591fb988cc17fa79e21cb5f1508d22781c
This commit is contained in:
Joshua Harlow
2014-06-10 17:04:15 -07:00
committed by Joshua Harlow
parent 89c4a30fd1
commit c558da07b6
44 changed files with 526 additions and 290 deletions

View File

@@ -78,7 +78,9 @@ def _build_rebind_dict(args, rebind_args):
def _build_arg_mapping(atom_name, reqs, rebind_args, function, do_infer, def _build_arg_mapping(atom_name, reqs, rebind_args, function, do_infer,
ignore_list=None): ignore_list=None):
"""Given a function, its requirements and a rebind mapping this helper """Builds an input argument mapping for a given function.
Given a function, its requirements and a rebind mapping this helper
function will build the correct argument mapping for the given function as function will build the correct argument mapping for the given function as
well as verify that the final argument mapping does not have missing or well as verify that the final argument mapping does not have missing or
extra arguments (where applicable). extra arguments (where applicable).

View File

@@ -24,7 +24,9 @@ from taskflow.utils import lock_utils
@six.add_metaclass(abc.ABCMeta) @six.add_metaclass(abc.ABCMeta)
class Conductor(object): class Conductor(object):
"""Conductors act as entities which extract jobs from a jobboard, assign """Conductors conduct jobs & assist in associated runtime interactions.
Conductors act as entities which extract jobs from a jobboard, assign
there work to some engine (using some desired configuration) and then wait there work to some engine (using some desired configuration) and then wait
for that work to complete. If the work fails then they abandon the claimed for that work to complete. If the work fails then they abandon the claimed
work (or if the process they are running in crashes or dies this work (or if the process they are running in crashes or dies this
@@ -99,13 +101,13 @@ class Conductor(object):
@abc.abstractmethod @abc.abstractmethod
def run(self): def run(self):
"""Continuously claims, runs, and consumes jobs, and waits for more """Continuously claims, runs, and consumes jobs (and repeat)."""
jobs when there are none left on the jobboard.
"""
@abc.abstractmethod @abc.abstractmethod
def _dispatch_job(self, job): def _dispatch_job(self, job):
"""Accepts a single (already claimed) job and causes it to be run in """Dispatches a claimed job for work completion.
Accepts a single (already claimed) job and causes it to be run in
an engine. Returns a boolean that signifies whether the job should an engine. Returns a boolean that signifies whether the job should
be consumed. The job is consumed upon completion (unless False is be consumed. The job is consumed upon completion (unless False is
returned which will signify the job should be abandoned instead). returned which will signify the job should be abandoned instead).

View File

@@ -67,10 +67,13 @@ class SingleThreadedConductor(base.Conductor):
@lock_utils.locked @lock_utils.locked
def stop(self, timeout=None): def stop(self, timeout=None):
"""Requests the conductor to stop dispatching and returns whether the """Requests the conductor to stop dispatching.
stop request was successfully completed. If the dispatching is still
occurring then False is returned otherwise True will be returned to This method can be used to request that a conductor stop its
signal that the conductor is no longer dispatching job requests. consumption & dispatching loop. It returns whether the stop request
was successfully completed. If the dispatching is still occurring
then False is returned otherwise True will be returned to signal that
the conductor is no longer consuming & dispatching job requests.
NOTE(harlowja): If a timeout is provided the dispatcher loop may NOTE(harlowja): If a timeout is provided the dispatcher loop may
not have ceased by the timeout reached (the request to cease will not have ceased by the timeout reached (the request to cease will

View File

@@ -22,11 +22,13 @@ from taskflow import states as st
class Analyzer(object): class Analyzer(object):
"""Analyzes a compilation output to get the next atoms for execution or """Analyzes a compilation and aids in execution processes.
reversion by utilizing the compilations underlying structures (graphs,
nodes and edge relations...) and using this information along with the Its primary purpose is to get the next atoms for execution or reversion
atom state/states stored in storage to provide useful analysis functions by utilizing the compilations underlying structures (graphs, nodes and
to the rest of the runtime system. edge relations...) and using this information along with the atom
state/states stored in storage to provide other useful functionality to
the rest of the runtime system.
""" """
def __init__(self, compilation, storage): def __init__(self, compilation, storage):
@@ -56,8 +58,11 @@ class Analyzer(object):
return [] return []
def browse_nodes_for_execute(self, node=None): def browse_nodes_for_execute(self, node=None):
"""Browse next nodes to execute for given node if specified and """Browse next nodes to execute.
for whole graph otherwise.
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.
""" """
if node: if node:
nodes = self._graph.successors(node) nodes = self._graph.successors(node)
@@ -71,8 +76,11 @@ class Analyzer(object):
return available_nodes return available_nodes
def browse_nodes_for_revert(self, node=None): def browse_nodes_for_revert(self, node=None):
"""Browse next nodes to revert for given node if specified and """Browse next nodes to revert.
for whole graph otherwise.
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.
""" """
if node: if node:
nodes = self._graph.predecessors(node) nodes = self._graph.predecessors(node)
@@ -87,7 +95,6 @@ class Analyzer(object):
def _is_ready_for_execute(self, task): def _is_ready_for_execute(self, task):
"""Checks if task is ready to be executed.""" """Checks if task is ready to be executed."""
state = self.get_state(task) state = self.get_state(task)
intention = self._storage.get_atom_intention(task.name) intention = self._storage.get_atom_intention(task.name)
transition = st.check_task_transition(state, st.RUNNING) transition = st.check_task_transition(state, st.RUNNING)
@@ -104,7 +111,6 @@ class Analyzer(object):
def _is_ready_for_revert(self, task): def _is_ready_for_revert(self, task):
"""Checks if task is ready to be reverted.""" """Checks if task is ready to be reverted."""
state = self.get_state(task) state = self.get_state(task)
intention = self._storage.get_atom_intention(task.name) intention = self._storage.get_atom_intention(task.name)
transition = st.check_task_transition(state, st.REVERTING) transition = st.check_task_transition(state, st.REVERTING)
@@ -120,15 +126,14 @@ class Analyzer(object):
for state, intention in six.itervalues(task_states)) for state, intention in six.itervalues(task_states))
def iterate_subgraph(self, retry): def iterate_subgraph(self, retry):
"""Iterates a subgraph connected to current retry controller, including """Iterates a subgraph connected to given retry controller."""
nested retry controllers and its nodes.
"""
for _src, dst in traversal.dfs_edges(self._graph, retry): for _src, dst in traversal.dfs_edges(self._graph, retry):
yield dst yield dst
def iterate_retries(self, state=None): def iterate_retries(self, state=None):
"""Iterates retry controllers of a graph with given state or all """Iterates retry controllers that match the provided state.
retries if state is None.
If no state is provided it will yield back all retry controllers.
""" """
for node in self._graph.nodes_iter(): for node in self._graph.nodes_iter():
if isinstance(node, retry_atom.Retry): if isinstance(node, retry_atom.Retry):

View File

@@ -42,8 +42,7 @@ class Compilation(object):
class PatternCompiler(object): class PatternCompiler(object):
"""Compiles patterns & atoms (potentially nested) into an compilation """Compiles patterns & atoms into a compilation unit.
unit with a *logically* equivalent directed acyclic graph representation.
NOTE(harlowja): during this pattern translation process any nested flows NOTE(harlowja): during this pattern translation process any nested flows
will be converted into there equivalent subgraphs. This currently implies will be converted into there equivalent subgraphs. This currently implies
@@ -51,8 +50,8 @@ class PatternCompiler(object):
be associated with there previously containing flow but instead will lose be associated with there previously containing flow but instead will lose
this identity and what will remain is the logical constraints that there this identity and what will remain is the logical constraints that there
contained flow mandated. In the future this may be changed so that this contained flow mandated. In the future this may be changed so that this
association is not lost via the compilation process (since it is sometime association is not lost via the compilation process (since it can be
useful to retain part of this relationship). useful to retain this relationship).
""" """
def compile(self, root): def compile(self, root):
graph = _Flattener(root).flatten() graph = _Flattener(root).flatten()
@@ -80,9 +79,11 @@ class _Flattener(object):
self._freeze = bool(freeze) self._freeze = bool(freeze)
def _add_new_edges(self, graph, nodes_from, nodes_to, edge_attrs): def _add_new_edges(self, graph, nodes_from, nodes_to, edge_attrs):
"""Adds new edges from nodes to other nodes in the specified graph, """Adds new edges from nodes to other nodes in the specified graph.
with the following edge attributes (defaulting to the class provided
edge_data if None), if the edge does not already exist. It will connect the nodes_from to the nodes_to if an edge currently
does *not* exist. When an edge is created the provided edge attributes
will be applied to the new edge between these two nodes.
""" """
nodes_to = list(nodes_to) nodes_to = list(nodes_to)
for u in nodes_from: for u in nodes_from:

View File

@@ -28,9 +28,11 @@ from taskflow.engines.action_engine import task_action as ta
class Runtime(object): class Runtime(object):
"""An object that contains various utility methods and properties that """A aggregate of runtime objects, properties, ... used during execution.
represent the collection of runtime components and functionality needed
for an action engine to run to completion. This object contains various utility methods and properties that represent
the collection of runtime components and functionality needed for an
action engine to run to completion.
""" """
def __init__(self, compilation, storage, task_notifier, task_executor): def __init__(self, compilation, storage, task_notifier, task_executor):
@@ -155,8 +157,13 @@ class Completer(object):
return False return False
def _process_atom_failure(self, atom, failure): def _process_atom_failure(self, atom, failure):
"""On atom failure find its retry controller, ask for the action to """Processes atom failure & applies resolution strategies.
perform with failed subflow and set proper intention for subflow nodes.
On atom failure this will find the atoms associated retry controller
and ask that controller for the strategy to perform to resolve that
failure. After getting a resolution strategy decision this method will
then adjust the needed other atoms intentions, and states, ... so that
the failure can be worked around.
""" """
retry = self._analyzer.find_atom_retry(atom) retry = self._analyzer.find_atom_retry(atom)
if retry: if retry:
@@ -195,6 +202,9 @@ class Scheduler(object):
def _schedule_node(self, node): def _schedule_node(self, node):
"""Schedule a single node for execution.""" """Schedule a single node for execution."""
# TODO(harlowja): we need to rework this so that we aren't doing type
# checking here, type checking usually means something isn't done right
# and usually will limit extensibility in the future.
if isinstance(node, task_atom.BaseTask): if isinstance(node, task_atom.BaseTask):
return self._schedule_task(node) return self._schedule_task(node)
elif isinstance(node, retry_atom.Retry): elif isinstance(node, retry_atom.Retry):
@@ -204,8 +214,10 @@ class Scheduler(object):
% (node, type(node))) % (node, type(node)))
def _schedule_retry(self, retry): def _schedule_retry(self, retry):
"""Schedules the given retry for revert or execute depending """Schedules the given retry atom for *future* completion.
on its intention.
Depending on the atoms stored intention this may schedule the retry
atom for reversion or execution.
""" """
intention = self._storage.get_atom_intention(retry.name) intention = self._storage.get_atom_intention(retry.name)
if intention == st.EXECUTE: if intention == st.EXECUTE:
@@ -221,8 +233,10 @@ class Scheduler(object):
" intention: %s" % intention) " intention: %s" % intention)
def _schedule_task(self, task): def _schedule_task(self, task):
"""Schedules the given task for revert or execute depending """Schedules the given task atom for *future* completion.
on its intention.
Depending on the atoms stored intention this may schedule the task
atom for reversion or execution.
""" """
intention = self._storage.get_atom_intention(task.name) intention = self._storage.get_atom_intention(task.name)
if intention == st.EXECUTE: if intention == st.EXECUTE:

View File

@@ -54,9 +54,12 @@ class EngineBase(object):
@abc.abstractmethod @abc.abstractmethod
def compile(self): def compile(self):
"""Compiles the contained flow into a structure which the engine can """Compiles the contained flow into a internal representation.
use to run or if this can not be done then an exception is thrown
indicating why this compilation could not be achieved. This internal representation is what the engine will *actually* use to
run. If this compilation can not be accomplished then an exception
is expected to be thrown with a message indicating why the compilation
could not be achieved.
""" """
@abc.abstractmethod @abc.abstractmethod

View File

@@ -50,7 +50,7 @@ def _fetch_validate_factory(flow_factory):
def load(flow, store=None, flow_detail=None, book=None, def load(flow, store=None, flow_detail=None, book=None,
engine_conf=None, backend=None, namespace=ENGINES_NAMESPACE, engine_conf=None, backend=None, namespace=ENGINES_NAMESPACE,
**kwargs): **kwargs):
"""Load flow into engine. """Load a flow into an engine.
This function creates and prepares engine to run the This function creates and prepares engine to run the
flow. All that is left is to run the engine with 'run()' method. flow. All that is left is to run the engine with 'run()' method.
@@ -151,8 +151,7 @@ def run(flow, store=None, flow_detail=None, book=None,
def save_factory_details(flow_detail, def save_factory_details(flow_detail,
flow_factory, factory_args, factory_kwargs, flow_factory, factory_args, factory_kwargs,
backend=None): backend=None):
"""Saves the given factories reimportable name, args, kwargs into the """Saves the given factories reimportable attributes into the flow detail.
flow detail.
This function saves the factory name, arguments, and keyword arguments This function saves the factory name, arguments, and keyword arguments
into the given flow details object and if a backend is provided it will into the given flow details object and if a backend is provided it will
@@ -227,9 +226,11 @@ def load_from_factory(flow_factory, factory_args=None, factory_kwargs=None,
def flow_from_detail(flow_detail): def flow_from_detail(flow_detail):
"""Recreate flow previously loaded with load_form_factory. """Reloads a flow previously saved.
Gets flow factory name from metadata, calls it to recreate the flow. Gets the flow factories name and any arguments and keyword arguments from
the flow details metadata, and then calls that factory to recreate the
flow.
:param flow_detail: FlowDetail that holds state of the flow to load :param flow_detail: FlowDetail that holds state of the flow to load
""" """
@@ -253,10 +254,10 @@ def flow_from_detail(flow_detail):
def load_from_detail(flow_detail, store=None, engine_conf=None, backend=None, def load_from_detail(flow_detail, store=None, engine_conf=None, backend=None,
namespace=ENGINES_NAMESPACE, **kwargs): namespace=ENGINES_NAMESPACE, **kwargs):
"""Reload flow previously loaded with load_form_factory function. """Reloads an engine previously saved.
Gets flow factory name from metadata, calls it to recreate the flow This reloads the flow using the flow_from_detail() function and then calls
and loads flow into engine with load(). into the load() function to create an engine from that flow.
:param flow_detail: FlowDetail that holds state of the flow to load :param flow_detail: FlowDetail that holds state of the flow to load
:param store: dict -- data to put to storage to satisfy flow requirements :param store: dict -- data to put to storage to satisfy flow requirements

View File

@@ -216,7 +216,7 @@ class WorkerTaskExecutor(executor.TaskExecutorBase):
return async_utils.wait_for_any(fs, timeout) return async_utils.wait_for_any(fs, timeout)
def start(self): def start(self):
"""Start proxy thread (and associated topic notification thread).""" """Starts proxy thread and associated topic notification thread."""
if not _is_alive(self._proxy_thread): if not _is_alive(self._proxy_thread):
self._proxy_thread = tu.daemon_thread(self._proxy.start) self._proxy_thread = tu.daemon_thread(self._proxy.start)
self._proxy_thread.start() self._proxy_thread.start()
@@ -227,9 +227,7 @@ class WorkerTaskExecutor(executor.TaskExecutorBase):
self._periodic_thread.start() self._periodic_thread.start()
def stop(self): def stop(self):
"""Stop proxy thread (and associated topic notification thread), so """Stops proxy thread and associated topic notification thread."""
those threads will be gracefully terminated.
"""
if self._periodic_thread is not None: if self._periodic_thread is not None:
self._periodic.stop() self._periodic.stop()
self._periodic_thread.join() self._periodic_thread.join()

View File

@@ -86,8 +86,10 @@ class Notify(Message):
class Request(Message): class Request(Message):
"""Represents request with execution results. Every request is created in """Represents request with execution results.
the WAITING state and is expired within the given timeout.
Every request is created in the WAITING state and is expired within the
given timeout.
""" """
TYPE = REQUEST TYPE = REQUEST
@@ -136,8 +138,11 @@ class Request(Message):
return False return False
def to_dict(self): def to_dict(self):
"""Return json-serializable request, converting all `misc.Failure` """Return json-serializable request.
objects into dictionaries.
To convert requests that have failed due to some exception this will
convert all `misc.Failure` objects into dictionaries (which will then
be reconstituted by the receiver).
""" """
request = dict(task_cls=self._task_cls, task_name=self._task.name, request = dict(task_cls=self._task_cls, task_name=self._task.name,
task_version=self._task.version, action=self._action, task_version=self._task.version, action=self._action,

View File

@@ -29,9 +29,7 @@ DRAIN_EVENTS_PERIOD = 1
class Proxy(object): class Proxy(object):
"""Proxy picks up messages from the named exchange, calls on_message """A proxy processes messages from/to the named exchange."""
callback when new message received and is used to publish messages.
"""
def __init__(self, topic, exchange_name, on_message, on_wait=None, def __init__(self, topic, exchange_name, on_message, on_wait=None,
**kwargs): **kwargs):
@@ -61,7 +59,7 @@ class Proxy(object):
@property @property
def is_running(self): def is_running(self):
"""Return whether proxy is running.""" """Return whether the proxy is running."""
return self._running.is_set() return self._running.is_set()
def _make_queue(self, name, exchange, **kwargs): def _make_queue(self, name, exchange, **kwargs):
@@ -74,7 +72,7 @@ class Proxy(object):
**kwargs) **kwargs)
def publish(self, msg, routing_key, **kwargs): def publish(self, msg, routing_key, **kwargs):
"""Publish message to the named exchange with routing key.""" """Publish message to the named exchange with given routing key."""
LOG.debug("Sending %s", msg) LOG.debug("Sending %s", msg)
if isinstance(routing_key, six.string_types): if isinstance(routing_key, six.string_types):
routing_keys = [routing_key] routing_keys = [routing_key]

View File

@@ -77,9 +77,10 @@ class Server(object):
@staticmethod @staticmethod
def _parse_request(task_cls, task_name, action, arguments, result=None, def _parse_request(task_cls, task_name, action, arguments, result=None,
failures=None, **kwargs): failures=None, **kwargs):
"""Parse request before it can be processed. All `misc.Failure` objects """Parse request before it can be further processed.
that have been converted to dict on the remote side to be serializable
are now converted back to objects. All `misc.Failure` objects that have been converted to dict on the
remote side will now converted back to `misc.Failure` objects.
""" """
action_args = dict(arguments=arguments, task_name=task_name) action_args = dict(arguments=arguments, task_name=task_name)
if result is not None: if result is not None:
@@ -96,9 +97,10 @@ class Server(object):
@staticmethod @staticmethod
def _parse_message(message): def _parse_message(message):
"""Parse broker message to get the `reply_to` and the `correlation_id` """Extracts required attributes out of the messages properties.
properties. If required properties are missing - the `ValueError` is
raised. This extracts the `reply_to` and the `correlation_id` properties. If
any of these required properties are missing a `ValueError` is raised.
""" """
properties = [] properties = []
for prop in ('reply_to', 'correlation_id'): for prop in ('reply_to', 'correlation_id'):

View File

@@ -96,8 +96,8 @@ class Worker(object):
LOG.info("Starting the '%s' topic worker using a %s.", self._topic, LOG.info("Starting the '%s' topic worker using a %s.", self._topic,
self._executor) self._executor)
LOG.info("Tasks list:") LOG.info("Tasks list:")
for endpoint in self._endpoints: for e in self._endpoints:
LOG.info("|-- %s", endpoint) LOG.info("|-- %s", e)
self._server.start() self._server.start()
def wait(self): def wait(self):

View File

@@ -84,9 +84,7 @@ class ExecutionFailure(TaskFlowException):
class RequestTimeout(ExecutionFailure): class RequestTimeout(ExecutionFailure):
"""Raised when a worker request was not finished within an allotted """Raised when a worker request was not finished within allotted time."""
timeout.
"""
class InvalidState(ExecutionFailure): class InvalidState(ExecutionFailure):

View File

@@ -55,8 +55,11 @@ class Flow(object):
@property @property
def retry(self): def retry(self):
"""A retry object that will affect control how (and if) this flow """The associated flow retry controller.
retries while execution is underway.
This retry controller object will affect & control how (and if) this
flow and its contained components retry when execution is underway and
a failure occurs.
""" """
return self._retry return self._retry

View File

@@ -31,9 +31,25 @@ LOG = logging.getLogger(__name__)
def fetch(name, conf, namespace=BACKEND_NAMESPACE, **kwargs): def fetch(name, conf, namespace=BACKEND_NAMESPACE, **kwargs):
"""Fetch a jobboard backend with the given configuration (and any board """Fetch a jobboard backend with the given configuration.
specific kwargs) in the given entrypoint namespace and create it with the
given name. This fetch method will look for the entrypoint name in the entrypoint
namespace, and then attempt to instantiate that entrypoint using the
provided name, configuration and any board specific 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
configuration specific to that board.
For example, given the following configuration uri:
zookeeper://<not-used>/?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).
""" """
if isinstance(conf, six.string_types): if isinstance(conf, six.string_types):
conf = {'board': conf} conf = {'board': conf}
@@ -58,8 +74,11 @@ def fetch(name, conf, namespace=BACKEND_NAMESPACE, **kwargs):
@contextlib.contextmanager @contextlib.contextmanager
def backend(name, conf, namespace=BACKEND_NAMESPACE, **kwargs): def backend(name, conf, namespace=BACKEND_NAMESPACE, **kwargs):
"""Fetches a jobboard backend, connects to it and allows it to be used in """Fetches a jobboard, connects to it and closes it on completion.
a context manager statement with the jobboard being closed upon completion.
This allows a board instance to fetched, connected to, and then used in a
context manager statement with the board being closed upon context
manager exit.
""" """
jb = fetch(name, conf, namespace=namespace, **kwargs) jb = fetch(name, conf, namespace=namespace, **kwargs)
jb.connect() jb.connect()

View File

@@ -24,16 +24,22 @@ from taskflow.openstack.common import uuidutils
@six.add_metaclass(abc.ABCMeta) @six.add_metaclass(abc.ABCMeta)
class Job(object): class Job(object):
"""A job is a higher level abstraction over a set of flows as well as the """A abstraction that represents a named and trackable unit of work.
*ownership* of those flows, it is the highest piece of work that can be
owned by an entity performing those flows.
Only one entity will be operating on the flows contained in a job at a A job connects a logbook, a owner, last modified and created on dates and
given time (for the foreseeable future). any associated state that the job has. Since it is a connector to a
logbook, which are each associated with a set of factories that can create
set of flows, it is the current top-level container for a piece of work
that can be owned by an entity (typically that entity will read those
logbooks and run any contained flows).
It is the object that should be transferred to another entity on failure of Only one entity will be allowed to own and operate on the flows contained
so that the contained flows ownership can be transferred to the secondary in a job at a given time (for the foreseeable future).
entity for resumption/continuation/reverting.
NOTE(harlowja): It is the object that will be transferred to another
entity on failure so that the contained flows ownership can be
transferred to the secondary entity/owner for resumption, continuation,
reverting...
""" """
def __init__(self, name, uuid=None, details=None): def __init__(self, name, uuid=None, details=None):

View File

@@ -24,10 +24,15 @@ from taskflow.utils import misc
@six.add_metaclass(abc.ABCMeta) @six.add_metaclass(abc.ABCMeta)
class JobBoard(object): class JobBoard(object):
"""A jobboard is an abstract representation of a place where jobs """A place where jobs can be posted, reposted, claimed and transferred.
can be posted, reposted, claimed and transferred. There can be multiple
implementations of this job board, depending on the desired semantics and There can be multiple implementations of this job board, depending on the
capabilities of the underlying jobboard implementation. desired semantics and capabilities of the underlying jobboard
implementation.
NOTE(harlowja): the name is meant to be an analogous to a board/posting
system that is used in newspapers, or elsewhere to solicit jobs that
people can interview and apply for (and then work on & complete).
""" """
def __init__(self, name, conf): def __init__(self, name, conf):
@@ -36,8 +41,7 @@ class JobBoard(object):
@abc.abstractmethod @abc.abstractmethod
def iterjobs(self, only_unclaimed=False, ensure_fresh=False): def iterjobs(self, only_unclaimed=False, ensure_fresh=False):
"""Returns an iterator that will provide back jobs that are currently """Returns an iterator of jobs that are currently on this board.
on this jobboard.
NOTE(harlowja): the ordering of this iteration should be by posting NOTE(harlowja): the ordering of this iteration should be by posting
order (oldest to newest) if possible, but it is left up to the backing order (oldest to newest) if possible, but it is left up to the backing
@@ -60,9 +64,10 @@ class JobBoard(object):
@abc.abstractmethod @abc.abstractmethod
def wait(self, timeout=None): def wait(self, timeout=None):
"""Waits a given amount of time for job/s to be posted, when jobs are """Waits a given amount of time for jobs to be posted.
found then an iterator will be returned that contains the jobs at
the given point in time. When jobs are found then an iterator will be returned that can be used
to iterate over those jobs.
NOTE(harlowja): since a jobboard can be mutated on by multiple external NOTE(harlowja): since a jobboard can be mutated on by multiple external
entities at the *same* time the iterator that can be returned *may* entities at the *same* time the iterator that can be returned *may*
@@ -75,8 +80,11 @@ class JobBoard(object):
@abc.abstractproperty @abc.abstractproperty
def job_count(self): def job_count(self):
"""Returns how many jobs are on this jobboard (this count may change as """Returns how many jobs are on this jobboard.
new jobs appear or are removed).
NOTE(harlowja): this count may change as jobs appear or are removed so
the accuracy of this count should not be used in a way that requires
it to be exact & absolute.
""" """
@abc.abstractmethod @abc.abstractmethod
@@ -90,11 +98,13 @@ class JobBoard(object):
@abc.abstractmethod @abc.abstractmethod
def consume(self, job, who): def consume(self, job, who):
"""Permanently (and atomically) removes a job from the jobboard, """Permanently (and atomically) removes a job from the jobboard.
signaling that this job has been completed by the entity assigned
to that job.
Only the entity that has claimed that job is able to consume a job. Consumption signals to the board (and any others examining the board)
that this job has been completed by the entity that previously claimed
that job.
Only the entity that has claimed that job is able to consume the job.
A job that has been consumed can not be reclaimed or reposted by A job that has been consumed can not be reclaimed or reposted by
another entity (job postings are immutable). Any entity consuming another entity (job postings are immutable). Any entity consuming
@@ -109,11 +119,13 @@ class JobBoard(object):
@abc.abstractmethod @abc.abstractmethod
def post(self, name, book, details=None): def post(self, name, book, details=None):
"""Atomically creates and posts a job to the jobboard, allowing others """Atomically creates and posts a job to the jobboard.
to attempt to claim that job (and subsequently work on that job). The
contents of the provided logbook must provide enough information for This posting allowing others to attempt to claim that job (and
others to reference to construct & work on the desired entries that subsequently work on that job). The contents of the provided logbook
are contained in that logbook. must provide *enough* information for others to reference to
construct & work on the desired entries that are contained in that
logbook.
Once a job has been posted it can only be removed by consuming that Once a job has been posted it can only be removed by consuming that
job (after that job is claimed). Any entity can post/propose jobs job (after that job is claimed). Any entity can post/propose jobs
@@ -124,13 +136,14 @@ class JobBoard(object):
@abc.abstractmethod @abc.abstractmethod
def claim(self, job, who): def claim(self, job, who):
"""Atomically attempts to claim the given job for the entity and either """Atomically attempts to claim the provided job.
succeeds or fails at claiming by throwing corresponding exceptions.
If a job is claimed it is expected that the entity that claims that job If a job is claimed it is expected that the entity that claims that job
will at sometime in the future work on that jobs flows and either fail will at sometime in the future work on that jobs contents and either
at completing them (resulting in a reposting) or consume that job from fail at completing them (resulting in a reposting) or consume that job
the jobboard (signaling its completion). from the jobboard (signaling its completion). If claiming fails then
a corresponding exception will be raised to signal this to the claim
attempter.
:param job: a job on this jobboard that can be claimed (if it does :param job: a job on this jobboard that can be claimed (if it does
not exist then a NotFound exception will be raised). not exist then a NotFound exception will be raised).
@@ -139,10 +152,12 @@ class JobBoard(object):
@abc.abstractmethod @abc.abstractmethod
def abandon(self, job, who): def abandon(self, job, who):
"""Atomically abandons the given job on the jobboard, allowing that job """Atomically attempts to abandon the provided job.
to be reclaimed by others. This would typically occur if the entity
that has claimed the job has failed or is unable to complete the job This abandonment signals to others that the job may now be reclaimed.
or jobs it has claimed. This would typically occur if the entity that has claimed the job has
failed or is unable to complete the job or jobs it had previously
claimed.
Only the entity that has claimed that job can abandon a job. Any entity Only the entity that has claimed that job can abandon a job. Any entity
abandoning a unclaimed job (or a job they do not own) will cause an abandoning a unclaimed job (or a job they do not own) will cause an
@@ -177,13 +192,14 @@ REMOVAL = 'REMOVAL' # existing job is/has been removed
class NotifyingJobBoard(JobBoard): class NotifyingJobBoard(JobBoard):
"""A jobboard subclass that can notify about jobs being created """A jobboard subclass that can notify others about board events.
and removed, which can remove the repeated usage of iterjobs() to achieve
the same operation. Implementers are expected to notify *at least* about jobs being posted
and removed.
NOTE(harlowja): notifications that are emitted *may* be emitted on a NOTE(harlowja): notifications that are emitted *may* be emitted on a
separate dedicated thread when they occur, so ensure that all callbacks separate dedicated thread when they occur, so ensure that all callbacks
registered are thread safe. registered are thread safe (and block for as little time as possible).
""" """
def __init__(self, name, conf): def __init__(self, name, conf):
super(NotifyingJobBoard, self).__init__(name, conf) super(NotifyingJobBoard, self).__init__(name, conf)

View File

@@ -72,9 +72,12 @@ class Flow(flow.Flow):
return graph return graph
def _swap(self, graph): def _swap(self, graph):
"""Validates the replacement graph and then swaps the underlying graph """Validates the replacement graph and then swaps the underlying graph.
with a frozen version of the replacement graph (this maintains the
invariant that the underlying graph is immutable). After swapping occurs the underlying graph will be frozen so that the
immutability invariant is maintained (we may be able to relax this
constraint in the future since our exposed public api does not allow
direct access to the underlying graph).
""" """
if not graph.is_directed_acyclic(): if not graph.is_directed_acyclic():
raise exc.DependencyFailure("No path through the items in the" raise exc.DependencyFailure("No path through the items in the"

View File

@@ -30,8 +30,25 @@ LOG = logging.getLogger(__name__)
def fetch(conf, namespace=BACKEND_NAMESPACE, **kwargs): def fetch(conf, namespace=BACKEND_NAMESPACE, **kwargs):
"""Fetches a given backend using the given configuration (and any backend """Fetch a persistence backend with the given configuration.
specific kwargs) in the given entrypoint namespace.
This fetch method will look for the entrypoint name in the entrypoint
namespace, and then attempt to instantiate that entrypoint using the
provided configuration and any persistence backend specific 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
configuration specific to that backend.
For example, given the following configuration uri:
mysql://<not-used>/?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
instance.
""" """
backend_name = conf['connection'] backend_name = conf['connection']
try: try:
@@ -54,8 +71,12 @@ def fetch(conf, namespace=BACKEND_NAMESPACE, **kwargs):
@contextlib.contextmanager @contextlib.contextmanager
def backend(conf, namespace=BACKEND_NAMESPACE, **kwargs): def backend(conf, namespace=BACKEND_NAMESPACE, **kwargs):
"""Fetches a persistence backend, ensures that it is upgraded and upon """Fetches a backend, connects, upgrades, then closes it on completion.
context manager completion closes the backend.
This allows a backend instance to be fetched, connected to, have its schema
upgraded (if the schema is already up to date this is a no-op) and then
used in a context manager statement with the backend being closed upon
context manager exit.
""" """
with contextlib.closing(fetch(conf, namespace=namespace, **kwargs)) as be: with contextlib.closing(fetch(conf, namespace=namespace, **kwargs)) as be:
with contextlib.closing(be.get_connection()) as conn: with contextlib.closing(be.get_connection()) as conn:

View File

@@ -70,9 +70,11 @@ class Connection(object):
@abc.abstractmethod @abc.abstractmethod
def validate(self): def validate(self):
"""Validates that a backend is still ok to be used (the semantics """Validates that a backend is still ok to be used.
of this vary depending on the backend). On failure a backend specific
exception is raised that will indicate why the failure occurred. The semantics of this *may* vary depending on the backend. On failure a
backend specific exception should be raised that will indicate why the
failure occurred.
""" """
pass pass

View File

@@ -33,10 +33,24 @@ LOG = logging.getLogger(__name__)
class DirBackend(base.Backend): class DirBackend(base.Backend):
"""A backend that writes logbooks, flow details, and task details to a """A directory and file based backend.
provided directory. This backend does *not* provide transactional semantics
although it does guarantee that there will be no race conditions when This backend writes logbooks, flow details, and atom details to a provided
writing/reading by using file level locking. 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.
Example conf:
conf = {
"path": "/tmp/taskflow",
}
""" """
def __init__(self, conf): def __init__(self, conf):
super(DirBackend, self).__init__(conf) super(DirBackend, self).__init__(conf)

View File

@@ -15,8 +15,6 @@
# License for the specific language governing permissions and limitations # License for the specific language governing permissions and limitations
# under the License. # under the License.
"""Implementation of in-memory backend."""
import logging import logging
import six import six
@@ -29,8 +27,10 @@ LOG = logging.getLogger(__name__)
class MemoryBackend(base.Backend): class MemoryBackend(base.Backend):
"""A backend that writes logbooks, flow details, and task details to in """A in-memory (non-persistent) backend.
memory dictionaries.
This backend writes logbooks, flow details, and atom details to in-memory
dictionaries and retrieves from those dictionaries as needed.
""" """
def __init__(self, conf=None): def __init__(self, conf=None):
super(MemoryBackend, self).__init__(conf) super(MemoryBackend, self).__init__(conf)

View File

@@ -167,6 +167,14 @@ def _ping_listener(dbapi_conn, connection_rec, connection_proxy):
class SQLAlchemyBackend(base.Backend): class SQLAlchemyBackend(base.Backend):
"""A sqlalchemy backend.
Example conf:
conf = {
"connection": "sqlite:////tmp/test.db",
}
"""
def __init__(self, conf, engine=None): def __init__(self, conf, engine=None):
super(SQLAlchemyBackend, self).__init__(conf) super(SQLAlchemyBackend, self).__init__(conf)
if engine is not None: if engine is not None:
@@ -337,9 +345,13 @@ class Connection(base.Connection):
failures[-1].reraise() failures[-1].reraise()
def _run_in_session(self, functor, *args, **kwargs): def _run_in_session(self, functor, *args, **kwargs):
"""Runs a function in a session and makes sure that sqlalchemy """Runs a callback in a session.
exceptions aren't emitted from that sessions actions (as that would
expose the underlying backends exception model). This function proxy will create a session, and then call the callback
with that session (along with the provided args and kwargs). It ensures
that the session is opened & closed and makes sure that sqlalchemy
exceptions aren't emitted from the callback or sessions actions (as
that would expose the underlying sqlalchemy exception model).
""" """
try: try:
session = self._make_session() session = self._make_session()

View File

@@ -34,9 +34,16 @@ MIN_ZK_VERSION = (3, 4, 0)
class ZkBackend(base.Backend): class ZkBackend(base.Backend):
"""ZooKeeper as backend storage implementation """A zookeeper backend.
Example conf (use Kazoo): 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.
Example conf:
conf = { conf = {
"hosts": "192.168.0.1:2181,192.168.0.2:2181,192.168.0.3:2181", "hosts": "192.168.0.1:2181,192.168.0.2:2181,192.168.0.3:2181",
@@ -126,8 +133,11 @@ class ZkConnection(base.Connection):
@contextlib.contextmanager @contextlib.contextmanager
def _exc_wrapper(self): def _exc_wrapper(self):
"""Exception wrapper which wraps kazoo exceptions and groups them """Exception context-manager which wraps kazoo exceptions.
to taskflow exceptions.
This is used to capture and wrap any kazoo specific exceptions and
then group them into corresponding taskflow exceptions (not doing
that would expose the underlying kazoo exception model).
""" """
try: try:
yield yield

View File

@@ -64,14 +64,20 @@ def _fix_meta(data):
class LogBook(object): class LogBook(object):
"""This class that contains a dict of flow detail entries for a """A container of flow details, a name and associated metadata.
given *job* so that the job can track what 'work' has been
completed for resumption/reverting and miscellaneous tracking 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. purposes.
The data contained within this class need *not* be backed by the backend 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 storage in real time. The data in this class will only be guaranteed to be
persisted when a save occurs via some backend connection. 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).
""" """
def __init__(self, name, uuid=None): def __init__(self, name, uuid=None):
if uuid: if uuid:
@@ -159,8 +165,11 @@ class LogBook(object):
class FlowDetail(object): class FlowDetail(object):
"""This class contains a dict of atom detail entries for a given """A container of atom details, a name and associated metadata.
flow along with any metadata associated with that flow.
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 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 storage in real time. The data in this class will only be guaranteed to be
@@ -241,13 +250,15 @@ class FlowDetail(object):
@six.add_metaclass(abc.ABCMeta) @six.add_metaclass(abc.ABCMeta)
class AtomDetail(object): class AtomDetail(object):
"""This is a base class that contains an entry that contains the """A base container of atom specific runtime information and metadata.
persistence of an atom after or before (or during) it is running including
any results it may have produced, any state that it may be in (failed This is a base class that contains attributes that are used to connect
for example), any exception that occurred when running and any associated a atom to the persistence layer during, after, or before it is running
stacktrace that may have occurring during that exception being thrown including any results it may have produced, any state that it may be
and any other metadata that should be stored along-side the details in (failed for example), any exception that occurred when running and any
about this atom. 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.
The data contained within this class need *not* backed by the backend 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 storage in real time. The data in this class will only be guaranteed to be
@@ -276,8 +287,11 @@ class AtomDetail(object):
@property @property
def last_results(self): def last_results(self):
"""Gets the atoms last result (if it has many results it should then """Gets the atoms last result.
return the last one of many).
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 return self.results

View File

@@ -34,8 +34,7 @@ RETRY = "RETRY"
@six.add_metaclass(abc.ABCMeta) @six.add_metaclass(abc.ABCMeta)
class Decider(object): class Decider(object):
"""A base class or mixin for an object that can decide how to resolve """A class/mixin object that can decide how to resolve execution failures.
execution failures.
A decider may be executed multiple times on subflow or other atom A decider may be executed multiple times on subflow or other atom
failure and it is expected to make a decision about what should be done failure and it is expected to make a decision about what should be done
@@ -45,10 +44,11 @@ class Decider(object):
@abc.abstractmethod @abc.abstractmethod
def on_failure(self, history, *args, **kwargs): def on_failure(self, history, *args, **kwargs):
"""On subflow failure makes a decision about the future flow """On failure makes a decision about the future.
execution using information about prior previous failures (if this
historical failure information is not available or was not persisted This method will typically use information about prior failures (if
this history will be empty). this historical failure information is not available or was not
persisted this history will be empty).
Returns retry action constant: Returns retry action constant:
@@ -63,9 +63,13 @@ class Decider(object):
@six.add_metaclass(abc.ABCMeta) @six.add_metaclass(abc.ABCMeta)
class Retry(atom.Atom, Decider): class Retry(atom.Atom, Decider):
"""A base class for a retry object that decides how to resolve subflow """A class that can decide how to resolve execution failures.
execution failures and may also provide execute and revert methods to alter
the inputs of subflow atoms. 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 execute and revert methods to alter
the inputs of connected atoms (depending on the desired strategy to be
used this can be quite useful).
""" """
default_provides = None default_provides = None
@@ -88,22 +92,32 @@ class Retry(atom.Atom, Decider):
@abc.abstractmethod @abc.abstractmethod
def execute(self, history, *args, **kwargs): def execute(self, history, *args, **kwargs):
"""Activate a given retry which will produce data required to """Executes the given retry atom.
start or restart a subflow using previously provided values and a
history of subflow failures from previous runs. This execution activates a given retry which will typically produce
Retry can provide same values multiple times (after each run), data required to start or restart a connected component using
the latest value will be used by tasks. Old values will be saved to previously provided values and a history of prior failures from
the history of retry that is a list of tuples (result, failures) previous runs. The historical data can be analyzed to alter the
where failures is a dictionary of failures by task names. resolution strategy that this retry controller will use.
This allows to make retries of subflow with different parameters.
For example, a retry can provide the same values multiple times (after
each run), the latest value or some other variation. Old values will be
saved to the history of the retry atom automatically, that is a list of
tuples (result, failures) are persisted where failures is a dictionary
of failures indexed by task names and the result is the execution
result returned by this retry controller during that failure resolution
attempt.
""" """
def revert(self, history, *args, **kwargs): def revert(self, history, *args, **kwargs):
"""Revert this retry using the given context, all results """Reverts this retry using the given context.
that had been provided by previous tries and all errors caused
a reversion. This method will be called only if a subflow must be On revert call all results that had been provided by previous tries
reverted without the retry. It won't be called on subflow retry, but and all errors caused during reversion are provided. This method
all subflow's tasks will be reverted before the retry. will be called *only* if a subflow must be reverted without the
retry (that is to say that the controller has ran out of resolution
options and has either given up resolution or has failed to handle
a execution failure).
""" """
@@ -146,9 +160,12 @@ class Times(Retry):
class ForEachBase(Retry): class ForEachBase(Retry):
"""Base class for retries that iterate given collection.""" """Base class for retries that iterate over a given collection."""
def _get_next_value(self, values, history): 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
# resolution strategy remaining.
items = (item for item, _failures in history) items = (item for item, _failures in history)
remaining = misc.sequence_minus(values, items) remaining = misc.sequence_minus(values, items)
if not remaining: if not remaining:
@@ -166,8 +183,10 @@ class ForEachBase(Retry):
class ForEach(ForEachBase): class ForEach(ForEachBase):
"""Accepts a collection of values to the constructor. Returns the next """Applies a statically provided collection of strategies.
element of the collection on each try.
Accepts a collection of decision strategies on construction and returns the
next element of the collection on each try.
""" """
def __init__(self, values, name=None, provides=None, requires=None, def __init__(self, values, name=None, provides=None, requires=None,
@@ -180,12 +199,17 @@ class ForEach(ForEachBase):
return self._on_failure(self._values, history) return self._on_failure(self._values, history)
def execute(self, history, *args, **kwargs): def execute(self, history, *args, **kwargs):
# NOTE(harlowja): This allows any connected components to know the
# current resolution strategy being attempted.
return self._get_next_value(self._values, history) return self._get_next_value(self._values, history)
class ParameterizedForEach(ForEachBase): class ParameterizedForEach(ForEachBase):
"""Accepts a collection of values from storage as a parameter of execute """Applies a dynamically provided collection of strategies.
method. Returns the next element of the collection on each try.
Accepts a collection of decision strategies from a predecessor (or from
storage) as a parameter and returns the next element of that collection on
each try.
""" """
def on_failure(self, values, history, *args, **kwargs): def on_failure(self, values, history, *args, **kwargs):

View File

@@ -53,7 +53,7 @@ SCHEDULING = 'SCHEDULING'
WAITING = 'WAITING' WAITING = 'WAITING'
ANALYZING = 'ANALYZING' ANALYZING = 'ANALYZING'
## Flow state transitions # Flow state transitions
# See: http://docs.openstack.org/developer/taskflow/states.html # See: http://docs.openstack.org/developer/taskflow/states.html
_ALLOWED_FLOW_TRANSITIONS = frozenset(( _ALLOWED_FLOW_TRANSITIONS = frozenset((
@@ -124,7 +124,7 @@ def check_flow_transition(old_state, new_state):
% pair) % pair)
## Task state transitions # Task state transitions
# See: http://docs.openstack.org/developer/taskflow/states.html # See: http://docs.openstack.org/developer/taskflow/states.html
_ALLOWED_TASK_TRANSITIONS = frozenset(( _ALLOWED_TASK_TRANSITIONS = frozenset((

View File

@@ -77,9 +77,12 @@ class Storage(object):
@abc.abstractproperty @abc.abstractproperty
def _lock_cls(self): def _lock_cls(self):
"""Lock class used to generate reader/writer locks for protecting """Lock class used to generate reader/writer locks.
read/write access to the underlying storage backend and internally
mutating operations. These locks are used for protecting read/write access to the
underlying storage backend when internally mutating operations occur.
They ensure that we read and write data in a consistent manner when
being used in a multithreaded situation.
""" """
def _with_connection(self, functor, *args, **kwargs): def _with_connection(self, functor, *args, **kwargs):
@@ -248,9 +251,12 @@ class Storage(object):
self._with_connection(self._save_atom_detail, ad) self._with_connection(self._save_atom_detail, ad)
def update_atom_metadata(self, atom_name, update_with): def update_atom_metadata(self, atom_name, update_with):
"""Updates a atoms metadata given another dictionary or a list of """Updates a atoms associated metadata.
(key, value) pairs to include in the updated metadata (newer keys will
overwrite older keys). This update will take a provided dictionary or a list of (key, value)
pairs to include in the updated metadata (newer keys will overwrite
older keys) and after merging saves the updated data into the
underlying persistence layer.
""" """
self._update_atom_metadata(atom_name, update_with) self._update_atom_metadata(atom_name, update_with)

View File

@@ -30,8 +30,12 @@ LOG = logging.getLogger(__name__)
@six.add_metaclass(abc.ABCMeta) @six.add_metaclass(abc.ABCMeta)
class BaseTask(atom.Atom): class BaseTask(atom.Atom):
"""An abstraction that defines a potential piece of work that can be """An abstraction that defines a potential piece of work.
applied and can be reverted to undo the work as a single task.
This potential piece of work is expected to be able to contain
functionality that defines what can be executed to accomplish that work
as well as a way of defining what can be executed to reverted/undo that
same piece of work.
""" """
TASK_EVENTS = ('update_progress', ) TASK_EVENTS = ('update_progress', )
@@ -101,8 +105,12 @@ class BaseTask(atom.Atom):
@contextlib.contextmanager @contextlib.contextmanager
def autobind(self, event_name, handler_func, **kwargs): def autobind(self, event_name, handler_func, **kwargs):
"""Binds a given function to the task for a given event name and then """Binds & unbinds a given event handler to the task.
unbinds that event name and associated function automatically on exit.
This function binds and unbinds using the context manager protocol.
When events are triggered on the task of the given event name this
handler will automatically be called with the provided keyword
arguments.
""" """
bound = False bound = False
if handler_func is not None: if handler_func is not None:
@@ -135,10 +143,11 @@ class BaseTask(atom.Atom):
self._events_listeners[event].append((handler, kwargs)) self._events_listeners[event].append((handler, kwargs))
def unbind(self, event, handler=None): def unbind(self, event, handler=None):
"""Remove a previously-attached event handler from the task. If handler """Remove a previously-attached event handler from the task.
function not passed, then unbind all event handlers for the provided
event. If multiple of the same handlers are bound, then the first If a handler function not passed, then this will unbind all event
match is removed (and only the first match). handlers for the provided event. If multiple of the same handlers are
bound, then the first match is removed (and only the first match).
:param event: event type :param event: event type
:param handler: handler previously bound :param handler: handler previously bound

View File

@@ -41,8 +41,11 @@ class GreaterThanEqual(object):
class FailureRegexpMatcher(object): class FailureRegexpMatcher(object):
"""Matches if the failure was caused by the given exception and its string """Matches if the failure was caused by the given exception and message.
matches to the given pattern.
This will match if a given failure contains and exception of the given
class type and if its string message matches to the given regular
expression pattern.
""" """
def __init__(self, exc_class, pattern): def __init__(self, exc_class, pattern):
@@ -59,8 +62,10 @@ class FailureRegexpMatcher(object):
class ItemsEqual(object): class ItemsEqual(object):
"""Matches the sequence that has same elements as reference """Matches the items in two sequences.
object, regardless of the order.
This matcher will validate that the provided sequence has the same elements
as a reference sequence, regardless of the order.
""" """
def __init__(self, seq): def __init__(self, seq):
@@ -167,9 +172,7 @@ class TestCase(testcase.TestCase):
def assertFailuresRegexp(self, exc_class, pattern, callable_obj, *args, def assertFailuresRegexp(self, exc_class, pattern, callable_obj, *args,
**kwargs): **kwargs):
"""Assert that the callable failed with the given exception and its """Asserts the callable failed with the given exception and message."""
string matches to the given pattern.
"""
try: try:
with utils.wrap_all_failures(): with utils.wrap_all_failures():
callable_obj(*args, **kwargs) callable_obj(*args, **kwargs)
@@ -200,8 +203,11 @@ class MockTestCase(TestCase):
return mocked return mocked
def _patch_class(self, module, name, autospec=True, attach_as=None): def _patch_class(self, module, name, autospec=True, attach_as=None):
"""Patch class, create class instance mock and attach them to """Patches a modules class.
the master mock.
This will create a class instance mock (using the provided name to
find the class in the module) and attach a mock class the master mock
to be cleaned up on test exit.
""" """
if autospec: if autospec:
instance_mock = mock.Mock(spec_set=getattr(module, name)) instance_mock = mock.Mock(spec_set=getattr(module, name))

View File

@@ -91,8 +91,12 @@ def list_examples():
class ExamplesTestCase(taskflow.test.TestCase): class ExamplesTestCase(taskflow.test.TestCase):
@classmethod @classmethod
def update(cls): def update(cls):
"""For each example, adds on a test method that the testing framework """For each example, adds on a test method.
will then run.
This newly created test method will then be activated by the testing
framework when it scans for and runs tests. This makes for a elegant
and simple way to ensure that all of the provided examples
actually work.
""" """
def add_test_method(name, method_name): def add_test_method(name, method_name):
def test_example(self): def test_example(self):

View File

@@ -1,15 +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.

View File

@@ -54,9 +54,7 @@ from taskflow.utils import lock_utils
def _get_connect_string(backend, user, passwd, database=None, variant=None): def _get_connect_string(backend, user, passwd, database=None, variant=None):
"""Try to get a connection with a very specific set of values, if we get """Forms a sqlalchemy database uri string for the given values."""
these then we'll run the tests, otherwise they are skipped.
"""
if backend == "postgres": if backend == "postgres":
if not variant: if not variant:
variant = 'psycopg2' variant = 'psycopg2'

View File

@@ -29,7 +29,7 @@ from taskflow.utils import lock_utils
NAPPY_TIME = 0.05 NAPPY_TIME = 0.05
# We will spend this amount of time doing some "fake" work. # We will spend this amount of time doing some "fake" work.
WORK_TIMES = [(0.01 + x/100.0) for x in range(0, 5)] WORK_TIMES = [(0.01 + x / 100.0) for x in range(0, 5)]
def _find_overlaps(times, start, end): def _find_overlaps(times, start, end):

View File

@@ -31,8 +31,7 @@ class DiGraph(nx.DiGraph):
return self return self
def get_edge_data(self, u, v, default=None): def get_edge_data(self, u, v, default=None):
"""Returns a *copy* of the attribute dictionary associated with edges """Returns a *copy* of the edge attribute dictionary between (u, v).
between (u, v).
NOTE(harlowja): this differs from the networkx get_edge_data() as that NOTE(harlowja): this differs from the networkx get_edge_data() as that
function does not return a copy (but returns a reference to the actual function does not return a copy (but returns a reference to the actual
@@ -48,7 +47,9 @@ class DiGraph(nx.DiGraph):
return nx.topological_sort(self) return nx.topological_sort(self)
def pformat(self): def pformat(self):
"""Pretty formats your graph into a string representation that includes """Pretty formats your graph into a string.
This pretty formatted string representation includes many useful
details about your graph, including; name, type, frozeness, node count, details about your graph, including; name, type, frozeness, node count,
nodes, edge count, edges, graph density and graph cycles (if any). nodes, edge count, edges, graph density and graph cycles (if any).
""" """

View File

@@ -46,8 +46,12 @@ def finalize_client(client):
def check_compatible(client, min_version=None, max_version=None): def check_compatible(client, min_version=None, max_version=None):
"""Checks if a kazoo client is backed by a zookeeper server version """Checks if a kazoo client is backed by a zookeeper server version.
that satisfies a given min (inclusive) and max (inclusive) version range.
This check will verify that the zookeeper server version that the client
is connected to satisfies a given minimum version (inclusive) and
maximum (inclusive) version range. If the server is not in the provided
version range then a exception is raised indiciating this.
""" """
server_version = None server_version = None
if min_version: if min_version:

View File

@@ -48,9 +48,16 @@ def try_lock(lock):
def locked(*args, **kwargs): def locked(*args, **kwargs):
"""A decorator that looks for a given attribute (typically a lock or a list """A locking decorator.
of locks) and before executing the decorated function uses the given lock
or list of locks as a context manager, automatically releasing on exit. 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 is provided then by default the attribute
named '_lock' is looked for.
""" """
def decorator(f): def decorator(f):
@@ -244,8 +251,11 @@ class ReaderWriterLock(_ReaderWriterLockBase):
class DummyReaderWriterLock(_ReaderWriterLockBase): class DummyReaderWriterLock(_ReaderWriterLockBase):
"""A dummy reader/writer lock that doesn't lock anything but provides same """A dummy reader/writer lock.
functions as a normal reader/writer lock class.
This dummy lock doesn't lock anything but provides the same functions as a
normal reader/writer lock class and can be useful in unit tests or other
similar scenarios (do *not* use it if locking is actually required).
""" """
@contextlib.contextmanager @contextlib.contextmanager
def write_lock(self): def write_lock(self):
@@ -271,11 +281,10 @@ class DummyReaderWriterLock(_ReaderWriterLockBase):
class MultiLock(object): class MultiLock(object):
"""A class which can attempt to obtain many locks at once and release """A class which attempts to obtain & release many locks at once.
said locks when exiting.
Useful as a context manager around many locks (instead of having to nest It is typically useful as a context manager around many locks (instead of
said individual context managers). having to nest individual lock context managers).
""" """
def __init__(self, locks): def __init__(self, locks):
@@ -318,7 +327,9 @@ class MultiLock(object):
class _InterProcessLock(object): class _InterProcessLock(object):
"""Lock implementation which allows multiple locks, working around """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 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 not require any cleanup. Since the lock is always held on a file
descriptor rather than outside of the process, the lock gets dropped descriptor rather than outside of the process, the lock gets dropped

View File

@@ -50,9 +50,11 @@ _SCHEME_REGEX = re.compile(r"^([A-Za-z][A-Za-z0-9+.-]*):")
def merge_uri(uri_pieces, conf): def merge_uri(uri_pieces, conf):
"""Merges the username, password, hostname, and query params of a uri into """Merges a parsed uri into the given configuration dictionary.
the given configuration (does not overwrite the configuration keys if they
already exist) and returns the adjusted configuration. 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.
NOTE(harlowja): does not merge the path, scheme or fragment. NOTE(harlowja): does not merge the path, scheme or fragment.
""" """
@@ -72,9 +74,7 @@ def merge_uri(uri_pieces, conf):
def parse_uri(uri, query_duplicates=False): def parse_uri(uri, query_duplicates=False):
"""Parses a uri into its components and returns a dictionary containing """Parses a uri into its components."""
those components.
"""
# Do some basic validation before continuing... # Do some basic validation before continuing...
if not isinstance(uri, six.string_types): if not isinstance(uri, six.string_types):
raise TypeError("Can only parse string types to uri data, " raise TypeError("Can only parse string types to uri data, "
@@ -176,9 +176,17 @@ def decode_json(raw_data, root_types=(dict,)):
class cachedproperty(object): class cachedproperty(object):
"""Descriptor that can be placed on instance methods to translate """A descriptor property that is only evaluated once..
This caching descriptor can be placed on instance methods to translate
those methods into properties that will be cached in the instance (avoiding those methods into properties that will be cached in the instance (avoiding
repeated creation checking logic to do the equivalent). repeated attribute checking logic to do the equivalent).
NOTE(harlowja): by default the property that will be saved will be under
the decorated methods name prefixed with an underscore. For example if we
were to attach this descriptor to an instance method 'get_thing(self)' the
cached property would be stored under '_get_thing' in the self object
after the first call to 'get_thing' occurs.
""" """
def __init__(self, fget): def __init__(self, fget):
# If a name is provided (as an argument) then this will be the string # If a name is provided (as an argument) then this will be the string
@@ -226,8 +234,10 @@ def wallclock():
def wraps(fn): def wraps(fn):
"""This will not be needed in python 3.2 or greater which already has this """Wraps a method and ensures the __wrapped__ attribute is set.
built-in to its functools.wraps method.
NOTE(harlowja): This will not be needed in python 3.2 or greater which
already has this built-in to its functools.wraps method.
""" """
def wrapper(f): def wrapper(f):
@@ -239,9 +249,7 @@ def wraps(fn):
def millis_to_datetime(milliseconds): def millis_to_datetime(milliseconds):
"""Converts a given number of milliseconds from the epoch into a datetime """Converts number of milliseconds (from epoch) into a datetime object."""
object.
"""
return datetime.datetime.fromtimestamp(float(milliseconds) / 1000) return datetime.datetime.fromtimestamp(float(milliseconds) / 1000)
@@ -313,9 +321,7 @@ _ASCII_WORD_SYMBOLS = frozenset(string.ascii_letters + string.digits + '_')
def is_valid_attribute_name(name, allow_self=False, allow_hidden=False): def is_valid_attribute_name(name, allow_self=False, allow_hidden=False):
"""Validates that a string name is a valid/invalid python attribute """Checks that a string is a valid/invalid python attribute name."""
name.
"""
return all(( return all((
isinstance(name, six.string_types), isinstance(name, six.string_types),
len(name) > 0, len(name) > 0,
@@ -332,8 +338,12 @@ def is_valid_attribute_name(name, allow_self=False, allow_hidden=False):
class AttrDict(dict): class AttrDict(dict):
"""Helper utility dict sub-class to create a class that can be accessed by """Dictionary subclass that allows for attribute based access.
attribute name from a dictionary that contains a set of keys and values.
This subclass allows for accessing a dictionaries keys and values by
accessing those keys as regular attributes. Keys that are not valid python
attribute names can not of course be acccessed/set (those keys must be
accessed/set by the traditional dictionary indexing operators instead).
""" """
NO_ATTRS = tuple(reflection.get_member_names(dict)) NO_ATTRS = tuple(reflection.get_member_names(dict))
@@ -392,9 +402,12 @@ class Timeout(object):
class ExponentialBackoff(object): class ExponentialBackoff(object):
"""An iterable object that will yield back an exponential delay sequence """An iterable object that will yield back an exponential delay sequence.
provided an exponent and a number of items to yield. This object may be
iterated over multiple times (yielding the same sequence each time). This objects provides for a configurable exponent, count of numbers
to generate, and a maximum number that will be returned. This object may
also be iterated over multiple times (yielding the same sequence each
time).
""" """
def __init__(self, count, exponent=2, max_backoff=3600): def __init__(self, count, exponent=2, max_backoff=3600):
self.count = max(0, int(count)) self.count = max(0, int(count))
@@ -541,9 +554,12 @@ class StopWatch(object):
class Notifier(object): class Notifier(object):
"""A utility helper class that can be used to subscribe to """A notification helper class.
notifications of events occurring as well as allow a entity to post said
notifications to subscribers. 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
associated subscribers without having either entity care about how this
notification occurs.
""" """
RESERVED_KEYS = ('details',) RESERVED_KEYS = ('details',)
@@ -665,12 +681,15 @@ def are_equal_exc_info_tuples(ei1, ei2):
@contextlib.contextmanager @contextlib.contextmanager
def capture_failure(): def capture_failure():
"""Save current exception, and yield back the failure (or raises a """Captures the occuring exception and provides a failure back.
runtime error if no active exception is being handled).
In some cases the exception context can be cleared, resulting in None This will save the current exception information and yield back a
being attempted to be saved after an exception handler is run. This failure object for the caller to use (it will raise a runtime error if
can happen when eventlet switches greenthreads or when running an no active exception is being handled).
This is useful since in some cases the exception context can be cleared,
resulting in None being attempted to be saved after an exception handler is
run. This can happen when eventlet switches greenthreads or when running an
exception handler, code raises and catches an exception. In both exception handler, code raises and catches an exception. In both
cases the exception context will be cleared. cases the exception context will be cleared.

View File

@@ -39,11 +39,10 @@ def temporary_log_book(backend=None):
def temporary_flow_detail(backend=None): def temporary_flow_detail(backend=None):
"""Creates a temporary flow detail and logbook for temporary usage in """Creates a temporary flow detail and logbook in the given backend.
the given backend.
Mainly useful for tests and other use cases where a temporary flow detail Mainly useful for tests and other use cases where a temporary flow detail
is needed for a short-period of time. and a temporary logbook is needed for a short-period of time.
""" """
flow_id = uuidutils.generate_uuid() flow_id = uuidutils.generate_uuid()
book = temporary_log_book(backend) book = temporary_log_book(backend)
@@ -57,9 +56,18 @@ def temporary_flow_detail(backend=None):
def create_flow_detail(flow, book=None, backend=None, meta=None): def create_flow_detail(flow, book=None, backend=None, meta=None):
"""Creates a flow detail for the given flow and adds it to the provided """Creates a flow detail for a flow & adds & saves it in a logbook.
logbook (if provided) and then uses the given backend (if provided) to
save the logbook then returns the created flow detail. This will create a flow detail for the given flow using the flow name,
and add it to the provided logbook and then uses the given backend to save
the logbook and then returns the created flow detail.
If no book is provided a temporary one will be created automatically (no
reference to the logbook will be returned, so this should nearly *always*
be provided or only used in situations where no logbook is needed, for
example in tests). If no backend is provided then no saving will occur and
the created flow detail will not be persisted even if the flow detail was
added to a given (or temporarily generated) logbook.
""" """
flow_id = uuidutils.generate_uuid() flow_id = uuidutils.generate_uuid()
flow_name = getattr(flow, 'name', None) flow_name = getattr(flow, 'name', None)

View File

@@ -31,8 +31,11 @@ def _get_members(obj, exclude_hidden):
def find_subclasses(locations, base_cls, exclude_hidden=True): def find_subclasses(locations, base_cls, exclude_hidden=True):
"""Examines the given locations for types which are subclasses of the base """Finds subclass types in the given locations.
class type provided and returns the found subclasses.
This will examines the given locations for types which are subclasses of
the base class type provided and returns the found subclasses (or fails
with exceptions if this introspection can not be accomplished).
If a string is provided as one of the locations it will be imported and If a string is provided as one of the locations it will be imported and
examined if it is a subclass of the base class. If a module is given, examined if it is a subclass of the base class. If a module is given,

View File

@@ -1,4 +1,4 @@
hacking>=0.8.0,<0.9 hacking>=0.9.1,<0.10
discover discover
coverage>=3.6 coverage>=3.6
mock>=1.0 mock>=1.0

View File

@@ -39,6 +39,11 @@ commands = python setup.py testr --coverage --testr-args='{posargs}'
commands = {posargs} commands = {posargs}
[flake8] [flake8]
# E265 block comment should start with '# '
# H305 imports not grouped correctly
# H307 like imports should be grouped together
# H904 Wrap long lines in parentheses instead of a backslash
ignore = H307,H305,H904,E265
builtins = _ builtins = _
exclude = .venv,.tox,dist,doc,./taskflow/openstack/common,*egg,.git,build,tools exclude = .venv,.tox,dist,doc,./taskflow/openstack/common,*egg,.git,build,tools

View File

@@ -68,6 +68,7 @@ commands = python setup.py testr --coverage --testr-args='{posargs}'
commands = {posargs} commands = {posargs}
[flake8] [flake8]
ignore = H307,H305,H904,E265
builtins = _ builtins = _
exclude = .venv,.tox,dist,doc,./taskflow/openstack/common,*egg,.git,build,tools exclude = .venv,.tox,dist,doc,./taskflow/openstack/common,*egg,.git,build,tools