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
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 .arvjob import RunnerJob, RunnerTemplate
35 from .runner import Runner, upload_docker, upload_job_order, upload_workflow_deps
36 from .arvtool import ArvadosCommandTool, validate_cluster_target, ArvadosExpressionTool
37 from .arvworkflow import ArvadosWorkflow, upload_workflow
38 from .fsaccess import CollectionFsAccess, CollectionFetcher, collectionResolver, CollectionCache, pdh_size
39 from .perf import Perf
40 from .pathmapper import NoFollowPathMapper
41 from .task_queue import TaskQueue
42 from .context import ArvLoadingContext, ArvRuntimeContext
43 from ._version import __version__
45 from cwltool.process import shortname, UnsupportedRequirement, use_custom_schema
46 from cwltool.pathmapper import adjustFileObjs, adjustDirObjs, get_listing, visit_class
47 from cwltool.command_line_tool import compute_checksums
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
63 def emit(self, record):
65 if record.levelno >= logging.ERROR:
67 elif record.levelno >= logging.WARNING:
70 log_msg = record.getMessage()
72 # If the logged message is multi-line, use its first line as status
73 # and the rest as detail.
74 status, detail = log_msg.split('\n', 1)
75 self.runtime_status_update(
77 "%s: %s" % (record.name, status),
81 self.runtime_status_update(
83 "%s: %s" % (record.name, record.getMessage())
86 class ArvCwlExecutor(object):
87 """Execute a CWL tool or workflow, submit work (using either jobs or
88 containers API), wait for them to complete, and report output.
92 def __init__(self, api_client,
99 arvargs = argparse.Namespace()
100 arvargs.work_api = None
101 arvargs.output_name = None
102 arvargs.output_tags = None
103 arvargs.thread_count = 1
104 arvargs.collection_cache_size = None
106 self.api = api_client
108 self.workflow_eval_lock = threading.Condition(threading.RLock())
109 self.final_output = None
110 self.final_status = None
111 self.num_retries = num_retries
113 self.stop_polling = threading.Event()
116 self.final_output_collection = None
117 self.output_name = arvargs.output_name
118 self.output_tags = arvargs.output_tags
119 self.project_uuid = None
120 self.intermediate_output_ttl = 0
121 self.intermediate_output_collections = []
122 self.trash_intermediate = False
123 self.thread_count = arvargs.thread_count
124 self.poll_interval = 12
125 self.loadingContext = None
126 self.should_estimate_cache_size = True
128 if keep_client is not None:
129 self.keep_client = keep_client
131 self.keep_client = arvados.keep.KeepClient(api_client=self.api, num_retries=self.num_retries)
133 if arvargs.collection_cache_size:
134 collection_cache_size = arvargs.collection_cache_size*1024*1024
135 self.should_estimate_cache_size = False
137 collection_cache_size = 256*1024*1024
139 self.collection_cache = CollectionCache(self.api, self.keep_client, self.num_retries,
140 cap=collection_cache_size)
142 self.fetcher_constructor = partial(CollectionFetcher,
144 fs_access=CollectionFsAccess("", collection_cache=self.collection_cache),
145 num_retries=self.num_retries)
148 expected_api = ["jobs", "containers"]
149 for api in expected_api:
151 methods = self.api._rootDesc.get('resources')[api]['methods']
152 if ('httpMethod' in methods['create'] and
153 (arvargs.work_api == api or arvargs.work_api is None)):
159 if not self.work_api:
160 if arvargs.work_api is None:
161 raise Exception("No supported APIs")
163 raise Exception("Unsupported API '%s', expected one of %s" % (arvargs.work_api, expected_api))
165 if self.work_api == "jobs":
167 *******************************
168 Using the deprecated 'jobs' API.
170 To get rid of this warning:
172 Users: read about migrating at
173 http://doc.arvados.org/user/cwl/cwl-style.html#migrate
174 and use the option --api=containers
176 Admins: configure the cluster to disable the 'jobs' API as described at:
177 http://doc.arvados.org/install/install-api-server.html#disable_api_methods
178 *******************************""")
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:
253 current = arvados_cwl.util.get_current_container(self.api, self.num_retries, logger)
256 runtime_status = current.get('runtime_status', {})
257 # In case of status being an error, only report the first one.
259 if not runtime_status.get('error'):
260 runtime_status.update({
263 if detail is not None:
264 runtime_status.update({
265 'errorDetail': detail
267 # Further errors are only mentioned as a count.
269 # Get anything before an optional 'and N more' string.
271 error_msg = re.match(
272 r'^(.*?)(?=\s*\(and \d+ more\)|$)', runtime_status.get('error')).groups()[0]
273 more_failures = re.match(
274 r'.*\(and (\d+) more\)', runtime_status.get('error'))
276 # Ignore tests stubbing errors
279 failure_qty = int(more_failures.groups()[0])
280 runtime_status.update({
281 'error': "%s (and %d more)" % (error_msg, failure_qty+1)
284 runtime_status.update({
285 'error': "%s (and 1 more)" % error_msg
287 elif kind in ['warning', 'activity']:
288 # Record the last warning/activity status without regard of
289 # previous occurences.
290 runtime_status.update({
293 if detail is not None:
294 runtime_status.update({
295 kind+"Detail": detail
298 # Ignore any other status kind
301 self.api.containers().update(uuid=current['uuid'],
303 'runtime_status': runtime_status,
304 }).execute(num_retries=self.num_retries)
305 except Exception as e:
306 logger.info("Couldn't update runtime_status: %s", e)
308 def wrapped_callback(self, cb, obj, st):
309 with self.workflow_eval_lock:
311 self.workflow_eval_lock.notifyAll()
313 def get_wrapped_callback(self, cb):
314 return partial(self.wrapped_callback, cb)
316 def on_message(self, event):
317 if event.get("object_uuid") in self.processes and event["event_type"] == "update":
318 uuid = event["object_uuid"]
319 if event["properties"]["new_attributes"]["state"] == "Running":
320 with self.workflow_eval_lock:
321 j = self.processes[uuid]
322 if j.running is False:
324 j.update_pipeline_component(event["properties"]["new_attributes"])
325 logger.info("%s %s is Running", self.label(j), uuid)
326 elif event["properties"]["new_attributes"]["state"] in ("Complete", "Failed", "Cancelled", "Final"):
327 self.process_done(uuid, event["properties"]["new_attributes"])
329 def label(self, obj):
330 return "[%s %s]" % (self.work_api[0:-1], obj.name)
332 def poll_states(self):
333 """Poll status of jobs or containers listed in the processes dict.
335 Runs in a separate thread.
339 remain_wait = self.poll_interval
342 self.stop_polling.wait(remain_wait)
343 if self.stop_polling.is_set():
345 with self.workflow_eval_lock:
346 keys = list(self.processes)
348 remain_wait = self.poll_interval
351 begin_poll = time.time()
352 if self.work_api == "containers":
353 table = self.poll_api.container_requests()
354 elif self.work_api == "jobs":
355 table = self.poll_api.jobs()
357 pageSize = self.poll_api._rootDesc.get('maxItemsPerResponse', 1000)
360 page = keys[:pageSize]
361 keys = keys[pageSize:]
363 proc_states = table.list(filters=[["uuid", "in", page]]).execute(num_retries=self.num_retries)
365 logger.exception("Error checking states on API server: %s")
366 remain_wait = self.poll_interval
369 for p in proc_states["items"]:
371 "object_uuid": p["uuid"],
372 "event_type": "update",
377 finish_poll = time.time()
378 remain_wait = self.poll_interval - (finish_poll - begin_poll)
380 logger.exception("Fatal error in state polling thread.")
381 with self.workflow_eval_lock:
382 self.processes.clear()
383 self.workflow_eval_lock.notifyAll()
385 self.stop_polling.set()
387 def add_intermediate_output(self, uuid):
389 self.intermediate_output_collections.append(uuid)
391 def trash_intermediate_output(self):
392 logger.info("Cleaning up intermediate output collections")
393 for i in self.intermediate_output_collections:
395 self.api.collections().delete(uuid=i).execute(num_retries=self.num_retries)
397 logger.warning("Failed to delete intermediate output: %s", sys.exc_info()[1], exc_info=(sys.exc_info()[1] if self.debug else False))
398 except (KeyboardInterrupt, SystemExit):
401 def check_features(self, obj):
402 if isinstance(obj, dict):
403 if obj.get("writable") and self.work_api != "containers":
404 raise SourceLine(obj, "writable", UnsupportedRequirement).makeError("InitialWorkDir feature 'writable: true' not supported with --api=jobs")
405 if obj.get("class") == "DockerRequirement":
406 if obj.get("dockerOutputDirectory"):
407 if self.work_api != "containers":
408 raise SourceLine(obj, "dockerOutputDirectory", UnsupportedRequirement).makeError(
409 "Option 'dockerOutputDirectory' of DockerRequirement not supported with --api=jobs.")
410 if not obj.get("dockerOutputDirectory").startswith('/'):
411 raise SourceLine(obj, "dockerOutputDirectory", validate.ValidationException).makeError(
412 "Option 'dockerOutputDirectory' must be an absolute path.")
413 if obj.get("class") == "http://commonwl.org/cwltool#Secrets" and self.work_api != "containers":
414 raise SourceLine(obj, "class", UnsupportedRequirement).makeError("Secrets not supported with --api=jobs")
415 for v in viewvalues(obj):
416 self.check_features(v)
417 elif isinstance(obj, list):
418 for i,v in enumerate(obj):
419 with SourceLine(obj, i, UnsupportedRequirement, logger.isEnabledFor(logging.DEBUG)):
420 self.check_features(v)
422 def make_output_collection(self, name, storage_classes, tagsString, outputObj):
423 outputObj = copy.deepcopy(outputObj)
426 def capture(fileobj):
427 files.append(fileobj)
429 adjustDirObjs(outputObj, capture)
430 adjustFileObjs(outputObj, capture)
432 generatemapper = NoFollowPathMapper(files, "", "", separateDirs=False)
434 final = arvados.collection.Collection(api_client=self.api,
435 keep_client=self.keep_client,
436 num_retries=self.num_retries)
438 for k,v in generatemapper.items():
439 if k.startswith("_:"):
440 if v.type == "Directory":
442 if v.type == "CreateFile":
443 with final.open(v.target, "wb") as f:
444 f.write(v.resolved.encode("utf-8"))
447 if not k.startswith("keep:"):
448 raise Exception("Output source is not in keep or a literal")
450 srccollection = sp[0][5:]
452 reader = self.collection_cache.get(srccollection)
453 srcpath = "/".join(sp[1:]) if len(sp) > 1 else "."
454 final.copy(srcpath, v.target, source_collection=reader, overwrite=False)
455 except arvados.errors.ArgumentError as e:
456 logger.error("Creating CollectionReader for '%s' '%s': %s", k, v, e)
459 logger.warning("While preparing output collection: %s", e)
461 def rewrite(fileobj):
462 fileobj["location"] = generatemapper.mapper(fileobj["location"]).target
463 for k in ("listing", "contents", "nameext", "nameroot", "dirname"):
467 adjustDirObjs(outputObj, rewrite)
468 adjustFileObjs(outputObj, rewrite)
470 with final.open("cwl.output.json", "w") as f:
471 res = str(json.dumps(outputObj, sort_keys=True, indent=4, separators=(',',': '), ensure_ascii=False))
474 final.save_new(name=name, owner_uuid=self.project_uuid, storage_classes=storage_classes, ensure_unique_name=True)
476 logger.info("Final output collection %s \"%s\" (%s)", final.portable_data_hash(),
477 final.api_response()["name"],
478 final.manifest_locator())
480 final_uuid = final.manifest_locator()
481 tags = tagsString.split(',')
483 self.api.links().create(body={
484 "head_uuid": final_uuid, "link_class": "tag", "name": tag
485 }).execute(num_retries=self.num_retries)
487 def finalcollection(fileobj):
488 fileobj["location"] = "keep:%s/%s" % (final.portable_data_hash(), fileobj["location"])
490 adjustDirObjs(outputObj, finalcollection)
491 adjustFileObjs(outputObj, finalcollection)
493 return (outputObj, final)
495 def set_crunch_output(self):
496 if self.work_api == "containers":
497 current = arvados_cwl.util.get_current_container(self.api, self.num_retries, logger)
501 self.api.containers().update(uuid=current['uuid'],
503 'output': self.final_output_collection.portable_data_hash(),
504 }).execute(num_retries=self.num_retries)
505 self.api.collections().update(uuid=self.final_output_collection.manifest_locator(),
508 }).execute(num_retries=self.num_retries)
510 logger.exception("Setting container output")
512 elif self.work_api == "jobs" and "TASK_UUID" in os.environ:
513 self.api.job_tasks().update(uuid=os.environ["TASK_UUID"],
515 'output': self.final_output_collection.portable_data_hash(),
516 'success': self.final_status == "success",
518 }).execute(num_retries=self.num_retries)
520 def arv_executor(self, tool, job_order, runtimeContext, logger=None):
521 self.debug = runtimeContext.debug
523 tool.visit(self.check_features)
525 self.project_uuid = runtimeContext.project_uuid
527 self.fs_access = runtimeContext.make_fs_access(runtimeContext.basedir)
528 self.secret_store = runtimeContext.secret_store
530 self.trash_intermediate = runtimeContext.trash_intermediate
531 if self.trash_intermediate and self.work_api != "containers":
532 raise Exception("--trash-intermediate is only supported with --api=containers.")
534 self.intermediate_output_ttl = runtimeContext.intermediate_output_ttl
535 if self.intermediate_output_ttl and self.work_api != "containers":
536 raise Exception("--intermediate-output-ttl is only supported with --api=containers.")
537 if self.intermediate_output_ttl < 0:
538 raise Exception("Invalid value %d for --intermediate-output-ttl, cannot be less than zero" % self.intermediate_output_ttl)
540 if runtimeContext.submit_request_uuid and self.work_api != "containers":
541 raise Exception("--submit-request-uuid requires containers API, but using '{}' api".format(self.work_api))
543 if not runtimeContext.name:
544 runtimeContext.name = self.name = tool.tool.get("label") or tool.metadata.get("label") or os.path.basename(tool.tool["id"])
546 # Upload direct dependencies of workflow steps, get back mapping of files to keep references.
547 # Also uploads docker images.
548 merged_map = upload_workflow_deps(self, tool)
550 # Reload tool object which may have been updated by
551 # upload_workflow_deps
552 # Don't validate this time because it will just print redundant errors.
553 loadingContext = self.loadingContext.copy()
554 loadingContext.loader = tool.doc_loader
555 loadingContext.avsc_names = tool.doc_schema
556 loadingContext.metadata = tool.metadata
557 loadingContext.do_validate = False
559 tool = self.arv_make_tool(tool.doc_loader.idx[tool.tool["id"]],
562 # Upload local file references in the job order.
563 job_order = upload_job_order(self, "%s input" % runtimeContext.name,
566 existing_uuid = runtimeContext.update_workflow
567 if existing_uuid or runtimeContext.create_workflow:
568 # Create a pipeline template or workflow record and exit.
569 if self.work_api == "jobs":
570 tmpl = RunnerTemplate(self, tool, job_order,
571 runtimeContext.enable_reuse,
573 submit_runner_ram=runtimeContext.submit_runner_ram,
574 name=runtimeContext.name,
575 merged_map=merged_map,
576 loadingContext=loadingContext)
578 # cwltool.main will write our return value to stdout.
579 return (tmpl.uuid, "success")
580 elif self.work_api == "containers":
581 return (upload_workflow(self, tool, job_order,
584 submit_runner_ram=runtimeContext.submit_runner_ram,
585 name=runtimeContext.name,
586 merged_map=merged_map),
589 self.ignore_docker_for_reuse = runtimeContext.ignore_docker_for_reuse
590 self.eval_timeout = runtimeContext.eval_timeout
592 runtimeContext = runtimeContext.copy()
593 runtimeContext.use_container = True
594 runtimeContext.tmpdir_prefix = "tmp"
595 runtimeContext.work_api = self.work_api
597 if self.work_api == "containers":
598 if self.ignore_docker_for_reuse:
599 raise Exception("--ignore-docker-for-reuse not supported with containers API.")
600 runtimeContext.outdir = "/var/spool/cwl"
601 runtimeContext.docker_outdir = "/var/spool/cwl"
602 runtimeContext.tmpdir = "/tmp"
603 runtimeContext.docker_tmpdir = "/tmp"
604 elif self.work_api == "jobs":
605 if runtimeContext.priority != DEFAULT_PRIORITY:
606 raise Exception("--priority not implemented for jobs API.")
607 runtimeContext.outdir = "$(task.outdir)"
608 runtimeContext.docker_outdir = "$(task.outdir)"
609 runtimeContext.tmpdir = "$(task.tmpdir)"
611 if runtimeContext.priority < 1 or runtimeContext.priority > 1000:
612 raise Exception("--priority must be in the range 1..1000.")
614 if self.should_estimate_cache_size:
617 def estimate_collection_cache(obj):
618 if obj.get("location", "").startswith("keep:"):
619 m = pdh_size.match(obj["location"][5:])
620 if m and m.group(1) not in visited:
621 visited.add(m.group(1))
622 estimated_size[0] += int(m.group(2))
623 visit_class(job_order, ("File", "Directory"), estimate_collection_cache)
624 runtimeContext.collection_cache_size = max(((estimated_size[0]*192) // (1024*1024))+1, 256)
625 self.collection_cache.set_cap(runtimeContext.collection_cache_size*1024*1024)
627 logger.info("Using collection cache size %s MiB", runtimeContext.collection_cache_size)
630 if runtimeContext.submit:
631 # Submit a runner job to run the workflow for us.
632 if self.work_api == "containers":
633 if tool.tool["class"] == "CommandLineTool" and runtimeContext.wait and (not runtimeContext.always_submit_runner):
634 runtimeContext.runnerjob = tool.tool["id"]
636 tool = RunnerContainer(self, tool, loadingContext, runtimeContext.enable_reuse,
639 submit_runner_ram=runtimeContext.submit_runner_ram,
640 name=runtimeContext.name,
641 on_error=runtimeContext.on_error,
642 submit_runner_image=runtimeContext.submit_runner_image,
643 intermediate_output_ttl=runtimeContext.intermediate_output_ttl,
644 merged_map=merged_map,
645 priority=runtimeContext.priority,
646 secret_store=self.secret_store,
647 collection_cache_size=runtimeContext.collection_cache_size,
648 collection_cache_is_default=self.should_estimate_cache_size)
649 elif self.work_api == "jobs":
650 tool = RunnerJob(self, 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 merged_map=merged_map)
658 elif runtimeContext.cwl_runner_job is None and self.work_api == "jobs":
659 # Create pipeline for local run
660 self.pipeline = self.api.pipeline_instances().create(
662 "owner_uuid": self.project_uuid,
663 "name": runtimeContext.name if runtimeContext.name else shortname(tool.tool["id"]),
665 "state": "RunningOnClient"}).execute(num_retries=self.num_retries)
666 logger.info("Pipeline instance %s", self.pipeline["uuid"])
668 if runtimeContext.cwl_runner_job is not None:
669 self.uuid = runtimeContext.cwl_runner_job.get('uuid')
671 jobiter = tool.job(job_order,
672 self.output_callback,
675 if runtimeContext.submit and not runtimeContext.wait:
676 runnerjob = next(jobiter)
677 runnerjob.run(runtimeContext)
678 return (runnerjob.uuid, "success")
680 current_container = arvados_cwl.util.get_current_container(self.api, self.num_retries, logger)
681 if current_container:
682 logger.info("Running inside container %s", current_container.get("uuid"))
684 self.poll_api = arvados.api('v1', timeout=runtimeContext.http_timeout)
685 self.polling_thread = threading.Thread(target=self.poll_states)
686 self.polling_thread.start()
688 self.task_queue = TaskQueue(self.workflow_eval_lock, self.thread_count)
691 self.workflow_eval_lock.acquire()
693 # Holds the lock while this code runs and releases it when
694 # it is safe to do so in self.workflow_eval_lock.wait(),
695 # at which point on_message can update job state and
696 # process output callbacks.
698 loopperf = Perf(metrics, "jobiter")
700 for runnable in jobiter:
703 if self.stop_polling.is_set():
706 if self.task_queue.error is not None:
707 raise self.task_queue.error
710 with Perf(metrics, "run"):
711 self.start_run(runnable, runtimeContext)
713 if (self.task_queue.in_flight + len(self.processes)) > 0:
714 self.workflow_eval_lock.wait(3)
716 logger.error("Workflow is deadlocked, no runnable processes and not waiting on any pending processes.")
719 if self.stop_polling.is_set():
725 while (self.task_queue.in_flight + len(self.processes)) > 0:
726 if self.task_queue.error is not None:
727 raise self.task_queue.error
728 self.workflow_eval_lock.wait(3)
730 except UnsupportedRequirement:
733 if sys.exc_info()[0] is KeyboardInterrupt or sys.exc_info()[0] is SystemExit:
734 logger.error("Interrupted, workflow will be cancelled")
735 elif isinstance(sys.exc_info()[1], WorkflowException):
736 logger.error("Workflow execution failed:\n%s", sys.exc_info()[1], exc_info=(sys.exc_info()[1] if self.debug else False))
738 logger.exception("Workflow execution failed")
741 self.api.pipeline_instances().update(uuid=self.pipeline["uuid"],
742 body={"state": "Failed"}).execute(num_retries=self.num_retries)
743 if runtimeContext.submit and isinstance(tool, Runner):
745 if runnerjob.uuid and self.work_api == "containers":
746 self.api.container_requests().update(uuid=runnerjob.uuid,
747 body={"priority": "0"}).execute(num_retries=self.num_retries)
749 self.workflow_eval_lock.release()
750 self.task_queue.drain()
751 self.stop_polling.set()
752 self.polling_thread.join()
753 self.task_queue.join()
755 if self.final_status == "UnsupportedRequirement":
756 raise UnsupportedRequirement("Check log for details.")
758 if self.final_output is None:
759 raise WorkflowException("Workflow did not return a result.")
761 if runtimeContext.submit and isinstance(tool, Runner):
762 logger.info("Final output collection %s", tool.final_output)
764 if self.output_name is None:
765 self.output_name = "Output of %s" % (shortname(tool.tool["id"]))
766 if self.output_tags is None:
767 self.output_tags = ""
769 storage_classes = runtimeContext.storage_classes.strip().split(",")
770 self.final_output, self.final_output_collection = self.make_output_collection(self.output_name, storage_classes, self.output_tags, self.final_output)
771 self.set_crunch_output()
773 if runtimeContext.compute_checksum:
774 adjustDirObjs(self.final_output, partial(get_listing, self.fs_access))
775 adjustFileObjs(self.final_output, partial(compute_checksums, self.fs_access))
777 if self.trash_intermediate and self.final_status == "success":
778 self.trash_intermediate_output()
780 return (self.final_output, self.final_status)