summaryrefslogtreecommitdiff
path: root/lib/spack/spack/ci.py
diff options
context:
space:
mode:
authorScott Wittenburg <scott.wittenburg@kitware.com>2021-05-28 10:38:07 -0600
committerGitHub <noreply@github.com>2021-05-28 09:38:07 -0700
commit91f66ea0a47bed49b1bf79461eb0e72e9ef6acf0 (patch)
tree70b0a2e08a24b33e9e555b50b02d00c46efdb10a /lib/spack/spack/ci.py
parent4262de6a32b086c388bf318be778e42b418d1dc2 (diff)
downloadspack-91f66ea0a47bed49b1bf79461eb0e72e9ef6acf0.tar.gz
spack-91f66ea0a47bed49b1bf79461eb0e72e9ef6acf0.tar.bz2
spack-91f66ea0a47bed49b1bf79461eb0e72e9ef6acf0.tar.xz
spack-91f66ea0a47bed49b1bf79461eb0e72e9ef6acf0.zip
Pipelines: reproducible builds (#22887)
### Overview The goal of this PR is to make gitlab pipeline builds (especially build failures) more reproducible outside of the pipeline environment. The two key changes here which aim to improve reproducibility are: 1. Produce a `spack.lock` during pipeline generation which is passed to child jobs via artifacts. This concretized environment is used both by generated child jobs as well as uploaded as an artifact to be used when reproducing the build locally. 2. In the `spack ci rebuild` command, if a spec needs to be rebuilt from source, do this by generating and running an `install.sh` shell script which is then also uploaded as a job artifact to be run during local reproduction. To make it easier to take advantage of improved build reproducibility, this PR also adds a new subcommand, `spack ci reproduce-build`, which, given a url to job artifacts: - fetches and unzips the job artifacts to a local directory - looks for the generated pipeline yaml and parses it to find details about the job to reproduce - attempts to provide a copy of the same version of spack used in the ci build - if the ci build used a docker image, the command prints a `docker run` command you can run to get an interactive shell for reproducing the build #### Some highlights One consequence of this change will be much smaller pipeline yaml files. By encoding the concrete environment in a `spack.lock` and passing to child jobs via artifacts, we will no longer need to encode the concrete root of each spec and write it into the job variables, greatly reducing the size of the generated pipeline yaml. Additionally `spack ci rebuild` output (stdout/stderr) is no longer internally redirected to a log file, so job output will appear directly in the gitlab job trace. With debug logging turned on, this often results in log files getting truncated because they exceed the maximum amount of log output gitlab allows. If this is a problem, you still have the option to `tee` command output to a file in the within the artifacts directory, as now each generated job exposes a `user_data` directory as an artifact, which you can fill with whatever you want in your custom job scripts. There are some changes to be aware of in how pipelines should be set up after this PR: #### Pipeline generation Because the pipeline generation job now writes a `spack.lock` artifact to be consumed by generated downstream jobs, `spack ci generate` takes a new option `--artifacts-root`, inside which it creates a `concrete_env` directory to place the lockfile. This artifacts root directory is also where the `user_data` directory will live, in case you want to generate any custom artifacts. If you do not provide `--artifacts-root`, the default is for it to create a `jobs_scratch_dir` within your `CI_PROJECT_DIR` (a gitlab predefined environment variable) or whatever is your current working directory if that variable isn't set. Here's the diff of the PR testing `.gitlab-ci.yml` taking advantage of the new option: ``` $ git diff develop..pipelines-reproducible-builds share/spack/gitlab/cloud_pipelines/.gitlab-ci.yml diff --git a/share/spack/gitlab/cloud_pipelines/.gitlab-ci.yml b/share/spack/gitlab/cloud_pipelines/.gitlab-ci.yml index 579d7b56f3..0247803a30 100644 --- a/share/spack/gitlab/cloud_pipelines/.gitlab-ci.yml +++ b/share/spack/gitlab/cloud_pipelines/.gitlab-ci.yml @@ -28,10 +28,11 @@ default: - cd share/spack/gitlab/cloud_pipelines/stacks/${SPACK_CI_STACK_NAME} - spack env activate --without-view . - spack ci generate --check-index-only + --artifacts-root "${CI_PROJECT_DIR}/jobs_scratch_dir" --output-file "${CI_PROJECT_DIR}/jobs_scratch_dir/cloud-ci-pipeline.yml" artifacts: paths: - - "${CI_PROJECT_DIR}/jobs_scratch_dir/cloud-ci-pipeline.yml" + - "${CI_PROJECT_DIR}/jobs_scratch_dir" tags: ["spack", "public", "medium", "x86_64"] interruptible: true ``` Notice how we replaced the specific pointer to the generated pipeline file with its containing folder, the same folder we passed as `--artifacts-root`. This way anything in that directory (the generated pipeline yaml, as well as the concrete environment directory containing the `spack.lock`) will be uploaded as an artifact and available to the downstream jobs. #### Rebuild jobs Rebuild jobs now must activate the concrete environment created by `spack ci generate` and provided via artifacts. When the pipeline is generated, a directory called `concrete_environment` is created within the artifacts root directory, and this is where the `spack.lock` file is written to be passed to the generated rebuild jobs. The artifacts root directory can be specified using the `--artifacts-root` option to `spack ci generate`, otherwise, it is assumed to be `$CI_PROJECT_DIR`. The directory containing the concrete environment files (`spack.yaml` and `spack.lock`) is then passed to generated child jobs via the `SPACK_CONCRETE_ENV_DIR` variable in the generated pipeline yaml file. When you don't provide custom `script` sections in your `mappings` within the `gitlab-ci` section of your `spack.yaml`, the default behavior of rebuild jobs is now to change into `SPACK_CONCRETE_ENV_DIR` and activate that environment. If you do provide custom rebuild scripts in your `spack.yaml`, be aware those scripts should do the same thing: assume `SPACK_CONCRETE_ENV_DIR` contains the concretized environment to activate. No other changes to existing custom rebuild scripts should be required as a result of this PR. As mentioned above, one key change made in this PR is the generation of the `install.sh` script by the rebuild jobs, as that same script is both run by the CI rebuild job as well as exported as an artifact to aid in subsequent attempts to reproduce the build outside of CI. The generated `install.sh` script contains only a single `spack install` command with arguments computed by `spack ci rebuild`. If the install fails, the job trace in gitlab will contain instructions on how to reproduce the build locally: ``` To reproduce this build locally, run: spack ci reproduce-build https://gitlab.next.spack.io/api/v4/projects/7/jobs/240607/artifacts [--working-dir <dir>] If this project does not have public pipelines, you will need to first: export GITLAB_PRIVATE_TOKEN=<generated_token> ... then follow the printed instructions. ``` When run locally, the `spack ci reproduce-build` command shown above will download and process the job artifacts from gitlab, then print out instructions you can copy-paste to run a local reproducer of the CI job. This PR includes a few other changes to the way pipelines work, see the documentation on pipelines for more details. This PR erelies on ~- [ ] #23194 to be able to refer to uninstalled specs by DAG hash~ EDIT: that is going to take longer to come to fruition, so for now, we will continue to install specs represented by a concrete `spec.yaml` file on disk. - [x] #22657 to support install a single spec already present in the active, concrete environment
Diffstat (limited to 'lib/spack/spack/ci.py')
-rw-r--r--lib/spack/spack/ci.py447
1 files changed, 404 insertions, 43 deletions
diff --git a/lib/spack/spack/ci.py b/lib/spack/spack/ci.py
index 4daffac44c..f94000e1f7 100644
--- a/lib/spack/spack/ci.py
+++ b/lib/spack/spack/ci.py
@@ -10,8 +10,9 @@ import json
import os
import re
import shutil
+import stat
import tempfile
-import zlib
+import zipfile
from six import iteritems
from six.moves.urllib.error import HTTPError, URLError
@@ -19,6 +20,7 @@ from six.moves.urllib.parse import urlencode
from six.moves.urllib.request import build_opener, HTTPHandler, Request
import llnl.util.tty as tty
+import llnl.util.filesystem as fs
import spack
import spack.binary_distribution as bindist
@@ -27,10 +29,12 @@ import spack.compilers as compilers
import spack.config as cfg
import spack.environment as ev
from spack.error import SpackError
-import spack.hash_types as ht
import spack.main
+import spack.mirror
+import spack.paths
import spack.repo
from spack.spec import Spec
+import spack.util.executable as exe
import spack.util.spack_yaml as syaml
import spack.util.web as web_util
import spack.util.gpg as gpg_util
@@ -197,10 +201,7 @@ def format_root_spec(spec, main_phase, strip_compiler):
return '{0}@{1} arch={2}'.format(
spec.name, spec.version, spec.architecture)
else:
- spec_yaml = spec.to_yaml(hash=ht.build_hash).encode('utf-8')
- return str(base64.b64encode(zlib.compress(spec_yaml)).decode('utf-8'))
- # return '{0}@{1}%{2} arch={3}'.format(
- # spec.name, spec.version, spec.compiler, spec.architecture)
+ return spec.build_hash()
def spec_deps_key(s):
@@ -513,28 +514,14 @@ def format_job_needs(phase_name, strip_compilers, dep_jobs,
return needs_list
-def add_pr_mirror(url):
- cfg_scope = cfg.default_modify_scope()
- mirrors = cfg.get('mirrors', scope=cfg_scope)
- items = [(n, u) for n, u in mirrors.items()]
- items.insert(0, ('ci_pr_mirror', url))
- cfg.set('mirrors', syaml.syaml_dict(items), scope=cfg_scope)
-
-
-def remove_pr_mirror():
- cfg_scope = cfg.default_modify_scope()
- mirrors = cfg.get('mirrors', scope=cfg_scope)
- mirrors.pop('ci_pr_mirror')
- cfg.set('mirrors', mirrors, scope=cfg_scope)
-
-
-def generate_gitlab_ci_yaml(env, print_summary, output_file, prune_dag=False,
- check_index_only=False, run_optimizer=False,
- use_dependencies=False):
- # FIXME: What's the difference between one that opens with 'spack'
- # and one that opens with 'env'? This will only handle the former.
+def generate_gitlab_ci_yaml(env, print_summary, output_file,
+ prune_dag=False, check_index_only=False,
+ run_optimizer=False, use_dependencies=False,
+ artifacts_root=None):
with spack.concretize.disable_compiler_existence_check():
- env.concretize()
+ with env.write_transaction():
+ env.concretize()
+ env.write()
yaml_root = ev.config_dict(env.yaml)
@@ -559,6 +546,9 @@ def generate_gitlab_ci_yaml(env, print_summary, output_file, prune_dag=False,
tty.verbose("Using CDash auth token from environment")
cdash_auth_token = os.environ.get('SPACK_CDASH_AUTH_TOKEN')
+ generate_job_name = os.environ.get('CI_JOB_NAME', None)
+ parent_pipeline_id = os.environ.get('CI_PIPELINE_ID', None)
+
is_pr_pipeline = (
os.environ.get('SPACK_IS_PR_PIPELINE', '').lower() == 'true'
)
@@ -574,6 +564,7 @@ def generate_gitlab_ci_yaml(env, print_summary, output_file, prune_dag=False,
ci_mirrors = yaml_root['mirrors']
mirror_urls = [url for url in ci_mirrors.values()]
+ remote_mirror_url = mirror_urls[0]
# Check for a list of "known broken" specs that we should not bother
# trying to build.
@@ -624,7 +615,32 @@ def generate_gitlab_ci_yaml(env, print_summary, output_file, prune_dag=False,
# Add this mirror if it's enabled, as some specs might be up to date
# here and thus not need to be rebuilt.
if pr_mirror_url:
- add_pr_mirror(pr_mirror_url)
+ spack.mirror.add(
+ 'ci_pr_mirror', pr_mirror_url, cfg.default_modify_scope())
+
+ pipeline_artifacts_dir = artifacts_root
+ if not pipeline_artifacts_dir:
+ proj_dir = os.environ.get('CI_PROJECT_DIR', os.getcwd())
+ pipeline_artifacts_dir = os.path.join(proj_dir, 'jobs_scratch_dir')
+
+ pipeline_artifacts_dir = os.path.abspath(pipeline_artifacts_dir)
+ concrete_env_dir = os.path.join(
+ pipeline_artifacts_dir, 'concrete_environment')
+
+ # Now that we've added the mirrors we know about, they should be properly
+ # reflected in the environment manifest file, so copy that into the
+ # concrete environment directory, along with the spack.lock file.
+ if not os.path.exists(concrete_env_dir):
+ os.makedirs(concrete_env_dir)
+ shutil.copyfile(env.manifest_path,
+ os.path.join(concrete_env_dir, 'spack.yaml'))
+ shutil.copyfile(env.lock_path,
+ os.path.join(concrete_env_dir, 'spack.lock'))
+
+ job_log_dir = os.path.join(pipeline_artifacts_dir, 'logs')
+ job_repro_dir = os.path.join(pipeline_artifacts_dir, 'reproduction')
+ local_mirror_dir = os.path.join(pipeline_artifacts_dir, 'mirror')
+ user_artifacts_dir = os.path.join(pipeline_artifacts_dir, 'user_data')
# Speed up staging by first fetching binary indices from all mirrors
# (including the per-PR mirror we may have just added above).
@@ -641,7 +657,7 @@ def generate_gitlab_ci_yaml(env, print_summary, output_file, prune_dag=False,
finally:
# Clean up PR mirror if enabled
if pr_mirror_url:
- remove_pr_mirror()
+ spack.mirror.remove('ci_pr_mirror', cfg.default_modify_scope())
all_job_names = []
output_object = {}
@@ -705,10 +721,16 @@ def generate_gitlab_ci_yaml(env, print_summary, output_file, prune_dag=False,
except AttributeError:
image_name = build_image
- job_script = [
- 'spack env activate --without-view .',
- 'spack ci rebuild',
- ]
+ job_script = ['spack env activate --without-view .']
+
+ if artifacts_root:
+ job_script.insert(0, 'cd {0}'.format(concrete_env_dir))
+
+ job_script.extend([
+ 'spack ci rebuild --prepare',
+ './install.sh'
+ ])
+
if 'script' in runner_attribs:
job_script = [s for s in runner_attribs['script']]
@@ -735,9 +757,9 @@ def generate_gitlab_ci_yaml(env, print_summary, output_file, prune_dag=False,
job_vars = {
'SPACK_ROOT_SPEC': format_root_spec(
root_spec, main_phase, strip_compilers),
+ 'SPACK_JOB_SPEC_DAG_HASH': release_spec.dag_hash(),
'SPACK_JOB_SPEC_PKG_NAME': release_spec.name,
- 'SPACK_COMPILER_ACTION': compiler_action,
- 'SPACK_IS_PR_PIPELINE': str(is_pr_pipeline),
+ 'SPACK_COMPILER_ACTION': compiler_action
}
job_dependencies = []
@@ -836,6 +858,12 @@ def generate_gitlab_ci_yaml(env, print_summary, output_file, prune_dag=False,
if prune_dag and not rebuild_spec:
continue
+ if artifacts_root:
+ job_dependencies.append({
+ 'job': generate_job_name,
+ 'pipeline': '{0}'.format(parent_pipeline_id)
+ })
+
job_vars['SPACK_SPEC_NEEDS_REBUILD'] = str(rebuild_spec)
if enable_cdash_reporting:
@@ -856,12 +884,14 @@ def generate_gitlab_ci_yaml(env, print_summary, output_file, prune_dag=False,
variables.update(job_vars)
artifact_paths = [
- 'jobs_scratch_dir',
- 'cdash_report',
+ job_log_dir,
+ job_repro_dir,
+ user_artifacts_dir
]
if enable_artifacts_buildcache:
- bc_root = 'local_mirror/build_cache'
+ bc_root = os.path.join(
+ local_mirror_dir, 'build_cache')
artifact_paths.extend([os.path.join(bc_root, p) for p in [
bindist.tarball_name(release_spec, '.spec.yaml'),
bindist.tarball_name(release_spec, '.cdashid'),
@@ -987,6 +1017,11 @@ def generate_gitlab_ci_yaml(env, print_summary, output_file, prune_dag=False,
]
final_job['when'] = 'always'
+ if artifacts_root:
+ final_job['variables'] = {
+ 'SPACK_CONCRETE_ENV_DIR': concrete_env_dir
+ }
+
output_object['rebuild-index'] = final_job
output_object['stages'] = stage_names
@@ -1007,8 +1042,15 @@ def generate_gitlab_ci_yaml(env, print_summary, output_file, prune_dag=False,
version_to_clone = spack_version
output_object['variables'] = {
+ 'SPACK_ARTIFACTS_ROOT': pipeline_artifacts_dir,
+ 'SPACK_CONCRETE_ENV_DIR': concrete_env_dir,
'SPACK_VERSION': spack_version,
'SPACK_CHECKOUT_VERSION': version_to_clone,
+ 'SPACK_REMOTE_MIRROR_URL': remote_mirror_url,
+ 'SPACK_JOB_LOG_DIR': job_log_dir,
+ 'SPACK_JOB_REPRO_DIR': job_repro_dir,
+ 'SPACK_LOCAL_MIRROR_DIR': local_mirror_dir,
+ 'SPACK_IS_PR_PIPELINE': str(is_pr_pipeline)
}
if pr_mirror_url:
@@ -1131,7 +1173,8 @@ def configure_compilers(compiler_action, scope=None):
return None
-def get_concrete_specs(root_spec, job_name, related_builds, compiler_action):
+def get_concrete_specs(env, root_spec, job_name, related_builds,
+ compiler_action):
spec_map = {
'root': None,
'deps': {},
@@ -1153,8 +1196,7 @@ def get_concrete_specs(root_spec, job_name, related_builds, compiler_action):
# again. The reason we take this path in the first case (bootstrapped
# compiler), is that we can't concretize a spec at this point if we're
# going to ask spack to "install_missing_compilers".
- concrete_root = Spec.from_yaml(
- str(zlib.decompress(base64.b64decode(root_spec)).decode('utf-8')))
+ concrete_root = env.specs_by_hash[root_spec]
spec_map['root'] = concrete_root
spec_map[job_name] = concrete_root[job_name]
@@ -1205,7 +1247,7 @@ def register_cdash_build(build_name, base_url, project, site, track):
def relate_cdash_builds(spec_map, cdash_base_url, job_build_id, cdash_project,
- cdashids_mirror_url):
+ cdashids_mirror_urls):
if not job_build_id:
return
@@ -1221,7 +1263,19 @@ def relate_cdash_builds(spec_map, cdash_base_url, job_build_id, cdash_project,
for dep_pkg_name in dep_map:
tty.debug('Fetching cdashid file for {0}'.format(dep_pkg_name))
dep_spec = dep_map[dep_pkg_name]
- dep_build_id = read_cdashid_from_mirror(dep_spec, cdashids_mirror_url)
+ dep_build_id = None
+
+ for url in cdashids_mirror_urls:
+ try:
+ if url:
+ dep_build_id = read_cdashid_from_mirror(dep_spec, url)
+ break
+ except web_util.SpackWebError:
+ tty.debug('Did not find cdashid for {0} on {1}'.format(
+ dep_pkg_name, url))
+ else:
+ raise SpackError('Did not find cdashid for {0} anywhere'.format(
+ dep_pkg_name))
payload = {
"project": cdash_project,
@@ -1335,3 +1389,310 @@ def copy_stage_logs_to_artifacts(job_spec, job_log_dir):
msg = ('Unable to copy build logs from stage to artifacts '
'due to exception: {0}').format(inst)
tty.error(msg)
+
+
+def download_and_extract_artifacts(url, work_dir):
+ tty.msg('Fetching artifacts from: {0}\n'.format(url))
+
+ headers = {
+ 'Content-Type': 'application/zip',
+ }
+
+ token = os.environ.get('GITLAB_PRIVATE_TOKEN', None)
+ if token:
+ headers['PRIVATE-TOKEN'] = token
+
+ opener = build_opener(HTTPHandler)
+
+ request = Request(url, headers=headers)
+ request.get_method = lambda: 'GET'
+
+ response = opener.open(request)
+ response_code = response.getcode()
+
+ if response_code != 200:
+ msg = 'Error response code ({0}) in reproduce_ci_job'.format(
+ response_code)
+ raise SpackError(msg)
+
+ artifacts_zip_path = os.path.join(work_dir, 'artifacts.zip')
+
+ if not os.path.exists(work_dir):
+ os.makedirs(work_dir)
+
+ with open(artifacts_zip_path, 'wb') as out_file:
+ shutil.copyfileobj(response, out_file)
+
+ zip_file = zipfile.ZipFile(artifacts_zip_path)
+ zip_file.extractall(work_dir)
+ zip_file.close()
+
+ os.remove(artifacts_zip_path)
+
+
+def get_spack_info():
+ git_path = os.path.join(spack.paths.prefix, ".git")
+ if os.path.exists(git_path):
+ git = exe.which("git")
+ if git:
+ with fs.working_dir(spack.paths.prefix):
+ git_log = git("log", "-1",
+ output=str, error=os.devnull,
+ fail_on_error=False)
+
+ return git_log
+
+ return 'no git repo, use spack {0}'.format(spack.spack_version)
+
+
+def setup_spack_repro_version(repro_dir, checkout_commit, merge_commit=None):
+ # figure out the path to the spack git version being used for the
+ # reproduction
+ print('checkout_commit: {0}'.format(checkout_commit))
+ print('merge_commit: {0}'.format(merge_commit))
+
+ dot_git_path = os.path.join(spack.paths.prefix, ".git")
+ if not os.path.exists(dot_git_path):
+ tty.error('Unable to find the path to your local spack clone')
+ return False
+
+ spack_git_path = spack.paths.prefix
+
+ git = exe.which("git")
+ if not git:
+ tty.error("reproduction of pipeline job requires git")
+ return False
+
+ # Check if we can find the tested commits in your local spack repo
+ with fs.working_dir(spack_git_path):
+ git("log", "-1", checkout_commit, output=str, error=os.devnull,
+ fail_on_error=False)
+
+ if git.returncode != 0:
+ tty.error('Missing commit: {0}'.format(checkout_commit))
+ return False
+
+ if merge_commit:
+ git("log", "-1", merge_commit, output=str, error=os.devnull,
+ fail_on_error=False)
+
+ if git.returncode != 0:
+ tty.error('Missing commit: {0}'.format(merge_commit))
+ return False
+
+ # Next attempt to clone your local spack repo into the repro dir
+ with fs.working_dir(repro_dir):
+ clone_out = git("clone", spack_git_path,
+ output=str, error=os.devnull,
+ fail_on_error=False)
+
+ if git.returncode != 0:
+ tty.error('Unable to clone your local spac repo:')
+ tty.msg(clone_out)
+ return False
+
+ # Finally, attempt to put the cloned repo into the same state used during
+ # the pipeline build job
+ repro_spack_path = os.path.join(repro_dir, 'spack')
+ with fs.working_dir(repro_spack_path):
+ co_out = git("checkout", checkout_commit,
+ output=str, error=os.devnull,
+ fail_on_error=False)
+
+ if git.returncode != 0:
+ tty.error('Unable to checkout {0}'.format(checkout_commit))
+ tty.msg(co_out)
+ return False
+
+ if merge_commit:
+ merge_out = git("-c", "user.name=cirepro", "-c",
+ "user.email=user@email.org", "merge",
+ "--no-edit", merge_commit,
+ output=str, error=os.devnull,
+ fail_on_error=False)
+
+ if git.returncode != 0:
+ tty.error('Unable to merge {0}'.format(merge_commit))
+ tty.msg(merge_out)
+ return False
+
+ return True
+
+
+def reproduce_ci_job(url, work_dir):
+ download_and_extract_artifacts(url, work_dir)
+
+ lock_file = fs.find(work_dir, 'spack.lock')[0]
+ concrete_env_dir = os.path.dirname(lock_file)
+
+ tty.debug('Concrete environment directory: {0}'.format(
+ concrete_env_dir))
+
+ yaml_files = fs.find(work_dir, ['*.yaml', '*.yml'])
+
+ tty.debug('yaml files:')
+ for yaml_file in yaml_files:
+ tty.debug(' {0}'.format(yaml_file))
+
+ pipeline_yaml = None
+ pipeline_variables = None
+
+ # Try to find the dynamically generated pipeline yaml file in the
+ # reproducer. If the user did not put it in the artifacts root,
+ # but rather somewhere else and exported it as an artifact from
+ # that location, we won't be able to find it.
+ for yf in yaml_files:
+ with open(yf) as y_fd:
+ yaml_obj = syaml.load(y_fd)
+ if 'variables' in yaml_obj and 'stages' in yaml_obj:
+ pipeline_yaml = yaml_obj
+ pipeline_variables = pipeline_yaml['variables']
+
+ if pipeline_yaml:
+ tty.debug('\n{0} is likely your pipeline file'.format(yf))
+
+ # Find the install script in the unzipped artifacts and make it executable
+ install_script = fs.find(work_dir, 'install.sh')[0]
+ st = os.stat(install_script)
+ os.chmod(install_script, st.st_mode | stat.S_IEXEC)
+
+ # Find the repro details file. This just includes some values we wrote
+ # during `spack ci rebuild` to make reproduction easier. E.g. the job
+ # name is written here so we can easily find the configuration of the
+ # job from the generated pipeline file.
+ repro_file = fs.find(work_dir, 'repro.json')[0]
+ repro_details = None
+ with open(repro_file) as fd:
+ repro_details = json.load(fd)
+
+ repro_dir = os.path.dirname(repro_file)
+ rel_repro_dir = repro_dir.replace(work_dir, '').lstrip(os.path.sep)
+
+ # Find the spack info text file that should contain the git log
+ # of the HEAD commit used during the CI build
+ spack_info_file = fs.find(work_dir, 'spack_info.txt')[0]
+ with open(spack_info_file) as fd:
+ spack_info = fd.read()
+
+ # Access the specific job configuration
+ job_name = repro_details['job_name']
+ job_yaml = None
+
+ if job_name in pipeline_yaml:
+ job_yaml = pipeline_yaml[job_name]
+
+ if job_yaml:
+ tty.debug('Found job:')
+ tty.debug(job_yaml)
+
+ job_image = None
+ setup_result = False
+ if 'image' in job_yaml:
+ job_image_elt = job_yaml['image']
+ if 'name' in job_image_elt:
+ job_image = job_image_elt['name']
+ else:
+ job_image = job_image_elt
+ tty.msg('Job ran with the following image: {0}'.format(job_image))
+
+ # Because we found this job was run with a docker image, so we will try
+ # to print a "docker run" command that bind-mounts the directory where
+ # we extracted the artifacts.
+
+ # Destination of bind-mounted reproduction directory. It makes for a
+ # more faithful reproducer if everything appears to run in the same
+ # absolute path used during the CI build.
+ mount_as_dir = '/work'
+ if pipeline_variables:
+ artifacts_root = pipeline_variables['SPACK_ARTIFACTS_ROOT']
+ mount_as_dir = os.path.dirname(artifacts_root)
+ mounted_repro_dir = os.path.join(mount_as_dir, rel_repro_dir)
+
+ # We will also try to clone spack from your local checkout and
+ # reproduce the state present during the CI build, and put that into
+ # the bind-mounted reproducer directory.
+
+ # Regular expressions for parsing that HEAD commit. If the pipeline
+ # was on the gitlab spack mirror, it will have been a merge commit made by
+ # gitub and pushed by the sync script. If the pipeline was run on some
+ # environment repo, then the tested spack commit will likely have been
+ # a regular commit.
+ commit_1 = None
+ commit_2 = None
+ commit_regex = re.compile(r"commit\s+([^\s]+)")
+ merge_commit_regex = re.compile(r"Merge\s+([^\s]+)\s+into\s+([^\s]+)")
+
+ # Try the more specific merge commit regex first
+ m = merge_commit_regex.search(spack_info)
+ if m:
+ # This was a merge commit and we captured the parents
+ commit_1 = m.group(1)
+ commit_2 = m.group(2)
+ else:
+ # Not a merge commit, just get the commit sha
+ m = commit_regex.search(spack_info)
+ if m:
+ commit_1 = m.group(1)
+
+ setup_result = False
+ if commit_1:
+ if commit_2:
+ setup_result = setup_spack_repro_version(
+ work_dir, commit_2, merge_commit=commit_1)
+ else:
+ setup_result = setup_spack_repro_version(work_dir, commit_1)
+
+ if not setup_result:
+ setup_msg = """
+ This can happen if the spack you are using to run this command is not a git
+ repo, or if it is a git repo, but it does not have the commits needed to
+ recreate the tested merge commit. If you are trying to reproduce a spack
+ PR pipeline job failure, try fetching the latest develop commits from
+ mainline spack and make sure you have the most recent commit of the PR
+ branch in your local spack repo. Then run this command again.
+ Alternatively, you can also manually clone spack if you know the version
+ you want to test.
+ """
+ tty.error('Failed to automatically setup the tested version of spack '
+ 'in your local reproduction directory.')
+ print(setup_msg)
+
+ # In cases where CI build was run on a shell runner, it might be useful
+ # to see what tags were applied to the job so the user knows what shell
+ # runner was used. But in that case in general, we cannot do nearly as
+ # much to set up the reproducer.
+ job_tags = None
+ if 'tags' in job_yaml:
+ job_tags = job_yaml['tags']
+ tty.msg('Job ran with the following tags: {0}'.format(job_tags))
+
+ inst_list = []
+
+ # Finally, print out some instructions to reproduce the build
+ if job_image:
+ inst_list.append('\nRun the following command:\n\n')
+ inst_list.append(' $ docker run --rm -v {0}:{1} -ti {2}\n'.format(
+ work_dir, mount_as_dir, job_image))
+ inst_list.append('\nOnce inside the container:\n\n')
+ else:
+ inst_list.append('\nOnce on the tagged runner:\n\n')
+
+ if not setup_result:
+ inst_list.append(' - Clone spack and acquire tested commit\n')
+ inst_list.append('{0}'.format(spack_info))
+ spack_root = '<spack-clone-path>'
+ else:
+ spack_root = '{0}/spack'.format(mount_as_dir)
+
+ inst_list.append(' - Activate the environment\n\n')
+ inst_list.append(' $ source {0}/share/spack/setup-env.sh\n'.format(
+ spack_root))
+ inst_list.append(
+ ' $ spack env activate --without-view {0}\n\n'.format(
+ mounted_repro_dir if job_image else repro_dir))
+ inst_list.append(' - Run the install script\n\n')
+ inst_list.append(' $ {0}\n'.format(
+ os.path.join(mounted_repro_dir, 'install.sh')
+ if job_image else install_script))
+
+ print(''.join(inst_list))