diff --git a/swift/cli/info.py b/swift/cli/info.py index 0eee781ba6..1969435285 100644 --- a/swift/cli/info.py +++ b/swift/cli/info.py @@ -298,6 +298,27 @@ def print_db_info_metadata(db_type, info, metadata, drop_prefixes=False): else: print('No user metadata found in db file') + if db_type == 'container': + print('Sharding Metadata:') + shard_type = 'root' if info['is_root'] else 'shard' + print(' Type: %s' % shard_type) + print(' State: %s' % info['db_state']) + if info.get('shard_ranges'): + print('Shard Ranges (%d):' % len(info['shard_ranges'])) + for srange in info['shard_ranges']: + srange = dict(srange, state_text=srange.state_text) + print(' Name: %(name)s' % srange) + print(' lower: %(lower)r, upper: %(upper)r' % srange) + print(' Object Count: %(object_count)d, Bytes Used: ' + '%(bytes_used)d, State: %(state_text)s (%(state)d)' + % srange) + print(' Created at: %s (%s)' + % (Timestamp(srange['timestamp']).isoformat, + srange['timestamp'])) + print(' Meta Timestamp: %s (%s)' + % (Timestamp(srange['meta_timestamp']).isoformat, + srange['meta_timestamp'])) + def print_obj_metadata(metadata, drop_prefixes=False): """ @@ -406,7 +427,13 @@ def print_info(db_type, db_file, swift_dir='/etc/swift', stale_reads_ok=False, raise InfoSystemExit() raise account = info['account'] - container = info['container'] if db_type == 'container' else None + container = None + if db_type == 'container': + container = info['container'] + info['is_root'] = broker.is_root_container() + sranges = broker.get_shard_ranges() + if sranges: + info['shard_ranges'] = sranges print_db_info_metadata(db_type, info, broker.metadata, drop_prefixes) try: ring = Ring(swift_dir, ring_name=db_type) diff --git a/swift/common/db_replicator.py b/swift/common/db_replicator.py index c27914bffb..0d063cd455 100644 --- a/swift/common/db_replicator.py +++ b/swift/common/db_replicator.py @@ -33,7 +33,8 @@ from swift.common.direct_client import quote from swift.common.utils import get_logger, whataremyips, storage_directory, \ renamer, mkdirs, lock_parent_directory, config_true_value, \ unlink_older_than, dump_recon_cache, rsync_module_interpolation, \ - json, parse_override_options, round_robin_iter, Everything + json, parse_override_options, round_robin_iter, Everything, get_db_files, \ + parse_db_filename from swift.common import ring from swift.common.ring.utils import is_local_device from swift.common.http import HTTP_NOT_FOUND, HTTP_INSUFFICIENT_STORAGE, \ @@ -120,14 +121,20 @@ def roundrobin_datadirs(datadirs): if not os.path.isdir(hash_dir): continue object_file = os.path.join(hash_dir, hsh + '.db') + # common case if os.path.exists(object_file): yield (partition, object_file, context) - else: - try: - os.rmdir(hash_dir) - except OSError as e: - if e.errno != errno.ENOTEMPTY: - raise + continue + # look for any alternate db filenames + db_files = get_db_files(object_file) + if db_files: + yield (partition, db_files[-1], context) + continue + try: + os.rmdir(hash_dir) + except OSError as e: + if e.errno != errno.ENOTEMPTY: + raise its = [walk_datadir(datadir, context, filt) for datadir, context, filt in datadirs] @@ -216,7 +223,7 @@ class Replicator(Daemon): self.stats = {'attempted': 0, 'success': 0, 'failure': 0, 'ts_repl': 0, 'no_change': 0, 'hashmatch': 0, 'rsync': 0, 'diff': 0, 'remove': 0, 'empty': 0, 'remote_merge': 0, - 'start': time.time(), 'diff_capped': 0, + 'start': time.time(), 'diff_capped': 0, 'deferred': 0, 'failure_nodes': {}} def _report_stats(self): @@ -313,12 +320,13 @@ class Replicator(Daemon): different_region=different_region): return False with Timeout(replicate_timeout or self.node_timeout): - response = http.replicate(replicate_method, local_id) + response = http.replicate(replicate_method, local_id, + os.path.basename(broker.db_file)) return response and 200 <= response.status < 300 - def _send_merge_items(self, http, local_id, items): + def _send_replicate_request(self, http, *repl_args): with Timeout(self.node_timeout): - response = http.replicate('merge_items', items, local_id) + response = http.replicate(*repl_args) if not response or not is_success(response.status): if response: self.logger.error('ERROR Bad response %s from %s', @@ -350,7 +358,8 @@ class Replicator(Daemon): diffs = 0 while len(objects) and diffs < self.max_diffs: diffs += 1 - if not self._send_merge_items(http, local_id, objects): + if not self._send_replicate_request( + http, 'merge_items', objects, local_id): return False # replication relies on db order to send the next merge batch in # order with no gaps @@ -413,9 +422,8 @@ class Replicator(Daemon): :returns: ReplConnection object """ - return ReplConnection(node, partition, - os.path.basename(db_file).split('.', 1)[0], - self.logger) + hsh, other, ext = parse_db_filename(db_file) + return ReplConnection(node, partition, hsh, self.logger) def _gather_sync_args(self, info): """ @@ -931,6 +939,8 @@ class ReplicatorRpc(object): def complete_rsync(self, drive, db_file, args): old_filename = os.path.join(self.root, drive, 'tmp', args[0]) + if args[1:]: + db_file = os.path.join(os.path.dirname(db_file), args[1]) if os.path.exists(db_file): return HTTPNotFound() if not os.path.exists(old_filename): @@ -944,6 +954,10 @@ class ReplicatorRpc(object): return not (self._db_file_exists(db_file) and os.path.exists(tmp_filename)) + def _post_rsync_then_merge_hook(self, existing_broker, new_broker): + # subclasses may override to make custom changes to the new broker + pass + def rsync_then_merge(self, drive, db_file, args): tmp_filename = os.path.join(self.root, drive, 'tmp', args[0]) if self._abort_rsync_then_merge(db_file, tmp_filename): @@ -959,6 +973,7 @@ class ReplicatorRpc(object): objects = existing_broker.get_items_since(point, 1000) sleep() new_broker.merge_syncs(existing_broker.get_syncs()) + self._post_rsync_then_merge_hook(existing_broker, new_broker) new_broker.newid(args[0]) new_broker.update_metadata(existing_broker.metadata) if self._abort_rsync_then_merge(db_file, tmp_filename): diff --git a/swift/common/utils.py b/swift/common/utils.py index 6641bce268..40c2cb7a8f 100644 --- a/swift/common/utils.py +++ b/swift/common/utils.py @@ -5300,3 +5300,86 @@ def distribute_evenly(items, num_buckets): for index, item in enumerate(items): out[index % num_buckets].append(item) return out + + +def parse_db_filename(filename): + """ + Splits a db filename into three parts: the hash, the epoch, and the + extension. + + >>> parse_db_filename("ab2134.db") + ('ab2134', None, '.db') + >>> parse_db_filename("ab2134_1234567890.12345.db") + ('ab2134', '1234567890.12345', '.db') + + :param filename: A db file basename or path to a db file. + :return: A tuple of (hash , epoch, extension). ``epoch`` may be None. + :raises ValueError: if ``filename`` is not a path to a file. + """ + filename = os.path.basename(filename) + if not filename: + raise ValueError('Path to a file required.') + name, ext = os.path.splitext(filename) + parts = name.split('_') + hash_ = parts.pop(0) + epoch = parts[0] if parts else None + return hash_, epoch, ext + + +def make_db_file_path(db_path, epoch): + """ + Given a path to a db file, return a modified path whose filename part has + the given epoch. + + A db filename takes the form [_].db; this method replaces the + part of the given ``db_path`` with the given ``epoch`` value. + + :param db_path: Path to a db file that does not necessarily exist. + :param epoch: A string that will be used as the epoch in the new path's + filename; the value will be normalized to the normal string + representation of a :class:`~swift.common.utils.Timestamp`. + :return: A modified path to a db file. + :raises ValueError: if the ``epoch`` is not valid for constructing a + :class:`~swift.common.utils.Timestamp`. + """ + if epoch is None: + raise ValueError('epoch must not be None') + epoch = Timestamp(epoch).normal + hash_, _, ext = parse_db_filename(db_path) + db_dir = os.path.dirname(db_path) + return os.path.join(db_dir, '%s_%s%s' % (hash_, epoch, ext)) + + +def get_db_files(db_path): + """ + Given the path to a db file, return a sorted list of all valid db files + that actually exist in that path's dir. A valid db filename has the form: + + [_].db + + where matches the part of the given db_path as would be + parsed by :meth:`~swift.utils.common.parse_db_filename`. + + :param db_path: Path to a db file that does not necessarily exist. + :return: List of valid db files that do exist in the dir of the + ``db_path``. This list may be empty. + """ + db_dir, db_file = os.path.split(db_path) + try: + files = os.listdir(db_dir) + except OSError as err: + if err.errno == errno.ENOENT: + return [] + raise + if not files: + return [] + match_hash, epoch, ext = parse_db_filename(db_file) + results = [] + for f in files: + hash_, epoch, ext = parse_db_filename(f) + if ext != '.db': + continue + if hash_ != match_hash: + continue + results.append(os.path.join(db_dir, f)) + return sorted(results) diff --git a/swift/container/backend.py b/swift/container/backend.py index c61c633739..9d75d0f680 100644 --- a/swift/container/backend.py +++ b/swift/container/backend.py @@ -15,6 +15,8 @@ """ Pluggable Back-ends for Container Server """ +import errno + import os from uuid import uuid4 @@ -22,16 +24,45 @@ import six import six.moves.cPickle as pickle from six.moves import range import sqlite3 +from eventlet import tpool +from swift.common.constraints import CONTAINER_LISTING_LIMIT +from swift.common.exceptions import LockTimeout from swift.common.utils import Timestamp, encode_timestamps, \ - decode_timestamps, extract_swift_bytes, storage_directory, hash_path -from swift.common.db import DatabaseBroker, utf8encode, \ + decode_timestamps, extract_swift_bytes, storage_directory, hash_path, \ + ShardRange, renamer, find_shard_range, MD5_OF_EMPTY_STRING, mkdirs, \ + get_db_files, parse_db_filename, make_db_file_path, split_path +from swift.common.db import DatabaseBroker, utf8encode, BROKER_TIMEOUT, \ zero_like, DatabaseAlreadyExists SQLITE_ARG_LIMIT = 999 DATADIR = 'containers' +RECORD_TYPE_OBJECT = 'object' +RECORD_TYPE_SHARD = 'shard' +SHARD_RANGE_TABLE = 'shard_range' + +NOTFOUND = 'not_found' +UNSHARDED = 'unsharded' +SHARDING = 'sharding' +SHARDED = 'sharded' +COLLAPSED = 'collapsed' + + +SHARD_STATS_STATES = [ShardRange.ACTIVE, ShardRange.SHARDING, + ShardRange.SHRINKING] +SHARD_LISTING_STATES = SHARD_STATS_STATES + [ShardRange.CLEAVED] +SHARD_UPDATE_STATES = [ShardRange.CREATED, ShardRange.CLEAVED, + ShardRange.ACTIVE, ShardRange.SHARDING] + + +# attribute names in order used when transforming shard ranges from dicts to +# tuples and vice-versa +SHARD_RANGE_KEYS = ('name', 'timestamp', 'lower', 'upper', 'object_count', + 'bytes_used', 'meta_timestamp', 'deleted', 'state', + 'state_timestamp', 'epoch') + POLICY_STAT_TABLE_CREATE = ''' CREATE TABLE policy_stat ( storage_policy_index INTEGER PRIMARY KEY, @@ -220,15 +251,99 @@ def update_new_item_from_existing(new_item, existing): return any(newer_than_existing) +def merge_shards(shard_data, existing): + """ + Compares ``shard_data`` with ``existing`` and updates ``shard_data`` with + any items of ``existing`` that take precedence over the corresponding item + in ``shard_data``. + + :param shard_data: a dict representation of shard range that may be + modified by this method. + :param existing: a dict representation of shard range. + :returns: True if ``shard data`` has any item(s) that are considered to + take precedence over the corresponding item in ``existing`` + """ + if not existing: + return True + if existing['timestamp'] < shard_data['timestamp']: + # note that currently we do not roll forward any meta or state from + # an item that was created at older time, newer created time trumps + return True + elif existing['timestamp'] > shard_data['timestamp']: + return False + + new_content = False + # timestamp must be the same, so preserve existing range bounds and deleted + for k in ('lower', 'upper', 'deleted'): + shard_data[k] = existing[k] + + # now we need to look for meta data updates + if existing['meta_timestamp'] >= shard_data['meta_timestamp']: + for k in ('object_count', 'bytes_used', 'meta_timestamp'): + shard_data[k] = existing[k] + else: + new_content = True + + if (existing['state_timestamp'] == shard_data['state_timestamp'] + and shard_data['state'] > existing['state']): + new_content = True + elif existing['state_timestamp'] >= shard_data['state_timestamp']: + for k in ('state', 'state_timestamp', 'epoch'): + shard_data[k] = existing[k] + else: + new_content = True + return new_content + + class ContainerBroker(DatabaseBroker): - """Encapsulates working with a container database.""" + """ + Encapsulates working with a container database. + + Note that this may involve multiple on-disk DB files if the container + becomes sharded: + + * :attr:`_db_file` is the path to the legacy container DB name, i.e. + ``.db``. This file should exist for an initialised broker that + has never been sharded, but will not exist once a container has been + sharded. + * :attr:`db_files` is a list of existing db files for the broker. This + list should have at least one entry for an initialised broker, and + should have two entries while a broker is in SHARDING state. + * :attr:`db_file` is the path to whichever db is currently authoritative + for the container. Depending on the container's state, this may not be + the same as the ``db_file`` argument given to :meth:`~__init__`, unless + ``force_db_file`` is True in which case :attr:`db_file` is always equal + to the ``db_file`` argument given to :meth:`~__init__`. + * :attr:`pending_file` is always equal to :attr:`_db_file` extended with + ``.pending``, i.e. ``.db.pending``. + """ db_type = 'container' db_contains_type = 'object' db_reclaim_timestamp = 'created_at' + def __init__(self, db_file, timeout=BROKER_TIMEOUT, logger=None, + account=None, container=None, pending_timeout=None, + stale_reads_ok=False, skip_commits=False, + force_db_file=False): + self._init_db_file = db_file + if db_file == ':memory:': + base_db_file = db_file + else: + db_dir = os.path.dirname(db_file) + hash_, other, ext = parse_db_filename(db_file) + base_db_file = os.path.join(db_dir, hash_ + ext) + super(ContainerBroker, self).__init__( + base_db_file, timeout, logger, account, container, pending_timeout, + stale_reads_ok, skip_commits=skip_commits) + # the root account and container are populated on demand + self._root_account = self._root_container = None + self._force_db_file = force_db_file + self._db_files = None + @classmethod def create_broker(self, device_path, part, account, container, logger=None, - put_timestamp=None, storage_policy_index=None): + epoch=None, put_timestamp=None, + storage_policy_index=None): """ Create a ContainerBroker instance. If the db doesn't exist, initialize the db file. @@ -238,6 +353,7 @@ class ContainerBroker(DatabaseBroker): :param account: account name string :param container: container name string :param logger: a logger instance + :param epoch: a timestamp to include in the db filename :param put_timestamp: initial timestamp if broker needs to be initialized :param storage_policy_index: the storage policy index @@ -246,6 +362,8 @@ class ContainerBroker(DatabaseBroker): hsh = hash_path(account, container) db_dir = storage_directory(DATADIR, part, hsh) db_path = os.path.join(device_path, db_dir, hsh + '.db') + if epoch: + db_path = make_db_file_path(db_path, epoch) broker = ContainerBroker(db_path, account=account, container=container, logger=logger) if not os.path.exists(broker.db_file): @@ -255,6 +373,98 @@ class ContainerBroker(DatabaseBroker): pass return broker + def get_db_state(self): + """ + Returns the current state of on disk db files. + """ + if self._db_file == ':memory:': + return UNSHARDED + if not self.db_files: + return NOTFOUND + if len(self.db_files) > 1: + return SHARDING + if self.db_epoch is None: + # never been sharded + return UNSHARDED + if self.db_epoch != self._own_shard_range().epoch: + return UNSHARDED + if not self.get_shard_ranges(): + return COLLAPSED + return SHARDED + + def sharding_initiated(self): + """ + Returns True if a broker has shard range state that would be necessary + for sharding to have been initiated, False otherwise. + """ + own_shard_range = self.get_own_shard_range() + if own_shard_range.state in (ShardRange.SHARDING, + ShardRange.SHRINKING, + ShardRange.SHARDED): + return bool(self.get_shard_ranges()) + return False + + def sharding_required(self): + """ + Returns True if a broker has shard range state that would be necessary + for sharding to have been initiated but has not yet completed sharding, + False otherwise. + """ + db_state = self.get_db_state() + return (db_state == SHARDING or + (db_state == UNSHARDED and self.sharding_initiated())) + + def is_sharded(self): + return self.get_db_state() == SHARDED + + def reload_db_files(self): + """ + Reloads the cached list of valid on disk db files for this broker. + """ + if self._db_file == ':memory:': + return + # reset connection so the next access will use the correct DB file + self.conn = None + self._db_files = get_db_files(self._init_db_file) + + @property + def db_files(self): + """ + Gets the cached list of valid db files that exist on disk for this + broker. + + The cached list may be refreshed by calling + :meth:`~swift.container.backend.ContainerBroker.reload_db_files`. + + :return: A list of paths to db files ordered by ascending epoch; + the list may be empty. + """ + if not self._db_files: + self.reload_db_files() + return self._db_files + + @property + def db_file(self): + """ + Get the path to the primary db file for this broker. This is typically + the db file for the most recent sharding epoch. However, if no db files + exist on disk, or if ``force_db_file`` was True when the broker was + constructed, then the primary db file is the file passed to the broker + constructor. + + :return: A path to a db file; the file does not necessarily exist. + """ + if self._force_db_file: + return self._init_db_file + if self.db_files: + return self.db_files[-1] + return self._init_db_file + + @property + def db_epoch(self): + hash_, epoch, ext = parse_db_filename(self.db_file) + return epoch + @property def storage_policy_index(self): if not hasattr(self, '_storage_policy_index'): @@ -262,6 +472,11 @@ class ContainerBroker(DatabaseBroker): self.get_info()['storage_policy_index'] return self._storage_policy_index + @property + def path(self): + self._populate_instance_cache() + return '%s/%s' % (self.account, self.container) + def _initialize(self, conn, put_timestamp, storage_policy_index): """ Create a brand new container database (tables, indices, triggers, etc.) @@ -278,6 +493,8 @@ class ContainerBroker(DatabaseBroker): self.create_policy_stat_table(conn, storage_policy_index) self.create_container_info_table(conn, put_timestamp, storage_policy_index) + self.create_shard_range_table(conn) + self._db_files = None def create_object_table(self, conn): """ @@ -359,6 +576,40 @@ class ContainerBroker(DatabaseBroker): VALUES (?) """, (storage_policy_index,)) + def create_shard_range_table(self, conn): + """ + Create the shard_range table which is specific to the container DB. + + :param conn: DB connection object + """ + # Use execute (not executescript) so we get the benefits of our + # GreenDBConnection. Creating a table requires a whole-DB lock; + # *any* in-progress cursor will otherwise trip a "database is locked" + # error. + conn.execute(""" + CREATE TABLE %s ( + ROWID INTEGER PRIMARY KEY AUTOINCREMENT, + name TEXT, + timestamp TEXT, + lower TEXT, + upper TEXT, + object_count INTEGER DEFAULT 0, + bytes_used INTEGER DEFAULT 0, + meta_timestamp TEXT, + deleted INTEGER DEFAULT 0, + state INTEGER, + state_timestamp TEXT, + epoch TEXT + ); + """ % SHARD_RANGE_TABLE) + + conn.execute(""" + CREATE TRIGGER shard_range_update BEFORE UPDATE ON %s + BEGIN + SELECT RAISE(FAIL, 'UPDATE not allowed; DELETE and INSERT'); + END; + """ % SHARD_RANGE_TABLE) + def get_db_version(self, conn): if self._db_version == -1: self._db_version = 0 @@ -368,6 +619,11 @@ class ContainerBroker(DatabaseBroker): self._db_version = 1 return self._db_version + def _get_deleted_key(self, connection): + if self.get_db_version(connection) < 1: + return '+deleted' + return 'deleted' + def _newid(self, conn): conn.execute(''' UPDATE container_stat @@ -411,12 +667,7 @@ class ContainerBroker(DatabaseBroker): 'ctype_timestamp': content_type_timestamp, 'meta_timestamp': meta_timestamp}) - def empty(self): - """ - Check if container DB is empty. - - :returns: True if the database has no active objects, False otherwise - """ + def _empty(self): self._commit_puts_stale_ok() with self.get() as conn: try: @@ -431,6 +682,26 @@ class ContainerBroker(DatabaseBroker): 'SELECT object_count from container_stat').fetchone() return zero_like(row[0]) + def empty(self): + """ + Check if container DB is empty. + + This method uses more stringent checks on object count than + :meth:`is_deleted`: this method checks that there are no objects in any + policy; if the container is in the process of sharding then both fresh + and retiring databases are checked to be empty; if a root container has + shard ranges then they are checked to be empty. + + :returns: True if the database has no active objects, False otherwise + """ + if not all(broker._empty() for broker in self.get_brokers()): + return False + if self.is_root_container() and self.sharding_initiated(): + # sharded shards don't get updates from their shards so their shard + # usage should not be relied upon + return self.get_shard_usage()['object_count'] <= 0 + return True + def delete_object(self, name, timestamp, storage_policy_index=0): """ Mark an object deleted. @@ -490,7 +761,12 @@ class ContainerBroker(DatabaseBroker): def _is_deleted(self, conn): """ - Check container_stat view and evaluate info. + Check if the DB is considered to be deleted. + + This object count used in this check is the same as the container + object count that would be returned in the result of :meth:`get_info` + and exposed to a client i.e. it is based on the container_stat view for + the current storage policy index or relevant shard range usage. :param conn: database conn @@ -499,6 +775,8 @@ class ContainerBroker(DatabaseBroker): info = conn.execute(''' SELECT put_timestamp, delete_timestamp, object_count FROM container_stat''').fetchone() + info = dict(info) + info.update(self._get_alternate_object_stats()[1]) return self._is_deleted_info(**info) def is_reclaimable(self, now, reclaim_age): @@ -524,6 +802,73 @@ class ContainerBroker(DatabaseBroker): info = self.get_info() return info, self._is_deleted_info(**info) + def get_replication_info(self): + info = super(ContainerBroker, self).get_replication_info() + info['shard_max_row'] = self.get_max_row('shard_ranges') + return info + + def _do_get_info_query(self, conn): + data = None + trailing_sync = 'x_container_sync_point1, x_container_sync_point2' + trailing_pol = 'storage_policy_index' + errors = set() + while not data: + try: + data = conn.execute((''' + SELECT account, container, created_at, put_timestamp, + delete_timestamp, status_changed_at, + object_count, bytes_used, + reported_put_timestamp, reported_delete_timestamp, + reported_object_count, reported_bytes_used, hash, + id, %s, %s + FROM container_stat + ''') % (trailing_sync, trailing_pol)).fetchone() + except sqlite3.OperationalError as err: + err_msg = str(err) + if err_msg in errors: + # only attempt migration once + raise + errors.add(err_msg) + if 'no such column: storage_policy_index' in err_msg: + trailing_pol = '0 AS storage_policy_index' + elif 'no such column: x_container_sync_point' in err_msg: + trailing_sync = '-1 AS x_container_sync_point1, ' \ + '-1 AS x_container_sync_point2' + else: + raise + data = dict(data) + # populate instance cache + self._storage_policy_index = data['storage_policy_index'] + self.account = data['account'] + self.container = data['container'] + return data + + def _get_info(self): + self._commit_puts_stale_ok() + with self.get() as conn: + return self._do_get_info_query(conn) + + def _populate_instance_cache(self, conn=None): + # load cached instance attributes from the database if necessary + if self.container is None: + if conn: + self._do_get_info_query(conn) + else: + with self.get() as conn: + self._do_get_info_query(conn) + + def _get_alternate_object_stats(self): + state = self.get_db_state() + if state == SHARDING: + other_info = self.get_brokers()[0]._get_info() + stats = {'object_count': other_info['object_count'], + 'bytes_used': other_info['bytes_used']} + elif state == SHARDED and self.is_root_container(): + stats = self.get_shard_usage() + else: + stats = {} + return state, stats + def get_info(self): """ Get global data for the container. @@ -533,44 +878,14 @@ class ContainerBroker(DatabaseBroker): object_count, bytes_used, reported_put_timestamp, reported_delete_timestamp, reported_object_count, reported_bytes_used, hash, id, x_container_sync_point1, - x_container_sync_point2, and storage_policy_index. + x_container_sync_point2, and storage_policy_index, + db_state. """ - self._commit_puts_stale_ok() - with self.get() as conn: - data = None - trailing_sync = 'x_container_sync_point1, x_container_sync_point2' - trailing_pol = 'storage_policy_index' - errors = set() - while not data: - try: - data = conn.execute((''' - SELECT account, container, created_at, put_timestamp, - delete_timestamp, status_changed_at, - object_count, bytes_used, - reported_put_timestamp, reported_delete_timestamp, - reported_object_count, reported_bytes_used, hash, - id, %s, %s - FROM container_stat - ''') % (trailing_sync, trailing_pol)).fetchone() - except sqlite3.OperationalError as err: - err_msg = str(err) - if err_msg in errors: - # only attempt migration once - raise - errors.add(err_msg) - if 'no such column: storage_policy_index' in err_msg: - trailing_pol = '0 AS storage_policy_index' - elif 'no such column: x_container_sync_point' in err_msg: - trailing_sync = '-1 AS x_container_sync_point1, ' \ - '-1 AS x_container_sync_point2' - else: - raise - data = dict(data) - # populate instance cache - self._storage_policy_index = data['storage_policy_index'] - self.account = data['account'] - self.container = data['container'] - return data + data = self._get_info() + state, stats = self._get_alternate_object_stats() + data.update(stats) + data['db_state'] = state + return data def set_x_container_sync_points(self, sync_point1, sync_point2): with self.get() as conn: @@ -696,7 +1011,9 @@ class ContainerBroker(DatabaseBroker): conn.commit() def list_objects_iter(self, limit, marker, end_marker, prefix, delimiter, - path=None, storage_policy_index=0, reverse=False): + path=None, storage_policy_index=0, reverse=False, + include_deleted=False, since_row=None, + transform_func=None, all_policies=False): """ Get a list of objects sorted by name starting at marker onward, up to limit entries. Entries will begin with the prefix and will not @@ -711,10 +1028,29 @@ class ContainerBroker(DatabaseBroker): the path :param storage_policy_index: storage policy index for query :param reverse: reverse the result order. - + :param include_deleted: if True, include only deleted objects; if + False (default), include only undeleted objects; otherwise, include + both deleted and undeleted objects. + :param since_row: include only items whose ROWID is greater than + the given row id; by default all rows are included. + :param transform_func: an optional function that if given will be + called for each object to get a transformed version of the object + to include in the listing; should have same signature as + :meth:`~_transform_record`; defaults to :meth:`~_transform_record`. + :param all_policies: if True, include objects for all storage policies + ignoring any value given for ``storage_policy_index`` :returns: list of tuples of (name, created_at, size, content_type, - etag) + etag, deleted) """ + if include_deleted is True: + deleted_arg = ' = 1' + elif include_deleted is False: + deleted_arg = ' = 0' + else: + deleted_arg = ' in (0, 1)' + + if transform_func is None: + transform_func = self._transform_record delim_force_gte = False (marker, end_marker, prefix, delimiter, path) = utf8encode( marker, end_marker, prefix, delimiter, path) @@ -734,60 +1070,71 @@ class ContainerBroker(DatabaseBroker): orig_marker = marker with self.get() as conn: results = [] + deleted_key = self._get_deleted_key(conn) + query_keys = ['name', 'created_at', 'size', 'content_type', + 'etag', deleted_key] while len(results) < limit: - query = '''SELECT name, created_at, size, content_type, etag - FROM object WHERE''' query_args = [] + query_conditions = [] if end_marker and (not prefix or end_marker < end_prefix): - query += ' name < ? AND' + query_conditions.append('name < ?') query_args.append(end_marker) elif prefix: - query += ' name < ? AND' + query_conditions.append('name < ?') query_args.append(end_prefix) if delim_force_gte: - query += ' name >= ? AND' + query_conditions.append('name >= ?') query_args.append(marker) # Always set back to False delim_force_gte = False elif marker and marker >= prefix: - query += ' name > ? AND' + query_conditions.append('name > ?') query_args.append(marker) elif prefix: - query += ' name >= ? AND' + query_conditions.append('name >= ?') query_args.append(prefix) - if self.get_db_version(conn) < 1: - query += ' +deleted = 0' - else: - query += ' deleted = 0' - orig_tail_query = ''' - ORDER BY name %s LIMIT ? - ''' % ('DESC' if reverse else '') - orig_tail_args = [limit - len(results)] + query_conditions.append(deleted_key + deleted_arg) + if since_row: + query_conditions.append('ROWID > ?') + query_args.append(since_row) + + def build_query(keys, conditions, args): + query = 'SELECT ' + ', '.join(keys) + ' FROM object ' + if conditions: + query += 'WHERE ' + ' AND '.join(conditions) + tail_query = ''' + ORDER BY name %s LIMIT ? + ''' % ('DESC' if reverse else '') + return query + tail_query, args + [limit - len(results)] + # storage policy filter - policy_tail_query = ''' - AND storage_policy_index = ? - ''' + orig_tail_query - policy_tail_args = [storage_policy_index] + orig_tail_args - tail_query, tail_args = \ - policy_tail_query, policy_tail_args + if all_policies: + query, args = build_query( + query_keys + ['storage_policy_index'], + query_conditions, + query_args) + else: + query, args = build_query( + query_keys + ['storage_policy_index'], + query_conditions + ['storage_policy_index = ?'], + query_args + [storage_policy_index]) try: - curs = conn.execute(query + tail_query, - tuple(query_args + tail_args)) + curs = conn.execute(query, tuple(args)) except sqlite3.OperationalError as err: if 'no such column: storage_policy_index' not in str(err): raise - tail_query, tail_args = \ - orig_tail_query, orig_tail_args - curs = conn.execute(query + tail_query, - tuple(query_args + tail_args)) + query, args = build_query( + query_keys + ['0 as storage_policy_index'], + query_conditions, query_args) + curs = conn.execute(query, tuple(args)) curs.row_factory = None # Delimiters without a prefix is ignored, further if there # is no delimiter then we can simply return the result as # prefixes are now handled in the SQL statement. if prefix is None or not delimiter: - return [self._transform_record(r) for r in curs] + return [transform_func(r) for r in curs] # We have a delimiter and a prefix (possibly empty string) to # handle @@ -826,19 +1173,51 @@ class ContainerBroker(DatabaseBroker): results.append([dir_name, '0', 0, None, '']) curs.close() break - results.append(self._transform_record(row)) + results.append(transform_func(row)) if not rowcount: break return results + def get_objects(self, limit=None, marker='', end_marker='', + include_deleted=None, since_row=None): + """ + Returns a list of objects, including deleted objects, in all policies. + Each object in the list is described by a dict with keys {'name', + 'created_at', 'size', 'content_type', 'etag', 'deleted', + 'storage_policy_index'}. + + :param limit: maximum number of entries to get + :param marker: if set, objects with names less than or equal to this + value will not be included in the list. + :param end_marker: if set, objects with names greater than or equal to + this value will not be included in the list. + :param include_deleted: if True, include only deleted objects; if + False, include only undeleted objects; otherwise (default), include + both deleted and undeleted objects. + :param since_row: include only items whose ROWID is greater than + the given row id; by default all rows are included. + :return: a list of dicts, each describing an object. + """ + + limit = CONTAINER_LISTING_LIMIT if limit is None else limit + return self.list_objects_iter( + limit, marker, end_marker, prefix=None, delimiter=None, path=None, + reverse=False, include_deleted=include_deleted, + transform_func=self._record_to_dict, since_row=since_row, + all_policies=True + ) + def _transform_record(self, record): """ - Decode the created_at timestamp into separate data, content-type and - meta timestamps and replace the created_at timestamp with the - metadata timestamp i.e. the last-modified time. + Returns a tuple of (name, last-modified time, size, content_type and + etag) for the given record. + + The given record's created_at timestamp is decoded into separate data, + content-type and meta timestamps and the metadata timestamp is used as + the last-modified time value. """ t_data, t_ctype, t_meta = decode_timestamps(record[1]) - return (record[0], t_meta.internal) + record[2:] + return (record[0], t_meta.internal) + record[2:5] def _record_to_dict(self, rec): if rec: @@ -861,7 +1240,7 @@ class ContainerBroker(DatabaseBroker): if isinstance(item['name'], six.text_type): item['name'] = item['name'].encode('utf-8') - def _really_merge_items(conn): + def _really_really_merge_items(conn): curs = conn.cursor() if self.get_db_version(conn) >= 1: query_mod = ' deleted IN (0, 1) AND ' @@ -924,6 +1303,9 @@ class ContainerBroker(DatabaseBroker): ''', (sync_point, source)) conn.commit() + def _really_merge_items(conn): + return tpool.execute(_really_really_merge_items, conn) + with self.get() as conn: try: return _really_merge_items(conn) @@ -933,6 +1315,86 @@ class ContainerBroker(DatabaseBroker): self._migrate_add_storage_policy(conn) return _really_merge_items(conn) + def merge_shard_ranges(self, shard_ranges): + """ + Merge shard ranges into the shard range table. + + :param shard_ranges: a shard range or a list of shard ranges; each + shard range should be an instance of + :class:`~swift.common.utils.ShardRange` or a dict representation of + a shard range having ``SHARD_RANGE_KEYS``. + """ + if not shard_ranges: + return + if not isinstance(shard_ranges, list): + shard_ranges = [shard_ranges] + + item_list = [] + for item in shard_ranges: + if isinstance(item, ShardRange): + item = dict(item) + for col in ('name', 'lower', 'upper'): + if isinstance(item[col], six.text_type): + item[col] = item[col].encode('utf-8') + item_list.append(item) + + def _really_merge_items(conn): + curs = conn.cursor() + curs.execute('BEGIN IMMEDIATE') + + # Get rows for items that already exist. + # We must chunk it up to avoid sqlite's limit of 999 args. + records = {} + for offset in range(0, len(item_list), SQLITE_ARG_LIMIT): + chunk = [record['name'] for record + in item_list[offset:offset + SQLITE_ARG_LIMIT]] + records.update( + (rec[0], rec) for rec in curs.execute( + 'SELECT %s FROM %s ' + 'WHERE deleted IN (0, 1) AND name IN (%s)' % + (', '.join(SHARD_RANGE_KEYS), SHARD_RANGE_TABLE, + ','.join('?' * len(chunk))), chunk)) + + # Sort item_list into things that need adding and deleting + to_delete = {} + to_add = {} + for item in item_list: + item_ident = item['name'] + existing = records.get(item_ident) + if existing: + existing = dict(zip(SHARD_RANGE_KEYS, existing)) + if merge_shards(item, existing): + # exists with older timestamp + if item_ident in records: + to_delete[item_ident] = item + # duplicate entries in item_list + if (item_ident not in to_add or + merge_shards(item, to_add[item_ident])): + to_add[item_ident] = item + + if to_delete: + curs.executemany( + 'DELETE FROM %s WHERE deleted in (0, 1) ' + 'AND name = ?' % SHARD_RANGE_TABLE, + ((item_ident,) for item_ident in to_delete)) + if to_add: + vals = ','.join('?' * len(SHARD_RANGE_KEYS)) + curs.executemany( + 'INSERT INTO %s (%s) VALUES (%s)' % + (SHARD_RANGE_TABLE, ','.join(SHARD_RANGE_KEYS), vals), + tuple([item[k] for k in SHARD_RANGE_KEYS] + for item in to_add.values())) + conn.commit() + + with self.get() as conn: + try: + return _really_merge_items(conn) + except sqlite3.OperationalError as err: + if ('no such table: %s' % SHARD_RANGE_TABLE) not in str(err): + raise + self.create_shard_range_table(conn) + return _really_merge_items(conn) + def get_reconciler_sync(self): with self.get() as conn: try: @@ -1078,3 +1540,644 @@ class ContainerBroker(DatabaseBroker): ''' % (column_names, column_names) + CONTAINER_STAT_VIEW_SCRIPT + 'COMMIT;') + + def _reclaim(self, conn, age_timestamp, sync_timestamp): + super(ContainerBroker, self)._reclaim(conn, age_timestamp, + sync_timestamp) + # populate instance cache, but use existing conn to avoid deadlock + # when it has a pending update + self._populate_instance_cache(conn=conn) + try: + conn.execute(''' + DELETE FROM %s WHERE deleted = 1 AND timestamp < ? + AND name != ? + ''' % SHARD_RANGE_TABLE, (sync_timestamp, self.path)) + except sqlite3.OperationalError as err: + if ('no such table: %s' % SHARD_RANGE_TABLE) not in str(err): + raise + + def _get_shard_range_rows(self, connection=None, include_deleted=False, + states=None, exclude_states=None, + include_own=False, exclude_others=False): + """ + Returns a list of shard range rows. + + To get all shard ranges use ``include_own=True``. To get only the + broker's own shard range use ``include_own=True`` and + ``exclude_others=True``. + + :param connection: db connection + :param include_deleted: include rows marked as deleted + :param states: include only rows matching the given state(s); can be an + int or a list of ints. + :param exclude_states: exclude rows matching the given state(s); can be + an int or a list of ints; takes precedence over ``state``. + :param include_own: boolean that governs whether the row whose name + matches the broker's path is included in the returned list. If + True, that row is included, otherwise it is not included. Default + is False. + :param exclude_others: boolean that governs whether the rows whose + names do not match the broker's path are included in the returned + list. If True, those rows are not included, otherwise they are + included. Default is False. + :return: a list of tuples. + """ + + if exclude_others and not include_own: + return [] + + def prep_states(states): + state_set = set() + if isinstance(states, (list, tuple, set)): + state_set.update(states) + elif states is not None: + state_set.add(states) + return state_set + + excluded_states = prep_states(exclude_states) + included_states = prep_states(states) + included_states -= excluded_states + + def do_query(conn): + try: + condition = '' + conditions = [] + params = [] + if not include_deleted: + conditions.append('deleted=0') + if included_states: + conditions.append('state in (%s)' % ','.join( + '?' * len(included_states))) + params.extend(included_states) + if excluded_states: + conditions.append('state not in (%s)' % ','.join( + '?' * len(excluded_states))) + params.extend(excluded_states) + if not include_own: + conditions.append('name != ?') + params.append(self.path) + if exclude_others: + conditions.append('name = ?') + params.append(self.path) + if conditions: + condition = ' WHERE ' + ' AND '.join(conditions) + sql = ''' + SELECT %s + FROM %s%s; + ''' % (', '.join(SHARD_RANGE_KEYS), SHARD_RANGE_TABLE, + condition) + data = conn.execute(sql, params) + data.row_factory = None + return [row for row in data] + except sqlite3.OperationalError as err: + if ('no such table: %s' % SHARD_RANGE_TABLE) not in str(err): + raise + return [] + + if connection: + return do_query(connection) + else: + with self.get() as conn: + return do_query(conn) + + @classmethod + def resolve_shard_range_states(cls, states): + """ + Given a list of values each of which may be the name of a state, the + number of a state, or an alias, return the set of state numbers + described by the list. + + The following alias values are supported: 'listing' maps to all states + that are considered valid when listing objects; 'updating' maps to all + states that are considered valid for redirecting an object update. + + :param states: a list of values each of which may be the name of a + state, the number of a state, or an alias + :return: a set of integer state numbers, or None if no states are given + :raises ValueError: if any value in the given list is neither a valid + state nor a valid alias + """ + if states: + resolved_states = set() + for state in states: + if state == 'listing': + resolved_states.update(SHARD_LISTING_STATES) + elif state == 'updating': + resolved_states.update(SHARD_UPDATE_STATES) + else: + resolved_states.add(ShardRange.resolve_state(state)[0]) + return resolved_states + return None + + def get_shard_ranges(self, marker=None, end_marker=None, includes=None, + reverse=False, include_deleted=False, states=None, + exclude_states=None, include_own=False, + exclude_others=False, fill_gaps=False): + """ + Returns a list of persisted shard ranges. + + :param marker: restricts the returned list to shard ranges whose + namespace includes or is greater than the marker value. + :param end_marker: restricts the returned list to shard ranges whose + namespace includes or is less than the end_marker value. + :param includes: restricts the returned list to the shard range that + includes the given value; if ``includes`` is specified then + ``marker`` and ``end_marker`` are ignored. + :param reverse: reverse the result order. + :param include_deleted: include items that have the delete marker set + :param states: if specified, restricts the returned list to shard + ranges that have the given state(s); can be a list of ints or a + single int. + :param exclude_states: exclude rows matching the given state(s); can be + an int or a list of ints; takes precedence over ``state``. + :param include_own: boolean that governs whether the row whose name + matches the broker's path is included in the returned list. If + True, that row is included, otherwise it is not included. Default + is False. + :param exclude_others: boolean that governs whether the rows whose + names do not match the broker's path are included in the returned + list. If True, those rows are not included, otherwise they are + included. Default is False. + :param fill_gaps: if True, insert own shard range to fill any gaps in + at the tail of other shard ranges. + :return: a list of instances of :class:`swift.common.utils.ShardRange` + """ + def shard_range_filter(sr): + end = start = True + if end_marker: + end = end_marker > sr.lower + if marker: + start = marker < sr.upper + return start and end + + if reverse: + marker, end_marker = end_marker, marker + if marker and end_marker and marker >= end_marker: + return [] + + shard_ranges = [ + ShardRange(*row) + for row in self._get_shard_range_rows( + include_deleted=include_deleted, states=states, + exclude_states=exclude_states, include_own=include_own, + exclude_others=exclude_others)] + # note if this ever changes to *not* sort by upper first then it breaks + # a key assumption for bisect, which is used by utils.find_shard_ranges + shard_ranges.sort(key=lambda sr: (sr.upper, sr.state, sr.lower)) + if includes: + shard_range = find_shard_range(includes, shard_ranges) + return [shard_range] if shard_range else [] + + if reverse: + shard_ranges.reverse() + if marker or end_marker: + shard_ranges = list(filter(shard_range_filter, shard_ranges)) + + if fill_gaps: + if reverse: + if shard_ranges: + last_upper = shard_ranges[0].upper + else: + last_upper = marker or ShardRange.MIN + required_upper = end_marker or ShardRange.MAX + filler_index = 0 + else: + if shard_ranges: + last_upper = shard_ranges[-1].upper + else: + last_upper = marker or ShardRange.MIN + required_upper = end_marker or ShardRange.MAX + filler_index = len(shard_ranges) + if required_upper > last_upper: + filler_sr = self.get_own_shard_range() + filler_sr.lower = last_upper + filler_sr.upper = required_upper + shard_ranges.insert(filler_index, filler_sr) + + return shard_ranges + + def _own_shard_range(self, no_default=False): + shard_ranges = self.get_shard_ranges(include_own=True, + include_deleted=True, + exclude_others=True) + if shard_ranges: + own_shard_range = shard_ranges[0] + elif no_default: + return None + else: + own_shard_range = ShardRange( + self.path, Timestamp.now(), ShardRange.MIN, ShardRange.MAX, + state=ShardRange.ACTIVE) + return own_shard_range + + def get_own_shard_range(self, no_default=False): + """ + Returns a shard range representing this broker's own shard range. If no + such range has been persisted in the broker's shard ranges table then a + default shard range representing the entire namespace will be returned. + + The returned shard range will be updated with the current object stats + for this broker and a meta timestamp set to the current time. For these + values to be persisted the caller must merge the shard range. + + :param no_default: if True and the broker's own shard range is not + found in the shard ranges table then None is returned, otherwise a + default shard range is returned. + :return: an instance of :class:`~swift.common.utils.ShardRange` + """ + own_shard_range = self._own_shard_range(no_default=no_default) + if own_shard_range: + info = self.get_info() + own_shard_range.update_meta( + info['object_count'], info['bytes_used']) + return own_shard_range + + def is_own_shard_range(self, shard_range): + return shard_range.name == self.path + + def enable_sharding(self, epoch): + """ + Updates this broker's own shard range with the given epoch, sets its + state to SHARDING and persists it in the DB. + + :param epoch: a :class:`~swift.utils.common.Timestamp` + :return: the broker's updated own shard range. + """ + own_shard_range = self._own_shard_range() + own_shard_range.update_state(ShardRange.SHARDING, epoch) + own_shard_range.epoch = epoch + self.merge_shard_ranges(own_shard_range) + return own_shard_range + + def get_shard_usage(self): + """ + Get the aggregate object stats for all shard ranges in states ACTIVE, + SHARDING or SHRINKING. + + :return: a dict with keys {bytes_used, object_count} + """ + shard_ranges = self.get_shard_ranges(states=SHARD_STATS_STATES) + return {'bytes_used': sum(sr.bytes_used for sr in shard_ranges), + 'object_count': sum(sr.object_count for sr in shard_ranges)} + + def get_all_shard_range_data(self): + """ + Returns a list of all shard range data, including own shard range and + deleted shard ranges. + + :return: A list of dict representations of a ShardRange. + """ + shard_ranges = self.get_shard_ranges(include_deleted=True, + include_own=True) + return [dict(sr) for sr in shard_ranges] + + def set_sharding_state(self): + """ + Creates and initializes a fresh DB file in preparation for sharding a + retiring DB. The broker's own shard range must have an epoch timestamp + for this method to succeed. + + :return: True if the fresh DB was successfully created, False + otherwise. + """ + epoch = self.get_own_shard_range().epoch + if not epoch: + self.logger.warning("Container '%s' cannot be set to sharding " + "state: missing epoch", self.path) + return False + state = self.get_db_state() + if not state == UNSHARDED: + self.logger.warning("Container '%s' cannot be set to sharding " + "state while in %s state", self.path, state) + return False + + info = self.get_info() + # The tmp_dir is cleaned up by the replicators after reclaim_age, so if + # we initially create the fresh DB there, we will already have cleanup + # covered if there is an error. + tmp_dir = os.path.join(self.get_device_path(), 'tmp') + if not os.path.exists(tmp_dir): + mkdirs(tmp_dir) + tmp_db_file = os.path.join(tmp_dir, "fresh%s.db" % str(uuid4())) + fresh_broker = ContainerBroker(tmp_db_file, self.timeout, self.logger, + self.account, self.container) + fresh_broker.initialize(info['put_timestamp'], + info['storage_policy_index']) + # copy relevant data from the retiring db to the fresh db + fresh_broker.update_metadata(self.metadata) + fresh_broker.merge_shard_ranges(self.get_all_shard_range_data()) + # copy sync points so that any peer in sync with retiring db will + # appear to be in sync with the fresh db, although the peer shouldn't + # attempt to replicate objects to a db with shard ranges. + for incoming in (True, False): + syncs = self.get_syncs(incoming) + fresh_broker.merge_syncs(syncs, incoming) + + max_row = self.get_max_row() + with fresh_broker.get() as fresh_broker_conn: + # Initialise the rowid to continue from where the retiring db ended + try: + sql = "INSERT into object " \ + "(ROWID, name, created_at, size, content_type, etag) " \ + "values (?, 'tmp_sharding', ?, 0, '', ?)" + fresh_broker_conn.execute( + sql, (max_row, Timestamp.now().internal, + MD5_OF_EMPTY_STRING)) + fresh_broker_conn.execute( + 'DELETE FROM object WHERE ROWID = ?', (max_row,)) + fresh_broker_conn.commit() + except sqlite3.OperationalError as err: + self.logger.error( + 'Failed to set the ROWID of the fresh database for %s: %s', + self.path, err) + return False + + # Set the created_at and hash in the container_info table the same + # in both brokers + try: + fresh_broker_conn.execute( + 'UPDATE container_stat SET created_at=?', + (info['created_at'],)) + fresh_broker_conn.commit() + except sqlite3.OperationalError as err: + self.logger.error('Failed to set matching created_at time in ' + 'the fresh database for %s: %s', + self.path, err) + return False + + # Rename to the new database + fresh_db_filename = make_db_file_path(self._db_file, epoch) + renamer(tmp_db_file, fresh_db_filename) + self.reload_db_files() + return True + + def set_sharded_state(self): + """ + Unlink's the broker's retiring DB file. + + :return: True if the retiring DB was successfully unlinked, False + otherwise. + """ + state = self.get_db_state() + if not state == SHARDING: + self.logger.warning("Container %r cannot be set to sharded " + "state while in %s state", + self.path, state) + return False + + self.reload_db_files() + if len(self.db_files) < 2: + self.logger.warning( + 'Refusing to delete db file for %r: no fresher db file found ' + 'in %r.', self.path, self.db_files) + return False + + retiring_file = self.db_files[-2] + try: + os.unlink(retiring_file) + self.logger.debug('Unlinked retiring db %r', retiring_file) + except OSError as err: + if err.errno != errno.ENOENT: + self.logger.exception('Failed to unlink %r' % self._db_file) + return False + + self.reload_db_files() + if len(self.db_files) >= 2: + self.logger.warning( + 'Still have multiple db files after unlinking %r: %r', + retiring_file, self.db_files) + return False + + return True + + def get_brokers(self): + """ + Return a list of brokers for component dbs. The list has two entries + while the db state is sharding: the first entry is a broker for the + retiring db with ``skip_commits`` set to ``True``; the second entry is + a broker for the fresh db with ``skip_commits`` set to ``False``. For + any other db state the list has one entry. + + :return: a list of :class:`~swift.container.backend.ContainerBroker` + """ + if len(self.db_files) > 2: + self.logger.warning('Unexpected db files will be ignored: %s' % + self.db_files[:-2]) + brokers = [] + db_files = self.db_files[-2:] + while db_files: + db_file = db_files.pop(0) + sub_broker = ContainerBroker( + db_file, self.timeout, self.logger, self.account, + self.container, self.pending_timeout, self.stale_reads_ok, + force_db_file=True, skip_commits=bool(db_files)) + brokers.append(sub_broker) + return brokers + + def set_sharding_sysmeta(self, key, value): + """ + Updates the broker's metadata metadata stored under the given key + prefixed with a sharding specific namespace. + + :param key: metadata key in the sharding metadata namespace. + :param value: metadata value + """ + self.update_metadata({'X-Container-Sysmeta-Shard-' + key: + (value, Timestamp.now().internal)}) + + def get_sharding_sysmeta(self, key=None): + """ + Returns sharding specific info from the broker's metadata. + + :param key: if given the value stored under ``key`` in the sharding + info will be returned. + :return: either a dict of sharding info or the value stored under + ``key`` in that dict. + """ + prefix = 'X-Container-Sysmeta-Shard-' + metadata = self.metadata + info = dict((k[len(prefix):], v[0]) for + k, v in metadata.items() if k.startswith(prefix)) + if key: + return info.get(key) + return info + + def _load_root_info(self): + """ + Load the root container name and account for the container represented + by this broker. + + The root container path, if set, is stored in sysmeta under the key + ``X-Container-Sysmeta-Shard-Root``. If this sysmeta is not set then the + container is considered to be a root container and ``_root_account`` + and ``_root_container`` are set equal to the broker ``account`` and + ``container`` attributes respectively. + + """ + path = self.get_sharding_sysmeta('Root') + if not path: + # Ensure account/container get populated + self._populate_instance_cache() + self._root_account = self.account + self._root_container = self.container + return + + try: + self._root_account, self._root_container = split_path( + '/' + path, 2, 2) + except ValueError: + raise ValueError("Expected X-Container-Sysmeta-Shard-Root to be " + "of the form 'account/container', got %r" % path) + + @property + def root_account(self): + if not self._root_account: + self._load_root_info() + return self._root_account + + @property + def root_container(self): + if not self._root_container: + self._load_root_info() + return self._root_container + + @property + def root_path(self): + return '%s/%s' % (self.root_account, self.root_container) + + def is_root_container(self): + """ + Returns True if this container is a root container, False otherwise. + + A root container is a container that is not a shard of another + container. + """ + self._populate_instance_cache() + return (self.root_account == self.account and + self.root_container == self.container) + + def _get_next_shard_range_upper(self, shard_size, last_upper=None): + """ + Returns the name of the object that is ``shard_size`` rows beyond + ``last_upper`` in the object table ordered by name. If ``last_upper`` + is not given then it defaults to the start of object table ordered by + name. + + :param last_upper: the upper bound of the last found shard range. + :return: an object name, or None if the number of rows beyond + ``last_upper`` is less than ``shard_size``. + """ + self._commit_puts_stale_ok() + with self.get() as connection: + sql = ('SELECT name FROM object WHERE %s=0 ' % + self._get_deleted_key(connection)) + args = [] + if last_upper: + sql += "AND name > ? " + args.append(str(last_upper)) + sql += "ORDER BY name LIMIT 1 OFFSET %d" % (shard_size - 1) + row = connection.execute(sql, args).fetchone() + return row['name'] if row else None + + def find_shard_ranges(self, shard_size, limit=-1, existing_ranges=None): + """ + Scans the container db for shard ranges. Scanning will start at the + upper bound of the any ``existing_ranges`` that are given, otherwise + at ``ShardRange.MIN``. Scanning will stop when ``limit`` shard ranges + have been found or when no more shard ranges can be found. In the + latter case, the upper bound of the final shard range will be equal to + the upper bound of the container namespace. + + This method does not modify the state of the db; callers are + responsible for persisting any shard range data in the db. + + :param shard_size: the size of each shard range + :param limit: the maximum number of shard points to be found; a + negative value (default) implies no limit. + :param existing_ranges: an optional list of existing ShardRanges; if + given, this list should be sorted in order of upper bounds; the + scan for new shard ranges will start at the upper bound of the last + existing ShardRange. + :return: a tuple; the first value in the tuple is a list of + dicts each having keys {'index', 'lower', 'upper', 'object_count'} + in order of ascending 'upper'; the second value in the tuple is a + boolean which is True if the last shard range has been found, False + otherwise. + """ + existing_ranges = existing_ranges or [] + object_count = self.get_info().get('object_count', 0) + if shard_size >= object_count: + # container not big enough to shard + return [], False + + own_shard_range = self.get_own_shard_range() + progress = 0 + progress_reliable = True + # update initial state to account for any existing shard ranges + if existing_ranges: + if all([sr.state == ShardRange.FOUND + for sr in existing_ranges]): + progress = sum([sr.object_count for sr in existing_ranges]) + else: + # else: object count in existing shard ranges may have changed + # since they were found so progress cannot be reliably + # calculated; use default progress of zero - that's ok, + # progress is used for optimisation not correctness + progress_reliable = False + last_shard_upper = existing_ranges[-1].upper + if last_shard_upper >= own_shard_range.upper: + # == implies all ranges were previously found + # > implies an acceptor range has been set into which this + # shard should cleave itself + return [], True + else: + last_shard_upper = own_shard_range.lower + + found_ranges = [] + sub_broker = self.get_brokers()[0] + index = len(existing_ranges) + while limit < 0 or len(found_ranges) < limit: + if progress + shard_size >= object_count: + # next shard point is at or beyond final object name so don't + # bother with db query + next_shard_upper = None + else: + try: + next_shard_upper = sub_broker._get_next_shard_range_upper( + shard_size, last_shard_upper) + except (sqlite3.OperationalError, LockTimeout): + self.logger.exception( + "Problem finding shard upper in %r: " % self.db_file) + break + + if (next_shard_upper is None or + next_shard_upper > own_shard_range.upper): + # We reached the end of the container namespace, or possibly + # beyond if the container has misplaced objects. In either case + # limit the final shard range to own_shard_range.upper. + next_shard_upper = own_shard_range.upper + if progress_reliable: + # object count may include misplaced objects so the final + # shard size may not be accurate until cleaved, but at + # least the sum of shard sizes will equal the unsharded + # object_count + shard_size = object_count - progress + + # NB shard ranges are created with a non-zero object count so that + # the apparent container object count remains constant, and the + # container is non-deletable while shards have been found but not + # yet cleaved + found_ranges.append( + {'index': index, + 'lower': str(last_shard_upper), + 'upper': str(next_shard_upper), + 'object_count': shard_size}) + + if next_shard_upper == own_shard_range.upper: + return found_ranges, True + + progress += shard_size + last_shard_upper = next_shard_upper + index += 1 + + return found_ranges, False diff --git a/swift/container/replicator.py b/swift/container/replicator.py index 9f3fdb53c7..b326ab70e3 100644 --- a/swift/container/replicator.py +++ b/swift/container/replicator.py @@ -26,9 +26,10 @@ from swift.container.reconciler import ( get_reconciler_container_name, get_row_to_q_entry_translator) from swift.common import db_replicator from swift.common.storage_policy import POLICIES +from swift.common.swob import HTTPOk, HTTPAccepted from swift.common.exceptions import DeviceUnavailable from swift.common.http import is_success -from swift.common.utils import Timestamp, majority_size +from swift.common.utils import Timestamp, majority_size, get_db_files class ContainerReplicator(db_replicator.Replicator): @@ -76,9 +77,51 @@ class ContainerReplicator(db_replicator.Replicator): if any(info[key] != remote_info[key] for key in sync_timestamps): broker.merge_timestamps(*(remote_info[key] for key in sync_timestamps)) + + # Grab remote's shard ranges, too + self._fetch_and_merge_shard_ranges(http, broker) + return super(ContainerReplicator, self)._handle_sync_response( node, response, info, broker, http, different_region) + def _sync_shard_ranges(self, broker, http, local_id): + # TODO: currently the number of shard ranges is expected to be _much_ + # less than normal objects so all are sync'd on each cycle. However, in + # future there should be sync points maintained much like for object + # syncing so that only new shard range rows are sync'd. + shard_range_data = broker.get_all_shard_range_data() + if shard_range_data: + if not self._send_replicate_request( + http, 'merge_shard_ranges', shard_range_data, local_id): + return False + self.logger.debug('%s synced %s shard ranges to %s', + broker.db_file, len(shard_range_data), + '%(ip)s:%(port)s/%(device)s' % http.node) + return True + + def _choose_replication_mode(self, node, rinfo, info, local_sync, broker, + http, different_region): + # Always replicate shard ranges + shard_range_success = self._sync_shard_ranges(broker, http, info['id']) + if broker.sharding_initiated(): + self.logger.warning( + '%s is able to shard -- refusing to replicate objects to peer ' + '%s; have shard ranges and will wait for cleaving', + broker.db_file, + '%(ip)s:%(port)s/%(device)s' % node) + self.stats['deferred'] += 1 + return shard_range_success + + success = super(ContainerReplicator, self)._choose_replication_mode( + node, rinfo, info, local_sync, broker, http, + different_region) + return shard_range_success and success + + def _fetch_and_merge_shard_ranges(self, http, broker): + response = http.replicate('get_shard_ranges') + if is_success(response.status): + broker.merge_shard_ranges(json.loads(response.data)) + def find_local_handoff_for_part(self, part): """ Look through devices in the ring for the first handoff device that was @@ -202,6 +245,18 @@ class ContainerReplicator(db_replicator.Replicator): # replication broker.update_reconciler_sync(max_sync) + def cleanup_post_replicate(self, broker, orig_info, responses): + debug_template = 'Not deleting db %s (%%s)' % broker.db_file + if broker.sharding_required(): + # despite being a handoff, since we're sharding we're not going to + # do any cleanup so we can continue cleaving - this is still + # considered "success" + reason = 'requires sharding, state %s' % broker.get_db_state() + self.logger.debug(debug_template, reason) + return True + return super(ContainerReplicator, self).cleanup_post_replicate( + broker, orig_info, responses) + def delete_db(self, broker): """ Ensure that reconciler databases are only cleaned up at the end of the @@ -255,9 +310,20 @@ class ContainerReplicator(db_replicator.Replicator): self.replicate_reconcilers() return rv + def _in_sync(self, rinfo, info, broker, local_sync): + # TODO: don't always sync shard ranges! + if broker.get_shard_ranges(include_own=True, include_deleted=True): + return False + + return super(ContainerReplicator, self)._in_sync( + rinfo, info, broker, local_sync) + class ContainerReplicatorRpc(db_replicator.ReplicatorRpc): + def _db_file_exists(self, db_path): + return bool(get_db_files(db_path)) + def _parse_sync_args(self, args): parent = super(ContainerReplicatorRpc, self) remote_info = parent._parse_sync_args(args) @@ -285,3 +351,27 @@ class ContainerReplicatorRpc(db_replicator.ReplicatorRpc): timestamp=status_changed_at) info = broker.get_replication_info() return info + + def _abort_rsync_then_merge(self, db_file, old_filename): + if super(ContainerReplicatorRpc, self)._abort_rsync_then_merge( + db_file, old_filename): + return True + # if the local db has started sharding since the original 'sync' + # request then abort object replication now; instantiate a fresh broker + # each time this check if performed so to get latest state + broker = ContainerBroker(db_file) + return broker.sharding_initiated() + + def _post_rsync_then_merge_hook(self, existing_broker, new_broker): + # Note the following hook will need to change to using a pointer and + # limit in the future. + new_broker.merge_shard_ranges( + existing_broker.get_all_shard_range_data()) + + def merge_shard_ranges(self, broker, args): + broker.merge_shard_ranges(args[0]) + return HTTPAccepted() + + def get_shard_ranges(self, broker, args): + return HTTPOk(headers={'Content-Type': 'application/json'}, + body=json.dumps(broker.get_all_shard_range_data())) diff --git a/test/unit/cli/test_info.py b/test/unit/cli/test_info.py index d1ea79cff3..1d5c56e9f4 100644 --- a/test/unit/cli/test_info.py +++ b/test/unit/cli/test_info.py @@ -31,6 +31,7 @@ from swift.cli.info import (print_db_info_metadata, print_ring_locations, parse_get_node_args) from swift.account.server import AccountController from swift.container.server import ContainerController +from swift.container.backend import UNSHARDED, SHARDED from swift.obj.diskfile import write_metadata @@ -103,17 +104,18 @@ class TestCliInfo(TestCliInfoBase): self.assertRaisesMessage(ValueError, 'Info is incomplete', print_db_info_metadata, 'container', {}, {}) - info = dict( - account='acct', - created_at=100.1, - put_timestamp=106.3, - delete_timestamp=107.9, - status_changed_at=108.3, - container_count='3', - object_count='20', - bytes_used='42') - info['hash'] = 'abaddeadbeefcafe' - info['id'] = 'abadf100d0ddba11' + info = { + 'account': 'acct', + 'created_at': 100.1, + 'put_timestamp': 106.3, + 'delete_timestamp': 107.9, + 'status_changed_at': 108.3, + 'container_count': '3', + 'object_count': '20', + 'bytes_used': '42', + 'hash': 'abaddeadbeefcafe', + 'id': 'abadf100d0ddba11', + } md = {'x-account-meta-mydata': ('swift', '0000000000.00000'), 'x-other-something': ('boo', '0000000000.00000')} out = StringIO() @@ -154,7 +156,9 @@ No system metadata found in db file reported_object_count='20', reported_bytes_used='42', x_container_foo='bar', - x_container_bar='goo') + x_container_bar='goo', + db_state=UNSHARDED, + is_root=True) info['hash'] = 'abaddeadbeefcafe' info['id'] = 'abadf100d0ddba11' md = {'x-container-sysmeta-mydata': ('swift', '0000000000.00000')} @@ -182,10 +186,88 @@ Metadata: X-Container-Bar: goo X-Container-Foo: bar System Metadata: {'mydata': 'swift'} -No user metadata found in db file''' % POLICIES[0].name +No user metadata found in db file +Sharding Metadata: + Type: root + State: unsharded''' % POLICIES[0].name self.assertEqual(sorted(out.getvalue().strip().split('\n')), sorted(exp_out.split('\n'))) + def test_print_db_info_metadata_with_shard_ranges(self): + + shard_ranges = [utils.ShardRange( + name='.sharded_a/shard_range_%s' % i, + timestamp=utils.Timestamp(i), lower='%da' % i, + upper='%dz' % i, object_count=i, bytes_used=i, + meta_timestamp=utils.Timestamp(i)) for i in range(1, 4)] + shard_ranges[0].state = utils.ShardRange.CLEAVED + shard_ranges[1].state = utils.ShardRange.CREATED + + info = dict( + account='acct', + container='cont', + storage_policy_index=0, + created_at='0000000100.10000', + put_timestamp='0000000106.30000', + delete_timestamp='0000000107.90000', + status_changed_at='0000000108.30000', + object_count='20', + bytes_used='42', + reported_put_timestamp='0000010106.30000', + reported_delete_timestamp='0000010107.90000', + reported_object_count='20', + reported_bytes_used='42', + db_state=SHARDED, + is_root=True, + shard_ranges=shard_ranges) + info['hash'] = 'abaddeadbeefcafe' + info['id'] = 'abadf100d0ddba11' + out = StringIO() + with mock.patch('sys.stdout', out): + print_db_info_metadata('container', info, {}) + exp_out = '''Path: /acct/cont + Account: acct + Container: cont + Container Hash: d49d0ecbb53be1fcc49624f2f7c7ccae +Metadata: + Created at: 1970-01-01T00:01:40.100000 (0000000100.10000) + Put Timestamp: 1970-01-01T00:01:46.300000 (0000000106.30000) + Delete Timestamp: 1970-01-01T00:01:47.900000 (0000000107.90000) + Status Timestamp: 1970-01-01T00:01:48.300000 (0000000108.30000) + Object Count: 20 + Bytes Used: 42 + Storage Policy: %s (0) + Reported Put Timestamp: 1970-01-01T02:48:26.300000 (0000010106.30000) + Reported Delete Timestamp: 1970-01-01T02:48:27.900000 (0000010107.90000) + Reported Object Count: 20 + Reported Bytes Used: 42 + Chexor: abaddeadbeefcafe + UUID: abadf100d0ddba11 +No system metadata found in db file +No user metadata found in db file +Sharding Metadata: + Type: root + State: sharded +Shard Ranges (3): + Name: .sharded_a/shard_range_1 + lower: '1a', upper: '1z' + Object Count: 1, Bytes Used: 1, State: cleaved (30) + Created at: 1970-01-01T00:00:01.000000 (0000000001.00000) + Meta Timestamp: 1970-01-01T00:00:01.000000 (0000000001.00000) + Name: .sharded_a/shard_range_2 + lower: '2a', upper: '2z' + Object Count: 2, Bytes Used: 2, State: created (20) + Created at: 1970-01-01T00:00:02.000000 (0000000002.00000) + Meta Timestamp: 1970-01-01T00:00:02.000000 (0000000002.00000) + Name: .sharded_a/shard_range_3 + lower: '3a', upper: '3z' + Object Count: 3, Bytes Used: 3, State: found (10) + Created at: 1970-01-01T00:00:03.000000 (0000000003.00000) + Meta Timestamp: 1970-01-01T00:00:03.000000 (0000000003.00000)''' %\ + POLICIES[0].name + self.assertEqual(sorted(out.getvalue().strip().split('\n')), + sorted(exp_out.strip().split('\n'))) + def test_print_ring_locations_invalid_args(self): self.assertRaises(ValueError, print_ring_locations, None, 'dir', 'acct') @@ -423,14 +505,8 @@ No user metadata found in db file''' % POLICIES[0].name '1', 'b47', 'dc5be2aa4347a22a0fee6bc7de505b47', 'dc5be2aa4347a22a0fee6bc7de505b47.db') - try: - print_info('account', db_file, swift_dir=self.testdir) - except Exception: - exp_raised = True - if exp_raised: - self.fail("Unexpected exception raised") - else: - self.assertGreater(len(out.getvalue().strip()), 800) + print_info('account', db_file, swift_dir=self.testdir) + self.assertGreater(len(out.getvalue().strip()), 800) controller = ContainerController( {'devices': self.testdir, 'mount_check': 'false'}) diff --git a/test/unit/common/test_db_replicator.py b/test/unit/common/test_db_replicator.py index 20c5d6738a..e4fdce8e91 100644 --- a/test/unit/common/test_db_replicator.py +++ b/test/unit/common/test_db_replicator.py @@ -274,6 +274,9 @@ class FakeBroker(object): self.put_timestamp = put_timestamp self.delete_timestamp = delete_timestamp + def get_brokers(self): + return [self] + class FakeAccountBroker(FakeBroker): db_type = 'account' @@ -1205,7 +1208,7 @@ class TestDBReplicator(unittest.TestCase): unit.mock_check_drive(isdir=True): mock_os.path.exists.side_effect = [False, True] response = rpc.dispatch(('drive', 'part', 'hash'), - ['complete_rsync', 'arg1', 'arg2']) + ['complete_rsync', 'arg1']) expected_calls = [call('/part/ash/hash/hash.db'), call('/drive/tmp/arg1')] self.assertEqual(mock_os.path.exists.call_args_list, @@ -1213,6 +1216,19 @@ class TestDBReplicator(unittest.TestCase): self.assertEqual('204 No Content', response.status) self.assertEqual(204, response.status_int) + with patch('swift.common.db_replicator.os', + new=mock.MagicMock(wraps=os)) as mock_os, \ + unit.mock_check_drive(isdir=True): + mock_os.path.exists.side_effect = [False, True] + response = rpc.dispatch(('drive', 'part', 'hash'), + ['complete_rsync', 'arg1', 'arg2']) + expected_calls = [call('/part/ash/hash/arg2'), + call('/drive/tmp/arg1')] + self.assertEqual(mock_os.path.exists.call_args_list, + expected_calls) + self.assertEqual('204 No Content', response.status) + self.assertEqual(204, response.status_int) + def test_rsync_then_merge_db_does_not_exist(self): rpc = db_replicator.ReplicatorRpc('/', '/', FakeBroker, mount_check=False) @@ -1267,13 +1283,22 @@ class TestDBReplicator(unittest.TestCase): rpc = db_replicator.ReplicatorRpc('/', '/', FakeBroker, mount_check=False) + with patch('swift.common.db_replicator.os', + new=mock.MagicMock(wraps=os)) as mock_os, \ + unit.mock_check_drive(isdir=True): + mock_os.path.exists.return_value = True + response = rpc.complete_rsync('drive', '/data/db.db', ['arg1']) + mock_os.path.exists.assert_called_with('/data/db.db') + self.assertEqual('404 Not Found', response.status) + self.assertEqual(404, response.status_int) + with patch('swift.common.db_replicator.os', new=mock.MagicMock(wraps=os)) as mock_os, \ unit.mock_check_drive(isdir=True): mock_os.path.exists.return_value = True response = rpc.complete_rsync('drive', '/data/db.db', ['arg1', 'arg2']) - mock_os.path.exists.assert_called_with('/data/db.db') + mock_os.path.exists.assert_called_with('/data/arg2') self.assertEqual('404 Not Found', response.status) self.assertEqual(404, response.status_int) @@ -1286,37 +1311,57 @@ class TestDBReplicator(unittest.TestCase): unit.mock_check_drive(isdir=True): mock_os.path.exists.return_value = False response = rpc.complete_rsync('drive', '/data/db.db', - ['arg1', 'arg2']) + ['arg1']) expected_calls = [call('/data/db.db'), call('/drive/tmp/arg1')] self.assertEqual(expected_calls, mock_os.path.exists.call_args_list) self.assertEqual('404 Not Found', response.status) self.assertEqual(404, response.status_int) + with patch('swift.common.db_replicator.os', + new=mock.MagicMock(wraps=os)) as mock_os, \ + unit.mock_check_drive(isdir=True): + mock_os.path.exists.return_value = False + response = rpc.complete_rsync('drive', '/data/db.db', + ['arg1', 'arg2']) + expected_calls = [call('/data/arg2'), call('/drive/tmp/arg1')] + self.assertEqual(expected_calls, + mock_os.path.exists.call_args_list) + self.assertEqual('404 Not Found', response.status) + self.assertEqual(404, response.status_int) + def test_complete_rsync_rename(self): rpc = db_replicator.ReplicatorRpc('/', '/', FakeBroker, mount_check=False) - def mock_exists(path): - if path == '/data/db.db': - return False - self.assertEqual('/drive/tmp/arg1', path) - return True - def mock_renamer(old, new): - self.assertEqual('/drive/tmp/arg1', old) - self.assertEqual('/data/db.db', new) + renamer_calls.append((old, new)) self._patch(patch.object, db_replicator, 'renamer', mock_renamer) + renamer_calls = [] + with patch('swift.common.db_replicator.os', + new=mock.MagicMock(wraps=os)) as mock_os, \ + unit.mock_check_drive(isdir=True): + mock_os.path.exists.side_effect = [False, True] + response = rpc.complete_rsync('drive', '/data/db.db', + ['arg1']) + self.assertEqual('204 No Content', response.status) + self.assertEqual(204, response.status_int) + self.assertEqual(('/drive/tmp/arg1', '/data/db.db'), renamer_calls[0]) + self.assertFalse(renamer_calls[1:]) + + renamer_calls = [] with patch('swift.common.db_replicator.os', new=mock.MagicMock(wraps=os)) as mock_os, \ unit.mock_check_drive(isdir=True): mock_os.path.exists.side_effect = [False, True] response = rpc.complete_rsync('drive', '/data/db.db', ['arg1', 'arg2']) - self.assertEqual('204 No Content', response.status) - self.assertEqual(204, response.status_int) + self.assertEqual('204 No Content', response.status) + self.assertEqual(204, response.status_int) + self.assertEqual(('/drive/tmp/arg1', '/data/arg2'), renamer_calls[0]) + self.assertFalse(renamer_calls[1:]) def test_replicator_sync_with_broker_replication_missing_table(self): rpc = db_replicator.ReplicatorRpc('/', '/', FakeBroker, @@ -1675,10 +1720,10 @@ class TestDBReplicator(unittest.TestCase): db_file = __file__ replicator = TestReplicator({}) replicator._http_connect(node, partition, db_file) + expected_hsh = os.path.basename(db_file).split('.', 1)[0] + expected_hsh = expected_hsh.split('_', 1)[0] db_replicator.ReplConnection.assert_has_calls([ - mock.call(node, partition, - os.path.basename(db_file).split('.', 1)[0], - replicator.logger)]) + mock.call(node, partition, expected_hsh, replicator.logger)]) class TestHandoffsOnly(unittest.TestCase): diff --git a/test/unit/common/test_utils.py b/test/unit/common/test_utils.py index 1f495ac876..33a437262a 100644 --- a/test/unit/common/test_utils.py +++ b/test/unit/common/test_utils.py @@ -3878,6 +3878,47 @@ cluster_dfw1 = http://dfw1.host/v1/ found = utils.find_shard_range('l', overlapping_ranges) self.assertEqual(found, ktol) + def test_parse_db_filename(self): + actual = utils.parse_db_filename('hash.db') + self.assertEqual(('hash', None, '.db'), actual) + actual = utils.parse_db_filename('hash_1234567890.12345.db') + self.assertEqual(('hash', '1234567890.12345', '.db'), actual) + actual = utils.parse_db_filename( + '/dev/containers/part/ash/hash/hash_1234567890.12345.db') + self.assertEqual(('hash', '1234567890.12345', '.db'), actual) + self.assertRaises(ValueError, utils.parse_db_filename, '/path/to/dir/') + # These shouldn't come up in practice; included for completeness + self.assertEqual(utils.parse_db_filename('hashunder_.db'), + ('hashunder', '', '.db')) + self.assertEqual(utils.parse_db_filename('lots_of_underscores.db'), + ('lots', 'of', '.db')) + + def test_make_db_file_path(self): + epoch = utils.Timestamp.now() + actual = utils.make_db_file_path('hash.db', epoch) + self.assertEqual('hash_%s.db' % epoch.internal, actual) + + actual = utils.make_db_file_path('hash_oldepoch.db', epoch) + self.assertEqual('hash_%s.db' % epoch.internal, actual) + + actual = utils.make_db_file_path('/path/to/hash.db', epoch) + self.assertEqual('/path/to/hash_%s.db' % epoch.internal, actual) + + epoch = utils.Timestamp.now() + actual = utils.make_db_file_path(actual, epoch) + self.assertEqual('/path/to/hash_%s.db' % epoch.internal, actual) + + # epochs shouldn't have offsets + epoch = utils.Timestamp.now(offset=10) + actual = utils.make_db_file_path(actual, epoch) + self.assertEqual('/path/to/hash_%s.db' % epoch.normal, actual) + + self.assertRaises(ValueError, utils.make_db_file_path, + '/path/to/hash.db', 'bad epoch') + + self.assertRaises(ValueError, utils.make_db_file_path, + '/path/to/hash.db', None) + def test_modify_priority(self): pid = os.getpid() logger = debug_logger() @@ -4168,6 +4209,70 @@ cluster_dfw1 = http://dfw1.host/v1/ # iterators self.assertListEqual([1, 4, 6, 2, 5, 7, 3, 8, 9], got) + @with_tempdir + def test_get_db_files(self, tempdir): + dbdir = os.path.join(tempdir, 'dbdir') + self.assertEqual([], utils.get_db_files(dbdir)) + path_1 = os.path.join(dbdir, 'dbfile.db') + self.assertEqual([], utils.get_db_files(path_1)) + os.mkdir(dbdir) + self.assertEqual([], utils.get_db_files(path_1)) + with open(path_1, 'wb'): + pass + self.assertEqual([path_1], utils.get_db_files(path_1)) + + path_2 = os.path.join(dbdir, 'dbfile_2.db') + self.assertEqual([path_1], utils.get_db_files(path_2)) + + with open(path_2, 'wb'): + pass + + self.assertEqual([path_1, path_2], utils.get_db_files(path_1)) + self.assertEqual([path_1, path_2], utils.get_db_files(path_2)) + + path_3 = os.path.join(dbdir, 'dbfile_3.db') + self.assertEqual([path_1, path_2], utils.get_db_files(path_3)) + + with open(path_3, 'wb'): + pass + + self.assertEqual([path_1, path_2, path_3], utils.get_db_files(path_1)) + self.assertEqual([path_1, path_2, path_3], utils.get_db_files(path_2)) + self.assertEqual([path_1, path_2, path_3], utils.get_db_files(path_3)) + + other_hash = os.path.join(dbdir, 'other.db') + self.assertEqual([], utils.get_db_files(other_hash)) + other_hash = os.path.join(dbdir, 'other_1.db') + self.assertEqual([], utils.get_db_files(other_hash)) + + pending = os.path.join(dbdir, 'dbfile.pending') + self.assertEqual([path_1, path_2, path_3], utils.get_db_files(pending)) + + with open(pending, 'wb'): + pass + self.assertEqual([path_1, path_2, path_3], utils.get_db_files(pending)) + + self.assertEqual([path_1, path_2, path_3], utils.get_db_files(path_1)) + self.assertEqual([path_1, path_2, path_3], utils.get_db_files(path_2)) + self.assertEqual([path_1, path_2, path_3], utils.get_db_files(path_3)) + self.assertEqual([], utils.get_db_files(dbdir)) + + os.unlink(path_1) + self.assertEqual([path_2, path_3], utils.get_db_files(path_1)) + self.assertEqual([path_2, path_3], utils.get_db_files(path_2)) + self.assertEqual([path_2, path_3], utils.get_db_files(path_3)) + + os.unlink(path_2) + self.assertEqual([path_3], utils.get_db_files(path_1)) + self.assertEqual([path_3], utils.get_db_files(path_2)) + self.assertEqual([path_3], utils.get_db_files(path_3)) + + os.unlink(path_3) + self.assertEqual([], utils.get_db_files(path_1)) + self.assertEqual([], utils.get_db_files(path_2)) + self.assertEqual([], utils.get_db_files(path_3)) + self.assertEqual([], utils.get_db_files('/path/to/nowhere')) + class ResellerConfReader(unittest.TestCase): diff --git a/test/unit/container/test_backend.py b/test/unit/container/test_backend.py index 9a3d86d4d4..0069f812e1 100644 --- a/test/unit/container/test_backend.py +++ b/test/unit/container/test_backend.py @@ -14,9 +14,10 @@ # limitations under the License. """ Tests for swift.container.backend """ - +import errno import os import hashlib +import inspect import unittest from time import sleep, time from uuid import uuid4 @@ -27,20 +28,34 @@ import sqlite3 import pickle import json +from swift.common.exceptions import LockTimeout from swift.container.backend import ContainerBroker, \ - update_new_item_from_existing -from swift.common.utils import Timestamp, encode_timestamps, hash_path + update_new_item_from_existing, UNSHARDED, SHARDING, SHARDED, \ + COLLAPSED, SHARD_LISTING_STATES, SHARD_UPDATE_STATES +from swift.common.db import DatabaseAlreadyExists, GreenDBConnection +from swift.common.utils import Timestamp, encode_timestamps, hash_path, \ + ShardRange, make_db_file_path from swift.common.storage_policy import POLICIES import mock +from test import annotate_failure from test.unit import (patch_policies, with_tempdir, make_timestamp_iter, - EMPTY_ETAG) + EMPTY_ETAG, FakeLogger, mock_timestamp_now) from test.unit.common import test_db class TestContainerBroker(unittest.TestCase): """Tests for ContainerBroker""" + expected_db_tables = {'outgoing_sync', 'incoming_sync', 'object', + 'sqlite_sequence', 'policy_stat', + 'container_info', 'shard_range'} + + def _assert_shard_ranges(self, broker, expected, include_own=False): + actual = broker.get_shard_ranges(include_deleted=True, + include_own=include_own) + self.assertEqual([dict(sr) for sr in expected], + [dict(sr) for sr in actual]) def test_creation(self): # Test ContainerBroker.__init__ @@ -51,6 +66,23 @@ class TestContainerBroker(unittest.TestCase): curs = conn.cursor() curs.execute('SELECT 1') self.assertEqual(curs.fetchall()[0][0], 1) + curs.execute("SELECT name FROM sqlite_master WHERE type='table';") + self.assertEqual(self.expected_db_tables, + {row[0] for row in curs.fetchall()}) + # check the update trigger + broker.put_object('blah', Timestamp.now().internal, 0, 'text/plain', + 'etag', 0, 0) + with broker.get() as conn: + with self.assertRaises(sqlite3.DatabaseError) as cm: + conn.execute('UPDATE object SET name="blah";') + self.assertIn('UPDATE not allowed', str(cm.exception)) + if 'shard_range' in self.expected_db_tables: + # check the update trigger + broker.merge_shard_ranges(broker.get_own_shard_range()) + with broker.get() as conn: + with self.assertRaises(sqlite3.DatabaseError) as cm: + conn.execute('UPDATE shard_range SET name="blah";') + self.assertIn('UPDATE not allowed', str(cm.exception)) @patch_policies def test_storage_policy_property(self): @@ -91,16 +123,296 @@ class TestContainerBroker(unittest.TestCase): pass self.assertTrue(broker.conn is None) - def test_empty(self): + @with_tempdir + def test_is_deleted(self, tempdir): + # Test ContainerBroker.is_deleted() and get_info_is_deleted() + ts_iter = make_timestamp_iter() + db_path = os.path.join( + tempdir, 'part', 'suffix', 'hash', 'container.db') + broker = ContainerBroker(db_path, account='a', container='c') + broker.initialize(next(ts_iter).internal, 0) + + self.assertFalse(broker.is_deleted()) + broker.delete_db(next(ts_iter).internal) + self.assertTrue(broker.is_deleted()) + + def check_object_counted(broker_to_test, broker_with_object): + obj = {'name': 'o', 'created_at': next(ts_iter).internal, + 'size': 0, 'content_type': 'text/plain', 'etag': EMPTY_ETAG, + 'deleted': 0} + broker_with_object.merge_items([dict(obj)]) + self.assertFalse(broker_to_test.is_deleted()) + info, deleted = broker_to_test.get_info_is_deleted() + self.assertFalse(deleted) + self.assertEqual(1, info['object_count']) + obj.update({'created_at': next(ts_iter).internal, 'deleted': 1}) + broker_with_object.merge_items([dict(obj)]) + self.assertTrue(broker_to_test.is_deleted()) + info, deleted = broker_to_test.get_info_is_deleted() + self.assertTrue(deleted) + self.assertEqual(0, info['object_count']) + + def check_object_not_counted(broker): + obj = {'name': 'o', 'created_at': next(ts_iter).internal, + 'size': 0, 'content_type': 'text/plain', 'etag': EMPTY_ETAG, + 'deleted': 0} + broker.merge_items([dict(obj)]) + self.assertTrue(broker.is_deleted()) + info, deleted = broker.get_info_is_deleted() + self.assertTrue(deleted) + self.assertEqual(0, info['object_count']) + obj.update({'created_at': next(ts_iter).internal, 'deleted': 1}) + broker.merge_items([dict(obj)]) + self.assertTrue(broker.is_deleted()) + info, deleted = broker.get_info_is_deleted() + self.assertTrue(deleted) + self.assertEqual(0, info['object_count']) + + def check_shard_ranges_not_counted(): + sr = ShardRange('.shards_a/shard_c', next(ts_iter), object_count=0) + sr.update_meta(13, 99, meta_timestamp=next(ts_iter)) + for state in ShardRange.STATES: + sr.update_state(state, state_timestamp=next(ts_iter)) + broker.merge_shard_ranges([sr]) + self.assertTrue(broker.is_deleted()) + info, deleted = broker.get_info_is_deleted() + self.assertTrue(deleted) + self.assertEqual(0, info['object_count']) + + def check_shard_ranges_counted(): + sr = ShardRange('.shards_a/shard_c', next(ts_iter), object_count=0) + sr.update_meta(13, 99, meta_timestamp=next(ts_iter)) + counted_states = (ShardRange.ACTIVE, ShardRange.SHARDING, + ShardRange.SHRINKING) + for state in ShardRange.STATES: + sr.update_state(state, state_timestamp=next(ts_iter)) + broker.merge_shard_ranges([sr]) + expected = state not in counted_states + self.assertEqual(expected, broker.is_deleted()) + info, deleted = broker.get_info_is_deleted() + self.assertEqual(expected, deleted) + self.assertEqual(0 if expected else 13, info['object_count']) + + sr.update_meta(0, 0, meta_timestamp=next(ts_iter)) + for state in ShardRange.STATES: + sr.update_state(state, state_timestamp=next(ts_iter)) + broker.merge_shard_ranges([sr]) + self.assertTrue(broker.is_deleted()) + info, deleted = broker.get_info_is_deleted() + self.assertTrue(deleted) + self.assertEqual(0, info['object_count']) + + # unsharded + check_object_counted(broker, broker) + check_shard_ranges_not_counted() + + # move to sharding state + broker.enable_sharding(next(ts_iter)) + self.assertTrue(broker.set_sharding_state()) + broker.delete_db(next(ts_iter).internal) + self.assertTrue(broker.is_deleted()) + + # check object in retiring db is considered + check_object_counted(broker, broker.get_brokers()[0]) + self.assertTrue(broker.is_deleted()) + check_shard_ranges_not_counted() + # misplaced object in fresh db is not considered + check_object_not_counted(broker) + + # move to sharded state + self.assertTrue(broker.set_sharded_state()) + check_object_not_counted(broker) + check_shard_ranges_counted() + + # own shard range has no influence + own_sr = broker.get_own_shard_range() + own_sr.update_meta(3, 4, meta_timestamp=next(ts_iter)) + broker.merge_shard_ranges([own_sr]) + self.assertTrue(broker.is_deleted()) + + @with_tempdir + def test_empty(self, tempdir): # Test ContainerBroker.empty - broker = ContainerBroker(':memory:', account='a', container='c') - broker.initialize(Timestamp('1').internal, 0) + ts_iter = make_timestamp_iter() + db_path = os.path.join( + tempdir, 'part', 'suffix', 'hash', 'container.db') + broker = ContainerBroker(db_path, account='a', container='c') + broker.initialize(next(ts_iter).internal, 0) + self.assertTrue(broker.is_root_container()) + + def check_object_counted(broker_to_test, broker_with_object): + obj = {'name': 'o', 'created_at': next(ts_iter).internal, + 'size': 0, 'content_type': 'text/plain', 'etag': EMPTY_ETAG, + 'deleted': 0} + broker_with_object.merge_items([dict(obj)]) + self.assertFalse(broker_to_test.empty()) + # and delete it + obj.update({'created_at': next(ts_iter).internal, 'deleted': 1}) + broker_with_object.merge_items([dict(obj)]) + self.assertTrue(broker_to_test.empty()) + + def check_shard_ranges_not_counted(): + sr = ShardRange('.shards_a/shard_c', next(ts_iter), object_count=0) + sr.update_meta(13, 99, meta_timestamp=next(ts_iter)) + for state in ShardRange.STATES: + sr.update_state(state, state_timestamp=next(ts_iter)) + broker.merge_shard_ranges([sr]) + self.assertTrue(broker.empty()) + + # empty other shard ranges do not influence result + sr.update_meta(0, 0, meta_timestamp=next(ts_iter)) + for state in ShardRange.STATES: + sr.update_state(state, state_timestamp=next(ts_iter)) + broker.merge_shard_ranges([sr]) + self.assertTrue(broker.empty()) + self.assertTrue(broker.empty()) - broker.put_object('o', Timestamp.now().internal, 0, 'text/plain', - 'd41d8cd98f00b204e9800998ecf8427e') - self.assertTrue(not broker.empty()) - sleep(.00001) - broker.delete_object('o', Timestamp.now().internal) + check_object_counted(broker, broker) + check_shard_ranges_not_counted() + + # own shard range is not considered for object count + own_sr = broker.get_own_shard_range() + self.assertEqual(0, own_sr.object_count) + broker.merge_shard_ranges([own_sr]) + self.assertTrue(broker.empty()) + + broker.put_object('o', next(ts_iter).internal, 0, 'text/plain', + EMPTY_ETAG) + own_sr = broker.get_own_shard_range() + self.assertEqual(1, own_sr.object_count) + broker.merge_shard_ranges([own_sr]) + self.assertFalse(broker.empty()) + broker.delete_object('o', next(ts_iter).internal) + self.assertTrue(broker.empty()) + + # have own shard range but in state ACTIVE + self.assertEqual(ShardRange.ACTIVE, own_sr.state) + check_object_counted(broker, broker) + check_shard_ranges_not_counted() + + def check_shard_ranges_counted(): + # other shard range is considered + sr = ShardRange('.shards_a/shard_c', next(ts_iter), object_count=0) + sr.update_meta(13, 99, meta_timestamp=next(ts_iter)) + counted_states = (ShardRange.ACTIVE, ShardRange.SHARDING, + ShardRange.SHRINKING) + for state in ShardRange.STATES: + sr.update_state(state, state_timestamp=next(ts_iter)) + broker.merge_shard_ranges([sr]) + self.assertEqual(state not in counted_states, broker.empty()) + + # empty other shard ranges do not influence result + sr.update_meta(0, 0, meta_timestamp=next(ts_iter)) + for state in ShardRange.STATES: + sr.update_state(state, state_timestamp=next(ts_iter)) + broker.merge_shard_ranges([sr]) + self.assertTrue(broker.empty()) + + # enable sharding + broker.enable_sharding(next(ts_iter)) + check_object_counted(broker, broker) + check_shard_ranges_counted() + + # move to sharding state + self.assertTrue(broker.set_sharding_state()) + # check object in retiring db is considered + check_object_counted(broker, broker.get_brokers()[0]) + self.assertTrue(broker.empty()) + # as well as misplaced objects in fresh db + check_object_counted(broker, broker) + check_shard_ranges_counted() + + # move to sharded state + self.assertTrue(broker.set_sharded_state()) + self.assertTrue(broker.empty()) + check_object_counted(broker, broker) + check_shard_ranges_counted() + + # own shard range still has no influence + own_sr = broker.get_own_shard_range() + own_sr.update_meta(3, 4, meta_timestamp=next(ts_iter)) + broker.merge_shard_ranges([own_sr]) + self.assertTrue(broker.empty()) + + @with_tempdir + def test_empty_shard_container(self, tempdir): + # Test ContainerBroker.empty for a shard container where shard range + # usage should not be considered + ts_iter = make_timestamp_iter() + db_path = os.path.join( + tempdir, 'part', 'suffix', 'hash', 'container.db') + broker = ContainerBroker(db_path, account='.shards_a', container='cc') + broker.initialize(next(ts_iter).internal, 0) + broker.set_sharding_sysmeta('Root', 'a/c') + self.assertFalse(broker.is_root_container()) + + def check_object_counted(broker_to_test, broker_with_object): + obj = {'name': 'o', 'created_at': next(ts_iter).internal, + 'size': 0, 'content_type': 'text/plain', 'etag': EMPTY_ETAG, + 'deleted': 0} + broker_with_object.merge_items([dict(obj)]) + self.assertFalse(broker_to_test.empty()) + # and delete it + obj.update({'created_at': next(ts_iter).internal, 'deleted': 1}) + broker_with_object.merge_items([dict(obj)]) + self.assertTrue(broker_to_test.empty()) + + self.assertTrue(broker.empty()) + check_object_counted(broker, broker) + + # own shard range is not considered for object count + own_sr = broker.get_own_shard_range() + self.assertEqual(0, own_sr.object_count) + broker.merge_shard_ranges([own_sr]) + self.assertTrue(broker.empty()) + + broker.put_object('o', next(ts_iter).internal, 0, 'text/plain', + EMPTY_ETAG) + own_sr = broker.get_own_shard_range() + self.assertEqual(1, own_sr.object_count) + broker.merge_shard_ranges([own_sr]) + self.assertFalse(broker.empty()) + broker.delete_object('o', next(ts_iter).internal) + self.assertTrue(broker.empty()) + + def check_shard_ranges_not_counted(): + sr = ShardRange('.shards_a/shard_c', next(ts_iter), object_count=0) + sr.update_meta(13, 99, meta_timestamp=next(ts_iter)) + for state in ShardRange.STATES: + sr.update_state(state, state_timestamp=next(ts_iter)) + broker.merge_shard_ranges([sr]) + self.assertTrue(broker.empty()) + + # empty other shard ranges do not influence result + sr.update_meta(0, 0, meta_timestamp=next(ts_iter)) + for state in ShardRange.STATES: + sr.update_state(state, state_timestamp=next(ts_iter)) + broker.merge_shard_ranges([sr]) + self.assertTrue(broker.empty()) + + check_shard_ranges_not_counted() + + # move to sharding state + broker.enable_sharding(next(ts_iter)) + self.assertTrue(broker.set_sharding_state()) + + # check object in retiring db is considered + check_object_counted(broker, broker.get_brokers()[0]) + self.assertTrue(broker.empty()) + # as well as misplaced objects in fresh db + check_object_counted(broker, broker) + check_shard_ranges_not_counted() + + # move to sharded state + self.assertTrue(broker.set_sharded_state()) + self.assertTrue(broker.empty()) + check_object_counted(broker, broker) + check_shard_ranges_not_counted() + + # own shard range still has no influence + own_sr = broker.get_own_shard_range() + own_sr.update_meta(3, 4, meta_timestamp=next(ts_iter)) + broker.merge_shard_ranges([own_sr]) self.assertTrue(broker.empty()) def test_reclaim(self): @@ -163,6 +475,77 @@ class TestContainerBroker(unittest.TestCase): broker.reclaim(Timestamp.now().internal, time()) broker.delete_db(Timestamp.now().internal) + @with_tempdir + def test_reclaim_deadlock(self, tempdir): + db_path = os.path.join( + tempdir, 'part', 'suffix', 'hash', '%s.db' % uuid4()) + broker = ContainerBroker(db_path, account='a', container='c') + broker.initialize(Timestamp(100).internal, 0) + # there's some magic count here that causes the failure, something + # about the size of object records and sqlite page size maybe? + count = 23000 + for i in range(count): + obj_name = 'o%d' % i + ts = Timestamp(200).internal + broker.delete_object(obj_name, ts) + broker._commit_puts() + with broker.get() as conn: + self.assertEqual(conn.execute( + "SELECT count(*) FROM object").fetchone()[0], count) + # make a broker whose container attribute is not yet set so that + # reclaim will need to query info to set it + broker = ContainerBroker(db_path, timeout=1) + # verify that reclaim doesn't get deadlocked and timeout + broker.reclaim(300, 300) + # check all objects were reclaimed + with broker.get() as conn: + self.assertEqual(conn.execute( + "SELECT count(*) FROM object" + ).fetchone()[0], 0) + + @with_tempdir + def test_reclaim_shard_ranges(self, tempdir): + ts_iter = make_timestamp_iter() + db_path = os.path.join( + tempdir, 'part', 'suffix', 'hash', '%s.db' % uuid4()) + broker = ContainerBroker(db_path, account='a', container='c') + broker.initialize(next(ts_iter).internal, 0) + older = next(ts_iter) + same = next(ts_iter) + newer = next(ts_iter) + shard_ranges = [ + ShardRange('.shards_a/older_deleted', older.internal, '', 'a', + deleted=True), + ShardRange('.shards_a/same_deleted', same.internal, 'a', 'b', + deleted=True), + ShardRange('.shards_a/newer_deleted', newer.internal, 'b', 'c', + deleted=True), + ShardRange('.shards_a/older', older.internal, 'c', 'd'), + ShardRange('.shards_a/same', same.internal, 'd', 'e'), + ShardRange('.shards_a/newer', newer.internal, 'e', 'f'), + # own shard range is never reclaimed, even if deleted + ShardRange('a/c', older.internal, '', '', deleted=True)] + broker.merge_shard_ranges( + random.sample(shard_ranges, len(shard_ranges))) + + def assert_row_count(expected): + with broker.get() as conn: + res = conn.execute("SELECT count(*) FROM shard_range") + self.assertEqual(expected, res.fetchone()[0]) + + broker.reclaim(older.internal, older.internal) + assert_row_count(7) + self._assert_shard_ranges(broker, shard_ranges, include_own=True) + broker.reclaim(older.internal, same.internal) + assert_row_count(6) + self._assert_shard_ranges(broker, shard_ranges[1:], include_own=True) + broker.reclaim(older.internal, newer.internal) + assert_row_count(5) + self._assert_shard_ranges(broker, shard_ranges[2:], include_own=True) + broker.reclaim(older.internal, next(ts_iter).internal) + assert_row_count(4) + self._assert_shard_ranges(broker, shard_ranges[3:], include_own=True) + def test_get_info_is_deleted(self): ts = make_timestamp_iter() start = next(ts) @@ -179,7 +562,8 @@ class TestContainerBroker(unittest.TestCase): self.assertEqual(info['delete_timestamp'], '0') if self.__class__ in (TestContainerBrokerBeforeMetadata, TestContainerBrokerBeforeXSync, - TestContainerBrokerBeforeSPI): + TestContainerBrokerBeforeSPI, + TestContainerBrokerBeforeShardRanges): self.assertEqual(info['status_changed_at'], '0') else: self.assertEqual(info['status_changed_at'], @@ -431,6 +815,273 @@ class TestContainerBroker(unittest.TestCase): self.assertEqual(conn.execute( "SELECT deleted FROM object").fetchone()[0], 0) + def test_merge_shard_range_single_record(self): + # Test ContainerBroker.merge_shard_range + broker = ContainerBroker(':memory:', account='a', container='c') + broker.initialize(Timestamp('1').internal, 0) + + ts_iter = make_timestamp_iter() + # Stash these for later + old_put_timestamp = next(ts_iter).internal + old_delete_timestamp = next(ts_iter).internal + + # Create initial object + timestamp = next(ts_iter).internal + meta_timestamp = next(ts_iter).internal + broker.merge_shard_ranges( + ShardRange('"a/{}"', timestamp, + 'low', 'up', meta_timestamp=meta_timestamp)) + with broker.get() as conn: + self.assertEqual(conn.execute( + "SELECT name FROM shard_range").fetchone()[0], + '"a/{}"') + self.assertEqual(conn.execute( + "SELECT timestamp FROM shard_range").fetchone()[0], + timestamp) + self.assertEqual(conn.execute( + "SELECT meta_timestamp FROM shard_range").fetchone()[0], + meta_timestamp) + self.assertEqual(conn.execute( + "SELECT lower FROM shard_range").fetchone()[0], 'low') + self.assertEqual(conn.execute( + "SELECT upper FROM shard_range").fetchone()[0], 'up') + self.assertEqual(conn.execute( + "SELECT deleted FROM shard_range").fetchone()[0], 0) + self.assertEqual(conn.execute( + "SELECT object_count FROM shard_range").fetchone()[0], 0) + self.assertEqual(conn.execute( + "SELECT bytes_used FROM shard_range").fetchone()[0], 0) + + # Reput same event + broker.merge_shard_ranges( + ShardRange('"a/{}"', timestamp, + 'low', 'up', meta_timestamp=meta_timestamp)) + with broker.get() as conn: + self.assertEqual(conn.execute( + "SELECT name FROM shard_range").fetchone()[0], + '"a/{}"') + self.assertEqual(conn.execute( + "SELECT timestamp FROM shard_range").fetchone()[0], + timestamp) + self.assertEqual(conn.execute( + "SELECT meta_timestamp FROM shard_range").fetchone()[0], + meta_timestamp) + self.assertEqual(conn.execute( + "SELECT lower FROM shard_range").fetchone()[0], 'low') + self.assertEqual(conn.execute( + "SELECT upper FROM shard_range").fetchone()[0], 'up') + self.assertEqual(conn.execute( + "SELECT deleted FROM shard_range").fetchone()[0], 0) + self.assertEqual(conn.execute( + "SELECT object_count FROM shard_range").fetchone()[0], 0) + self.assertEqual(conn.execute( + "SELECT bytes_used FROM shard_range").fetchone()[0], 0) + + # Put new event + timestamp = next(ts_iter).internal + meta_timestamp = next(ts_iter).internal + broker.merge_shard_ranges( + ShardRange('"a/{}"', timestamp, + 'lower', 'upper', 1, 2, meta_timestamp=meta_timestamp)) + with broker.get() as conn: + self.assertEqual(conn.execute( + "SELECT name FROM shard_range").fetchone()[0], + '"a/{}"') + self.assertEqual(conn.execute( + "SELECT timestamp FROM shard_range").fetchone()[0], + timestamp) + self.assertEqual(conn.execute( + "SELECT meta_timestamp FROM shard_range").fetchone()[0], + meta_timestamp) + self.assertEqual(conn.execute( + "SELECT lower FROM shard_range").fetchone()[0], 'lower') + self.assertEqual(conn.execute( + "SELECT upper FROM shard_range").fetchone()[0], 'upper') + self.assertEqual(conn.execute( + "SELECT deleted FROM shard_range").fetchone()[0], 0) + self.assertEqual(conn.execute( + "SELECT object_count FROM shard_range").fetchone()[0], 1) + self.assertEqual(conn.execute( + "SELECT bytes_used FROM shard_range").fetchone()[0], 2) + + # Put old event + broker.merge_shard_ranges( + ShardRange('"a/{}"', old_put_timestamp, + 'lower', 'upper', 1, 2, meta_timestamp=meta_timestamp)) + with broker.get() as conn: + self.assertEqual(conn.execute( + "SELECT name FROM shard_range").fetchone()[0], + '"a/{}"') + self.assertEqual(conn.execute( + "SELECT timestamp FROM shard_range").fetchone()[0], + timestamp) # Not old_put_timestamp! + self.assertEqual(conn.execute( + "SELECT meta_timestamp FROM shard_range").fetchone()[0], + meta_timestamp) + self.assertEqual(conn.execute( + "SELECT lower FROM shard_range").fetchone()[0], 'lower') + self.assertEqual(conn.execute( + "SELECT upper FROM shard_range").fetchone()[0], 'upper') + self.assertEqual(conn.execute( + "SELECT deleted FROM shard_range").fetchone()[0], 0) + self.assertEqual(conn.execute( + "SELECT object_count FROM shard_range").fetchone()[0], 1) + self.assertEqual(conn.execute( + "SELECT bytes_used FROM shard_range").fetchone()[0], 2) + + # Put old delete event + broker.merge_shard_ranges( + ShardRange('"a/{}"', old_delete_timestamp, + 'lower', 'upper', meta_timestamp=meta_timestamp, + deleted=1)) + with broker.get() as conn: + self.assertEqual(conn.execute( + "SELECT name FROM shard_range").fetchone()[0], + '"a/{}"') + self.assertEqual(conn.execute( + "SELECT timestamp FROM shard_range").fetchone()[0], + timestamp) # Not old_delete_timestamp! + self.assertEqual(conn.execute( + "SELECT meta_timestamp FROM shard_range").fetchone()[0], + meta_timestamp) + self.assertEqual(conn.execute( + "SELECT lower FROM shard_range").fetchone()[0], 'lower') + self.assertEqual(conn.execute( + "SELECT upper FROM shard_range").fetchone()[0], 'upper') + self.assertEqual(conn.execute( + "SELECT deleted FROM shard_range").fetchone()[0], 0) + self.assertEqual(conn.execute( + "SELECT object_count FROM shard_range").fetchone()[0], 1) + self.assertEqual(conn.execute( + "SELECT bytes_used FROM shard_range").fetchone()[0], 2) + + # Put new delete event + timestamp = next(ts_iter).internal + broker.merge_shard_ranges( + ShardRange('"a/{}"', timestamp, + 'lower', 'upper', meta_timestamp=meta_timestamp, + deleted=1)) + with broker.get() as conn: + self.assertEqual(conn.execute( + "SELECT name FROM shard_range").fetchone()[0], + '"a/{}"') + self.assertEqual(conn.execute( + "SELECT timestamp FROM shard_range").fetchone()[0], + timestamp) + self.assertEqual(conn.execute( + "SELECT deleted FROM shard_range").fetchone()[0], 1) + + # Put new event + timestamp = next(ts_iter).internal + meta_timestamp = next(ts_iter).internal + broker.merge_shard_ranges( + ShardRange('"a/{}"', timestamp, + 'lowerer', 'upperer', 3, 4, + meta_timestamp=meta_timestamp)) + with broker.get() as conn: + self.assertEqual(conn.execute( + "SELECT name FROM shard_range").fetchone()[0], + '"a/{}"') + self.assertEqual(conn.execute( + "SELECT timestamp FROM shard_range").fetchone()[0], + timestamp) + self.assertEqual(conn.execute( + "SELECT meta_timestamp FROM shard_range").fetchone()[0], + meta_timestamp) + self.assertEqual(conn.execute( + "SELECT lower FROM shard_range").fetchone()[0], 'lowerer') + self.assertEqual(conn.execute( + "SELECT upper FROM shard_range").fetchone()[0], 'upperer') + self.assertEqual(conn.execute( + "SELECT deleted FROM shard_range").fetchone()[0], 0) + self.assertEqual(conn.execute( + "SELECT object_count FROM shard_range").fetchone()[0], 3) + self.assertEqual(conn.execute( + "SELECT bytes_used FROM shard_range").fetchone()[0], 4) + + # We'll use this later + in_between_timestamp = next(ts_iter).internal + + # New update event, meta_timestamp increases + meta_timestamp = next(ts_iter).internal + broker.merge_shard_ranges( + ShardRange('"a/{}"', timestamp, + 'lowerer', 'upperer', 3, 4, + meta_timestamp=meta_timestamp)) + with broker.get() as conn: + self.assertEqual(conn.execute( + "SELECT name FROM shard_range").fetchone()[0], + '"a/{}"') + self.assertEqual(conn.execute( + "SELECT timestamp FROM shard_range").fetchone()[0], + timestamp) + self.assertEqual(conn.execute( + "SELECT meta_timestamp FROM shard_range").fetchone()[0], + meta_timestamp) + self.assertEqual(conn.execute( + "SELECT lower FROM shard_range").fetchone()[0], 'lowerer') + self.assertEqual(conn.execute( + "SELECT upper FROM shard_range").fetchone()[0], 'upperer') + self.assertEqual(conn.execute( + "SELECT deleted FROM shard_range").fetchone()[0], 0) + self.assertEqual(conn.execute( + "SELECT object_count FROM shard_range").fetchone()[0], 3) + self.assertEqual(conn.execute( + "SELECT bytes_used FROM shard_range").fetchone()[0], 4) + + # Put event from after last put but before last post + timestamp = in_between_timestamp + broker.merge_shard_ranges( + ShardRange('"a/{}"', timestamp, + 'lowererer', 'uppererer', 5, 6, + meta_timestamp=meta_timestamp)) + with broker.get() as conn: + self.assertEqual(conn.execute( + "SELECT name FROM shard_range").fetchone()[0], + '"a/{}"') + self.assertEqual(conn.execute( + "SELECT timestamp FROM shard_range").fetchone()[0], + timestamp) + self.assertEqual(conn.execute( + "SELECT meta_timestamp FROM shard_range").fetchone()[0], + meta_timestamp) + self.assertEqual(conn.execute( + "SELECT lower FROM shard_range").fetchone()[0], 'lowererer') + self.assertEqual(conn.execute( + "SELECT upper FROM shard_range").fetchone()[0], 'uppererer') + self.assertEqual(conn.execute( + "SELECT deleted FROM shard_range").fetchone()[0], 0) + self.assertEqual(conn.execute( + "SELECT object_count FROM shard_range").fetchone()[0], 5) + self.assertEqual(conn.execute( + "SELECT bytes_used FROM shard_range").fetchone()[0], 6) + + def test_merge_shard_ranges_deleted(self): + # Test ContainerBroker.merge_shard_ranges sets deleted attribute + ts_iter = make_timestamp_iter() + broker = ContainerBroker(':memory:', account='a', container='c') + broker.initialize(Timestamp('1').internal, 0) + # put shard range + broker.merge_shard_ranges(ShardRange('a/o', next(ts_iter).internal)) + with broker.get() as conn: + self.assertEqual(conn.execute( + "SELECT count(*) FROM shard_range " + "WHERE deleted = 0").fetchone()[0], 1) + self.assertEqual(conn.execute( + "SELECT count(*) FROM shard_range " + "WHERE deleted = 1").fetchone()[0], 0) + + # delete shard range + broker.merge_shard_ranges(ShardRange('a/o', next(ts_iter).internal, + deleted=1)) + with broker.get() as conn: + self.assertEqual(conn.execute( + "SELECT count(*) FROM shard_range " + "WHERE deleted = 0").fetchone()[0], 0) + self.assertEqual(conn.execute( + "SELECT count(*) FROM shard_range " + "WHERE deleted = 1").fetchone()[0], 1) + def test_make_tuple_for_pickle(self): record = {'name': 'obj', 'created_at': '1234567890.12345', @@ -618,6 +1269,194 @@ class TestContainerBroker(unittest.TestCase): broker = ContainerBroker(':memory:', account='a', container='c') self._test_put_object_multiple_encoded_timestamps(broker) + @with_tempdir + def test_get_db_state(self, tempdir): + acct = 'account' + cont = 'container' + hsh = hash_path(acct, cont) + db_file = "%s.db" % hsh + epoch = Timestamp.now() + fresh_db_file = "%s_%s.db" % (hsh, epoch.normal) + db_path = os.path.join(tempdir, db_file) + fresh_db_path = os.path.join(tempdir, fresh_db_file) + ts = Timestamp.now() + + # First test NOTFOUND state + broker = ContainerBroker(db_path, account=acct, container=cont) + self.assertEqual(broker.get_db_state(), 'not_found') + + # Test UNSHARDED state, that is when db_file exists and fresh_db_file + # doesn't + broker.initialize(ts.internal, 0) + self.assertEqual(broker.get_db_state(), 'unsharded') + + # Test the SHARDING state, this is the period when both the db_file and + # the fresh_db_file exist + fresh_broker = ContainerBroker(fresh_db_path, account=acct, + container=cont, force_db_file=True) + fresh_broker.initialize(ts.internal, 0) + own_shard_range = fresh_broker.get_own_shard_range() + own_shard_range.update_state(ShardRange.SHARDING) + own_shard_range.epoch = epoch + shard_range = ShardRange( + '.shards_%s/%s' % (acct, cont), Timestamp.now()) + fresh_broker.merge_shard_ranges([own_shard_range, shard_range]) + + self.assertEqual(fresh_broker.get_db_state(), 'sharding') + # old broker will also change state if we reload its db files + broker.reload_db_files() + self.assertEqual(broker.get_db_state(), 'sharding') + + # Test the SHARDED state, this is when only fresh_db_file exists. + os.unlink(db_path) + fresh_broker.reload_db_files() + self.assertEqual(fresh_broker.get_db_state(), 'sharded') + + # Test the COLLAPSED state, this is when only fresh_db_file exists. + shard_range.deleted = 1 + shard_range.timestamp = Timestamp.now() + fresh_broker.merge_shard_ranges([shard_range]) + self.assertEqual(fresh_broker.get_db_state(), 'collapsed') + + # back to UNSHARDED if the desired epoch changes + own_shard_range.update_state(ShardRange.SHRINKING, + state_timestamp=Timestamp.now()) + own_shard_range.epoch = Timestamp.now() + fresh_broker.merge_shard_ranges([own_shard_range]) + self.assertEqual(fresh_broker.get_db_state(), 'unsharded') + + @with_tempdir + def test_db_file(self, tempdir): + acct = 'account' + cont = 'continer' + hsh = hash_path(acct, cont) + db_file = "%s.db" % hsh + ts_epoch = Timestamp.now() + fresh_db_file = "%s_%s.db" % (hsh, ts_epoch.normal) + db_path = os.path.join(tempdir, db_file) + fresh_db_path = os.path.join(tempdir, fresh_db_file) + ts = Timestamp.now() + + # First test NOTFOUND state, this will return the db_file passed + # in the constructor + def check_unfound_db_files(broker, init_db_file): + self.assertEqual(init_db_file, broker.db_file) + self.assertEqual(broker._db_file, db_path) + self.assertFalse(os.path.exists(db_path)) + self.assertFalse(os.path.exists(fresh_db_path)) + self.assertEqual([], broker.db_files) + + broker = ContainerBroker(db_path, account=acct, container=cont) + check_unfound_db_files(broker, db_path) + broker = ContainerBroker(fresh_db_path, account=acct, container=cont) + check_unfound_db_files(broker, fresh_db_path) + + # Test UNSHARDED state, that is when db_file exists and fresh_db_file + # doesn't, so it should return the db_path + def check_unsharded_db_files(broker): + self.assertEqual(broker.db_file, db_path) + self.assertEqual(broker._db_file, db_path) + self.assertTrue(os.path.exists(db_path)) + self.assertFalse(os.path.exists(fresh_db_path)) + self.assertEqual([db_path], broker.db_files) + + broker = ContainerBroker(db_path, account=acct, container=cont) + broker.initialize(ts.internal, 0) + check_unsharded_db_files(broker) + broker = ContainerBroker(fresh_db_path, account=acct, container=cont) + check_unsharded_db_files(broker) + # while UNSHARDED db_path is still used despite giving fresh_db_path + # to init, so we cannot initialize this broker + with self.assertRaises(DatabaseAlreadyExists): + broker.initialize(ts.internal, 0) + + # Test the SHARDING state, this is the period when both the db_file and + # the fresh_db_file exist, in this case it should return the + # fresh_db_path. + def check_sharding_db_files(broker): + self.assertEqual(broker.db_file, fresh_db_path) + self.assertEqual(broker._db_file, db_path) + self.assertTrue(os.path.exists(db_path)) + self.assertTrue(os.path.exists(fresh_db_path)) + self.assertEqual([db_path, fresh_db_path], broker.db_files) + + # Use force_db_file to have db_shard_path created when initializing + broker = ContainerBroker(fresh_db_path, account=acct, + container=cont, force_db_file=True) + self.assertEqual([db_path], broker.db_files) + broker.initialize(ts.internal, 0) + check_sharding_db_files(broker) + broker = ContainerBroker(db_path, account=acct, container=cont) + check_sharding_db_files(broker) + broker = ContainerBroker(fresh_db_path, account=acct, container=cont) + check_sharding_db_files(broker) + + # force_db_file can be used to open db_path specifically + forced_broker = ContainerBroker(db_path, account=acct, + container=cont, force_db_file=True) + self.assertEqual(forced_broker.db_file, db_path) + self.assertEqual(forced_broker._db_file, db_path) + + def check_sharded_db_files(broker): + self.assertEqual(broker.db_file, fresh_db_path) + self.assertEqual(broker._db_file, db_path) + self.assertFalse(os.path.exists(db_path)) + self.assertTrue(os.path.exists(fresh_db_path)) + self.assertEqual([fresh_db_path], broker.db_files) + + # Test the SHARDED state, this is when only fresh_db_file exists, so + # obviously this should return the fresh_db_path + os.unlink(db_path) + broker.reload_db_files() + check_sharded_db_files(broker) + broker = ContainerBroker(db_path, account=acct, container=cont) + check_sharded_db_files(broker) + + @with_tempdir + def test_sharding_initiated_and_required(self, tempdir): + db_path = os.path.join( + tempdir, 'part', 'suffix', 'hash', '%s.db' % uuid4()) + broker = ContainerBroker(db_path, account='a', container='c') + broker.initialize(Timestamp.now().internal, 0) + # no shard ranges + self.assertIs(False, broker.sharding_initiated()) + self.assertIs(False, broker.sharding_required()) + # only own shard range + own_sr = broker.get_own_shard_range() + for state in ShardRange.STATES: + own_sr.update_state(state, state_timestamp=Timestamp.now()) + broker.merge_shard_ranges(own_sr) + self.assertIs(False, broker.sharding_initiated()) + self.assertIs(False, broker.sharding_required()) + + # shard ranges, still ACTIVE + own_sr.update_state(ShardRange.ACTIVE, + state_timestamp=Timestamp.now()) + broker.merge_shard_ranges(own_sr) + broker.merge_shard_ranges(ShardRange('.shards_a/cc', Timestamp.now())) + self.assertIs(False, broker.sharding_initiated()) + self.assertIs(False, broker.sharding_required()) + + # shard ranges and SHARDING, SHRINKING or SHARDED + broker.enable_sharding(Timestamp.now()) + self.assertTrue(broker.set_sharding_state()) + self.assertIs(True, broker.sharding_initiated()) + self.assertIs(True, broker.sharding_required()) + + epoch = broker.db_epoch + own_sr.update_state(ShardRange.SHRINKING, + state_timestamp=Timestamp.now()) + own_sr.epoch = epoch + broker.merge_shard_ranges(own_sr) + self.assertIs(True, broker.sharding_initiated()) + self.assertIs(True, broker.sharding_required()) + + own_sr.update_state(ShardRange.SHARDED) + broker.merge_shard_ranges(own_sr) + self.assertTrue(broker.set_sharded_state()) + self.assertIs(True, broker.sharding_initiated()) + self.assertIs(False, broker.sharding_required()) + @with_tempdir def test_put_object_multiple_encoded_timestamps_using_file(self, tempdir): # Test ContainerBroker.put_object with differing data, content-type @@ -968,7 +1807,8 @@ class TestContainerBroker(unittest.TestCase): self.assertEqual(info['delete_timestamp'], '0') if self.__class__ in (TestContainerBrokerBeforeMetadata, TestContainerBrokerBeforeXSync, - TestContainerBrokerBeforeSPI): + TestContainerBrokerBeforeSPI, + TestContainerBrokerBeforeShardRanges): self.assertEqual(info['status_changed_at'], '0') else: self.assertEqual(info['status_changed_at'], @@ -1014,6 +1854,84 @@ class TestContainerBroker(unittest.TestCase): self.assertEqual(info['x_container_sync_point1'], -1) self.assertEqual(info['x_container_sync_point2'], -1) + @with_tempdir + def test_get_info_sharding_states(self, tempdir): + ts_iter = make_timestamp_iter() + db_path = os.path.join(tempdir, 'part', 'suffix', 'hash', 'hash.db') + broker = ContainerBroker( + db_path, account='myaccount', container='mycontainer') + broker.initialize(next(ts_iter).internal, 0) + broker.put_object('o1', next(ts_iter).internal, 123, 'text/plain', + 'fake etag') + sr = ShardRange('.shards_a/c', next(ts_iter)) + broker.merge_shard_ranges(sr) + + def check_info(expected): + errors = [] + for k, v in expected.items(): + if info.get(k) != v: + errors.append((k, v, info.get(k))) + if errors: + self.fail('Mismatches: %s' % ', '.join( + ['%s should be %s but got %s' % error + for error in errors])) + + # unsharded + with mock.patch.object( + broker, 'get_shard_usage') as mock_get_shard_usage: + info = broker.get_info() + mock_get_shard_usage.assert_not_called() + check_info({'account': 'myaccount', + 'container': 'mycontainer', + 'object_count': 1, + 'bytes_used': 123, + 'db_state': 'unsharded'}) + + # sharding + epoch = next(ts_iter) + broker.enable_sharding(epoch) + self.assertTrue(broker.set_sharding_state()) + broker.put_object('o2', next(ts_iter).internal, 1, 'text/plain', + 'fake etag') + broker.put_object('o3', next(ts_iter).internal, 320, 'text/plain', + 'fake etag') + with mock.patch.object( + broker, 'get_shard_usage') as mock_get_shard_usage: + info = broker.get_info() + mock_get_shard_usage.assert_not_called() + check_info({'account': 'myaccount', + 'container': 'mycontainer', + 'object_count': 1, + 'bytes_used': 123, + 'db_state': 'sharding'}) + + # sharded + self.assertTrue(broker.set_sharded_state()) + shard_stats = {'object_count': 1001, 'bytes_used': 3003} + with mock.patch.object( + broker, 'get_shard_usage') as mock_get_shard_usage: + mock_get_shard_usage.return_value = shard_stats + info = broker.get_info() + mock_get_shard_usage.assert_called_once_with() + check_info({'account': 'myaccount', + 'container': 'mycontainer', + 'object_count': 1001, + 'bytes_used': 3003, + 'db_state': 'sharded'}) + + # collapsed + sr.set_deleted(next(ts_iter)) + broker.merge_shard_ranges(sr) + with mock.patch.object( + broker, 'get_shard_usage') as mock_get_shard_usage: + info = broker.get_info() + mock_get_shard_usage.assert_not_called() + check_info({'account': 'myaccount', + 'container': 'mycontainer', + 'object_count': 2, + 'bytes_used': 321, + 'db_state': 'collapsed'}) + def test_set_x_syncs(self): broker = ContainerBroker(':memory:', account='test1', container='test2') @@ -1095,6 +2013,105 @@ class TestContainerBroker(unittest.TestCase): self.assertEqual(info['reported_object_count'], 2) self.assertEqual(info['reported_bytes_used'], 1123) + def test_get_objects(self): + broker = ContainerBroker(':memory:', account='a', container='c') + broker.initialize(Timestamp('1').internal, 0) + ts_iter = make_timestamp_iter() + objects_0 = [{'name': 'obj_0_%d' % i, + 'created_at': next(ts_iter).normal, + 'content_type': 'text/plain', + 'etag': 'etag_%d' % i, + 'size': 1024 * i, + 'deleted': i % 2, + 'storage_policy_index': 0 + } for i in range(1, 8)] + objects_1 = [{'name': 'obj_1_%d' % i, + 'created_at': next(ts_iter).normal, + 'content_type': 'text/plain', + 'etag': 'etag_%d' % i, + 'size': 1024 * i, + 'deleted': i % 2, + 'storage_policy_index': 1 + } for i in range(1, 8)] + # merge_objects mutates items + broker.merge_items([dict(obj) for obj in objects_0 + objects_1]) + + actual = broker.get_objects() + self.assertEqual(objects_0 + objects_1, actual) + + with mock.patch('swift.container.backend.CONTAINER_LISTING_LIMIT', 2): + actual = broker.get_objects() + self.assertEqual(objects_0[:2], actual) + + with mock.patch('swift.container.backend.CONTAINER_LISTING_LIMIT', 2): + actual = broker.get_objects(limit=9) + self.assertEqual(objects_0 + objects_1[:2], actual) + + actual = broker.get_objects(marker=objects_0[2]['name']) + self.assertEqual(objects_0[3:] + objects_1, actual) + + actual = broker.get_objects(end_marker=objects_0[2]['name']) + self.assertEqual(objects_0[:2], actual) + + actual = broker.get_objects(include_deleted=True) + self.assertEqual(objects_0[::2] + objects_1[::2], actual) + + actual = broker.get_objects(include_deleted=False) + self.assertEqual(objects_0[1::2] + objects_1[1::2], actual) + + actual = broker.get_objects(include_deleted=None) + self.assertEqual(objects_0 + objects_1, actual) + + def test_get_objects_since_row(self): + ts_iter = make_timestamp_iter() + broker = ContainerBroker(':memory:', account='a', container='c') + broker.initialize(Timestamp('1').internal, 0) + obj_names = ['obj%03d' % i for i in range(20)] + timestamps = [next(ts_iter) for o in obj_names] + for name, timestamp in zip(obj_names, timestamps): + broker.put_object(name, timestamp.internal, + 0, 'text/plain', EMPTY_ETAG) + broker._commit_puts() # ensure predictable row order + timestamps = [next(ts_iter) for o in obj_names[10:]] + for name, timestamp in zip(obj_names[10:], timestamps): + broker.put_object(name, timestamp.internal, + 0, 'text/plain', EMPTY_ETAG, deleted=1) + broker._commit_puts() # ensure predictable row order + + # sanity check + self.assertEqual(30, broker.get_max_row()) + actual = broker.get_objects() + self.assertEqual(obj_names, [o['name'] for o in actual]) + + # all rows included + actual = broker.get_objects(since_row=None) + self.assertEqual(obj_names, [o['name'] for o in actual]) + + actual = broker.get_objects(since_row=-1) + self.assertEqual(obj_names, [o['name'] for o in actual]) + + # selected rows + for since_row in range(10): + actual = broker.get_objects(since_row=since_row) + with annotate_failure(since_row): + self.assertEqual(obj_names[since_row:], + [o['name'] for o in actual]) + + for since_row in range(10, 20): + actual = broker.get_objects(since_row=since_row) + with annotate_failure(since_row): + self.assertEqual(obj_names[10:], + [o['name'] for o in actual]) + + for since_row in range(20, len(obj_names) + 1): + actual = broker.get_objects(since_row=since_row) + with annotate_failure(since_row): + self.assertEqual(obj_names[since_row - 10:], + [o['name'] for o in actual]) + + self.assertFalse(broker.get_objects(end_marker=obj_names[5], + since_row=5)) + def test_list_objects_iter(self): # Test ContainerBroker.list_objects_iter broker = ContainerBroker(':memory:', account='a', container='c') @@ -1827,6 +2844,21 @@ class TestContainerBroker(unittest.TestCase): self.assertEqual(['a', 'b', 'c'], sorted([rec['name'] for rec in items])) + @with_tempdir + def test_merge_items_is_green(self, tempdir): + ts = make_timestamp_iter() + db_path = os.path.join(tempdir, 'container.db') + + broker = ContainerBroker(db_path, account='a', container='c') + broker.initialize(next(ts).internal, 1) + + broker.put_object('b', next(ts).internal, 0, 'text/plain', + EMPTY_ETAG) + + with mock.patch('swift.container.backend.tpool') as mock_tpool: + broker.get_info() + mock_tpool.execute.assert_called_once() + def test_merge_items_overwrite_unicode(self): # test DatabaseBroker.merge_items snowman = u'\N{SNOWMAN}'.encode('utf-8') @@ -1937,7 +2969,8 @@ class TestContainerBroker(unittest.TestCase): self.assertEqual(0, info['bytes_used']) if self.__class__ in (TestContainerBrokerBeforeMetadata, TestContainerBrokerBeforeXSync, - TestContainerBrokerBeforeSPI): + TestContainerBrokerBeforeSPI, + TestContainerBrokerBeforeShardRanges): self.assertEqual(info['status_changed_at'], '0') else: self.assertEqual(timestamp.internal, info['status_changed_at']) @@ -2116,6 +3149,1237 @@ class TestContainerBroker(unittest.TestCase): self.assertEqual(ts.internal, broker.get_info()['put_timestamp']) self.assertEqual(0, broker.get_info()['storage_policy_index']) + epoch = Timestamp.now() + broker = ContainerBroker.create_broker(tempdir, 0, 'a', 'c3', + epoch=epoch) + hsh = hash_path('a', 'c3') + expected_path = os.path.join( + tempdir, 'containers', '0', hsh[-3:], + hsh, '%s_%s.db' % (hsh, epoch.internal)) + self.assertEqual(expected_path, broker.db_file) + + @with_tempdir + def test_pending_file_name(self, tempdir): + # pending file should have same name for sharded or unsharded db + expected_pending_path = os.path.join(tempdir, 'container.db.pending') + + db_path = os.path.join(tempdir, 'container.db') + fresh_db_path = os.path.join(tempdir, 'container_epoch.db') + + def do_test(given_db_file, expected_db_file): + broker = ContainerBroker(given_db_file, account='a', container='c') + self.assertEqual(expected_pending_path, broker.pending_file) + self.assertEqual(expected_db_file, broker.db_file) + + # no files exist + do_test(db_path, db_path) + do_test(fresh_db_path, fresh_db_path) + + # only container.db exists - unsharded + with open(db_path, 'wb'): + pass + do_test(db_path, db_path) + do_test(fresh_db_path, db_path) + + # container.db and container_shard.db exist - sharding + with open(fresh_db_path, 'wb'): + pass + do_test(db_path, fresh_db_path) + do_test(fresh_db_path, fresh_db_path) + + # only container_shard.db exists - sharded + os.unlink(db_path) + do_test(db_path, fresh_db_path) + do_test(fresh_db_path, fresh_db_path) + + @with_tempdir + def test_sharding_sysmeta(self, tempdir): + db_path = os.path.join(tempdir, 'container.db') + broker = ContainerBroker( + db_path, account='myaccount', container='mycontainer') + broker.initialize(Timestamp.now().internal) + + expected = 'aaa/ccc' + with mock_timestamp_now() as now: + broker.set_sharding_sysmeta('Root', expected) + actual = broker.metadata + self.assertEqual([expected, now.internal], + actual.get('X-Container-Sysmeta-Shard-Root')) + self.assertEqual(expected, broker.get_sharding_sysmeta('Root')) + + expected = {'key': 'value'} + with mock_timestamp_now() as now: + broker.set_sharding_sysmeta('test', expected) + actual = broker.metadata + self.assertEqual([expected, now.internal], + actual.get('X-Container-Sysmeta-Shard-test')) + self.assertEqual(expected, broker.get_sharding_sysmeta('test')) + + @with_tempdir + def test_path(self, tempdir): + ts_iter = make_timestamp_iter() + db_path = os.path.join(tempdir, 'container.db') + broker = ContainerBroker( + db_path, account='myaccount', container='mycontainer') + broker.initialize(next(ts_iter).internal, 1) + # make sure we can cope with unitialized account and container + broker.account = broker.container = None + self.assertEqual('myaccount/mycontainer', broker.path) + + @with_tempdir + def test_root_account_container_path(self, tempdir): + ts_iter = make_timestamp_iter() + db_path = os.path.join(tempdir, 'container.db') + broker = ContainerBroker( + db_path, account='root_a', container='root_c') + broker.initialize(next(ts_iter).internal, 1) + # make sure we can cope with unitialized account and container + broker.account = broker.container = None + + self.assertEqual('root_a', broker.root_account) + self.assertEqual('root_c', broker.root_container) + self.assertEqual('root_a/root_c', broker.root_path) + self.assertTrue(broker.is_root_container()) + self.assertEqual('root_a', broker.account) # sanity check + self.assertEqual('root_c', broker.container) # sanity check + + # we don't expect root containers to have this sysmeta set but if it is + # the broker should still behave like a root container + metadata = { + 'X-Container-Sysmeta-Shard-Root': + ('root_a/root_c', next(ts_iter).internal)} + broker = ContainerBroker( + db_path, account='root_a', container='root_c') + broker.update_metadata(metadata) + broker.account = broker.container = None + self.assertEqual('root_a', broker.root_account) + self.assertEqual('root_c', broker.root_container) + self.assertEqual('root_a/root_c', broker.root_path) + self.assertTrue(broker.is_root_container()) + + # if root is marked deleted, it still considers itself to be a root + broker.delete_db(next(ts_iter).internal) + self.assertEqual('root_a', broker.root_account) + self.assertEqual('root_c', broker.root_container) + self.assertEqual('root_a/root_c', broker.root_path) + self.assertTrue(broker.is_root_container()) + # check the values are not just being cached + broker = ContainerBroker(db_path) + self.assertEqual('root_a', broker.root_account) + self.assertEqual('root_c', broker.root_container) + self.assertEqual('root_a/root_c', broker.root_path) + self.assertTrue(broker.is_root_container()) + + # check a shard container + db_path = os.path.join(tempdir, 'shard_container.db') + broker = ContainerBroker( + db_path, account='.shards_root_a', container='c_shard') + broker.initialize(next(ts_iter).internal, 1) + # now the metadata is significant... + metadata = { + 'X-Container-Sysmeta-Shard-Root': + ('root_a/root_c', next(ts_iter).internal)} + broker.update_metadata(metadata) + broker.account = broker.container = None + broker._root_account = broker._root_container = None + + self.assertEqual('root_a', broker.root_account) + self.assertEqual('root_c', broker.root_container) + self.assertEqual('root_a/root_c', broker.root_path) + self.assertFalse(broker.is_root_container()) + + # check validation + def check_validation(root_value): + metadata = { + 'X-Container-Sysmeta-Shard-Root': + (root_value, next(ts_iter).internal)} + broker.update_metadata(metadata) + broker.account = broker.container = None + broker._root_account = broker._root_container = None + with self.assertRaises(ValueError) as cm: + broker.root_account + self.assertIn('Expected X-Container-Sysmeta-Shard-Root', + str(cm.exception)) + with self.assertRaises(ValueError): + broker.root_container + + check_validation('root_a') + check_validation('/root_a') + check_validation('/root_a/root_c') + check_validation('/root_a/root_c/blah') + check_validation('/') + + def test_resolve_shard_range_states(self): + self.assertIsNone(ContainerBroker.resolve_shard_range_states(None)) + self.assertIsNone(ContainerBroker.resolve_shard_range_states([])) + + for state_num, state_name in ShardRange.STATES.items(): + self.assertEqual({state_num}, + ContainerBroker.resolve_shard_range_states( + [state_name])) + self.assertEqual({state_num}, + ContainerBroker.resolve_shard_range_states( + [state_num])) + + self.assertEqual(set(ShardRange.STATES), + ContainerBroker.resolve_shard_range_states( + ShardRange.STATES_BY_NAME)) + + self.assertEqual( + set(ShardRange.STATES), + ContainerBroker.resolve_shard_range_states(ShardRange.STATES)) + + # check aliases + self.assertEqual( + {ShardRange.CLEAVED, ShardRange.ACTIVE, ShardRange.SHARDING, + ShardRange.SHRINKING}, + ContainerBroker.resolve_shard_range_states(['listing'])) + + self.assertEqual( + {ShardRange.CLEAVED, ShardRange.ACTIVE, ShardRange.SHARDING, + ShardRange.SHRINKING}, + ContainerBroker.resolve_shard_range_states(['listing', 'active'])) + + self.assertEqual( + {ShardRange.CLEAVED, ShardRange.ACTIVE, ShardRange.SHARDING, + ShardRange.SHRINKING, ShardRange.CREATED}, + ContainerBroker.resolve_shard_range_states(['listing', 'created'])) + + self.assertEqual( + {ShardRange.CREATED, ShardRange.CLEAVED, ShardRange.ACTIVE, + ShardRange.SHARDING}, + ContainerBroker.resolve_shard_range_states(['updating'])) + + self.assertEqual( + {ShardRange.CREATED, ShardRange.CLEAVED, ShardRange.ACTIVE, + ShardRange.SHARDING, ShardRange.SHRINKING}, + ContainerBroker.resolve_shard_range_states( + ['updating', 'listing'])) + + def check_bad_value(value): + with self.assertRaises(ValueError) as cm: + ContainerBroker.resolve_shard_range_states(value) + self.assertIn('Invalid state', str(cm.exception)) + + check_bad_value(['bad_state', 'active']) + check_bad_value(['']) + check_bad_value('active') + + @with_tempdir + def test_get_shard_ranges(self, tempdir): + ts_iter = make_timestamp_iter() + db_path = os.path.join(tempdir, 'container.db') + broker = ContainerBroker(db_path, account='a', container='c') + broker.initialize(next(ts_iter).internal, 0) + + # no rows + self.assertFalse(broker.get_shard_ranges()) + # check that a default own shard range is not generated + self.assertFalse(broker.get_shard_ranges(include_own=True)) + + # merge row for own shard range + own_shard_range = ShardRange(broker.path, next(ts_iter), 'l', 'u', + state=ShardRange.SHARDING) + broker.merge_shard_ranges([own_shard_range]) + self.assertFalse(broker.get_shard_ranges()) + self.assertFalse(broker.get_shard_ranges(include_own=False)) + + actual = broker.get_shard_ranges(include_own=True) + self.assertEqual([dict(sr) for sr in [own_shard_range]], + [dict(sr) for sr in actual]) + + # merge rows for other shard ranges + shard_ranges = [ + ShardRange('.a/c0', next(ts_iter), 'a', 'c'), + ShardRange('.a/c1', next(ts_iter), 'c', 'd'), + ShardRange('.a/c2', next(ts_iter), 'd', 'f', + state=ShardRange.ACTIVE), + ShardRange('.a/c3', next(ts_iter), 'e', 'f', deleted=1, + state=ShardRange.SHARDED,), + ShardRange('.a/c4', next(ts_iter), 'f', 'h', + state=ShardRange.CREATED), + ShardRange('.a/c5', next(ts_iter), 'h', 'j', deleted=1) + ] + broker.merge_shard_ranges(shard_ranges) + actual = broker.get_shard_ranges() + undeleted = shard_ranges[:3] + shard_ranges[4:5] + self.assertEqual([dict(sr) for sr in undeleted], + [dict(sr) for sr in actual]) + + actual = broker.get_shard_ranges(include_deleted=True) + self.assertEqual([dict(sr) for sr in shard_ranges], + [dict(sr) for sr in actual]) + + actual = broker.get_shard_ranges(reverse=True) + self.assertEqual([dict(sr) for sr in reversed(undeleted)], + [dict(sr) for sr in actual]) + + actual = broker.get_shard_ranges(marker='c', end_marker='e') + self.assertEqual([dict(sr) for sr in shard_ranges[1:3]], + [dict(sr) for sr in actual]) + + actual = broker.get_shard_ranges(marker='c', end_marker='e', + states=ShardRange.ACTIVE) + self.assertEqual([dict(sr) for sr in shard_ranges[2:3]], + [dict(sr) for sr in actual]) + + actual = broker.get_shard_ranges(marker='e', end_marker='e') + self.assertFalse([dict(sr) for sr in actual]) + + actual = broker.get_shard_ranges(includes='f') + self.assertEqual([dict(sr) for sr in shard_ranges[2:3]], + [dict(sr) for sr in actual]) + + actual = broker.get_shard_ranges(includes='i') + self.assertFalse(actual) + + actual = broker.get_shard_ranges( + states=[ShardRange.CREATED, ShardRange.ACTIVE]) + self.assertEqual( + [dict(sr) for sr in [shard_ranges[2], shard_ranges[4]]], + [dict(sr) for sr in actual]) + + actual = broker.get_shard_ranges(exclude_states=ShardRange.CREATED) + self.assertEqual([dict(sr) for sr in shard_ranges[:3]], + [dict(sr) for sr in actual]) + + actual = broker.get_shard_ranges( + exclude_states=[ShardRange.CREATED, ShardRange.ACTIVE]) + self.assertEqual([dict(sr) for sr in shard_ranges[:2]], + [dict(sr) for sr in actual]) + + # exclude_states takes precedence + actual = broker.get_shard_ranges( + states=ShardRange.CREATED, exclude_states=ShardRange.CREATED) + self.assertEqual([dict(sr) for sr in shard_ranges[:3]], + [dict(sr) for sr in actual]) + + actual = broker.get_shard_ranges(states=[ShardRange.CREATED], + exclude_states=[ShardRange.ACTIVE]) + self.assertEqual([dict(sr) for sr in shard_ranges[4:5]], + [dict(sr) for sr in actual]) + + # get everything + actual = broker.get_shard_ranges(include_own=True) + self.assertEqual([dict(sr) for sr in undeleted + [own_shard_range]], + [dict(sr) for sr in actual]) + + # get just own range + actual = broker.get_shard_ranges(include_own=True, exclude_others=True) + self.assertEqual([dict(sr) for sr in [own_shard_range]], + [dict(sr) for sr in actual]) + + # exclude_states overrides include_own + actual = broker.get_shard_ranges(include_own=True, + exclude_states=ShardRange.SHARDING, + exclude_others=True) + self.assertFalse(actual) + + # if you ask for nothing you'll get nothing + actual = broker.get_shard_ranges( + include_own=False, exclude_others=True) + self.assertFalse(actual) + + @with_tempdir + def test_get_shard_ranges_with_sharding_overlaps(self, tempdir): + ts_iter = make_timestamp_iter() + db_path = os.path.join(tempdir, 'container.db') + broker = ContainerBroker(db_path, account='a', container='c') + broker.initialize(next(ts_iter).internal, 0) + shard_ranges = [ + ShardRange('.shards_a/c0', next(ts_iter), 'a', 'd', + state=ShardRange.ACTIVE), + ShardRange('.shards_a/c1_0', next(ts_iter), 'd', 'g', + state=ShardRange.CLEAVED), + ShardRange('.shards_a/c1_1', next(ts_iter), 'g', 'j', + state=ShardRange.CLEAVED), + ShardRange('.shards_a/c1_2', next(ts_iter), 'j', 'm', + state=ShardRange.CREATED), + ShardRange('.shards_a/c1', next(ts_iter), 'd', 'm', + state=ShardRange.SHARDING), + ShardRange('.shards_a/c2', next(ts_iter), 'm', '', + state=ShardRange.ACTIVE), + ] + broker.merge_shard_ranges( + random.sample(shard_ranges, len(shard_ranges))) + actual = broker.get_shard_ranges() + self.assertEqual([dict(sr) for sr in shard_ranges], + [dict(sr) for sr in actual]) + + actual = broker.get_shard_ranges(states=SHARD_LISTING_STATES) + self.assertEqual( + [dict(sr) for sr in shard_ranges[:3] + shard_ranges[4:]], + [dict(sr) for sr in actual]) + + actual = broker.get_shard_ranges(states=SHARD_UPDATE_STATES, + includes='e') + self.assertEqual([shard_ranges[1]], actual) + actual = broker.get_shard_ranges(states=SHARD_UPDATE_STATES, + includes='j') + self.assertEqual([shard_ranges[2]], actual) + actual = broker.get_shard_ranges(states=SHARD_UPDATE_STATES, + includes='k') + self.assertEqual([shard_ranges[3]], actual) + + @with_tempdir + def test_get_shard_ranges_with_shrinking_overlaps(self, tempdir): + ts_iter = make_timestamp_iter() + db_path = os.path.join(tempdir, 'container.db') + broker = ContainerBroker(db_path, account='a', container='c') + broker.initialize(next(ts_iter).internal, 0) + shard_ranges = [ + ShardRange('.shards_a/c0', next(ts_iter), 'a', 'k', + state=ShardRange.ACTIVE), + ShardRange('.shards_a/c1', next(ts_iter), 'k', 'm', + state=ShardRange.SHRINKING), + ShardRange('.shards_a/c2', next(ts_iter), 'k', 't', + state=ShardRange.ACTIVE), + ShardRange('.shards_a/c3', next(ts_iter), 't', '', + state=ShardRange.ACTIVE), + ] + broker.merge_shard_ranges( + random.sample(shard_ranges, len(shard_ranges))) + actual = broker.get_shard_ranges() + self.assertEqual([dict(sr) for sr in shard_ranges], + [dict(sr) for sr in actual]) + + actual = broker.get_shard_ranges(states=SHARD_UPDATE_STATES, + includes='l') + self.assertEqual([shard_ranges[2]], actual) + + @with_tempdir + def test_get_own_shard_range(self, tempdir): + ts_iter = make_timestamp_iter() + db_path = os.path.join(tempdir, 'container.db') + broker = ContainerBroker( + db_path, account='.shards_a', container='shard_c') + broker.initialize(next(ts_iter).internal, 0) + + # no row for own shard range - expect entire namespace default + now = Timestamp.now() + expected = ShardRange(broker.path, now, '', '', 0, 0, now, + state=ShardRange.ACTIVE) + with mock.patch('swift.container.backend.Timestamp.now', + return_value=now): + actual = broker.get_own_shard_range() + self.assertEqual(dict(expected), dict(actual)) + + actual = broker.get_own_shard_range(no_default=True) + self.assertIsNone(actual) + + # row for own shard range and others + ts_1 = next(ts_iter) + own_sr = ShardRange(broker.path, ts_1, 'l', 'u') + broker.merge_shard_ranges( + [own_sr, + ShardRange('.a/c1', next(ts_iter), 'b', 'c'), + ShardRange('.a/c2', next(ts_iter), 'c', 'd')]) + expected = ShardRange(broker.path, ts_1, 'l', 'u', 0, 0, now) + with mock.patch('swift.container.backend.Timestamp.now', + return_value=now): + actual = broker.get_own_shard_range() + self.assertEqual(dict(expected), dict(actual)) + + # check stats get updated + broker.put_object( + 'o1', next(ts_iter).internal, 100, 'text/plain', 'etag1') + broker.put_object( + 'o2', next(ts_iter).internal, 99, 'text/plain', 'etag2') + expected = ShardRange( + broker.path, ts_1, 'l', 'u', 2, 199, now) + with mock.patch('swift.container.backend.Timestamp.now', + return_value=now): + actual = broker.get_own_shard_range() + self.assertEqual(dict(expected), dict(actual)) + + # still returned when deleted + delete_ts = next(ts_iter) + own_sr.set_deleted(timestamp=delete_ts) + broker.merge_shard_ranges(own_sr) + with mock.patch('swift.container.backend.Timestamp.now', + return_value=now): + actual = broker.get_own_shard_range() + expected = ShardRange( + broker.path, delete_ts, 'l', 'u', 2, 199, now, deleted=True) + self.assertEqual(dict(expected), dict(actual)) + + # still in table after reclaim_age + broker.reclaim(next(ts_iter).internal, next(ts_iter).internal) + with mock.patch('swift.container.backend.Timestamp.now', + return_value=now): + actual = broker.get_own_shard_range() + self.assertEqual(dict(expected), dict(actual)) + + # entire namespace + ts_2 = next(ts_iter) + broker.merge_shard_ranges( + [ShardRange(broker.path, ts_2, '', '')]) + expected = ShardRange( + broker.path, ts_2, '', '', 2, 199, now) + with mock.patch('swift.container.backend.Timestamp.now', + return_value=now): + actual = broker.get_own_shard_range() + self.assertEqual(dict(expected), dict(actual)) + + @with_tempdir + def test_enable_sharding(self, tempdir): + ts_iter = make_timestamp_iter() + db_path = os.path.join(tempdir, 'container.db') + broker = ContainerBroker( + db_path, account='.shards_a', container='shard_c') + broker.initialize(next(ts_iter).internal, 0) + epoch = next(ts_iter) + broker.enable_sharding(epoch) + own_sr = broker.get_own_shard_range(no_default=True) + self.assertEqual(epoch, own_sr.epoch) + self.assertEqual(epoch, own_sr.state_timestamp) + self.assertEqual(ShardRange.SHARDING, own_sr.state) + + @with_tempdir + def test_get_shard_usage(self, tempdir): + ts_iter = make_timestamp_iter() + shard_range_by_state = dict( + (state, ShardRange('.shards_a/c_%s' % state, next(ts_iter), + str(state), str(state + 1), + 2 * state, 2 * state + 1, 2, + state=state)) + for state in ShardRange.STATES) + + def make_broker(a, c): + db_path = os.path.join(tempdir, '%s.db' % uuid4()) + broker = ContainerBroker(db_path, account=a, container=c) + broker.initialize(next(ts_iter).internal, 0) + broker.set_sharding_sysmeta('Root', 'a/c') + broker.merge_shard_ranges(shard_range_by_state.values()) + return broker + + # make broker appear to be a root container + broker = make_broker('a', 'c') + self.assertTrue(broker.is_root_container()) + included_states = (ShardRange.ACTIVE, ShardRange.SHARDING, + ShardRange.SHRINKING) + included = [shard_range_by_state[state] for state in included_states] + expected = { + 'object_count': sum([sr.object_count for sr in included]), + 'bytes_used': sum([sr.bytes_used for sr in included]) + } + self.assertEqual(expected, broker.get_shard_usage()) + + @with_tempdir + def _check_find_shard_ranges(self, c_lower, c_upper, tempdir): + ts_iter = make_timestamp_iter() + ts_now = Timestamp.now() + container_name = 'test_container' + + def do_test(expected_bounds, expected_last_found, shard_size, limit, + start_index=0, existing=None): + # expected_bounds is a list of tuples (lower, upper, object_count) + # build expected shard ranges + expected_shard_ranges = [ + dict(lower=lower, upper=upper, index=index, + object_count=object_count) + for index, (lower, upper, object_count) + in enumerate(expected_bounds, start_index)] + + with mock.patch('swift.common.utils.time.time', + return_value=float(ts_now.normal)): + ranges, last_found = broker.find_shard_ranges( + shard_size, limit=limit, existing_ranges=existing) + self.assertEqual(expected_shard_ranges, ranges) + self.assertEqual(expected_last_found, last_found) + + db_path = os.path.join(tempdir, 'test_container.db') + broker = ContainerBroker( + db_path, account='a', container=container_name) + # shard size > object count, no objects + broker.initialize(next(ts_iter).internal, 0) + + ts = next(ts_iter) + if c_lower or c_upper: + # testing a shard, so set its own shard range + own_shard_range = ShardRange(broker.path, ts, c_lower, c_upper) + broker.merge_shard_ranges([own_shard_range]) + + self.assertEqual(([], False), broker.find_shard_ranges(10)) + + for i in range(10): + broker.put_object( + 'obj%02d' % i, next(ts_iter).internal, 0, 'text/plain', 'etag') + + expected_bounds = [(c_lower, 'obj04', 5), ('obj04', c_upper, 5)] + do_test(expected_bounds, True, shard_size=5, limit=None) + + expected = [(c_lower, 'obj06', 7), ('obj06', c_upper, 3)] + do_test(expected, True, shard_size=7, limit=None) + expected = [(c_lower, 'obj08', 9), ('obj08', c_upper, 1)] + do_test(expected, True, shard_size=9, limit=None) + # shard size >= object count + do_test([], False, shard_size=10, limit=None) + do_test([], False, shard_size=11, limit=None) + + # check use of limit + do_test([], False, shard_size=4, limit=0) + expected = [(c_lower, 'obj03', 4)] + do_test(expected, False, shard_size=4, limit=1) + expected = [(c_lower, 'obj03', 4), ('obj03', 'obj07', 4)] + do_test(expected, False, shard_size=4, limit=2) + expected = [(c_lower, 'obj03', 4), ('obj03', 'obj07', 4), + ('obj07', c_upper, 2)] + do_test(expected, True, shard_size=4, limit=3) + do_test(expected, True, shard_size=4, limit=4) + do_test(expected, True, shard_size=4, limit=-1) + + # increase object count to 11 + broker.put_object( + 'obj10', next(ts_iter).internal, 0, 'text/plain', 'etag') + expected = [(c_lower, 'obj03', 4), ('obj03', 'obj07', 4), + ('obj07', c_upper, 3)] + do_test(expected, True, shard_size=4, limit=None) + + expected = [(c_lower, 'obj09', 10), ('obj09', c_upper, 1)] + do_test(expected, True, shard_size=10, limit=None) + do_test([], False, shard_size=11, limit=None) + + # now pass in a pre-existing shard range + existing = [ShardRange( + '.shards_a/srange-0', Timestamp.now(), '', 'obj03', + object_count=4, state=ShardRange.FOUND)] + + expected = [('obj03', 'obj07', 4), ('obj07', c_upper, 3)] + do_test(expected, True, shard_size=4, limit=None, start_index=1, + existing=existing) + expected = [('obj03', 'obj07', 4)] + do_test(expected, False, shard_size=4, limit=1, start_index=1, + existing=existing) + # using increased shard size should not distort estimation of progress + expected = [('obj03', 'obj09', 6), ('obj09', c_upper, 1)] + do_test(expected, True, shard_size=6, limit=None, start_index=1, + existing=existing) + + # add another existing... + existing.append(ShardRange( + '.shards_a/srange-1', Timestamp.now(), '', 'obj07', + object_count=4, state=ShardRange.FOUND)) + expected = [('obj07', c_upper, 3)] + do_test(expected, True, shard_size=10, limit=None, start_index=2, + existing=existing) + # an existing shard range not in FOUND state should not distort + # estimation of progress, but may cause final range object count to + # default to shard_size + existing[-1].state = ShardRange.CREATED + existing[-1].object_count = 10 + # there's only 3 objects left to scan but progress cannot be reliably + # calculated, so final shard range has object count of 2 + expected = [('obj07', 'obj09', 2), ('obj09', c_upper, 2)] + do_test(expected, True, shard_size=2, limit=None, start_index=2, + existing=existing) + + # add last shard range so there's none left to find + existing.append(ShardRange( + '.shards_a/srange-2', Timestamp.now(), 'obj07', c_upper, + object_count=4, state=ShardRange.FOUND)) + do_test([], True, shard_size=4, limit=None, existing=existing) + + def test_find_shard_ranges(self): + self._check_find_shard_ranges('', '') + self._check_find_shard_ranges('', 'upper') + self._check_find_shard_ranges('lower', '') + self._check_find_shard_ranges('lower', 'upper') + + @with_tempdir + def test_find_shard_ranges_with_misplaced_objects(self, tempdir): + # verify that misplaced objects outside of a shard's range do not + # influence choice of shard ranges (but do distort the object counts) + ts_iter = make_timestamp_iter() + ts_now = Timestamp.now() + container_name = 'test_container' + + db_path = os.path.join(tempdir, 'test_container.db') + broker = ContainerBroker( + db_path, account='a', container=container_name) + # shard size > object count, no objects + broker.initialize(next(ts_iter).internal, 0) + + ts = next(ts_iter) + own_shard_range = ShardRange(broker.path, ts, 'l', 'u') + broker.merge_shard_ranges([own_shard_range]) + + self.assertEqual(([], False), broker.find_shard_ranges(10)) + + for name in ('a-misplaced', 'm', 'n', 'p', 'q', 'r', 'z-misplaced'): + broker.put_object( + name, next(ts_iter).internal, 0, 'text/plain', 'etag') + + expected_bounds = ( + ('l', 'n', 2), # contains m, n + ('n', 'q', 2), # contains p, q + ('q', 'u', 3) # contains r; object count distorted by 2 misplaced + ) + expected_shard_ranges = [ + dict(lower=lower, upper=upper, index=index, + object_count=object_count) + for index, (lower, upper, object_count) + in enumerate(expected_bounds)] + + with mock.patch('swift.common.utils.time.time', + return_value=float(ts_now.normal)): + actual_shard_ranges, last_found = broker.find_shard_ranges(2, -1) + self.assertEqual(expected_shard_ranges, actual_shard_ranges) + + ts_iter = make_timestamp_iter() + ts_now = Timestamp.now() + container_name = 'test_container' + + @with_tempdir + def test_find_shard_ranges_errors(self, tempdir): + ts_iter = make_timestamp_iter() + db_path = os.path.join(tempdir, 'test_container.db') + broker = ContainerBroker(db_path, account='a', container='c', + logger=FakeLogger()) + broker.initialize(next(ts_iter).internal, 0) + for i in range(2): + broker.put_object( + 'obj%d' % i, next(ts_iter).internal, 0, 'text/plain', 'etag') + + klass = 'swift.container.backend.ContainerBroker' + with mock.patch(klass + '._get_next_shard_range_upper', + side_effect=LockTimeout()): + ranges, last_found = broker.find_shard_ranges(1) + self.assertFalse(ranges) + self.assertFalse(last_found) + lines = broker.logger.get_lines_for_level('error') + self.assertIn('Problem finding shard upper', lines[0]) + self.assertFalse(lines[1:]) + + broker.logger.clear() + with mock.patch(klass + '._get_next_shard_range_upper', + side_effect=sqlite3.OperationalError()): + ranges, last_found = broker.find_shard_ranges(1) + self.assertFalse(last_found) + self.assertFalse(ranges) + lines = broker.logger.get_lines_for_level('error') + self.assertIn('Problem finding shard upper', lines[0]) + self.assertFalse(lines[1:]) + + @with_tempdir + def test_set_db_states(self, tempdir): + ts_iter = make_timestamp_iter() + db_path = os.path.join( + tempdir, 'part', 'suffix', 'hash', 'container.db') + broker = ContainerBroker(db_path, account='a', container='c') + broker.initialize(next(ts_iter).internal, 0) + + # load up the broker with some objects + objects = [{'name': 'obj_%d' % i, + 'created_at': next(ts_iter).normal, + 'content_type': 'text/plain', + 'etag': 'etag_%d' % i, + 'size': 1024 * i, + 'deleted': 0, + 'storage_policy_index': 0, + } for i in range(1, 6)] + # merge_items mutates items + broker.merge_items([dict(obj) for obj in objects]) + original_info = broker.get_info() + + # Add some metadata + meta = { + 'X-Container-Meta-Color': ['Blue', next(ts_iter).normal], + 'X-Container-Meta-Cleared': ['', next(ts_iter).normal], + 'X-Container-Sysmeta-Shape': ['Circle', next(ts_iter).normal], + } + broker.update_metadata(meta) + + # Add some syncs + incoming_sync = {'remote_id': 'incoming_123', 'sync_point': 1} + outgoing_sync = {'remote_id': 'outgoing_123', 'sync_point': 2} + broker.merge_syncs([outgoing_sync], incoming=False) + broker.merge_syncs([incoming_sync], incoming=True) + + # Add some ShardRanges + shard_ranges = [ShardRange( + name='.shards_a/shard_range_%s' % i, + timestamp=next(ts_iter), lower='obj_%d' % i, + upper='obj_%d' % (i + 2), + object_count=len(objects[i:i + 2]), + bytes_used=sum(obj['size'] for obj in objects[i:i + 2]), + meta_timestamp=next(ts_iter)) for i in range(0, 6, 2)] + deleted_range = ShardRange('.shards_a/shard_range_z', next(ts_iter), + 'z', '', state=ShardRange.SHARDED, + deleted=1) + own_sr = ShardRange(name='a/c', timestamp=next(ts_iter), + state=ShardRange.ACTIVE) + broker.merge_shard_ranges([own_sr] + shard_ranges + [deleted_range]) + ts_epoch = next(ts_iter) + new_db_path = os.path.join(tempdir, 'part', 'suffix', 'hash', + 'container_%s.db' % ts_epoch.normal) + + def check_broker_properties(broker): + # these broker properties should remain unchanged as state changes + self.assertEqual(broker.get_max_row(), 5) + all_metadata = broker.metadata + original_meta = dict((k, all_metadata[k]) for k in meta) + self.assertEqual(original_meta, meta) + self.assertEqual(broker.get_syncs(True)[0], incoming_sync) + self.assertEqual(broker.get_syncs(False)[0], outgoing_sync) + self.assertEqual(shard_ranges + [own_sr, deleted_range], + broker.get_shard_ranges(include_own=True, + include_deleted=True)) + + def check_broker_info(actual_info): + for key in ('db_state', 'id', 'hash'): + actual_info.pop(key, None) + original_info.pop(key, None) + self.assertEqual(original_info, actual_info) + + def check_unsharded_state(broker): + # these are expected properties in unsharded state + self.assertEqual(len(broker.get_brokers()), 1) + self.assertEqual(broker.get_db_state(), UNSHARDED) + self.assertTrue(os.path.exists(db_path)) + self.assertFalse(os.path.exists(new_db_path)) + self.assertEqual(objects, broker.get_objects()) + + # Sanity checks + check_broker_properties(broker) + check_unsharded_state(broker) + check_broker_info(broker.get_info()) + + # first test that moving from UNSHARDED to SHARDED doesn't work + self.assertFalse(broker.set_sharded_state()) + # check nothing changed + check_broker_properties(broker) + check_broker_info(broker.get_info()) + check_unsharded_state(broker) + + # cannot go to SHARDING without an epoch set + self.assertFalse(broker.set_sharding_state()) + + # now set sharding epoch and make sure everything moves. + broker.enable_sharding(ts_epoch) + self.assertTrue(broker.set_sharding_state()) + check_broker_properties(broker) + check_broker_info(broker.get_info()) + + def check_sharding_state(broker): + self.assertEqual(len(broker.get_brokers()), 2) + self.assertEqual(broker.get_db_state(), SHARDING) + self.assertTrue(os.path.exists(db_path)) + self.assertTrue(os.path.exists(new_db_path)) + self.assertEqual([], broker.get_objects()) + self.assertEqual(objects, broker.get_brokers()[0].get_objects()) + check_sharding_state(broker) + + # to confirm we're definitely looking at the shard db + broker2 = ContainerBroker(new_db_path) + check_broker_properties(broker2) + check_broker_info(broker2.get_info()) + self.assertEqual([], broker2.get_objects()) + + # Try to set sharding state again + self.assertFalse(broker.set_sharding_state()) + # check nothing changed + check_broker_properties(broker) + check_broker_info(broker.get_info()) + check_sharding_state(broker) + + # Now move to the final state - update shard ranges' state + broker.merge_shard_ranges( + [dict(sr, state=ShardRange.ACTIVE, + state_timestamp=next(ts_iter).internal) + for sr in shard_ranges]) + # pretend all ranges have been cleaved + self.assertTrue(broker.set_sharded_state()) + check_broker_properties(broker) + check_broker_info(broker.get_info()) + + def check_sharded_state(broker): + self.assertEqual(broker.get_db_state(), SHARDED) + self.assertEqual(len(broker.get_brokers()), 1) + self.assertFalse(os.path.exists(db_path)) + self.assertTrue(os.path.exists(new_db_path)) + self.assertEqual([], broker.get_objects()) + check_sharded_state(broker) + + # Try to set sharded state again + self.assertFalse(broker.set_sharded_state()) + # check nothing changed + check_broker_properties(broker) + check_broker_info(broker.get_info()) + check_sharded_state(broker) + + # delete the container - sharding sysmeta gets erased + broker.delete_db(next(ts_iter).internal) + # but it is not considered deleted while shards have content + self.assertFalse(broker.is_deleted()) + check_sharded_state(broker) + # empty the shard ranges + empty_shard_ranges = [sr.copy(object_count=0, bytes_used=0, + meta_timestamp=next(ts_iter)) + for sr in shard_ranges] + broker.merge_shard_ranges(empty_shard_ranges) + # and no it is deleted + self.assertTrue(broker.is_deleted()) + check_sharded_state(broker) + + def do_revive_shard_delete(shard_ranges): + # delete all shard ranges + deleted_shard_ranges = [sr.copy(timestamp=next(ts_iter), deleted=1) + for sr in shard_ranges] + broker.merge_shard_ranges(deleted_shard_ranges) + self.assertEqual(COLLAPSED, broker.get_db_state()) + + # add new shard ranges and go to sharding state - need to force + # broker time to be after the delete time in order to write new + # sysmeta + broker.enable_sharding(next(ts_iter)) + shard_ranges = [sr.copy(timestamp=next(ts_iter)) + for sr in shard_ranges] + broker.merge_shard_ranges(shard_ranges) + with mock.patch('swift.common.db.time.time', + lambda: float(next(ts_iter))): + self.assertTrue(broker.set_sharding_state()) + self.assertEqual(SHARDING, broker.get_db_state()) + + # go to sharded + self.assertTrue( + broker.set_sharded_state()) + self.assertEqual(SHARDED, broker.get_db_state()) + + # delete again + broker.delete_db(next(ts_iter).internal) + self.assertTrue(broker.is_deleted()) + self.assertEqual(SHARDED, broker.get_db_state()) + + do_revive_shard_delete(shard_ranges) + do_revive_shard_delete(shard_ranges) + + @with_tempdir + def test_set_sharding_state_errors(self, tempdir): + ts_iter = make_timestamp_iter() + db_path = os.path.join( + tempdir, 'part', 'suffix', 'hash', 'container.db') + broker = ContainerBroker(db_path, account='a', container='c', + logger=FakeLogger()) + broker.initialize(next(ts_iter).internal, 0) + broker.enable_sharding(next(ts_iter)) + + orig_execute = GreenDBConnection.execute + trigger = 'INSERT into object' + + def mock_execute(conn, *args, **kwargs): + if trigger in args[0]: + raise sqlite3.OperationalError() + return orig_execute(conn, *args, **kwargs) + + with mock.patch('swift.common.db.GreenDBConnection.execute', + mock_execute): + res = broker.set_sharding_state() + self.assertFalse(res) + lines = broker.logger.get_lines_for_level('error') + self.assertIn('Failed to set the ROWID', lines[0]) + self.assertFalse(lines[1:]) + + broker.logger.clear() + trigger = 'UPDATE container_stat SET created_at' + with mock.patch('swift.common.db.GreenDBConnection.execute', + mock_execute): + res = broker.set_sharding_state() + self.assertFalse(res) + lines = broker.logger.get_lines_for_level('error') + self.assertIn('Failed to set matching', lines[0]) + self.assertFalse(lines[1:]) + + @with_tempdir + def test_set_sharded_state_errors(self, tempdir): + ts_iter = make_timestamp_iter() + retiring_db_path = os.path.join( + tempdir, 'part', 'suffix', 'hash', 'container.db') + broker = ContainerBroker(retiring_db_path, account='a', container='c', + logger=FakeLogger()) + broker.initialize(next(ts_iter).internal, 0) + pre_epoch = next(ts_iter) + broker.enable_sharding(next(ts_iter)) + self.assertTrue(broker.set_sharding_state()) + # unlink fails + with mock.patch('os.unlink', side_effect=OSError(errno.EPERM)): + self.assertFalse(broker.set_sharded_state()) + lines = broker.logger.get_lines_for_level('error') + self.assertIn('Failed to unlink', lines[0]) + self.assertFalse(lines[1:]) + self.assertFalse(broker.logger.get_lines_for_level('warning')) + self.assertTrue(os.path.exists(retiring_db_path)) + self.assertTrue(os.path.exists(broker.db_file)) + + # extra files + extra_filename = make_db_file_path(broker.db_file, pre_epoch) + self.assertNotEqual(extra_filename, broker.db_file) # sanity check + with open(extra_filename, 'wb'): + pass + broker.logger.clear() + self.assertFalse(broker.set_sharded_state()) + lines = broker.logger.get_lines_for_level('warning') + self.assertIn('Still have multiple db files', lines[0]) + self.assertFalse(lines[1:]) + self.assertFalse(broker.logger.get_lines_for_level('error')) + self.assertTrue(os.path.exists(retiring_db_path)) + self.assertTrue(os.path.exists(broker.db_file)) + + # retiring file missing + broker.logger.clear() + os.unlink(retiring_db_path) + self.assertFalse(broker.set_sharded_state()) + lines = broker.logger.get_lines_for_level('warning') + self.assertIn('Refusing to delete', lines[0]) + self.assertFalse(lines[1:]) + self.assertFalse(broker.logger.get_lines_for_level('error')) + self.assertTrue(os.path.exists(broker.db_file)) + + @with_tempdir + def test_get_brokers(self, tempdir): + ts_iter = make_timestamp_iter() + retiring_db_path = os.path.join( + tempdir, 'part', 'suffix', 'hash', 'container.db') + broker = ContainerBroker(retiring_db_path, account='a', container='c', + logger=FakeLogger()) + broker.initialize(next(ts_iter).internal, 0) + brokers = broker.get_brokers() + self.assertEqual(retiring_db_path, brokers[0].db_file) + self.assertFalse(brokers[0].skip_commits) + self.assertFalse(brokers[1:]) + + broker.enable_sharding(next(ts_iter)) + self.assertTrue(broker.set_sharding_state()) + brokers = broker.get_brokers() + self.assertEqual(retiring_db_path, brokers[0].db_file) + self.assertTrue(brokers[0].skip_commits) + self.assertEqual(broker.db_file, brokers[1].db_file) + self.assertFalse(brokers[1].skip_commits) + self.assertFalse(brokers[2:]) + + # same outcome when called on retiring db broker + brokers = brokers[0].get_brokers() + self.assertEqual(retiring_db_path, brokers[0].db_file) + self.assertTrue(brokers[0].skip_commits) + self.assertEqual(broker.db_file, brokers[1].db_file) + self.assertFalse(brokers[1].skip_commits) + self.assertFalse(brokers[2:]) + + self.assertTrue(broker.set_sharded_state()) + brokers = broker.get_brokers() + self.assertEqual(broker.db_file, brokers[0].db_file) + self.assertFalse(brokers[0].skip_commits) + self.assertFalse(brokers[1:]) + + # unexpected extra file should be ignored + with open(retiring_db_path, 'wb'): + pass + retiring_db_path = broker.db_file + broker.enable_sharding(next(ts_iter)) + self.assertTrue(broker.set_sharding_state()) + broker.reload_db_files() + self.assertEqual(3, len(broker.db_files)) # sanity check + brokers = broker.get_brokers() + self.assertEqual(retiring_db_path, brokers[0].db_file) + self.assertTrue(brokers[0].skip_commits) + self.assertEqual(broker.db_file, brokers[1].db_file) + self.assertFalse(brokers[1].skip_commits) + self.assertFalse(brokers[2:]) + lines = broker.logger.get_lines_for_level('warning') + self.assertIn('Unexpected db files', lines[0]) + self.assertFalse(lines[1:]) + + @with_tempdir + def test_merge_shard_ranges(self, tempdir): + ts_iter = make_timestamp_iter() + ts = [next(ts_iter) for _ in range(13)] + db_path = os.path.join( + tempdir, 'part', 'suffix', 'hash', 'container.db') + broker = ContainerBroker( + db_path, account='a', container='c') + broker.initialize(next(ts_iter).internal, 0) + + # sanity check + self.assertFalse(broker.get_shard_ranges(include_deleted=True)) + + broker.merge_shard_ranges(None) + self.assertFalse(broker.get_shard_ranges(include_deleted=True)) + + # merge item at ts1 + # sr___ + sr_b_1_1 = ShardRange('a/c_b', ts[1], lower='a', upper='b', + object_count=2) + broker.merge_shard_ranges([sr_b_1_1]) + self._assert_shard_ranges(broker, [sr_b_1_1]) + + # merge older item - ignored + sr_b_0_0 = ShardRange('a/c_b', ts[0], lower='a', upper='b', + object_count=1) + broker.merge_shard_ranges([sr_b_0_0]) + self._assert_shard_ranges(broker, [sr_b_1_1]) + + # merge same timestamp - ignored + broker.merge_shard_ranges([dict(sr_b_1_1, lower='', upper='c')]) + self._assert_shard_ranges(broker, [sr_b_1_1]) + broker.merge_shard_ranges([dict(sr_b_1_1, object_count=99)]) + self._assert_shard_ranges(broker, [sr_b_1_1]) + + # merge list with older item *after* newer item + sr_c_2_2 = ShardRange('a/c_c', ts[2], lower='b', upper='c', + object_count=3) + sr_c_3_3 = ShardRange('a/c_c', ts[3], lower='b', upper='c', + object_count=4) + broker.merge_shard_ranges([sr_c_3_3, sr_c_2_2]) + self._assert_shard_ranges(broker, [sr_b_1_1, sr_c_3_3]) + + # merge newer item - updated + sr_c_5_5 = ShardRange('a/c_c', ts[5], lower='b', upper='c', + object_count=5) + broker.merge_shard_ranges([sr_c_5_5]) + self._assert_shard_ranges(broker, [sr_b_1_1, sr_c_5_5]) + + # merge older metadata item - ignored + sr_c_5_4 = ShardRange('a/c_c', ts[5], lower='b', upper='c', + object_count=6, meta_timestamp=ts[4]) + broker.merge_shard_ranges([sr_c_5_4]) + self._assert_shard_ranges(broker, [sr_b_1_1, sr_c_5_5]) + + # merge newer metadata item - only metadata is updated + sr_c_5_6 = ShardRange('a/c_c', ts[5], lower='b', upper='c', + object_count=7, meta_timestamp=ts[6]) + broker.merge_shard_ranges([dict(sr_c_5_6, lower='', upper='d')]) + self._assert_shard_ranges(broker, [sr_b_1_1, sr_c_5_6]) + + # merge older created_at, newer metadata item - ignored + sr_c_4_7 = ShardRange('a/c_c', ts[4], lower='b', upper='c', + object_count=8, meta_timestamp=ts[7]) + broker.merge_shard_ranges([sr_c_4_7]) + self._assert_shard_ranges(broker, [sr_b_1_1, sr_c_5_6]) + + # merge list with older metadata item *after* newer metadata item + sr_c_5_11 = ShardRange('a/c_c', ts[5], lower='b', upper='c', + object_count=9, meta_timestamp=ts[11]) + broker.merge_shard_ranges([sr_c_5_11, sr_c_5_6]) + self._assert_shard_ranges(broker, [sr_b_1_1, sr_c_5_11]) + + # deleted item at *same timestamp* as existing - deleted ignored + broker.merge_shard_ranges([dict(sr_b_1_1, deleted=1, object_count=0)]) + self._assert_shard_ranges(broker, [sr_b_1_1, sr_c_5_11]) + sr_b_1_1.meta_timestamp = ts[11] + broker.merge_shard_ranges([dict(sr_b_1_1, deleted=1)]) + self._assert_shard_ranges(broker, [sr_b_1_1, sr_c_5_11]) + sr_b_1_1.state_timestamp = ts[11] + broker.merge_shard_ranges([dict(sr_b_1_1, deleted=1)]) + self._assert_shard_ranges(broker, [sr_b_1_1, sr_c_5_11]) + + # delete item at *newer timestamp* - updated + sr_b_2_2_deleted = ShardRange('a/c_b', ts[2], lower='a', upper='b', + object_count=0, deleted=1) + broker.merge_shard_ranges([sr_b_2_2_deleted]) + self._assert_shard_ranges(broker, [sr_b_2_2_deleted, sr_c_5_11]) + + # merge list with older undeleted item *after* newer deleted item + # NB deleted timestamp trumps newer meta timestamp + sr_c_9_12 = ShardRange('a/c_c', ts[9], lower='b', upper='c', + object_count=10, meta_timestamp=ts[12]) + sr_c_10_10_deleted = ShardRange('a/c_c', ts[10], lower='b', upper='c', + object_count=0, deleted=1) + broker.merge_shard_ranges([sr_c_10_10_deleted, sr_c_9_12]) + self._assert_shard_ranges( + broker, [sr_b_2_2_deleted, sr_c_10_10_deleted]) + + @with_tempdir + def test_merge_shard_ranges_state(self, tempdir): + ts_iter = make_timestamp_iter() + db_path = os.path.join( + tempdir, 'part', 'suffix', 'hash', 'container.db') + broker = ContainerBroker(db_path, account='a', container='c') + broker.initialize(next(ts_iter).internal, 0) + expected_shard_ranges = [] + + def do_test(orig_state, orig_timestamp, test_state, test_timestamp, + expected_state, expected_timestamp): + index = len(expected_shard_ranges) + sr = ShardRange('a/%s' % index, orig_timestamp, '%03d' % index, + '%03d' % (index + 1), state=orig_state) + broker.merge_shard_ranges([sr]) + sr.state = test_state + sr.state_timestamp = test_timestamp + broker.merge_shard_ranges([sr]) + sr.state = expected_state + sr.state_timestamp = expected_timestamp + expected_shard_ranges.append(sr) + self._assert_shard_ranges(broker, expected_shard_ranges) + + # state at older state_timestamp is not merged + for orig_state in ShardRange.STATES: + for test_state in ShardRange.STATES: + ts_older = next(ts_iter) + ts = next(ts_iter) + do_test(orig_state, ts, test_state, ts_older, orig_state, ts) + + # more advanced state at same timestamp is merged + for orig_state in ShardRange.STATES: + for test_state in ShardRange.STATES: + ts = next(ts_iter) + do_test(orig_state, ts, test_state, ts, + test_state if test_state > orig_state else orig_state, + ts) + + # any state at newer timestamp is merged + for orig_state in ShardRange.STATES: + for test_state in ShardRange.STATES: + ts = next(ts_iter) + ts_newer = next(ts_iter) + do_test(orig_state, ts, test_state, ts_newer, test_state, + ts_newer) + + def _check_object_stats_when_sharded(self, a, c, root_a, root_c, tempdir): + # common setup and assertions for root and shard containers + ts_iter = make_timestamp_iter() + db_path = os.path.join( + tempdir, 'part', 'suffix', 'hash', 'container.db') + broker = ContainerBroker( + db_path, account=a, container=c) + broker.initialize(next(ts_iter).internal, 0) + broker.set_sharding_sysmeta('Root', '%s/%s' % (root_a, root_c)) + broker.merge_items([{'name': 'obj', 'size': 14, 'etag': 'blah', + 'content_type': 'text/plain', 'deleted': 0, + 'created_at': Timestamp.now().internal}]) + self.assertEqual(1, broker.get_info()['object_count']) + self.assertEqual(14, broker.get_info()['bytes_used']) + + broker.enable_sharding(next(ts_iter)) + self.assertTrue(broker.set_sharding_state()) + sr_1 = ShardRange( + '%s/%s1' % (root_a, root_c), Timestamp.now(), lower='', upper='m', + object_count=99, bytes_used=999, state=ShardRange.ACTIVE) + sr_2 = ShardRange( + '%s/%s2' % (root_a, root_c), Timestamp.now(), lower='m', upper='', + object_count=21, bytes_used=1000, state=ShardRange.ACTIVE) + broker.merge_shard_ranges([sr_1, sr_2]) + self.assertEqual(1, broker.get_info()['object_count']) + self.assertEqual(14, broker.get_info()['bytes_used']) + return broker + + @with_tempdir + def test_object_stats_root_container(self, tempdir): + broker = self._check_object_stats_when_sharded( + 'a', 'c', 'a', 'c', tempdir) + self.assertTrue(broker.is_root_container()) # sanity + self.assertTrue(broker.set_sharded_state()) + self.assertEqual(120, broker.get_info()['object_count']) + self.assertEqual(1999, broker.get_info()['bytes_used']) + + @with_tempdir + def test_object_stats_shard_container(self, tempdir): + broker = self._check_object_stats_when_sharded( + '.shard_a', 'c-blah', 'a', 'c', tempdir) + self.assertFalse(broker.is_root_container()) # sanity + self.assertTrue(broker.set_sharded_state()) + self.assertEqual(0, broker.get_info()['object_count']) + self.assertEqual(0, broker.get_info()['bytes_used']) + class TestCommonContainerBroker(test_db.TestExampleBroker): @@ -2144,6 +4408,8 @@ class ContainerBrokerMigrationMixin(object): ContainerBroker.create_object_table ContainerBroker.create_object_table = \ prespi_create_object_table + self._imported_create_shard_ranges_table = \ + ContainerBroker.create_shard_range_table self._imported_create_container_info_table = \ ContainerBroker.create_container_info_table ContainerBroker.create_container_info_table = \ @@ -2168,6 +4434,8 @@ class ContainerBrokerMigrationMixin(object): self._imported_create_container_info_table ContainerBroker.create_object_table = \ self._imported_create_object_table + ContainerBroker.create_shard_range_table = \ + self._imported_create_shard_ranges_table ContainerBroker.create_policy_stat_table = \ self._imported_create_policy_stat_table @@ -2221,6 +4489,8 @@ class TestContainerBrokerBeforeMetadata(ContainerBrokerMigrationMixin, Tests for ContainerBroker against databases created before the metadata column was added. """ + expected_db_tables = {'outgoing_sync', 'incoming_sync', 'object', + 'sqlite_sequence', 'container_stat', 'shard_range'} def setUp(self): super(TestContainerBrokerBeforeMetadata, self).setUp() @@ -2293,6 +4563,8 @@ class TestContainerBrokerBeforeXSync(ContainerBrokerMigrationMixin, Tests for ContainerBroker against databases created before the x_container_sync_point[12] columns were added. """ + expected_db_tables = {'outgoing_sync', 'incoming_sync', 'object', + 'sqlite_sequence', 'container_stat', 'shard_range'} def setUp(self): super(TestContainerBrokerBeforeXSync, self).setUp() @@ -2407,6 +4679,8 @@ class TestContainerBrokerBeforeSPI(ContainerBrokerMigrationMixin, Tests for ContainerBroker against databases created before the storage_policy_index column was added. """ + expected_db_tables = {'outgoing_sync', 'incoming_sync', 'object', + 'sqlite_sequence', 'container_stat', 'shard_range'} def setUp(self): super(TestContainerBrokerBeforeSPI, self).setUp() @@ -2611,6 +4885,48 @@ class TestContainerBrokerBeforeSPI(ContainerBrokerMigrationMixin, self.assertEqual(info['bytes_used'], 456) +class TestContainerBrokerBeforeShardRanges(ContainerBrokerMigrationMixin, + TestContainerBroker): + """ + Tests for ContainerBroker against databases created + before the shard_ranges table was added. + """ + expected_db_tables = {'outgoing_sync', 'incoming_sync', 'object', + 'sqlite_sequence', 'container_stat'} + + class Override(object): + def __init__(self, func): + self.func = func + + def __get__(self, obj, obj_type): + if inspect.stack()[1][3] == '_initialize': + return lambda *a, **kw: None + return self.func.__get__(obj, obj_type) + + def setUp(self): + super(TestContainerBrokerBeforeShardRanges, self).setUp() + ContainerBroker.create_shard_range_table = self.Override( + ContainerBroker.create_shard_range_table) + broker = ContainerBroker(':memory:', account='a', container='c') + broker.initialize(Timestamp('1').internal, 0) + exc = None + with broker.get() as conn: + try: + conn.execute('''SELECT * + FROM shard_range''') + except BaseException as err: + exc = err + self.assertTrue('no such table: shard_range' in str(exc)) + + def tearDown(self): + super(TestContainerBrokerBeforeShardRanges, self).tearDown() + broker = ContainerBroker(':memory:', account='a', container='c') + broker.initialize(Timestamp('1').internal, 0) + with broker.get() as conn: + conn.execute('''SELECT * + FROM shard_range''') + + class TestUpdateNewItemFromExisting(unittest.TestCase): # TODO: add test scenarios that have swift_bytes in content_type t0 = '1234567890.00000' diff --git a/test/unit/container/test_replicator.py b/test/unit/container/test_replicator.py index ff63a2992c..23f06ddc97 100644 --- a/test/unit/container/test_replicator.py +++ b/test/unit/container/test_replicator.py @@ -26,13 +26,17 @@ from swift.common import db_replicator from swift.container import replicator, backend, server, sync_store from swift.container.reconciler import ( MISPLACED_OBJECTS_ACCOUNT, get_reconciler_container_name) -from swift.common.utils import Timestamp, encode_timestamps +from swift.common.utils import Timestamp, encode_timestamps, ShardRange, \ + get_db_files, make_db_file_path from swift.common.storage_policy import POLICIES from test.unit.common import test_db_replicator -from test.unit import patch_policies, make_timestamp_iter, mock_check_drive +from test.unit import patch_policies, make_timestamp_iter, mock_check_drive, \ + debug_logger from contextlib import contextmanager +from test.unit.common.test_db_replicator import attach_fake_replication_rpc + @patch_policies class TestReplicatorSync(test_db_replicator.TestReplicatorSync): @@ -42,6 +46,16 @@ class TestReplicatorSync(test_db_replicator.TestReplicatorSync): replicator_daemon = replicator.ContainerReplicator replicator_rpc = replicator.ContainerReplicatorRpc + def assertShardRangesEqual(self, x, y): + # ShardRange.__eq__ only compares lower and upper; here we generate + # dict representations to compare all attributes + self.assertEqual([dict(sr) for sr in x], [dict(sr) for sr in y]) + + def assertShardRangesNotEqual(self, x, y): + # ShardRange.__eq__ only compares lower and upper; here we generate + # dict representations to compare all attributes + self.assertNotEqual([dict(sr) for sr in x], [dict(sr) for sr in y]) + def test_report_up_to_date(self): broker = self._get_broker('a', 'c', node_index=0) broker.initialize(Timestamp(1).internal, int(POLICIES.default)) @@ -1148,6 +1162,1037 @@ class TestReplicatorSync(test_db_replicator.TestReplicatorSync): self.assertEqual(1, mock_remove.call_count) self.assertEqual(broker_2.db_file, mock_remove.call_args[0][0].db_file) + def test_cleanup_post_replicate(self): + broker = self._get_broker('a', 'c', node_index=0) + put_timestamp = Timestamp.now() + broker.initialize(put_timestamp.internal, POLICIES.default.idx) + orig_info = broker.get_replication_info() + daemon = replicator.ContainerReplicator({}, logger=self.logger) + + # db should not be here, replication ok, deleted + res = daemon.cleanup_post_replicate(broker, orig_info, [True] * 3) + self.assertTrue(res) + self.assertFalse(os.path.exists(broker.db_file)) + self.assertEqual(['Successfully deleted db %s' % broker.db_file], + daemon.logger.get_lines_for_level('debug')) + daemon.logger.clear() + + # failed replication, not deleted + broker.initialize(put_timestamp.internal, POLICIES.default.idx) + orig_info = broker.get_replication_info() + res = daemon.cleanup_post_replicate(broker, orig_info, + [False, True, True]) + self.assertTrue(res) + self.assertTrue(os.path.exists(broker.db_file)) + self.assertEqual(['Not deleting db %s (2/3 success)' % broker.db_file], + daemon.logger.get_lines_for_level('debug')) + daemon.logger.clear() + + # db has shard ranges, not deleted + broker.enable_sharding(Timestamp.now()) + broker.merge_shard_ranges( + [ShardRange('.shards_a/c', Timestamp.now(), '', 'm')]) + self.assertTrue(broker.sharding_required()) # sanity check + res = daemon.cleanup_post_replicate(broker, orig_info, [True] * 3) + self.assertTrue(res) + self.assertTrue(os.path.exists(broker.db_file)) + self.assertEqual( + ['Not deleting db %s (requires sharding, state unsharded)' % + broker.db_file], + daemon.logger.get_lines_for_level('debug')) + daemon.logger.clear() + + # db sharding, not deleted + self._goto_sharding_state(broker, Timestamp.now()) + self.assertTrue(broker.sharding_required()) # sanity check + orig_info = broker.get_replication_info() + res = daemon.cleanup_post_replicate(broker, orig_info, [True] * 3) + self.assertTrue(res) + self.assertTrue(os.path.exists(broker.db_file)) + self.assertEqual( + ['Not deleting db %s (requires sharding, state sharding)' % + broker.db_file], + daemon.logger.get_lines_for_level('debug')) + daemon.logger.clear() + + # db sharded, should not be here, failed replication, not deleted + self._goto_sharded_state(broker) + self.assertFalse(broker.sharding_required()) # sanity check + res = daemon.cleanup_post_replicate(broker, orig_info, + [True, False, True]) + self.assertTrue(res) + self.assertTrue(os.path.exists(broker.db_file)) + self.assertEqual(['Not deleting db %s (2/3 success)' % + broker.db_file], + daemon.logger.get_lines_for_level('debug')) + daemon.logger.clear() + + # db sharded, should not be here, new shard ranges (e.g. from reverse + # replication), deleted + broker.merge_shard_ranges( + [ShardRange('.shards_a/c', Timestamp.now(), '', 'm')]) + res = daemon.cleanup_post_replicate(broker, orig_info, [True] * 3) + self.assertTrue(res) + self.assertFalse(os.path.exists(broker.db_file)) + daemon.logger.clear() + + # db sharded, should not be here, replication ok, deleted + broker.initialize(put_timestamp.internal, POLICIES.default.idx) + self.assertTrue(os.path.exists(broker.db_file)) + orig_info = broker.get_replication_info() + res = daemon.cleanup_post_replicate(broker, orig_info, [True] * 3) + self.assertTrue(res) + self.assertFalse(os.path.exists(broker.db_file)) + self.assertEqual(['Successfully deleted db %s' % broker.db_file], + daemon.logger.get_lines_for_level('debug')) + daemon.logger.clear() + + def test_sync_shard_ranges(self): + put_timestamp = Timestamp.now().internal + # create "local" broker + broker = self._get_broker('a', 'c', node_index=0) + broker.initialize(put_timestamp, POLICIES.default.idx) + # create "remote" broker + remote_broker = self._get_broker('a', 'c', node_index=1) + remote_broker.initialize(put_timestamp, POLICIES.default.idx) + + def check_replicate(expected_shard_ranges, from_broker, to_broker): + daemon = replicator.ContainerReplicator({}) + part, node = self._get_broker_part_node(to_broker) + info = broker.get_replication_info() + success = daemon._repl_to_node(node, from_broker, part, info) + self.assertTrue(success) + self.assertEqual( + expected_shard_ranges, + to_broker.get_all_shard_range_data() + ) + self.assertEqual(1, daemon.stats['deferred']) + self.assertEqual(0, daemon.stats['rsync']) + self.assertEqual(0, daemon.stats['diff']) + local_info = self._get_broker( + 'a', 'c', node_index=0).get_info() + remote_info = self._get_broker( + 'a', 'c', node_index=1).get_info() + for k, v in local_info.items(): + if k == 'id': + continue + self.assertEqual(remote_info[k], v, + "mismatch remote %s %r != %r" % ( + k, remote_info[k], v)) + + bounds = (('', 'g'), ('g', 'r'), ('r', '')) + shard_ranges = [ + ShardRange('.shards_a/sr-%s' % upper, Timestamp.now(), lower, + upper, i + 1, 10 * (i + 1)) + for i, (lower, upper) in enumerate(bounds) + ] + # add first two shard_ranges to both brokers + for shard_range in shard_ranges[:2]: + for db in (broker, remote_broker): + db.merge_shard_ranges(shard_range) + # now add a shard range to the "local" broker only + own_sr = broker.enable_sharding(Timestamp.now()) + broker.merge_shard_ranges(shard_ranges[2]) + broker_ranges = broker.get_all_shard_range_data() + self.assertShardRangesEqual(shard_ranges + [own_sr], broker_ranges) + check_replicate(broker_ranges, broker, remote_broker) + + # update one shard range + shard_ranges[1].update_meta(99, 0) + broker.merge_shard_ranges(shard_ranges[1]) + # sanity check + broker_ranges = broker.get_all_shard_range_data() + self.assertShardRangesEqual(shard_ranges + [own_sr], broker_ranges) + check_replicate(broker_ranges, broker, remote_broker) + + # delete one shard range + shard_ranges[0].deleted = 1 + shard_ranges[0].timestamp = Timestamp.now() + broker.merge_shard_ranges(shard_ranges[0]) + # sanity check + broker_ranges = broker.get_all_shard_range_data() + self.assertShardRangesEqual(shard_ranges + [own_sr], broker_ranges) + check_replicate(broker_ranges, broker, remote_broker) + + # put a shard range again + shard_ranges[2].timestamp = Timestamp.now() + shard_ranges[2].object_count = 0 + broker.merge_shard_ranges(shard_ranges[2]) + # sanity check + broker_ranges = broker.get_all_shard_range_data() + self.assertShardRangesEqual(shard_ranges + [own_sr], broker_ranges) + check_replicate(broker_ranges, broker, remote_broker) + + # update same shard range on local and remote, remote later + shard_ranges[-1].meta_timestamp = Timestamp.now() + shard_ranges[-1].bytes_used += 1000 + broker.merge_shard_ranges(shard_ranges[-1]) + remote_shard_ranges = remote_broker.get_shard_ranges( + include_deleted=True) + remote_shard_ranges[-1].meta_timestamp = Timestamp.now() + remote_shard_ranges[-1].bytes_used += 2000 + remote_broker.merge_shard_ranges(remote_shard_ranges[-1]) + # sanity check + remote_broker_ranges = remote_broker.get_all_shard_range_data() + self.assertShardRangesEqual(remote_shard_ranges + [own_sr], + remote_broker_ranges) + self.assertShardRangesNotEqual(shard_ranges, remote_shard_ranges) + check_replicate(remote_broker_ranges, broker, remote_broker) + + # undelete shard range *on the remote* + deleted_ranges = [sr for sr in remote_shard_ranges if sr.deleted] + self.assertEqual([shard_ranges[0]], deleted_ranges) + deleted_ranges[0].deleted = 0 + deleted_ranges[0].timestamp = Timestamp.now() + remote_broker.merge_shard_ranges(deleted_ranges[0]) + # sanity check + remote_broker_ranges = remote_broker.get_all_shard_range_data() + self.assertShardRangesEqual(remote_shard_ranges + [own_sr], + remote_broker_ranges) + self.assertShardRangesNotEqual(shard_ranges, remote_shard_ranges) + check_replicate(remote_broker_ranges, broker, remote_broker) + + # reverse replication direction and expect syncs to propagate + check_replicate(remote_broker_ranges, remote_broker, broker) + + def test_sync_shard_ranges_with_rsync(self): + broker = self._get_broker('a', 'c', node_index=0) + put_timestamp = time.time() + broker.initialize(put_timestamp, POLICIES.default.idx) + + bounds = (('', 'g'), ('g', 'r'), ('r', '')) + shard_ranges = [ + ShardRange('.shards_a/sr-%s' % upper, Timestamp.now(), lower, + upper, i + 1, 10 * (i + 1)) + for i, (lower, upper) in enumerate(bounds) + ] + # add first shard range + own_sr = broker.enable_sharding(Timestamp.now()) + broker.merge_shard_ranges(shard_ranges[:1]) + + # "replicate" + part, node = self._get_broker_part_node(broker) + daemon = self._run_once(node) + self.assertEqual(2, daemon.stats['rsync']) + + # complete rsync to all other nodes + def check_replicate(expected_ranges): + for i in range(1, 3): + remote_broker = self._get_broker('a', 'c', node_index=i) + self.assertTrue(os.path.exists(remote_broker.db_file)) + self.assertShardRangesEqual( + expected_ranges, + remote_broker.get_shard_ranges(include_deleted=True, + include_own=True) + ) + remote_info = remote_broker.get_info() + local_info = self._get_broker( + 'a', 'c', node_index=0).get_info() + for k, v in local_info.items(): + if k == 'id': + continue + if k == 'hash': + self.assertEqual(remote_info[k], '0' * 32) + continue + if k == 'object_count': + self.assertEqual(remote_info[k], 0) + continue + self.assertEqual(remote_info[k], v, + "mismatch remote %s %r != %r" % ( + k, remote_info[k], v)) + + check_replicate([shard_ranges[0], own_sr]) + + # delete and add some more shard ranges + shard_ranges[0].deleted = 1 + shard_ranges[0].timestamp = Timestamp.now() + for shard_range in shard_ranges: + broker.merge_shard_ranges(shard_range) + daemon = self._run_once(node) + self.assertEqual(2, daemon.stats['deferred']) + check_replicate(shard_ranges + [own_sr]) + + def check_replicate(self, from_broker, remote_node_index, repl_conf=None, + expect_success=True, errors=None): + repl_conf = repl_conf or {} + repl_calls = [] + rsync_calls = [] + + def repl_hook(op, *sync_args): + repl_calls.append((op, sync_args)) + + fake_repl_connection = attach_fake_replication_rpc( + self.rpc, replicate_hook=repl_hook, errors=errors) + db_replicator.ReplConnection = fake_repl_connection + daemon = replicator.ContainerReplicator( + repl_conf, logger=debug_logger()) + self._install_fake_rsync_file(daemon, rsync_calls) + part, nodes = self._ring.get_nodes(from_broker.account, + from_broker.container) + + def find_node(node_index): + for node in nodes: + if node['index'] == node_index: + return node + else: + self.fail('Failed to find node index %s' % remote_node_index) + + remote_node = find_node(remote_node_index) + info = from_broker.get_replication_info() + success = daemon._repl_to_node(remote_node, from_broker, part, info) + self.assertEqual(expect_success, success) + return daemon, repl_calls, rsync_calls + + def assert_synced_shard_ranges(self, expected, synced_items): + expected.sort(key=lambda sr: (sr.lower, sr.upper)) + for item in synced_items: + item.pop('record_type', None) + self.assertEqual([dict(ex) for ex in expected], synced_items) + + def assert_info_synced(self, local, remote_node_index, mismatches=None): + mismatches = mismatches or [] + mismatches.append('id') + remote = self._get_broker(local.account, local.container, + node_index=remote_node_index) + local_info = local.get_info() + remote_info = remote.get_info() + errors = [] + for k, v in local_info.items(): + if remote_info.get(k) == v: + if k in mismatches: + errors.append( + "unexpected match remote %s %r == %r" % ( + k, remote_info[k], v)) + continue + else: + if k not in mismatches: + errors.append( + "unexpected mismatch remote %s %r != %r" % ( + k, remote_info[k], v)) + if errors: + self.fail('Found sync errors:\n' + '\n'.join(errors)) + + def assert_shard_ranges_synced(self, local_broker, remote_broker): + self.assertShardRangesEqual( + local_broker.get_shard_ranges(include_deleted=True, + include_own=True), + remote_broker.get_shard_ranges(include_deleted=True, + include_own=True) + ) + + def _setup_replication_test(self, node_index): + ts_iter = make_timestamp_iter() + policy_idx = POLICIES.default.idx + put_timestamp = Timestamp.now().internal + # create "local" broker + broker = self._get_broker('a', 'c', node_index=node_index) + broker.initialize(put_timestamp, policy_idx) + + objs = [{'name': 'blah%03d' % i, 'created_at': next(ts_iter).internal, + 'size': i, 'content_type': 'text/plain', 'etag': 'etag%s' % i, + 'deleted': 0, 'storage_policy_index': policy_idx} + for i in range(20)] + bounds = (('', 'a'), ('a', 'b'), ('b', 'c'), ('c', '')) + shard_ranges = [ + ShardRange( + '.sharded_a/sr-%s' % upper, Timestamp.now(), lower, upper) + for i, (lower, upper) in enumerate(bounds) + ] + return {'broker': broker, + 'objects': objs, + 'shard_ranges': shard_ranges} + + def _merge_object(self, broker, objects, index, **kwargs): + if not isinstance(index, slice): + index = slice(index, index + 1) + objs = [dict(obj) for obj in objects[index]] + broker.merge_items(objs) + + def _merge_shard_range(self, broker, shard_ranges, index, **kwargs): + broker.merge_shard_ranges(shard_ranges[index:index + 1]) + + def _goto_sharding_state(self, broker, epoch): + broker.enable_sharding(epoch) + self.assertTrue(broker.set_sharding_state()) + self.assertEqual(backend.SHARDING, broker.get_db_state()) + + def _goto_sharded_state(self, broker): + self.assertTrue(broker.set_sharded_state()) + self.assertEqual(backend.SHARDED, broker.get_db_state()) + + def _assert_local_sharded_in_sync(self, local_broker, local_id): + daemon, repl_calls, rsync_calls = self.check_replicate(local_broker, 1) + self.assertEqual(['sync', 'get_shard_ranges', 'merge_shard_ranges'], + [call[0] for call in repl_calls]) + self.assertEqual(1, daemon.stats['deferred']) + self.assertEqual(0, daemon.stats['rsync']) + self.assertEqual(0, daemon.stats['diff']) + self.assertFalse(rsync_calls) + # new db sync + self.assertEqual(local_id, repl_calls[0][1][2]) + # ...but we still get a merge_shard_ranges for shard ranges + self.assert_synced_shard_ranges( + local_broker.get_shard_ranges(include_own=True), + repl_calls[2][1][0]) + self.assertEqual(local_id, repl_calls[2][1][1]) + + def _check_only_shard_ranges_replicated(self, local_broker, + remote_node_index, + repl_conf, + expected_shard_ranges, + expect_success=True): + # expected_shard_ranges is expected final list of sync'd ranges + daemon, repl_calls, rsync_calls = self.check_replicate( + local_broker, remote_node_index, repl_conf, + expect_success=expect_success) + + # we always expect only shard ranges to end in abort + self.assertEqual(1, daemon.stats['deferred']) + self.assertEqual(0, daemon.stats['diff']) + self.assertEqual(0, daemon.stats['rsync']) + self.assertEqual(['sync', 'get_shard_ranges', 'merge_shard_ranges'], + [call[0] for call in repl_calls]) + self.assertFalse(rsync_calls) + # sync + local_id = local_broker.get_info()['id'] + self.assertEqual(local_id, repl_calls[0][1][2]) + # get_shard_ranges + self.assertEqual((), repl_calls[1][1]) + # merge_shard_ranges for sending local shard ranges + self.assertShardRangesEqual(expected_shard_ranges, repl_calls[2][1][0]) + self.assertEqual(local_id, repl_calls[2][1][1]) + remote_broker = self._get_broker( + local_broker.account, local_broker.container, node_index=1) + self.assertNotEqual(local_id, remote_broker.get_info()['id']) + self.assert_shard_ranges_synced(remote_broker, local_broker) + + def test_replication_local_unsharded_remote_missing(self): + context = self._setup_replication_test(0) + local_broker = context['broker'] + local_id = local_broker.get_info()['id'] + objs = context['objects'] + self._merge_object(index=0, **context) + + daemon, repl_calls, rsync_calls = self.check_replicate(local_broker, 1) + + self.assert_info_synced(local_broker, 1) + self.assertEqual(1, daemon.stats['rsync']) + self.assertEqual(['sync', 'complete_rsync'], + [call[0] for call in repl_calls]) + self.assertEqual(local_id, repl_calls[1][1][0]) + self.assertEqual(os.path.basename(local_broker.db_file), + repl_calls[1][1][1]) + self.assertEqual(local_broker.db_file, rsync_calls[0][0]) + self.assertEqual(local_id, os.path.basename(rsync_calls[0][1])) + self.assertFalse(rsync_calls[1:]) + remote_broker = self._get_broker('a', 'c', node_index=1) + self.assert_shard_ranges_synced(local_broker, remote_broker) + self.assertTrue(os.path.exists(remote_broker._db_file)) + self.assertNotEqual(local_id, remote_broker.get_info()['id']) + self.assertEqual(objs[:1], remote_broker.get_objects()) + + def _check_replication_local_unsharded_remote_sharded(self, repl_conf): + context = self._setup_replication_test(0) + local_broker = context['broker'] + local_id = local_broker.get_info()['id'] + self._merge_object(index=slice(0, 6), **context) + + remote_context = self._setup_replication_test(1) + self._merge_object(index=4, **remote_context) + remote_broker = remote_context['broker'] + epoch = Timestamp.now() + self._goto_sharding_state(remote_broker, epoch=epoch) + remote_context['shard_ranges'][0].object_count = 101 + remote_context['shard_ranges'][0].bytes_used = 1010 + remote_context['shard_ranges'][0].state = ShardRange.ACTIVE + self._merge_shard_range(index=0, **remote_context) + self._merge_object(index=5, **remote_context) + self._goto_sharded_state(remote_broker) + self.assertEqual(backend.SHARDED, remote_broker.get_db_state()) + + self._check_only_shard_ranges_replicated( + local_broker, 1, repl_conf, + remote_broker.get_shard_ranges(include_own=True)) + + remote_broker = self._get_broker( + local_broker.account, local_broker.container, node_index=1) + self.assertEqual(backend.SHARDED, remote_broker.get_db_state()) + self.assertFalse(os.path.exists(remote_broker._db_file)) + self.assertNotEqual(local_id, remote_broker.get_info()['id']) + self.assertEqual(remote_context['objects'][5:6], + remote_broker.get_objects()) + + # Now that we have shard ranges, we're never considered in-sync :-/ + self._check_only_shard_ranges_replicated( + local_broker, 1, repl_conf, + remote_broker.get_shard_ranges(include_own=True)) + + def test_replication_local_unsharded_remote_sharded(self): + self._check_replication_local_unsharded_remote_sharded({}) + + def test_replication_local_unsharded_remote_sharded_large_diff(self): + self._check_replication_local_unsharded_remote_sharded({'per_diff': 1}) + + def _check_replication_local_sharding_remote_missing(self, repl_conf): + local_context = self._setup_replication_test(0) + local_broker = local_context['broker'] + self._merge_object(index=0, **local_context) + self._merge_object(index=1, **local_context) + epoch = Timestamp.now() + self._goto_sharding_state(local_broker, epoch) + self._merge_shard_range(index=0, **local_context) + self._merge_object(index=slice(2, 8), **local_context) + objs = local_context['objects'] + + daemon, repl_calls, rsync_calls = self.check_replicate( + local_broker, 1, repl_conf=repl_conf) + + self.assertEqual(['sync', 'complete_rsync'], + [call[0] for call in repl_calls]) + self.assertEqual(1, daemon.stats['rsync']) + self.assertEqual(0, daemon.stats['deferred']) + self.assertEqual(0, daemon.stats['diff']) + + # fresh db is sync'd first... + fresh_id = local_broker.get_info()['id'] + self.assertEqual(fresh_id, repl_calls[0][1][2]) + self.assertEqual(fresh_id, repl_calls[1][1][0]) + # retired db is not sync'd at all + old_broker = self.backend( + local_broker._db_file, account=local_broker.account, + container=local_broker.container, force_db_file=True) + old_id = old_broker.get_info()['id'] + bad_calls = [] + for call in repl_calls: + if old_id in call[1]: + bad_calls.append( + 'old db id %r in %r call args %r' % ( + old_id, call[0], call[1])) + if bad_calls: + self.fail('Found some bad calls:\n' + '\n'.join(bad_calls)) + # complete_rsync + self.assertEqual(os.path.basename(local_broker.db_file), + repl_calls[1][1][1]) + self.assertEqual(local_broker.db_file, rsync_calls[0][0]) + self.assertEqual(fresh_id, os.path.basename(rsync_calls[0][1])) + self.assertFalse(rsync_calls[1:]) + + # TODO: make these stats better; in sharding state local broker pulls + # stats for 2 objects from old db, whereas remote thinks it's sharded + # and has an empty shard range table + self.assert_info_synced(local_broker, 1, mismatches=[ + 'object_count', 'bytes_used', 'db_state']) + + remote_broker = self._get_broker('a', 'c', node_index=1) + remote_id = remote_broker.get_info()['id'] + self.assertNotEqual(old_id, remote_id) + self.assertNotEqual(fresh_id, remote_id) + self.assertEqual( + [remote_broker.db_file], get_db_files(remote_broker.db_file)) + self.assertEqual(os.path.basename(remote_broker.db_file), + os.path.basename(local_broker.db_file)) + self.assertEqual(epoch, remote_broker.db_epoch) + # remote db has only the misplaced objects + self.assertEqual(objs[2:8], remote_broker.get_objects()) + self.assert_shard_ranges_synced(local_broker, remote_broker) + + # replicate again, check asserts abort + self._check_only_shard_ranges_replicated( + local_broker, 1, repl_conf, + local_broker.get_shard_ranges(include_own=True)) + + # sanity + remote_broker = self._get_broker('a', 'c', node_index=1) + self.assertEqual( + [remote_broker.db_file], get_db_files(remote_broker.db_file)) + self.assertEqual(os.path.basename(remote_broker.db_file), + os.path.basename(local_broker.db_file)) + self.assertEqual(objs[2:8], remote_broker.get_objects()) + self.assertEqual(epoch, remote_broker.db_epoch) + + def test_replication_local_sharding_remote_missing(self): + self._check_replication_local_sharding_remote_missing({}) + + def test_replication_local_sharding_remote_missing_large_diff(self): + # the local shard db has large diff with respect to the old db + self._check_replication_local_sharding_remote_missing({'per_diff': 1}) + + def _check_replication_local_sharding_remote_unsharded(self, repl_conf): + local_context = self._setup_replication_test(0) + self._merge_object(index=slice(0, 3), **local_context) + local_broker = local_context['broker'] + epoch = Timestamp.now() + self._goto_sharding_state(local_broker, epoch) + self._merge_shard_range(index=0, **local_context) + self._merge_object(index=slice(3, 11), **local_context) + + remote_context = self._setup_replication_test(1) + self._merge_object(index=11, **remote_context) + + self._check_only_shard_ranges_replicated( + local_broker, 1, repl_conf, + local_broker.get_shard_ranges(include_own=True)) + + remote_broker = self._get_broker('a', 'c', node_index=1) + self.assertEqual( + [remote_broker._db_file], get_db_files(remote_broker.db_file)) + self.assertEqual(remote_context['objects'][11:12], + remote_broker.get_objects()) + + self.assert_info_synced( + local_broker, 1, + mismatches=['db_state', 'object_count', 'bytes_used', + 'status_changed_at', 'hash']) + + self._check_only_shard_ranges_replicated( + local_broker, 1, repl_conf, + local_broker.get_shard_ranges(include_own=True)) + + def test_replication_local_sharding_remote_unsharded(self): + self._check_replication_local_sharding_remote_unsharded({}) + + def test_replication_local_sharding_remote_unsharded_large_diff(self): + self._check_replication_local_sharding_remote_unsharded( + {'per_diff': 1}) + + def _check_replication_local_sharding_remote_sharding(self, repl_conf): + local_context = self._setup_replication_test(0) + self._merge_object(index=slice(0, 5), **local_context) + local_broker = local_context['broker'] + epoch = Timestamp.now() + self._goto_sharding_state(local_broker, epoch) + self._merge_shard_range(index=0, **local_context) + self._merge_object(index=slice(5, 10), **local_context) + + remote_context = self._setup_replication_test(1) + self._merge_object(index=12, **remote_context) + # take snapshot of info now before transition to sharding... + orig_remote_info = remote_context['broker'].get_info() + remote_broker = remote_context['broker'] + self._goto_sharding_state(remote_broker, epoch) + self._merge_shard_range(index=0, **remote_context) + self._merge_object(index=13, **remote_context) + + self._check_only_shard_ranges_replicated( + local_broker, 1, repl_conf, + remote_broker.get_shard_ranges(include_own=True)) + + # in sharding state brokers only reports object stats from old db, and + # they are different + self.assert_info_synced( + local_broker, 1, mismatches=['object_count', 'bytes_used', + 'status_changed_at', 'hash']) + + remote_broker = self._get_broker('a', 'c', node_index=1) + shard_db = make_db_file_path(remote_broker._db_file, epoch) + self.assertEqual([remote_broker._db_file, shard_db], + get_db_files(remote_broker.db_file)) + shard_db = make_db_file_path(remote_broker._db_file, epoch) + self.assertEqual([remote_broker._db_file, shard_db], + get_db_files(remote_broker.db_file)) + # no local objects have been sync'd to remote shard db + self.assertEqual(remote_context['objects'][13:14], + remote_broker.get_objects()) + # remote *old db* is unchanged + remote_old_broker = self.backend( + remote_broker._db_file, account=remote_broker.account, + container=remote_broker.container, force_db_file=True) + self.assertEqual(remote_context['objects'][12:13], + remote_old_broker.get_objects()) + self.assertFalse(remote_old_broker.get_shard_ranges()) + remote_old_info = remote_old_broker.get_info() + orig_remote_info.pop('db_state') + remote_old_info.pop('db_state') + self.assertEqual(orig_remote_info, remote_old_info) + + self._check_only_shard_ranges_replicated( + local_broker, 1, repl_conf, + local_broker.get_shard_ranges(include_own=True)) + + def test_replication_local_sharding_remote_sharding(self): + self._check_replication_local_sharding_remote_sharding({}) + + def test_replication_local_sharding_remote_sharding_large_diff(self): + self._check_replication_local_sharding_remote_sharding({'per_diff': 1}) + + def test_replication_local_sharded_remote_missing(self): + local_context = self._setup_replication_test(0) + local_broker = local_context['broker'] + epoch = Timestamp.now() + self._goto_sharding_state(local_broker, epoch) + local_context['shard_ranges'][0].object_count = 99 + local_context['shard_ranges'][0].state = ShardRange.ACTIVE + self._merge_shard_range(index=0, **local_context) + self._merge_object(index=slice(0, 3), **local_context) + self._goto_sharded_state(local_broker) + objs = local_context['objects'] + + daemon, repl_calls, rsync_calls = self.check_replicate(local_broker, 1) + + self.assertEqual(['sync', 'complete_rsync'], + [call[0] for call in repl_calls]) + self.assertEqual(1, daemon.stats['rsync']) + + # sync + local_id = local_broker.get_info()['id'] + self.assertEqual(local_id, repl_calls[0][1][2]) + # complete_rsync + self.assertEqual(local_id, repl_calls[1][1][0]) + self.assertEqual( + os.path.basename(local_broker.db_file), repl_calls[1][1][1]) + self.assertEqual(local_broker.db_file, rsync_calls[0][0]) + self.assertEqual(local_id, os.path.basename(rsync_calls[0][1])) + self.assertFalse(rsync_calls[1:]) + + self.assert_info_synced(local_broker, 1) + + remote_broker = self._get_broker('a', 'c', node_index=1) + remote_id = remote_broker.get_info()['id'] + self.assertNotEqual(local_id, remote_id) + shard_db = make_db_file_path(remote_broker._db_file, epoch) + self.assertEqual([shard_db], + get_db_files(remote_broker.db_file)) + self.assertEqual(objs[:3], remote_broker.get_objects()) + self.assertEqual(local_broker.get_shard_ranges(), + remote_broker.get_shard_ranges()) + + # sanity check - in sync + self._assert_local_sharded_in_sync(local_broker, local_id) + + remote_broker = self._get_broker('a', 'c', node_index=1) + shard_db = make_db_file_path(remote_broker._db_file, epoch) + self.assertEqual([shard_db], + get_db_files(remote_broker.db_file)) + # the remote broker object_count comes from replicated shard range... + self.assertEqual(99, remote_broker.get_info()['object_count']) + # these are replicated misplaced objects... + self.assertEqual(objs[:3], remote_broker.get_objects()) + self.assertEqual(local_broker.get_shard_ranges(), + remote_broker.get_shard_ranges()) + + def _check_replication_local_sharded_remote_unsharded(self, repl_conf): + local_context = self._setup_replication_test(0) + local_broker = local_context['broker'] + epoch = Timestamp.now() + self._goto_sharding_state(local_broker, epoch) + local_context['shard_ranges'][0].object_count = 99 + local_context['shard_ranges'][0].state = ShardRange.ACTIVE + self._merge_shard_range(index=0, **local_context) + self._merge_object(index=slice(0, 3), **local_context) + self._goto_sharded_state(local_broker) + + remote_context = self._setup_replication_test(1) + self._merge_object(index=4, **remote_context) + + self._check_only_shard_ranges_replicated( + local_broker, 1, repl_conf, + local_broker.get_shard_ranges(include_own=True), + expect_success=True) + + # sharded broker takes object count from shard range whereas remote + # unsharded broker takes it from object table + self.assert_info_synced( + local_broker, 1, + mismatches=['db_state', 'object_count', 'bytes_used', + 'status_changed_at', 'hash']) + + remote_broker = self._get_broker('a', 'c', node_index=1) + self.assertEqual([remote_broker._db_file], + get_db_files(remote_broker.db_file)) + self.assertEqual(remote_context['objects'][4:5], + remote_broker.get_objects()) + + self._check_only_shard_ranges_replicated( + local_broker, 1, repl_conf, + local_broker.get_shard_ranges(include_own=True), + expect_success=True) + + remote_broker = self._get_broker('a', 'c', node_index=1) + self.assertEqual([remote_broker._db_file], + get_db_files(remote_broker.db_file)) + self.assertEqual(remote_context['objects'][4:5], + remote_broker.get_objects()) + + def test_replication_local_sharded_remote_unsharded(self): + self._check_replication_local_sharded_remote_unsharded({}) + + def test_replication_local_sharded_remote_unsharded_large_diff(self): + self._check_replication_local_sharded_remote_unsharded({'per_diff': 1}) + + def _check_replication_local_sharded_remote_sharding(self, repl_conf): + local_context = self._setup_replication_test(0) + local_broker = local_context['broker'] + epoch = Timestamp.now() + self._goto_sharding_state(local_broker, epoch=epoch) + local_context['shard_ranges'][0].object_count = 99 + local_context['shard_ranges'][0].bytes_used = 999 + local_context['shard_ranges'][0].state = ShardRange.ACTIVE + self._merge_shard_range(index=0, **local_context) + self._merge_object(index=slice(0, 5), **local_context) + self._goto_sharded_state(local_broker) + + remote_context = self._setup_replication_test(1) + self._merge_object(index=6, **remote_context) + remote_broker = remote_context['broker'] + remote_info_orig = remote_broker.get_info() + self._goto_sharding_state(remote_broker, epoch=epoch) + self._merge_shard_range(index=0, **remote_context) + self._merge_object(index=7, **remote_context) + + self._check_only_shard_ranges_replicated( + local_broker, 1, repl_conf, + # remote has newer timestamp for shard range + remote_broker.get_shard_ranges(include_own=True), + expect_success=True) + + # sharded broker takes object count from shard range whereas remote + # sharding broker takes it from object table + self.assert_info_synced( + local_broker, 1, + mismatches=['db_state', 'object_count', 'bytes_used', + 'status_changed_at', 'hash']) + + remote_broker = self._get_broker('a', 'c', node_index=1) + shard_db = make_db_file_path(remote_broker._db_file, epoch) + self.assertEqual([remote_broker._db_file, shard_db], + get_db_files(remote_broker.db_file)) + # remote fresh db objects are unchanged + self.assertEqual(remote_context['objects'][7:8], + remote_broker.get_objects()) + # remote old hash.db objects are unchanged + remote_old_broker = self.backend( + remote_broker._db_file, account=remote_broker.account, + container=remote_broker.container, force_db_file=True) + self.assertEqual( + remote_context['objects'][6:7], + remote_old_broker.get_objects()) + remote_info = remote_old_broker.get_info() + remote_info_orig.pop('db_state') + remote_info.pop('db_state') + self.assertEqual(remote_info_orig, remote_info) + self.assertEqual(local_broker.get_shard_ranges(), + remote_broker.get_shard_ranges()) + + self._check_only_shard_ranges_replicated( + local_broker, 1, repl_conf, + remote_broker.get_shard_ranges(include_own=True), + expect_success=True) + + def test_replication_local_sharded_remote_sharding(self): + self._check_replication_local_sharded_remote_sharding({}) + + def test_replication_local_sharded_remote_sharding_large_diff(self): + self._check_replication_local_sharded_remote_sharding({'per_diff': 1}) + + def _check_replication_local_sharded_remote_sharded(self, repl_conf): + local_context = self._setup_replication_test(0) + local_broker = local_context['broker'] + epoch = Timestamp.now() + self._goto_sharding_state(local_broker, epoch) + local_context['shard_ranges'][0].object_count = 99 + local_context['shard_ranges'][0].bytes_used = 999 + local_context['shard_ranges'][0].state = ShardRange.ACTIVE + self._merge_shard_range(index=0, **local_context) + self._merge_object(index=slice(0, 6), **local_context) + self._goto_sharded_state(local_broker) + + remote_context = self._setup_replication_test(1) + self._merge_object(index=6, **remote_context) + remote_broker = remote_context['broker'] + self._goto_sharding_state(remote_broker, epoch) + remote_context['shard_ranges'][0].object_count = 101 + remote_context['shard_ranges'][0].bytes_used = 1010 + remote_context['shard_ranges'][0].state = ShardRange.ACTIVE + self._merge_shard_range(index=0, **remote_context) + self._merge_object(index=7, **remote_context) + self._goto_sharded_state(remote_broker) + + self._check_only_shard_ranges_replicated( + local_broker, 1, repl_conf, + # remote has newer timestamp for shard range + remote_broker.get_shard_ranges(include_own=True), + expect_success=True) + + self.assert_info_synced( + local_broker, 1, + mismatches=['status_changed_at', 'hash']) + + remote_broker = self._get_broker('a', 'c', node_index=1) + shard_db = make_db_file_path(remote_broker._db_file, epoch) + self.assertEqual([shard_db], + get_db_files(remote_broker.db_file)) + self.assertEqual(remote_context['objects'][7:8], + remote_broker.get_objects()) + # remote shard range was newer than local so object count is not + # updated by sync'd shard range + self.assertEqual( + 101, remote_broker.get_shard_ranges()[0].object_count) + + self._check_only_shard_ranges_replicated( + local_broker, 1, repl_conf, + # remote has newer timestamp for shard range + remote_broker.get_shard_ranges(include_own=True), + expect_success=True) + + def test_replication_local_sharded_remote_sharded(self): + self._check_replication_local_sharded_remote_sharded({}) + + def test_replication_local_sharded_remote_sharded_large_diff(self): + self._check_replication_local_sharded_remote_sharded({'per_diff': 1}) + + def test_replication_rsync_then_merge_aborts_before_merge_sharding(self): + # verify that rsync_then_merge aborts if remote starts sharding during + # the rsync + local_context = self._setup_replication_test(0) + local_broker = local_context['broker'] + self._merge_object(index=slice(0, 3), **local_context) + remote_context = self._setup_replication_test(1) + remote_broker = remote_context['broker'] + remote_broker.logger = debug_logger() + self._merge_object(index=5, **remote_context) + + orig_func = replicator.ContainerReplicatorRpc.rsync_then_merge + + def mock_rsync_then_merge(*args): + remote_broker.merge_shard_ranges( + ShardRange('.shards_a/cc', Timestamp.now())) + self._goto_sharding_state(remote_broker, Timestamp.now()) + return orig_func(*args) + + with mock.patch( + 'swift.container.replicator.ContainerReplicatorRpc.' + 'rsync_then_merge', + mock_rsync_then_merge): + with mock.patch( + 'swift.container.backend.ContainerBroker.' + 'get_items_since') as mock_get_items_since: + daemon, repl_calls, rsync_calls = self.check_replicate( + local_broker, 1, expect_success=False, + repl_conf={'per_diff': 1}) + + mock_get_items_since.assert_not_called() + self.assertEqual(['sync', 'get_shard_ranges', 'rsync_then_merge'], + [call[0] for call in repl_calls]) + self.assertEqual(local_broker.db_file, rsync_calls[0][0]) + self.assertEqual(local_broker.get_info()['id'], + os.path.basename(rsync_calls[0][1])) + self.assertFalse(rsync_calls[1:]) + + def test_replication_rsync_then_merge_aborts_before_merge_sharded(self): + # verify that rsync_then_merge aborts if remote completes sharding + # during the rsync + local_context = self._setup_replication_test(0) + local_broker = local_context['broker'] + self._merge_object(index=slice(0, 3), **local_context) + remote_context = self._setup_replication_test(1) + remote_broker = remote_context['broker'] + remote_broker.logger = debug_logger() + self._merge_object(index=5, **remote_context) + + orig_func = replicator.ContainerReplicatorRpc.rsync_then_merge + + def mock_rsync_then_merge(*args): + remote_broker.merge_shard_ranges( + ShardRange('.shards_a/cc', Timestamp.now())) + self._goto_sharding_state(remote_broker, Timestamp.now()) + self._goto_sharded_state(remote_broker) + return orig_func(*args) + + with mock.patch( + 'swift.container.replicator.ContainerReplicatorRpc.' + 'rsync_then_merge', + mock_rsync_then_merge): + with mock.patch( + 'swift.container.backend.ContainerBroker.' + 'get_items_since') as mock_get_items_since: + daemon, repl_calls, rsync_calls = self.check_replicate( + local_broker, 1, expect_success=False, + repl_conf={'per_diff': 1}) + + mock_get_items_since.assert_not_called() + self.assertEqual(['sync', 'get_shard_ranges', 'rsync_then_merge'], + [call[0] for call in repl_calls]) + self.assertEqual(local_broker.db_file, rsync_calls[0][0]) + self.assertEqual(local_broker.get_info()['id'], + os.path.basename(rsync_calls[0][1])) + self.assertFalse(rsync_calls[1:]) + + def test_replication_rsync_then_merge_aborts_after_merge_sharding(self): + # verify that rsync_then_merge aborts if remote starts sharding during + # the merge + local_context = self._setup_replication_test(0) + local_broker = local_context['broker'] + self._merge_object(index=slice(0, 3), **local_context) + remote_context = self._setup_replication_test(1) + remote_broker = remote_context['broker'] + remote_broker.logger = debug_logger() + self._merge_object(index=5, **remote_context) + + orig_get_items_since = backend.ContainerBroker.get_items_since + calls = [] + + def fake_get_items_since(broker, *args): + # remote starts sharding while rpc call is merging + if not calls: + remote_broker.merge_shard_ranges( + ShardRange('.shards_a/cc', Timestamp.now())) + self._goto_sharding_state(remote_broker, Timestamp.now()) + calls.append(args) + return orig_get_items_since(broker, *args) + + with mock.patch( + 'swift.container.backend.ContainerBroker.get_items_since', + fake_get_items_since): + daemon, repl_calls, rsync_calls = self.check_replicate( + local_broker, 1, expect_success=False, + repl_conf={'per_diff': 1}) + + self.assertEqual(['sync', 'get_shard_ranges', 'rsync_then_merge'], + [call[0] for call in repl_calls]) + self.assertEqual(local_broker.db_file, rsync_calls[0][0]) + self.assertEqual(local_broker.get_info()['id'], + os.path.basename(rsync_calls[0][1])) + self.assertFalse(rsync_calls[1:]) + + def test_replication_rsync_then_merge_aborts_after_merge_sharded(self): + # verify that rsync_then_merge aborts if remote completes sharding + # during the merge + local_context = self._setup_replication_test(0) + local_broker = local_context['broker'] + self._merge_object(index=slice(0, 3), **local_context) + remote_context = self._setup_replication_test(1) + remote_broker = remote_context['broker'] + remote_broker.logger = debug_logger() + self._merge_object(index=5, **remote_context) + + orig_get_items_since = backend.ContainerBroker.get_items_since + calls = [] + + def fake_get_items_since(broker, *args): + # remote starts sharding while rpc call is merging + result = orig_get_items_since(broker, *args) + if calls: + remote_broker.merge_shard_ranges( + ShardRange('.shards_a/cc', Timestamp.now())) + self._goto_sharding_state(remote_broker, Timestamp.now()) + self._goto_sharded_state(remote_broker) + calls.append(args) + return result + + with mock.patch( + 'swift.container.backend.ContainerBroker.get_items_since', + fake_get_items_since): + daemon, repl_calls, rsync_calls = self.check_replicate( + local_broker, 1, expect_success=False, + repl_conf={'per_diff': 1}) + + self.assertEqual(['sync', 'get_shard_ranges', 'rsync_then_merge'], + [call[0] for call in repl_calls]) + self.assertEqual(local_broker.db_file, rsync_calls[0][0]) + self.assertEqual(local_broker.get_info()['id'], + os.path.basename(rsync_calls[0][1])) + self.assertFalse(rsync_calls[1:]) + if __name__ == '__main__': unittest.main() diff --git a/test/unit/container/test_server.py b/test/unit/container/test_server.py index 8327e8754b..0e4201b08a 100644 --- a/test/unit/container/test_server.py +++ b/test/unit/container/test_server.py @@ -1460,7 +1460,7 @@ class TestContainerController(unittest.TestCase): self.assertEqual(True, db.is_deleted()) # now save a copy of this db (and remove it from the "current node") db = self.controller._get_container_broker('sda1', 'p', 'a', 'c') - db_path = db.db_file + db_path = db._db_file other_path = os.path.join(self.testdir, 'othernode.db') os.rename(db_path, other_path) # that should make it missing on this node @@ -1474,6 +1474,8 @@ class TestContainerController(unittest.TestCase): def mock_exists(db_path): rv = _real_exists(db_path) + if db_path != db._db_file: + return rv if not mock_called: # be as careful as we might hope backend replication can be... with lock_parent_directory(db_path, timeout=1):