Synchronize test-prepare-workspace-git to prepare-workspace-git

This updates prepare-workspace-git to the new module-based system.

Change-Id: I99f1673775c161f1f8b5d95ce5daee3363571e3b
This commit is contained in:
James E. Blair 2024-08-01 14:48:31 -07:00
parent f1714ea75b
commit e3d891d16d
11 changed files with 492 additions and 87 deletions

View File

@ -24,10 +24,8 @@ The cached repos need to be placed using the canonical name under the
list to only use projects which are required by the job.
.. zuul:rolevar:: mirror_workspace_quiet
:default: false
If `true` git operations will be silenced and won't print every
changed reference.
This value is ignored; it should be removed from job configuration.
.. zuul:rolevar:: zuul_workspace_root
:default: "{{ ansible_user_dir }}"

View File

@ -0,0 +1,82 @@
# Copyright 2024 Acme Gating, LLC
#
# Licensed under the Apache License, Version 2.0 (the "License"); you may
# not use this file except in compliance with the License. You may obtain
# a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
# License for the specific language governing permissions and limitations
# under the License.
import os
import time
from ansible.module_utils.basic import AnsibleModule
try:
# Ansible context
from ansible.module_utils.zuul_jobs.workspace_utils import (
run,
for_each_project,
)
except ImportError:
# Test context
from ..module_utils.zuul_jobs.workspace_utils import (
run,
for_each_project,
)
def prep_one_project(args, project, output):
start = time.monotonic()
dest = f"{args['zuul_workspace_root']}/{project['src_dir']}"
output['dest'] = dest
if not os.path.isdir(dest):
cache = f"{args['cached_repos_root']}/{project['canonical_name']}"
if os.path.isdir(cache):
# We do a bare clone here first so that we skip creating a working
# copy that will be overwritten later anyway.
output['initial_state'] = 'cloned-from-cache'
out = run(f"git clone --bare {cache} {dest}/.git")
output['clone'] = out.stdout.decode('utf8').strip()
else:
output['initial_state'] = 'git-init'
out = run(f"git init {dest}")
output['init'] = out.stdout.decode('utf8').strip()
else:
output['initial_state'] = 'pre-existing'
run("git config --local --bool core.bare false", cwd=dest)
# Allow pushing to non-bare repo
run("git config --local receive.denyCurrentBranch ignore", cwd=dest)
# Allow deleting current branch
run("git config --local receive.denyDeleteCurrent ignore", cwd=dest)
run("git remote rm origin", cwd=dest, check=False)
run("git remote add origin file:///dev/null", cwd=dest)
end = time.monotonic()
output['elapsed'] = end - start
def ansible_main():
module = AnsibleModule(
argument_spec=dict(
cached_repos_root=dict(type='path'),
executor_work_root=dict(type='path'),
zuul_projects=dict(type='dict'),
zuul_workspace_root=dict(type='path'),
)
)
output = {}
if for_each_project(prep_one_project, module.params, output):
module.exit_json(changed=True, output=output)
else:
module.fail_json("Failure preparing repos", output=output)
if __name__ == '__main__':
ansible_main()

View File

@ -0,0 +1,109 @@
# Copyright 2024 Acme Gating, LLC
#
# Licensed under the Apache License, Version 2.0 (the "License"); you may
# not use this file except in compliance with the License. You may obtain
# a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
# License for the specific language governing permissions and limitations
# under the License.
import os
import time
from ansible.module_utils.basic import AnsibleModule
try:
# Ansible context
from ansible.module_utils.zuul_jobs.workspace_utils import (
run,
for_each_project,
)
except ImportError:
# Test context
from ..module_utils.zuul_jobs.workspace_utils import (
run,
for_each_project,
)
def get_ssh_dest(args, dest):
return (
f"git+ssh://{args['ansible_user']}"
f"@{args['ansible_host']}"
f":{args['ansible_port']}"
f"/{dest}"
)
def get_k8s_dest(args, dest):
resources = args['zuul_resources'][args['inventory_hostname']]
return (
f"\"ext::kubectl "
f"--context {resources['context']} "
f"-n {resources['namespace']} "
f"exec -i {resources['pod']} "
f"-- %S {dest}\""
)
def sync_one_project(args, project, output):
cwd = f"{args['executor_work_root']}/{project['src_dir']}"
dest = f"{args['zuul_workspace_root']}/{project['src_dir']}"
output['src'] = cwd
output['dest'] = dest
env = os.environ.copy()
env['GIT_ALLOW_PROTOCOL'] = 'ext:ssh'
# We occasionally see git pushes in the middle of this loop fail then
# subsequent pushes for other repos succeed. The entire loop ends up
# failing because one of the pushes failed. Mitigate this by retrying
# on failure.
max_tries = 3
start = time.monotonic()
for count in range(max_tries):
try:
if args['ansible_connection'] == "kubectl":
git_dest = get_k8s_dest(args, dest)
else:
git_dest = get_ssh_dest(args, dest)
out = run(f"git push --quiet --mirror {git_dest}",
cwd=cwd, env=env)
output['push'] = out.stdout.decode('utf8').strip()
break
except Exception:
if count + 1 >= max_tries:
raise
end = time.monotonic()
output['attempts'] = count + 1
output['elapsed'] = end - start
def ansible_main():
module = AnsibleModule(
argument_spec=dict(
ansible_connection=dict(type='str'),
ansible_host=dict(type='str'),
ansible_port=dict(type='int'),
ansible_user=dict(type='str'),
executor_work_root=dict(type='path'),
inventory_hostname=dict(type='str'),
mirror_workspace_quiet=dict(type='bool'),
zuul_projects=dict(type='dict'),
zuul_resources=dict(type='dict'),
zuul_workspace_root=dict(type='path'),
)
)
output = {}
if for_each_project(sync_one_project, module.params, output):
module.exit_json(changed=True, output=output)
else:
module.fail_json("Failure synchronizing repos", output=output)
if __name__ == '__main__':
ansible_main()

View File

@ -0,0 +1,71 @@
# Copyright 2024 Acme Gating, LLC
#
# Licensed under the Apache License, Version 2.0 (the "License"); you may
# not use this file except in compliance with the License. You may obtain
# a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
# License for the specific language governing permissions and limitations
# under the License.
import time
from ansible.module_utils.basic import AnsibleModule
try:
# Ansible context
from ansible.module_utils.zuul_jobs.workspace_utils import (
run,
for_each_project,
)
except ImportError:
# Test context
from ..module_utils.zuul_jobs.workspace_utils import (
run,
for_each_project,
)
def update_one_project(args, project, output):
cwd = f"{args['zuul_workspace_root']}/{project['src_dir']}"
output['dest'] = cwd
start = time.monotonic()
# Reset is needed because we pushed to a non-bare repo
run("git reset --hard", cwd=cwd)
# Clean is needed because we pushed to a non-bare repo
run("git clean -xdf", cwd=cwd)
# Undo the config setting we did in repo_prep
run("git config --local --unset receive.denyCurrentBranch", cwd=cwd)
run("git config --local --unset receive.denyDeleteCurrent", cwd=cwd)
# checkout the branch matching the branch set up by the executor
out = run(f"git checkout --quiet {project['checkout']}", cwd=cwd)
output['checkout'] = out.stdout.decode('utf8').strip()
# put out a status line with the current HEAD
out = run("git log --pretty=oneline -1", cwd=cwd)
end = time.monotonic()
output['HEAD'] = out.stdout.decode('utf8').strip()
output['elapsed'] = end - start
def ansible_main():
module = AnsibleModule(
argument_spec=dict(
zuul_projects=dict(type='dict'),
zuul_workspace_root=dict(type='path'),
)
)
output = {}
if for_each_project(update_one_project, module.params, output):
module.exit_json(changed=True, output=output)
else:
module.fail_json("Failure updating repos", output=output)
if __name__ == '__main__':
ansible_main()

View File

@ -0,0 +1,155 @@
# Copyright 2024 Acme Gating, LLC
#
# Licensed under the Apache License, Version 2.0 (the "License"); you may
# not use this file except in compliance with the License. You may obtain
# a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
# License for the specific language governing permissions and limitations
# under the License.
import os
import pprint
import shutil
import testtools
import fixtures
from ..module_utils.zuul_jobs.workspace_utils import for_each_project, run
from .repo_prep import prep_one_project
from .repo_sync import sync_one_project
from .repo_update import update_one_project
from . import repo_sync
class TestPrepareWorkspace(testtools.TestCase):
def _test_prepare_workspace(self, connection, cached):
executor_root = self.useFixture(fixtures.TempDir()).path
project_root = os.path.join(executor_root,
'example.com/org/test-project')
os.makedirs(project_root)
try:
run("git init .", cwd=project_root)
run("git config --local user.email user@example.com",
cwd=project_root)
run("git config --local user.name username",
cwd=project_root)
with open(os.path.join(project_root, 'README'), 'w') as f:
f.write('test')
run("git add README", cwd=project_root)
run("git commit -a -m init", cwd=project_root)
except Exception as e:
if hasattr(e, 'output'):
msg = f'{str(e)} : {e.output}'
else:
msg = str(e)
print(msg)
raise
if cached:
cache_root = self.useFixture(fixtures.TempDir()).path
shutil.copytree(executor_root, cache_root, dirs_exist_ok=True)
else:
cache_root = '/doesnotexist'
work_root = self.useFixture(fixtures.TempDir()).path
params = {
"ansible_connection": connection,
"ansible_host": "testhost",
"ansible_port": 22,
"ansible_user": "zuul",
"cached_repos_root": cache_root,
"executor_work_root": executor_root,
"inventory_hostname": "testhost",
"zuul_workspace_root": work_root,
"zuul_projects": {
"example.com/org/test-project": {
"canonical_name": "example.com/org/test-project",
"checkout": "master",
"required": False,
"src_dir": "example.com/org/test-project"
},
},
"zuul_resources": {
"testhost": {
"context": "testcontext",
"namespace": "testnamespace",
"pod": "testpod",
},
},
}
# Verify the original behavior since we override it
self.assertEqual('git+ssh://zuul@testhost:22/project',
repo_sync.get_ssh_dest(params, 'project'))
self.assertEqual(
'"ext::kubectl --context testcontext -n testnamespace '
'exec -i testpod -- %S project"',
repo_sync.get_k8s_dest(params, 'project'))
def my_get_dest(args, dest):
return dest
def my_run(*args, **kw):
env = kw.get('env', {})
env.pop('GIT_ALLOW_PROTOCOL', None)
return run(*args, **kw)
# Override the destination to use a file instead
self.patch(repo_sync, 'get_ssh_dest', my_get_dest)
self.patch(repo_sync, 'get_k8s_dest', my_get_dest)
self.patch(repo_sync, 'run', my_run)
dest = os.path.join(work_root, 'example.com/org/test-project')
output = {}
ret = for_each_project(prep_one_project, params, output)
pprint.pprint(output)
self.assertTrue(ret)
project_output = output['example.com/org/test-project']
self.assertEqual(dest, project_output['dest'])
if cached:
self.assertEqual('cloned-from-cache',
project_output['initial_state'])
self.assertTrue("Cloning into bare" in project_output['clone'])
else:
self.assertEqual('git-init', project_output['initial_state'])
self.assertTrue("Initialized empty" in project_output['init'])
self.assertTrue(project_output['elapsed'] > 0)
output = {}
ret = for_each_project(sync_one_project, params, output)
pprint.pprint(output)
self.assertTrue(ret)
project_output = output['example.com/org/test-project']
self.assertEqual(dest, project_output['dest'])
self.assertEqual(1, project_output['attempts'])
self.assertEqual('', project_output['push'])
output = {}
ret = for_each_project(update_one_project, params, output)
pprint.pprint(output)
self.assertTrue(ret)
project_output = output['example.com/org/test-project']
self.assertEqual(dest, project_output['dest'])
head = run("git rev-parse HEAD", cwd=dest,
).stdout.decode('utf8').strip()
self.assertEqual(f'{head} init', project_output['HEAD'])
self.assertTrue(project_output['elapsed'] > 0)
def test_prepare_workspace_ssh_new(self):
self._test_prepare_workspace('local', cached=False)
def test_prepare_workspace_k8s_new(self):
self._test_prepare_workspace('kubectl', cached=False)
def test_prepare_workspace_ssh_cached(self):
self._test_prepare_workspace('local', cached=True)
def test_prepare_workspace_k8s_cached(self):
self._test_prepare_workspace('kubectl', cached=True)

View File

@ -0,0 +1,53 @@
# Copyright 2024 Acme Gating, LLC
#
# Licensed under the Apache License, Version 2.0 (the "License"); you may
# not use this file except in compliance with the License. You may obtain
# a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
# License for the specific language governing permissions and limitations
# under the License.
import shlex
import subprocess
from concurrent.futures import ThreadPoolExecutor
def run(cmd, shell=False, cwd=None, check=True, env=None):
if not shell:
cmd = shlex.split(cmd)
return subprocess.run(cmd, shell=shell, cwd=cwd, env=env,
stderr=subprocess.STDOUT,
stdout=subprocess.PIPE,
check=check)
def for_each_project(func, args, output):
# Run a function for each zuul project in a threadpool executor.
# An output dictionary specific to that project is passed to the
# function.
success = True
with ThreadPoolExecutor(max_workers=10) as executor:
futures = []
for project in args['zuul_projects'].values():
project_out = {}
output[project['canonical_name']] = project_out
f = executor.submit(
func, args, project, project_out)
futures.append((project_out, f))
for (project_out, f) in futures:
try:
f.result()
except Exception as e:
msg = str(e)
if hasattr(e, 'output'):
msg = f'{str(e)} : {e.output}'
else:
msg = str(e)
project_out['error'] = msg
success = False
return success

View File

@ -15,67 +15,27 @@
_zuul_projects: "{{ zuul.projects }}"
when: not prepare_workspace_sync_required_projects_only
# Do all the steps in a single shell script. This reduces the number of times
# ansible must loop over the list of projects which reduces the amount of
# task startup time we incur.
- name: Set initial repo states in workspace
shell: |
set -ex
if [ ! -d "{{ zuul_workspace_root }}/{{ zj_project.src_dir }}" ] ; then
if [ -d "{{ cached_repos_root }}/{{ zj_project.canonical_name }}" ] ; then
# We do a bare clone here first so that we skip creating a working
# copy that will be overwritten later anyway.
git clone --bare {{ cached_repos_root }}/{{ zj_project.canonical_name }} {{ zuul_workspace_root }}/{{ zj_project.src_dir }}/.git
else
git init {{ zuul_workspace_root }}/{{ zj_project.src_dir }}
fi
cd {{ zuul_workspace_root }}/{{ zj_project.src_dir }}
fi
cd {{ zuul_workspace_root }}/{{ zj_project.src_dir }}
git config --local --bool core.bare false
# Allow pushing to non-bare repo
git config --local receive.denyCurrentBranch ignore
# Allow deleting current branch
git config --local receive.denyDeleteCurrent ignore
git remote -v | grep origin && git remote rm origin || true
git remote add origin file:///dev/null
with_items: "{{ _zuul_projects.values() }}"
loop_control:
loop_var: zj_project
# We're using git in a shell script because it is faster and the module
# doesn't support features we need.
tags:
- skip_ansible_lint
repo_prep:
cached_repos_root: "{{ cached_repos_root }}"
executor_work_root: "{{ zuul.executor.work_root }}"
zuul_projects: "{{ _zuul_projects }}"
zuul_workspace_root: "{{ zuul_workspace_root }}"
- name: Include tasks to synchronize src repos to workspace directory
include_tasks: sync-project.yaml
with_dict: "{{ _zuul_projects }}"
loop_control:
loop_var: zj_project
- name: Synchronize src repos to workspace directory
delegate_to: localhost
repo_sync:
ansible_connection: "{{ ansible_connection }}"
ansible_host: "{{ ansible_host | ansible.utils.ipwrap }}"
ansible_port: "{{ ansible_port }}"
ansible_user: "{{ ansible_user }}"
executor_work_root: "{{ zuul.executor.work_root }}"
inventory_hostname: "{{ inventory_hostname }}"
zuul_projects: "{{ _zuul_projects }}"
zuul_resources: "{{ zuul.resources | default({}) }}"
zuul_workspace_root: "{{ zuul_workspace_root }}"
# Do this as a multi-line shell so that we can do the loop once
- name: Update remote repository state correctly
shell: |
set -eu
# Reset is needed because we pushed to a non-bare repo
git reset --hard
# Clean is needed because we pushed to a non-bare repo
git clean -xdf
# Undo the config setting we did above
git config --local --unset receive.denyCurrentBranch
git config --local --unset receive.denyDeleteCurrent
# checkout the branch matching the branch set up by the executor
git checkout {% if mirror_workspace_quiet %}--quiet{% endif %} {{ zj_project.value.checkout }}
# put out a status line with the current HEAD
echo "{{ zj_project.value.canonical_name }} checked out to:"
git log --pretty=oneline -1
args:
chdir: "{{ zuul_workspace_root }}/{{ zj_project.value.src_dir }}"
with_dict: "{{ _zuul_projects }}"
loop_control:
loop_var: zj_project
# ANSIBLE0006: Skip linting since it triggers on the "git" command,
# but we prefer the shell above
tags:
- skip_ansible_lint
- name: Update remote repository state
repo_update:
zuul_projects: "{{ _zuul_projects }}"
zuul_workspace_root: "{{ zuul_workspace_root }}"

View File

@ -1,23 +0,0 @@
- name: "Synchronize {{ zj_project.value.canonical_name }}"
command: |-
{% if ansible_connection == "kubectl" %}
git push {% if mirror_workspace_quiet %}--quiet{% endif %} --mirror "ext::kubectl --context {{ zuul.resources[inventory_hostname].context }} -n {{ zuul.resources[inventory_hostname].namespace }} exec -i {{ zuul.resources[inventory_hostname].pod }} -- %S {{ zuul_workspace_root }}/{{ zj_project.value.src_dir }}"
{% else %}
git push {% if mirror_workspace_quiet %}--quiet{% endif %} --mirror git+ssh://{{ ansible_user }}@{{ ansible_host | ipwrap }}:{{ ansible_port }}/{{ zuul_workspace_root }}/{{ zj_project.value.src_dir }}
{% endif %}
args:
chdir: "{{ zuul.executor.work_root }}/{{ zj_project.value.src_dir }}"
environment:
GIT_ALLOW_PROTOCOL: ext:ssh
delegate_to: localhost
# We occasionally see git pushes in the middle of this loop fail then
# subsequent pushes for other repos succeed. The entire loop ends up
# failing because one of the pushes failed. Mitigate this by retrying
# on failure.
register: git_push
until: git_push is success
retries: 3
# ANSIBLE0006: Skip linting since it triggers on the "git" command,
# but push is not supported by ansible git module.
tags:
- skip_ansible_lint