diff --git a/sql/updates/15.sql b/sql/updates/15.sql new file mode 100644 index 0000000..75dd44a --- /dev/null +++ b/sql/updates/15.sql @@ -0,0 +1,23 @@ +insert into dbversion (version, release, description) + values (15, now(), 'Work In Progress'); + +create or replace function swh_scheduler_task_to_archive( + ts_after timestamptz, ts_before timestamptz, last_id bigint default -1, + lim bigint default 10) + returns setof task_record + language sql stable +as $$ + select t.id as task_id, t.policy as task_policy, + t.status as task_status, tr.id as task_run_id, + t.arguments, t.type, tr.backend_id, tr.metadata, + tr.scheduled, tr.started, tr.ended, tr.status as task_run_status + from task_run tr inner join task t on tr.task=t.id + where ((t.policy = 'oneshot' and t.status in ('completed', 'disabled')) or + (t.policy = 'recurring' and t.status = 'disabled')) and + ((ts_after <= tr.started and tr.started < ts_before) or tr.started is null) and + t.id >= last_id + order by tr.task, tr.started + limit lim; +$$; + +comment on function swh_scheduler_task_to_archive (timestamptz, timestamptz, bigint, bigint) is 'Read archivable tasks function'; diff --git a/swh/scheduler/backend.py b/swh/scheduler/backend.py index f57ca46..30ce479 100644 --- a/swh/scheduler/backend.py +++ b/swh/scheduler/backend.py @@ -1,493 +1,498 @@ -# Copyright (C) 2015-2018 The Software Heritage developers +# Copyright (C) 2015-2019 The Software Heritage developers # See the AUTHORS file at the top-level directory of this distribution # License: GNU General Public License version 3, or any later version # See top-level LICENSE file for more information import json import logging from arrow import Arrow, utcnow import psycopg2.pool import psycopg2.extras + +from typing import Any, Dict from psycopg2.extensions import AsIs from swh.core.db import BaseDb -from swh.core.db.common import db_transaction, db_transaction_generator +from swh.core.db.common import db_transaction logger = logging.getLogger(__name__) def adapt_arrow(arrow): return AsIs("'%s'::timestamptz" % arrow.isoformat()) psycopg2.extensions.register_adapter(dict, psycopg2.extras.Json) psycopg2.extensions.register_adapter(Arrow, adapt_arrow) def format_query(query, keys): """Format a query with the given keys""" query_keys = ', '.join(keys) placeholders = ', '.join(['%s'] * len(keys)) return query.format(keys=query_keys, placeholders=placeholders) class SchedulerBackend: """Backend for the Software Heritage scheduling database. """ def __init__(self, db, min_pool_conns=1, max_pool_conns=10): """ Args: db_conn: either a libpq connection string, or a psycopg2 connection """ if isinstance(db, psycopg2.extensions.connection): self._pool = None self._db = BaseDb(db) else: self._pool = psycopg2.pool.ThreadedConnectionPool( min_pool_conns, max_pool_conns, db, cursor_factory=psycopg2.extras.RealDictCursor, ) self._db = None def get_db(self): if self._db: return self._db return BaseDb.from_pool(self._pool) def put_db(self, db): if db is not self._db: db.put_conn() task_type_keys = [ 'type', 'description', 'backend_name', 'default_interval', 'min_interval', 'max_interval', 'backoff_factor', 'max_queue_length', 'num_retries', 'retry_delay', ] @db_transaction() def create_task_type(self, task_type, db=None, cur=None): """Create a new task type ready for scheduling. Args: task_type (dict): a dictionary with the following keys: - type (str): an identifier for the task type - description (str): a human-readable description of what the task does - backend_name (str): the name of the task in the job-scheduling backend - default_interval (datetime.timedelta): the default interval between two task runs - min_interval (datetime.timedelta): the minimum interval between two task runs - max_interval (datetime.timedelta): the maximum interval between two task runs - backoff_factor (float): the factor by which the interval changes at each run - max_queue_length (int): the maximum length of the task queue for this task type """ keys = [key for key in self.task_type_keys if key in task_type] query = format_query( """insert into task_type ({keys}) values ({placeholders})""", keys) cur.execute(query, [task_type[key] for key in keys]) @db_transaction() def get_task_type(self, task_type_name, db=None, cur=None): """Retrieve the task type with id task_type_name""" query = format_query( "select {keys} from task_type where type=%s", self.task_type_keys, ) cur.execute(query, (task_type_name,)) return cur.fetchone() @db_transaction() def get_task_types(self, db=None, cur=None): """Retrieve all registered task types""" query = format_query( "select {keys} from task_type", self.task_type_keys, ) cur.execute(query) return cur.fetchall() task_create_keys = [ 'type', 'arguments', 'next_run', 'policy', 'status', 'retries_left', 'priority' ] task_keys = task_create_keys + ['id', 'current_interval'] @db_transaction() def create_tasks(self, tasks, policy='recurring', db=None, cur=None): """Create new tasks. Args: tasks (list): each task is a dictionary with the following keys: - type (str): the task type - arguments (dict): the arguments for the task runner, keys: - args (list of str): arguments - kwargs (dict str -> str): keyword arguments - next_run (datetime.datetime): the next scheduled run for the task Returns: a list of created tasks. """ cur.execute('select swh_scheduler_mktemp_task()') db.copy_to(tasks, 'tmp_task', self.task_create_keys, default_values={ 'policy': policy, 'status': 'next_run_not_scheduled' }, cur=cur) query = format_query( 'select {keys} from swh_scheduler_create_tasks_from_temp()', self.task_keys, ) cur.execute(query) return cur.fetchall() @db_transaction() def set_status_tasks(self, task_ids, status='disabled', next_run=None, db=None, cur=None): """Set the tasks' status whose ids are listed. If given, also set the next_run date. """ if not task_ids: return query = ["UPDATE task SET status = %s"] args = [status] if next_run: query.append(', next_run = %s') args.append(next_run) query.append(" WHERE id IN %s") args.append(tuple(task_ids)) cur.execute(''.join(query), args) @db_transaction() def disable_tasks(self, task_ids, db=None, cur=None): """Disable the tasks whose ids are listed.""" return self.set_status_tasks(task_ids, db=db, cur=cur) @db_transaction() def search_tasks(self, task_id=None, task_type=None, status=None, priority=None, policy=None, before=None, after=None, limit=None, db=None, cur=None): """Search tasks from selected criterions""" where = [] args = [] if task_id: if isinstance(task_id, (str, int)): where.append('id = %s') else: where.append('id in %s') task_id = tuple(task_id) args.append(task_id) if task_type: if isinstance(task_type, str): where.append('type = %s') else: where.append('type in %s') task_type = tuple(task_type) args.append(task_type) if status: if isinstance(status, str): where.append('status = %s') else: where.append('status in %s') status = tuple(status) args.append(status) if priority: if isinstance(priority, str): where.append('priority = %s') else: priority = tuple(priority) where.append('priority in %s') args.append(priority) if policy: where.append('policy = %s') args.append(policy) if before: where.append('next_run <= %s') args.append(before) if after: where.append('next_run >= %s') args.append(after) query = 'select * from task' if where: query += ' where ' + ' and '.join(where) if limit: query += ' limit %s :: bigint' args.append(limit) cur.execute(query, args) return cur.fetchall() @db_transaction() def get_tasks(self, task_ids, db=None, cur=None): """Retrieve the info of tasks whose ids are listed.""" query = format_query('select {keys} from task where id in %s', self.task_keys) cur.execute(query, (tuple(task_ids),)) return cur.fetchall() @db_transaction() def peek_ready_tasks(self, task_type, timestamp=None, num_tasks=None, num_tasks_priority=None, db=None, cur=None): """Fetch the list of ready tasks Args: task_type (str): filtering task per their type timestamp (datetime.datetime): peek tasks that need to be executed before that timestamp num_tasks (int): only peek at num_tasks tasks (with no priority) num_tasks_priority (int): only peek at num_tasks_priority tasks (with priority) Returns: a list of tasks """ if timestamp is None: timestamp = utcnow() cur.execute( '''select * from swh_scheduler_peek_ready_tasks( %s, %s, %s :: bigint, %s :: bigint)''', (task_type, timestamp, num_tasks, num_tasks_priority) ) logger.debug('PEEK %s => %s' % (task_type, cur.rowcount)) return cur.fetchall() @db_transaction() def grab_ready_tasks(self, task_type, timestamp=None, num_tasks=None, num_tasks_priority=None, db=None, cur=None): """Fetch the list of ready tasks, and mark them as scheduled Args: task_type (str): filtering task per their type timestamp (datetime.datetime): grab tasks that need to be executed before that timestamp num_tasks (int): only grab num_tasks tasks (with no priority) num_tasks_priority (int): only grab oneshot num_tasks tasks (with priorities) Returns: a list of tasks """ if timestamp is None: timestamp = utcnow() cur.execute( '''select * from swh_scheduler_grab_ready_tasks( %s, %s, %s :: bigint, %s :: bigint)''', (task_type, timestamp, num_tasks, num_tasks_priority) ) logger.debug('GRAB %s => %s' % (task_type, cur.rowcount)) return cur.fetchall() task_run_create_keys = ['task', 'backend_id', 'scheduled', 'metadata'] @db_transaction() def schedule_task_run(self, task_id, backend_id, metadata=None, timestamp=None, db=None, cur=None): """Mark a given task as scheduled, adding a task_run entry in the database. Args: task_id (int): the identifier for the task being scheduled backend_id (str): the identifier of the job in the backend metadata (dict): metadata to add to the task_run entry timestamp (datetime.datetime): the instant the event occurred Returns: a fresh task_run entry """ if metadata is None: metadata = {} if timestamp is None: timestamp = utcnow() cur.execute( 'select * from swh_scheduler_schedule_task_run(%s, %s, %s, %s)', (task_id, backend_id, metadata, timestamp) ) return cur.fetchone() @db_transaction() def mass_schedule_task_runs(self, task_runs, db=None, cur=None): """Schedule a bunch of task runs. Args: task_runs (list): a list of dicts with keys: - task (int): the identifier for the task being scheduled - backend_id (str): the identifier of the job in the backend - metadata (dict): metadata to add to the task_run entry - scheduled (datetime.datetime): the instant the event occurred Returns: None """ cur.execute('select swh_scheduler_mktemp_task_run()') db.copy_to(task_runs, 'tmp_task_run', self.task_run_create_keys, cur=cur) cur.execute('select swh_scheduler_schedule_task_run_from_temp()') @db_transaction() def start_task_run(self, backend_id, metadata=None, timestamp=None, db=None, cur=None): """Mark a given task as started, updating the corresponding task_run entry in the database. Args: backend_id (str): the identifier of the job in the backend metadata (dict): metadata to add to the task_run entry timestamp (datetime.datetime): the instant the event occurred Returns: the updated task_run entry """ if metadata is None: metadata = {} if timestamp is None: timestamp = utcnow() cur.execute( 'select * from swh_scheduler_start_task_run(%s, %s, %s)', (backend_id, metadata, timestamp) ) return cur.fetchone() @db_transaction() def end_task_run(self, backend_id, status, metadata=None, timestamp=None, result=None, db=None, cur=None): """Mark a given task as ended, updating the corresponding task_run entry in the database. Args: backend_id (str): the identifier of the job in the backend status (str): how the task ended; one of: 'eventful', 'uneventful', 'failed' metadata (dict): metadata to add to the task_run entry timestamp (datetime.datetime): the instant the event occurred Returns: the updated task_run entry """ if metadata is None: metadata = {} if timestamp is None: timestamp = utcnow() cur.execute( 'select * from swh_scheduler_end_task_run(%s, %s, %s, %s)', (backend_id, status, metadata, timestamp) ) return cur.fetchone() - @db_transaction_generator() - def filter_task_to_archive(self, after_ts, before_ts, limit=10, last_id=-1, - db=None, cur=None): - """Returns the list of task/task_run prior to a given date to archive. + @db_transaction() + def filter_task_to_archive( + self, after_ts: str, before_ts: str, limit: int = 10, + last_id: int = -1, db=None, cur=None) -> Dict[str, Any]: + """Compute the tasks to archive between after_ts and before_ts interval. + The result is paginated """ - last_task_run_id = None - while True: - row = None - cur.execute( - "select * from swh_scheduler_task_to_archive(%s, %s, %s, %s)", - (after_ts, before_ts, last_id, limit) - ) - for row in cur: - # nested type index does not accept bare values - # transform it as a dict to comply with this - row['arguments']['args'] = { - i: v for i, v in enumerate(row['arguments']['args']) - } - kwargs = row['arguments']['kwargs'] - row['arguments']['kwargs'] = json.dumps(kwargs) - yield row - - if not row: - break - _id = row.get('task_id') - _task_run_id = row.get('task_run_id') - if last_id == _id and last_task_run_id == _task_run_id: - break - last_id = _id - last_task_run_id = _task_run_id + tasks = [] + cur.execute( + "select * from swh_scheduler_task_to_archive(%s, %s, %s, %s)", + (after_ts, before_ts, last_id, limit + 1) + ) + for row in cur: + # nested type index does not accept bare values + # transform it as a dict to comply with this + row['arguments']['args'] = { + i: v for i, v in enumerate(row['arguments']['args']) + } + kwargs = row['arguments']['kwargs'] + row['arguments']['kwargs'] = json.dumps(kwargs) + tasks.append(row) + + if len(tasks) >= limit + 1: # remains data, add pagination information + result = { + 'tasks': tasks[:limit], + 'next_task_id': tasks[-1]['task_id'], + } + else: + result = { + 'tasks': tasks + } + + return result @db_transaction() def delete_archived_tasks(self, task_ids, db=None, cur=None): """Delete archived tasks as much as possible. Only the task_ids whose complete associated task_run have been cleaned up will be. """ _task_ids = _task_run_ids = [] for task_id in task_ids: _task_ids.append(task_id['task_id']) _task_run_ids.append(task_id['task_run_id']) cur.execute( "select * from swh_scheduler_delete_archived_tasks(%s, %s)", (_task_ids, _task_run_ids)) task_run_keys = ['id', 'task', 'backend_id', 'scheduled', 'started', 'ended', 'metadata', 'status', ] @db_transaction() def get_task_runs(self, task_ids, limit=None, db=None, cur=None): """Search task run for a task id""" where = [] args = [] if task_ids: if isinstance(task_ids, (str, int)): where.append('task = %s') else: where.append('task in %s') task_ids = tuple(task_ids) args.append(task_ids) else: return () query = 'select * from task_run where ' + ' and '.join(where) if limit: query += ' limit %s :: bigint' args.append(limit) cur.execute(query, args) return cur.fetchall() @db_transaction() def get_priority_ratios(self, db=None, cur=None): cur.execute('select id, ratio from priority_ratio') return {row['id']: row['ratio'] for row in cur.fetchall()} diff --git a/swh/scheduler/cli/task.py b/swh/scheduler/cli/task.py index 24ff757..36145d2 100644 --- a/swh/scheduler/cli/task.py +++ b/swh/scheduler/cli/task.py @@ -1,567 +1,571 @@ # Copyright (C) 2016-2019 The Software Heritage developers # See the AUTHORS file at the top-level directory of this distribution # License: GNU General Public License version 3, or any later version # See top-level LICENSE file for more information import datetime import json import itertools import locale import logging import arrow import csv import click from . import cli locale.setlocale(locale.LC_ALL, '') ARROW_LOCALE = locale.getlocale(locale.LC_TIME)[0] class DateTimeType(click.ParamType): name = 'time and date' def convert(self, value, param, ctx): if not isinstance(value, arrow.Arrow): value = arrow.get(value) return value DATETIME = DateTimeType() CONTEXT_SETTINGS = dict(help_option_names=['-h', '--help']) def format_dict(d): ret = {} for k, v in d.items(): if isinstance(v, (arrow.Arrow, datetime.date, datetime.datetime)): v = arrow.get(v).format() elif isinstance(v, dict): v = format_dict(v) ret[k] = v return ret def pretty_print_list(list, indent=0): """Pretty-print a list""" return ''.join('%s%r\n' % (' ' * indent, item) for item in list) def pretty_print_dict(dict, indent=0): """Pretty-print a list""" return ''.join('%s%s: %r\n' % (' ' * indent, click.style(key, bold=True), value) for key, value in sorted(dict.items())) def pretty_print_run(run, indent=4): fmt = ('{indent}{backend_id} [{status}]\n' '{indent} scheduled: {scheduled} [{started}:{ended}]') return fmt.format(indent=' '*indent, **format_dict(run)) def pretty_print_task(task, full=False): """Pretty-print a task If 'full' is True, also print the status and priority fields. >>> task = { ... 'id': 1234, ... 'arguments': { ... 'args': ['foo', 'bar', True], ... 'kwargs': {'key': 'value', 'key2': 42}, ... }, ... 'current_interval': datetime.timedelta(hours=1), ... 'next_run': datetime.datetime(2019, 2, 21, 13, 52, 35, 407818), ... 'policy': 'oneshot', ... 'priority': None, ... 'status': 'next_run_not_scheduled', ... 'type': 'test_task', ... } >>> print(click.unstyle(pretty_print_task(task))) Task 1234 Next run: ... (2019-02-21 13:52:35+00:00) Interval: 1:00:00 Type: test_task Policy: oneshot Args: 'foo' 'bar' True Keyword args: key: 'value' key2: 42 >>> print(click.unstyle(pretty_print_task(task, full=True))) Task 1234 Next run: ... (2019-02-21 13:52:35+00:00) Interval: 1:00:00 Type: test_task Policy: oneshot Status: next_run_not_scheduled Priority:\x20 Args: 'foo' 'bar' True Keyword args: key: 'value' key2: 42 """ next_run = arrow.get(task['next_run']) lines = [ '%s %s\n' % (click.style('Task', bold=True), task['id']), click.style(' Next run: ', bold=True), "%s (%s)" % (next_run.humanize(locale=ARROW_LOCALE), next_run.format()), '\n', click.style(' Interval: ', bold=True), str(task['current_interval']), '\n', click.style(' Type: ', bold=True), task['type'] or '', '\n', click.style(' Policy: ', bold=True), task['policy'] or '', '\n', ] if full: lines += [ click.style(' Status: ', bold=True), task['status'] or '', '\n', click.style(' Priority: ', bold=True), task['priority'] or '', '\n', ] lines += [ click.style(' Args:\n', bold=True), pretty_print_list(task['arguments']['args'], indent=4), click.style(' Keyword args:\n', bold=True), pretty_print_dict(task['arguments']['kwargs'], indent=4), ] return ''.join(lines) @cli.group('task') @click.pass_context def task(ctx): """Manipulate tasks.""" pass @task.command('schedule') @click.option('--columns', '-c', multiple=True, default=['type', 'args', 'kwargs', 'next_run'], type=click.Choice([ 'type', 'args', 'kwargs', 'policy', 'next_run']), help='columns present in the CSV file') @click.option('--delimiter', '-d', default=',') @click.argument('file', type=click.File(encoding='utf-8')) @click.pass_context def schedule_tasks(ctx, columns, delimiter, file): """Schedule tasks from a CSV input file. The following columns are expected, and can be set through the -c option: - type: the type of the task to be scheduled (mandatory) - args: the arguments passed to the task (JSON list, defaults to an empty list) - kwargs: the keyword arguments passed to the task (JSON object, defaults to an empty dict) - next_run: the date at which the task should run (datetime, defaults to now) The CSV can be read either from a named file, or from stdin (use - as filename). Use sample: cat scheduling-task.txt | \ python3 -m swh.scheduler.cli \ --database 'service=swh-scheduler-dev' \ task schedule \ --columns type --columns kwargs --columns policy \ --delimiter ';' - """ tasks = [] now = arrow.utcnow() scheduler = ctx.obj['scheduler'] if not scheduler: raise ValueError('Scheduler class (local/remote) must be instantiated') reader = csv.reader(file, delimiter=delimiter) for line in reader: task = dict(zip(columns, line)) args = json.loads(task.pop('args', '[]')) kwargs = json.loads(task.pop('kwargs', '{}')) task['arguments'] = { 'args': args, 'kwargs': kwargs, } task['next_run'] = DATETIME.convert(task.get('next_run', now), None, None) tasks.append(task) created = scheduler.create_tasks(tasks) output = [ 'Created %d tasks\n' % len(created), ] for task in created: output.append(pretty_print_task(task)) click.echo_via_pager('\n'.join(output)) @task.command('add') @click.argument('type', nargs=1, required=True) @click.argument('options', nargs=-1) @click.option('--policy', '-p', default='recurring', type=click.Choice(['recurring', 'oneshot'])) @click.option('--priority', '-P', default=None, type=click.Choice(['low', 'normal', 'high'])) @click.option('--next-run', '-n', default=None) @click.pass_context def schedule_task(ctx, type, options, policy, priority, next_run): """Schedule one task from arguments. The first argument is the name of the task type, further ones are positional and keyword argument(s) of the task, in YAML format. Keyword args are of the form key=value. Usage sample: swh-scheduler --database 'service=swh-scheduler' \ task add list-pypi swh-scheduler --database 'service=swh-scheduler' \ task add list-debian-distribution --policy=oneshot distribution=stretch Note: if the priority is not given, the task won't have the priority set, which is considered as the lowest priority level. """ from .utils import parse_options scheduler = ctx.obj['scheduler'] if not scheduler: raise ValueError('Scheduler class (local/remote) must be instantiated') now = arrow.utcnow() (args, kw) = parse_options(options) task = {'type': type, 'policy': policy, 'priority': priority, 'arguments': { 'args': args, 'kwargs': kw, }, 'next_run': DATETIME.convert(next_run or now, None, None), } created = scheduler.create_tasks([task]) output = [ 'Created %d tasks\n' % len(created), ] for task in created: output.append(pretty_print_task(task)) click.echo('\n'.join(output)) @task.command('schedule_origins') @click.argument('type', nargs=1, required=True) @click.argument('options', nargs=-1) @click.option('--batch-size', '-b', 'origin_batch_size', default=10, show_default=True, type=int, help="Number of origins per task") @click.option('--min-id', default=0, show_default=True, type=int, help="Only schedule tasks for origins whose ID is greater") @click.option('--max-id', default=None, type=int, help="Only schedule tasks for origins whose ID is lower") @click.option('--storage-url', '-g', help="URL of the (graph) storage API") @click.option('--dry-run/--no-dry-run', is_flag=True, default=False, help='List only what would be scheduled.') @click.pass_context def schedule_origin_metadata_index( ctx, type, options, storage_url, origin_batch_size, min_id, max_id, dry_run): """Schedules tasks for origins that are already known. The first argument is the name of the task type, further ones are keyword argument(s) of the task in the form key=value, where value is in YAML format. Usage sample: swh-scheduler --database 'service=swh-scheduler' \ task schedule_origins index-origin-metadata """ from swh.storage import get_storage from swh.storage.algos.origin import iter_origins from .utils import parse_options, schedule_origin_batches scheduler = ctx.obj['scheduler'] storage = get_storage('remote', {'url': storage_url}) if dry_run: scheduler = None (args, kw) = parse_options(options) if args: raise click.ClickException('Only keywords arguments are allowed.') origins = iter_origins(storage, origin_from=min_id, origin_to=max_id) origin_urls = (origin['url'] for origin in origins) schedule_origin_batches( scheduler, type, origin_urls, origin_batch_size, kw) @task.command('list-pending') @click.argument('task-types', required=True, nargs=-1) @click.option('--limit', '-l', required=False, type=click.INT, help='The maximum number of tasks to fetch') @click.option('--before', '-b', required=False, type=DATETIME, help='List all jobs supposed to run before the given date') @click.pass_context def list_pending_tasks(ctx, task_types, limit, before): """List the tasks that are going to be run. You can override the number of tasks to fetch """ from swh.scheduler import compute_nb_tasks_from scheduler = ctx.obj['scheduler'] if not scheduler: raise ValueError('Scheduler class (local/remote) must be instantiated') num_tasks, num_tasks_priority = compute_nb_tasks_from(limit) output = [] for task_type in task_types: pending = scheduler.peek_ready_tasks( task_type, timestamp=before, num_tasks=num_tasks, num_tasks_priority=num_tasks_priority) output.append('Found %d %s tasks\n' % ( len(pending), task_type)) for task in pending: output.append(pretty_print_task(task)) click.echo('\n'.join(output)) @task.command('list') @click.option('--task-id', '-i', default=None, multiple=True, metavar='ID', help='List only tasks whose id is ID.') @click.option('--task-type', '-t', default=None, multiple=True, metavar='TYPE', help='List only tasks of type TYPE') @click.option('--limit', '-l', required=False, type=click.INT, help='The maximum number of tasks to fetch.') @click.option('--status', '-s', multiple=True, metavar='STATUS', type=click.Choice( ('next_run_not_scheduled', 'next_run_scheduled', 'completed', 'disabled')), default=None, help='List tasks whose status is STATUS.') @click.option('--policy', '-p', default=None, type=click.Choice(['recurring', 'oneshot']), help='List tasks whose policy is POLICY.') @click.option('--priority', '-P', default=None, multiple=True, type=click.Choice(['all', 'low', 'normal', 'high']), help='List tasks whose priority is PRIORITY.') @click.option('--before', '-b', required=False, type=DATETIME, metavar='DATETIME', help='Limit to tasks supposed to run before the given date.') @click.option('--after', '-a', required=False, type=DATETIME, metavar='DATETIME', help='Limit to tasks supposed to run after the given date.') @click.option('--list-runs', '-r', is_flag=True, default=False, help='Also list past executions of each task.') @click.pass_context def list_tasks(ctx, task_id, task_type, limit, status, policy, priority, before, after, list_runs): """List tasks. """ scheduler = ctx.obj['scheduler'] if not scheduler: raise ValueError('Scheduler class (local/remote) must be instantiated') if not task_type: task_type = [x['type'] for x in scheduler.get_task_types()] # if task_id is not given, default value for status is # 'next_run_not_scheduled' # if task_id is given, default status is 'all' if task_id is None and status is None: status = ['next_run_not_scheduled'] if status and 'all' in status: status = None if priority and 'all' in priority: priority = None output = [] tasks = scheduler.search_tasks( task_id=task_id, task_type=task_type, status=status, priority=priority, policy=policy, before=before, after=after, limit=limit) if list_runs: runs = {t['id']: [] for t in tasks} for r in scheduler.get_task_runs([task['id'] for task in tasks]): runs[r['task']].append(r) else: runs = {} output.append('Found %d tasks\n' % ( len(tasks))) for task in tasks: output.append(pretty_print_task(task, full=True)) if runs.get(task['id']): output.append(click.style(' Executions:', bold=True)) for run in runs[task['id']]: output.append(pretty_print_run(run, indent=4)) click.echo('\n'.join(output)) @task.command('respawn') @click.argument('task-ids', required=True, nargs=-1) @click.option('--next-run', '-n', required=False, type=DATETIME, metavar='DATETIME', default=None, help='Re spawn the selected tasks at this date') @click.pass_context def respawn_tasks(ctx, task_ids, next_run): """Respawn tasks. Respawn tasks given by their ids (see the 'task list' command to find task ids) at the given date (immediately by default). Eg. swh-scheduler task respawn 1 3 12 """ scheduler = ctx.obj['scheduler'] if not scheduler: raise ValueError('Scheduler class (local/remote) must be instantiated') if next_run is None: next_run = arrow.utcnow() output = [] scheduler.set_status_tasks( task_ids, status='next_run_not_scheduled', next_run=next_run) output.append('Respawn tasks %s\n' % (task_ids,)) click.echo('\n'.join(output)) @task.command('archive') @click.option('--before', '-b', default=None, help='''Task whose ended date is anterior will be archived. Default to current month's first day.''') @click.option('--after', '-a', default=None, help='''Task whose ended date is after the specified date will be archived. Default to prior month's first day.''') @click.option('--batch-index', default=1000, type=click.INT, help='Batch size of tasks to read from db to archive') @click.option('--bulk-index', default=200, type=click.INT, help='Batch size of tasks to bulk index') @click.option('--batch-clean', default=1000, type=click.INT, help='Batch size of task to clean after archival') @click.option('--dry-run/--no-dry-run', is_flag=True, default=False, help='Default to list only what would be archived.') @click.option('--verbose', is_flag=True, default=False, help='Verbose mode') @click.option('--cleanup/--no-cleanup', is_flag=True, default=True, help='Clean up archived tasks (default)') @click.option('--start-from', type=click.INT, default=-1, help='(Optional) default task id to start from. Default is -1.') @click.pass_context def archive_tasks(ctx, before, after, batch_index, bulk_index, batch_clean, dry_run, verbose, cleanup, start_from): """Archive task/task_run whose (task_type is 'oneshot' and task_status is 'completed') or (task_type is 'recurring' and task_status is 'disabled'). With --dry-run flag set (default), only list those. """ from swh.core.utils import grouper from swh.scheduler.backend_es import SWHElasticSearchClient scheduler = ctx.obj['scheduler'] if not scheduler: raise ValueError('Scheduler class (local/remote) must be instantiated') es_client = SWHElasticSearchClient() logging.basicConfig(level=logging.DEBUG if verbose else logging.INFO) log = logging.getLogger('swh.scheduler.cli.archive') logging.getLogger('urllib3').setLevel(logging.WARN) logging.getLogger('elasticsearch').setLevel(logging.WARN) if dry_run: log.info('**DRY-RUN** (only reading db)') if not cleanup: log.info('**NO CLEANUP**') now = arrow.utcnow() # Default to archive tasks from a rolling month starting the week # prior to the current one if not before: before = now.shift(weeks=-1).format('YYYY-MM-DD') if not after: after = now.shift(weeks=-1).shift(months=-1).format('YYYY-MM-DD') log.debug('index: %s; cleanup: %s; period: [%s ; %s]' % ( not dry_run, not dry_run and cleanup, after, before)) def group_by_index_name(data, es_client=es_client): """Given a data record, determine the index's name through its ending date. This varies greatly depending on the task_run's status. """ date = data.get('started') if not date: date = data['scheduled'] return es_client.compute_index_name(date.year, date.month) def index_data(before, last_id, batch_index): - tasks_in = scheduler.filter_task_to_archive( - after, before, last_id=last_id, limit=batch_index) - for index_name, tasks_group in itertools.groupby( - tasks_in, key=group_by_index_name): - log.debug('Index tasks to %s' % index_name) - if dry_run: - for task in tasks_group: - yield task - continue - - yield from es_client.streaming_bulk( - index_name, tasks_group, source=['task_id', 'task_run_id'], - chunk_size=bulk_index, log=log) + while last_id is not None: + result = scheduler.filter_task_to_archive( + after, before, last_id=last_id, limit=batch_index) + tasks_in = result['tasks'] + for index_name, tasks_group in itertools.groupby( + tasks_in, key=group_by_index_name): + log.debug('Index tasks to %s' % index_name) + if dry_run: + for task in tasks_group: + yield task + continue + + yield from es_client.streaming_bulk( + index_name, tasks_group, source=['task_id', 'task_run_id'], + chunk_size=bulk_index, log=log) + + last_id = result.get('next_task_id') gen = index_data(before, last_id=start_from, batch_index=batch_index) if cleanup: for task_ids in grouper(gen, n=batch_clean): task_ids = list(task_ids) log.info('Clean up %s tasks: [%s, ...]' % ( len(task_ids), task_ids[0])) if dry_run: # no clean up continue ctx.obj['scheduler'].delete_archived_tasks(task_ids) else: for task_ids in grouper(gen, n=batch_index): task_ids = list(task_ids) log.info('Indexed %s tasks: [%s, ...]' % ( len(task_ids), task_ids[0])) diff --git a/swh/scheduler/sql/30-swh-schema.sql b/swh/scheduler/sql/30-swh-schema.sql index c4c8a4c..a5f0b0e 100644 --- a/swh/scheduler/sql/30-swh-schema.sql +++ b/swh/scheduler/sql/30-swh-schema.sql @@ -1,113 +1,112 @@ create table dbversion ( version int primary key, release timestamptz not null, description text not null ); comment on table dbversion is 'Schema update tracking'; comment on column dbversion.version is 'SQL schema version'; comment on column dbversion.release is 'Version deployment timestamp'; comment on column dbversion.description is 'Version description'; insert into dbversion (version, release, description) - values (14, now(), 'Work In Progress'); + values (15, now(), 'Work In Progress'); create table task_type ( type text primary key, description text not null, backend_name text not null, default_interval interval, min_interval interval, max_interval interval, backoff_factor float, max_queue_length bigint, num_retries bigint, retry_delay interval ); comment on table task_type is 'Types of schedulable tasks'; comment on column task_type.type is 'Short identifier for the task type'; comment on column task_type.description is 'Human-readable task description'; comment on column task_type.backend_name is 'Name of the task in the job-running backend'; comment on column task_type.default_interval is 'Default interval for newly scheduled tasks'; comment on column task_type.min_interval is 'Minimum interval between two runs of a task'; comment on column task_type.max_interval is 'Maximum interval between two runs of a task'; comment on column task_type.backoff_factor is 'Adjustment factor for the backoff between two task runs'; comment on column task_type.max_queue_length is 'Maximum length of the queue for this type of tasks'; comment on column task_type.num_retries is 'Default number of retries on transient failures'; comment on column task_type.retry_delay is 'Retry delay for the task'; create type task_status as enum ('next_run_not_scheduled', 'next_run_scheduled', 'completed', 'disabled'); comment on type task_status is 'Status of a given task'; create type task_policy as enum ('recurring', 'oneshot'); comment on type task_policy is 'Recurrence policy of the given task'; create type task_priority as enum('high', 'normal', 'low'); comment on type task_priority is 'Priority of the given task'; create table priority_ratio( id task_priority primary key, ratio float not null ); comment on table priority_ratio is 'Oneshot task''s reading ratio per priority'; comment on column priority_ratio.id is 'Task priority id'; comment on column priority_ratio.ratio is 'Percentage of tasks to read per priority'; insert into priority_ratio (id, ratio) values ('high', 0.5); insert into priority_ratio (id, ratio) values ('normal', 0.3); insert into priority_ratio (id, ratio) values ('low', 0.2); create table task ( id bigserial primary key, type text not null references task_type(type), arguments jsonb not null, next_run timestamptz not null, current_interval interval, status task_status not null, policy task_policy not null default 'recurring', retries_left bigint not null default 0, priority task_priority references priority_ratio(id), check (policy <> 'recurring' or current_interval is not null) ); comment on table task is 'Schedule of recurring tasks'; comment on column task.arguments is 'Arguments passed to the underlying job scheduler. ' 'Contains two keys, ''args'' (list) and ''kwargs'' (object).'; comment on column task.next_run is 'The next run of this task should be run on or after that time'; comment on column task.current_interval is 'The interval between two runs of this task, ' 'taking into account the backoff factor'; comment on column task.policy is 'Whether the task is one-shot or recurring'; comment on column task.retries_left is 'The number of "short delay" retries of the task in case of ' 'transient failure'; comment on column task.priority is 'Policy of the given task'; comment on column task.id is 'Task Identifier'; comment on column task.type is 'References task_type table'; comment on column task.status is 'Task status (''next_run_not_scheduled'', ''next_run_scheduled'', ''completed'', ''disabled'')'; create type task_run_status as enum ('scheduled', 'started', 'eventful', 'uneventful', 'failed', 'permfailed', 'lost'); comment on type task_run_status is 'Status of a given task run'; create table task_run ( id bigserial primary key, task bigint not null references task(id), backend_id text, scheduled timestamptz, started timestamptz, ended timestamptz, metadata jsonb, status task_run_status not null default 'scheduled' ); comment on table task_run is 'History of task runs sent to the job-running backend'; comment on column task_run.backend_id is 'id of the task run in the job-running backend'; comment on column task_run.metadata is 'Useful metadata for the given task run. ' 'For instance, the worker that took on the job, ' 'or the logs for the run.'; comment on column task_run.id is 'Task run identifier'; comment on column task_run.task is 'References task table'; comment on column task_run.scheduled is 'Scheduled run time for task'; comment on column task_run.started is 'Task starting time'; comment on column task_run.ended is 'Task ending time'; - diff --git a/swh/scheduler/sql/40-swh-func.sql b/swh/scheduler/sql/40-swh-func.sql index e01aa93..770e667 100644 --- a/swh/scheduler/sql/40-swh-func.sql +++ b/swh/scheduler/sql/40-swh-func.sql @@ -1,408 +1,407 @@ create or replace function swh_scheduler_mktemp_task () returns void language sql as $$ create temporary table tmp_task ( like task excluding indexes ) on commit drop; alter table tmp_task alter column retries_left drop not null, drop column id; $$; comment on function swh_scheduler_mktemp_task () is 'Create a temporary table for bulk task creation'; create or replace function swh_scheduler_create_tasks_from_temp () returns setof task language plpgsql as $$ begin -- update the default values in one go -- this is separated from the insert/select to avoid too much -- juggling update tmp_task t set current_interval = tt.default_interval, retries_left = coalesce(retries_left, tt.num_retries, 0) from task_type tt where tt.type=t.type; insert into task (type, arguments, next_run, status, current_interval, policy, retries_left, priority) select type, arguments, next_run, status, current_interval, policy, retries_left, priority from tmp_task t where not exists(select 1 from task where type = t.type and md5(arguments::text) = md5(t.arguments::text) and arguments = t.arguments and policy = t.policy and priority is not distinct from t.priority and status = t.status); return query select distinct t.* from tmp_task tt inner join task t on ( tt.type = t.type and md5(tt.arguments::text) = md5(t.arguments::text) and tt.arguments = t.arguments and tt.policy = t.policy and tt.priority is not distinct from t.priority and tt.status = t.status ); end; $$; comment on function swh_scheduler_create_tasks_from_temp () is 'Create tasks in bulk from the temporary table'; create or replace function swh_scheduler_peek_no_priority_tasks (task_type text, ts timestamptz default now(), num_tasks bigint default NULL) returns setof task language sql stable as $$ select * from task where next_run <= ts and type = task_type and status = 'next_run_not_scheduled' and priority is null order by next_run limit num_tasks for update skip locked; $$; comment on function swh_scheduler_peek_no_priority_tasks (text, timestamptz, bigint) is 'Retrieve tasks without priority'; create or replace function swh_scheduler_nb_priority_tasks(num_tasks_priority bigint, task_priority task_priority) returns numeric language sql stable as $$ select ceil(num_tasks_priority * (select ratio from priority_ratio where id = task_priority)) :: numeric $$; comment on function swh_scheduler_nb_priority_tasks (bigint, task_priority) is 'Given a priority task and a total number, compute the number of tasks to read'; create or replace function swh_scheduler_peek_tasks_with_priority (task_type text, ts timestamptz default now(), num_tasks_priority bigint default NULL, task_priority task_priority default 'normal') returns setof task language sql stable as $$ select * from task t where t.next_run <= ts and t.type = task_type and t.status = 'next_run_not_scheduled' and t.priority = task_priority order by t.next_run limit num_tasks_priority for update skip locked; $$; comment on function swh_scheduler_peek_tasks_with_priority(text, timestamptz, bigint, task_priority) is 'Retrieve tasks with a given priority'; create or replace function swh_scheduler_peek_priority_tasks (task_type text, ts timestamptz default now(), num_tasks_priority bigint default NULL) returns setof task language plpgsql as $$ declare r record; count_row bigint; nb_diff bigint; nb_high bigint; nb_normal bigint; nb_low bigint; begin -- expected values to fetch select swh_scheduler_nb_priority_tasks(num_tasks_priority, 'high') into nb_high; select swh_scheduler_nb_priority_tasks(num_tasks_priority, 'normal') into nb_normal; select swh_scheduler_nb_priority_tasks(num_tasks_priority, 'low') into nb_low; nb_diff := 0; count_row := 0; for r in select * from swh_scheduler_peek_tasks_with_priority(task_type, ts, nb_high, 'high') loop count_row := count_row + 1; return next r; end loop; if count_row < nb_high then nb_normal := nb_normal + nb_high - count_row; end if; count_row := 0; for r in select * from swh_scheduler_peek_tasks_with_priority(task_type, ts, nb_normal, 'normal') loop count_row := count_row + 1; return next r; end loop; if count_row < nb_normal then nb_low := nb_low + nb_normal - count_row; end if; return query select * from swh_scheduler_peek_tasks_with_priority(task_type, ts, nb_low, 'low'); end $$; comment on function swh_scheduler_peek_priority_tasks(text, timestamptz, bigint) is 'Retrieve priority tasks'; create or replace function swh_scheduler_peek_ready_tasks (task_type text, ts timestamptz default now(), num_tasks bigint default NULL, num_tasks_priority bigint default NULL) returns setof task language plpgsql as $$ declare r record; count_row bigint; nb_diff bigint; nb_tasks bigint; begin count_row := 0; for r in select * from swh_scheduler_peek_priority_tasks(task_type, ts, num_tasks_priority) order by priority, next_run loop count_row := count_row + 1; return next r; end loop; if count_row < num_tasks_priority then nb_tasks := num_tasks + num_tasks_priority - count_row; else nb_tasks := num_tasks; end if; for r in select * from swh_scheduler_peek_no_priority_tasks(task_type, ts, nb_tasks) order by priority, next_run loop return next r; end loop; return; end $$; comment on function swh_scheduler_peek_ready_tasks(text, timestamptz, bigint, bigint) is 'Retrieve tasks with/without priority in order'; create or replace function swh_scheduler_grab_ready_tasks (task_type text, ts timestamptz default now(), num_tasks bigint default NULL, num_tasks_priority bigint default NULL) returns setof task language sql as $$ update task set status='next_run_scheduled' from ( select id from swh_scheduler_peek_ready_tasks(task_type, ts, num_tasks, num_tasks_priority) ) next_tasks where task.id = next_tasks.id returning task.*; $$; comment on function swh_scheduler_grab_ready_tasks (text, timestamptz, bigint, bigint) is 'Grab tasks ready for scheduling and change their status'; create or replace function swh_scheduler_schedule_task_run (task_id bigint, backend_id text, metadata jsonb default '{}'::jsonb, ts timestamptz default now()) returns task_run language sql as $$ insert into task_run (task, backend_id, metadata, scheduled, status) values (task_id, backend_id, metadata, ts, 'scheduled') returning *; $$; create or replace function swh_scheduler_mktemp_task_run () returns void language sql as $$ create temporary table tmp_task_run ( like task_run excluding indexes ) on commit drop; alter table tmp_task_run drop column id, drop column status; $$; comment on function swh_scheduler_mktemp_task_run () is 'Create a temporary table for bulk task run scheduling'; create or replace function swh_scheduler_schedule_task_run_from_temp () returns void language plpgsql as $$ begin insert into task_run (task, backend_id, metadata, scheduled, status) select task, backend_id, metadata, scheduled, 'scheduled' from tmp_task_run; return; end; $$; create or replace function swh_scheduler_start_task_run (backend_id text, metadata jsonb default '{}'::jsonb, ts timestamptz default now()) returns task_run language sql as $$ update task_run set started = ts, status = 'started', metadata = coalesce(task_run.metadata, '{}'::jsonb) || swh_scheduler_start_task_run.metadata where task_run.backend_id = swh_scheduler_start_task_run.backend_id returning *; $$; create or replace function swh_scheduler_end_task_run (backend_id text, status task_run_status, metadata jsonb default '{}'::jsonb, ts timestamptz default now()) returns task_run language sql as $$ update task_run set ended = ts, status = swh_scheduler_end_task_run.status, metadata = coalesce(task_run.metadata, '{}'::jsonb) || swh_scheduler_end_task_run.metadata where task_run.backend_id = swh_scheduler_end_task_run.backend_id returning *; $$; create type task_record as ( task_id bigint, task_policy task_policy, task_status task_status, task_run_id bigint, arguments jsonb, type text, backend_id text, metadata jsonb, scheduled timestamptz, started timestamptz, ended timestamptz, task_run_status task_run_status ); create or replace function swh_scheduler_task_to_archive( ts_after timestamptz, ts_before timestamptz, last_id bigint default -1, lim bigint default 10) returns setof task_record language sql stable as $$ select t.id as task_id, t.policy as task_policy, t.status as task_status, tr.id as task_run_id, t.arguments, t.type, tr.backend_id, tr.metadata, tr.scheduled, tr.started, tr.ended, tr.status as task_run_status from task_run tr inner join task t on tr.task=t.id where ((t.policy = 'oneshot' and t.status in ('completed', 'disabled')) or (t.policy = 'recurring' and t.status = 'disabled')) and ((ts_after <= tr.started and tr.started < ts_before) or tr.started is null) and - t.id > last_id + t.id >= last_id order by tr.task, tr.started limit lim; $$; comment on function swh_scheduler_task_to_archive (timestamptz, timestamptz, bigint, bigint) is 'Read archivable tasks function'; create or replace function swh_scheduler_delete_archived_tasks( task_ids bigint[], task_run_ids bigint[]) returns void language sql as $$ -- clean up task_run_ids delete from task_run where id in (select * from unnest(task_run_ids)); -- clean up only tasks whose associated task_run are all cleaned up. -- Remaining tasks will stay there and will be cleaned up when -- remaining data have been indexed delete from task where id in (select t.id from task t left outer join task_run tr on t.id=tr.task where t.id in (select * from unnest(task_ids)) and tr.task is null); $$; comment on function swh_scheduler_delete_archived_tasks(bigint[], bigint[]) is 'Clean up archived tasks function'; create or replace function swh_scheduler_update_task_on_task_end () returns trigger language plpgsql as $$ declare cur_task task%rowtype; cur_task_type task_type%rowtype; adjustment_factor float; new_interval interval; begin select * from task where id = new.task into cur_task; select * from task_type where type = cur_task.type into cur_task_type; case when new.status = 'permfailed' then update task set status = 'disabled' where id = cur_task.id; when new.status in ('eventful', 'uneventful') then case when cur_task.policy = 'oneshot' then update task set status = 'completed' where id = cur_task.id; when cur_task.policy = 'recurring' then if new.status = 'uneventful' then adjustment_factor := 1/cur_task_type.backoff_factor; else adjustment_factor := 1/cur_task_type.backoff_factor; end if; new_interval := greatest( cur_task_type.min_interval, least( cur_task_type.max_interval, adjustment_factor * cur_task.current_interval)); update task set status = 'next_run_not_scheduled', next_run = now() + new_interval, current_interval = new_interval, retries_left = coalesce(cur_task_type.num_retries, 0) where id = cur_task.id; end case; else -- new.status in 'failed', 'lost' if cur_task.retries_left > 0 then update task set status = 'next_run_not_scheduled', next_run = now() + coalesce(cur_task_type.retry_delay, interval '1 hour'), retries_left = cur_task.retries_left - 1 where id = cur_task.id; else -- no retries left case when cur_task.policy = 'oneshot' then update task set status = 'disabled' where id = cur_task.id; when cur_task.policy = 'recurring' then update task set status = 'next_run_not_scheduled', next_run = now() + cur_task.current_interval, retries_left = coalesce(cur_task_type.num_retries, 0) where id = cur_task.id; end case; end if; -- retries end case; return null; end; $$; create trigger update_task_on_task_end after update of status on task_run for each row when (new.status NOT IN ('scheduled', 'started')) execute procedure swh_scheduler_update_task_on_task_end (); - diff --git a/swh/scheduler/tests/test_scheduler.py b/swh/scheduler/tests/test_scheduler.py index 3172dcb..25cfc32 100644 --- a/swh/scheduler/tests/test_scheduler.py +++ b/swh/scheduler/tests/test_scheduler.py @@ -1,596 +1,629 @@ # Copyright (C) 2017-2018 The Software Heritage developers # See the AUTHORS file at the top-level directory of this distribution # License: GNU General Public License version 3, or any later version # See top-level LICENSE file for more information import copy import datetime import random import uuid from collections import defaultdict import psycopg2 from arrow import utcnow import pytest TASK_TYPES = { 'git': { 'type': 'update-git', 'description': 'Update a git repository', 'backend_name': 'swh.loader.git.tasks.UpdateGitRepository', 'default_interval': datetime.timedelta(days=64), 'min_interval': datetime.timedelta(hours=12), 'max_interval': datetime.timedelta(days=64), 'backoff_factor': 2, 'max_queue_length': None, 'num_retries': 7, 'retry_delay': datetime.timedelta(hours=2), }, 'hg': { 'type': 'update-hg', 'description': 'Update a mercurial repository', 'backend_name': 'swh.loader.mercurial.tasks.UpdateHgRepository', 'default_interval': datetime.timedelta(days=64), 'min_interval': datetime.timedelta(hours=12), 'max_interval': datetime.timedelta(days=64), 'backoff_factor': 2, 'max_queue_length': None, 'num_retries': 7, 'retry_delay': datetime.timedelta(hours=2), }, } TEMPLATES = { 'git': { 'type': 'update-git', 'arguments': { 'args': [], 'kwargs': {}, }, 'next_run': None, }, 'hg': { 'type': 'update-hg', 'arguments': { 'args': [], 'kwargs': {}, }, 'next_run': None, 'policy': 'oneshot', } } def subdict(d, keys=None, excl=()): if keys is None: keys = [k for k in d.keys()] return {k: d[k] for k in keys if k not in excl} @pytest.mark.db class TestScheduler: def test_get_priority_ratios(self, swh_scheduler): assert swh_scheduler.get_priority_ratios() == { 'high': 0.5, 'normal': 0.3, 'low': 0.2, } def test_add_task_type(self, swh_scheduler): tt = TASK_TYPES['git'] swh_scheduler.create_task_type(tt) assert tt == swh_scheduler.get_task_type(tt['type']) with pytest.raises(psycopg2.IntegrityError, match=r'\(type\)=\(%s\)' % tt['type']): swh_scheduler.create_task_type(tt) tt2 = TASK_TYPES['hg'] swh_scheduler.create_task_type(tt2) assert tt == swh_scheduler.get_task_type(tt['type']) assert tt2 == swh_scheduler.get_task_type(tt2['type']) def test_get_task_types(self, swh_scheduler): tt, tt2 = TASK_TYPES['git'], TASK_TYPES['hg'] swh_scheduler.create_task_type(tt) swh_scheduler.create_task_type(tt2) actual_task_types = swh_scheduler.get_task_types() assert tt in actual_task_types assert tt2 in actual_task_types def test_create_tasks(self, swh_scheduler): priority_ratio = self._priority_ratio(swh_scheduler) self._create_task_types(swh_scheduler) num_tasks_priority = 100 tasks_1 = self._tasks_from_template( TEMPLATES['git'], utcnow(), 100) tasks_2 = self._tasks_from_template( TEMPLATES['hg'], utcnow(), 100, num_tasks_priority, priorities=priority_ratio) tasks = tasks_1 + tasks_2 # tasks are returned only once with their ids ret1 = swh_scheduler.create_tasks(tasks + tasks_1 + tasks_2) set_ret1 = set([t['id'] for t in ret1]) # creating the same set result in the same ids ret = swh_scheduler.create_tasks(tasks) set_ret = set([t['id'] for t in ret]) # Idempotence results assert set_ret == set_ret1 assert len(ret) == len(ret1) ids = set() actual_priorities = defaultdict(int) for task, orig_task in zip(ret, tasks): task = copy.deepcopy(task) task_type = TASK_TYPES[orig_task['type'].split('-')[-1]] assert task['id'] not in ids assert task['status'] == 'next_run_not_scheduled' assert task['current_interval'] == task_type['default_interval'] assert task['policy'] == orig_task.get('policy', 'recurring') priority = task.get('priority') if priority: actual_priorities[priority] += 1 assert task['retries_left'] == (task_type['num_retries'] or 0) ids.add(task['id']) del task['id'] del task['status'] del task['current_interval'] del task['retries_left'] if 'policy' not in orig_task: del task['policy'] if 'priority' not in orig_task: del task['priority'] assert task == orig_task assert dict(actual_priorities) == { priority: int(ratio * num_tasks_priority) for priority, ratio in priority_ratio.items() } def test_peek_ready_tasks_no_priority(self, swh_scheduler): self._create_task_types(swh_scheduler) t = utcnow() task_type = TEMPLATES['git']['type'] tasks = self._tasks_from_template(TEMPLATES['git'], t, 100) random.shuffle(tasks) swh_scheduler.create_tasks(tasks) ready_tasks = swh_scheduler.peek_ready_tasks(task_type) assert len(ready_tasks) == len(tasks) for i in range(len(ready_tasks) - 1): assert ready_tasks[i]['next_run'] <= ready_tasks[i+1]['next_run'] # Only get the first few ready tasks limit = random.randrange(5, 5 + len(tasks)//2) ready_tasks_limited = swh_scheduler.peek_ready_tasks( task_type, num_tasks=limit) assert len(ready_tasks_limited) == limit assert ready_tasks_limited == ready_tasks[:limit] # Limit by timestamp max_ts = tasks[limit-1]['next_run'] ready_tasks_timestamped = swh_scheduler.peek_ready_tasks( task_type, timestamp=max_ts) for ready_task in ready_tasks_timestamped: assert ready_task['next_run'] <= max_ts # Make sure we get proper behavior for the first ready tasks assert ready_tasks[:len(ready_tasks_timestamped)] \ == ready_tasks_timestamped # Limit by both ready_tasks_both = swh_scheduler.peek_ready_tasks( task_type, timestamp=max_ts, num_tasks=limit//3) assert len(ready_tasks_both) <= limit//3 for ready_task in ready_tasks_both: assert ready_task['next_run'] <= max_ts assert ready_task in ready_tasks[:limit//3] def _priority_ratio(self, swh_scheduler): return swh_scheduler.get_priority_ratios() def test_peek_ready_tasks_mixed_priorities(self, swh_scheduler): priority_ratio = self._priority_ratio(swh_scheduler) self._create_task_types(swh_scheduler) t = utcnow() task_type = TEMPLATES['git']['type'] num_tasks_priority = 100 num_tasks_no_priority = 100 # Create tasks with and without priorities tasks = self._tasks_from_template( TEMPLATES['git'], t, num=num_tasks_no_priority, num_priority=num_tasks_priority, priorities=priority_ratio) random.shuffle(tasks) swh_scheduler.create_tasks(tasks) # take all available tasks ready_tasks = swh_scheduler.peek_ready_tasks( task_type) assert len(ready_tasks) == len(tasks) assert num_tasks_priority + num_tasks_no_priority \ == len(ready_tasks) count_tasks_per_priority = defaultdict(int) for task in ready_tasks: priority = task.get('priority') if priority: count_tasks_per_priority[priority] += 1 assert dict(count_tasks_per_priority) == { priority: int(ratio * num_tasks_priority) for priority, ratio in priority_ratio.items() } # Only get some ready tasks num_tasks = random.randrange(5, 5 + num_tasks_no_priority//2) num_tasks_priority = random.randrange(5, num_tasks_priority//2) ready_tasks_limited = swh_scheduler.peek_ready_tasks( task_type, num_tasks=num_tasks, num_tasks_priority=num_tasks_priority) count_tasks_per_priority = defaultdict(int) for task in ready_tasks_limited: priority = task.get('priority') count_tasks_per_priority[priority] += 1 import math for priority, ratio in priority_ratio.items(): expected_count = math.ceil(ratio * num_tasks_priority) actual_prio = count_tasks_per_priority[priority] assert (actual_prio == expected_count or actual_prio == expected_count + 1) assert count_tasks_per_priority[None] == num_tasks def test_grab_ready_tasks(self, swh_scheduler): priority_ratio = self._priority_ratio(swh_scheduler) self._create_task_types(swh_scheduler) t = utcnow() task_type = TEMPLATES['git']['type'] num_tasks_priority = 100 num_tasks_no_priority = 100 # Create tasks with and without priorities tasks = self._tasks_from_template( TEMPLATES['git'], t, num=num_tasks_no_priority, num_priority=num_tasks_priority, priorities=priority_ratio) random.shuffle(tasks) swh_scheduler.create_tasks(tasks) first_ready_tasks = swh_scheduler.peek_ready_tasks( task_type, num_tasks=10, num_tasks_priority=10) grabbed_tasks = swh_scheduler.grab_ready_tasks( task_type, num_tasks=10, num_tasks_priority=10) for peeked, grabbed in zip(first_ready_tasks, grabbed_tasks): assert peeked['status'] == 'next_run_not_scheduled' del peeked['status'] assert grabbed['status'] == 'next_run_scheduled' del grabbed['status'] assert peeked == grabbed assert peeked['priority'] == grabbed['priority'] def test_get_tasks(self, swh_scheduler): self._create_task_types(swh_scheduler) t = utcnow() tasks = self._tasks_from_template(TEMPLATES['git'], t, 100) tasks = swh_scheduler.create_tasks(tasks) random.shuffle(tasks) while len(tasks) > 1: length = random.randrange(1, len(tasks)) cur_tasks = sorted(tasks[:length], key=lambda x: x['id']) tasks[:length] = [] ret = swh_scheduler.get_tasks(task['id'] for task in cur_tasks) # result is not guaranteed to be sorted ret.sort(key=lambda x: x['id']) assert ret == cur_tasks def test_search_tasks(self, swh_scheduler): def make_real_dicts(l): """RealDictRow is not a real dict.""" return [dict(d.items()) for d in l] self._create_task_types(swh_scheduler) t = utcnow() tasks = self._tasks_from_template(TEMPLATES['git'], t, 100) tasks = swh_scheduler.create_tasks(tasks) assert make_real_dicts(swh_scheduler.search_tasks()) \ == make_real_dicts(tasks) def test_filter_task_to_archive(self, swh_scheduler): """Filtering only list disabled recurring or completed oneshot tasks """ self._create_task_types(swh_scheduler) _time = utcnow() recurring = self._tasks_from_template(TEMPLATES['git'], _time, 12) oneshots = self._tasks_from_template(TEMPLATES['hg'], _time, 12) total_tasks = len(recurring) + len(oneshots) # simulate scheduling tasks pending_tasks = swh_scheduler.create_tasks(recurring + oneshots) backend_tasks = [{ 'task': task['id'], 'backend_id': str(uuid.uuid4()), 'scheduled': utcnow(), } for task in pending_tasks] swh_scheduler.mass_schedule_task_runs(backend_tasks) # we simulate the task are being done _tasks = [] for task in backend_tasks: t = swh_scheduler.end_task_run( task['backend_id'], status='eventful') _tasks.append(t) # Randomly update task's status per policy status_per_policy = {'recurring': 0, 'oneshot': 0} status_choice = { # policy: [tuple (1-for-filtering, 'associated-status')] 'recurring': [(1, 'disabled'), (0, 'completed'), (0, 'next_run_not_scheduled')], 'oneshot': [(0, 'next_run_not_scheduled'), (1, 'disabled'), (1, 'completed')] } tasks_to_update = defaultdict(list) _task_ids = defaultdict(list) # randomize 'disabling' recurring task or 'complete' oneshot task for task in pending_tasks: policy = task['policy'] _task_ids[policy].append(task['id']) status = random.choice(status_choice[policy]) if status[0] != 1: continue # elected for filtering status_per_policy[policy] += status[0] tasks_to_update[policy].append(task['id']) swh_scheduler.disable_tasks(tasks_to_update['recurring']) # hack: change the status to something else than completed/disabled swh_scheduler.set_status_tasks( _task_ids['oneshot'], status='next_run_not_scheduled') # complete the tasks to update swh_scheduler.set_status_tasks( tasks_to_update['oneshot'], status='completed') total_tasks_filtered = (status_per_policy['recurring'] + status_per_policy['oneshot']) + # no pagination scenario + # retrieve tasks to archive after = _time.shift(days=-1).format('YYYY-MM-DD') before = utcnow().shift(days=1).format('YYYY-MM-DD') - tasks_to_archive = list(swh_scheduler.filter_task_to_archive( - after_ts=after, before_ts=before, limit=total_tasks)) + tasks_result = swh_scheduler.filter_task_to_archive( + after_ts=after, before_ts=before, limit=total_tasks) + + tasks_to_archive = tasks_result['tasks'] assert len(tasks_to_archive) == total_tasks_filtered + assert tasks_result.get('next_task_id') is None actual_filtered_per_status = {'recurring': 0, 'oneshot': 0} for task in tasks_to_archive: actual_filtered_per_status[task['task_policy']] += 1 assert actual_filtered_per_status == status_per_policy + # pagination scenario + + limit = 3 + tasks_result = swh_scheduler.filter_task_to_archive( + after_ts=after, before_ts=before, limit=limit) + + tasks_to_archive2 = tasks_result['tasks'] + + assert len(tasks_to_archive2) == limit + next_id = tasks_result['next_task_id'] + assert next_id is not None + + all_tasks = tasks_to_archive2 + while next_id is not None: + tasks_result = swh_scheduler.filter_task_to_archive( + after_ts=after, before_ts=before, last_id=next_id, + limit=limit) + tasks_to_archive2 = tasks_result['tasks'] + assert len(tasks_to_archive2) <= limit + all_tasks.extend(tasks_to_archive2) + next_id = tasks_result.get('next_task_id') + + actual_filtered_per_status = {'recurring': 0, 'oneshot': 0} + for task in all_tasks: + actual_filtered_per_status[task['task_policy']] += 1 + + assert actual_filtered_per_status == status_per_policy + def test_delete_archived_tasks(self, swh_scheduler): self._create_task_types(swh_scheduler) _time = utcnow() recurring = self._tasks_from_template( TEMPLATES['git'], _time, 12) oneshots = self._tasks_from_template( TEMPLATES['hg'], _time, 12) total_tasks = len(recurring) + len(oneshots) pending_tasks = swh_scheduler.create_tasks(recurring + oneshots) backend_tasks = [{ 'task': task['id'], 'backend_id': str(uuid.uuid4()), 'scheduled': utcnow(), } for task in pending_tasks] swh_scheduler.mass_schedule_task_runs(backend_tasks) _tasks = [] percent = random.randint(0, 100) # random election removal boundary for task in backend_tasks: t = swh_scheduler.end_task_run( task['backend_id'], status='eventful') c = random.randint(0, 100) if c <= percent: _tasks.append({'task_id': t['task'], 'task_run_id': t['id']}) swh_scheduler.delete_archived_tasks(_tasks) all_tasks = [task['id'] for task in swh_scheduler.search_tasks()] tasks_count = len(all_tasks) tasks_run_count = len(swh_scheduler.get_task_runs(all_tasks)) assert tasks_count == total_tasks - len(_tasks) assert tasks_run_count == total_tasks - len(_tasks) def test_get_task_runs_no_task(self, swh_scheduler): '''No task exist in the scheduler's db, get_task_runs() should always return an empty list. ''' assert not swh_scheduler.get_task_runs(task_ids=()) assert not swh_scheduler.get_task_runs(task_ids=(1, 2, 3)) assert not swh_scheduler.get_task_runs(task_ids=(1, 2, 3), limit=10) def test_get_task_runs_no_task_executed(self, swh_scheduler): '''No task has been executed yet, get_task_runs() should always return an empty list. ''' self._create_task_types(swh_scheduler) _time = utcnow() recurring = self._tasks_from_template( TEMPLATES['git'], _time, 12) oneshots = self._tasks_from_template( TEMPLATES['hg'], _time, 12) swh_scheduler.create_tasks(recurring + oneshots) assert not swh_scheduler.get_task_runs(task_ids=()) assert not swh_scheduler.get_task_runs(task_ids=(1, 2, 3)) assert not swh_scheduler.get_task_runs(task_ids=(1, 2, 3), limit=10) def test_get_task_runs_with_scheduled(self, swh_scheduler): '''Some tasks have been scheduled but not executed yet, get_task_runs() should not return an empty list. limit should behave as expected. ''' self._create_task_types(swh_scheduler) _time = utcnow() recurring = self._tasks_from_template( TEMPLATES['git'], _time, 12) oneshots = self._tasks_from_template( TEMPLATES['hg'], _time, 12) total_tasks = len(recurring) + len(oneshots) pending_tasks = swh_scheduler.create_tasks(recurring + oneshots) backend_tasks = [{ 'task': task['id'], 'backend_id': str(uuid.uuid4()), 'scheduled': utcnow(), } for task in pending_tasks] swh_scheduler.mass_schedule_task_runs(backend_tasks) assert not swh_scheduler.get_task_runs( task_ids=[total_tasks + 1]) btask = backend_tasks[0] runs = swh_scheduler.get_task_runs( task_ids=[btask['task']]) assert len(runs) == 1 run = runs[0] assert subdict(run, excl=('id',)) == { 'task': btask['task'], 'backend_id': btask['backend_id'], 'scheduled': btask['scheduled'], 'started': None, 'ended': None, 'metadata': None, 'status': 'scheduled', } runs = swh_scheduler.get_task_runs( task_ids=[bt['task'] for bt in backend_tasks], limit=2) assert len(runs) == 2 runs = swh_scheduler.get_task_runs( task_ids=[bt['task'] for bt in backend_tasks]) assert len(runs) == total_tasks keys = ('task', 'backend_id', 'scheduled') assert sorted([subdict(x, keys) for x in runs], key=lambda x: x['task']) == backend_tasks def test_get_task_runs_with_executed(self, swh_scheduler): '''Some tasks have been executed, get_task_runs() should not return an empty list. limit should behave as expected. ''' self._create_task_types(swh_scheduler) _time = utcnow() recurring = self._tasks_from_template( TEMPLATES['git'], _time, 12) oneshots = self._tasks_from_template( TEMPLATES['hg'], _time, 12) pending_tasks = swh_scheduler.create_tasks(recurring + oneshots) backend_tasks = [{ 'task': task['id'], 'backend_id': str(uuid.uuid4()), 'scheduled': utcnow(), } for task in pending_tasks] swh_scheduler.mass_schedule_task_runs(backend_tasks) btask = backend_tasks[0] ts = utcnow() swh_scheduler.start_task_run(btask['backend_id'], metadata={'something': 'stupid'}, timestamp=ts) runs = swh_scheduler.get_task_runs(task_ids=[btask['task']]) assert len(runs) == 1 assert subdict(runs[0], excl=('id')) == { 'task': btask['task'], 'backend_id': btask['backend_id'], 'scheduled': btask['scheduled'], 'started': ts, 'ended': None, 'metadata': {'something': 'stupid'}, 'status': 'started', } ts2 = utcnow() swh_scheduler.end_task_run(btask['backend_id'], metadata={'other': 'stuff'}, timestamp=ts2, status='eventful') runs = swh_scheduler.get_task_runs(task_ids=[btask['task']]) assert len(runs) == 1 assert subdict(runs[0], excl=('id')) == { 'task': btask['task'], 'backend_id': btask['backend_id'], 'scheduled': btask['scheduled'], 'started': ts, 'ended': ts2, 'metadata': {'something': 'stupid', 'other': 'stuff'}, 'status': 'eventful', } @staticmethod def _task_from_template(template, next_run, priority, *args, **kwargs): ret = copy.deepcopy(template) ret['next_run'] = next_run if priority: ret['priority'] = priority if args: ret['arguments']['args'] = list(args) if kwargs: ret['arguments']['kwargs'] = kwargs return ret def _pop_priority(self, priorities): if not priorities: return None for priority, remains in priorities.items(): if remains > 0: priorities[priority] = remains - 1 return priority return None def _tasks_from_template(self, template, max_timestamp, num, num_priority=0, priorities=None): if num_priority and priorities: priorities = { priority: ratio * num_priority for priority, ratio in priorities.items() } tasks = [] for i in range(num + num_priority): priority = self._pop_priority(priorities) tasks.append(self._task_from_template( template, max_timestamp - datetime.timedelta(microseconds=i), priority, 'argument-%03d' % i, **{'kwarg%03d' % i: 'bogus-kwarg'} )) return tasks def _create_task_types(self, scheduler): for tt in TASK_TYPES.values(): scheduler.create_task_type(tt)