diff --git a/dev/tasks/linux-packages/travis.linux.yml b/ci/conda_env_crossbow.txt similarity index 50% rename from dev/tasks/linux-packages/travis.linux.yml rename to ci/conda_env_crossbow.txt index 7546587d123..347294650ca 100644 --- a/dev/tasks/linux-packages/travis.linux.yml +++ b/ci/conda_env_crossbow.txt @@ -15,45 +15,11 @@ # specific language governing permissions and limitations # under the License. -os: linux -dist: xenial -language: ruby - -services: - - docker - -# don't build twice -if: tag IS blank - -env: - global: - - TRAVIS_TAG={{ task.tag }} - - BUILD_REF={{ arrow.head }} - - ARROW_VERSION={{ arrow.version }} - -before_script: - - git clone --no-checkout {{ arrow.remote }} arrow - - git -C arrow fetch -t {{ arrow.remote }} {{ arrow.branch }} - - git -C arrow checkout FETCH_HEAD - -script: - - pushd arrow/dev/tasks/linux-packages - - rake version:update - - rake dist - - {{ build_command }} - -deploy: - provider: releases - api_key: $CROSSBOW_GITHUB_TOKEN - file_glob: true - file: - {% for extension in upload_extensions -%} - - "**/*{{ extension }}" - {% endfor -%} - skip_cleanup: true - on: - tags: true - -notifications: - email: - - {{ job.email }} +click +github3.py +jinja2 +jira +pygit2 +ruamel.yaml +setuptools_scm +toolz diff --git a/dev/tasks/conda-recipes/azure.linux.yml b/dev/tasks/conda-recipes/azure.linux.yml index e398dca82f2..ce58ba0f7b4 100755 --- a/dev/tasks/conda-recipes/azure.linux.yml +++ b/dev/tasks/conda-recipes/azure.linux.yml @@ -30,23 +30,24 @@ jobs: displayName: Run docker build workingDirectory: arrow/dev/tasks/conda-recipes - # Using github release tries to find a common ancestor between the currently - # pushed tag and the latest tag of the github repository (don't know why). - # The tag upload took 43 minutes because of this scan, so use an alternative - # upload script. - - task: UsePythonVersion@0 + # Using github release tries to find a common ancestor between the + # currently pushed tag and the latest tag of the github repository + # (don't know why). + # The tag upload took 43 minutes because of this scan, so use an + # alternative upload script. + - task: CondaEnvironment@1 inputs: - versionSpec: '3.6' - architecture: 'x64' - + packageSpecs: 'click github3.py jinja2 jira pygit2 ruamel.yaml setuptools_scm toolz' + installOptions: '-c conda-forge' + updateConda: false - script: | - python -m pip install github3-py click - python upload-assets.py \ + python arrow/dev/tasks/crossbow.py \ + --queue-path . \ + --queue-remote {{ queue.remote_url }} \ + upload-artifacts \ --sha {{ task.branch }} \ --tag {{ task.tag }} \ - --pattern "conda-recipes/build_artifacts/linux-64/*.tar.bz2" + --pattern "arrow/dev/tasks/conda-recipes/build_artifacts/linux-64/*.tar.bz2" env: - CROSSBOW_GITHUB_REPO: $(Build.Repository.Name) CROSSBOW_GITHUB_TOKEN: $(CROSSBOW_GITHUB_TOKEN) displayName: Upload packages as a GitHub release - workingDirectory: arrow/dev/tasks diff --git a/dev/tasks/conda-recipes/azure.osx.yml b/dev/tasks/conda-recipes/azure.osx.yml index 95f76f0fc53..5cd27944ba6 100755 --- a/dev/tasks/conda-recipes/azure.osx.yml +++ b/dev/tasks/conda-recipes/azure.osx.yml @@ -73,19 +73,24 @@ jobs: workingDirectory: arrow/dev/tasks/conda-recipes displayName: Build recipes - # Using github release tries to find a common ancestor between the currently - # pushed tag and the latest tag of the github repository (don't know why). - # The tag upload took 43 minutes because of this scan, so use an alternative - # upload script. + # Using github release tries to find a common ancestor between the + # currently pushed tag and the latest tag of the github repository + # (don't know why). + # The tag upload took 43 minutes because of this scan, so use an + # alternative upload script. + - task: CondaEnvironment@1 + inputs: + packageSpecs: 'click github3.py jinja2 jira pygit2 ruamel.yaml setuptools_scm toolz' + installOptions: '-c conda-forge' + updateConda: false - script: | - source activate base - conda install -c conda-forge -y github3.py click - python upload-assets.py \ + python arrow/dev/tasks/crossbow.py \ + --queue-path . \ + --queue-remote {{ queue.remote_url }} \ + upload-artifacts \ --sha {{ task.branch }} \ --tag {{ task.tag }} \ - --pattern "conda-recipes/build_artifacts/osx-64/*.tar.bz2" + --pattern "arrow/dev/tasks/conda-recipes/build_artifacts/osx-64/*.tar.bz2" env: - CROSSBOW_GITHUB_REPO: $(Build.Repository.Name) CROSSBOW_GITHUB_TOKEN: $(CROSSBOW_GITHUB_TOKEN) displayName: Upload packages as a GitHub release - workingDirectory: arrow/dev/tasks diff --git a/dev/tasks/conda-recipes/azure.win.yml b/dev/tasks/conda-recipes/azure.win.yml index 69029088f75..366f3edb648 100755 --- a/dev/tasks/conda-recipes/azure.win.yml +++ b/dev/tasks/conda-recipes/azure.win.yml @@ -88,22 +88,24 @@ jobs: PYTHONUNBUFFERED: 1 condition: not(contains(variables['CONFIG'], 'vs2008')) - # Using github release tries to find a common ancestor between the currently - # pushed tag and the latest tag of the github repository (don't know why). - # The tag upload took 43 minutes because of this scan, so use an alternative upload script. - - task: UsePythonVersion@0 + # Using github release tries to find a common ancestor between the + # currently pushed tag and the latest tag of the github repository + # (don't know why). + # The tag upload took 43 minutes because of this scan, so use an + # alternative upload script. + - task: CondaEnvironment@1 inputs: - versionSpec: '3.6' - architecture: 'x64' - + packageSpecs: 'click github3.py jinja2 jira pygit2 ruamel.yaml setuptools_scm toolz' + installOptions: '-c conda-forge' + updateConda: false - script: | - python -m pip install github3-py click - python upload-assets.py \ + python arrow/dev/tasks/crossbow.py \ + --queue-path . \ + --queue-remote {{ queue.remote_url }} \ + upload-artifacts \ --sha {{ task.branch }} \ --tag {{ task.tag }} \ - --pattern "conda-recipes/build_artifacts/*.tar.bz2" + --pattern "arrow/dev/tasks/conda-recipes/build_artifacts/win-64/*.tar.bz2" env: - CROSSBOW_GITHUB_REPO: $(Build.Repository.Name) CROSSBOW_GITHUB_TOKEN: $(CROSSBOW_GITHUB_TOKEN) displayName: Upload packages as a GitHub release - workingDirectory: arrow/dev/tasks diff --git a/dev/tasks/crossbow.py b/dev/tasks/crossbow.py index e737f977ead..4e4dabb4d32 100755 --- a/dev/tasks/crossbow.py +++ b/dev/tasks/crossbow.py @@ -19,8 +19,9 @@ import os import re +import glob import time -import hashlib +import mimetypes from io import StringIO from pathlib import Path from textwrap import dedent @@ -28,7 +29,6 @@ from collections import namedtuple import click -import gnupg import toolz import pygit2 import github3 @@ -244,11 +244,13 @@ class Repo: require_https : boolean, default False Raise exception for SSH origin URLs """ - def __init__(self, path, github_token=None, require_https=False): + def __init__(self, path, github_token=None, remote_url=None, + require_https=False): self.path = Path(path) self.repo = pygit2.Repository(str(self.path)) self.github_token = github_token self.require_https = require_https + self._remote_url = remote_url self._github_repo = None # set by as_github_repo() self._updated_refs = [] @@ -294,17 +296,18 @@ def head(self): @property def branch(self): """Currently checked out branch""" - return self.repo.branches[self.repo.head.shorthand] + try: + return self.repo.branches[self.repo.head.shorthand] + except KeyError: + return None # detached @property def remote(self): """Currently checked out branch's remote counterpart""" - if self.branch.upstream is None: - raise RuntimeError('Cannot determine git remote to push to, try ' - 'to push the branch first to have a remote ' - 'tracking counterpart.') - else: + try: return self.repo.remotes[self.branch.upstream.remote_name] + except (AttributeError, KeyError): + return None # cannot detect @property def remote_url(self): @@ -313,7 +316,10 @@ def remote_url(self): If an SSH github url is set, it will be replaced by the https equivalent usable with Github OAuth token. """ - return _git_ssh_to_https(self.remote.url) + try: + return self._remote_url or _git_ssh_to_https(self.remote.url) + except AttributeError: + return None @property def user_name(self): @@ -389,7 +395,8 @@ def file_contents(self, commit_id, file): return blob.data def _parse_github_user_repo(self): - m = re.match(r'.*\/([^\/]+)\/([^\/\.]+)(\.git)?$', self.remote_url) + url = self._remote_url if self._remote_url else self.remote_url + m = re.match(r'.*\/([^\/]+)\/([^\/\.]+)(\.git)?$', url) user, repo = m.group(1), m.group(2) return user, repo @@ -401,53 +408,7 @@ def as_github_repo(self): return gh.repository(username, reponame) return self._github_repo - -CombinedStatus = namedtuple('CombinedStatus', ('state', 'total_count')) - - -class Queue(Repo): - - def _next_job_id(self, prefix): - """Auto increments the branch's identifier based on the prefix""" - pattern = re.compile(r'[\w\/-]*{}-(\d+)'.format(prefix)) - matches = list(filter(None, map(pattern.match, self.repo.branches))) - if matches: - latest = max(int(m.group(1)) for m in matches) - else: - latest = 0 - return '{}-{}'.format(prefix, latest + 1) - - def get(self, job_name): - branch_name = 'origin/{}'.format(job_name) - branch = self.repo.branches[branch_name] - content = self.file_contents(branch.target, 'job.yml') - buffer = StringIO(content.decode('utf-8')) - return yaml.load(buffer) - - def put(self, job, prefix='build'): - if not isinstance(job, Job): - raise ValueError('`job` must be an instance of Job') - if job.branch is not None: - raise ValueError('`job.branch` is automatically generated, thus ' - 'it must be blank') - - # auto increment and set next job id, e.g. build-85 - job.branch = self._next_job_id(prefix) - - # create tasks' branches - for task_name, task in job.tasks.items(): - # adding CI's name to the end of the branch in order to use skip - # patterns on travis and circleci - task.branch = '{}-{}-{}'.format(job.branch, task.ci, task_name) - files = task.render_files(job=job, arrow=job.target) - branch = self.create_branch(task.branch, files=files) - self.create_tag(task.tag, branch.target) - task.commit = str(branch.target) - - # create job's branch with its description - return self.create_branch(job.branch, files=job.render_files()) - - def combined_status(self, task): + def github_commit_status(self, commit): """Combine the results from status and checks API to a single state. Azure pipelines uses checks API which doesn't provide a combined @@ -470,8 +431,8 @@ def combined_status(self, task): Parameters ---------- - task : Task - Task to query the combined status for. + commit : str + Commit to query the combined status for. Returns ------- @@ -481,7 +442,7 @@ def combined_status(self, task): ) """ repo = self.as_github_repo() - commit = repo.commit(task.commit) + commit = repo.commit(commit) states = [] for status in commit.status().statuses: @@ -510,48 +471,106 @@ def combined_status(self, task): return CombinedStatus(state=combined_state, total_count=len(states)) - def github_statuses(self, job): - return toolz.valmap(self.combined_status, job.tasks) - - def github_assets(self, task): + def github_release_assets(self, tag): repo = self.as_github_repo() try: - release = repo.release_from_tag(task.tag) + release = repo.release_from_tag(tag) except github3.exceptions.NotFoundError: return {} + else: + return {a.name: a for a in release.assets()} - assets = {a.name: a for a in release.assets()} + def github_overwrite_release_assets(self, tag_name, target_commitish, + patterns): + repo = self.as_github_repo() + if not tag_name: + raise ValueError('Empty tag name') + if not target_commitish: + raise ValueError('Empty target commit for the release tag') - artifacts = {} - for artifact in task.artifacts: - # artifact can be a regex pattern - pattern = re.compile(artifact) - matches = list(filter(None, map(pattern.match, assets.keys()))) - num_matches = len(matches) + # remove the whole release if it already exists + try: + release = repo.release_from_tag(tag_name) + except github3.exceptions.NotFoundError: + pass + else: + release.delete() - # validate artifact pattern matches single asset - if num_matches > 1: - raise ValueError( - 'Only a single asset should match pattern `{}`, there are ' - 'multiple ones: {}'.format(', '.join(matches)) - ) - elif num_matches == 1: - artifacts[artifact] = assets[matches[0].group(0)] + release = repo.create_release(tag_name, target_commitish) + default_mime = 'application/octet-stream' - return artifacts + for pattern in patterns: + for path in glob.glob(pattern, recursive=True): + name = os.path.basename(path) + mime = mimetypes.guess_type(name)[0] or default_mime - def upload_assets(self, job, files, content_type): - repo = self.as_github_repo() - release = repo.release_from_tag(job.branch) - assets = {a.name: a for a in release.assets()} + # TODO(kszucs): use logging + click.echo('Uploading asset `{}`...'.format(name)) + with open(path, 'rb') as fp: + release.upload_asset(name=name, asset=fp, + content_type=mime) - for path in files: - if path.name in assets: - # remove already uploaded asset - assets[path.name].delete() - with path.open('rb') as fp: - release.upload_asset(name=path.name, asset=fp, - content_type=content_type) + +CombinedStatus = namedtuple('CombinedStatus', ('state', 'total_count')) + + +class Queue(Repo): + + def _next_job_id(self, prefix): + """Auto increments the branch's identifier based on the prefix""" + pattern = re.compile(r'[\w\/-]*{}-(\d+)'.format(prefix)) + matches = list(filter(None, map(pattern.match, self.repo.branches))) + if matches: + latest = max(int(m.group(1)) for m in matches) + else: + latest = 0 + return '{}-{}'.format(prefix, latest + 1) + + def get(self, job_name): + branch_name = 'origin/{}'.format(job_name) + branch = self.repo.branches[branch_name] + content = self.file_contents(branch.target, 'job.yml') + buffer = StringIO(content.decode('utf-8')) + job = yaml.load(buffer) + job.queue = self + return job + + def put(self, job, prefix='build'): + if not isinstance(job, Job): + raise ValueError('`job` must be an instance of Job') + if job.branch is not None: + raise ValueError('`job.branch` is automatically generated, thus ' + 'it must be blank') + + if job.target.remote is None: + raise RuntimeError( + 'Cannot determine git remote for the Arrow repository to ' + 'clone or push to, try to push the branch first to have a ' + 'remote tracking counterpart.' + ) + if job.target.branch is None: + raise RuntimeError( + 'Cannot determine the current branch of the Arrow repository ' + 'to clone or push to, perhaps it is in detached HEAD state. ' + 'Please checkout a branch.' + ) + + # auto increment and set next job id, e.g. build-85 + job._queue = self + job.branch = self._next_job_id(prefix) + + # create tasks' branches + for task_name, task in job.tasks.items(): + # adding CI's name to the end of the branch in order to use skip + # patterns on travis and circleci + task.branch = '{}-{}-{}'.format(job.branch, task.ci, task_name) + files = task.render_files(job=job, arrow=job.target, queue=self) + branch = self.create_branch(task.branch, files=files) + self.create_tag(task.tag, branch.target) + task.commit = str(branch.target) + + # create job's branch with its description + return self.create_branch(job.branch, files=job.render_files()) def get_version(root, **kwargs): @@ -564,7 +583,16 @@ def get_version(root, **kwargs): return parse_git_version(root, **kwargs) -class Target: +class Serializable: + + @classmethod + def to_yaml(cls, representer, data): + tag = '!{}'.format(cls.__name__) + dct = {k: v for k, v in data.__dict__.items() if not k.startswith('_')} + return representer.represent_mapping(tag, dct) + + +class Target(Serializable): """Describes target repository and revision the builds run against This serializable data container holding information about arrow's @@ -604,7 +632,7 @@ def from_repo(cls, repo, head=None, branch=None, remote=None, version=None, version=version) -class Task: +class Task(Serializable): """Describes a build task and metadata required to render CI templates A task is represented as a single git commit and branch containing jinja2 @@ -624,7 +652,8 @@ def __init__(self, platform, ci, template, artifacts=None, params=None): self.artifacts = artifacts or [] self.params = params or {} self.branch = None # filled after adding to a queue - self.commit = None + self.commit = None # filled after adding to a queue + self._queue = None # set by the queue object after put or get def render_files(self, **extra_params): path = CWD / self.template @@ -647,8 +676,33 @@ def filename(self): } return config_files[self.ci] + def status(self): + return self._queue.github_commit_status(self.commit) -class Job: + def assets(self): + assets = self._queue.github_release_assets(self.tag) + + # validate the artifacts + artifacts = {} + for artifact in self.artifacts: + # artifact can be a regex pattern + pattern = re.compile(artifact) + matches = list(filter(None, map(pattern.match, assets.keys()))) + num_matches = len(matches) + + # validate artifact pattern matches single asset + if num_matches > 1: + raise ValueError( + 'Only a single asset should match pattern `{}`, there are ' + 'multiple ones: {}'.format(', '.join(matches)) + ) + elif num_matches == 1: + artifacts[artifact] = assets[matches[0].group(0)] + + return artifacts + + +class Job(Serializable): """Describes multiple tasks against a single target repository""" def __init__(self, target, tasks): @@ -661,6 +715,7 @@ def __init__(self, target, tasks): self.target = target self.tasks = tasks self.branch = None # filled after adding to a queue + self._queue = None # set by the queue object after put or get def render_files(self): with StringIO() as buf: @@ -668,10 +723,61 @@ def render_files(self): content = buf.getvalue() return toolz.merge(_default_tree, {'job.yml': content}) + @property + def queue(self): + assert isinstance(self._queue, Queue) + return self._queue + + @queue.setter + def queue(self, queue): + assert isinstance(queue, Queue) + self._queue = queue + for task in self.tasks.values(): + task._queue = queue + @property def email(self): return os.environ.get('CROSSBOW_EMAIL', self.target.email) + @classmethod + def from_config(cls, config, target, task_whitelist=None, + group_whitelist=None): + config_groups = dict(config['groups']) + config_tasks = dict(config['tasks']) + valid_groups = set(config_groups.keys()) + valid_tasks = set(config_tasks.keys()) + group_whitelist = list(group_whitelist or []) + task_whitelist = list(task_whitelist or []) + + requested_groups = set(group_whitelist) + invalid_groups = requested_groups - valid_groups + if invalid_groups: + msg = 'Invalid group(s) {!r}. Must be one of {!r}'.format( + invalid_groups, valid_groups + ) + raise click.ClickException(msg) + + requested_tasks = [list(config_groups[name]) + for name in group_whitelist] + requested_tasks = set(sum(requested_tasks, task_whitelist)) + invalid_tasks = requested_tasks - valid_tasks + if invalid_tasks: + msg = 'Invalid task(s) {!r}. Must be one of {!r}'.format( + invalid_tasks, valid_tasks + ) + raise click.ClickException(msg) + + tasks = {} + versions = {'version': target.version, + 'no_rc_version': target.no_rc_version} + for task_name in requested_tasks: + task = config_tasks[task_name] + artifacts = task.pop('artifacts', None) or [] # because of yaml + artifacts = [fn.format(**versions) for fn in artifacts] + tasks[task_name] = Task(artifacts=artifacts, **task) + + return cls(target=target, tasks=tasks) + # configure yaml serializer yaml = YAML() @@ -704,8 +810,10 @@ def email(self): type=click.Path(exists=True), default=DEFAULT_QUEUE_PATH, help='The repository path used for scheduling the tasks. ' 'Defaults to crossbow directory placed next to arrow') +@click.option('--queue-remote', '-qr', default=None, + help='Force to use this remote URL for the Queue repository') @click.pass_context -def crossbow(ctx, github_token, arrow_path, queue_path): +def crossbow(ctx, github_token, arrow_path, queue_path, queue_remote): if github_token is None: raise click.ClickException( 'Could not determine GitHub token. Please set the ' @@ -714,8 +822,8 @@ def crossbow(ctx, github_token, arrow_path, queue_path): ) ctx.obj['arrow'] = Repo(arrow_path) - ctx.obj['queue'] = Queue(queue_path, github_token=github_token, - require_https=True) + ctx.obj['queue'] = Queue(queue_path, remote_url=queue_remote, + github_token=github_token, require_https=True) @crossbow.command() @@ -750,35 +858,9 @@ def changelog(ctx, changelog_path, arrow_version, is_website, jira_username, 'changes') -def load_tasks_from_config(config_path, task_names, group_names): - with Path(config_path).open() as fp: - config = yaml.load(fp) - - groups = config['groups'] - tasks = config['tasks'] - - valid_groups = set(groups.keys()) - valid_tasks = set(tasks.keys()) - - requested_groups = set(group_names) - invalid_groups = requested_groups - valid_groups - if invalid_groups: - raise click.ClickException('Invalid group(s) {!r}. Must be one of {!r}' - .format(invalid_groups, valid_groups)) - - requested_tasks = [list(groups[name]) for name in group_names] - requested_tasks = set(sum(requested_tasks, list(task_names))) - invalid_tasks = requested_tasks - valid_tasks - if invalid_tasks: - raise click.ClickException('Invalid task(s) {!r}. Must be one of {!r}' - .format(invalid_tasks, valid_tasks)) - - return {t: config['tasks'][t] for t in requested_tasks} - - @crossbow.command() -@click.argument('task', nargs=-1, required=False) -@click.option('--group', '-g', multiple=True, +@click.argument('tasks', nargs=-1, required=False) +@click.option('--group', '-g', 'groups', multiple=True, help='Submit task groups as defined in task.yml') @click.option('--job-prefix', default='build', help='Arbitrary prefix for branch names, e.g. nightly') @@ -804,10 +886,14 @@ def load_tasks_from_config(config_path, task_names, group_names): type=click.File('w', encoding='utf8'), default='-', help='Capture output result into file.') @click.pass_context -def submit(ctx, task, group, job_prefix, config_path, arrow_version, +def submit(ctx, tasks, groups, job_prefix, config_path, arrow_version, arrow_remote, arrow_branch, arrow_sha, dry_run, output): queue, arrow = ctx.obj['queue'], ctx.obj['arrow'] + # load available tasks configuration and groups from yaml + with Path(config_path).open() as fp: + config = yaml.load(fp) + # Override the detected repo url / remote, branch and sha - this aims to # make release procedure a bit simpler. # Note, that the target resivion's crossbow templates must be @@ -817,22 +903,10 @@ def submit(ctx, task, group, job_prefix, config_path, arrow_version, # which will be reduced to a single command in the future. target = Target.from_repo(arrow, remote=arrow_remote, branch=arrow_branch, head=arrow_sha, version=arrow_version) - params = { - 'version': target.version, - 'no_rc_version': target.no_rc_version, - } - - # task and group variables are lists, containing multiple values - tasks = {} - task_configs = load_tasks_from_config(config_path, task, group) - for name, task in task_configs.items(): - # replace version number and create task instance from configuration - artifacts = task.pop('artifacts', None) or [] # because of yaml - artifacts = [fn.format(**params) for fn in artifacts] - tasks[name] = Task(artifacts=artifacts, **task) - - # create job instance, doesn't mutate git data yet - job = Job(target=target, tasks=tasks) + + # instantiate the job object + job = Job.from_config(config, target=target, task_whitelist=tasks, + group_whitelist=groups) if dry_run: yaml.dump(job, output) @@ -860,11 +934,10 @@ def status(ctx, job_name, output): click.echo('-' * len(header), file=output) job = queue.get(job_name) - statuses = queue.github_statuses(job) for task_name, task in sorted(job.tasks.items()): - status = statuses[task_name] - assets = queue.github_assets(task) + status = task.status() + assets = task.assets() uploaded = 'uploaded {} / {}'.format( sum(a in assets for a in task.artifacts), @@ -888,48 +961,14 @@ def status(ctx, job_name, output): file=output) -def hashbytes(bytes, algoname): - """Hash `bytes` using the algorithm named `algoname`. - - Parameters - ---------- - bytes : bytes - The bytes to hash - algoname : str - The name of class in the hashlib standard library module - - Returns - ------- - str - Hexadecimal digest of `bytes` hashed using `algoname` - """ - algo = getattr(hashlib, algoname)() - algo.update(bytes) - result = algo.hexdigest() - return result - - @crossbow.command() @click.argument('job-name', required=True) -@click.option('-g', '--gpg-homedir', default=None, - type=click.Path(exists=True, file_okay=False, dir_okay=True), - help=('Full pathname to directory containing the public and ' - 'private keyrings. Default is whatever GnuPG defaults to')) @click.option('-t', '--target-dir', default=DEFAULT_ARROW_PATH / 'packages', type=click.Path(file_okay=False, dir_okay=True), help='Directory to download the build artifacts') -@click.option('-a', '--algorithm', - default=['sha256', 'sha512'], - show_default=True, - type=click.Choice(sorted(hashlib.algorithms_guaranteed)), - multiple=True, - help=('Algorithm(s) used to generate checksums. Pass multiple ' - 'algorithms by passing -a/--algorithm multiple times')) @click.pass_context -def sign(ctx, job_name, gpg_homedir, target_dir, algorithm): +def download_artifacts(ctx, job_name, target_dir): """Download and sign build artifacts from github releases""" - gpg = gnupg.GPG(gnupghome=gpg_homedir) - # fetch the queue repository queue = ctx.obj['queue'] queue.fetch() @@ -947,11 +986,11 @@ def sign(ctx, job_name, gpg_homedir, target_dir, algorithm): ntasks = len(task_items) for i, (task_name, task) in enumerate(task_items, start=1): - assets = queue.github_assets(task) + assets = task.assets() artifact_dir = target_dir / task_name artifact_dir.mkdir(exist_ok=True) - basemsg = 'Downloading and signing assets for task {}'.format( + basemsg = 'Downloading assets for task {}'.format( click.style(task_name, bold=True) ) click.echo( @@ -978,30 +1017,22 @@ def sign(ctx, job_name, gpg_homedir, target_dir, algorithm): artifact_path = artifact_dir / asset.name asset.download(artifact_path) - # sign the artifact - signature_path = Path(str(artifact_path) + '.asc') - with artifact_path.open('rb') as fp: - gpg.sign_file(fp, detach=True, clearsign=False, - binary=False, - output=str(signature_path)) - - # compute checksums for the artifact - artifact_bytes = artifact_path.read_bytes() - for algo in algorithm: - suffix = '.{}'.format(algo) - checksum_path = Path(str(artifact_path) + suffix) - checksum = '{} {}'.format( - hashbytes(artifact_bytes, algo), artifact_path.name - ) - checksum_path.write_text(checksum) - msg = click.style( - '[{:>13}]'.format('{} HASHED'.format(algo)), - fg='blue' - ) - click.echo(tpl.format(msg, checksum_path.name)) - - msg = click.style('[{:>13}]'.format('SIGNED'), fg=COLORS['ok']) - click.echo(tpl.format(msg, str(signature_path.name))) + msg = click.style('[{:>13}]'.format('DOWNLOADED'), + fg=COLORS['ok']) + click.echo(tpl.format(msg, str(asset.name))) + + +@crossbow.command() +@click.option('--sha', required=True, help='Target committish') +@click.option('--tag', required=True, help='Target tag') +@click.option('--pattern', '-p', 'patterns', required=True, multiple=True, + help='File pattern to upload as assets') +@click.pass_context +def upload_artifacts(ctx, tag, sha, patterns): + queue = ctx.obj['queue'] + queue.github_overwrite_release_assets( + tag_name=tag, target_commitish=sha, patterns=patterns + ) if __name__ == '__main__': diff --git a/dev/tasks/linux-packages/azure.linux.arm64.yml b/dev/tasks/linux-packages/azure.linux.arm64.yml new file mode 100644 index 00000000000..07e96efad16 --- /dev/null +++ b/dev/tasks/linux-packages/azure.linux.arm64.yml @@ -0,0 +1,82 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. + +jobs: +- job: linux + pool: + vmImage: ubuntu-latest + timeoutInMinutes: 360 + steps: + # for building the linux packages + - task: UseRubyVersion@0 + inputs: + addToPath: true + + - script: | + git clone --no-checkout {{ arrow.remote }} arrow + git -C arrow fetch -t {{ arrow.remote }} {{ arrow.branch }} + git -C arrow checkout FETCH_HEAD + git -C arrow submodule update --init --recursive + displayName: Clone arrow + + # We can remove this with binfmt-support 2.1.7 or later and + # qemu-user-static 2.12 or later. It requires Debian buster or later, + # or Ubuntu 18.10 or later. + - script: | + sudo apt install -y qemu-user-static unar + wget http://archive.ubuntu.com/ubuntu/pool/universe/q/qemu/qemu-user-static_3.1+dfsg-2ubuntu3.4_amd64.deb + unar *.deb + rm *.deb + pushd qemu-user-static* + unar data.* + for arm64_image in ../arrow/dev/tasks/linux-packages/apt/*-arm64/; do + cp data/usr/bin/qemu-aarch64-static ${arm64_image} + done + popd + displayName: Prepare qemu-user-static + + - script: | + pushd arrow/dev/tasks/linux-packages + rake version:update + rake dist + {{ build_command }} + popd + displayName: Build + + # Using github release tries to find a common ancestor between the + # currently pushed tag and the latest tag of the github repository + # (don't know why). + # The tag upload took 43 minutes because of this scan, so use an + # alternative upload script. + - task: CondaEnvironment@1 + inputs: + packageSpecs: 'click github3.py jinja2 jira pygit2 ruamel.yaml setuptools_scm toolz' + installOptions: '-c conda-forge' + updateConda: false + - script: | + python arrow/dev/tasks/crossbow.py \ + --queue-path . \ + --queue-remote {{ queue.remote_url }} \ + upload-artifacts \ + {%- for extension in upload_extensions %} + --pattern "**/*{{ extension }}" \ + {%- endfor %} + --sha {{ task.branch }} \ + --tag {{ task.tag }} + env: + CROSSBOW_GITHUB_TOKEN: $(CROSSBOW_GITHUB_TOKEN) + displayName: Upload packages as a GitHub release diff --git a/dev/tasks/linux-packages/azure.linux.yml b/dev/tasks/linux-packages/azure.linux.yml new file mode 100644 index 00000000000..44e9b204eab --- /dev/null +++ b/dev/tasks/linux-packages/azure.linux.yml @@ -0,0 +1,66 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. + +jobs: +- job: linux + pool: + vmImage: ubuntu-latest + timeoutInMinutes: 360 + steps: + # for building the linux packages + - task: UseRubyVersion@0 + inputs: + addToPath: true + + - script: | + git clone --no-checkout {{ arrow.remote }} arrow + git -C arrow fetch -t {{ arrow.remote }} {{ arrow.branch }} + git -C arrow checkout FETCH_HEAD + git -C arrow submodule update --init --recursive + displayName: Clone arrow + + - script: | + pushd arrow/dev/tasks/linux-packages + rake version:update + rake dist + {{ build_command }} + popd + displayName: Build + + # Using github release tries to find a common ancestor between the + # currently pushed tag and the latest tag of the github repository + # (don't know why). + # The tag upload took 43 minutes because of this scan, so use an + # alternative upload script. + - task: CondaEnvironment@1 + inputs: + packageSpecs: 'click github3.py jinja2 jira pygit2 ruamel.yaml setuptools_scm toolz' + installOptions: '-c conda-forge' + updateConda: false + - script: | + python arrow/dev/tasks/crossbow.py \ + --queue-path . \ + --queue-remote {{ queue.remote_url }} \ + upload-artifacts \ + {%- for extension in upload_extensions %} + --pattern "**/*{{ extension }}" \ + {%- endfor %} + --sha {{ task.branch }} \ + --tag {{ task.tag }} + env: + CROSSBOW_GITHUB_TOKEN: $(CROSSBOW_GITHUB_TOKEN) + displayName: Upload packages as a GitHub release diff --git a/dev/tasks/linux-packages/package-task.rb b/dev/tasks/linux-packages/package-task.rb index 2db7bdc2750..113f4de15cf 100644 --- a/dev/tasks/linux-packages/package-task.rb +++ b/dev/tasks/linux-packages/package-task.rb @@ -294,11 +294,23 @@ def package_changelog_message end def packager_name - ENV["DEBFULLNAME"] || ENV["NAME"] || `git config --get user.name`.chomp + ENV["DEBFULLNAME"] || ENV["NAME"] || guess_packager_name_from_git + end + + def guess_packager_name_from_git + name = `git config --get user.name`.chomp + return name unless name.empty? + `git log -n 1 --format=%aN`.chomp end def packager_email - ENV["DEBEMAIL"] || ENV["EMAIL"] || `git config --get user.email`.chomp + ENV["DEBEMAIL"] || ENV["EMAIL"] || guess_packager_email_from_git + end + + def guess_packager_email_from_git + email = `git config --get user.email`.chomp + return email unless email.empty? + `git log -n 1 --format=%aE`.chomp end def update_content(path) @@ -342,5 +354,4 @@ def update_spec content.rstrip end end - end diff --git a/dev/tasks/linux-packages/travis.linux.arm64.yml b/dev/tasks/linux-packages/travis.linux.arm64.yml deleted file mode 100644 index 7bb5ad0496f..00000000000 --- a/dev/tasks/linux-packages/travis.linux.arm64.yml +++ /dev/null @@ -1,75 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you 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. - -os: linux -dist: xenial -language: ruby - -# don't build twice -if: tag IS blank - -services: - - docker - -addons: - apt: - packages: - - qemu-user-static - - unar - -env: - global: - - TRAVIS_TAG={{ task.tag }} - - BUILD_REF={{ arrow.head }} - - ARROW_VERSION={{ arrow.version }} - -before_script: - - git clone --no-checkout {{ arrow.remote }} arrow - - git -C arrow fetch -t {{ arrow.remote }} {{ arrow.branch }} - - git -C arrow checkout FETCH_HEAD - -script: - - pushd arrow/dev/tasks/linux-packages - - rake version:update - - rake dist - # We can remove this with binfmt-support 2.1.7 or later and - # qemu-user-static 2.12 or later. It requires Debian buster or later, - # or Ubuntu 18.10 or later. - - wget http://archive.ubuntu.com/ubuntu/pool/universe/q/qemu/qemu-user-static_3.1+dfsg-2ubuntu5_amd64.deb - - unar *.deb - - rm *.deb - - pushd qemu-user-static* - - unar data.* - - cp data/usr/bin/qemu-aarch64-static ../apt/debian-stretch-arm64/ - - popd - - {{ build_command }} - -deploy: - provider: releases - api_key: $CROSSBOW_GITHUB_TOKEN - file_glob: true - file: - {% for extension in upload_extensions -%} - - "**/*{{ extension }}" - {% endfor -%} - skip_cleanup: true - on: - tags: true - -notifications: - email: - - {{ job.email }} diff --git a/dev/tasks/tasks.yml b/dev/tasks/tasks.yml index e72d5676339..9bc7a0d9f4a 100644 --- a/dev/tasks/tasks.yml +++ b/dev/tasks/tasks.yml @@ -47,8 +47,7 @@ groups: - wheel-win-cp37m linux: - debian-stretch - # We can't finish this on Travis CI for now. - # - debian-stretch-arm64 + - debian-stretch-arm64 - debian-buster - ubuntu-xenial - ubuntu-bionic @@ -366,9 +365,9 @@ tasks: ############################## Linux PKGS #################################### debian-stretch: - ci: travis + ci: azure platform: linux - template: linux-packages/travis.linux.yml + template: linux-packages/azure.linux.yml params: build_command: rake apt:build APT_TARGETS=debian-stretch upload_extensions: @@ -431,11 +430,10 @@ tasks: - plasma-store-server-dbgsym_{no_rc_version}-1_amd64.deb - plasma-store-server_{no_rc_version}-1_amd64.deb - # Don't use this for now. We can't complete this task in 50min on Travis CI. debian-stretch-arm64: - ci: travis + ci: azure platform: linux - template: linux-packages/travis.linux.arm64.yml + template: linux-packages/azure.linux.arm64.yml params: build_command: rake apt:build APT_TARGETS=debian-stretch-arm64 upload_extensions: @@ -474,9 +472,9 @@ tasks: - libparquet15_{no_rc_version}-1_amd64.deb debian-buster: - ci: travis + ci: azure platform: linux - template: linux-packages/travis.linux.yml + template: linux-packages/azure.linux.yml params: build_command: rake apt:build APT_TARGETS=debian-buster upload_extensions: @@ -540,9 +538,9 @@ tasks: - plasma-store-server_{no_rc_version}-1_amd64.deb ubuntu-xenial: - ci: travis + ci: azure platform: linux - template: linux-packages/travis.linux.yml + template: linux-packages/azure.linux.yml params: build_command: rake apt:build APT_TARGETS=ubuntu-xenial upload_extensions: @@ -592,9 +590,9 @@ tasks: - plasma-store-server_{no_rc_version}-1_amd64.deb ubuntu-bionic: - ci: travis + ci: azure platform: linux - template: linux-packages/travis.linux.yml + template: linux-packages/azure.linux.yml params: build_command: rake apt:build APT_TARGETS=ubuntu-bionic upload_extensions: @@ -647,9 +645,9 @@ tasks: - plasma-store-server_{no_rc_version}-1_amd64.deb ubuntu-cosmic: - ci: travis + ci: azure platform: linux - template: linux-packages/travis.linux.yml + template: linux-packages/azure.linux.yml params: build_command: rake apt:build APT_TARGETS=ubuntu-cosmic upload_extensions: @@ -702,9 +700,9 @@ tasks: - plasma-store-server_{no_rc_version}-1_amd64.deb ubuntu-disco: - ci: travis + ci: azure platform: linux - template: linux-packages/travis.linux.yml + template: linux-packages/azure.linux.yml params: build_command: rake apt:build APT_TARGETS=ubuntu-disco upload_extensions: @@ -757,9 +755,9 @@ tasks: - plasma-store-server_{no_rc_version}-1_amd64.deb centos-6: - ci: travis + ci: azure platform: linux - template: linux-packages/travis.linux.yml + template: linux-packages/azure.linux.yml params: build_command: rake yum:build CENTOS_VERSIONS=6 upload_extensions: @@ -777,9 +775,9 @@ tasks: - plasma-libs-{no_rc_version}-1.el6.x86_64.rpm centos-7: - ci: travis + ci: azure platform: linux - template: linux-packages/travis.linux.yml + template: linux-packages/azure.linux.yml params: build_command: rake yum:build CENTOS_VERSIONS=7 upload_extensions: diff --git a/dev/tasks/upload-assets.py b/dev/tasks/upload-assets.py deleted file mode 100755 index 24356335991..00000000000 --- a/dev/tasks/upload-assets.py +++ /dev/null @@ -1,64 +0,0 @@ -#!/usr/bin/env python - -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you 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 mimetypes -from glob import glob - -import click -import github3 - - -# TODO(kszucs): move it to crossbow.py - - -@click.command() -@click.option('--sha', help='Target committish') -@click.option('--tag', help='Target tag') -@click.option('--pattern', help='File pattern') -def upload_assets(tag, sha, pattern): - token = os.environ['CROSSBOW_GITHUB_TOKEN'] - owner, repository = os.environ['CROSSBOW_GITHUB_REPO'].split('/') - - gh = github3.login(token=token) - repo = gh.repository(owner, repository) - click.echo('Selected repository: {}/{}'.format(owner, repository)) - - try: - release = repo.release_from_tag(tag) - except github3.exceptions.NotFoundError: - pass - else: - click.echo('Removing release `{}`'.format(release.tag_name)) - release.delete() - - click.echo('Creating release `{}`'.format(tag)) - release = repo.create_release(tag, sha) - - for path in glob(pattern): - name = os.path.basename(path) - mime = mimetypes.guess_type(name)[0] or 'application/octet-stream' - - click.echo('Uploading asset `{}`...'.format(name)) - with open(path, 'rb') as fp: - release.upload_asset(name=name, asset=fp, content_type=mime) - - -if __name__ == '__main__': - upload_assets()