Commit Graph

170 Commits

Author SHA1 Message Date
James E. Blair 1f026bd49c Finish circular dependency refactor
This change completes the circular dependency refactor.

The principal change is that queue items may now include
more than one change simultaneously in the case of circular
dependencies.

In dependent pipelines, the two-phase reporting process is
simplified because it happens during processing of a single
item.

In independent pipelines, non-live items are still used for
linear depnedencies, but multi-change items are used for
circular dependencies.

Previously changes were enqueued recursively and then
bundles were made out of the resulting items.  Since we now
need to enqueue entire cycles in one queue item, the
dependency graph generation is performed at the start of
enqueing the first change in a cycle.

Some tests exercise situations where Zuul is processing
events for old patchsets of changes.  The new change query
sequence mentioned in the previous paragraph necessitates
more accurate information about out-of-date patchsets than
the previous sequence, therefore the Gerrit driver has been
updated to query and return more data about non-current
patchsets.

This change is not backwards compatible with the existing
ZK schema, and will require Zuul systems delete all pipeline
states during the upgrade.  A later change will implement
a helper command for this.

All backwards compatability handling for the last several
model_api versions which were added to prepare for this
upgrade have been removed.  In general, all model data
structures involving frozen jobs are now indexed by the
frozen job's uuid and no longer include the job name since
a job name no longer uniquely identifies a job in a buildset
(either the uuid or the (job name, change) tuple must be
used to identify it).

Job deduplication is simplified and now only needs to
consider jobs within the same buildset.

The fake github driver had a bug (fakegithub.py line 694) where
it did not correctly increment the check run counter, so our
tests that verified that we closed out obsolete check runs
when re-enqueing were not valid.  This has been corrected, and
in doing so, has necessitated some changes around quiet dequeing
when we re-enqueue a change.

The reporting in several drivers has been updated to support
reporting information about multiple changes in a queue item.

Change-Id: I0b9e4d3f9936b1e66a08142fc36866269dc287f1
Depends-On: https://review.opendev.org/907627
2024-02-09 07:39:40 -08:00
James E. Blair 663931cba3 Include job_uuid in BuildRequests/Events
This is part of the circular dependency refactor.

It updates BuildRequests to include the job uuid (similar to the
previous change with NodeRequests).  We also use the job uuid when
sending BuildResultEvents.

Change-Id: If7a55138d0cb5a358c62e8cd97ee322087b09a6b
2024-01-08 08:54:56 -08:00
James E. Blair 9201f9ee28 Store builds on buildset by uuid
This is part of the circular dependency refactor.

This updates the buildset object in memory (and zk) to store builds
indexed by frozen job uuid rather than job name.  This also updates
everal related fields and also temporary dictionaries to do the same.

This will allow us, in the future, to have more than one job/build
in a buildset with the same name (for different changes/refs).

Change-Id: I70865ec8d70fb9105633f0d03ba7c7e3e6cd147d
2023-12-12 11:58:21 -08:00
James E. Blair dd60903a95 Retry lingering deduplicated builds
We intend to handle the case where two queue items in check share
a deduplicated build and one of them completes and is dequeued while
the other continues.  To handle this, we avoid removing any queue
items (and therefore their associated builds) from ZK if their builds
show up in any currently-enqueued queue item.  However, we don't
actually have a mechanism to load a build from ZK in that situation
if it isn't in a queue item that is currently enqueued.

Adding such a mechanism is complex and risky, whereas the circular
dependency refactoring effort currently in progress will address this
issue in a comprehensive manner.

This change addresses the issue by detecting the situation at the
point where we would try to launch the build (since we failed to
restore it from ZK) and instead of raising an exception as we currently
do, we tell the scheduler to retry the build.  This results in the
buildset not actually taking advantage of the potential deduplication,
but it does at least provide a working build result to the user in
the form of a brand new build.

Change-Id: I86c159c82b858e67433bdaa1e479471b21ea8b86
2023-11-16 14:59:31 -08:00
Simon Westphahl b8be20c5db
Expose max. attempts to the job as a Zuul variable
This allows a job to determine if the current execution is the last
attempt.

Change-Id: I989db9f924aab997496851ce99ad71bbacf2379e
2023-04-14 08:13:35 +02:00
James E. Blair 279d7fb5cd
Fix deduplication exceptions in pipeline processing
If a build is to be deduplicated and has not started yet and has
a pending node request, we store a dictionary describing the target
deduplicated build in the node_requests dictionary on the buildset.

There were a few places where we directly accessed that dictionary
and assumed the results would be the node request id.  Notably, this
could cause an error in pipeline processing (as well os potentially
some other edge cases such as reconfiguring).

Most of the time we can just ignore deduplicated node requests since
the "real" buildset will take care of them.  This change enriches
the API to help with that.  In other places, we add a check for the
type.

To test this, we enable relative_priority in the config file which
is used in the deduplication tests, and we also add an assertion
which runs at the end of every test that ensures there were no
pipeline exceptions during the test (almost all the existing dedup
tests fail this assertion before this change).

Change-Id: Ia0c3f000426011b59542d8e56b43767fccc89a22
2022-11-21 09:22:25 +01:00
James E. Blair c355adf44e Add playbook semaphores
This adds the ability to specify that the Zuul executor should
acquire a semaphore before running an individual playbook.  This
is useful for long running jobs which need exclusive access to
a resources for only a small amount of time.

Change-Id: I90f5e0f570ef6c4b0986b0143318a78ddc27bbde
2022-11-07 08:41:10 -08:00
Simon Westphahl 075bdd0178
Add span for builds and propagate via request
Change-Id: Ib10452862e7aa1355502bb381d3ff07c65ac7187
Co-Authored-By: James E. Blair <jim@acmegating.com>
Co-Authored-By: Tristan Cacqueray <tdecacqu@redhat.com>
2022-09-19 08:42:28 +02:00
Felix Edel 41fcc6d53b Implement job freezing API in zuul-web
This implements the job freezing API directly in zuul-web, so there is
no need to call the scheduler via RPC.

Change-Id: Ibc7737a51fe5428bacdcb4763b3e6155cea29036
2021-11-10 09:25:49 +01:00
Simon Westphahl a7dde6c189 Store builds in Zookeeper
Change-Id: I66740e36b63c9a5e6f6bb39a0cd5c7a93a9c08b4
2021-10-29 12:04:44 +02:00
Felix Edel 645c2d5098 Remove the local builds list from the executor client
This removes the local builds list (executor.builds) from the executor
client.

To make this work, we have to adapt a few tests which are still using
this list. To provide a similar functionality for those tests, the
ZuulTestCase class now provides a method getCurrentBuilds() that looks
up the builds from the active pipelines/queues.

Change-Id: If74bc7164d1156cdb7ba06f12868c60bf1617ea8
2021-09-24 16:26:27 -07:00
Felix Edel c6ce4ae2bb Don't use executor.builds when processing build result events
The executor client still holds a list of local builds objects which is
used in various places. One use case is to look up necessary
information of the original build when a build result event is handled.

Using such a local list won't work with multiple schedulers in place. As
a first step we will avoid using this list for handling build result
events and instead provide all necessary information to the build result
itself and look up the remaining information from the pipeline directly.

This change also improves the log output when processing build result
events in the scheduler.

Change-Id: I9c4e573de2ce63259ec6cfb7d69c2f5be48f33ef
2021-09-24 16:25:25 -07:00
James E. Blair 97a76de403 Fix race involving job request locks
It's possible for the following sequence to occur (prefixed by
thread ids):

2> process job request cache update

1> finish job
1> set job request state to complete
1> unlock job request
1> delete job request
1> delete job request lock

2> get cached list of running jobs for lostRequests, start examining job
2> check if the job is unlocked (this will re-create the lock dir and return true)
2> attempt to set job request state to complete (this will raise JobRequestNotFound)
2> bail

This leaves a lock node laying around.  We have a cleanup process that
will eventually remove it in production, but it's existence can cause
the clean-state checks at the end of unit tests to fail.

To correct this:

a) Try to avoid re-creating the lock (though this is not possible in all cases)
b) If we encounter JobRequestNotFound error in the cleanup, attempt to
   delete the job nonetheless (so that we re-delete the lock dir)

The remove method is also made entirely idemptotent to support this.

Change-Id: I49ad5c38a3c6cbaf0962e805b6c228e36b97a3d2
2021-09-14 09:10:34 -07:00
James E. Blair 6dc1178fc3 Don't store node requests/nodesets on queue items
To prepare for queue items moving into ZooKeeper, stop storing the
NodeRequest and NodeSet objects on them.  Instead, reference requests
by ID and consult ZK when necessary, and store only the info about
nodesets that the scheduler needs.  The result are simple dicts
than can easily be serialized.

The deleteNodeRequest method is updated to accept IDs instead of
NodeRequest objects to minimize the number of times we need to
use a full NodeRequest object.

Change-Id: I3587a42eb5a151f41369385e482b7f36b1c41bf6
2021-09-10 08:51:20 -07:00
James E. Blair dbe13ce076 Remove nodeset from NodeRequest
To make things simpler for schedulers to handle node provisioned
events for node requests which they may not have in their local
pipeline state, we need to make the pipeline storage of node requests
simpler.  That starts by removing the nodeset object as an attribute
of the NodeRequest object.  This means that the scheduler can work
with a node request object without relying on having the associated
nodeset.  It also simplifies the ZooKeeper code that deserializes
NodeRequests (as it doesn't have to create fake NodeSet objects too).
And finally, it simplifies what must be stored in the pipeline
and queue item structures, which will also come in handy later.

Two tests designed to verify that the request->nodeset magic
deserialization worked have been removed since they are no longer
applicable.

Change-Id: I70ae083765d5cd9a4fd1afc2442bf22d6c52ba0b
2021-09-02 09:29:44 -07:00
Tobias Henkel a746cc75c1
Remove noop builds from builds list
Those are not submitted to zookeeper but still need to be removed from
the builds list to prevent a memory leak.

Change-Id: I60d2e5e957aa4b7d877dd5e8ba78977eb3e7d7e6
2021-08-17 10:25:49 +02:00
James E. Blair a729d6c6e8 Refactor Merger/Executor API
The Merger and executor APIs have a lot in common, but they behave
slightly differently.  A merger needs to sometimes return results.
An executor needs to have separate queues for zones and be able to
pause or cancel jobs.

This refactors them both into a common class which can handle job
state changes (like pause/cancel) and return results if requested.

The MergerApi can subclass this fairly trivially.

The ExecutorApi adds an intermediate layer which uses a
DefaultKeyDict to maintain a distinct queue for every zone and then
transparently dispatches method calls to the queue object for
that zone.

The ZK paths for both are significantly altered in this change.

Change-Id: I3adedcc4ea293e43070ba6ef0fe29e7889a0b502
2021-08-06 15:40:46 -07:00
James E. Blair c237772ac8 Move executor api cleanup to apscheduler
We don't need to clean up leaked build parameters every time we
clean up lost build requests.  The latter happens every 60 seconds
because it's performance critical, but the param cleanup can happen
much less frequently.

Move it into the scheduled tasks that the scheduler runs, but rather
than continuing to add new apsched events, combine this with the
config cache cleanup.  We can also add the similar cleanup task which
will be needed by the merger api.  These can all run infrequently
(every 60 minutes).

The only downside to this is that we will have one scheduler running
all of these combined tasks, but they are relatively low impact so
it's probably not important to load-balance them.

Change-Id: I76ce9df242ebdaf9f1b7113530775bd702ccaaa0
2021-08-02 15:58:38 -07:00
James E. Blair a99b2dace3 Cleanup leaked build request parameters
Now that build request parameters are sharded outside of a transaction,
we might write them but fail to write the actual build request.  To
address that, add a periodic checke for leaked parameters which is run
after the search for lost build requests.

Change-Id: I0787d94513572f90ae66616a6619d488c9592a5d
2021-08-02 15:57:12 -07:00
James E. Blair 03e98df9da Use the nodeset build parameter instead of hosts/groups
The serialized nodeset is now supplied as a build parameter,
which makes the synthetic hosts and groups parameters which are
derived from it redundant.

Update the executor to rely entirely on the deserialized nodeset.

We also rename the method which creates the parameters since they
are not used for gearman any more.

A subsequent change can remove the hosts and nodes parameters.

Change-Id: Ied7f78c332485e5c66b5721c1007c25660d4238e
2021-07-20 11:04:24 -07:00
Felix Edel fee46c25bc Lock/unlock nodes on executor server
Currently, the nodes are locked in the scheduler/pipeline manager before
the actual build is created in the executor client. When the nodes are
locked, the corresponding NodeRequest is also deleted.

With this change, the executor will lock the nodes directly before
starting the build and unlock them when the build is completed.

To keep the order of events intact, the nodepool.acceptNodes() method is
split up into two:
    1. nodepool.acceptNodeRequest() does most of the old acceptNodes()
       method except for locking the nodes and deleting the node
       request. It is called on the scheduler side when the
       NodesProvisionedEvent is handled (which is also where
       acceptNodes() was previously called).
    2. nodepool.acceptNodes() is now called on the executor side when
       the job is started. It locks the nodes and deletes the node
       request in ZooKeeper.

Finally, it's also necessary to move the autohold processing to the
executor, as this requires a lock on the node. To allow processing of
autoholds, the executor now also determines the build attempts and sets
the RETRY_LIMIT result if necessary.

Change-Id: I7392ce47e84dcfb8079c16e34e0ed2062ebf4136
2021-07-01 05:46:02 +00:00
James E. Blair db6697e5af Move build request cleanup from executor to scheduler
The scheduler is a better home for the build request cleanup because
the cleanup process should run across all zones, and the executor
api object in the executors are tied to specific zones, so they
have an incomplete cache.

It also means that if all the executors go offline, the schedulers
can still cancel and reset the builds, so it doesn't look like
dead builds are still running.

Since we now have two cleanup actions, instead of making a thread
for each, we use APScheduler to run the cleanup functions on an
interval.  We use ZK locks to make sure that they aren't running
on more than one scheduler.

Change-Id: I0245b71f31aae9616d8e65d27c63b25b2c27f815
2021-06-29 14:37:15 -07:00
Felix Edel 6ac14615a0 Execute builds via ZooKeeper
This is the second part of I5de26afdf6774944b35472e2054b93d12fe21793.
It uses the executor api.

Three tests are disabled until the next change.

Change-Id: Ie08fa9dfb4bb3adb9a02e0a2e8b11309e1ec27cd
2021-06-29 14:37:15 -07:00
James E. Blair 638b1e1090 Fix unknown job detection
If geard loses track of a job, this code is supposed to detect it,
but we've moved classes, so we can't use dunderscores anymore,
we need to change it to a single underscore.

This also corrects the mirror situation where the scheduler is
disconnected from the gearman server; that would eventually fall
through to the lost build handler, but since we get an unambiguous
disconnect event, let's restore the handling of that.

A test case for each of these is added.

Change-Id: I41a926c488717f8bc4cd8a173beaa9458e7cb6e2
2021-06-09 11:51:05 -07:00
Felix Edel 8bf5bcd1bb Calculate build start and end time on executor server
This sets the build's start and end time on the executor server rather
than the scheduler. This is necessary to calculate the in-use duration
of locked nodes later on when the nodes are locked on the executor.

Since the scheduler must still set the start and end time on the local
build object, we use the BuildStarted and BuildCompleted to provide the
timestamps to the scheduler. Once the builds are stored in ZooKeeper,
the executor can directly set the timestamps on the build.

In case of noop builds the timestamps will be set by the executor client
as those builds don't show up on any executor server.

Change-Id: I08ac9618b3c12280a8720657f97d0577803e1a59
2021-05-11 09:01:13 +02:00
Simon Westphahl e6199985d1 Fix exception on build started event for noop jobs
Use an empty dictionary for the data of the BuildStartedEvent for noop
jobs instead of passing None. Otherwise the dictionary access will
fail when trying to set the worker info.

2021-05-03 13:20:50,287 zuul.Scheduler  ERROR  Exception in run handler:
Traceback (most recent call last):
  File "/home/westphahl/src/opendev/zuul/zuul/zuul/scheduler.py", line 1186, in run
    self.process_result_queue()
  File "/home/westphahl/src/opendev/zuul/zuul/zuul/scheduler.py", line 1441, in process_result_queue
    self._process_result_event(event)
  File "/home/westphahl/src/opendev/zuul/zuul/zuul/scheduler.py", line 1467, in _process_result_event
    self._doBuildStartedEvent(event)
  File "/home/westphahl/src/opendev/zuul/zuul/zuul/scheduler.py", line 1486, in _doBuildStartedEvent
    self.executor.setWorkerInfo(build, event.data)
  File "/home/westphahl/src/opendev/zuul/zuul/zuul/executor/client.py", line 302, in setWorkerInfo
    build.worker.updateFromData(data)
  File "/home/westphahl/src/opendev/zuul/zuul/zuul/model.py", line 2020, in updateFromData
    self.name = data.get('worker_name', self.name)
AttributeError: 'NoneType' object has no attribute 'get'

Change-Id: I09370e9b69b42cbc2d42f32b3c375c49961a8390
2021-05-03 13:21:16 +02:00
Felix Edel 9b5b90047f Switch to ZooKeeper backed build result events
This puts the build result events (BuildStarted, BuildStatus,
BuildPaused and BuildCompleted) into ZooKeeper.

This change moves most of the result event logic into the scheduler
class. Previously, the executor client did a lot before forwarding the
events to the scheduler. Now, the scheduler picks up the events from the
result event queues and does everything by its own before notifying the
pipeline manager.

To make this work, noop builds now get an entry in the local self.builds
list in the executor client. Like before, they are started and completed
immediately (by putting the corresponding events in the result event
queue) without acquiring a compute node.

To properly deal with the pending build results from tenants/pipelines
that were removed during a reconfiguration, we have to put some cleanup
logic into the Scheduler. This is necessary as the result events of
those builds cannot be processed anymore as the tenant/pipeline is no
longer part of the current abide.

The builds itself are still executed via Gearman, but the results aren't
looked up from the Gearman job anymore, but from the result queues in
ZooKeeper.

Change-Id: I067c38cec8ac14f3a3bb6f607ede943c88ce493d
2021-04-26 18:39:57 -07:00
Felix Edel 9674806662 Make buildset mandatory on build
With the introduction of type checking, many of the Optional attributes
must be checked in various places before accessing. Making the buildset
mandatory on the build removes some of those otherwise necessary
NoneType checks and will avoid more in future changes.

Change-Id: I70409551f0f786ad6ea9c4f2cf0ab0f9eb2d1555
2021-03-31 16:30:47 +02:00
Joshua Hesketh 7d424ee477 Get executor job params
Move the param generation for an execution job into a library for reuse.
The param preparation takes care of determining projects and connections
for dependant roles which saves a zuul-runner from needing to understand
canonical names for which it would need to query a scheduler.

Implement a basic API to freeze and grab these job params. These could
then be passed to another zuul-executor or other runner.

Change-Id: I681f2a3384c9a65ae0acc3fce966e8ec47005b64
Co-Authored-By: Tristan Cacqueray <tdecacqu@redhat.com>
2021-03-12 00:16:09 +00:00
Zuul 6098cb066c Merge "Defer setting build pause to event queue" 2021-02-27 00:37:11 +00:00
Albin Vass deece4d236 Add post_review pipeline variable to jobs
Change-Id: If70539e075663e12b21a5b76d8b1456e34c6dfec
2020-12-22 00:14:34 +01:00
Zuul bd68de673f Merge "Fix override variables in zuul_return" 2020-12-15 20:17:54 +00:00
Benedikt Loeffler 042514acdc
Fix override variables in zuul_return
When a job is retried, the variables set by the zuul_return are not
overwritten.

Change-Id: Ia78eac82a978740766ef5a3f5e5e32a02a8d4479
2020-12-14 15:13:03 +01:00
Tobias Henkel 313dcebb9f
Defer setting build pause to event queue
We currently set the paused flag on the build object directly when we
receive the notification from the executor and afterwards enqueue a
BuildPaused event to let the pipeline process the pause. As soon a the
build is marked as paused the pipeline processor will request
resources for the child jobs. However during processing of the
BuildPaused event it can be that the paused job requested to skip some
jobs. In this case we have a race between requesting the resources and
setting the child job results to SKIPPED. The result is that if the
resources are requested earlier they will be leaked because they will
never be released by the scheduler as no builds will be enqueued for
them.

This can be solved by deferring to set the paused flag to the event
queue which eliminates the race.

This is similar to the deferred result setting in
I0631b97e6ae2bf7d883c6c6a189e9d41d99ece5d.

Change-Id: Ie0fa8a2613fca65884d54b59273769f4984552cd
2020-10-12 19:52:32 +02:00
Benedikt Loeffler 99c6db0895
Report retried builds in a build set via mqtt.
If a build must be retried, the previous build information get lost.
To be informed that a build was retried, the retry builds are now part
of the mqtt message.

Change-Id: I8c93376f844c3d1c55c89a250384a7f835763677
Depends-On: https://review.opendev.org/704983
2020-04-09 11:57:29 +02:00
Simon Westphahl e00ce4ce4e Add Zuul's event id to Ansible inventory
With the event id in the inventory we can use it in the emit-job-header
role. This can aid in debugging, e.g. when a job is still running.

Change-Id: Id2d38bb3d121fa9cf959b9765ac8025d42001c02
2020-02-06 11:32:48 +01:00
Zuul 83175b3fe0 Merge "Fix canceling builds in starting phase" 2020-01-24 16:06:26 +00:00
Tobias Henkel 17f6b7cf75
Fix canceling builds in starting phase
Currently the algorithm for canceling is as follows:
1. check url of build, if exists abort and return
2. remove from gearman queue and return if successful
3. sleep a second
4. check url of build again

The url part of the build is used to determine if it has been started
on the executor. However the url is actually set just before the
ansible playbooks are started after all repo fetching and merging has
happened. This most of the time takes more than the one second wait
time in step three. This leaves a large window open where builds
cannot be canceled at all and will run completely.

This can be fixed by reporting the worker_name earlier just after the
job has been accepted and use the worker name instead of the url to
indicate if the job is running.

Change-Id: I37925c7a6207e8d92d47097c9d66200343b3d090
2019-12-13 12:52:07 +01:00
Clark Boylan 9f18243dab Record job build attempts in inventory
Record the number of attempts zuul has made to run this job in the job
inventory. This will help expose reliability information in job logs and
in job log indexers. We want to try and expose job reliability as much
as possible and this is one way to do that.

Change-Id: I2f7c31ce510f59bc569c5db89ce6626d9e3ef436
2019-12-06 09:26:00 -08:00
Tobias Henkel bb16d52991
Annotate cancelJobInQueue with build id
When debugging cancel issues the message from cancelJobInQueue easily
gets overlooked without the annotation.

Change-Id: Ic9198c0cdcee2d2cd7771dcc17448328a71dadd6
2019-09-24 17:08:05 +02:00
Zuul b741beb18f Merge "model: add cleanup-run to the job configuration" 2019-07-03 14:43:01 +00:00
Tobias Henkel a703f42c24
Annotate logs around build states
It's useful to have logs around build states annotated with the event
ids that caused it.

Change-Id: Ib2173210516823cfe16e34710e8d01319aba8cce
2019-05-30 19:18:01 +02:00
Tobias Henkel 6b37468a38
Annotate logs around build completion and cancellation
The logs around build completion and cancelations should be annotated
with the event that triggered the action.

Change-Id: Ifd039d0314a32647a0ec9ebe9bff064e503e0452
2019-05-30 19:18:00 +02:00
Tobias Henkel 6f3bcdd6b6
Annotate builds with event id
It's useful to be able to trace an event through the system including
the builds.

Change-Id: If852cbe8aecc4cf346dccc1b8fc34272c8ff483d
2019-05-30 19:18:00 +02:00
Tristan Cacqueray a4ccc9c3e5 model: add cleanup-run to the job configuration
This change adds a cleanup-run job attribute to specify a cleanup playbook to
be executed after the job execution.

Change-Id: Iebd93b2d6d13e1d4ccf2878e79e563cc9da2cce9
2019-05-29 05:42:48 +00:00
Tobias Henkel b0d7c3c69a
Support fail-fast in project pipelines
In some cases like resource constrained environments it is beneficial
to report on changes in a fail fast manner to immediately report if
one job fails. This can be useful especially if a project has many
expensive long-running jobs. This introduces a fail-fast flag in the
project pipeline that let's the project choose the trade off between
full information and quick feedback.

Change-Id: Ie4a5ac8e025362dbaacd3ae82f2e8369f7447a62
2019-04-29 08:26:58 +02:00
Tobias Henkel 5c2b61e638
Make ansible version configurable
Currently the default ansible version is selected by the version of
zuul itself. However we want to make this configurable per deployment
(zuul.conf), tenant and job.

Change-Id: Iccbb124ac7f7a8260c730fbc109ccfc1dec09f8b
2019-03-15 09:09:16 +01:00
James E. Blair 1317391323 Add provides/requires support
Adds support for expressing artifact dependencies between jobs
which may run in different projects.

Change-Id: If8cce8750d296d607841800e4bbf688a24c40e08
2019-01-30 14:07:42 -08:00
Zuul 91e7e680a1 Merge "Use gearman client keepalive" 2019-01-28 20:09:30 +00:00
Simon Westphahl 5b4c5299ad Add change message to Zuul vars in inventory
Having the change message available via the Zuul vars simplifies cases
where a job e.g. needs to update a GitHub/Jira/... ticket.

Those ticket numbers are usually referenced in the commit/PR message.

This avoids having to deal with secrets etc. to get this information
'out-of-band'.

Change-Id: Ib88db7f724dadfb8a4f86e76692f3e1c2c63a258
2019-01-17 08:45:14 +01:00