diff --git a/doc/source/account.rst b/doc/source/account.rst index 2ddb1f7d33..7d632579a0 100644 --- a/doc/source/account.rst +++ b/doc/source/account.rst @@ -34,3 +34,12 @@ Account Reaper :undoc-members: :show-inheritance: +.. _account-backend: + +Account Backend +=============== + +.. automodule:: swift.account.backend + :members: + :undoc-members: + :show-inheritance: diff --git a/doc/source/backends.rst b/doc/source/backends.rst deleted file mode 100644 index 4f37ffca1c..0000000000 --- a/doc/source/backends.rst +++ /dev/null @@ -1,16 +0,0 @@ -====================================== -Pluggable Back-ends: API Documentation -====================================== - -.. automodule:: swift.account.backend - :private-members: - :members: - :undoc-members: - -.. automodule:: swift.container.backend - :private-members: - :members: - :undoc-members: - -.. automodule:: swift.obj.diskfile - :members: diff --git a/doc/source/container.rst b/doc/source/container.rst index d80adcaa32..c6b3c46b1b 100644 --- a/doc/source/container.rst +++ b/doc/source/container.rst @@ -41,3 +41,13 @@ Container Sync :members: :undoc-members: :show-inheritance: + +.. _container-backend: + +Container Backend +================= + +.. automodule:: swift.container.backend + :members: + :undoc-members: + :show-inheritance: diff --git a/doc/source/development_ondisk_backends.rst b/doc/source/development_ondisk_backends.rst new file mode 100644 index 0000000000..6762de5346 --- /dev/null +++ b/doc/source/development_ondisk_backends.rst @@ -0,0 +1,37 @@ +=============================== +Pluggable On-Disk Back-end APIs +=============================== + +The internal REST API used between the proxy server and the account, container +and object server is almost identical to public Swift REST API, but with a few +internal extentsions (for example, update an account with a new container). + +The pluggable back-end APIs for the three REST API servers (account, +container, object) abstracts the needs for servicing the various REST APIs +from the details of how data is laid out and stored on-disk. + +The APIs are documented in the reference implementations for all three +servers. For historical reasons, the object server backend reference +implementation module is named `diskfile`, while the account and container +server backend reference implementation modules are named appropriately. + +----------------------------------------- +Back-end API for Account Server REST APIs +----------------------------------------- +.. automodule:: swift.account.backend + :noindex: + :members: + +------------------------------------------- +Back-end API for Container Server REST APIs +------------------------------------------- +.. automodule:: swift.container.backend + :noindex: + :members: + +---------------------------------------- +Back-end API for Object Server REST APIs +---------------------------------------- +.. automodule:: swift.obj.diskfile + :noindex: + :members: diff --git a/doc/source/index.rst b/doc/source/index.rst index 18223efd9c..e934f553e0 100644 --- a/doc/source/index.rst +++ b/doc/source/index.rst @@ -13,16 +13,16 @@ WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. - + Welcome to Swift's documentation! ================================= -Swift is a highly available, distributed, eventually consistent object/blob +Swift is a highly available, distributed, eventually consistent object/blob store. Organizations can use Swift to store lots of data efficiently, safely, and cheaply. This documentation is generated by the Sphinx toolkit and lives in the source tree. Additional documentation on Swift and other components of OpenStack can -be found on the `OpenStack wiki`_ and at http://docs.openstack.org. +be found on the `OpenStack wiki`_ and at http://docs.openstack.org. .. _`OpenStack wiki`: http://wiki.openstack.org @@ -33,7 +33,7 @@ be found on the `OpenStack wiki`_ and at http://docs.openstack.org. .. toctree:: :maxdepth: 1 - + getting_started Overview and Concepts @@ -66,14 +66,14 @@ Developer Documentation development_guidelines development_saio development_auth - backends + development_ondisk_backends Administrator Documentation =========================== .. toctree:: :maxdepth: 1 - + howto_installmultinode deployment_guide apache_deployment_guide @@ -101,5 +101,3 @@ Indices and tables * :ref:`genindex` * :ref:`modindex` * :ref:`search` - - diff --git a/doc/source/object.rst b/doc/source/object.rst index 9a2643d4a7..6da9b9a18b 100644 --- a/doc/source/object.rst +++ b/doc/source/object.rst @@ -44,3 +44,12 @@ Object Auditor :undoc-members: :show-inheritance: +.. _object-diskfile: + +Object Backend +============== + +.. automodule:: swift.obj.diskfile + :members: + :undoc-members: + :show-inheritance: diff --git a/swift/common/exceptions.py b/swift/common/exceptions.py index d6322f9793..b7d677a5b6 100644 --- a/swift/common/exceptions.py +++ b/swift/common/exceptions.py @@ -38,7 +38,11 @@ class DiskFileError(SwiftException): pass -class DiskFileNotOpenError(DiskFileError): +class DiskFileNotOpen(DiskFileError): + pass + + +class DiskFileQuarantined(DiskFileError): pass @@ -50,6 +54,10 @@ class DiskFileNotExist(DiskFileError): pass +class DiskFileDeleted(DiskFileNotExist): + pass + + class DiskFileNoSpace(DiskFileError): pass diff --git a/swift/obj/auditor.py b/swift/obj/auditor.py index 2a491a87f0..2fce5703f6 100644 --- a/swift/obj/auditor.py +++ b/swift/obj/auditor.py @@ -16,14 +16,14 @@ import os import time from swift import gettext_ as _ +from contextlib import closing from eventlet import Timeout from swift.obj import diskfile -from swift.obj import server as object_server from swift.common.utils import get_logger, audit_location_generator, \ - ratelimit_sleep, config_true_value, dump_recon_cache, list_from_csv, json -from swift.common.exceptions import AuditException, DiskFileError, \ + ratelimit_sleep, dump_recon_cache, list_from_csv, json +from swift.common.exceptions import AuditException, DiskFileQuarantined, \ DiskFileNotExist from swift.common.daemon import Daemon @@ -37,7 +37,7 @@ class AuditorWorker(object): self.conf = conf self.logger = logger self.devices = conf.get('devices', '/srv/node') - self.mount_check = config_true_value(conf.get('mount_check', 'true')) + self.diskfile_mgr = diskfile.DiskFileManager(conf, self.logger) self.max_files_per_second = float(conf.get('files_per_second', 20)) self.max_bytes_per_second = float(conf.get('bytes_per_second', 10000000)) @@ -72,10 +72,10 @@ class AuditorWorker(object): total_quarantines = 0 total_errors = 0 time_auditing = 0 - all_locs = audit_location_generator(self.devices, - object_server.DATADIR, '.data', - mount_check=self.mount_check, - logger=self.logger) + all_locs = audit_location_generator( + self.devices, diskfile.DATADIR, '.data', + mount_check=self.diskfile_mgr.mount_check, + logger=self.logger) for path, device, partition in all_locs: loop_time = time.time() self.failsafe_object_audit(path, device, partition) @@ -176,41 +176,46 @@ class AuditorWorker(object): except (Exception, Timeout) as exc: raise AuditException('Error when reading metadata: %s' % exc) _junk, account, container, obj = name.split('/', 3) - df = diskfile.DiskFile(self.devices, device, partition, - account, container, obj, self.logger) - df.open() + df = self.diskfile_mgr.get_diskfile( + device, partition, account, container, obj) try: - try: - obj_size = df.get_data_file_size() - except DiskFileNotExist: - return - except DiskFileError as e: - raise AuditException(str(e)) - if self.stats_sizes: - self.record_stats(obj_size) - if self.zero_byte_only_at_fps and obj_size: - self.passes += 1 - return - for chunk in df: - self.bytes_running_time = ratelimit_sleep( - self.bytes_running_time, self.max_bytes_per_second, - incr_by=len(chunk)) - self.bytes_processed += len(chunk) - self.total_bytes_processed += len(chunk) - df.close() - if df.quarantined_dir: + with df.open(): + metadata = df.get_metadata() + obj_size = int(metadata['Content-Length']) + if self.stats_sizes: + self.record_stats(obj_size) + if self.zero_byte_only_at_fps and obj_size: + self.passes += 1 + return + reader = df.reader() + with closing(reader): + for chunk in reader: + chunk_len = len(chunk) + self.bytes_running_time = ratelimit_sleep( + self.bytes_running_time, + self.max_bytes_per_second, + incr_by=chunk_len) + self.bytes_processed += chunk_len + self.total_bytes_processed += chunk_len + if reader.was_quarantined: self.quarantines += 1 - self.logger.error( - _("ERROR Object %(path)s failed audit and will be " - "quarantined: ETag and file's md5 do not match"), - {'path': path}) - finally: - df.close(verify_file=False) + self.logger.error(_('ERROR Object %(obj)s failed audit and' + ' was quarantined: %(err)s'), + {'obj': path, + 'err': reader.was_quarantined}) + return + except DiskFileNotExist: + return + except DiskFileQuarantined as err: + self.quarantines += 1 + self.logger.error(_('ERROR Object %(obj)s failed audit and was' + ' quarantined: %(err)s'), + {'obj': path, 'err': err}) except AuditException as err: self.logger.increment('quarantines') self.quarantines += 1 - self.logger.error(_('ERROR Object %(obj)s failed audit and will ' - 'be quarantined: %(err)s'), + self.logger.error(_('ERROR Object %(obj)s failed audit and will' + ' be quarantined: %(err)s'), {'obj': path, 'err': err}) diskfile.quarantine_renamer( os.path.join(self.devices, device), path) diff --git a/swift/obj/diskfile.py b/swift/obj/diskfile.py index 656c1797c9..f7d8023c01 100644 --- a/swift/obj/diskfile.py +++ b/swift/obj/diskfile.py @@ -13,7 +13,22 @@ # See the License for the specific language governing permissions and # limitations under the License. -""" Disk File Interface for Swift Object Server""" +""" +Disk File Interface for the Swift Object Server + +The `DiskFile`, `DiskFileWriter` and `DiskFileReader` classes combined define +the on-disk abstraction layer for supporting the object server REST API +interfaces (excluding `REPLICATE`). Other implementations wishing to provide +an alternative backend for the object server must implement the three +classes. An example alternative implementation can be found in the +`mem_server.py` and `mem_diskfile.py` modules along size this one. + +The `DiskFileManager` is a reference implemenation specific class and is not +part of the backend API. + +The remaining methods in this module are considered implementation specifc and +are also not considered part of the backend API. +""" from __future__ import with_statement import cPickle as pickle @@ -24,9 +39,10 @@ import uuid import hashlib import logging import traceback -from os.path import basename, dirname, exists, getmtime, getsize, join +from os.path import basename, dirname, exists, getmtime, join from tempfile import mkstemp from contextlib import contextmanager +from collections import defaultdict from xattr import getxattr, setxattr from eventlet import Timeout @@ -35,10 +51,11 @@ from swift import gettext_ as _ from swift.common.constraints import check_mount from swift.common.utils import mkdirs, normalize_timestamp, \ storage_directory, hash_path, renamer, fallocate, fsync, \ - fdatasync, drop_buffer_cache, ThreadPool, lock_path, write_pickle -from swift.common.exceptions import DiskFileError, DiskFileNotExist, \ + fdatasync, drop_buffer_cache, ThreadPool, lock_path, write_pickle, \ + config_true_value +from swift.common.exceptions import DiskFileQuarantined, DiskFileNotExist, \ DiskFileCollision, DiskFileNoSpace, DiskFileDeviceUnavailable, \ - PathNotDir, DiskFileNotOpenError + DiskFileDeleted, DiskFileError, DiskFileNotOpen, PathNotDir from swift.common.swob import multi_range_iterator @@ -49,6 +66,8 @@ METADATA_KEY = 'user.swift.metadata' # These are system-set metadata keys that cannot be changed with a POST. # They should be lowercase. DATAFILE_SYSTEM_META = set('content-length content-type deleted etag'.split()) +DATADIR = 'objects' +ASYNCDIR = 'async_pending' def read_metadata(fd): @@ -271,300 +290,302 @@ def get_hashes(partition_dir, recalculate=None, do_listdir=False, return hashed, hashes -class DiskWriter(object): +class DiskFileManager(object): + """ + Management class for devices, providing common place for shared parameters + and methods not provided by the DiskFile class (which primarily services + the object server REST API layer). + + The `get_diskfile()` method is how this implementation creates a `DiskFile` + object. + + .. note:: + + This class is reference implementation specific and not part of the + pluggable on-disk backend API. + + .. note:: + + TODO(portante): Not sure what the right name to recommend here, as + "manager" seemed generic enough, though suggestions are welcome. + + :param conf: caller provided configuration object + :param logger: caller provided logger + """ + def __init__(self, conf, logger): + self.logger = logger + self.devices = conf.get('devices', '/srv/node/') + self.disk_chunk_size = int(conf.get('disk_chunk_size', 65536)) + self.keep_cache_size = int(conf.get('keep_cache_size', 5242880)) + self.bytes_per_sync = int(conf.get('mb_per_sync', 512)) * 1024 * 1024 + self.mount_check = config_true_value(conf.get('mount_check', 'true')) + threads_per_disk = int(conf.get('threads_per_disk', '0')) + self.threadpools = defaultdict( + lambda: ThreadPool(nthreads=threads_per_disk)) + + def construct_dev_path(self, device): + """ + Construct the path to a device without checking if it is mounted. + + :param device: name of target device + :returns: full path to the device + """ + return os.path.join(self.devices, device) + + def get_dev_path(self, device): + """ + Return the path to a device, checking to see that it is a proper mount + point based on a configuration parameter. + + :param device: name of target device + :returns: full path to the device, None if the path to the device is + not a proper mount point. + """ + if self.mount_check and not check_mount(self.devices, device): + dev_path = None + else: + dev_path = os.path.join(self.devices, device) + return dev_path + + def pickle_async_update(self, device, account, container, obj, data, + timestamp): + device_path = self.construct_dev_path(device) + async_dir = os.path.join(device_path, ASYNCDIR) + ohash = hash_path(account, container, obj) + self.threadpools[device].run_in_thread( + write_pickle, + data, + os.path.join(async_dir, ohash[-3:], ohash + '-' + + normalize_timestamp(timestamp)), + os.path.join(device_path, 'tmp')) + self.logger.increment('async_pendings') + + def get_diskfile(self, device, partition, account, container, obj, + **kwargs): + dev_path = self.get_dev_path(device) + if not dev_path: + raise DiskFileDeviceUnavailable() + return DiskFile(self, dev_path, self.threadpools[device], + partition, account, container, obj, **kwargs) + + def get_hashes(self, device, partition, suffix): + dev_path = self.get_dev_path(device) + if not dev_path: + raise DiskFileDeviceUnavailable() + partition_path = os.path.join(dev_path, DATADIR, partition) + if not os.path.exists(partition_path): + mkdirs(partition_path) + suffixes = suffix.split('-') if suffix else [] + _junk, hashes = self.threadpools[device].force_run_in_thread( + get_hashes, partition_path, recalculate=suffixes) + return hashes + + +class DiskFileWriter(object): """ Encapsulation of the write context for servicing PUT REST API - requests. Serves as the context manager object for DiskFile's create() - method. + requests. Serves as the context manager object for the + :class:`swift.obj.diskfile.DiskFile` class's + :func:`swift.obj.diskfile.DiskFile.create` method. + + .. note:: + + It is the responsibility of the + :func:`swift.obj.diskfile.DiskFile.create` method context manager to + close the open file descriptor. + + .. note:: + + The arguments to the constructor are considered implementation + specific. The API does not define the constructor arguments. + + :param name: name of object from REST API + :param datadir: on-disk directory object will end up in on + :func:`swift.obj.diskfile.DiskFileWriter.put` + :param fd: open file descriptor of temporary file to receive data + :param tmppath: full path name of the opened file descriptor + :param bytes_per_sync: number bytes written between sync calls + :param threadpool: internal thread pool to use for disk operations """ - def __init__(self, disk_file, fd, tmppath, threadpool): - self.disk_file = disk_file - self.fd = fd - self.tmppath = tmppath - self.upload_size = 0 - self.last_sync = 0 - self.threadpool = threadpool + def __init__(self, name, datadir, fd, tmppath, bytes_per_sync, threadpool): + # Parameter tracking + self._name = name + self._datadir = datadir + self._fd = fd + self._tmppath = tmppath + self._bytes_per_sync = bytes_per_sync + self._threadpool = threadpool + + # Internal attributes + self._upload_size = 0 + self._last_sync = 0 + self._extension = '.data' def write(self, chunk): """ - Write a chunk of data into the temporary file. + Write a chunk of data to disk. All invocations of this method must + come before invoking the :func: + + For this implementation, the data is written into a temporary file. :param chunk: the chunk of data to write as a string object + + :returns: the total number of bytes written to an object """ def _write_entire_chunk(chunk): while chunk: - written = os.write(self.fd, chunk) - self.upload_size += written + written = os.write(self._fd, chunk) + self._upload_size += written chunk = chunk[written:] - self.threadpool.run_in_thread(_write_entire_chunk, chunk) + self._threadpool.run_in_thread(_write_entire_chunk, chunk) # For large files sync every 512MB (by default) written - diff = self.upload_size - self.last_sync - if diff >= self.disk_file.bytes_per_sync: - self.threadpool.force_run_in_thread(fdatasync, self.fd) - drop_buffer_cache(self.fd, self.last_sync, diff) - self.last_sync = self.upload_size + diff = self._upload_size - self._last_sync + if diff >= self._bytes_per_sync: + self._threadpool.force_run_in_thread(fdatasync, self._fd) + drop_buffer_cache(self._fd, self._last_sync, diff) + self._last_sync = self._upload_size + + return self._upload_size def _finalize_put(self, metadata, target_path): # Write the metadata before calling fsync() so that both data and # metadata are flushed to disk. - write_metadata(self.fd, metadata) - # We call fsync() before calling drop_cache() to lower the amount - # of redundant work the drop cache code will perform on the pages - # (now that after fsync the pages will be all clean). - fsync(self.fd) - # From the Department of the Redundancy Department, make sure - # we call drop_cache() after fsync() to avoid redundant work - # (pages all clean). - drop_buffer_cache(self.fd, 0, self.upload_size) - invalidate_hash(dirname(self.disk_file.datadir)) - # After the rename completes, this object will be available for - # other requests to reference. - renamer(self.tmppath, target_path) - hash_cleanup_listdir(self.disk_file.datadir) + write_metadata(self._fd, metadata) + # We call fsync() before calling drop_cache() to lower the amount of + # redundant work the drop cache code will perform on the pages (now + # that after fsync the pages will be all clean). + fsync(self._fd) + # From the Department of the Redundancy Department, make sure we call + # drop_cache() after fsync() to avoid redundant work (pages all + # clean). + drop_buffer_cache(self._fd, 0, self._upload_size) + invalidate_hash(dirname(self._datadir)) + # After the rename completes, this object will be available for other + # requests to reference. + renamer(self._tmppath, target_path) + hash_cleanup_listdir(self._datadir) - def put(self, metadata, extension='.data'): + def put(self, metadata): """ - Finalize writing the file on disk, and renames it from the temp file - to the real location. This should be called after the data has been - written to the temp file. + Finalize writing the file on disk. - :param metadata: dictionary of metadata to be written - :param extension: extension to be used when making the file + For this implementation, this method is responsible for renaming the + temporary file to the final name and directory location. This method + should be called after the final call to + :func:`swift.obj.diskfile.DiskFileWriter.write`. + + :param metadata: dictionary of metadata to be associated with the + object """ - if not self.tmppath: + if not self._tmppath: raise ValueError("tmppath is unusable.") timestamp = normalize_timestamp(metadata['X-Timestamp']) - metadata['name'] = self.disk_file.name - target_path = join(self.disk_file.datadir, timestamp + extension) + metadata['name'] = self._name + target_path = join(self._datadir, timestamp + self._extension) - self.threadpool.force_run_in_thread( + self._threadpool.force_run_in_thread( self._finalize_put, metadata, target_path) -class DiskFile(object): +class DiskFileReader(object): """ - Manage object files on disk. + Encapsulation of the WSGI read context for servicing GET REST API + requests. Serves as the context manager object for the + :class:`swift.obj.diskfile.DiskFile` class's + :func:`swift.obj.diskfile.DiskFile.reader` method. - :param path: path to devices on the node - :param device: device name - :param partition: partition on the device the object lives in - :param account: account name for the object - :param container: container name for the object - :param obj: object name for the object - :param disk_chunk_size: size of chunks on file reads - :param bytes_per_sync: number of bytes between fdatasync calls + .. note:: + + The quarantining behavior of this method is considered implementation + specific, and is not required of the API. + + .. note:: + + The arguments to the constructor are considered implementation + specific. The API does not define the constructor arguments. + + :param fp: open file object pointer reference + :param data_file: on-disk data file name for the object + :param obj_size: verified on-disk size of the object + :param etag: expected metadata etag value for entire file + :param threadpool: thread pool to use for read operations + :param disk_chunk_size: size of reads from disk in bytes + :param keep_cache_size: maximum object size that will be kept in cache + :param device_path: on-disk device path, used when quarantining an obj + :param logger: logger caller wants this object to use :param iter_hook: called when __iter__ returns a chunk - :param threadpool: thread pool in which to do blocking operations + :param keep_cache: should resulting reads be kept in the buffer cache """ - - def __init__(self, path, device, partition, account, container, obj, - logger, disk_chunk_size=65536, - bytes_per_sync=(512 * 1024 * 1024), - iter_hook=None, threadpool=None, obj_dir='objects', - mount_check=False): - if mount_check and not check_mount(path, device): - raise DiskFileDeviceUnavailable() - self.disk_chunk_size = disk_chunk_size - self.bytes_per_sync = bytes_per_sync - self.iter_hook = iter_hook - self.name = '/' + '/'.join((account, container, obj)) - name_hash = hash_path(account, container, obj) - self.datadir = join( - path, device, storage_directory(obj_dir, partition, name_hash)) - self.device_path = join(path, device) - self.tmpdir = join(path, device, 'tmp') - self.logger = logger - self._metadata = None - self.data_file = None - self._data_file_size = None - self.fp = None - self.iter_etag = None - self.started_at_0 = False - self.read_to_eof = False - self.quarantined_dir = None - self.suppress_file_closing = False - self._verify_close = False - self.threadpool = threadpool or ThreadPool(nthreads=0) - - # FIXME(clayg): this attribute is set after open and affects the - # behavior of the class (i.e. public interface) - self.keep_cache = False - - def open(self, verify_close=False): - """ - Open the file and read the metadata. - - This method must populate the _metadata attribute. - - :param verify_close: force implicit close to verify_file, no effect on - explicit close. - - :raises DiskFileCollision: on md5 collision - """ - data_file, meta_file, ts_file = self._get_ondisk_file() - if not data_file: - if ts_file: - self._construct_from_ts_file(ts_file) + def __init__(self, fp, data_file, obj_size, etag, threadpool, + disk_chunk_size, keep_cache_size, device_path, logger, + iter_hook=None, keep_cache=False): + # Parameter tracking + self._fp = fp + self._data_file = data_file + self._obj_size = obj_size + self._etag = etag + self._threadpool = threadpool + self._disk_chunk_size = disk_chunk_size + self._device_path = device_path + self._logger = logger + self._iter_hook = iter_hook + if keep_cache: + # Caller suggests we keep this in cache, only do it if the + # object's size is less than the maximum. + self._keep_cache = obj_size < keep_cache_size else: - self.fp = self._construct_from_data_file(data_file, meta_file) - self._verify_close = verify_close - self._metadata = self._metadata or {} - return self + self._keep_cache = False - def __enter__(self): - if self._metadata is None: - raise DiskFileNotOpenError() - return self - - def __exit__(self, t, v, tb): - self.close(verify_file=self._verify_close) - - def _get_ondisk_file(self): - """ - Do the work to figure out if the data directory exists, and if so, - determine the on-disk files to use. - - :returns: a tuple of data, meta and ts (tombstone) files, in one of - three states: - - 1. all three are None - - data directory does not exist, or there are no files in - that directory - - 2. ts_file is not None, data_file is None, meta_file is None - - object is considered deleted - - 3. data_file is not None, ts_file is None - - object exists, and optionally has fast-POST metadata - """ - data_file = meta_file = ts_file = None - try: - files = sorted(os.listdir(self.datadir), reverse=True) - except OSError as err: - if err.errno != errno.ENOENT: - raise - # The data directory does not exist, so the object cannot exist. - else: - for afile in files: - assert ts_file is None, "On-disk file search loop" \ - " continuing after tombstone, %s, encountered" % ts_file - assert data_file is None, "On-disk file search loop" \ - " continuing after data file, %s, encountered" % data_file - if afile.endswith('.ts'): - meta_file = None - ts_file = join(self.datadir, afile) - break - if afile.endswith('.meta') and not meta_file: - meta_file = join(self.datadir, afile) - # NOTE: this does not exit this loop, since a fast-POST - # operation just updates metadata, writing one or more - # .meta files, the data file will have an older timestamp, - # so we keep looking. - continue - if afile.endswith('.data'): - data_file = join(self.datadir, afile) - break - assert ((data_file is None and meta_file is None and ts_file is None) - or (ts_file is not None and data_file is None - and meta_file is None) - or (data_file is not None and ts_file is None)), \ - "On-disk file search algorithm contract is broken: data_file:" \ - " %s, meta_file: %s, ts_file: %s" % (data_file, meta_file, ts_file) - return data_file, meta_file, ts_file - - def _construct_from_ts_file(self, ts_file): - """ - A tombstone means the object is considered deleted. We just need to - pull the metadata from the tombstone file which has the timestamp. - """ - with open(ts_file) as fp: - self._metadata = read_metadata(fp) - self._metadata['deleted'] = True - - def _verify_name(self): - """ - Verify the metadata's name value matches what we think the object is - named. - """ - try: - mname = self._metadata['name'] - except KeyError: - pass - else: - if mname != self.name: - self.logger.error(_('Client path %(client)s does not match ' - 'path stored in object metadata %(meta)s'), - {'client': self.name, 'meta': mname}) - raise DiskFileCollision('Client path does not match path ' - 'stored in object metadata') - - def _construct_from_data_file(self, data_file, meta_file): - """ - Open the data file to fetch its metadata, and fetch the metadata from - the fast-POST .meta file as well if it exists, merging them properly. - - :returns: the opened data file pointer - """ - fp = open(data_file, 'rb') - datafile_metadata = read_metadata(fp) - if meta_file: - with open(meta_file) as mfp: - self._metadata = read_metadata(mfp) - sys_metadata = dict( - [(key, val) for key, val in datafile_metadata.iteritems() - if key.lower() in DATAFILE_SYSTEM_META]) - self._metadata.update(sys_metadata) - else: - self._metadata = datafile_metadata - self._verify_name() - self.data_file = data_file - return fp + # Internal Attributes + self._iter_etag = None + self._bytes_read = 0 + self._started_at_0 = False + self._read_to_eof = False + self._suppress_file_closing = False + self._quarantined_dir = None + # Currently referenced by the object Auditor only + self.was_quarantined = '' def __iter__(self): """Returns an iterator over the data file.""" - if self.fp is None: - raise DiskFileNotOpenError() try: dropped_cache = 0 - read = 0 - self.started_at_0 = False - self.read_to_eof = False - if self.fp.tell() == 0: - self.started_at_0 = True - self.iter_etag = hashlib.md5() + self._bytes_read = 0 + self._started_at_0 = False + self._read_to_eof = False + if self._fp.tell() == 0: + self._started_at_0 = True + self._iter_etag = hashlib.md5() while True: - chunk = self.threadpool.run_in_thread( - self.fp.read, self.disk_chunk_size) + chunk = self._threadpool.run_in_thread( + self._fp.read, self._disk_chunk_size) if chunk: - if self.iter_etag: - self.iter_etag.update(chunk) - read += len(chunk) - if read - dropped_cache > (1024 * 1024): - self._drop_cache(self.fp.fileno(), dropped_cache, - read - dropped_cache) - dropped_cache = read + if self._iter_etag: + self._iter_etag.update(chunk) + self._bytes_read += len(chunk) + if self._bytes_read - dropped_cache > (1024 * 1024): + self._drop_cache(self._fp.fileno(), dropped_cache, + self._bytes_read - dropped_cache) + dropped_cache = self._bytes_read yield chunk - if self.iter_hook: - self.iter_hook() + if self._iter_hook: + self._iter_hook() else: - self.read_to_eof = True - self._drop_cache(self.fp.fileno(), dropped_cache, - read - dropped_cache) + self._read_to_eof = True + self._drop_cache(self._fp.fileno(), dropped_cache, + self._bytes_read - dropped_cache) break finally: - if not self.suppress_file_closing: + if not self._suppress_file_closing: self.close() def app_iter_range(self, start, stop): """Returns an iterator over the data file for range (start, stop)""" if start or start == 0: - self.fp.seek(start) + self._fp.seek(start) if stop is not None: length = stop - start else: @@ -579,7 +600,7 @@ class DiskFile(object): break yield chunk finally: - if not self.suppress_file_closing: + if not self._suppress_file_closing: self.close() def app_iter_ranges(self, ranges, content_type, boundary, size): @@ -588,101 +609,425 @@ class DiskFile(object): yield '' else: try: - self.suppress_file_closing = True + self._suppress_file_closing = True for chunk in multi_range_iterator( ranges, content_type, boundary, size, self.app_iter_range): yield chunk finally: - self.suppress_file_closing = False - self.close() + self._suppress_file_closing = False + try: + self.close() + except DiskFileQuarantined: + pass + + def _drop_cache(self, fd, offset, length): + """Method for no-oping buffer cache drop method.""" + if not self._keep_cache: + drop_buffer_cache(fd, offset, length) + + def _quarantine(self, msg): + self._quarantined_dir = self._threadpool.run_in_thread( + quarantine_renamer, self._device_path, self._data_file) + self._logger.increment('quarantines') + self.was_quarantined = msg def _handle_close_quarantine(self): """Check if file needs to be quarantined""" - try: - self.get_data_file_size() - except DiskFileNotExist: - return - except DiskFileError: - self.quarantine() - return + if self._bytes_read != self._obj_size: + self._quarantine( + "Bytes read: %s, does not match metadata: %s" % ( + self._bytes_read, self._obj_size)) + elif self._iter_etag and \ + self._etag != self._iter_etag.hexdigest(): + self._quarantine( + "ETag %s and file's md5 %s do not match" % ( + self._etag, self._iter_etag.hexdigest())) - if self.iter_etag and self.started_at_0 and self.read_to_eof and \ - self.iter_etag.hexdigest() != self._metadata['ETag']: - self.quarantine() - - def close(self, verify_file=True): + def close(self): """ - Close the file. Will handle quarantining file if necessary. + Close the open file handle if present. - :param verify_file: Defaults to True. If false, will not check - file to see if it needs quarantining. + For this specific implementation, this method will handle quarantining + the file if necessary. """ - if self.fp: + if self._fp: try: - if verify_file: + if self._started_at_0 and self._read_to_eof: self._handle_close_quarantine() except (Exception, Timeout) as e: - self.logger.error(_( - 'ERROR DiskFile %(data_file)s in ' - '%(data_dir)s close failure: %(exc)s : %(stack)s'), + self._logger.error(_( + 'ERROR DiskFile %(data_file)s' + ' close failure: %(exc)s : %(stack)s'), {'exc': e, 'stack': ''.join(traceback.format_stack()), - 'data_file': self.data_file, 'data_dir': self.datadir}) + 'data_file': self._data_file}) finally: - self.fp.close() - self.fp = None + fp, self._fp = self._fp, None + fp.close() + + +class DiskFile(object): + """ + Manage object files. + + This specific implementation manages object files on a disk formatted with + a POSIX-compliant file system that supports extended attributes as + metadata on a file or directory. + + .. note:: + + The arguments to the constructor are considered implementation + specific. The API does not define the constructor arguments. + + :param mgr: associated DiskFileManager instance + :param device_path: path to the target device or drive + :param threadpool: thread pool to use for blocking operations + :param partition: partition on the device in which the object lives + :param account: account name for the object + :param container: container name for the object + :param obj: object name for the object + """ + + def __init__(self, mgr, device_path, threadpool, partition, + account, container, obj): + self._mgr = mgr + self._device_path = device_path + self._threadpool = threadpool or ThreadPool(nthreads=0) + self._logger = mgr.logger + self._disk_chunk_size = mgr.disk_chunk_size + self._bytes_per_sync = mgr.bytes_per_sync + self._name = '/' + '/'.join((account, container, obj)) + name_hash = hash_path(account, container, obj) + self._datadir = join( + device_path, storage_directory(DATADIR, partition, name_hash)) + self._tmpdir = join(device_path, 'tmp') self._metadata = None - self._data_file_size = None - self._verify_close = False + self._data_file = None + self._fp = None + self._quarantined_dir = None + + def open(self): + """ + Open the object. + + This implementation opens the data file representing the object, reads + the associated metadata in the extended attributes, additionally + combining metadata from fast-POST `.meta` files. + + .. note:: + + An implementation is allowed to raise any of the following + exceptions, but is only required to raise `DiskFileNotExist` when + the object representation does not exist. + + :raises DiskFileCollision: on name mis-match with metadata + :raises DiskFileNotExist: if the object does not exist + :raises DiskFileDeleted: if the object was previously deleted + :raises DiskFileQuarantined: if while reading metadata of the file + some data did pass cross checks + :returns: itself for use as a context manager + """ + data_file, meta_file, ts_file = self._get_ondisk_file() + if not data_file: + raise self._construct_exception_from_ts_file(ts_file) + self._fp = self._construct_from_data_file( + data_file, meta_file) + # This method must populate the internal _metadata attribute. + self._metadata = self._metadata or {} + self._data_file = data_file + return self + + def __enter__(self): + """ + Context enter. + + .. note:: + + An implemenation shall raise `DiskFileNotOpen` when has not + previously invoked the :func:`swift.obj.diskfile.DiskFile.open` + method. + """ + if self._metadata is None: + raise DiskFileNotOpen() + return self + + def __exit__(self, t, v, tb): + """ + Context exit. + + .. note:: + + This method will be invoked by the object server while servicing + the REST API *before* the object has actually been read. It is the + responsibility of the implementation to properly handle that. + """ + if self._fp is not None: + fp, self._fp = self._fp, None + fp.close() + + def _quarantine(self, data_file, msg): + """ + Quarantine a file; responsible for incrementing the associated loggers + count of quarantines. + + :param data_file: full path of data file to quarantine + :param msg: reason for quarantining to be included in the exception + :raises DiskFileQuarantine: + """ + self._quarantined_dir = self._threadpool.run_in_thread( + quarantine_renamer, self._device_path, data_file) + self._logger.increment('quarantines') + raise DiskFileQuarantined(msg) + + def _get_ondisk_file(self): + """ + Do the work to figure out if the data directory exists, and if so, + determine the on-disk files to use. + + :returns: a tuple of data, meta and ts (tombstone) files, in one of + three states: + + * all three are None + + data directory does not exist, or there are no files in + that directory + + * ts_file is not None, data_file is None, meta_file is None + + object is considered deleted + + * data_file is not None, ts_file is None + + object exists, and optionally has fast-POST metadata + """ + data_file = meta_file = ts_file = None + try: + files = sorted(os.listdir(self._datadir), reverse=True) + except OSError as err: + if err.errno != errno.ENOENT: + raise DiskFileError() + # The data directory does not exist, so the object cannot exist. + else: + for afile in files: + assert ts_file is None, "On-disk file search loop" \ + " continuing after tombstone, %s, encountered" % ts_file + assert data_file is None, "On-disk file search loop" \ + " continuing after data file, %s, encountered" % data_file + if afile.endswith('.ts'): + meta_file = None + ts_file = join(self._datadir, afile) + break + if afile.endswith('.meta') and not meta_file: + meta_file = join(self._datadir, afile) + # NOTE: this does not exit this loop, since a fast-POST + # operation just updates metadata, writing one or more + # .meta files, the data file will have an older timestamp, + # so we keep looking. + continue + if afile.endswith('.data'): + data_file = join(self._datadir, afile) + break + assert ((data_file is None and meta_file is None and ts_file is None) + or (ts_file is not None and data_file is None + and meta_file is None) + or (data_file is not None and ts_file is None)), \ + "On-disk file search algorithm contract is broken: data_file:" \ + " %s, meta_file: %s, ts_file: %s" % (data_file, meta_file, ts_file) + return data_file, meta_file, ts_file + + def _construct_exception_from_ts_file(self, ts_file): + """ + If a tombstone is present it means the object is considered + deleted. We just need to pull the metadata from the tombstone file + which has the timestamp to construct the deleted exception. If there + was no tombstone, just report it does not exist. + + :param ts_file: the tombstone file name found on disk + :returns: DiskFileDeleted if the ts_file was provided, else + DiskFileNotExist + """ + if not ts_file: + exc = DiskFileNotExist() + else: + with open(ts_file) as fp: + metadata = read_metadata(fp) + # All well and good that we have found a tombstone file, but + # we don't have a data file so we are just going to raise an + # exception that we could not find the object, providing the + # tombstone's timestamp. + exc = DiskFileDeleted() + exc.timestamp = metadata['X-Timestamp'] + return exc + + def _verify_data_file(self, data_file, fp): + """ + Verify the metadata's name value matches what we think the object is + named. + + :param data_file: data file name being consider, used when quarantines + occur + :param fp: open file pointer so that we can `fstat()` the file to + verify the on-disk size with Content-Length metadata value + :raises DiskFileCollision: if the metadata stored name does not match + the referenced name of the file + :raises DiskFileNotExist: if the object has expired + :raises DiskFileQuarantined: if data inconsistencies were detected + between the metadata and the file-system + metadata + """ + try: + mname = self._metadata['name'] + except KeyError: + self._quarantine(data_file, "missing name metadata") + else: + if mname != self._name: + self._logger.error( + _('Client path %(client)s does not match ' + 'path stored in object metadata %(meta)s'), + {'client': self._name, 'meta': mname}) + raise DiskFileCollision('Client path does not match path ' + 'stored in object metadata') + try: + x_delete_at = int(self._metadata['X-Delete-At']) + except KeyError: + pass + except ValueError: + # Quarantine, the x-delete-at key is present but not an + # integer. + self._quarantine( + data_file, "bad metadata x-delete-at value %s" % ( + self._metadata['X-Delete-At'])) + else: + if x_delete_at <= time.time(): + raise DiskFileNotExist('Expired') + try: + metadata_size = int(self._metadata['Content-Length']) + except KeyError: + self._quarantine( + data_file, "missing content-length in metadata") + except ValueError: + # Quarantine, the content-length key is present but not an + # integer. + self._quarantine( + data_file, "bad metadata content-length value %s" % ( + self._metadata['Content-Length'])) + fd = fp.fileno() + try: + statbuf = os.fstat(fd) + except OSError as err: + # Quarantine, we can't successfully stat the file. + self._quarantine(data_file, "not stat-able: %s" % err) + else: + obj_size = statbuf.st_size + if metadata_size is not None and obj_size != metadata_size: + self._quarantine( + data_file, "metadata content-length %s does" + " not match actual object size %s" % ( + metadata_size, statbuf.st_size)) + return obj_size + + def _construct_from_data_file(self, data_file, meta_file): + """ + Open the `.data` file to fetch its metadata, and fetch the metadata + from the fast-POST `.meta` file as well if it exists, merging them + properly. + + :param data_file: on-disk `.data` file being considered + :param meta_file: on-disk fast-POST `.meta` file being considered + :returns: an opened data file pointer + :raises DiskFileError: various exceptions from + :func:`swift.obj.diskfile.DiskFile._verify_data_file` + """ + fp = open(data_file, 'rb') + datafile_metadata = read_metadata(fp) + if meta_file: + with open(meta_file) as mfp: + self._metadata = read_metadata(mfp) + sys_metadata = dict( + [(key, val) for key, val in datafile_metadata.iteritems() + if key.lower() in DATAFILE_SYSTEM_META]) + self._metadata.update(sys_metadata) + else: + self._metadata = datafile_metadata + self._verify_data_file(data_file, fp) + return fp def get_metadata(self): """ - Provide the metadata for an object as a dictionary. + Provide the metadata for a previously opened object as a dictionary. :returns: object's metadata dictionary + :raises DiskFileNotOpen: if the + :func:`swift.obj.diskfile.DiskFile.open` method was not previously + invoked """ if self._metadata is None: - raise DiskFileNotOpenError() + raise DiskFileNotOpen() return self._metadata - def is_deleted(self): + def read_metadata(self): """ - Check if the file is deleted. + Return the metadata for an object without requiring the caller to open + the object first. - :returns: True if the file doesn't exist or has been flagged as - deleted. + :returns: metadata dictionary for an object + :raises DiskFileError: this implementation will raise the same + errors as the `open()` method. """ - return not self.data_file or 'deleted' in self._metadata + with self.open(): + return self.get_metadata() - def is_expired(self): + def reader(self, iter_hook=None, keep_cache=False): """ - Check if the file is expired. + Return a :class:`swift.common.swob.Response` class compatible + "`app_iter`" object as defined by + :class:`swift.obj.diskfile.DiskFileReader`. - :returns: True if the file has an X-Delete-At in the past + For this implementation, the responsibility of closing the open file + is passed to the :class:`swift.obj.diskfile.DiskFileReader` object. + + :param iter_hook: called when __iter__ returns a chunk + :param keep_cache: caller's preference for keeping data read in the + OS buffer cache + :returns: a :class:`swift.obj.diskfile.DiskFileReader` object """ - return ('X-Delete-At' in self._metadata and - int(self._metadata['X-Delete-At']) <= time.time()) + dr = DiskFileReader( + self._fp, self._data_file, int(self._metadata['Content-Length']), + self._metadata['ETag'], self._threadpool, self._disk_chunk_size, + self._mgr.keep_cache_size, self._device_path, self._logger, + iter_hook=iter_hook, keep_cache=keep_cache) + # At this point the reader object is now responsible for closing + # the file pointer. + self._fp = None + return dr @contextmanager def create(self, size=None): """ Context manager to create a file. We create a temporary file first, and - then return a DiskWriter object to encapsulate the state. + then return a DiskFileWriter object to encapsulate the state. + + .. note:: + + An implementation is not required to perform on-disk + preallocations even if the parameter is specified. But if it does + and it fails, it must raise a `DiskFileNoSpace` exception. :param size: optional initial size of file to explicitly allocate on disk :raises DiskFileNoSpace: if a size is specified and allocation fails """ - if not exists(self.tmpdir): - mkdirs(self.tmpdir) - fd, tmppath = mkstemp(dir=self.tmpdir) + if not exists(self._tmpdir): + mkdirs(self._tmpdir) + fd, tmppath = mkstemp(dir=self._tmpdir) try: if size is not None and size > 0: try: fallocate(fd, size) except OSError: raise DiskFileNoSpace() - yield DiskWriter(self, fd, tmppath, self.threadpool) + yield DiskFileWriter(self._name, self._datadir, fd, tmppath, + self._bytes_per_sync, self._threadpool) finally: try: os.close(fd) @@ -693,75 +1038,39 @@ class DiskFile(object): except OSError: pass - def put_metadata(self, metadata, tombstone=False): + def write_metadata(self, metadata): """ - Short hand for putting metadata to .meta and .ts files. + Write a block of metadata to an object without requiring the caller to + create the object first. Supports fast-POST behavior semantics. - :param metadata: dictionary of metadata to be written - :param tombstone: whether or not we are writing a tombstone + :param metadata: dictionary of metadata to be associated with the + object + :raises DiskFileError: this implementation will raise the same + errors as the `create()` method. """ - extension = '.ts' if tombstone else '.meta' with self.create() as writer: - writer.put(metadata, extension=extension) + writer._extension = '.meta' + writer.put(metadata) def delete(self, timestamp): """ - Simple short hand for marking an object as deleted. Provides - a layer of abstraction. + Delete the object. - :param timestamp: time stamp to mark the object deleted at + This implementation creates a tombstone file using the given + timestamp, and removes any older versions of the object file. Any + file that has an older timestamp than timestamp will be deleted. + + .. note:: + + An implementation is free to use or ignore the timestamp + parameter. + + :param timestamp: timestamp to compare with each file + :raises DiskFileError: this implementation will raise the same + errors as the `create()` method. """ - self.put_metadata({'X-Timestamp': timestamp}, tombstone=True) + timestamp = normalize_timestamp(timestamp) - def _drop_cache(self, fd, offset, length): - """Method for no-oping buffer cache drop method.""" - if not self.keep_cache: - drop_buffer_cache(fd, offset, length) - - def quarantine(self): - """ - In the case that a file is corrupted, move it to a quarantined - area to allow replication to fix it. - - :returns: if quarantine is successful, path to quarantined - directory otherwise None - """ - if not (self.is_deleted() or self.quarantined_dir): - self.quarantined_dir = self.threadpool.run_in_thread( - quarantine_renamer, self.device_path, self.data_file) - self.logger.increment('quarantines') - return self.quarantined_dir - - def get_data_file_size(self): - """ - Returns the os.path.getsize for the file. Raises an exception if this - file does not match the Content-Length stored in the metadata. Or if - self.data_file does not exist. - - :returns: file size as an int - :raises DiskFileError: on file size mismatch. - :raises DiskFileNotExist: on file not existing (including deleted) - """ - if self._data_file_size is None: - self._data_file_size = self._get_data_file_size() - return self._data_file_size - - def _get_data_file_size(self): - # ensure file is opened - metadata = self.get_metadata() - try: - file_size = 0 - if self.data_file: - file_size = self.threadpool.run_in_thread( - getsize, self.data_file) - if 'Content-Length' in metadata: - metadata_size = int(metadata['Content-Length']) - if file_size != metadata_size: - raise DiskFileError( - 'Content-Length of %s does not match file size ' - 'of %s' % (metadata_size, file_size)) - return file_size - except OSError as err: - if err.errno != errno.ENOENT: - raise - raise DiskFileNotExist('Data File does not exist.') + with self.create() as deleter: + deleter._extension = '.ts' + deleter.put({'X-Timestamp': timestamp}) diff --git a/swift/obj/mem_diskfile.py b/swift/obj/mem_diskfile.py new file mode 100644 index 0000000000..d29ed625da --- /dev/null +++ b/swift/obj/mem_diskfile.py @@ -0,0 +1,408 @@ +# Copyright (c) 2010-2013 OpenStack, LLC. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +# implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +""" In-Memory Disk File Interface for Swift Object Server""" + +from __future__ import with_statement +import cStringIO +import time +import hashlib +from contextlib import contextmanager + +from eventlet import Timeout + +from swift.common.utils import normalize_timestamp +from swift.common.exceptions import DiskFileQuarantined, DiskFileNotExist, \ + DiskFileCollision, DiskFileDeleted, DiskFileNotOpen +from swift.common.swob import multi_range_iterator + + +class InMemoryFileSystem(object): + """ + A very simplistic in-memory file system scheme. + + There is one dictionary mapping a given object name to a tuple. The first + entry in the tuble is the cStringIO buffer representing the file contents, + the second entry is the metadata dictionary. + """ + + def __init__(self): + self._filesystem = {} + + def get_object(self, name): + val = self._filesystem.get(name) + if val is None: + data, metadata = None, None + else: + data, metadata = val + return data, metadata + + def put_object(self, name, data, metadata): + self._filesystem[name] = (data, metadata) + + def del_object(self, name): + del self._filesystem[name] + + def get_diskfile(self, account, container, obj, **kwargs): + return DiskFile(self, account, container, obj) + + +class DiskFileWriter(object): + """ + .. note:: + Sample alternative pluggable on-disk backend implementation. + + Encapsulation of the write context for servicing PUT REST API + requests. Serves as the context manager object for DiskFile's create() + method. + + :param fs: internal file system object to use + :param name: standard object name + :param fp: `StringIO` in-memory representation object + """ + def __init__(self, fs, name, fp): + self._filesystem = fs + self._name = name + self._fp = fp + self._upload_size = 0 + + def write(self, chunk): + """ + Write a chunk of data into the `StringIO` object. + + :param chunk: the chunk of data to write as a string object + """ + self._fp.write(chunk) + self._upload_size += len(chunk) + return self._upload_size + + def put(self, metadata): + """ + Make the final association in the in-memory file system for this name + with the `StringIO` object. + + :param metadata: dictionary of metadata to be written + :param extension: extension to be used when making the file + """ + metadata['name'] = self._name + self._filesystem.put_object(self._name, self._fp, metadata) + + +class DiskFileReader(object): + """ + .. note:: + Sample alternative pluggable on-disk backend implementation. + + Encapsulation of the read context for servicing GET REST API + requests. Serves as the context manager object for DiskFile's reader() + method. + + :param name: object name + :param fp: open file object pointer reference + :param obj_size: on-disk size of object in bytes + :param etag: MD5 hash of object from metadata + :param iter_hook: called when __iter__ returns a chunk + """ + def __init__(self, name, fp, obj_size, etag, iter_hook=None): + self._name = name + self._fp = fp + self._obj_size = obj_size + self._etag = etag + self._iter_hook = iter_hook + # + self._iter_etag = None + self._bytes_read = 0 + self._started_at_0 = False + self._read_to_eof = False + self._suppress_file_closing = False + # + self.was_quarantined = '' + + def __iter__(self): + try: + self._bytes_read = 0 + self._started_at_0 = False + self._read_to_eof = False + if self._fp.tell() == 0: + self._started_at_0 = True + self._iter_etag = hashlib.md5() + while True: + chunk = self._fp.read() + if chunk: + if self._iter_etag: + self._iter_etag.update(chunk) + self._bytes_read += len(chunk) + yield chunk + if self._iter_hook: + self._iter_hook() + else: + self._read_to_eof = True + break + finally: + if not self._suppress_file_closing: + self.close() + + def app_iter_range(self, start, stop): + if start or start == 0: + self._fp.seek(start) + if stop is not None: + length = stop - start + else: + length = None + try: + for chunk in self: + if length is not None: + length -= len(chunk) + if length < 0: + # Chop off the extra: + yield chunk[:length] + break + yield chunk + finally: + if not self._suppress_file_closing: + self.close() + + def app_iter_ranges(self, ranges, content_type, boundary, size): + if not ranges: + yield '' + else: + try: + self._suppress_file_closing = True + for chunk in multi_range_iterator( + ranges, content_type, boundary, size, + self.app_iter_range): + yield chunk + finally: + self._suppress_file_closing = False + try: + self.close() + except DiskFileQuarantined: + pass + + def _quarantine(self, msg): + self.was_quarantined = msg + + def _handle_close_quarantine(self): + if self._bytes_read != self._obj_size: + self._quarantine( + "Bytes read: %s, does not match metadata: %s" % ( + self.bytes_read, self._obj_size)) + elif self._iter_etag and \ + self._etag != self._iter_etag.hexdigest(): + self._quarantine( + "ETag %s and file's md5 %s do not match" % ( + self._etag, self._iter_etag.hexdigest())) + + def close(self): + """ + Close the file. Will handle quarantining file if necessary. + """ + if self._fp: + try: + if self._started_at_0 and self._read_to_eof: + self._handle_close_quarantine() + except (Exception, Timeout): + pass + finally: + self._fp = None + + +class DiskFile(object): + """ + .. note:: + + Sample alternative pluggable on-disk backend implementation. This + example duck-types the reference implementation DiskFile class. + + Manage object files in-memory. + + :param mgr: DiskFileManager + :param device_path: path to the target device or drive + :param threadpool: thread pool to use for blocking operations + :param partition: partition on the device in which the object lives + :param account: account name for the object + :param container: container name for the object + :param obj: object name for the object + :param iter_hook: called when __iter__ returns a chunk + :param keep_cache: caller's preference for keeping data read in the cache + """ + + def __init__(self, fs, account, container, obj): + self._name = '/' + '/'.join((account, container, obj)) + self._metadata = None + self._fp = None + self._filesystem = fs + + def open(self): + """ + Open the file and read the metadata. + + This method must populate the _metadata attribute. + :raises DiskFileCollision: on name mis-match with metadata + :raises DiskFileDeleted: if it does not exist, or a tombstone is + present + :raises DiskFileQuarantined: if while reading metadata of the file + some data did pass cross checks + """ + fp, self._metadata = self._filesystem.get_object(self._name) + if fp is None: + raise DiskFileDeleted() + self._fp = self._verify_data_file(fp) + self._metadata = self._metadata or {} + return self + + def __enter__(self): + if self._metadata is None: + raise DiskFileNotOpen() + return self + + def __exit__(self, t, v, tb): + if self._fp is not None: + self._fp = None + + def _verify_data_file(self, fp): + """ + Verify the metadata's name value matches what we think the object is + named. + + :raises DiskFileCollision: if the metadata stored name does not match + the referenced name of the file + :raises DiskFileNotExist: if the object has expired + :raises DiskFileQuarantined: if data inconsistencies were detected + between the metadata and the file-system + metadata + """ + try: + mname = self._metadata['name'] + except KeyError: + self._quarantine(self._name, "missing name metadata") + else: + if mname != self._name: + raise DiskFileCollision('Client path does not match path ' + 'stored in object metadata') + try: + x_delete_at = int(self._metadata['X-Delete-At']) + except KeyError: + pass + except ValueError: + # Quarantine, the x-delete-at key is present but not an + # integer. + self._quarantine( + self._name, "bad metadata x-delete-at value %s" % ( + self._metadata['X-Delete-At'])) + else: + if x_delete_at <= time.time(): + raise DiskFileNotExist('Expired') + try: + metadata_size = int(self._metadata['Content-Length']) + except KeyError: + self._quarantine( + self._name, "missing content-length in metadata") + except ValueError: + # Quarantine, the content-length key is present but not an + # integer. + self._quarantine( + self._name, "bad metadata content-length value %s" % ( + self._metadata['Content-Length'])) + try: + fp.seek(0, 2) + obj_size = fp.tell() + fp.seek(0, 0) + except OSError as err: + # Quarantine, we can't successfully stat the file. + self._quarantine(self._name, "not stat-able: %s" % err) + if obj_size != metadata_size: + self._quarantine( + self._name, "metadata content-length %s does" + " not match actual object size %s" % ( + metadata_size, obj_size)) + return fp + + def get_metadata(self): + """ + Provide the metadata for an object as a dictionary. + + :returns: object's metadata dictionary + """ + if self._metadata is None: + raise DiskFileNotOpen() + return self._metadata + + def read_metadata(self): + """ + Return the metadata for an object. + + :returns: metadata dictionary for an object + """ + with self.open(): + return self.get_metadata() + + def reader(self, iter_hook=None, keep_cache=False): + """ + Return a swift.common.swob.Response class compatible "app_iter" + object. The responsibility of closing the open file is passed to the + DiskFileReader object. + + :param iter_hook: + :param keep_cache: + """ + dr = DiskFileReader(self._name, self._fp, + int(self._metadata['Content-Length']), + self._metadata['ETag'], + iter_hook=iter_hook) + # At this point the reader object is now responsible for + # the file pointer. + self._fp = None + return dr + + @contextmanager + def create(self, size=None): + """ + Context manager to create a file. We create a temporary file first, and + then return a DiskFileWriter object to encapsulate the state. + + :param size: optional initial size of file to explicitly allocate on + disk + :raises DiskFileNoSpace: if a size is specified and allocation fails + """ + fp = cStringIO.StringIO() + try: + yield DiskFileWriter(self._filesystem, self._name, fp) + finally: + del fp + + def write_metadata(self, metadata): + """ + Write a block of metadata to an object. + """ + cur_fp = self._filesystem.get(self._name) + if cur_fp is not None: + self._filesystem[self._name] = (cur_fp, metadata) + + def delete(self, timestamp): + """ + Perform a delete for the given object in the given container under the + given account. + + This creates a tombstone file with the given timestamp, and removes + any older versions of the object file. Any file that has an older + timestamp than timestamp will be deleted. + + :param timestamp: timestamp to compare with each file + """ + timestamp = normalize_timestamp(timestamp) + fp, md = self._filesystem.get_object(self._name) + if md['X-Timestamp'] < timestamp: + self._filesystem.del_object(self._name) diff --git a/swift/obj/mem_server.py b/swift/obj/mem_server.py new file mode 100644 index 0000000000..a1f985ebc3 --- /dev/null +++ b/swift/obj/mem_server.py @@ -0,0 +1,111 @@ +# Copyright (c) 2010-2013 OpenStack, LLC. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +# implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +""" In-Memory Object Server for Swift """ + +from __future__ import with_statement +import os +from swift import gettext_ as _ + +from eventlet import Timeout + +from swift.common.bufferedhttp import http_connect +from swift.common.exceptions import ConnectionTimeout + +from swift.common.http import is_success +from swift.obj.mem_diskfile import InMemoryFileSystem +from swift.obj import server + + +class ObjectController(server.ObjectController): + """ + Implements the WSGI application for the Swift In-Memory Object Server. + """ + + def setup(self, conf): + """ + Nothing specific to do for the in-memory version. + + :param conf: WSGI configuration parameter + """ + self._filesystem = InMemoryFileSystem() + + def get_diskfile(self, device, partition, account, container, obj, + **kwargs): + """ + Utility method for instantiating a DiskFile object supporting a given + REST API. + + An implementation of the object server that wants to use a different + DiskFile class would simply over-ride this method to provide that + behavior. + """ + return self._filesystem.get_diskfile(account, container, obj, **kwargs) + + def async_update(self, op, account, container, obj, host, partition, + contdevice, headers_out, objdevice): + """ + Sends or saves an async update. + + :param op: operation performed (ex: 'PUT', or 'DELETE') + :param account: account name for the object + :param container: container name for the object + :param obj: object name + :param host: host that the container is on + :param partition: partition that the container is on + :param contdevice: device name that the container is on + :param headers_out: dictionary of headers to send in the container + request + :param objdevice: device name that the object is in + """ + headers_out['user-agent'] = 'obj-server %s' % os.getpid() + full_path = '/%s/%s/%s' % (account, container, obj) + if all([host, partition, contdevice]): + try: + with ConnectionTimeout(self.conn_timeout): + ip, port = host.rsplit(':', 1) + conn = http_connect(ip, port, contdevice, partition, op, + full_path, headers_out) + with Timeout(self.node_timeout): + response = conn.getresponse() + response.read() + if is_success(response.status): + return + else: + self.logger.error(_( + 'ERROR Container update failed: %(status)d ' + 'response from %(ip)s:%(port)s/%(dev)s'), + {'status': response.status, 'ip': ip, 'port': port, + 'dev': contdevice}) + except (Exception, Timeout): + self.logger.exception(_( + 'ERROR container update failed with ' + '%(ip)s:%(port)s/%(dev)s'), + {'ip': ip, 'port': port, 'dev': contdevice}) + # FIXME: For now don't handle async updates + + def REPLICATE(self, request): + """ + Handle REPLICATE requests for the Swift Object Server. This is used + by the object replicator to get hashes for directories. + """ + pass + + +def app_factory(global_conf, **local_conf): + """paste.deploy app factory for creating WSGI object server apps""" + conf = global_conf.copy() + conf.update(local_conf) + return ObjectController(conf) diff --git a/swift/obj/server.py b/swift/obj/server.py index ab70ff21cf..e8593fedab 100644 --- a/swift/obj/server.py +++ b/swift/obj/server.py @@ -20,21 +20,19 @@ import cPickle as pickle import os import time import traceback -from collections import defaultdict from datetime import datetime from swift import gettext_ as _ from hashlib import md5 from eventlet import sleep, Timeout -from swift.common.utils import mkdirs, normalize_timestamp, public, \ - hash_path, get_logger, write_pickle, config_true_value, timing_stats, \ - ThreadPool, replication +from swift.common.utils import public, get_logger, \ + config_true_value, timing_stats, replication from swift.common.bufferedhttp import http_connect -from swift.common.constraints import check_object_creation, check_mount, \ +from swift.common.constraints import check_object_creation, \ check_float, check_utf8 -from swift.common.exceptions import ConnectionTimeout, DiskFileError, \ - DiskFileNotExist, DiskFileCollision, DiskFileNoSpace, \ +from swift.common.exceptions import ConnectionTimeout, DiskFileQuarantined, \ + DiskFileNotExist, DiskFileCollision, DiskFileNoSpace, DiskFileDeleted, \ DiskFileDeviceUnavailable from swift.common.http import is_success from swift.common.request_helpers import split_and_validate_path @@ -44,13 +42,7 @@ from swift.common.swob import HTTPAccepted, HTTPBadRequest, HTTPCreated, \ HTTPClientDisconnect, HTTPMethodNotAllowed, Request, Response, UTC, \ HTTPInsufficientStorage, HTTPForbidden, HTTPException, HeaderKeyDict, \ HTTPConflict -from swift.obj.diskfile import DATAFILE_SYSTEM_META, DiskFile, \ - get_hashes - - -DATADIR = 'objects' -ASYNCDIR = 'async_pending' -MAX_OBJECT_NAME_LENGTH = 1024 +from swift.obj.diskfile import DATAFILE_SYSTEM_META, DiskFileManager class ObjectController(object): @@ -64,26 +56,19 @@ class ObjectController(object): /etc/swift/object-server.conf-sample. """ self.logger = get_logger(conf, log_route='object-server') - self.devices = conf.get('devices', '/srv/node/') - self.mount_check = config_true_value(conf.get('mount_check', 'true')) self.node_timeout = int(conf.get('node_timeout', 3)) self.conn_timeout = float(conf.get('conn_timeout', 0.5)) - self.disk_chunk_size = int(conf.get('disk_chunk_size', 65536)) self.network_chunk_size = int(conf.get('network_chunk_size', 65536)) - self.keep_cache_size = int(conf.get('keep_cache_size', 5242880)) - self.keep_cache_private = \ - config_true_value(conf.get('keep_cache_private', 'false')) self.log_requests = config_true_value(conf.get('log_requests', 'true')) self.max_upload_time = int(conf.get('max_upload_time', 86400)) self.slow = int(conf.get('slow', 0)) - self.bytes_per_sync = int(conf.get('mb_per_sync', 512)) * 1024 * 1024 + self.keep_cache_private = \ + config_true_value(conf.get('keep_cache_private', 'false')) replication_server = conf.get('replication_server', None) if replication_server is not None: replication_server = config_true_value(replication_server) self.replication_server = replication_server - self.threads_per_disk = int(conf.get('threads_per_disk', '0')) - self.threadpools = defaultdict( - lambda: ThreadPool(nthreads=self.threads_per_disk)) + default_allowed_headers = ''' content-disposition, content-encoding, @@ -106,15 +91,34 @@ class ObjectController(object): self.expiring_objects_container_divisor = \ int(conf.get('expiring_objects_container_divisor') or 86400) - def _diskfile(self, device, partition, account, container, obj, **kwargs): - """Utility method for instantiating a DiskFile.""" - kwargs.setdefault('mount_check', self.mount_check) - kwargs.setdefault('bytes_per_sync', self.bytes_per_sync) - kwargs.setdefault('disk_chunk_size', self.disk_chunk_size) - kwargs.setdefault('threadpool', self.threadpools[device]) - kwargs.setdefault('obj_dir', DATADIR) - return DiskFile(self.devices, device, partition, account, - container, obj, self.logger, **kwargs) + # Provide further setup sepecific to an object server implemenation. + self.setup(conf) + + def setup(self, conf): + """ + Implementation specific setup. This method is called at the very end + by the constructor to allow a specific implementation to modify + existing attributes or add its own attributes. + + :param conf: WSGI configuration parameter + """ + + # Common on-disk hierarchy shared across account, container and object + # servers. + self._diskfile_mgr = DiskFileManager(conf, self.logger) + + def get_diskfile(self, device, partition, account, container, obj, + **kwargs): + """ + Utility method for instantiating a DiskFile object supporting a given + REST API. + + An implementation of the object server that wants to use a different + DiskFile class would simply over-ride this method to provide that + behavior. + """ + return self._diskfile_mgr.get_diskfile( + device, partition, account, container, obj, **kwargs) def async_update(self, op, account, container, obj, host, partition, contdevice, headers_out, objdevice): @@ -157,16 +161,11 @@ class ObjectController(object): 'ERROR container update failed with ' '%(ip)s:%(port)s/%(dev)s (saving for async update later)'), {'ip': ip, 'port': port, 'dev': contdevice}) - async_dir = os.path.join(self.devices, objdevice, ASYNCDIR) - ohash = hash_path(account, container, obj) - self.logger.increment('async_pendings') - self.threadpools[objdevice].run_in_thread( - write_pickle, - {'op': op, 'account': account, 'container': container, - 'obj': obj, 'headers': headers_out}, - os.path.join(async_dir, ohash[-3:], ohash + '-' + - normalize_timestamp(headers_out['x-timestamp'])), - os.path.join(self.devices, objdevice, 'tmp')) + data = {'op': op, 'account': account, 'container': container, + 'obj': obj, 'headers': headers_out} + timestamp = headers_out['x-timestamp'] + self._diskfile_mgr.pickle_async_update(objdevice, account, container, + obj, data, timestamp) def container_update(self, op, account, container, obj, request, headers_out, objdevice): @@ -295,19 +294,14 @@ class ObjectController(object): return HTTPBadRequest(body='X-Delete-At in past', request=request, content_type='text/plain') try: - disk_file = self._diskfile(device, partition, account, container, - obj) + disk_file = self.get_diskfile( + device, partition, account, container, obj) except DiskFileDeviceUnavailable: return HTTPInsufficientStorage(drive=device, request=request) - with disk_file.open(): - if disk_file.is_deleted() or disk_file.is_expired(): - return HTTPNotFound(request=request) - try: - disk_file.get_data_file_size() - except (DiskFileError, DiskFileNotExist): - disk_file.quarantine() - return HTTPNotFound(request=request) - orig_metadata = disk_file.get_metadata() + try: + orig_metadata = disk_file.read_metadata() + except (DiskFileNotExist, DiskFileQuarantined): + return HTTPNotFound(request=request) orig_timestamp = orig_metadata.get('X-Timestamp', '0') if orig_timestamp >= request.headers['x-timestamp']: return HTTPConflict(request=request) @@ -318,16 +312,20 @@ class ObjectController(object): if header_key in request.headers: header_caps = header_key.title() metadata[header_caps] = request.headers[header_key] - old_delete_at = int(orig_metadata.get('X-Delete-At') or 0) - if old_delete_at != new_delete_at: + orig_delete_at = int(orig_metadata.get('X-Delete-At') or 0) + if orig_delete_at != new_delete_at: if new_delete_at: self.delete_at_update('PUT', new_delete_at, account, container, obj, request, device) - if old_delete_at: - self.delete_at_update('DELETE', old_delete_at, account, + if orig_delete_at: + self.delete_at_update('DELETE', orig_delete_at, account, container, obj, request, device) - disk_file.put_metadata(metadata) - return HTTPAccepted(request=request) + try: + disk_file.write_metadata(metadata) + except (DiskFileNotExist, DiskFileQuarantined): + return HTTPNotFound(request=request) + else: + return HTTPAccepted(request=request) @public @timing_stats() @@ -353,21 +351,24 @@ class ObjectController(object): return HTTPBadRequest(body=str(e), request=request, content_type='text/plain') try: - disk_file = self._diskfile(device, partition, account, container, - obj) + disk_file = self.get_diskfile( + device, partition, account, container, obj) except DiskFileDeviceUnavailable: return HTTPInsufficientStorage(drive=device, request=request) - with disk_file.open(): - orig_metadata = disk_file.get_metadata() - old_delete_at = int(orig_metadata.get('X-Delete-At') or 0) + try: + orig_metadata = disk_file.read_metadata() + except (DiskFileNotExist, DiskFileQuarantined): + orig_metadata = {} orig_timestamp = orig_metadata.get('X-Timestamp') if orig_timestamp and orig_timestamp >= request.headers['x-timestamp']: return HTTPConflict(request=request) + orig_delete_at = int(orig_metadata.get('X-Delete-At') or 0) upload_expiration = time.time() + self.max_upload_time etag = md5() elapsed_time = 0 try: with disk_file.create(size=fsize) as writer: + upload_size = 0 reader = request.environ['wsgi.input'].read for chunk in iter(lambda: reader(self.network_chunk_size), ''): start_time = time.time() @@ -375,10 +376,9 @@ class ObjectController(object): self.logger.increment('PUT.timeouts') return HTTPRequestTimeout(request=request) etag.update(chunk) - writer.write(chunk) + upload_size = writer.write(chunk) sleep() elapsed_time += time.time() - start_time - upload_size = writer.upload_size if upload_size: self.logger.transfer_rate( 'PUT.' + device + '.timing', elapsed_time, @@ -405,14 +405,14 @@ class ObjectController(object): writer.put(metadata) except DiskFileNoSpace: return HTTPInsufficientStorage(drive=device, request=request) - if old_delete_at != new_delete_at: + if orig_delete_at != new_delete_at: if new_delete_at: self.delete_at_update( 'PUT', new_delete_at, account, container, obj, request, device) - if old_delete_at: + if orig_delete_at: self.delete_at_update( - 'DELETE', old_delete_at, account, container, obj, + 'DELETE', orig_delete_at, account, container, obj, request, device) if not orig_timestamp or \ orig_timestamp < request.headers['x-timestamp']: @@ -424,8 +424,7 @@ class ObjectController(object): 'x-timestamp': metadata['X-Timestamp'], 'x-etag': metadata['ETag']}), device) - resp = HTTPCreated(request=request, etag=etag) - return resp + return HTTPCreated(request=request, etag=etag) @public @timing_stats() @@ -433,75 +432,74 @@ class ObjectController(object): """Handle HTTP GET requests for the Swift Object Server.""" device, partition, account, container, obj = \ split_and_validate_path(request, 5, 5, True) + keep_cache = self.keep_cache_private or ( + 'X-Auth-Token' not in request.headers and + 'X-Storage-Token' not in request.headers) try: - disk_file = self._diskfile(device, partition, account, container, - obj, iter_hook=sleep) + disk_file = self.get_diskfile( + device, partition, account, container, obj) except DiskFileDeviceUnavailable: return HTTPInsufficientStorage(drive=device, request=request) - disk_file.open() - if disk_file.is_deleted() or disk_file.is_expired(): + try: + with disk_file.open(): + metadata = disk_file.get_metadata() + obj_size = int(metadata['Content-Length']) + if request.headers.get('if-match') not in (None, '*') and \ + metadata['ETag'] not in request.if_match: + return HTTPPreconditionFailed(request=request) + if request.headers.get('if-none-match') is not None: + if metadata['ETag'] in request.if_none_match: + resp = HTTPNotModified(request=request) + resp.etag = metadata['ETag'] + return resp + file_x_ts = metadata['X-Timestamp'] + file_x_ts_flt = float(file_x_ts) + try: + if_unmodified_since = request.if_unmodified_since + except (OverflowError, ValueError): + # catches timestamps before the epoch + return HTTPPreconditionFailed(request=request) + file_x_ts_utc = datetime.fromtimestamp(file_x_ts_flt, UTC) + if if_unmodified_since and file_x_ts_utc > if_unmodified_since: + return HTTPPreconditionFailed(request=request) + try: + if_modified_since = request.if_modified_since + except (OverflowError, ValueError): + # catches timestamps before the epoch + return HTTPPreconditionFailed(request=request) + if if_modified_since and file_x_ts_utc < if_modified_since: + return HTTPNotModified(request=request) + keep_cache = (self.keep_cache_private or + ('X-Auth-Token' not in request.headers and + 'X-Storage-Token' not in request.headers)) + response = Response( + app_iter=disk_file.reader(iter_hook=sleep, + keep_cache=keep_cache), + request=request, conditional_response=True) + response.headers['Content-Type'] = metadata.get( + 'Content-Type', 'application/octet-stream') + for key, value in metadata.iteritems(): + if key.lower().startswith('x-object-meta-') or \ + key.lower() in self.allowed_headers: + response.headers[key] = value + response.etag = metadata['ETag'] + response.last_modified = file_x_ts_flt + response.content_length = obj_size + try: + response.content_encoding = metadata[ + 'Content-Encoding'] + except KeyError: + pass + response.headers['X-Timestamp'] = file_x_ts + resp = request.get_response(response) + except DiskFileNotExist: if request.headers.get('if-match') == '*': - return HTTPPreconditionFailed(request=request) + resp = HTTPPreconditionFailed(request=request) else: - return HTTPNotFound(request=request) - try: - file_size = disk_file.get_data_file_size() - except (DiskFileError, DiskFileNotExist): - disk_file.quarantine() - return HTTPNotFound(request=request) - metadata = disk_file.get_metadata() - if request.headers.get('if-match') not in (None, '*') and \ - metadata['ETag'] not in request.if_match: - disk_file.close() - return HTTPPreconditionFailed(request=request) - if request.headers.get('if-none-match') is not None: - if metadata['ETag'] in request.if_none_match: - resp = HTTPNotModified(request=request) - resp.etag = metadata['ETag'] - disk_file.close() - return resp - try: - if_unmodified_since = request.if_unmodified_since - except (OverflowError, ValueError): - # catches timestamps before the epoch - return HTTPPreconditionFailed(request=request) - if if_unmodified_since and \ - datetime.fromtimestamp( - float(metadata['X-Timestamp']), UTC) > \ - if_unmodified_since: - disk_file.close() - return HTTPPreconditionFailed(request=request) - try: - if_modified_since = request.if_modified_since - except (OverflowError, ValueError): - # catches timestamps before the epoch - return HTTPPreconditionFailed(request=request) - if if_modified_since and \ - datetime.fromtimestamp( - float(metadata['X-Timestamp']), UTC) < \ - if_modified_since: - disk_file.close() - return HTTPNotModified(request=request) - response = Response(app_iter=disk_file, - request=request, conditional_response=True) - response.headers['Content-Type'] = metadata.get( - 'Content-Type', 'application/octet-stream') - for key, value in metadata.iteritems(): - if key.lower().startswith('x-object-meta-') or \ - key.lower() in self.allowed_headers: - response.headers[key] = value - response.etag = metadata['ETag'] - response.last_modified = float(metadata['X-Timestamp']) - response.content_length = file_size - if response.content_length < self.keep_cache_size and \ - (self.keep_cache_private or - ('X-Auth-Token' not in request.headers and - 'X-Storage-Token' not in request.headers)): - disk_file.keep_cache = True - if 'Content-Encoding' in metadata: - response.content_encoding = metadata['Content-Encoding'] - response.headers['X-Timestamp'] = metadata['X-Timestamp'] - return request.get_response(response) + resp = HTTPNotFound(request=request) + except DiskFileQuarantined: + resp = HTTPNotFound(request=request) + return resp @public @timing_stats(sample_rate=0.8) @@ -510,19 +508,14 @@ class ObjectController(object): device, partition, account, container, obj = \ split_and_validate_path(request, 5, 5, True) try: - disk_file = self._diskfile(device, partition, account, container, - obj) + disk_file = self.get_diskfile( + device, partition, account, container, obj) except DiskFileDeviceUnavailable: return HTTPInsufficientStorage(drive=device, request=request) - with disk_file.open(): - if disk_file.is_deleted() or disk_file.is_expired(): - return HTTPNotFound(request=request) - try: - file_size = disk_file.get_data_file_size() - except (DiskFileError, DiskFileNotExist): - disk_file.quarantine() - return HTTPNotFound(request=request) - metadata = disk_file.get_metadata() + try: + metadata = disk_file.read_metadata() + except (DiskFileNotExist, DiskFileQuarantined): + return HTTPNotFound(request=request) response = Response(request=request, conditional_response=True) response.headers['Content-Type'] = metadata.get( 'Content-Type', 'application/octet-stream') @@ -531,12 +524,15 @@ class ObjectController(object): key.lower() in self.allowed_headers: response.headers[key] = value response.etag = metadata['ETag'] - response.last_modified = float(metadata['X-Timestamp']) + ts = metadata['X-Timestamp'] + response.last_modified = float(ts) # Needed for container sync feature - response.headers['X-Timestamp'] = metadata['X-Timestamp'] - response.content_length = file_size - if 'Content-Encoding' in metadata: + response.headers['X-Timestamp'] = ts + response.content_length = int(metadata['Content-Length']) + try: response.content_encoding = metadata['Content-Encoding'] + except KeyError: + pass return response @public @@ -550,41 +546,44 @@ class ObjectController(object): return HTTPBadRequest(body='Missing timestamp', request=request, content_type='text/plain') try: - disk_file = self._diskfile(device, partition, account, container, - obj) + disk_file = self.get_diskfile( + device, partition, account, container, obj) except DiskFileDeviceUnavailable: return HTTPInsufficientStorage(drive=device, request=request) - with disk_file.open(): - orig_metadata = disk_file.get_metadata() - is_deleted = disk_file.is_deleted() - is_expired = disk_file.is_expired() + try: + orig_metadata = disk_file.read_metadata() + except DiskFileDeleted as e: + orig_timestamp = e.timestamp + orig_metadata = {} + response_class = HTTPNotFound + except (DiskFileNotExist, DiskFileQuarantined): + orig_timestamp = 0 + orig_metadata = {} + response_class = HTTPNotFound + else: + orig_timestamp = orig_metadata.get('X-Timestamp', 0) + if orig_timestamp < request.headers['x-timestamp']: + response_class = HTTPNoContent + else: + response_class = HTTPConflict if 'x-if-delete-at' in request.headers and \ int(request.headers['x-if-delete-at']) != \ int(orig_metadata.get('X-Delete-At') or 0): return HTTPPreconditionFailed( request=request, body='X-If-Delete-At and X-Delete-At do not match') - old_delete_at = int(orig_metadata.get('X-Delete-At') or 0) - if old_delete_at: - self.delete_at_update('DELETE', old_delete_at, account, + orig_delete_at = int(orig_metadata.get('X-Delete-At') or 0) + if orig_delete_at: + self.delete_at_update('DELETE', orig_delete_at, account, container, obj, request, device) - orig_timestamp = orig_metadata.get('X-Timestamp', 0) req_timestamp = request.headers['X-Timestamp'] - if is_deleted or is_expired: - response_class = HTTPNotFound - else: - if orig_timestamp < req_timestamp: - response_class = HTTPNoContent - else: - response_class = HTTPConflict if orig_timestamp < req_timestamp: disk_file.delete(req_timestamp) self.container_update( 'DELETE', account, container, obj, request, HeaderKeyDict({'x-timestamp': req_timestamp}), device) - resp = response_class(request=request) - return resp + return response_class(request=request) @public @replication @@ -596,16 +595,13 @@ class ObjectController(object): """ device, partition, suffix = split_and_validate_path( request, 2, 3, True) - - if self.mount_check and not check_mount(self.devices, device): - return HTTPInsufficientStorage(drive=device, request=request) - path = os.path.join(self.devices, device, DATADIR, partition) - if not os.path.exists(path): - mkdirs(path) - suffixes = suffix.split('-') if suffix else [] - _junk, hashes = self.threadpools[device].force_run_in_thread( - get_hashes, path, recalculate=suffixes) - return Response(body=pickle.dumps(hashes)) + try: + hashes = self._diskfile_mgr.get_hashes(device, partition, suffix) + except DiskFileDeviceUnavailable: + resp = HTTPInsufficientStorage(drive=device, request=request) + else: + resp = Response(body=pickle.dumps(hashes)) + return resp def __call__(self, env, start_response): """WSGI Application entry point for the Swift Object Server.""" diff --git a/swift/obj/updater.py b/swift/obj/updater.py index f519cd341f..af4eab4d6d 100644 --- a/swift/obj/updater.py +++ b/swift/obj/updater.py @@ -29,7 +29,7 @@ from swift.common.ring import Ring from swift.common.utils import get_logger, renamer, write_pickle, \ dump_recon_cache, config_true_value from swift.common.daemon import Daemon -from swift.obj.server import ASYNCDIR +from swift.obj.diskfile import ASYNCDIR from swift.common.http import is_success, HTTP_NOT_FOUND, \ HTTP_INTERNAL_SERVER_ERROR diff --git a/test/unit/account/test_auditor.py b/test/unit/account/test_auditor.py index f7abbed47c..83ca833447 100644 --- a/test/unit/account/test_auditor.py +++ b/test/unit/account/test_auditor.py @@ -85,9 +85,9 @@ class TestAuditor(unittest.TestCase): files = os.listdir(self.testdir) return [(os.path.join(self.testdir, f), '', '') for f in files] - auditor.audit_location_generator = fake_audit_location_generator - - self.assertRaises(ValueError, test_auditor.run_forever) + with mock.patch('swift.account.auditor.audit_location_generator', + fake_audit_location_generator): + self.assertRaises(ValueError, test_auditor.run_forever) self.assertEqual(test_auditor.account_failures, 2 * call_times) self.assertEqual(test_auditor.account_passes, 3 * call_times) @@ -100,9 +100,9 @@ class TestAuditor(unittest.TestCase): files = os.listdir(self.testdir) return [(os.path.join(self.testdir, f), '', '') for f in files] - auditor.audit_location_generator = fake_audit_location_generator - - test_auditor.run_once() + with mock.patch('swift.account.auditor.audit_location_generator', + fake_audit_location_generator): + test_auditor.run_once() self.assertEqual(test_auditor.account_failures, 2) self.assertEqual(test_auditor.account_passes, 3) diff --git a/test/unit/container/test_auditor.py b/test/unit/container/test_auditor.py index 18f832d933..5173141bfa 100644 --- a/test/unit/container/test_auditor.py +++ b/test/unit/container/test_auditor.py @@ -85,9 +85,9 @@ class TestAuditor(unittest.TestCase): files = os.listdir(self.testdir) return [(os.path.join(self.testdir, f), '', '') for f in files] - auditor.audit_location_generator = fake_audit_location_generator - - self.assertRaises(ValueError, test_auditor.run_forever) + with mock.patch('swift.container.auditor.audit_location_generator', + fake_audit_location_generator): + self.assertRaises(ValueError, test_auditor.run_forever) self.assertEquals(test_auditor.container_failures, 2 * call_times) self.assertEquals(test_auditor.container_passes, 3 * call_times) @@ -100,9 +100,9 @@ class TestAuditor(unittest.TestCase): files = os.listdir(self.testdir) return [(os.path.join(self.testdir, f), '', '') for f in files] - auditor.audit_location_generator = fake_audit_location_generator - - test_auditor.run_once() + with mock.patch('swift.container.auditor.audit_location_generator', + fake_audit_location_generator): + test_auditor.run_once() self.assertEquals(test_auditor.container_failures, 2) self.assertEquals(test_auditor.container_passes, 3) diff --git a/test/unit/container/test_sync.py b/test/unit/container/test_sync.py index f2fc294ffb..f4862c8cbd 100644 --- a/test/unit/container/test_sync.py +++ b/test/unit/container/test_sync.py @@ -134,16 +134,16 @@ class TestContainerSync(unittest.TestCase): orig_time = sync.time orig_sleep = sync.sleep - orig_audit_location_generator = sync.audit_location_generator orig_ContainerBroker = sync.ContainerBroker + orig_audit_location_generator = sync.audit_location_generator try: sync.ContainerBroker = lambda p: FakeContainerBroker( p, info={'account': 'a', 'container': 'c'}) sync.time = fake_time sync.sleep = fake_sleep - sync.audit_location_generator = fake_audit_location_generator cs = sync.ContainerSync({}, container_ring=FakeRing(), object_ring=FakeRing()) + sync.audit_location_generator = fake_audit_location_generator cs.run_forever() except Exception as err: if str(err) != 'we are now done': @@ -196,9 +196,9 @@ class TestContainerSync(unittest.TestCase): sync.ContainerBroker = lambda p: FakeContainerBroker( p, info={'account': 'a', 'container': 'c'}) sync.time = fake_time - sync.audit_location_generator = fake_audit_location_generator cs = sync.ContainerSync({}, container_ring=FakeRing(), object_ring=FakeRing()) + sync.audit_location_generator = fake_audit_location_generator cs.run_once() self.assertEquals(time_calls, [6]) self.assertEquals(audit_location_generator_calls, [1]) diff --git a/test/unit/obj/test_auditor.py b/test/unit/obj/test_auditor.py index 23ade8fe8a..115ffe7658 100644 --- a/test/unit/obj/test_auditor.py +++ b/test/unit/obj/test_auditor.py @@ -23,8 +23,8 @@ from hashlib import md5 from tempfile import mkdtemp from test.unit import FakeLogger from swift.obj import auditor -from swift.obj.diskfile import DiskFile, write_metadata, invalidate_hash -from swift.obj.server import DATADIR +from swift.obj.diskfile import DiskFile, write_metadata, invalidate_hash, \ + DATADIR, DiskFileManager from swift.common.utils import hash_path, mkdirs, normalize_timestamp, \ storage_directory @@ -52,8 +52,8 @@ class TestAuditor(unittest.TestCase): devices=self.devices, mount_check='false', object_size_stats='10,100,1024,10240') - self.disk_file = DiskFile(self.devices, 'sda', '0', 'a', 'c', 'o', - self.logger) + self.df_mgr = DiskFileManager(self.conf, self.logger) + self.disk_file = self.df_mgr.get_diskfile('sda', '0', 'a', 'c', 'o') def tearDown(self): rmtree(os.path.dirname(self.testdir), ignore_errors=1) @@ -71,19 +71,19 @@ class TestAuditor(unittest.TestCase): metadata = { 'ETag': etag, 'X-Timestamp': timestamp, - 'Content-Length': str(os.fstat(writer.fd).st_size), + 'Content-Length': str(os.fstat(writer._fd).st_size), } writer.put(metadata) pre_quarantines = auditor_worker.quarantines auditor_worker.object_audit( - os.path.join(self.disk_file.datadir, timestamp + '.data'), + os.path.join(self.disk_file._datadir, timestamp + '.data'), 'sda', '0') self.assertEquals(auditor_worker.quarantines, pre_quarantines) - os.write(writer.fd, 'extra_data') + os.write(writer._fd, 'extra_data') auditor_worker.object_audit( - os.path.join(self.disk_file.datadir, timestamp + '.data'), + os.path.join(self.disk_file._datadir, timestamp + '.data'), 'sda', '0') self.assertEquals(auditor_worker.quarantines, pre_quarantines + 1) @@ -99,17 +99,16 @@ class TestAuditor(unittest.TestCase): metadata = { 'ETag': etag, 'X-Timestamp': timestamp, - 'Content-Length': str(os.fstat(writer.fd).st_size), + 'Content-Length': str(os.fstat(writer._fd).st_size), } writer.put(metadata) pre_quarantines = auditor_worker.quarantines # remake so it will have metadata - self.disk_file = DiskFile(self.devices, 'sda', '0', 'a', 'c', 'o', - self.logger) + self.disk_file = self.df_mgr.get_diskfile('sda', '0', 'a', 'c', 'o') auditor_worker.object_audit( - os.path.join(self.disk_file.datadir, timestamp + '.data'), + os.path.join(self.disk_file._datadir, timestamp + '.data'), 'sda', '0') self.assertEquals(auditor_worker.quarantines, pre_quarantines) etag = md5() @@ -122,29 +121,29 @@ class TestAuditor(unittest.TestCase): writer.put(metadata) auditor_worker.object_audit( - os.path.join(self.disk_file.datadir, timestamp + '.data'), + os.path.join(self.disk_file._datadir, timestamp + '.data'), 'sda', '0') self.assertEquals(auditor_worker.quarantines, pre_quarantines + 1) def test_object_audit_no_meta(self): timestamp = str(normalize_timestamp(time.time())) - path = os.path.join(self.disk_file.datadir, timestamp + '.data') - mkdirs(self.disk_file.datadir) + path = os.path.join(self.disk_file._datadir, timestamp + '.data') + mkdirs(self.disk_file._datadir) fp = open(path, 'w') fp.write('0' * 1024) fp.close() - invalidate_hash(os.path.dirname(self.disk_file.datadir)) + invalidate_hash(os.path.dirname(self.disk_file._datadir)) auditor_worker = auditor.AuditorWorker(self.conf, self.logger) pre_quarantines = auditor_worker.quarantines auditor_worker.object_audit( - os.path.join(self.disk_file.datadir, timestamp + '.data'), + os.path.join(self.disk_file._datadir, timestamp + '.data'), 'sda', '0') self.assertEquals(auditor_worker.quarantines, pre_quarantines + 1) def test_object_audit_will_not_swallow_errors_in_tests(self): timestamp = str(normalize_timestamp(time.time())) - path = os.path.join(self.disk_file.datadir, timestamp + '.data') - mkdirs(self.disk_file.datadir) + path = os.path.join(self.disk_file._datadir, timestamp + '.data') + mkdirs(self.disk_file._datadir) with open(path, 'w') as f: write_metadata(f, {'name': '/a/c/o'}) auditor_worker = auditor.AuditorWorker(self.conf, self.logger) @@ -158,8 +157,8 @@ class TestAuditor(unittest.TestCase): def test_failsafe_object_audit_will_swallow_errors_in_tests(self): timestamp = str(normalize_timestamp(time.time())) - path = os.path.join(self.disk_file.datadir, timestamp + '.data') - mkdirs(self.disk_file.datadir) + path = os.path.join(self.disk_file._datadir, timestamp + '.data') + mkdirs(self.disk_file._datadir) with open(path, 'w') as f: write_metadata(f, {'name': '/a/c/o'}) auditor_worker = auditor.AuditorWorker(self.conf, self.logger) @@ -184,7 +183,7 @@ class TestAuditor(unittest.TestCase): metadata = { 'ETag': etag, 'X-Timestamp': timestamp, - 'Content-Length': str(os.fstat(writer.fd).st_size), + 'Content-Length': str(os.fstat(writer._fd).st_size), } writer.put(metadata) with mock.patch('swift.obj.diskfile.DiskFile', @@ -206,7 +205,7 @@ class TestAuditor(unittest.TestCase): metadata = { 'ETag': etag, 'X-Timestamp': timestamp, - 'Content-Length': str(os.fstat(writer.fd).st_size), + 'Content-Length': str(os.fstat(writer._fd).st_size), } writer.put(metadata) auditor_worker.audit_all_objects() @@ -227,10 +226,10 @@ class TestAuditor(unittest.TestCase): metadata = { 'ETag': etag, 'X-Timestamp': timestamp, - 'Content-Length': str(os.fstat(writer.fd).st_size), + 'Content-Length': str(os.fstat(writer._fd).st_size), } writer.put(metadata) - os.write(writer.fd, 'extra_data') + os.write(writer._fd, 'extra_data') auditor_worker.audit_all_objects() self.assertEquals(auditor_worker.quarantines, pre_quarantines + 1) @@ -247,12 +246,11 @@ class TestAuditor(unittest.TestCase): metadata = { 'ETag': etag, 'X-Timestamp': timestamp, - 'Content-Length': str(os.fstat(writer.fd).st_size), + 'Content-Length': str(os.fstat(writer._fd).st_size), } writer.put(metadata) auditor_worker.audit_all_objects() - self.disk_file = DiskFile(self.devices, 'sdb', '0', 'a', 'c', - 'ob', self.logger) + self.disk_file = self.df_mgr.get_diskfile('sda', '0', 'a', 'c', 'ob') data = '1' * 10 etag = md5() with self.disk_file.create() as writer: @@ -262,10 +260,10 @@ class TestAuditor(unittest.TestCase): metadata = { 'ETag': etag, 'X-Timestamp': timestamp, - 'Content-Length': str(os.fstat(writer.fd).st_size), + 'Content-Length': str(os.fstat(writer._fd).st_size), } writer.put(metadata) - os.write(writer.fd, 'extra_data') + os.write(writer._fd, 'extra_data') auditor_worker.audit_all_objects() self.assertEquals(auditor_worker.quarantines, pre_quarantines + 1) @@ -281,14 +279,14 @@ class TestAuditor(unittest.TestCase): metadata = { 'ETag': etag, 'X-Timestamp': str(normalize_timestamp(time.time())), - 'Content-Length': str(os.fstat(writer.fd).st_size), + 'Content-Length': str(os.fstat(writer._fd).st_size), } writer.put(metadata) etag = md5() etag.update('1' + '0' * 1023) etag = etag.hexdigest() metadata['ETag'] = etag - write_metadata(writer.fd, metadata) + write_metadata(writer._fd, metadata) quarantine_path = os.path.join(self.devices, 'sda', 'quarantined', 'objects') @@ -302,7 +300,6 @@ class TestAuditor(unittest.TestCase): self.auditor.log_time = 0 ts_file_path = '' if with_ts: - name_hash = hash_path('a', 'c', 'o') dir_path = os.path.join( self.devices, 'sda', @@ -325,9 +322,7 @@ class TestAuditor(unittest.TestCase): etag = md5() etag = etag.hexdigest() metadata['ETag'] = etag - write_metadata(writer.fd, metadata) - if self.disk_file.data_file: - return self.disk_file.data_file + write_metadata(writer._fd, metadata) return ts_file_path def test_object_run_fast_track_all(self): @@ -344,6 +339,27 @@ class TestAuditor(unittest.TestCase): 'sda', 'quarantined', 'objects') self.assertTrue(os.path.isdir(quarantine_path)) + def test_object_run_fast_track_zero_check_closed(self): + rat = [False] + + class FakeFile(DiskFile): + + def _quarantine(self, data_file, msg): + rat[0] = True + DiskFile._quarantine(self, data_file, msg) + + self.setup_bad_zero_byte() + was_df = auditor.diskfile.DiskFile + try: + auditor.diskfile.DiskFile = FakeFile + self.auditor.run_once(zero_byte_fps=50) + quarantine_path = os.path.join(self.devices, + 'sda', 'quarantined', 'objects') + self.assertTrue(os.path.isdir(quarantine_path)) + self.assertTrue(rat[0]) + finally: + auditor.diskfile.DiskFile = was_df + def test_with_tombstone(self): ts_file_path = self.setup_bad_zero_byte(with_ts=True) self.auditor.run_once() @@ -359,26 +375,6 @@ class TestAuditor(unittest.TestCase): self.assert_(delta_t > 0.08) self.assert_(delta_t < 0.12) - def test_object_run_fast_track_zero_check_closed(self): - rat = [False] - - class FakeFile(DiskFile): - - def close(self, verify_file=True): - rat[0] = True - DiskFile.close(self, verify_file=verify_file) - self.setup_bad_zero_byte() - was_df = auditor.diskfile.DiskFile - try: - auditor.diskfile.DiskFile = FakeFile - self.auditor.run_once(zero_byte_fps=50) - quarantine_path = os.path.join(self.devices, - 'sda', 'quarantined', 'objects') - self.assertTrue(os.path.isdir(quarantine_path)) - self.assertTrue(rat[0]) - finally: - auditor.diskfile.DiskFile = was_df - def test_run_forever(self): class StopForever(Exception): diff --git a/test/unit/obj/test_diskfile.py b/test/unit/obj/test_diskfile.py index 3962870ec2..1990546795 100644 --- a/test/unit/obj/test_diskfile.py +++ b/test/unit/obj/test_diskfile.py @@ -35,11 +35,13 @@ from gzip import GzipFile from eventlet import tpool from test.unit import FakeLogger, mock as unit_mock from test.unit import _setxattr as setxattr + from swift.obj import diskfile from swift.common import utils from swift.common.utils import hash_path, mkdirs, normalize_timestamp from swift.common import ring -from swift.common.exceptions import DiskFileNotExist, DiskFileDeviceUnavailable +from swift.common.exceptions import DiskFileNotExist, DiskFileQuarantined, \ + DiskFileDeviceUnavailable, DiskFileDeleted, DiskFileNotOpen, DiskFileError def _create_test_ring(path): @@ -91,15 +93,15 @@ class TestDiskFileModuleMethods(unittest.TestCase): self.conf = dict( swift_dir=self.testdir, devices=self.devices, mount_check='false', timeout='300', stats_interval='1') + self.df_mgr = diskfile.DiskFileManager(self.conf, FakeLogger()) def tearDown(self): rmtree(self.testdir, ignore_errors=1) def test_hash_suffix_hash_dir_is_file_quarantine(self): - df = diskfile.DiskFile(self.devices, 'sda', '0', 'a', 'c', 'o', - FakeLogger()) - mkdirs(os.path.dirname(df.datadir)) - open(df.datadir, 'wb').close() + df = self.df_mgr.get_diskfile('sda', '0', 'a', 'c', 'o') + mkdirs(os.path.dirname(df._datadir)) + open(df._datadir, 'wb').close() ohash = hash_path('a', 'c', 'o') data_dir = ohash[-3:] whole_path_from = os.path.join(self.objects, '0', data_dir) @@ -118,11 +120,10 @@ class TestDiskFileModuleMethods(unittest.TestCase): self.assertTrue(called[0]) def test_hash_suffix_one_file(self): - df = diskfile.DiskFile(self.devices, 'sda', '0', 'a', 'c', 'o', - FakeLogger()) - mkdirs(df.datadir) + df = self.df_mgr.get_diskfile('sda', '0', 'a', 'c', 'o') + mkdirs(df._datadir) f = open( - os.path.join(df.datadir, + os.path.join(df._datadir, normalize_timestamp(time() - 100) + '.ts'), 'wb') f.write('1234567890') @@ -137,14 +138,13 @@ class TestDiskFileModuleMethods(unittest.TestCase): self.assertEquals(len(os.listdir(self.parts['0'])), 0) def test_hash_suffix_multi_file_one(self): - df = diskfile.DiskFile(self.devices, 'sda', '0', 'a', 'c', 'o', - FakeLogger()) - mkdirs(df.datadir) + df = self.df_mgr.get_diskfile('sda', '0', 'a', 'c', 'o') + mkdirs(df._datadir) for tdiff in [1, 50, 100, 500]: for suff in ['.meta', '.data', '.ts']: f = open( os.path.join( - df.datadir, + df._datadir, normalize_timestamp(int(time()) - tdiff) + suff), 'wb') f.write('1234567890') @@ -161,9 +161,8 @@ class TestDiskFileModuleMethods(unittest.TestCase): self.assertEquals(len(os.listdir(whole_hsh_path)), 1) def test_hash_suffix_multi_file_two(self): - df = diskfile.DiskFile(self.devices, 'sda', '0', 'a', 'c', 'o', - FakeLogger()) - mkdirs(df.datadir) + df = self.df_mgr.get_diskfile('sda', '0', 'a', 'c', 'o') + mkdirs(df._datadir) for tdiff in [1, 50, 100, 500]: suffs = ['.meta', '.data'] if tdiff > 50: @@ -171,7 +170,7 @@ class TestDiskFileModuleMethods(unittest.TestCase): for suff in suffs: f = open( os.path.join( - df.datadir, + df._datadir, normalize_timestamp(int(time()) - tdiff) + suff), 'wb') f.write('1234567890') @@ -194,9 +193,8 @@ class TestDiskFileModuleMethods(unittest.TestCase): fdata = fp.read() self.assertEquals(pickle.loads(fdata), pickle.loads(data)) - df = diskfile.DiskFile(self.devices, 'sda', '0', 'a', 'c', 'o', - FakeLogger()) - mkdirs(df.datadir) + df = self.df_mgr.get_diskfile('sda', '0', 'a', 'c', 'o') + mkdirs(df._datadir) ohash = hash_path('a', 'c', 'o') data_dir = ohash[-3:] whole_path_from = os.path.join(self.objects, '0', data_dir) @@ -215,11 +213,10 @@ class TestDiskFileModuleMethods(unittest.TestCase): assertFileData(hashes_file, check_pickle_data) def test_get_hashes(self): - df = diskfile.DiskFile(self.devices, 'sda', '0', 'a', 'c', 'o', - FakeLogger()) - mkdirs(df.datadir) + df = self.df_mgr.get_diskfile('sda', '0', 'a', 'c', 'o') + mkdirs(df._datadir) with open( - os.path.join(df.datadir, + os.path.join(df._datadir, normalize_timestamp(time()) + '.ts'), 'wb') as f: f.write('1234567890') @@ -235,9 +232,8 @@ class TestDiskFileModuleMethods(unittest.TestCase): self.assert_('a83' in hashes) def test_get_hashes_bad_dir(self): - df = diskfile.DiskFile(self.devices, 'sda', '0', 'a', 'c', 'o', - FakeLogger()) - mkdirs(df.datadir) + df = self.df_mgr.get_diskfile('sda', '0', 'a', 'c', 'o') + mkdirs(df._datadir) with open(os.path.join(self.objects, '0', 'bad'), 'wb') as f: f.write('1234567890') part = os.path.join(self.objects, '0') @@ -247,11 +243,10 @@ class TestDiskFileModuleMethods(unittest.TestCase): self.assert_('bad' not in hashes) def test_get_hashes_unmodified(self): - df = diskfile.DiskFile(self.devices, 'sda', '0', 'a', 'c', 'o', - FakeLogger()) - mkdirs(df.datadir) + df = self.df_mgr.get_diskfile('sda', '0', 'a', 'c', 'o') + mkdirs(df._datadir) with open( - os.path.join(df.datadir, + os.path.join(df._datadir, normalize_timestamp(time()) + '.ts'), 'wb') as f: f.write('1234567890') @@ -268,9 +263,8 @@ class TestDiskFileModuleMethods(unittest.TestCase): self.assertEquals(i[0], 2) def test_get_hashes_unmodified_and_zero_bytes(self): - df = diskfile.DiskFile(self.devices, 'sda', '0', 'a', 'c', 'o', - FakeLogger()) - mkdirs(df.datadir) + df = self.df_mgr.get_diskfile('sda', '0', 'a', 'c', 'o') + mkdirs(df._datadir) part = os.path.join(self.objects, '0') open(os.path.join(part, diskfile.HASH_FILE), 'w') # Now the hash file is zero bytes. @@ -290,11 +284,10 @@ class TestDiskFileModuleMethods(unittest.TestCase): self.assertTrue('a83' in hashes) def test_get_hashes_modified(self): - df = diskfile.DiskFile(self.devices, 'sda', '0', 'a', 'c', 'o', - FakeLogger()) - mkdirs(df.datadir) + df = self.df_mgr.get_diskfile('sda', '0', 'a', 'c', 'o') + mkdirs(df._datadir) with open( - os.path.join(df.datadir, + os.path.join(df._datadir, normalize_timestamp(time()) + '.ts'), 'wb') as f: f.write('1234567890') @@ -361,30 +354,43 @@ class TestDiskFile(unittest.TestCase): mkdirs(os.path.join(self.testdir, 'sda1', 'tmp')) self._orig_tpool_exc = tpool.execute tpool.execute = lambda f, *args, **kwargs: f(*args, **kwargs) + self.conf = dict(devices=self.testdir, mount_check='false', + keep_cache_size=2 * 1024) + self.df_mgr = diskfile.DiskFileManager(self.conf, FakeLogger()) def tearDown(self): """Tear down for testing swift.obj.diskfile""" rmtree(os.path.dirname(self.testdir)) tpool.execute = self._orig_tpool_exc - def _create_ondisk_file(self, df, data, timestamp, ext='.data'): - mkdirs(df.datadir) - timestamp = normalize_timestamp(timestamp) - data_file = os.path.join(df.datadir, timestamp + ext) - with open(data_file, 'wb') as f: - f.write(data) - md = {'X-Timestamp': timestamp} - setxattr(f.fileno(), diskfile.METADATA_KEY, - pickle.dumps(md, diskfile.PICKLE_PROTOCOL)) - - def _create_test_file(self, data, timestamp=None): - df = diskfile.DiskFile(self.testdir, 'sda1', '0', 'a', 'c', 'o', - FakeLogger()) + def _create_ondisk_file(self, df, data, timestamp, metadata=None, + ext='.data'): + mkdirs(df._datadir) if timestamp is None: timestamp = time() - self._create_ondisk_file(df, data, timestamp) - df = diskfile.DiskFile(self.testdir, 'sda1', '0', 'a', 'c', 'o', - FakeLogger()) + timestamp = normalize_timestamp(timestamp) + if not metadata: + metadata = {} + if 'X-Timestamp' not in metadata: + metadata['X-Timestamp'] = normalize_timestamp(timestamp) + if 'ETag' not in metadata: + etag = md5() + etag.update(data) + metadata['ETag'] = etag.hexdigest() + if 'name' not in metadata: + metadata['name'] = '/a/c/o' + if 'Content-Length' not in metadata: + metadata['Content-Length'] = str(len(data)) + data_file = os.path.join(df._datadir, timestamp + ext) + with open(data_file, 'wb') as f: + f.write(data) + setxattr(f.fileno(), diskfile.METADATA_KEY, + pickle.dumps(metadata, diskfile.PICKLE_PROTOCOL)) + + def _create_test_file(self, data, timestamp=None, metadata=None): + df = self.df_mgr.get_diskfile('sda', '0', 'a', 'c', 'o') + self._create_ondisk_file(df, data, timestamp, metadata) + df = self.df_mgr.get_diskfile('sda', '0', 'a', 'c', 'o') df.open() return df @@ -393,19 +399,32 @@ class TestDiskFile(unittest.TestCase): md = df.get_metadata() self.assertEquals(md['X-Timestamp'], normalize_timestamp(42)) + def test_get_metadata_not_opened(self): + df = self.df_mgr.get_diskfile('sda1', '0', 'a', 'c', 'o') + self.assertRaises(DiskFileNotOpen, df.get_metadata) + + def test_not_opened(self): + df = self.df_mgr.get_diskfile('sda1', '0', 'a', 'c', 'o') + try: + with df: + pass + except DiskFileNotOpen: + pass + else: + self.fail("Expected DiskFileNotOpen exception") + def test_disk_file_default_disallowed_metadata(self): # build an object with some meta (ts 41) orig_metadata = {'X-Object-Meta-Key1': 'Value1', 'Content-Type': 'text/garbage'} - df = self._get_disk_file(ts=41, extra_metadata=orig_metadata) + df = self._get_open_disk_file(ts=41, extra_metadata=orig_metadata) with df.open(): self.assertEquals('1024', df._metadata['Content-Length']) # write some new metadata (fast POST, don't send orig meta, ts 42) - df = diskfile.DiskFile(self.testdir, 'sda1', '0', 'a', 'c', 'o', - FakeLogger()) - df.put_metadata({'X-Timestamp': '42', 'X-Object-Meta-Key2': 'Value2'}) - df = diskfile.DiskFile(self.testdir, 'sda1', '0', 'a', 'c', 'o', - FakeLogger()) + df = self.df_mgr.get_diskfile('sda1', '0', 'a', 'c', 'o') + df.write_metadata({'X-Timestamp': normalize_timestamp(42), + 'X-Object-Meta-Key2': 'Value2'}) + df = self.df_mgr.get_diskfile('sda1', '0', 'a', 'c', 'o') with df.open(): # non-fast-post updateable keys are preserved self.assertEquals('text/garbage', df._metadata['Content-Type']) @@ -416,24 +435,30 @@ class TestDiskFile(unittest.TestCase): def test_disk_file_app_iter_corners(self): df = self._create_test_file('1234567890') - self.assertEquals(''.join(df.app_iter_range(0, None)), '1234567890') - - df = diskfile.DiskFile(self.testdir, 'sda1', '0', 'a', 'c', 'o', - FakeLogger()) + reader = df.reader() + self.assertEquals(''.join(reader.app_iter_range(0, None)), + '1234567890') + self.assertFalse(reader.was_quarantined) + df = self.df_mgr.get_diskfile('sda', '0', 'a', 'c', 'o') with df.open(): - self.assertEqual(''.join(df.app_iter_range(5, None)), '67890') + reader = df.reader() + self.assertEqual(''.join(reader.app_iter_range(5, None)), '67890') def test_disk_file_app_iter_partial_closes(self): df = self._create_test_file('1234567890') - with df.open(): - it = df.app_iter_range(0, 5) - self.assertEqual(''.join(it), '12345') - self.assertEqual(df.fp, None) + reader = df.reader() + it = reader.app_iter_range(0, 5) + self.assertFalse(reader.was_quarantined) + self.assertEqual(''.join(it), '12345') + self.assertTrue(reader._fp is None) def test_disk_file_app_iter_ranges(self): df = self._create_test_file('012345678911234567892123456789') - it = df.app_iter_ranges([(0, 10), (10, 20), (20, 30)], 'plain/text', - '\r\n--someheader\r\n', 30) + reader = df.reader() + it = reader.app_iter_ranges([(0, 10), (10, 20), (20, 30)], + 'plain/text', + '\r\n--someheader\r\n', 30) + self.assertFalse(reader.was_quarantined) value = ''.join(it) self.assert_('0123456789' in value) self.assert_('1123456789' in value) @@ -441,124 +466,83 @@ class TestDiskFile(unittest.TestCase): def test_disk_file_app_iter_ranges_edges(self): df = self._create_test_file('012345678911234567892123456789') - it = df.app_iter_ranges([(3, 10), (0, 2)], 'application/whatever', - '\r\n--someheader\r\n', 30) + reader = df.reader() + it = reader.app_iter_ranges([(3, 10), (0, 2)], 'application/whatever', + '\r\n--someheader\r\n', 30) value = ''.join(it) + self.assertFalse(reader.was_quarantined) self.assert_('3456789' in value) self.assert_('01' in value) def test_disk_file_large_app_iter_ranges(self): - """ - This test case is to make sure that the disk file app_iter_ranges - method all the paths being tested. - """ + # This test case is to make sure that the disk file app_iter_ranges + # method all the paths being tested. long_str = '01234567890' * 65536 target_strs = ['3456789', long_str[0:65590]] df = self._create_test_file(long_str) + reader = df.reader() + it = reader.app_iter_ranges([(3, 10), (0, 65590)], 'plain/text', + '5e816ff8b8b8e9a5d355497e5d9e0301', 655360) - it = df.app_iter_ranges([(3, 10), (0, 65590)], 'plain/text', - '5e816ff8b8b8e9a5d355497e5d9e0301', 655360) - - """ - the produced string actually missing the MIME headers - need to add these headers to make it as real MIME message. - The body of the message is produced by method app_iter_ranges - off of DiskFile object. - """ + # The produced string actually missing the MIME headers + # need to add these headers to make it as real MIME message. + # The body of the message is produced by method app_iter_ranges + # off of DiskFile object. header = ''.join(['Content-Type: multipart/byteranges;', 'boundary=', '5e816ff8b8b8e9a5d355497e5d9e0301\r\n']) value = header + ''.join(it) + self.assertFalse(reader.was_quarantined) parts = map(lambda p: p.get_payload(decode=True), email.message_from_string(value).walk())[1:3] self.assertEqual(parts, target_strs) def test_disk_file_app_iter_ranges_empty(self): - """ - This test case tests when empty value passed into app_iter_ranges - When ranges passed into the method is either empty array or None, - this method will yield empty string - """ + # This test case tests when empty value passed into app_iter_ranges + # When ranges passed into the method is either empty array or None, + # this method will yield empty string df = self._create_test_file('012345678911234567892123456789') - it = df.app_iter_ranges([], 'application/whatever', - '\r\n--someheader\r\n', 100) + reader = df.reader() + it = reader.app_iter_ranges([], 'application/whatever', + '\r\n--someheader\r\n', 100) self.assertEqual(''.join(it), '') - df = diskfile.DiskFile(self.testdir, 'sda1', '0', 'a', 'c', 'o', - FakeLogger()) + df = self.df_mgr.get_diskfile('sda', '0', 'a', 'c', 'o') with df.open(): - it = df.app_iter_ranges(None, 'app/something', - '\r\n--someheader\r\n', 150) + reader = df.reader() + it = reader.app_iter_ranges(None, 'app/something', + '\r\n--someheader\r\n', 150) + self.assertFalse(reader.was_quarantined) self.assertEqual(''.join(it), '') def test_disk_file_mkstemp_creates_dir(self): tmpdir = os.path.join(self.testdir, 'sda1', 'tmp') os.rmdir(tmpdir) - with diskfile.DiskFile(self.testdir, 'sda1', '0', 'a', 'c', - 'o', FakeLogger()).create(): + df = self.df_mgr.get_diskfile('sda1', '0', 'a', 'c', 'o') + with df.create(): self.assert_(os.path.exists(tmpdir)) - def test_iter_hook(self): - hook_call_count = [0] - - def hook(): - hook_call_count[0] += 1 - - df = self._get_disk_file(fsize=65, csize=8, iter_hook=hook) - with df.open(): - for _ in df: - pass - - self.assertEquals(hook_call_count[0], 9) - - def test_quarantine(self): - df = self._create_test_file('') # empty - df.quarantine() - quar_dir = os.path.join(self.testdir, 'sda1', 'quarantined', - 'objects', os.path.basename(os.path.dirname( - df.data_file))) - self.assert_(os.path.isdir(quar_dir)) - - def test_quarantine_same_file(self): - df = self._create_test_file('empty') - new_dir = df.quarantine() - quar_dir = os.path.join(self.testdir, 'sda1', 'quarantined', - 'objects', os.path.basename(os.path.dirname( - df.data_file))) - self.assert_(os.path.isdir(quar_dir)) - self.assertEquals(quar_dir, new_dir) - # have to remake the datadir and file - self._create_ondisk_file(df, '', time()) # still empty - df = diskfile.DiskFile(self.testdir, 'sda1', '0', 'a', 'c', 'o', - FakeLogger()) - df.open() - double_uuid_path = df.quarantine() - self.assert_(os.path.isdir(double_uuid_path)) - self.assert_('-' in os.path.basename(double_uuid_path)) - - def _get_disk_file(self, invalid_type=None, obj_name='o', - fsize=1024, csize=8, mark_deleted=False, ts=None, - iter_hook=None, mount_check=False, - extra_metadata=None): + def _get_open_disk_file(self, invalid_type=None, obj_name='o', fsize=1024, + csize=8, mark_deleted=False, ts=None, + mount_check=False, extra_metadata=None): '''returns a DiskFile''' - df = diskfile.DiskFile(self.testdir, 'sda1', '0', 'a', 'c', - obj_name, FakeLogger()) + df = self.df_mgr.get_diskfile('sda1', '0', 'a', 'c', obj_name) data = '0' * fsize etag = md5() if ts: timestamp = ts else: - timestamp = str(normalize_timestamp(time())) + timestamp = normalize_timestamp(time()) with df.create() as writer: - writer.write(data) + upload_size = writer.write(data) etag.update(data) etag = etag.hexdigest() metadata = { 'ETag': etag, 'X-Timestamp': timestamp, - 'Content-Length': str(os.fstat(writer.fd).st_size), + 'Content-Length': str(upload_size), } metadata.update(extra_metadata or {}) writer.put(metadata) @@ -567,209 +551,327 @@ class TestDiskFile(unittest.TestCase): etag.update('1' + '0' * (fsize - 1)) etag = etag.hexdigest() metadata['ETag'] = etag - diskfile.write_metadata(writer.fd, metadata) + diskfile.write_metadata(writer._fd, metadata) if invalid_type == 'Content-Length': metadata['Content-Length'] = fsize - 1 - diskfile.write_metadata(writer.fd, metadata) + diskfile.write_metadata(writer._fd, metadata) + if invalid_type == 'Bad-Content-Length': + metadata['Content-Length'] = 'zero' + diskfile.write_metadata(writer._fd, metadata) + if invalid_type == 'Missing-Content-Length': + del metadata['Content-Length'] + diskfile.write_metadata(writer._fd, metadata) if mark_deleted: - metadata = { - 'X-Timestamp': timestamp, - 'deleted': True - } - df.put_metadata(metadata, tombstone=True) + df.delete(timestamp) - df = diskfile.DiskFile(self.testdir, 'sda1', '0', 'a', 'c', - obj_name, FakeLogger(), - disk_chunk_size=csize, - iter_hook=iter_hook, mount_check=mount_check) + self.conf['disk_chunk_size'] = csize + self.conf['mount_check'] = mount_check + self.df_mgr = diskfile.DiskFileManager(self.conf, FakeLogger()) + df = self.df_mgr.get_diskfile('sda1', '0', 'a', 'c', obj_name) df.open() if invalid_type == 'Zero-Byte': - fp = open(df.data_file, 'w') + fp = open(df._data_file, 'w') fp.close() df.unit_test_len = fsize return df + def test_iter_hook(self): + hook_call_count = [0] + + def hook(): + hook_call_count[0] += 1 + + df = self._get_open_disk_file(fsize=65, csize=8) + with df.open(): + for _ in df.reader(iter_hook=hook): + pass + + self.assertEquals(hook_call_count[0], 9) + + def test_keep_cache(self): + df = self._get_open_disk_file(fsize=65) + with mock.patch("swift.obj.diskfile.drop_buffer_cache") as foo: + for _ in df.reader(): + pass + self.assertTrue(foo.called) + + df = self._get_open_disk_file(fsize=65) + with mock.patch("swift.obj.diskfile.drop_buffer_cache") as bar: + for _ in df.reader(keep_cache=False): + pass + self.assertTrue(bar.called) + + df = self._get_open_disk_file(fsize=65) + with mock.patch("swift.obj.diskfile.drop_buffer_cache") as boo: + for _ in df.reader(keep_cache=True): + pass + self.assertFalse(boo.called) + + df = self._get_open_disk_file(fsize=5 * 1024, csize=256) + with mock.patch("swift.obj.diskfile.drop_buffer_cache") as goo: + for _ in df.reader(keep_cache=True): + pass + self.assertTrue(goo.called) + def test_quarantine_valids(self): - df = self._get_disk_file(obj_name='1') - for chunk in df: - pass - self.assertFalse(df.quarantined_dir) - df = self._get_disk_file(obj_name='2', csize=1) - for chunk in df: - pass - self.assertFalse(df.quarantined_dir) + def verify(*args, **kwargs): + try: + df = self._get_open_disk_file(**kwargs) + reader = df.reader() + for chunk in reader: + pass + except DiskFileQuarantined: + self.fail( + "Unexpected quarantining occurred: args=%r, kwargs=%r" % ( + args, kwargs)) + else: + pass - df = self._get_disk_file(obj_name='3', csize=100000) - for chunk in df: - pass - self.assertFalse(df.quarantined_dir) + verify(obj_name='1') + + verify(obj_name='2', csize=1) + + verify(obj_name='3', csize=100000) def run_quarantine_invalids(self, invalid_type): - df = self._get_disk_file(invalid_type=invalid_type, obj_name='1') - for chunk in df: - pass - self.assertTrue(df.quarantined_dir) - df = self._get_disk_file(invalid_type=invalid_type, - obj_name='2', csize=1) - for chunk in df: - pass - self.assertTrue(df.quarantined_dir) - df = self._get_disk_file(invalid_type=invalid_type, - obj_name='3', csize=100000) - for chunk in df: - pass - self.assertTrue(df.quarantined_dir) - df = self._get_disk_file(invalid_type=invalid_type, obj_name='4') - self.assertFalse(df.quarantined_dir) - df = self._get_disk_file(invalid_type=invalid_type, obj_name='5') - for chunk in df.app_iter_range(0, df.unit_test_len): - pass - self.assertTrue(df.quarantined_dir) - df = self._get_disk_file(invalid_type=invalid_type, obj_name='6') - for chunk in df.app_iter_range(0, df.unit_test_len + 100): - pass - self.assertTrue(df.quarantined_dir) - expected_quar = False - # for the following, Content-Length/Zero-Byte errors will always result - # in a quarantine, even if the whole file isn't check-summed - if invalid_type in ('Zero-Byte', 'Content-Length'): - expected_quar = True - df = self._get_disk_file(invalid_type=invalid_type, obj_name='7') - for chunk in df.app_iter_range(1, df.unit_test_len): - pass - self.assertEquals(bool(df.quarantined_dir), expected_quar) - df = self._get_disk_file(invalid_type=invalid_type, obj_name='8') - for chunk in df.app_iter_range(0, df.unit_test_len - 1): - pass - self.assertEquals(bool(df.quarantined_dir), expected_quar) - df = self._get_disk_file(invalid_type=invalid_type, obj_name='8') - for chunk in df.app_iter_range(1, df.unit_test_len + 1): - pass - self.assertEquals(bool(df.quarantined_dir), expected_quar) - def test_quarantine_invalids(self): + def verify(*args, **kwargs): + open_exc = invalid_type in ('Content-Length', 'Bad-Content-Length') + reader = None + try: + df = self._get_open_disk_file(**kwargs) + reader = df.reader() + except DiskFileQuarantined as err: + if not open_exc: + self.fail( + "Unexpected DiskFileQuarantine raised: :%r" % err) + return + else: + if open_exc: + self.fail("Expected DiskFileQuarantine exception") + try: + for chunk in reader: + pass + except DiskFileQuarantined as err: + self.fail("Unexpected DiskFileQuarantine raised: :%r" % err) + else: + if not open_exc: + self.assertTrue(reader.was_quarantined) + + verify(invalid_type=invalid_type, obj_name='1') + + verify(invalid_type=invalid_type, obj_name='2', csize=1) + + verify(invalid_type=invalid_type, obj_name='3', csize=100000) + + verify(invalid_type=invalid_type, obj_name='4') + + def verify_air(params, start=0, adjustment=0): + """verify (a)pp (i)ter (r)ange""" + open_exc = invalid_type in ('Content-Length', 'Bad-Content-Length') + reader = None + try: + df = self._get_open_disk_file(**params) + reader = df.reader() + except DiskFileQuarantined as err: + if not open_exc: + self.fail( + "Unexpected DiskFileQuarantine raised: :%r" % err) + return + else: + if open_exc: + self.fail("Expected DiskFileQuarantine exception") + try: + for chunk in reader.app_iter_range( + start, + df.unit_test_len + adjustment): + pass + except DiskFileQuarantined as err: + self.fail("Unexpected DiskFileQuarantine raised: :%r" % err) + + verify_air(dict(invalid_type=invalid_type, obj_name='5')) + + verify_air(dict(invalid_type=invalid_type, obj_name='6'), 0, 100) + + verify_air(dict(invalid_type=invalid_type, obj_name='7'), 1) + + verify_air(dict(invalid_type=invalid_type, obj_name='8'), 0, -1) + + verify_air(dict(invalid_type=invalid_type, obj_name='8'), 1, 1) + + def test_quarantine_invalid_etag(self): self.run_quarantine_invalids('ETag') + + def test_quarantine_invalid_content_length(self): self.run_quarantine_invalids('Content-Length') + + def test_quarantine_invalid_content_length_bad(self): + self.run_quarantine_invalids('Bad-Content-Length') + + def test_quarantine_invalid_zero_byte(self): self.run_quarantine_invalids('Zero-Byte') def test_quarantine_deleted_files(self): - df = self._get_disk_file(invalid_type='Content-Length') - df.close() - self.assertTrue(df.quarantined_dir) - df = self._get_disk_file(invalid_type='Content-Length', - mark_deleted=True) - df.close() - self.assertFalse(df.quarantined_dir) - df = self._get_disk_file(invalid_type='Content-Length', - mark_deleted=True) - self.assertRaises(DiskFileNotExist, df.get_data_file_size) + try: + self._get_open_disk_file(invalid_type='Content-Length') + except DiskFileQuarantined: + pass + else: + self.fail("Expected DiskFileQuarantined exception") + try: + self._get_open_disk_file(invalid_type='Content-Length', + mark_deleted=True) + except DiskFileQuarantined as err: + self.fail("Unexpected DiskFileQuarantined exception" + " encountered: %r" % err) + except DiskFileNotExist: + pass + else: + self.fail("Expected DiskFileNotExist exception") + try: + self._get_open_disk_file(invalid_type='Content-Length', + mark_deleted=True) + except DiskFileNotExist: + pass + else: + self.fail("Expected DiskFileNotExist exception") - def test_put_metadata(self): - df = self._get_disk_file() - ts = time() - metadata = {'X-Timestamp': ts, 'X-Object-Meta-test': 'data'} - df.put_metadata(metadata) - exp_name = '%s.meta' % str(normalize_timestamp(ts)) - dl = os.listdir(df.datadir) + def test_quarantine_missing_content_length(self): + try: + self._get_open_disk_file( + invalid_type='Missing-Content-Length') + except DiskFileQuarantined: + pass + + def test_quarantine_bad_content_length(self): + try: + self._get_open_disk_file( + invalid_type='Bad-Content-Length') + except DiskFileQuarantined: + pass + else: + self.fail("Expected DiskFileQuarantined exception") + + def test_quarantine_fstat_oserror(self): + invocations = [0] + orig_os_fstat = os.fstat + + def bad_fstat(fd): + invocations[0] += 1 + if invocations[0] == 4: + # FIXME - yes, this an icky way to get code coverage ... worth + # it? + raise OSError() + return orig_os_fstat(fd) + + with mock.patch('os.fstat', bad_fstat): + try: + self._get_open_disk_file() + except DiskFileQuarantined: + pass + else: + self.fail("Expected DiskFileQuarantined exception") + + def test_write_metadata(self): + df = self._create_test_file('1234567890') + timestamp = normalize_timestamp(time()) + metadata = {'X-Timestamp': timestamp, 'X-Object-Meta-test': 'data'} + df.write_metadata(metadata) + dl = os.listdir(df._datadir) self.assertEquals(len(dl), 2) - self.assertTrue(exp_name in set(dl)) - - def test_put_metadata_ts(self): - df = self._get_disk_file() - ts = time() - metadata = {'X-Timestamp': ts, 'X-Object-Meta-test': 'data'} - df.put_metadata(metadata, tombstone=True) - exp_name = '%s.ts' % str(normalize_timestamp(ts)) - dl = os.listdir(df.datadir) - self.assertEquals(len(dl), 1) + exp_name = '%s.meta' % timestamp self.assertTrue(exp_name in set(dl)) def test_delete(self): - df = self._get_disk_file() + df = self._get_open_disk_file() ts = time() df.delete(ts) exp_name = '%s.ts' % str(normalize_timestamp(ts)) - dl = os.listdir(df.datadir) + dl = os.listdir(df._datadir) self.assertEquals(len(dl), 1) self.assertTrue(exp_name in set(dl)) + def test_open_deleted(self): + df = self._get_open_disk_file() + ts = time() + df.delete(ts) + exp_name = '%s.ts' % str(normalize_timestamp(ts)) + dl = os.listdir(df._datadir) + self.assertEquals(len(dl), 1) + self.assertTrue(exp_name in set(dl)) + df = self.df_mgr.get_diskfile('sda1', '0', 'a', 'c', 'o') + self.assertRaises(DiskFileDeleted, df.open) + def test_close_error(self): - def err(): - raise Exception("bad") + def mock_handle_close_quarantine(): + raise Exception("Bad") - df = self._get_disk_file(fsize=1024 * 2) - df._handle_close_quarantine = err - with df.open(): - for chunk in df: - pass + df = self._get_open_disk_file(fsize=1024 * 1024 * 2) + reader = df.reader() + reader._handle_close_quarantine = mock_handle_close_quarantine + for chunk in reader: + pass # close is called at the end of the iterator - self.assertEquals(df.fp, None) - self.assertEquals(len(df.logger.log_dict['error']), 1) - - def test_quarantine_twice(self): - df = self._get_disk_file(invalid_type='Content-Length') - self.assert_(os.path.isfile(df.data_file)) - quar_dir = df.quarantine() - self.assertFalse(os.path.isfile(df.data_file)) - self.assert_(os.path.isdir(quar_dir)) - self.assertEquals(df.quarantine(), None) + self.assertEquals(reader._fp, None) + self.assertEquals(len(df._logger.log_dict['error']), 1) def test_mount_checking(self): - def _mock_ismount(*args, **kwargs): + + def _mock_cm(*args, **kwargs): return False - with mock.patch("os.path.ismount", _mock_ismount): - self.assertRaises(DiskFileDeviceUnavailable, self._get_disk_file, - mount_check=True) + + with mock.patch("swift.common.constraints.check_mount", _mock_cm): + try: + self._get_open_disk_file(mount_check=True) + except DiskFileDeviceUnavailable: + pass + else: + self.fail("Expected DiskFileDeviceUnavailable exception") def test_ondisk_search_loop_ts_meta_data(self): - df = diskfile.DiskFile(self.testdir, 'sda1', '0', 'a', 'c', 'o', - FakeLogger()) + df = self.df_mgr.get_diskfile('sda1', '0', 'a', 'c', 'o') self._create_ondisk_file(df, '', ext='.ts', timestamp=10) self._create_ondisk_file(df, '', ext='.ts', timestamp=9) self._create_ondisk_file(df, '', ext='.meta', timestamp=8) self._create_ondisk_file(df, '', ext='.meta', timestamp=7) self._create_ondisk_file(df, 'B', ext='.data', timestamp=6) self._create_ondisk_file(df, 'A', ext='.data', timestamp=5) - df = diskfile.DiskFile(self.testdir, 'sda1', '0', 'a', 'c', 'o', - FakeLogger()) - with df.open(): - self.assertTrue('X-Timestamp' in df._metadata) - self.assertEquals(df._metadata['X-Timestamp'], - normalize_timestamp(10)) - self.assertTrue('deleted' in df._metadata) - self.assertTrue(df._metadata['deleted']) + df = self.df_mgr.get_diskfile('sda1', '0', 'a', 'c', 'o') + try: + df.open() + except DiskFileDeleted as d: + self.assertEquals(d.timestamp, normalize_timestamp(10)) + else: + self.fail("Expected DiskFileDeleted exception") def test_ondisk_search_loop_meta_ts_data(self): - df = diskfile.DiskFile(self.testdir, 'sda1', '0', 'a', 'c', 'o', - FakeLogger()) + df = self.df_mgr.get_diskfile('sda1', '0', 'a', 'c', 'o') self._create_ondisk_file(df, '', ext='.meta', timestamp=10) self._create_ondisk_file(df, '', ext='.meta', timestamp=9) self._create_ondisk_file(df, '', ext='.ts', timestamp=8) self._create_ondisk_file(df, '', ext='.ts', timestamp=7) self._create_ondisk_file(df, 'B', ext='.data', timestamp=6) self._create_ondisk_file(df, 'A', ext='.data', timestamp=5) - df = diskfile.DiskFile(self.testdir, 'sda1', '0', 'a', 'c', 'o', - FakeLogger()) - with df.open(): - self.assertTrue('X-Timestamp' in df._metadata) - self.assertEquals(df._metadata['X-Timestamp'], - normalize_timestamp(8)) - self.assertTrue('deleted' in df._metadata) + df = self.df_mgr.get_diskfile('sda1', '0', 'a', 'c', 'o') + try: + df.open() + except DiskFileDeleted as d: + self.assertEquals(d.timestamp, normalize_timestamp(8)) + else: + self.fail("Expected DiskFileDeleted exception") def test_ondisk_search_loop_meta_data_ts(self): - df = diskfile.DiskFile(self.testdir, 'sda1', '0', 'a', 'c', 'o', - FakeLogger()) + df = self.df_mgr.get_diskfile('sda1', '0', 'a', 'c', 'o') self._create_ondisk_file(df, '', ext='.meta', timestamp=10) self._create_ondisk_file(df, '', ext='.meta', timestamp=9) self._create_ondisk_file(df, 'B', ext='.data', timestamp=8) self._create_ondisk_file(df, 'A', ext='.data', timestamp=7) self._create_ondisk_file(df, '', ext='.ts', timestamp=6) self._create_ondisk_file(df, '', ext='.ts', timestamp=5) - df = diskfile.DiskFile(self.testdir, 'sda1', '0', 'a', 'c', 'o', - FakeLogger()) + df = self.df_mgr.get_diskfile('sda1', '0', 'a', 'c', 'o') with df.open(): self.assertTrue('X-Timestamp' in df._metadata) self.assertEquals(df._metadata['X-Timestamp'], @@ -777,16 +879,14 @@ class TestDiskFile(unittest.TestCase): self.assertTrue('deleted' not in df._metadata) def test_ondisk_search_loop_data_meta_ts(self): - df = diskfile.DiskFile(self.testdir, 'sda1', '0', 'a', 'c', 'o', - FakeLogger()) + df = self.df_mgr.get_diskfile('sda1', '0', 'a', 'c', 'o') self._create_ondisk_file(df, 'B', ext='.data', timestamp=10) self._create_ondisk_file(df, 'A', ext='.data', timestamp=9) self._create_ondisk_file(df, '', ext='.ts', timestamp=8) self._create_ondisk_file(df, '', ext='.ts', timestamp=7) self._create_ondisk_file(df, '', ext='.meta', timestamp=6) self._create_ondisk_file(df, '', ext='.meta', timestamp=5) - df = diskfile.DiskFile(self.testdir, 'sda1', '0', 'a', 'c', 'o', - FakeLogger()) + df = self.df_mgr.get_diskfile('sda1', '0', 'a', 'c', 'o') with df.open(): self.assertTrue('X-Timestamp' in df._metadata) self.assertEquals(df._metadata['X-Timestamp'], @@ -794,8 +894,7 @@ class TestDiskFile(unittest.TestCase): self.assertTrue('deleted' not in df._metadata) def test_ondisk_search_loop_wayward_files_ignored(self): - df = diskfile.DiskFile(self.testdir, 'sda1', '0', 'a', 'c', 'o', - FakeLogger()) + df = self.df_mgr.get_diskfile('sda1', '0', 'a', 'c', 'o') self._create_ondisk_file(df, 'X', ext='.bar', timestamp=11) self._create_ondisk_file(df, 'B', ext='.data', timestamp=10) self._create_ondisk_file(df, 'A', ext='.data', timestamp=9) @@ -803,8 +902,7 @@ class TestDiskFile(unittest.TestCase): self._create_ondisk_file(df, '', ext='.ts', timestamp=7) self._create_ondisk_file(df, '', ext='.meta', timestamp=6) self._create_ondisk_file(df, '', ext='.meta', timestamp=5) - df = diskfile.DiskFile(self.testdir, 'sda1', '0', 'a', 'c', 'o', - FakeLogger()) + df = self.df_mgr.get_diskfile('sda1', '0', 'a', 'c', 'o') with df.open(): self.assertTrue('X-Timestamp' in df._metadata) self.assertEquals(df._metadata['X-Timestamp'], @@ -812,8 +910,7 @@ class TestDiskFile(unittest.TestCase): self.assertTrue('deleted' not in df._metadata) def test_ondisk_search_loop_listdir_error(self): - df = diskfile.DiskFile(self.testdir, 'sda1', '0', 'a', 'c', 'o', - FakeLogger()) + df = self.df_mgr.get_diskfile('sda1', '0', 'a', 'c', 'o') def mock_listdir_exp(*args, **kwargs): raise OSError(errno.EACCES, os.strerror(errno.EACCES)) @@ -826,17 +923,19 @@ class TestDiskFile(unittest.TestCase): self._create_ondisk_file(df, '', ext='.ts', timestamp=7) self._create_ondisk_file(df, '', ext='.meta', timestamp=6) self._create_ondisk_file(df, '', ext='.meta', timestamp=5) - df = diskfile.DiskFile(self.testdir, 'sda1', '0', 'a', 'c', 'o', - FakeLogger()) - self.assertRaises(OSError, df.open) + df = self.df_mgr.get_diskfile('sda1', '0', 'a', 'c', 'o') + self.assertRaises(DiskFileError, df.open) def test_exception_in_handle_close_quarantine(self): - df = self._get_disk_file() + df = self._get_open_disk_file() def blow_up(): raise Exception('a very special error') - df._handle_close_quarantine = blow_up - df.close() - log_lines = df.logger.get_lines_for_level('error') + reader = df.reader() + reader._handle_close_quarantine = blow_up + for _ in reader: + pass + reader.close() + log_lines = df._logger.get_lines_for_level('error') self.assert_('a very special error' in log_lines[-1]) diff --git a/test/unit/obj/test_replicator.py b/test/unit/obj/test_replicator.py index 9e7f21991d..e8fcb30f45 100644 --- a/test/unit/obj/test_replicator.py +++ b/test/unit/obj/test_replicator.py @@ -156,9 +156,10 @@ class TestObjectReplicator(unittest.TestCase): self.conf = dict( swift_dir=self.testdir, devices=self.devices, mount_check='false', timeout='300', stats_interval='1') - self.replicator = object_replicator.ObjectReplicator( - self.conf) + self.replicator = object_replicator.ObjectReplicator(self.conf) self.replicator.logger = FakeLogger() + self.df_mgr = diskfile.DiskFileManager(self.conf, + self.replicator.logger) def tearDown(self): rmtree(self.testdir, ignore_errors=1) @@ -170,10 +171,9 @@ class TestObjectReplicator(unittest.TestCase): was_connector = object_replicator.http_connect object_replicator.http_connect = mock_http_connect(200) cur_part = '0' - df = diskfile.DiskFile(self.devices, 'sda', cur_part, 'a', 'c', 'o', - FakeLogger()) - mkdirs(df.datadir) - f = open(os.path.join(df.datadir, + df = self.df_mgr.get_diskfile('sda', cur_part, 'a', 'c', 'o') + mkdirs(df._datadir) + f = open(os.path.join(df._datadir, normalize_timestamp(time.time()) + '.data'), 'wb') f.write('1234567890') @@ -297,11 +297,10 @@ class TestObjectReplicator(unittest.TestCase): def test_delete_partition(self): with mock.patch('swift.obj.replicator.http_connect', mock_http_connect(200)): - df = diskfile.DiskFile(self.devices, - 'sda', '1', 'a', 'c', 'o', FakeLogger()) - mkdirs(df.datadir) - print df.datadir - f = open(os.path.join(df.datadir, + df = self.df_mgr.get_diskfile('sda', '1', 'a', 'c', 'o') + mkdirs(df._datadir) + print df._datadir + f = open(os.path.join(df._datadir, normalize_timestamp(time.time()) + '.data'), 'wb') f.write('1234567890') @@ -326,11 +325,10 @@ class TestObjectReplicator(unittest.TestCase): def test_delete_partition_with_failures(self): with mock.patch('swift.obj.replicator.http_connect', mock_http_connect(200)): - df = diskfile.DiskFile(self.devices, - 'sda', '1', 'a', 'c', 'o', FakeLogger()) - mkdirs(df.datadir) - print df.datadir - f = open(os.path.join(df.datadir, + df = self.df_mgr.get_diskfile('sda', '1', 'a', 'c', 'o') + mkdirs(df._datadir) + print df._datadir + f = open(os.path.join(df._datadir, normalize_timestamp(time.time()) + '.data'), 'wb') f.write('1234567890') @@ -362,11 +360,10 @@ class TestObjectReplicator(unittest.TestCase): with mock.patch('swift.obj.replicator.http_connect', mock_http_connect(200)): self.replicator.handoff_delete = 2 - df = diskfile.DiskFile(self.devices, - 'sda', '1', 'a', 'c', 'o', FakeLogger()) - mkdirs(df.datadir) - print df.datadir - f = open(os.path.join(df.datadir, + df = self.df_mgr.get_diskfile('sda', '1', 'a', 'c', 'o') + mkdirs(df._datadir) + print df._datadir + f = open(os.path.join(df._datadir, normalize_timestamp(time.time()) + '.data'), 'wb') f.write('1234567890') @@ -397,11 +394,10 @@ class TestObjectReplicator(unittest.TestCase): with mock.patch('swift.obj.replicator.http_connect', mock_http_connect(200)): self.replicator.handoff_delete = 2 - df = diskfile.DiskFile(self.devices, - 'sda', '1', 'a', 'c', 'o', FakeLogger()) - mkdirs(df.datadir) - print df.datadir - f = open(os.path.join(df.datadir, + df = self.df_mgr.get_diskfile('sda', '1', 'a', 'c', 'o') + mkdirs(df._datadir) + print df._datadir + f = open(os.path.join(df._datadir, normalize_timestamp(time.time()) + '.data'), 'wb') f.write('1234567890') @@ -430,9 +426,8 @@ class TestObjectReplicator(unittest.TestCase): self.assertTrue(os.access(part_path, os.F_OK)) def test_delete_partition_override_params(self): - df = diskfile.DiskFile(self.devices, 'sda', '0', 'a', 'c', 'o', - FakeLogger()) - mkdirs(df.datadir) + df = self.df_mgr.get_diskfile('sda', '0', 'a', 'c', 'o') + mkdirs(df._datadir) part_path = os.path.join(self.objects, '1') self.assertTrue(os.access(part_path, os.F_OK)) self.replicator.replicate(override_devices=['sdb']) @@ -453,10 +448,9 @@ class TestObjectReplicator(unittest.TestCase): # Write some files into '1' and run replicate- they should be moved # to the other partitoins and then node should get deleted. cur_part = '1' - df = diskfile.DiskFile( - self.devices, 'sda', cur_part, 'a', 'c', 'o', FakeLogger()) - mkdirs(df.datadir) - f = open(os.path.join(df.datadir, + df = self.df_mgr.get_diskfile('sda', cur_part, 'a', 'c', 'o') + mkdirs(df._datadir) + f = open(os.path.join(df._datadir, normalize_timestamp(time.time()) + '.data'), 'wb') f.write('1234567890') @@ -517,10 +511,9 @@ class TestObjectReplicator(unittest.TestCase): # Write some files into '1' and run replicate- they should be moved # to the other partitions and then node should get deleted. cur_part = '1' - df = diskfile.DiskFile( - self.devices, 'sda', cur_part, 'a', 'c', 'o', FakeLogger()) - mkdirs(df.datadir) - f = open(os.path.join(df.datadir, + df = self.df_mgr.get_diskfile('sda', cur_part, 'a', 'c', 'o') + mkdirs(df._datadir) + f = open(os.path.join(df._datadir, normalize_timestamp(time.time()) + '.data'), 'wb') f.write('1234567890') diff --git a/test/unit/obj/test_server.py b/test/unit/obj/test_server.py index b4f2bd35df..bc8b491245 100755 --- a/test/unit/obj/test_server.py +++ b/test/unit/obj/test_server.py @@ -40,6 +40,10 @@ from eventlet import tpool from swift.common.swob import Request, HeaderKeyDict +def mock_time(*args, **kwargs): + return 5000.0 + + class TestObjectController(unittest.TestCase): """Test swift.obj.server.ObjectController""" @@ -55,6 +59,7 @@ class TestObjectController(unittest.TestCase): self.object_controller.bytes_per_sync = 1 self._orig_tpool_exc = tpool.execute tpool.execute = lambda f, *args, **kwargs: f(*args, **kwargs) + self.df_mgr = diskfile.DiskFileManager(conf, FakeLogger()) def tearDown(self): """Tear down for testing swift.object.server.ObjectController""" @@ -360,18 +365,17 @@ class TestObjectController(unittest.TestCase): req.body = 'VERIFY' resp = req.get_response(self.object_controller) self.assertEquals(resp.status_int, 201) - objfile = diskfile.DiskFile(self.testdir, 'sda1', 'p', 'a', 'c', 'o', - FakeLogger()) + objfile = self.df_mgr.get_diskfile('sda1', 'p', 'a', 'c', 'o') objfile.open() - file_name = os.path.basename(objfile.data_file) - with open(objfile.data_file) as fp: + file_name = os.path.basename(objfile._data_file) + with open(objfile._data_file) as fp: metadata = diskfile.read_metadata(fp) - os.unlink(objfile.data_file) - with open(objfile.data_file, 'w') as fp: + os.unlink(objfile._data_file) + with open(objfile._data_file, 'w') as fp: diskfile.write_metadata(fp, metadata) - self.assertEquals(os.listdir(objfile.datadir)[0], file_name) + self.assertEquals(os.listdir(objfile._datadir)[0], file_name) req = Request.blank( '/sda1/p/a/c/o', headers={'X-Timestamp': normalize_timestamp(time())}) @@ -380,7 +384,7 @@ class TestObjectController(unittest.TestCase): quar_dir = os.path.join( self.testdir, 'sda1', 'quarantined', 'objects', - os.path.basename(os.path.dirname(objfile.data_file))) + os.path.basename(os.path.dirname(objfile._data_file))) self.assertEquals(os.listdir(quar_dir)[0], file_name) def test_PUT_invalid_path(self): @@ -446,7 +450,7 @@ class TestObjectController(unittest.TestCase): self.assertEquals(resp.status_int, 201) objfile = os.path.join( self.testdir, 'sda1', - storage_directory(object_server.DATADIR, 'p', + storage_directory(diskfile.DATADIR, 'p', hash_path('a', 'c', 'o')), timestamp + '.data') self.assert_(os.path.isfile(objfile)) @@ -479,7 +483,7 @@ class TestObjectController(unittest.TestCase): self.assertEquals(resp.status_int, 201) objfile = os.path.join( self.testdir, 'sda1', - storage_directory(object_server.DATADIR, 'p', + storage_directory(diskfile.DATADIR, 'p', hash_path('a', 'c', 'o')), timestamp + '.data') self.assert_(os.path.isfile(objfile)) @@ -553,7 +557,7 @@ class TestObjectController(unittest.TestCase): self.assertEquals(resp.status_int, 201) objfile = os.path.join( self.testdir, 'sda1', - storage_directory(object_server.DATADIR, 'p', + storage_directory(diskfile.DATADIR, 'p', hash_path('a', 'c', 'o')), timestamp + '.data') self.assert_(os.path.isfile(objfile)) @@ -676,7 +680,7 @@ class TestObjectController(unittest.TestCase): objfile = os.path.join( self.testdir, 'sda1', - storage_directory(object_server.DATADIR, 'p', + storage_directory(diskfile.DATADIR, 'p', hash_path('a', 'c', 'o')), timestamp + '.data') os.unlink(objfile) @@ -718,18 +722,18 @@ class TestObjectController(unittest.TestCase): req.body = 'VERIFY' resp = req.get_response(self.object_controller) self.assertEquals(resp.status_int, 201) - objfile = diskfile.DiskFile(self.testdir, 'sda1', 'p', 'a', 'c', 'o', - FakeLogger()) - objfile.open() + disk_file = self.df_mgr.get_diskfile('sda1', 'p', 'a', 'c', 'o') + disk_file.open() - file_name = os.path.basename(objfile.data_file) - with open(objfile.data_file) as fp: + file_name = os.path.basename(disk_file._data_file) + with open(disk_file._data_file) as fp: metadata = diskfile.read_metadata(fp) - os.unlink(objfile.data_file) - with open(objfile.data_file, 'w') as fp: + os.unlink(disk_file._data_file) + with open(disk_file._data_file, 'w') as fp: diskfile.write_metadata(fp, metadata) - self.assertEquals(os.listdir(objfile.datadir)[0], file_name) + file_name = os.path.basename(disk_file._data_file) + self.assertEquals(os.listdir(disk_file._datadir)[0], file_name) req = Request.blank('/sda1/p/a/c/o', environ={'REQUEST_METHOD': 'HEAD'}) resp = req.get_response(self.object_controller) @@ -737,7 +741,7 @@ class TestObjectController(unittest.TestCase): quar_dir = os.path.join( self.testdir, 'sda1', 'quarantined', 'objects', - os.path.basename(os.path.dirname(objfile.data_file))) + os.path.basename(os.path.dirname(disk_file._data_file))) self.assertEquals(os.listdir(quar_dir)[0], file_name) def test_GET(self): @@ -799,7 +803,7 @@ class TestObjectController(unittest.TestCase): objfile = os.path.join( self.testdir, 'sda1', - storage_directory(object_server.DATADIR, 'p', + storage_directory(diskfile.DATADIR, 'p', hash_path('a', 'c', 'o')), timestamp + '.data') os.unlink(objfile) @@ -1017,23 +1021,22 @@ class TestObjectController(unittest.TestCase): req.body = 'VERIFY' resp = req.get_response(self.object_controller) self.assertEquals(resp.status_int, 201) - objfile = diskfile.DiskFile(self.testdir, 'sda1', 'p', 'a', 'c', 'o', - FakeLogger()) - objfile.open() - file_name = os.path.basename(objfile.data_file) + disk_file = self.df_mgr.get_diskfile('sda1', 'p', 'a', 'c', 'o') + disk_file.open() + file_name = os.path.basename(disk_file._data_file) etag = md5() etag.update('VERIF') etag = etag.hexdigest() - metadata = {'X-Timestamp': timestamp, + metadata = {'X-Timestamp': timestamp, 'name': '/a/c/o', 'Content-Length': 6, 'ETag': etag} - diskfile.write_metadata(objfile.fp, metadata) - self.assertEquals(os.listdir(objfile.datadir)[0], file_name) + diskfile.write_metadata(disk_file._fp, metadata) + self.assertEquals(os.listdir(disk_file._datadir)[0], file_name) req = Request.blank('/sda1/p/a/c/o') resp = req.get_response(self.object_controller) quar_dir = os.path.join( self.testdir, 'sda1', 'quarantined', 'objects', - os.path.basename(os.path.dirname(objfile.data_file))) - self.assertEquals(os.listdir(objfile.datadir)[0], file_name) + os.path.basename(os.path.dirname(disk_file._data_file))) + self.assertEquals(os.listdir(disk_file._datadir)[0], file_name) body = resp.body # actually does quarantining self.assertEquals(body, 'VERIFY') self.assertEquals(os.listdir(quar_dir)[0], file_name) @@ -1050,24 +1053,23 @@ class TestObjectController(unittest.TestCase): req.body = 'VERIFY' resp = req.get_response(self.object_controller) self.assertEquals(resp.status_int, 201) - objfile = diskfile.DiskFile(self.testdir, 'sda1', 'p', 'a', 'c', 'o', - FakeLogger()) - objfile.open() - file_name = os.path.basename(objfile.data_file) - with open(objfile.data_file) as fp: + disk_file = self.df_mgr.get_diskfile('sda1', 'p', 'a', 'c', 'o') + disk_file.open() + file_name = os.path.basename(disk_file._data_file) + with open(disk_file._data_file) as fp: metadata = diskfile.read_metadata(fp) - os.unlink(objfile.data_file) - with open(objfile.data_file, 'w') as fp: + os.unlink(disk_file._data_file) + with open(disk_file._data_file, 'w') as fp: diskfile.write_metadata(fp, metadata) - self.assertEquals(os.listdir(objfile.datadir)[0], file_name) + self.assertEquals(os.listdir(disk_file._datadir)[0], file_name) req = Request.blank('/sda1/p/a/c/o') resp = req.get_response(self.object_controller) self.assertEquals(resp.status_int, 404) quar_dir = os.path.join( self.testdir, 'sda1', 'quarantined', 'objects', - os.path.basename(os.path.dirname(objfile.data_file))) + os.path.basename(os.path.dirname(disk_file._data_file))) self.assertEquals(os.listdir(quar_dir)[0], file_name) def test_GET_quarantine_range(self): @@ -1079,25 +1081,24 @@ class TestObjectController(unittest.TestCase): req.body = 'VERIFY' resp = req.get_response(self.object_controller) self.assertEquals(resp.status_int, 201) - objfile = diskfile.DiskFile(self.testdir, 'sda1', 'p', 'a', 'c', 'o', - FakeLogger()) - objfile.open() - file_name = os.path.basename(objfile.data_file) + disk_file = self.df_mgr.get_diskfile('sda1', 'p', 'a', 'c', 'o') + disk_file.open() + file_name = os.path.basename(disk_file._data_file) etag = md5() etag.update('VERIF') etag = etag.hexdigest() - metadata = {'X-Timestamp': timestamp, + metadata = {'X-Timestamp': timestamp, 'name': '/a/c/o', 'Content-Length': 6, 'ETag': etag} - diskfile.write_metadata(objfile.fp, metadata) - self.assertEquals(os.listdir(objfile.datadir)[0], file_name) + diskfile.write_metadata(disk_file._fp, metadata) + self.assertEquals(os.listdir(disk_file._datadir)[0], file_name) req = Request.blank('/sda1/p/a/c/o') req.range = 'bytes=0-4' # partial resp = req.get_response(self.object_controller) quar_dir = os.path.join( self.testdir, 'sda1', 'quarantined', 'objects', - os.path.basename(os.path.dirname(objfile.data_file))) + os.path.basename(os.path.dirname(disk_file._data_file))) resp.body - self.assertEquals(os.listdir(objfile.datadir)[0], file_name) + self.assertEquals(os.listdir(disk_file._datadir)[0], file_name) self.assertFalse(os.path.isdir(quar_dir)) req = Request.blank('/sda1/p/a/c/o') resp = req.get_response(self.object_controller) @@ -1108,9 +1109,9 @@ class TestObjectController(unittest.TestCase): resp = req.get_response(self.object_controller) quar_dir = os.path.join( self.testdir, 'sda1', 'quarantined', 'objects', - os.path.basename(os.path.dirname(objfile.data_file))) + os.path.basename(os.path.dirname(disk_file._data_file))) resp.body - self.assertEquals(os.listdir(objfile.datadir)[0], file_name) + self.assertEquals(os.listdir(disk_file._datadir)[0], file_name) self.assertFalse(os.path.isdir(quar_dir)) req = Request.blank('/sda1/p/a/c/o') @@ -1118,14 +1119,15 @@ class TestObjectController(unittest.TestCase): resp = req.get_response(self.object_controller) quar_dir = os.path.join( self.testdir, 'sda1', 'quarantined', 'objects', - os.path.basename(os.path.dirname(objfile.data_file))) - self.assertEquals(os.listdir(objfile.datadir)[0], file_name) + os.path.basename(os.path.dirname(disk_file._data_file))) + self.assertEquals(os.listdir(disk_file._datadir)[0], file_name) resp.body self.assertTrue(os.path.isdir(quar_dir)) req = Request.blank('/sda1/p/a/c/o') resp = req.get_response(self.object_controller) self.assertEquals(resp.status_int, 404) + @mock.patch("time.time", mock_time) def test_DELETE(self): # Test swift.obj.server.ObjectController.DELETE req = Request.blank('/sda1/p/a/c', @@ -1140,37 +1142,38 @@ class TestObjectController(unittest.TestCase): # self.assertRaises(KeyError, self.object_controller.DELETE, req) # The following should have created a tombstone file - timestamp = normalize_timestamp(time()) + timestamp = normalize_timestamp(1000) req = Request.blank('/sda1/p/a/c/o', environ={'REQUEST_METHOD': 'DELETE'}, headers={'X-Timestamp': timestamp}) resp = req.get_response(self.object_controller) - resp = req.get_response(self.object_controller) self.assertEquals(resp.status_int, 404) - objfile = os.path.join( + ts_1000_file = os.path.join( self.testdir, 'sda1', - storage_directory(object_server.DATADIR, 'p', + storage_directory(diskfile.DATADIR, 'p', hash_path('a', 'c', 'o')), timestamp + '.ts') - self.assert_(os.path.isfile(objfile)) + self.assertTrue(os.path.isfile(ts_1000_file)) + # There should now be a 1000 ts file. + self.assertEquals(len(os.listdir(os.path.dirname(ts_1000_file))), 1) # The following should *not* have created a tombstone file. - timestamp = normalize_timestamp(float(timestamp) - 1) + timestamp = normalize_timestamp(999) req = Request.blank('/sda1/p/a/c/o', environ={'REQUEST_METHOD': 'DELETE'}, headers={'X-Timestamp': timestamp}) resp = req.get_response(self.object_controller) self.assertEquals(resp.status_int, 404) - objfile = os.path.join( + ts_999_file = os.path.join( self.testdir, 'sda1', - storage_directory(object_server.DATADIR, 'p', + storage_directory(diskfile.DATADIR, 'p', hash_path('a', 'c', 'o')), timestamp + '.ts') - self.assertFalse(os.path.isfile(objfile)) - self.assertEquals(len(os.listdir(os.path.dirname(objfile))), 1) + self.assertFalse(os.path.isfile(ts_999_file)) + self.assertTrue(os.path.isfile(ts_1000_file)) + self.assertEquals(len(os.listdir(os.path.dirname(ts_1000_file))), 1) - sleep(.00001) - timestamp = normalize_timestamp(time()) + timestamp = normalize_timestamp(1002) req = Request.blank('/sda1/p/a/c/o', environ={'REQUEST_METHOD': 'PUT'}, headers={ 'X-Timestamp': timestamp, @@ -1180,35 +1183,44 @@ class TestObjectController(unittest.TestCase): req.body = 'test' resp = req.get_response(self.object_controller) self.assertEquals(resp.status_int, 201) + # There should now be 1000 ts and a 1001 data file. + data_1002_file = os.path.join( + self.testdir, 'sda1', + storage_directory(diskfile.DATADIR, 'p', + hash_path('a', 'c', 'o')), + timestamp + '.data') + self.assertTrue(os.path.isfile(data_1002_file)) + self.assertEquals(len(os.listdir(os.path.dirname(data_1002_file))), 1) # The following should *not* have created a tombstone file. - timestamp = normalize_timestamp(float(timestamp) - 1) + timestamp = normalize_timestamp(1001) req = Request.blank('/sda1/p/a/c/o', environ={'REQUEST_METHOD': 'DELETE'}, headers={'X-Timestamp': timestamp}) resp = req.get_response(self.object_controller) self.assertEquals(resp.status_int, 409) - objfile = os.path.join( + ts_1001_file = os.path.join( self.testdir, 'sda1', - storage_directory(object_server.DATADIR, 'p', + storage_directory(diskfile.DATADIR, 'p', hash_path('a', 'c', 'o')), timestamp + '.ts') - self.assertFalse(os.path.isfile(objfile)) - self.assertEquals(len(os.listdir(os.path.dirname(objfile))), 1) + self.assertFalse(os.path.isfile(ts_1001_file)) + self.assertTrue(os.path.isfile(data_1002_file)) + self.assertEquals(len(os.listdir(os.path.dirname(ts_1001_file))), 1) - sleep(.00001) - timestamp = normalize_timestamp(time()) + timestamp = normalize_timestamp(1003) req = Request.blank('/sda1/p/a/c/o', environ={'REQUEST_METHOD': 'DELETE'}, headers={'X-Timestamp': timestamp}) resp = req.get_response(self.object_controller) self.assertEquals(resp.status_int, 204) - objfile = os.path.join( + ts_1003_file = os.path.join( self.testdir, 'sda1', - storage_directory(object_server.DATADIR, 'p', + storage_directory(diskfile.DATADIR, 'p', hash_path('a', 'c', 'o')), timestamp + '.ts') - self.assert_(os.path.isfile(objfile)) + self.assertTrue(os.path.isfile(ts_1003_file)) + self.assertEquals(len(os.listdir(os.path.dirname(ts_1003_file))), 1) def test_DELETE_container_updates(self): # Test swift.obj.server.ObjectController.DELETE and container @@ -1243,7 +1255,7 @@ class TestObjectController(unittest.TestCase): self.assertEquals(resp.status_int, 409) objfile = os.path.join( self.testdir, 'sda1', - storage_directory(object_server.DATADIR, 'p', + storage_directory(diskfile.DATADIR, 'p', hash_path('a', 'c', 'o')), timestamp + '.ts') self.assertFalse(os.path.isfile(objfile)) @@ -1263,7 +1275,7 @@ class TestObjectController(unittest.TestCase): self.assertEquals(resp.status_int, 204) objfile = os.path.join( self.testdir, 'sda1', - storage_directory(object_server.DATADIR, 'p', + storage_directory(diskfile.DATADIR, 'p', hash_path('a', 'c', 'o')), timestamp + '.ts') self.assert_(os.path.isfile(objfile)) @@ -1283,7 +1295,7 @@ class TestObjectController(unittest.TestCase): self.assertEquals(resp.status_int, 404) objfile = os.path.join( self.testdir, 'sda1', - storage_directory(object_server.DATADIR, 'p', + storage_directory(diskfile.DATADIR, 'p', hash_path('a', 'c', 'o')), timestamp + '.ts') self.assert_(os.path.isfile(objfile)) @@ -1302,7 +1314,7 @@ class TestObjectController(unittest.TestCase): self.assertEquals(resp.status_int, 404) objfile = os.path.join( self.testdir, 'sda1', - storage_directory(object_server.DATADIR, 'p', + storage_directory(diskfile.DATADIR, 'p', hash_path('a', 'c', 'o')), timestamp + '.ts') self.assertFalse(os.path.isfile(objfile)) @@ -1311,7 +1323,7 @@ class TestObjectController(unittest.TestCase): finally: self.object_controller.container_update = orig_cu - def test_call(self): + def test_call_bad_request(self): # Test swift.obj.server.ObjectController.__call__ inbuf = StringIO() errbuf = StringIO() @@ -1339,9 +1351,15 @@ class TestObjectController(unittest.TestCase): self.assertEquals(errbuf.getvalue(), '') self.assertEquals(outbuf.getvalue()[:4], '400 ') + def test_call_not_found(self): inbuf = StringIO() errbuf = StringIO() outbuf = StringIO() + + def start_response(*args): + """Sends args to outbuf""" + outbuf.writelines(args) + self.object_controller.__call__({'REQUEST_METHOD': 'GET', 'SCRIPT_NAME': '', 'PATH_INFO': '/sda1/p/a/c/o', @@ -1360,9 +1378,15 @@ class TestObjectController(unittest.TestCase): self.assertEquals(errbuf.getvalue(), '') self.assertEquals(outbuf.getvalue()[:4], '404 ') + def test_call_bad_method(self): inbuf = StringIO() errbuf = StringIO() outbuf = StringIO() + + def start_response(*args): + """Sends args to outbuf""" + outbuf.writelines(args) + self.object_controller.__call__({'REQUEST_METHOD': 'INVALID', 'SCRIPT_NAME': '', 'PATH_INFO': '/sda1/p/a/c/o', @@ -1381,66 +1405,73 @@ class TestObjectController(unittest.TestCase): self.assertEquals(errbuf.getvalue(), '') self.assertEquals(outbuf.getvalue()[:4], '405 ') + def test_call_name_collision(self): def my_check(*args): return False - def my_storage_directory(*args): - return os.path.join(self.testdir, 'collide') + def my_hash_path(*args): + return md5('collide').hexdigest() - _storage_directory = diskfile.storage_directory - _check = object_server.check_object_creation - try: - diskfile.storage_directory = my_storage_directory - object_server.check_object_creation = my_check - inbuf = StringIO() - errbuf = StringIO() - outbuf = StringIO() - self.object_controller.__call__({'REQUEST_METHOD': 'PUT', - 'SCRIPT_NAME': '', - 'PATH_INFO': '/sda1/p/a/c/o', - 'SERVER_NAME': '127.0.0.1', - 'SERVER_PORT': '8080', - 'SERVER_PROTOCOL': 'HTTP/1.0', - 'CONTENT_LENGTH': '0', - 'CONTENT_TYPE': 'text/html', - 'HTTP_X_TIMESTAMP': '1.2', - 'wsgi.version': (1, 0), - 'wsgi.url_scheme': 'http', - 'wsgi.input': inbuf, - 'wsgi.errors': errbuf, - 'wsgi.multithread': False, - 'wsgi.multiprocess': False, - 'wsgi.run_once': False}, - start_response) - self.assertEquals(errbuf.getvalue(), '') - self.assertEquals(outbuf.getvalue()[:4], '201 ') + with mock.patch("swift.obj.diskfile.hash_path", my_hash_path): + with mock.patch("swift.obj.server.check_object_creation", + my_check): + inbuf = StringIO() + errbuf = StringIO() + outbuf = StringIO() - inbuf = StringIO() - errbuf = StringIO() - outbuf = StringIO() - self.object_controller.__call__({'REQUEST_METHOD': 'PUT', - 'SCRIPT_NAME': '', - 'PATH_INFO': '/sda1/q/b/d/x', - 'SERVER_NAME': '127.0.0.1', - 'SERVER_PORT': '8080', - 'SERVER_PROTOCOL': 'HTTP/1.0', - 'CONTENT_LENGTH': '0', - 'CONTENT_TYPE': 'text/html', - 'HTTP_X_TIMESTAMP': '1.3', - 'wsgi.version': (1, 0), - 'wsgi.url_scheme': 'http', - 'wsgi.input': inbuf, - 'wsgi.errors': errbuf, - 'wsgi.multithread': False, - 'wsgi.multiprocess': False, - 'wsgi.run_once': False}, - start_response) - self.assertEquals(errbuf.getvalue(), '') - self.assertEquals(outbuf.getvalue()[:4], '403 ') + def start_response(*args): + """Sends args to outbuf""" + outbuf.writelines(args) - finally: - diskfile.storage_directory = _storage_directory - object_server.check_object_creation = _check + self.object_controller.__call__({ + 'REQUEST_METHOD': 'PUT', + 'SCRIPT_NAME': '', + 'PATH_INFO': '/sda1/p/a/c/o', + 'SERVER_NAME': '127.0.0.1', + 'SERVER_PORT': '8080', + 'SERVER_PROTOCOL': 'HTTP/1.0', + 'CONTENT_LENGTH': '0', + 'CONTENT_TYPE': 'text/html', + 'HTTP_X_TIMESTAMP': normalize_timestamp(1.2), + 'wsgi.version': (1, 0), + 'wsgi.url_scheme': 'http', + 'wsgi.input': inbuf, + 'wsgi.errors': errbuf, + 'wsgi.multithread': False, + 'wsgi.multiprocess': False, + 'wsgi.run_once': False}, + start_response) + self.assertEquals(errbuf.getvalue(), '') + self.assertEquals(outbuf.getvalue()[:4], '201 ') + + inbuf = StringIO() + errbuf = StringIO() + outbuf = StringIO() + + def start_response(*args): + """Sends args to outbuf""" + outbuf.writelines(args) + + self.object_controller.__call__({ + 'REQUEST_METHOD': 'PUT', + 'SCRIPT_NAME': '', + 'PATH_INFO': '/sda1/p/b/d/x', + 'SERVER_NAME': '127.0.0.1', + 'SERVER_PORT': '8080', + 'SERVER_PROTOCOL': 'HTTP/1.0', + 'CONTENT_LENGTH': '0', + 'CONTENT_TYPE': 'text/html', + 'HTTP_X_TIMESTAMP': normalize_timestamp(1.3), + 'wsgi.version': (1, 0), + 'wsgi.url_scheme': 'http', + 'wsgi.input': inbuf, + 'wsgi.errors': errbuf, + 'wsgi.multithread': False, + 'wsgi.multiprocess': False, + 'wsgi.run_once': False}, + start_response) + self.assertEquals(errbuf.getvalue(), '') + self.assertEquals(outbuf.getvalue()[:4], '403 ') def test_invalid_method_doesnt_exist(self): errbuf = StringIO() @@ -1448,9 +1479,10 @@ class TestObjectController(unittest.TestCase): def start_response(*args): outbuf.writelines(args) - self.object_controller.__call__( - {'REQUEST_METHOD': 'method_doesnt_exist', - 'PATH_INFO': '/sda1/p/a/c/o'}, + + self.object_controller.__call__({ + 'REQUEST_METHOD': 'method_doesnt_exist', + 'PATH_INFO': '/sda1/p/a/c/o'}, start_response) self.assertEquals(errbuf.getvalue(), '') self.assertEquals(outbuf.getvalue()[:4], '405 ') @@ -1461,6 +1493,7 @@ class TestObjectController(unittest.TestCase): def start_response(*args): outbuf.writelines(args) + self.object_controller.__call__({'REQUEST_METHOD': '__init__', 'PATH_INFO': '/sda1/p/a/c/o'}, start_response) @@ -1476,9 +1509,10 @@ class TestObjectController(unittest.TestCase): fd = sock.makefile() fd.write('PUT /sda1/p/a/c/o HTTP/1.1\r\nHost: localhost\r\n' 'Content-Type: text/plain\r\n' - 'Connection: close\r\nX-Timestamp: 1.0\r\n' + 'Connection: close\r\nX-Timestamp: %s\r\n' 'Transfer-Encoding: chunked\r\n\r\n' - '2\r\noh\r\n4\r\n hai\r\n0\r\n\r\n') + '2\r\noh\r\n4\r\n hai\r\n0\r\n\r\n' % normalize_timestamp( + 1.0)) fd.flush() headers = readuntil2crlfs(fd) exp = 'HTTP/1.1 201' @@ -1504,10 +1538,11 @@ class TestObjectController(unittest.TestCase): fd = sock.makefile() fd.write('PUT /sda1/p/a/c/o HTTP/1.1\r\nHost: localhost\r\n' 'Content-Type: text/plain\r\n' - 'Connection: close\r\nX-Timestamp: 1.0\r\n' + 'Connection: close\r\nX-Timestamp: %s\r\n' 'Content-Length: 0\r\n' 'Transfer-Encoding: chunked\r\n\r\n' - '2\r\noh\r\n4\r\n hai\r\n0\r\n\r\n') + '2\r\noh\r\n4\r\n hai\r\n0\r\n\r\n' % normalize_timestamp( + 1.0)) fd.flush() headers = readuntil2crlfs(fd) exp = 'HTTP/1.1 201' @@ -1657,7 +1692,7 @@ class TestObjectController(unittest.TestCase): self.assertEquals(resp.status_int, 201) objfile = os.path.join( self.testdir, 'sda1', - storage_directory(object_server.DATADIR, 'p', + storage_directory(diskfile.DATADIR, 'p', hash_path('a', 'c', 'o')), timestamp + '.data') self.assert_(os.path.isfile(objfile)) @@ -1686,7 +1721,7 @@ class TestObjectController(unittest.TestCase): resp = req.get_response(self.object_controller) objfile = os.path.join( self.testdir, 'sda1', - storage_directory(object_server.DATADIR, 'p', + storage_directory(diskfile.DATADIR, 'p', hash_path('a', 'c', 'o')), timestamp + '.data') self.assert_(os.path.isfile(objfile)) @@ -1724,7 +1759,9 @@ class TestObjectController(unittest.TestCase): def fake_http_connect(ipaddr, port, device, partition, method, path, headers=None, query_string=None, ssl=False): + class SuccessfulFakeConn(object): + @property def status(self): return 200 @@ -1828,7 +1865,9 @@ class TestObjectController(unittest.TestCase): def fake_http_connect(ipaddr, port, device, partition, method, path, headers=None, query_string=None, ssl=False): + class SuccessfulFakeConn(object): + @property def status(self): return 200 @@ -2708,22 +2747,21 @@ class TestObjectController(unittest.TestCase): def my_tpool_execute(func, *args, **kwargs): return func(*args, **kwargs) - was_get_hashes = object_server.get_hashes - object_server.get_hashes = fake_get_hashes + was_get_hashes = diskfile.get_hashes was_tpool_exe = tpool.execute - tpool.execute = my_tpool_execute try: - req = Request.blank( - '/sda1/p/suff', - environ={'REQUEST_METHOD': 'REPLICATE'}, - headers={}) + diskfile.get_hashes = fake_get_hashes + tpool.execute = my_tpool_execute + req = Request.blank('/sda1/p/suff', + environ={'REQUEST_METHOD': 'REPLICATE'}, + headers={}) resp = req.get_response(self.object_controller) self.assertEquals(resp.status_int, 200) p_data = pickle.loads(resp.body) self.assertEquals(p_data, {1: 2}) finally: tpool.execute = was_tpool_exe - object_server.get_hashes = was_get_hashes + diskfile.get_hashes = was_get_hashes def test_REPLICATE_timeout(self): @@ -2733,19 +2771,18 @@ class TestObjectController(unittest.TestCase): def my_tpool_execute(func, *args, **kwargs): return func(*args, **kwargs) - was_get_hashes = object_server.get_hashes - object_server.get_hashes = fake_get_hashes + was_get_hashes = diskfile.get_hashes was_tpool_exe = tpool.execute - tpool.execute = my_tpool_execute try: - req = Request.blank( - '/sda1/p/suff', - environ={'REQUEST_METHOD': 'REPLICATE'}, - headers={}) + diskfile.get_hashes = fake_get_hashes + tpool.execute = my_tpool_execute + req = Request.blank('/sda1/p/suff', + environ={'REQUEST_METHOD': 'REPLICATE'}, + headers={}) self.assertRaises(Timeout, self.object_controller.REPLICATE, req) finally: tpool.execute = was_tpool_exe - object_server.get_hashes = was_get_hashes + diskfile.get_hashes = was_get_hashes def test_PUT_with_full_drive(self): diff --git a/test/unit/obj/test_updater.py b/test/unit/obj/test_updater.py index 4dbbac0599..eb9f07c0ac 100644 --- a/test/unit/obj/test_updater.py +++ b/test/unit/obj/test_updater.py @@ -24,8 +24,8 @@ from distutils.dir_util import mkpath from eventlet import spawn, Timeout, listen -from swift.obj import updater as object_updater, server as object_server -from swift.obj.server import ASYNCDIR +from swift.obj import updater as object_updater +from swift.obj.diskfile import ASYNCDIR from swift.common.ring import RingData from swift.common import utils from swift.common.utils import hash_path, normalize_timestamp, mkdirs, \ @@ -123,7 +123,7 @@ class TestObjectUpdater(unittest.TestCase): 'concurrency': '1', 'node_timeout': '15'}) cu.run_once() - async_dir = os.path.join(self.sda1, object_server.ASYNCDIR) + async_dir = os.path.join(self.sda1, ASYNCDIR) os.mkdir(async_dir) cu.run_once() self.assert_(os.path.exists(async_dir)) diff --git a/test/unit/proxy/test_mem_server.py b/test/unit/proxy/test_mem_server.py new file mode 100644 index 0000000000..626cda8ee0 --- /dev/null +++ b/test/unit/proxy/test_mem_server.py @@ -0,0 +1,62 @@ +# Copyright (c) 2010-2013 OpenStack, LLC. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +# implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +import unittest + +from test.unit.proxy import test_server +from test.unit.proxy.test_server import teardown + +from swift.obj import mem_server + + +def setup(): + test_server.do_setup(mem_server) + + +class TestController(test_server.TestController): + pass + + +class TestProxyServer(test_server.TestProxyServer): + pass + + +class TestObjectController(test_server.TestObjectController): + pass + + +class TestContainerController(test_server.TestContainerController): + pass + + +class TestAccountController(test_server.TestAccountController): + pass + + +class TestAccountControllerFakeGetResponse( + test_server.TestAccountControllerFakeGetResponse): + pass + + +class TestSegmentedIterable(test_server.TestSegmentedIterable): + pass + + +if __name__ == '__main__': + setup() + try: + unittest.main() + finally: + teardown() diff --git a/test/unit/proxy/test_server.py b/test/unit/proxy/test_server.py index b59c5d7b35..071f62ae11 100644 --- a/test/unit/proxy/test_server.py +++ b/test/unit/proxy/test_server.py @@ -71,7 +71,7 @@ def request_init(self, *args, **kwargs): _request_instances[self] = None -def setup(): +def do_setup(the_object_server): utils.HASH_PATH_SUFFIX = 'endcap' global _testdir, _test_servers, _test_sockets, \ _orig_container_listing_limit, _test_coros, _orig_SysLogHandler @@ -135,8 +135,8 @@ def setup(): acc2srv = account_server.AccountController(conf) con1srv = container_server.ContainerController(conf) con2srv = container_server.ContainerController(conf) - obj1srv = object_server.ObjectController(conf) - obj2srv = object_server.ObjectController(conf) + obj1srv = the_object_server.ObjectController(conf) + obj2srv = the_object_server.ObjectController(conf) _test_servers = \ (prosrv, acc1srv, acc2srv, con1srv, con2srv, obj1srv, obj2srv) nl = NullLogger() @@ -174,6 +174,10 @@ def setup(): exp, headers[:len(exp)]) +def setup(): + do_setup(object_server) + + def teardown(): for server in _test_coros: server.kill()