DiskFile API, with reference implementation

Refactor on-disk knowledge out of the object server by pushing the
async update pickle creation to the new DiskFileManager class (name is
not the best, so suggestions welcome), along with the REPLICATOR
method logic. We also move the mount checking and thread pool storage
to the new ondisk.Devices object, which then also becomes the new home
of the audit_location_generator method.

For the object server, a new setup() method is now called at the end
of the controller's construction, and the _diskfile() method has been
renamed to get_diskfile(), to allow implementation specific behavior.

We then hide the need for the REST API layer to know how and where
quarantining needs to be performed. There are now two places it is
checked internally, on open() where we verify the content-length,
name, and x-timestamp metadata, and in the reader on close where the
etag metadata is checked if the entire file was read.

We add a reader class to allow implementations to isolate the WSGI
handling code for that specific environment (it is used no-where else
in the REST APIs). This simplifies the caller's code to just use a
"with" statement once open to avoid multiple points where close needs
to be called.

For a full historical comparison, including the usage patterns see:
https://gist.github.com/portante/5488238

(as of master, 2b639f5, Merge
 "Fix 500 from account-quota     This Commit
 middleware")
--------------------------------+------------------------------------
                                 DiskFileManager(conf)

                                   Methods:
                                     .pickle_async_update()
                                     .get_diskfile()
                                     .get_hashes()

                                   Attributes:
                                     .devices
                                     .logger
                                     .disk_chunk_size
                                     .keep_cache_size
                                     .bytes_per_sync

DiskFile(a,c,o,keep_data_fp=)    DiskFile(a,c,o)

  Methods:                         Methods:
   *.__iter__()
    .close(verify_file=)
    .is_deleted()
    .is_expired()
    .quarantine()
    .get_data_file_size()
                                     .open()
                                     .read_metadata()
    .create()                        .create()
                                     .write_metadata()
    .delete()                        .delete()

  Attributes:                      Attributes:
    .quarantined_dir
    .keep_cache
    .metadata
                                *DiskFileReader()

                                   Methods:
                                     .__iter__()
                                     .close()

                                   Attributes:
                                    +.was_quarantined

DiskWriter()                     DiskFileWriter()

  Methods:                         Methods:
    .write()                         .write()
    .put()                           .put()

* Note that the DiskFile class   * Note that the DiskReader() object
  implements all the methods       returned by the
  necessary for a WSGI app         DiskFileOpened.reader() method
  iterator                         implements all the methods
                                   necessary for a WSGI app iterator

                                 + Note that if the auditor is
                                   refactored to not use the DiskFile
                                   class, see
                                   https://review.openstack.org/44787
                                   then we don't need the
                                   was_quarantined attribute

A reference "in-memory" object server implementation of a backend
DiskFile class in swift/obj/mem_server.py and
swift/obj/mem_diskfile.py.

One can also reference
https://github.com/portante/gluster-swift/commits/diskfile for the
proposed integration with the gluster-swift code based on these
changes.

Change-Id: I44e153fdb405a5743e9c05349008f94136764916
Signed-off-by: Peter Portante <peter.portante@redhat.com>
This commit is contained in:
Peter Portante 2013-09-12 19:51:18 -04:00
parent 2b639f5ecc
commit 5202b0e586
23 changed files with 2249 additions and 1174 deletions

View File

@ -34,3 +34,12 @@ Account Reaper
:undoc-members:
:show-inheritance:
.. _account-backend:
Account Backend
===============
.. automodule:: swift.account.backend
:members:
:undoc-members:
:show-inheritance:

View File

@ -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:

View File

@ -41,3 +41,13 @@ Container Sync
:members:
:undoc-members:
:show-inheritance:
.. _container-backend:
Container Backend
=================
.. automodule:: swift.container.backend
:members:
:undoc-members:
:show-inheritance:

View File

@ -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:

View File

@ -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`

View File

@ -44,3 +44,12 @@ Object Auditor
:undoc-members:
:show-inheritance:
.. _object-diskfile:
Object Backend
==============
.. automodule:: swift.obj.diskfile
:members:
:undoc-members:
:show-inheritance:

View File

@ -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

View File

@ -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)

File diff suppressed because it is too large Load Diff

408
swift/obj/mem_diskfile.py Normal file
View File

@ -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)

111
swift/obj/mem_server.py Normal file
View File

@ -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)

View File

@ -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."""

View File

@ -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

View File

@ -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)

View File

@ -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)

View File

@ -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])

View File

@ -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):

View File

@ -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])

View File

@ -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')

View File

@ -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):

View File

@ -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))

View File

@ -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()

View File

@ -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()