Add prune-database command

This adds a zuul-admin command which allows operators to delete old
database entries.

Change-Id: I4e277a07394aa4852a563f4c9cdc39b5801ab4ba
This commit is contained in:
James E. Blair 2022-05-28 17:34:08 -07:00
parent 326ede243f
commit 3ffbf10f25
9 changed files with 320 additions and 4 deletions

View File

@ -108,6 +108,15 @@ Example::
zuul-admin delete-pipeline-state tenant pipeline
prune-database
^^^^^^^^^^^^^^
.. program-output:: zuul-admin prune-database --help
Example::
zuul-admin prune-database --older-than 180d
Deprecated commands
-------------------
@ -256,4 +265,4 @@ ordering.
If items in independent pipelines are promoted, no jobs will be
restarted, but their change queues within the pipeline will be
re-ordered so that they will be processed first and their node request
priorities will increase.
priorities will increase.

View File

@ -0,0 +1,21 @@
---
features:
- |
A new command, ``prune-database`` has been added to `zuul-admin`
in order to remove database entries older than a certain age.
upgrade:
- |
A SQL schema migration is added in order to add an ``updated``
column to the buildset table which will be used by the
`prune-database` command to determine which buildsets to delete.
The migration will attempt to use the most recent timestamp
associated with the buildset when initializing this column,
however, if no timestamps are available, it will initialize it to
1970-01-01. Since this is considerably before Zuul's birthdate,
this means that any buildsets without timestamp information will
be pruned the first time `prune-database` is run. It is expected
that buildsets with no timestamps, even very recent ones, are
typically uninteresting and therefore this should not cause a
hardship. If this is not the case for your installation, you may
want to inspect the database and change the ``updated`` column to
a more recent value in these cases.

View File

@ -1,4 +1,5 @@
# Copyright 2018 Red Hat, Inc.
# 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
@ -17,18 +18,22 @@ import os
import sys
import subprocess
import time
import configparser
import datetime
import dateutil.tz
import fixtures
import jwt
import testtools
from kazoo.exceptions import NoNodeError
from zuul.zk import ZooKeeperClient
from zuul.cmd.client import parse_cutoff
from tests.base import BaseTestCase, ZuulTestCase
from tests.base import FIXTURE_DIR
from kazoo.exceptions import NoNodeError
class BaseClientTestCase(BaseTestCase):
config_file = 'zuul.conf'
@ -429,3 +434,118 @@ class TestOnlineZKOperations(ZuulTestCase):
dict(name='project-test1', result='SUCCESS', changes='2,1'),
dict(name='project-test2', result='SUCCESS', changes='2,1'),
], ordered=False)
class TestDBPruneParse(BaseTestCase):
def test_db_prune_parse(self):
now = datetime.datetime(year=2023, month=5, day=28,
hour=22, minute=15, second=1,
tzinfo=dateutil.tz.tzutc())
reference = datetime.datetime(year=2022, month=5, day=28,
hour=22, minute=15, second=1,
tzinfo=dateutil.tz.tzutc())
# Test absolute times
self.assertEqual(
reference,
parse_cutoff(now, '2022-05-28 22:15:01 UTC', None))
self.assertEqual(
reference,
parse_cutoff(now, '2022-05-28 22:15:01', None))
# Test relative times
self.assertEqual(reference,
parse_cutoff(now, None, '8760h'))
self.assertEqual(reference,
parse_cutoff(now, None, '365d'))
with testtools.ExpectedException(RuntimeError):
self.assertEqual(reference,
parse_cutoff(now, None, '1y'))
class DBPruneTestCase(ZuulTestCase):
tenant_config_file = 'config/single-tenant/main.yaml'
def _setup(self):
config_file = os.path.join(self.test_root, 'zuul.conf')
with open(config_file, 'w') as f:
self.config.write(f)
A = self.fake_gerrit.addFakeChange('org/project', 'master', 'A')
self.fake_gerrit.addEvent(A.getPatchsetCreatedEvent(1))
self.waitUntilSettled()
time.sleep(1)
B = self.fake_gerrit.addFakeChange('org/project', 'master', 'B')
self.fake_gerrit.addEvent(B.getPatchsetCreatedEvent(1))
self.waitUntilSettled()
connection = self.scheds.first.sched.sql.connection
buildsets = connection.getBuildsets()
builds = connection.getBuilds()
self.assertEqual(len(buildsets), 2)
self.assertEqual(len(builds), 6)
for build in builds:
self.log.debug("Build %s %s %s",
build, build.start_time, build.end_time)
return config_file
def test_db_prune_before(self):
# Test pruning buildsets before a specific date
config_file = self._setup()
connection = self.scheds.first.sched.sql.connection
# Builds are reverse ordered; 0 is most recent
buildsets = connection.getBuildsets()
start_time = buildsets[0].first_build_start_time
self.log.debug("Cutoff %s", start_time)
p = subprocess.Popen(
[os.path.join(sys.prefix, 'bin/zuul-admin'),
'-c', config_file,
'prune-database',
'--before', str(start_time),
],
stdout=subprocess.PIPE)
out, _ = p.communicate()
self.log.debug(out.decode('utf8'))
buildsets = connection.getBuildsets()
builds = connection.getBuilds()
self.assertEqual(len(buildsets), 1)
self.assertEqual(len(builds), 3)
for build in builds:
self.log.debug("Build %s %s %s",
build, build.start_time, build.end_time)
def test_db_prune_older_than(self):
# Test pruning buildsets older than a relative time
config_file = self._setup()
connection = self.scheds.first.sched.sql.connection
# We use 0d as the relative time here since the earliest we
# support is 1d and that's tricky in unit tests. The
# prune_before test handles verifying that we don't just
# always delete everything.
p = subprocess.Popen(
[os.path.join(sys.prefix, 'bin/zuul-admin'),
'-c', config_file,
'prune-database',
'--older-than', '0d',
],
stdout=subprocess.PIPE)
out, _ = p.communicate()
self.log.debug(out.decode('utf8'))
buildsets = connection.getBuildsets()
builds = connection.getBuilds()
self.assertEqual(len(buildsets), 0)
self.assertEqual(len(builds), 0)
class TestDBPruneMysql(DBPruneTestCase):
config_file = 'zuul-sql-driver-mysql.conf'
class TestDBPrunePostgres(DBPruneTestCase):
config_file = 'zuul-sql-driver-postgres.conf'

View File

@ -73,7 +73,7 @@ class TestSQLConnectionMysql(ZuulTestCase):
buildset_table = table_prefix + 'zuul_buildset'
build_table = table_prefix + 'zuul_build'
self.assertEqual(19, len(insp.get_columns(buildset_table)))
self.assertEqual(20, len(insp.get_columns(buildset_table)))
self.assertEqual(13, len(insp.get_columns(build_table)))
def test_sql_tables_created(self):

View File

@ -106,6 +106,7 @@ class TestMysqlDatabase(DBBaseTestCase):
connection.exec_driver_sql(f"drop table {table}")
connection.exec_driver_sql("set foreign_key_checks=1")
self.connection.force_migrations = True
self.connection._migrate('c57e9e76b812')
with self.connection.engine.begin() as connection:
connection.exec_driver_sql(
@ -122,6 +123,49 @@ class TestMysqlDatabase(DBBaseTestCase):
"select result from zuul_buildset")]
self.assertEqual(results, ['SUCCESS', 'MERGE_CONFLICT'])
def test_migration_c7467b642498(self):
with self.connection.engine.begin() as connection:
connection.exec_driver_sql("set foreign_key_checks=0")
for table in connection.exec_driver_sql("show tables"):
table = table[0]
connection.exec_driver_sql(f"drop table {table}")
connection.exec_driver_sql("set foreign_key_checks=1")
self.connection.force_migrations = True
self.connection._migrate('4647def24b32')
with self.connection.engine.begin() as connection:
connection.exec_driver_sql(
"insert into zuul_buildset (result) values ('SUCCESS')")
connection.exec_driver_sql(
"insert into zuul_buildset (result, first_build_start_time) "
"values ('SUCCESS', '2022-05-01 12:34:56')")
connection.exec_driver_sql(
"insert into zuul_buildset (result, last_build_end_time) "
"values ('SUCCESS', '2022-05-02 12:34:56')")
connection.exec_driver_sql(
"insert into zuul_buildset (result, event_timestamp) "
"values ('SUCCESS', '2022-05-03 12:34:56')")
connection.exec_driver_sql(
"insert into zuul_buildset (result, "
"first_build_start_time, "
"last_build_end_time, "
"event_timestamp)"
"values ('SUCCESS', "
"'2022-05-11 12:34:56', "
"'2022-05-12 12:34:56', "
"'2022-05-13 12:34:56')")
self.connection._migrate()
with self.connection.engine.begin() as connection:
results = [str(r[0]) for r in connection.exec_driver_sql(
"select updated from zuul_buildset")]
self.assertEqual(results,
['1970-01-01 00:00:00',
'2022-05-01 12:34:56',
'2022-05-02 12:34:56',
'2022-05-03 12:34:56',
'2022-05-13 12:34:56'])
def test_buildsets(self):
tenant = 'tenant1',
buildset_uuid = 'deadbeef'

View File

@ -1,5 +1,6 @@
# Copyright 2012 Hewlett-Packard Development Company, L.P.
# Copyright 2013 OpenStack Foundation
# 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
@ -16,6 +17,8 @@
import argparse
import babel.dates
import datetime
import dateutil.parser
import dateutil.tz
import json
import jwt
import logging
@ -40,6 +43,28 @@ from zuul.zk.layout import LayoutState, LayoutStateStore
from zuul.zk.components import COMPONENT_REGISTRY
def parse_cutoff(now, before, older_than):
if before and not older_than:
cutoff = dateutil.parser.parse(before)
if cutoff.tzinfo and cutoff.tzinfo != dateutil.tz.tzutc():
raise RuntimeError("Timestamp must be specified as UTC")
cutoff = cutoff.replace(tzinfo=dateutil.tz.tzutc())
return cutoff
elif older_than and not before:
value = older_than[:-1]
suffix = older_than[-1]
if suffix == 'd':
delta = datetime.timedelta(days=int(value))
elif suffix == 'h':
delta = datetime.timedelta(hours=int(value))
else:
raise RuntimeError("Unsupported relative time")
return now - delta
else:
raise RuntimeError(
"Either --before or --older-than must be supplied")
# todo This should probably live somewhere else
class ZuulRESTClient(object):
"""Basic client for Zuul's REST API"""
@ -498,6 +523,27 @@ class Client(zuul.cmd.ZuulApp):
help='tenant name')
cmd_delete_pipeline_state.add_argument('pipeline', type=str,
help='pipeline name')
# DB Maintenance
cmd_prune_database = subparsers.add_parser(
'prune-database',
help='prune old database entries',
formatter_class=argparse.RawDescriptionHelpFormatter,
description=textwrap.dedent('''\
Prune old database entries
This command will delete database entries older than the
specified cutoff (which can be specified as either an
absolute or relative time).'''))
cmd_prune_database.set_defaults(command='prune-database')
cmd_prune_database.add_argument(
'--before',
help='absolute timestamp (e.g., "2022-01-31 12:00:00")')
cmd_prune_database.add_argument(
'--older-than',
help='relative time (e.g., "24h" or "180d")')
cmd_prune_database.set_defaults(func=self.prune_database)
return parser
def parseArguments(self, args=None):
@ -1007,6 +1053,16 @@ class Client(zuul.cmd.ZuulApp):
sys.exit(0)
def prune_database(self):
logging.basicConfig(level=logging.INFO)
args = self.args
now = datetime.datetime.now(dateutil.tz.tzutc())
cutoff = parse_cutoff(now, args.before, args.older_than)
self.configure_connections(source_only=False, require_sql=True)
connection = self.connections.getSqlConnection()
connection.deleteBuildsets(cutoff)
sys.exit(0)
def main():
if sys.argv[0].endswith('zuul'):

View File

@ -0,0 +1,47 @@
# 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.
"""buildset_updated
Revision ID: c7467b642498
Revises: 4647def24b32
Create Date: 2022-05-28 16:21:50.035877
"""
# revision identifiers, used by Alembic.
revision = 'c7467b642498'
down_revision = '4647def24b32'
branch_labels = None
depends_on = None
from alembic import op
import sqlalchemy as sa
def upgrade(table_prefix=''):
op.add_column(
table_prefix + "zuul_buildset", sa.Column('updated', sa.DateTime))
connection = op.get_bind()
connection.execute(
"""
UPDATE {buildset_table}
SET updated=greatest(
coalesce(first_build_start_time, '1970-01-01 00:00:00'),
coalesce(last_build_end_time, '1970-01-01 00:00:00'),
coalesce(event_timestamp, '1970-01-01 00:00:00'))
""".format(buildset_table=table_prefix + "zuul_buildset"))
def downgrade():
raise Exception("Downgrades not supported")

View File

@ -173,6 +173,7 @@ class DatabaseSession(object):
def getBuildsets(self, tenant=None, project=None, pipeline=None,
change=None, branch=None, patchset=None, ref=None,
newrev=None, uuid=None, result=None, complete=None,
updated_max=None,
limit=50, offset=0, idx_min=None, idx_max=None):
buildset_table = self.connection.zuul_buildset_table
@ -202,6 +203,9 @@ class DatabaseSession(object):
elif complete is False:
q = q.filter(buildset_table.c.result == None) # noqa
if updated_max:
q = q.filter(buildset_table.c.updated < updated_max)
q = q.order_by(buildset_table.c.id.desc()).\
limit(limit).\
offset(offset)
@ -232,6 +236,13 @@ class DatabaseSession(object):
except sqlalchemy.orm.exc.MultipleResultsFound:
raise Exception("Multiple buildset found with uuid %s", uuid)
def deleteBuildsets(self, cutoff):
"""Delete buildsets before the cutoff"""
# delete buildsets updated before the cutoff
for buildset in self.getBuildsets(updated_max=cutoff):
self.session().delete(buildset)
class SQLConnection(BaseConnection):
driver_name = 'sql'
@ -349,6 +360,7 @@ class SQLConnection(BaseConnection):
event_timestamp = sa.Column(sa.DateTime, nullable=True)
first_build_start_time = sa.Column(sa.DateTime, nullable=True)
last_build_end_time = sa.Column(sa.DateTime, nullable=True)
updated = sa.Column(sa.DateTime, nullable=True)
sa.Index(self.table_prefix + 'project_pipeline_idx',
project, pipeline)
@ -472,3 +484,8 @@ class SQLConnection(BaseConnection):
"""Return a BuildSet objects"""
with self.getSession() as db:
return db.getBuildset(*args, **kw)
def deleteBuildsets(self, *args, **kw):
"""Delete buildsets"""
with self.getSession() as db:
return db.deleteBuildsets(*args, **kw)

View File

@ -68,6 +68,7 @@ class SQLReporter(BaseReporter):
ref_url=item.change.url,
event_id=event_id,
event_timestamp=event_timestamp,
updated=datetime.datetime.utcnow(),
)
return db_buildset
@ -111,6 +112,7 @@ class SQLReporter(BaseReporter):
and build.end_time > end_time):
end_time = build.end_time
db_buildset.last_build_end_time = end_time
db_buildset.updated = datetime.datetime.utcnow()
return
except sqlalchemy.exc.DBAPIError:
if retry_count < self.retry_count - 1: