1 # Copyright (C) The Arvados Authors. All rights reserved.
3 # SPDX-License-Identifier: Apache-2.0
5 from __future__ import division
6 from builtins import next
7 from builtins import object
8 from builtins import str
9 from future.utils import viewvalues, viewitems
19 from functools import partial
22 from cwltool.errors import WorkflowException
23 import cwltool.workflow
24 from schema_salad.sourceline import SourceLine
25 import schema_salad.validate as validate
29 from arvados.keep import KeepClient
30 from arvados.errors import ApiError
32 import arvados_cwl.util
33 from .arvcontainer import RunnerContainer
34 from .runner import Runner, upload_docker, upload_job_order, upload_workflow_deps
35 from .arvtool import ArvadosCommandTool, validate_cluster_target, ArvadosExpressionTool
36 from .arvworkflow import ArvadosWorkflow, upload_workflow
37 from .fsaccess import CollectionFsAccess, CollectionFetcher, collectionResolver, CollectionCache, pdh_size
38 from .perf import Perf
39 from .pathmapper import NoFollowPathMapper
40 from cwltool.task_queue import TaskQueue
41 from .context import ArvLoadingContext, ArvRuntimeContext
42 from ._version import __version__
44 from cwltool.process import shortname, UnsupportedRequirement, use_custom_schema
45 from cwltool.utils import adjustFileObjs, adjustDirObjs, get_listing, visit_class
46 from cwltool.command_line_tool import compute_checksums
47 from cwltool.load_tool import load_tool
49 logger = logging.getLogger('arvados.cwl-runner')
50 metrics = logging.getLogger('arvados.cwl-runner.metrics')
52 DEFAULT_PRIORITY = 500
54 class RuntimeStatusLoggingHandler(logging.Handler):
56 Intercepts logging calls and report them as runtime statuses on runner
59 def __init__(self, runtime_status_update_func):
60 super(RuntimeStatusLoggingHandler, self).__init__()
61 self.runtime_status_update = runtime_status_update_func
62 self.updatingRuntimeStatus = False
64 def emit(self, record):
66 if record.levelno >= logging.ERROR:
68 elif record.levelno >= logging.WARNING:
70 if kind is not None and self.updatingRuntimeStatus is not True:
71 self.updatingRuntimeStatus = True
73 log_msg = record.getMessage()
75 # If the logged message is multi-line, use its first line as status
76 # and the rest as detail.
77 status, detail = log_msg.split('\n', 1)
78 self.runtime_status_update(
80 "%s: %s" % (record.name, status),
84 self.runtime_status_update(
86 "%s: %s" % (record.name, record.getMessage())
89 self.updatingRuntimeStatus = False
92 class ArvCwlExecutor(object):
93 """Execute a CWL tool or workflow, submit work (using containers API),
94 wait for them to complete, and report output.
98 def __init__(self, api_client,
105 arvargs = argparse.Namespace()
106 arvargs.work_api = None
107 arvargs.output_name = None
108 arvargs.output_tags = None
109 arvargs.thread_count = 1
110 arvargs.collection_cache_size = None
112 self.api = api_client
114 self.workflow_eval_lock = threading.Condition(threading.RLock())
115 self.final_output = None
116 self.final_status = None
117 self.num_retries = num_retries
119 self.stop_polling = threading.Event()
122 self.final_output_collection = None
123 self.output_name = arvargs.output_name
124 self.output_tags = arvargs.output_tags
125 self.project_uuid = None
126 self.intermediate_output_ttl = 0
127 self.intermediate_output_collections = []
128 self.trash_intermediate = False
129 self.thread_count = arvargs.thread_count
130 self.poll_interval = 12
131 self.loadingContext = None
132 self.should_estimate_cache_size = True
133 self.fs_access = None
134 self.secret_store = None
136 if keep_client is not None:
137 self.keep_client = keep_client
139 self.keep_client = arvados.keep.KeepClient(api_client=self.api, num_retries=self.num_retries)
141 if arvargs.collection_cache_size:
142 collection_cache_size = arvargs.collection_cache_size*1024*1024
143 self.should_estimate_cache_size = False
145 collection_cache_size = 256*1024*1024
147 self.collection_cache = CollectionCache(self.api, self.keep_client, self.num_retries,
148 cap=collection_cache_size)
150 self.fetcher_constructor = partial(CollectionFetcher,
152 fs_access=CollectionFsAccess("", collection_cache=self.collection_cache),
153 num_retries=self.num_retries)
156 expected_api = ["containers"]
157 for api in expected_api:
159 methods = self.api._rootDesc.get('resources')[api]['methods']
160 if ('httpMethod' in methods['create'] and
161 (arvargs.work_api == api or arvargs.work_api is None)):
167 if not self.work_api:
168 if arvargs.work_api is None:
169 raise Exception("No supported APIs")
171 raise Exception("Unsupported API '%s', expected one of %s" % (arvargs.work_api, expected_api))
173 if self.work_api == "jobs":
175 *******************************
176 The 'jobs' API is no longer supported.
177 *******************************""")
180 self.loadingContext = ArvLoadingContext(vars(arvargs))
181 self.loadingContext.fetcher_constructor = self.fetcher_constructor
182 self.loadingContext.resolver = partial(collectionResolver, self.api, num_retries=self.num_retries)
183 self.loadingContext.construct_tool_object = self.arv_make_tool
185 # Add a custom logging handler to the root logger for runtime status reporting
186 # if running inside a container
187 if arvados_cwl.util.get_current_container(self.api, self.num_retries, logger):
188 root_logger = logging.getLogger('')
190 # Remove existing RuntimeStatusLoggingHandlers if they exist
191 handlers = [h for h in root_logger.handlers if not isinstance(h, RuntimeStatusLoggingHandler)]
192 root_logger.handlers = handlers
194 handler = RuntimeStatusLoggingHandler(self.runtime_status_update)
195 root_logger.addHandler(handler)
197 self.runtimeContext = ArvRuntimeContext(vars(arvargs))
198 self.runtimeContext.make_fs_access = partial(CollectionFsAccess,
199 collection_cache=self.collection_cache)
201 validate_cluster_target(self, self.runtimeContext)
204 def arv_make_tool(self, toolpath_object, loadingContext):
205 if "class" in toolpath_object and toolpath_object["class"] == "CommandLineTool":
206 return ArvadosCommandTool(self, toolpath_object, loadingContext)
207 elif "class" in toolpath_object and toolpath_object["class"] == "Workflow":
208 return ArvadosWorkflow(self, toolpath_object, loadingContext)
209 elif "class" in toolpath_object and toolpath_object["class"] == "ExpressionTool":
210 return ArvadosExpressionTool(self, toolpath_object, loadingContext)
212 raise Exception("Unknown tool %s" % toolpath_object.get("class"))
214 def output_callback(self, out, processStatus):
215 with self.workflow_eval_lock:
216 if processStatus == "success":
217 logger.info("Overall process status is %s", processStatus)
220 logger.error("Overall process status is %s", processStatus)
223 self.api.pipeline_instances().update(uuid=self.pipeline["uuid"],
224 body={"state": state}).execute(num_retries=self.num_retries)
225 self.final_status = processStatus
226 self.final_output = out
227 self.workflow_eval_lock.notifyAll()
230 def start_run(self, runnable, runtimeContext):
231 self.task_queue.add(partial(runnable.run, runtimeContext),
232 self.workflow_eval_lock, self.stop_polling)
234 def process_submitted(self, container):
235 with self.workflow_eval_lock:
236 self.processes[container.uuid] = container
238 def process_done(self, uuid, record):
239 with self.workflow_eval_lock:
240 j = self.processes[uuid]
241 logger.info("%s %s is %s", self.label(j), uuid, record["state"])
242 self.task_queue.add(partial(j.done, record),
243 self.workflow_eval_lock, self.stop_polling)
244 del self.processes[uuid]
246 def runtime_status_update(self, kind, message, detail=None):
248 Updates the runtime_status field on the runner container.
249 Called when there's a need to report errors, warnings or just
250 activity statuses, for example in the RuntimeStatusLoggingHandler.
252 with self.workflow_eval_lock:
255 current = arvados_cwl.util.get_current_container(self.api, self.num_retries, logger)
256 except Exception as e:
257 logger.info("Couldn't get current container: %s", e)
260 runtime_status = current.get('runtime_status', {})
261 # In case of status being an error, only report the first one.
263 if not runtime_status.get('error'):
264 runtime_status.update({
267 if detail is not None:
268 runtime_status.update({
269 'errorDetail': detail
271 # Further errors are only mentioned as a count.
273 # Get anything before an optional 'and N more' string.
275 error_msg = re.match(
276 r'^(.*?)(?=\s*\(and \d+ more\)|$)', runtime_status.get('error')).groups()[0]
277 more_failures = re.match(
278 r'.*\(and (\d+) more\)', runtime_status.get('error'))
280 # Ignore tests stubbing errors
283 failure_qty = int(more_failures.groups()[0])
284 runtime_status.update({
285 'error': "%s (and %d more)" % (error_msg, failure_qty+1)
288 runtime_status.update({
289 'error': "%s (and 1 more)" % error_msg
291 elif kind in ['warning', 'activity']:
292 # Record the last warning/activity status without regard of
293 # previous occurences.
294 runtime_status.update({
297 if detail is not None:
298 runtime_status.update({
299 kind+"Detail": detail
302 # Ignore any other status kind
305 self.api.containers().update(uuid=current['uuid'],
307 'runtime_status': runtime_status,
308 }).execute(num_retries=self.num_retries)
309 except Exception as e:
310 logger.info("Couldn't update runtime_status: %s", e)
312 def wrapped_callback(self, cb, obj, st):
313 with self.workflow_eval_lock:
315 self.workflow_eval_lock.notifyAll()
317 def get_wrapped_callback(self, cb):
318 return partial(self.wrapped_callback, cb)
320 def on_message(self, event):
321 if event.get("object_uuid") in self.processes and event["event_type"] == "update":
322 uuid = event["object_uuid"]
323 if event["properties"]["new_attributes"]["state"] == "Running":
324 with self.workflow_eval_lock:
325 j = self.processes[uuid]
326 if j.running is False:
328 j.update_pipeline_component(event["properties"]["new_attributes"])
329 logger.info("%s %s is Running", self.label(j), uuid)
330 elif event["properties"]["new_attributes"]["state"] in ("Complete", "Failed", "Cancelled", "Final"):
331 self.process_done(uuid, event["properties"]["new_attributes"])
333 def label(self, obj):
334 return "[%s %s]" % (self.work_api[0:-1], obj.name)
336 def poll_states(self):
337 """Poll status of containers listed in the processes dict.
339 Runs in a separate thread.
343 remain_wait = self.poll_interval
346 self.stop_polling.wait(remain_wait)
347 if self.stop_polling.is_set():
349 with self.workflow_eval_lock:
350 keys = list(self.processes)
352 remain_wait = self.poll_interval
355 begin_poll = time.time()
356 if self.work_api == "containers":
357 table = self.poll_api.container_requests()
359 pageSize = self.poll_api._rootDesc.get('maxItemsPerResponse', 1000)
362 page = keys[:pageSize]
364 proc_states = table.list(filters=[["uuid", "in", page]]).execute(num_retries=self.num_retries)
366 logger.exception("Error checking states on API server: %s")
367 remain_wait = self.poll_interval
370 for p in proc_states["items"]:
372 "object_uuid": p["uuid"],
373 "event_type": "update",
378 keys = keys[pageSize:]
380 finish_poll = time.time()
381 remain_wait = self.poll_interval - (finish_poll - begin_poll)
383 logger.exception("Fatal error in state polling thread.")
384 with self.workflow_eval_lock:
385 self.processes.clear()
386 self.workflow_eval_lock.notifyAll()
388 self.stop_polling.set()
390 def add_intermediate_output(self, uuid):
392 self.intermediate_output_collections.append(uuid)
394 def trash_intermediate_output(self):
395 logger.info("Cleaning up intermediate output collections")
396 for i in self.intermediate_output_collections:
398 self.api.collections().delete(uuid=i).execute(num_retries=self.num_retries)
400 logger.warning("Failed to delete intermediate output: %s", sys.exc_info()[1], exc_info=(sys.exc_info()[1] if self.debug else False))
401 except (KeyboardInterrupt, SystemExit):
404 def check_features(self, obj, parentfield=""):
405 if isinstance(obj, dict):
406 if obj.get("class") == "DockerRequirement":
407 if obj.get("dockerOutputDirectory"):
408 if not obj.get("dockerOutputDirectory").startswith('/'):
409 raise SourceLine(obj, "dockerOutputDirectory", validate.ValidationException).makeError(
410 "Option 'dockerOutputDirectory' must be an absolute path.")
411 if obj.get("class") == "InplaceUpdateRequirement":
412 if obj["inplaceUpdate"] and parentfield == "requirements":
413 raise SourceLine(obj, "class", UnsupportedRequirement).makeError("InplaceUpdateRequirement not supported for keep collections.")
414 for k,v in viewitems(obj):
415 self.check_features(v, parentfield=k)
416 elif isinstance(obj, list):
417 for i,v in enumerate(obj):
418 with SourceLine(obj, i, UnsupportedRequirement, logger.isEnabledFor(logging.DEBUG)):
419 self.check_features(v, parentfield=parentfield)
421 def make_output_collection(self, name, storage_classes, tagsString, outputObj):
422 outputObj = copy.deepcopy(outputObj)
425 def capture(fileobj):
426 files.append(fileobj)
428 adjustDirObjs(outputObj, capture)
429 adjustFileObjs(outputObj, capture)
431 generatemapper = NoFollowPathMapper(files, "", "", separateDirs=False)
433 final = arvados.collection.Collection(api_client=self.api,
434 keep_client=self.keep_client,
435 num_retries=self.num_retries)
437 for k,v in generatemapper.items():
438 if v.type == "Directory" and v.resolved.startswith("_:"):
440 if v.type == "CreateFile" and (k.startswith("_:") or v.resolved.startswith("_:")):
441 with final.open(v.target, "wb") as f:
442 f.write(v.resolved.encode("utf-8"))
445 if not v.resolved.startswith("keep:"):
446 raise Exception("Output source is not in keep or a literal")
447 sp = v.resolved.split("/")
448 srccollection = sp[0][5:]
450 reader = self.collection_cache.get(srccollection)
451 srcpath = "/".join(sp[1:]) if len(sp) > 1 else "."
452 final.copy(srcpath, v.target, source_collection=reader, overwrite=False)
453 except arvados.errors.ArgumentError as e:
454 logger.error("Creating CollectionReader for '%s' '%s': %s", k, v, e)
457 logger.error("While preparing output collection: %s", e)
460 def rewrite(fileobj):
461 fileobj["location"] = generatemapper.mapper(fileobj["location"]).target
462 for k in ("listing", "contents", "nameext", "nameroot", "dirname"):
466 adjustDirObjs(outputObj, rewrite)
467 adjustFileObjs(outputObj, rewrite)
469 with final.open("cwl.output.json", "w") as f:
470 res = str(json.dumps(outputObj, sort_keys=True, indent=4, separators=(',',': '), ensure_ascii=False))
473 final.save_new(name=name, owner_uuid=self.project_uuid, storage_classes=storage_classes, ensure_unique_name=True)
475 logger.info("Final output collection %s \"%s\" (%s)", final.portable_data_hash(),
476 final.api_response()["name"],
477 final.manifest_locator())
479 final_uuid = final.manifest_locator()
480 tags = tagsString.split(',')
482 self.api.links().create(body={
483 "head_uuid": final_uuid, "link_class": "tag", "name": tag
484 }).execute(num_retries=self.num_retries)
486 def finalcollection(fileobj):
487 fileobj["location"] = "keep:%s/%s" % (final.portable_data_hash(), fileobj["location"])
489 adjustDirObjs(outputObj, finalcollection)
490 adjustFileObjs(outputObj, finalcollection)
492 return (outputObj, final)
494 def set_crunch_output(self):
495 if self.work_api == "containers":
496 current = arvados_cwl.util.get_current_container(self.api, self.num_retries, logger)
500 self.api.containers().update(uuid=current['uuid'],
502 'output': self.final_output_collection.portable_data_hash(),
503 }).execute(num_retries=self.num_retries)
504 self.api.collections().update(uuid=self.final_output_collection.manifest_locator(),
507 }).execute(num_retries=self.num_retries)
509 logger.exception("Setting container output")
512 def apply_reqs(self, job_order_object, tool):
513 if "https://w3id.org/cwl/cwl#requirements" in job_order_object:
514 if tool.metadata.get("http://commonwl.org/cwltool#original_cwlVersion") == 'v1.0':
515 raise WorkflowException(
516 "`cwl:requirements` in the input object is not part of CWL "
517 "v1.0. You can adjust to use `cwltool:overrides` instead; or you "
518 "can set the cwlVersion to v1.1 or greater and re-run with "
520 job_reqs = job_order_object["https://w3id.org/cwl/cwl#requirements"]
522 tool.requirements.append(req)
524 def arv_executor(self, updated_tool, job_order, runtimeContext, logger=None):
525 self.debug = runtimeContext.debug
527 workbench1 = self.api.config()["Services"]["Workbench1"]["ExternalURL"]
528 workbench2 = self.api.config()["Services"]["Workbench2"]["ExternalURL"]
529 controller = self.api.config()["Services"]["Controller"]["ExternalURL"]
530 logger.info("Using cluster %s (%s)", self.api.config()["ClusterID"], workbench2 or workbench1 or controller)
532 updated_tool.visit(self.check_features)
534 self.project_uuid = runtimeContext.project_uuid
536 self.fs_access = runtimeContext.make_fs_access(runtimeContext.basedir)
537 self.secret_store = runtimeContext.secret_store
539 self.trash_intermediate = runtimeContext.trash_intermediate
540 if self.trash_intermediate and self.work_api != "containers":
541 raise Exception("--trash-intermediate is only supported with --api=containers.")
543 self.intermediate_output_ttl = runtimeContext.intermediate_output_ttl
544 if self.intermediate_output_ttl and self.work_api != "containers":
545 raise Exception("--intermediate-output-ttl is only supported with --api=containers.")
546 if self.intermediate_output_ttl < 0:
547 raise Exception("Invalid value %d for --intermediate-output-ttl, cannot be less than zero" % self.intermediate_output_ttl)
549 if runtimeContext.submit_request_uuid and self.work_api != "containers":
550 raise Exception("--submit-request-uuid requires containers API, but using '{}' api".format(self.work_api))
552 if not runtimeContext.name:
553 runtimeContext.name = self.name = updated_tool.tool.get("label") or updated_tool.metadata.get("label") or os.path.basename(updated_tool.tool["id"])
555 # Upload local file references in the job order.
556 job_order = upload_job_order(self, "%s input" % runtimeContext.name,
557 updated_tool, job_order)
559 # the last clause means: if it is a command line tool, and we
560 # are going to wait for the result, and always_submit_runner
561 # is false, then we don't submit a runner process.
563 submitting = (runtimeContext.update_workflow or
564 runtimeContext.create_workflow or
565 (runtimeContext.submit and not
566 (updated_tool.tool["class"] == "CommandLineTool" and
567 runtimeContext.wait and
568 not runtimeContext.always_submit_runner)))
570 loadingContext = self.loadingContext.copy()
571 loadingContext.do_validate = False
572 loadingContext.do_update = False
574 # Document may have been auto-updated. Reload the original
575 # document with updating disabled because we want to
576 # submit the document with its original CWL version, not
577 # the auto-updated one.
578 tool = load_tool(updated_tool.tool["id"], loadingContext)
582 # Upload direct dependencies of workflow steps, get back mapping of files to keep references.
583 # Also uploads docker images.
584 merged_map = upload_workflow_deps(self, tool)
586 # Recreate process object (ArvadosWorkflow or
587 # ArvadosCommandTool) because tool document may have been
588 # updated by upload_workflow_deps in ways that modify
589 # inheritance of hints or requirements.
590 loadingContext.loader = tool.doc_loader
591 loadingContext.avsc_names = tool.doc_schema
592 loadingContext.metadata = tool.metadata
593 tool = load_tool(tool.tool, loadingContext)
595 existing_uuid = runtimeContext.update_workflow
596 if existing_uuid or runtimeContext.create_workflow:
597 # Create a pipeline template or workflow record and exit.
598 if self.work_api == "containers":
599 return (upload_workflow(self, tool, job_order,
602 submit_runner_ram=runtimeContext.submit_runner_ram,
603 name=runtimeContext.name,
604 merged_map=merged_map,
605 submit_runner_image=runtimeContext.submit_runner_image),
608 self.apply_reqs(job_order, tool)
610 self.ignore_docker_for_reuse = runtimeContext.ignore_docker_for_reuse
611 self.eval_timeout = runtimeContext.eval_timeout
613 runtimeContext = runtimeContext.copy()
614 runtimeContext.use_container = True
615 runtimeContext.tmpdir_prefix = "tmp"
616 runtimeContext.work_api = self.work_api
618 if self.work_api == "containers":
619 if self.ignore_docker_for_reuse:
620 raise Exception("--ignore-docker-for-reuse not supported with containers API.")
621 runtimeContext.outdir = "/var/spool/cwl"
622 runtimeContext.docker_outdir = "/var/spool/cwl"
623 runtimeContext.tmpdir = "/tmp"
624 runtimeContext.docker_tmpdir = "/tmp"
626 if runtimeContext.priority < 1 or runtimeContext.priority > 1000:
627 raise Exception("--priority must be in the range 1..1000.")
629 if self.should_estimate_cache_size:
632 def estimate_collection_cache(obj):
633 if obj.get("location", "").startswith("keep:"):
634 m = pdh_size.match(obj["location"][5:])
635 if m and m.group(1) not in visited:
636 visited.add(m.group(1))
637 estimated_size[0] += int(m.group(2))
638 visit_class(job_order, ("File", "Directory"), estimate_collection_cache)
639 runtimeContext.collection_cache_size = max(((estimated_size[0]*192) // (1024*1024))+1, 256)
640 self.collection_cache.set_cap(runtimeContext.collection_cache_size*1024*1024)
642 logger.info("Using collection cache size %s MiB", runtimeContext.collection_cache_size)
645 if runtimeContext.submit:
646 # Submit a runner job to run the workflow for us.
647 if self.work_api == "containers":
649 tool = RunnerContainer(self, updated_tool,
650 tool, loadingContext, runtimeContext.enable_reuse,
653 submit_runner_ram=runtimeContext.submit_runner_ram,
654 name=runtimeContext.name,
655 on_error=runtimeContext.on_error,
656 submit_runner_image=runtimeContext.submit_runner_image,
657 intermediate_output_ttl=runtimeContext.intermediate_output_ttl,
658 merged_map=merged_map,
659 priority=runtimeContext.priority,
660 secret_store=self.secret_store,
661 collection_cache_size=runtimeContext.collection_cache_size,
662 collection_cache_is_default=self.should_estimate_cache_size)
664 runtimeContext.runnerjob = tool.tool["id"]
666 if runtimeContext.cwl_runner_job is not None:
667 self.uuid = runtimeContext.cwl_runner_job.get('uuid')
669 jobiter = tool.job(job_order,
670 self.output_callback,
673 if runtimeContext.submit and not runtimeContext.wait:
674 runnerjob = next(jobiter)
675 runnerjob.run(runtimeContext)
676 return (runnerjob.uuid, "success")
678 current_container = arvados_cwl.util.get_current_container(self.api, self.num_retries, logger)
679 if current_container:
680 logger.info("Running inside container %s", current_container.get("uuid"))
682 self.poll_api = arvados.api('v1', timeout=runtimeContext.http_timeout)
683 self.polling_thread = threading.Thread(target=self.poll_states)
684 self.polling_thread.start()
686 self.task_queue = TaskQueue(self.workflow_eval_lock, self.thread_count)
689 self.workflow_eval_lock.acquire()
691 # Holds the lock while this code runs and releases it when
692 # it is safe to do so in self.workflow_eval_lock.wait(),
693 # at which point on_message can update job state and
694 # process output callbacks.
696 loopperf = Perf(metrics, "jobiter")
698 for runnable in jobiter:
701 if self.stop_polling.is_set():
704 if self.task_queue.error is not None:
705 raise self.task_queue.error
708 with Perf(metrics, "run"):
709 self.start_run(runnable, runtimeContext)
711 if (self.task_queue.in_flight + len(self.processes)) > 0:
712 self.workflow_eval_lock.wait(3)
714 logger.error("Workflow is deadlocked, no runnable processes and not waiting on any pending processes.")
717 if self.stop_polling.is_set():
723 while (self.task_queue.in_flight + len(self.processes)) > 0:
724 if self.task_queue.error is not None:
725 raise self.task_queue.error
726 self.workflow_eval_lock.wait(3)
728 except UnsupportedRequirement:
731 if sys.exc_info()[0] is KeyboardInterrupt or sys.exc_info()[0] is SystemExit:
732 logger.error("Interrupted, workflow will be cancelled")
733 elif isinstance(sys.exc_info()[1], WorkflowException):
734 logger.error("Workflow execution failed:\n%s", sys.exc_info()[1], exc_info=(sys.exc_info()[1] if self.debug else False))
736 logger.exception("Workflow execution failed")
739 self.api.pipeline_instances().update(uuid=self.pipeline["uuid"],
740 body={"state": "Failed"}).execute(num_retries=self.num_retries)
742 if self.work_api == "containers" and not current_container:
743 # Not running in a crunch container, so cancel any outstanding processes.
744 for p in self.processes:
746 self.api.container_requests().update(uuid=p,
747 body={"priority": "0"}
748 ).execute(num_retries=self.num_retries)
752 self.workflow_eval_lock.release()
753 self.task_queue.drain()
754 self.stop_polling.set()
755 self.polling_thread.join()
756 self.task_queue.join()
758 if self.final_status == "UnsupportedRequirement":
759 raise UnsupportedRequirement("Check log for details.")
761 if self.final_output is None:
762 raise WorkflowException("Workflow did not return a result.")
764 if runtimeContext.submit and isinstance(tool, Runner):
765 logger.info("Final output collection %s", tool.final_output)
766 if workbench2 or workbench1:
767 logger.info("Output at %scollections/%s", workbench2 or workbench1, tool.final_output)
769 if self.output_name is None:
770 self.output_name = "Output of %s" % (shortname(tool.tool["id"]))
771 if self.output_tags is None:
772 self.output_tags = ""
774 storage_classes = runtimeContext.storage_classes.strip().split(",")
775 self.final_output, self.final_output_collection = self.make_output_collection(self.output_name, storage_classes, self.output_tags, self.final_output)
776 self.set_crunch_output()
778 if runtimeContext.compute_checksum:
779 adjustDirObjs(self.final_output, partial(get_listing, self.fs_access))
780 adjustFileObjs(self.final_output, partial(compute_checksums, self.fs_access))
782 if self.trash_intermediate and self.final_status == "success":
783 self.trash_intermediate_output()
785 return (self.final_output, self.final_status)