From 46e8aecd0347115498d1158dadbc57f1c54baf34 Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Thu, 5 Aug 2010 13:57:26 -0500 Subject: [PATCH 01/89] initial stats system commit. includes the log uploader --- bin/swift-log-uploader | 83 ++++++++++++ etc/log-processing.conf-sample | 26 ++++ swift/common/compressed_file_reader.py | 72 ++++++++++ swift/common/internal_proxy.py | 174 +++++++++++++++++++++++++ swift/stats/__init__.py | 0 swift/stats/log_uploader.py | 135 +++++++++++++++++++ 6 files changed, 490 insertions(+) create mode 100755 bin/swift-log-uploader create mode 100644 etc/log-processing.conf-sample create mode 100644 swift/common/compressed_file_reader.py create mode 100644 swift/common/internal_proxy.py create mode 100644 swift/stats/__init__.py create mode 100644 swift/stats/log_uploader.py diff --git a/bin/swift-log-uploader b/bin/swift-log-uploader new file mode 100755 index 0000000000..4ca9c17795 --- /dev/null +++ b/bin/swift-log-uploader @@ -0,0 +1,83 @@ +#!/usr/bin/python +# Copyright (c) 2010 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 os +import signal +import sys +import time +from ConfigParser import ConfigParser + +from swift.stats.log_uploader import LogUploader +from swift.common.utils import get_logger + +if __name__ == '__main__': + if len(sys.argv) < 3: + print "Usage: swift-log-uploader CONFIG_FILE plugin" + sys.exit() + + c = ConfigParser() + if not c.read(sys.argv[1]): + print "Unable to read config file." + sys.exit(1) + + if c.has_section('log-processor'): + parser_conf = dict(c.items('log-processor')) + else: + print "Unable to find log-processor config section in %s." % sys.argv[1] + sys.exit(1) + + plugin = sys.argv[2] + section_name = 'log-processor-%s' % plugin + if c.has_section(section_name): + uploader_conf.update(dict(c.items(section_name))) + else: + print "Unable to find %s config section in %s." % (section_name, + sys.argv[1]) + sys.exit(1) + + try: + os.setsid() + except OSError: + pass + + logger = get_logger(uploader_conf, 'swift-log-uploader') + + def kill_children(*args): + signal.signal(signal.SIGTERM, signal.SIG_IGN) + os.killpg(0, signal.SIGTERM) + sys.exit() + + signal.signal(signal.SIGTERM, kill_children) + + log_dir = uploader_conf.get('log_dir', '/var/log/swift/') + swift_account = uploader_conf['swift_account'] + container_name = uploader_conf['container_name'] + source_filename_format = uploader_conf['source_filename_format'] + proxy_server_conf_loc = uploader_conf.get('proxy_server_conf', + '/etc/swift/proxy-server.conf') + try: + c = ConfigParser() + c.read(proxy_server_conf_loc) + proxy_server_conf = dict(c.items('proxy-server')) + except: + proxy_server_conf = None + uploader = LogUploader(log_dir, swift_account, container_name, + source_filename_format, proxy_server_conf, logger) + logger.info("Uploading logs") + start = time.time() + uploader.upload_all_logs() + logger.info("Uploading logs complete (%0.2f minutes)" % + ((time.time()-start)/60)) diff --git a/etc/log-processing.conf-sample b/etc/log-processing.conf-sample new file mode 100644 index 0000000000..a73f9a0bc3 --- /dev/null +++ b/etc/log-processing.conf-sample @@ -0,0 +1,26 @@ +# plugin section format is named "log-processor-" +# section "log-processor" is the generic defaults (overridden by plugins) + +[log-processor] +# working_dir = /tmp/swift/ +# proxy_server_conf = /etc/swift/proxy-server.conf +# log_facility = LOG_LOCAL0 +# log_level = INFO +# lookback_hours = 120 +# lookback_window = 120 + +[log-processor-access] +# log_dir = /var/log/swift/ +swift_account = AUTH_7abbc116-8a07-4b63-819d-02715d3e0f31 +container_name = log_data +source_filename_format = %Y%m%d%H* +class_path = swift.stats.access_processor +# service ips is for client ip addresses that should be counted as servicenet +# service_ips = + +[log-processor-stats] +# log_dir = /var/log/swift/ +swift_account = AUTH_7abbc116-8a07-4b63-819d-02715d3e0f31 +container_name = account_stats +source_filename_format = %Y%m%d%H* +class_path = swift.stats.stats_processor \ No newline at end of file diff --git a/swift/common/compressed_file_reader.py b/swift/common/compressed_file_reader.py new file mode 100644 index 0000000000..9eef0200c7 --- /dev/null +++ b/swift/common/compressed_file_reader.py @@ -0,0 +1,72 @@ +# Copyright (c) 2010 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 zlib +import struct + + +class CompressedFileReader(object): + ''' + Wraps a file object and provides a read method that returns gzip'd data. + + One warning: if read is called with a small value, the data returned may + be bigger than the value. In this case, the "compressed" data will be + bigger than the original data. To solve this, use a bigger read buffer. + + An example use case: + Given an uncompressed file on disk, provide a way to read compressed data + without buffering the entire file data in memory. Using this class, an + uncompressed log file could be uploaded as compressed data with chunked + transfer encoding. + + gzip header and footer code taken from the python stdlib gzip module + + :param file_obj: File object to read from + :param compresslevel: compression level + ''' + def __init__(self, file_obj, compresslevel=9): + self._f = file_obj + self._compressor = zlib.compressobj(compresslevel, + zlib.DEFLATED, + -zlib.MAX_WBITS, + zlib.DEF_MEM_LEVEL, + 0) + self.done = False + self.first = True + self.crc32 = 0 + self.total_size = 0 + + def read(self, *a, **kw): + if self.done: + return '' + x = self._f.read(*a, **kw) + if x: + self.crc32 = zlib.crc32(x, self.crc32) & 0xffffffffL + self.total_size += len(x) + compressed = self._compressor.compress(x) + if not compressed: + compressed = self._compressor.flush(zlib.Z_SYNC_FLUSH) + else: + compressed = self._compressor.flush(zlib.Z_FINISH) + crc32 = struct.pack(" 299) \ + and tries <= self.retries: + resp = self.upload_app.handle_request(self.upload_app.update_request(req)) + tries += 1 + if not (200 <= resp.status_int < 300): + return False + return True + + def get_object(self, account, container, object_name): + """ + Get object. + + :param account: account name object is in + :param container: container name object is in + :param object_name: name of object to get + :returns: iterator for object data + """ + req = webob.Request.blank('/v1/%s/%s/%s' % + (account, container, object_name), + environ={'REQUEST_METHOD': 'GET'}) + req.account = account + resp = self.upload_app.handle_request(self.upload_app.update_request(req)) + tries = 1 + while (resp.status_int < 200 or resp.status_int > 299) \ + and tries <= self.retries: + resp = self.upload_app.handle_request(self.upload_app.update_request(req)) + tries += 1 + for x in resp.app_iter: + yield x + + def create_container(self, account, container): + """ + Create container. + + :param account: account name to put the container in + :param container: container name to create + :returns: True if successful, otherwise False + """ + req = webob.Request.blank('/v1/%s/%s' % (account, container), + environ={'REQUEST_METHOD': 'PUT'}) + req.account = account + resp = self.upload_app.handle_request(self.upload_app.update_request(req)) + tries = 1 + while (resp.status_int < 200 or resp.status_int > 299) \ + and tries <= self.retries: + resp = self.upload_app.handle_request(self.upload_app.update_request(req)) + tries += 1 + return 200 <= resp.status_int < 300 + + def get_container_list(self, account, container, marker=None, limit=None, + prefix=None, delimiter=None, full_listing=True): + """ + Get container listing. + + :param account: account name for the container + :param container: container name to get the listing of + :param marker: marker query + :param limit: limit to query + :param prefix: prefix query + :param delimeter: delimeter for query + :param full_listing: if True, make enough requests to get all listings + :returns: list of objects + """ + if full_listing: + rv = [] + listing = self.get_container_list(account, container, marker, + limit, prefix, delimiter, full_listing=False) + while listing: + rv.extend(listing) + if not delimiter: + marker = listing[-1]['name'] + else: + marker = listing[-1].get('name', listing[-1].get('subdir')) + listing = self.get_container_list(account, container, marker, + limit, prefix, delimiter, full_listing=False) + return rv + path = '/v1/%s/%s' % (account, container) + qs = 'format=json' + if marker: + qs += '&marker=%s' % quote(marker) + if limit: + qs += '&limit=%d' % limit + if prefix: + qs += '&prefix=%s' % quote(prefix) + if delimiter: + qs += '&delimiter=%s' % quote(delimiter) + path += '?%s' % qs + req = webob.Request.blank(path, environ={'REQUEST_METHOD': 'GET'}) + req.account = account + resp = self.upload_app.handle_request(self.upload_app.update_request(req)) + tries = 1 + while (resp.status_int < 200 or resp.status_int > 299) \ + and tries <= self.retries: + resp = self.upload_app.handle_request(self.upload_app.update_request(req)) + tries += 1 + if resp.status_int == 204: + return [] + if 200 <= resp.status_int < 300: + return json_loads(resp.body) diff --git a/swift/stats/__init__.py b/swift/stats/__init__.py new file mode 100644 index 0000000000..e69de29bb2 diff --git a/swift/stats/log_uploader.py b/swift/stats/log_uploader.py new file mode 100644 index 0000000000..8c45d64db5 --- /dev/null +++ b/swift/stats/log_uploader.py @@ -0,0 +1,135 @@ +# Copyright (c) 2010 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. + +from __future__ import with_statement +import os +import hashlib +import time +import gzip +import glob + +from swift.common.internal_proxy import InternalProxy + +class LogUploader(object): + ''' + Given a local directory, a swift account, and a container name, LogParser + will upload all files in the local directory to the given account/container. + All but the newest files will be uploaded, and the files' md5 sum will be + computed. The hash is used to prevent duplicate data from being uploaded + multiple times in different files (ex: log lines). Since the hash is + computed, it is also used as the uploaded object's etag to ensure data + integrity. + + Note that after the file is successfully uploaded, it will be unlinked. + + The given proxy server config is used to instantiate a proxy server for + the object uploads. + ''' + + def __init__(self, log_dir, swift_account, container_name, filename_format, + proxy_server_conf, logger): + if not log_dir.endswith('/'): + log_dir = log_dir + '/' + self.log_dir = log_dir + self.swift_account = swift_account + self.container_name = container_name + self.filename_format = filename_format + self.internal_proxy = InternalProxy(proxy_server_conf, logger) + self.logger = logger + + def upload_all_logs(self): + i = [(c,self.filename_format.index(c)) for c in '%Y %m %d %H'.split()] + i.sort() + year_offset = month_offset = day_offset = hour_offset = None + for c, start in i: + if c == '%Y': + year_offset = start, start+4 + elif c == '%m': + month_offset = start, start+2 + elif c == '%d': + day_offset = start, start+2 + elif c == '%H': + hour_offset = start, start+2 + if not (year_offset and month_offset and day_offset and hour_offset): + # don't have all the parts, can't upload anything + return + glob_pattern = self.filename_format + glob_pattern = glob_pattern.replace('%Y', '????') + glob_pattern = glob_pattern.replace('%m', '??') + glob_pattern = glob_pattern.replace('%d', '??') + glob_pattern = glob_pattern.replace('%H', '??') + filelist = glob.iglob(os.path.join(self.log_dir, glob_pattern)) + current_hour = int(time.strftime('%H')) + today = int(time.strftime('%Y%m%d')) + self.internal_proxy.create_container(self.swift_account, + self.container_name) + for filename in filelist: + try: + # From the filename, we need to derive the year, month, day, + # and hour for the file. These values are used in the uploaded + # object's name, so they should be a reasonably accurate + # representation of the time for which the data in the file was + # collected. The file's last modified time is not a reliable + # representation of the data in the file. For example, an old + # log file (from hour A) may be uploaded or moved into the + # log_dir in hour Z. The file's modified time will be for hour + # Z, and therefore the object's name in the system will not + # represent the data in it. + # If the filename doesn't match the format, it shouldn't be + # uploaded. + year = filename[slice(*year_offset)] + month = filename[slice(*month_offset)] + day = filename[slice(*day_offset)] + hour = filename[slice(*hour_offset)] + except IndexError: + # unexpected filename format, move on + self.logger.error("Unexpected log: %s" % filename) + continue + if (time.time() - os.stat(filename).st_mtime) < 7200: + # don't process very new logs + self.logger.debug("Skipping log: %s (< 2 hours old)" % filename) + continue + self.upload_one_log(filename, year, month, day, hour) + + def upload_one_log(self, filename, year, month, day, hour): + if os.path.getsize(filename) == 0: + self.logger.debug("Log %s is 0 length, skipping" % filename) + return + self.logger.debug("Processing log: %s" % filename) + filehash = hashlib.md5() + already_compressed = True if filename.endswith('.gz') else False + opener = gzip.open if already_compressed else open + f = opener(filename, 'rb') + try: + for line in f: + # filter out bad lines here? + filehash.update(line) + finally: + f.close() + filehash = filehash.hexdigest() + # By adding a hash to the filename, we ensure that uploaded files + # have unique filenames and protect against uploading one file + # more than one time. By using md5, we get an etag for free. + target_filename = '/'.join([year, month, day, hour, filehash+'.gz']) + if self.internal_proxy.upload_file(filename, + self.swift_account, + self.container_name, + target_filename, + compress=(not already_compressed)): + self.logger.debug("Uploaded log %s to %s" % + (filename, target_filename)) + os.unlink(filename) + else: + self.logger.error("ERROR: Upload of log %s failed!" % filename) From 485799fc5489d2d4a93771ff7915b0aa9f0e15dc Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Thu, 5 Aug 2010 23:09:53 -0500 Subject: [PATCH 02/89] added account stats logger to stats system --- bin/swift-account-stats-logger.py | 81 +++++++++++++++++++++++++++++++ etc/log-processing.conf-sample | 3 +- swift/stats/account_stats.py | 69 ++++++++++++++++++++++++++ 3 files changed, 152 insertions(+), 1 deletion(-) create mode 100644 bin/swift-account-stats-logger.py create mode 100644 swift/stats/account_stats.py diff --git a/bin/swift-account-stats-logger.py b/bin/swift-account-stats-logger.py new file mode 100644 index 0000000000..050545af9a --- /dev/null +++ b/bin/swift-account-stats-logger.py @@ -0,0 +1,81 @@ +#!/usr/bin/python +# Copyright (c) 2010 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 os +import signal +import sys +import time +from ConfigParser import ConfigParser + +from swift.account_stats import AccountStat +from swift.common import utils + +if __name__ == '__main__': + if len(sys.argv) < 2: + print "Usage: swift-account-stats-logger CONFIG_FILE" + sys.exit() + + c = ConfigParser() + if not c.read(sys.argv[1]): + print "Unable to read config file." + sys.exit(1) + + if c.has_section('log-processor-stats'): + stats_conf = dict(c.items('log-processor-stats')) + else: + print "Unable to find log-processor-stats config section in %s." % \ + sys.argv[1] + sys.exit(1) + + # reference this from the account stats conf + + target_dir = stats.conf.get('log_dir', '/var/log/swift') + account_server_conf_loc = stats_conf.get('account_server_conf', + '/etc/swift/account-server.conf') + filename_format = stats.conf['source_filename_format'] + try: + c = ConfigParser() + c.read(account_server_conf_loc) + account_server_conf = dict(c.items('account-server')) + except: + print "Unable to load account server conf from %s" % account_server_conf_loc + sys.exit(1) + + utils.drop_privileges(account_server_conf.get('user', 'swift')) + + try: + os.setsid() + except OSError: + pass + + logger = utils.get_logger(stats_conf, 'swift-account-stats-logger') + + def kill_children(*args): + signal.signal(signal.SIGTERM, signal.SIG_IGN) + os.killpg(0, signal.SIGTERM) + sys.exit() + + signal.signal(signal.SIGTERM, kill_children) + + stats = AccountStat(filename_format, + target_dir, + account_server_conf, + logger) + logger.info("Gathering account stats") + start = time.time() + stats.find_and_process() + logger.info("Gathering account stats complete (%0.2f minutes)" % + ((time.time()-start)/60)) diff --git a/etc/log-processing.conf-sample b/etc/log-processing.conf-sample index a73f9a0bc3..2d9293c4c5 100644 --- a/etc/log-processing.conf-sample +++ b/etc/log-processing.conf-sample @@ -23,4 +23,5 @@ class_path = swift.stats.access_processor swift_account = AUTH_7abbc116-8a07-4b63-819d-02715d3e0f31 container_name = account_stats source_filename_format = %Y%m%d%H* -class_path = swift.stats.stats_processor \ No newline at end of file +class_path = swift.stats.stats_processor +# account_server_conf = /etc/swift/account-server.conf diff --git a/swift/stats/account_stats.py b/swift/stats/account_stats.py new file mode 100644 index 0000000000..1624807cfe --- /dev/null +++ b/swift/stats/account_stats.py @@ -0,0 +1,69 @@ +# Copyright (c) 2010 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 os +import time + +from swift.account.server import DATADIR as account_server_data_dir +from swift.common.db import AccountBroker +from swift.common.internal_proxy import InternalProxy +from swift.common.utils import renamer + +class AccountStat(object): + def __init__(self, filename_format, target_dir, server_conf, logger): + self.filename_format = filename_format + self.target_dir = target_dir + self.devices = server_conf.get('devices', '/srv/node') + self.mount_check = server_conf.get('mount_check', 'true').lower() in \ + ('true', 't', '1', 'on', 'yes', 'y') + self.logger = logger + + def find_and_process(self): + src_filename = time.strftime(self.filename_format) + tmp_filename = os.path.join('/tmp', src_filename) + with open(tmp_filename, 'wb') as statfile: + #statfile.write('Account Name, Container Count, Object Count, Bytes Used, Created At\n') + for device in os.listdir(self.devices): + if self.mount_check and \ + not os.path.ismount(os.path.join(self.devices, device)): + self.logger.error("Device %s is not mounted, skipping." % + device) + continue + accounts = os.path.join(self.devices, + device, + account_server_data_dir) + if not os.path.exists(accounts): + self.logger.debug("Path %s does not exist, skipping." % + accounts) + continue + for root, dirs, files in os.walk(accounts, topdown=False): + for filename in files: + if filename.endswith('.db'): + broker = AccountBroker(os.path.join(root, filename)) + if not broker.is_deleted(): + account_name, + created_at, + _, _, + container_count, + object_count, + bytes_used, + _, _ = broker.get_info() + line_data = '"%s",%d,%d,%d,%s\n' % (account_name, + container_count, + object_count, + bytes_used, + created_at) + statfile.write(line_data) + renamer(tmp_filename, os.path.join(self.target_dir, src_filename)) From 6258b4adfd8573ae65f5d049a105d7de11f126e4 Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Thu, 12 Aug 2010 14:26:34 -0500 Subject: [PATCH 03/89] added log_processor and a stats plugin --- etc/log-processing.conf-sample | 4 +- swift/stats/log_processor.py | 226 +++++++++++++++++++++++++++++++++ swift/stats/stats_processor.py | 55 ++++++++ 3 files changed, 283 insertions(+), 2 deletions(-) create mode 100644 swift/stats/log_processor.py create mode 100644 swift/stats/stats_processor.py diff --git a/etc/log-processing.conf-sample b/etc/log-processing.conf-sample index 2d9293c4c5..cc7ea6e276 100644 --- a/etc/log-processing.conf-sample +++ b/etc/log-processing.conf-sample @@ -14,7 +14,7 @@ swift_account = AUTH_7abbc116-8a07-4b63-819d-02715d3e0f31 container_name = log_data source_filename_format = %Y%m%d%H* -class_path = swift.stats.access_processor +class_path = swift.stats.access_processor.AccessLogProcessor # service ips is for client ip addresses that should be counted as servicenet # service_ips = @@ -23,5 +23,5 @@ class_path = swift.stats.access_processor swift_account = AUTH_7abbc116-8a07-4b63-819d-02715d3e0f31 container_name = account_stats source_filename_format = %Y%m%d%H* -class_path = swift.stats.stats_processor +class_path = swift.stats.stats_processor.StatsLogProcessor # account_server_conf = /etc/swift/account-server.conf diff --git a/swift/stats/log_processor.py b/swift/stats/log_processor.py new file mode 100644 index 0000000000..b802ba4404 --- /dev/null +++ b/swift/stats/log_processor.py @@ -0,0 +1,226 @@ +# Copyright (c) 2010 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. + +class LogProcessor(object): + + def __init__(self, conf, logger): + stats_conf = conf.get('log-processor', {}) + + working_dir = stats_conf.get('working_dir', '/tmp/swift/') + if working_dir.endswith('/') and len(working_dir) > 1: + working_dir = working_dir[:-1] + self.working_dir = working_dir + proxy_server_conf_loc = stats_conf.get('proxy_server_conf', + '/etc/swift/proxy-server.conf') + try: + c = ConfigParser() + c.read(proxy_server_conf_loc) + proxy_server_conf = dict(c.items('proxy-server')) + except: + proxy_server_conf = None + self.proxy_server_conf = proxy_server_conf + if isinstance(logger, tuple): + self.logger = get_logger(*logger) + else: + self.logger = logger + + # load the processing plugins + self.plugins = {} + plugin_prefix = 'log-processor-' + for section in (x for x in conf if x.startswith(plugin_prefix)): + plugin_name = section[len(plugin_prefix):] + plugin_conf = conf.get(section, {}) + self.plugins[plugin_name] = plugin_conf + import_target, class_name = plugin_conf['class_path'].rsplit('.', 1) + module = __import__(import_target, fromlist=[import_target]) + klass = getattr(module, class_name) + self.plugins[plugin_name]['instance'] = klass(plugin_conf) + + def process_one_file(self, plugin_name, account, container, object_name): + # get an iter of the object data + compressed = object_name.endswith('.gz') + stream = self.get_object_data(account, container, object_name, + compressed=compressed) + # look up the correct plugin and send the stream to it + return self.plugins[plugin_name]['instance'].process(stream) + + def get_data_list(self, start_date=None, end_date=None, listing_filter=None): + total_list = [] + for p in self.plugins: + account = p['swift_account'] + container = p['container_name'] + l = self.get_container_listing(account, container, start_date, + end_date, listing_filter) + for i in l: + total_list.append((p, account, container, i)) + return total_list + + def get_container_listing(self, swift_account, container_name, start_date=None, + end_date=None, listing_filter=None): + ''' + Get a container listing, filtered by start_date, end_date, and + listing_filter. Dates, if given, should be in YYYYMMDDHH format + ''' + search_key = None + if start_date is not None: + date_parts = [] + try: + year, start_date = start_date[:4], start_date[4:] + if year: + date_parts.append(year) + month, start_date = start_date[:2], start_date[2:] + if month: + date_parts.append(month) + day, start_date = start_date[:2], start_date[2:] + if day: + date_parts.append(day) + hour, start_date = start_date[:2], start_date[2:] + if hour: + date_parts.append(hour) + except IndexError: + pass + else: + search_key = '/'.join(date_parts) + end_key = None + if end_date is not None: + date_parts = [] + try: + year, end_date = end_date[:4], end_date[4:] + if year: + date_parts.append(year) + month, end_date = end_date[:2], end_date[2:] + if month: + date_parts.append(month) + day, end_date = end_date[:2], end_date[2:] + if day: + date_parts.append(day) + hour, end_date = end_date[:2], end_date[2:] + if hour: + date_parts.append(hour) + except IndexError: + pass + else: + end_key = '/'.join(date_parts) + container_listing = self.private_proxy.get_container_list( + swift_account, + container_name, + marker=search_key) + results = [] + if container_listing is not None: + if listing_filter is None: + listing_filter = set() + for item in container_listing: + name = item['name'] + if end_key and name > end_key: + break + if name not in listing_filter: + results.append(name) + return results + + def get_object_data(self, swift_account, container_name, object_name, + compressed=False): + '''reads an object and yields its lines''' + o = self.private_proxy.get_object(swift_account, + container_name, + object_name) + tmp_file = tempfile.TemporaryFile(dir=self.working_dir) + with tmp_file as f: + bad_file = False + try: + for chunk in o: + f.write(chunk) + except ChunkReadTimeout: + bad_file = True + if bad_file: + raise BadFileDownload() + f.flush() + f.seek(0) # rewind to start reading + last_part = '' + last_compressed_part = '' + # magic in the following zlib.decompressobj argument is courtesy of + # http://stackoverflow.com/questions/2423866/python-decompressing-gzip-chunk-by-chunk + d = zlib.decompressobj(16+zlib.MAX_WBITS) + for chunk in iter(lambda: f.read(16384), ''): + if compressed: + try: + chunk = d.decompress(chunk) + except zlib.error: + raise BadFileDownload() # bad compressed data + parts = chunk.split('\n') + parts[0] = last_part + parts[0] + for part in parts[:-1]: + yield part + last_part = parts[-1] + if last_part: + yield last_part + +def multiprocess_collate(processor_args, + start_date=None, + end_date=None, + listing_filter=None): + '''get listing of files and yield hourly data from them''' + p = LogProcessor(*processor_args) + all_files = p.get_data_list(start_date, end_date, listing_filter) + + p.logger.info('loaded %d files to process' % len(all_files)) + + if not all_files: + # no work to do + return + + worker_count = multiprocessing.cpu_count() - 1 + results = [] + in_queue = multiprocessing.Queue() + out_queue = multiprocessing.Queue() + for _ in range(worker_count): + p = multiprocessing.Process(target=collate_worker, + args=(processor_args, + in_queue, + out_queue)) + p.start() + results.append(p) + for x in all_files: + in_queue.put(x) + for _ in range(worker_count): + in_queue.put(None) + count = 0 + while True: + try: + item, data = out_queue.get_nowait() + count += 1 + if data: + yield item, data + if count >= len(all_files): + # this implies that one result will come from every request + break + except Queue.Empty: + time.sleep(.1) + for r in results: + r.join() + +def collate_worker(processor_args, in_queue, out_queue): + '''worker process for multiprocess_collate''' + p = LogProcessor(*processor_args) + while True: + try: + item = in_queue.get_nowait() + if item is None: + break + except Queue.Empty: + time.sleep(.1) + else: + ret = None + ret = p.process_one_file(item) + out_queue.put((item, ret)) \ No newline at end of file diff --git a/swift/stats/stats_processor.py b/swift/stats/stats_processor.py new file mode 100644 index 0000000000..793963ff04 --- /dev/null +++ b/swift/stats/stats_processor.py @@ -0,0 +1,55 @@ +# Copyright (c) 2010 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. + +class StatsLogProcessor(object): + + def __init__(self, conf): + pass + + def process(self, obj_stream): + '''generate hourly groupings of data from one stats log file''' + account_totals = {} + year, month, day, hour, _ = item.split('/') + for line in obj_stream: + if not line: + continue + try: + (account, + container_count, + object_count, + bytes_used, + created_at) = line.split(',') + account = account.strip('"') + if account_name and account_name != account: + continue + container_count = int(container_count.strip('"')) + object_count = int(object_count.strip('"')) + bytes_used = int(bytes_used.strip('"')) + aggr_key = account + aggr_key = (account, year, month, day, hour) + d = account_totals.get(aggr_key, {}) + d['count'] = d.setdefault('count', 0) + 1 + d['container_count'] = d.setdefault('container_count', 0) + \ + container_count + d['object_count'] = d.setdefault('object_count', 0) + \ + object_count + d['bytes_used'] = d.setdefault('bytes_used', 0) + \ + bytes_used + d['created_at'] = created_at + account_totals[aggr_key] = d + except (IndexError, ValueError): + # bad line data + pass + return account_totals, item From 401c22f3bc376cc540a941c8cb46eeac1981b71c Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Sat, 14 Aug 2010 13:37:21 -0500 Subject: [PATCH 04/89] added access log processing plugin --- etc/log-processing.conf-sample | 1 + swift/stats/access_processor.py | 168 ++++++++++++++++++++++++++++++++ swift/stats/stats_processor.py | 1 - 3 files changed, 169 insertions(+), 1 deletion(-) create mode 100644 swift/stats/access_processor.py diff --git a/etc/log-processing.conf-sample b/etc/log-processing.conf-sample index cc7ea6e276..79d48dc3d6 100644 --- a/etc/log-processing.conf-sample +++ b/etc/log-processing.conf-sample @@ -17,6 +17,7 @@ source_filename_format = %Y%m%d%H* class_path = swift.stats.access_processor.AccessLogProcessor # service ips is for client ip addresses that should be counted as servicenet # service_ips = +# server_name = proxy [log-processor-stats] # log_dir = /var/log/swift/ diff --git a/swift/stats/access_processor.py b/swift/stats/access_processor.py new file mode 100644 index 0000000000..0fe254a4ca --- /dev/null +++ b/swift/stats/access_processor.py @@ -0,0 +1,168 @@ +# Copyright (c) 2010 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. + +class AccessLogProcessor(object): + + def __init__(self, conf): + self.server_name = conf.get('server_name', 'proxy') + + def _log_line_parser(self, raw_log): + '''given a raw access log line, return a dict of the good parts''' + d = {} + try: + (_, + server, + client_ip, + lb_ip, + timestamp, + method, + request, + http_version, + code, + referrer, + user_agent, + auth_token, + bytes_in, + bytes_out, + etag, + trans_id, + headers, + processing_time) = (unquote(x) for x in raw_log[16:].split(' ')) + if server != self.server_name: + raise ValueError('incorrect server name in log line') + (version, + account, + container_name, + object_name) = split_path(request, 2, 4, True) + if container_name is not None: + container_name = container_name.split('?', 1)[0] + if object_name is not None: + object_name = object_name.split('?', 1)[0] + account = account.split('?', 1)[0] + query = None + if '?' in request: + request, query = request.split('?', 1) + args = query.split('&') + # Count each query argument. This is used later to aggregate + # the number of format, prefix, etc. queries. + for q in args: + if '=' in q: + k, v = q.split('=', 1) + else: + k = q + # Certain keys will get summmed in stats reporting + # (format, path, delimiter, etc.). Save a "1" here + # to indicate that this request is 1 request for + # its respective key. + d[k] = 1 + except ValueError: + pass + else: + d['client_ip'] = client_ip + d['lb_ip'] = lb_ip + d['method'] = method + d['request'] = request + if query: + d['query'] = query + d['http_version'] = http_version + d['code'] = code + d['referrer'] = referrer + d['user_agent'] = user_agent + d['auth_token'] = auth_token + d['bytes_in'] = bytes_in + d['bytes_out'] = bytes_out + d['etag'] = etag + d['trans_id'] = trans_id + d['processing_time'] = processing_time + day, month, year, hour, minute, second = timestamp.split('/') + d['day'] = day + month = ('%02s' % month_map.index(month)).replace(' ', '0') + d['month'] = month + d['year'] = year + d['hour'] = hour + d['minute'] = minute + d['second'] = second + d['tz'] = '+0000' + d['account'] = account + d['container_name'] = container_name + d['object_name'] = object_name + d['bytes_out'] = int(d['bytes_out'].replace('-','0')) + d['bytes_in'] = int(d['bytes_in'].replace('-','0')) + d['code'] = int(d['code']) + return d + + def process(self, obj_stream): + '''generate hourly groupings of data from one access log file''' + hourly_aggr_info = {} + aggr_account_logs = {} + container_line_counts = collections.defaultdict(int) + log_buffer = collections.defaultdict(list) + for line in obj_stream: + line_data = self._log_line_parser(line) + if not line_data: + continue + account = line_data['account'] + container_name = line_data['container_name'] + year = line_data['year'] + month = line_data['month'] + day = line_data['day'] + hour = line_data['hour'] + bytes_out = line_data['bytes_out'] + bytes_in = line_data['bytes_in'] + method = line_data['method'] + code = int(line_data['code']) + object_name = line_data['object_name'] + client_ip = line_data['client_ip'] + + op_level = None + if not container_name: + op_level = 'account' + elif container_name and not object_name: + op_level = 'container' + elif object_name: + op_level = 'object' + + aggr_key = (account, year, month, day, hour) + d = hourly_aggr_info.get(aggr_key, {}) + if line_data['lb_ip'] in self.lb_private_ips: + source = 'service' + else: + source = 'public' + + if line_data['client_ip'] in self.service_ips: + source = 'service' + + d[(source, 'bytes_out')] = d.setdefault((source, 'bytes_out'), 0) + \ + bytes_out + d[(source, 'bytes_in')] = d.setdefault((source, 'bytes_in'), 0) + \ + bytes_in + + d['format_query'] = d.setdefault('format_query', 0) + \ + line_data.get('format', 0) + d['marker_query'] = d.setdefault('marker_query', 0) + \ + line_data.get('marker', 0) + d['prefix_query'] = d.setdefault('prefix_query', 0) + \ + line_data.get('prefix', 0) + d['delimiter_query'] = d.setdefault('delimiter_query', 0) + \ + line_data.get('delimiter', 0) + path = line_data.get('path', 0) + d['path_query'] = d.setdefault('path_query', 0) + path + + code = '%dxx' % (code/100) + key = (source, op_level, method, code) + d[key] = d.setdefault(key, 0) + 1 + + hourly_aggr_info[aggr_key] = d + return hourly_aggr_info, item, aggr_account_logs diff --git a/swift/stats/stats_processor.py b/swift/stats/stats_processor.py index 793963ff04..7f94e53b66 100644 --- a/swift/stats/stats_processor.py +++ b/swift/stats/stats_processor.py @@ -37,7 +37,6 @@ class StatsLogProcessor(object): container_count = int(container_count.strip('"')) object_count = int(object_count.strip('"')) bytes_used = int(bytes_used.strip('"')) - aggr_key = account aggr_key = (account, year, month, day, hour) d = account_totals.get(aggr_key, {}) d['count'] = d.setdefault('count', 0) + 1 From 1f83f7e6fd3bf412e82f5bd22ac83fdbc51c4e31 Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Wed, 1 Sep 2010 09:35:54 -0500 Subject: [PATCH 05/89] initial tests for the stats system --- ...s-logger.py => swift-account-stats-logger} | 0 swift/stats/log_processor.py | 50 +++--- test/unit/stats/__init__.py | 0 test/unit/stats/test_log_processor.py | 170 ++++++++++++++++++ 4 files changed, 197 insertions(+), 23 deletions(-) rename bin/{swift-account-stats-logger.py => swift-account-stats-logger} (100%) create mode 100644 test/unit/stats/__init__.py create mode 100644 test/unit/stats/test_log_processor.py diff --git a/bin/swift-account-stats-logger.py b/bin/swift-account-stats-logger similarity index 100% rename from bin/swift-account-stats-logger.py rename to bin/swift-account-stats-logger diff --git a/swift/stats/log_processor.py b/swift/stats/log_processor.py index b802ba4404..6179bc64e6 100644 --- a/swift/stats/log_processor.py +++ b/swift/stats/log_processor.py @@ -13,6 +13,16 @@ # See the License for the specific language governing permissions and # limitations under the License. +from swift.common.internal_proxy import InternalProxy +from swift.common.exceptions import ChunkReadTimeout +from swift.common.utils import get_logger + +class ConfigError(Exception): + pass + +class MissingProxyConfig(ConfigError): + pass + class LogProcessor(object): def __init__(self, conf, logger): @@ -29,12 +39,15 @@ class LogProcessor(object): c.read(proxy_server_conf_loc) proxy_server_conf = dict(c.items('proxy-server')) except: - proxy_server_conf = None + raise MissingProxyConfig() self.proxy_server_conf = proxy_server_conf if isinstance(logger, tuple): self.logger = get_logger(*logger) else: self.logger = logger + self.internal_proxy = InternalProxy(self.proxy_server_conf, + self.logger, + retries=3) # load the processing plugins self.plugins = {} @@ -113,7 +126,7 @@ class LogProcessor(object): pass else: end_key = '/'.join(date_parts) - container_listing = self.private_proxy.get_container_list( + container_listing = self.internal_proxy.get_container_list( swift_account, container_name, marker=search_key) @@ -132,27 +145,16 @@ class LogProcessor(object): def get_object_data(self, swift_account, container_name, object_name, compressed=False): '''reads an object and yields its lines''' - o = self.private_proxy.get_object(swift_account, - container_name, - object_name) - tmp_file = tempfile.TemporaryFile(dir=self.working_dir) - with tmp_file as f: - bad_file = False - try: - for chunk in o: - f.write(chunk) - except ChunkReadTimeout: - bad_file = True - if bad_file: - raise BadFileDownload() - f.flush() - f.seek(0) # rewind to start reading - last_part = '' - last_compressed_part = '' - # magic in the following zlib.decompressobj argument is courtesy of - # http://stackoverflow.com/questions/2423866/python-decompressing-gzip-chunk-by-chunk - d = zlib.decompressobj(16+zlib.MAX_WBITS) - for chunk in iter(lambda: f.read(16384), ''): + o = self.internal_proxy.get_object(swift_account, + container_name, + object_name) + last_part = '' + last_compressed_part = '' + # magic in the following zlib.decompressobj argument is courtesy of + # http://stackoverflow.com/questions/2423866/python-decompressing-gzip-chunk-by-chunk + d = zlib.decompressobj(16+zlib.MAX_WBITS) + try: + for chunk in o: if compressed: try: chunk = d.decompress(chunk) @@ -165,6 +167,8 @@ class LogProcessor(object): last_part = parts[-1] if last_part: yield last_part + except ChunkReadTimeout: + raise BadFileDownload() def multiprocess_collate(processor_args, start_date=None, diff --git a/test/unit/stats/__init__.py b/test/unit/stats/__init__.py new file mode 100644 index 0000000000..e69de29bb2 diff --git a/test/unit/stats/test_log_processor.py b/test/unit/stats/test_log_processor.py new file mode 100644 index 0000000000..70367d2215 --- /dev/null +++ b/test/unit/stats/test_log_processor.py @@ -0,0 +1,170 @@ +import unittest + +from swift import log_processor + +class DumbLogger(object): + def __getattr__(self, n): + return self.foo + + def foo(self, *a, **kw): + pass + +class DumbInternalProxy(object): + def get_container_list(self, account, container, marker=None): + n = '2010/03/14/13/obj1' + if marker is None or n > marker: + return [{'name': n}] + else: + return [] + + def get_object(self, account, container, object_name): + if object_name.endswith('.gz'): + # same data as below, compressed with gzip -9 + yield '\x1f\x8b\x08' + yield '\x08"\xd79L' + yield '\x02\x03te' + yield 'st\x00\xcbO' + yield '\xca\xe2JI,I' + yield '\xe4\x02\x00O\xff' + yield '\xa3Y\t\x00\x00\x00' + else: + yield 'obj\n' + yield 'data' + +class TestLogProcessor(unittest.TestCase): + + access_test_line = 'Jul 9 04:14:30 saio proxy 1.2.3.4 4.5.6.7 '\ + '09/Jul/2010/04/14/30 GET '\ + '/v1/AUTH_acct/foo/bar?format=json&foo HTTP/1.0 200 - '\ + 'curl tk4e350daf-9338-4cc6-aabb-090e49babfbd '\ + '6 95 - txfa431231-7f07-42fd-8fc7-7da9d8cc1f90 - 0.0262' + stats_test_line = 'account,1,2,3' + proxy_config = {'log-processor': { + + } + } + + def test_log_line_parser(self): + p = log_processor.LogProcessor(self.proxy_config, DumbLogger()) + result = p.log_line_parser(self.access_test_line) + self.assertEquals(result, {'code': 200, + 'processing_time': '0.0262', + 'auth_token': 'tk4e350daf-9338-4cc6-aabb-090e49babfbd', + 'month': '07', + 'second': '30', + 'year': '2010', + 'query': 'format=json&foo', + 'tz': '+0000', + 'http_version': 'HTTP/1.0', + 'object_name': 'bar', + 'etag': '-', + 'foo': 1, + 'method': 'GET', + 'trans_id': 'txfa431231-7f07-42fd-8fc7-7da9d8cc1f90', + 'client_ip': '1.2.3.4', + 'format': 1, + 'bytes_out': 95, + 'container_name': 'foo', + 'day': '09', + 'minute': '14', + 'account': 'acct', + 'reseller': 'AUTH', + 'hour': '04', + 'referrer': '-', + 'request': '/v1/AUTH_acct', + 'user_agent': 'curl', + 'bytes_in': 6, + 'lb_ip': '4.5.6.7'}) + + def test_process_one_access_file(self): + p = log_processor.LogProcessor(self.proxy_config, DumbLogger()) + def get_object_data(*a,**kw): + return [self.access_test_line] + p.get_object_data = get_object_data + result = p.process_one_access_file('yarr', None) + expected = ({('AUTH_acct', '2010', '07', '09', '04'): + {('public', 'object', 'GET', '2xx'): 1, + ('public', 'bytes_out'): 95, + 'marker_query': 0, + 'format_query': 1, + 'delimiter_query': 0, + 'path_query': 0, + ('public', 'bytes_in'): 6, + 'prefix_query': 0}}, + 'yarr', {}) + self.assertEquals(result, expected) + + def test_process_one_stats_file(self): + p = log_processor.LogProcessor(self.proxy_config, DumbLogger()) + def get_object_data(*a,**kw): + return [self.stats_test_line] + p.get_object_data = get_object_data + result = p.process_one_stats_file('y/m/d/h/f', None) + expected = ({('account', 'y', 'm', 'd', 'h'): + {'count': 1, + 'object_count': 2, + 'container_count': 1, + 'bytes_used': 3}}, + 'y/m/d/h/f') + self.assertEquals(result, expected) + + def test_get_data_listing(self): + p = log_processor.LogProcessor(self.proxy_config, DumbLogger()) + p.private_proxy = DumbPrivateProxy() + result = p.get_data_listing('foo') + expected = ['2010/03/14/13/obj1'] + self.assertEquals(result, expected) + result = p.get_data_listing('foo', listing_filter=expected) + expected = [] + self.assertEquals(result, expected) + result = p.get_data_listing('foo', start_date='2010031412', + end_date='2010031414') + expected = ['2010/03/14/13/obj1'] + self.assertEquals(result, expected) + result = p.get_data_listing('foo', start_date='2010031414') + expected = [] + self.assertEquals(result, expected) + + def test_get_object_data(self): + p = log_processor.LogProcessor(self.proxy_config, DumbLogger()) + p.private_proxy = DumbPrivateProxy() + result = list(p.get_object_data('c', 'o', False)) + expected = ['obj','data'] + self.assertEquals(result, expected) + result = list(p.get_object_data('c', 'o.gz', True)) + self.assertEquals(result, expected) + + def test_get_stat_totals(self): + p = log_processor.LogProcessor(self.proxy_config, DumbLogger()) + p.private_proxy = DumbPrivateProxy() + def get_object_data(*a,**kw): + return [self.stats_test_line] + p.get_object_data = get_object_data + result = list(p.get_stat_totals()) + expected = [({('account', '2010', '03', '14', '13'): + {'count': 1, + 'object_count': 2, + 'container_count': 1, + 'bytes_used': 3}}, + '2010/03/14/13/obj1')] + self.assertEquals(result, expected) + + def test_get_aggr_access_logs(self): + p = log_processor.LogProcessor(self.proxy_config, DumbLogger()) + p.private_proxy = DumbPrivateProxy() + def get_object_data(*a,**kw): + return [self.access_test_line] + p.get_object_data = get_object_data + result = list(p.get_aggr_access_logs()) + expected = [({('AUTH_7abbc116-8a07-4b63-819d-02715d3e0f31', '2010', '07', '09', '04'): + {('public', 'object', 'GET', '2xx'): 1, + ('public', 'bytes_out'): 95, + 'marker_query': 0, + 'format_query': 1, + 'delimiter_query': 0, + 'path_query': 0, + ('public', 'bytes_in'): 0, + 'prefix_query': 0}}, + '2010/03/14/13/obj1', + {})] + self.assertEquals(result, expected) \ No newline at end of file From fed5e315ebe50ebb77aaabacf91b193a0f78150a Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Wed, 1 Sep 2010 15:02:54 -0500 Subject: [PATCH 06/89] first test working --- swift/stats/log_processor.py | 4 ++++ test/unit/stats/test_log_processor.py | 33 +++++++++++++++++---------- 2 files changed, 25 insertions(+), 12 deletions(-) diff --git a/swift/stats/log_processor.py b/swift/stats/log_processor.py index 6179bc64e6..10fd456757 100644 --- a/swift/stats/log_processor.py +++ b/swift/stats/log_processor.py @@ -13,6 +13,9 @@ # See the License for the specific language governing permissions and # limitations under the License. +from ConfigParser import ConfigParser +import zlib + from swift.common.internal_proxy import InternalProxy from swift.common.exceptions import ChunkReadTimeout from swift.common.utils import get_logger @@ -39,6 +42,7 @@ class LogProcessor(object): c.read(proxy_server_conf_loc) proxy_server_conf = dict(c.items('proxy-server')) except: + raise raise MissingProxyConfig() self.proxy_server_conf = proxy_server_conf if isinstance(logger, tuple): diff --git a/test/unit/stats/test_log_processor.py b/test/unit/stats/test_log_processor.py index 70367d2215..0f7699abd6 100644 --- a/test/unit/stats/test_log_processor.py +++ b/test/unit/stats/test_log_processor.py @@ -1,6 +1,6 @@ import unittest -from swift import log_processor +from swift.stats import log_processor class DumbLogger(object): def __getattr__(self, n): @@ -45,6 +45,7 @@ class TestLogProcessor(unittest.TestCase): } def test_log_line_parser(self): + return p = log_processor.LogProcessor(self.proxy_config, DumbLogger()) result = p.log_line_parser(self.access_test_line) self.assertEquals(result, {'code': 200, @@ -77,6 +78,7 @@ class TestLogProcessor(unittest.TestCase): 'lb_ip': '4.5.6.7'}) def test_process_one_access_file(self): + return p = log_processor.LogProcessor(self.proxy_config, DumbLogger()) def get_object_data(*a,**kw): return [self.access_test_line] @@ -95,6 +97,7 @@ class TestLogProcessor(unittest.TestCase): self.assertEquals(result, expected) def test_process_one_stats_file(self): + return p = log_processor.LogProcessor(self.proxy_config, DumbLogger()) def get_object_data(*a,**kw): return [self.stats_test_line] @@ -108,35 +111,40 @@ class TestLogProcessor(unittest.TestCase): 'y/m/d/h/f') self.assertEquals(result, expected) - def test_get_data_listing(self): + def test_get_container_listing(self): p = log_processor.LogProcessor(self.proxy_config, DumbLogger()) - p.private_proxy = DumbPrivateProxy() - result = p.get_data_listing('foo') + p.internal_proxy = DumbInternalProxy() + result = p.get_container_listing('a', 'foo') expected = ['2010/03/14/13/obj1'] self.assertEquals(result, expected) - result = p.get_data_listing('foo', listing_filter=expected) + result = p.get_container_listing('a', 'foo', listing_filter=expected) expected = [] self.assertEquals(result, expected) - result = p.get_data_listing('foo', start_date='2010031412', + result = p.get_container_listing('a', 'foo', start_date='2010031412', end_date='2010031414') expected = ['2010/03/14/13/obj1'] self.assertEquals(result, expected) - result = p.get_data_listing('foo', start_date='2010031414') + result = p.get_container_listing('a', 'foo', start_date='2010031414') + expected = [] + self.assertEquals(result, expected) + result = p.get_container_listing('a', 'foo', start_date='2010031410', + end_date='2010031412') expected = [] self.assertEquals(result, expected) def test_get_object_data(self): p = log_processor.LogProcessor(self.proxy_config, DumbLogger()) - p.private_proxy = DumbPrivateProxy() - result = list(p.get_object_data('c', 'o', False)) + p.internal_proxy = DumbInternalProxy() + result = list(p.get_object_data('a', 'c', 'o', False)) expected = ['obj','data'] self.assertEquals(result, expected) - result = list(p.get_object_data('c', 'o.gz', True)) + result = list(p.get_object_data('a', 'c', 'o.gz', True)) self.assertEquals(result, expected) def test_get_stat_totals(self): + return p = log_processor.LogProcessor(self.proxy_config, DumbLogger()) - p.private_proxy = DumbPrivateProxy() + p.internal_proxy = DumbInternalProxy() def get_object_data(*a,**kw): return [self.stats_test_line] p.get_object_data = get_object_data @@ -150,8 +158,9 @@ class TestLogProcessor(unittest.TestCase): self.assertEquals(result, expected) def test_get_aggr_access_logs(self): + return p = log_processor.LogProcessor(self.proxy_config, DumbLogger()) - p.private_proxy = DumbPrivateProxy() + p.internal_proxy = DumbInternalProxy() def get_object_data(*a,**kw): return [self.access_test_line] p.get_object_data = get_object_data From fc9e26a44e4d01cf2c0d44e472f282d6a4ad5cf2 Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Wed, 1 Sep 2010 16:33:44 -0500 Subject: [PATCH 07/89] access log parsing tests pass --- etc/log-processing.conf-sample | 3 +++ swift/stats/access_processor.py | 22 +++++++++++----- test/unit/stats/test_log_processor.py | 38 +++++++++++++++++---------- 3 files changed, 43 insertions(+), 20 deletions(-) diff --git a/etc/log-processing.conf-sample b/etc/log-processing.conf-sample index 79d48dc3d6..6ea8d2b0cb 100644 --- a/etc/log-processing.conf-sample +++ b/etc/log-processing.conf-sample @@ -17,6 +17,9 @@ source_filename_format = %Y%m%d%H* class_path = swift.stats.access_processor.AccessLogProcessor # service ips is for client ip addresses that should be counted as servicenet # service_ips = +# load balancer private ips is for load balancer ip addresses that should be +# counted as servicenet +# lb_private_ips = # server_name = proxy [log-processor-stats] diff --git a/swift/stats/access_processor.py b/swift/stats/access_processor.py index 0fe254a4ca..a20d1741ed 100644 --- a/swift/stats/access_processor.py +++ b/swift/stats/access_processor.py @@ -13,12 +13,25 @@ # See the License for the specific language governing permissions and # limitations under the License. +import collections +from urllib import unquote + +from swift.common.utils import split_path + +month_map = '_ Jan Feb Mar Apr May Jun Jul Aug Sep Oct Nov Dec'.split() + class AccessLogProcessor(object): def __init__(self, conf): self.server_name = conf.get('server_name', 'proxy') + self.lb_private_ips = [x.strip() for x in \ + conf.get('lb_private_ips', '').split(',')\ + if x.strip()] + self.service_ips = [x.strip() for x in \ + conf.get('service_ips', '').split(',')\ + if x.strip()] - def _log_line_parser(self, raw_log): + def log_line_parser(self, raw_log): '''given a raw access log line, return a dict of the good parts''' d = {} try: @@ -106,11 +119,8 @@ class AccessLogProcessor(object): def process(self, obj_stream): '''generate hourly groupings of data from one access log file''' hourly_aggr_info = {} - aggr_account_logs = {} - container_line_counts = collections.defaultdict(int) - log_buffer = collections.defaultdict(list) for line in obj_stream: - line_data = self._log_line_parser(line) + line_data = self.log_line_parser(line) if not line_data: continue account = line_data['account'] @@ -165,4 +175,4 @@ class AccessLogProcessor(object): d[key] = d.setdefault(key, 0) + 1 hourly_aggr_info[aggr_key] = d - return hourly_aggr_info, item, aggr_account_logs + return hourly_aggr_info diff --git a/test/unit/stats/test_log_processor.py b/test/unit/stats/test_log_processor.py index 0f7699abd6..ca99890908 100644 --- a/test/unit/stats/test_log_processor.py +++ b/test/unit/stats/test_log_processor.py @@ -35,7 +35,7 @@ class TestLogProcessor(unittest.TestCase): access_test_line = 'Jul 9 04:14:30 saio proxy 1.2.3.4 4.5.6.7 '\ '09/Jul/2010/04/14/30 GET '\ - '/v1/AUTH_acct/foo/bar?format=json&foo HTTP/1.0 200 - '\ + '/v1/acct/foo/bar?format=json&foo HTTP/1.0 200 - '\ 'curl tk4e350daf-9338-4cc6-aabb-090e49babfbd '\ '6 95 - txfa431231-7f07-42fd-8fc7-7da9d8cc1f90 - 0.0262' stats_test_line = 'account,1,2,3' @@ -44,10 +44,16 @@ class TestLogProcessor(unittest.TestCase): } } - def test_log_line_parser(self): - return - p = log_processor.LogProcessor(self.proxy_config, DumbLogger()) - result = p.log_line_parser(self.access_test_line) + def test_access_log_line_parser(self): + access_proxy_config = self.proxy_config + access_proxy_config.update({ + 'log-processor-access': { + 'source_filename_format':'%Y%m%d%H*', + 'class_path': + 'swift.stats.access_processor.AccessLogProcessor' + }}) + p = log_processor.LogProcessor(access_proxy_config, DumbLogger()) + result = p.plugins['access']['instance'].log_line_parser(self.access_test_line) self.assertEquals(result, {'code': 200, 'processing_time': '0.0262', 'auth_token': 'tk4e350daf-9338-4cc6-aabb-090e49babfbd', @@ -69,22 +75,27 @@ class TestLogProcessor(unittest.TestCase): 'day': '09', 'minute': '14', 'account': 'acct', - 'reseller': 'AUTH', 'hour': '04', 'referrer': '-', - 'request': '/v1/AUTH_acct', + 'request': '/v1/acct/foo/bar', 'user_agent': 'curl', 'bytes_in': 6, 'lb_ip': '4.5.6.7'}) def test_process_one_access_file(self): - return - p = log_processor.LogProcessor(self.proxy_config, DumbLogger()) - def get_object_data(*a,**kw): + access_proxy_config = self.proxy_config + access_proxy_config.update({ + 'log-processor-access': { + 'source_filename_format':'%Y%m%d%H*', + 'class_path': + 'swift.stats.access_processor.AccessLogProcessor' + }}) + p = log_processor.LogProcessor(access_proxy_config, DumbLogger()) + def get_object_data(*a, **kw): return [self.access_test_line] p.get_object_data = get_object_data - result = p.process_one_access_file('yarr', None) - expected = ({('AUTH_acct', '2010', '07', '09', '04'): + result = p.process_one_file('access', 'a', 'c', 'o') + expected = {('acct', '2010', '07', '09', '04'): {('public', 'object', 'GET', '2xx'): 1, ('public', 'bytes_out'): 95, 'marker_query': 0, @@ -92,8 +103,7 @@ class TestLogProcessor(unittest.TestCase): 'delimiter_query': 0, 'path_query': 0, ('public', 'bytes_in'): 6, - 'prefix_query': 0}}, - 'yarr', {}) + 'prefix_query': 0}} self.assertEquals(result, expected) def test_process_one_stats_file(self): From 7763e541eb3d1fa727828eac9e15d0857b892184 Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Wed, 1 Sep 2010 17:06:01 -0500 Subject: [PATCH 08/89] added (working) stats tests --- swift/stats/stats_processor.py | 11 ++---- test/unit/stats/test_log_processor.py | 55 ++++++--------------------- 2 files changed, 16 insertions(+), 50 deletions(-) diff --git a/swift/stats/stats_processor.py b/swift/stats/stats_processor.py index 7f94e53b66..4921915176 100644 --- a/swift/stats/stats_processor.py +++ b/swift/stats/stats_processor.py @@ -21,7 +21,6 @@ class StatsLogProcessor(object): def process(self, obj_stream): '''generate hourly groupings of data from one stats log file''' account_totals = {} - year, month, day, hour, _ = item.split('/') for line in obj_stream: if not line: continue @@ -32,13 +31,11 @@ class StatsLogProcessor(object): bytes_used, created_at) = line.split(',') account = account.strip('"') - if account_name and account_name != account: - continue container_count = int(container_count.strip('"')) object_count = int(object_count.strip('"')) bytes_used = int(bytes_used.strip('"')) - aggr_key = (account, year, month, day, hour) - d = account_totals.get(aggr_key, {}) + created_at = created_at.strip('"') + d = account_totals.get(account, {}) d['count'] = d.setdefault('count', 0) + 1 d['container_count'] = d.setdefault('container_count', 0) + \ container_count @@ -47,8 +44,8 @@ class StatsLogProcessor(object): d['bytes_used'] = d.setdefault('bytes_used', 0) + \ bytes_used d['created_at'] = created_at - account_totals[aggr_key] = d + account_totals[account] = d except (IndexError, ValueError): # bad line data pass - return account_totals, item + return account_totals diff --git a/test/unit/stats/test_log_processor.py b/test/unit/stats/test_log_processor.py index ca99890908..24a37a5877 100644 --- a/test/unit/stats/test_log_processor.py +++ b/test/unit/stats/test_log_processor.py @@ -38,7 +38,7 @@ class TestLogProcessor(unittest.TestCase): '/v1/acct/foo/bar?format=json&foo HTTP/1.0 200 - '\ 'curl tk4e350daf-9338-4cc6-aabb-090e49babfbd '\ '6 95 - txfa431231-7f07-42fd-8fc7-7da9d8cc1f90 - 0.0262' - stats_test_line = 'account,1,2,3' + stats_test_line = 'account,1,2,3,1283378584.881391' proxy_config = {'log-processor': { } @@ -106,21 +106,6 @@ class TestLogProcessor(unittest.TestCase): 'prefix_query': 0}} self.assertEquals(result, expected) - def test_process_one_stats_file(self): - return - p = log_processor.LogProcessor(self.proxy_config, DumbLogger()) - def get_object_data(*a,**kw): - return [self.stats_test_line] - p.get_object_data = get_object_data - result = p.process_one_stats_file('y/m/d/h/f', None) - expected = ({('account', 'y', 'm', 'd', 'h'): - {'count': 1, - 'object_count': 2, - 'container_count': 1, - 'bytes_used': 3}}, - 'y/m/d/h/f') - self.assertEquals(result, expected) - def test_get_container_listing(self): p = log_processor.LogProcessor(self.proxy_config, DumbLogger()) p.internal_proxy = DumbInternalProxy() @@ -152,38 +137,22 @@ class TestLogProcessor(unittest.TestCase): self.assertEquals(result, expected) def test_get_stat_totals(self): - return - p = log_processor.LogProcessor(self.proxy_config, DumbLogger()) + stats_proxy_config = self.proxy_config + stats_proxy_config.update({ + 'log-processor-stats': { + 'class_path': + 'swift.stats.stats_processor.StatsLogProcessor' + }}) + p = log_processor.LogProcessor(stats_proxy_config, DumbLogger()) p.internal_proxy = DumbInternalProxy() def get_object_data(*a,**kw): return [self.stats_test_line] p.get_object_data = get_object_data - result = list(p.get_stat_totals()) - expected = [({('account', '2010', '03', '14', '13'): + result = p.process_one_file('stats', 'a', 'c', 'o') + expected = {'account': {'count': 1, 'object_count': 2, 'container_count': 1, - 'bytes_used': 3}}, - '2010/03/14/13/obj1')] - self.assertEquals(result, expected) - - def test_get_aggr_access_logs(self): - return - p = log_processor.LogProcessor(self.proxy_config, DumbLogger()) - p.internal_proxy = DumbInternalProxy() - def get_object_data(*a,**kw): - return [self.access_test_line] - p.get_object_data = get_object_data - result = list(p.get_aggr_access_logs()) - expected = [({('AUTH_7abbc116-8a07-4b63-819d-02715d3e0f31', '2010', '07', '09', '04'): - {('public', 'object', 'GET', '2xx'): 1, - ('public', 'bytes_out'): 95, - 'marker_query': 0, - 'format_query': 1, - 'delimiter_query': 0, - 'path_query': 0, - ('public', 'bytes_in'): 0, - 'prefix_query': 0}}, - '2010/03/14/13/obj1', - {})] + 'bytes_used': 3, + 'created_at': '1283378584.881391'}} self.assertEquals(result, expected) \ No newline at end of file From 560243f2f05d1622ee70411b614587466cd506ea Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Tue, 7 Sep 2010 09:53:26 -0500 Subject: [PATCH 09/89] updated stats binaries to be DRY compliant --- bin/swift-account-stats-logger | 58 ++------------------------------ bin/swift-log-uploader | 61 +++------------------------------- swift/stats/account_stats.py | 22 +++++++++--- swift/stats/log_uploader.py | 26 ++++++++++++--- 4 files changed, 47 insertions(+), 120 deletions(-) diff --git a/bin/swift-account-stats-logger b/bin/swift-account-stats-logger index 050545af9a..fc7b128270 100644 --- a/bin/swift-account-stats-logger +++ b/bin/swift-account-stats-logger @@ -14,11 +14,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -import os -import signal import sys -import time -from ConfigParser import ConfigParser from swift.account_stats import AccountStat from swift.common import utils @@ -27,55 +23,5 @@ if __name__ == '__main__': if len(sys.argv) < 2: print "Usage: swift-account-stats-logger CONFIG_FILE" sys.exit() - - c = ConfigParser() - if not c.read(sys.argv[1]): - print "Unable to read config file." - sys.exit(1) - - if c.has_section('log-processor-stats'): - stats_conf = dict(c.items('log-processor-stats')) - else: - print "Unable to find log-processor-stats config section in %s." % \ - sys.argv[1] - sys.exit(1) - - # reference this from the account stats conf - - target_dir = stats.conf.get('log_dir', '/var/log/swift') - account_server_conf_loc = stats_conf.get('account_server_conf', - '/etc/swift/account-server.conf') - filename_format = stats.conf['source_filename_format'] - try: - c = ConfigParser() - c.read(account_server_conf_loc) - account_server_conf = dict(c.items('account-server')) - except: - print "Unable to load account server conf from %s" % account_server_conf_loc - sys.exit(1) - - utils.drop_privileges(account_server_conf.get('user', 'swift')) - - try: - os.setsid() - except OSError: - pass - - logger = utils.get_logger(stats_conf, 'swift-account-stats-logger') - - def kill_children(*args): - signal.signal(signal.SIGTERM, signal.SIG_IGN) - os.killpg(0, signal.SIGTERM) - sys.exit() - - signal.signal(signal.SIGTERM, kill_children) - - stats = AccountStat(filename_format, - target_dir, - account_server_conf, - logger) - logger.info("Gathering account stats") - start = time.time() - stats.find_and_process() - logger.info("Gathering account stats complete (%0.2f minutes)" % - ((time.time()-start)/60)) + stats_conf = utils.readconf(sys.argv[1], 'log-processor-stats') + stats = AccountStat(stats_conf).run(once=True) diff --git a/bin/swift-log-uploader b/bin/swift-log-uploader index 4ca9c17795..83c8f67303 100755 --- a/bin/swift-log-uploader +++ b/bin/swift-log-uploader @@ -14,70 +14,19 @@ # See the License for the specific language governing permissions and # limitations under the License. -import os -import signal import sys import time -from ConfigParser import ConfigParser from swift.stats.log_uploader import LogUploader -from swift.common.utils import get_logger +from swift.common import utils if __name__ == '__main__': if len(sys.argv) < 3: print "Usage: swift-log-uploader CONFIG_FILE plugin" sys.exit() - - c = ConfigParser() - if not c.read(sys.argv[1]): - print "Unable to read config file." - sys.exit(1) - - if c.has_section('log-processor'): - parser_conf = dict(c.items('log-processor')) - else: - print "Unable to find log-processor config section in %s." % sys.argv[1] - sys.exit(1) - + uploader_conf = utils.readconf(sys.argv[1], 'log-processor') plugin = sys.argv[2] section_name = 'log-processor-%s' % plugin - if c.has_section(section_name): - uploader_conf.update(dict(c.items(section_name))) - else: - print "Unable to find %s config section in %s." % (section_name, - sys.argv[1]) - sys.exit(1) - - try: - os.setsid() - except OSError: - pass - - logger = get_logger(uploader_conf, 'swift-log-uploader') - - def kill_children(*args): - signal.signal(signal.SIGTERM, signal.SIG_IGN) - os.killpg(0, signal.SIGTERM) - sys.exit() - - signal.signal(signal.SIGTERM, kill_children) - - log_dir = uploader_conf.get('log_dir', '/var/log/swift/') - swift_account = uploader_conf['swift_account'] - container_name = uploader_conf['container_name'] - source_filename_format = uploader_conf['source_filename_format'] - proxy_server_conf_loc = uploader_conf.get('proxy_server_conf', - '/etc/swift/proxy-server.conf') - try: - c = ConfigParser() - c.read(proxy_server_conf_loc) - proxy_server_conf = dict(c.items('proxy-server')) - except: - proxy_server_conf = None - uploader = LogUploader(log_dir, swift_account, container_name, - source_filename_format, proxy_server_conf, logger) - logger.info("Uploading logs") - start = time.time() - uploader.upload_all_logs() - logger.info("Uploading logs complete (%0.2f minutes)" % - ((time.time()-start)/60)) + plugin_conf = utils.readconf(sys.argv[1], section_name) + uploader_conf.update(plugin_conf) + uploader = LogUploader(uploader_conf, plugin).run(once=True) diff --git a/swift/stats/account_stats.py b/swift/stats/account_stats.py index 1624807cfe..a514aec2e8 100644 --- a/swift/stats/account_stats.py +++ b/swift/stats/account_stats.py @@ -19,16 +19,30 @@ import time from swift.account.server import DATADIR as account_server_data_dir from swift.common.db import AccountBroker from swift.common.internal_proxy import InternalProxy -from swift.common.utils import renamer +from swift.common.utils import renamer, get_logger +from swift.common.daemon import Daemon -class AccountStat(object): - def __init__(self, filename_format, target_dir, server_conf, logger): +class AccountStat(Daemon): + def __init__(self, stats_conf): + super(self, AccountStat).__init__(stats_conf) + target_dir = stats_conf.get('log_dir', '/var/log/swift') + account_server_conf_loc = stats_conf.get('account_server_conf', + '/etc/swift/account-server.conf') + server_conf = utils.readconf(account_server_conf_loc, 'account-server') + filename_format = stats_conf['source_filename_format'] self.filename_format = filename_format self.target_dir = target_dir self.devices = server_conf.get('devices', '/srv/node') self.mount_check = server_conf.get('mount_check', 'true').lower() in \ ('true', 't', '1', 'on', 'yes', 'y') - self.logger = logger + self.logger = get_logger(stats_conf, 'swift-account-stats-logger') + + def run_once(self): + self.logger.info("Gathering account stats") + start = time.time() + self.find_and_process() + self.logger.info("Gathering account stats complete (%0.2f minutes)" % + ((time.time()-start)/60)) def find_and_process(self): src_filename = time.strftime(self.filename_format) diff --git a/swift/stats/log_uploader.py b/swift/stats/log_uploader.py index 8c45d64db5..6634611767 100644 --- a/swift/stats/log_uploader.py +++ b/swift/stats/log_uploader.py @@ -21,8 +21,10 @@ import gzip import glob from swift.common.internal_proxy import InternalProxy +from swift.common.daemon import Daemon +from swift.common import utils -class LogUploader(object): +class LogUploader(Daemon): ''' Given a local directory, a swift account, and a container name, LogParser will upload all files in the local directory to the given account/container. @@ -38,8 +40,16 @@ class LogUploader(object): the object uploads. ''' - def __init__(self, log_dir, swift_account, container_name, filename_format, - proxy_server_conf, logger): + def __init__(self, uploader_conf, plugin_name): + super(self, LogUploader).__init__(uploader_conf) + log_dir = uploader_conf.get('log_dir', '/var/log/swift/') + swift_account = uploader_conf['swift_account'] + container_name = uploader_conf['container_name'] + source_filename_format = uploader_conf['source_filename_format'] + proxy_server_conf_loc = uploader_conf.get('proxy_server_conf', + '/etc/swift/proxy-server.conf') + proxy_server_conf = utils.readconf(proxy_server_conf_loc, + 'proxy-server') if not log_dir.endswith('/'): log_dir = log_dir + '/' self.log_dir = log_dir @@ -47,7 +57,15 @@ class LogUploader(object): self.container_name = container_name self.filename_format = filename_format self.internal_proxy = InternalProxy(proxy_server_conf, logger) - self.logger = logger + log_name = 'swift-log-uploader-%s' % plugin_name + self.logger = utils.get_logger(uploader_conf, plugin_name) + + def run_once(self): + self.logger.info("Uploading logs") + start = time.time() + self.upload_all_logs() + self.logger.info("Uploading logs complete (%0.2f minutes)" % + ((time.time()-start)/60)) def upload_all_logs(self): i = [(c,self.filename_format.index(c)) for c in '%Y %m %d %H'.split()] From d8ad8ae473c84f21ca98d6583b3ca7d1b6b9e5a5 Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Fri, 10 Sep 2010 15:08:06 -0500 Subject: [PATCH 10/89] set up log-stats-collector as a daemon process to create csv files --- bin/swift-log-stats-collector | 27 +++++ bin/swift-log-uploader | 1 - etc/log-processing.conf-sample | 4 +- swift/common/utils.py | 27 +++-- swift/stats/access_processor.py | 48 ++++++++- swift/stats/account_stats.py | 10 +- swift/stats/log_processor.py | 186 ++++++++++++++++++++++++++------ swift/stats/stats_processor.py | 21 +++- 8 files changed, 265 insertions(+), 59 deletions(-) create mode 100644 bin/swift-log-stats-collector diff --git a/bin/swift-log-stats-collector b/bin/swift-log-stats-collector new file mode 100644 index 0000000000..d21135b35c --- /dev/null +++ b/bin/swift-log-stats-collector @@ -0,0 +1,27 @@ +#!/usr/bin/python +# Copyright (c) 2010 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 sys + +from swift.stats.log_processor import LogProcessorDaemon +from swift.common import utils + +if __name__ == '__main__': + if len(sys.argv) < 2: + print "Usage: swift-log-stats-collector CONFIG_FILE" + sys.exit() + conf = utils.readconf(sys.argv[1], log_name='log-stats-collector') + stats = LogProcessorDaemon(conf).run(once=True) diff --git a/bin/swift-log-uploader b/bin/swift-log-uploader index 83c8f67303..b557e4c167 100755 --- a/bin/swift-log-uploader +++ b/bin/swift-log-uploader @@ -15,7 +15,6 @@ # limitations under the License. import sys -import time from swift.stats.log_uploader import LogUploader from swift.common import utils diff --git a/etc/log-processing.conf-sample b/etc/log-processing.conf-sample index 6ea8d2b0cb..97d33c15d3 100644 --- a/etc/log-processing.conf-sample +++ b/etc/log-processing.conf-sample @@ -1,8 +1,8 @@ # plugin section format is named "log-processor-" -# section "log-processor" is the generic defaults (overridden by plugins) [log-processor] -# working_dir = /tmp/swift/ +swift_account = AUTH_7abbc116-8a07-4b63-819d-02715d3e0f31 +# container_name = log_processing_data # proxy_server_conf = /etc/swift/proxy-server.conf # log_facility = LOG_LOCAL0 # log_level = INFO diff --git a/swift/common/utils.py b/swift/common/utils.py index 49813be668..5e0f62269a 100644 --- a/swift/common/utils.py +++ b/swift/common/utils.py @@ -530,19 +530,26 @@ def item_from_env(env, item_name): def cache_from_env(env): return item_from_env(env, 'swift.cache') -def readconf(conf, section_name, log_name=None): +def readconf(conf, section_name=None, log_name=None): c = ConfigParser() if not c.read(conf): print "Unable to read config file %s" % conf sys.exit(1) - if c.has_section(section_name): - conf = dict(c.items(section_name)) - else: - print "Unable to find %s config section in %s" % (section_name, conf) - sys.exit(1) - if "log_name" not in conf: - if log_name is not None: - conf['log_name'] = log_name + if section_name: + if c.has_section(section_name): + conf = dict(c.items(section_name)) else: - conf['log_name'] = section_name + print "Unable to find %s config section in %s" % (section_name, conf) + sys.exit(1) + if "log_name" not in conf: + if log_name is not None: + conf['log_name'] = log_name + else: + conf['log_name'] = section_name + else: + conf = {} + for s in c.sections(): + conf.update({s:dict(c.items(s))}) + if 'log_name' not in conf: + conf['log_name'] = log_name return conf diff --git a/swift/stats/access_processor.py b/swift/stats/access_processor.py index a20d1741ed..08d2f1bf08 100644 --- a/swift/stats/access_processor.py +++ b/swift/stats/access_processor.py @@ -15,6 +15,7 @@ import collections from urllib import unquote +import copy from swift.common.utils import split_path @@ -116,7 +117,7 @@ class AccessLogProcessor(object): d['code'] = int(d['code']) return d - def process(self, obj_stream): + def process(self, obj_stream, account, container, object_name): '''generate hourly groupings of data from one access log file''' hourly_aggr_info = {} for line in obj_stream: @@ -176,3 +177,48 @@ class AccessLogProcessor(object): hourly_aggr_info[aggr_key] = d return hourly_aggr_info + + def keylist_mapping(self): + source_keys = 'service public'.split() + level_keys = 'account container object'.split() + verb_keys = 'GET PUT POST DELETE HEAD COPY'.split() + code_keys = '2xx 4xx 5xx'.split() + + keylist_mapping = { + # : or + 'service_bw_in': ('service', 'bytes_in'), + 'service_bw_out': ('service', 'bytes_out'), + 'public_bw_in': ('public', 'bytes_in'), + 'public_bw_out': ('public', 'bytes_out'), + 'account_requests': set(), + 'container_requests': set(), + 'object_requests': set(), + 'service_request': set(), + 'public_request': set(), + 'ops_count': set(), + } + for verb in verb_keys: + keylist_mapping[verb] = set() + for code in code_keys: + keylist_mapping[code] = set() + for source in source_keys: + for level in level_keys: + for verb in verb_keys: + for code in code_keys: + keylist_mapping['account_requests'].add( + (source, 'account', verb, code)) + keylist_mapping['container_requests'].add( + (source, 'container', verb, code)) + keylist_mapping['object_requests'].add( + (source, 'object', verb, code)) + keylist_mapping['service_request'].add( + ('service', level, verb, code)) + keylist_mapping['public_request'].add( + ('public', level, verb, code)) + keylist_mapping[verb].add( + (source, level, verb, code)) + keylist_mapping[code].add( + (source, level, verb, code)) + keylist_mapping['ops_count'].add( + (source,level,verb,code)) + return keylist_mapping diff --git a/swift/stats/account_stats.py b/swift/stats/account_stats.py index a514aec2e8..12c473cfdf 100644 --- a/swift/stats/account_stats.py +++ b/swift/stats/account_stats.py @@ -48,7 +48,7 @@ class AccountStat(Daemon): src_filename = time.strftime(self.filename_format) tmp_filename = os.path.join('/tmp', src_filename) with open(tmp_filename, 'wb') as statfile: - #statfile.write('Account Name, Container Count, Object Count, Bytes Used, Created At\n') + #statfile.write('Account Name, Container Count, Object Count, Bytes Used\n') for device in os.listdir(self.devices): if self.mount_check and \ not os.path.ismount(os.path.join(self.devices, device)): @@ -68,16 +68,14 @@ class AccountStat(Daemon): broker = AccountBroker(os.path.join(root, filename)) if not broker.is_deleted(): account_name, - created_at, - _, _, + _, _, _, container_count, object_count, bytes_used, _, _ = broker.get_info() - line_data = '"%s",%d,%d,%d,%s\n' % (account_name, + line_data = '"%s",%d,%d,%d\n' % (account_name, container_count, object_count, - bytes_used, - created_at) + bytes_used) statfile.write(line_data) renamer(tmp_filename, os.path.join(self.target_dir, src_filename)) diff --git a/swift/stats/log_processor.py b/swift/stats/log_processor.py index 10fd456757..66344ae106 100644 --- a/swift/stats/log_processor.py +++ b/swift/stats/log_processor.py @@ -15,36 +15,23 @@ from ConfigParser import ConfigParser import zlib +import time +import datetime +import cStringIO +import collections from swift.common.internal_proxy import InternalProxy from swift.common.exceptions import ChunkReadTimeout -from swift.common.utils import get_logger - -class ConfigError(Exception): - pass - -class MissingProxyConfig(ConfigError): - pass +from swift.common.utils import get_logger, readconf class LogProcessor(object): def __init__(self, conf, logger): stats_conf = conf.get('log-processor', {}) - working_dir = stats_conf.get('working_dir', '/tmp/swift/') - if working_dir.endswith('/') and len(working_dir) > 1: - working_dir = working_dir[:-1] - self.working_dir = working_dir proxy_server_conf_loc = stats_conf.get('proxy_server_conf', '/etc/swift/proxy-server.conf') - try: - c = ConfigParser() - c.read(proxy_server_conf_loc) - proxy_server_conf = dict(c.items('proxy-server')) - except: - raise - raise MissingProxyConfig() - self.proxy_server_conf = proxy_server_conf + self.proxy_server_conf = readconf(proxy_server_conf_loc, 'proxy-server') if isinstance(logger, tuple): self.logger = get_logger(*logger) else: @@ -71,7 +58,10 @@ class LogProcessor(object): stream = self.get_object_data(account, container, object_name, compressed=compressed) # look up the correct plugin and send the stream to it - return self.plugins[plugin_name]['instance'].process(stream) + return self.plugins[plugin_name]['instance'].process(stream, + account, + container, + object_name) def get_data_list(self, start_date=None, end_date=None, listing_filter=None): total_list = [] @@ -81,6 +71,8 @@ class LogProcessor(object): l = self.get_container_listing(account, container, start_date, end_date, listing_filter) for i in l: + # The items in this list end up being passed as positional + # parameters to process_one_file. total_list.append((p, account, container, i)) return total_list @@ -174,21 +166,146 @@ class LogProcessor(object): except ChunkReadTimeout: raise BadFileDownload() -def multiprocess_collate(processor_args, - start_date=None, - end_date=None, - listing_filter=None): - '''get listing of files and yield hourly data from them''' - p = LogProcessor(*processor_args) - all_files = p.get_data_list(start_date, end_date, listing_filter) + def generate_keylist_mapping(self): + keylist = {} + for plugin in self.plugins: + plugin_keylist = self.plugins['instance'].keylist_mapping() + for k, v in plugin_keylist.items(): + o = keylist.get(k) + if o: + if isinstance(o, set): + if isinstance(v, set): + o.update(v) + else: + o.update([v]) + else: + o = set(o) + if isinstance(v, set): + o.update(v) + else: + o.update([v]) + else: + o = v + keylist[k] = o + return keylist - p.logger.info('loaded %d files to process' % len(all_files)) - if not all_files: - # no work to do - return +class LogProcessorDaemon(Daemon): + def __init__(self, conf): + super(self, LogProcessorDaemon).__init__(conf) + self.log_processor = LogProcessor(conf, self.logger) + c = readconf(conf) + self.lookback_hours = int(c.get('lookback_hours', '120')) + self.lookback_window = int(c.get('lookback_window', '%s'%lookback_hours)) + self.log_processor_account = c['swift_account'] + self.log_processor_container = c.get('container_name', 'log_processing_data') - worker_count = multiprocessing.cpu_count() - 1 + def run_once(self): + self.logger.info("Beginning log processing") + start = time.time() + if lookback_hours == 0: + lookback_start = None + lookback_end = None + else: + lookback_start = datetime.datetime.now() - \ + datetime.timedelta(hours=lookback_hours) + lookback_start = lookback_start.strftime('%Y%m%d') + if lookback_window == 0: + lookback_end = None + else: + lookback_end = datetime.datetime.now() - \ + datetime.timedelta(hours=lookback_hours) + \ + datetime.timedelta(hours=lookback_window) + lookback_end = lookback_end.strftime('%Y%m%d') + + try: + processed_files_stream = self.log_processor,get_object_data( + self.log_processor_account, + self.log_processor_container, + 'processed_files.pickle.gz', + compressed=True) + buf = ''.join(x for x in processed_files_stream) + already_processed_files = cPickle.loads(buf) + except: + already_processed_files = set() + + logs_to_process = self.log_processor.get_data_list(lookback_start, + lookback_end, + already_processed_files) + self.logger.info('loaded %d files to process' % len(logs_to_process)) + if not logs_to_process: + self.logger.info("Log processing done (%0.2f minutes)" % + ((time.time()-start)/60)) + return + + # map + processor_args = (conf, self.logger) + results = multiprocess_collate(processor_args, logs_to_process) + + #reduce + aggr_data = {} + processed_files = already_processed_files + for item, data in results.items(): + # since item contains the plugin and the log name, new plugins will + # "reprocess" the file and the results will be in the final csv. + processed_files.append(item) + for k, d in data.items(): + existing_data = aggr_data.get(k, {}) + for i, j in d.items(): + current = existing_data.get(i, 0) + # merging strategy for key collisions is addition + # processing plugins need to realize this + existing_data[i] = current + j + aggr_data[k] = existing_data + + # group + # reduce a large number of keys in aggr_data[k] to a small number of + # output keys + keylist_mapping = generate_keylist_mapping() + final_info = collections.defaultdict(dict) + for account, data in rows.items(): + for key, mapping in keylist_mapping.items(): + if isinstance(mapping, (list, set)): + value = 0 + for k in mapping: + try: + value += data[k] + except KeyError: + pass + else: + try: + value = data[mapping] + except KeyError: + value = 0 + final_info[account][key] = value + + # output + sorted_keylist_mapping = sorted(keylist_mapping) + columns = 'bill_ts,data_ts,account,' + ','.join(sorted_keylist_mapping) + print columns + for (account, year, month, day, hour), d in final_info.items(): + bill_ts = '' + data_ts = '%s/%s/%s %s:00:00' % (year, month, day, hour) + row = [bill_ts, data_ts] + row.append('%s' % account) + for k in sorted_keylist_mapping: + row.append('%s'%d[k]) + print ','.join(row) + + # cleanup + s = cPickle.dumps(processed_files, cPickle.HIGHEST_PROTOCOL) + f = cStringIO.StringIO(s) + self.log_processor.internal_proxy.upload_file(s, + self.log_processor_account, + self.log_processor_container, + 'processed_files.pickle.gz') + + self.logger.info("Log processing done (%0.2f minutes)" % + ((time.time()-start)/60)) + +def multiprocess_collate(processor_args, logs_to_process): + '''yield hourly data from logs_to_process''' + worker_count = multiprocessing.cpu_count() results = [] in_queue = multiprocessing.Queue() out_queue = multiprocessing.Queue() @@ -199,7 +316,7 @@ def multiprocess_collate(processor_args, out_queue)) p.start() results.append(p) - for x in all_files: + for x in logs_to_process: in_queue.put(x) for _ in range(worker_count): in_queue.put(None) @@ -229,6 +346,5 @@ def collate_worker(processor_args, in_queue, out_queue): except Queue.Empty: time.sleep(.1) else: - ret = None - ret = p.process_one_file(item) + ret = p.process_one_file(*item) out_queue.put((item, ret)) \ No newline at end of file diff --git a/swift/stats/stats_processor.py b/swift/stats/stats_processor.py index 4921915176..c7b89c11e6 100644 --- a/swift/stats/stats_processor.py +++ b/swift/stats/stats_processor.py @@ -18,9 +18,10 @@ class StatsLogProcessor(object): def __init__(self, conf): pass - def process(self, obj_stream): + def process(self, obj_stream, account, container, object_name): '''generate hourly groupings of data from one stats log file''' account_totals = {} + year, month, day, hour, _ = object_name.split('/') for line in obj_stream: if not line: continue @@ -35,17 +36,29 @@ class StatsLogProcessor(object): object_count = int(object_count.strip('"')) bytes_used = int(bytes_used.strip('"')) created_at = created_at.strip('"') - d = account_totals.get(account, {}) - d['count'] = d.setdefault('count', 0) + 1 + aggr_key = (account, year, month, day, hour) + d = account_totals.get(aggr_key, {}) + d['replica_count'] = d.setdefault('count', 0) + 1 d['container_count'] = d.setdefault('container_count', 0) + \ container_count d['object_count'] = d.setdefault('object_count', 0) + \ object_count d['bytes_used'] = d.setdefault('bytes_used', 0) + \ bytes_used - d['created_at'] = created_at account_totals[account] = d except (IndexError, ValueError): # bad line data pass return account_totals + + def keylist_mapping(self): + ''' + returns a dictionary of final keys mapped to source keys + ''' + keylist_mapping = { + # : or + 'bytes_used': 'bytes_used', + 'container_count': 'container_count', + 'object_count': 'object_count', + 'replica_count': 'replica_count', + } From e94faaff7f81091a566d3b2b07ad6015d1424662 Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Sat, 11 Sep 2010 00:01:04 -0500 Subject: [PATCH 11/89] added execute perms to stats processor binaries --- bin/swift-account-stats-logger | 0 bin/swift-log-stats-collector | 0 2 files changed, 0 insertions(+), 0 deletions(-) mode change 100644 => 100755 bin/swift-account-stats-logger mode change 100644 => 100755 bin/swift-log-stats-collector diff --git a/bin/swift-account-stats-logger b/bin/swift-account-stats-logger old mode 100644 new mode 100755 diff --git a/bin/swift-log-stats-collector b/bin/swift-log-stats-collector old mode 100644 new mode 100755 From 25c775516c9f60abdd22ec329cb6adab5037092f Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Mon, 13 Sep 2010 11:11:12 -0500 Subject: [PATCH 12/89] updated config file loading to work with paste.deploy configs --- bin/swift-account-stats-logger | 2 +- swift/stats/account_stats.py | 9 ++++++--- swift/stats/log_processor.py | 6 ++++-- swift/stats/log_uploader.py | 7 ++++--- 4 files changed, 15 insertions(+), 9 deletions(-) diff --git a/bin/swift-account-stats-logger b/bin/swift-account-stats-logger index fc7b128270..c42554de82 100755 --- a/bin/swift-account-stats-logger +++ b/bin/swift-account-stats-logger @@ -16,7 +16,7 @@ import sys -from swift.account_stats import AccountStat +from swift.stats.account_stats import AccountStat from swift.common import utils if __name__ == '__main__': diff --git a/swift/stats/account_stats.py b/swift/stats/account_stats.py index 12c473cfdf..60b7bbf684 100644 --- a/swift/stats/account_stats.py +++ b/swift/stats/account_stats.py @@ -15,20 +15,23 @@ import os import time +from paste.deploy import appconfig from swift.account.server import DATADIR as account_server_data_dir from swift.common.db import AccountBroker from swift.common.internal_proxy import InternalProxy -from swift.common.utils import renamer, get_logger +from swift.common.utils import renamer, get_logger, readconf from swift.common.daemon import Daemon class AccountStat(Daemon): def __init__(self, stats_conf): - super(self, AccountStat).__init__(stats_conf) + super(AccountStat, self).__init__(stats_conf) target_dir = stats_conf.get('log_dir', '/var/log/swift') + #TODO: figure out the server configs. also figure out internal_proxy account_server_conf_loc = stats_conf.get('account_server_conf', '/etc/swift/account-server.conf') - server_conf = utils.readconf(account_server_conf_loc, 'account-server') + server_conf = appconfig('config:%s' % account_server_conf_loc, + name='account-server') filename_format = stats_conf['source_filename_format'] self.filename_format = filename_format self.target_dir = target_dir diff --git a/swift/stats/log_processor.py b/swift/stats/log_processor.py index 66344ae106..d02cabbbf6 100644 --- a/swift/stats/log_processor.py +++ b/swift/stats/log_processor.py @@ -19,6 +19,7 @@ import time import datetime import cStringIO import collections +from paste.deploy import appconfig from swift.common.internal_proxy import InternalProxy from swift.common.exceptions import ChunkReadTimeout @@ -31,7 +32,8 @@ class LogProcessor(object): proxy_server_conf_loc = stats_conf.get('proxy_server_conf', '/etc/swift/proxy-server.conf') - self.proxy_server_conf = readconf(proxy_server_conf_loc, 'proxy-server') + self.proxy_server_conf = appconfig('config:%s' % proxy_server_conf_loc, + name='proxy-server') if isinstance(logger, tuple): self.logger = get_logger(*logger) else: @@ -192,7 +194,7 @@ class LogProcessor(object): class LogProcessorDaemon(Daemon): def __init__(self, conf): - super(self, LogProcessorDaemon).__init__(conf) + super(LogProcessorDaemon, stats).__init__(conf) self.log_processor = LogProcessor(conf, self.logger) c = readconf(conf) self.lookback_hours = int(c.get('lookback_hours', '120')) diff --git a/swift/stats/log_uploader.py b/swift/stats/log_uploader.py index 6634611767..6385574860 100644 --- a/swift/stats/log_uploader.py +++ b/swift/stats/log_uploader.py @@ -19,6 +19,7 @@ import hashlib import time import gzip import glob +from paste.deploy import appconfig from swift.common.internal_proxy import InternalProxy from swift.common.daemon import Daemon @@ -41,15 +42,15 @@ class LogUploader(Daemon): ''' def __init__(self, uploader_conf, plugin_name): - super(self, LogUploader).__init__(uploader_conf) + super(LogUploader, self).__init__(uploader_conf) log_dir = uploader_conf.get('log_dir', '/var/log/swift/') swift_account = uploader_conf['swift_account'] container_name = uploader_conf['container_name'] source_filename_format = uploader_conf['source_filename_format'] proxy_server_conf_loc = uploader_conf.get('proxy_server_conf', '/etc/swift/proxy-server.conf') - proxy_server_conf = utils.readconf(proxy_server_conf_loc, - 'proxy-server') + proxy_server_conf = appconfig('config:%s' % proxy_server_conf_loc, + name='proxy-server') if not log_dir.endswith('/'): log_dir = log_dir + '/' self.log_dir = log_dir From de70ddce11b120be8803909367f39d8c241a2d39 Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Mon, 13 Sep 2010 13:35:19 -0500 Subject: [PATCH 13/89] fixed typos --- etc/log-processing.conf-sample | 3 +++ swift/stats/account_stats.py | 4 ++-- swift/stats/log_uploader.py | 2 +- 3 files changed, 6 insertions(+), 3 deletions(-) diff --git a/etc/log-processing.conf-sample b/etc/log-processing.conf-sample index 97d33c15d3..433613efee 100644 --- a/etc/log-processing.conf-sample +++ b/etc/log-processing.conf-sample @@ -8,6 +8,7 @@ swift_account = AUTH_7abbc116-8a07-4b63-819d-02715d3e0f31 # log_level = INFO # lookback_hours = 120 # lookback_window = 120 +# user = swift [log-processor-access] # log_dir = /var/log/swift/ @@ -21,6 +22,7 @@ class_path = swift.stats.access_processor.AccessLogProcessor # counted as servicenet # lb_private_ips = # server_name = proxy +# user = swift [log-processor-stats] # log_dir = /var/log/swift/ @@ -29,3 +31,4 @@ container_name = account_stats source_filename_format = %Y%m%d%H* class_path = swift.stats.stats_processor.StatsLogProcessor # account_server_conf = /etc/swift/account-server.conf +# user = swift \ No newline at end of file diff --git a/swift/stats/account_stats.py b/swift/stats/account_stats.py index 60b7bbf684..b436a5b12c 100644 --- a/swift/stats/account_stats.py +++ b/swift/stats/account_stats.py @@ -70,12 +70,12 @@ class AccountStat(Daemon): if filename.endswith('.db'): broker = AccountBroker(os.path.join(root, filename)) if not broker.is_deleted(): - account_name, + (account_name, _, _, _, container_count, object_count, bytes_used, - _, _ = broker.get_info() + _, _) = broker.get_info() line_data = '"%s",%d,%d,%d\n' % (account_name, container_count, object_count, diff --git a/swift/stats/log_uploader.py b/swift/stats/log_uploader.py index 6385574860..579f1901ab 100644 --- a/swift/stats/log_uploader.py +++ b/swift/stats/log_uploader.py @@ -56,7 +56,7 @@ class LogUploader(Daemon): self.log_dir = log_dir self.swift_account = swift_account self.container_name = container_name - self.filename_format = filename_format + self.filename_format = source_filename_format self.internal_proxy = InternalProxy(proxy_server_conf, logger) log_name = 'swift-log-uploader-%s' % plugin_name self.logger = utils.get_logger(uploader_conf, plugin_name) From 42d9bab9c93624e2a45878cd0c79c48147993a4c Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Tue, 14 Sep 2010 16:53:57 -0500 Subject: [PATCH 14/89] fixed internal proxy loading --- swift/common/internal_proxy.py | 2 +- swift/proxy/server.py | 9 +++++++++ swift/stats/log_uploader.py | 2 +- 3 files changed, 11 insertions(+), 2 deletions(-) diff --git a/swift/common/internal_proxy.py b/swift/common/internal_proxy.py index 3b17329598..c42b17ae16 100644 --- a/swift/common/internal_proxy.py +++ b/swift/common/internal_proxy.py @@ -32,7 +32,7 @@ class InternalProxy(object): :param retries: number of times to retry each request """ def __init__(self, proxy_server_conf=None, logger=None, retries=0): - self.upload_app = BaseApplication(proxy_server_conf, logger) + self.upload_app = BaseApplication(proxy_server_conf, logger=logger) self.retries = retries def upload_file(self, source_file, account, container, object_name, diff --git a/swift/proxy/server.py b/swift/proxy/server.py index 892c4f4769..375d481c5b 100644 --- a/swift/proxy/server.py +++ b/swift/proxy/server.py @@ -1202,6 +1202,15 @@ class BaseApplication(object): try: if self.memcache is None: self.memcache = cache_from_env(env) + if self.memcache is None: + class MemcacheStub(object): + def get(self, *a, **kw): return None + def set(self, *a, **kw): return None + def incr(self, *a, **kw): return 0 + def delete(self, *a, **kw): return None + def set_multi(self, *a, **kw): return None + def get_multi(self, *a, **kw): return [] + self.memcache = MemcacheStub() req = self.update_request(Request(env)) if 'eventlet.posthooks' in env: env['eventlet.posthooks'].append( diff --git a/swift/stats/log_uploader.py b/swift/stats/log_uploader.py index 579f1901ab..d2f4f6d7be 100644 --- a/swift/stats/log_uploader.py +++ b/swift/stats/log_uploader.py @@ -57,7 +57,7 @@ class LogUploader(Daemon): self.swift_account = swift_account self.container_name = container_name self.filename_format = source_filename_format - self.internal_proxy = InternalProxy(proxy_server_conf, logger) + self.internal_proxy = InternalProxy(proxy_server_conf) log_name = 'swift-log-uploader-%s' % plugin_name self.logger = utils.get_logger(uploader_conf, plugin_name) From e4ac6f1cbba1aacf35702d02994f8c127e096068 Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Tue, 14 Sep 2010 17:24:48 -0500 Subject: [PATCH 15/89] made a memcache stub for the internal proxy server --- swift/common/internal_proxy.py | 11 ++++++++++- swift/proxy/server.py | 11 +---------- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/swift/common/internal_proxy.py b/swift/common/internal_proxy.py index c42b17ae16..3da3319497 100644 --- a/swift/common/internal_proxy.py +++ b/swift/common/internal_proxy.py @@ -20,6 +20,13 @@ from json import loads as json_loads from swift.common.compressed_file_reader import CompressedFileReader from swift.proxy.server import BaseApplication +class MemcacheStub(object): + def get(self, *a, **kw): return None + def set(self, *a, **kw): return None + def incr(self, *a, **kw): return 0 + def delete(self, *a, **kw): return None + def set_multi(self, *a, **kw): return None + def get_multi(self, *a, **kw): return [] class InternalProxy(object): """ @@ -32,7 +39,9 @@ class InternalProxy(object): :param retries: number of times to retry each request """ def __init__(self, proxy_server_conf=None, logger=None, retries=0): - self.upload_app = BaseApplication(proxy_server_conf, logger=logger) + self.upload_app = BaseApplication(proxy_server_conf, + memcache=MemcacheStub(), + logger=logger) self.retries = retries def upload_file(self, source_file, account, container, object_name, diff --git a/swift/proxy/server.py b/swift/proxy/server.py index 375d481c5b..b7bea00e43 100644 --- a/swift/proxy/server.py +++ b/swift/proxy/server.py @@ -1200,17 +1200,8 @@ class BaseApplication(object): :param start_response: WSGI callable """ try: - if self.memcache is None: + if self.mamcache is None: self.memcache = cache_from_env(env) - if self.memcache is None: - class MemcacheStub(object): - def get(self, *a, **kw): return None - def set(self, *a, **kw): return None - def incr(self, *a, **kw): return 0 - def delete(self, *a, **kw): return None - def set_multi(self, *a, **kw): return None - def get_multi(self, *a, **kw): return [] - self.memcache = MemcacheStub() req = self.update_request(Request(env)) if 'eventlet.posthooks' in env: env['eventlet.posthooks'].append( From e63cc70b3e3234e293f0b6993534110e78884527 Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Wed, 15 Sep 2010 01:01:06 -0500 Subject: [PATCH 16/89] fixed internal proxy put_container reference --- etc/log-processing.conf-sample | 4 ++-- swift/common/internal_proxy.py | 2 +- swift/stats/account_stats.py | 2 ++ 3 files changed, 5 insertions(+), 3 deletions(-) diff --git a/etc/log-processing.conf-sample b/etc/log-processing.conf-sample index 433613efee..dea016f0b9 100644 --- a/etc/log-processing.conf-sample +++ b/etc/log-processing.conf-sample @@ -14,7 +14,7 @@ swift_account = AUTH_7abbc116-8a07-4b63-819d-02715d3e0f31 # log_dir = /var/log/swift/ swift_account = AUTH_7abbc116-8a07-4b63-819d-02715d3e0f31 container_name = log_data -source_filename_format = %Y%m%d%H* +source_filename_format = access-%Y%m%d%H class_path = swift.stats.access_processor.AccessLogProcessor # service ips is for client ip addresses that should be counted as servicenet # service_ips = @@ -28,7 +28,7 @@ class_path = swift.stats.access_processor.AccessLogProcessor # log_dir = /var/log/swift/ swift_account = AUTH_7abbc116-8a07-4b63-819d-02715d3e0f31 container_name = account_stats -source_filename_format = %Y%m%d%H* +source_filename_format = stats-%Y%m%d%H class_path = swift.stats.stats_processor.StatsLogProcessor # account_server_conf = /etc/swift/account-server.conf # user = swift \ No newline at end of file diff --git a/swift/common/internal_proxy.py b/swift/common/internal_proxy.py index 3da3319497..ddfbbd5ca8 100644 --- a/swift/common/internal_proxy.py +++ b/swift/common/internal_proxy.py @@ -60,7 +60,7 @@ class InternalProxy(object): log_create_pattern = '/v1/%s/%s/%s' % (account, container, object_name) # create the container - if not self.put_container(account, container): + if not self.create_container(account, container): return False # upload the file to the account diff --git a/swift/stats/account_stats.py b/swift/stats/account_stats.py index b436a5b12c..3eac63c0f6 100644 --- a/swift/stats/account_stats.py +++ b/swift/stats/account_stats.py @@ -48,7 +48,9 @@ class AccountStat(Daemon): ((time.time()-start)/60)) def find_and_process(self): + #TODO: handle a counter in the filename to prevent overwrites? src_filename = time.strftime(self.filename_format) + #TODO: don't use /tmp? tmp_filename = os.path.join('/tmp', src_filename) with open(tmp_filename, 'wb') as statfile: #statfile.write('Account Name, Container Count, Object Count, Bytes Used\n') From 031a29c651433c8ca29884e4ad85c172ac24400f Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Wed, 15 Sep 2010 01:10:51 -0500 Subject: [PATCH 17/89] fixed some log uploading glob patterns --- swift/stats/log_uploader.py | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/swift/stats/log_uploader.py b/swift/stats/log_uploader.py index d2f4f6d7be..331b01651d 100644 --- a/swift/stats/log_uploader.py +++ b/swift/stats/log_uploader.py @@ -69,10 +69,10 @@ class LogUploader(Daemon): ((time.time()-start)/60)) def upload_all_logs(self): - i = [(c,self.filename_format.index(c)) for c in '%Y %m %d %H'.split()] + i = [(self.filename_format.index(c), c) for c in '%Y %m %d %H'.split()] i.sort() year_offset = month_offset = day_offset = hour_offset = None - for c, start in i: + for start, c in i: if c == '%Y': year_offset = start, start+4 elif c == '%m': @@ -85,10 +85,10 @@ class LogUploader(Daemon): # don't have all the parts, can't upload anything return glob_pattern = self.filename_format - glob_pattern = glob_pattern.replace('%Y', '????') - glob_pattern = glob_pattern.replace('%m', '??') - glob_pattern = glob_pattern.replace('%d', '??') - glob_pattern = glob_pattern.replace('%H', '??') + glob_pattern = glob_pattern.replace('%Y', '????', 1) + glob_pattern = glob_pattern.replace('%m', '??', 1) + glob_pattern = glob_pattern.replace('%d', '??', 1) + glob_pattern = glob_pattern.replace('%H', '??', 1) filelist = glob.iglob(os.path.join(self.log_dir, glob_pattern)) current_hour = int(time.strftime('%H')) today = int(time.strftime('%Y%m%d')) From 603d6a31758e92755ca2a1a80c47312dec3e9a55 Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Wed, 15 Sep 2010 01:21:24 -0500 Subject: [PATCH 18/89] fixed bug in calculating offsets for filename patterns --- swift/stats/log_uploader.py | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/swift/stats/log_uploader.py b/swift/stats/log_uploader.py index 331b01651d..f330f798e3 100644 --- a/swift/stats/log_uploader.py +++ b/swift/stats/log_uploader.py @@ -72,15 +72,17 @@ class LogUploader(Daemon): i = [(self.filename_format.index(c), c) for c in '%Y %m %d %H'.split()] i.sort() year_offset = month_offset = day_offset = hour_offset = None + base_offset = len(self.log_dir) for start, c in i: + offset = base_offset + start if c == '%Y': - year_offset = start, start+4 + year_offset = offset, offset+4 elif c == '%m': - month_offset = start, start+2 + month_offset = offset, offset+2 elif c == '%d': - day_offset = start, start+2 + day_offset = offset, offset+2 elif c == '%H': - hour_offset = start, start+2 + hour_offset = offset, offset+2 if not (year_offset and month_offset and day_offset and hour_offset): # don't have all the parts, can't upload anything return From 20596ffcce2f09920f963b32121f4763ad66bb3a Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Wed, 15 Sep 2010 01:36:02 -0500 Subject: [PATCH 19/89] fixed typos in log processor --- swift/stats/log_processor.py | 21 ++++++++++++--------- 1 file changed, 12 insertions(+), 9 deletions(-) diff --git a/swift/stats/log_processor.py b/swift/stats/log_processor.py index d02cabbbf6..e8030ed911 100644 --- a/swift/stats/log_processor.py +++ b/swift/stats/log_processor.py @@ -24,6 +24,7 @@ from paste.deploy import appconfig from swift.common.internal_proxy import InternalProxy from swift.common.exceptions import ChunkReadTimeout from swift.common.utils import get_logger, readconf +from swift.common.daemon import Daemon class LogProcessor(object): @@ -194,30 +195,32 @@ class LogProcessor(object): class LogProcessorDaemon(Daemon): def __init__(self, conf): - super(LogProcessorDaemon, stats).__init__(conf) + super(LogProcessorDaemon, self).__init__(conf) self.log_processor = LogProcessor(conf, self.logger) - c = readconf(conf) + c = conf.get('log-processor') self.lookback_hours = int(c.get('lookback_hours', '120')) - self.lookback_window = int(c.get('lookback_window', '%s'%lookback_hours)) + self.lookback_window = int(c.get('lookback_window', + str(self.lookback_hours))) self.log_processor_account = c['swift_account'] - self.log_processor_container = c.get('container_name', 'log_processing_data') + self.log_processor_container = c.get('container_name', + 'log_processing_data') def run_once(self): self.logger.info("Beginning log processing") start = time.time() - if lookback_hours == 0: + if self.lookback_hours == 0: lookback_start = None lookback_end = None else: lookback_start = datetime.datetime.now() - \ - datetime.timedelta(hours=lookback_hours) + datetime.timedelta(hours=self.lookback_hours) lookback_start = lookback_start.strftime('%Y%m%d') - if lookback_window == 0: + if self.lookback_window == 0: lookback_end = None else: lookback_end = datetime.datetime.now() - \ - datetime.timedelta(hours=lookback_hours) + \ - datetime.timedelta(hours=lookback_window) + datetime.timedelta(hours=self.lookback_hours) + \ + datetime.timedelta(hours=self.lookback_window) lookback_end = lookback_end.strftime('%Y%m%d') try: From 8e19ef973a5dab82272db964cc6fbce516df80c7 Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Wed, 15 Sep 2010 01:42:23 -0500 Subject: [PATCH 20/89] fixed get_data_list in log_processor --- swift/stats/log_processor.py | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/swift/stats/log_processor.py b/swift/stats/log_processor.py index e8030ed911..2b88214926 100644 --- a/swift/stats/log_processor.py +++ b/swift/stats/log_processor.py @@ -68,15 +68,15 @@ class LogProcessor(object): def get_data_list(self, start_date=None, end_date=None, listing_filter=None): total_list = [] - for p in self.plugins: - account = p['swift_account'] - container = p['container_name'] + for name, data in self.plugins.items(): + account = data['swift_account'] + container = data['container_name'] l = self.get_container_listing(account, container, start_date, end_date, listing_filter) for i in l: # The items in this list end up being passed as positional # parameters to process_one_file. - total_list.append((p, account, container, i)) + total_list.append((name, account, container, i)) return total_list def get_container_listing(self, swift_account, container_name, start_date=None, @@ -195,9 +195,9 @@ class LogProcessor(object): class LogProcessorDaemon(Daemon): def __init__(self, conf): - super(LogProcessorDaemon, self).__init__(conf) - self.log_processor = LogProcessor(conf, self.logger) c = conf.get('log-processor') + super(LogProcessorDaemon, self).__init__(c) + self.log_processor = LogProcessor(conf, self.logger) self.lookback_hours = int(c.get('lookback_hours', '120')) self.lookback_window = int(c.get('lookback_window', str(self.lookback_hours))) From d037c5f92e7e7321f5b676456928180bb3104fa3 Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Wed, 15 Sep 2010 01:55:51 -0500 Subject: [PATCH 21/89] added some debug output --- etc/log-processing.conf-sample | 1 + swift/stats/log_processor.py | 8 ++++++-- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/etc/log-processing.conf-sample b/etc/log-processing.conf-sample index dea016f0b9..e18f0ff2f9 100644 --- a/etc/log-processing.conf-sample +++ b/etc/log-processing.conf-sample @@ -9,6 +9,7 @@ swift_account = AUTH_7abbc116-8a07-4b63-819d-02715d3e0f31 # lookback_hours = 120 # lookback_window = 120 # user = swift +log_name = log-processor [log-processor-access] # log_dir = /var/log/swift/ diff --git a/swift/stats/log_processor.py b/swift/stats/log_processor.py index 2b88214926..091d9286f4 100644 --- a/swift/stats/log_processor.py +++ b/swift/stats/log_processor.py @@ -197,6 +197,7 @@ class LogProcessorDaemon(Daemon): def __init__(self, conf): c = conf.get('log-processor') super(LogProcessorDaemon, self).__init__(c) + self.logger = get_logger(conf) self.log_processor = LogProcessor(conf, self.logger) self.lookback_hours = int(c.get('lookback_hours', '120')) self.lookback_window = int(c.get('lookback_window', @@ -222,7 +223,8 @@ class LogProcessorDaemon(Daemon): datetime.timedelta(hours=self.lookback_hours) + \ datetime.timedelta(hours=self.lookback_window) lookback_end = lookback_end.strftime('%Y%m%d') - + self.logger.debug('lookback_start: %s' % lookback_start) + self.logger.debug('lookback_end: %s' % lookback_end) try: processed_files_stream = self.log_processor,get_object_data( self.log_processor_account, @@ -233,7 +235,7 @@ class LogProcessorDaemon(Daemon): already_processed_files = cPickle.loads(buf) except: already_processed_files = set() - + self.logger.debug('found %d processed files' % len(already_processed_files)) logs_to_process = self.log_processor.get_data_list(lookback_start, lookback_end, already_processed_files) @@ -300,6 +302,8 @@ class LogProcessorDaemon(Daemon): # cleanup s = cPickle.dumps(processed_files, cPickle.HIGHEST_PROTOCOL) f = cStringIO.StringIO(s) + self.internal_proxy.create_container(self.log_processor_account, + self.log_processor_container) self.log_processor.internal_proxy.upload_file(s, self.log_processor_account, self.log_processor_container, From c28f5550a039b7b94717db5fe4b86713ef253e04 Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Wed, 15 Sep 2010 02:34:10 -0500 Subject: [PATCH 22/89] fixed logging and log uploading --- etc/log-processing.conf-sample | 1 - swift/stats/log_processor.py | 2 +- swift/stats/log_uploader.py | 4 ++++ 3 files changed, 5 insertions(+), 2 deletions(-) diff --git a/etc/log-processing.conf-sample b/etc/log-processing.conf-sample index e18f0ff2f9..dea016f0b9 100644 --- a/etc/log-processing.conf-sample +++ b/etc/log-processing.conf-sample @@ -9,7 +9,6 @@ swift_account = AUTH_7abbc116-8a07-4b63-819d-02715d3e0f31 # lookback_hours = 120 # lookback_window = 120 # user = swift -log_name = log-processor [log-processor-access] # log_dir = /var/log/swift/ diff --git a/swift/stats/log_processor.py b/swift/stats/log_processor.py index 091d9286f4..4aacfbe7d3 100644 --- a/swift/stats/log_processor.py +++ b/swift/stats/log_processor.py @@ -197,7 +197,7 @@ class LogProcessorDaemon(Daemon): def __init__(self, conf): c = conf.get('log-processor') super(LogProcessorDaemon, self).__init__(c) - self.logger = get_logger(conf) + self.logger = get_logger(c) self.log_processor = LogProcessor(conf, self.logger) self.lookback_hours = int(c.get('lookback_hours', '120')) self.lookback_window = int(c.get('lookback_window', diff --git a/swift/stats/log_uploader.py b/swift/stats/log_uploader.py index f330f798e3..f3fff42df5 100644 --- a/swift/stats/log_uploader.py +++ b/swift/stats/log_uploader.py @@ -77,6 +77,10 @@ class LogUploader(Daemon): offset = base_offset + start if c == '%Y': year_offset = offset, offset+4 + # Add in the difference between len(%Y) and the expanded + # version of %Y (????). This makes sure the codes after this + # one will align properly in the final filename. + base_offset += 2 elif c == '%m': month_offset = offset, offset+2 elif c == '%d': From c7773ee6f8cfab403df07f54cfde6a1a2029db84 Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Wed, 15 Sep 2010 02:49:09 -0500 Subject: [PATCH 23/89] fixed copy/paste errors and missing imports --- swift/stats/access_processor.py | 86 ++++++++++++++++----------------- swift/stats/log_processor.py | 13 +++-- 2 files changed, 51 insertions(+), 48 deletions(-) diff --git a/swift/stats/access_processor.py b/swift/stats/access_processor.py index 08d2f1bf08..e4be212efb 100644 --- a/swift/stats/access_processor.py +++ b/swift/stats/access_processor.py @@ -178,47 +178,47 @@ class AccessLogProcessor(object): hourly_aggr_info[aggr_key] = d return hourly_aggr_info - def keylist_mapping(self): - source_keys = 'service public'.split() - level_keys = 'account container object'.split() - verb_keys = 'GET PUT POST DELETE HEAD COPY'.split() - code_keys = '2xx 4xx 5xx'.split() + def keylist_mapping(self): + source_keys = 'service public'.split() + level_keys = 'account container object'.split() + verb_keys = 'GET PUT POST DELETE HEAD COPY'.split() + code_keys = '2xx 4xx 5xx'.split() - keylist_mapping = { - # : or - 'service_bw_in': ('service', 'bytes_in'), - 'service_bw_out': ('service', 'bytes_out'), - 'public_bw_in': ('public', 'bytes_in'), - 'public_bw_out': ('public', 'bytes_out'), - 'account_requests': set(), - 'container_requests': set(), - 'object_requests': set(), - 'service_request': set(), - 'public_request': set(), - 'ops_count': set(), - } - for verb in verb_keys: - keylist_mapping[verb] = set() - for code in code_keys: - keylist_mapping[code] = set() - for source in source_keys: - for level in level_keys: - for verb in verb_keys: - for code in code_keys: - keylist_mapping['account_requests'].add( - (source, 'account', verb, code)) - keylist_mapping['container_requests'].add( - (source, 'container', verb, code)) - keylist_mapping['object_requests'].add( - (source, 'object', verb, code)) - keylist_mapping['service_request'].add( - ('service', level, verb, code)) - keylist_mapping['public_request'].add( - ('public', level, verb, code)) - keylist_mapping[verb].add( - (source, level, verb, code)) - keylist_mapping[code].add( - (source, level, verb, code)) - keylist_mapping['ops_count'].add( - (source,level,verb,code)) - return keylist_mapping + keylist_mapping = { + # : or + 'service_bw_in': ('service', 'bytes_in'), + 'service_bw_out': ('service', 'bytes_out'), + 'public_bw_in': ('public', 'bytes_in'), + 'public_bw_out': ('public', 'bytes_out'), + 'account_requests': set(), + 'container_requests': set(), + 'object_requests': set(), + 'service_request': set(), + 'public_request': set(), + 'ops_count': set(), + } + for verb in verb_keys: + keylist_mapping[verb] = set() + for code in code_keys: + keylist_mapping[code] = set() + for source in source_keys: + for level in level_keys: + for verb in verb_keys: + for code in code_keys: + keylist_mapping['account_requests'].add( + (source, 'account', verb, code)) + keylist_mapping['container_requests'].add( + (source, 'container', verb, code)) + keylist_mapping['object_requests'].add( + (source, 'object', verb, code)) + keylist_mapping['service_request'].add( + ('service', level, verb, code)) + keylist_mapping['public_request'].add( + ('public', level, verb, code)) + keylist_mapping[verb].add( + (source, level, verb, code)) + keylist_mapping[code].add( + (source, level, verb, code)) + keylist_mapping['ops_count'].add( + (source,level,verb,code)) + return keylist_mapping diff --git a/swift/stats/log_processor.py b/swift/stats/log_processor.py index 4aacfbe7d3..8f7a10a40e 100644 --- a/swift/stats/log_processor.py +++ b/swift/stats/log_processor.py @@ -20,6 +20,8 @@ import datetime import cStringIO import collections from paste.deploy import appconfig +import multiprocessing +import Queue from swift.common.internal_proxy import InternalProxy from swift.common.exceptions import ChunkReadTimeout @@ -172,7 +174,7 @@ class LogProcessor(object): def generate_keylist_mapping(self): keylist = {} for plugin in self.plugins: - plugin_keylist = self.plugins['instance'].keylist_mapping() + plugin_keylist = self.plugins[plugin]['instance'].keylist_mapping() for k, v in plugin_keylist.items(): o = keylist.get(k) if o: @@ -197,6 +199,7 @@ class LogProcessorDaemon(Daemon): def __init__(self, conf): c = conf.get('log-processor') super(LogProcessorDaemon, self).__init__(c) + self.total_conf = conf self.logger = get_logger(c) self.log_processor = LogProcessor(conf, self.logger) self.lookback_hours = int(c.get('lookback_hours', '120')) @@ -246,13 +249,13 @@ class LogProcessorDaemon(Daemon): return # map - processor_args = (conf, self.logger) + processor_args = (self.total_conf, self.logger) results = multiprocess_collate(processor_args, logs_to_process) #reduce aggr_data = {} processed_files = already_processed_files - for item, data in results.items(): + for item, data in results: # since item contains the plugin and the log name, new plugins will # "reprocess" the file and the results will be in the final csv. processed_files.append(item) @@ -268,7 +271,7 @@ class LogProcessorDaemon(Daemon): # group # reduce a large number of keys in aggr_data[k] to a small number of # output keys - keylist_mapping = generate_keylist_mapping() + keylist_mapping = self.log_processor.generate_keylist_mapping() final_info = collections.defaultdict(dict) for account, data in rows.items(): for key, mapping in keylist_mapping.items(): @@ -336,7 +339,7 @@ def multiprocess_collate(processor_args, logs_to_process): count += 1 if data: yield item, data - if count >= len(all_files): + if count >= len(logs_to_process): # this implies that one result will come from every request break except Queue.Empty: From 157e01e84b3902ba0e823422b80ada0849ef6af5 Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Wed, 15 Sep 2010 02:53:06 -0500 Subject: [PATCH 24/89] added error handling and missing return statement --- swift/stats/log_processor.py | 2 ++ swift/stats/stats_processor.py | 1 + 2 files changed, 3 insertions(+) diff --git a/swift/stats/log_processor.py b/swift/stats/log_processor.py index 8f7a10a40e..c9a21704a6 100644 --- a/swift/stats/log_processor.py +++ b/swift/stats/log_processor.py @@ -175,6 +175,8 @@ class LogProcessor(object): keylist = {} for plugin in self.plugins: plugin_keylist = self.plugins[plugin]['instance'].keylist_mapping() + if not plugin_keylist: + continue for k, v in plugin_keylist.items(): o = keylist.get(k) if o: diff --git a/swift/stats/stats_processor.py b/swift/stats/stats_processor.py index c7b89c11e6..2c7a5b4259 100644 --- a/swift/stats/stats_processor.py +++ b/swift/stats/stats_processor.py @@ -62,3 +62,4 @@ class StatsLogProcessor(object): 'object_count': 'object_count', 'replica_count': 'replica_count', } + return keylist_mapping From 51218437768a0c04590f9896dfc902d2d67d2aa8 Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Wed, 15 Sep 2010 03:17:56 -0500 Subject: [PATCH 25/89] handled some typos and better handling of missing data in internal proxy --- swift/common/internal_proxy.py | 3 +-- swift/stats/log_processor.py | 17 ++++++++++++----- test/unit/stats/test_log_processor.py | 22 +++++++++++++--------- 3 files changed, 26 insertions(+), 16 deletions(-) diff --git a/swift/common/internal_proxy.py b/swift/common/internal_proxy.py index ddfbbd5ca8..13dd7d621a 100644 --- a/swift/common/internal_proxy.py +++ b/swift/common/internal_proxy.py @@ -109,8 +109,7 @@ class InternalProxy(object): and tries <= self.retries: resp = self.upload_app.handle_request(self.upload_app.update_request(req)) tries += 1 - for x in resp.app_iter: - yield x + return resp.status_int, resp.app_iter def create_container(self, account, container): """ diff --git a/swift/stats/log_processor.py b/swift/stats/log_processor.py index c9a21704a6..375f20416d 100644 --- a/swift/stats/log_processor.py +++ b/swift/stats/log_processor.py @@ -22,12 +22,16 @@ import collections from paste.deploy import appconfig import multiprocessing import Queue +import cPickle from swift.common.internal_proxy import InternalProxy from swift.common.exceptions import ChunkReadTimeout from swift.common.utils import get_logger, readconf from swift.common.daemon import Daemon +class BadFileDownload(Exception): + pass + class LogProcessor(object): def __init__(self, conf, logger): @@ -146,9 +150,11 @@ class LogProcessor(object): def get_object_data(self, swift_account, container_name, object_name, compressed=False): '''reads an object and yields its lines''' - o = self.internal_proxy.get_object(swift_account, + code, o = self.internal_proxy.get_object(swift_account, container_name, object_name) + if code < 200 or code >= 300: + return last_part = '' last_compressed_part = '' # magic in the following zlib.decompressobj argument is courtesy of @@ -231,7 +237,7 @@ class LogProcessorDaemon(Daemon): self.logger.debug('lookback_start: %s' % lookback_start) self.logger.debug('lookback_end: %s' % lookback_end) try: - processed_files_stream = self.log_processor,get_object_data( + processed_files_stream = self.log_processor.get_object_data( self.log_processor_account, self.log_processor_container, 'processed_files.pickle.gz', @@ -275,7 +281,7 @@ class LogProcessorDaemon(Daemon): # output keys keylist_mapping = self.log_processor.generate_keylist_mapping() final_info = collections.defaultdict(dict) - for account, data in rows.items(): + for account, data in aggr_data.items(): for key, mapping in keylist_mapping.items(): if isinstance(mapping, (list, set)): value = 0 @@ -307,9 +313,10 @@ class LogProcessorDaemon(Daemon): # cleanup s = cPickle.dumps(processed_files, cPickle.HIGHEST_PROTOCOL) f = cStringIO.StringIO(s) - self.internal_proxy.create_container(self.log_processor_account, + self.log_processor.internal_proxy.create_container( + self.log_processor_account, self.log_processor_container) - self.log_processor.internal_proxy.upload_file(s, + self.log_processor.internal_proxy.upload_file(f, self.log_processor_account, self.log_processor_container, 'processed_files.pickle.gz') diff --git a/test/unit/stats/test_log_processor.py b/test/unit/stats/test_log_processor.py index 24a37a5877..cf7049bb3a 100644 --- a/test/unit/stats/test_log_processor.py +++ b/test/unit/stats/test_log_processor.py @@ -18,18 +18,22 @@ class DumbInternalProxy(object): return [] def get_object(self, account, container, object_name): + code = 200 if object_name.endswith('.gz'): # same data as below, compressed with gzip -9 - yield '\x1f\x8b\x08' - yield '\x08"\xd79L' - yield '\x02\x03te' - yield 'st\x00\xcbO' - yield '\xca\xe2JI,I' - yield '\xe4\x02\x00O\xff' - yield '\xa3Y\t\x00\x00\x00' + def data(): + yield '\x1f\x8b\x08' + yield '\x08"\xd79L' + yield '\x02\x03te' + yield 'st\x00\xcbO' + yield '\xca\xe2JI,I' + yield '\xe4\x02\x00O\xff' + yield '\xa3Y\t\x00\x00\x00' else: - yield 'obj\n' - yield 'data' + def data(): + yield 'obj\n' + yield 'data' + return code, data class TestLogProcessor(unittest.TestCase): From ff37407f8f78f3246a4f2bcaf574075529bc6ce8 Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Wed, 15 Sep 2010 03:32:47 -0500 Subject: [PATCH 26/89] fixed tests, typos, and added error handling --- swift/proxy/server.py | 2 +- swift/stats/log_processor.py | 8 ++++---- test/unit/stats/test_log_processor.py | 9 ++++----- 3 files changed, 9 insertions(+), 10 deletions(-) diff --git a/swift/proxy/server.py b/swift/proxy/server.py index b7bea00e43..892c4f4769 100644 --- a/swift/proxy/server.py +++ b/swift/proxy/server.py @@ -1200,7 +1200,7 @@ class BaseApplication(object): :param start_response: WSGI callable """ try: - if self.mamcache is None: + if self.memcache is None: self.memcache = cache_from_env(env) req = self.update_request(Request(env)) if 'eventlet.posthooks' in env: diff --git a/swift/stats/log_processor.py b/swift/stats/log_processor.py index 375f20416d..be9aa8155c 100644 --- a/swift/stats/log_processor.py +++ b/swift/stats/log_processor.py @@ -243,7 +243,10 @@ class LogProcessorDaemon(Daemon): 'processed_files.pickle.gz', compressed=True) buf = ''.join(x for x in processed_files_stream) - already_processed_files = cPickle.loads(buf) + if buf: + already_processed_files = cPickle.loads(buf) + else: + already_processed_files = set() except: already_processed_files = set() self.logger.debug('found %d processed files' % len(already_processed_files)) @@ -313,9 +316,6 @@ class LogProcessorDaemon(Daemon): # cleanup s = cPickle.dumps(processed_files, cPickle.HIGHEST_PROTOCOL) f = cStringIO.StringIO(s) - self.log_processor.internal_proxy.create_container( - self.log_processor_account, - self.log_processor_container) self.log_processor.internal_proxy.upload_file(f, self.log_processor_account, self.log_processor_container, diff --git a/test/unit/stats/test_log_processor.py b/test/unit/stats/test_log_processor.py index cf7049bb3a..ea80399659 100644 --- a/test/unit/stats/test_log_processor.py +++ b/test/unit/stats/test_log_processor.py @@ -33,7 +33,7 @@ class DumbInternalProxy(object): def data(): yield 'obj\n' yield 'data' - return code, data + return code, data() class TestLogProcessor(unittest.TestCase): @@ -42,7 +42,7 @@ class TestLogProcessor(unittest.TestCase): '/v1/acct/foo/bar?format=json&foo HTTP/1.0 200 - '\ 'curl tk4e350daf-9338-4cc6-aabb-090e49babfbd '\ '6 95 - txfa431231-7f07-42fd-8fc7-7da9d8cc1f90 - 0.0262' - stats_test_line = 'account,1,2,3,1283378584.881391' + stats_test_line = 'account,1,2,3' proxy_config = {'log-processor': { } @@ -152,11 +152,10 @@ class TestLogProcessor(unittest.TestCase): def get_object_data(*a,**kw): return [self.stats_test_line] p.get_object_data = get_object_data - result = p.process_one_file('stats', 'a', 'c', 'o') + result = p.process_one_file('stats', 'a', 'c', 'y/m/d/h/o') expected = {'account': {'count': 1, 'object_count': 2, 'container_count': 1, - 'bytes_used': 3, - 'created_at': '1283378584.881391'}} + 'bytes_used': 3}} self.assertEquals(result, expected) \ No newline at end of file From e08fb7c0cbfb8635812d31b0af8e9f242413948b Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Wed, 15 Sep 2010 13:40:47 -0500 Subject: [PATCH 27/89] fixed bug in account stats log processing --- swift/stats/log_processor.py | 4 ++-- swift/stats/stats_processor.py | 4 +--- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/swift/stats/log_processor.py b/swift/stats/log_processor.py index be9aa8155c..3f9d042f30 100644 --- a/swift/stats/log_processor.py +++ b/swift/stats/log_processor.py @@ -242,7 +242,7 @@ class LogProcessorDaemon(Daemon): self.log_processor_container, 'processed_files.pickle.gz', compressed=True) - buf = ''.join(x for x in processed_files_stream) + buf = '\n'.join(x for x in processed_files_stream) if buf: already_processed_files = cPickle.loads(buf) else: @@ -269,7 +269,7 @@ class LogProcessorDaemon(Daemon): for item, data in results: # since item contains the plugin and the log name, new plugins will # "reprocess" the file and the results will be in the final csv. - processed_files.append(item) + processed_files.add(item) for k, d in data.items(): existing_data = aggr_data.get(k, {}) for i, j in d.items(): diff --git a/swift/stats/stats_processor.py b/swift/stats/stats_processor.py index 2c7a5b4259..dd698c32ee 100644 --- a/swift/stats/stats_processor.py +++ b/swift/stats/stats_processor.py @@ -29,13 +29,11 @@ class StatsLogProcessor(object): (account, container_count, object_count, - bytes_used, - created_at) = line.split(',') + bytes_used) = line.split(',') account = account.strip('"') container_count = int(container_count.strip('"')) object_count = int(object_count.strip('"')) bytes_used = int(bytes_used.strip('"')) - created_at = created_at.strip('"') aggr_key = (account, year, month, day, hour) d = account_totals.get(aggr_key, {}) d['replica_count'] = d.setdefault('count', 0) + 1 From 6ed3724880c99bc8714c52464aa629526b31e145 Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Wed, 15 Sep 2010 14:01:17 -0500 Subject: [PATCH 28/89] fixed listing filter in log processing --- swift/stats/log_processor.py | 17 +++++++++++------ swift/stats/stats_processor.py | 2 +- 2 files changed, 12 insertions(+), 7 deletions(-) diff --git a/swift/stats/log_processor.py b/swift/stats/log_processor.py index 3f9d042f30..7221ff15ca 100644 --- a/swift/stats/log_processor.py +++ b/swift/stats/log_processor.py @@ -77,12 +77,16 @@ class LogProcessor(object): for name, data in self.plugins.items(): account = data['swift_account'] container = data['container_name'] - l = self.get_container_listing(account, container, start_date, - end_date, listing_filter) + l = self.get_container_listing(account, + container, + start_date, + end_date) for i in l: # The items in this list end up being passed as positional # parameters to process_one_file. - total_list.append((name, account, container, i)) + x = (name, account, container, i) + if x not in listing_filter: + total_list.append(x) return total_list def get_container_listing(self, swift_account, container_name, start_date=None, @@ -249,10 +253,11 @@ class LogProcessorDaemon(Daemon): already_processed_files = set() except: already_processed_files = set() - self.logger.debug('found %d processed files' % len(already_processed_files)) + self.logger.debug('found %d processed files' % \ + len(already_processed_files)) logs_to_process = self.log_processor.get_data_list(lookback_start, - lookback_end, - already_processed_files) + lookback_end, + already_processed_files) self.logger.info('loaded %d files to process' % len(logs_to_process)) if not logs_to_process: self.logger.info("Log processing done (%0.2f minutes)" % diff --git a/swift/stats/stats_processor.py b/swift/stats/stats_processor.py index dd698c32ee..b392af6cf2 100644 --- a/swift/stats/stats_processor.py +++ b/swift/stats/stats_processor.py @@ -43,7 +43,7 @@ class StatsLogProcessor(object): object_count d['bytes_used'] = d.setdefault('bytes_used', 0) + \ bytes_used - account_totals[account] = d + account_totals[aggr_key] = d except (IndexError, ValueError): # bad line data pass From 4412ec7ac813489e6e15c4f49d8f4b2e68814ecf Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Wed, 15 Sep 2010 14:11:10 -0500 Subject: [PATCH 29/89] fixed stdout capturing for generating csv files --- bin/swift-log-stats-collector | 2 +- swift/common/daemon.py | 7 +++++-- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/bin/swift-log-stats-collector b/bin/swift-log-stats-collector index d21135b35c..00ed827348 100755 --- a/bin/swift-log-stats-collector +++ b/bin/swift-log-stats-collector @@ -24,4 +24,4 @@ if __name__ == '__main__': print "Usage: swift-log-stats-collector CONFIG_FILE" sys.exit() conf = utils.readconf(sys.argv[1], log_name='log-stats-collector') - stats = LogProcessorDaemon(conf).run(once=True) + stats = LogProcessorDaemon(conf).run(once=True, capture_stdout=False) diff --git a/swift/common/daemon.py b/swift/common/daemon.py index 80d3eff4c7..997bcb98f0 100644 --- a/swift/common/daemon.py +++ b/swift/common/daemon.py @@ -33,12 +33,15 @@ class Daemon(object): """Override this to run forever""" raise NotImplementedError('run_forever not implemented') - def run(self, once=False): + def run(self, once=False, capture_stdout=True, capture_stderr=True): """Run the daemon""" # log uncaught exceptions sys.excepthook = lambda *exc_info: \ self.logger.critical('UNCAUGHT EXCEPTION', exc_info=exc_info) - sys.stdout = sys.stderr = utils.LoggerFileObject(self.logger) + if capture_stdout: + sys.stdout = utils.LoggerFileObject(self.logger) + if capture_stderr: + sys.stderr = utils.LoggerFileObject(self.logger) utils.drop_privileges(self.conf.get('user', 'swift')) From 457e8560ae891e5793898fb69c78e326cdc74bea Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Wed, 15 Sep 2010 14:23:45 -0500 Subject: [PATCH 30/89] fixed replica count reporting error --- swift/stats/stats_processor.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/swift/stats/stats_processor.py b/swift/stats/stats_processor.py index b392af6cf2..eb31d8c668 100644 --- a/swift/stats/stats_processor.py +++ b/swift/stats/stats_processor.py @@ -36,7 +36,7 @@ class StatsLogProcessor(object): bytes_used = int(bytes_used.strip('"')) aggr_key = (account, year, month, day, hour) d = account_totals.get(aggr_key, {}) - d['replica_count'] = d.setdefault('count', 0) + 1 + d['replica_count'] = d.setdefault('replica_count', 0) + 1 d['container_count'] = d.setdefault('container_count', 0) + \ container_count d['object_count'] = d.setdefault('object_count', 0) + \ From ddfc1f2951ab2303b9ed13f64267d42858b56abc Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Wed, 15 Sep 2010 15:07:53 -0500 Subject: [PATCH 31/89] fixed lookback in log processor --- swift/stats/log_processor.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/swift/stats/log_processor.py b/swift/stats/log_processor.py index 7221ff15ca..04c4a3b649 100644 --- a/swift/stats/log_processor.py +++ b/swift/stats/log_processor.py @@ -230,14 +230,14 @@ class LogProcessorDaemon(Daemon): else: lookback_start = datetime.datetime.now() - \ datetime.timedelta(hours=self.lookback_hours) - lookback_start = lookback_start.strftime('%Y%m%d') + lookback_start = lookback_start.strftime('%Y%m%d%H') if self.lookback_window == 0: lookback_end = None else: lookback_end = datetime.datetime.now() - \ datetime.timedelta(hours=self.lookback_hours) + \ datetime.timedelta(hours=self.lookback_window) - lookback_end = lookback_end.strftime('%Y%m%d') + lookback_end = lookback_end.strftime('%Y%m%d%H') self.logger.debug('lookback_start: %s' % lookback_start) self.logger.debug('lookback_end: %s' % lookback_end) try: From 872a2f67a3f8c47c39bf2a8de2c92cd19a0e4caa Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Mon, 20 Sep 2010 13:26:00 -0500 Subject: [PATCH 32/89] fixed tests to account for changed key name --- test/unit/stats/test_log_processor.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/unit/stats/test_log_processor.py b/test/unit/stats/test_log_processor.py index ea80399659..eadc3c4423 100644 --- a/test/unit/stats/test_log_processor.py +++ b/test/unit/stats/test_log_processor.py @@ -154,7 +154,7 @@ class TestLogProcessor(unittest.TestCase): p.get_object_data = get_object_data result = p.process_one_file('stats', 'a', 'c', 'y/m/d/h/o') expected = {'account': - {'count': 1, + {'replica_count': 1, 'object_count': 2, 'container_count': 1, 'bytes_used': 3}} From 55c997aa9417073d13a564cf0ebbc5c9dcc88d22 Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Mon, 20 Sep 2010 13:28:52 -0500 Subject: [PATCH 33/89] fixed test bug --- test/unit/stats/test_log_processor.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/unit/stats/test_log_processor.py b/test/unit/stats/test_log_processor.py index eadc3c4423..1f4f14f513 100644 --- a/test/unit/stats/test_log_processor.py +++ b/test/unit/stats/test_log_processor.py @@ -153,7 +153,7 @@ class TestLogProcessor(unittest.TestCase): return [self.stats_test_line] p.get_object_data = get_object_data result = p.process_one_file('stats', 'a', 'c', 'y/m/d/h/o') - expected = {'account': + expected = {('account', 'y', 'm', 'd', 'h'): {'replica_count': 1, 'object_count': 2, 'container_count': 1, From 0bb5857da3af3afc2872255441efa1c0fbc23f3f Mon Sep 17 00:00:00 2001 From: Clay Gerrard Date: Mon, 20 Sep 2010 17:52:58 -0500 Subject: [PATCH 34/89] pep8 issues, refactored log_processor a tiny bit for testing (lazy load internal proxy), added a few comments --- swift/stats/access_processor.py | 16 ++++--- swift/stats/account_stats.py | 24 ++++++---- swift/stats/log_processor.py | 66 +++++++++++++++++---------- swift/stats/log_uploader.py | 32 +++++++------ swift/stats/stats_processor.py | 2 + test/unit/stats/test_log_processor.py | 8 ++-- 6 files changed, 89 insertions(+), 59 deletions(-) diff --git a/swift/stats/access_processor.py b/swift/stats/access_processor.py index e4be212efb..6fcdafc59c 100644 --- a/swift/stats/access_processor.py +++ b/swift/stats/access_processor.py @@ -21,7 +21,9 @@ from swift.common.utils import split_path month_map = '_ Jan Feb Mar Apr May Jun Jul Aug Sep Oct Nov Dec'.split() + class AccessLogProcessor(object): + """Transform proxy server access logs""" def __init__(self, conf): self.server_name = conf.get('server_name', 'proxy') @@ -112,8 +114,8 @@ class AccessLogProcessor(object): d['account'] = account d['container_name'] = container_name d['object_name'] = object_name - d['bytes_out'] = int(d['bytes_out'].replace('-','0')) - d['bytes_in'] = int(d['bytes_in'].replace('-','0')) + d['bytes_out'] = int(d['bytes_out'].replace('-', '0')) + d['bytes_in'] = int(d['bytes_in'].replace('-', '0')) d['code'] = int(d['code']) return d @@ -151,12 +153,12 @@ class AccessLogProcessor(object): source = 'service' else: source = 'public' - + if line_data['client_ip'] in self.service_ips: source = 'service' - d[(source, 'bytes_out')] = d.setdefault((source, 'bytes_out'), 0) + \ - bytes_out + d[(source, 'bytes_out')] = d.setdefault(( + source, 'bytes_out'), 0) + bytes_out d[(source, 'bytes_in')] = d.setdefault((source, 'bytes_in'), 0) + \ bytes_in @@ -171,7 +173,7 @@ class AccessLogProcessor(object): path = line_data.get('path', 0) d['path_query'] = d.setdefault('path_query', 0) + path - code = '%dxx' % (code/100) + code = '%dxx' % (code / 100) key = (source, op_level, method, code) d[key] = d.setdefault(key, 0) + 1 @@ -220,5 +222,5 @@ class AccessLogProcessor(object): keylist_mapping[code].add( (source, level, verb, code)) keylist_mapping['ops_count'].add( - (source,level,verb,code)) + (source, level, verb, code)) return keylist_mapping diff --git a/swift/stats/account_stats.py b/swift/stats/account_stats.py index 3eac63c0f6..3ed8b6d88b 100644 --- a/swift/stats/account_stats.py +++ b/swift/stats/account_stats.py @@ -21,9 +21,15 @@ from swift.account.server import DATADIR as account_server_data_dir from swift.common.db import AccountBroker from swift.common.internal_proxy import InternalProxy from swift.common.utils import renamer, get_logger, readconf +from swift.common.constraints import check_mount from swift.common.daemon import Daemon + class AccountStat(Daemon): + """Extract storage stats from account databases on the account + storage nodes + """ + def __init__(self, stats_conf): super(AccountStat, self).__init__(stats_conf) target_dir = stats_conf.get('log_dir', '/var/log/swift') @@ -45,7 +51,7 @@ class AccountStat(Daemon): start = time.time() self.find_and_process() self.logger.info("Gathering account stats complete (%0.2f minutes)" % - ((time.time()-start)/60)) + ((time.time() - start) / 60)) def find_and_process(self): #TODO: handle a counter in the filename to prevent overwrites? @@ -53,10 +59,10 @@ class AccountStat(Daemon): #TODO: don't use /tmp? tmp_filename = os.path.join('/tmp', src_filename) with open(tmp_filename, 'wb') as statfile: - #statfile.write('Account Name, Container Count, Object Count, Bytes Used\n') + #statfile.write( + # 'Account Name, Container Count, Object Count, Bytes Used\n') for device in os.listdir(self.devices): - if self.mount_check and \ - not os.path.ismount(os.path.join(self.devices, device)): + if self.mount_check and not check_mount(self.devices, device): self.logger.error("Device %s is not mounted, skipping." % device) continue @@ -70,7 +76,8 @@ class AccountStat(Daemon): for root, dirs, files in os.walk(accounts, topdown=False): for filename in files: if filename.endswith('.db'): - broker = AccountBroker(os.path.join(root, filename)) + db_path = os.path.join(root, filename) + broker = AccountBroker(db_path) if not broker.is_deleted(): (account_name, _, _, _, @@ -78,9 +85,8 @@ class AccountStat(Daemon): object_count, bytes_used, _, _) = broker.get_info() - line_data = '"%s",%d,%d,%d\n' % (account_name, - container_count, - object_count, - bytes_used) + line_data = '"%s",%d,%d,%d\n' % ( + account_name, container_count, + object_count, bytes_used) statfile.write(line_data) renamer(tmp_filename, os.path.join(self.target_dir, src_filename)) diff --git a/swift/stats/log_processor.py b/swift/stats/log_processor.py index 04c4a3b649..473beda112 100644 --- a/swift/stats/log_processor.py +++ b/swift/stats/log_processor.py @@ -29,26 +29,22 @@ from swift.common.exceptions import ChunkReadTimeout from swift.common.utils import get_logger, readconf from swift.common.daemon import Daemon + class BadFileDownload(Exception): pass + class LogProcessor(object): + """Load plugins, process logs""" def __init__(self, conf, logger): stats_conf = conf.get('log-processor', {}) - - proxy_server_conf_loc = stats_conf.get('proxy_server_conf', - '/etc/swift/proxy-server.conf') - self.proxy_server_conf = appconfig('config:%s' % proxy_server_conf_loc, - name='proxy-server') + if isinstance(logger, tuple): self.logger = get_logger(*logger) else: self.logger = logger - self.internal_proxy = InternalProxy(self.proxy_server_conf, - self.logger, - retries=3) - + # load the processing plugins self.plugins = {} plugin_prefix = 'log-processor-' @@ -56,11 +52,25 @@ class LogProcessor(object): plugin_name = section[len(plugin_prefix):] plugin_conf = conf.get(section, {}) self.plugins[plugin_name] = plugin_conf - import_target, class_name = plugin_conf['class_path'].rsplit('.', 1) + class_path = self.plugins[plugin_name]['class_path'] + import_target, class_name = class_path.rsplit('.', 1) module = __import__(import_target, fromlist=[import_target]) klass = getattr(module, class_name) self.plugins[plugin_name]['instance'] = klass(plugin_conf) + @property + def internal_proxy(self): + '''Lazy load internal proxy''' + if self._internal_proxy is None: + proxy_server_conf_loc = stats_conf.get('proxy_server_conf', + '/etc/swift/proxy-server.conf') + self.proxy_server_conf = appconfig('config:%s' % proxy_server_conf_loc, + name='proxy-server') + self._internal_proxy = InternalProxy(self.proxy_server_conf, + self.logger, + retries=3) + return self._internal_proxy + def process_one_file(self, plugin_name, account, container, object_name): # get an iter of the object data compressed = object_name.endswith('.gz') @@ -72,7 +82,8 @@ class LogProcessor(object): container, object_name) - def get_data_list(self, start_date=None, end_date=None, listing_filter=None): + def get_data_list(self, start_date=None, end_date=None, + listing_filter=None): total_list = [] for name, data in self.plugins.items(): account = data['swift_account'] @@ -89,8 +100,9 @@ class LogProcessor(object): total_list.append(x) return total_list - def get_container_listing(self, swift_account, container_name, start_date=None, - end_date=None, listing_filter=None): + def get_container_listing(self, swift_account, container_name, + start_date=None, end_date=None, + listing_filter=None): ''' Get a container listing, filtered by start_date, end_date, and listing_filter. Dates, if given, should be in YYYYMMDDHH format @@ -162,15 +174,16 @@ class LogProcessor(object): last_part = '' last_compressed_part = '' # magic in the following zlib.decompressobj argument is courtesy of - # http://stackoverflow.com/questions/2423866/python-decompressing-gzip-chunk-by-chunk - d = zlib.decompressobj(16+zlib.MAX_WBITS) + # Python decompressing gzip chunk-by-chunk + # http://stackoverflow.com/questions/2423866 + d = zlib.decompressobj(16 + zlib.MAX_WBITS) try: for chunk in o: if compressed: try: chunk = d.decompress(chunk) except zlib.error: - raise BadFileDownload() # bad compressed data + raise BadFileDownload() # bad compressed data parts = chunk.split('\n') parts[0] = last_part + parts[0] for part in parts[:-1]: @@ -208,6 +221,8 @@ class LogProcessor(object): class LogProcessorDaemon(Daemon): + """Gather raw log data and farm proccessing, results output via print""" + def __init__(self, conf): c = conf.get('log-processor') super(LogProcessorDaemon, self).__init__(c) @@ -228,15 +243,16 @@ class LogProcessorDaemon(Daemon): lookback_start = None lookback_end = None else: - lookback_start = datetime.datetime.now() - \ - datetime.timedelta(hours=self.lookback_hours) + delta_hours = datetime.timedelta(hours=self.lookback_hours) + lookback_start = datetime.datetime.now() - delta_hours lookback_start = lookback_start.strftime('%Y%m%d%H') if self.lookback_window == 0: lookback_end = None else: + delta_window = datetime.timedelta(hours=self.lookback_window) lookback_end = datetime.datetime.now() - \ - datetime.timedelta(hours=self.lookback_hours) + \ - datetime.timedelta(hours=self.lookback_window) + delta_hours + \ + delta_window lookback_end = lookback_end.strftime('%Y%m%d%H') self.logger.debug('lookback_start: %s' % lookback_start) self.logger.debug('lookback_end: %s' % lookback_end) @@ -261,7 +277,7 @@ class LogProcessorDaemon(Daemon): self.logger.info('loaded %d files to process' % len(logs_to_process)) if not logs_to_process: self.logger.info("Log processing done (%0.2f minutes)" % - ((time.time()-start)/60)) + ((time.time() - start) / 60)) return # map @@ -315,7 +331,7 @@ class LogProcessorDaemon(Daemon): row = [bill_ts, data_ts] row.append('%s' % account) for k in sorted_keylist_mapping: - row.append('%s'%d[k]) + row.append('%s' % d[k]) print ','.join(row) # cleanup @@ -327,7 +343,8 @@ class LogProcessorDaemon(Daemon): 'processed_files.pickle.gz') self.logger.info("Log processing done (%0.2f minutes)" % - ((time.time()-start)/60)) + ((time.time() - start) / 60)) + def multiprocess_collate(processor_args, logs_to_process): '''yield hourly data from logs_to_process''' @@ -361,6 +378,7 @@ def multiprocess_collate(processor_args, logs_to_process): for r in results: r.join() + def collate_worker(processor_args, in_queue, out_queue): '''worker process for multiprocess_collate''' p = LogProcessor(*processor_args) @@ -373,4 +391,4 @@ def collate_worker(processor_args, in_queue, out_queue): time.sleep(.1) else: ret = p.process_one_file(*item) - out_queue.put((item, ret)) \ No newline at end of file + out_queue.put((item, ret)) diff --git a/swift/stats/log_uploader.py b/swift/stats/log_uploader.py index f3fff42df5..d695e27338 100644 --- a/swift/stats/log_uploader.py +++ b/swift/stats/log_uploader.py @@ -25,18 +25,19 @@ from swift.common.internal_proxy import InternalProxy from swift.common.daemon import Daemon from swift.common import utils + class LogUploader(Daemon): ''' Given a local directory, a swift account, and a container name, LogParser - will upload all files in the local directory to the given account/container. - All but the newest files will be uploaded, and the files' md5 sum will be - computed. The hash is used to prevent duplicate data from being uploaded - multiple times in different files (ex: log lines). Since the hash is - computed, it is also used as the uploaded object's etag to ensure data - integrity. - + will upload all files in the local directory to the given account/ + container. All but the newest files will be uploaded, and the files' md5 + sum will be computed. The hash is used to prevent duplicate data from + being uploaded multiple times in different files (ex: log lines). Since + the hash is computed, it is also used as the uploaded object's etag to + ensure data integrity. + Note that after the file is successfully uploaded, it will be unlinked. - + The given proxy server config is used to instantiate a proxy server for the object uploads. ''' @@ -66,7 +67,7 @@ class LogUploader(Daemon): start = time.time() self.upload_all_logs() self.logger.info("Uploading logs complete (%0.2f minutes)" % - ((time.time()-start)/60)) + ((time.time() - start) / 60)) def upload_all_logs(self): i = [(self.filename_format.index(c), c) for c in '%Y %m %d %H'.split()] @@ -76,17 +77,17 @@ class LogUploader(Daemon): for start, c in i: offset = base_offset + start if c == '%Y': - year_offset = offset, offset+4 + year_offset = offset, offset + 4 # Add in the difference between len(%Y) and the expanded # version of %Y (????). This makes sure the codes after this # one will align properly in the final filename. base_offset += 2 elif c == '%m': - month_offset = offset, offset+2 + month_offset = offset, offset + 2 elif c == '%d': - day_offset = offset, offset+2 + day_offset = offset, offset + 2 elif c == '%H': - hour_offset = offset, offset+2 + hour_offset = offset, offset + 2 if not (year_offset and month_offset and day_offset and hour_offset): # don't have all the parts, can't upload anything return @@ -124,7 +125,8 @@ class LogUploader(Daemon): continue if (time.time() - os.stat(filename).st_mtime) < 7200: # don't process very new logs - self.logger.debug("Skipping log: %s (< 2 hours old)" % filename) + self.logger.debug( + "Skipping log: %s (< 2 hours old)" % filename) continue self.upload_one_log(filename, year, month, day, hour) @@ -147,7 +149,7 @@ class LogUploader(Daemon): # By adding a hash to the filename, we ensure that uploaded files # have unique filenames and protect against uploading one file # more than one time. By using md5, we get an etag for free. - target_filename = '/'.join([year, month, day, hour, filehash+'.gz']) + target_filename = '/'.join([year, month, day, hour, filehash + '.gz']) if self.internal_proxy.upload_file(filename, self.swift_account, self.container_name, diff --git a/swift/stats/stats_processor.py b/swift/stats/stats_processor.py index eb31d8c668..30500dfe95 100644 --- a/swift/stats/stats_processor.py +++ b/swift/stats/stats_processor.py @@ -13,7 +13,9 @@ # See the License for the specific language governing permissions and # limitations under the License. + class StatsLogProcessor(object): + """Transform account storage stat logs""" def __init__(self, conf): pass diff --git a/test/unit/stats/test_log_processor.py b/test/unit/stats/test_log_processor.py index 1f4f14f513..f6576f6a43 100644 --- a/test/unit/stats/test_log_processor.py +++ b/test/unit/stats/test_log_processor.py @@ -112,7 +112,7 @@ class TestLogProcessor(unittest.TestCase): def test_get_container_listing(self): p = log_processor.LogProcessor(self.proxy_config, DumbLogger()) - p.internal_proxy = DumbInternalProxy() + p._internal_proxy = DumbInternalProxy() result = p.get_container_listing('a', 'foo') expected = ['2010/03/14/13/obj1'] self.assertEquals(result, expected) @@ -133,7 +133,7 @@ class TestLogProcessor(unittest.TestCase): def test_get_object_data(self): p = log_processor.LogProcessor(self.proxy_config, DumbLogger()) - p.internal_proxy = DumbInternalProxy() + p._internal_proxy = DumbInternalProxy() result = list(p.get_object_data('a', 'c', 'o', False)) expected = ['obj','data'] self.assertEquals(result, expected) @@ -148,7 +148,7 @@ class TestLogProcessor(unittest.TestCase): 'swift.stats.stats_processor.StatsLogProcessor' }}) p = log_processor.LogProcessor(stats_proxy_config, DumbLogger()) - p.internal_proxy = DumbInternalProxy() + p._internal_proxy = DumbInternalProxy() def get_object_data(*a,**kw): return [self.stats_test_line] p.get_object_data = get_object_data @@ -158,4 +158,4 @@ class TestLogProcessor(unittest.TestCase): 'object_count': 2, 'container_count': 1, 'bytes_used': 3}} - self.assertEquals(result, expected) \ No newline at end of file + self.assertEquals(result, expected) From 08c497dea01598c691258dfa3ca639d551c63643 Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Wed, 22 Sep 2010 09:43:33 -0500 Subject: [PATCH 35/89] added stats overview --- doc/source/overview_stats.rst | 79 +++++++++++++++++++++++++++++++++++ 1 file changed, 79 insertions(+) create mode 100644 doc/source/overview_stats.rst diff --git a/doc/source/overview_stats.rst b/doc/source/overview_stats.rst new file mode 100644 index 0000000000..3406f8522c --- /dev/null +++ b/doc/source/overview_stats.rst @@ -0,0 +1,79 @@ +================== +Swift stats system +================== + +The swift stats system is composed of three parts parts: log creation, log +uploading, and log processing. The system handles two types of logs (access +and storage stats), but it can be extended to handle other types of logs. + +--------- +Log Types +--------- + +*********** +Access logs +*********** + +Access logs are the proxy server logs. + +****************** +Storage stats logs +****************** + +Storage logs (also referred to as stats logs) are generated by a stats system +process. swift-account-stats-logger runs on each account server (via cron) and +walks the filesystem looking for account databases. When an account database +is found, the logger selects the account hash, bytes_used, container_count, +and object_count. These values are then written out as one line in a csv file. +One csv file is produced for every run of swift-account-stats-logger. This +means that, system wide, one csv file is produced for every storage node. +Rackspace runs the account stats logger every hour. Therefore, in a cluster of +ten account servers, ten csv files are produced every hour. Also, every +account will have one entry for every replica in the system. On average, there +will be three copies of each account in the aggreagate of all account stat csv +files created in one system-wide run. + +---------------------- +Log Processing plugins +---------------------- + +The swift stats system is written to allow a plugin to be defined for every +log type. Swift includes plugins for both access logs and storage stats logs. +Each plugin is responsible for defining, in a config section, where the logs +are stored on disk, where the logs will be stored in swift (account and +container), the filename format of the logs on disk, the location of the +plugin class definition, and any plugin-specific config values. + +The plugin class definition defines three methods. The constuctor must accept +one argument (the dict representation of the plugin's config section). The +process method must accept an iterator, and the account, container, and object +name of the log. The keylist_mapping accepts no parameters. + +------------- +Log Uploading +------------- + +swift-log-uploader accepts a config file and a plugin name. It finds the log +files on disk according to the plugin config section and uploads them to the +swift cluster. This means one uploader process will run on each proxy server +node and each account server node. To not upload partially-written log files, +the uploader will not upload files with an mtime of less than two hours ago. +Rackspace runs this process once an hour via cron. + +-------------- +Log Processing +-------------- + +swift-log-stats-collector accepts a config file and generates a csv that is +uploaded to swift. It loads all plugins defined in the config file, generates +a list of all log files in swift that need to be processed, and passes an +iterable of the log file data to the appropriate plugin's process method. The +process method returns a dictionary of data in the log file keyed on (account, +year, month, day, hour). The log-stats-collector process then combines all +dictionaries from all calls to a process method into one dictionary. Key +collisions within each (account, year, month, day, hour) dictionary are +summed. Finally, the summed dictionary is mapped to the final csv values with +each plugin's keylist_mapping method. + +The resulting csv file has one line per (account, year, month, day, hour) for +all log files processed in that run of swift-log-stats-collector. From 6cef24dcd757ad36d4d861353d3fc911a36a8e7a Mon Sep 17 00:00:00 2001 From: Chuck Thier Date: Mon, 27 Sep 2010 23:10:09 +0000 Subject: [PATCH 36/89] Initial checkin of benchmark code --- bin/swift-bench | 104 ++++++++++++++++ setup.py | 3 +- swift/common/bench.py | 273 ++++++++++++++++++++++++++++++++++++++++++ swift/common/utils.py | 6 +- 4 files changed, 383 insertions(+), 3 deletions(-) create mode 100755 bin/swift-bench create mode 100644 swift/common/bench.py diff --git a/bin/swift-bench b/bin/swift-bench new file mode 100755 index 0000000000..10df6a74a3 --- /dev/null +++ b/bin/swift-bench @@ -0,0 +1,104 @@ +#!/usr/bin/python + +import logging +import os +import sys +import signal +import uuid +from optparse import OptionParser + +from swift.common.bench import BenchController +from swift.common.utils import readconf, NamedLogger + +# The defaults should be sufficient to run the slapper on a SAIO +CONF_DEFAULTS = { + 'auth': 'http://saio:11000/v1.0', + 'user': 'test:tester', + 'key': 'testing', + 'object_sources': '', + 'put_concurrency': '10', + 'get_concurrency': '10', + 'del_concurrency': '10', + 'concurrency': '', + 'object_size': '1', + 'num_objects': '1000', + 'num_gets': '10000', + 'delete': 'yes', + 'container_name': uuid.uuid4().hex, + 'use_proxy': 'yes', + 'url': '', + 'devices': 'sdb', + 'logging_level': 'INFO', + 'timeout': '10', + } + +if __name__ == '__main__': + usage = "usage: %prog [OPTIONS] [CONF_FILE]" + usage += """\n\nConf file (with defaults): + + [bench] + auth = http://saio:11000/v1.0 + user = test:tester + key = testing + concurrency = 10 + object_size = 1 + num_objects = 1000 + num_gets = 10000 + delete = yes + """ + parser = OptionParser(usage=usage) + parser.add_option('-A', '--auth', dest='auth', + help='URL for obtaining an auth token') + parser.add_option('-U', '--user', dest='user', + help='User name for obtaining an auth token') + parser.add_option('-K', '--key', dest='key', + help='Key for obtaining an auth token') + parser.add_option('-u', '--url', dest='url', + help='Storage URL') + parser.add_option('-c', '--concurrency', dest='concurrency', + help='Number of concurrent connections to use') + parser.add_option('-s', '--object-size', dest='object_size', + help='Size of objects to PUT (in bytes)') + parser.add_option('-n', '--num-objects', dest='num_objects', + help='Number of objects to PUT') + parser.add_option('-g', '--num-gets', dest='num_gets', + help='Number of GET operations to perform') + parser.add_option('-x', '--no-delete', dest='delete', action='store_false', + help='If set, will not delete the objects created') + + _, args = parser.parse_args() + if args: + conf = args[0] + if not os.path.exists(conf): + sys.exit("No such conf file: %s" % conf) + conf = readconf(conf, 'bench', log_name='swift-bench', + defaults=CONF_DEFAULTS) + else: + conf = CONF_DEFAULTS + parser.set_defaults(**conf) + options, _ = parser.parse_args() + if options.concurrency is not '': + options.put_concurrency = options.concurrency + options.get_concurrency = options.concurrency + options.del_concurrency = options.concurrency + + def sigterm(signum, frame): + sys.exit('Termination signal received.') + signal.signal(signal.SIGTERM, sigterm) + + logger = logging.getLogger() + logger.setLevel({ + 'debug': logging.DEBUG, + 'info': logging.INFO, + 'warning': logging.WARNING, + 'error': logging.ERROR, + 'critical': logging.CRITICAL}.get( + options.logging_level, logging.INFO)) + loghandler = logging.StreamHandler() + logformat = logging.Formatter('%(asctime)s %(levelname)s %(message)s') + loghandler.setFormatter(logformat) + logger.addHandler(loghandler) + logger = NamedLogger(logger, 'swift-bench') + + controller = BenchController(logger, options) + controller.run() diff --git a/setup.py b/setup.py index 56977c97d2..936bf92f8e 100644 --- a/setup.py +++ b/setup.py @@ -74,7 +74,8 @@ setup( 'bin/swift-object-server', 'bin/swift-object-updater', 'bin/swift-proxy-server', 'bin/swift-ring-builder', 'bin/swift-stats-populate', - 'bin/swift-stats-report' + 'bin/swift-stats-report', + 'bin/swift-bench', ], entry_points={ 'paste.app_factory': [ diff --git a/swift/common/bench.py b/swift/common/bench.py new file mode 100644 index 0000000000..49c4873809 --- /dev/null +++ b/swift/common/bench.py @@ -0,0 +1,273 @@ +import hashlib +import uuid +import time +import random +from urlparse import urlparse + +import eventlet.pools +import eventlet.corolocal +from eventlet.green.httplib import HTTPSConnection, \ + HTTPResponse, CannotSendRequest, _UNKNOWN + +from swift.common.bufferedhttp \ + import BufferedHTTPConnection as HTTPConnection + + +class ConnectionPool(eventlet.pools.Pool): + def __init__(self, url, size): + self.url = url + self.url_parsed = urlparse(self.url) + eventlet.pools.Pool.__init__(self, size, size) + + def create(self): + if self.url_parsed[0] == 'https': + hc = HTTPSConnection(self.url_parsed[1]) + elif self.url_parsed[0] == 'http': + hc = HTTPConnection(self.url_parsed[1]) + else: + raise Exception("Can't handle %s" % self.url_parsed[0]) + return hc + + +class Bench(object): + def __init__(self, logger, conf, names): + self.logger = logger + self.user = conf.user + self.key = conf.key + self.auth_url = conf.auth + self.use_proxy = conf.use_proxy in ('1', 'yes', 'true', 'True') + if self.use_proxy: + # Get the auth token + parsed = urlparse(self.auth_url) + if parsed.scheme == 'http': + hc = HTTPConnection(parsed.netloc) + elif parsed.scheme == 'https': + hc = HTTPSConnection(parsed.netloc) + else: + raise ClientException( + 'Cannot handle protocol scheme %s for url %s' % + (parsed.scheme, self.auth_url)) + hc_args = ('GET', parsed.path, None, + {'X-Auth-User': self.user, 'X-Auth-Key': self.key}) + hc.request(*hc_args) + hcr = hc.getresponse() + hcrd = hcr.read() + if hcr.status != 204: + raise Exception("Could not authenticate (%s)" % hcr.status) + self.token = hcr.getheader('x-auth-token') + self.account = hcr.getheader('x-storage-url').split('/')[-1] + if conf.url == '': + self.url = hcr.getheader('x-storage-url') + else: + self.url = conf.url + else: + self.token = 'SlapChop!' + self.account = conf.account + self.url = conf.url + self.container_name = conf.container_name + + self.object_size = int(conf.object_size) + self.object_sources = conf.object_sources + self.files = [] + if self.object_sources: + self.object_sources = self.object_sources.split() + self.files = [file(f, 'rb').read() for f in self.object_sources] + + self.put_concurrency = int(conf.put_concurrency) + self.get_concurrency = int(conf.get_concurrency) + self.del_concurrency = int(conf.del_concurrency) + self.total_objects = int(conf.num_objects) + self.total_gets = int(conf.num_gets) + self.timeout = int(conf.timeout) + self.url_parsed = urlparse(self.url) + self.devices = conf.devices.split() + self.names = names + self.local = eventlet.corolocal.local() + self.conn_pool = ConnectionPool(self.url, + max(self.put_concurrency, self.get_concurrency, + self.del_concurrency)) + + def _log_status(self, title): + total = time.time() - self.beginbeat + self.logger.info('%s %s [%s failures], %.01f/s' % ( + self.complete, title, self.failures, + (float(self.complete) / total), + )) + + def _create_connection(self): + if self.url_parsed[0] == 'https': + hc = HTTPSConnection(self.url_parsed[1]) + elif self.url_parsed[0] == 'http': + hc = HTTPConnection(self.url_parsed[1]) + else: + raise Exception("Can't handle %s" % self.url_parsed[0]) + return hc + + def _send_request(self, *args): + hc = self.conn_pool.get() + try: + start = time.time() + try: + hc.request(*args) + hcr = hc.getresponse() + hcrd = hcr.read() + hcr.close() + except CannotSendRequest: + self.logger.info("CannonSendRequest. Skipping...") + try: + hc.close() + except: + pass + self.failures += 1 + hc = self._create_connection() + return + total = time.time() - start + self.logger.debug("%s %s: %04f" % + (args[0], args[1], total)) + if hcr.status < 200 or hcr.status > 299: + self.failures += 1 + return False + else: + return True + finally: + self.conn_pool.put(hc) + + def run(self): + pool = eventlet.GreenPool(self.concurrency) + events = [] + self.beginbeat = self.heartbeat = time.time() + self.heartbeat -= 13 # just to get the first report quicker + self.failures = 0 + self.complete = 0 + for i in xrange(self.total): + pool.spawn_n(self._run, i) + pool.waitall() + self._log_status(self.msg + ' **FINAL**') + + def _run(self, thread): + return + + +class BenchController(object): + def __init__(self, logger, conf): + self.logger = logger + self.conf = conf + self.names = [] + self.delete = conf.delete in ('1', 'true', 'True', 'yes') + self.gets = int(conf.num_gets) + + def run(self): + puts = BenchPUT(self.logger, self.conf, self.names) + puts.run() + if self.gets: + gets = BenchGET(self.logger, self.conf, self.names) + gets.run() + if self.delete: + dels = BenchDELETE(self.logger, self.conf, self.names) + dels.run() + + +class BenchDELETE(Bench): + def __init__(self, logger, conf, names): + Bench.__init__(self, logger, conf, names) + self.concurrency = self.del_concurrency + self.total = len(names) + self.msg = 'DEL' + + def _run(self, thread): + if time.time() - self.heartbeat >= 15: + self.heartbeat = time.time() + self._log_status('DEL') + device, partition, path = self.names.pop() + headers = { + 'X-Timestamp': "%013.05f" % time.time(), + 'X-ID': str(uuid.uuid4()), + 'X-Auth-Token': self.token, + } + if self.use_proxy: + hc_args = ('DELETE', "/v1/%s/%s/%s" % + (self.account, self.container_name, path), '', headers) + else: + hc_args = ('DELETE', "/%s/%s/%s/%s/%s" % + (device, partition, self.account, self.container_name, path), + '', headers) + self._send_request(*hc_args) + self.complete += 1 + + +class BenchGET(Bench): + def __init__(self, logger, conf, names): + Bench.__init__(self, logger, conf, names) + self.concurrency = self.get_concurrency + self.total = self.total_gets + self.msg = 'GETS' + + def _run(self, thread): + if time.time() - self.heartbeat >= 15: + self.heartbeat = time.time() + self._log_status('GETS') + device, partition, name = random.choice(self.names) + headers = { + 'X-Auth-Token': self.token, + 'X-Timestamp': "%013.05f" % time.time(), + } + if self.use_proxy: + hc_args = ('GET', '/v1/%s/%s/%s' % + (self.account, self.container_name, name), '', headers) + else: + hc_args = ('GET', '/%s/%s/%s/%s/%s' % + (device, partition, self.account, self.container_name, name), + '', headers) + self._send_request(*hc_args) + self.complete += 1 + + +class BenchPUT(Bench): + def __init__(self, logger, conf, names): + Bench.__init__(self, logger, conf, names) + self.concurrency = self.put_concurrency + self.total = self.total_objects + self.msg = 'PUTS' + if self.use_proxy: + # Create the container + if self.url.startswith('http://'): + hc = HTTPConnection(self.url.split('/')[2]) + else: + hc = HTTPSConnection(self.url.split('/')[2]) + hc_args = ('PUT', + '/v1/%s/%s' % (self.account, self.container_name), + None, {'X-Auth-Token': self.token}) + hc.request(*hc_args) + hcr = hc.getresponse() + hcrd = hcr.read() + if hcr.status < 200 or hcr.status > 299: + raise Exception('Could not create container %s: code: %s' % + (self.container_name, hcr.status)) + + def _run(self, thread): + if time.time() - self.heartbeat >= 15: + self.heartbeat = time.time() + self._log_status('PUTS') + name = uuid.uuid4().hex + if self.object_sources: + source = random.choice(self.files) + else: + source = '0' * self.object_size + headers = { + 'Content-Type': 'application/octet-stream', + 'X-ID': str(uuid.uuid4()), + 'X-Auth-Token': self.token, + 'X-Timestamp': "%013.05f" % time.time(), + } + device = random.choice(self.devices) + partition = str(random.randint(1, 3000)) + if self.use_proxy: + hc_args = ('PUT', '/v1/%s/%s/%s' % + (self.account, self.container_name, name), source, headers) + else: + hc_args = ('PUT', '/%s/%s/%s/%s/%s' % + (device, partition, self.account, self.container_name, name), + source, headers) + if self._send_request(*hc_args): + self.names.append((device, partition, name)) + self.complete += 1 diff --git a/swift/common/utils.py b/swift/common/utils.py index f2e186c03e..7f193e66c3 100644 --- a/swift/common/utils.py +++ b/swift/common/utils.py @@ -534,8 +534,10 @@ def cache_from_env(env): return item_from_env(env, 'swift.cache') -def readconf(conf, section_name, log_name=None): - c = ConfigParser() +def readconf(conf, section_name, log_name=None, defaults=None): + if defaults is None: + defaults = {} + c = ConfigParser(defaults) if not c.read(conf): print "Unable to read config file %s" % conf sys.exit(1) From fddbbc013429f9eb1d5f5eb4033ccb3b6faaf9a4 Mon Sep 17 00:00:00 2001 From: Chuck Thier Date: Tue, 28 Sep 2010 15:00:59 +0000 Subject: [PATCH 37/89] Removed some code that was no longer used --- swift/common/bench.py | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/swift/common/bench.py b/swift/common/bench.py index 49c4873809..51087a5cb1 100644 --- a/swift/common/bench.py +++ b/swift/common/bench.py @@ -1,13 +1,11 @@ -import hashlib import uuid import time import random from urlparse import urlparse import eventlet.pools -import eventlet.corolocal from eventlet.green.httplib import HTTPSConnection, \ - HTTPResponse, CannotSendRequest, _UNKNOWN + HTTPResponse, CannotSendRequest from swift.common.bufferedhttp \ import BufferedHTTPConnection as HTTPConnection @@ -82,7 +80,6 @@ class Bench(object): self.url_parsed = urlparse(self.url) self.devices = conf.devices.split() self.names = names - self.local = eventlet.corolocal.local() self.conn_pool = ConnectionPool(self.url, max(self.put_concurrency, self.get_concurrency, self.del_concurrency)) @@ -113,7 +110,7 @@ class Bench(object): hcrd = hcr.read() hcr.close() except CannotSendRequest: - self.logger.info("CannonSendRequest. Skipping...") + self.logger.info("CannotSendRequest. Skipping...") try: hc.close() except: From f71a85a05bd3d12d44def96f2152cf4dd8b0f180 Mon Sep 17 00:00:00 2001 From: Chuck Thier Date: Tue, 28 Sep 2010 15:05:08 +0000 Subject: [PATCH 38/89] Missed a rename --- bin/swift-bench | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bin/swift-bench b/bin/swift-bench index 10df6a74a3..0662726233 100755 --- a/bin/swift-bench +++ b/bin/swift-bench @@ -10,7 +10,7 @@ from optparse import OptionParser from swift.common.bench import BenchController from swift.common.utils import readconf, NamedLogger -# The defaults should be sufficient to run the slapper on a SAIO +# The defaults should be sufficient to run swift-bench on a SAIO CONF_DEFAULTS = { 'auth': 'http://saio:11000/v1.0', 'user': 'test:tester', From ed2a3e9bd2dfbaccacf25c804f7414738cf10a16 Mon Sep 17 00:00:00 2001 From: Chuck Thier Date: Tue, 28 Sep 2010 16:06:35 +0000 Subject: [PATCH 39/89] Now uses TRUE_VALUES from utils.py --- swift/common/bench.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/swift/common/bench.py b/swift/common/bench.py index 51087a5cb1..4dc5cf7419 100644 --- a/swift/common/bench.py +++ b/swift/common/bench.py @@ -9,6 +9,7 @@ from eventlet.green.httplib import HTTPSConnection, \ from swift.common.bufferedhttp \ import BufferedHTTPConnection as HTTPConnection +from swift.common.utils import TRUE_VALUES class ConnectionPool(eventlet.pools.Pool): @@ -33,7 +34,7 @@ class Bench(object): self.user = conf.user self.key = conf.key self.auth_url = conf.auth - self.use_proxy = conf.use_proxy in ('1', 'yes', 'true', 'True') + self.use_proxy = conf.use_proxy in TRUE_VALUES if self.use_proxy: # Get the auth token parsed = urlparse(self.auth_url) @@ -150,7 +151,7 @@ class BenchController(object): self.logger = logger self.conf = conf self.names = [] - self.delete = conf.delete in ('1', 'true', 'True', 'yes') + self.delete = conf.delete in TRUE_VALUES self.gets = int(conf.num_gets) def run(self): From b44296d146fdeaffde511408eeea18160cd22afe Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Wed, 29 Sep 2010 15:22:04 -0500 Subject: [PATCH 40/89] added additional docs --- doc/source/overview_stats.rst | 88 +++++++++++++++++++++++++++++++++-- 1 file changed, 85 insertions(+), 3 deletions(-) diff --git a/doc/source/overview_stats.rst b/doc/source/overview_stats.rst index 3406f8522c..ef54169a1f 100644 --- a/doc/source/overview_stats.rst +++ b/doc/source/overview_stats.rst @@ -14,7 +14,10 @@ Log Types Access logs *********** -Access logs are the proxy server logs. +Access logs are the proxy server logs. Rackspace uses syslog-ng to redirect +the proxy log output to an hourly log file. For example, a proxy request that +is made on August 4, 2010 at 12:37 gets logged in a file named 2010080412. +This allows easy log rotation and easy per-hour log processing. ****************** Storage stats logs @@ -30,7 +33,7 @@ means that, system wide, one csv file is produced for every storage node. Rackspace runs the account stats logger every hour. Therefore, in a cluster of ten account servers, ten csv files are produced every hour. Also, every account will have one entry for every replica in the system. On average, there -will be three copies of each account in the aggreagate of all account stat csv +will be three copies of each account in the aggregate of all account stat csv files created in one system-wide run. ---------------------- @@ -44,7 +47,7 @@ are stored on disk, where the logs will be stored in swift (account and container), the filename format of the logs on disk, the location of the plugin class definition, and any plugin-specific config values. -The plugin class definition defines three methods. The constuctor must accept +The plugin class definition defines three methods. The constructor must accept one argument (the dict representation of the plugin's config section). The process method must accept an iterator, and the account, container, and object name of the log. The keylist_mapping accepts no parameters. @@ -77,3 +80,82 @@ each plugin's keylist_mapping method. The resulting csv file has one line per (account, year, month, day, hour) for all log files processed in that run of swift-log-stats-collector. + + +================================ +Running the stats system on SAIO +================================ + +#. Create a swift account to use for storing stats information, and note the + account hash. The hash will be used in config files. + +#. Install syslog-ng + + sudo apt-get install syslog-ng + +#. Add a destination rule to `/etc/syslog-ng/syslog-ng.conf` + + destination df_syslog_hourly { file("/var/log/swift/access-$YEAR$MONTH$DAY$HOUR"); }; + +#. Edit the destination rules to standard logging in + `/etc/syslog-ng/syslog-ng.conf` by adding the destination just created. + This will cause syslog messages to be also put into a file, named by the + current hour, in `/var/log/swift`. + + log { + source(s_all); + filter(f_syslog); + destination(df_syslog); + destination(df_syslog_hourly); + }; + +#. Restart syslog-ng + +#. Create `/etc/swift/log-processor.conf` + + [log-processor] + swift_account = + user = + + [log-processor-access] + swift_account = + container_name = log_data + source_filename_format = access-%Y%m%d%H + class_path = swift.stats.access_processor.AccessLogProcessor + user = + + [log-processor-stats] + swift_account = + container_name = account_stats + source_filename_format = stats-%Y%m%d%H_* + class_path = swift.stats.stats_processor.StatsLogProcessor + account_server_conf = /etc/swift/account-server/1.conf + user = + +#. Create a `cron` job to run once per hour to create the stats logs. In + `/etc/cron.d/swift-stats-log-creator` + + 0 * * * * swift-account-stats-logger /etc/swift/log-processor.conf + +#. Create a `cron` job to run once per hour to upload the stats logs. In + `/etc/cron.d/swift-stats-log-uploader` + + 10 * * * * swift-log-uploader /etc/swift/log-processor.conf stats + +#. Create a `cron` job to run once per hour to upload the access logs. In + `/etc/cron.d/swift-access-log-uploader` + + 5 * * * * swift-log-uploader /etc/swift/log-processor.conf access + +#. Create a `cron` job to run once per hour to process the logs. In + `/etc/cron.d/swift-stats-processor` + + 30 * * * * swift-log-stats-collector /etc/swift/log-processor.conf + +After running for a few hours, you should start to see .csv files in the +log_processing_data container in the swift stats account that was created +earlier. This file will have one entry per account per hour for each account +with activity in that hour. One .csv file should be produced per hour. Note +that the stats will be delayed by at least two hours by default. This can be +changed with the new_log_cutoff variable in the config file. See +`log-processing.conf-sample` for more details. \ No newline at end of file From a2f7209eac9453843010f5fbf05f9f3868ebe4e3 Mon Sep 17 00:00:00 2001 From: Chuck Thier Date: Thu, 30 Sep 2010 10:35:10 -0500 Subject: [PATCH 41/89] Updated saio with instructions for using a loopback device --- doc/source/development_saio.rst | 25 +++++++++++++++++++++---- 1 file changed, 21 insertions(+), 4 deletions(-) diff --git a/doc/source/development_saio.rst b/doc/source/development_saio.rst index 9365665cb9..37e6613068 100644 --- a/doc/source/development_saio.rst +++ b/doc/source/development_saio.rst @@ -41,11 +41,23 @@ good idea what to do on other environments. python-xattr sqlite3 xfsprogs python-webob python-eventlet python-greenlet python-pastedeploy` #. Install anything else you want, like screen, ssh, vim, etc. - #. `fdisk /dev/sdb` (set up a single partition) - #. `mkfs.xfs -i size=1024 /dev/sdb1` + #. If you are using another partition: + + #. `fdisk /dev/sdb` (set up a single partition) + #. `mkfs.xfs -i size=1024 /dev/sdb1` + #. Edit `/etc/fstab` and add + `/dev/sdb1 /mnt/sdb1 xfs noatime,nodiratime,nobarrier,logbufs=8 0 0` + + #. If you would like to use a loopback device instead of another partition: + + #. `dd if=/dev/zero of=/swift-disk bs=1024 count=1000000` (modify count to + make a larger or smaller partition) + #. `losetup /dev/loop0 /swift-disk` + #. `mkfs.xfs -i size=1024 /dev/loop0` + #. Edit `/etc/fstab` and add + `/dev/loop0 /mnt/sdb1 xfs noatime,nodiratime,nobarrier,logbufs=8 0 0` + #. `mkdir /mnt/sdb1` - #. Edit `/etc/fstab` and add - `/dev/sdb1 /mnt/sdb1 xfs noatime,nodiratime,nobarrier,logbufs=8 0 0` #. `mount /mnt/sdb1` #. `mkdir /mnt/sdb1/1 /mnt/sdb1/2 /mnt/sdb1/3 /mnt/sdb1/4 /mnt/sdb1/test` #. `chown : /mnt/sdb1/*` @@ -472,6 +484,11 @@ good idea what to do on other environments. sudo service rsyslog restart sudo service memcached restart + .. note:: + + If you are using a loopback device, substitute `/dev/sdb1` above with + `/dev/loop0` + #. Create `~/bin/remakerings`:: #!/bin/bash From 41906b15fdf7830f3aadaf4b9a0dbaf1ec344515 Mon Sep 17 00:00:00 2001 From: Chuck Thier Date: Thu, 30 Sep 2010 10:47:57 -0500 Subject: [PATCH 42/89] Changed rsync on saio to only listen on localhost --- doc/source/development_saio.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/doc/source/development_saio.rst b/doc/source/development_saio.rst index 37e6613068..fafe72482a 100644 --- a/doc/source/development_saio.rst +++ b/doc/source/development_saio.rst @@ -76,7 +76,7 @@ good idea what to do on other environments. gid = log file = /var/log/rsyncd.log pid file = /var/run/rsyncd.pid - + address = 127.0.0.1 [account6012] max connections = 25 From 58cc9078dad4fcbb4286ee89db8a480e668f819b Mon Sep 17 00:00:00 2001 From: Chuck Thier Date: Thu, 30 Sep 2010 11:18:37 -0500 Subject: [PATCH 43/89] Updated to be a bit more generic about the VM --- doc/source/development_saio.rst | 22 ++++------------------ 1 file changed, 4 insertions(+), 18 deletions(-) diff --git a/doc/source/development_saio.rst b/doc/source/development_saio.rst index fafe72482a..19e1059d65 100644 --- a/doc/source/development_saio.rst +++ b/doc/source/development_saio.rst @@ -7,9 +7,7 @@ Instructions for setting up a dev VM ------------------------------------ This documents setting up a virtual machine for doing Swift development. The -virtual machine will emulate running a four node Swift cluster. It assumes -you're using *VMware Fusion 3* on *Mac OS X Snow Leopard*, but should give a -good idea what to do on other environments. +virtual machine will emulate running a four node Swift cluster. * Get the *Ubuntu 10.04 LTS (Lucid Lynx)* server image: @@ -17,20 +15,7 @@ good idea what to do on other environments. - Ubuntu Live/Install: http://cdimage.ubuntu.com/releases/10.04/release/ubuntu-10.04-dvd-amd64.iso (4.1 GB) - Ubuntu Mirrors: https://launchpad.net/ubuntu/+cdmirrors -* Create guest virtual machine: - - #. `Continue without disc` - #. `Use operating system installation disc image file`, pick the .iso - from above. - #. Select `Linux` and `Ubuntu 64-bit`. - #. Fill in the *Linux Easy Install* details. - #. `Customize Settings`, name the image whatever you want - (`SAIO` for instance.) - #. When the `Settings` window comes up, select `Hard Disk`, create an - extra disk (the defaults are fine). - #. Start the virtual machine up and wait for the easy install to - finish. - +* Create guest virtual machine from the Ubuntu image * As root on guest (you'll have to log in as you, then `sudo su -`): #. `apt-get install python-software-properties` @@ -41,8 +26,9 @@ good idea what to do on other environments. python-xattr sqlite3 xfsprogs python-webob python-eventlet python-greenlet python-pastedeploy` #. Install anything else you want, like screen, ssh, vim, etc. - #. If you are using another partition: + #. If you would like to use another partition for storage: + #. Create the device through your VM software #. `fdisk /dev/sdb` (set up a single partition) #. `mkfs.xfs -i size=1024 /dev/sdb1` #. Edit `/etc/fstab` and add From a9508c70b53eee4dccfdb4ad10e81adda4793448 Mon Sep 17 00:00:00 2001 From: Chuck Thier Date: Thu, 30 Sep 2010 11:23:39 -0500 Subject: [PATCH 44/89] Moved the message about adding an extra device to the step when creating the VM --- doc/source/development_saio.rst | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/doc/source/development_saio.rst b/doc/source/development_saio.rst index 19e1059d65..b5d23ab63e 100644 --- a/doc/source/development_saio.rst +++ b/doc/source/development_saio.rst @@ -15,7 +15,9 @@ virtual machine will emulate running a four node Swift cluster. - Ubuntu Live/Install: http://cdimage.ubuntu.com/releases/10.04/release/ubuntu-10.04-dvd-amd64.iso (4.1 GB) - Ubuntu Mirrors: https://launchpad.net/ubuntu/+cdmirrors -* Create guest virtual machine from the Ubuntu image +* Create guest virtual machine from the Ubuntu image (if you are going to use + a separate partition for swift data, be sure to add another device when + creating the VM) * As root on guest (you'll have to log in as you, then `sudo su -`): #. `apt-get install python-software-properties` @@ -28,7 +30,6 @@ virtual machine will emulate running a four node Swift cluster. #. Install anything else you want, like screen, ssh, vim, etc. #. If you would like to use another partition for storage: - #. Create the device through your VM software #. `fdisk /dev/sdb` (set up a single partition) #. `mkfs.xfs -i size=1024 /dev/sdb1` #. Edit `/etc/fstab` and add From 9de5e9292704fe2634e2b14698c0df0f14065ee0 Mon Sep 17 00:00:00 2001 From: Chuck Thier Date: Thu, 30 Sep 2010 12:41:49 -0500 Subject: [PATCH 45/89] Updated loopback instructions per redbo --- doc/source/development_saio.rst | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/doc/source/development_saio.rst b/doc/source/development_saio.rst index b5d23ab63e..d850c76514 100644 --- a/doc/source/development_saio.rst +++ b/doc/source/development_saio.rst @@ -37,12 +37,11 @@ virtual machine will emulate running a four node Swift cluster. #. If you would like to use a loopback device instead of another partition: - #. `dd if=/dev/zero of=/swift-disk bs=1024 count=1000000` (modify count to - make a larger or smaller partition) - #. `losetup /dev/loop0 /swift-disk` - #. `mkfs.xfs -i size=1024 /dev/loop0` + #. `dd if=/dev/zero of=/srv/swift-disk bs=1024 count=0 seek=1000000` + (modify seek to make a larger or smaller partition) + #. `mkfs.xfs -i size=1024 /srv/swift-disk` #. Edit `/etc/fstab` and add - `/dev/loop0 /mnt/sdb1 xfs noatime,nodiratime,nobarrier,logbufs=8 0 0` + `/srv/swift-disk /mnt/sdb1 xfs loop,noatime,nodiratime,nobarrier,logbufs=8 0 0` #. `mkdir /mnt/sdb1` #. `mount /mnt/sdb1` @@ -55,7 +54,7 @@ virtual machine will emulate running a four node Swift cluster. #. Add to `/etc/rc.local` (before the `exit 0`):: mkdir /var/run/swift - chown : /var/run/swift + chown : /var/run/swift #. Create /etc/rsyncd.conf:: From 1bb40411cf22b3bfdd94699699e8e6d487aa62d8 Mon Sep 17 00:00:00 2001 From: Chuck Thier Date: Thu, 30 Sep 2010 12:45:54 -0500 Subject: [PATCH 46/89] updated the note for the resetswift script to be in line with the new loopback device --- doc/source/development_saio.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/doc/source/development_saio.rst b/doc/source/development_saio.rst index d850c76514..17a443d692 100644 --- a/doc/source/development_saio.rst +++ b/doc/source/development_saio.rst @@ -473,7 +473,7 @@ virtual machine will emulate running a four node Swift cluster. .. note:: If you are using a loopback device, substitute `/dev/sdb1` above with - `/dev/loop0` + `/srv/swift-disk` #. Create `~/bin/remakerings`:: From 880f5af2ffc1f2a145e03a179c4826c8b260efde Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Thu, 30 Sep 2010 15:48:56 -0500 Subject: [PATCH 47/89] documentation clarification and pep8 fixes --- doc/source/overview_stats.rst | 28 +++++++++++----------- swift/common/compressing_file_reader.py | 2 ++ swift/common/internal_proxy.py | 31 +++++++++++++++++++------ swift/common/utils.py | 6 +++-- 4 files changed, 44 insertions(+), 23 deletions(-) diff --git a/doc/source/overview_stats.rst b/doc/source/overview_stats.rst index ef54169a1f..b151674514 100644 --- a/doc/source/overview_stats.rst +++ b/doc/source/overview_stats.rst @@ -4,7 +4,7 @@ Swift stats system The swift stats system is composed of three parts parts: log creation, log uploading, and log processing. The system handles two types of logs (access -and storage stats), but it can be extended to handle other types of logs. +and account stats), but it can be extended to handle other types of logs. --------- Log Types @@ -20,21 +20,21 @@ is made on August 4, 2010 at 12:37 gets logged in a file named 2010080412. This allows easy log rotation and easy per-hour log processing. ****************** -Storage stats logs +Account stats logs ****************** -Storage logs (also referred to as stats logs) are generated by a stats system -process. swift-account-stats-logger runs on each account server (via cron) and -walks the filesystem looking for account databases. When an account database -is found, the logger selects the account hash, bytes_used, container_count, -and object_count. These values are then written out as one line in a csv file. -One csv file is produced for every run of swift-account-stats-logger. This -means that, system wide, one csv file is produced for every storage node. -Rackspace runs the account stats logger every hour. Therefore, in a cluster of -ten account servers, ten csv files are produced every hour. Also, every -account will have one entry for every replica in the system. On average, there -will be three copies of each account in the aggregate of all account stat csv -files created in one system-wide run. +Account stats logs are generated by a stats system process. +swift-account-stats-logger runs on each account server (via cron) and walks +the filesystem looking for account databases. When an account database is +found, the logger selects the account hash, bytes_used, container_count, and +object_count. These values are then written out as one line in a csv file. One +csv file is produced for every run of swift-account-stats-logger. This means +that, system wide, one csv file is produced for every storage node. Rackspace +runs the account stats logger every hour. Therefore, in a cluster of ten +account servers, ten csv files are produced every hour. Also, every account +will have one entry for every replica in the system. On average, there will be +three copies of each account in the aggregate of all account stat csv files +created in one system-wide run. ---------------------- Log Processing plugins diff --git a/swift/common/compressing_file_reader.py b/swift/common/compressing_file_reader.py index bfe62aee81..d6de9154eb 100644 --- a/swift/common/compressing_file_reader.py +++ b/swift/common/compressing_file_reader.py @@ -16,6 +16,7 @@ import zlib import struct + class CompressingFileReader(object): ''' Wraps a file object and provides a read method that returns gzip'd data. @@ -35,6 +36,7 @@ class CompressingFileReader(object): :param file_obj: File object to read from :param compresslevel: compression level ''' + def __init__(self, file_obj, compresslevel=9): self._f = file_obj self._compressor = zlib.compressobj(compresslevel, diff --git a/swift/common/internal_proxy.py b/swift/common/internal_proxy.py index f548a3a2fa..f308e1c5bd 100644 --- a/swift/common/internal_proxy.py +++ b/swift/common/internal_proxy.py @@ -20,13 +20,27 @@ from json import loads as json_loads from swift.common.compressing_file_reader import CompressingFileReader from swift.proxy.server import BaseApplication + class MemcacheStub(object): - def get(self, *a, **kw): return None - def set(self, *a, **kw): return None - def incr(self, *a, **kw): return 0 - def delete(self, *a, **kw): return None - def set_multi(self, *a, **kw): return None - def get_multi(self, *a, **kw): return [] + + def get(self, *a, **kw): + return None + + def set(self, *a, **kw): + return None + + def incr(self, *a, **kw): + return 0 + + def delete(self, *a, **kw): + return None + + def set_multi(self, *a, **kw): + return None + + def get_multi(self, *a, **kw): + return [] + class InternalProxy(object): """ @@ -38,6 +52,7 @@ class InternalProxy(object): :param logger: logger to log requests to :param retries: number of times to retry each request """ + def __init__(self, proxy_server_conf=None, logger=None, retries=0): self.upload_app = BaseApplication(proxy_server_conf, memcache=MemcacheStub(), @@ -56,6 +71,7 @@ class InternalProxy(object): :param object_name: name of object being uploaded :param compress: if True, compresses object as it is uploaded :param content_type: content-type of object + :param etag: etag for object to check successful upload :returns: True if successful, False otherwise """ log_create_pattern = '/v1/%s/%s/%s' % (account, container, object_name) @@ -72,7 +88,8 @@ class InternalProxy(object): if hasattr(source_file, 'read'): compressed_file = CompressingFileReader(source_file) else: - compressed_file = CompressingFileReader(open(source_file, 'rb')) + compressed_file = CompressingFileReader( + open(source_file, 'rb')) req.body_file = compressed_file else: if not hasattr(source_file, 'read'): diff --git a/swift/common/utils.py b/swift/common/utils.py index ab498e3c05..0dc843080b 100644 --- a/swift/common/utils.py +++ b/swift/common/utils.py @@ -552,6 +552,7 @@ def cache_from_env(env): """ return item_from_env(env, 'swift.cache') + def readconf(conf, section_name, log_name=None): """ Read config file and return config items as a dict @@ -571,7 +572,8 @@ def readconf(conf, section_name, log_name=None): if c.has_section(section_name): conf = dict(c.items(section_name)) else: - print "Unable to find %s config section in %s" % (section_name, conf) + print "Unable to find %s config section in %s" % (section_name, + conf) sys.exit(1) if "log_name" not in conf: if log_name is not None: @@ -581,7 +583,7 @@ def readconf(conf, section_name, log_name=None): else: conf = {} for s in c.sections(): - conf.update({s:dict(c.items(s))}) + conf.update({s: dict(c.items(s))}) if 'log_name' not in conf: conf['log_name'] = log_name return conf From 745ee99a00221079fc15f7b0eb2b67410aa3a7c4 Mon Sep 17 00:00:00 2001 From: Anne Gentle Date: Thu, 30 Sep 2010 15:50:20 -0500 Subject: [PATCH 48/89] Minor spelling and typo fixes with added words to intro the Proxy Server config. --- doc/source/deployment_guide.rst | 47 ++++++++++++++++++--------------- 1 file changed, 26 insertions(+), 21 deletions(-) diff --git a/doc/source/deployment_guide.rst b/doc/source/deployment_guide.rst index 0c2e52e59a..df5b4f642d 100644 --- a/doc/source/deployment_guide.rst +++ b/doc/source/deployment_guide.rst @@ -47,7 +47,7 @@ If you need more throughput to either Account or Container Services, they may each be deployed to their own servers. For example you might use faster (but more expensive) SAS or even SSD drives to get faster disk I/O to the databases. -Load balancing and network design is left as an excercise to the reader, +Load balancing and network design is left as an exercise to the reader, but this is a very important part of the cluster, so time should be spent designing the network for a Swift cluster. @@ -59,7 +59,7 @@ Preparing the Ring The first step is to determine the number of partitions that will be in the ring. We recommend that there be a minimum of 100 partitions per drive to -insure even distribution accross the drives. A good starting point might be +insure even distribution across the drives. A good starting point might be to figure out the maximum number of drives the cluster will contain, and then multiply by 100, and then round up to the nearest power of two. @@ -154,8 +154,8 @@ Option Default Description ------------------ ---------- --------------------------------------------- swift_dir /etc/swift Swift configuration directory devices /srv/node Parent directory of where devices are mounted -mount_check true Weather or not check if the devices are - mounted to prevent accidently writing +mount_check true Whether or not check if the devices are + mounted to prevent accidentally writing to the root device bind_ip 0.0.0.0 IP Address for server to bind to bind_port 6000 Port for server to bind to @@ -173,7 +173,7 @@ use paste.deploy entry point for the object log_name object-server Label used when logging log_facility LOG_LOCAL0 Syslog log facility log_level INFO Logging level -log_requests True Weather or not to log each request +log_requests True Whether or not to log each request user swift User to run as node_timeout 3 Request timeout to external services conn_timeout 0.5 Connection timeout to external services @@ -193,7 +193,7 @@ Option Default Description log_name object-replicator Label used when logging log_facility LOG_LOCAL0 Syslog log facility log_level INFO Logging level -daemonize yes Weather or not to run replication as a +daemonize yes Whether or not to run replication as a daemon run_pause 30 Time in seconds to wait between replication passes @@ -249,9 +249,9 @@ The following configuration options are available: Option Default Description ------------------ ---------- -------------------------------------------- swift_dir /etc/swift Swift configuration directory -devices /srv/node Parent irectory of where devices are mounted -mount_check true Weather or not check if the devices are - mounted to prevent accidently writing +devices /srv/node Parent directory of where devices are mounted +mount_check true Whether or not check if the devices are + mounted to prevent accidentally writing to the root device bind_ip 0.0.0.0 IP Address for server to bind to bind_port 6001 Port for server to bind to @@ -339,8 +339,8 @@ Option Default Description ------------------ ---------- --------------------------------------------- swift_dir /etc/swift Swift configuration directory devices /srv/node Parent directory or where devices are mounted -mount_check true Weather or not check if the devices are - mounted to prevent accidently writing +mount_check true Whether or not check if the devices are + mounted to prevent accidentally writing to the root device bind_ip 0.0.0.0 IP Address for server to bind to bind_port 6002 Port for server to bind to @@ -353,7 +353,7 @@ user swift User to run as ================== ============== ========================================== Option Default Description ------------------ -------------- ------------------------------------------ -use paste.deploy entry point for the account +use Entry point for paste.deploy for the account server. For most cases, this should be `egg:swift#account`. log_name account-server Label used when logging @@ -412,6 +412,11 @@ conn_timeout 0.5 Connection timeout to external services Proxy Server Configuration -------------------------- +An example Proxy Server configuration can be found at +etc/proxy-server.conf-sample in the source code repository. + +The following configuration options are available: + [DEFAULT] ============================ =============== ============================= @@ -432,7 +437,7 @@ key_file Path to the ssl .key ============================ =============== ============================= Option Default Description ---------------------------- --------------- ----------------------------- -use paste.deploy entry point for +use Entry point for paste.deploy for the proxy server. For most cases, this should be `egg:swift#proxy`. @@ -443,10 +448,10 @@ log_headers True If True, log headers in each request recheck_account_existence 60 Cache timeout in seconds to send memcached for account - existance + existence recheck_container_existence 60 Cache timeout in seconds to send memcached for container - existance + existence object_chunk_size 65536 Chunk size to read from object servers client_chunk_size 65536 Chunk size to read from @@ -474,7 +479,7 @@ rate_limit_account_whitelist Comma separated list of rate limit rate_limit_account_blacklist Comma separated list of account name hashes to block - completly + completely ============================ =============== ============================= [auth] @@ -482,7 +487,7 @@ rate_limit_account_blacklist Comma separated list of ============ =================================== ======================== Option Default Description ------------ ----------------------------------- ------------------------ -use paste.deploy entry point +use Entry point for paste.deploy to use for auth. To use the swift dev auth, set to: @@ -500,7 +505,7 @@ Memcached Considerations ------------------------ Several of the Services rely on Memcached for caching certain types of -lookups, such as auth tokens, and container/account existance. Swift does +lookups, such as auth tokens, and container/account existence. Swift does not do any caching of actual object data. Memcached should be able to run on any servers that have available RAM and CPU. At Rackspace, we run Memcached on the proxy servers. The `memcache_servers` config option @@ -526,7 +531,7 @@ Most services support either a worker or concurrency value in the settings. This allows the services to make effective use of the cores available. A good starting point to set the concurrency level for the proxy and storage services to 2 times the number of cores available. If more than one service is -sharing a server, then some experimentaiton may be needed to find the best +sharing a server, then some experimentation may be needed to find the best balance. At Rackspace, our Proxy servers have dual quad core processors, giving us 8 @@ -548,7 +553,7 @@ Filesystem Considerations ------------------------- Swift is designed to be mostly filesystem agnostic--the only requirement -beeing that the filesystem supports extended attributes (xattrs). After +being that the filesystem supports extended attributes (xattrs). After thorough testing with our use cases and hardware configurations, XFS was the best all-around choice. If you decide to use a filesystem other than XFS, we highly recommend thorough testing. @@ -611,5 +616,5 @@ Logging Considerations Swift is set up to log directly to syslog. Every service can be configured with the `log_facility` option to set the syslog log facility destination. -It is recommended to use syslog-ng to route the logs to specific log +We recommended using syslog-ng to route the logs to specific log files locally on the server and also to remote log collecting servers. From f51df2cd2674c16ae4cb2bd91ab474d2dcaefc50 Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Thu, 30 Sep 2010 15:56:39 -0500 Subject: [PATCH 49/89] added overview stats to the doc index --- doc/source/index.rst | 1 + 1 file changed, 1 insertion(+) diff --git a/doc/source/index.rst b/doc/source/index.rst index 8760852f13..ee10ae67f0 100644 --- a/doc/source/index.rst +++ b/doc/source/index.rst @@ -24,6 +24,7 @@ Overview: overview_reaper overview_auth overview_replication + overview_stats Development: From 4f93661b9d695b9a12615dfd3eaa75ac86bd9397 Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Thu, 30 Sep 2010 16:02:45 -0500 Subject: [PATCH 50/89] made long lines wrap (grr pep8) --- swift/common/internal_proxy.py | 24 ++++++++++++++++-------- 1 file changed, 16 insertions(+), 8 deletions(-) diff --git a/swift/common/internal_proxy.py b/swift/common/internal_proxy.py index f308e1c5bd..2df7580967 100644 --- a/swift/common/internal_proxy.py +++ b/swift/common/internal_proxy.py @@ -100,11 +100,13 @@ class InternalProxy(object): req.content_length = None # to make sure we send chunked data if etag: req.etag = etag - resp = self.upload_app.handle_request(self.upload_app.update_request(req)) + resp = self.upload_app.handle_request( + self.upload_app.update_request(req)) tries = 1 while (resp.status_int < 200 or resp.status_int > 299) \ and tries <= self.retries: - resp = self.upload_app.handle_request(self.upload_app.update_request(req)) + resp = self.upload_app.handle_request( + self.upload_app.update_request(req)) tries += 1 if not (200 <= resp.status_int < 300): return False @@ -123,11 +125,13 @@ class InternalProxy(object): (account, container, object_name), environ={'REQUEST_METHOD': 'GET'}) req.account = account - resp = self.upload_app.handle_request(self.upload_app.update_request(req)) + resp = self.upload_app.handle_request( + self.upload_app.update_request(req)) tries = 1 while (resp.status_int < 200 or resp.status_int > 299) \ and tries <= self.retries: - resp = self.upload_app.handle_request(self.upload_app.update_request(req)) + resp = self.upload_app.handle_request( + self.upload_app.update_request(req)) tries += 1 return resp.status_int, resp.app_iter @@ -142,11 +146,13 @@ class InternalProxy(object): req = webob.Request.blank('/v1/%s/%s' % (account, container), environ={'REQUEST_METHOD': 'PUT'}) req.account = account - resp = self.upload_app.handle_request(self.upload_app.update_request(req)) + resp = self.upload_app.handle_request( + self.upload_app.update_request(req)) tries = 1 while (resp.status_int < 200 or resp.status_int > 299) \ and tries <= self.retries: - resp = self.upload_app.handle_request(self.upload_app.update_request(req)) + resp = self.upload_app.handle_request( + self.upload_app.update_request(req)) tries += 1 return 200 <= resp.status_int < 300 @@ -190,11 +196,13 @@ class InternalProxy(object): path += '?%s' % qs req = webob.Request.blank(path, environ={'REQUEST_METHOD': 'GET'}) req.account = account - resp = self.upload_app.handle_request(self.upload_app.update_request(req)) + resp = self.upload_app.handle_request( + self.upload_app.update_request(req)) tries = 1 while (resp.status_int < 200 or resp.status_int > 299) \ and tries <= self.retries: - resp = self.upload_app.handle_request(self.upload_app.update_request(req)) + resp = self.upload_app.handle_request( + self.upload_app.update_request(req)) tries += 1 if resp.status_int == 204: return [] From 117aef456d637598def40edef2b1dbc6b8988a88 Mon Sep 17 00:00:00 2001 From: Chuck Thier Date: Fri, 1 Oct 2010 19:46:35 +0000 Subject: [PATCH 51/89] Refactored swift-bench to reuse client.py and direct_client.py --- bin/swift-bench | 7 +- swift/common/bench.py | 178 ++++++++++++---------------------- swift/common/client.py | 4 +- swift/common/direct_client.py | 54 +++++++++++ 4 files changed, 125 insertions(+), 118 deletions(-) diff --git a/bin/swift-bench b/bin/swift-bench index 0662726233..aa1433ddc8 100755 --- a/bin/swift-bench +++ b/bin/swift-bench @@ -27,8 +27,9 @@ CONF_DEFAULTS = { 'container_name': uuid.uuid4().hex, 'use_proxy': 'yes', 'url': '', - 'devices': 'sdb', - 'logging_level': 'INFO', + 'account': '', + 'devices': 'sdb1', + 'log_level': 'INFO', 'timeout': '10', } @@ -93,7 +94,7 @@ if __name__ == '__main__': 'warning': logging.WARNING, 'error': logging.ERROR, 'critical': logging.CRITICAL}.get( - options.logging_level, logging.INFO)) + options.log_level.lower(), logging.INFO)) loghandler = logging.StreamHandler() logformat = logging.Formatter('%(asctime)s %(levelname)s %(message)s') loghandler.setFormatter(logformat) diff --git a/swift/common/bench.py b/swift/common/bench.py index 4dc5cf7419..e921ec9158 100644 --- a/swift/common/bench.py +++ b/swift/common/bench.py @@ -2,33 +2,28 @@ import uuid import time import random from urlparse import urlparse +from contextlib import contextmanager import eventlet.pools -from eventlet.green.httplib import HTTPSConnection, \ - HTTPResponse, CannotSendRequest +from eventlet.green.httplib import CannotSendRequest -from swift.common.bufferedhttp \ - import BufferedHTTPConnection as HTTPConnection from swift.common.utils import TRUE_VALUES +from swift.common import client +from swift.common import direct_client class ConnectionPool(eventlet.pools.Pool): + def __init__(self, url, size): self.url = url - self.url_parsed = urlparse(self.url) eventlet.pools.Pool.__init__(self, size, size) def create(self): - if self.url_parsed[0] == 'https': - hc = HTTPSConnection(self.url_parsed[1]) - elif self.url_parsed[0] == 'http': - hc = HTTPConnection(self.url_parsed[1]) - else: - raise Exception("Can't handle %s" % self.url_parsed[0]) - return hc + return client.http_connection(self.url) class Bench(object): + def __init__(self, logger, conf, names): self.logger = logger self.user = conf.user @@ -36,33 +31,18 @@ class Bench(object): self.auth_url = conf.auth self.use_proxy = conf.use_proxy in TRUE_VALUES if self.use_proxy: - # Get the auth token - parsed = urlparse(self.auth_url) - if parsed.scheme == 'http': - hc = HTTPConnection(parsed.netloc) - elif parsed.scheme == 'https': - hc = HTTPSConnection(parsed.netloc) - else: - raise ClientException( - 'Cannot handle protocol scheme %s for url %s' % - (parsed.scheme, self.auth_url)) - hc_args = ('GET', parsed.path, None, - {'X-Auth-User': self.user, 'X-Auth-Key': self.key}) - hc.request(*hc_args) - hcr = hc.getresponse() - hcrd = hcr.read() - if hcr.status != 204: - raise Exception("Could not authenticate (%s)" % hcr.status) - self.token = hcr.getheader('x-auth-token') - self.account = hcr.getheader('x-storage-url').split('/')[-1] + url, token = client.get_auth(self.auth_url, self.user, self.key) + self.token = token + self.account = url.split('/')[-1] if conf.url == '': - self.url = hcr.getheader('x-storage-url') + self.url = url else: self.url = conf.url else: self.token = 'SlapChop!' self.account = conf.account self.url = conf.url + self.ip, self.port = self.url.split('/')[2].split(':') self.container_name = conf.container_name self.object_size = int(conf.object_size) @@ -78,7 +58,6 @@ class Bench(object): self.total_objects = int(conf.num_objects) self.total_gets = int(conf.num_gets) self.timeout = int(conf.timeout) - self.url_parsed = urlparse(self.url) self.devices = conf.devices.split() self.names = names self.conn_pool = ConnectionPool(self.url, @@ -92,24 +71,12 @@ class Bench(object): (float(self.complete) / total), )) - def _create_connection(self): - if self.url_parsed[0] == 'https': - hc = HTTPSConnection(self.url_parsed[1]) - elif self.url_parsed[0] == 'http': - hc = HTTPConnection(self.url_parsed[1]) - else: - raise Exception("Can't handle %s" % self.url_parsed[0]) - return hc - - def _send_request(self, *args): - hc = self.conn_pool.get() + @contextmanager + def connection(self): try: - start = time.time() + hc = self.conn_pool.get() try: - hc.request(*args) - hcr = hc.getresponse() - hcrd = hcr.read() - hcr.close() + yield hc except CannotSendRequest: self.logger.info("CannotSendRequest. Skipping...") try: @@ -117,16 +84,7 @@ class Bench(object): except: pass self.failures += 1 - hc = self._create_connection() - return - total = time.time() - start - self.logger.debug("%s %s: %04f" % - (args[0], args[1], total)) - if hcr.status < 200 or hcr.status > 299: - self.failures += 1 - return False - else: - return True + hc = self.conn_pool.create() finally: self.conn_pool.put(hc) @@ -147,6 +105,7 @@ class Bench(object): class BenchController(object): + def __init__(self, logger, conf): self.logger = logger self.conf = conf @@ -166,6 +125,7 @@ class BenchController(object): class BenchDELETE(Bench): + def __init__(self, logger, conf, names): Bench.__init__(self, logger, conf, names) self.concurrency = self.del_concurrency @@ -176,24 +136,24 @@ class BenchDELETE(Bench): if time.time() - self.heartbeat >= 15: self.heartbeat = time.time() self._log_status('DEL') - device, partition, path = self.names.pop() - headers = { - 'X-Timestamp': "%013.05f" % time.time(), - 'X-ID': str(uuid.uuid4()), - 'X-Auth-Token': self.token, - } - if self.use_proxy: - hc_args = ('DELETE', "/v1/%s/%s/%s" % - (self.account, self.container_name, path), '', headers) - else: - hc_args = ('DELETE', "/%s/%s/%s/%s/%s" % - (device, partition, self.account, self.container_name, path), - '', headers) - self._send_request(*hc_args) + device, partition, name = self.names.pop() + with self.connection() as conn: + try: + if self.use_proxy: + client.delete_object(self.url, self.token, + self.container_name, name, http_conn=conn) + else: + node = {'ip': self.ip, 'port': self.port, 'device': device} + direct_client.direct_delete_object(node, partition, + self.account, self.container_name, name) + except client.ClientException, e: + self.logger.debug(str(e)) + self.failures += 1 self.complete += 1 class BenchGET(Bench): + def __init__(self, logger, conf, names): Bench.__init__(self, logger, conf, names) self.concurrency = self.get_concurrency @@ -205,42 +165,32 @@ class BenchGET(Bench): self.heartbeat = time.time() self._log_status('GETS') device, partition, name = random.choice(self.names) - headers = { - 'X-Auth-Token': self.token, - 'X-Timestamp': "%013.05f" % time.time(), - } - if self.use_proxy: - hc_args = ('GET', '/v1/%s/%s/%s' % - (self.account, self.container_name, name), '', headers) - else: - hc_args = ('GET', '/%s/%s/%s/%s/%s' % - (device, partition, self.account, self.container_name, name), - '', headers) - self._send_request(*hc_args) + with self.connection() as conn: + try: + if self.use_proxy: + client.get_object(self.url, self.token, + self.container_name, name, http_conn=conn) + else: + node = {'ip': self.ip, 'port': self.port, 'device': device} + direct_client.direct_get_object(node, partition, + self.account, self.container_name, name) + except client.ClientException, e: + self.logger.debug(str(e)) + self.failures += 1 self.complete += 1 class BenchPUT(Bench): + def __init__(self, logger, conf, names): Bench.__init__(self, logger, conf, names) self.concurrency = self.put_concurrency self.total = self.total_objects self.msg = 'PUTS' if self.use_proxy: - # Create the container - if self.url.startswith('http://'): - hc = HTTPConnection(self.url.split('/')[2]) - else: - hc = HTTPSConnection(self.url.split('/')[2]) - hc_args = ('PUT', - '/v1/%s/%s' % (self.account, self.container_name), - None, {'X-Auth-Token': self.token}) - hc.request(*hc_args) - hcr = hc.getresponse() - hcrd = hcr.read() - if hcr.status < 200 or hcr.status > 299: - raise Exception('Could not create container %s: code: %s' % - (self.container_name, hcr.status)) + with self.connection() as conn: + client.put_container(self.url, self.token, + self.container_name, http_conn=conn) def _run(self, thread): if time.time() - self.heartbeat >= 15: @@ -251,21 +201,21 @@ class BenchPUT(Bench): source = random.choice(self.files) else: source = '0' * self.object_size - headers = { - 'Content-Type': 'application/octet-stream', - 'X-ID': str(uuid.uuid4()), - 'X-Auth-Token': self.token, - 'X-Timestamp': "%013.05f" % time.time(), - } device = random.choice(self.devices) partition = str(random.randint(1, 3000)) - if self.use_proxy: - hc_args = ('PUT', '/v1/%s/%s/%s' % - (self.account, self.container_name, name), source, headers) - else: - hc_args = ('PUT', '/%s/%s/%s/%s/%s' % - (device, partition, self.account, self.container_name, name), - source, headers) - if self._send_request(*hc_args): - self.names.append((device, partition, name)) + with self.connection() as conn: + try: + if self.use_proxy: + client.put_object(self.url, self.token, + self.container_name, name, source, + content_length=len(source), http_conn=conn) + else: + node = {'ip': self.ip, 'port': self.port, 'device': device} + direct_client.direct_put_object(node, partition, + self.account, self.container_name, name, source, + content_length=len(source)) + except client.ClientException, e: + self.logger.debug(str(e)) + self.failures += 1 + self.names.append((device, partition, name)) self.complete += 1 diff --git a/swift/common/client.py b/swift/common/client.py index 22562ede37..06c3dab067 100644 --- a/swift/common/client.py +++ b/swift/common/client.py @@ -18,7 +18,7 @@ Cloud Files client library used internally """ import socket from cStringIO import StringIO -from httplib import HTTPConnection, HTTPException, HTTPSConnection +from httplib import HTTPException, HTTPSConnection from re import compile, DOTALL from tokenize import generate_tokens, STRING, NAME, OP from urllib import quote as _quote, unquote @@ -29,6 +29,8 @@ try: except: from time import sleep +from swift.common.bufferedhttp \ + import BufferedHTTPConnection as HTTPConnection def quote(value, safe='/'): """ diff --git a/swift/common/direct_client.py b/swift/common/direct_client.py index e17c86ce53..886ab4ca44 100644 --- a/swift/common/direct_client.py +++ b/swift/common/direct_client.py @@ -229,6 +229,60 @@ def direct_get_object(node, part, account, container, obj, conn_timeout=5, resp_headers[header.lower()] = value return resp_headers, object_body +def direct_put_object(node, part, account, container, name, contents, + content_length=None, etag=None, content_type=None, + headers=None, conn_timeout=5, response_timeout=15, + resp_chunk_size=None): + """ + Put object directly from the object server. + + :param node: node dictionary from the ring + :param part: partition the container is on + :param account: account name + :param container: container name + :param name: object name + :param contents: a string to read object data from + :param content_length: value to send as content-length header + :param etag: etag of contents + :param content_type: value to send as content-type header + :param headers: additional headers to include in the request + :param conn_timeout: timeout in seconds for establishing the connection + :param response_timeout: timeout in seconds for getting the response + :param chunk_size: if defined, chunk size of data to send. + :returns: etag from the server response + """ + # TODO: Add chunked puts + path = '/%s/%s/%s' % (account, container, name) + if headers is None: + headers = {} + if etag: + headers['ETag'] = etag.strip('"') + if content_length is not None: + headers['Content-Length'] = str(content_length) + if content_type is not None: + headers['Content-Type'] = content_type + else: + headers['Content-Type'] = 'application/octet-stream' + if not contents: + headers['Content-Length'] = '0' + headers['X-Timestamp'] = normalize_timestamp(time()) + with Timeout(conn_timeout): + conn = http_connect(node['ip'], node['port'], node['device'], part, + 'PUT', path, headers=headers) + conn.send(contents) + with Timeout(response_timeout): + resp = conn.getresponse() + resp.read() + if resp.status < 200 or resp.status >= 300: + raise ClientException( + 'Object server %s:%s direct PUT %s gave status %s' % + (node['ip'], node['port'], + repr('/%s/%s%s' % (node['device'], part, path)), + resp.status), + http_host=node['ip'], http_port=node['port'], + http_device=node['device'], http_status=resp.status, + http_reason=resp.reason) + return resp.getheader('etag').strip('"') def direct_delete_object(node, part, account, container, obj, conn_timeout=5, response_timeout=15, headers={}): From d7d0bd425377ac9c798de79f6661e6fdb20adfad Mon Sep 17 00:00:00 2001 From: Chuck Thier Date: Fri, 1 Oct 2010 19:50:34 +0000 Subject: [PATCH 52/89] added apache license to the new files --- bin/swift-bench | 14 ++++++++++++++ swift/common/bench.py | 14 ++++++++++++++ 2 files changed, 28 insertions(+) diff --git a/bin/swift-bench b/bin/swift-bench index aa1433ddc8..a8898bf069 100755 --- a/bin/swift-bench +++ b/bin/swift-bench @@ -1,4 +1,18 @@ #!/usr/bin/python +# Copyright (c) 2010 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 logging import os diff --git a/swift/common/bench.py b/swift/common/bench.py index e921ec9158..872262d938 100644 --- a/swift/common/bench.py +++ b/swift/common/bench.py @@ -179,6 +179,20 @@ class BenchGET(Bench): self.failures += 1 self.complete += 1 +# Copyright (c) 2010 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. class BenchPUT(Bench): From 5d92d43cacac7c6bdf218af8f381218f2d2a32eb Mon Sep 17 00:00:00 2001 From: Chuck Thier Date: Fri, 1 Oct 2010 19:55:08 +0000 Subject: [PATCH 53/89] Copy paste fail --- swift/common/bench.py | 29 +++++++++++++++-------------- swift/common/direct_client.py | 4 +++- 2 files changed, 18 insertions(+), 15 deletions(-) diff --git a/swift/common/bench.py b/swift/common/bench.py index 872262d938..bb1dbf3af2 100644 --- a/swift/common/bench.py +++ b/swift/common/bench.py @@ -1,3 +1,18 @@ +# Copyright (c) 2010 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 uuid import time import random @@ -179,20 +194,6 @@ class BenchGET(Bench): self.failures += 1 self.complete += 1 -# Copyright (c) 2010 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. class BenchPUT(Bench): diff --git a/swift/common/direct_client.py b/swift/common/direct_client.py index 886ab4ca44..7952a1ceaf 100644 --- a/swift/common/direct_client.py +++ b/swift/common/direct_client.py @@ -229,6 +229,7 @@ def direct_get_object(node, part, account, container, obj, conn_timeout=5, resp_headers[header.lower()] = value return resp_headers, object_body + def direct_put_object(node, part, account, container, name, contents, content_length=None, etag=None, content_type=None, headers=None, conn_timeout=5, response_timeout=15, @@ -251,7 +252,7 @@ def direct_put_object(node, part, account, container, name, contents, :param chunk_size: if defined, chunk size of data to send. :returns: etag from the server response """ - # TODO: Add chunked puts + # TODO: Add chunked puts path = '/%s/%s/%s' % (account, container, name) if headers is None: headers = {} @@ -284,6 +285,7 @@ def direct_put_object(node, part, account, container, name, contents, http_reason=resp.reason) return resp.getheader('etag').strip('"') + def direct_delete_object(node, part, account, container, obj, conn_timeout=5, response_timeout=15, headers={}): """ From 18245c1adc7a25e2cdf5fda396dadae13be99d76 Mon Sep 17 00:00:00 2001 From: Chuck Thier Date: Mon, 4 Oct 2010 15:09:53 +0000 Subject: [PATCH 54/89] Updated to print usage by default if no args are passed, and added --saio to run with saio defaults --- bin/swift-bench | 23 ++++++++++++++++++----- 1 file changed, 18 insertions(+), 5 deletions(-) diff --git a/bin/swift-bench b/bin/swift-bench index a8898bf069..3b82653f2c 100755 --- a/bin/swift-bench +++ b/bin/swift-bench @@ -26,9 +26,9 @@ from swift.common.utils import readconf, NamedLogger # The defaults should be sufficient to run swift-bench on a SAIO CONF_DEFAULTS = { - 'auth': 'http://saio:11000/v1.0', - 'user': 'test:tester', - 'key': 'testing', + 'auth': '', + 'user': '', + 'key': '', 'object_sources': '', 'put_concurrency': '10', 'get_concurrency': '10', @@ -47,9 +47,15 @@ CONF_DEFAULTS = { 'timeout': '10', } +SAIO_DEFAULTS = { + 'auth': 'http://saio:11000/v1.0', + 'user': 'test:tester', + 'key': 'testing', + } + if __name__ == '__main__': usage = "usage: %prog [OPTIONS] [CONF_FILE]" - usage += """\n\nConf file (with defaults): + usage += """\n\nConf file with SAIO defaults: [bench] auth = http://saio:11000/v1.0 @@ -62,6 +68,8 @@ if __name__ == '__main__': delete = yes """ parser = OptionParser(usage=usage) + parser.add_option('', '--saio', dest='saio', action='store_true', + default=False, help='Run benchmark with SAIO defaults') parser.add_option('-A', '--auth', dest='auth', help='URL for obtaining an auth token') parser.add_option('-U', '--user', dest='user', @@ -81,7 +89,12 @@ if __name__ == '__main__': parser.add_option('-x', '--no-delete', dest='delete', action='store_false', help='If set, will not delete the objects created') - _, args = parser.parse_args() + if len(sys.argv) == 1: + parser.print_usage() + sys.exit(1) + options, args = parser.parse_args() + if options.saio: + CONF_DEFAULTS.update(SAIO_DEFAULTS) if args: conf = args[0] if not os.path.exists(conf): From a6becf6a3f290fd56b628703df8e346204c6a80b Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Mon, 4 Oct 2010 15:48:57 -0500 Subject: [PATCH 55/89] fixed stats docs --- doc/source/overview_stats.rst | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/doc/source/overview_stats.rst b/doc/source/overview_stats.rst index b151674514..d85eb286d7 100644 --- a/doc/source/overview_stats.rst +++ b/doc/source/overview_stats.rst @@ -89,18 +89,18 @@ Running the stats system on SAIO #. Create a swift account to use for storing stats information, and note the account hash. The hash will be used in config files. -#. Install syslog-ng +#. Install syslog-ng:: sudo apt-get install syslog-ng -#. Add a destination rule to `/etc/syslog-ng/syslog-ng.conf` +#. Add a destination rule to `/etc/syslog-ng/syslog-ng.conf`:: destination df_syslog_hourly { file("/var/log/swift/access-$YEAR$MONTH$DAY$HOUR"); }; #. Edit the destination rules to standard logging in `/etc/syslog-ng/syslog-ng.conf` by adding the destination just created. This will cause syslog messages to be also put into a file, named by the - current hour, in `/var/log/swift`. + current hour, in `/var/log/swift`.:: log { source(s_all); @@ -111,7 +111,7 @@ Running the stats system on SAIO #. Restart syslog-ng -#. Create `/etc/swift/log-processor.conf` +#. Create `/etc/swift/log-processor.conf`:: [log-processor] swift_account = @@ -133,22 +133,22 @@ Running the stats system on SAIO user = #. Create a `cron` job to run once per hour to create the stats logs. In - `/etc/cron.d/swift-stats-log-creator` + `/etc/cron.d/swift-stats-log-creator`:: 0 * * * * swift-account-stats-logger /etc/swift/log-processor.conf #. Create a `cron` job to run once per hour to upload the stats logs. In - `/etc/cron.d/swift-stats-log-uploader` + `/etc/cron.d/swift-stats-log-uploader`:: 10 * * * * swift-log-uploader /etc/swift/log-processor.conf stats #. Create a `cron` job to run once per hour to upload the access logs. In - `/etc/cron.d/swift-access-log-uploader` + `/etc/cron.d/swift-access-log-uploader`:: 5 * * * * swift-log-uploader /etc/swift/log-processor.conf access #. Create a `cron` job to run once per hour to process the logs. In - `/etc/cron.d/swift-stats-processor` + `/etc/cron.d/swift-stats-processor`:: 30 * * * * swift-log-stats-collector /etc/swift/log-processor.conf From 9a915a00f7c497fa56a255b34c1a345002bb33cd Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Mon, 4 Oct 2010 16:12:43 -0500 Subject: [PATCH 56/89] added a bad lines check to the access log parser --- etc/log-processing.conf-sample | 1 + swift/common/internal_proxy.py | 4 ++-- swift/stats/access_processor.py | 9 +++++++++ 3 files changed, 12 insertions(+), 2 deletions(-) diff --git a/etc/log-processing.conf-sample b/etc/log-processing.conf-sample index 931c0525ca..11805add0b 100644 --- a/etc/log-processing.conf-sample +++ b/etc/log-processing.conf-sample @@ -25,6 +25,7 @@ class_path = swift.stats.access_processor.AccessLogProcessor # lb_private_ips = # server_name = proxy # user = swift +# warn_percent = 0.8 [log-processor-stats] # log_dir = /var/log/swift/ diff --git a/swift/common/internal_proxy.py b/swift/common/internal_proxy.py index 2df7580967..9951083ac6 100644 --- a/swift/common/internal_proxy.py +++ b/swift/common/internal_proxy.py @@ -74,14 +74,14 @@ class InternalProxy(object): :param etag: etag for object to check successful upload :returns: True if successful, False otherwise """ - log_create_pattern = '/v1/%s/%s/%s' % (account, container, object_name) + target_name = '/v1/%s/%s/%s' % (account, container, object_name) # create the container if not self.create_container(account, container): return False # upload the file to the account - req = webob.Request.blank(log_create_pattern, + req = webob.Request.blank(target_name, environ={'REQUEST_METHOD': 'PUT'}, headers={'Transfer-Encoding': 'chunked'}) if compress: diff --git a/swift/stats/access_processor.py b/swift/stats/access_processor.py index 6fcdafc59c..7ce3837723 100644 --- a/swift/stats/access_processor.py +++ b/swift/stats/access_processor.py @@ -33,6 +33,7 @@ class AccessLogProcessor(object): self.service_ips = [x.strip() for x in \ conf.get('service_ips', '').split(',')\ if x.strip()] + self.warn_percent = float(conf.get('warn_percent', '0.8')) def log_line_parser(self, raw_log): '''given a raw access log line, return a dict of the good parts''' @@ -122,9 +123,13 @@ class AccessLogProcessor(object): def process(self, obj_stream, account, container, object_name): '''generate hourly groupings of data from one access log file''' hourly_aggr_info = {} + total_lines = 0 + bad_lines = 0 for line in obj_stream: line_data = self.log_line_parser(line) + total_lines += 1 if not line_data: + bad_lines += 1 continue account = line_data['account'] container_name = line_data['container_name'] @@ -178,6 +183,10 @@ class AccessLogProcessor(object): d[key] = d.setdefault(key, 0) + 1 hourly_aggr_info[aggr_key] = d + if bad_lines > (total_lines * self.warn_percent): + name = '/'.join([account, container, object_name]) + print >>sys.stderr, 'I found a bunch of bad lines in %s '\ + '(%d bad, %d total)' % (name, bad_lines, total_lines) return hourly_aggr_info def keylist_mapping(self): From 3a841f20497269ceb0784ee562022964c6e3bac5 Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Mon, 4 Oct 2010 16:40:06 -0500 Subject: [PATCH 57/89] added tests for compressing file reader --- .../common/test_compressing_file_reader.py | 33 +++++++++++++++++++ 1 file changed, 33 insertions(+) create mode 100644 test/unit/common/test_compressing_file_reader.py diff --git a/test/unit/common/test_compressing_file_reader.py b/test/unit/common/test_compressing_file_reader.py new file mode 100644 index 0000000000..ce7c1c1b64 --- /dev/null +++ b/test/unit/common/test_compressing_file_reader.py @@ -0,0 +1,33 @@ +# Copyright (c) 2010 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. + +""" Tests for swift.common.compressing_file_reader """ + +import unittest +import cStringIO + +from swift.common.compressing_file_reader import CompressingFileReader + +class TestCompressingFileReader(unittest.TestCase): + + def test_read(self): + plain = 'obj\ndata' + s = cStringIO.StringIO(plain) + expected = '\x1f\x8b\x08\x00\x00\x00\x00\x00\x02\xff'\ + '\xcaO\xca\xe2JI,I\x04\x00\x00\x00\xff\xff' + x = CompressingFileReader(s) + self.assertEquals(x.read(), expected) + d = zlib.decompressobj(16 + zlib.MAX_WBITS) + self.asserEquals(plain, d.decompress(expected)) From 683aa3b3c23fa8cdab51df37ad9b58dd7cbf2ca5 Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Mon, 4 Oct 2010 16:41:59 -0500 Subject: [PATCH 58/89] fixed compressing file reader test --- test/unit/common/test_compressing_file_reader.py | 1 + 1 file changed, 1 insertion(+) diff --git a/test/unit/common/test_compressing_file_reader.py b/test/unit/common/test_compressing_file_reader.py index ce7c1c1b64..7d9ff21994 100644 --- a/test/unit/common/test_compressing_file_reader.py +++ b/test/unit/common/test_compressing_file_reader.py @@ -17,6 +17,7 @@ import unittest import cStringIO +import zlib from swift.common.compressing_file_reader import CompressingFileReader From 6b2106ae33e06a5322ce3e00669b6d4173a6b870 Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Mon, 4 Oct 2010 16:43:03 -0500 Subject: [PATCH 59/89] fixed compressing file reader test --- test/unit/common/test_compressing_file_reader.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/unit/common/test_compressing_file_reader.py b/test/unit/common/test_compressing_file_reader.py index 7d9ff21994..b3e340a6d5 100644 --- a/test/unit/common/test_compressing_file_reader.py +++ b/test/unit/common/test_compressing_file_reader.py @@ -31,4 +31,4 @@ class TestCompressingFileReader(unittest.TestCase): x = CompressingFileReader(s) self.assertEquals(x.read(), expected) d = zlib.decompressobj(16 + zlib.MAX_WBITS) - self.asserEquals(plain, d.decompress(expected)) + self.assertEquals(plain, d.decompress(expected)) From 0ca5286ad9ec47cbe0502d421194f47fd71ab181 Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Mon, 4 Oct 2010 16:55:08 -0500 Subject: [PATCH 60/89] improved compressing file reader test --- test/unit/common/test_compressing_file_reader.py | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/test/unit/common/test_compressing_file_reader.py b/test/unit/common/test_compressing_file_reader.py index b3e340a6d5..0da4d5dd65 100644 --- a/test/unit/common/test_compressing_file_reader.py +++ b/test/unit/common/test_compressing_file_reader.py @@ -17,7 +17,6 @@ import unittest import cStringIO -import zlib from swift.common.compressing_file_reader import CompressingFileReader @@ -29,6 +28,6 @@ class TestCompressingFileReader(unittest.TestCase): expected = '\x1f\x8b\x08\x00\x00\x00\x00\x00\x02\xff'\ '\xcaO\xca\xe2JI,I\x04\x00\x00\x00\xff\xff' x = CompressingFileReader(s) - self.assertEquals(x.read(), expected) - d = zlib.decompressobj(16 + zlib.MAX_WBITS) - self.assertEquals(plain, d.decompress(expected)) + compressed = ''.join(iter(lambda: x.read(), '')) + self.assertEquals(compressed, expected) + self.assertEquals(x.read(), '') From 42cc1352cca06bd9a1793c810cf4937762bf37df Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Mon, 4 Oct 2010 16:59:10 -0500 Subject: [PATCH 61/89] fixed compressing file reader test --- test/unit/common/test_compressing_file_reader.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/test/unit/common/test_compressing_file_reader.py b/test/unit/common/test_compressing_file_reader.py index 0da4d5dd65..5394a97a72 100644 --- a/test/unit/common/test_compressing_file_reader.py +++ b/test/unit/common/test_compressing_file_reader.py @@ -25,8 +25,9 @@ class TestCompressingFileReader(unittest.TestCase): def test_read(self): plain = 'obj\ndata' s = cStringIO.StringIO(plain) - expected = '\x1f\x8b\x08\x00\x00\x00\x00\x00\x02\xff'\ - '\xcaO\xca\xe2JI,I\x04\x00\x00\x00\xff\xff' + expected = '\x1f\x8b\x08\x00\x00\x00\x00\x00\x02\xff\xcaO\xca\xe2JI,'\ + 'I\x04\x00\x00\x00\xff\xff\x03\x00P(\xa8\x1f\x08\x00\x00'\ + '\x00' x = CompressingFileReader(s) compressed = ''.join(iter(lambda: x.read(), '')) self.assertEquals(compressed, expected) From f4d4856629aa157a364d097843d8a1da62fceac1 Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Tue, 5 Oct 2010 09:02:45 -0500 Subject: [PATCH 62/89] made try/except much less inclusive in access log processor --- swift/stats/access_processor.py | 120 ++++++++++++++++---------------- 1 file changed, 60 insertions(+), 60 deletions(-) diff --git a/swift/stats/access_processor.py b/swift/stats/access_processor.py index 7ce3837723..7039f7b658 100644 --- a/swift/stats/access_processor.py +++ b/swift/stats/access_processor.py @@ -57,67 +57,67 @@ class AccessLogProcessor(object): trans_id, headers, processing_time) = (unquote(x) for x in raw_log[16:].split(' ')) - if server != self.server_name: - raise ValueError('incorrect server name in log line') - (version, - account, - container_name, - object_name) = split_path(request, 2, 4, True) - if container_name is not None: - container_name = container_name.split('?', 1)[0] - if object_name is not None: - object_name = object_name.split('?', 1)[0] - account = account.split('?', 1)[0] - query = None - if '?' in request: - request, query = request.split('?', 1) - args = query.split('&') - # Count each query argument. This is used later to aggregate - # the number of format, prefix, etc. queries. - for q in args: - if '=' in q: - k, v = q.split('=', 1) - else: - k = q - # Certain keys will get summmed in stats reporting - # (format, path, delimiter, etc.). Save a "1" here - # to indicate that this request is 1 request for - # its respective key. - d[k] = 1 except ValueError: - pass - else: - d['client_ip'] = client_ip - d['lb_ip'] = lb_ip - d['method'] = method - d['request'] = request - if query: - d['query'] = query - d['http_version'] = http_version - d['code'] = code - d['referrer'] = referrer - d['user_agent'] = user_agent - d['auth_token'] = auth_token - d['bytes_in'] = bytes_in - d['bytes_out'] = bytes_out - d['etag'] = etag - d['trans_id'] = trans_id - d['processing_time'] = processing_time - day, month, year, hour, minute, second = timestamp.split('/') - d['day'] = day - month = ('%02s' % month_map.index(month)).replace(' ', '0') - d['month'] = month - d['year'] = year - d['hour'] = hour - d['minute'] = minute - d['second'] = second - d['tz'] = '+0000' - d['account'] = account - d['container_name'] = container_name - d['object_name'] = object_name - d['bytes_out'] = int(d['bytes_out'].replace('-', '0')) - d['bytes_in'] = int(d['bytes_in'].replace('-', '0')) - d['code'] = int(d['code']) + return {} + if server != self.server_name: + # incorrect server name in log line + return {} + (version, + account, + container_name, + object_name) = split_path(request, 2, 4, True) + if container_name is not None: + container_name = container_name.split('?', 1)[0] + if object_name is not None: + object_name = object_name.split('?', 1)[0] + account = account.split('?', 1)[0] + query = None + if '?' in request: + request, query = request.split('?', 1) + args = query.split('&') + # Count each query argument. This is used later to aggregate + # the number of format, prefix, etc. queries. + for q in args: + if '=' in q: + k, v = q.split('=', 1) + else: + k = q + # Certain keys will get summmed in stats reporting + # (format, path, delimiter, etc.). Save a "1" here + # to indicate that this request is 1 request for + # its respective key. + d[k] = 1 + d['client_ip'] = client_ip + d['lb_ip'] = lb_ip + d['method'] = method + d['request'] = request + if query: + d['query'] = query + d['http_version'] = http_version + d['code'] = code + d['referrer'] = referrer + d['user_agent'] = user_agent + d['auth_token'] = auth_token + d['bytes_in'] = bytes_in + d['bytes_out'] = bytes_out + d['etag'] = etag + d['trans_id'] = trans_id + d['processing_time'] = processing_time + day, month, year, hour, minute, second = timestamp.split('/') + d['day'] = day + month = ('%02s' % month_map.index(month)).replace(' ', '0') + d['month'] = month + d['year'] = year + d['hour'] = hour + d['minute'] = minute + d['second'] = second + d['tz'] = '+0000' + d['account'] = account + d['container_name'] = container_name + d['object_name'] = object_name + d['bytes_out'] = int(d['bytes_out'].replace('-', '0')) + d['bytes_in'] = int(d['bytes_in'].replace('-', '0')) + d['code'] = int(d['code']) return d def process(self, obj_stream, account, container, object_name): From 2ad602b68b8d5840fbb2a6039360cecf5107091f Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Tue, 5 Oct 2010 09:30:25 -0500 Subject: [PATCH 63/89] added keylist mapping tests and fixed other tests --- test/unit/stats/test_log_processor.py | 27 ++++++++++++++++++++++++--- 1 file changed, 24 insertions(+), 3 deletions(-) diff --git a/test/unit/stats/test_log_processor.py b/test/unit/stats/test_log_processor.py index f6576f6a43..2915ce91d2 100644 --- a/test/unit/stats/test_log_processor.py +++ b/test/unit/stats/test_log_processor.py @@ -49,7 +49,7 @@ class TestLogProcessor(unittest.TestCase): } def test_access_log_line_parser(self): - access_proxy_config = self.proxy_config + access_proxy_config = self.proxy_config.copy() access_proxy_config.update({ 'log-processor-access': { 'source_filename_format':'%Y%m%d%H*', @@ -87,7 +87,7 @@ class TestLogProcessor(unittest.TestCase): 'lb_ip': '4.5.6.7'}) def test_process_one_access_file(self): - access_proxy_config = self.proxy_config + access_proxy_config = self.proxy_config.copy() access_proxy_config.update({ 'log-processor-access': { 'source_filename_format':'%Y%m%d%H*', @@ -141,7 +141,7 @@ class TestLogProcessor(unittest.TestCase): self.assertEquals(result, expected) def test_get_stat_totals(self): - stats_proxy_config = self.proxy_config + stats_proxy_config = self.proxy_config.copy() stats_proxy_config.update({ 'log-processor-stats': { 'class_path': @@ -159,3 +159,24 @@ class TestLogProcessor(unittest.TestCase): 'container_count': 1, 'bytes_used': 3}} self.assertEquals(result, expected) + + def test_generate_keylist_mapping(self): + p = log_processor.LogProcessor(self.proxy_config, DumbLogger()) + result = p.generate_keylist_mapping() + expected = {} + print p.plugins + self.assertEquals(result, expected) + + def test_generate_keylist_mapping_with_plugins(self): + class Plugin1(object): + def keylist_mapping(self): + return {'a': 'b', 'c': 'd'} + class Plugin2(object): + def keylist_mapping(self): + return {'a': '1', 'e': '2'} + p = log_processor.LogProcessor(self.proxy_config, DumbLogger()) + p.plugins['plugin1'] = {'instance': Plugin1()} + p.plugins['plugin2'] = {'instance': Plugin2()} + result = p.generate_keylist_mapping() + expected = {'a': set(['b', '1']), 'c': 'd', 'e': '2'} + self.assertEquals(result, expected) From 7ec88950eb8860322657c98dc8f6e78f1c1a0000 Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Wed, 6 Oct 2010 10:13:51 -0500 Subject: [PATCH 64/89] updated stats system tests --- swift/stats/stats_processor.py | 30 ++++++++++----------- test/unit/stats/test_log_processor.py | 38 ++++++++++++++++++++++++--- 2 files changed, 49 insertions(+), 19 deletions(-) diff --git a/swift/stats/stats_processor.py b/swift/stats/stats_processor.py index 30500dfe95..cf3c6e6446 100644 --- a/swift/stats/stats_processor.py +++ b/swift/stats/stats_processor.py @@ -32,23 +32,23 @@ class StatsLogProcessor(object): container_count, object_count, bytes_used) = line.split(',') - account = account.strip('"') - container_count = int(container_count.strip('"')) - object_count = int(object_count.strip('"')) - bytes_used = int(bytes_used.strip('"')) - aggr_key = (account, year, month, day, hour) - d = account_totals.get(aggr_key, {}) - d['replica_count'] = d.setdefault('replica_count', 0) + 1 - d['container_count'] = d.setdefault('container_count', 0) + \ - container_count - d['object_count'] = d.setdefault('object_count', 0) + \ - object_count - d['bytes_used'] = d.setdefault('bytes_used', 0) + \ - bytes_used - account_totals[aggr_key] = d except (IndexError, ValueError): # bad line data - pass + continue + account = account.strip('"') + container_count = int(container_count.strip('"')) + object_count = int(object_count.strip('"')) + bytes_used = int(bytes_used.strip('"')) + aggr_key = (account, year, month, day, hour) + d = account_totals.get(aggr_key, {}) + d['replica_count'] = d.setdefault('replica_count', 0) + 1 + d['container_count'] = d.setdefault('container_count', 0) + \ + container_count + d['object_count'] = d.setdefault('object_count', 0) + \ + object_count + d['bytes_used'] = d.setdefault('bytes_used', 0) + \ + bytes_used + account_totals[aggr_key] = d return account_totals def keylist_mapping(self): diff --git a/test/unit/stats/test_log_processor.py b/test/unit/stats/test_log_processor.py index 2915ce91d2..fc4176f3f3 100644 --- a/test/unit/stats/test_log_processor.py +++ b/test/unit/stats/test_log_processor.py @@ -167,16 +167,46 @@ class TestLogProcessor(unittest.TestCase): print p.plugins self.assertEquals(result, expected) - def test_generate_keylist_mapping_with_plugins(self): + def test_generate_keylist_mapping_with_dummy_plugins(self): class Plugin1(object): def keylist_mapping(self): - return {'a': 'b', 'c': 'd'} + return {'a': 'b', 'c': 'd', 'e': ['f', 'g']} class Plugin2(object): def keylist_mapping(self): - return {'a': '1', 'e': '2'} + return {'a': '1', 'e': '2', 'h': '3'} p = log_processor.LogProcessor(self.proxy_config, DumbLogger()) p.plugins['plugin1'] = {'instance': Plugin1()} p.plugins['plugin2'] = {'instance': Plugin2()} result = p.generate_keylist_mapping() - expected = {'a': set(['b', '1']), 'c': 'd', 'e': '2'} + expected = {'a': set(['b', '1']), 'c': 'd', 'e': set(['2', 'f', 'g']), + 'h': '3'} self.assertEquals(result, expected) + + def test_access_keylist_mapping_format(self): + proxy_config = self.proxy_config.copy() + proxy_config.update({ + 'log-processor-access': { + 'source_filename_format':'%Y%m%d%H*', + 'class_path': + 'swift.stats.access_processor.AccessLogProcessor' + }}) + p = log_processor.LogProcessor(proxy_config, DumbLogger()) + mapping = p.generate_keylist_mapping() + for k, v in mapping.items(): + # these only work for Py2.7+ + #self.assertIsInstance(k, str) + self.assertTrue(isinstance(k, str), type(k)) + + def test_stats_keylist_mapping_format(self): + proxy_config = self.proxy_config.copy() + proxy_config.update({ + 'log-processor-stats': { + 'class_path': + 'swift.stats.stats_processor.StatsLogProcessor' + }}) + p = log_processor.LogProcessor(proxy_config, DumbLogger()) + mapping = p.generate_keylist_mapping() + for k, v in mapping.items(): + # these only work for Py2.7+ + #self.assertIsInstance(k, str) + self.assertTrue(isinstance(k, str), type(k)) From 456dea9295fb391af7dde594f938f04a9b608461 Mon Sep 17 00:00:00 2001 From: Clay Gerrard Date: Wed, 6 Oct 2010 16:10:04 -0500 Subject: [PATCH 65/89] confirmed pending patch with webob 1.0; resolved only known issue with webob trunk, some pep8 cleanup --- test/unit/common/middleware/test_auth.py | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/test/unit/common/middleware/test_auth.py b/test/unit/common/middleware/test_auth.py index a7ec9199a2..800ecb4cb7 100644 --- a/test/unit/common/middleware/test_auth.py +++ b/test/unit/common/middleware/test_auth.py @@ -67,25 +67,33 @@ def mock_http_connect(response, headers=None, with_exc=False): self.headers = headers if self.headers is None: self.headers = {} + def getresponse(self): if self.with_exc: raise Exception('test') return self + def getheader(self, header): return self.headers[header] + def read(self, amt=None): return '' + def close(self): return + return lambda *args, **kwargs: FakeConn(response, headers, with_exc) class Logger(object): + def __init__(self): self.error_value = None self.exception_value = None + def error(self, msg, *args, **kwargs): self.error_value = (msg, args, kwargs) + def exception(self, msg, *args, **kwargs): _, exc, _ = sys.exc_info() self.exception_value = (msg, @@ -99,7 +107,7 @@ class FakeApp(object): def __call__(self, env, start_response): self.i_was_called = True - req = Request(env) + req = Request.blank('', environ=env) if 'swift.authorize' in env: resp = env['swift.authorize'](req) if resp: @@ -110,6 +118,7 @@ class FakeApp(object): def start_response(*args): pass + class TestAuth(unittest.TestCase): def setUp(self): @@ -418,6 +427,5 @@ class TestAuth(unittest.TestCase): self.assert_(resp.startswith('403'), resp) - if __name__ == '__main__': unittest.main() From ccf47f825ff5222cd93fb247b8c9c8edafaa9fe9 Mon Sep 17 00:00:00 2001 From: gholt Date: Thu, 7 Oct 2010 08:08:42 -0700 Subject: [PATCH 66/89] Adding test stubs for missing test areas --- test/unit/common/test_internal_proxy.py | 29 ++++++++++++++++++++++++ test/unit/stats/test_access_processor.py | 29 ++++++++++++++++++++++++ test/unit/stats/test_account_stats.py | 29 ++++++++++++++++++++++++ test/unit/stats/test_log_uploader.py | 29 ++++++++++++++++++++++++ test/unit/stats/test_stats_processor.py | 29 ++++++++++++++++++++++++ 5 files changed, 145 insertions(+) create mode 100644 test/unit/common/test_internal_proxy.py create mode 100644 test/unit/stats/test_access_processor.py create mode 100644 test/unit/stats/test_account_stats.py create mode 100644 test/unit/stats/test_log_uploader.py create mode 100644 test/unit/stats/test_stats_processor.py diff --git a/test/unit/common/test_internal_proxy.py b/test/unit/common/test_internal_proxy.py new file mode 100644 index 0000000000..248bf1cf23 --- /dev/null +++ b/test/unit/common/test_internal_proxy.py @@ -0,0 +1,29 @@ +# Copyright (c) 2010 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. + +# TODO: Tests + +import unittest +from swift.common import internal_proxy + + +class TestInternalProxy(unittest.TestCase): + + def test_placeholder(self): + pass + + +if __name__ == '__main__': + unittest.main() diff --git a/test/unit/stats/test_access_processor.py b/test/unit/stats/test_access_processor.py new file mode 100644 index 0000000000..47013ca8ae --- /dev/null +++ b/test/unit/stats/test_access_processor.py @@ -0,0 +1,29 @@ +# Copyright (c) 2010 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. + +# TODO: Tests + +import unittest +from swift.stats import access_processor + + +class TestAccessProcessor(unittest.TestCase): + + def test_placeholder(self): + pass + + +if __name__ == '__main__': + unittest.main() diff --git a/test/unit/stats/test_account_stats.py b/test/unit/stats/test_account_stats.py new file mode 100644 index 0000000000..e318739dda --- /dev/null +++ b/test/unit/stats/test_account_stats.py @@ -0,0 +1,29 @@ +# Copyright (c) 2010 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. + +# TODO: Tests + +import unittest +from swift.stats import account_stats + + +class TestAccountStats(unittest.TestCase): + + def test_placeholder(self): + pass + + +if __name__ == '__main__': + unittest.main() diff --git a/test/unit/stats/test_log_uploader.py b/test/unit/stats/test_log_uploader.py new file mode 100644 index 0000000000..8e889ad918 --- /dev/null +++ b/test/unit/stats/test_log_uploader.py @@ -0,0 +1,29 @@ +# Copyright (c) 2010 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. + +# TODO: Tests + +import unittest +from swift.stats import log_uploader + + +class TestLogUploader(unittest.TestCase): + + def test_placeholder(self): + pass + + +if __name__ == '__main__': + unittest.main() diff --git a/test/unit/stats/test_stats_processor.py b/test/unit/stats/test_stats_processor.py new file mode 100644 index 0000000000..4720d1f035 --- /dev/null +++ b/test/unit/stats/test_stats_processor.py @@ -0,0 +1,29 @@ +# Copyright (c) 2010 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. + +# TODO: Tests + +import unittest +from swift.stats import stats_processor + + +class TestStatsProcessor(unittest.TestCase): + + def test_placeholder(self): + pass + + +if __name__ == '__main__': + unittest.main() From e6e354c4839619f38c4db77fca844921e61e44eb Mon Sep 17 00:00:00 2001 From: gholt Date: Thu, 7 Oct 2010 08:23:17 -0700 Subject: [PATCH 67/89] Added some missing test stubs so we can better see coverage (and get a little syntax-level "testing"). --- test/unit/account/test_replicator.py | 32 ++++++++++++++++++++++++++ test/unit/common/test_bench.py | 29 +++++++++++++++++++++++ test/unit/common/test_daemon.py | 29 +++++++++++++++++++++++ test/unit/common/test_direct_client.py | 5 +++- test/unit/common/test_exceptions.py | 3 ++- test/unit/container/test_replicator.py | 32 ++++++++++++++++++++++++++ 6 files changed, 128 insertions(+), 2 deletions(-) create mode 100644 test/unit/account/test_replicator.py create mode 100644 test/unit/common/test_bench.py create mode 100644 test/unit/common/test_daemon.py create mode 100644 test/unit/container/test_replicator.py diff --git a/test/unit/account/test_replicator.py b/test/unit/account/test_replicator.py new file mode 100644 index 0000000000..6b3d045eaa --- /dev/null +++ b/test/unit/account/test_replicator.py @@ -0,0 +1,32 @@ +# Copyright (c) 2010 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 swift.account import replicator + + +class TestReplicator(unittest.TestCase): + """ + swift.account.replicator is currently just a subclass with some class + variables overridden, but at least this test stub will ensure proper Python + syntax. + """ + + def test_placeholder(self): + pass + + +if __name__ == '__main__': + unittest.main() diff --git a/test/unit/common/test_bench.py b/test/unit/common/test_bench.py new file mode 100644 index 0000000000..7b75aba79e --- /dev/null +++ b/test/unit/common/test_bench.py @@ -0,0 +1,29 @@ +# Copyright (c) 2010 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. + +# TODO: Tests + +import unittest +from swift.common import bench + + +class TestBench(unittest.TestCase): + + def test_placeholder(self): + pass + + +if __name__ == '__main__': + unittest.main() diff --git a/test/unit/common/test_daemon.py b/test/unit/common/test_daemon.py new file mode 100644 index 0000000000..e2db43caa6 --- /dev/null +++ b/test/unit/common/test_daemon.py @@ -0,0 +1,29 @@ +# Copyright (c) 2010 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. + +# TODO: Tests + +import unittest +from swift.common import daemon + + +class TestDaemon(unittest.TestCase): + + def test_placeholder(self): + pass + + +if __name__ == '__main__': + unittest.main() diff --git a/test/unit/common/test_direct_client.py b/test/unit/common/test_direct_client.py index a925c118bb..35ed07ffd7 100644 --- a/test/unit/common/test_direct_client.py +++ b/test/unit/common/test_direct_client.py @@ -17,7 +17,10 @@ import unittest -class TestAuditor(unittest.TestCase): +from swift.common import direct_client + + +class TestDirectClient(unittest.TestCase): def test_placeholder(self): pass diff --git a/test/unit/common/test_exceptions.py b/test/unit/common/test_exceptions.py index bfb251b139..35a5801e77 100644 --- a/test/unit/common/test_exceptions.py +++ b/test/unit/common/test_exceptions.py @@ -18,7 +18,8 @@ import unittest from swift.common import exceptions -class TestAuditor(unittest.TestCase): + +class TestExceptions(unittest.TestCase): def test_placeholder(self): pass diff --git a/test/unit/container/test_replicator.py b/test/unit/container/test_replicator.py new file mode 100644 index 0000000000..8f7a032893 --- /dev/null +++ b/test/unit/container/test_replicator.py @@ -0,0 +1,32 @@ +# Copyright (c) 2010 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 swift.container import replicator + + +class TestReplicator(unittest.TestCase): + """ + swift.container.replicator is currently just a subclass with some class + variables overridden, but at least this test stub will ensure proper Python + syntax. + """ + + def test_placeholder(self): + pass + + +if __name__ == '__main__': + unittest.main() From dfbb80693a5c82d199dd1ed46421fe1c4f85d4a9 Mon Sep 17 00:00:00 2001 From: Clay Gerrard Date: Thu, 7 Oct 2010 11:28:59 -0500 Subject: [PATCH 68/89] added -v option to st, ties into -q option, for higher levels of verbosity - currently on the stat command does anything with options.verbose > 1 - it will give you the storage url and token in case you need to drop into curl --- bin/st | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/bin/st b/bin/st index 51a7637881..1a4bf34312 100755 --- a/bin/st +++ b/bin/st @@ -1132,6 +1132,11 @@ def st_stat(options, args): if not args: try: headers = conn.head_account() + if options.verbose > 1: + options.print_queue.put(''' +StorageURL: %s +Auth Token: %s +'''.strip('\n') % (conn.url, conn.token)) container_count = int(headers.get('x-account-container-count', 0)) object_count = int(headers.get('x-account-object-count', 0)) bytes_used = int(headers.get('x-account-bytes-used', 0)) @@ -1397,8 +1402,10 @@ Example: '''.strip('\n') % globals()) parser.add_option('-s', '--snet', action='store_true', dest='snet', default=False, help='Use SERVICENET internal network') - parser.add_option('-q', '--quiet', action='store_false', dest='verbose', - default=True, help='Suppress status output') + parser.add_option('-v', '--verbose', action='count', dest='verbose', + default=1, help='Print more info') + parser.add_option('-q', '--quiet', action='store_const', dest='verbose', + const=0, default=1, help='Suppress status output') parser.add_option('-a', '--all', action='store_true', dest='yes_all', default=False, help='Indicate that you really want the ' 'whole account for commands that require --all in such ' From 09cbe002edfc0a46c7a2a8c69cd9bb6b25b6a496 Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Thu, 7 Oct 2010 15:50:56 -0500 Subject: [PATCH 69/89] updated SAIO instructions for the stats system --- doc/source/overview_stats.rst | 37 +++++++++++++++++++++++------------ 1 file changed, 25 insertions(+), 12 deletions(-) diff --git a/doc/source/overview_stats.rst b/doc/source/overview_stats.rst index d85eb286d7..a7da82b7c8 100644 --- a/doc/source/overview_stats.rst +++ b/doc/source/overview_stats.rst @@ -93,20 +93,31 @@ Running the stats system on SAIO sudo apt-get install syslog-ng -#. Add a destination rule to `/etc/syslog-ng/syslog-ng.conf`:: +#. Add the following to the end of `/etc/syslog-ng/syslog-ng.conf`:: - destination df_syslog_hourly { file("/var/log/swift/access-$YEAR$MONTH$DAY$HOUR"); }; + # Added for swift logging + destination df_local1 { file("/var/log/swift/proxy.log"); }; + destination df_local1_err { file("/var/log/swift/proxy.error"); }; + destination df_local1_hourly { file("/var/log/swift/hourly/$YEAR$MONTH$DAY$HOUR"); }; + filter f_local1 { facility(local1) and level(info); }; -#. Edit the destination rules to standard logging in - `/etc/syslog-ng/syslog-ng.conf` by adding the destination just created. - This will cause syslog messages to be also put into a file, named by the - current hour, in `/var/log/swift`.:: + filter f_local1_err { facility(local1) and not level(info); }; - log { - source(s_all); - filter(f_syslog); - destination(df_syslog); - destination(df_syslog_hourly); + # local1.info -/var/log/swift/proxy.log + # write to local file and to remove log server + log { + source(s_all); + filter(f_local1); + destination(df_local1); + destination(df_local1_hourly); + }; + + # local1.error -/var/log/swift/proxy.error + # write to local file and to remove log server + log { + source(s_all); + filter(f_local1_err); + destination(df_local1_err); }; #. Restart syslog-ng @@ -120,6 +131,7 @@ Running the stats system on SAIO [log-processor-access] swift_account = container_name = log_data + log_dir = /var/log/swift/hourly/ source_filename_format = access-%Y%m%d%H class_path = swift.stats.access_processor.AccessLogProcessor user = @@ -127,7 +139,8 @@ Running the stats system on SAIO [log-processor-stats] swift_account = container_name = account_stats - source_filename_format = stats-%Y%m%d%H_* + log_dir = /var/log/swift/stats/ + source_filename_format = %Y%m%d%H_* class_path = swift.stats.stats_processor.StatsLogProcessor account_server_conf = /etc/swift/account-server/1.conf user = From 4349bfa3afabe511cc088dee00d94b2a57262d0c Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Thu, 7 Oct 2010 16:12:55 -0500 Subject: [PATCH 70/89] fixed stats system saio docs --- doc/source/overview_stats.rst | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/doc/source/overview_stats.rst b/doc/source/overview_stats.rst index a7da82b7c8..99c543214c 100644 --- a/doc/source/overview_stats.rst +++ b/doc/source/overview_stats.rst @@ -132,7 +132,7 @@ Running the stats system on SAIO swift_account = container_name = log_data log_dir = /var/log/swift/hourly/ - source_filename_format = access-%Y%m%d%H + source_filename_format = %Y%m%d%H class_path = swift.stats.access_processor.AccessLogProcessor user = @@ -145,6 +145,10 @@ Running the stats system on SAIO account_server_conf = /etc/swift/account-server/1.conf user = +#. Add the following to the end of `/etc/swift/proxy-server.conf`:: + + log_facility = LOG_LOCAL1 + #. Create a `cron` job to run once per hour to create the stats logs. In `/etc/cron.d/swift-stats-log-creator`:: From 8e581a66b4555968bff82d4e0f7315fb8e89e990 Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Thu, 7 Oct 2010 16:16:55 -0500 Subject: [PATCH 71/89] fixed stats system saio docs --- doc/source/overview_stats.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/doc/source/overview_stats.rst b/doc/source/overview_stats.rst index 99c543214c..b75b34b523 100644 --- a/doc/source/overview_stats.rst +++ b/doc/source/overview_stats.rst @@ -145,7 +145,7 @@ Running the stats system on SAIO account_server_conf = /etc/swift/account-server/1.conf user = -#. Add the following to the end of `/etc/swift/proxy-server.conf`:: +#. Add the following under [app:proxy-server] in `/etc/swift/proxy-server.conf`:: log_facility = LOG_LOCAL1 From 84a163d736924d6eb3aae45fe68e76bf314e0873 Mon Sep 17 00:00:00 2001 From: Chuck Thier Date: Thu, 7 Oct 2010 21:36:39 +0000 Subject: [PATCH 72/89] Somehow I missed the commit to print the whole help, not just usage --- bin/swift-bench | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bin/swift-bench b/bin/swift-bench index 3b82653f2c..e058882acb 100755 --- a/bin/swift-bench +++ b/bin/swift-bench @@ -90,7 +90,7 @@ if __name__ == '__main__': help='If set, will not delete the objects created') if len(sys.argv) == 1: - parser.print_usage() + parser.print_help() sys.exit(1) options, args = parser.parse_args() if options.saio: From 61226339e6c3bfb13232ffdbb4adb88c76b470cf Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Thu, 7 Oct 2010 20:36:51 -0500 Subject: [PATCH 73/89] added openstack copyright/license to test_log_processor.py --- test/unit/stats/test_log_processor.py | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/test/unit/stats/test_log_processor.py b/test/unit/stats/test_log_processor.py index fc4176f3f3..4ff73eccf3 100644 --- a/test/unit/stats/test_log_processor.py +++ b/test/unit/stats/test_log_processor.py @@ -1,3 +1,18 @@ +# Copyright (c) 2010 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 swift.stats import log_processor From cf5542beb93f551b8ef134f8645b32fb9f1d8854 Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Thu, 7 Oct 2010 20:56:31 -0500 Subject: [PATCH 74/89] improved logging in log processors --- swift/stats/access_processor.py | 10 +++++++--- swift/stats/log_processor.py | 10 ++++++++++ swift/stats/stats_processor.py | 4 +++- 3 files changed, 20 insertions(+), 4 deletions(-) diff --git a/swift/stats/access_processor.py b/swift/stats/access_processor.py index 7039f7b658..5d8766b9df 100644 --- a/swift/stats/access_processor.py +++ b/swift/stats/access_processor.py @@ -17,7 +17,7 @@ import collections from urllib import unquote import copy -from swift.common.utils import split_path +from swift.common.utils import split_path, get_logger month_map = '_ Jan Feb Mar Apr May Jun Jul Aug Sep Oct Nov Dec'.split() @@ -34,6 +34,7 @@ class AccessLogProcessor(object): conf.get('service_ips', '').split(',')\ if x.strip()] self.warn_percent = float(conf.get('warn_percent', '0.8')) + self.logger = get_logger(conf) def log_line_parser(self, raw_log): '''given a raw access log line, return a dict of the good parts''' @@ -58,9 +59,12 @@ class AccessLogProcessor(object): headers, processing_time) = (unquote(x) for x in raw_log[16:].split(' ')) except ValueError: + self.logger.debug('Bad line data: %s' % repr(raw_log)) return {} if server != self.server_name: # incorrect server name in log line + self.logger.debug('Bad server name: found "%s" expected "%s"' \ + % (server, self.server_name)) return {} (version, account, @@ -185,8 +189,8 @@ class AccessLogProcessor(object): hourly_aggr_info[aggr_key] = d if bad_lines > (total_lines * self.warn_percent): name = '/'.join([account, container, object_name]) - print >>sys.stderr, 'I found a bunch of bad lines in %s '\ - '(%d bad, %d total)' % (name, bad_lines, total_lines) + self.logger.warning('I found a bunch of bad lines in %s '\ + '(%d bad, %d total)' % (name, bad_lines, total_lines)) return hourly_aggr_info def keylist_mapping(self): diff --git a/swift/stats/log_processor.py b/swift/stats/log_processor.py index cad3461d08..d1030ae36f 100644 --- a/swift/stats/log_processor.py +++ b/swift/stats/log_processor.py @@ -58,6 +58,7 @@ class LogProcessor(object): module = __import__(import_target, fromlist=[import_target]) klass = getattr(module, class_name) self.plugins[plugin_name]['instance'] = klass(plugin_conf) + self.logger.debug('Loaded plugin "%s"' % plugin_name) @property def internal_proxy(self): @@ -74,6 +75,11 @@ class LogProcessor(object): return self._internal_proxy def process_one_file(self, plugin_name, account, container, object_name): + self.logger.info('Processing %s/%s/%s with plugin "%s"' % (account, + container, + object_name, + plugin_name) + ) # get an iter of the object data compressed = object_name.endswith('.gz') stream = self.get_object_data(account, container, object_name, @@ -185,6 +191,10 @@ class LogProcessor(object): try: chunk = d.decompress(chunk) except zlib.error: + self.logger.debug('Bad compressed data for %s/%s/%s' % + (swift_account, + container_name, + object_name)) raise BadFileDownload() # bad compressed data parts = chunk.split('\n') parts[0] = last_part + parts[0] diff --git a/swift/stats/stats_processor.py b/swift/stats/stats_processor.py index cf3c6e6446..3fd352bab7 100644 --- a/swift/stats/stats_processor.py +++ b/swift/stats/stats_processor.py @@ -13,12 +13,13 @@ # See the License for the specific language governing permissions and # limitations under the License. +from swift.common.utils import get_logger class StatsLogProcessor(object): """Transform account storage stat logs""" def __init__(self, conf): - pass + self.logger = get_logger(conf) def process(self, obj_stream, account, container, object_name): '''generate hourly groupings of data from one stats log file''' @@ -34,6 +35,7 @@ class StatsLogProcessor(object): bytes_used) = line.split(',') except (IndexError, ValueError): # bad line data + self.logger.debug('Bad line data: %s' % repr(line)) continue account = account.strip('"') container_count = int(container_count.strip('"')) From 690262b3628ea2a0dfce6a3c52be27fe731843d6 Mon Sep 17 00:00:00 2001 From: Michael Barton Date: Fri, 8 Oct 2010 15:00:30 +0000 Subject: [PATCH 75/89] Make wsgi listen queue configurable, with a bigger default --- swift/common/wsgi.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/swift/common/wsgi.py b/swift/common/wsgi.py index 306f0e2980..5628517264 100644 --- a/swift/common/wsgi.py +++ b/swift/common/wsgi.py @@ -95,7 +95,7 @@ def run_wsgi(conf_file, app_section, *args, **kwargs): # pragma: no cover retry_until = time.time() + 30 while not sock and time.time() < retry_until: try: - sock = listen(bind_addr) + sock = listen(bind_addr, backlog=int(conf.get('backlog', 4096))) if 'cert_file' in conf: sock = ssl.wrap_socket(sock, certfile=conf['cert_file'], keyfile=conf['key_file']) From f3397977d3d2fd1a7919e342e9b7396dc86deec2 Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Fri, 8 Oct 2010 13:31:17 -0500 Subject: [PATCH 76/89] fixed missing working directory bug in account stats --- swift/stats/account_stats.py | 1 + 1 file changed, 1 insertion(+) diff --git a/swift/stats/account_stats.py b/swift/stats/account_stats.py index 1f5279ac70..feae842414 100644 --- a/swift/stats/account_stats.py +++ b/swift/stats/account_stats.py @@ -60,6 +60,7 @@ class AccountStat(Daemon): src_filename = time.strftime(self.filename_format) working_dir = os.path.join(self.target_dir, '.stats_tmp') shutil.rmtree(working_dir, ignore_errors=True) + mkdirs(working_dir) tmp_filename = os.path.join(working_dir, src_filename) hasher = hashlib.md5() with open(tmp_filename, 'wb') as statfile: From 0c5e34e9a19c7f4b6e2504ad61b12f028790cebb Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Fri, 8 Oct 2010 13:40:51 -0500 Subject: [PATCH 77/89] fixed readconf parameter that was broken with a previous merge --- swift/common/utils.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/swift/common/utils.py b/swift/common/utils.py index de60e38f14..f8feb73968 100644 --- a/swift/common/utils.py +++ b/swift/common/utils.py @@ -553,7 +553,7 @@ def cache_from_env(env): return item_from_env(env, 'swift.cache') -def readconf(conf, section_name, log_name=None, defaults=None): +def readconf(conf, section_name=None, log_name=None, defaults=None): """ Read config file and return config items as a dict From 8f0ae7a0ebc6ddba5c5cefb3d0ecb3a25e552c1f Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Fri, 8 Oct 2010 13:50:21 -0500 Subject: [PATCH 78/89] updated setup.py and saio docs for syats system --- doc/source/overview_stats.rst | 6 ++++++ setup.py | 3 +++ 2 files changed, 9 insertions(+) diff --git a/doc/source/overview_stats.rst b/doc/source/overview_stats.rst index b75b34b523..985635ba08 100644 --- a/doc/source/overview_stats.rst +++ b/doc/source/overview_stats.rst @@ -122,6 +122,12 @@ Running the stats system on SAIO #. Restart syslog-ng +#. Create the log directories:: + + mkdir /var/log/swift/hourly + mkdir /var/log/swift/stats + chown -R : /var/log/swift + #. Create `/etc/swift/log-processor.conf`:: [log-processor] diff --git a/setup.py b/setup.py index 936bf92f8e..b50fe7d4b1 100644 --- a/setup.py +++ b/setup.py @@ -76,6 +76,9 @@ setup( 'bin/swift-ring-builder', 'bin/swift-stats-populate', 'bin/swift-stats-report', 'bin/swift-bench', + 'bin/swift-log-uploader', + 'bin/swift-log-stats-collector', + 'bin/swift-account-stats-logger', ], entry_points={ 'paste.app_factory': [ From 665b7f97fbc1ef3d8b69a3261603c1206ff6497f Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Fri, 8 Oct 2010 14:14:08 -0500 Subject: [PATCH 79/89] added readconf unit test --- test/unit/common/test_utils.py | 27 +++++++++++++++++++++++++++ 1 file changed, 27 insertions(+) diff --git a/test/unit/common/test_utils.py b/test/unit/common/test_utils.py index 344cee4ec8..b8d2523f04 100644 --- a/test/unit/common/test_utils.py +++ b/test/unit/common/test_utils.py @@ -247,5 +247,32 @@ class TestUtils(unittest.TestCase): self.assert_(callable( utils.load_libc_function('some_not_real_function'))) + def test_readconf(self): + conf = ''' + [section1] + foo = bar + + [section2] + log_name = yarr + ''' + result = utils.readconf(StringIO(conf)) + expected = {'section1': {'foo': 'bar'}, 'section2': {'log_name': 'yarr'}} + self.assertEquals(result, expected) + result = utils.readconf(StringIO(conf), 'section1') + expected = {'foo': 'bar'} + self.assertEquals(result, expected) + result = utils.readconf(StringIO(conf), 'section2').get('log_name') + expected = 'yarr' + self.assertEquals(result, expected) + result = utils.readconf(StringIO(conf), 'section1').get('log_name') + expected = 'section1' + self.assertEquals(result, expected) + result = utils.readconf(StringIO(conf), 'section1', log_name='foo').get('log_name') + expected = 'foo' + self.assertEquals(result, expected) + result = utils.readconf(StringIO(conf), 'section1', defaults={'bar': 'baz'}) + expected = {'foo': 'bar', 'bar': 'baz'} + self.assertEquals(result, expected) + if __name__ == '__main__': unittest.main() From adfb92443f4a6d9dc96551974094adfb3bfe2cf0 Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Fri, 8 Oct 2010 14:23:20 -0500 Subject: [PATCH 80/89] updated stats saio docs to create logs with the appropriate permissions --- doc/source/overview_stats.rst | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/doc/source/overview_stats.rst b/doc/source/overview_stats.rst index 985635ba08..6364de4611 100644 --- a/doc/source/overview_stats.rst +++ b/doc/source/overview_stats.rst @@ -96,9 +96,9 @@ Running the stats system on SAIO #. Add the following to the end of `/etc/syslog-ng/syslog-ng.conf`:: # Added for swift logging - destination df_local1 { file("/var/log/swift/proxy.log"); }; - destination df_local1_err { file("/var/log/swift/proxy.error"); }; - destination df_local1_hourly { file("/var/log/swift/hourly/$YEAR$MONTH$DAY$HOUR"); }; + destination df_local1 { file("/var/log/swift/proxy.log" owner() group()); }; + destination df_local1_err { file("/var/log/swift/proxy.error" owner() group()); }; + destination df_local1_hourly { file("/var/log/swift/hourly/$YEAR$MONTH$DAY$HOUR" owner() group()); }; filter f_local1 { facility(local1) and level(info); }; filter f_local1_err { facility(local1) and not level(info); }; From f6e6fd7599aa22f0d764f9e317d6d9d70e0ebdbe Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Fri, 8 Oct 2010 14:25:45 -0500 Subject: [PATCH 81/89] fixed bug in log processor internal proxy lazy load code --- swift/stats/log_processor.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/swift/stats/log_processor.py b/swift/stats/log_processor.py index d1030ae36f..a304a878e1 100644 --- a/swift/stats/log_processor.py +++ b/swift/stats/log_processor.py @@ -40,6 +40,8 @@ class LogProcessor(object): def __init__(self, conf, logger): stats_conf = conf.get('log-processor', {}) + + self._internal_proxy = None if isinstance(logger, tuple): self.logger = get_logger(*logger) From 4111a16189e4ddc9b0a9085d4bda0d4a403cc960 Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Fri, 8 Oct 2010 14:29:24 -0500 Subject: [PATCH 82/89] updated readconf test --- test/unit/common/test_utils.py | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/test/unit/common/test_utils.py b/test/unit/common/test_utils.py index b8d2523f04..e876690b1f 100644 --- a/test/unit/common/test_utils.py +++ b/test/unit/common/test_utils.py @@ -248,12 +248,11 @@ class TestUtils(unittest.TestCase): utils.load_libc_function('some_not_real_function'))) def test_readconf(self): - conf = ''' - [section1] - foo = bar - - [section2] - log_name = yarr + conf = '''[section1] +foo = bar + +[section2] +log_name = yarr ''' result = utils.readconf(StringIO(conf)) expected = {'section1': {'foo': 'bar'}, 'section2': {'log_name': 'yarr'}} From 8ee2cbaeee9eeff1b07588880b1ad3cc59714ab3 Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Fri, 8 Oct 2010 14:47:40 -0500 Subject: [PATCH 83/89] updated readconf test --- test/unit/common/test_utils.py | 19 +++++++++++-------- 1 file changed, 11 insertions(+), 8 deletions(-) diff --git a/test/unit/common/test_utils.py b/test/unit/common/test_utils.py index e876690b1f..11f0fa5bcf 100644 --- a/test/unit/common/test_utils.py +++ b/test/unit/common/test_utils.py @@ -252,26 +252,29 @@ class TestUtils(unittest.TestCase): foo = bar [section2] -log_name = yarr - ''' - result = utils.readconf(StringIO(conf)) +log_name = yarr''' + f = open('/tmp/test', 'wb') + f.write(conf) + f.close() + result = utils.readconf('/tmp/test') expected = {'section1': {'foo': 'bar'}, 'section2': {'log_name': 'yarr'}} self.assertEquals(result, expected) - result = utils.readconf(StringIO(conf), 'section1') + result = utils.readconf('/tmp/test', 'section1') expected = {'foo': 'bar'} self.assertEquals(result, expected) - result = utils.readconf(StringIO(conf), 'section2').get('log_name') + result = utils.readconf('/tmp/test', 'section2').get('log_name') expected = 'yarr' self.assertEquals(result, expected) - result = utils.readconf(StringIO(conf), 'section1').get('log_name') + result = utils.readconf('/tmp/test', 'section1').get('log_name') expected = 'section1' self.assertEquals(result, expected) - result = utils.readconf(StringIO(conf), 'section1', log_name='foo').get('log_name') + result = utils.readconf('/tmp/test', 'section1', log_name='foo').get('log_name') expected = 'foo' self.assertEquals(result, expected) - result = utils.readconf(StringIO(conf), 'section1', defaults={'bar': 'baz'}) + result = utils.readconf('/tmp/test', 'section1', defaults={'bar': 'baz'}) expected = {'foo': 'bar', 'bar': 'baz'} self.assertEquals(result, expected) + os.unlink('/tmp/test') if __name__ == '__main__': unittest.main() From 60d159fd39478f59914723042cf198cdd99ea95c Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Fri, 8 Oct 2010 14:55:43 -0500 Subject: [PATCH 84/89] updated readconf test --- test/unit/common/test_utils.py | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/test/unit/common/test_utils.py b/test/unit/common/test_utils.py index 11f0fa5bcf..92be1077c0 100644 --- a/test/unit/common/test_utils.py +++ b/test/unit/common/test_utils.py @@ -257,22 +257,21 @@ log_name = yarr''' f.write(conf) f.close() result = utils.readconf('/tmp/test') - expected = {'section1': {'foo': 'bar'}, 'section2': {'log_name': 'yarr'}} + expected = {'log_name': None, + 'section1': {'foo': 'bar'}, + 'section2': {'log_name': 'yarr'}} self.assertEquals(result, expected) result = utils.readconf('/tmp/test', 'section1') - expected = {'foo': 'bar'} + expected = {'log_name': 'section1', 'foo': 'bar'} self.assertEquals(result, expected) result = utils.readconf('/tmp/test', 'section2').get('log_name') expected = 'yarr' self.assertEquals(result, expected) - result = utils.readconf('/tmp/test', 'section1').get('log_name') - expected = 'section1' - self.assertEquals(result, expected) result = utils.readconf('/tmp/test', 'section1', log_name='foo').get('log_name') expected = 'foo' self.assertEquals(result, expected) result = utils.readconf('/tmp/test', 'section1', defaults={'bar': 'baz'}) - expected = {'foo': 'bar', 'bar': 'baz'} + expected = {'log_name': 'section1', 'foo': 'bar', 'bar': 'baz'} self.assertEquals(result, expected) os.unlink('/tmp/test') From 0235c5cd86f793b1fb737b4cd673f4febb4e5d4d Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Fri, 8 Oct 2010 15:23:23 -0500 Subject: [PATCH 85/89] fixed internal proxy references in log processor --- swift/stats/log_processor.py | 25 +++++++++---------------- test/unit/stats/test_log_processor.py | 6 +++--- 2 files changed, 12 insertions(+), 19 deletions(-) diff --git a/swift/stats/log_processor.py b/swift/stats/log_processor.py index a304a878e1..1b71b61397 100644 --- a/swift/stats/log_processor.py +++ b/swift/stats/log_processor.py @@ -40,14 +40,21 @@ class LogProcessor(object): def __init__(self, conf, logger): stats_conf = conf.get('log-processor', {}) - - self._internal_proxy = None if isinstance(logger, tuple): self.logger = get_logger(*logger) else: self.logger = logger + proxy_server_conf_loc = stats_conf.get('proxy_server_conf', + '/etc/swift/proxy-server.conf') + self.proxy_server_conf = appconfig( + 'config:%s' % proxy_server_conf_loc, + name='proxy-server') + self.internal_proxy = InternalProxy(self.proxy_server_conf, + self.logger, + retries=3) + # load the processing plugins self.plugins = {} plugin_prefix = 'log-processor-' @@ -62,20 +69,6 @@ class LogProcessor(object): self.plugins[plugin_name]['instance'] = klass(plugin_conf) self.logger.debug('Loaded plugin "%s"' % plugin_name) - @property - def internal_proxy(self): - '''Lazy load internal proxy''' - if self._internal_proxy is None: - proxy_server_conf_loc = stats_conf.get('proxy_server_conf', - '/etc/swift/proxy-server.conf') - self.proxy_server_conf = appconfig( - 'config:%s' % proxy_server_conf_loc, - name='proxy-server') - self._internal_proxy = InternalProxy(self.proxy_server_conf, - self.logger, - retries=3) - return self._internal_proxy - def process_one_file(self, plugin_name, account, container, object_name): self.logger.info('Processing %s/%s/%s with plugin "%s"' % (account, container, diff --git a/test/unit/stats/test_log_processor.py b/test/unit/stats/test_log_processor.py index 4ff73eccf3..cee34e55b7 100644 --- a/test/unit/stats/test_log_processor.py +++ b/test/unit/stats/test_log_processor.py @@ -127,7 +127,7 @@ class TestLogProcessor(unittest.TestCase): def test_get_container_listing(self): p = log_processor.LogProcessor(self.proxy_config, DumbLogger()) - p._internal_proxy = DumbInternalProxy() + p.internal_proxy = DumbInternalProxy() result = p.get_container_listing('a', 'foo') expected = ['2010/03/14/13/obj1'] self.assertEquals(result, expected) @@ -148,7 +148,7 @@ class TestLogProcessor(unittest.TestCase): def test_get_object_data(self): p = log_processor.LogProcessor(self.proxy_config, DumbLogger()) - p._internal_proxy = DumbInternalProxy() + p.internal_proxy = DumbInternalProxy() result = list(p.get_object_data('a', 'c', 'o', False)) expected = ['obj','data'] self.assertEquals(result, expected) @@ -163,7 +163,7 @@ class TestLogProcessor(unittest.TestCase): 'swift.stats.stats_processor.StatsLogProcessor' }}) p = log_processor.LogProcessor(stats_proxy_config, DumbLogger()) - p._internal_proxy = DumbInternalProxy() + p.internal_proxy = DumbInternalProxy() def get_object_data(*a,**kw): return [self.stats_test_line] p.get_object_data = get_object_data From 041933a706514708527225b2ce3994c1d0c11f62 Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Fri, 8 Oct 2010 16:22:23 -0500 Subject: [PATCH 86/89] fixed account stats filename creation --- swift/stats/account_stats.py | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/swift/stats/account_stats.py b/swift/stats/account_stats.py index feae842414..3f2a23d83d 100644 --- a/swift/stats/account_stats.py +++ b/swift/stats/account_stats.py @@ -41,6 +41,8 @@ class AccountStat(Daemon): server_conf = appconfig('config:%s' % account_server_conf_loc, name='account-server') filename_format = stats_conf['source_filename_format'] + if filename_format.count('*') > 1: + raise Exception('source filename format should have at max one *') self.filename_format = filename_format self.target_dir = target_dir mkdirs(self.target_dir) @@ -96,6 +98,14 @@ class AccountStat(Daemon): statfile.write(line_data) hasher.update(line_data) file_hash = hasher.hexdigest() - src_filename = '_'.join([src_filename, file_hash]) + hash_index = src_filename.find('*') + if hash_index < 0: + # if there is no * in the target filename, the uploader probably + # won't work because we are crafting a filename that doesn't + # fit the pattern + src_filename = '_'.join([src_filename, file_hash]) + else: + parts = src_filename[:hash_index], src_filename[hash_index+1:] + src_filename = ''.join([parts[0], file_hash, parts[1]]) renamer(tmp_filename, os.path.join(self.target_dir, src_filename)) shutil.rmtree(working_dir, ignore_errors=True) From 9ed1d023128e84789b0d385439312b9add3031ba Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Fri, 8 Oct 2010 16:31:55 -0500 Subject: [PATCH 87/89] pep8 tomfoolery --- swift/stats/account_stats.py | 2 +- swift/stats/stats_processor.py | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/swift/stats/account_stats.py b/swift/stats/account_stats.py index 3f2a23d83d..ddf4192119 100644 --- a/swift/stats/account_stats.py +++ b/swift/stats/account_stats.py @@ -105,7 +105,7 @@ class AccountStat(Daemon): # fit the pattern src_filename = '_'.join([src_filename, file_hash]) else: - parts = src_filename[:hash_index], src_filename[hash_index+1:] + parts = src_filename[:hash_index], src_filename[hash_index + 1:] src_filename = ''.join([parts[0], file_hash, parts[1]]) renamer(tmp_filename, os.path.join(self.target_dir, src_filename)) shutil.rmtree(working_dir, ignore_errors=True) diff --git a/swift/stats/stats_processor.py b/swift/stats/stats_processor.py index 3fd352bab7..6caaae7840 100644 --- a/swift/stats/stats_processor.py +++ b/swift/stats/stats_processor.py @@ -15,6 +15,7 @@ from swift.common.utils import get_logger + class StatsLogProcessor(object): """Transform account storage stat logs""" From 86faf56ee11008f01b6a5dee9fa96d5f74c9a5dd Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Fri, 8 Oct 2010 16:34:27 -0500 Subject: [PATCH 88/89] moved paren --- swift/stats/log_processor.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/swift/stats/log_processor.py b/swift/stats/log_processor.py index 1b71b61397..39679f0503 100644 --- a/swift/stats/log_processor.py +++ b/swift/stats/log_processor.py @@ -73,8 +73,7 @@ class LogProcessor(object): self.logger.info('Processing %s/%s/%s with plugin "%s"' % (account, container, object_name, - plugin_name) - ) + plugin_name)) # get an iter of the object data compressed = object_name.endswith('.gz') stream = self.get_object_data(account, container, object_name, From 9572fa62f557e590949d68ebdc41dbf2d9e7837e Mon Sep 17 00:00:00 2001 From: John Dickinson Date: Fri, 8 Oct 2010 17:20:43 -0500 Subject: [PATCH 89/89] added lazy load of internal proxy to log processor (you were right clay) --- swift/stats/log_processor.py | 27 +++++++++++++++++---------- test/unit/stats/test_log_processor.py | 6 +++--- 2 files changed, 20 insertions(+), 13 deletions(-) diff --git a/swift/stats/log_processor.py b/swift/stats/log_processor.py index 39679f0503..6fd6c68597 100644 --- a/swift/stats/log_processor.py +++ b/swift/stats/log_processor.py @@ -39,21 +39,13 @@ class LogProcessor(object): """Load plugins, process logs""" def __init__(self, conf, logger): - stats_conf = conf.get('log-processor', {}) - if isinstance(logger, tuple): self.logger = get_logger(*logger) else: self.logger = logger - proxy_server_conf_loc = stats_conf.get('proxy_server_conf', - '/etc/swift/proxy-server.conf') - self.proxy_server_conf = appconfig( - 'config:%s' % proxy_server_conf_loc, - name='proxy-server') - self.internal_proxy = InternalProxy(self.proxy_server_conf, - self.logger, - retries=3) + self.conf = conf + self._internal_proxy = None # load the processing plugins self.plugins = {} @@ -69,6 +61,21 @@ class LogProcessor(object): self.plugins[plugin_name]['instance'] = klass(plugin_conf) self.logger.debug('Loaded plugin "%s"' % plugin_name) + @property + def internal_proxy(self): + if self._internal_proxy is None: + stats_conf = self.conf.get('log-processor', {}) + proxy_server_conf_loc = stats_conf.get('proxy_server_conf', + '/etc/swift/proxy-server.conf') + proxy_server_conf = appconfig( + 'config:%s' % proxy_server_conf_loc, + name='proxy-server') + self._internal_proxy = InternalProxy(proxy_server_conf, + self.logger, + retries=3) + else: + return self._internal_proxy + def process_one_file(self, plugin_name, account, container, object_name): self.logger.info('Processing %s/%s/%s with plugin "%s"' % (account, container, diff --git a/test/unit/stats/test_log_processor.py b/test/unit/stats/test_log_processor.py index cee34e55b7..4ff73eccf3 100644 --- a/test/unit/stats/test_log_processor.py +++ b/test/unit/stats/test_log_processor.py @@ -127,7 +127,7 @@ class TestLogProcessor(unittest.TestCase): def test_get_container_listing(self): p = log_processor.LogProcessor(self.proxy_config, DumbLogger()) - p.internal_proxy = DumbInternalProxy() + p._internal_proxy = DumbInternalProxy() result = p.get_container_listing('a', 'foo') expected = ['2010/03/14/13/obj1'] self.assertEquals(result, expected) @@ -148,7 +148,7 @@ class TestLogProcessor(unittest.TestCase): def test_get_object_data(self): p = log_processor.LogProcessor(self.proxy_config, DumbLogger()) - p.internal_proxy = DumbInternalProxy() + p._internal_proxy = DumbInternalProxy() result = list(p.get_object_data('a', 'c', 'o', False)) expected = ['obj','data'] self.assertEquals(result, expected) @@ -163,7 +163,7 @@ class TestLogProcessor(unittest.TestCase): 'swift.stats.stats_processor.StatsLogProcessor' }}) p = log_processor.LogProcessor(stats_proxy_config, DumbLogger()) - p.internal_proxy = DumbInternalProxy() + p._internal_proxy = DumbInternalProxy() def get_object_data(*a,**kw): return [self.stats_test_line] p.get_object_data = get_object_data