py3: port the container

This started with ShardRanges and its CLI. The sharder is at the
bottom of the dependency chain. Even container backend needs it.
Once we started tinkering with the sharder, it all snowballed to
include the rest of the container services.

Beware, this does affect some of Python 2 code. Mostly it's trivial
and obviously correct, but needs checking by reviewers.

About killing the stray "from __future__ import unicode_literals":
we do not do it in general. The specific problem it caused was
a failure of functional tests because unicode leaked into a field
that was supposed to be encoded. It is just too hard to track the
types when rules change from file to file, so off with its head.

Change-Id: Iba4e65d0e46d8c1f5a91feb96c2c07f99ca7c666
This commit is contained in:
Pete Zaitcev 2018-05-22 16:17:12 -05:00
parent 3c224af80c
commit 575538b55b
20 changed files with 494 additions and 313 deletions

View File

@ -171,7 +171,7 @@ from swift.container.sharder import make_shard_ranges, sharding_enabled, \
def _load_and_validate_shard_data(args):
try:
with open(args.input, 'rb') as fd:
with open(args.input, 'r') as fd:
try:
data = json.load(fd)
if not isinstance(data, list):
@ -329,7 +329,7 @@ def delete_shard_ranges(broker, args):
return 0
def _replace_shard_ranges(broker, args, shard_data, timeout=None):
def _replace_shard_ranges(broker, args, shard_data, timeout=0):
own_shard_range = _check_own_shard_range(broker, args)
shard_ranges = make_shard_ranges(
broker, shard_data, args.shards_account_prefix)
@ -435,7 +435,7 @@ def _add_enable_args(parser):
def _make_parser():
parser = argparse.ArgumentParser(description='Manage shard ranges')
parser.add_argument('container_db')
parser.add_argument('--verbose', '-v', action='count',
parser.add_argument('--verbose', '-v', action='count', default=0,
help='Increase output verbosity')
subparsers = parser.add_subparsers(
help='Sub-command help', title='Sub-commands')

View File

@ -986,7 +986,8 @@ class SwiftRecon(object):
help="Print verbose info")
args.add_option('--suppress', action="store_true",
help="Suppress most connection related errors")
args.add_option('--async', '-a', action="store_true",
args.add_option('--async', '-a',
action="store_true", dest="async_check",
help="Get async stats")
args.add_option('--replication', '-r', action="store_true",
help="Get replication stats")
@ -1104,7 +1105,7 @@ class SwiftRecon(object):
self.time_check(hosts, options.jitter)
self.version_check(hosts)
else:
if options.async:
if options.async_check:
if self.server_type == 'object':
self.async_check(hosts)
else:

View File

@ -875,15 +875,10 @@ class DatabaseBroker(object):
meta_count = 0
meta_size = 0
for key, (value, timestamp) in metadata.items():
if key and not isinstance(key, six.text_type):
if not check_utf8(key):
raise HTTPBadRequest('Metadata must be valid UTF-8')
# Promote to a natural string for the checks below
if six.PY3:
key = key.decode('utf8')
if value and not isinstance(value, six.text_type):
if not check_utf8(value):
raise HTTPBadRequest('Metadata must be valid UTF-8')
if key and not check_utf8(key):
raise HTTPBadRequest('Metadata must be valid UTF-8')
if value and not check_utf8(value):
raise HTTPBadRequest('Metadata must be valid UTF-8')
key = key.lower()
if value and key.startswith(('x-account-meta-',
'x-container-meta-')):

View File

@ -78,7 +78,6 @@ from six.moves import range, http_client
from six.moves.urllib.parse import ParseResult
from six.moves.urllib.parse import quote as _quote
from six.moves.urllib.parse import urlparse as stdlib_urlparse
from six import string_types
from swift import gettext_ as _
import swift.common.exceptions
@ -3974,7 +3973,10 @@ class Spliterator(object):
yield to_yield
while n > 0:
chunk = next(self.input_iterator)
try:
chunk = next(self.input_iterator)
except StopIteration:
return
cl = len(chunk)
if cl <= n:
n -= cl
@ -4719,12 +4721,17 @@ class ShardRange(object):
def _encode(cls, value):
if six.PY2 and isinstance(value, six.text_type):
return value.encode('utf-8')
if six.PY3 and isinstance(value, six.binary_type):
# This should never fail -- the value should always be coming from
# valid swift paths, which means UTF-8
return value.decode('utf-8')
return value
def _encode_bound(self, bound):
if isinstance(bound, ShardRange.OuterBound):
return bound
if not isinstance(bound, string_types):
if not (isinstance(bound, six.text_type) or
isinstance(bound, six.binary_type)):
raise TypeError('must be a string type')
return self._encode(bound)
@ -4812,7 +4819,7 @@ class ShardRange(object):
@lower.setter
def lower(self, value):
if value in (None, ''):
if value in (None, b'', u''):
value = ShardRange.MIN
try:
value = self._encode_bound(value)
@ -4838,7 +4845,7 @@ class ShardRange(object):
@upper.setter
def upper(self, value):
if value in (None, ''):
if value in (None, b'', u''):
value = ShardRange.MAX
try:
value = self._encode_bound(value)
@ -5027,7 +5034,7 @@ class ShardRange(object):
elif other is None:
return True
else:
return self.upper < other
return self.upper < self._encode(other)
def __gt__(self, other):
# a ShardRange is greater than other if its entire namespace is greater
@ -5041,7 +5048,7 @@ class ShardRange(object):
elif other is None:
return False
else:
return self.lower >= other
return self.lower >= self._encode(other)
def __eq__(self, other):
# test for equality of range bounds only
@ -5049,6 +5056,12 @@ class ShardRange(object):
return False
return self.lower == other.lower and self.upper == other.upper
# A by-the-book implementation should probably hash the value, which
# in our case would be account+container+lower+upper (+timestamp ?).
# But we seem to be okay with just the identity.
def __hash__(self):
return id(self)
def __ne__(self, other):
return not (self == other)

View File

@ -1081,8 +1081,9 @@ class ContainerBroker(DatabaseBroker):
if transform_func is None:
transform_func = self._transform_record
delim_force_gte = False
(marker, end_marker, prefix, delimiter, path) = utf8encode(
marker, end_marker, prefix, delimiter, path)
if six.PY2:
(marker, end_marker, prefix, delimiter, path) = utf8encode(
marker, end_marker, prefix, delimiter, path)
self._commit_puts_stale_ok()
if reverse:
# Reverse the markers if we are reversing the listing.
@ -1117,7 +1118,7 @@ class ContainerBroker(DatabaseBroker):
query_args.append(marker)
# Always set back to False
delim_force_gte = False
elif marker and marker >= prefix:
elif marker and (not prefix or marker >= prefix):
query_conditions.append('name > ?')
query_args.append(marker)
elif prefix:
@ -1268,6 +1269,8 @@ class ContainerBroker(DatabaseBroker):
for item in item_list:
if six.PY2 and isinstance(item['name'], six.text_type):
item['name'] = item['name'].encode('utf-8')
elif not six.PY2 and isinstance(item['name'], six.binary_type):
item['name'] = item['name'].decode('utf-8')
def _really_really_merge_items(conn):
curs = conn.cursor()
@ -1364,6 +1367,8 @@ class ContainerBroker(DatabaseBroker):
for col in ('name', 'lower', 'upper'):
if six.PY2 and isinstance(item[col], six.text_type):
item[col] = item[col].encode('utf-8')
elif not six.PY2 and isinstance(item[col], six.binary_type):
item[col] = item[col].decode('utf-8')
item_list.append(item)
def _really_merge_items(conn):
@ -1418,6 +1423,11 @@ class ContainerBroker(DatabaseBroker):
try:
return _really_merge_items(conn)
except sqlite3.OperationalError as err:
# Without the rollback, new enough (>= py37) python/sqlite3
# will panic:
# sqlite3.OperationalError: cannot start a transaction
# within a transaction
conn.rollback()
if ('no such table: %s' % SHARD_RANGE_TABLE) not in str(err):
raise
self.create_shard_range_table(conn)
@ -2137,7 +2147,7 @@ class ContainerBroker(DatabaseBroker):
found_ranges = []
sub_broker = self.get_brokers()[0]
index = len(existing_ranges)
while limit < 0 or len(found_ranges) < limit:
while limit is None or limit < 0 or len(found_ranges) < limit:
if progress + shard_size >= object_count:
# next shard point is at or beyond final object name so don't
# bother with db query

View File

@ -13,11 +13,13 @@
import time
from collections import defaultdict
import functools
import socket
import itertools
import logging
from eventlet import GreenPile, GreenPool, Timeout
import six
from swift.common import constraints
from swift.common.daemon import Daemon
@ -117,8 +119,9 @@ def translate_container_headers_to_info(headers):
def best_policy_index(headers):
container_info = map(translate_container_headers_to_info, headers)
container_info.sort(cmp=cmp_policy_info)
container_info = [translate_container_headers_to_info(header_set)
for header_set in headers]
container_info.sort(key=functools.cmp_to_key(cmp_policy_info))
return container_info[0]['storage_policy_index']
@ -261,7 +264,10 @@ def parse_raw_obj(obj_info):
:returns: a queue entry dict with the keys: q_policy_index, account,
container, obj, q_op, q_ts, q_record, and path
"""
raw_obj_name = obj_info['name'].encode('utf-8')
if six.PY2:
raw_obj_name = obj_info['name'].encode('utf-8')
else:
raw_obj_name = obj_info['name']
policy_index, obj_name = raw_obj_name.split(':', 1)
q_policy_index = int(policy_index)
@ -691,8 +697,10 @@ class ContainerReconciler(Daemon):
break
# reversed order since we expect older containers to be empty
for c in reversed(one_page):
# encoding here is defensive
container = c['name'].encode('utf8')
container = c['name']
if six.PY2:
# encoding here is defensive
container = container.encode('utf8')
if container == current_container:
continue # we've already hit this one this pass
yield container

View File

@ -51,7 +51,8 @@ from swift.common.header_key_dict import HeaderKeyDict
from swift.common.swob import HTTPAccepted, HTTPBadRequest, HTTPConflict, \
HTTPCreated, HTTPInternalServerError, HTTPNoContent, HTTPNotFound, \
HTTPPreconditionFailed, HTTPMethodNotAllowed, Request, Response, \
HTTPInsufficientStorage, HTTPException, HTTPMovedPermanently
HTTPInsufficientStorage, HTTPException, HTTPMovedPermanently, \
wsgi_to_str, str_to_wsgi
def gen_resp_headers(info, is_deleted=False):
@ -418,7 +419,7 @@ class ContainerController(BaseStorageServer):
def _update_metadata(self, req, broker, req_timestamp, method):
metadata = {}
metadata.update(
(key, (value, req_timestamp.internal))
(wsgi_to_str(key), (wsgi_to_str(value), req_timestamp.internal))
for key, value in req.headers.items()
if key.lower() in self.save_headers or
is_sys_or_user_meta('container', key))
@ -465,11 +466,12 @@ class ContainerController(BaseStorageServer):
broker.put_object(obj, req_timestamp.internal,
int(req.headers['x-size']),
req.headers['x-content-type'],
req.headers['x-etag'], 0,
wsgi_to_str(req.headers['x-content-type']),
wsgi_to_str(req.headers['x-etag']), 0,
obj_policy_index,
req.headers.get('x-content-type-timestamp'),
req.headers.get('x-meta-timestamp'))
wsgi_to_str(req.headers.get(
'x-content-type-timestamp')),
wsgi_to_str(req.headers.get('x-meta-timestamp')))
return HTTPCreated(request=req)
record_type = req.headers.get('x-backend-record-type', '').lower()
@ -530,7 +532,7 @@ class ContainerController(BaseStorageServer):
if is_deleted:
return HTTPNotFound(request=req, headers=headers)
headers.update(
(key, value)
(str_to_wsgi(key), str_to_wsgi(value))
for key, (value, timestamp) in broker.metadata.items()
if value != '' and (key.lower() in self.save_headers or
is_sys_or_user_meta('container', key)))
@ -709,7 +711,7 @@ class ContainerController(BaseStorageServer):
for key, (value, timestamp) in metadata.items():
if value and (key.lower() in self.save_headers or
is_sys_or_user_meta('container', key)):
resp_headers[key] = value
resp_headers[str_to_wsgi(key)] = str_to_wsgi(value)
listing = [self.update_data_record(record)
for record in container_list]
if out_content_type.endswith('/xml'):

View File

@ -29,6 +29,7 @@ from swift.common.direct_client import (direct_put_container,
DirectClientException)
from swift.common.exceptions import DeviceUnavailable
from swift.common.ring.utils import is_local_device
from swift.common.swob import str_to_wsgi
from swift.common.utils import get_logger, config_true_value, \
dump_recon_cache, whataremyips, Timestamp, ShardRange, GreenAsyncPile, \
config_float_value, config_positive_int_value, \
@ -571,7 +572,7 @@ class ContainerSharder(ContainerReplicator):
def _send_shard_ranges(self, account, container, shard_ranges,
headers=None):
body = json.dumps([dict(sr) for sr in shard_ranges])
body = json.dumps([dict(sr) for sr in shard_ranges]).encode('ascii')
part, nodes = self.ring.get_nodes(account, container)
headers = headers or {}
headers.update({'X-Backend-Record-Type': RECORD_TYPE_SHARD,
@ -676,8 +677,8 @@ class ContainerSharder(ContainerReplicator):
if own_shard_range:
shard_ranges = self._fetch_shard_ranges(
broker, newest=True,
params={'marker': own_shard_range.lower,
'end_marker': own_shard_range.upper},
params={'marker': str_to_wsgi(own_shard_range.lower_str),
'end_marker': str_to_wsgi(own_shard_range.upper_str)},
include_deleted=True)
if shard_ranges:
for shard_range in shard_ranges:
@ -940,8 +941,10 @@ class ContainerSharder(ContainerReplicator):
ranges = self._fetch_shard_ranges(
broker, newest=True,
params={'states': 'updating',
'marker': src_shard_range.lower_str,
'end_marker': src_shard_range.end_marker})
'marker': str_to_wsgi(
src_shard_range.lower_str),
'end_marker': str_to_wsgi(
src_shard_range.end_marker)})
outer['ranges'] = iter(ranges)
return outer['ranges']
return shard_range_fetcher
@ -992,7 +995,7 @@ class ContainerSharder(ContainerReplicator):
their correct shard containers, False otherwise
"""
self.logger.debug('Looking for misplaced objects in %s (%s)',
broker.path.decode('utf-8'), broker.db_file)
broker.path, broker.db_file)
self._increment_stat('misplaced', 'attempted')
src_broker = src_broker or broker
if src_bounds is None:
@ -1135,7 +1138,7 @@ class ContainerSharder(ContainerReplicator):
source_max_row = source_broker.get_max_row()
sync_point = shard_broker.get_sync(source_db_id)
if sync_point < source_max_row:
sync_from_row = max(cleaving_context.last_cleave_to_row,
sync_from_row = max(cleaving_context.last_cleave_to_row or -1,
sync_point)
for objects, info in self.yield_objects(
source_broker, shard_range,

View File

@ -19,9 +19,11 @@ from __future__ import print_function
import os
import copy
import logging
import logging.handlers
import sys
from contextlib import contextmanager, closing
from collections import defaultdict, Iterable
from hashlib import md5
import itertools
from numbers import Number
from tempfile import NamedTemporaryFile
@ -37,6 +39,11 @@ import random
import errno
import xattr
import six.moves.cPickle as pickle
from six import BytesIO
from six.moves import range
from six.moves.http_client import HTTPException
from swift.common import storage_policy, swob, utils
from swift.common.storage_policy import (StoragePolicy, ECStoragePolicy,
VALID_EC_TYPES)
@ -45,15 +52,8 @@ from test import get_config
from swift.common.header_key_dict import HeaderKeyDict
from swift.common.ring import Ring, RingData, RingBuilder
from swift.obj import server
from hashlib import md5
import logging.handlers
from six.moves import range
from six import BytesIO
from six.moves.http_client import HTTPException
import functools
import six.moves.cPickle as pickle
from gzip import GzipFile
import mock as mocklib
import inspect

View File

@ -19,6 +19,7 @@ import mock
from shutil import rmtree
from tempfile import mkdtemp
import six
from six.moves import cStringIO as StringIO
from test.unit import patch_policies, write_fake_ring, skip_if_no_xattrs
@ -268,6 +269,87 @@ Shard Ranges (3):
self.assertEqual(sorted(out.getvalue().strip().split('\n')),
sorted(exp_out.strip().split('\n')))
def test_print_db_info_metadata_with_shard_ranges_bis(self):
shard_ranges = [utils.ShardRange(
name='.sharded_a/shard_range_%s' % i,
timestamp=utils.Timestamp(i), lower=u'%d\u30a2' % i,
upper=u'%d\u30e4' % i, object_count=i, bytes_used=i,
meta_timestamp=utils.Timestamp(i)) for i in range(1, 4)]
shard_ranges[0].state = utils.ShardRange.CLEAVED
shard_ranges[1].state = utils.ShardRange.CREATED
info = dict(
account='acct',
container='cont',
storage_policy_index=0,
created_at='0000000100.10000',
put_timestamp='0000000106.30000',
delete_timestamp='0000000107.90000',
status_changed_at='0000000108.30000',
object_count='20',
bytes_used='42',
reported_put_timestamp='0000010106.30000',
reported_delete_timestamp='0000010107.90000',
reported_object_count='20',
reported_bytes_used='42',
db_state=SHARDED,
is_root=True,
shard_ranges=shard_ranges)
info['hash'] = 'abaddeadbeefcafe'
info['id'] = 'abadf100d0ddba11'
out = StringIO()
with mock.patch('sys.stdout', out):
print_db_info_metadata('container', info, {})
if six.PY2:
s_a = '\\xe3\\x82\\xa2'
s_ya = '\\xe3\\x83\\xa4'
else:
s_a = '\u30a2'
s_ya = '\u30e4'
exp_out = '''Path: /acct/cont
Account: acct
Container: cont
Container Hash: d49d0ecbb53be1fcc49624f2f7c7ccae
Metadata:
Created at: 1970-01-01T00:01:40.100000 (0000000100.10000)
Put Timestamp: 1970-01-01T00:01:46.300000 (0000000106.30000)
Delete Timestamp: 1970-01-01T00:01:47.900000 (0000000107.90000)
Status Timestamp: 1970-01-01T00:01:48.300000 (0000000108.30000)
Object Count: 20
Bytes Used: 42
Storage Policy: %s (0)
Reported Put Timestamp: 1970-01-01T02:48:26.300000 (0000010106.30000)
Reported Delete Timestamp: 1970-01-01T02:48:27.900000 (0000010107.90000)
Reported Object Count: 20
Reported Bytes Used: 42
Chexor: abaddeadbeefcafe
UUID: abadf100d0ddba11
No system metadata found in db file
No user metadata found in db file
Sharding Metadata:
Type: root
State: sharded
Shard Ranges (3):
Name: .sharded_a/shard_range_1
lower: '1%s', upper: '1%s'
Object Count: 1, Bytes Used: 1, State: cleaved (30)
Created at: 1970-01-01T00:00:01.000000 (0000000001.00000)
Meta Timestamp: 1970-01-01T00:00:01.000000 (0000000001.00000)
Name: .sharded_a/shard_range_2
lower: '2%s', upper: '2%s'
Object Count: 2, Bytes Used: 2, State: created (20)
Created at: 1970-01-01T00:00:02.000000 (0000000002.00000)
Meta Timestamp: 1970-01-01T00:00:02.000000 (0000000002.00000)
Name: .sharded_a/shard_range_3
lower: '3%s', upper: '3%s'
Object Count: 3, Bytes Used: 3, State: found (10)
Created at: 1970-01-01T00:00:03.000000 (0000000003.00000)
Meta Timestamp: 1970-01-01T00:00:03.000000 (0000000003.00000)''' %\
(POLICIES[0].name, s_a, s_ya, s_a, s_ya, s_a, s_ya)
self.assertEqual(out.getvalue().strip().split('\n'),
exp_out.strip().split('\n'))
def test_print_ring_locations_invalid_args(self):
self.assertRaises(ValueError, print_ring_locations,
None, 'dir', 'acct')

View File

@ -10,8 +10,6 @@
# License for the specific language governing permissions and limitations
# under the License.
from __future__ import unicode_literals
import json
import os
import unittest
@ -184,34 +182,36 @@ class TestManageShardRanges(unittest.TestCase):
main([broker.db_file, 'info'])
expected = ['Sharding enabled = True',
'Own shard range: {',
' "bytes_used": 0, ',
' "deleted": 0, ',
' "epoch": "%s", ' % epoch.internal,
' "lower": "", ',
' "meta_timestamp": "%s", ' % now.internal,
' "name": "a/c", ',
' "object_count": 0, ',
' "state": "sharding", ',
' "state_timestamp": "%s", ' % now.internal,
' "timestamp": "%s", ' % now.internal,
' "bytes_used": 0,',
' "deleted": 0,',
' "epoch": "%s",' % epoch.internal,
' "lower": "",',
' "meta_timestamp": "%s",' % now.internal,
' "name": "a/c",',
' "object_count": 0,',
' "state": "sharding",',
' "state_timestamp": "%s",' % now.internal,
' "timestamp": "%s",' % now.internal,
' "upper": ""',
'}',
'db_state = sharding',
'Retiring db id: %s' % retiring_db_id,
'Cleaving context: {',
' "cleave_to_row": null, ',
' "cleaving_done": false, ',
' "cursor": "", ',
' "last_cleave_to_row": null, ',
' "max_row": -1, ',
' "misplaced_done": false, ',
' "ranges_done": 0, ',
' "ranges_todo": 0, ',
' "cleave_to_row": null,',
' "cleaving_done": false,',
' "cursor": "",',
' "last_cleave_to_row": null,',
' "max_row": -1,',
' "misplaced_done": false,',
' "ranges_done": 0,',
' "ranges_todo": 0,',
' "ref": "%s"' % retiring_db_id,
'}',
'Metadata:',
' X-Container-Sysmeta-Sharding = True']
self.assertEqual(expected, out.getvalue().splitlines())
# The json.dumps() in py2 produces trailing space, not in py3.
result = [x.rstrip() for x in out.getvalue().splitlines()]
self.assertEqual(expected, result)
self.assertEqual(['Loaded db broker for a/c.'],
err.getvalue().splitlines())
@ -223,22 +223,23 @@ class TestManageShardRanges(unittest.TestCase):
main([broker.db_file, 'info'])
expected = ['Sharding enabled = True',
'Own shard range: {',
' "bytes_used": 0, ',
' "deleted": 0, ',
' "epoch": "%s", ' % epoch.internal,
' "lower": "", ',
' "meta_timestamp": "%s", ' % now.internal,
' "name": "a/c", ',
' "object_count": 0, ',
' "state": "sharding", ',
' "state_timestamp": "%s", ' % now.internal,
' "timestamp": "%s", ' % now.internal,
' "bytes_used": 0,',
' "deleted": 0,',
' "epoch": "%s",' % epoch.internal,
' "lower": "",',
' "meta_timestamp": "%s",' % now.internal,
' "name": "a/c",',
' "object_count": 0,',
' "state": "sharding",',
' "state_timestamp": "%s",' % now.internal,
' "timestamp": "%s",' % now.internal,
' "upper": ""',
'}',
'db_state = sharded',
'Metadata:',
' X-Container-Sysmeta-Sharding = True']
self.assertEqual(expected, out.getvalue().splitlines())
self.assertEqual(expected,
[x.rstrip() for x in out.getvalue().splitlines()])
self.assertEqual(['Loaded db broker for a/c.'],
err.getvalue().splitlines())
@ -247,7 +248,7 @@ class TestManageShardRanges(unittest.TestCase):
broker.update_metadata({'X-Container-Sysmeta-Sharding':
(True, Timestamp.now().internal)})
input_file = os.path.join(self.testdir, 'shards')
with open(input_file, 'wb') as fd:
with open(input_file, 'w') as fd:
json.dump(self.shard_data, fd)
out = StringIO()
err = StringIO()

View File

@ -7274,7 +7274,7 @@ class TestShardRange(unittest.TestCase):
def test_lower_setter(self):
sr = utils.ShardRange('a/c', utils.Timestamp.now(), 'b', '')
# sanity checks
self.assertEqual('b', sr.lower)
self.assertEqual('b', sr.lower_str)
self.assertEqual(sr.MAX, sr.upper)
def do_test(good_value, expected):
@ -7284,11 +7284,19 @@ class TestShardRange(unittest.TestCase):
do_test(utils.ShardRange.MIN, utils.ShardRange.MIN)
do_test(utils.ShardRange.MAX, utils.ShardRange.MAX)
do_test('', utils.ShardRange.MIN)
do_test(b'', utils.ShardRange.MIN)
do_test(u'', utils.ShardRange.MIN)
do_test(None, utils.ShardRange.MIN)
do_test('a', 'a')
do_test('y', 'y')
do_test(b'a', 'a')
do_test(b'y', 'y')
do_test(u'a', 'a')
do_test(u'y', 'y')
expected = u'\N{SNOWMAN}'
if six.PY2:
expected = expected.encode('utf-8')
do_test(u'\N{SNOWMAN}', expected)
do_test(u'\N{SNOWMAN}'.encode('utf-8'), expected)
sr = utils.ShardRange('a/c', utils.Timestamp.now(), 'b', 'y')
sr.lower = ''
@ -7297,17 +7305,16 @@ class TestShardRange(unittest.TestCase):
sr = utils.ShardRange('a/c', utils.Timestamp.now(), 'b', 'y')
with self.assertRaises(ValueError) as cm:
sr.lower = 'z'
self.assertIn("lower ('z') must be less than or equal to upper ('y')",
str(cm.exception))
self.assertEqual('b', sr.lower)
self.assertEqual('y', sr.upper)
self.assertIn("must be less than or equal to upper", str(cm.exception))
self.assertEqual('b', sr.lower_str)
self.assertEqual('y', sr.upper_str)
def do_test(bad_value):
with self.assertRaises(TypeError) as cm:
sr.lower = bad_value
self.assertIn("lower must be a string", str(cm.exception))
self.assertEqual('b', sr.lower)
self.assertEqual('y', sr.upper)
self.assertEqual('b', sr.lower_str)
self.assertEqual('y', sr.upper_str)
do_test(1)
do_test(1.234)
@ -7316,7 +7323,7 @@ class TestShardRange(unittest.TestCase):
sr = utils.ShardRange('a/c', utils.Timestamp.now(), '', 'y')
# sanity checks
self.assertEqual(sr.MIN, sr.lower)
self.assertEqual('y', sr.upper)
self.assertEqual('y', sr.upper_str)
def do_test(good_value, expected):
sr.upper = good_value
@ -7325,11 +7332,19 @@ class TestShardRange(unittest.TestCase):
do_test(utils.ShardRange.MIN, utils.ShardRange.MIN)
do_test(utils.ShardRange.MAX, utils.ShardRange.MAX)
do_test('', utils.ShardRange.MAX)
do_test(b'', utils.ShardRange.MAX)
do_test(u'', utils.ShardRange.MAX)
do_test(None, utils.ShardRange.MAX)
do_test('z', 'z')
do_test('b', 'b')
do_test(b'z', 'z')
do_test(b'b', 'b')
do_test(u'z', 'z')
do_test(u'b', 'b')
expected = u'\N{SNOWMAN}'
if six.PY2:
expected = expected.encode('utf-8')
do_test(u'\N{SNOWMAN}', expected)
do_test(u'\N{SNOWMAN}'.encode('utf-8'), expected)
sr = utils.ShardRange('a/c', utils.Timestamp.now(), 'b', 'y')
sr.upper = ''
@ -7339,17 +7354,17 @@ class TestShardRange(unittest.TestCase):
with self.assertRaises(ValueError) as cm:
sr.upper = 'a'
self.assertIn(
"upper ('a') must be greater than or equal to lower ('b')",
"must be greater than or equal to lower",
str(cm.exception))
self.assertEqual('b', sr.lower)
self.assertEqual('y', sr.upper)
self.assertEqual('b', sr.lower_str)
self.assertEqual('y', sr.upper_str)
def do_test(bad_value):
with self.assertRaises(TypeError) as cm:
sr.upper = bad_value
self.assertIn("upper must be a string", str(cm.exception))
self.assertEqual('b', sr.lower)
self.assertEqual('y', sr.upper)
self.assertEqual('b', sr.lower_str)
self.assertEqual('y', sr.upper_str)
do_test(1)
do_test(1.234)
@ -7373,18 +7388,16 @@ class TestShardRange(unittest.TestCase):
upper = u'\u00fb'
sr = utils.ShardRange('a/%s-%s' % (lower, upper),
utils.Timestamp.now(), lower, upper)
if six.PY3:
self.assertEqual(u'\u00e4', sr.lower)
self.assertEqual(u'\u00e4', sr.lower_str)
self.assertEqual(u'\u00fb', sr.upper)
self.assertEqual(u'\u00fb', sr.upper_str)
self.assertEqual(u'\u00fb\x00', sr.end_marker)
else:
self.assertEqual(u'\u00e4'.encode('utf8'), sr.lower)
self.assertEqual(u'\u00e4'.encode('utf8'), sr.lower_str)
self.assertEqual(u'\u00fb'.encode('utf8'), sr.upper)
self.assertEqual(u'\u00fb'.encode('utf8'), sr.upper_str)
self.assertEqual(u'\u00fb\x00'.encode('utf8'), sr.end_marker)
exp_lower = lower
exp_upper = upper
if six.PY2:
exp_lower = exp_lower.encode('utf-8')
exp_upper = exp_upper.encode('utf-8')
self.assertEqual(exp_lower, sr.lower)
self.assertEqual(exp_lower, sr.lower_str)
self.assertEqual(exp_upper, sr.upper)
self.assertEqual(exp_upper, sr.upper_str)
self.assertEqual(exp_upper + '\x00', sr.end_marker)
def test_entire_namespace(self):
# test entire range (no boundaries)
@ -7606,9 +7619,10 @@ class TestShardRange(unittest.TestCase):
state=utils.ShardRange.ACTIVE,
state_timestamp=state_ts)
self.assertEqual(
"ShardRange<'l' to 'u' as of %s, (100, 1000) as of %s, "
"ShardRange<%r to %r as of %s, (100, 1000) as of %s, "
"active as of %s>"
% (ts.internal, meta_ts.internal, state_ts.internal), str(sr))
% ('l', 'u',
ts.internal, meta_ts.internal, state_ts.internal), str(sr))
ts.offset = 0
meta_ts.offset = 2

View File

@ -60,8 +60,9 @@ class TestAuditor(unittest.TestCase):
def tearDown(self):
rmtree(os.path.dirname(self.testdir), ignore_errors=1)
@mock.patch('swift.container.auditor.dump_recon_cache')
@mock.patch('swift.container.auditor.ContainerBroker', FakeContainerBroker)
def test_run_forever(self):
def test_run_forever(self, mock_recon):
sleep_times = random.randint(5, 10)
call_times = sleep_times - 1
@ -100,8 +101,9 @@ class TestAuditor(unittest.TestCase):
with mock.patch('swift.container.auditor.time', FakeTime()):
self.assertRaises(ValueError, test_auditor.run_forever)
@mock.patch('swift.container.auditor.dump_recon_cache')
@mock.patch('swift.container.auditor.ContainerBroker', FakeContainerBroker)
def test_run_once(self):
def test_run_once(self, mock_recon):
conf = {}
test_auditor = auditor.ContainerAuditor(conf, logger=self.logger)
@ -115,8 +117,9 @@ class TestAuditor(unittest.TestCase):
self.assertEqual(test_auditor.container_failures, 2)
self.assertEqual(test_auditor.container_passes, 3)
@mock.patch('swift.container.auditor.dump_recon_cache')
@mock.patch('swift.container.auditor.ContainerBroker', FakeContainerBroker)
def test_one_audit_pass(self):
def test_one_audit_pass(self, mock_recon):
conf = {}
test_auditor = auditor.ContainerAuditor(conf, logger=self.logger)
@ -147,7 +150,8 @@ class TestAuditor(unittest.TestCase):
class TestAuditorMigrations(unittest.TestCase):
@with_tempdir
def test_db_migration(self, tempdir):
@mock.patch('swift.container.auditor.dump_recon_cache')
def test_db_migration(self, tempdir, mock_recon):
db_path = os.path.join(tempdir, 'sda', 'containers', '0', '0', '0',
'test.db')
with test_backend.TestContainerBrokerBeforeSPI.old_broker() as \

View File

@ -14,6 +14,7 @@
# limitations under the License.
""" Tests for swift.container.backend """
import base64
import errno
import os
import hashlib
@ -28,6 +29,8 @@ import sqlite3
import pickle
import json
import six
from swift.common.exceptions import LockTimeout
from swift.container.backend import ContainerBroker, \
update_new_item_from_existing, UNSHARDED, SHARDING, SHARDED, \
@ -2877,22 +2880,25 @@ class TestContainerBroker(unittest.TestCase):
self.assertEqual([row[0] for row in listing], ['b:a', 'b:b'])
def test_chexor(self):
def md5_str(s):
if not isinstance(s, bytes):
s = s.encode('utf8')
return hashlib.md5(s).hexdigest()
broker = ContainerBroker(':memory:', account='a', container='c')
broker.initialize(Timestamp('1').internal, 0)
broker.put_object('a', Timestamp(1).internal, 0,
'text/plain', 'd41d8cd98f00b204e9800998ecf8427e')
broker.put_object('b', Timestamp(2).internal, 0,
'text/plain', 'd41d8cd98f00b204e9800998ecf8427e')
hasha = hashlib.md5('%s-%s' % ('a', Timestamp(1).internal)).digest()
hashb = hashlib.md5('%s-%s' % ('b', Timestamp(2).internal)).digest()
hashc = ''.join(
('%02x' % (ord(a) ^ ord(b)) for a, b in zip(hasha, hashb)))
hasha = md5_str('%s-%s' % ('a', Timestamp(1).internal))
hashb = md5_str('%s-%s' % ('b', Timestamp(2).internal))
hashc = '%032x' % (int(hasha, 16) ^ int(hashb, 16))
self.assertEqual(broker.get_info()['hash'], hashc)
broker.put_object('b', Timestamp(3).internal, 0,
'text/plain', 'd41d8cd98f00b204e9800998ecf8427e')
hashb = hashlib.md5('%s-%s' % ('b', Timestamp(3).internal)).digest()
hashc = ''.join(
('%02x' % (ord(a) ^ ord(b)) for a, b in zip(hasha, hashb)))
hashb = md5_str('%s-%s' % ('b', Timestamp(3).internal))
hashc = '%032x' % (int(hasha, 16) ^ int(hashb, 16))
self.assertEqual(broker.get_info()['hash'], hashc)
def test_newid(self):
@ -2968,7 +2974,9 @@ class TestContainerBroker(unittest.TestCase):
def test_merge_items_overwrite_unicode(self):
# test DatabaseBroker.merge_items
snowman = u'\N{SNOWMAN}'.encode('utf-8')
snowman = u'\N{SNOWMAN}'
if six.PY2:
snowman = snowman.encode('utf-8')
broker1 = ContainerBroker(':memory:', account='a', container='c')
broker1.initialize(Timestamp('1').internal, 0)
id = broker1.get_info()['id']
@ -3151,10 +3159,10 @@ class TestContainerBroker(unittest.TestCase):
for i in range(10):
name, timestamp, size, content_type, etag, deleted = (
'o%s' % i, next(ts).internal, 0, 'c', 'e', 0)
fp.write(':')
fp.write(pickle.dumps(
fp.write(b':')
fp.write(base64.b64encode(pickle.dumps(
(name, timestamp, size, content_type, etag, deleted),
protocol=2).encode('base64'))
protocol=2)))
fp.flush()
# use put_object to append some more entries with different
@ -3198,10 +3206,10 @@ class TestContainerBroker(unittest.TestCase):
for i in range(10):
name, timestamp, size, content_type, etag, deleted = (
'o%s' % i, next(ts).internal, 0, 'c', 'e', 0)
fp.write(':')
fp.write(pickle.dumps(
fp.write(b':')
fp.write(base64.b64encode(pickle.dumps(
(name, timestamp, size, content_type, etag, deleted),
protocol=2).encode('base64'))
protocol=2)))
fp.flush()
broker._commit_puts = mock_commit_puts
@ -3227,10 +3235,10 @@ class TestContainerBroker(unittest.TestCase):
for i in range(10):
name, timestamp, size, content_type, etag, deleted = (
'o%s' % i, next(ts).internal, 0, 'c', 'e', 0)
fp.write(':')
fp.write(pickle.dumps(
fp.write(b':')
fp.write(base64.b64encode(pickle.dumps(
(name, timestamp, size, content_type, etag, deleted),
protocol=2).encode('base64'))
protocol=2)))
fp.flush()
broker._commit_puts = mock_commit_puts
@ -3731,7 +3739,7 @@ class TestContainerBroker(unittest.TestCase):
broker = ContainerBroker(db_path, account=a, container=c)
broker.initialize(next(ts_iter).internal, 0)
broker.set_sharding_sysmeta('Root', 'a/c')
broker.merge_shard_ranges(shard_range_by_state.values())
broker.merge_shard_ranges(list(shard_range_by_state.values()))
return broker
# make broker appear to be a root container

View File

@ -26,6 +26,7 @@ import random
from collections import defaultdict
from datetime import datetime
import six
from six.moves import urllib
from swift.container import reconciler
from swift.container.server import gen_resp_headers
@ -105,8 +106,10 @@ class FakeInternalClient(reconciler.InternalClient):
else:
timestamp, content_type = timestamp, 'application/x-put'
storage_policy_index, path = item
if six.PY2 and isinstance(path, six.text_type):
path = path.encode('utf-8')
account, container_name, obj_name = split_path(
path.encode('utf-8'), 0, 3, rest_with_last=True)
path, 0, 3, rest_with_last=True)
self.accounts[account][container_name].append(
(obj_name, storage_policy_index, timestamp, content_type))
for account_name, containers in self.accounts.items():
@ -124,7 +127,8 @@ class FakeInternalClient(reconciler.InternalClient):
if storage_policy_index is None and not obj_name:
# empty container
continue
obj_path = container_path + '/' + obj_name
obj_path = swob.str_to_wsgi(
container_path + '/' + obj_name)
ts = Timestamp(timestamp)
headers = {'X-Timestamp': ts.normal,
'X-Backend-Timestamp': ts.internal}
@ -139,12 +143,15 @@ class FakeInternalClient(reconciler.InternalClient):
# strings, so normalize here
if isinstance(timestamp, numbers.Number):
timestamp = '%f' % timestamp
if six.PY2:
obj_name = obj_name.decode('utf-8')
timestamp = timestamp.decode('utf-8')
obj_data = {
'bytes': 0,
# listing data is unicode
'name': obj_name.decode('utf-8'),
'name': obj_name,
'last_modified': last_modified,
'hash': timestamp.decode('utf-8'),
'hash': timestamp,
'content_type': content_type,
}
container_listing_data.append(obj_data)
@ -752,7 +759,7 @@ class TestReconciler(unittest.TestCase):
with mock.patch.multiple(reconciler, **items) as mocks:
self.mock_delete_container_entry = \
mocks['direct_delete_container_entry']
with mock.patch('time.time', mock_time_iter.next):
with mock.patch('time.time', lambda: next(mock_time_iter)):
self.reconciler.run_once()
return [c[1][1:4] for c in
@ -974,7 +981,10 @@ class TestReconciler(unittest.TestCase):
# functions where we call them with (account, container, obj)
obj_name = u"AUTH_bob/c \u062a/o1 \u062a"
# anytime we talk about a call made to swift for a path
obj_path = obj_name.encode('utf-8')
if six.PY2:
obj_path = obj_name.encode('utf-8')
else:
obj_path = obj_name.encode('utf-8').decode('latin-1')
# this mock expects unquoted unicode because it handles container
# listings as well as paths
self._mock_listing({

View File

@ -37,7 +37,8 @@ from six import StringIO
from swift import __version__ as swift_version
from swift.common.header_key_dict import HeaderKeyDict
from swift.common.swob import (Request, WsgiBytesIO, HTTPNoContent)
from swift.common.swob import (Request, WsgiBytesIO, HTTPNoContent,
bytes_to_wsgi)
import swift.container
from swift.container import server as container_server
from swift.common import constraints
@ -134,7 +135,7 @@ class TestContainerController(unittest.TestCase):
})
resp = req.get_response(self.controller)
self.assertEqual(400, resp.status_int)
self.assertTrue('invalid' in resp.body.lower())
self.assertIn(b'invalid', resp.body.lower())
# good policies
for policy in POLICIES:
@ -337,7 +338,7 @@ class TestContainerController(unittest.TestCase):
headers={'Accept': 'application/plain;q'})
resp = req.get_response(self.controller)
self.assertEqual(resp.status_int, 400)
self.assertEqual(resp.body, '')
self.assertEqual(resp.body, b'')
def test_HEAD_invalid_format(self):
format = '%D1%BD%8A9' # invalid UTF-8; should be %E1%BD%8A9 (E -> D)
@ -1157,23 +1158,25 @@ class TestContainerController(unittest.TestCase):
bindsock = listen_zero()
def accept(return_code, expected_timestamp):
if not isinstance(expected_timestamp, bytes):
expected_timestamp = expected_timestamp.encode('ascii')
try:
with Timeout(3):
sock, addr = bindsock.accept()
inc = sock.makefile('rb')
out = sock.makefile('wb')
out.write('HTTP/1.1 %d OK\r\nContent-Length: 0\r\n\r\n' %
out.write(b'HTTP/1.1 %d OK\r\nContent-Length: 0\r\n\r\n' %
return_code)
out.flush()
self.assertEqual(inc.readline(),
'PUT /sda1/123/a/c HTTP/1.1\r\n')
b'PUT /sda1/123/a/c HTTP/1.1\r\n')
headers = {}
line = inc.readline()
while line and line != '\r\n':
headers[line.split(':')[0].lower()] = \
line.split(':')[1].strip()
while line and line != b'\r\n':
headers[line.split(b':')[0].lower()] = \
line.split(b':')[1].strip()
line = inc.readline()
self.assertEqual(headers['x-put-timestamp'],
self.assertEqual(headers[b'x-put-timestamp'],
expected_timestamp)
except BaseException as err:
return err
@ -1391,7 +1394,7 @@ class TestContainerController(unittest.TestCase):
req = Request.blank('/sda1/p/a/',
environ={'REQUEST_METHOD': 'REPLICATE'},
headers={})
json_string = '["rsync_then_merge", "a.db"]'
json_string = b'["rsync_then_merge", "a.db"]'
inbuf = WsgiBytesIO(json_string)
req.environ['wsgi.input'] = inbuf
resp = req.get_response(self.controller)
@ -1405,7 +1408,7 @@ class TestContainerController(unittest.TestCase):
req = Request.blank('/sda1/p/a/',
environ={'REQUEST_METHOD': 'REPLICATE'},
headers={})
json_string = '["complete_rsync", "a.db"]'
json_string = b'["complete_rsync", "a.db"]'
inbuf = WsgiBytesIO(json_string)
req.environ['wsgi.input'] = inbuf
resp = req.get_response(self.controller)
@ -1416,7 +1419,7 @@ class TestContainerController(unittest.TestCase):
environ={'REQUEST_METHOD': 'REPLICATE'},
headers={})
# check valuerror
wsgi_input_valuerror = '["sync" : sync, "-1"]'
wsgi_input_valuerror = b'["sync" : sync, "-1"]'
inbuf1 = WsgiBytesIO(wsgi_input_valuerror)
req.environ['wsgi.input'] = inbuf1
resp = req.get_response(self.controller)
@ -1427,7 +1430,7 @@ class TestContainerController(unittest.TestCase):
req = Request.blank('/sda1/p/a/',
environ={'REQUEST_METHOD': 'REPLICATE'},
headers={})
json_string = '["unknown_sync", "a.db"]'
json_string = b'["unknown_sync", "a.db"]'
inbuf = WsgiBytesIO(json_string)
req.environ['wsgi.input'] = inbuf
resp = req.get_response(self.controller)
@ -1441,7 +1444,7 @@ class TestContainerController(unittest.TestCase):
req = Request.blank('/sda1/p/a/',
environ={'REQUEST_METHOD': 'REPLICATE'},
headers={})
json_string = '["unknown_sync", "a.db"]'
json_string = b'["unknown_sync", "a.db"]'
inbuf = WsgiBytesIO(json_string)
req.environ['wsgi.input'] = inbuf
resp = req.get_response(self.controller)
@ -1930,7 +1933,7 @@ class TestContainerController(unittest.TestCase):
return req.get_response(self.controller)
ts = (Timestamp(t) for t in itertools.count(int(time.time())))
t0 = ts.next()
t0 = next(ts)
# create container
req = Request.blank('/sda1/p/a/c', method='PUT', headers={
@ -1944,7 +1947,7 @@ class TestContainerController(unittest.TestCase):
self.assertEqual(resp.status_int, 204)
# create object at t1
t1 = ts.next()
t1 = next(ts)
resp = do_update(t1, 'etag_at_t1', 1, 'ctype_at_t1')
self.assertEqual(resp.status_int, 201)
@ -1965,9 +1968,9 @@ class TestContainerController(unittest.TestCase):
self.assertEqual(obj['last_modified'], t1.isoformat)
# send an update with a content type timestamp at t4
t2 = ts.next()
t3 = ts.next()
t4 = ts.next()
t2 = next(ts)
t3 = next(ts)
t4 = next(ts)
resp = do_update(t1, 'etag_at_t1', 1, 'ctype_at_t4', t_type=t4)
self.assertEqual(resp.status_int, 201)
@ -2028,7 +2031,7 @@ class TestContainerController(unittest.TestCase):
self.assertEqual(obj['last_modified'], t4.isoformat)
# now update with an in-between meta timestamp at t5
t5 = ts.next()
t5 = next(ts)
resp = do_update(t2, 'etag_at_t2', 2, 'ctype_at_t3', t_type=t3,
t_meta=t5)
self.assertEqual(resp.status_int, 201)
@ -2050,7 +2053,7 @@ class TestContainerController(unittest.TestCase):
self.assertEqual(obj['last_modified'], t5.isoformat)
# delete object at t6
t6 = ts.next()
t6 = next(ts)
req = Request.blank(
'/sda1/p/a/c/o', method='DELETE', headers={
'X-Timestamp': t6.internal})
@ -2069,9 +2072,9 @@ class TestContainerController(unittest.TestCase):
self.assertEqual(0, len(listing_data))
# subsequent content type timestamp at t8 should leave object deleted
t7 = ts.next()
t8 = ts.next()
t9 = ts.next()
t7 = next(ts)
t8 = next(ts)
t9 = next(ts)
resp = do_update(t2, 'etag_at_t2', 2, 'ctype_at_t8', t_type=t8,
t_meta=t9)
self.assertEqual(resp.status_int, 201)
@ -2110,25 +2113,29 @@ class TestContainerController(unittest.TestCase):
bindsock = listen_zero()
def accept(return_code, expected_timestamp):
if not isinstance(expected_timestamp, bytes):
expected_timestamp = expected_timestamp.encode('ascii')
try:
with Timeout(3):
sock, addr = bindsock.accept()
inc = sock.makefile('rb')
out = sock.makefile('wb')
out.write('HTTP/1.1 %d OK\r\nContent-Length: 0\r\n\r\n' %
out.write(b'HTTP/1.1 %d OK\r\nContent-Length: 0\r\n\r\n' %
return_code)
out.flush()
self.assertEqual(inc.readline(),
'PUT /sda1/123/a/c HTTP/1.1\r\n')
b'PUT /sda1/123/a/c HTTP/1.1\r\n')
headers = {}
line = inc.readline()
while line and line != '\r\n':
headers[line.split(':')[0].lower()] = \
line.split(':')[1].strip()
while line and line != b'\r\n':
headers[line.split(b':')[0].lower()] = \
line.split(b':')[1].strip()
line = inc.readline()
self.assertEqual(headers['x-delete-timestamp'],
self.assertEqual(headers[b'x-delete-timestamp'],
expected_timestamp)
except BaseException as err:
import traceback
traceback.print_exc()
return err
return None
@ -2248,7 +2255,7 @@ class TestContainerController(unittest.TestCase):
body=json.dumps([dict(shard_range)]))
resp = req.get_response(self.controller)
self.assertEqual(400, resp.status_int)
self.assertIn('X-Backend-Storage-Policy-Index header is required',
self.assertIn(b'X-Backend-Storage-Policy-Index header is required',
resp.body)
# PUT shard range to non-existent container with autocreate prefix
@ -2457,7 +2464,7 @@ class TestContainerController(unittest.TestCase):
'/sda1/p/a/c', method='PUT', headers=headers, body=body)
resp = req.get_response(self.controller)
self.assertEqual(400, resp.status_int)
self.assertIn('Invalid body', resp.body)
self.assertIn(b'Invalid body', resp.body)
self.assertEqual(
exp_meta, dict((k, v[0]) for k, v in broker.metadata.items()))
self._assert_shard_ranges_equal(
@ -3451,7 +3458,7 @@ class TestContainerController(unittest.TestCase):
noodles = [u"Spätzle", u"ラーメン"]
for n in noodles:
req = Request.blank(
'/sda1/p/a/jsonc/%s' % n.encode("utf-8"),
'/sda1/p/a/jsonc/%s' % bytes_to_wsgi(n.encode("utf-8")),
environ={'REQUEST_METHOD': 'PUT',
'HTTP_X_TIMESTAMP': '1',
'HTTP_X_CONTENT_TYPE': 'text/plain',
@ -3512,7 +3519,7 @@ class TestContainerController(unittest.TestCase):
self._update_object_put_headers(req)
resp = req.get_response(self.controller)
self.assertEqual(resp.status_int, 201)
plain_body = '0\n1\n2\n'
plain_body = b'0\n1\n2\n'
req = Request.blank('/sda1/p/a/plainc',
environ={'REQUEST_METHOD': 'GET'})
@ -3629,21 +3636,21 @@ class TestContainerController(unittest.TestCase):
self._update_object_put_headers(req)
resp = req.get_response(self.controller)
self.assertEqual(resp.status_int, 201)
xml_body = '<?xml version="1.0" encoding="UTF-8"?>\n' \
'<container name="xmlc">' \
'<object><name>0</name><hash>x</hash><bytes>0</bytes>' \
'<content_type>text/plain</content_type>' \
'<last_modified>1970-01-01T00:00:01.000000' \
'</last_modified></object>' \
'<object><name>1</name><hash>x</hash><bytes>0</bytes>' \
'<content_type>text/plain</content_type>' \
'<last_modified>1970-01-01T00:00:01.000000' \
'</last_modified></object>' \
'<object><name>2</name><hash>x</hash><bytes>0</bytes>' \
'<content_type>text/plain</content_type>' \
'<last_modified>1970-01-01T00:00:01.000000' \
'</last_modified></object>' \
'</container>'
xml_body = b'<?xml version="1.0" encoding="UTF-8"?>\n' \
b'<container name="xmlc">' \
b'<object><name>0</name><hash>x</hash><bytes>0</bytes>' \
b'<content_type>text/plain</content_type>' \
b'<last_modified>1970-01-01T00:00:01.000000' \
b'</last_modified></object>' \
b'<object><name>1</name><hash>x</hash><bytes>0</bytes>' \
b'<content_type>text/plain</content_type>' \
b'<last_modified>1970-01-01T00:00:01.000000' \
b'</last_modified></object>' \
b'<object><name>2</name><hash>x</hash><bytes>0</bytes>' \
b'<content_type>text/plain</content_type>' \
b'<last_modified>1970-01-01T00:00:01.000000' \
b'</last_modified></object>' \
b'</container>'
# tests
req = Request.blank(
@ -3701,7 +3708,7 @@ class TestContainerController(unittest.TestCase):
headers={'Accept': 'application/plain;q'})
resp = req.get_response(self.controller)
self.assertEqual(resp.status_int, 400)
self.assertEqual(resp.body, 'Invalid Accept header')
self.assertEqual(resp.body, b'Invalid Accept header')
def test_GET_marker(self):
# make a container
@ -3724,26 +3731,26 @@ class TestContainerController(unittest.TestCase):
req = Request.blank('/sda1/p/a/c?limit=2&marker=1',
environ={'REQUEST_METHOD': 'GET'})
resp = req.get_response(self.controller)
result = resp.body.split()
self.assertEqual(result, ['2', ])
result = resp.body.split(b'\n')
self.assertEqual(result, [b'2', b''])
# test limit with end_marker
req = Request.blank('/sda1/p/a/c?limit=2&end_marker=1',
environ={'REQUEST_METHOD': 'GET'})
resp = req.get_response(self.controller)
result = resp.body.split()
self.assertEqual(result, ['0', ])
result = resp.body.split(b'\n')
self.assertEqual(result, [b'0', b''])
# test limit, reverse with end_marker
req = Request.blank('/sda1/p/a/c?limit=2&end_marker=1&reverse=True',
environ={'REQUEST_METHOD': 'GET'})
resp = req.get_response(self.controller)
result = resp.body.split()
self.assertEqual(result, ['2', ])
result = resp.body.split(b'\n')
self.assertEqual(result, [b'2', b''])
# test marker > end_marker
req = Request.blank('/sda1/p/a/c?marker=2&end_marker=1',
environ={'REQUEST_METHOD': 'GET'})
resp = req.get_response(self.controller)
result = resp.body.split()
self.assertEqual(result, [])
result = resp.body.split(b'\n')
self.assertEqual(result, [b''])
def test_weird_content_types(self):
snowman = u'\u2603'
@ -3752,11 +3759,12 @@ class TestContainerController(unittest.TestCase):
'HTTP_X_TIMESTAMP': '0'})
resp = req.get_response(self.controller)
for i, ctype in enumerate((snowman.encode('utf-8'),
'text/plain; charset="utf-8"')):
b'text/plain; charset="utf-8"')):
req = Request.blank(
'/sda1/p/a/c/%s' % i, environ={
'REQUEST_METHOD': 'PUT',
'HTTP_X_TIMESTAMP': '1', 'HTTP_X_CONTENT_TYPE': ctype,
'HTTP_X_TIMESTAMP': '1',
'HTTP_X_CONTENT_TYPE': bytes_to_wsgi(ctype),
'HTTP_X_ETAG': 'x', 'HTTP_X_SIZE': 0})
self._update_object_put_headers(req)
resp = req.get_response(self.controller)
@ -3764,6 +3772,7 @@ class TestContainerController(unittest.TestCase):
req = Request.blank('/sda1/p/a/c?format=json',
environ={'REQUEST_METHOD': 'GET'})
resp = req.get_response(self.controller)
self.assertEqual(resp.status_int, 200)
result = [x['content_type'] for x in json.loads(resp.body)]
self.assertEqual(result, [u'\u2603', 'text/plain;charset="utf-8"'])
@ -3842,8 +3851,8 @@ class TestContainerController(unittest.TestCase):
req = Request.blank(
'/sda1/p/a/c?limit=2', environ={'REQUEST_METHOD': 'GET'})
resp = req.get_response(self.controller)
result = resp.body.split()
self.assertEqual(result, ['0', '1'])
result = resp.body.split(b'\n')
self.assertEqual(result, [b'0', b'1', b''])
def test_GET_prefix(self):
req = Request.blank(
@ -3865,7 +3874,7 @@ class TestContainerController(unittest.TestCase):
req = Request.blank(
'/sda1/p/a/c?prefix=a', environ={'REQUEST_METHOD': 'GET'})
resp = req.get_response(self.controller)
self.assertEqual(resp.body.split(), ['a1', 'a2', 'a3'])
self.assertEqual(resp.body.split(b'\n'), [b'a1', b'a2', b'a3', b''])
def test_GET_delimiter_too_long(self):
req = Request.blank('/sda1/p/a/c?delimiter=xx',
@ -3906,7 +3915,7 @@ class TestContainerController(unittest.TestCase):
resp = req.get_response(self.controller)
for obj_name in [u"a/❥/1", u"a/❥/2", u"a/ꙮ/1", u"a/ꙮ/2"]:
req = Request.blank(
'/sda1/p/a/c/%s' % obj_name.encode('utf-8'),
'/sda1/p/a/c/%s' % bytes_to_wsgi(obj_name.encode('utf-8')),
environ={
'REQUEST_METHOD': 'PUT', 'HTTP_X_TIMESTAMP': '1',
'HTTP_X_CONTENT_TYPE': 'text/plain', 'HTTP_X_ETAG': 'x',
@ -3976,11 +3985,11 @@ class TestContainerController(unittest.TestCase):
environ={'REQUEST_METHOD': 'GET'})
resp = req.get_response(self.controller)
self.assertEqual(
resp.body, '<?xml version="1.0" encoding="UTF-8"?>'
'\n<container name="c"><subdir name="US-OK-">'
'<name>US-OK-</name></subdir>'
'<subdir name="US-TX-"><name>US-TX-</name></subdir>'
'<subdir name="US-UT-"><name>US-UT-</name></subdir></container>')
resp.body, b'<?xml version="1.0" encoding="UTF-8"?>'
b'\n<container name="c"><subdir name="US-OK-">'
b'<name>US-OK-</name></subdir>'
b'<subdir name="US-TX-"><name>US-TX-</name></subdir>'
b'<subdir name="US-UT-"><name>US-UT-</name></subdir></container>')
def test_GET_delimiter_xml_with_quotes(self):
req = Request.blank(
@ -4045,8 +4054,8 @@ class TestContainerController(unittest.TestCase):
errbuf = StringIO()
outbuf = StringIO()
def start_response(*args):
outbuf.writelines(args)
def start_response(status, headers):
outbuf.writelines(status)
self.controller.__call__({'REQUEST_METHOD': 'GET',
'SCRIPT_NAME': '',
@ -4071,8 +4080,8 @@ class TestContainerController(unittest.TestCase):
errbuf = StringIO()
outbuf = StringIO()
def start_response(*args):
outbuf.writelines(args)
def start_response(status, headers):
outbuf.writelines(status)
self.controller.__call__({'REQUEST_METHOD': 'GET',
'SCRIPT_NAME': '',
@ -4097,8 +4106,8 @@ class TestContainerController(unittest.TestCase):
errbuf = StringIO()
outbuf = StringIO()
def start_response(*args):
outbuf.writelines(args)
def start_response(status, headers):
outbuf.writelines(status)
self.controller.__call__({'REQUEST_METHOD': 'GET',
'SCRIPT_NAME': '',
@ -4122,8 +4131,8 @@ class TestContainerController(unittest.TestCase):
errbuf = StringIO()
outbuf = StringIO()
def start_response(*args):
outbuf.writelines(args)
def start_response(status, headers):
outbuf.writelines(status)
self.controller.__call__({'REQUEST_METHOD': 'method_doesnt_exist',
'PATH_INFO': '/sda1/p/a/c'},
@ -4135,8 +4144,8 @@ class TestContainerController(unittest.TestCase):
errbuf = StringIO()
outbuf = StringIO()
def start_response(*args):
outbuf.writelines(args)
def start_response(status, headers):
outbuf.writelines(status)
self.controller.__call__({'REQUEST_METHOD': '__init__',
'PATH_INFO': '/sda1/p/a/c'},
@ -4388,9 +4397,9 @@ class TestContainerController(unittest.TestCase):
{'devices': self.testdir, 'mount_check': 'false',
'replication_server': 'false'})
def start_response(*args):
def start_response(status, headers):
"""Sends args to outbuf"""
outbuf.writelines(args)
outbuf.writelines(status)
method = 'PUT'
@ -4414,6 +4423,9 @@ class TestContainerController(unittest.TestCase):
with mock.patch.object(self.controller, method, new=mock_method):
response = self.controller(env, start_response)
self.assertEqual(response, method_res)
# The controller passed responsibility of calling start_response
# to the mock, which never did
self.assertEqual(outbuf.getvalue(), '')
def test_not_allowed_method(self):
# Test correct work for NOT allowed method using
@ -4425,9 +4437,9 @@ class TestContainerController(unittest.TestCase):
{'devices': self.testdir, 'mount_check': 'false',
'replication_server': 'false'})
def start_response(*args):
def start_response(status, headers):
"""Sends args to outbuf"""
outbuf.writelines(args)
outbuf.writelines(status)
method = 'PUT'
@ -4446,12 +4458,13 @@ class TestContainerController(unittest.TestCase):
'wsgi.multiprocess': False,
'wsgi.run_once': False}
answer = ['<html><h1>Method Not Allowed</h1><p>The method is not '
'allowed for this resource.</p></html>']
answer = [b'<html><h1>Method Not Allowed</h1><p>The method is not '
b'allowed for this resource.</p></html>']
mock_method = replication(public(lambda x: mock.MagicMock()))
with mock.patch.object(self.controller, method, new=mock_method):
response = self.controller.__call__(env, start_response)
self.assertEqual(response, answer)
self.assertEqual(outbuf.getvalue()[:4], '405 ')
def test_call_incorrect_replication_method(self):
inbuf = BytesIO()
@ -4461,9 +4474,9 @@ class TestContainerController(unittest.TestCase):
{'devices': self.testdir, 'mount_check': 'false',
'replication_server': 'true'})
def start_response(*args):
def start_response(status, headers):
"""Sends args to outbuf"""
outbuf.writelines(args)
outbuf.writelines(status)
obj_methods = ['DELETE', 'PUT', 'HEAD', 'GET', 'POST', 'OPTIONS']
for method in obj_methods:
@ -4495,9 +4508,9 @@ class TestContainerController(unittest.TestCase):
'replication_server': 'false', 'log_requests': 'false'},
logger=self.logger)
def start_response(*args):
def start_response(status, headers):
# Sends args to outbuf
outbuf.writelines(args)
outbuf.writelines(status)
method = 'PUT'
@ -4524,12 +4537,13 @@ class TestContainerController(unittest.TestCase):
new=mock_put_method):
response = self.container_controller.__call__(env, start_response)
self.assertTrue(response[0].startswith(
'Traceback (most recent call last):'))
b'Traceback (most recent call last):'))
self.assertEqual(self.logger.get_lines_for_level('error'), [
'ERROR __call__ error with %(method)s %(path)s : ' % {
'method': 'PUT', 'path': '/sda1/p/a/c'},
])
self.assertEqual(self.logger.get_lines_for_level('info'), [])
self.assertEqual(outbuf.getvalue()[:4], '500 ')
def test_GET_log_requests_true(self):
self.controller.log_requests = True

View File

@ -31,6 +31,8 @@ import time
from copy import deepcopy
import six
from swift.common import internal_client
from swift.container import replicator
from swift.container.backend import ContainerBroker, UNSHARDED, SHARDING, \
@ -61,7 +63,7 @@ class BaseTestSharder(unittest.TestCase):
def _make_broker(self, account='a', container='c', epoch=None,
device='sda', part=0, hash_=None):
hash_ = hash_ or hashlib.md5(container).hexdigest()
hash_ = hash_ or hashlib.md5(container.encode('utf-8')).hexdigest()
datadir = os.path.join(
self.tempdir, device, 'containers', str(part), hash_[-3:], hash_)
if epoch:
@ -211,14 +213,14 @@ class TestSharder(BaseTestSharder):
with self.assertRaises(ValueError) as cm:
do_test({'shard_shrink_point': 101}, {})
self.assertIn(
'greater than 0, less than 100, not "101"', cm.exception.message)
self.assertIn('shard_shrink_point', cm.exception.message)
'greater than 0, less than 100, not "101"', str(cm.exception))
self.assertIn('shard_shrink_point', str(cm.exception))
with self.assertRaises(ValueError) as cm:
do_test({'shard_shrink_merge_point': 101}, {})
self.assertIn(
'greater than 0, less than 100, not "101"', cm.exception.message)
self.assertIn('shard_shrink_merge_point', cm.exception.message)
'greater than 0, less than 100, not "101"', str(cm.exception))
self.assertIn('shard_shrink_merge_point', str(cm.exception))
def test_init_internal_client_conf_loading_error(self):
with mock.patch('swift.common.db_replicator.ring.Ring') \
@ -350,7 +352,7 @@ class TestSharder(BaseTestSharder):
with self.assertRaises(Exception) as cm:
sharder.run_forever()
self.assertEqual('Test over', cm.exception.message)
self.assertEqual('Test over', str(cm.exception))
# four cycles are started, two brokers visited per cycle, but
# fourth never completes
self.assertEqual(8, len(fake_process_broker_calls))
@ -836,7 +838,8 @@ class TestSharder(BaseTestSharder):
'GET', '/v1/a/c', expected_headers, acceptable_statuses=(2,),
params=params)
params = {'format': 'json', 'end_marker': 'there', 'marker': 'here'}
params = {'format': 'json',
'end_marker': 'there', 'marker': 'here'}
actual, mock_call = do_test(json.dumps([]), params=params)
self._assert_shard_ranges_equal([], actual)
mock_call.assert_called_once_with(
@ -1248,7 +1251,7 @@ class TestSharder(BaseTestSharder):
context = CleavingContext.load(broker)
self.assertTrue(context.misplaced_done)
self.assertFalse(context.cleaving_done)
self.assertEqual(str(shard_ranges[1].upper), context.cursor)
self.assertEqual(shard_ranges[1].upper_str, context.cursor)
self.assertEqual(8, context.cleave_to_row)
self.assertEqual(8, context.max_row)
@ -1281,7 +1284,7 @@ class TestSharder(BaseTestSharder):
context = CleavingContext.load(broker)
self.assertTrue(context.misplaced_done)
self.assertFalse(context.cleaving_done)
self.assertEqual(str(shard_ranges[1].upper), context.cursor)
self.assertEqual(shard_ranges[1].upper_str, context.cursor)
self.assertEqual(8, context.cleave_to_row)
self.assertEqual(8, context.max_row)
@ -1313,7 +1316,7 @@ class TestSharder(BaseTestSharder):
context = CleavingContext.load(broker)
self.assertTrue(context.misplaced_done)
self.assertTrue(context.cleaving_done)
self.assertEqual(str(shard_ranges[2].upper), context.cursor)
self.assertEqual(shard_ranges[2].upper_str, context.cursor)
self.assertEqual(8, context.cleave_to_row)
self.assertEqual(8, context.max_row)
@ -1388,7 +1391,7 @@ class TestSharder(BaseTestSharder):
context = CleavingContext.load(broker)
self.assertFalse(context.misplaced_done)
self.assertFalse(context.cleaving_done)
self.assertEqual(str(shard_ranges[0].upper), context.cursor)
self.assertEqual(shard_ranges[0].upper_str, context.cursor)
self.assertEqual(6, context.cleave_to_row)
self.assertEqual(6, context.max_row)
@ -1433,7 +1436,7 @@ class TestSharder(BaseTestSharder):
context = CleavingContext.load(broker)
self.assertTrue(context.misplaced_done)
self.assertTrue(context.cleaving_done)
self.assertEqual(str(shard_ranges[1].upper), context.cursor)
self.assertEqual(shard_ranges[1].upper_str, context.cursor)
self.assertEqual(6, context.cleave_to_row)
self.assertEqual(6, context.max_row)
@ -1492,7 +1495,7 @@ class TestSharder(BaseTestSharder):
context = CleavingContext.load(broker)
self.assertTrue(context.misplaced_done)
self.assertTrue(context.cleaving_done)
self.assertEqual(str(acceptor.upper), context.cursor)
self.assertEqual(acceptor.upper_str, context.cursor)
self.assertEqual(2, context.cleave_to_row)
self.assertEqual(2, context.max_row)
@ -3058,9 +3061,9 @@ class TestSharder(BaseTestSharder):
sharder._move_misplaced_objects(broker)
sharder._fetch_shard_ranges.assert_has_calls(
[mock.call(broker, newest=True, params={'states': 'updating',
'marker': '',
'end_marker': 'here\x00'}),
[mock.call(broker, newest=True,
params={'states': 'updating',
'marker': '', 'end_marker': 'here\x00'}),
mock.call(broker, newest=True, params={'states': 'updating',
'marker': 'where',
'end_marker': ''})])
@ -3147,12 +3150,12 @@ class TestSharder(BaseTestSharder):
sharder._move_misplaced_objects(broker)
sharder._fetch_shard_ranges.assert_has_calls(
[mock.call(broker, newest=True, params={'states': 'updating',
'marker': '',
'end_marker': 'here\x00'}),
mock.call(broker, newest=True, params={'states': 'updating',
'marker': 'where',
'end_marker': ''})])
[mock.call(broker, newest=True,
params={'states': 'updating',
'marker': '', 'end_marker': 'here\x00'}),
mock.call(broker, newest=True,
params={'states': 'updating',
'marker': 'where', 'end_marker': ''})])
sharder._replicate_object.assert_has_calls(
[mock.call(0, expected_shard_dbs[-1], 0)],
)
@ -3614,7 +3617,7 @@ class TestSharder(BaseTestSharder):
shard_ranges = self._make_shard_ranges((('', 'h'), ('h', '')))
def do_test(replicas, *resp_codes):
sent_data = defaultdict(str)
sent_data = defaultdict(bytes)
def on_send(fake_conn, data):
sent_data[fake_conn] += data
@ -3627,6 +3630,7 @@ class TestSharder(BaseTestSharder):
self.assertEqual(sharder.ring.replica_count, len(conn.requests))
expected_body = json.dumps([dict(sr) for sr in shard_ranges])
expected_body = expected_body.encode('ascii')
expected_headers = {'Content-Type': 'application/json',
'Content-Length': str(len(expected_body)),
'X-Timestamp': now.internal,
@ -4513,11 +4517,28 @@ class TestCleavingContext(BaseTestSharder):
for curs in ('curs', u'curs\u00e4\u00fb'):
with annotate_failure('%r' % curs):
expected = curs.encode('utf-8') if six.PY2 else curs
ctx = CleavingContext(ref, curs, 12, 11, 10, False, True)
self.assertEqual(curs.encode('utf8'), ctx.cursor)
self.assertEqual(dict(ctx), {
'cursor': expected,
'max_row': 12,
'cleave_to_row': 11,
'last_cleave_to_row': 10,
'cleaving_done': False,
'misplaced_done': True,
'ranges_done': 0,
'ranges_todo': 0,
'ref': ref,
})
self.assertEqual(expected, ctx.cursor)
ctx.store(broker)
ctx = CleavingContext.load(broker)
self.assertEqual(curs.encode('utf8'), ctx.cursor)
reloaded_ctx = CleavingContext.load(broker)
self.assertEqual(expected, reloaded_ctx.cursor)
# Since we reloaded, the max row gets updated from the broker
self.assertEqual(reloaded_ctx.max_row, -1)
# reset it so the dict comparison will succeed
reloaded_ctx.max_row = 12
self.assertEqual(dict(ctx), dict(reloaded_ctx))
def test_load(self):
broker = self._make_broker()

View File

@ -86,8 +86,8 @@ class TestContainerSync(unittest.TestCase):
def test_FileLikeIter(self):
# Retained test to show new FileLikeIter acts just like the removed
# _Iter2FileLikeObject did.
flo = sync.FileLikeIter(iter(['123', '4567', '89', '0']))
expect = '1234567890'
flo = sync.FileLikeIter(iter([b'123', b'4567', b'89', b'0']))
expect = b'1234567890'
got = flo.read(2)
self.assertTrue(len(got) <= 2)
@ -100,13 +100,13 @@ class TestContainerSync(unittest.TestCase):
expect = expect[len(got):]
self.assertEqual(flo.read(), expect)
self.assertEqual(flo.read(), '')
self.assertEqual(flo.read(2), '')
self.assertEqual(flo.read(), b'')
self.assertEqual(flo.read(2), b'')
flo = sync.FileLikeIter(iter(['123', '4567', '89', '0']))
self.assertEqual(flo.read(), '1234567890')
self.assertEqual(flo.read(), '')
self.assertEqual(flo.read(2), '')
flo = sync.FileLikeIter(iter([b'123', b'4567', b'89', b'0']))
self.assertEqual(flo.read(), b'1234567890')
self.assertEqual(flo.read(), b'')
self.assertEqual(flo.read(2), b'')
def assertLogMessage(self, msg_level, expected, skip=0):
for line in self.logger.get_lines_for_level(msg_level)[skip:]:
@ -659,7 +659,7 @@ class TestContainerSync(unittest.TestCase):
def fake_hash_path(account, container, obj, raw_digest=False):
# Ensures that no rows match for second loop, ordinal is 0 and
# all hashes are 1
return '\x01' * 16
return b'\x01' * 16
sync.hash_path = fake_hash_path
fcb = FakeContainerBroker(
@ -685,7 +685,7 @@ class TestContainerSync(unittest.TestCase):
def fake_hash_path(account, container, obj, raw_digest=False):
# Ensures that all rows match for second loop, ordinal is 0 and
# all hashes are 0
return '\x00' * 16
return b'\x00' * 16
def fake_delete_object(*args, **kwargs):
pass
@ -787,10 +787,9 @@ class TestContainerSync(unittest.TestCase):
cs._myips = ['10.0.0.0'] # Match
cs._myport = 1000 # Match
cs.allowed_sync_hosts = ['127.0.0.1']
funcType = type(sync.ContainerSync.container_sync_row)
cs.container_sync_row = funcType(fake_container_sync_row,
cs, sync.ContainerSync)
cs.container_sync('isa.db')
with mock.patch.object(cs, 'container_sync_row',
fake_container_sync_row):
cs.container_sync('isa.db')
# Succeeds because no rows match
log_line = cs.logger.get_lines_for_level('info')[0]
lines = log_line.split(',')
@ -954,7 +953,7 @@ class TestContainerSync(unittest.TestCase):
'x-container-sync-key': 'key'})
expected_headers.update(extra_headers)
self.assertDictEqual(expected_headers, headers)
self.assertEqual(contents.read(), 'contents')
self.assertEqual(contents.read(), b'contents')
self.assertEqual(proxy, 'http://proxy')
self.assertEqual(timeout, 5.0)
self.assertEqual(logger, self.logger)
@ -978,7 +977,7 @@ class TestContainerSync(unittest.TestCase):
'etag': '"etagvalue"',
'x-timestamp': timestamp.internal,
'content-type': 'text/plain; swift_bytes=123'},
iter('contents'))
iter([b'contents']))
cs.swift.get_object = fake_get_object
# Success as everything says it worked.
@ -1019,7 +1018,7 @@ class TestContainerSync(unittest.TestCase):
'other-header': 'other header value',
'etag': '"etagvalue"',
'content-type': 'text/plain; swift_bytes=123'},
iter('contents'))
iter([b'contents']))
cs.swift.get_object = fake_get_object
@ -1073,7 +1072,7 @@ class TestContainerSync(unittest.TestCase):
'etag': '"etagvalue"',
'x-static-large-object': 'true',
'content-type': 'text/plain; swift_bytes=123'},
iter('contents'))
iter([b'contents']))
cs.swift.get_object = fake_get_object
@ -1156,7 +1155,7 @@ class TestContainerSync(unittest.TestCase):
return (200, {'other-header': 'other header value',
'x-timestamp': timestamp.internal,
'etag': '"etagvalue"'},
iter('contents'))
iter([b'contents']))
def fake_put_object(*args, **kwargs):
raise ClientException('test client exception', http_status=401)

View File

@ -13,6 +13,7 @@
# See the License for the specific language governing permissions and
# limitations under the License.
import six
import six.moves.cPickle as pickle
import mock
import os
@ -196,7 +197,8 @@ class TestContainerUpdater(unittest.TestCase):
self.assertFalse(log_lines[1:])
self.assertEqual(1, len(mock_dump_recon.mock_calls))
def test_run_once(self):
@mock.patch('swift.container.updater.dump_recon_cache')
def test_run_once(self, mock_recon):
cu = self._get_container_updater()
cu.run_once()
containers_dir = os.path.join(self.sda1, DATADIR)
@ -230,21 +232,21 @@ class TestContainerUpdater(unittest.TestCase):
with Timeout(3):
inc = sock.makefile('rb')
out = sock.makefile('wb')
out.write('HTTP/1.1 %d OK\r\nContent-Length: 0\r\n\r\n' %
out.write(b'HTTP/1.1 %d OK\r\nContent-Length: 0\r\n\r\n' %
return_code)
out.flush()
self.assertEqual(inc.readline(),
'PUT /sda1/0/a/c HTTP/1.1\r\n')
b'PUT /sda1/0/a/c HTTP/1.1\r\n')
headers = {}
line = inc.readline()
while line and line != '\r\n':
headers[line.split(':')[0].lower()] = \
line.split(':')[1].strip()
while line and line != b'\r\n':
headers[line.split(b':')[0].lower()] = \
line.split(b':')[1].strip()
line = inc.readline()
self.assertTrue('x-put-timestamp' in headers)
self.assertTrue('x-delete-timestamp' in headers)
self.assertTrue('x-object-count' in headers)
self.assertTrue('x-bytes-used' in headers)
self.assertIn(b'x-put-timestamp', headers)
self.assertIn(b'x-delete-timestamp', headers)
self.assertIn(b'x-object-count', headers)
self.assertIn(b'x-bytes-used', headers)
except BaseException as err:
import traceback
traceback.print_exc()
@ -303,7 +305,10 @@ class TestContainerUpdater(unittest.TestCase):
cb = ContainerBroker(os.path.join(subdir, 'hash.db'), account='a',
container='\xce\xa9')
cb.initialize(normalize_timestamp(1), 0)
cb.put_object('\xce\xa9', normalize_timestamp(2), 3, 'text/plain',
obj_name = u'\N{GREEK CAPITAL LETTER OMEGA}'
if six.PY2:
obj_name = obj_name.encode('utf-8')
cb.put_object(obj_name, normalize_timestamp(2), 3, 'text/plain',
'68b329da9893e34099c7d8ad5cb9c940')
def accept(sock, addr):
@ -311,7 +316,7 @@ class TestContainerUpdater(unittest.TestCase):
with Timeout(3):
inc = sock.makefile('rb')
out = sock.makefile('wb')
out.write('HTTP/1.1 201 OK\r\nContent-Length: 0\r\n\r\n')
out.write(b'HTTP/1.1 201 OK\r\nContent-Length: 0\r\n\r\n')
out.flush()
inc.read()
except BaseException as err:
@ -388,21 +393,21 @@ class TestContainerUpdater(unittest.TestCase):
with Timeout(3):
inc = sock.makefile('rb')
out = sock.makefile('wb')
out.write('HTTP/1.1 %d OK\r\nContent-Length: 0\r\n\r\n' %
out.write(b'HTTP/1.1 %d OK\r\nContent-Length: 0\r\n\r\n' %
return_code)
out.flush()
self.assertEqual(inc.readline(),
'PUT /sda1/2/.shards_a/c HTTP/1.1\r\n')
b'PUT /sda1/2/.shards_a/c HTTP/1.1\r\n')
headers = {}
line = inc.readline()
while line and line != '\r\n':
headers[line.split(':')[0].lower()] = \
line.split(':')[1].strip()
while line and line != b'\r\n':
headers[line.split(b':')[0].lower()] = \
line.split(b':')[1].strip()
line = inc.readline()
self.assertTrue('x-put-timestamp' in headers)
self.assertTrue('x-delete-timestamp' in headers)
self.assertTrue('x-object-count' in headers)
self.assertTrue('x-bytes-used' in headers)
self.assertIn(b'x-put-timestamp', headers)
self.assertIn(b'x-delete-timestamp', headers)
self.assertIn(b'x-object-count', headers)
self.assertIn(b'x-bytes-used', headers)
except BaseException as err:
import traceback
traceback.print_exc()

13
tox.ini
View File

@ -35,14 +35,7 @@ setenv = VIRTUAL_ENV={envdir}
commands =
nosetests {posargs:\
test/unit/account \
test/unit/cli/test_dispersion_report.py \
test/unit/cli/test_form_signature.py \
test/unit/cli/test_info.py \
test/unit/cli/test_recon.py \
test/unit/cli/test_relinker.py \
test/unit/cli/test_ring_builder_analyzer.py \
test/unit/cli/test_ringbuilder.py \
test/unit/cli/test_ringcomposer.py \
test/unit/cli \
test/unit/common/middleware/crypto \
test/unit/common/middleware/test_account_quotas.py \
test/unit/common/middleware/test_acl.py \
@ -83,9 +76,7 @@ commands =
test/unit/common/test_swob.py \
test/unit/common/test_utils.py \
test/unit/common/test_wsgi.py \
test/unit/container/test_auditor.py \
test/unit/container/test_replicator.py \
test/unit/container/test_sync_store.py \
test/unit/container \
test/unit/obj/test_replicator.py \
test/unit/obj/test_server.py \
test/unit/proxy/controllers/test_base.py \