zuul/zuul/zk/layout.py
James E. Blair 9a279725f9 Strictly sequence reconfiguration events
In the before times when we only had a single scheduler, it was
naturally the case that reconfiguration events were processed as they
were encountered and no trigger events which arrived after them would
be processed until the reconfiguration was complete.  As we added more
event queues to support SOS, it became possible for trigger events
which arrived at the scheduler to be processed before a tenant
reconfiguration caused by a preceding event to be complete.  This is
now even possible with a single scheduler.

As a concrete example, imagine a change merges which updates the jobs
which should run on a tag, and then a tag is created.  A scheduler
will process both of those events in succession.  The first will cause
it to submit a tenant reconfiguration event, and then forward the
trigger event to any matching pipelines.  The second event will also
be forwarded to pipeline event queues.  The pipeline events will then
be processed, and then only at that point will the scheduler return to
the start of the run loop and process the reconfiguration event.

To correct this, we can take one of two approaches: make the
reconfiguration more synchronous, or make it safer to be
asynchronous.  To make reconfiguration more synchronous, we would need
to be able to upgrade a tenant read lock into a tenant write lock
without releasing it.  The lock recipes we use from kazoo do not
support this.  While it would be possible to extend them to do so, it
would lead us further from parity with the upstream kazoo recipes, so
this aproach is not used.

Instead, we will make it safer for reconfiguration to be asynchronous
by annotating every trigger event we forward with the last
reconfiguration event that was seen before it.  This means that every
trigger event now specifies the minimum reconfiguration time for that
event.  If our local scheduler has not reached that time, we should
stop processing trigger events and wait for it to catch up.  This
means that schedulers may continue to process events up to the point
of a reconfiguration, but will then stop.  The already existing
short-circuit to abort processing once a scheduler is ready to
reconfigure a tenant (where we check the tenant write lock contenders
for a waiting reconfiguration) helps us get out of the way of pending
reconfigurations as well.  In short, once a reconfiguration is ready
to start, we won't start processing tenant events anymore because of
the existing lock check.  And up until that happens, we will process
as many events as possible until any further events require the
reconfiguration.

We will use the ltime of the tenant trigger event as our timestamp.
As we forward tenant trigger events to the pipeline trigger event
queues, we decide whether an event should cause a reconfiguration.
Whenever one does, we note the ltime of that event and store it as
metadata on the tenant trigger event queue so that we always know what
the most recent required minimum ltime is (ie, the ltime of the most
recently seen event that should cause a reconfiguration).  Every event
that we forward to the pipeline trigger queue will be annotated to
specify that its minimum required reconfiguration ltime is that most
recently seen ltime.  And each time we reconfigure a tenant, we store
the ltime of the event that prompted the reconfiguration in the layout
state.  If we later process a pipeline trigger event with a minimum
required reconfigure ltime greater than the current one, we know we
need to stop and wait for a reconfiguration, so we abort early.

Because this system involves several event queues and objects each of
which may be serialized at any point during a rolling upgrade, every
involved object needs to have appropriate default value handling, and
a synchronized model api change is not helpful.  The remainder of this
commit message is a description of what happens with each object when
handled by either an old or new scheduler component during a rolling
upgrade.

When forwarding a trigger event and submitting a tenant
reconfiguration event:

The tenant trigger event zuul_event_ltime is initialized
from zk, so will always have a value.

The pipeline management event trigger_event_ltime is initialzed to the
tenant trigger event zuul_event_ltime, so a new scheduler will write
out the value.  If an old scheduler creates the tenant reconfiguration
event, it will be missing the trigger_event_ltime.

The _reconfigureTenant method is called with a
last_reconfigure_event_ltime parameter, which is either the
trigger_event_ltime above in the case of a tenant reconfiguration
event forwarded by a new scheduler, or -1 in all other cases
(including other types of reconfiguration, or a tenant reconfiguration
event forwarded by an old scheduler).  If it is -1, it will use the
current ltime so that if we process an event from an old scheduler
which is missing the event ltime, or we are bootstrapping a tenant or
otherwise reconfiguring in a context where we don't have a triggering
event ltime, we will use an ltime which is very new so that we don't
defer processing trigger events.  We also ensure we never go backward,
so that if we process an event from an old scheduler (and thus use the
current ltime) then process an event from a new scheduler with an
older (than "now") ltime, we retain the newer ltime.

Each time a tenant reconfiguration event is submitted, the ltime of
that reconfiguration event is stored on the trigger event queue.  This
is then used as the min_reconfigure_ltime attribute on the forwarded
trigger events.  This is updated by new schedulers, and ignored by old
ones, so if an old scheduler process a tenant trigger event queue it
won't update the min ltime.  That will just mean that any events
processed by a new scheduler may continue to use an older ltime as
their minimum, which should not cause a problem.  Any events forwarded
by an old scheduler will omit the min_reconfigure_ltime field; that
field will be initialized to -1 when loaded on a new scheduler.

When processing pipeline trigger events:

In process_pipeline_trigger_queue we compare two values: the
last_reconfigure_event_ltime on the layout state which is either set
to a value as above (by a new scheduler), or will be -1 if it was last
written by an old scheduler (including in the case it was overwritten
by an old scheduler; it will re-initialize to -1 in that case).  The
event.min_reconfigure_ltime field will either be the most recent
reconfiguration ltime seen by a new scheduler forwarding trigger
events, or -1 otherwise.  If the min_reconfigure_ltime of an event is
-1, we retain the old behavior of processing the event regardless.
Only if we have a min_reconfigure_ltime > -1 and it is greater than
the layout state last_reconfigure_event_ltime (which itself may be -1,
and thus less than the min_reconfigure_ltime) do we abort processing
the event.

(The test_config_update test for the Gerrit checks plugin is updated
to include an extra waitUntilSettled since a potential test race was
observed during development.)

Change-Id: Icb6a7858591ab867e7006c7c80bfffeb582b28ee
2022-07-18 10:51:59 -07:00

222 lines
7.6 KiB
Python

# Copyright 2020 BMW Group
# Copyright 2022 Acme Gating, LLC
#
# Licensed under the Apache License, Version 2.0 (the "License"); you may
# not use this file except in compliance with the License. You may obtain
# a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
# License for the specific language governing permissions and limitations
# under the License.
import json
from collections.abc import MutableMapping
from contextlib import suppress
from functools import total_ordering
import logging
import time
from kazoo.exceptions import NoNodeError
from zuul.zk.components import COMPONENT_REGISTRY
from zuul.zk import sharding, ZooKeeperBase
@total_ordering
class LayoutState:
"""Representation of a tenant's layout state.
The layout state holds information about a certain version of a
tenant's layout. It is used to coordinate reconfigurations across
multiple schedulers by comparing a local tenant layout state
against the current version in Zookeeper. In case it detects that
a local layout state is outdated, this scheduler is not allowed to
process this tenant (events, pipelines, ...) until the layout is
updated.
The important information of the layout state is the logical
timestamp (ltime) that is used to detect if the layout on a
scheduler needs to be updated. The ltime is the last modified
transaction ID (mzxid) of the corresponding Znode in Zookeeper.
The hostname of the scheduler creating the new layout state and the
timestamp of the last reconfiguration are only informational and
may aid in debugging.
"""
def __init__(self, tenant_name, hostname, last_reconfigured, uuid,
branch_cache_min_ltimes, last_reconfigure_event_ltime,
ltime=-1):
self.uuid = uuid
self.ltime = ltime
self.tenant_name = tenant_name
self.hostname = hostname
self.last_reconfigured = last_reconfigured
self.last_reconfigure_event_ltime =\
last_reconfigure_event_ltime
self.branch_cache_min_ltimes = branch_cache_min_ltimes
def toDict(self):
return {
"tenant_name": self.tenant_name,
"hostname": self.hostname,
"last_reconfigured": self.last_reconfigured,
"last_reconfigure_event_ltime":
self.last_reconfigure_event_ltime,
"uuid": self.uuid,
"branch_cache_min_ltimes": self.branch_cache_min_ltimes,
}
@classmethod
def fromDict(cls, data):
return cls(
data["tenant_name"],
data["hostname"],
data["last_reconfigured"],
data.get("uuid"),
data.get("branch_cache_min_ltimes"),
data.get("last_reconfigure_event_ltime", -1),
data.get("ltime", -1),
)
def __eq__(self, other):
if not isinstance(other, LayoutState):
return False
return self.uuid == other.uuid
def __gt__(self, other):
if not isinstance(other, LayoutState):
return False
return self.ltime > other.ltime
def __repr__(self):
return (
f"<{self.__class__.__name__} {self.tenant_name}: "
f"ltime={self.ltime}, "
f"hostname={self.hostname}, "
f"last_reconfigured={self.last_reconfigured}>"
)
class LayoutStateStore(ZooKeeperBase, MutableMapping):
log = logging.getLogger("zuul.LayoutStore")
layout_root = "/zuul/layout"
layout_data_root = "/zuul/layout-data"
def __init__(self, client, callback):
super().__init__(client)
self._watched_tenants = set()
self._callback = callback
self.kazoo_client.ensure_path(self.layout_root)
self.kazoo_client.ChildrenWatch(self.layout_root, self._layoutCallback)
def _layoutCallback(self, tenant_list, event=None):
new_tenants = set(tenant_list) - self._watched_tenants
for tenant_name in new_tenants:
self.kazoo_client.DataWatch(f"{self.layout_root}/{tenant_name}",
self._callbackWrapper)
def _callbackWrapper(self, data, stat, event):
self._callback()
def __getitem__(self, tenant_name):
try:
data, zstat = self.kazoo_client.get(
f"{self.layout_root}/{tenant_name}")
except NoNodeError:
raise KeyError(tenant_name)
if not data:
raise KeyError(tenant_name)
return LayoutState.fromDict({
"ltime": zstat.last_modified_transaction_id,
**json.loads(data)
})
def __setitem__(self, tenant_name, state):
path = f"{self.layout_root}/{tenant_name}"
data = json.dumps(state.toDict(), sort_keys=True).encode("utf-8")
if self.kazoo_client.exists(path):
zstat = self.kazoo_client.set(path, data)
else:
_, zstat = self.kazoo_client.create(path, data, include_data=True)
# Set correct ltime of the layout in Zookeeper
state.ltime = zstat.last_modified_transaction_id
def __delitem__(self, tenant_name):
try:
self.kazoo_client.delete(f"{self.layout_root}/{tenant_name}")
except NoNodeError:
raise KeyError(tenant_name)
def __iter__(self):
try:
tenant_names = self.kazoo_client.get_children(self.layout_root)
except NoNodeError:
return
yield from tenant_names
def __len__(self):
zstat = self.kazoo_client.exists(self.layout_root)
if zstat is None:
return 0
return zstat.children_count
def getMinLtimes(self, layout_state):
if COMPONENT_REGISTRY.model_api < 6:
return None
try:
path = f"{self.layout_data_root}/{layout_state.uuid}"
with sharding.BufferedShardReader(
self.kazoo_client, path) as stream:
data = stream.read()
except NoNodeError:
return None
try:
return json.loads(data)['min_ltimes']
except Exception:
return None
def setMinLtimes(self, layout_state, min_ltimes):
data = dict(min_ltimes=min_ltimes)
encoded_data = json.dumps(data).encode("utf-8")
path = f"{self.layout_data_root}/{layout_state.uuid}"
with sharding.BufferedShardWriter(
self.kazoo_client, path) as stream:
stream.write(encoded_data)
def cleanup(self, delay=300):
self.log.debug("Starting layout data cleanup")
known_layouts = set()
for tenant in self.kazoo_client.get_children(
self.layout_root):
state = self.get(tenant)
if state:
known_layouts.add(state.uuid)
for layout_uuid in self.kazoo_client.get_children(
self.layout_data_root):
if layout_uuid in known_layouts:
continue
path = f"{self.layout_data_root}/{layout_uuid}"
zstat = self.kazoo_client.exists(path)
if zstat is None:
continue
now = time.time()
if now - zstat.created >= delay:
self.log.debug("Deleting unused layout data for %s",
layout_uuid)
with suppress(NoNodeError):
self.kazoo_client.delete(path, recursive=True)
self.log.debug("Finished layout data cleanup")