Add the ability to specify ranges for SLO segments

Users can now include an optional 'range' field in segment descriptions
to specify which bytes from the underlying object should be used for the
segment data. Only one range may be specified per segment. Note that the
'etag' and 'size_bytes' fields still describe the backing object as a
whole. So, if a user uploads a manifest like:

    [{"path": "/con/obj_seg_1", "etag": null, "size_bytes": 1048576,
      "range": "0-1023"},
     {"path": "/con/obj_seg_2", "etag": null, "size_bytes": 1048576,
      "range": "512-4095"},
     {"path": "/con/obj_seg_1", "etag": null, "size_bytes": 1048576,
      "range": "-2048"}]

then the segment will consist of the first 1024 bytes of /con/obj_seg_1,
followed by bytes 513 through 4096 (inclusive) of /con/obj_seg_2, and
finally bytes 1046528 through 1048576 (i.e., the last 2048 bytes) of
/con/obj_seg_1.

ETag generation for SLOs had been updated to prevent collisions when
using different ranges for the same set of objects.

Additionally, there are two performance enhancements:

 * On download, multiple sequential requests for segments from the same
   underlying object will be coalesced into a single ranged request,
   provided it still does not meet Swift's "egregious range requests"
   critieria.

 * On upload, multiple sequential segments referencing the same object
   will be validated against the response from a single HEAD request.

Change-Id: Ia21d51c2cef4e2ee5162161dd2c1d3069009b52c
DocImpact
This commit is contained in:
Tim Burke 2015-08-11 00:42:30 -05:00 committed by John Dickinson
parent a7837c785a
commit 25d5e686a1
5 changed files with 725 additions and 81 deletions

View File

@ -34,17 +34,22 @@ uploaded. The request must be a PUT with the query parameter::
The body of this request will be an ordered list of files in
json data format. The data to be supplied for each segment is::
path: the path to the segment (not including account)
path: the path to the segment object (not including account)
/container/object_name
etag: the etag given back when the segment was PUT, or null
size_bytes: the size of the segment in bytes, or null
etag: the etag given back when the segment object was PUT,
or null
size_bytes: the size of the complete segment object in
bytes, or null
range: (Optional) the range within the object to use as a
segment. If omitted, the entire object is used.
The format of the list will be::
json:
[{"path": "/cont/object",
"etag": "etagoftheobjectsegment",
"size_bytes": 1048576}, ...]
"size_bytes": 10485760,
"range": "1048576-2097151"}, ...]
The number of object segments is limited to a configurable amount, default
1000. Each segment, except for the final one, must be at least 1 megabyte
@ -53,14 +58,16 @@ verify:
1. the segment exists (i.e. the HEAD was successful);
2. the segment meets minimum size requirements (if not the last segment);
3. if the user provided a non-null etag, the etag matches; and
4. if the user provided a non-null size_bytes, the size_bytes matches.
3. if the user provided a non-null etag, the etag matches;
4. if the user provided a non-null size_bytes, the size_bytes matches; and
5. if the user provided a range, it is a singular, syntactically correct range
that is satisfiable given the size of the object.
Note that the etag and size_bytes keys are still required; this acts as a guard
against user errors such as typos. If any of the objects fail to verify (not
found, size/etag mismatch, below minimum size) then the user will receive a 4xx
error response. If everything does match, the user will receive a 2xx response
and the SLO object is ready for downloading.
found, size/etag mismatch, below minimum size, invalid range) then the user
will receive a 4xx error response. If everything does match, the user will
receive a 2xx response and the SLO object is ready for downloading.
Behind the scenes, on success, a json manifest generated from the user input is
sent to object servers with an extra "X-Static-Large-Object: True" header
@ -159,7 +166,7 @@ from swift.common.swob import Request, HTTPBadRequest, HTTPServerError, \
HTTPMethodNotAllowed, HTTPRequestEntityTooLarge, HTTPLengthRequired, \
HTTPOk, HTTPPreconditionFailed, HTTPException, HTTPNotFound, \
HTTPUnauthorized, HTTPConflict, HTTPRequestedRangeNotSatisfiable,\
Response
Response, Range
from swift.common.utils import json, get_logger, config_true_value, \
get_valid_utf8_str, override_bytes_from_content_type, split_path, \
register_swift_info, RateLimitedIterator, quote, close_if_possible, \
@ -189,11 +196,18 @@ def parse_input(raw_data):
raise HTTPBadRequest("Manifest must be valid json.")
req_keys = set(['path', 'etag', 'size_bytes'])
opt_keys = set(['range'])
try:
for seg_dict in parsed_data:
if (set(seg_dict) != req_keys or
if (not (req_keys <= set(seg_dict) <= req_keys | opt_keys) or
'/' not in seg_dict['path'].lstrip('/')):
raise HTTPBadRequest('Invalid SLO Manifest File')
if seg_dict.get('range'):
try:
seg_dict['range'] = Range('bytes=%s' % seg_dict['range'])
except ValueError:
raise HTTPBadRequest('Invalid SLO Manifest File')
except (AttributeError, TypeError):
raise HTTPBadRequest('Invalid SLO Manifest File')
@ -256,6 +270,22 @@ class SloGetContext(WSGIContext):
'ERROR: while fetching %s, JSON-decoding of submanifest %s '
'failed with %s' % (req.path, sub_req.path, err))
def _segment_length(self, seg_dict):
"""
Returns the number of bytes that will be fetched from the specified
segment on a plain GET request for this SLO manifest.
"""
seg_range = seg_dict.get('range')
if seg_range is not None:
# The range is of the form N-M, where N and M are both positive
# decimal integers. We know this because this middleware is the
# only thing that creates the SLO manifests stored in the
# cluster.
range_start, range_end = [int(x) for x in seg_range.split('-')]
return range_end - range_start + 1
else:
return int(seg_dict['bytes'])
def _segment_listing_iterator(self, req, version, account, segments,
recursion_depth=1):
for seg_dict in segments:
@ -265,30 +295,38 @@ class SloGetContext(WSGIContext):
# We handle the range stuff here so that we can be smart about
# skipping unused submanifests. For example, if our first segment is a
# submanifest referencing 50 MiB total, but self.first_byte falls in
# submanifest referencing 50 MiB total, but start_byte falls in
# the 51st MiB, then we can avoid fetching the first submanifest.
#
# If we were to make SegmentedIterable handle all the range
# calculations, we would be unable to make this optimization.
total_length = sum(int(seg['bytes']) for seg in segments)
total_length = sum(self._segment_length(seg) for seg in segments)
if self.first_byte is None:
self.first_byte = 0
if self.last_byte is None:
self.last_byte = total_length - 1
last_sub_path = None
for seg_dict in segments:
seg_length = int(seg_dict['bytes'])
seg_length = self._segment_length(seg_dict)
if self.first_byte >= seg_length:
# don't need any bytes from this segment
self.first_byte = max(self.first_byte - seg_length, -1)
self.last_byte = max(self.last_byte - seg_length, -1)
self.first_byte -= seg_length
self.last_byte -= seg_length
continue
if self.last_byte < 0:
# no bytes are needed from this or any future segment
break
range = seg_dict.get('range')
if range is None:
range_start, range_end = 0, seg_length - 1
else:
# We already validated and supplied concrete values
# for the range on upload
range_start, range_end = map(int, range.split('-'))
if config_true_value(seg_dict.get('sub_slo')):
# do this check here so that we can avoid fetching this last
# manifest before raising the exception
@ -297,22 +335,34 @@ class SloGetContext(WSGIContext):
sub_path = get_valid_utf8_str(seg_dict['name'])
sub_cont, sub_obj = split_path(sub_path, 2, 2, True)
sub_segments = self._fetch_sub_slo_segments(
req, version, account, sub_cont, sub_obj)
if last_sub_path != sub_path:
sub_segments = self._fetch_sub_slo_segments(
req, version, account, sub_cont, sub_obj)
last_sub_path = sub_path
# Use the existing machinery to slice into the sub-SLO.
# This requires that we save off our current state, and
# restore at the other end.
orig_start, orig_end = self.first_byte, self.last_byte
self.first_byte = range_start + max(0, self.first_byte)
self.last_byte = min(range_end, range_start + self.last_byte)
for sub_seg_dict, sb, eb in self._segment_listing_iterator(
req, version, account, sub_segments,
recursion_depth=recursion_depth + 1):
yield sub_seg_dict, sb, eb
# Restore the first/last state
self.first_byte, self.last_byte = orig_start, orig_end
else:
if isinstance(seg_dict['name'], unicode):
seg_dict['name'] = seg_dict['name'].encode("utf-8")
seg_length = int(seg_dict['bytes'])
yield (seg_dict,
(None if self.first_byte <= 0 else self.first_byte),
(None if self.last_byte >=
seg_length - 1 else self.last_byte))
self.first_byte = max(self.first_byte - seg_length, -1)
self.last_byte = max(self.last_byte - seg_length, -1)
max(0, self.first_byte) + range_start,
min(range_end, range_start + self.last_byte))
self.first_byte -= seg_length
self.last_byte -= seg_length
def _need_to_refetch_manifest(self, req):
"""
@ -426,12 +476,15 @@ class SloGetContext(WSGIContext):
etag = md5()
content_length = 0
for seg_dict in segments:
etag.update(seg_dict['hash'])
if seg_dict.get('range'):
etag.update('%s:%s;' % (seg_dict['hash'], seg_dict['range']))
else:
etag.update(seg_dict['hash'])
if config_true_value(seg_dict.get('sub_slo')):
override_bytes_from_content_type(
seg_dict, logger=self.slo.logger)
content_length += int(seg_dict['bytes'])
content_length += self._segment_length(seg_dict)
response_headers = [(h, v) for h, v in resp_headers
if h.lower() not in ('etag', 'content-length')]
@ -599,6 +652,7 @@ class StaticLargeObject(object):
out_content_type = 'text/plain'
data_for_storage = []
slo_etag = md5()
last_obj_path = None
for index, seg_dict in enumerate(parsed_data):
obj_name = seg_dict['path']
if isinstance(obj_name, unicode):
@ -631,21 +685,50 @@ class StaticLargeObject(object):
new_env['CONTENT_LENGTH'] = 0
new_env['HTTP_USER_AGENT'] = \
'%s MultipartPUT' % req.environ.get('HTTP_USER_AGENT')
head_seg_resp = \
Request.blank(obj_path, new_env).get_response(self)
if obj_path != last_obj_path:
last_obj_path = obj_path
head_seg_resp = \
Request.blank(obj_path, new_env).get_response(self)
if head_seg_resp.is_success:
if head_seg_resp.content_length < self.min_segment_size and \
segment_length = head_seg_resp.content_length
if seg_dict.get('range'):
# Since we now know the length, we can normalize the ranges
ranges = seg_dict['range'].ranges_for_length(
head_seg_resp.content_length)
if not ranges:
problem_segments.append([quote(obj_name),
'Unsatisfiable Range'])
elif len(ranges) > 1:
problem_segments.append([quote(obj_name),
'Multiple Ranges'])
elif ranges == [(0, seg_dict['size_bytes'])]:
# Just one range, and it exactly matches the object.
# Why'd we do this again?
seg_dict['range'] = None
segment_length = head_seg_resp.content_length
else:
range = ranges[0]
seg_dict['range'] = '%d-%d' % (range[0], range[1] - 1)
segment_length = range[1] - range[0]
if segment_length < self.min_segment_size and \
index < len(parsed_data) - 1:
raise HTTPBadRequest(
'Each segment, except the last, must be at least '
'%d bytes.' % self.min_segment_size)
total_size += head_seg_resp.content_length
total_size += segment_length
if seg_size is not None and \
seg_size != head_seg_resp.content_length:
problem_segments.append([quote(obj_name), 'Size Mismatch'])
if seg_dict['etag'] is None or \
seg_dict['etag'] == head_seg_resp.etag:
slo_etag.update(head_seg_resp.etag)
if seg_dict.get('range'):
slo_etag.update('%s:%s;' % (head_seg_resp.etag,
seg_dict['range']))
else:
slo_etag.update(head_seg_resp.etag)
else:
problem_segments.append([quote(obj_name), 'Etag Mismatch'])
if head_seg_resp.last_modified:
@ -661,6 +744,9 @@ class StaticLargeObject(object):
'hash': head_seg_resp.etag,
'content_type': head_seg_resp.content_type,
'last_modified': last_modified_formatted}
if seg_dict.get('range'):
seg_data['range'] = seg_dict['range']
if config_true_value(
head_seg_resp.headers.get('X-Static-Large-Object')):
seg_data['sub_slo'] = True

View File

@ -30,9 +30,10 @@ from swift.common.constraints import FORMAT2CONTENT_TYPE
from swift.common.exceptions import ListingIterError, SegmentError
from swift.common.http import is_success
from swift.common.swob import (HTTPBadRequest, HTTPNotAcceptable,
HTTPServiceUnavailable)
HTTPServiceUnavailable, Range)
from swift.common.utils import split_path, validate_device_partition, \
close_if_possible
close_if_possible, maybe_multipart_byteranges_to_document_iters
from swift.common.wsgi import make_subrequest
@ -286,13 +287,17 @@ class SegmentedIterable(object):
self.validated_first_segment = False
self.current_resp = None
def _internal_iter(self):
def _coalesce_requests(self):
start_time = time.time()
bytes_left = self.response_body_length
pending_req = None
pending_etag = None
pending_size = None
try:
for seg_path, seg_etag, seg_size, first_byte, last_byte \
in self.listing_iter:
first_byte = first_byte or 0
go_to_end = last_byte is None or (
seg_size is not None and last_byte == seg_size - 1)
if time.time() - start_time > self.max_get_time:
raise SegmentError(
'ERROR: While processing manifest %s, '
@ -307,20 +312,51 @@ class SegmentedIterable(object):
'x-auth-token')},
agent=('%(orig)s ' + self.ua_suffix),
swift_source=self.swift_source)
if first_byte is not None or last_byte is not None:
seg_req.headers['Range'] = "bytes=%s-%s" % (
# The 0 is to avoid having a range like "bytes=-10",
# which actually means the *last* 10 bytes.
'0' if first_byte is None else first_byte,
'' if last_byte is None else last_byte)
if first_byte != 0 or not go_to_end:
seg_req.headers['Range'] = "bytes=%s-%s" % (
first_byte, '' if go_to_end else last_byte)
# We can only coalesce if paths match and we know the segment
# size (so we can check that the ranges will be allowed)
if pending_req and pending_req.path == seg_req.path and \
seg_size is not None:
new_range = '%s,%s' % (
pending_req.headers.get('Range',
'bytes=0-%s' % (seg_size - 1)),
seg_req.headers['Range'].split('bytes=')[1])
if Range(new_range).ranges_for_length(seg_size):
# Good news! We can coalesce the requests
pending_req.headers['Range'] = new_range
continue
# else, Too many ranges, or too much backtracking, or ...
if pending_req:
yield pending_req, pending_etag, pending_size
pending_req = seg_req
pending_etag = seg_etag
pending_size = seg_size
finally:
if time.time() - start_time > self.max_get_time:
raise SegmentError(
'ERROR: While processing manifest %s, '
'max LO GET time of %ds exceeded' %
(self.name, self.max_get_time))
if pending_req:
yield pending_req, pending_etag, pending_size
def _internal_iter(self):
bytes_left = self.response_body_length
try:
for seg_req, seg_etag, seg_size in self._coalesce_requests():
seg_resp = seg_req.get_response(self.app)
if not is_success(seg_resp.status_int):
close_if_possible(seg_resp.app_iter)
raise SegmentError(
'ERROR: While processing manifest %s, '
'got %d while retrieving %s' %
(self.name, seg_resp.status_int, seg_path))
(self.name, seg_resp.status_int, seg_req.path))
elif ((seg_etag and (seg_resp.etag != seg_etag)) or
(seg_size and (seg_resp.content_length != seg_size) and
@ -344,9 +380,19 @@ class SegmentedIterable(object):
else:
self.current_resp = seg_resp
seg_hash = hashlib.md5()
for chunk in seg_resp.app_iter:
seg_hash.update(chunk)
seg_hash = None
if seg_resp.etag and not seg_req.headers.get('Range'):
# Only calculate the MD5 if it we can use it to validate
seg_hash = hashlib.md5()
document_iters = maybe_multipart_byteranges_to_document_iters(
seg_resp.app_iter,
seg_resp.headers['Content-Type'])
for chunk in itertools.chain.from_iterable(document_iters):
if seg_hash:
seg_hash.update(chunk)
if bytes_left is None:
yield chunk
elif bytes_left >= len(chunk):
@ -363,8 +409,7 @@ class SegmentedIterable(object):
'left': bytes_left})
close_if_possible(seg_resp.app_iter)
if seg_resp.etag and seg_hash.hexdigest() != seg_resp.etag \
and first_byte is None and last_byte is None:
if seg_hash and seg_hash.hexdigest() != seg_resp.etag:
raise SegmentError(
"Bad MD5 checksum in %(name)s for %(seg)s: headers had"
" %(etag)s, but object MD5 was actually %(actual)s" %

View File

@ -3436,6 +3436,27 @@ def mime_to_document_iters(input_file, boundary, read_chunk_size=4096):
yield (headers, doc_file)
def maybe_multipart_byteranges_to_document_iters(app_iter, content_type):
"""
Takes an iterator that may or may not contain a multipart MIME document
as well as content type and returns an iterator of body iterators.
:param app_iter: iterator that may contain a multipart MIME document
:param content_type: content type of the app_iter, used to determine
whether it conains a multipart document and, if
so, what the boundary is between documents
"""
content_type, params_list = parse_content_type(content_type)
if content_type != 'multipart/byteranges':
yield app_iter
return
body_file = FileLikeIter(app_iter)
boundary = dict(params_list)['boundary']
for _headers, body in mime_to_document_iters(body_file, boundary):
yield (chunk for chunk in iter(lambda: body.read(65536), ''))
def document_iters_to_multipart_byteranges(ranges_iter, boundary):
"""
Takes an iterator of range iters and yields a multipart/byteranges MIME

View File

@ -17,6 +17,7 @@
from datetime import datetime
import hashlib
import hmac
import itertools
import json
import locale
import random
@ -2137,7 +2138,7 @@ class TestSloEnv(object):
if not cls.container.create():
raise ResponseError(cls.conn.response)
seg_info = {}
cls.seg_info = seg_info = {}
for letter, size in (('a', 1024 * 1024),
('b', 1024 * 1024),
('c', 1024 * 1024),
@ -2188,7 +2189,59 @@ class TestSloEnv(object):
'manifest-bcd-submanifest')},
seg_info['seg_e']]),
parms={'multipart-manifest': 'put'})
cls.seg_info = seg_info
abcde_submanifest_etag = hashlib.md5(
seg_info['seg_a']['etag'] + bcd_submanifest_etag +
seg_info['seg_e']['etag']).hexdigest()
abcde_submanifest_size = (seg_info['seg_a']['size_bytes'] +
seg_info['seg_b']['size_bytes'] +
seg_info['seg_c']['size_bytes'] +
seg_info['seg_d']['size_bytes'] +
seg_info['seg_e']['size_bytes'])
file_item = cls.container.file("ranged-manifest")
file_item.write(
json.dumps([
{'etag': abcde_submanifest_etag,
'size_bytes': abcde_submanifest_size,
'path': '/%s/%s' % (cls.container.name,
'manifest-abcde-submanifest'),
'range': '-1048578'}, # 'c' + ('d' * 2**20) + 'e'
{'etag': abcde_submanifest_etag,
'size_bytes': abcde_submanifest_size,
'path': '/%s/%s' % (cls.container.name,
'manifest-abcde-submanifest'),
'range': '524288-1572863'}, # 'a' * 2**19 + 'b' * 2**19
{'etag': abcde_submanifest_etag,
'size_bytes': abcde_submanifest_size,
'path': '/%s/%s' % (cls.container.name,
'manifest-abcde-submanifest'),
'range': '3145727-3145728'}]), # 'cd'
parms={'multipart-manifest': 'put'})
ranged_manifest_etag = hashlib.md5(
abcde_submanifest_etag + ':3145727-4194304;' +
abcde_submanifest_etag + ':524288-1572863;' +
abcde_submanifest_etag + ':3145727-3145728;').hexdigest()
ranged_manifest_size = 2 * 1024 * 1024 + 4
file_item = cls.container.file("ranged-submanifest")
file_item.write(
json.dumps([
seg_info['seg_c'],
{'etag': ranged_manifest_etag,
'size_bytes': ranged_manifest_size,
'path': '/%s/%s' % (cls.container.name,
'ranged-manifest')},
{'etag': ranged_manifest_etag,
'size_bytes': ranged_manifest_size,
'path': '/%s/%s' % (cls.container.name,
'ranged-manifest'),
'range': '524289-1572865'},
{'etag': ranged_manifest_etag,
'size_bytes': ranged_manifest_size,
'path': '/%s/%s' % (cls.container.name,
'ranged-manifest'),
'range': '-3'}]),
parms={'multipart-manifest': 'put'})
file_item = cls.container.file("manifest-db")
file_item.write(
@ -2234,6 +2287,39 @@ class TestSlo(Base):
self.assertEqual('d', file_contents[-2])
self.assertEqual('e', file_contents[-1])
def test_slo_get_ranged_manifest(self):
file_item = self.env.container.file('ranged-manifest')
grouped_file_contents = [
(char, sum(1 for _char in grp))
for char, grp in itertools.groupby(file_item.read())]
self.assertEqual([
('c', 1),
('d', 1024 * 1024),
('e', 1),
('a', 512 * 1024),
('b', 512 * 1024),
('c', 1),
('d', 1)], grouped_file_contents)
def test_slo_get_ranged_submanifest(self):
file_item = self.env.container.file('ranged-submanifest')
grouped_file_contents = [
(char, sum(1 for _char in grp))
for char, grp in itertools.groupby(file_item.read())]
self.assertEqual([
('c', 1024 * 1024 + 1),
('d', 1024 * 1024),
('e', 1),
('a', 512 * 1024),
('b', 512 * 1024),
('c', 1),
('d', 512 * 1024 + 1),
('e', 1),
('a', 512 * 1024),
('b', 1),
('c', 1),
('d', 1)], grouped_file_contents)
def test_slo_ranged_get(self):
file_item = self.env.container.file('manifest-abcde')
file_contents = file_item.read(size=1024 * 1024 + 2,

View File

@ -121,12 +121,27 @@ class TestSloMiddleware(SloTestCase):
def test_parse_input(self):
self.assertRaises(HTTPException, slo.parse_input, 'some non json')
self.assertRaises(HTTPException, slo.parse_input, '[{}]')
self.assertRaises(HTTPException, slo.parse_input, json.dumps(
[{'path': '/cont/object', 'etag': 'etagoftheobjecitsegment',
'size_bytes': 100, 'foo': 'bar'}]))
self.assertRaises(HTTPException, slo.parse_input, json.dumps(
[{'path': '/cont/object', 'etag': 'etagoftheobjecitsegment',
'size_bytes': 100, 'range': 'non-range value'}]))
data = json.dumps(
[{'path': '/cont/object', 'etag': 'etagoftheobjecitsegment',
'size_bytes': 100}])
self.assertEquals('/cont/object',
slo.parse_input(data)[0]['path'])
data = json.dumps(
[{'path': '/cont/object', 'etag': 'etagoftheobjecitsegment',
'size_bytes': 100, 'range': '0-40,30-90'}])
parsed = slo.parse_input(data)
self.assertEquals('/cont/object', parsed[0]['path'])
self.assertEquals([(0, 40), (30, 90)], parsed[0]['range'].ranges)
class TestSloPutManifest(SloTestCase):
@ -501,6 +516,81 @@ class TestSloPutManifest(SloTestCase):
self.assertEquals('a', manifest_data[0]['hash'])
self.assertEquals('b', manifest_data[1]['hash'])
def test_handle_unsatisfiable_ranges(self):
bad_data = json.dumps(
[{'path': '/checktest/a_1', 'etag': None,
'size_bytes': None, 'range': '1-'}])
req = Request.blank(
'/v1/AUTH_test/checktest/man_3?multipart-manifest=put',
environ={'REQUEST_METHOD': 'PUT'}, body=bad_data)
with self.assertRaises(HTTPException) as catcher:
self.slo.handle_multipart_put(req, fake_start_response)
self.assertEqual(400, catcher.exception.status_int)
def test_handle_single_ranges(self):
good_data = json.dumps(
[{'path': '/checktest/a_1', 'etag': None,
'size_bytes': 1, 'range': '0-0'},
{'path': '/checktest/b_2', 'etag': None,
'size_bytes': 2, 'range': '-1'},
{'path': '/checktest/b_2', 'etag': None,
'size_bytes': 2, 'range': '0-0'},
{'path': '/cont/object', 'etag': None,
'size_bytes': None, 'range': '10-40'}])
req = Request.blank(
'/v1/AUTH_test/checktest/man_3?multipart-manifest=put',
environ={'REQUEST_METHOD': 'PUT'}, body=good_data)
status, headers, body = self.call_slo(req)
expected_etag = '"%s"' % md5('ab:1-1;b:0-0;etagoftheobjectsegment:'
'10-40;').hexdigest()
self.assertEqual(expected_etag, dict(headers)['Etag'])
self.assertEqual([
('HEAD', '/v1/AUTH_test/checktest/a_1'),
('HEAD', '/v1/AUTH_test/checktest/b_2'), # Only once!
('HEAD', '/v1/AUTH_test/cont/object'),
('PUT', '/v1/AUTH_test/checktest/man_3?multipart-manifest=put'),
], self.app.calls)
# Check that we still populated the manifest properly from our HEADs
req = Request.blank(
# this string looks weird, but it's just an artifact
# of FakeSwift
'/v1/AUTH_test/checktest/man_3?multipart-manifest=put',
environ={'REQUEST_METHOD': 'GET'})
status, headers, body = self.call_app(req)
manifest_data = json.loads(body)
self.assertEquals('a', manifest_data[0]['hash'])
self.assertNotIn('range', manifest_data[0])
self.assertNotIn('segment_bytes', manifest_data[0])
self.assertEquals('b', manifest_data[1]['hash'])
self.assertEquals('1-1', manifest_data[1]['range'])
self.assertEquals('b', manifest_data[2]['hash'])
self.assertEquals('0-0', manifest_data[2]['range'])
self.assertEquals('etagoftheobjectsegment', manifest_data[3]['hash'])
self.assertEquals('10-40', manifest_data[3]['range'])
def test_handle_multiple_ranges_error(self):
good_data = json.dumps(
[{'path': '/checktest/a_1', 'etag': None,
'size_bytes': 1, 'range': '0-100'},
{'path': '/checktest/b_2', 'etag': None,
'size_bytes': 2, 'range': '-1,0-0'},
{'path': '/cont/object', 'etag': None,
'size_bytes': None, 'range': '10-30,20-40'}])
req = Request.blank(
'/v1/AUTH_test/checktest/man_3?multipart-manifest=put',
environ={'REQUEST_METHOD': 'PUT'}, body=good_data)
status, headers, body = self.call_slo(req)
self.assertEqual(status, '400 Bad Request')
self.assertEquals(self.app.call_count, 3)
self.assertEqual(body, '\n'.join([
'Errors:',
'/checktest/b_2, Multiple Ranges',
'/cont/object, Multiple Ranges']))
class TestSloDeleteManifest(SloTestCase):
@ -891,12 +981,6 @@ class TestSloGetManifest(SloTestCase):
def setUp(self):
super(TestSloGetManifest, self).setUp()
_bc_manifest_json = json.dumps(
[{'name': '/gettest/b_10', 'hash': md5hex('b' * 10), 'bytes': '10',
'content_type': 'text/plain'},
{'name': '/gettest/c_15', 'hash': md5hex('c' * 15), 'bytes': '15',
'content_type': 'text/plain'}])
# some plain old objects
self.app.register(
'GET', '/v1/AUTH_test/gettest/a_5',
@ -919,6 +1003,11 @@ class TestSloGetManifest(SloTestCase):
'Etag': md5hex('d' * 20)},
'd' * 20)
_bc_manifest_json = json.dumps(
[{'name': '/gettest/b_10', 'hash': md5hex('b' * 10), 'bytes': '10',
'content_type': 'text/plain'},
{'name': '/gettest/c_15', 'hash': md5hex('c' * 15), 'bytes': '15',
'content_type': 'text/plain'}])
self.app.register(
'GET', '/v1/AUTH_test/gettest/manifest-bc',
swob.HTTPOk, {'Content-Type': 'application/json;swift_bytes=25',
@ -947,6 +1036,86 @@ class TestSloGetManifest(SloTestCase):
md5hex("a" * 5) + md5hex(md5hex("b" * 10) + md5hex("c" * 15)) +
md5hex("d" * 20))
_bc_ranges_manifest_json = json.dumps(
[{'name': '/gettest/b_10', 'hash': md5hex('b' * 10),
'content_type': 'text/plain', 'bytes': '10',
'range': '4-7'},
{'name': '/gettest/b_10', 'hash': md5hex('b' * 10),
'content_type': 'text/plain', 'bytes': '10',
'range': '2-5'},
{'name': '/gettest/c_15', 'hash': md5hex('c' * 15),
'content_type': 'text/plain', 'bytes': '15',
'range': '0-3'},
{'name': '/gettest/c_15', 'hash': md5hex('c' * 15),
'content_type': 'text/plain', 'bytes': '15',
'range': '11-14'}])
self.bc_ranges_etag = md5hex(_bc_ranges_manifest_json)
self.app.register(
'GET', '/v1/AUTH_test/gettest/manifest-bc-ranges',
swob.HTTPOk, {'Content-Type': 'application/json;swift_bytes=16',
'X-Static-Large-Object': 'true',
'X-Object-Meta-Plant': 'Ficus',
'Etag': self.bc_ranges_etag},
_bc_ranges_manifest_json)
_abcd_ranges_manifest_json = json.dumps(
[{'name': '/gettest/a_5', 'hash': md5hex("a" * 5),
'content_type': 'text/plain', 'bytes': '5',
'range': '0-3'},
{'name': '/gettest/a_5', 'hash': md5hex("a" * 5),
'content_type': 'text/plain', 'bytes': '5',
'range': '1-4'},
{'name': '/gettest/manifest-bc-ranges', 'sub_slo': True,
'content_type': 'application/json;swift_bytes=16',
'hash': self.bc_ranges_etag,
'bytes': len(_bc_ranges_manifest_json),
'range': '8-15'},
{'name': '/gettest/manifest-bc-ranges', 'sub_slo': True,
'content_type': 'application/json;swift_bytes=16',
'hash': self.bc_ranges_etag,
'bytes': len(_bc_ranges_manifest_json),
'range': '0-7'},
{'name': '/gettest/d_20', 'hash': md5hex("d" * 20),
'content_type': 'text/plain', 'bytes': '20',
'range': '0-3'},
{'name': '/gettest/d_20', 'hash': md5hex("d" * 20),
'content_type': 'text/plain', 'bytes': '20',
'range': '8-11'}])
self.app.register(
'GET', '/v1/AUTH_test/gettest/manifest-abcd-ranges',
swob.HTTPOk, {'Content-Type': 'application/json',
'X-Static-Large-Object': 'true',
'Etag': md5hex(_abcd_ranges_manifest_json)},
_abcd_ranges_manifest_json)
_abcd_subranges_manifest_json = json.dumps(
[{'name': '/gettest/manifest-abcd-ranges', 'sub_slo': True,
'hash': md5hex("a" * 8),
'content_type': 'text/plain', 'bytes': '32',
'range': '6-10'},
{'name': '/gettest/manifest-abcd-ranges', 'sub_slo': True,
'hash': md5hex("a" * 8),
'content_type': 'text/plain', 'bytes': '32',
'range': '31-31'},
{'name': '/gettest/manifest-abcd-ranges', 'sub_slo': True,
'hash': md5hex("a" * 8),
'content_type': 'text/plain', 'bytes': '32',
'range': '14-18'},
{'name': '/gettest/manifest-abcd-ranges', 'sub_slo': True,
'hash': md5hex("a" * 8),
'content_type': 'text/plain', 'bytes': '32',
'range': '0-0'},
{'name': '/gettest/manifest-abcd-ranges', 'sub_slo': True,
'hash': md5hex("a" * 8),
'content_type': 'text/plain', 'bytes': '32',
'range': '22-26'}])
self.app.register(
'GET', '/v1/AUTH_test/gettest/manifest-abcd-subranges',
swob.HTTPOk, {'Content-Type': 'application/json',
'X-Static-Large-Object': 'true',
'Etag': md5hex(_abcd_subranges_manifest_json)},
_abcd_subranges_manifest_json)
self.app.register(
'GET', '/v1/AUTH_test/gettest/manifest-badjson',
swob.HTTPOk, {'Content-Type': 'application/json',
@ -1104,21 +1273,23 @@ class TestSloGetManifest(SloTestCase):
self.app.calls,
[('GET', '/v1/AUTH_test/gettest/manifest-abcd'),
('GET', '/v1/AUTH_test/gettest/manifest-abcd'),
('GET', '/v1/AUTH_test/gettest/a_5?multipart-manifest=get'),
('GET', '/v1/AUTH_test/gettest/manifest-bc'),
('GET', '/v1/AUTH_test/gettest/a_5?multipart-manifest=get'),
('GET', '/v1/AUTH_test/gettest/b_10?multipart-manifest=get'),
('GET', '/v1/AUTH_test/gettest/c_15?multipart-manifest=get')])
headers = [c[2] for c in self.app.calls_with_headers]
self.assertEqual(headers[0].get('Range'), 'bytes=3-17')
self.assertEqual(headers[1].get('Range'), None)
self.assertEqual(headers[2].get('Range'), 'bytes=3-')
self.assertEqual(headers[3].get('Range'), None)
self.assertEqual(headers[4].get('Range'), None)
self.assertEqual(headers[5].get('Range'), 'bytes=0-2')
ranges = [c[2].get('Range') for c in self.app.calls_with_headers]
self.assertEqual(ranges, [
'bytes=3-17',
None,
None,
'bytes=3-',
None,
'bytes=0-2'])
# we set swift.source for everything but the first request
self.assertEqual(self.app.swift_sources,
[None, 'SLO', 'SLO', 'SLO', 'SLO', 'SLO'])
self.assertIsNone(self.app.swift_sources[0])
self.assertEqual(self.app.swift_sources[1:],
['SLO'] * (len(self.app.swift_sources) - 1))
def test_range_get_includes_whole_manifest(self):
# If the first range GET results in retrieval of the entire manifest
@ -1139,8 +1310,8 @@ class TestSloGetManifest(SloTestCase):
self.assertEqual(
self.app.calls,
[('GET', '/v1/AUTH_test/gettest/manifest-abcd'),
('GET', '/v1/AUTH_test/gettest/a_5?multipart-manifest=get'),
('GET', '/v1/AUTH_test/gettest/manifest-bc'),
('GET', '/v1/AUTH_test/gettest/a_5?multipart-manifest=get'),
('GET', '/v1/AUTH_test/gettest/b_10?multipart-manifest=get'),
('GET', '/v1/AUTH_test/gettest/c_15?multipart-manifest=get'),
('GET', '/v1/AUTH_test/gettest/d_20?multipart-manifest=get')])
@ -1170,7 +1341,9 @@ class TestSloGetManifest(SloTestCase):
headers = swob.HeaderKeyDict(headers)
self.assertEqual(status, '206 Partial Content')
self.assertEqual(body, 'e' * 100000)
count_e = sum(1 if x == 'e' else 0 for x in body)
self.assertEqual(count_e, 100000)
self.assertEqual(len(body) - count_e, 0)
self.assertEqual(
self.app.calls, [
@ -1212,8 +1385,8 @@ class TestSloGetManifest(SloTestCase):
self.app.calls,
[('GET', '/v1/AUTH_test/gettest/manifest-abcd'),
('GET', '/v1/AUTH_test/gettest/manifest-abcd'),
('GET', '/v1/AUTH_test/gettest/a_5?multipart-manifest=get'),
('GET', '/v1/AUTH_test/gettest/manifest-bc'),
('GET', '/v1/AUTH_test/gettest/a_5?multipart-manifest=get'),
('GET', '/v1/AUTH_test/gettest/b_10?multipart-manifest=get'),
('GET', '/v1/AUTH_test/gettest/c_15?multipart-manifest=get'),
('GET', '/v1/AUTH_test/gettest/d_20?multipart-manifest=get')])
@ -1349,6 +1522,237 @@ class TestSloGetManifest(SloTestCase):
self.assertEqual(status, '200 OK')
self.assertEqual(body, segment_body)
def test_get_range_manifest(self):
req = Request.blank(
'/v1/AUTH_test/gettest/manifest-abcd-ranges',
environ={'REQUEST_METHOD': 'GET'})
status, headers, body = self.call_slo(req)
headers = swob.HeaderKeyDict(headers)
self.assertEqual(status, '200 OK')
self.assertEqual(headers['Content-Length'], '32')
self.assertEqual(headers['Content-Type'], 'application/json')
self.assertEqual(body, 'aaaaaaaaccccccccbbbbbbbbdddddddd')
self.assertEqual(
self.app.calls,
[('GET', '/v1/AUTH_test/gettest/manifest-abcd-ranges'),
('GET', '/v1/AUTH_test/gettest/manifest-bc-ranges'),
('GET', '/v1/AUTH_test/gettest/a_5?multipart-manifest=get'),
('GET', '/v1/AUTH_test/gettest/c_15?multipart-manifest=get'),
('GET', '/v1/AUTH_test/gettest/b_10?multipart-manifest=get'),
('GET', '/v1/AUTH_test/gettest/d_20?multipart-manifest=get')])
ranges = [c[2].get('Range') for c in self.app.calls_with_headers]
self.assertEqual(ranges, [
None,
None,
'bytes=0-3,1-',
'bytes=0-3,11-',
'bytes=4-7,2-5',
'bytes=0-3,8-11'])
# we set swift.source for everything but the first request
self.assertIsNone(self.app.swift_sources[0])
self.assertEqual(self.app.swift_sources[1:],
['SLO'] * (len(self.app.swift_sources) - 1))
self.assertEqual(md5hex(''.join([
md5hex('a' * 5), ':0-3;',
md5hex('a' * 5), ':1-4;',
self.bc_ranges_etag, ':8-15;',
self.bc_ranges_etag, ':0-7;',
md5hex('d' * 20), ':0-3;',
md5hex('d' * 20), ':8-11;',
])), headers['Etag'].strip('"'))
def test_get_subrange_manifest(self):
req = Request.blank(
'/v1/AUTH_test/gettest/manifest-abcd-subranges',
environ={'REQUEST_METHOD': 'GET'})
status, headers, body = self.call_slo(req)
headers = swob.HeaderKeyDict(headers)
self.assertEqual(status, '200 OK')
self.assertEqual(headers['Content-Length'], '17')
self.assertEqual(headers['Content-Type'], 'application/json')
self.assertEqual(body, 'aacccdccbbbabbddd')
self.assertEqual(
self.app.calls,
[('GET', '/v1/AUTH_test/gettest/manifest-abcd-subranges'),
('GET', '/v1/AUTH_test/gettest/manifest-abcd-ranges'),
('GET', '/v1/AUTH_test/gettest/manifest-bc-ranges'),
('GET', '/v1/AUTH_test/gettest/a_5?multipart-manifest=get'),
('GET', '/v1/AUTH_test/gettest/c_15?multipart-manifest=get'),
('GET', '/v1/AUTH_test/gettest/manifest-bc-ranges'),
('GET', '/v1/AUTH_test/gettest/d_20?multipart-manifest=get'),
('GET', '/v1/AUTH_test/gettest/c_15?multipart-manifest=get'),
('GET', '/v1/AUTH_test/gettest/b_10?multipart-manifest=get'),
('GET', '/v1/AUTH_test/gettest/manifest-bc-ranges'),
('GET', '/v1/AUTH_test/gettest/a_5?multipart-manifest=get'),
('GET', '/v1/AUTH_test/gettest/b_10?multipart-manifest=get'),
('GET', '/v1/AUTH_test/gettest/d_20?multipart-manifest=get')])
ranges = [c[2].get('Range') for c in self.app.calls_with_headers]
self.assertEqual(ranges, [
None,
None,
None,
'bytes=3-',
'bytes=0-2',
None,
'bytes=11-11',
'bytes=13-',
'bytes=4-6',
None,
'bytes=0-0',
'bytes=4-5',
'bytes=0-2'])
# we set swift.source for everything but the first request
self.assertIsNone(self.app.swift_sources[0])
self.assertEqual(self.app.swift_sources[1:],
['SLO'] * (len(self.app.swift_sources) - 1))
def test_range_get_range_manifest(self):
req = Request.blank(
'/v1/AUTH_test/gettest/manifest-abcd-ranges',
environ={'REQUEST_METHOD': 'GET'},
headers={'Range': 'bytes=7-26'})
status, headers, body = self.call_slo(req)
headers = swob.HeaderKeyDict(headers)
self.assertEqual(status, '206 Partial Content')
self.assertEqual(headers['Content-Length'], '20')
self.assertEqual(headers['Content-Type'], 'application/json')
self.assertNotIn('Etag', headers)
self.assertEqual(body, 'accccccccbbbbbbbbddd')
self.assertEqual(
self.app.calls,
[('GET', '/v1/AUTH_test/gettest/manifest-abcd-ranges'),
('GET', '/v1/AUTH_test/gettest/manifest-abcd-ranges'),
('GET', '/v1/AUTH_test/gettest/manifest-bc-ranges'),
('GET', '/v1/AUTH_test/gettest/a_5?multipart-manifest=get'),
('GET', '/v1/AUTH_test/gettest/c_15?multipart-manifest=get'),
('GET', '/v1/AUTH_test/gettest/b_10?multipart-manifest=get'),
('GET', '/v1/AUTH_test/gettest/d_20?multipart-manifest=get')])
ranges = [c[2].get('Range') for c in self.app.calls_with_headers]
self.assertEqual(ranges, [
'bytes=7-26',
None,
None,
'bytes=4-',
'bytes=0-3,11-',
'bytes=4-7,2-5',
'bytes=0-2'])
# we set swift.source for everything but the first request
self.assertIsNone(self.app.swift_sources[0])
self.assertEqual(self.app.swift_sources[1:],
['SLO'] * (len(self.app.swift_sources) - 1))
def test_range_get_subrange_manifest(self):
req = Request.blank(
'/v1/AUTH_test/gettest/manifest-abcd-subranges',
environ={'REQUEST_METHOD': 'GET'},
headers={'Range': 'bytes=4-12'})
status, headers, body = self.call_slo(req)
headers = swob.HeaderKeyDict(headers)
self.assertEqual(status, '206 Partial Content')
self.assertEqual(headers['Content-Length'], '9')
self.assertEqual(headers['Content-Type'], 'application/json')
self.assertEqual(body, 'cdccbbbab')
self.assertEqual(
self.app.calls,
[('GET', '/v1/AUTH_test/gettest/manifest-abcd-subranges'),
('GET', '/v1/AUTH_test/gettest/manifest-abcd-subranges'),
('GET', '/v1/AUTH_test/gettest/manifest-abcd-ranges'),
('GET', '/v1/AUTH_test/gettest/manifest-bc-ranges'),
('GET', '/v1/AUTH_test/gettest/c_15?multipart-manifest=get'),
('GET', '/v1/AUTH_test/gettest/manifest-bc-ranges'),
('GET', '/v1/AUTH_test/gettest/d_20?multipart-manifest=get'),
('GET', '/v1/AUTH_test/gettest/c_15?multipart-manifest=get'),
('GET', '/v1/AUTH_test/gettest/b_10?multipart-manifest=get'),
('GET', '/v1/AUTH_test/gettest/manifest-bc-ranges'),
('GET', '/v1/AUTH_test/gettest/a_5?multipart-manifest=get'),
('GET', '/v1/AUTH_test/gettest/b_10?multipart-manifest=get')])
ranges = [c[2].get('Range') for c in self.app.calls_with_headers]
self.assertEqual(ranges, [
'bytes=4-12',
None,
None,
None,
'bytes=2-2',
None,
'bytes=11-11',
'bytes=13-',
'bytes=4-6',
None,
'bytes=0-0',
'bytes=4-4'])
# we set swift.source for everything but the first request
self.assertIsNone(self.app.swift_sources[0])
self.assertEqual(self.app.swift_sources[1:],
['SLO'] * (len(self.app.swift_sources) - 1))
def test_range_get_includes_whole_range_manifest(self):
# If the first range GET results in retrieval of the entire manifest
# body (which we can detect by looking at Content-Range), then we
# should not go make a second, non-ranged request just to retrieve the
# same bytes again.
req = Request.blank(
'/v1/AUTH_test/gettest/manifest-abcd-ranges',
environ={'REQUEST_METHOD': 'GET'},
headers={'Range': 'bytes=0-999999999'})
status, headers, body = self.call_slo(req)
headers = swob.HeaderKeyDict(headers)
self.assertEqual(status, '206 Partial Content')
self.assertEqual(headers['Content-Length'], '32')
self.assertEqual(headers['Content-Type'], 'application/json')
self.assertEqual(body, 'aaaaaaaaccccccccbbbbbbbbdddddddd')
self.assertEqual(
self.app.calls,
[('GET', '/v1/AUTH_test/gettest/manifest-abcd-ranges'),
('GET', '/v1/AUTH_test/gettest/manifest-bc-ranges'),
('GET', '/v1/AUTH_test/gettest/a_5?multipart-manifest=get'),
('GET', '/v1/AUTH_test/gettest/c_15?multipart-manifest=get'),
('GET', '/v1/AUTH_test/gettest/b_10?multipart-manifest=get'),
('GET', '/v1/AUTH_test/gettest/d_20?multipart-manifest=get')])
ranges = [c[2].get('Range') for c in self.app.calls_with_headers]
self.assertEqual(ranges, [
'bytes=0-999999999',
None,
'bytes=0-3,1-',
'bytes=0-3,11-',
'bytes=4-7,2-5',
'bytes=0-3,8-11'])
# we set swift.source for everything but the first request
self.assertIsNone(self.app.swift_sources[0])
self.assertEqual(self.app.swift_sources[1:],
['SLO'] * (len(self.app.swift_sources) - 1))
def test_multi_range_get_range_manifest(self):
# SLO doesn't support multi-range GETs. The way that you express
# "unsupported" in HTTP is to return a 200 and the whole entity.
req = Request.blank(
'/v1/AUTH_test/gettest/manifest-abcd-ranges',
environ={'REQUEST_METHOD': 'GET'},
headers={'Range': 'bytes=0-0,2-2'})
status, headers, body = self.call_slo(req)
headers = swob.HeaderKeyDict(headers)
self.assertEqual(status, '200 OK')
self.assertEqual(headers['Content-Type'], 'application/json')
self.assertEqual(body, 'aaaaaaaaccccccccbbbbbbbbdddddddd')
self.assertNotIn('Transfer-Encoding', headers)
self.assertNotIn('Content-Range', headers)
self.assertEqual(headers['Content-Length'], '32')
def test_get_bogus_manifest(self):
req = Request.blank(
'/v1/AUTH_test/gettest/manifest-badjson',
@ -1426,7 +1830,7 @@ class TestSloGetManifest(SloTestCase):
status, headers, body, exc = self.call_slo(req, expect_exception=True)
headers = swob.HeaderKeyDict(headers)
self.assertTrue(isinstance(exc, ListingIterError))
self.assertIsInstance(exc, ListingIterError)
# we don't know at header-sending time that things are going to go
# wrong, so we end up with a 200 and a truncated body
self.assertEqual(status, '200 OK')
@ -1574,12 +1978,12 @@ class TestSloGetManifest(SloTestCase):
status, headers, body, exc = self.call_slo(req, expect_exception=True)
headers = swob.HeaderKeyDict(headers)
self.assertTrue(isinstance(exc, SegmentError))
self.assertIsInstance(exc, SegmentError)
self.assertEqual(status, '200 OK')
self.assertEqual(self.app.calls, [
('GET', '/v1/AUTH_test/gettest/manifest-abcd'),
('GET', '/v1/AUTH_test/gettest/a_5?multipart-manifest=get'),
('GET', '/v1/AUTH_test/gettest/manifest-bc'),
('GET', '/v1/AUTH_test/gettest/a_5?multipart-manifest=get'),
('GET', '/v1/AUTH_test/gettest/b_10?multipart-manifest=get'),
# This one has the error, and so is the last one we fetch.
('GET', '/v1/AUTH_test/gettest/c_15?multipart-manifest=get')])
@ -1592,14 +1996,16 @@ class TestSloGetManifest(SloTestCase):
environ={'REQUEST_METHOD': 'GET'})
status, headers, body, exc = self.call_slo(req, expect_exception=True)
self.assertTrue(isinstance(exc, ListingIterError))
self.assertIsInstance(exc, ListingIterError)
self.assertEqual("200 OK", status)
self.assertEqual("aaaaa", body)
self.assertEqual(self.app.calls, [
('GET', '/v1/AUTH_test/gettest/manifest-abcd'),
('GET', '/v1/AUTH_test/gettest/a_5?multipart-manifest=get'),
# This one has the error, and so is the last one we fetch.
('GET', '/v1/AUTH_test/gettest/manifest-bc')])
('GET', '/v1/AUTH_test/gettest/manifest-bc'),
# But we were looking ahead to see if we could combine ranges,
# so we still get the first segment out
('GET', '/v1/AUTH_test/gettest/a_5?multipart-manifest=get')])
def test_error_fetching_first_segment_submanifest(self):
# This differs from the normal submanifest error because this one
@ -1640,7 +2046,7 @@ class TestSloGetManifest(SloTestCase):
environ={'REQUEST_METHOD': 'GET'})
status, headers, body, exc = self.call_slo(req, expect_exception=True)
self.assertTrue(isinstance(exc, ListingIterError))
self.assertIsInstance(exc, ListingIterError)
self.assertEqual('200 OK', status)
self.assertEqual(body, 'aaaaa')
@ -1661,7 +2067,7 @@ class TestSloGetManifest(SloTestCase):
environ={'REQUEST_METHOD': 'GET'})
status, headers, body, exc = self.call_slo(req, expect_exception=True)
self.assertTrue(isinstance(exc, SegmentError))
self.assertIsInstance(exc, SegmentError)
self.assertEqual('200 OK', status)
self.assertEqual(body, 'aaaaa')
@ -1682,7 +2088,7 @@ class TestSloGetManifest(SloTestCase):
environ={'REQUEST_METHOD': 'GET'})
status, headers, body, exc = self.call_slo(req, expect_exception=True)
self.assertTrue(isinstance(exc, SegmentError))
self.assertIsInstance(exc, SegmentError)
self.assertEqual('200 OK', status)
self.assertEqual(body, 'aaaaa')
@ -1748,12 +2154,12 @@ class TestSloGetManifest(SloTestCase):
status, headers, body, exc = self.call_slo(
req, expect_exception=True)
self.assertTrue(isinstance(exc, SegmentError))
self.assertIsInstance(exc, SegmentError)
self.assertEqual(status, '200 OK')
self.assertEqual(self.app.calls, [
('GET', '/v1/AUTH_test/gettest/manifest-abcd'),
('GET', '/v1/AUTH_test/gettest/a_5?multipart-manifest=get'),
('GET', '/v1/AUTH_test/gettest/manifest-bc'),
('GET', '/v1/AUTH_test/gettest/a_5?multipart-manifest=get'),
('GET', '/v1/AUTH_test/gettest/b_10?multipart-manifest=get'),
('GET', '/v1/AUTH_test/gettest/c_15?multipart-manifest=get')])