Added a top level SpiffWorklowException that all exceptions inherit from. Aside from a message string you can append information to these exceptions with "add_note", which is a new method that all exceptions have starting in python 3.11

Switched arguments to the WorkflowException, WorkflowTaskException - which now always takes a string message as the first argument, and named arguments thereafter to be consistent with all other error messages in Python.
Consistently raise ValidationExceptions whenever we encounter an error anywhere during parsing of xml.
The BPMN/WorkflowTaskExecException is removed, in favor of just calling a WorkflowTaskException.  There is nothing BPMN Specific in the logic, so no need for this.
Consolidated error message logic so that things like "Did you mean" just get added by default if possible.  So we don't have to separately deal with that logic each time.
Better Error messages for DMN (include row number as a part of the error information)

Validators now catch more general exceptions from lxml
When catching errors, check to see if they are SpiffWorkflow errors already, and if so, just add a note to help provide additional context for the error.
This commit is contained in:
Dan 2023-01-16 15:13:45 -05:00
parent be26100bcb
commit 8119abd140
42 changed files with 650 additions and 251 deletions

View File

@ -168,7 +168,7 @@ def feelParseISODuration(input):
"""
if input[0] != 'P':
raise Exception("Oh Crap!")
raise Exception("ISO Duration format must begin with the letter P")
input = input[1:]
days, time = input.split("T")
lookups = [("Y",days,timedelta(days=365)),
@ -239,7 +239,7 @@ fixes = [(r'string\s+length\((.+?)\)','len(\\1)'),
('true','True'),
('false','False')
]
externalFuncs = {
'feelConvertTime':feelConvertTime,
'FeelInterval':FeelInterval,

View File

@ -4,7 +4,7 @@ import copy
import sys
import traceback
from SpiffWorkflow.bpmn.exceptions import WorkflowTaskExecException
from ..exceptions import SpiffWorkflowException, WorkflowTaskException
from ..operators import Operator
@ -118,10 +118,11 @@ class PythonScriptEngine(object):
return expression._matches(task)
else:
return self._evaluate(expression, task.data, external_methods)
except SpiffWorkflowException as se:
se.add_note(f"Error evaluating expression '{expression}'")
raise se
except Exception as e:
raise WorkflowTaskExecException(task,
f"Error evaluating expression {expression}",
e)
raise WorkflowTaskException(f"Error evaluating expression '{expression}'", task=task, exception=e)
def execute(self, task, script, external_methods=None):
"""
@ -141,13 +142,17 @@ class PythonScriptEngine(object):
raise NotImplementedError("To call external services override the script engine and implement `call_service`.")
def create_task_exec_exception(self, task, script, err):
if isinstance(err, WorkflowTaskExecException):
if isinstance(err, SpiffWorkflowException):
line_number, error_line = self.get_error_line_number_and_content(script)
err.add_note(f"Python script error on line {line_number}: '{error_line}'")
return err
detail = err.__class__.__name__
if len(err.args) > 0:
detail += ":" + err.args[0]
line_number, error_line = self.get_error_line_number_and_content(script)
return WorkflowTaskException(detail, task=task, line_number=line_number, error_line=error_line)
def get_error_line_number_and_content(self, script):
line_number = 0
error_line = ''
cl, exc, tb = sys.exc_info()
@ -158,8 +163,7 @@ class PythonScriptEngine(object):
if frame_summary.filename == '<string>':
line_number = frame_summary.lineno
error_line = script.splitlines()[line_number - 1]
return WorkflowTaskExecException(task, detail, err, line_number,
error_line)
return line_number, error_line
def check_for_overwrite(self, task, external_methods):
"""It's possible that someone will define a variable with the
@ -172,7 +176,7 @@ class PythonScriptEngine(object):
msg = f"You have task data that overwrites a predefined " \
f"function(s). Please change the following variable or " \
f"field name(s) to something else: {func_overwrites}"
raise WorkflowTaskExecException(task, msg)
raise WorkflowTaskException(msg, task=task)
def convert_to_box(self, data):
if isinstance(data, dict):

View File

@ -1,50 +1,4 @@
import re
from SpiffWorkflow.exceptions import WorkflowException, WorkflowTaskException
from SpiffWorkflow.util import levenshtein
class WorkflowTaskExecException(WorkflowTaskException):
"""
Exception during execution of task "payload". For example:
* ScriptTask during execution of embedded script,
* ServiceTask during external service call.
"""
def __init__(self, task, error_msg, exception=None, line_number=0, error_line=""):
"""
Exception initialization.
:param task: the task that threw the exception
:type task: Task
:param exception: a human readable error message
:type exception: Exception
"""
self.offset = 0
self.line_number = line_number
self.error_line = error_line
if isinstance(exception, SyntaxError):
# Prefer line number from syntax error if available.
self.line_number = exception.lineno
self.offset = exception.offset
elif isinstance(exception, NameError):
def_match = re.match("name '(.+)' is not defined", str(exception))
if def_match:
bad_variable = re.match("name '(.+)' is not defined", str(exception)).group(1)
most_similar = levenshtein.most_similar(bad_variable, task.data.keys(), 3)
error_msg = f'something you are referencing does not exist: ' \
f'"{exception}".'
if len(most_similar) == 1:
error_msg += f' Did you mean \'{most_similar[0]}\'?'
if len(most_similar) > 1:
error_msg += f' Did you mean one of \'{most_similar}\'?'
else:
error_msg = str(exception)
super().__init__(task, error_msg, exception)
from SpiffWorkflow.exceptions import WorkflowException
class WorkflowDataException(WorkflowException):

View File

@ -21,7 +21,7 @@ import glob
import os
from lxml import etree
from lxml.etree import DocumentInvalid
from lxml.etree import DocumentInvalid, LxmlError
from SpiffWorkflow.bpmn.specs.events.event_definitions import NoneEventDefinition
@ -72,8 +72,10 @@ class BpmnValidator:
def validate(self, bpmn, filename=None):
try:
self.validator.assertValid(bpmn)
except DocumentInvalid as di:
raise DocumentInvalid(str(di) + "file: " + filename)
except LxmlError as le:
last_error = self.validator.error_log.last_error
raise ValidationException(last_error.message, filename=filename,
sourceline=last_error.line)
class BpmnParser(object):
"""

View File

@ -197,8 +197,12 @@ class TaskParser(NodeParser):
self.connect_outgoing(c, sequence_flow, sequence_flow.get('id') == default_outgoing)
return parent if boundary_event_nodes else self.task
except ValidationException:
raise
except ValidationException as ve:
exc_info = sys.exc_info()
tb = "".join(traceback.format_exception(
exc_info[0], exc_info[1], exc_info[2]))
ve.add_note(tb)
raise ve
except Exception as ex:
exc_info = sys.exc_info()
tb = "".join(traceback.format_exception(

View File

@ -1,5 +1,5 @@
# -*- coding: utf-8 -*-
# Copyright (C) 2012 Matthew Hampton
# Copyright (C) 2012 Matthew Hampton, 2023 Dan Funk
#
# This library is free software; you can redistribute it and/or
# modify it under the terms of the GNU Lesser General Public
@ -17,10 +17,10 @@
# 02110-1301 USA
from .util import BPMN_MODEL_NS
from ...exceptions import SpiffWorkflowException
class ValidationException(Exception):
class ValidationException(SpiffWorkflowException):
"""
A ValidationException should be thrown with enough information for the user
to diagnose the problem and sort it out.
@ -31,20 +31,28 @@ class ValidationException(Exception):
def __init__(self, msg, node=None, filename=None, *args, **kwargs):
if node is not None:
self.tag = self._shorten_tag(node.tag)
self.id = node.get('id', '<Unknown>')
self.name = node.get('name', '<Unknown>')
self.sourceline = getattr(node, 'sourceline', '<Unknown>')
self.id = node.get('id', '')
self.name = node.get('name', '')
self.sourceline = getattr(node, 'sourceline', '')
else:
self.tag = '<Unknown>'
self.id = '<Unknown>'
self.name = '<Unknown>'
self.sourceline = '<Unknown>'
self.filename = filename or '<Unknown File>'
message = ('%s\nSource Details: '
'%s (id:%s), name \'%s\', line %s in %s') % (
msg, self.tag, self.id, self.name, self.sourceline, self.filename)
self.tag = kwargs.get('tag', '')
self.id = kwargs.get('id', '')
self.name = kwargs.get('name', '')
self.sourceline = kwargs.get('sourceline', '')
self.filename = filename or ''
message = msg
if self.tag:
msg += f", tag: {self.tag}"
if self.id:
msg += f", id: {self.id}"
if self.name:
msg += f", name: {self.name}"
if self.sourceline:
msg += f", line #: {self.sourceline}"
if self.filename:
msg += f", file: {self.filename}"
super(ValidationException, self).__init__(message, *args, **kwargs)
super(ValidationException, self).__init__(message, *args)
@classmethod
def _shorten_tag(cls, tag):

View File

@ -41,11 +41,11 @@ class ExclusiveGateway(ExclusiveChoice, BpmnSpecMixin):
# raise WorkflowException(self, 'At least one output required.')
for condition, name in self.cond_task_specs:
if name is None:
raise WorkflowException(self, 'Condition with no task spec.')
raise WorkflowException('Condition with no task spec.', sender=self)
task_spec = self._wf_spec.get_task_spec_from_name(name)
if task_spec is None:
msg = 'Condition leads to non-existent task ' + repr(name)
raise WorkflowException(self, msg)
raise WorkflowException(msg, sender=self)
if condition is None:
continue

View File

@ -22,12 +22,12 @@ from builtins import range
from uuid import uuid4
import re
from SpiffWorkflow.bpmn.exceptions import WorkflowTaskExecException
from .SubWorkflowTask import SubWorkflowTask, CallActivity
from .ParallelGateway import ParallelGateway
from .ScriptTask import ScriptTask
from .ExclusiveGateway import ExclusiveGateway
from ...dmn.specs.BusinessRuleTask import BusinessRuleTask
from ...exceptions import WorkflowTaskException
from ...operators import valueof, is_number
from ...specs.SubWorkflow import SubWorkflow
from ...specs.base import TaskSpec
@ -396,9 +396,10 @@ class MultiInstanceTask(TaskSpec):
# look for variable in context, if we don't find it, default to 1
variable = valueof(my_task, self.times, 1)
if self.times.name == self.collection.name and type(variable) == type([]):
raise WorkflowTaskExecException(my_task,
'If we are updating a collection, then the collection must be a dictionary.')
raise WorkflowTaskException(
'If we are updating a collection, then the collection must be a dictionary.',
task=my_task)
def _get_current_var(self, my_task, pos):
variable = valueof(my_task, self.times, 1)
if is_number(variable):
@ -418,7 +419,7 @@ class MultiInstanceTask(TaskSpec):
msg = f"There is a mismatch between runtimes and the number " \
f"items in the collection, please check for empty " \
f"collection {self.collection.name}."
raise WorkflowTaskExecException(my_task, msg)
raise WorkflowTaskException(msg, task=my_task)
runtimesvar = keys[runtimes - 1]
else:
# Use an integer (for arrays)

View File

@ -56,9 +56,8 @@ class UnstructuredJoin(Join, BpmnSpecMixin):
task._has_state(TaskState.WAITING) or task == my_task):
if task.parent.task_spec in completed_inputs:
raise(WorkflowException
(task.task_spec,
"Unsupported looping behaviour: two threads waiting"
" on the same sequence flow."))
("Unsupported looping behaviour: two threads waiting"
" on the same sequence flow.", sender=self))
completed_inputs.add(task.parent.task_spec)
else:
waiting_tasks.append(task.parent)

View File

@ -218,12 +218,11 @@ class BpmnWorkflow(Workflow):
def _find_task(self, task_id):
if task_id is None:
raise WorkflowException(self.spec, 'task_id is None')
raise WorkflowException('task_id is None', sender=self.spec)
for task in self.get_tasks():
if task.id == task_id:
return task
raise WorkflowException(self.spec,
f'A task with the given task_id ({task_id}) was not found')
raise WorkflowException(f'A task with the given task_id ({task_id}) was not found', sender=self.spec)
def complete_task_from_id(self, task_id):
# I don't even know why we use this stupid function instead of calling task.complete,

View File

@ -44,7 +44,7 @@ class UserTaskParser(TaskParser):
"""
Base class for parsing User Tasks
"""
def __init__(self, process_parser, spec_class, node, lane=None):
nsmap = DEFAULT_NSMAP.copy()
nsmap.update({'camunda': CAMUNDA_MODEL_NS})
@ -63,7 +63,7 @@ class UserTaskParser(TaskParser):
form = Form()
try:
form.key = self.node.attrib['{' + CAMUNDA_MODEL_NS + '}formKey']
except (KeyError):
except KeyError:
return form
for xml_field in self.xpath('.//camunda:formData/camunda:formField'):
if xml_field.get('type') == 'enum':

View File

@ -2,6 +2,7 @@ import logging
import re
from ..specs.model import HitPolicy
from ...exceptions import SpiffWorkflowException, WorkflowTaskException
from ...util import levenshtein
from ...workflow import WorkflowException
@ -54,18 +55,11 @@ class DMNEngine:
try:
if not self.evaluate(input_val, lhs, task):
return False
except NameError as e:
# Add a bit of info, re-raise as Name Error
raise NameError(str(e) + "Failed to execute "
"expression: '%s' is '%s' in the "
"Row with annotation '%s'")
except WorkflowException as we:
raise we
except SpiffWorkflowException as se:
se.add_note(f"Rule failed on row {rule.row_number}")
raise se
except Exception as e:
raise Exception("Failed to execute "
"expression: '%s' is '%s' in the "
"Row with annotation '%s', %s" % (
input_val, lhs, rule.description, str(e)))
raise WorkflowTaskException(f"Failed to execute DMN Rule on row {rule.row_number}", task=task, exception=e)
else:
# Empty means ignore decision value
continue # Check the other operators/columns
@ -111,10 +105,7 @@ class DMNEngine:
external_methods=external_methods)
# The input expression just has to be something that can be parsed as is by the engine.
try:
script_engine.validate(input_expr)
except Exception as e:
raise WorkflowException(f"Input Expression '{input_expr}' is malformed. " + str(e))
script_engine.validate(input_expr)
# If we get here, we need to check whether the match expression includes
# an operator or if can use '=='

View File

@ -1,6 +1,7 @@
import ast
from SpiffWorkflow.bpmn.parser.node_parser import NodeParser, DEFAULT_NSMAP
from ...bpmn.parser.ValidationException import ValidationException
from ...bpmn.parser.util import xpath_eval
@ -69,25 +70,20 @@ class DMNParser(NodeParser):
def _parse_decision(self, root):
decision_elements = list(root)
if len(decision_elements) == 0:
raise Exception('No decisions found')
raise ValidationException('No decisions found', filename=self.filename,
node=root)
if len(decision_elements) > 1:
raise Exception('Multiple decisions found')
raise ValidationException('Multiple decision tables are not current supported.',
filename=self.filename, node=root)
decision_element = decision_elements[0]
assert decision_element.tag.endswith(
'decision'), 'Element %r is not of type "decision"' % (
decision_element.tag)
decision = Decision(decision_element.attrib['id'],
decision_element.attrib.get('name', ''))
# Parse decision tables
try:
self._parse_decision_tables(decision, decision_element)
except Exception as e:
raise Exception(
"Error in Decision '%s': %s" % (decision.name, str(e)))
self._parse_decision_tables(decision, decision_element)
return decision
@ -104,6 +100,7 @@ class DMNParser(NodeParser):
def _parse_inputs_outputs(self, decisionTable,
decisionTableElement):
rule_counter = 0
for element in decisionTableElement:
if element.tag.endswith('input'):
e_input = self._parse_input(element)
@ -112,11 +109,13 @@ class DMNParser(NodeParser):
output = self._parse_output(element)
decisionTable.outputs.append(output)
elif element.tag.endswith('rule'):
rule = self._parse_rule(decisionTable, element)
rule_counter += 1
rule = self._parse_rule(decisionTable, element, rule_counter)
decisionTable.rules.append(rule)
else:
raise Exception(
'Unknown type in decision table: %r' % element.tag)
raise ValidationException(
'Unknown type in decision table: %r' % element.tag,
node=element, filename=self.filename)
def _parse_input(self, input_element):
type_ref = None
@ -142,9 +141,9 @@ class DMNParser(NodeParser):
outputElement.attrib.get('typeRef', ''))
return output
def _parse_rule(self, decisionTable, ruleElement):
def _parse_rule(self, decisionTable, ruleElement, rowNumber):
rule = Rule(ruleElement.attrib['id'])
rule.row_number = rowNumber
input_idx = 0
output_idx = 0
for child in ruleElement:
@ -189,6 +188,7 @@ class DMNParser(NodeParser):
try:
ast.parse(entry.text)
except Exception as e:
raise Exception(
"Malformed Output Expression '%s'. %s " % (entry.text, str(e)))
raise ValidationException(
"Malformed Output Expression '%s'. %s " % (entry.text, str(e)),
node=element, filename=self.filename)
return entry

View File

@ -47,6 +47,7 @@ class BusinessRuleTaskConverter(BpmnTaskSpecConverter):
def rule_to_dict(self, rule):
return {
'id': rule.id,
'row_number': rule.row_number,
'description': rule.description,
'input_entries': [self.input_entry_to_dict(entry) for entry in rule.inputEntries],
'output_entries': [self.output_entry_to_dict(entry) for entry in rule.outputEntries],
@ -91,6 +92,7 @@ class BusinessRuleTaskConverter(BpmnTaskSpecConverter):
def rule_from_dict(self, dct, inputs, outputs):
rule = Rule(dct['id'])
rule.description = dct['description']
rule.row_number = dct['row_number']
rule.inputEntries = [self.input_entry_from_dict(entry, inputs)
for entry in dct['input_entries']]
rule.outputEntries = [self.output_entry_from_dict(entry, outputs)

View File

@ -1,4 +1,5 @@
from SpiffWorkflow.bpmn.exceptions import WorkflowTaskExecException
from SpiffWorkflow.exceptions import WorkflowTaskException, WorkflowException, \
SpiffWorkflowException
from ...specs.Simple import Simple
@ -29,5 +30,8 @@ class BusinessRuleTask(Simple, BpmnSpecMixin):
my_task.data = DeepMerge.merge(my_task.data,
self.dmnEngine.result(my_task))
super(BusinessRuleTask, self)._on_complete_hook(my_task)
except SpiffWorkflowException as we:
we.add_note(f"Business Rule Task '{my_task.task_spec.description}'.")
raise we
except Exception as e:
raise WorkflowTaskExecException(my_task, str(e))
raise WorkflowTaskExecException("Unexpected error in Business Rule", task=my_task) from e

View File

@ -75,7 +75,7 @@ class OutputEntry:
class Rule:
def __init__(self, id):
self.id = id
self.row_number = 0
self.description = ''
self.inputEntries = []
self.outputEntries = []

View File

@ -15,23 +15,43 @@
# License along with this library; if not, write to the Free Software
# Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA
# 02110-1301 USA
import re
from SpiffWorkflow.util import levenshtein
class WorkflowException(Exception):
class SpiffWorkflowException(Exception):
"""
Base class for all SpiffWorkflow-generated exceptions.
"""
def __init__(self, msg):
super().__init__(msg)
self.notes = []
def add_note(self, note):
"""add_note is a python 3.11 feature, this can be removed when we
stop supporting versions prior to 3.11"""
self.notes.append(note)
def __str__(self):
return super().__str__() + ". " + ". ".join(self.notes)
class WorkflowException(SpiffWorkflowException):
"""
Base class for all SpiffWorkflow-generated exceptions.
"""
def __init__(self, sender, error):
def __init__(self, message, sender=None):
"""
Standard exception class.
:param sender: the task spec that threw the exception
:type sender: TaskSpec
:param error: a human readable error message
:param error: a human-readable error message
:type error: string
"""
Exception.__init__(self, str(error))
super().__init__(str(message))
# Points to the TaskSpec that generated the exception.
self.sender = sender
@ -45,10 +65,32 @@ class WorkflowException(Exception):
task_trace.append(f"{workflow.spec.task_specs[caller].description} ({workflow.spec.file})")
return task_trace
@staticmethod
def did_you_mean_from_name_error(name_exception, options):
"""Returns a string along the lines of 'did you mean 'dog'? Given
a name_error, and a set of possible things that could have been called,
or an empty string if no valid suggestions come up. """
if isinstance(name_exception, NameError):
def_match = re.match("name '(.+)' is not defined", str(name_exception))
if def_match:
bad_variable = re.match("name '(.+)' is not defined",
str(name_exception)).group(1)
most_similar = levenshtein.most_similar(bad_variable,
options, 3)
error_msg = f'This variable or function does not exist: ' \
f'"{bad_variable}".'
if len(most_similar) == 1:
error_msg += f' Did you mean \'{most_similar[0]}\'?'
if len(most_similar) > 1:
error_msg += f' Did you mean one of \'{most_similar}\'?'
return error_msg
class WorkflowTaskException(WorkflowException):
"""WorkflowException that provides task_trace information."""
def __init__(self, task, error_msg, exception=None):
def __init__(self, error_msg, task=None, exception=None,
line_number=None, offset=None, error_line=None):
"""
Exception initialization.
@ -60,17 +102,27 @@ class WorkflowTaskException(WorkflowException):
:type exception: Exception
"""
self.exception = exception
self.task = task
self.line_number = line_number
self.offset = offset
self.error_line = error_line
super().__init__(error_msg, sender=task.task_spec)
# If encountered in a sub-workflow, this traces back up the stack
# so we can tell how we got to this paticular task, no matter how
if isinstance(exception, SyntaxError):
# Prefer line number from syntax error if available.
self.error_msg = f"Python syntax error. {str(exception)}"
self.line_number = exception.lineno
self.offset = exception.offset
elif isinstance(exception, NameError):
self.add_note(self.did_you_mean_from_name_error(exception, list(task.data.keys())))
# If encountered in a sub-workflow, this traces back up the stack,
# so we can tell how we got to this particular task, no matter how
# deeply nested in sub-workflows it is. Takes the form of:
# task-description (file-name)
self.task_trace = self.get_task_trace(task)
super().__init__(task.task_spec, error_msg)
class StorageException(Exception):
class StorageException(SpiffWorkflowException):
pass

View File

@ -225,7 +225,7 @@ class Operator(Term):
return values
def _matches(self, task):
raise Exception("Abstract class, do not call")
raise NotImplementedError("Abstract class, do not call")
def serialize(self, serializer):
"""

View File

@ -53,7 +53,7 @@ class Cancel(TaskSpec):
"""
TaskSpec.test(self)
if len(self.outputs) > 0:
raise WorkflowException(self, 'Cancel with an output.')
raise WorkflowException('Cancel with an output.', sender=self)
def _on_complete_hook(self, my_task):
my_task.workflow.cancel(self.cancel_successfully)

View File

@ -64,7 +64,7 @@ class ExclusiveChoice(MultiChoice):
"""
MultiChoice.test(self)
if self.default_task_spec is None:
raise WorkflowException(self, 'A default output is required.')
raise WorkflowException('A default output is required.', sender=self)
def _predict_hook(self, my_task):
# If the task's status is not predicted, we default to MAYBE
@ -84,8 +84,7 @@ class ExclusiveChoice(MultiChoice):
break
if output is None:
raise WorkflowException(self,
f'No conditions satisfied for {my_task.task_spec.name}')
raise WorkflowException(f'No conditions satisfied for {my_task.task_spec.name}', sender=self)
my_task._sync_children([output], TaskState.FUTURE)

View File

@ -154,7 +154,7 @@ class Join(TaskSpec):
split_task = my_task._find_ancestor_from_name(self.split_task)
if split_task is None:
msg = 'Join with %s, which was not reached' % self.split_task
raise WorkflowException(self, msg)
raise WorkflowException(msg, sender=self)
tasks = split_task.task_spec._get_activated_tasks(split_task, my_task)
# The default threshold is the number of branches that were started.

View File

@ -57,10 +57,11 @@ class LoopResetTask(TaskSpec):
# maybe upstream someone will be able to handle this situation
task._set_state(TaskState.WAITING)
if isinstance(e, WorkflowTaskException):
e.add_note('Error occurred during a loop back to a previous step.')
raise e
else:
raise WorkflowTaskException(
task, 'Error during loop back:' + str(e), e)
'Error during loop back:' + str(e), task=task, exception=e)
super(LoopResetTask, self)._on_complete_hook(task)
def serialize(self, serializer):

View File

@ -70,14 +70,14 @@ class MultiChoice(TaskSpec):
"""
TaskSpec.test(self)
if len(self.cond_task_specs) < 1:
raise WorkflowException(self, 'At least one output required.')
raise WorkflowException('At least one output required.', sender=self)
for condition, name in self.cond_task_specs:
if name is None:
raise WorkflowException(self, 'Condition with no task spec.')
raise WorkflowException('Condition with no task spec.', sender=self)
task_spec = self._wf_spec.get_task_spec_from_name(name)
if task_spec is None:
msg = 'Condition leads to non-existent task ' + repr(name)
raise WorkflowException(self, msg)
raise WorkflowException(msg, sender=self)
if condition is None:
continue

View File

@ -44,7 +44,7 @@ class StartTask(TaskSpec):
"""
Called by the previous task to let us know that it exists.
"""
raise WorkflowException(self, 'StartTask can not have any inputs.')
raise WorkflowException('StartTask can not have any inputs.', sender=self)
def test(self):
"""
@ -52,9 +52,9 @@ class StartTask(TaskSpec):
if an error was detected.
"""
if len(self.inputs) != 0:
raise WorkflowException(self, 'StartTask with an input.')
raise WorkflowException('StartTask with an input.', sender=self)
elif len(self.outputs) < 1:
raise WorkflowException(self, 'No output task connected.')
raise WorkflowException('No output task connected.', sender=self)
def serialize(self, serializer):
return serializer.serialize_start_task(self)

View File

@ -72,8 +72,7 @@ class SubWorkflow(TaskSpec):
def test(self):
TaskSpec.test(self)
if self.file is not None and not os.path.exists(self.file):
raise WorkflowException(
self, 'File does not exist: %s' % self.file)
raise WorkflowException('File does not exist: %s' % self.file, sender=self)
def _predict_hook(self, my_task):
# Modifying the task spec is a TERRIBLE idea, but if we don't do it, sync_children won't work

View File

@ -63,7 +63,7 @@ class ThreadMerge(Join):
split_task = my_task._find_ancestor_from_name(self.split_task)
if split_task is None:
msg = 'Join with %s, which was not reached' % self.split_task
raise WorkflowException(self, msg)
raise WorkflowException(msg, sender=self)
tasks = split_task.task_spec._get_activated_threads(split_task)
# The default threshold is the number of threads that were started.

View File

@ -182,8 +182,7 @@ class Task(object, metaclass=DeprecatedMetaTask):
# Assure we don't recurse forever.
self.count += 1
if self.count > self.MAX_ITERATIONS:
raise WorkflowException(current,
"Task Iterator entered infinite recursion loop" )
raise WorkflowException("Task Iterator entered infinite recursion loop", sender=current)
# If the current task has children, the first child is the next
@ -266,8 +265,8 @@ class Task(object, metaclass=DeprecatedMetaTask):
def state(self, value):
if value < self._state:
raise WorkflowException(
self.task_spec,
'state went from %s to %s!' % (self.get_state_name(), TaskStateNames[value])
'state went from %s to %s!' % (self.get_state_name(), TaskStateNames[value]),
sender=self.task_spec
)
self._set_state(value)
@ -345,8 +344,8 @@ class Task(object, metaclass=DeprecatedMetaTask):
if self.is_looping():
self.terminate_current_loop = True
else:
raise WorkflowException(self.task_spec,
'The method terminate_loop should only be called in the case of a BPMN Loop Task')
raise WorkflowException('The method terminate_loop should only be called in the case of a BPMN Loop Task',
sender=self)
def is_looping(self):
"""Returns true if this is a looping task."""
@ -475,7 +474,7 @@ class Task(object, metaclass=DeprecatedMetaTask):
raise ValueError(self, '_add_child() requires a TaskSpec')
if self._is_predicted() and state & TaskState.PREDICTED_MASK == 0:
msg = 'Attempt to add non-predicted child to predicted task'
raise WorkflowException(self.task_spec, msg)
raise WorkflowException(msg, sender=self.task_spec)
task = Task(self.workflow, task_spec, self, state=state)
task.thread_id = self.thread_id
if state == TaskState.READY:
@ -551,7 +550,7 @@ class Task(object, metaclass=DeprecatedMetaTask):
# Definite tasks must not be removed, so they HAVE to be in the given task spec list.
if child._is_definite():
raise WorkflowException(self.task_spec, f'removal of non-predicted child {child}')
raise WorkflowException(f'removal of non-predicted child {child}', sender=self.task_spec)
unneeded_children.append(child)
# Remove and add the children accordingly.

View File

@ -257,7 +257,7 @@ class Workflow(object):
:param task_id: The id of the Task object.
"""
if task_id is None:
raise WorkflowException(self.spec, 'task_id is None')
raise WorkflowException('task_id is None', sender=self.spec)
data = {}
if self.last_task and self.last_task.data:
data = self.last_task.data
@ -265,7 +265,7 @@ class Workflow(object):
if task.id == task_id:
return task.reset_token(data)
msg = 'A task with the given task_id (%s) was not found' % task_id
raise WorkflowException(self.spec, msg)
raise WorkflowException(msg, sender=self.spec)
def get_reset_task_spec(self, destination):
"""
@ -300,12 +300,12 @@ class Workflow(object):
:param task_id: The id of the Task object.
"""
if task_id is None:
raise WorkflowException(self.spec, 'task_id is None')
raise WorkflowException('task_id is None', sender=self.spec)
for task in self.task_tree:
if task.id == task_id:
return task.complete()
msg = 'A task with the given task_id (%s) was not found' % task_id
raise WorkflowException(self.spec, msg)
raise WorkflowException(msg, sender=self.spec)
def complete_next(self, pick_up=True, halt_on_manual=True):
"""

100
doc/errors.rst Normal file
View File

@ -0,0 +1,100 @@
SpiffWorkflow Exceptions
====================================
Details about the exceptions and exception hierarchy within SpiffWorkflow
SpiffWorkflowException
----------
Base exception for all exceptions raised by SpiffWorkflow
ValidationException
----------
**Extends**
SpiffWorkflowException
Thrown during the parsing of a workflow.
**Attributes/Methods**
- **tag**: The type of xml tag being parsed
- **id**: the id attribute of the xml tag, if available.
- **name**: the name attribute of the xml tag, if available.
- **sourceline**: the line number where the tag occurs.
- **filename**: The name of the file where the error occurred.
- **message**: a human readable error message.
WorkflowException
--------
When an error occurs with a Task Specification (maybe should have been called
a SpecException)
**Extends**
SpiffWorkflowException
**Attributes/Methods**
- **sender**: The TaskSpec - the specific Task, Gateway, etc... that caused the error to happen.
- **error**: a human readable error message describing the problem.
- **get_task_trace**: Provided a specific Task, will work it's way through the workflow / sub-processes
and call activities to show where an error occurred. Useful if the error happened within a deeply nested structure (where call activities include call activities ....)
WorkflowDataException
------------------
When an exception occurs moving data between tasks and Data Objects (including
data inputs and data outputs.)
**Extends**
WorkflowException
**Attributes/Methods**
(in addition to the values in a WorkflowException)
- **task**: The specific task (not the task spec, but the actual executing task)
- **data_input**: The spec of the input variable
- **data_output**: The spec of the output variable
WorkflowTaskException
--------
**Extends**
WorkflowException
**Attributes/Methods**
(in addition to the values in a WorkflowException)
- **task**: The specific task (not the task spec, but the actual executing task)
- **error_msg**: The detailed human readable message. (conflicts with error above)
- **exception**: The original exception this wraps around.
WorkflowTaskExecException
------------------
**Extends**
WorkflowTaskException
(in addition to the values in a WorkflowTaskException)
- **line_number** The line number that contains the error
- **offset** The point in the line that caused the error
- **error_line** The content of the line that caused the error.
It will accept the line_number and error_line as arguments - if the
underlying error provided is a SyntaxError it will try to derive this
information from the error.
If this is a name error, it will attempt to calculate a did-you-mean
error_msg.
Unused / Deprecated errors
--------------------
** StorageException **
Deprecated -- Used only by the PrettyXmlSerializer - which is not under active
support.
** UnrecoverableWorkflowChange **
Only in use by the outdated CompactWorkflowSerializer
** DeadMethodCalled **
Something related to WeakMethod -- which doesn't look to be utilized anymore.

View File

@ -26,13 +26,13 @@
showgrid="false"
showguides="true"
inkscape:guide-bbox="true"
inkscape:zoom="0.27433373"
inkscape:cx="-586.87643"
inkscape:cy="1882.7433"
inkscape:zoom="1.5518659"
inkscape:cx="2265.0153"
inkscape:cy="3541.2209"
inkscape:window-width="1916"
inkscape:window-height="1076"
inkscape:window-height="916"
inkscape:window-x="0"
inkscape:window-y="0"
inkscape:window-y="162"
inkscape:window-maximized="1"
inkscape:current-layer="layer1">
<sodipodi:guide
@ -70,6 +70,18 @@
position="30.565253,-441.07447"
orientation="1,0"
id="guide37861" />
<sodipodi:guide
position="700.40724,-579.16044"
orientation="0,-1"
id="guide49435" />
<sodipodi:guide
position="639.13561,-614.28335"
orientation="0,-1"
id="guide49437" />
<sodipodi:guide
position="734.70955,-597.73685"
orientation="1,0"
id="guide83679" />
</sodipodi:namedview>
<defs
id="defs2">
@ -839,6 +851,91 @@
x="-580.08496"
y="716.69928">Draw the code</tspan></text>
</g>
<path
id="path73257-7-2-9-159-9-8"
style="vector-effect:none;fill:#126d82;fill-opacity:1;fill-rule:evenodd;stroke:none;stroke-width:4.433;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-dasharray:none;stroke-dashoffset:0;stroke-opacity:1"
d="m 599.99956,911.28335 v 0.91595 c 0.96107,0.84548 1.66181,1.94689 2.02041,3.17567 h -17.54738 c -3.11685,0 -5.68164,2.56482 -5.68164,5.68159 v 3.91192 c 0,3.11682 2.56479,5.68164 5.68164,5.68164 h 5.6851 4.69113 5.6878 c 1.21493,0 2.13344,0.91846 2.13344,2.13344 v 3.91632 c 0,1.21498 -0.91851,2.13346 -2.13344,2.13346 h -14.79076 c -0.75878,-2.29982 -2.93713,-3.97943 -5.47735,-3.97943 -3.16218,0 -5.76138,2.60267 -5.76138,5.76489 0,3.1622 2.5992,5.76399 5.76138,5.76399 2.54974,0 4.73517,-1.69176 5.48615,-4.00482 h 14.78196 c 3.1168,0 5.67808,-2.5613 5.67808,-5.67809 v -3.91632 c 0,-3.11677 -2.56128,-5.68164 -5.67808,-5.68164 h -5.6878 -4.69113 -5.6851 c -1.21497,0 -2.13609,-0.91837 -2.13609,-2.13344 v -3.91192 c 0,-1.21499 0.92112,-2.13696 2.13609,-2.13696 h 17.60609 c -0.33391,1.31874 -1.05865,2.50576 -2.07912,3.4053 v 0.68721 l 11.72877,-5.86483 z m -19.73105,27.11871 c 1.24555,0 2.21936,0.97116 2.21936,2.21674 0,1.24556 -0.97381,2.21936 -2.21936,2.21936 -1.24559,0 -2.21675,-0.9738 -2.21675,-2.21936 0,-1.24558 0.97116,-2.21674 2.21675,-2.21674 z"
sodipodi:nodetypes="cccssssccsssscssscssssccsssscccccsssss"
inkscape:export-filename="/home/dan/spiffworkflow.png"
inkscape:export-xdpi="136.48"
inkscape:export-ydpi="136.48" />
<path
id="path85339"
style="color:#000000;fill:#126d82;fill-rule:evenodd;stroke-width:3.77953;stroke-linecap:round;stroke-linejoin:round;-inkscape-stroke:none"
d="M 2575.416 3444.2207 L 2575.416 3447.6816 C 2579.0484 3450.8772 2581.6954 3455.0413 2583.0508 3459.6855 L 2516.7305 3459.6855 C 2504.9502 3459.6855 2495.2559 3469.3783 2495.2559 3481.1582 L 2495.2559 3487.6719 L 2475.9883 3506.9375 A 6.6148345 6.6148345 0 0 0 2475.9883 3516.293 L 2495.2559 3535.5586 L 2495.2559 3540.2656 C 2495.2559 3552.0455 2504.9502 3561.7402 2516.7305 3561.7402 L 2583.0508 3561.7402 C 2581.6954 3566.3844 2579.0484 3570.5467 2575.416 3573.7422 L 2575.416 3577.2051 L 2619.7441 3555.041 L 2575.416 3532.875 L 2575.416 3535.4727 C 2579.2729 3538.8725 2582.0114 3543.3595 2583.2734 3548.3438 L 2516.7305 3548.3438 C 2512.1385 3548.3437 2508.6582 3544.8577 2508.6582 3540.2656 L 2508.6582 3535.6543 L 2526.8945 3517.418 L 2538.2168 3517.418 L 2555.9473 3517.418 L 2577.4453 3517.418 L 2577.4453 3504.0078 L 2555.9473 3504.0078 L 2538.2168 3504.0078 L 2525.0801 3504.0078 L 2508.6582 3487.5879 L 2508.6582 3481.1582 C 2508.6582 3476.5661 2512.1385 3473.082 2516.7305 3473.082 L 2583.2734 3473.082 C 2582.0114 3478.0662 2579.2729 3482.5533 2575.416 3485.9531 L 2575.416 3488.5488 L 2619.7441 3466.3828 L 2575.416 3444.2207 z M 2502 3499.6348 L 2513.9844 3511.6211 L 2502.0039 3523.5996 L 2490.0195 3511.6152 L 2502 3499.6348 z "
transform="scale(0.26458333)" />
<path
id="path28509-8"
style="color:#000000;fill:#126d82;fill-rule:evenodd;stroke-width:3.77953;stroke-linecap:round;-inkscape-stroke:none"
d="M 2399.5488 3444.8691 C 2399.2119 3444.8591 2398.8728 3444.8589 2398.5312 3444.8789 C 2393.0057 3445.089 2388.3208 3448.0519 2384.6758 3451.8398 C 2381.0308 3455.6276 2378.0721 3460.419 2375.375 3465.8945 C 2373.2508 3470.2068 2371.3058 3474.9651 2369.5156 3479.9805 L 2345.8223 3493.6602 A 6.6148344 6.6148344 0 0 0 2343.4023 3502.6953 L 2357.3008 3526.7676 C 2353.7347 3544.8143 2351.9297 3558.8242 2351.9297 3558.8242 C 2351.4636 3562.4489 2354.0254 3565.7647 2357.6504 3566.2285 C 2361.2731 3566.6948 2364.5879 3564.1363 2365.0547 3560.5137 C 2365.0547 3560.5137 2366.8057 3547.0108 2370.2109 3529.6914 L 2391.2676 3517.5332 L 2423.0762 3519.1836 C 2421.901 3514.4062 2420.5091 3511.1637 2419.0352 3506.375 L 2394.916 3505.125 L 2382.3672 3483.3887 C 2383.9174 3479.1251 2385.5484 3475.1771 2387.2441 3471.7344 C 2389.5662 3467.0204 2392.0321 3463.2741 2394.2129 3461.0078 C 2396.3937 3458.7414 2397.9351 3458.1336 2399.0332 3458.0918 C 2400.1917 3458.0478 2401.6224 3458.4993 2403.7949 3460.4766 C 2405.9674 3462.454 2408.4992 3465.8585 2410.9258 3470.2285 C 2415.779 3478.9684 2420.2895 3491.437 2423.9551 3504.0078 C 2428.2848 3518.856 2431.4634 3533.6918 2433.3691 3543.4609 C 2428.6211 3543.3167 2423.9981 3541.8521 2420.0293 3539.2148 L 2417.5215 3539.8867 L 2450.4043 3576.9688 L 2460.3398 3528.4141 L 2456.9961 3529.3105 C 2454.8496 3533.6462 2451.5141 3537.2818 2447.3789 3539.793 L 2441.7285 3519.5527 C 2440.2594 3513.4161 2438.5627 3506.8668 2436.6504 3500.3086 C 2432.8362 3487.2285 2428.209 3474.1159 2422.4844 3463.8066 C 2419.6221 3458.652 2416.508 3454.1637 2412.6973 3450.6953 C 2409.1248 3447.4436 2404.6033 3444.9934 2399.5488 3444.8711 L 2399.5488 3444.8691 z M 2375.7188 3477.7051 C 2375.7747 3477.7091 2375.8288 3477.727 2375.8848 3477.7324 A 6.6148344 6.6148344 0 0 0 2375.666 3477.707 C 2375.6835 3477.7081 2375.7013 3477.7038 2375.7188 3477.7051 z M 2372.8359 3493.3379 L 2381.3066 3508.0098 L 2366.6348 3516.4805 L 2358.1641 3501.8086 L 2372.8359 3493.3379 z "
transform="scale(0.26458333)" />
<g
id="g38755">
<path
d="m 590.37968,815.93796 q 2.10753,0 3.40104,1.63919 1.30466,1.63918 1.30466,4.80606 0,2.11867 -0.6133,3.56829 -0.61331,1.43847 -1.69494,2.17443 -1.08165,0.73597 -2.48666,0.73597 -0.90323,0 -1.54998,-0.22302 -0.64675,-0.23417 -1.10395,-0.591 -0.45719,-0.36798 -0.79171,-0.78057 h -0.17842 q 0.0892,0.44604 0.13383,0.91439 0.0446,0.46833 0.0446,0.91437 v 5.02907 h -3.40104 v -17.95302 h 2.76543 l 0.47949,1.6169 h 0.15605 q 0.33452,-0.5018 0.81401,-0.92554 0.47949,-0.42374 1.14855,-0.66905 0.68021,-0.25647 1.57228,-0.25647 z m -1.09279,2.72082 q -0.89207,0 -1.41617,0.36799 -0.52408,0.36798 -0.76942,1.10393 -0.23416,0.73597 -0.25646,1.86222 v 0.36798 q 0,1.2043 0.22301,2.04062 0.23418,0.83631 0.76942,1.27121 0.54639,0.43488 1.49423,0.43488 0.78056,0 1.28236,-0.43488 0.50179,-0.4349 0.74711,-1.27121 0.25646,-0.84748 0.25646,-2.06292 0,-1.82875 -0.56869,-2.75428 -0.56869,-0.92554 -1.76185,-0.92554 z"
id="path105589-9-3"
style="font-weight:bold;font-size:13.4639px;line-height:125%;font-family:'Open Sans';-inkscape-font-specification:'Open Sans Bold';letter-spacing:0px;word-spacing:0px;fill:#126d82;fill-opacity:1;stroke-width:1.42731px" />
<path
d="m 600.11661,816.17212 v 12.46676 h -3.40104 v -12.46676 z m -1.69494,-4.8841 q 0.75826,0 1.30466,0.35683 0.54639,0.34568 0.54639,1.30466 0,0.94782 -0.54639,1.31581 -0.5464,0.35682 -1.30466,0.35682 -0.76942,0 -1.31581,-0.35682 -0.53524,-0.36799 -0.53524,-1.31581 0,-0.95898 0.53524,-1.30466 0.54639,-0.35683 1.31581,-0.35683 z"
id="path105591-5-6"
style="font-weight:bold;font-size:13.4639px;line-height:125%;font-family:'Open Sans';-inkscape-font-specification:'Open Sans Bold';letter-spacing:0px;word-spacing:0px;fill:#126d82;fill-opacity:1;stroke-width:1.42731px" />
<path
style="font-weight:bold;font-size:13.4639px;line-height:125%;font-family:'Open Sans';-inkscape-font-specification:'Open Sans Bold';letter-spacing:0px;word-spacing:0px;fill:#126d82;fill-opacity:1;stroke-width:1.42731px"
d="m 619.0778,818.72569 h -2.94385 v 9.91319 h -3.40103 v -9.91319 h -1.87336 v -1.63918 l 1.87336,-0.91439 v -0.91437 q 0,-1.59458 0.53524,-2.4755 0.54639,-0.89209 1.52768,-1.24891 0.99243,-0.36799 2.34169,-0.36799 0.99245,0 1.80646,0.16725 0.81401,0.15605 1.32696,0.35684 l -0.86978,2.4978 q -0.39027,-0.12263 -0.84747,-0.22301 -0.45719,-0.10041 -1.04819,-0.10041 -0.71365,0 -1.04818,0.43488 -0.32338,0.42374 -0.32338,1.09279 v 0.78056 h 2.94385 z"
id="path106049-0-7" />
<path
style="font-weight:bold;font-size:13.4639px;line-height:125%;font-family:'Open Sans';-inkscape-font-specification:'Open Sans Bold';letter-spacing:0px;word-spacing:0px;fill:#126d82;fill-opacity:1;stroke-width:1.42731px"
d="m 609.97621,818.72569 h -2.94384 v 9.91319 h -3.40104 v -9.91319 h -1.87335 v -1.63918 l 1.87335,-0.91439 v -0.91437 q 0,-1.59458 0.53525,-2.4755 0.5464,-0.89209 1.52767,-1.24891 0.99244,-0.36799 2.34171,-0.36799 0.99243,0 1.80644,0.16725 0.81403,0.15605 1.32697,0.35684 l -0.86978,2.4978 q -0.39028,-0.12263 -0.84746,-0.22301 -0.45719,-0.10041 -1.04819,-0.10041 -0.71366,0 -1.0482,0.43488 -0.32337,0.42374 -0.32337,1.09279 v 0.78056 h 2.94384 z"
id="path105593-48-5" />
<path
d="m 642.93756,812.33621 -4.14815,16.30267 h -3.93629 l -2.20788,-8.56393 q -0.0668,-0.24532 -0.17842,-0.74711 -0.11144,-0.50178 -0.23417,-1.09278 -0.12264,-0.60216 -0.22302,-1.12626 -0.0892,-0.53524 -0.12263,-0.84746 -0.0334,0.31222 -0.13383,0.83632 -0.0892,0.5241 -0.21187,1.11509 -0.11144,0.591 -0.22303,1.10394 -0.11144,0.51294 -0.1784,0.78057 l -2.19675,8.54162 H 625.018 l -4.1593,-16.30267 h 3.40103 l 2.08522,8.89845 q 0.0892,0.40143 0.20073,0.95898 0.12263,0.55753 0.23417,1.17084 0.12264,0.60216 0.21185,1.17085 0.10042,0.55755 0.14503,0.97013 0.0558,-0.42373 0.14502,-0.98128 0.0892,-0.56869 0.18956,-1.14854 0.11144,-0.591 0.22303,-1.0928 0.11144,-0.50179 0.20071,-0.81401 l 2.37516,-9.13262 h 3.26722 l 2.37515,9.13262 q 0.078,0.30107 0.17842,0.81401 0.11144,0.5018 0.22301,1.0928 0.11144,0.591 0.20073,1.15969 0.10041,0.55755 0.14502,0.97013 0.078,-0.55754 0.21187,-1.34926 0.14502,-0.80287 0.30107,-1.59459 0.16724,-0.79171 0.28993,-1.32695 l 2.07408,-8.89845 z"
id="path105595-7-3"
style="font-weight:bold;font-size:13.4639px;line-height:125%;font-family:'Open Sans';-inkscape-font-specification:'Open Sans Bold';letter-spacing:0px;word-spacing:0px;fill:#126d82;fill-opacity:1;stroke-width:1.42731px" />
<path
d="m 654.71528,822.38321 q 0,1.56112 -0.42374,2.76542 -0.41257,1.2043 -1.21544,2.04062 -0.79173,0.82517 -1.91797,1.24891 -1.11508,0.42374 -2.52011,0.42374 -1.3158,0 -2.41974,-0.42374 -1.09279,-0.42374 -1.90682,-1.24891 -0.80287,-0.83632 -1.2489,-2.04062 -0.43489,-1.2043 -0.43489,-2.76542 0,-2.07409 0.73597,-3.51256 0.73595,-1.43846 2.09636,-2.18557 1.36042,-0.74712 3.24494,-0.74712 1.75069,0 3.09996,0.74712 1.3604,0.74711 2.12982,2.18557 0.78056,1.43847 0.78056,3.51256 z m -8.61967,0 q 0,1.2266 0.26762,2.06292 0.26763,0.83631 0.83632,1.26005 0.56871,0.42374 1.48308,0.42374 0.90323,0 1.46077,-0.42374 0.56871,-0.42374 0.82517,-1.26005 0.26762,-0.83632 0.26762,-2.06292 0,-1.23777 -0.26762,-2.05178 -0.25646,-0.82517 -0.82517,-1.23775 -0.5687,-0.41259 -1.48307,-0.41259 -1.34927,0 -1.96256,0.92553 -0.60216,0.92552 -0.60216,2.77659 z"
id="path105597-17-5"
style="font-weight:bold;font-size:13.4639px;line-height:125%;font-family:'Open Sans';-inkscape-font-specification:'Open Sans Bold';letter-spacing:0px;word-spacing:0px;fill:#126d82;fill-opacity:1;stroke-width:1.42731px" />
<path
d="m 663.15875,815.93796 q 0.25648,0 0.59102,0.0334 0.34566,0.0222 0.55753,0.0668 l -0.25646,3.18918 q -0.16725,-0.0558 -0.47949,-0.078 -0.30108,-0.0334 -0.52411,-0.0334 -0.65789,0 -1.28235,0.16724 -0.61331,0.16725 -1.10394,0.54639 -0.49064,0.36799 -0.78056,0.98129 -0.27879,0.60214 -0.27879,1.48307 v 6.34489 h -3.40103 v -12.46675 h 2.57588 l 0.50178,2.09639 h 0.16724 q 0.36797,-0.63561 0.91437,-1.15971 0.55755,-0.53525 1.26006,-0.84746 0.71366,-0.32338 1.53882,-0.32338 z"
id="path105599-2-6"
style="font-weight:bold;font-size:13.4639px;line-height:125%;font-family:'Open Sans';-inkscape-font-specification:'Open Sans Bold';letter-spacing:0px;word-spacing:0px;fill:#126d82;fill-opacity:1;stroke-width:1.42731px" />
<path
d="m 669.19362,811.28802 v 7.76105 q 0,0.7025 -0.0558,1.40503 -0.0558,0.7025 -0.12264,1.40501 h 0.0446 q 0.34568,-0.49064 0.70252,-0.97013 0.36797,-0.47949 0.78057,-0.92554 l 3.49023,-3.79132 h 3.83593 l -4.95101,5.40822 5.25208,7.05854 h -3.92513 l -3.5906,-5.05137 -1.46078,1.17085 v 3.88052 h -3.40103 v -17.35086 z"
id="path105601-7-2"
style="font-weight:bold;font-size:13.4639px;line-height:125%;font-family:'Open Sans';-inkscape-font-specification:'Open Sans Bold';letter-spacing:0px;word-spacing:0px;fill:#126d82;fill-opacity:1;stroke-width:1.42731px" />
<path
d="m 686.97604,818.72569 h -2.94385 v 9.91319 h -3.40103 v -9.91319 h -1.87336 v -1.63918 l 1.87336,-0.91439 v -0.91437 q 0,-1.59458 0.53524,-2.4755 0.54639,-0.89209 1.52768,-1.24891 0.99243,-0.36799 2.34169,-0.36799 0.99245,0 1.80646,0.16725 0.81401,0.15605 1.32696,0.35684 l -0.86978,2.4978 q -0.39027,-0.12263 -0.84747,-0.22301 -0.45719,-0.10041 -1.04819,-0.10041 -0.71365,0 -1.04818,0.43488 -0.32338,0.42374 -0.32338,1.09279 v 0.78056 h 2.94385 z"
id="path105603-22-9"
style="font-weight:bold;font-size:13.4639px;line-height:125%;font-family:'Open Sans';-inkscape-font-specification:'Open Sans Bold';letter-spacing:0px;word-spacing:0px;fill:#126d82;fill-opacity:1;stroke-width:1.42731px" />
<path
d="m 693.05548,828.63888 h -3.40103 v -17.35086 h 3.40103 z"
id="path105605-6-1"
style="font-weight:bold;font-size:13.4639px;line-height:125%;font-family:'Open Sans';-inkscape-font-specification:'Open Sans Bold';letter-spacing:0px;word-spacing:0px;fill:#126d82;fill-opacity:1;stroke-width:1.42731px" />
<path
d="m 706.62835,822.38321 q 0,1.56112 -0.42372,2.76542 -0.41259,1.2043 -1.21546,2.04062 -0.79171,0.82517 -1.91797,1.24891 -1.11508,0.42374 -2.5201,0.42374 -1.31581,0 -2.41975,-0.42374 -1.09279,-0.42374 -1.90682,-1.24891 -0.80287,-0.83632 -1.24889,-2.04062 -0.4349,-1.2043 -0.4349,-2.76542 0,-2.07409 0.73597,-3.51256 0.73595,-1.43846 2.09637,-2.18557 1.36041,-0.74712 3.24493,-0.74712 1.75069,0 3.09996,0.74712 1.3604,0.74711 2.12982,2.18557 0.78056,1.43847 0.78056,3.51256 z m -8.61967,0 q 0,1.2266 0.26763,2.06292 0.26762,0.83631 0.83631,1.26005 0.56871,0.42374 1.48309,0.42374 0.90323,0 1.46076,-0.42374 0.56871,-0.42374 0.82517,-1.26005 0.26762,-0.83632 0.26762,-2.06292 0,-1.23777 -0.26762,-2.05178 -0.25646,-0.82517 -0.82517,-1.23775 -0.5687,-0.41259 -1.48307,-0.41259 -1.34925,0 -1.96256,0.92553 -0.60216,0.92552 -0.60216,2.77659 z"
id="path105607-1-2"
style="font-weight:bold;font-size:13.4639px;line-height:125%;font-family:'Open Sans';-inkscape-font-specification:'Open Sans Bold';letter-spacing:0px;word-spacing:0px;fill:#126d82;fill-opacity:1;stroke-width:1.42731px" />
<path
d="m 719.02496,828.63888 -0.95898,-4.36001 q -0.078,-0.39029 -0.25648,-1.14855 -0.17842,-0.76942 -0.39027,-1.6392 -0.20073,-0.88092 -0.37915,-1.62804 -0.16724,-0.7471 -0.24532,-1.09278 h -0.10041 q -0.078,0.34568 -0.24532,1.09278 -0.16724,0.74712 -0.37913,1.62804 -0.20071,0.88093 -0.37913,1.66149 -0.17842,0.76942 -0.26762,1.17085 l -1.00359,4.31542 h -3.6575 l -3.546,-12.46676 h 3.38989 l 1.43847,5.51973 q 0.14503,0.57985 0.27877,1.38272 0.13383,0.7917 0.23417,1.53882 0.11144,0.73597 0.16725,1.17085 h 0.0892 q 0.0222,-0.32338 0.0892,-0.85862 0.078,-0.53525 0.16725,-1.10394 0.10041,-0.57985 0.17842,-1.03704 0.0892,-0.46835 0.13383,-0.63561 l 1.53882,-5.97691 h 3.74671 l 1.46078,5.97691 q 0.078,0.32338 0.20071,1.02588 0.13382,0.70252 0.23417,1.44964 0.10041,0.73595 0.11144,1.15969 h 0.0892 q 0.0446,-0.37913 0.15604,-1.12624 0.11144,-0.74712 0.25648,-1.56115 0.15605,-0.82515 0.31223,-1.405 l 1.49423,-5.51973 h 3.33412 l -3.5906,12.46676 z"
id="path105609-0-7"
style="font-weight:bold;font-size:13.4639px;line-height:125%;font-family:'Open Sans';-inkscape-font-specification:'Open Sans Bold';letter-spacing:0px;word-spacing:0px;fill:#126d82;fill-opacity:1;stroke-width:1.42731px" />
<path
id="path73257-7-2-9-159-9"
style="vector-effect:none;fill:#126d82;fill-opacity:1;fill-rule:evenodd;stroke:none;stroke-width:4.433;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-dasharray:none;stroke-dashoffset:0;stroke-opacity:1"
d="m 574.18839,797.73763 v 0.91595 c 0.96107,0.84548 1.66181,1.94689 2.02041,3.17567 h -17.54738 c -3.11685,0 -5.68164,2.56482 -5.68164,5.68159 v 3.91192 c 0,3.11682 2.56479,5.68164 5.68164,5.68164 h 5.6851 4.69113 5.6878 c 1.21493,0 2.13344,0.91846 2.13344,2.13344 v 3.91632 c 0,1.21498 -0.91851,2.13346 -2.13344,2.13346 h -14.79076 c -0.75878,-2.29982 -2.93713,-3.97943 -5.47735,-3.97943 -3.16218,0 -5.76138,2.60267 -5.76138,5.76489 0,3.1622 2.5992,5.76399 5.76138,5.76399 2.54974,0 4.73517,-1.69176 5.48615,-4.00482 h 14.78196 c 3.1168,0 5.67808,-2.5613 5.67808,-5.67809 v -3.91632 c 0,-3.11677 -2.56128,-5.68164 -5.67808,-5.68164 h -5.6878 -4.69113 -5.6851 c -1.21497,0 -2.13609,-0.91837 -2.13609,-2.13344 v -3.91192 c 0,-1.21499 0.92112,-2.13696 2.13609,-2.13696 h 17.60609 c -0.33391,1.31874 -1.05865,2.50576 -2.07912,3.4053 v 0.68721 l 11.72877,-5.86483 z m -19.73105,27.11871 c 1.24555,0 2.21936,0.97116 2.21936,2.21674 0,1.24556 -0.97381,2.21936 -2.21936,2.21936 -1.24559,0 -2.21675,-0.9738 -2.21675,-2.21936 0,-1.24558 0.97116,-2.21674 2.21675,-2.21674 z"
sodipodi:nodetypes="cccssssccsssscssscssssccsssscccccsssss" />
<text
xml:space="preserve"
style="font-style:normal;font-variant:normal;font-weight:600;font-stretch:normal;font-size:7.18046px;line-height:125%;font-family:'Open Sans';-inkscape-font-specification:'Open Sans Semi-Bold';letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;stroke-width:0.448779px;stroke-linecap:butt;stroke-linejoin:miter;stroke-opacity:1"
x="587.66064"
y="805.95227"
id="text109870-49-3"><tspan
sodipodi:role="line"
id="tspan109868-0-6"
style="font-style:normal;font-variant:normal;font-weight:600;font-stretch:normal;font-family:'Open Sans';-inkscape-font-specification:'Open Sans Semi-Bold';fill:#000000;fill-opacity:1;stroke:none;stroke-width:0.448779px"
x="587.66064"
y="805.95227">Draw the code</tspan></text>
</g>
<path
d="m 129.58873,798.16413 q 2.10753,0 3.40104,1.63919 1.30466,1.63918 1.30466,4.80606 0,2.11867 -0.6133,3.56829 -0.61331,1.43847 -1.69494,2.17443 -1.08165,0.73597 -2.48666,0.73597 -0.90323,0 -1.54998,-0.22302 -0.64675,-0.23417 -1.10395,-0.591 -0.45719,-0.36798 -0.79171,-0.78057 h -0.17842 q 0.0892,0.44604 0.13383,0.91439 0.0446,0.46833 0.0446,0.91437 v 5.02907 h -3.40104 v -17.95302 h 2.76543 l 0.47949,1.6169 h 0.15605 q 0.33452,-0.5018 0.81401,-0.92554 0.47949,-0.42374 1.14855,-0.66905 0.68021,-0.25647 1.57228,-0.25647 z m -1.09279,2.72082 q -0.89207,0 -1.41617,0.36799 -0.52408,0.36798 -0.76942,1.10393 -0.23416,0.73597 -0.25646,1.86222 v 0.36798 q 0,1.2043 0.22301,2.04062 0.23418,0.83631 0.76942,1.27121 0.54639,0.43488 1.49423,0.43488 0.78056,0 1.28236,-0.43488 0.50179,-0.4349 0.74711,-1.27121 0.25646,-0.84748 0.25646,-2.06292 0,-1.82875 -0.56869,-2.75428 -0.56869,-0.92554 -1.76185,-0.92554 z"
id="path105589-78"
@ -1029,5 +1126,29 @@
x="141.17513"
y="396.69467"
style="fill:#19788f;fill-opacity:1" />
<circle
style="color:#000000;overflow:visible;fill:none;fill-rule:evenodd;stroke:#126d82;stroke-width:3.5;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-dasharray:none;stroke-opacity:1"
id="path1309-1"
cx="636.84607"
cy="925.89471"
r="0" />
<circle
style="color:#000000;overflow:visible;fill:none;fill-rule:evenodd;stroke:#126d82;stroke-width:3.5;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-dasharray:none;stroke-opacity:1"
id="path1309-28"
cx="622.87994"
cy="928.04297"
r="0" />
<circle
style="color:#000000;overflow:visible;fill:none;fill-rule:evenodd;stroke:#126d82;stroke-width:3.5;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-dasharray:none;stroke-opacity:1"
id="path1309-2"
cx="622.87994"
cy="928.04297"
r="0" />
<circle
style="color:#000000;overflow:visible;fill:none;fill-rule:evenodd;stroke:#126d82;stroke-width:3.5;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-dasharray:none;stroke-opacity:1"
id="path1309-9"
cx="622.87994"
cy="928.04297"
r="0" />
</g>
</svg>

Before

Width:  |  Height:  |  Size: 7.0 MiB

After

Width:  |  Height:  |  Size: 7.0 MiB

View File

@ -3,9 +3,9 @@
import unittest
from SpiffWorkflow.bpmn.PythonScriptEngine import PythonScriptEngine
from SpiffWorkflow.bpmn.exceptions import WorkflowTaskExecException
from SpiffWorkflow.bpmn.workflow import BpmnWorkflow
from SpiffWorkflow.exceptions import WorkflowTaskException
from tests.SpiffWorkflow.bpmn.BpmnWorkflowTestCase import BpmnWorkflowTestCase
__author__ = 'kellym'
@ -60,7 +60,7 @@ class CallActivityTest(BpmnWorkflowTestCase):
def test_call_acitivity_errors_include_task_trace(self):
error_spec = self.subprocesses.get('ErroringBPMN')
error_spec, subprocesses = self.load_workflow_spec('call_activity_*.bpmn', 'ErroringBPMN')
with self.assertRaises(WorkflowTaskExecException) as context:
with self.assertRaises(WorkflowTaskException) as context:
self.workflow = BpmnWorkflow(error_spec, subprocesses)
self.workflow.do_engine_steps()
self.assertEquals(2, len(context.exception.task_trace))

View File

@ -1,10 +1,10 @@
# -*- coding: utf-8 -*-
import unittest
from SpiffWorkflow.exceptions import WorkflowTaskException
from SpiffWorkflow.task import TaskState
from SpiffWorkflow.bpmn.PythonScriptEngine import PythonScriptEngine
from SpiffWorkflow.bpmn.workflow import BpmnWorkflow
from SpiffWorkflow.bpmn.exceptions import WorkflowTaskExecException
from tests.SpiffWorkflow.bpmn.BpmnWorkflowTestCase import BpmnWorkflowTestCase
__author__ = 'McDonald, danfunk'
@ -46,7 +46,7 @@ class CustomInlineScriptTest(BpmnWorkflowTestCase):
def test_overwrite_function_with_local_variable(self):
ready_task = self.workflow.get_tasks(TaskState.READY)[0]
ready_task.data = {'custom_function': "bill"}
with self.assertRaises(WorkflowTaskExecException) as e:
with self.assertRaises(WorkflowTaskException) as e:
self.workflow.do_engine_steps()
self.assertTrue('' in str(e.exception))
self.assertTrue('custom_function' in str(e.exception))

View File

@ -31,18 +31,11 @@ class InvalidWorkflowsTest(BpmnWorkflowTestCase):
except ValidationException as ex:
self.assertTrue('No start event found' in ('%r' % ex),
'\'No start event found\' should be a substring of error message: \'%r\'' % ex)
self.assertTrue('No-Start-Event.bpmn20.xml' in ('%r' % ex),
self.assertTrue('No-Start-Event.bpmn20.xml' in ex.filename,
'\'No-Start-Event.bpmn20.xml\' should be a substring of error message: \'%r\'' % ex)
self.assertTrue('process' in ('%r' % ex),
'\'process\' should be a substring of error message: \'%r\'' % ex)
self.assertTrue(
'sid-669ddebf-4196-41ee-8b04-bcc90bc5f983' in ('%r' % ex),
'\'sid-669ddebf-4196-41ee-8b04-bcc90bc5f983\' should be a substring of error message: \'%r\'' % ex)
self.assertTrue('No Start Event' in ('%r' % ex),
'\'No Start Event\' should be a substring of error message: \'%r\'' % ex)
def testSubprocessNotFound(self):
with self.assertRaises(ValidationException) as exc:
self.load_workflow_spec('Invalid-Workflows/Subprocess-Not-Found.bpmn20.xml', 'Subprocess Not Found')
self.assertIn("The process 'Missing subprocess' was not found.", str(exc))
@ -60,15 +53,12 @@ class InvalidWorkflowsTest(BpmnWorkflowTestCase):
'There is no support implemented for this task type' in (
'%r' % ex),
'\'There is no support implemented for this task type\' should be a substring of error message: \'%r\'' % ex)
self.assertTrue('Unsupported-Task.bpmn20.xml' in ('%r' % ex),
self.assertTrue('Unsupported-Task.bpmn20.xml' in ex.filename,
'\'Unsupported-Task.bpmn20.xml\' should be a substring of error message: \'%r\'' % ex)
self.assertTrue('businessRuleTask' in ('%r' % ex),
'\'businessRuleTask\' should be a substring of error message: \'%r\'' % ex)
self.assertTrue(
'sid-75EEAB28-3B69-4282-B91A-0F3C97931834' in ('%r' % ex),
'\'sid-75EEAB28-3B69-4282-B91A-0F3C97931834\' should be a substring of error message: \'%r\'' % ex)
self.assertTrue('Business Rule Task' in ('%r' % ex),
'\'Business Rule Task\' should be a substring of error message: \'%r\'' % ex)
self.assertTrue('businessRuleTask' in ex.tag,
'\'businessRuleTask\' should be a substring of the tag: \'%r\'' % ex)
self.assertTrue('Business Rule Task' in ex.name,
'\'Business Rule Task\' should be the name: \'%s\'' % ex.name)
def suite():

View File

@ -1,7 +1,8 @@
import unittest
import os
from SpiffWorkflow.bpmn.parser.BpmnParser import BpmnParser
from SpiffWorkflow.bpmn.parser.BpmnParser import BpmnParser, BpmnValidator
from SpiffWorkflow.bpmn.parser.ValidationException import ValidationException
class ParserTest(unittest.TestCase):
@ -27,3 +28,17 @@ class ParserTest(unittest.TestCase):
self.assertEqual(generate.data_output_associations[0].name, 'obj_1')
self.assertEqual(len(read.data_input_associations), 1)
self.assertEqual(read.data_input_associations[0].name, 'obj_1')
def testValidatorError(self):
parser = BpmnParser(validator=BpmnValidator())
bpmn_file = os.path.join(os.path.dirname(__file__), 'data',
'data_object_invalid.bpmn')
errored = False
try:
parser.add_bpmn_file(bpmn_file)
except ValidationException as ex:
errored = True
self.assertEqual(ex.filename, bpmn_file)
self.assertEqual(14, ex.sourceline)
self.assertIn('DataObjectReference_0cm8dnh', str(ex))
assert(errored, "This should have errored out with a validation exception.")

View File

@ -2,7 +2,7 @@
import unittest
from SpiffWorkflow.bpmn.exceptions import WorkflowTaskExecException
from SpiffWorkflow.exceptions import WorkflowTaskException
from SpiffWorkflow.task import TaskState
from SpiffWorkflow.bpmn.workflow import BpmnWorkflow
from tests.SpiffWorkflow.bpmn.BpmnWorkflowTestCase import BpmnWorkflowTestCase
@ -39,7 +39,7 @@ class InlineScriptTest(BpmnWorkflowTestCase):
# StartTask doesn't know about testvar, it happened earlier.
# calling an exec that references testvar, in the context of the
# start task should fail.
with self.assertRaises(WorkflowTaskExecException):
with self.assertRaises(WorkflowTaskException):
result = self.workflow.script_engine.evaluate(startTask, 'testvar == True')

View File

@ -8,16 +8,15 @@ sys.path.insert(0, os.path.join(dirname, '..', '..', '..'))
from SpiffWorkflow.bpmn.PythonScriptEngine import PythonScriptEngine
from SpiffWorkflow.bpmn.workflow import BpmnWorkflow
from SpiffWorkflow.bpmn.exceptions import WorkflowTaskExecException
from tests.SpiffWorkflow.bpmn.BpmnWorkflowTestCase import BpmnWorkflowTestCase
class ServiceTaskTest(BpmnWorkflowTestCase):
def setUp(self):
spec, subprocesses = self.load_workflow_spec('service_task.bpmn',
spec, subprocesses = self.load_workflow_spec('service_task.bpmn',
'service_task_example1')
self.workflow = BpmnWorkflow(spec, subprocesses)
self.workflow = BpmnWorkflow(spec, subprocesses)
def testRunThroughHappy(self):
self.workflow.do_engine_steps()

View File

@ -0,0 +1,152 @@
<?xml version="1.0" encoding="UTF-8"?>
<bpmn:definitions xmlns:bpmn="http://www.omg.org/spec/BPMN/20100524/MODEL" xmlns:bpmndi="http://www.omg.org/spec/BPMN/20100524/DI" xmlns:dc="http://www.omg.org/spec/DD/20100524/DC" xmlns:di="http://www.omg.org/spec/DD/20100524/DI" xmlns:modeler="http://camunda.org/schema/modeler/1.0" id="Definitions_19o7vxg" targetNamespace="http://bpmn.io/schema/bpmn" exporter="Camunda Modeler" exporterVersion="4.11.1" modeler:executionPlatform="Camunda Platform" modeler:executionPlatformVersion="7.17.0">
<bpmn:process id="Process" isExecutable="true">
<bpmn:dataObject id="obj_1" />
<bpmn:startEvent id="Event_0kmwi7u">
<bpmn:outgoing>Flow_18858hr</bpmn:outgoing>
</bpmn:startEvent>
<bpmn:sequenceFlow id="Flow_18858hr" sourceRef="Event_0kmwi7u" targetRef="generate_data" />
<bpmn:sequenceFlow id="Flow_0gbxq9s" sourceRef="generate_data" targetRef="task_1" />
<bpmn:sequenceFlow id="Flow_1r7v9yo" sourceRef="task_1" targetRef="read_data" />
<!-- All the Data objects here have the same id, which is not valid according the to the xml schema. -->
<bpmn:dataObjectReference id="DataObjectReference_0cm8dnh" name="Data" dataObjectRef="obj_1" />
<bpmn:dataObjectReference id="DataObjectReference_0cm8dnh" name="Data" dataObjectRef="obj_1" />
<bpmn:dataObjectReference id="DataObjectReference_0cm8dnh" name="Data" dataObjectRef="obj_1" />
<bpmn:endEvent id="Event_0qw1yr0">
<bpmn:incoming>Flow_19pyf8s</bpmn:incoming>
</bpmn:endEvent>
<bpmn:sequenceFlow id="Flow_1tnu3ej" sourceRef="read_data" targetRef="subprocess" />
<bpmn:userTask id="read_data" name="Read Data">
<bpmn:incoming>Flow_1r7v9yo</bpmn:incoming>
<bpmn:outgoing>Flow_1tnu3ej</bpmn:outgoing>
<bpmn:property id="Property_1uusomz" name="__targetRef_placeholder" />
<bpmn:dataInputAssociation id="DataInputAssociation_1vaag83">
<bpmn:sourceRef>DataObjectReference_0pztwm3</bpmn:sourceRef>
<bpmn:targetRef>Property_1uusomz</bpmn:targetRef>
</bpmn:dataInputAssociation>
</bpmn:userTask>
<bpmn:userTask id="generate_data" name="Generate Data">
<bpmn:incoming>Flow_18858hr</bpmn:incoming>
<bpmn:outgoing>Flow_0gbxq9s</bpmn:outgoing>
<bpmn:dataOutputAssociation id="DataOutputAssociation_053pozp">
<bpmn:targetRef>DataObjectReference_17fhr1j</bpmn:targetRef>
</bpmn:dataOutputAssociation>
</bpmn:userTask>
<bpmn:userTask id="task_1" name="Task">
<bpmn:incoming>Flow_0gbxq9s</bpmn:incoming>
<bpmn:outgoing>Flow_1r7v9yo</bpmn:outgoing>
</bpmn:userTask>
<bpmn:subProcess id="subprocess" name="Subprocess">
<bpmn:incoming>Flow_1tnu3ej</bpmn:incoming>
<bpmn:outgoing>Flow_19pyf8s</bpmn:outgoing>
<bpmn:property id="Property_1q5wp77" name="__targetRef_placeholder" />
<bpmn:dataInputAssociation id="DataInputAssociation_0w2qahx">
<bpmn:sourceRef>DataObjectReference_0cm8dnh</bpmn:sourceRef>
<bpmn:targetRef>Property_1q5wp77</bpmn:targetRef>
</bpmn:dataInputAssociation>
<bpmn:startEvent id="Event_1wuwx2f">
<bpmn:outgoing>Flow_0yx8lkz</bpmn:outgoing>
</bpmn:startEvent>
<bpmn:task id="placeholder">
<bpmn:incoming>Flow_0yx8lkz</bpmn:incoming>
<bpmn:outgoing>Flow_0rk4i35</bpmn:outgoing>
</bpmn:task>
<bpmn:sequenceFlow id="Flow_0yx8lkz" sourceRef="Event_1wuwx2f" targetRef="placeholder" />
<bpmn:endEvent id="Event_1qcnmnt">
<bpmn:incoming>Flow_0rk4i35</bpmn:incoming>
</bpmn:endEvent>
<bpmn:sequenceFlow id="Flow_0rk4i35" sourceRef="placeholder" targetRef="Event_1qcnmnt" />
</bpmn:subProcess>
<bpmn:sequenceFlow id="Flow_19pyf8s" sourceRef="subprocess" targetRef="Event_0qw1yr0" />
</bpmn:process>
<bpmndi:BPMNDiagram id="BPMNDiagram_1">
<bpmndi:BPMNPlane id="BPMNPlane_1" bpmnElement="Process">
<bpmndi:BPMNEdge id="Flow_19pyf8s_di" bpmnElement="Flow_19pyf8s">
<di:waypoint x="1110" y="180" />
<di:waypoint x="1192" y="180" />
</bpmndi:BPMNEdge>
<bpmndi:BPMNEdge id="Flow_1tnu3ej_di" bpmnElement="Flow_1tnu3ej">
<di:waypoint x="680" y="180" />
<di:waypoint x="740" y="180" />
</bpmndi:BPMNEdge>
<bpmndi:BPMNEdge id="Flow_1r7v9yo_di" bpmnElement="Flow_1r7v9yo">
<di:waypoint x="520" y="180" />
<di:waypoint x="580" y="180" />
</bpmndi:BPMNEdge>
<bpmndi:BPMNEdge id="Flow_0gbxq9s_di" bpmnElement="Flow_0gbxq9s">
<di:waypoint x="360" y="180" />
<di:waypoint x="420" y="180" />
</bpmndi:BPMNEdge>
<bpmndi:BPMNEdge id="Flow_18858hr_di" bpmnElement="Flow_18858hr">
<di:waypoint x="208" y="180" />
<di:waypoint x="260" y="180" />
</bpmndi:BPMNEdge>
<bpmndi:BPMNShape id="Event_0kmwi7u_di" bpmnElement="Event_0kmwi7u">
<dc:Bounds x="172" y="162" width="36" height="36" />
</bpmndi:BPMNShape>
<bpmndi:BPMNShape id="DataObjectReference_17fhr1j_di" bpmnElement="DataObjectReference_17fhr1j">
<dc:Bounds x="292" y="275" width="36" height="50" />
<bpmndi:BPMNLabel>
<dc:Bounds x="299" y="332" width="25" height="14" />
</bpmndi:BPMNLabel>
</bpmndi:BPMNShape>
<bpmndi:BPMNShape id="DataObjectReference_0pztwm3_di" bpmnElement="DataObjectReference_0pztwm3">
<dc:Bounds x="612" y="275" width="36" height="50" />
<bpmndi:BPMNLabel>
<dc:Bounds x="619" y="332" width="25" height="14" />
</bpmndi:BPMNLabel>
</bpmndi:BPMNShape>
<bpmndi:BPMNShape id="DataObjectReference_0cm8dnh_di" bpmnElement="DataObjectReference_0cm8dnh">
<dc:Bounds x="907" y="375" width="36" height="50" />
<bpmndi:BPMNLabel>
<dc:Bounds x="913" y="432" width="25" height="14" />
</bpmndi:BPMNLabel>
</bpmndi:BPMNShape>
<bpmndi:BPMNShape id="Event_0qw1yr0_di" bpmnElement="Event_0qw1yr0">
<dc:Bounds x="1192" y="162" width="36" height="36" />
</bpmndi:BPMNShape>
<bpmndi:BPMNShape id="Activity_08lb08m_di" bpmnElement="read_data">
<dc:Bounds x="580" y="140" width="100" height="80" />
<bpmndi:BPMNLabel />
</bpmndi:BPMNShape>
<bpmndi:BPMNShape id="Activity_00pwxgv_di" bpmnElement="generate_data">
<dc:Bounds x="260" y="140" width="100" height="80" />
</bpmndi:BPMNShape>
<bpmndi:BPMNShape id="Activity_1xmp3i1_di" bpmnElement="task_1">
<dc:Bounds x="420" y="140" width="100" height="80" />
</bpmndi:BPMNShape>
<bpmndi:BPMNShape id="Activity_10c32lr_di" bpmnElement="subprocess" isExpanded="true">
<dc:Bounds x="740" y="80" width="370" height="200" />
</bpmndi:BPMNShape>
<bpmndi:BPMNEdge id="Flow_0rk4i35_di" bpmnElement="Flow_0rk4i35">
<di:waypoint x="970" y="180" />
<di:waypoint x="1032" y="180" />
</bpmndi:BPMNEdge>
<bpmndi:BPMNEdge id="Flow_0yx8lkz_di" bpmnElement="Flow_0yx8lkz">
<di:waypoint x="816" y="180" />
<di:waypoint x="870" y="180" />
</bpmndi:BPMNEdge>
<bpmndi:BPMNShape id="Event_1wuwx2f_di" bpmnElement="Event_1wuwx2f">
<dc:Bounds x="780" y="162" width="36" height="36" />
</bpmndi:BPMNShape>
<bpmndi:BPMNShape id="Activity_01ooqcb_di" bpmnElement="placeholder">
<dc:Bounds x="870" y="140" width="100" height="80" />
</bpmndi:BPMNShape>
<bpmndi:BPMNShape id="Event_1qcnmnt_di" bpmnElement="Event_1qcnmnt">
<dc:Bounds x="1032" y="162" width="36" height="36" />
</bpmndi:BPMNShape>
<bpmndi:BPMNEdge id="DataInputAssociation_1vaag83_di" bpmnElement="DataInputAssociation_1vaag83">
<di:waypoint x="630" y="275" />
<di:waypoint x="630" y="220" />
</bpmndi:BPMNEdge>
<bpmndi:BPMNEdge id="DataOutputAssociation_053pozp_di" bpmnElement="DataOutputAssociation_053pozp">
<di:waypoint x="309" y="220" />
<di:waypoint x="308" y="275" />
</bpmndi:BPMNEdge>
<bpmndi:BPMNEdge id="DataInputAssociation_0w2qahx_di" bpmnElement="DataInputAssociation_0w2qahx">
<di:waypoint x="925" y="375" />
<di:waypoint x="925" y="280" />
</bpmndi:BPMNEdge>
</bpmndi:BPMNPlane>
</bpmndi:BPMNDiagram>
</bpmn:definitions>

View File

@ -1,7 +1,7 @@
import os
import unittest
from SpiffWorkflow.bpmn.exceptions import WorkflowTaskExecException
from SpiffWorkflow.exceptions import SpiffWorkflowException
from SpiffWorkflow.task import TaskState
from SpiffWorkflow.bpmn.workflow import BpmnWorkflow
@ -15,17 +15,23 @@ class BusinessRuleTaskParserTest(BaseTestCase):
'invalid/InvalidDecision.bpmn', 'Process_1', 'invalid_decision.dmn')
self.workflow = BpmnWorkflow(self.spec)
def testExceptionPrint(self):
e1 = Exception("test 1")
print (e1)
e = SpiffWorkflowException("test")
print (e)
def testDmnRaisesTaskErrors(self):
self.workflow = BpmnWorkflow(self.spec)
self.workflow.get_tasks(TaskState.READY)[0].set_data(x=3)
try:
self.workflow.do_engine_steps()
self.assertTrue(False, "An error should have been raised.")
except WorkflowTaskExecException as we:
except SpiffWorkflowException as we:
self.assertTrue(True, "An error was raised..")
self.assertEquals("InvalidDecisionTaskId", we.sender.name)
self.assertEqual("InvalidDecisionTaskId", we.sender.name)
self.maxDiff = 1000
self.assertEquals("Error evaluating expression spam= 1", str(we))
self.assertEquals("Error evaluating expression 'spam= 1'. Rule failed on row 1. Business Rule Task 'Invalid Decision'.", str(we))
def suite():
return unittest.TestLoader().loadTestsFromTestCase(BusinessRuleTaskParserTest)

View File

@ -1,5 +1,5 @@
<?xml version="1.0" encoding="UTF-8"?>
<bpmn:definitions xmlns:bpmn="http://www.omg.org/spec/BPMN/20100524/MODEL" xmlns:bpmndi="http://www.omg.org/spec/BPMN/20100524/DI" xmlns:di="http://www.omg.org/spec/DD/20100524/DI" xmlns:dc="http://www.omg.org/spec/DD/20100524/DC" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xmlns:camunda="http://camunda.org/schema/1.0/bpmn" id="Definitions_1" targetNamespace="http://bpmn.io/schema/bpmn" exporter="Camunda Modeler" exporterVersion="3.7.0">
<bpmn:definitions xmlns:bpmn="http://www.omg.org/spec/BPMN/20100524/MODEL" xmlns:bpmndi="http://www.omg.org/spec/BPMN/20100524/DI" xmlns:di="http://www.omg.org/spec/DD/20100524/DI" xmlns:dc="http://www.omg.org/spec/DD/20100524/DC" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xmlns:camunda="http://camunda.org/schema/1.0/bpmn" id="Definitions_1" targetNamespace="http://bpmn.io/schema/bpmn" exporter="Camunda Modeler" exporterVersion="5.0.0">
<bpmn:process id="Process_1" isExecutable="true">
<bpmn:startEvent id="StartEvent_1">
<bpmn:outgoing>Flow_1b29lxw</bpmn:outgoing>
@ -46,59 +46,59 @@ of documentation</bpmn:documentation>
</bpmn:process>
<bpmndi:BPMNDiagram id="BPMNDiagram_1">
<bpmndi:BPMNPlane id="BPMNPlane_1" bpmnElement="Process_1">
<bpmndi:BPMNShape id="_BPMNShape_StartEvent_2" bpmnElement="StartEvent_1">
<dc:Bounds x="122" y="106" width="36" height="36" />
<bpmndi:BPMNLabel>
<dc:Bounds x="164" y="322" width="90" height="20" />
</bpmndi:BPMNLabel>
</bpmndi:BPMNShape>
<bpmndi:BPMNShape id="EndEvent_0n32cxd_di" bpmnElement="EndEvent_0n32cxd">
<dc:Bounds x="1132" y="106" width="36" height="36" />
<bpmndi:BPMNLabel>
<dc:Bounds x="933" y="505" width="0" height="12" />
</bpmndi:BPMNLabel>
</bpmndi:BPMNShape>
<bpmndi:BPMNEdge id="Flow_066d5e1_di" bpmnElement="Flow_066d5e1">
<di:waypoint x="850" y="124" />
<di:waypoint x="930" y="124" />
</bpmndi:BPMNEdge>
<bpmndi:BPMNEdge id="Flow_0z7tfh1_di" bpmnElement="Flow_0z7tfh1">
<di:waypoint x="510" y="124" />
<di:waypoint x="575" y="124" />
</bpmndi:BPMNEdge>
<bpmndi:BPMNEdge id="Flow_09ciw49_di" bpmnElement="Flow_09ciw49">
<di:waypoint x="340" y="124" />
<di:waypoint x="410" y="124" />
</bpmndi:BPMNEdge>
<bpmndi:BPMNEdge id="Flow_0fusz9y_di" bpmnElement="Flow_0fusz9y">
<di:waypoint x="1030" y="124" />
<di:waypoint x="1092" y="124" />
</bpmndi:BPMNEdge>
<bpmndi:BPMNEdge id="Flow_1b29lxw_di" bpmnElement="Flow_1b29lxw">
<di:waypoint x="188" y="124" />
<di:waypoint x="240" y="124" />
</bpmndi:BPMNEdge>
<bpmndi:BPMNEdge id="SequenceFlow_06fnqj2_di" bpmnElement="SequenceFlow_06fnqj2">
<di:waypoint x="650" y="124" />
<di:waypoint x="700" y="124" />
<di:waypoint x="675" y="124" />
<di:waypoint x="750" y="124" />
<bpmndi:BPMNLabel>
<dc:Bounds x="850" y="462" width="0" height="12" />
</bpmndi:BPMNLabel>
</bpmndi:BPMNEdge>
<bpmndi:BPMNShape id="BusinessRuleTask_1ipm12w_di" bpmnElement="Task_067fajl">
<dc:Bounds x="550" y="84" width="100" height="80" />
</bpmndi:BPMNShape>
<bpmndi:BPMNEdge id="Flow_1b29lxw_di" bpmnElement="Flow_1b29lxw">
<di:waypoint x="158" y="124" />
<di:waypoint x="210" y="124" />
</bpmndi:BPMNEdge>
<bpmndi:BPMNEdge id="Flow_0fusz9y_di" bpmnElement="Flow_0fusz9y">
<di:waypoint x="980" y="124" />
<di:waypoint x="1132" y="124" />
</bpmndi:BPMNEdge>
<bpmndi:BPMNShape id="Activity_1uk7uyi_di" bpmnElement="Activity_0w0chd2">
<dc:Bounds x="880" y="84" width="100" height="80" />
<dc:Bounds x="930" y="84" width="100" height="80" />
</bpmndi:BPMNShape>
<bpmndi:BPMNEdge id="Flow_09ciw49_di" bpmnElement="Flow_09ciw49">
<di:waypoint x="310" y="124" />
<di:waypoint x="360" y="124" />
</bpmndi:BPMNEdge>
<bpmndi:BPMNShape id="Activity_1v5khzq_di" bpmnElement="Activity_0qh0jpg">
<dc:Bounds x="210" y="84" width="100" height="80" />
</bpmndi:BPMNShape>
<bpmndi:BPMNEdge id="Flow_0z7tfh1_di" bpmnElement="Flow_0z7tfh1">
<di:waypoint x="460" y="124" />
<di:waypoint x="550" y="124" />
</bpmndi:BPMNEdge>
<bpmndi:BPMNShape id="Activity_0vne2ba_di" bpmnElement="Activity_1ftn207">
<dc:Bounds x="360" y="84" width="100" height="80" />
<dc:Bounds x="410" y="84" width="100" height="80" />
</bpmndi:BPMNShape>
<bpmndi:BPMNEdge id="Flow_066d5e1_di" bpmnElement="Flow_066d5e1">
<di:waypoint x="800" y="124" />
<di:waypoint x="880" y="124" />
</bpmndi:BPMNEdge>
<bpmndi:BPMNShape id="Activity_0z6lv9u_di" bpmnElement="Activity_1mu3z8p">
<dc:Bounds x="700" y="84" width="100" height="80" />
<dc:Bounds x="750" y="84" width="100" height="80" />
</bpmndi:BPMNShape>
<bpmndi:BPMNShape id="_BPMNShape_StartEvent_2" bpmnElement="StartEvent_1">
<dc:Bounds x="152" y="106" width="36" height="36" />
<bpmndi:BPMNLabel>
<dc:Bounds x="164" y="322" width="90" height="20" />
</bpmndi:BPMNLabel>
</bpmndi:BPMNShape>
<bpmndi:BPMNShape id="Activity_1v5khzq_di" bpmnElement="Activity_0qh0jpg">
<dc:Bounds x="240" y="84" width="100" height="80" />
</bpmndi:BPMNShape>
<bpmndi:BPMNShape id="EndEvent_0n32cxd_di" bpmnElement="EndEvent_0n32cxd">
<dc:Bounds x="1092" y="106" width="36" height="36" />
<bpmndi:BPMNLabel>
<dc:Bounds x="933" y="505" width="0" height="12" />
</bpmndi:BPMNLabel>
</bpmndi:BPMNShape>
<bpmndi:BPMNShape id="BusinessRuleTask_1ipm12w_di" bpmnElement="Task_067fajl">
<dc:Bounds x="575" y="84" width="100" height="80" />
</bpmndi:BPMNShape>
</bpmndi:BPMNPlane>
</bpmndi:BPMNDiagram>

View File

@ -1,8 +1,8 @@
<?xml version="1.0" encoding="UTF-8"?>
<definitions xmlns="http://www.omg.org/spec/DMN/20151101/dmn.xsd" xmlns:camunda="http://camunda.org/schema/1.0/dmn" id="definitions_1jblnbx" name="definitions" namespace="http://camunda.org/schema/1.0/dmn" exporter="Camunda Modeler" exporterVersion="3.7.0">
<definitions xmlns="https://www.omg.org/spec/DMN/20191111/MODEL/" xmlns:camunda="http://camunda.org/schema/1.0/dmn" xmlns:biodi="http://bpmn.io/schema/dmn/biodi/2.0" id="definitions_1jblnbx" name="definitions" namespace="http://camunda.org/schema/1.0/dmn" exporter="Camunda Modeler" exporterVersion="5.0.0">
<decision id="IntegerDecisionStringOutputTable" name="IntegerDecisionStringOutput">
<decisionTable id="decisionTable">
<input id="InputClause_1tm0ceq" label="x" camunda:inputVariable="">
<input id="InputClause_1tm0ceq" label="x" biodi:width="192" camunda:inputVariable="">
<inputExpression id="LiteralExpression_04o7chw" typeRef="integer">
<text>item.x</text>
</inputExpression>

View File

@ -9,7 +9,6 @@ sys.path.insert(0, os.path.join(dirname, '..', '..', '..'))
from SpiffWorkflow.bpmn.PythonScriptEngine import PythonScriptEngine
from SpiffWorkflow.bpmn.workflow import BpmnWorkflow
from SpiffWorkflow.bpmn.exceptions import WorkflowTaskExecException
from .BaseTestCase import BaseTestCase
class ServiceTaskDelegate:

View File

@ -9,7 +9,6 @@ sys.path.insert(0, os.path.join(dirname, '..', '..', '..'))
from SpiffWorkflow.bpmn.PythonScriptEngine import PythonScriptEngine
from SpiffWorkflow.bpmn.workflow import BpmnWorkflow
from SpiffWorkflow.bpmn.exceptions import WorkflowTaskExecException
from .BaseTestCase import BaseTestCase
class ServiceTaskDelegate: