Merge branch 'main' into 18842-arv-mount-disk-config
[arvados.git] / sdk / cwl / arvados_cwl / executor.py
index db1db28162583899d13e8248a3fc639dd639198c..447c14b8bfad4c8339addcbce9c6899aa2e06b72 100644 (file)
@@ -17,12 +17,15 @@ import copy
 import json
 import re
 from functools import partial
+import subprocess
 import time
+import urllib
 
 from cwltool.errors import WorkflowException
 import cwltool.workflow
 from schema_salad.sourceline import SourceLine
 import schema_salad.validate as validate
+from schema_salad.ref_resolver import file_uri, uri_file_path
 
 import arvados
 import arvados.config
@@ -30,8 +33,8 @@ from arvados.keep import KeepClient
 from arvados.errors import ApiError
 
 import arvados_cwl.util
-from .arvcontainer import RunnerContainer
-from .runner import Runner, upload_docker, upload_job_order, upload_workflow_deps
+from .arvcontainer import RunnerContainer, cleanup_name_for_collection
+from .runner import Runner, upload_docker, upload_job_order, upload_workflow_deps, make_builder
 from .arvtool import ArvadosCommandTool, validate_cluster_target, ArvadosExpressionTool
 from .arvworkflow import ArvadosWorkflow, upload_workflow
 from .fsaccess import CollectionFsAccess, CollectionFetcher, collectionResolver, CollectionCache, pdh_size
@@ -67,6 +70,10 @@ class RuntimeStatusLoggingHandler(logging.Handler):
             kind = 'error'
         elif record.levelno >= logging.WARNING:
             kind = 'warning'
+        if kind == 'warning' and record.name == "salad":
+            # Don't send validation warnings to runtime status,
+            # they're noisy and unhelpful.
+            return
         if kind is not None and self.updatingRuntimeStatus is not True:
             self.updatingRuntimeStatus = True
             try:
@@ -99,7 +106,8 @@ class ArvCwlExecutor(object):
                  arvargs=None,
                  keep_client=None,
                  num_retries=4,
-                 thread_count=4):
+                 thread_count=4,
+                 stdout=sys.stdout):
 
         if arvargs is None:
             arvargs = argparse.Namespace()
@@ -108,6 +116,9 @@ class ArvCwlExecutor(object):
             arvargs.output_tags = None
             arvargs.thread_count = 1
             arvargs.collection_cache_size = None
+            arvargs.git_info = True
+            arvargs.submit = False
+            arvargs.defer_downloads = False
 
         self.api = api_client
         self.processes = {}
@@ -132,6 +143,9 @@ class ArvCwlExecutor(object):
         self.should_estimate_cache_size = True
         self.fs_access = None
         self.secret_store = None
+        self.stdout = stdout
+        self.fast_submit = False
+        self.git_info = arvargs.git_info
 
         if keep_client is not None:
             self.keep_client = keep_client
@@ -194,11 +208,13 @@ The 'jobs' API is no longer supported.
             handler = RuntimeStatusLoggingHandler(self.runtime_status_update)
             root_logger.addHandler(handler)
 
-        self.runtimeContext = ArvRuntimeContext(vars(arvargs))
-        self.runtimeContext.make_fs_access = partial(CollectionFsAccess,
+        self.toplevel_runtimeContext = ArvRuntimeContext(vars(arvargs))
+        self.toplevel_runtimeContext.make_fs_access = partial(CollectionFsAccess,
                                                      collection_cache=self.collection_cache)
 
-        validate_cluster_target(self, self.runtimeContext)
+        self.defer_downloads = arvargs.submit and arvargs.defer_downloads
+
+        validate_cluster_target(self, self.toplevel_runtimeContext)
 
 
     def arv_make_tool(self, toolpath_object, loadingContext):
@@ -249,6 +265,11 @@ The 'jobs' API is no longer supported.
         Called when there's a need to report errors, warnings or just
         activity statuses, for example in the RuntimeStatusLoggingHandler.
         """
+
+        if kind not in ('error', 'warning'):
+            # Ignore any other status kind
+            return
+
         with self.workflow_eval_lock:
             current = None
             try:
@@ -258,49 +279,35 @@ The 'jobs' API is no longer supported.
             if current is None:
                 return
             runtime_status = current.get('runtime_status', {})
-            # In case of status being an error, only report the first one.
-            if kind == 'error':
-                if not runtime_status.get('error'):
-                    runtime_status.update({
-                        'error': message
-                    })
-                    if detail is not None:
-                        runtime_status.update({
-                            'errorDetail': detail
-                        })
-                # Further errors are only mentioned as a count.
-                else:
-                    # Get anything before an optional 'and N more' string.
-                    try:
-                        error_msg = re.match(
-                            r'^(.*?)(?=\s*\(and \d+ more\)|$)', runtime_status.get('error')).groups()[0]
-                        more_failures = re.match(
-                            r'.*\(and (\d+) more\)', runtime_status.get('error'))
-                    except TypeError:
-                        # Ignore tests stubbing errors
-                        return
-                    if more_failures:
-                        failure_qty = int(more_failures.groups()[0])
-                        runtime_status.update({
-                            'error': "%s (and %d more)" % (error_msg, failure_qty+1)
-                        })
-                    else:
-                        runtime_status.update({
-                            'error': "%s (and 1 more)" % error_msg
-                        })
-            elif kind in ['warning', 'activity']:
-                # Record the last warning/activity status without regard of
-                # previous occurences.
-                runtime_status.update({
-                    kind: message
-                })
-                if detail is not None:
-                    runtime_status.update({
-                        kind+"Detail": detail
-                    })
-            else:
-                # Ignore any other status kind
+
+            original_updatemessage = updatemessage = runtime_status.get(kind, "")
+            if not updatemessage:
+                updatemessage = message
+
+            # Subsequent messages tacked on in detail
+            original_updatedetail = updatedetail = runtime_status.get(kind+'Detail', "")
+            maxlines = 40
+            if updatedetail.count("\n") < maxlines:
+                if updatedetail:
+                    updatedetail += "\n"
+                updatedetail += message + "\n"
+
+                if detail:
+                    updatedetail += detail + "\n"
+
+                if updatedetail.count("\n") >= maxlines:
+                    updatedetail += "\nSome messages may have been omitted.  Check the full log."
+
+            if updatemessage == original_updatemessage and updatedetail == original_updatedetail:
+                # don't waste time doing an update if nothing changed
+                # (usually because we exceeded the max lines)
                 return
+
+            runtime_status.update({
+                kind: updatemessage,
+                kind+'Detail': updatedetail,
+            })
+
             try:
                 self.api.containers().update(uuid=current['uuid'],
                                             body={
@@ -362,8 +369,8 @@ The 'jobs' API is no longer supported.
                     page = keys[:pageSize]
                     try:
                         proc_states = table.list(filters=[["uuid", "in", page]]).execute(num_retries=self.num_retries)
-                    except Exception:
-                        logger.exception("Error checking states on API server: %s")
+                    except Exception as e:
+                        logger.exception("Error checking states on API server: %s", e)
                         remain_wait = self.poll_interval
                         continue
 
@@ -418,7 +425,7 @@ The 'jobs' API is no longer supported.
                 with SourceLine(obj, i, UnsupportedRequirement, logger.isEnabledFor(logging.DEBUG)):
                     self.check_features(v, parentfield=parentfield)
 
-    def make_output_collection(self, name, storage_classes, tagsString, outputObj):
+    def make_output_collection(self, name, storage_classes, tagsString, output_properties, outputObj):
         outputObj = copy.deepcopy(outputObj)
 
         files = []
@@ -448,7 +455,7 @@ The 'jobs' API is no longer supported.
             srccollection = sp[0][5:]
             try:
                 reader = self.collection_cache.get(srccollection)
-                srcpath = "/".join(sp[1:]) if len(sp) > 1 else "."
+                srcpath = urllib.parse.unquote("/".join(sp[1:]) if len(sp) > 1 else ".")
                 final.copy(srcpath, v.target, source_collection=reader, overwrite=False)
             except arvados.errors.ArgumentError as e:
                 logger.error("Creating CollectionReader for '%s' '%s': %s", k, v, e)
@@ -470,7 +477,9 @@ The 'jobs' API is no longer supported.
             res = str(json.dumps(outputObj, sort_keys=True, indent=4, separators=(',',': '), ensure_ascii=False))
             f.write(res)
 
-        final.save_new(name=name, owner_uuid=self.project_uuid, storage_classes=storage_classes, ensure_unique_name=True)
+
+        final.save_new(name=name, owner_uuid=self.project_uuid, storage_classes=storage_classes,
+                       ensure_unique_name=True, properties=output_properties)
 
         logger.info("Final output collection %s \"%s\" (%s)", final.portable_data_hash(),
                     final.api_response()["name"],
@@ -500,6 +509,7 @@ The 'jobs' API is no longer supported.
                 self.api.containers().update(uuid=current['uuid'],
                                              body={
                                                  'output': self.final_output_collection.portable_data_hash(),
+                                                 'output_properties': self.final_output_collection.get_properties(),
                                              }).execute(num_retries=self.num_retries)
                 self.api.collections().update(uuid=self.final_output_collection.manifest_locator(),
                                               body={
@@ -521,17 +531,83 @@ The 'jobs' API is no longer supported.
             for req in job_reqs:
                 tool.requirements.append(req)
 
+    @staticmethod
+    def get_git_info(tool):
+        in_a_git_repo = False
+        cwd = None
+        filepath = None
+
+        if tool.tool["id"].startswith("file://"):
+            # check if git is installed
+            try:
+                filepath = uri_file_path(tool.tool["id"])
+                cwd = os.path.dirname(filepath)
+                subprocess.run(["git", "log", "--format=%H", "-n1", "HEAD"], cwd=cwd, check=True, capture_output=True, text=True)
+                in_a_git_repo = True
+            except Exception as e:
+                pass
+
+        gitproperties = {}
+
+        if in_a_git_repo:
+            git_commit = subprocess.run(["git", "log", "--format=%H", "-n1", "HEAD"], cwd=cwd, capture_output=True, text=True).stdout
+            git_date = subprocess.run(["git", "log", "--format=%cD", "-n1", "HEAD"], cwd=cwd, capture_output=True, text=True).stdout
+            git_committer = subprocess.run(["git", "log", "--format=%cn <%ce>", "-n1", "HEAD"], cwd=cwd, capture_output=True, text=True).stdout
+            git_branch = subprocess.run(["git", "rev-parse", "--abbrev-ref", "HEAD"], cwd=cwd, capture_output=True, text=True).stdout
+            git_origin = subprocess.run(["git", "remote", "get-url", "origin"], cwd=cwd, capture_output=True, text=True).stdout
+            git_status = subprocess.run(["git", "status", "--untracked-files=no", "--porcelain"], cwd=cwd, capture_output=True, text=True).stdout
+            git_describe = subprocess.run(["git", "describe", "--always", "--tags"], cwd=cwd, capture_output=True, text=True).stdout
+            git_toplevel = subprocess.run(["git", "rev-parse", "--show-toplevel"], cwd=cwd, capture_output=True, text=True).stdout
+            git_path = filepath[len(git_toplevel):]
+
+            gitproperties = {
+                "http://arvados.org/cwl#gitCommit": git_commit.strip(),
+                "http://arvados.org/cwl#gitDate": git_date.strip(),
+                "http://arvados.org/cwl#gitCommitter": git_committer.strip(),
+                "http://arvados.org/cwl#gitBranch": git_branch.strip(),
+                "http://arvados.org/cwl#gitOrigin": git_origin.strip(),
+                "http://arvados.org/cwl#gitStatus": git_status.strip(),
+                "http://arvados.org/cwl#gitDescribe": git_describe.strip(),
+                "http://arvados.org/cwl#gitPath": git_path.strip(),
+            }
+        else:
+            for g in ("http://arvados.org/cwl#gitCommit",
+                      "http://arvados.org/cwl#gitDate",
+                      "http://arvados.org/cwl#gitCommitter",
+                      "http://arvados.org/cwl#gitBranch",
+                      "http://arvados.org/cwl#gitOrigin",
+                      "http://arvados.org/cwl#gitStatus",
+                      "http://arvados.org/cwl#gitDescribe",
+                      "http://arvados.org/cwl#gitPath"):
+                if g in tool.metadata:
+                    gitproperties[g] = tool.metadata[g]
+
+        return gitproperties
+
+    def set_container_request_properties(self, container, properties):
+        resp = self.api.container_requests().list(filters=[["container_uuid", "=", container["uuid"]]], select=["uuid", "properties"]).execute(num_retries=self.num_retries)
+        for cr in resp["items"]:
+            cr["properties"].update({k.replace("http://arvados.org/cwl#", "arv:"): v for k, v in properties.items()})
+            self.api.container_requests().update(uuid=cr["uuid"], body={"container_request": {"properties": cr["properties"]}}).execute(num_retries=self.num_retries)
+
     def arv_executor(self, updated_tool, job_order, runtimeContext, logger=None):
         self.debug = runtimeContext.debug
 
+        git_info = self.get_git_info(updated_tool) if self.git_info else {}
+        if git_info:
+            logger.info("Git provenance")
+            for g in git_info:
+                if git_info[g]:
+                    logger.info("  %s: %s", g.split("#", 1)[1], git_info[g])
+
         workbench1 = self.api.config()["Services"]["Workbench1"]["ExternalURL"]
         workbench2 = self.api.config()["Services"]["Workbench2"]["ExternalURL"]
         controller = self.api.config()["Services"]["Controller"]["ExternalURL"]
         logger.info("Using cluster %s (%s)", self.api.config()["ClusterID"], workbench2 or workbench1 or controller)
 
-        updated_tool.visit(self.check_features)
+        if not self.fast_submit:
+            updated_tool.visit(self.check_features)
 
-        self.project_uuid = runtimeContext.project_uuid
         self.pipeline = None
         self.fs_access = runtimeContext.make_fs_access(runtimeContext.basedir)
         self.secret_store = runtimeContext.secret_store
@@ -549,6 +625,8 @@ The 'jobs' API is no longer supported.
         if runtimeContext.submit_request_uuid and self.work_api != "containers":
             raise Exception("--submit-request-uuid requires containers API, but using '{}' api".format(self.work_api))
 
+        runtimeContext = runtimeContext.copy()
+
         default_storage_classes = ",".join([k for k,v in self.api.config().get("StorageClasses", {"default": {"Default": True}}).items() if v.get("Default") is True])
         if runtimeContext.storage_classes == "default":
             runtimeContext.storage_classes = default_storage_classes
@@ -556,11 +634,31 @@ The 'jobs' API is no longer supported.
             runtimeContext.intermediate_storage_classes = default_storage_classes
 
         if not runtimeContext.name:
-            runtimeContext.name = self.name = updated_tool.tool.get("label") or updated_tool.metadata.get("label") or os.path.basename(updated_tool.tool["id"])
+            self.name = updated_tool.tool.get("label") or updated_tool.metadata.get("label") or os.path.basename(updated_tool.tool["id"])
+            if git_info.get("http://arvados.org/cwl#gitDescribe"):
+                self.name = "%s (%s)" % (self.name, git_info.get("http://arvados.org/cwl#gitDescribe"))
+            runtimeContext.name = self.name
+
+        if runtimeContext.copy_deps is None and (runtimeContext.create_workflow or runtimeContext.update_workflow):
+            # When creating or updating workflow record, by default
+            # always copy dependencies and ensure Docker images are up
+            # to date.
+            runtimeContext.copy_deps = True
+            runtimeContext.match_local_docker = True
+
+        if runtimeContext.update_workflow and self.project_uuid is None:
+            # If we are updating a workflow, make sure anything that
+            # gets uploaded goes into the same parent project, unless
+            # an alternate --project-uuid was provided.
+            existing_wf = self.api.workflows().get(uuid=runtimeContext.update_workflow).execute()
+            runtimeContext.project_uuid = existing_wf["owner_uuid"]
+
+        self.project_uuid = runtimeContext.project_uuid
 
         # Upload local file references in the job order.
-        job_order = upload_job_order(self, "%s input" % runtimeContext.name,
-                                     updated_tool, job_order)
+        with Perf(metrics, "upload_job_order"):
+            job_order = upload_job_order(self, "%s input" % runtimeContext.name,
+                                         updated_tool, job_order, runtimeContext)
 
         # the last clause means: if it is a command line tool, and we
         # are going to wait for the result, and always_submit_runner
@@ -575,19 +673,26 @@ The 'jobs' API is no longer supported.
 
         loadingContext = self.loadingContext.copy()
         loadingContext.do_validate = False
-        loadingContext.do_update = False
-        if submitting:
+        loadingContext.disable_js_validation = True
+        if submitting and not self.fast_submit:
+            loadingContext.do_update = False
             # Document may have been auto-updated. Reload the original
             # document with updating disabled because we want to
             # submit the document with its original CWL version, not
             # the auto-updated one.
-            tool = load_tool(updated_tool.tool["id"], loadingContext)
+            with Perf(metrics, "load_tool original"):
+                tool = load_tool(updated_tool.tool["id"], loadingContext)
         else:
             tool = updated_tool
 
         # Upload direct dependencies of workflow steps, get back mapping of files to keep references.
         # Also uploads docker images.
-        merged_map = upload_workflow_deps(self, tool)
+        if not self.fast_submit:
+            logger.info("Uploading workflow dependencies")
+            with Perf(metrics, "upload_workflow_deps"):
+                merged_map = upload_workflow_deps(self, tool, runtimeContext)
+        else:
+            merged_map = {}
 
         # Recreate process object (ArvadosWorkflow or
         # ArvadosCommandTool) because tool document may have been
@@ -596,31 +701,38 @@ The 'jobs' API is no longer supported.
         loadingContext.loader = tool.doc_loader
         loadingContext.avsc_names = tool.doc_schema
         loadingContext.metadata = tool.metadata
-        tool = load_tool(tool.tool, loadingContext)
+        with Perf(metrics, "load_tool"):
+            tool = load_tool(tool.tool, loadingContext)
 
-        existing_uuid = runtimeContext.update_workflow
-        if existing_uuid or runtimeContext.create_workflow:
+        if runtimeContext.update_workflow or runtimeContext.create_workflow:
             # Create a pipeline template or workflow record and exit.
             if self.work_api == "containers":
-                return (upload_workflow(self, tool, job_order,
-                                        self.project_uuid,
-                                        uuid=existing_uuid,
-                                        submit_runner_ram=runtimeContext.submit_runner_ram,
-                                        name=runtimeContext.name,
-                                        merged_map=merged_map,
-                                        submit_runner_image=runtimeContext.submit_runner_image),
-                        "success")
+                uuid = upload_workflow(self, tool, job_order,
+                                       runtimeContext.project_uuid,
+                                       runtimeContext,
+                                       uuid=runtimeContext.update_workflow,
+                                       submit_runner_ram=runtimeContext.submit_runner_ram,
+                                       name=runtimeContext.name,
+                                       merged_map=merged_map,
+                                       submit_runner_image=runtimeContext.submit_runner_image,
+                                       git_info=git_info)
+                self.stdout.write(uuid + "\n")
+                return (None, "success")
 
         self.apply_reqs(job_order, tool)
 
         self.ignore_docker_for_reuse = runtimeContext.ignore_docker_for_reuse
         self.eval_timeout = runtimeContext.eval_timeout
 
-        runtimeContext = runtimeContext.copy()
         runtimeContext.use_container = True
         runtimeContext.tmpdir_prefix = "tmp"
         runtimeContext.work_api = self.work_api
 
+        if not self.output_name:
+             self.output_name = "Output from workflow %s" % runtimeContext.name
+
+        self.output_name  = cleanup_name_for_collection(self.output_name)
+
         if self.work_api == "containers":
             if self.ignore_docker_for_reuse:
                 raise Exception("--ignore-docker-for-reuse not supported with containers API.")
@@ -665,7 +777,8 @@ The 'jobs' API is no longer supported.
                                            priority=runtimeContext.priority,
                                            secret_store=self.secret_store,
                                            collection_cache_size=runtimeContext.collection_cache_size,
-                                           collection_cache_is_default=self.should_estimate_cache_size)
+                                           collection_cache_is_default=self.should_estimate_cache_size,
+                                           git_info=git_info)
                 else:
                     runtimeContext.runnerjob = tool.tool["id"]
 
@@ -679,11 +792,13 @@ The 'jobs' API is no longer supported.
         if runtimeContext.submit and not runtimeContext.wait:
             runnerjob = next(jobiter)
             runnerjob.run(runtimeContext)
-            return (runnerjob.uuid, "success")
+            self.stdout.write(runnerjob.uuid+"\n")
+            return (None, "success")
 
         current_container = arvados_cwl.util.get_current_container(self.api, self.num_retries, logger)
         if current_container:
             logger.info("Running inside container %s", current_container.get("uuid"))
+            self.set_container_request_properties(current_container, git_info)
 
         self.poll_api = arvados.api('v1', timeout=runtimeContext.http_timeout)
         self.polling_thread = threading.Thread(target=self.poll_states)
@@ -772,8 +887,6 @@ The 'jobs' API is no longer supported.
             if workbench2 or workbench1:
                 logger.info("Output at %scollections/%s", workbench2 or workbench1, tool.final_output)
         else:
-            if self.output_name is None:
-                self.output_name = "Output of %s" % (shortname(tool.tool["id"]))
             if self.output_tags is None:
                 self.output_tags = ""
 
@@ -784,7 +897,16 @@ The 'jobs' API is no longer supported.
             else:
                 storage_classes = runtimeContext.storage_classes.strip().split(",")
 
-            self.final_output, self.final_output_collection = self.make_output_collection(self.output_name, storage_classes, self.output_tags, self.final_output)
+            output_properties = {}
+            output_properties_req, _ = tool.get_requirement("http://arvados.org/cwl#OutputCollectionProperties")
+            if output_properties_req:
+                builder = make_builder(job_order, tool.hints, tool.requirements, runtimeContext, tool.metadata)
+                for pr in output_properties_req["outputProperties"]:
+                    output_properties[pr["propertyName"]] = builder.do_eval(pr["propertyValue"])
+
+            self.final_output, self.final_output_collection = self.make_output_collection(self.output_name, storage_classes,
+                                                                                          self.output_tags, output_properties,
+                                                                                          self.final_output)
             self.set_crunch_output()
 
         if runtimeContext.compute_checksum: