summaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
-rwxr-xr-x.functests6
-rwxr-xr-xbin/gluster-swift-object-expirer33
-rw-r--r--etc/object-expirer.conf-gluster3
-rw-r--r--gluster/swift/common/DiskDir.py2
-rw-r--r--gluster/swift/common/utils.py27
-rw-r--r--gluster/swift/obj/expirer.py160
-rw-r--r--gluster/swift/obj/server.py78
-rw-r--r--glusterfs-openstack-swift.spec1
-rw-r--r--setup.py1
-rw-r--r--test/functional_auth/common_conf/object-expirer.conf8
-rw-r--r--test/object_expirer_functional/test_object_expirer.py31
-rw-r--r--test/unit/common/test_diskdir.py2
-rwxr-xr-xtools/object_expirer_functional.sh4
13 files changed, 314 insertions, 42 deletions
diff --git a/.functests b/.functests
index c129712..0090015 100755
--- a/.functests
+++ b/.functests
@@ -26,4 +26,10 @@ bash tools/tempauth_functional_tests.sh || EXIT_STATUS=$?
# Run functional tests with gswauth as auth middleware
bash tools/gswauth_functional_tests.sh || EXIT_STATUS=$?
+# Run object expirer functional tests if gsexpiring volume is found.
+if mount | grep "gsexpiring on /mnt/gluster-object/gsexpiring type fuse.glusterfs"; then
+ echo "Running object expirer functional tests"
+ bash tools/object_expirer_functional.sh || EXIT_STATUS=$?
+fi
+
exit $EXIT_STATUS
diff --git a/bin/gluster-swift-object-expirer b/bin/gluster-swift-object-expirer
new file mode 100755
index 0000000..b5db80f
--- /dev/null
+++ b/bin/gluster-swift-object-expirer
@@ -0,0 +1,33 @@
+#!/usr/bin/env python
+# Copyright (c) 2016 Red Hat
+#
+# 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.
+
+from swift.common.daemon import run_daemon
+from swift.common.utils import parse_options
+from gluster.swift.obj.expirer import ObjectExpirer
+from optparse import OptionParser
+
+if __name__ == '__main__':
+ parser = OptionParser("%prog CONFIG [options]")
+ parser.add_option('--processes', dest='processes',
+ help="Number of processes to use to do the work, don't "
+ "use this option to do all the work in one process")
+ parser.add_option('--process', dest='process',
+ help="Process number for this process, don't use "
+ "this option to do all the work in one process, this "
+ "is used to determine which part of the work this "
+ "process should do")
+ conf_file, options = parse_options(parser=parser, once=True)
+ run_daemon(ObjectExpirer, conf_file, **options)
diff --git a/etc/object-expirer.conf-gluster b/etc/object-expirer.conf-gluster
index 55c4fbf..32be5a1 100644
--- a/etc/object-expirer.conf-gluster
+++ b/etc/object-expirer.conf-gluster
@@ -1,5 +1,8 @@
[DEFAULT]
user = root
+# Default gluster mount point to be used for object store,can be changed by
+# setting the following value in {account,container,object}-server.conf files.
+devices = /mnt/gluster-object
[object-expirer]
user = root
diff --git a/gluster/swift/common/DiskDir.py b/gluster/swift/common/DiskDir.py
index 36504a6..d314a1f 100644
--- a/gluster/swift/common/DiskDir.py
+++ b/gluster/swift/common/DiskDir.py
@@ -559,7 +559,7 @@ class DiskDir(DiskCommon):
self.metadata[X_PUT_TIMESTAMP] = (timestamp, 0)
write_metadata(self.datadir, self.metadata)
- def delete_object(self, name, timestamp):
+ def delete_object(self, name, timestamp, obj_policy_index):
# NOOP - should never be called since object file removal occurs
# within a directory implicitly.
return
diff --git a/gluster/swift/common/utils.py b/gluster/swift/common/utils.py
index 8958717..1bbc56c 100644
--- a/gluster/swift/common/utils.py
+++ b/gluster/swift/common/utils.py
@@ -556,7 +556,7 @@ def dir_is_object(metadata):
return metadata.get(X_OBJECT_TYPE, "") == DIR_OBJECT
-def rmobjdir(dir_path):
+def rmobjdir(dir_path, marker_dir_check=True):
"""
Removes the directory as long as there are no objects stored in it. This
works for containers also.
@@ -580,18 +580,19 @@ def rmobjdir(dir_path):
for directory in dirs:
fullpath = os.path.join(path, directory)
- try:
- metadata = read_metadata(fullpath)
- except GlusterFileSystemIOError as err:
- if err.errno in (errno.ENOENT, errno.ESTALE):
- # Ignore removal from another entity.
- continue
- raise
- else:
- if dir_is_object(metadata):
- # Wait, this is an object created by the caller
- # We cannot delete
- return False
+ if marker_dir_check:
+ try:
+ metadata = read_metadata(fullpath)
+ except GlusterFileSystemIOError as err:
+ if err.errno in (errno.ENOENT, errno.ESTALE):
+ # Ignore removal from another entity.
+ continue
+ raise
+ else:
+ if dir_is_object(metadata):
+ # Wait, this is an object created by the caller
+ # We cannot delete
+ return False
# Directory is not an object created by the caller
# so we can go ahead and delete it.
diff --git a/gluster/swift/obj/expirer.py b/gluster/swift/obj/expirer.py
new file mode 100644
index 0000000..564a2c9
--- /dev/null
+++ b/gluster/swift/obj/expirer.py
@@ -0,0 +1,160 @@
+# Copyright (c) 2016 Red Hat
+#
+# 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.
+
+# This import will monkey-patch Ring and other classes.
+# Do not remove.
+import gluster.swift.common.constraints # noqa
+
+import errno
+import os
+
+from gluster.swift.common.utils import rmobjdir
+
+from swift.obj.expirer import ObjectExpirer as SwiftObjectExpirer
+from swift.common.http import HTTP_NOT_FOUND
+from swift.common.internal_client import InternalClient, UnexpectedResponse
+from swift.common.utils import ThreadPool
+
+EXCLUDE_DIRS = ('.trashcan', '.glusterfs')
+
+
+class GlusterSwiftInternalClient(InternalClient):
+
+ def __init__(self, conf_path, user_agent, request_tries,
+ allow_modify_pipeline=False, devices=None):
+ super(GlusterSwiftInternalClient, self).__init__(
+ conf_path, user_agent, request_tries, allow_modify_pipeline)
+ self.devices = devices
+
+ def get_account_info(self, account):
+ # Supposed to return container count and object count in gsexpiring
+ # account. This is used by object expirer only for logging.
+ return (0, 0)
+
+ def delete_container(self, account, container, acceptable_statuses=None):
+ container_path = os.path.join(self.devices, account, container)
+ try:
+ os.rmdir(container_path)
+ except OSError as err:
+ if err.errno != errno.ENOENT:
+ raise
+
+ def iter_containers(self, account):
+ account_path = os.path.join(self.devices, account)
+ for container in os.listdir(account_path):
+ if container in EXCLUDE_DIRS:
+ continue
+ container_path = os.path.join(account_path, container)
+ if os.path.isdir(container_path):
+ yield {'name': container.encode('utf8')}
+
+ def iter_objects(self, account, container):
+ container_path = os.path.join(self.devices, account, container)
+ # TODO: Use a slightly better implementation of os.walk()
+ for (root, dirs, files) in os.walk(container_path):
+ for f in files:
+ obj_path = os.path.join(root, f)
+ obj = obj_path[(len(container_path) + 1):]
+ yield {'name': obj.encode('utf8')}
+
+
+class ObjectExpirer(SwiftObjectExpirer):
+
+ def __init__(self, conf, logger=None, swift=None):
+
+ conf_path = conf.get('__file__') or '/etc/swift/object-expirer.conf'
+ self.devices = conf.get('devices', '/mnt/gluster-object')
+ # Do not retry DELETEs on getting 404. Hence default is set to 1.
+ request_tries = int(conf.get('request_tries') or 1)
+ # Use our extended version of InternalClient
+ swift = GlusterSwiftInternalClient(
+ conf_path, 'Gluster Swift Object Expirer', request_tries,
+ devices=self.devices)
+ # Let the parent class initialize self.swift
+ super(ObjectExpirer, self).__init__(conf, logger=logger, swift=swift)
+
+ self.reseller_prefix = conf.get('reseller_prefix', 'AUTH').strip()
+ if not self.reseller_prefix.endswith('_'):
+ self.reseller_prefix = self.reseller_prefix + '_'
+
+ # nthread=0 is intentional. This ensures that no green pool is
+ # used. Call to force_run_in_thread() will ensure that the method
+ # passed as arg is run in a real external thread using eventlet.tpool
+ # which has a threadpool of 20 threads (default)
+ self.threadpool = ThreadPool(nthreads=0)
+
+ def _delete_tracker_object(self, container, obj):
+ container_path = os.path.join(self.devices,
+ self.expiring_objects_account,
+ container)
+ tracker_object_path = os.path.join(container_path, obj)
+
+ try:
+ os.unlink(tracker_object_path)
+ except OSError as err:
+ if err.errno in (errno.ENOENT, errno.ESTALE):
+ # Ignore removal from another entity.
+ return
+ elif err.errno == errno.EISDIR:
+ # Handle race: Was a file during crawl, but now it's a
+ # directory. There are no 'directory marker' objects in
+ # gsexpiring volume.
+ return
+ else:
+ raise
+
+ # This part of code is very similar to DiskFile._unlinkold()
+ dirname = os.path.dirname(tracker_object_path)
+ while dirname and dirname != container_path:
+ if not rmobjdir(dirname, marker_dir_check=False):
+ # If a directory with objects has been found, we can stop
+ # garbage collection
+ break
+ else:
+ # Traverse upwards till the root of container
+ dirname = os.path.dirname(dirname)
+
+ def pop_queue(self, container, obj):
+ """
+ In Swift, this method removes tracker object entry directly from
+ container database. In gluster-swift, this method deletes tracker
+ object directly from filesystem.
+ """
+ self.threadpool.force_run_in_thread(self._delete_tracker_object,
+ container, obj)
+
+ def delete_actual_object(self, actual_obj, timestamp):
+ """
+ Swift's expirer will re-attempt expiring if the source object is not
+ available (404 or ANY other error) up to self.reclaim_age seconds
+ before it gives up and deletes the entry in the queue.
+
+ Don't do this in gluster-swift. GlusterFS isn't eventually consistent
+ and has no concept of hand-off nodes. If actual data object doesn't
+ exist (404), remove tracker object from the queue (filesystem).
+
+ However if DELETE fails due a reason other than 404, do not remove
+ tracker object yet, follow Swift's behaviour of waiting till
+ self.reclaim_age seconds.
+
+ This method is just a wrapper around parent class's method. All this
+ wrapper does is ignore 404 failures.
+ """
+ try:
+ super(ObjectExpirer, self).delete_actual_object(
+ actual_obj, timestamp)
+ except UnexpectedResponse as err:
+ if err.resp.status_int != HTTP_NOT_FOUND:
+ raise
diff --git a/gluster/swift/obj/server.py b/gluster/swift/obj/server.py
index 1d8d418..3e27cc3 100644
--- a/gluster/swift/obj/server.py
+++ b/gluster/swift/obj/server.py
@@ -14,16 +14,19 @@
# limitations under the License.
""" Object Server for Gluster for Swift """
+import errno
+import os
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.utils import public, timing_stats, replication, mkdirs
from swift.common.request_helpers import split_and_validate_path
-
from swift.obj import server
from gluster.swift.obj.diskfile import DiskFileManager
+from gluster.swift.common.fs_utils import do_ismount
+from gluster.swift.common.ring import Ring
+from gluster.swift.common.exceptions import AlreadyExistsAsFile, \
+ AlreadyExistsAsDir
class GlusterSwiftDiskFileRouter(object):
@@ -55,6 +58,9 @@ class ObjectController(server.ObjectController):
"""
# Replaces Swift's DiskFileRouter object reference with ours.
self._diskfile_router = GlusterSwiftDiskFileRouter(conf, self.logger)
+ self.devices = conf.get('devices', '/mnt/gluster-object')
+ self.swift_dir = conf.get('swift_dir', '/etc/swift')
+ self.object_ring = self.get_object_ring()
def container_update(self, *args, **kwargs):
"""
@@ -65,15 +71,65 @@ class ObjectController(server.ObjectController):
"""
return
- def delete_at_update(self, *args, **kwargs):
- """
- Update the expiring objects container when objects are updated.
+ def get_object_ring(self):
+ return Ring(self.swift_dir, ring_name='object')
- 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.
+ def _create_expiring_tracker_object(self, object_path):
+ try:
+
+ # Check if gsexpiring volume is present in ring
+ if not any(d.get('device', None) == self.expiring_objects_account
+ for d in self.object_ring.devs):
+ raise Exception("%s volume not in ring" %
+ self.expiring_objects_account)
+
+ # Check if gsexpiring is mounted.
+ expiring_objects_account_path = \
+ os.path.join(self.devices, self.expiring_objects_account)
+ mount_check = self._diskfile_router['junk'].mount_check
+ if mount_check and not do_ismount(expiring_objects_account_path):
+ raise Exception("Path %s doesn't exist or is not a mount "
+ "point" % expiring_objects_account_path)
+
+ # Create object directory
+ object_dir = os.path.dirname(object_path)
+ try:
+ mkdirs(object_dir)
+ except OSError as err:
+ mkdirs(object_dir) # handle race
+
+ # Create zero-byte file
+ try:
+ os.mknod(object_path)
+ except OSError as err:
+ if err.errno != errno.EEXIST:
+ raise
+ except Exception as e:
+ self.logger.error("Creation of tracker object %s failed: %s" %
+ (object_path, str(e)))
+
+ def async_update(self, op, account, container, obj, host, partition,
+ contdevice, headers_out, objdevice, policy):
"""
- return
+ 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.
+ """
+ object_path = os.path.join(self.devices, account, container, obj)
+
+ threadpool = self._diskfile_router[policy].threadpools[objdevice]
+ threadpool.run_in_thread(self._create_expiring_tracker_object,
+ object_path)
@public
@timing_stats()
diff --git a/glusterfs-openstack-swift.spec b/glusterfs-openstack-swift.spec
index 77bb324..9acabb5 100644
--- a/glusterfs-openstack-swift.spec
+++ b/glusterfs-openstack-swift.spec
@@ -78,6 +78,7 @@ done
%{_bindir}/gluster-swift-gen-builders
%{_bindir}/gluster-swift-print-metadata
%{_bindir}/gluster-swift-migrate-metadata
+%{_bindir}/gluster-swift-object-expirer
%{_bindir}/gswauth-add-account
%{_bindir}/gswauth-add-user
%{_bindir}/gswauth-cleanup-tokens
diff --git a/setup.py b/setup.py
index 214d8f1..a8787c6 100644
--- a/setup.py
+++ b/setup.py
@@ -44,6 +44,7 @@ setup(
'bin/gluster-swift-gen-builders',
'bin/gluster-swift-print-metadata',
'bin/gluster-swift-migrate-metadata',
+ 'bin/gluster-swift-object-expirer',
'gluster/swift/common/middleware/gswauth/bin/gswauth-add-account',
'gluster/swift/common/middleware/gswauth/bin/gswauth-add-user',
'gluster/swift/common/middleware/gswauth/bin/gswauth-cleanup-tokens',
diff --git a/test/functional_auth/common_conf/object-expirer.conf b/test/functional_auth/common_conf/object-expirer.conf
index 4449ee2..971367b 100644
--- a/test/functional_auth/common_conf/object-expirer.conf
+++ b/test/functional_auth/common_conf/object-expirer.conf
@@ -2,6 +2,7 @@
#For now, refer: https://github.com/openstack/swift/blob/master/etc/object-expirer.conf-sample
[DEFAULT]
+devices = /mnt/gluster-object
[object-expirer]
user = root
@@ -15,13 +16,18 @@ expiring_objects_account_name = expiring
interval = 30
[pipeline:main]
-pipeline = catch_errors cache proxy-server
+pipeline = catch_errors cache proxy-logging proxy-server
[app:proxy-server]
use = egg:gluster_swift#proxy
+log_facility = LOG_LOCAL1
+log_level = DEBUG
[filter:cache]
use = egg:swift#memcache
[filter:catch_errors]
use = egg:swift#catch_errors
+
+[filter:proxy-logging]
+use = egg:swift#proxy_logging
diff --git a/test/object_expirer_functional/test_object_expirer.py b/test/object_expirer_functional/test_object_expirer.py
index aaec75e..279994f 100644
--- a/test/object_expirer_functional/test_object_expirer.py
+++ b/test/object_expirer_functional/test_object_expirer.py
@@ -15,14 +15,19 @@
import os
import time
+import logging
-from swift.common.manager import Manager
-from swift.common.internal_client import InternalClient
+from gluster.swift.obj.expirer import ObjectExpirer, GlusterSwiftInternalClient
-from test.functional.tests import Base, config, Utils
+from swift.common.utils import readconf
+
+from test import get_config
+from test.functional.tests import Base, Utils
from test.functional.swift_test_client import Account, Connection, \
ResponseError
+config = get_config('func_test')
+
class TestObjectExpirerEnv:
@classmethod
@@ -39,9 +44,12 @@ class TestObjectExpirerEnv:
cls.file_size = 8
cls.root_dir = os.path.join('/mnt/gluster-object',
cls.account.conn.storage_url.split('/')[2].split('_')[1])
- cls.client = InternalClient('/etc/swift/object-expirer.conf',
- 'Test Object Expirer', 1)
- cls.expirer = Manager(['object-expirer'])
+ devices = config.get('devices', '/mnt/gluster-object')
+ cls.client = GlusterSwiftInternalClient('/etc/swift/object-expirer.conf',
+ 'Test Object Expirer', 1,
+ devices=devices)
+ conf = readconf('/etc/swift/object-expirer.conf', 'object-expirer')
+ cls.expirer = ObjectExpirer(conf)
class TestObjectExpirer(Base):
@@ -91,7 +99,7 @@ class TestObjectExpirer(Base):
self.fail("Tracker object not found.")
# Run expirer daemon once.
- self.env.expirer.once()
+ self.env.expirer.run_once()
# Ensure object is physically deleted from filesystem.
self.assertFalse(os.path.exists(os.path.join(self.env.root_dir,
@@ -151,7 +159,7 @@ class TestObjectExpirer(Base):
self.fail("Tracker object not found.")
# Run expirer daemon once.
- self.env.expirer.once()
+ self.env.expirer.run_once()
# Ensure object is physically deleted from filesystem.
self.assertFalse(os.path.exists(os.path.join(self.env.root_dir,
@@ -169,7 +177,6 @@ class TestObjectExpirer(Base):
# GET on container should no longer list the object.
self.assertFalse(obj.name in self.env.container.files())
-
def test_object_expiry_X_Delete_At_POST(self):
# Create normal object
@@ -221,8 +228,7 @@ class TestObjectExpirer(Base):
self.fail("Tracker object not found.")
# Run expirer daemon once.
- self.env.expirer.once()
- time.sleep(3)
+ self.env.expirer.run_once()
# Ensure object is physically deleted from filesystem.
self.assertFalse(os.path.exists(os.path.join(self.env.root_dir,
@@ -291,8 +297,7 @@ class TestObjectExpirer(Base):
self.fail("Tracker object not found.")
# Run expirer daemon once.
- self.env.expirer.once()
- time.sleep(3)
+ self.env.expirer.run_once()
# Ensure object is physically deleted from filesystem.
self.assertFalse(os.path.exists(os.path.join(self.env.root_dir,
diff --git a/test/unit/common/test_diskdir.py b/test/unit/common/test_diskdir.py
index 3c91016..623164c 100644
--- a/test/unit/common/test_diskdir.py
+++ b/test/unit/common/test_diskdir.py
@@ -461,7 +461,7 @@ class TestContainerBroker(unittest.TestCase):
self.assert_(broker.empty())
obj = self._create_file('o.txt')
self.assert_(not broker.empty())
- broker.delete_object('o', normalize_timestamp(time()))
+ broker.delete_object('o', normalize_timestamp(time()), 0)
# delete_object() should be a NOOP
self.assert_(not broker.empty())
os.unlink(obj)
diff --git a/tools/object_expirer_functional.sh b/tools/object_expirer_functional.sh
index 40f2f81..e8587bf 100755
--- a/tools/object_expirer_functional.sh
+++ b/tools/object_expirer_functional.sh
@@ -30,7 +30,7 @@ cleanup()
sudo rm -rf /etc/swift > /dev/null 2>&1
sudo rm -rf /mnt/gluster-object/test{,2}/* > /dev/null 2>&1
sudo rm -rf /mnt/gluster-object/gsexpiring/* > /dev/null 2>&1
- sudo setfattr -x user.swift.metadata /mnt/gluster-object/test{,2} > /dev/null 2>&1
+ sudo setfattr -x user.swift.metadata /mnt/gluster-object/test > /dev/null 2>&1
sudo setfattr -x user.swift.metadata /mnt/gluster-object/gsexpiring > /dev/null 2>&1
}
@@ -55,7 +55,7 @@ if [ -x /etc/swift ] ; then
fi
# Check the directories exist
-DIRS="/mnt/gluster-object /mnt/gluster-object/test /mnt/gluster-object/test2 /mnt/gluster-object/gsexpiring"
+DIRS="/mnt/gluster-object /mnt/gluster-object/test /mnt/gluster-object/gsexpiring"
for d in $DIRS ; do
if [ ! -x $d ] ; then
quit "$d must exist on an XFS or GlusterFS volume"