summaryrefslogtreecommitdiffstats
path: root/gluster
diff options
context:
space:
mode:
Diffstat (limited to 'gluster')
-rw-r--r--gluster/swift/common/DiskDir.py90
-rw-r--r--gluster/swift/common/Glusterfs.py2
-rw-r--r--gluster/swift/common/middleware/gswauth/swauth/middleware.py1
-rw-r--r--gluster/swift/common/utils.py41
-rw-r--r--gluster/swift/obj/diskfile.py137
-rw-r--r--gluster/swift/obj/server.py155
-rw-r--r--gluster/swift/proxy/server.py2
7 files changed, 193 insertions, 235 deletions
diff --git a/gluster/swift/common/DiskDir.py b/gluster/swift/common/DiskDir.py
index 6112709..e8dba35 100644
--- a/gluster/swift/common/DiskDir.py
+++ b/gluster/swift/common/DiskDir.py
@@ -255,7 +255,7 @@ class DiskDir(DiskCommon):
:param uid: user ID container object should assume
:param gid: group ID container object should assume
- Usage pattern from container/server.py (Havana, 1.8.0+):
+ Usage pattern from container/server.py (Kilo, 2.3.0):
DELETE:
if auto-create and obj and not .db_file:
# Creates container
@@ -287,36 +287,43 @@ class DiskDir(DiskCommon):
return 404
.put_object()
else:
- if not .db_file:
- # Creates container
- .initialize()
- else:
- # Update container timestamp
- .is_deleted()
+ _update_or_create():
+ if not .db_file:
+ # Creates container
+ .initialize()
+ recreated = .is_deleted():
+ if recreated:
+ .set_storage_policy_index()
+ .storage_policy_index
.update_put_timestamp()
if .is_deleted()
return conflict
- if metadata:
+ if recreated:
+ .update_status_changed_at()
+
+ if 'X-Container-Sync-To' in metadata:
if .metadata
.set_x_container_sync_points()
.update_metadata()
account_update():
.get_info()
HEAD:
- .pending_timeout
- .stale_reads_ok
- if .is_deleted():
- return 404
- .get_info()
+ info, is_deleted = .get_info_is_deleted()
+ .get_info_is_deleted():
+ if not .db_file:
+ return {}, True
+ info = .get_info()
+ return info, ._is_deleted_info()
.metadata
GET:
- .pending_timeout
- .stale_reads_ok
- if .is_deleted():
- return 404
- .get_info()
- .metadata
+ info, is_deleted = .get_info_is_deleted()
+ .get_info_is_deleted():
+ if not .db_file:
+ return {}, True
+ info = .get_info()
+ return info, ._is_deleted_info()
.list_objects_iter()
+ .metadata
POST:
if .is_deleted():
return 404
@@ -346,8 +353,22 @@ class DiskDir(DiskCommon):
create_container_metadata(self.datadir)
self.metadata = _read_metadata(self.datadir)
+ def update_status_changed_at(self, timestamp):
+ return
+
+ @property
+ def storage_policy_index(self):
+ if not hasattr(self, '_storage_policy_index'):
+ self._storage_policy_index = \
+ self.get_info()['storage_policy_index']
+ return self._storage_policy_index
+
+ def set_storage_policy_index(self, policy_index, timestamp=None):
+ self._storage_policy_index = policy_index
+
def list_objects_iter(self, limit, marker, end_marker,
- prefix, delimiter, path=None):
+ prefix, delimiter, path=None,
+ storage_policy_index=0):
"""
Returns tuple of name, created_at, size, content_type, etag.
"""
@@ -451,6 +472,12 @@ class DiskDir(DiskCommon):
return objects
+ def get_info_is_deleted(self):
+ if not do_exists(self.datadir):
+ return {}, True
+ info = self.get_info()
+ return info, False
+
def get_info(self):
"""
Get global data for the container.
@@ -477,7 +504,10 @@ class DiskDir(DiskCommon):
'x_container_sync_point1', -1),
'x_container_sync_point2': self.metadata.get(
'x_container_sync_point2', -1),
+ 'storage_policy_index': self.metadata.get(
+ 'storage_policy_index', 0)
}
+ self._storage_policy_index = data['storage_policy_index']
return data
def put_object(self, name, timestamp, size, content_type, etag, deleted=0):
@@ -540,13 +570,14 @@ class DiskDir(DiskCommon):
class DiskAccount(DiskCommon):
"""
- Usage pattern from account/server.py (Havana, 1.8.0+):
+ Usage pattern from account/server.py (Kilo, 2.3.0):
DELETE:
.is_deleted()
+ .is_status_deleted()
.delete_db()
+ .is_status_deleted()
PUT:
container:
- .pending_timeout
.db_file
.initialize()
.is_deleted()
@@ -555,25 +586,27 @@ class DiskAccount(DiskCommon):
.db_file
.initialize()
.is_status_deleted()
+ .is_status_deleted()
.is_deleted()
.update_put_timestamp()
- .is_deleted() ???
+ .is_deleted()
.update_metadata()
HEAD:
- .pending_timeout
- .stale_reads_ok
.is_deleted()
+ .is_status_deleted()
.get_info()
+ .get_policy_stats()
.metadata
GET:
- .pending_timeout
- .stale_reads_ok
.is_deleted()
+ .is_status_deleted()
.get_info()
+ .get_policy_stats()
.metadata
.list_containers_iter()
POST:
.is_deleted()
+ .is_status_deleted()
.update_metadata()
"""
@@ -748,3 +781,6 @@ class DiskAccount(DiskCommon):
'bytes_used': self.metadata.get(X_BYTES_USED, (0, 0))[0],
'hash': '', 'id': ''}
return data
+
+ def get_policy_stats(self, do_migrations=False):
+ return {}
diff --git a/gluster/swift/common/Glusterfs.py b/gluster/swift/common/Glusterfs.py
index 6a2fdb2..910f752 100644
--- a/gluster/swift/common/Glusterfs.py
+++ b/gluster/swift/common/Glusterfs.py
@@ -148,7 +148,7 @@ def _get_unique_id():
# own the lock.
continue
raise
- except:
+ except Exception:
os.close(fd)
raise
else:
diff --git a/gluster/swift/common/middleware/gswauth/swauth/middleware.py b/gluster/swift/common/middleware/gswauth/swauth/middleware.py
index cdcc638..3cd9cf7 100644
--- a/gluster/swift/common/middleware/gswauth/swauth/middleware.py
+++ b/gluster/swift/common/middleware/gswauth/swauth/middleware.py
@@ -28,6 +28,7 @@ import base64
from eventlet.timeout import Timeout
from eventlet import TimeoutError
+from swift import gettext_ as _
from swift.common.swob import HTTPAccepted, HTTPBadRequest, HTTPConflict, \
HTTPCreated, HTTPForbidden, HTTPMethodNotAllowed, HTTPMovedPermanently, \
HTTPNoContent, HTTPNotFound, HTTPUnauthorized, \
diff --git a/gluster/swift/common/utils.py b/gluster/swift/common/utils.py
index b6a5a09..e6f4bcc 100644
--- a/gluster/swift/common/utils.py
+++ b/gluster/swift/common/utils.py
@@ -17,6 +17,7 @@ import os
import stat
import json
import errno
+import random
import logging
from hashlib import md5
from eventlet import sleep
@@ -29,7 +30,7 @@ from swift.common.db import utf8encodekeys
from gluster.swift.common.fs_utils import do_getctime, do_getmtime, do_stat, \
do_listdir, do_walk, do_rmdir, do_log_rl, get_filename_from_fd, do_open, \
do_isdir, do_getsize, do_getxattr, do_setxattr, do_removexattr, do_read, \
- do_close, do_dup, do_lseek, do_fstat
+ do_close, do_dup, do_lseek, do_fstat, do_fsync, do_rename
from gluster.swift.common import Glusterfs
X_CONTENT_TYPE = 'Content-Type'
@@ -607,3 +608,41 @@ def rmobjdir(dir_path):
raise
else:
return True
+
+
+def write_pickle(obj, dest, tmp=None, pickle_protocol=0):
+ """
+ Ensure that a pickle file gets written to disk. The file is first written
+ to a tmp file location in the destination directory path, ensured it is
+ synced to disk, then moved to its final destination name.
+
+ This version takes advantage of Gluster's dot-prefix-dot-suffix naming
+ where the a file named ".thefile.name.9a7aasv" is hashed to the same
+ Gluster node as "thefile.name". This ensures the renaming of a temp file
+ once written does not move it to another Gluster node.
+
+ :param obj: python object to be pickled
+ :param dest: path of final destination file
+ :param tmp: path to tmp to use, defaults to None (ignored)
+ :param pickle_protocol: protocol to pickle the obj with, defaults to 0
+ """
+ dirname = os.path.dirname(dest)
+ # Create destination directory
+ try:
+ os.makedirs(dirname)
+ except OSError as err:
+ if err.errno != errno.EEXIST:
+ raise
+ basename = os.path.basename(dest)
+ tmpname = '.' + basename + '.' + \
+ md5(basename + str(random.random())).hexdigest()
+ tmppath = os.path.join(dirname, tmpname)
+ with open(tmppath, 'wb') as fo:
+ pickle.dump(obj, fo, pickle_protocol)
+ # TODO: This flush() method call turns into a flush() system call
+ # We'll need to wrap this as well, but we would do this by writing
+ # a context manager for our own open() method which returns an object
+ # in fo which makes the gluster API call.
+ fo.flush()
+ do_fsync(fo)
+ do_rename(tmppath, dest)
diff --git a/gluster/swift/obj/diskfile.py b/gluster/swift/obj/diskfile.py
index eb180a2..21e6cee 100644
--- a/gluster/swift/obj/diskfile.py
+++ b/gluster/swift/obj/diskfile.py
@@ -23,22 +23,18 @@ except ImportError:
import random
import logging
import time
-from collections import defaultdict
-from socket import gethostname
-from hashlib import md5
+from uuid import uuid4
from eventlet import sleep
-from greenlet import getcurrent
from contextlib import contextmanager
from gluster.swift.common.exceptions import AlreadyExistsAsFile, \
AlreadyExistsAsDir
-from swift.common.utils import TRUE_VALUES, ThreadPool, config_true_value
+from swift.common.utils import ThreadPool
from swift.common.exceptions import DiskFileNotExist, DiskFileError, \
DiskFileNoSpace, DiskFileDeviceUnavailable, DiskFileNotOpen, \
DiskFileExpired
from swift.common.swob import multi_range_iterator
from gluster.swift.common.exceptions import GlusterFileSystemOSError
-from gluster.swift.common.Glusterfs import mount
from gluster.swift.common.fs_utils import do_fstat, do_open, do_close, \
do_unlink, do_chown, do_fsync, do_fchown, do_stat, do_write, do_read, \
do_fadvise64, do_rename, do_fdatasync, do_lseek, do_mkdir
@@ -49,24 +45,15 @@ from gluster.swift.common.utils import X_CONTENT_TYPE, \
X_TIMESTAMP, X_TYPE, X_OBJECT_TYPE, FILE, OBJECT, DIR_TYPE, \
FILE_TYPE, DEFAULT_UID, DEFAULT_GID, DIR_NON_OBJECT, DIR_OBJECT, \
X_ETAG, X_CONTENT_LENGTH
-from ConfigParser import ConfigParser, NoSectionError, NoOptionError
+from swift.obj.diskfile import DiskFileManager as SwiftDiskFileManager
# FIXME: Hopefully we'll be able to move to Python 2.7+ where O_CLOEXEC will
# be back ported. See http://www.python.org/dev/peps/pep-0433/
O_CLOEXEC = 02000000
-DEFAULT_DISK_CHUNK_SIZE = 65536
-DEFAULT_KEEP_CACHE_SIZE = (5 * 1024 * 1024)
-DEFAULT_MB_PER_SYNC = 512
-# keep these lower-case
-DISALLOWED_HEADERS = set('content-length content-type deleted etag'.split())
-
MAX_RENAME_ATTEMPTS = 10
MAX_OPEN_ATTEMPTS = 10
-_cur_pid = str(os.getpid())
-_cur_host = str(gethostname())
-
def _random_sleep():
sleep(random.uniform(0.5, 0.15))
@@ -181,24 +168,6 @@ def make_directory(full_path, uid, gid, metadata=None):
return True, metadata
-_fs_conf = ConfigParser()
-if _fs_conf.read(os.path.join('/etc/swift', 'fs.conf')):
- try:
- _mkdir_locking = _fs_conf.get('DEFAULT', 'mkdir_locking', "no") \
- in TRUE_VALUES
- logging.warn("The option mkdir_locking has been deprecated and is"
- " no longer supported")
- except (NoSectionError, NoOptionError):
- pass
- try:
- _use_put_mount = _fs_conf.get('DEFAULT', 'use_put_mount', "no") \
- in TRUE_VALUES
- except (NoSectionError, NoOptionError):
- _use_put_mount = False
-else:
- _use_put_mount = False
-
-
def _adjust_metadata(metadata):
# Fix up the metadata to ensure it has a proper value for the
# Content-Type metadata, as well as an X_TYPE and X_OBJECT_TYPE
@@ -223,7 +192,7 @@ def _adjust_metadata(metadata):
return metadata
-class OnDiskManager(object):
+class DiskFileManager(SwiftDiskFileManager):
"""
Management class for devices, providing common place for shared parameters
and methods not provided by the DiskFile class (which primarily services
@@ -240,42 +209,14 @@ class OnDiskManager(object):
:param conf: caller provided configuration object
:param logger: caller provided logger
"""
- def __init__(self, conf, logger):
- self.logger = logger
- self.disk_chunk_size = int(conf.get('disk_chunk_size',
- DEFAULT_DISK_CHUNK_SIZE))
- self.keep_cache_size = int(conf.get('keep_cache_size',
- DEFAULT_KEEP_CACHE_SIZE))
- self.bytes_per_sync = int(conf.get('mb_per_sync',
- DEFAULT_MB_PER_SYNC)) * 1024 * 1024
- self.devices = conf.get('devices', '/srv/node/')
- self.mount_check = config_true_value(conf.get('mount_check', 'true'))
- threads_per_disk = int(conf.get('threads_per_disk', '0'))
- self.threadpools = defaultdict(
- lambda: ThreadPool(nthreads=threads_per_disk))
-
- def _get_dev_path(self, device):
- """
- Return the path to a device, checking to see that it is a proper mount
- point based on a configuration parameter.
-
- :param device: name of target device
- :returns: full path to the device, None if the path to the device is
- not a proper mount point.
- """
- if self.mount_check and not mount(self.devices, device):
- dev_path = None
- else:
- dev_path = os.path.join(self.devices, device)
- return dev_path
-
- def get_diskfile(self, device, account, container, obj,
- **kwargs):
- dev_path = self._get_dev_path(device)
+ def get_diskfile(self, device, partition, account, container, obj,
+ policy=None, **kwargs):
+ dev_path = self.get_dev_path(device, self.mount_check)
if not dev_path:
raise DiskFileDeviceUnavailable()
return DiskFile(self, dev_path, self.threadpools[device],
- account, container, obj, **kwargs)
+ partition, account, container, obj,
+ policy=policy, **kwargs)
class DiskFileWriter(object):
@@ -447,10 +388,20 @@ class DiskFileWriter(object):
df._threadpool.force_run_in_thread(self._finalize_put, metadata)
- # Avoid the unlink() system call as part of the DiskFile.create()
- # context cleanup
+ # Avoid the unlink() system call as part of the mkstemp context
+ # cleanup
self._tmppath = None
+ def commit(self, timestamp):
+ """
+ Perform any operations necessary to mark the object as durable. For
+ replication policy type this is a no-op.
+
+ :param timestamp: object put timestamp, an instance of
+ :class:`~swift.common.utils.Timestamp`
+ """
+ pass
+
class DiskFileReader(object):
"""
@@ -579,9 +530,9 @@ class DiskFile(object):
Manage object files on disk.
Object names ending or beginning with a '/' as in /a, a/, /a/b/,
- etc, or object names with multiple consecutive slahes, like a//b,
- are not supported. The proxy server's contraints filter
- gluster.common.constrains.gluster_check_object_creation() should
+ etc, or object names with multiple consecutive slashes, like a//b,
+ are not supported. The proxy server's constraints filter
+ gluster.common.constrains.check_object_creation() should
reject such requests.
:param mgr: associated on-disk manager instance
@@ -593,36 +544,37 @@ class DiskFile(object):
:param uid: user ID disk object should assume (file or directory)
:param gid: group ID disk object should assume (file or directory)
"""
- def __init__(self, mgr, dev_path, threadpool, account, container, obj,
- uid=DEFAULT_UID, gid=DEFAULT_GID):
+ def __init__(self, mgr, dev_path, threadpool, partition,
+ account=None, container=None, obj=None,
+ policy=None, uid=DEFAULT_UID, gid=DEFAULT_GID):
+ # Variables partition and policy is currently unused.
self._mgr = mgr
self._device_path = dev_path
self._threadpool = threadpool or ThreadPool(nthreads=0)
self._uid = int(uid)
self._gid = int(gid)
self._is_dir = False
- self._logger = mgr.logger
self._metadata = None
self._fd = None
self._stat = None
# Don't store a value for data_file until we know it exists.
self._data_file = None
- self._container_path = os.path.join(self._device_path, container)
+ self._account = account # Unused, account = volume
+ self._container = container
+
+ self._container_path = os.path.join(self._device_path, self._container)
+
obj = obj.strip(os.path.sep)
obj_path, self._obj = os.path.split(obj)
if obj_path:
self._obj_path = obj_path.strip(os.path.sep)
- self._datadir = os.path.join(self._container_path, self._obj_path)
+ self._put_datadir = os.path.join(self._container_path,
+ self._obj_path)
else:
self._obj_path = ''
- self._datadir = self._container_path
+ self._put_datadir = self._container_path
- if _use_put_mount:
- self._put_datadir = os.path.join(
- self._device_path + '_PUT', container, self._obj_path)
- else:
- self._put_datadir = self._datadir
self._data_file = os.path.join(self._put_datadir, self._obj)
def open(self):
@@ -687,7 +639,8 @@ class DiskFile(object):
raise DiskFileNotExist
else:
# Re-raise the original exception after fd has been closed
- raise err
+ raise
+
return self
def _is_object_expired(self, metadata):
@@ -874,16 +827,14 @@ class DiskFile(object):
:raises AlreadyExistsAsFile: if path or part of a path is not a \
directory
"""
+
data_file = os.path.join(self._put_datadir, self._obj)
# Assume the full directory path exists to the file already, and
# construct the proper name for the temporary file.
attempts = 1
- cur_thread = str(getcurrent())
while True:
- postfix = md5(self._obj + _cur_host + _cur_pid + cur_thread
- + str(random.random())).hexdigest()
- tmpfile = '.' + self._obj + '.' + postfix
+ tmpfile = '.' + self._obj + '.' + uuid4().hex
tmppath = os.path.join(self._put_datadir, tmpfile)
try:
fd = do_open(tmppath,
@@ -905,7 +856,7 @@ class DiskFile(object):
if attempts >= MAX_OPEN_ATTEMPTS:
# We failed after N attempts to create the temporary
# file.
- raise DiskFileError('DiskFile.create(): failed to'
+ raise DiskFileError('DiskFile.mkstemp(): failed to'
' successfully create a temporary file'
' without running into a name conflict'
' after %d of %d attempts for: %s' % (
@@ -918,7 +869,7 @@ class DiskFile(object):
# FIXME: Possible FUSE issue or race condition, let's
# sleep on it and retry the operation.
_random_sleep()
- logging.warn("DiskFile.create(): %s ... retrying in"
+ logging.warn("DiskFile.mkstemp(): %s ... retrying in"
" 0.1 secs", gerr)
attempts += 1
elif not self._obj_path:
@@ -927,7 +878,7 @@ class DiskFile(object):
# could be a FUSE issue or some race condition, so let's
# sleep a bit and retry.
_random_sleep()
- logging.warn("DiskFile.create(): %s ... retrying in"
+ logging.warn("DiskFile.mkstemp(): %s ... retrying in"
" 0.1 secs", gerr)
attempts += 1
elif attempts > 1:
@@ -935,7 +886,7 @@ class DiskFile(object):
# also be a FUSE issue or some race condition, nap and
# retry.
_random_sleep()
- logging.warn("DiskFile.create(): %s ... retrying in"
+ logging.warn("DiskFile.mkstemp(): %s ... retrying in"
" 0.1 secs" % gerr)
attempts += 1
else:
diff --git a/gluster/swift/obj/server.py b/gluster/swift/obj/server.py
index 8bc080a..1d8d418 100644
--- a/gluster/swift/obj/server.py
+++ b/gluster/swift/obj/server.py
@@ -15,26 +15,27 @@
""" Object Server for Gluster for Swift """
-# Simply importing this monkey patches the constraint handling to fit our
-# needs
-import gluster.swift.common.constraints # noqa
-from swift.common.swob import HTTPConflict
-from swift.common.utils import public, timing_stats
+from swift.common.swob import HTTPConflict, HTTPNotImplemented
+from swift.common.utils import public, timing_stats, replication
from gluster.swift.common.exceptions import AlreadyExistsAsFile, \
AlreadyExistsAsDir
from swift.common.request_helpers import split_and_validate_path
from swift.obj import server
-from gluster.swift.obj.diskfile import OnDiskManager
+from gluster.swift.obj.diskfile import DiskFileManager
-import os
-from swift.common.exceptions import ConnectionTimeout
-from swift.common.bufferedhttp import http_connect
-from eventlet import Timeout
-from swift.common.http import is_success
-from gluster.swift.common.ring import Ring
-from swift import gettext_ as _
+
+class GlusterSwiftDiskFileRouter(object):
+ """
+ Replacement for Swift's DiskFileRouter object.
+ Always returns GlusterSwift's DiskFileManager implementation.
+ """
+ def __init__(self, *args, **kwargs):
+ self.manager_cls = DiskFileManager(*args, **kwargs)
+
+ def __getitem__(self, policy):
+ return self.manager_cls
class ObjectController(server.ObjectController):
@@ -52,23 +53,8 @@ class ObjectController(server.ObjectController):
:param conf: WSGI configuration parameter
"""
- # Common on-disk hierarchy shared across account, container and object
- # servers.
- self._ondisk_mgr = OnDiskManager(conf, self.logger)
- self.swift_dir = conf.get('swift_dir', '/etc/swift')
-
- 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._ondisk_mgr.get_diskfile(device, account, container, obj,
- **kwargs)
+ # Replaces Swift's DiskFileRouter object reference with ours.
+ self._diskfile_router = GlusterSwiftDiskFileRouter(conf, self.logger)
def container_update(self, *args, **kwargs):
"""
@@ -79,102 +65,47 @@ class ObjectController(server.ObjectController):
"""
return
- def get_object_ring(self):
- if hasattr(self, 'object_ring'):
- if not self.object_ring:
- self.object_ring = Ring(self.swift_dir, ring_name='object')
- else:
- self.object_ring = Ring(self.swift_dir, ring_name='object')
- return self.object_ring
-
- def async_update(self, op, account, container, obj, host, partition,
- contdevice, headers_out, objdevice):
- """
- In Openstack Swift, this method is called by:
- * container_update (a no-op in gluster-swift)
- * delete_at_update (to PUT objects into .expiring_objects account)
-
- The Swift's version of async_update only sends the request to
- container-server to PUT the object. The container-server calls
- container_update method which makes an entry for the object in it's
- database. No actual object is created on disk.
-
- But in gluster-swift container_update is a no-op, so we'll
- have to PUT an actual object. We override async_update to create a
- container first and then the corresponding "tracker object" which
- tracks expired objects scheduled for deletion.
+ def delete_at_update(self, *args, **kwargs):
"""
+ Update the expiring objects container when objects are updated.
- headers_out['user-agent'] = 'obj-server %s' % os.getpid()
- if all([host, partition, contdevice]):
- # PUT the container. Send request directly to container-server
- container_path = '/%s/%s' % (account, container)
- try:
- with ConnectionTimeout(self.conn_timeout):
- ip, port = host.rsplit(':', 1)
- conn = http_connect(ip, port, contdevice, partition, op,
- container_path, headers_out)
- with Timeout(self.node_timeout):
- response = conn.getresponse()
- response.read()
- if not is_success(response.status):
- self.logger.error(_(
- 'async_update : '
- 'ERROR Container update failed :%(status)d '
- 'response from %(ip)s:%(port)s/%(dev)s'),
- {'status': response.status, 'ip': ip, 'port': port,
- 'dev': contdevice})
- return
- except (Exception, Timeout):
- self.logger.exception(_(
- 'async_update : '
- 'ERROR Container update failed :%(ip)s:%(port)s/%(dev)s'),
- {'ip': ip, 'port': port, 'dev': contdevice})
-
- # PUT the tracker object. Send request directly to object-server
- object_path = '/%s/%s/%s' % (account, container, obj)
- headers_out['Content-Length'] = 0
- headers_out['Content-Type'] = 'text/plain'
- try:
- with ConnectionTimeout(self.conn_timeout):
- # FIXME: Assuming that get_nodes returns single node
- part, nodes = self.get_object_ring().get_nodes(account,
- container,
- obj)
- ip = nodes[0]['ip']
- port = nodes[0]['port']
- objdevice = nodes[0]['device']
- conn = http_connect(ip, port, objdevice, partition, op,
- object_path, headers_out)
- with Timeout(self.node_timeout):
- response = conn.getresponse()
- response.read()
- if is_success(response.status):
- return
- else:
- self.logger.error(_(
- 'async_update : '
- 'ERROR Object PUT failed : %(status)d '
- 'response from %(ip)s:%(port)s/%(dev)s'),
- {'status': response.status, 'ip': ip, 'port': port,
- 'dev': objdevice})
- except (Exception, Timeout):
- self.logger.exception(_(
- 'async_update : '
- 'ERROR Object PUT failed :%(ip)s:%(port)s/%(dev)s'),
- {'ip': ip, 'port': port, 'dev': objdevice})
+ For Gluster, this is a no-op as there are no container DB entries
+ to be created that tracks objects to be expired. Objects to be
+ expired will be determined by crawling the filesystem directly.
+ """
return
@public
@timing_stats()
def PUT(self, request):
try:
+ # now call swift's PUT method
return server.ObjectController.PUT(self, request)
except (AlreadyExistsAsFile, AlreadyExistsAsDir):
device = \
split_and_validate_path(request, 1, 5, True)
return HTTPConflict(drive=device, request=request)
+ @public
+ @replication
+ @timing_stats(sample_rate=0.1)
+ def REPLICATE(self, request):
+ """
+ In Swift, this method handles REPLICATE requests for the Swift
+ Object Server. This is used by the object replicator to get hashes
+ for directories.
+
+ Gluster-Swift does not support this as it expects the underlying
+ GlusterFS to take care of replication
+ """
+ return HTTPNotImplemented(request=request)
+
+ @public
+ @replication
+ @timing_stats(sample_rate=0.1)
+ def REPLICATION(self, request):
+ return HTTPNotImplemented(request=request)
+
def app_factory(global_conf, **local_conf):
"""paste.deploy app factory for creating WSGI object server apps"""
diff --git a/gluster/swift/proxy/server.py b/gluster/swift/proxy/server.py
index 7b2f58e..1621990 100644
--- a/gluster/swift/proxy/server.py
+++ b/gluster/swift/proxy/server.py
@@ -20,7 +20,7 @@ import gluster.swift.common.constraints # noqa
from swift.proxy.server import Application, mimetypes # noqa
from swift.proxy.controllers import AccountController # noqa
-from swift.proxy.controllers import ObjectController # noqa
+from swift.proxy.controllers import ObjectControllerRouter # noqa
from swift.proxy.controllers import ContainerController # noqa