Auto merge of #21776 - servo:taskcluster, r=Manishearth

Rewrite Taskcluster integration

This rewrites this repository’s integration with Taskcluster to have a decision task that runs an in-tree Python script to schedule a graph of other tasks with inter-dependencies, saved build artifacts, Docker images built from in-tree `Dockerfile`s, etc. This also adds support fail daily tasks, run automatically every 24 hours. See `etc/ci/taskcluster/README.md` for more details.

Example of a green build with web-platform-tests enabled: https://tools.taskcluster.net/groups/aDL0pUBkSmuhezT1OF80_g

For now this PR only enables one task, equivalent to Buildbot’s `linux-dev` job. (Running Tidy and other linting tools, compiling in debug mode, and running unit tests.) At this time Homu does **not** yet block PRs from merging on a successful result from Taskcluster, but I plan to do this soon after this PR lands.

<!-- Reviewable:start -->
---
This change is [<img src="https://reviewable.io/review_button.svg" height="34" align="absmiddle" alt="Reviewable"/>](https://reviewable.io/reviews/servo/servo/21776)
<!-- Reviewable:end -->
This commit is contained in:
bors-servo 2018-09-25 03:26:09 -04:00 committed by GitHub
commit 10cf425069
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
11 changed files with 926 additions and 66 deletions

View file

@ -1,37 +1,56 @@
version: 0
metadata:
name: servo
description: >-
A modern, high-performance browser engine designed for both application
and embedded use.
owner: '{{ event.head.user.email }}'
source: '{{ event.head.repo.url }}'
version: 1
policy:
# https://docs.taskcluster.net/docs/reference/integrations/taskcluster-github/docs/taskcluster-yml-v1#pull-requests
pullRequests: collaborators
tasks:
- provisionerId: '{{ taskcluster.docker.provisionerId }}'
workerType: 'servo-docker-worker'
extra:
github:
events:
- pull_request.opened
- pull_request.reopened
- pull_request.synchronize
payload:
maxRunTime: 7200
image: 'servobrowser/servo-linux-dev:servo-linux-build-deps-2017-06-30'
command:
- /bin/bash
- '--login'
- '-c'
- '-x'
- >-
git clone {{event.head.repo.url}} servo &&
cd servo &&
git config advice.detachedHead false &&
git checkout {{event.head.sha}} &&
etc/ci/taskcluster-test.sh
metadata:
name: linux-tests
description: Run Linux tests.
owner: '{{ event.head.user.email }}'
source: '{{ event.head.repo.url }}'
allowPullRequests: public
- $if: 'tasks_for == "github-push"'
then:
$if: 'event.ref in ["refs/heads/auto", "refs/heads/try"]'
then:
# NOTE: when updating this consider whether the daily hook needs similar changes:
# https://tools.taskcluster.net/hooks/project-servo/daily
taskGroupId: {$eval: as_slugid("decision_task")}
taskId: {$eval: as_slugid("decision_task")}
provisionerId: aws-provisioner-v1
workerType: servo-docker-worker
created: {$fromNow: ''}
deadline: {$fromNow: '1 day'}
metadata:
name: "Servo: GitHub push decision task"
description: ""
owner: &task_owner ${event.pusher.name}@users.noreply.github.com
source: &task_source ${event.compare}
scopes:
- "queue:scheduler-id:taskcluster-github"
# Granted to role "repo:github.com/servo/servo:branch:*"
- "queue:create-task:highest:aws-provisioner-v1/servo-*"
- "queue:route:index.project.servo.servo.*"
- "docker-worker:cache:cargo-*"
payload:
maxRunTime: {$eval: '20 * 60'}
# https://github.com/servo/taskcluster-bootstrap-docker-images#decision-task
image: "servobrowser/taskcluster-bootstrap:decision-task@sha256:28045b7ec0485ef363f8cb14f194008b47e9ede99f2ea40a1e945e921fce976e"
features:
taskclusterProxy: true
env:
GIT_URL: ${event.repository.clone_url}
GIT_REF: ${event.ref}
GIT_SHA: ${event.after}
TASK_FOR: ${tasks_for}
TASK_OWNER: *task_owner
TASK_SOURCE: *task_source
command:
- /bin/bash
- '--login'
- '-e'
- '-c'
- >-
git init repo &&
cd repo &&
git fetch --depth 1 "$GIT_URL" "$GIT_REF" &&
git reset --hard "$GIT_SHA" &&
python3 etc/ci/taskcluster/decision-task.py

View file

@ -1,30 +0,0 @@
#!/usr/bin/env bash
# This Source Code Form is subject to the terms of the Mozilla Public
# License, v. 2.0. If a copy of the MPL was not distributed with this
# file, You can obtain one at http://mozilla.org/MPL/2.0/.
set -o errexit
set -o nounset
set -o pipefail
set -x
curl https://sh.rustup.rs -sSf | sh -s -- --default-toolchain none -y
export PATH="${HOME}/.cargo/bin:${PATH}"
# Update this from the linux-dev builder in etc/ci/buildbot_steps.yml
export RUST_BACKTRACE=1
export RUSTFLAGS="-Dwarnings"
export CARGO_INCREMENTAL=0
export SCCACHE_IDLE_TIMEOUT=1200
./mach test-tidy --no-progress --all
./mach test-tidy --no-progress --self-test
env CC=gcc-5 CXX=g++-5 ./mach build --dev
env ./mach test-unit
env ./mach package --dev
env ./mach build --dev --no-default-features --features default-except-unstable
bash ./etc/ci/lockfile_changed.sh
bash ./etc/ci/check_no_panic.sh

View file

@ -0,0 +1,237 @@
# Testing Servo on Taskcluster
## Homu
When a pull request is reviewed and the appropriate command is given,
[Homu] creates a merge commit of `master` and the PRs branch, and pushes it to the `auto` branch.
One or more CI system (through their own means) get notified of this push by GitHub,
start testing the merge commit, and use the [GitHub Status API] to report results.
Through a [Webhook], Homu gets notified of changes to these statues.
If all of the required statuses are reported successful,
Homu pushes its merge commit to the `master` branch
and goes on to testing the next pull request in its queue.
[Homu]: https://github.com/servo/servo/wiki/Homu
[GitHub Status API]: https://developer.github.com/v3/repos/statuses/
[Webhook]: https://developer.github.com/webhooks/
## Taskcluster GitHub integration
Taskcluster is very flexible and not necessarily tied to GitHub,
but it does have an optional [GitHub integration service] that you can enable
on a repository [as a GitHub App].
When enabled, this service gets notified for every push, pull request, or GitHub release.
It then schedules some tasks based on reading [`.taskcluster.yml`] in the corresponding commit.
This file contains templates for creating one or more tasks,
but the logic it can support is fairly limited.
So a common pattern is to have it only run a single initial task called a *decision task*
that can have complex logic based on code and data in the repository
to build an arbitrary [task graph].
[GitHub integration service]: https://docs.taskcluster.net/docs/manual/using/github
[as a GitHub App]: https://github.com/apps/taskcluster
[`.taskcluster.yml`]: https://docs.taskcluster.net/docs/reference/integrations/taskcluster-github/docs/taskcluster-yml-v1
[task graph]: https://docs.taskcluster.net/docs/manual/using/task-graph
## Servos decision task
This repositorys [`.taskcluster.yml`][tc.yml] schedules a single task
that runs the Python 3 script [`etc/ci/taskcluster/decision-task.py`](decision-task.py).
It is called a *decision task* as it is responsible for deciding what other tasks to schedule.
The Docker image that runs the decision task
is hosted on Docker Hub at [`servobrowser/taskcluster-bootstrap`][hub].
It is built by [Docker Hub automated builds] based on a `Dockerfile`
in the [`taskcluster-bootstrap-docker-images`] GitHub repository.
Hopefully, this image does not need to be modified often
as it only needs to clone the repository and run Python.
[tc.yml]: ../../../.taskcluster.yml
[hub]: https://hub.docker.com/r/servobrowser/taskcluster-bootstrap/
[Docker Hub automated builds]: https://docs.docker.com/docker-hub/builds/
[`taskcluster-bootstrap-docker-images`]: https://github.com/servo/taskcluster-bootstrap-docker-images/
## In-tree Docker images
[Similar to Firefox][firefox], Servos decision task supports running other tasks
in Docker images built on-demand, based on `Dockerfile`s in the main repository.
Modifying a `Dockerfile` and relying on those new changes
can be done in the same pull request or commit.
To avoid rebuilding images on every pull request,
they are cached based on a hash of the source `Dockerfile`.
For now, to support this hashing, we make `Dockerfile`s be self-contained (with one exception).
Images are built without a [context],
so instructions like [`COPY`] cannot be used because there is nothing to copy from.
The exception is that the decision task adds support for a non-standard include directive:
when a `Dockerfile` first line is `% include` followed by a filename,
that line is replaced with the content of that file.
For example,
[`etc/ci/taskcluster/docker/build.dockerfile`](docker/build.dockerfile) starts like so:
```Dockerfile
% include base.dockerfile
RUN \
apt-get install -qy --no-install-recommends \
# […]
```
[firefox]: https://firefox-source-docs.mozilla.org/taskcluster/taskcluster/docker-images.html
[context]: https://docs.docker.com/engine/reference/commandline/build/#extended-description
[`COPY`]: https://docs.docker.com/engine/reference/builder/#copy
## Build artifacts
[web-platform-tests] (WPT) is large enough that running all of a it takes a long time.
So it supports *chunking*,
such as multiple chunks of the test suite can be run in parallel on different machines.
As of this writing,
Servos current Buildbot setup for this has each machine start by compiling its own copy of Servo.
On Taskcluster with a decision task,
we can have a single build task save its resulting binary executable as an [artifact],
together with multiple testing tasks that each depend on the build task
(wait until it successfully finishes before they can start)
and start by downloading the artifact that was saved earlier.
The logic for all this is in [`decision-task.py`](decision-task.py)
and can be modified in any pull request.
[web-platform-tests]: https://github.com/web-platform-tests/wpt
[artifact]: https://docs.taskcluster.net/docs/manual/using/artifacts
## Log artifacts
Taskcluster automatically save the `stdio` output of a task as an artifact,
and as special support for seeing and streaming that output while the task is still running.
Servos decision task additionally looks for `*.log` arguments to its taskss commands,
assumes they instruct a program to create a log file with that name,
and saves those log files as individual artifacts.
For example, WPT tasks have a `filtered-wpt-errorsummary.log` artifact
that is typically the most relevant output when such a task fails.
## Scopes and roles
[Scopes] are what Taskcluster calls permissions.
They control access to everything.
Anyone logged in in the [web UI] has (access to) a set of scopes,
which is visible on the [credentials] page
(reachable from clicking on ones own name on the top-right of any page).
A running task has a set of scopes allowing it access to various functionality and APIs.
It can grant those scopes (and at most only thoses) to sub-tasks that it schedules
(if it has the scope allowing it to schedule new tasks in the first place).
[Roles] represent each a set of scopes.
They can be granted to… things,
and then configured separately to modify what scopes they [expand] to.
For example, when Taskcluster-GitHub schedules tasks based on the `.taskcluster.yml` file
in a push to the `auto` branch of this repository,
those tasks are granted the scope `assume:repo:github.com/servo/servo:branch:auto`.
Scopes that start with `assume:` are special,
they expand to the scopes defined in the matching roles.
In this case, the [`repo:github.com/servo/servo:branch:*`][branches] role matches.
Servo admins have scope `auth:update-role:repo:github.com/servo/*` which allows them
to edit that role in the web UI and grant more scopes to these tasks
(if that person has the new scope themselves).
[Scopes]: https://docs.taskcluster.net/docs/manual/design/apis/hawk/scopes
[web UI]: https://tools.taskcluster.net/
[credentials]: https://tools.taskcluster.net/credentials
[Roles]: https://docs.taskcluster.net/docs/manual/design/apis/hawk/roles
[expand]: https://docs.taskcluster.net/docs/reference/platform/taskcluster-auth/docs/roles
[branches]: https://tools.taskcluster.net/auth/roles/repo%3Agithub.com%2Fservo%2Fservo%3Abranch%3A*
## Daily tasks
The [`project-servo/daily`] hook in Taskclusters [Hooks service]
is used to run some tasks automatically ever 24 hours.
In this case as well we use a decision task.
The `decision-task.py` script can differenciate this from a GitHub push
based on the `$TASK_FOR` environment variable.
Daily tasks can also be triggered manually.
Scopes available to the daily decision task need to be both requested in the hook definition
and granted through the [`hook-id:project-servo/daily`] role.
Because they do not have something similar to GitHub statuses that link to them,
daily tasks are indexed under the [`project.servo.servo.daily`] namespace.
[`project.servo.servo.daily`]: https://tools.taskcluster.net/index/project.servo.servo.daily
[`project-servo/daily`]: https://tools.taskcluster.net/hooks/project-servo/daily
[Hooks service]: https://docs.taskcluster.net/docs/manual/using/scheduled-tasks
[`hook-id:project-servo/daily`]: https://tools.taskcluster.net/auth/roles/hook-id%3Aproject-servo%2Fdaily
## AWS EC2 workers
As of this writing, Servo on Taskcluster can only use the `servo-docker-worker` worker type.
Tasks scheduled with this worker type run in a Linux environment,
in a Docker container, on an AWS EC2 virtual machine.
These machines are short-lived “spot instances”.
They are started automatically as needed by the [AWS provisioner]
when the existing capacity is insufficient to execute queued tasks.
They terminate themselves after being idle without work for a while,
or unconditionally after a few days.
Because these workers are short-lived,
we dont need to worry about evicting old entries from Cargos or rustups download cache,
for example.
Servo admins can view and edit the [worker type definition] which configures the provisioner,
in particular with the types of EC2 instances to be used.
[AWS provisioner]: https://docs.taskcluster.net/docs/reference/integrations/aws-provisioner/references/api
[worker type definition]: https://tools.taskcluster.net/aws-provisioner/servo-docker-worker/view
## Self-service, Bugzilla, and IRC
Taskcluster is designed to be “self-service” as much as possible,
with features like in-tree `.taskcluster.yml`
or the web UI for modifying the worker type definitions.
However some changes like adding a new worker type still require Taskcluster admin access.
For those, file requests on Bugzilla under [Taskcluster :: Service Request][req].
For asking for help less formally, try the `#servo` or `#taskcluster` channels on Mozilla IRC.
[req]: https://bugzilla.mozilla.org/enter_bug.cgi?product=Taskcluster&component=Service%20Request
## Configuration recap
We try to keep as much as possible of our Taskcluster configuration in this repository.
To modify those, submit a pull request.
* The [`.taskcluster.yml`][tc.yml] file,
for starting decision tasks in reaction to GitHub events
* The [`etc/ci/decision-task.py`](decision-task.py) file,
defining what other tasks to schedule
However some configuration needs to be handled separately.
Modifying those requires Servo-project-level administrative access.
* The [`aws-provisioner/servo-docker-worker`][worker type definition] worker type definition,
for EC2 instances configuration
* The [`project-servo/daily`] hook definition,
for starting daily decision tasks
* The [`hook-id:project-servo/daily`] role,
for scopes granted to those tasks
* The [`repo:github.com/servo/servo:branch:*`][branches] role,
for scopes granted to tasks responding to a GitHub push to the repository (includin by Homu)

View file

@ -0,0 +1,23 @@
#!/usr/bin/env bash
# This Source Code Form is subject to the terms of the Mozilla Public
# License, v. 2.0. If a copy of the MPL was not distributed with this
# file, You can obtain one at http://mozilla.org/MPL/2.0/.
set -o errexit
set -o nounset
set -o pipefail
task_id="${1}"
artifact="${2}"
shift 2
queue="https://queue.taskcluster.net/v1"
url="${queue}/task/${task_id}/artifacts/public/${artifact}"
echo "Fetching ${url}" >&2
curl \
--retry 5 \
--connect-timeout 10 \
--location \
--fail \
"${url}" \
"${@}"

View file

@ -0,0 +1,221 @@
# coding: utf8
# This Source Code Form is subject to the terms of the Mozilla Public
# License, v. 2.0. If a copy of the MPL was not distributed with this
# file, You can obtain one at http://mozilla.org/MPL/2.0/.
import os.path
import subprocess
from decisionlib import DecisionTask
def main():
task_for = os.environ["TASK_FOR"]
if task_for == "github-push":
linux_tidy_unit()
#linux_wpt()
# https://tools.taskcluster.net/hooks/project-servo/daily
elif task_for == "daily":
daily_tasks_setup()
with_rust_nightly()
else:
raise ValueError("Unrecognized $TASK_FOR value: %r", task_for)
ping_on_daily_task_failure = "SimonSapin, nox, emilio"
build_artifacts_expiry = "1 week"
log_artifacts_expiry = "1 year"
build_env = {
"RUST_BACKTRACE": "1",
"RUSTFLAGS": "-Dwarnings",
"CARGO_INCREMENTAL": "0",
"SCCACHE_IDLE_TIMEOUT": "1200",
"CCACHE": "sccache",
"RUSTC_WRAPPER": "sccache",
"SHELL": "/bin/dash", # For SpiderMonkeys build system
}
def linux_tidy_unit():
return decision.create_task(
task_name="Linux x86_64: tidy + dev build + unit tests",
script="""
./mach test-tidy --no-progress --all
./mach build --dev
./mach test-unit
./mach package --dev
./mach test-tidy --no-progress --self-test
python2.7 ./etc/memory_reports_over_time.py --test
python3 ./etc/ci/taskcluster/mock.py
./etc/ci/lockfile_changed.sh
./etc/ci/check_no_panic.sh
""",
**build_kwargs
)
def with_rust_nightly():
return decision.create_task(
task_name="Linux x86_64: with Rust Nightly",
script="""
echo "nightly" > rust-toolchain
./mach build --dev
./mach test-unit
""",
**build_kwargs
)
def linux_wpt():
release_build_task = linux_release_build()
total_chunks = 2
for i in range(total_chunks):
this_chunk = i + 1
wpt_chunk(release_build_task, total_chunks, this_chunk, extra=(this_chunk == 1))
def linux_release_build():
return decision.find_or_create_task(
index_bucket="build.linux_x86-64_release",
index_key=os.environ["GIT_SHA"], # Set in .taskcluster.yml
index_expiry=build_artifacts_expiry,
task_name="Linux x86_64: release build",
script="""
./mach build --release --with-debug-assertions -p servo
./etc/ci/lockfile_changed.sh
tar -czf /target.tar.gz \
target/release/servo \
target/release/build/osmesa-src-*/output \
target/release/build/osmesa-src-*/out/lib/gallium
""",
artifacts=[
"/target.tar.gz",
],
**build_kwargs
)
def wpt_chunk(release_build_task, total_chunks, this_chunk, extra):
if extra:
name_extra = " + extra"
script_extra = """
./mach test-wpt-failure
./mach test-wpt --release --binary-arg=--multiprocess --processes 24 \
--log-raw test-wpt-mp.log \
--log-errorsummary wpt-mp-errorsummary.log \
eventsource
"""
else:
name_extra = ""
script_extra = ""
script = """
./mach test-wpt \
--release \
--processes 24 \
--total-chunks "$TOTAL_CHUNKS" \
--this-chunk "$THIS_CHUNK" \
--log-raw test-wpt.log \
--log-errorsummary wpt-errorsummary.log \
--always-succeed
./mach filter-intermittents\
wpt-errorsummary.log \
--log-intermittents intermittents.log \
--log-filteredsummary filtered-wpt-errorsummary.log \
--tracker-api default
"""
# FIXME: --reporter-api default
# IndexError: list index out of range
# File "/repo/python/servo/testing_commands.py", line 533, in filter_intermittents
# pull_request = int(last_merge.split(' ')[4][1:])
create_run_task(
build_task=release_build_task,
task_name="Linux x86_64: WPT chunk %s / %s%s" % (this_chunk, total_chunks, name_extra),
script=script_extra + script,
env={
"TOTAL_CHUNKS": total_chunks,
"THIS_CHUNK": this_chunk,
},
)
def create_run_task(*, build_task, script, **kwargs):
fetch_build = """
./etc/ci/taskcluster/curl-artifact.sh ${BUILD_TASK_ID} target.tar.gz | tar -xz
"""
kwargs.setdefault("env", {})["BUILD_TASK_ID"] = build_task
kwargs.setdefault("dependencies", []).append(build_task)
kwargs.setdefault("artifacts", []).extend(
("/repo/" + word, log_artifacts_expiry)
for word in script.split() if word.endswith(".log")
)
return decision.create_task(
script=fetch_build + script,
max_run_time_minutes=60,
dockerfile=dockerfile_path("run"),
**kwargs
)
def daily_tasks_setup():
# ':' is not accepted in an index namepspace:
# https://docs.taskcluster.net/docs/reference/core/taskcluster-index/references/api
now = decision.now.strftime("%Y-%m-%d_%H-%M-%S")
index_path = "%s.daily.%s" % (decision.index_prefix, now)
# Index this task manually rather than with a route,
# so that it is indexed even if it fails.
decision.index_service.insertTask(index_path, {
"taskId": os.environ["TASK_ID"],
"rank": 0,
"data": {},
"expires": decision.from_now_json(log_artifacts_expiry),
})
# Unlike when reacting to a GitHub event,
# the commit hash is not known until we clone the repository.
os.environ["GIT_SHA"] = \
subprocess.check_output(["git", "rev-parse", "HEAD"]).decode("utf8").strip()
# On failure, notify a few people on IRC
# https://docs.taskcluster.net/docs/reference/core/taskcluster-notify/docs/usage
notify_route = "notify.irc-channel.#servo.on-failed"
decision.routes_for_all_subtasks.append(notify_route)
decision.scopes_for_all_subtasks.append("queue:route:" + notify_route)
decision.task_name_template = "Servo daily: %s. On failure, ping: " + ping_on_daily_task_failure
def dockerfile_path(name):
return os.path.join(os.path.dirname(__file__), "docker", name + ".dockerfile")
decision = DecisionTask(
task_name_template="Servo: %s",
index_prefix="project.servo.servo",
worker_type="servo-docker-worker",
)
# https://docs.taskcluster.net/docs/reference/workers/docker-worker/docs/caches
cache_scopes = [
"docker-worker:cache:cargo-*",
]
build_caches = {
"cargo-registry-cache": "/root/.cargo/registry",
"cargo-git-cache": "/root/.cargo/git",
"cargo-rustup": "/root/.rustup",
"cargo-sccache": "/root/.cache/sccache",
}
build_kwargs = {
"max_run_time_minutes": 60,
"dockerfile": dockerfile_path("build"),
"env": build_env,
"scopes": cache_scopes,
"cache": build_caches,
}
if __name__ == "__main__":
main()

View file

@ -0,0 +1,267 @@
# coding: utf8
# Copyright 2018 The Servo Project Developers. See the COPYRIGHT
# file at the top-level directory of this distribution.
#
# Licensed under the Apache License, Version 2.0 <LICENSE-APACHE or
# http://www.apache.org/licenses/LICENSE-2.0> or the MIT license
# <LICENSE-MIT or http://opensource.org/licenses/MIT>, at your
# option. This file may not be copied, modified, or distributed
# except according to those terms.
"""
Project-independent library for Taskcluster decision tasks
"""
import datetime
import hashlib
import json
import os
import re
import sys
import taskcluster
class DecisionTask:
"""
Holds some project-specific configuration and provides higher-level functionality
on top of the `taskcluster` package a.k.a. `taskcluster-client.py`.
"""
DOCKER_IMAGE_ARTIFACT_FILENAME = "image.tar.lz4"
# https://github.com/servo/taskcluster-bootstrap-docker-images#image-builder
DOCKER_IMAGE_BUILDER_IMAGE = "servobrowser/taskcluster-bootstrap:image-builder@sha256:" \
"0a7d012ce444d62ffb9e7f06f0c52fedc24b68c2060711b313263367f7272d9d"
def __init__(self, *, index_prefix="garbage.servo-decisionlib", task_name_template="%s",
worker_type="github-worker", docker_image_cache_expiry="1 year",
routes_for_all_subtasks=None, scopes_for_all_subtasks=None):
self.task_name_template = task_name_template
self.index_prefix = index_prefix
self.worker_type = worker_type
self.docker_image_cache_expiry = docker_image_cache_expiry
self.routes_for_all_subtasks = routes_for_all_subtasks or []
self.scopes_for_all_subtasks = scopes_for_all_subtasks or []
# https://docs.taskcluster.net/docs/reference/workers/docker-worker/docs/features#feature-taskclusterproxy
self.queue_service = taskcluster.Queue(options={"baseUrl": "http://taskcluster/queue/v1/"})
self.index_service = taskcluster.Index(options={"baseUrl": "http://taskcluster/index/v1/"})
self.now = datetime.datetime.utcnow()
self.found_or_created_indices = {}
def from_now_json(self, offset):
"""
Same as `taskcluster.fromNowJSON`, but uses the creation time of `self` for now.
"""
return taskcluster.stringDate(taskcluster.fromNow(offset, dateObj=self.now))
def find_or_create_task(self, *, index_bucket, index_key, index_expiry, artifacts, **kwargs):
"""
Find a task indexed in the given bucket (kind, category, ) and cache key,
on schedule a new one if there isnt one yet.
Returns the task ID.
"""
index_path = "%s.%s.%s" % (self.index_prefix, index_bucket, index_key)
task_id = self.found_or_created_indices.get(index_path)
if task_id is not None:
return task_id
try:
result = self.index_service.findTask(index_path)
task_id = result["taskId"]
except taskcluster.TaskclusterRestFailure as e:
if e.status_code == 404:
task_id = self.create_task(
routes=[
"index." + index_path,
],
extra={
"index": {
"expires": self.from_now_json(self.docker_image_cache_expiry),
},
},
artifacts=[
(artifact, index_expiry)
for artifact in artifacts
],
**kwargs
)
else:
raise
self.found_or_created_indices[index_path] = task_id
return task_id
def find_or_build_docker_image(self, dockerfile):
"""
Find a task that built a Docker image based on this `dockerfile`,
or schedule a new image-building task if needed.
Returns the task ID.
"""
dockerfile_contents = expand_dockerfile(dockerfile)
digest = hashlib.sha256(dockerfile_contents).hexdigest()
return self.find_or_create_task(
index_bucket="docker-image",
index_key=digest,
index_expiry=self.docker_image_cache_expiry,
task_name="Docker image: " + image_name(dockerfile),
script="""
echo "$DOCKERFILE" | docker build -t taskcluster-built -
docker save taskcluster-built | lz4 > /%s
""" % self.DOCKER_IMAGE_ARTIFACT_FILENAME,
env={
"DOCKERFILE": dockerfile_contents,
},
artifacts=[
"/" + self.DOCKER_IMAGE_ARTIFACT_FILENAME,
],
max_run_time_minutes=20,
docker_image=self.DOCKER_IMAGE_BUILDER_IMAGE,
features={
"dind": True, # docker-in-docker
},
with_repo=False,
)
def create_task(self, *, task_name, script, max_run_time_minutes,
docker_image=None, dockerfile=None, # One of these is required
artifacts=None, dependencies=None, env=None, cache=None, scopes=None,
routes=None, extra=None, features=None,
with_repo=True):
"""
Schedule a new task. Only supports `docker-worker` for now.
Returns the new task ID.
One of `docker_image` or `dockerfile` (but not both) must be given.
If `dockerfile` is given, the corresponding Docker image is built as needed and cached.
`with_repo` indicates whether `script` should start in a clone of the git repository.
"""
if docker_image and dockerfile:
raise TypeError("cannot use both `docker_image` or `dockerfile`")
if not docker_image and not dockerfile:
raise TypeError("need one of `docker_image` or `dockerfile`")
# https://docs.taskcluster.net/docs/reference/workers/docker-worker/docs/environment
decision_task_id = os.environ["TASK_ID"]
dependencies = [decision_task_id] + (dependencies or [])
if dockerfile:
image_build_task = self.find_or_build_docker_image(dockerfile)
dependencies.append(image_build_task)
docker_image = {
"type": "task-image",
"taskId": image_build_task,
"path": "public/" + self.DOCKER_IMAGE_ARTIFACT_FILENAME,
}
# Set in .taskcluster.yml
task_owner = os.environ["TASK_OWNER"]
task_source = os.environ["TASK_SOURCE"]
env = env or {}
if with_repo:
# Set in .taskcluster.yml
for k in ["GIT_URL", "GIT_REF", "GIT_SHA"]:
env[k] = os.environ[k]
script = """
git init repo
cd repo
git fetch --depth 1 "$GIT_URL" "$GIT_REF"
git reset --hard "$GIT_SHA"
""" + script
payload = {
"taskGroupId": decision_task_id,
"dependencies": dependencies or [],
"schedulerId": "taskcluster-github",
"provisionerId": "aws-provisioner-v1",
"workerType": self.worker_type,
"created": self.from_now_json(""),
"deadline": self.from_now_json("1 day"),
"metadata": {
"name": self.task_name_template % task_name,
"description": "",
"owner": task_owner,
"source": task_source,
},
"scopes": (scopes or []) + self.scopes_for_all_subtasks,
"routes": (routes or []) + self.routes_for_all_subtasks,
"extra": extra or {},
"payload": {
"cache": cache or {},
"maxRunTime": max_run_time_minutes * 60,
"image": docker_image,
"command": [
"/bin/bash",
"--login",
"-x",
"-e",
"-c",
deindent(script)
],
"env": env,
"artifacts": {
"public/" + os.path.basename(path): {
"type": "file",
"path": path,
"expires": self.from_now_json(expires),
}
for path, expires in artifacts or []
},
"features": features or {},
},
}
task_id = taskcluster.slugId().decode("utf8")
self.queue_service.createTask(task_id, payload)
print("Scheduled %s" % task_name)
return task_id
def image_name(dockerfile):
"""
Guess a short name based on the path `dockerfile`.
"""
basename = os.path.basename(dockerfile)
suffix = ".dockerfile"
if basename == "Dockerfile":
return os.path.basename(os.path.dirname(os.path.abspath(dockerfile)))
elif basename.endswith(suffix):
return basename[:-len(suffix)]
else:
return basename
def expand_dockerfile(dockerfile):
"""
Read the file at path `dockerfile`,
and transitively expand the non-standard `% include` header if it is present.
"""
with open(dockerfile, "rb") as f:
dockerfile_contents = f.read()
include_marker = b"% include"
if not dockerfile_contents.startswith(include_marker):
return dockerfile_contents
include_line, _, rest = dockerfile_contents.partition(b"\n")
included = include_line[len(include_marker):].strip().decode("utf8")
path = os.path.join(os.path.dirname(dockerfile), included)
return b"\n".join([expand_dockerfile(path), rest])
def deindent(string):
return re.sub("\n +", " \n ", string).strip()

View file

@ -0,0 +1,23 @@
FROM ubuntu:bionic-20180821
ENV \
#
# Some APT packages like 'tzdata' wait for user input on install by default.
# https://stackoverflow.com/questions/44331836/apt-get-install-tzdata-noninteractive
DEBIAN_FRONTEND=noninteractive
RUN \
apt-get update -q && \
apt-get install -qy --no-install-recommends \
#
# Cloning the repository
git \
ca-certificates \
#
# Running mach
python2.7 \
virtualenv \
#
# Installing rustup and sccache (build dockerfile) or fetching build artifacts (run tasks)
curl

View file

@ -0,0 +1,43 @@
% include base.dockerfile
RUN \
apt-get install -qy --no-install-recommends \
#
# Multiple C/C++ dependencies built from source
g++ \
make \
cmake \
#
# ANGLE
xorg-dev \
#
# mozjs (SpiderMonkey)
autoconf2.13 \
#
# Bindgen (for SpiderMonkey bindings)
clang \
#
# GStreamer
libgstreamer-plugins-bad1.0-dev \
#
# OpenSSL
libssl1.0-dev \
#
# blurz
libdbus-1-dev \
#
# Skia
libglu1-mesa-dev \
libbz2-dev \
#
#
&& \
#
#
curl https://sh.rustup.rs -sSf | sh -s -- --default-toolchain none -y && \
#
#
curl -sSfL \
https://github.com/mozilla/sccache/releases/download/0.2.7/sccache-0.2.7-x86_64-unknown-linux-musl.tar.gz \
| tar -xz --strip-components=1 -C /usr/local/bin/ \
sccache-0.2.7-x86_64-unknown-linux-musl/sccache

View file

@ -0,0 +1,9 @@
% include base.dockerfile
# Servos runtime dependencies
RUN apt-get install -qy --no-install-recommends \
libgl1 \
libssl1.0.0 \
libdbus-1-3 \
libgstreamer-plugins-bad1.0-0 \
gstreamer1.0-plugins-good

45
etc/ci/taskcluster/mock.py Executable file
View file

@ -0,0 +1,45 @@
#!/usr/bin/python3
# Copyright 2018 The Servo Project Developers. See the COPYRIGHT
# file at the top-level directory of this distribution.
#
# Licensed under the Apache License, Version 2.0 <LICENSE-APACHE or
# http://www.apache.org/licenses/LICENSE-2.0> or the MIT license
# <LICENSE-MIT or http://opensource.org/licenses/MIT>, at your
# option. This file may not be copied, modified, or distributed
# except according to those terms.
"""
Run the decision task with fake Taskcluster APIs, to catch Python errors before pushing.
"""
import os
import sys
from unittest.mock import MagicMock
class TaskclusterRestFailure(Exception):
status_code = 404
class Index:
__init__ = insertTask = lambda *_, **__: None
def findTask(self, _):
raise TaskclusterRestFailure
Queue = stringDate = fromNow = slugId = MagicMock()
sys.modules["taskcluster"] = sys.modules[__name__]
sys.dont_write_bytecode = True
code = open(os.path.join(os.path.dirname(__file__), "decision-task.py"), "rb").read()
for k in "TASK_ID TASK_OWNER TASK_SOURCE GIT_URL GIT_REF GIT_SHA".split():
os.environ[k] = k
print("Push:")
os.environ["TASK_FOR"] = "github-push"
exec(code)
print("Daily:")
os.environ["TASK_FOR"] = "daily"
exec(code)

View file

@ -68,6 +68,9 @@ files = [
"./tests/wpt/mozilla/tests/css/fonts",
"./tests/wpt/mozilla/tests/css/pre_with_tab.html",
"./tests/wpt/mozilla/tests/mozilla/textarea_placeholder.html",
# Python 3 syntax causes "E901 SyntaxError" when flake8 runs in Python 2
"./etc/ci/taskcluster/decision-task.py",
"./etc/ci/taskcluster/decisionlib.py",
]
# Directories that are ignored for the non-WPT tidy check.
directories = [