20933: Use acrContainerImage where available
[arvados.git] / sdk / cwl / arvados_cwl / executor.py
index 778af58ac3f7a1b71c040d5ec4f3332ecba11964..330dba3dbe9f9ee5401f476b096753f39bd9ccb8 100644 (file)
@@ -17,13 +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
+from schema_salad.sourceline import SourceLine, cmap
 import schema_salad.validate as validate
+from schema_salad.ref_resolver import file_uri, uri_file_path
 
 import arvados
 import arvados.config
@@ -32,9 +34,9 @@ from arvados.errors import ApiError
 
 import arvados_cwl.util
 from .arvcontainer import RunnerContainer, cleanup_name_for_collection
-from .runner import Runner, upload_docker, upload_job_order, upload_workflow_deps, make_builder
+from .runner import Runner, upload_docker, upload_job_order, upload_workflow_deps, make_builder, update_from_merged_map, print_keep_deps
 from .arvtool import ArvadosCommandTool, validate_cluster_target, ArvadosExpressionTool
-from .arvworkflow import ArvadosWorkflow, upload_workflow
+from .arvworkflow import ArvadosWorkflow, upload_workflow, make_workflow_record
 from .fsaccess import CollectionFsAccess, CollectionFetcher, collectionResolver, CollectionCache, pdh_size
 from .perf import Perf
 from .pathmapper import NoFollowPathMapper
@@ -68,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:
@@ -110,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 = {}
@@ -135,6 +144,8 @@ class ArvCwlExecutor(object):
         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
@@ -201,6 +212,8 @@ The 'jobs' API is no longer supported.
         self.toplevel_runtimeContext.make_fs_access = partial(CollectionFsAccess,
                                                      collection_cache=self.collection_cache)
 
+        self.defer_downloads = arvargs.submit and arvargs.defer_downloads
+
         validate_cluster_target(self, self.toplevel_runtimeContext)
 
 
@@ -252,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', 'activity'):
+            # Ignore any other status kind
+            return
+
         with self.workflow_eval_lock:
             current = None
             try:
@@ -261,32 +279,35 @@ The 'jobs' API is no longer supported.
             if current is None:
                 return
             runtime_status = current.get('runtime_status', {})
-            if kind in ('error', 'warning'):
-                updatemessage = runtime_status.get(kind, "")
-                if not updatemessage:
-                    updatemessage = message
-
-                # Subsequent messages tacked on in detail
-                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."
-
-                runtime_status.update({
-                    kind: updatemessage,
-                    kind+'Detail': updatedetail,
-                })
-            else:
-                # Ignore any other status kind
+
+            original_updatemessage = updatemessage = runtime_status.get(kind, "")
+            if kind == "activity" or 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={
@@ -347,9 +368,10 @@ The 'jobs' API is no longer supported.
                 while keys:
                     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")
+                        proc_states = table.list(filters=[["uuid", "in", page]], select=["uuid", "container_uuid", "state", "log_uuid",
+                                                                                         "output_uuid", "modified_at", "properties"]).execute(num_retries=self.num_retries)
+                    except Exception as e:
+                        logger.warning("Temporary error checking states on API server: %s", e)
                         remain_wait = self.poll_interval
                         continue
 
@@ -510,15 +532,84 @@ 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
 
+        self.runtime_status_update("activity", "initialization")
+
+        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.pipeline = None
         self.fs_access = runtimeContext.make_fs_access(runtimeContext.basedir)
@@ -546,7 +637,10 @@ 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
@@ -555,6 +649,10 @@ The 'jobs' API is no longer supported.
             runtimeContext.copy_deps = True
             runtimeContext.match_local_docker = True
 
+        if runtimeContext.print_keep_deps:
+            runtimeContext.copy_deps = False
+            runtimeContext.match_local_docker = False
+
         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
@@ -564,60 +662,95 @@ The 'jobs' API is no longer supported.
 
         self.project_uuid = runtimeContext.project_uuid
 
+        self.runtime_status_update("activity", "data transfer")
+
         # Upload local file references in the job order.
-        job_order = upload_job_order(self, "%s input" % runtimeContext.name,
-                                     updated_tool, job_order, runtimeContext)
+        with Perf(metrics, "upload_job_order"):
+            job_order, jobmapper = upload_job_order(self, "%s input" % runtimeContext.name,
+                                         updated_tool, job_order, runtimeContext)
 
+        # determine if we are submitting or directly executing the workflow.
+        #
         # the last clause means: if it is a command line tool, and we
         # are going to wait for the result, and always_submit_runner
         # is false, then we don't submit a runner process.
 
-        submitting = (runtimeContext.update_workflow or
-                      runtimeContext.create_workflow or
-                      (runtimeContext.submit and not
+        submitting = (runtimeContext.submit and not
                        (updated_tool.tool["class"] == "CommandLineTool" and
                         runtimeContext.wait and
-                        not runtimeContext.always_submit_runner)))
+                        not runtimeContext.always_submit_runner))
 
         loadingContext = self.loadingContext.copy()
         loadingContext.do_validate = False
-        if submitting:
-            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)
-        else:
-            tool = updated_tool
+        loadingContext.disable_js_validation = True
+        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, runtimeContext)
+        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:
+            # in the fast submit case, we are running a workflow that
+            # has already been uploaded to Arvados, so we assume all
+            # the dependencies have been pinned to keep references and
+            # there is nothing to do.
+            merged_map = {}
 
-        # Recreate process object (ArvadosWorkflow or
-        # ArvadosCommandTool) because tool document may have been
-        # updated by upload_workflow_deps in ways that modify
-        # inheritance of hints or requirements.
         loadingContext.loader = tool.doc_loader
         loadingContext.avsc_names = tool.doc_schema
         loadingContext.metadata = tool.metadata
-        tool = load_tool(tool.tool, loadingContext)
-
-        if runtimeContext.update_workflow or runtimeContext.create_workflow:
-            # Create a pipeline template or workflow record and exit.
-            if self.work_api == "containers":
-                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)
+        loadingContext.skip_resolve_all = True
+
+        workflow_wrapper = None
+        if (submitting and not self.fast_submit) or runtimeContext.update_workflow or runtimeContext.create_workflow or runtimeContext.print_keep_deps:
+            # upload workflow and get back the workflow wrapper
+
+            workflow_wrapper = 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,
+                                               set_defaults=(runtimeContext.update_workflow or runtimeContext.create_workflow),
+                                               jobmapper=jobmapper)
+
+            if runtimeContext.update_workflow or runtimeContext.create_workflow:
+                # We're registering the workflow, so create or update
+                # the workflow record and then exit.
+                uuid = make_workflow_record(self, workflow_wrapper, runtimeContext.name, tool,
+                                            runtimeContext.project_uuid, runtimeContext.update_workflow)
                 self.stdout.write(uuid + "\n")
                 return (None, "success")
 
+            if runtimeContext.print_keep_deps:
+                # Just find and print out all the collection dependencies and exit
+                print_keep_deps(tool)
+                return (None, "success")
+
+            # Did not register a workflow, we're going to submit
+            # it instead.
+            loadingContext.loader.idx.clear()
+            loadingContext.loader.idx["_:main"] = workflow_wrapper
+            workflow_wrapper["id"] = "_:main"
+
+            # Reload the minimal wrapper workflow.
+            self.fast_submit = True
+            tool = load_tool(workflow_wrapper, loadingContext)
+            loadingContext.loader.idx["_:main"] = workflow_wrapper
+
+        if not submitting:
+            # If we are going to run the workflow now (rather than
+            # submit it), we need to update the workflow document
+            # replacing file references with keep references.  If we
+            # are just going to construct a run submission, we don't
+            # need to do this.
+            update_from_merged_map(tool, merged_map)
+
         self.apply_reqs(job_order, tool)
 
         self.ignore_docker_for_reuse = runtimeContext.ignore_docker_for_reuse
@@ -660,11 +793,14 @@ The 'jobs' API is no longer supported.
 
         runnerjob = None
         if runtimeContext.submit:
-            # Submit a runner job to run the workflow for us.
+            # We are submitting instead of running immediately.
+            #
+            # Create a "Runner job" that when run() is invoked,
+            # creates the container request to run the workflow.
             if self.work_api == "containers":
                 if submitting:
-                    tool = RunnerContainer(self, updated_tool,
-                                           tool, loadingContext, runtimeContext.enable_reuse,
+                    loadingContext.metadata = updated_tool.metadata.copy()
+                    tool = RunnerContainer(self, tool, loadingContext, runtimeContext.enable_reuse,
                                            self.output_name,
                                            self.output_tags,
                                            submit_runner_ram=runtimeContext.submit_runner_ram,
@@ -676,7 +812,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"]
 
@@ -688,14 +825,22 @@ The 'jobs' API is no longer supported.
                            runtimeContext)
 
         if runtimeContext.submit and not runtimeContext.wait:
+            # User provided --no-wait so submit the container request,
+            # get the container request uuid, print it out, and exit.
             runnerjob = next(jobiter)
             runnerjob.run(runtimeContext)
             self.stdout.write(runnerjob.uuid+"\n")
             return (None, "success")
 
+        # We either running the workflow directly, or submitting it
+        # and will wait for a final result.
+
+        self.runtime_status_update("activity", "workflow execution")
+
         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)