Merge "Ignore 404s from handoffs for objects when calculating quorum"
This commit is contained in:
commit
9aaae58bdd
@ -1282,10 +1282,10 @@ class ResumingGetter(object):
|
||||
if not self.newest: # one good source is enough
|
||||
return True
|
||||
else:
|
||||
if self.server_type != 'Object' and 'handoff_index' in node and \
|
||||
if 'handoff_index' in node and \
|
||||
possible_source.status == HTTP_NOT_FOUND and \
|
||||
not Timestamp(src_headers.get('x-backend-timestamp', 0)):
|
||||
# throw out 404s from handoff nodes unless the db is really
|
||||
# throw out 404s from handoff nodes unless the data is really
|
||||
# on disk and had been DELETEd
|
||||
return False
|
||||
self.statuses.append(possible_source.status)
|
||||
|
@ -2002,6 +2002,10 @@ class ECGetResponseBucket(object):
|
||||
def set_durable(self):
|
||||
self._durable = True
|
||||
|
||||
@property
|
||||
def durable(self):
|
||||
return self._durable
|
||||
|
||||
def add_response(self, getter, parts_iter):
|
||||
if not self.gets:
|
||||
self.status = getter.last_status
|
||||
@ -2017,7 +2021,8 @@ class ECGetResponseBucket(object):
|
||||
# metadata headers for self.headers by selecting the source with
|
||||
# the latest X-Timestamp.
|
||||
self.headers = getter.last_headers
|
||||
elif (getter.last_headers.get('X-Object-Sysmeta-Ec-Etag') !=
|
||||
elif (self.timestamp_str is not None and # ie, not bad_bucket
|
||||
getter.last_headers.get('X-Object-Sysmeta-Ec-Etag') !=
|
||||
self.headers.get('X-Object-Sysmeta-Ec-Etag')):
|
||||
# Fragments at the same timestamp with different etags are never
|
||||
# expected. If somehow it happens then ignore those fragments
|
||||
@ -2054,9 +2059,8 @@ class ECGetResponseBucket(object):
|
||||
|
||||
@property
|
||||
def shortfall(self):
|
||||
# A non-durable bucket always has a shortfall of at least 1
|
||||
result = self.policy.ec_ndata - len(self.get_responses())
|
||||
return max(result, 0 if self._durable else 1)
|
||||
return max(result, 0)
|
||||
|
||||
@property
|
||||
def shortfall_with_alts(self):
|
||||
@ -2064,7 +2068,7 @@ class ECGetResponseBucket(object):
|
||||
# for frags on the alt nodes.
|
||||
alts = set(self.alt_nodes.keys()).difference(set(self.gets.keys()))
|
||||
result = self.policy.ec_ndata - (len(self.get_responses()) + len(alts))
|
||||
return max(result, 0 if self._durable else 1)
|
||||
return max(result, 0)
|
||||
|
||||
def __str__(self):
|
||||
# return a string summarising bucket state, useful for debugging.
|
||||
@ -2141,12 +2145,14 @@ class ECGetResponseCollection(object):
|
||||
def _sort_buckets(self):
|
||||
def key_fn(bucket):
|
||||
# Returns a tuple to use for sort ordering:
|
||||
# buckets with no shortfall sort higher,
|
||||
# durable buckets with no shortfall sort higher,
|
||||
# then durable buckets with no shortfall_with_alts,
|
||||
# then non-durable buckets with no shortfall,
|
||||
# otherwise buckets with lowest shortfall_with_alts sort higher,
|
||||
# finally buckets with newer timestamps sort higher.
|
||||
return (bucket.shortfall <= 0,
|
||||
(not (bucket.shortfall <= 0) and
|
||||
(-1 * bucket.shortfall_with_alts)),
|
||||
return (bucket.durable,
|
||||
bucket.shortfall <= 0,
|
||||
-1 * bucket.shortfall_with_alts,
|
||||
bucket.timestamp_str)
|
||||
|
||||
return sorted(self.buckets.values(), key=key_fn, reverse=True)
|
||||
@ -2196,7 +2202,7 @@ class ECGetResponseCollection(object):
|
||||
return None
|
||||
|
||||
bucket = self.best_bucket
|
||||
if (bucket is None) or (bucket.shortfall <= 0):
|
||||
if (bucket is None) or (bucket.shortfall <= 0) or not bucket.durable:
|
||||
return None
|
||||
|
||||
alt_frags = set(bucket.alt_nodes.keys())
|
||||
@ -2381,7 +2387,11 @@ class ECObjectController(BaseObjectController):
|
||||
shortfall = bad_bucket.shortfall
|
||||
best_bucket = buckets.best_bucket
|
||||
if best_bucket:
|
||||
shortfall = min(best_bucket.shortfall, shortfall)
|
||||
shortfall = best_bucket.shortfall
|
||||
if not best_bucket.durable and shortfall <= 0:
|
||||
# be willing to go a *little* deeper, slowly
|
||||
shortfall = 1
|
||||
shortfall = min(shortfall, bad_bucket.shortfall)
|
||||
if (extra_requests < max_extra_requests and
|
||||
shortfall > pile._pending and
|
||||
(node_iter.nodes_left > 0 or
|
||||
@ -2395,7 +2405,7 @@ class ECObjectController(BaseObjectController):
|
||||
buckets.get_extra_headers)
|
||||
|
||||
req.range = orig_range
|
||||
if best_bucket and best_bucket.shortfall <= 0:
|
||||
if best_bucket and best_bucket.shortfall <= 0 and best_bucket.durable:
|
||||
# headers can come from any of the getters
|
||||
resp_headers = best_bucket.headers
|
||||
resp_headers.pop('Content-Range', None)
|
||||
@ -2435,10 +2445,28 @@ class ECObjectController(BaseObjectController):
|
||||
bodies = []
|
||||
headers = []
|
||||
for getter, _parts_iter in bad_bucket.get_responses():
|
||||
if best_bucket and best_bucket.durable:
|
||||
headers = HeaderKeyDict(getter.last_headers)
|
||||
t_data_file = headers.get('X-Backend-Data-Timestamp')
|
||||
t_obj = headers.get('X-Backend-Timestamp',
|
||||
headers.get('X-Timestamp'))
|
||||
bad_ts = Timestamp(t_data_file or t_obj or '0')
|
||||
if bad_ts <= Timestamp(best_bucket.timestamp_str):
|
||||
# We have reason to believe there's still good data
|
||||
# out there, it's just currently unavailable
|
||||
continue
|
||||
statuses.extend(getter.statuses)
|
||||
reasons.extend(getter.reasons)
|
||||
bodies.extend(getter.bodies)
|
||||
headers.extend(getter.source_headers)
|
||||
|
||||
if not statuses and best_bucket and not best_bucket.durable:
|
||||
# pretend that non-durable bucket was 404s
|
||||
statuses.append(404)
|
||||
reasons.append('404 Not Found')
|
||||
bodies.append(b'')
|
||||
headers.append({})
|
||||
|
||||
resp = self.best_response(
|
||||
req, statuses, reasons, bodies, 'Object',
|
||||
headers=headers)
|
||||
|
@ -14,6 +14,8 @@
|
||||
# See the License for the specific language governing permissions and
|
||||
# limitations under the License.
|
||||
|
||||
import json
|
||||
import os
|
||||
import time
|
||||
from os import listdir, unlink
|
||||
from os.path import join as path_join
|
||||
@ -26,7 +28,7 @@ from swift.common import direct_client
|
||||
from swift.common.exceptions import ClientException
|
||||
from swift.common.utils import hash_path, readconf
|
||||
from swift.obj.diskfile import write_metadata, read_metadata, get_data_dir
|
||||
from test.probe.common import ReplProbeTest
|
||||
from test.probe.common import ReplProbeTest, ECProbeTest
|
||||
|
||||
|
||||
RETRIES = 5
|
||||
@ -198,5 +200,61 @@ class TestObjectFailures(ReplProbeTest):
|
||||
self.run_quarantine_zero_byte_post()
|
||||
|
||||
|
||||
class TestECObjectFailures(ECProbeTest):
|
||||
|
||||
def test_ec_missing_all_durable_fragments(self):
|
||||
# This tests helps assert the behavior that when
|
||||
# the proxy has enough fragments to reconstruct the object
|
||||
# but none are marked as durable, the proxy should return a 404.
|
||||
|
||||
container_name = 'container-%s' % uuid4()
|
||||
object_name = 'object-%s' % uuid4()
|
||||
|
||||
# create EC container
|
||||
headers = {'X-Storage-Policy': self.policy.name}
|
||||
client.put_container(self.url, self.token, container_name,
|
||||
headers=headers)
|
||||
|
||||
# PUT object, should go to primary nodes
|
||||
client.put_object(self.url, self.token, container_name,
|
||||
object_name, contents='object contents')
|
||||
|
||||
# get our node lists
|
||||
opart, onodes = self.object_ring.get_nodes(
|
||||
self.account, container_name, object_name)
|
||||
|
||||
# sanity test
|
||||
odata = client.get_object(self.url, self.token, container_name,
|
||||
object_name)[-1]
|
||||
self.assertEqual('object contents', odata)
|
||||
|
||||
# make all fragments non-durable
|
||||
for node in onodes:
|
||||
part_dir = self.storage_dir('object', node, part=opart)
|
||||
for dirs, subdirs, files in os.walk(part_dir):
|
||||
for fname in files:
|
||||
if fname.endswith('.data'):
|
||||
non_durable_fname = fname.replace('#d', '')
|
||||
os.rename(os.path.join(dirs, fname),
|
||||
os.path.join(dirs, non_durable_fname))
|
||||
break
|
||||
headers = direct_client.direct_head_object(
|
||||
node, opart, self.account, container_name, object_name,
|
||||
headers={
|
||||
'X-Backend-Storage-Policy-Index': self.policy.idx,
|
||||
'X-Backend-Fragment-Preferences': json.dumps([])})
|
||||
self.assertNotIn('X-Backend-Durable-Timestamp', headers)
|
||||
|
||||
# Now a new GET should return *404* because all fragments
|
||||
# are non-durable, even if they are reconstructable
|
||||
try:
|
||||
client.get_object(self.url, self.token, container_name,
|
||||
object_name)
|
||||
except client.ClientException as err:
|
||||
self.assertEqual(err.http_status, 404)
|
||||
else:
|
||||
self.fail("Expected ClientException but didn't get it")
|
||||
|
||||
|
||||
if __name__ == '__main__':
|
||||
main()
|
||||
|
@ -309,6 +309,62 @@ class TestObjectHandoff(ReplProbeTest):
|
||||
else:
|
||||
self.fail("Expected ClientException but didn't get it")
|
||||
|
||||
def test_missing_primaries(self):
|
||||
# Create container
|
||||
container = 'container-%s' % uuid4()
|
||||
client.put_container(self.url, self.token, container,
|
||||
headers={'X-Storage-Policy':
|
||||
self.policy.name})
|
||||
|
||||
# Create container/obj (goes to all three primaries)
|
||||
obj = 'object-%s' % uuid4()
|
||||
client.put_object(self.url, self.token, container, obj, 'VERIFY')
|
||||
odata = client.get_object(self.url, self.token, container, obj)[-1]
|
||||
if odata != 'VERIFY':
|
||||
raise Exception('Object GET did not return VERIFY, instead it '
|
||||
'returned: %s' % repr(odata))
|
||||
|
||||
# Kill all primaries obj server
|
||||
obj = 'object-%s' % uuid4()
|
||||
opart, onodes = self.object_ring.get_nodes(
|
||||
self.account, container, obj)
|
||||
for onode in onodes:
|
||||
kill_server((onode['ip'], onode['port']), self.ipport2server)
|
||||
|
||||
# Indirectly (i.e., through proxy) try to GET object, it should return
|
||||
# a 503, since all primaries will Timeout and handoffs return a 404.
|
||||
try:
|
||||
client.get_object(self.url, self.token, container, obj)
|
||||
except client.ClientException as err:
|
||||
self.assertEqual(err.http_status, 503)
|
||||
else:
|
||||
self.fail("Expected ClientException but didn't get it")
|
||||
|
||||
# Restart the first container/obj primary server again
|
||||
onode = onodes[0]
|
||||
start_server((onode['ip'], onode['port']), self.ipport2server)
|
||||
|
||||
# Send a delete that will reach first primary and handoff.
|
||||
# Sure, the DELETE will return a 404 since the handoff doesn't
|
||||
# have a .data file, but object server will still write a
|
||||
# Tombstone in the handoff node!
|
||||
try:
|
||||
client.delete_object(self.url, self.token, container, obj)
|
||||
except client.ClientException as err:
|
||||
self.assertEqual(err.http_status, 404)
|
||||
|
||||
# kill the first container/obj primary server again
|
||||
kill_server((onode['ip'], onode['port']), self.ipport2server)
|
||||
|
||||
# a new GET should return a 404, since all primaries will Timeout
|
||||
# and the handoff will return a 404 but this time with a tombstone
|
||||
try:
|
||||
client.get_object(self.url, self.token, container, obj)
|
||||
except client.ClientException as err:
|
||||
self.assertEqual(err.http_status, 404)
|
||||
else:
|
||||
self.fail("Expected ClientException but didn't get it")
|
||||
|
||||
|
||||
class TestECObjectHandoff(ECProbeTest):
|
||||
|
||||
@ -523,5 +579,54 @@ class TestECObjectHandoff(ECProbeTest):
|
||||
# ... all six unique
|
||||
self.assertEqual(len(frag2count), 6)
|
||||
|
||||
def test_ec_primary_timeout(self):
|
||||
container_name = 'container-%s' % uuid4()
|
||||
object_name = 'object-%s' % uuid4()
|
||||
|
||||
# create EC container
|
||||
headers = {'X-Storage-Policy': self.policy.name}
|
||||
client.put_container(self.url, self.token, container_name,
|
||||
headers=headers)
|
||||
|
||||
# PUT object, should go to primary nodes
|
||||
old_contents = Body()
|
||||
client.put_object(self.url, self.token, container_name,
|
||||
object_name, contents=old_contents)
|
||||
|
||||
# get our node lists
|
||||
opart, onodes = self.object_ring.get_nodes(
|
||||
self.account, container_name, object_name)
|
||||
|
||||
# shutdown three of the primary data nodes
|
||||
for i in range(3):
|
||||
failed_primary = onodes[i]
|
||||
failed_primary_device_path = self.device_dir('object',
|
||||
failed_primary)
|
||||
self.kill_drive(failed_primary_device_path)
|
||||
|
||||
# Indirectly (i.e., through proxy) try to GET object, it should return
|
||||
# a 503, since all primaries will Timeout and handoffs return a 404.
|
||||
try:
|
||||
client.get_object(self.url, self.token, container_name,
|
||||
object_name)
|
||||
except client.ClientException as err:
|
||||
self.assertEqual(err.http_status, 503)
|
||||
else:
|
||||
self.fail("Expected ClientException but didn't get it")
|
||||
|
||||
# Send a delete to write down tombstones in the handoff nodes
|
||||
client.delete_object(self.url, self.token, container_name, object_name)
|
||||
|
||||
# Now a new GET should return 404 because the handoff nodes
|
||||
# return a 404 with a Tombstone.
|
||||
try:
|
||||
client.get_object(self.url, self.token, container_name,
|
||||
object_name)
|
||||
except client.ClientException as err:
|
||||
self.assertEqual(err.http_status, 404)
|
||||
else:
|
||||
self.fail("Expected ClientException but didn't get it")
|
||||
|
||||
|
||||
if __name__ == '__main__':
|
||||
main()
|
||||
|
@ -2136,7 +2136,7 @@ class ECObjectControllerMixin(CommonObjectControllerMixin):
|
||||
{'obj': obj1, 'frag': 13},
|
||||
]
|
||||
|
||||
# ... and the rests are 404s which is limited by request_count
|
||||
# ... and the rest are 404s which is limited by request_count
|
||||
# (2 * replicas in default) rather than max_extra_requests limitation
|
||||
# because the retries will be in ResumingGetter if the responses
|
||||
# are 404s
|
||||
@ -2147,7 +2147,7 @@ class ECObjectControllerMixin(CommonObjectControllerMixin):
|
||||
with capture_http_requests(fake_response) as log:
|
||||
resp = req.get_response(self.app)
|
||||
|
||||
self.assertEqual(resp.status_int, 404)
|
||||
self.assertEqual(resp.status_int, 503)
|
||||
|
||||
# expect a request to all nodes
|
||||
self.assertEqual(2 * self.replicas(), len(log))
|
||||
@ -2693,7 +2693,7 @@ class TestECObjController(ECObjectControllerMixin, unittest.TestCase):
|
||||
with capture_http_requests(fake_response) as log:
|
||||
resp = req.get_response(self.app)
|
||||
|
||||
self.assertEqual(resp.status_int, 404)
|
||||
self.assertEqual(resp.status_int, 503)
|
||||
|
||||
collected_responses = defaultdict(set)
|
||||
for conn in log:
|
||||
@ -2792,7 +2792,7 @@ class TestECObjController(ECObjectControllerMixin, unittest.TestCase):
|
||||
collected_indexes[fi].append(conn)
|
||||
self.assertEqual(len(collected_indexes), 7)
|
||||
|
||||
def test_GET_with_mixed_frags_and_no_quorum_will_503(self):
|
||||
def test_GET_with_mixed_nondurable_frags_and_no_quorum_will_503(self):
|
||||
# all nodes have a frag but there is no one set that reaches quorum,
|
||||
# which means there is no backend 404 response, but proxy should still
|
||||
# return 404 rather than 503
|
||||
@ -2801,6 +2801,68 @@ class TestECObjController(ECObjectControllerMixin, unittest.TestCase):
|
||||
obj3 = self._make_ec_object_stub(pattern='obj3')
|
||||
obj4 = self._make_ec_object_stub(pattern='obj4')
|
||||
|
||||
node_frags = [
|
||||
{'obj': obj1, 'frag': 0, 'durable': False},
|
||||
{'obj': obj2, 'frag': 0, 'durable': False},
|
||||
{'obj': obj3, 'frag': 0, 'durable': False},
|
||||
{'obj': obj1, 'frag': 1, 'durable': False},
|
||||
{'obj': obj2, 'frag': 1, 'durable': False},
|
||||
{'obj': obj3, 'frag': 1, 'durable': False},
|
||||
{'obj': obj1, 'frag': 2, 'durable': False},
|
||||
{'obj': obj2, 'frag': 2, 'durable': False},
|
||||
{'obj': obj3, 'frag': 2, 'durable': False},
|
||||
{'obj': obj1, 'frag': 3, 'durable': False},
|
||||
{'obj': obj2, 'frag': 3, 'durable': False},
|
||||
{'obj': obj3, 'frag': 3, 'durable': False},
|
||||
{'obj': obj1, 'frag': 4, 'durable': False},
|
||||
{'obj': obj2, 'frag': 4, 'durable': False},
|
||||
{'obj': obj3, 'frag': 4, 'durable': False},
|
||||
{'obj': obj1, 'frag': 5, 'durable': False},
|
||||
{'obj': obj2, 'frag': 5, 'durable': False},
|
||||
{'obj': obj3, 'frag': 5, 'durable': False},
|
||||
{'obj': obj1, 'frag': 6, 'durable': False},
|
||||
{'obj': obj2, 'frag': 6, 'durable': False},
|
||||
{'obj': obj3, 'frag': 6, 'durable': False},
|
||||
{'obj': obj1, 'frag': 7, 'durable': False},
|
||||
{'obj': obj2, 'frag': 7, 'durable': False},
|
||||
{'obj': obj3, 'frag': 7, 'durable': False},
|
||||
{'obj': obj1, 'frag': 8, 'durable': False},
|
||||
{'obj': obj2, 'frag': 8, 'durable': False},
|
||||
{'obj': obj3, 'frag': 8, 'durable': False},
|
||||
{'obj': obj4, 'frag': 8, 'durable': False},
|
||||
]
|
||||
|
||||
fake_response = self._fake_ec_node_response(node_frags)
|
||||
|
||||
req = swob.Request.blank('/v1/a/c/o')
|
||||
with capture_http_requests(fake_response) as log:
|
||||
resp = req.get_response(self.app)
|
||||
|
||||
self.assertEqual(resp.status_int, 404)
|
||||
|
||||
collected_etags = set()
|
||||
collected_status = set()
|
||||
for conn in log:
|
||||
etag = conn.resp.headers['X-Object-Sysmeta-Ec-Etag']
|
||||
collected_etags.add(etag)
|
||||
collected_status.add(conn.resp.status)
|
||||
|
||||
# default node_iter will exhaust at 2 * replicas
|
||||
self.assertEqual(len(log), 2 * self.replicas())
|
||||
self.assertEqual(
|
||||
{obj1['etag'], obj2['etag'], obj3['etag'], obj4['etag']},
|
||||
collected_etags)
|
||||
self.assertEqual({200}, collected_status)
|
||||
|
||||
def test_GET_with_mixed_frags_and_no_quorum_will_503(self):
|
||||
# all nodes have a frag but there is no one set that reaches quorum,
|
||||
# but since they're all marked durable (so we *should* be able to
|
||||
# reconstruct), proxy will 503
|
||||
obj1 = self._make_ec_object_stub(pattern='obj1')
|
||||
obj2 = self._make_ec_object_stub(pattern='obj2')
|
||||
obj3 = self._make_ec_object_stub(pattern='obj3')
|
||||
obj4 = self._make_ec_object_stub(pattern='obj4')
|
||||
|
||||
node_frags = [
|
||||
{'obj': obj1, 'frag': 0},
|
||||
{'obj': obj2, 'frag': 0},
|
||||
@ -3229,9 +3291,9 @@ class TestECObjController(ECObjectControllerMixin, unittest.TestCase):
|
||||
with capture_http_requests(fake_response) as log:
|
||||
resp = req.get_response(self.app)
|
||||
# read body to provoke any EC decode errors
|
||||
self.assertFalse(resp.body)
|
||||
self.assertTrue(resp.body)
|
||||
|
||||
self.assertEqual(resp.status_int, 404)
|
||||
self.assertEqual(resp.status_int, 503)
|
||||
self.assertEqual(len(log), self.replicas() * 2)
|
||||
collected_etags = set()
|
||||
for conn in log:
|
||||
@ -3240,7 +3302,10 @@ class TestECObjController(ECObjectControllerMixin, unittest.TestCase):
|
||||
self.assertEqual({obj1['etag'], obj2['etag'], None}, collected_etags)
|
||||
log_lines = self.app.logger.get_lines_for_level('error')
|
||||
self.assertEqual(log_lines,
|
||||
['Problem with fragment response: ETag mismatch'] * 7)
|
||||
['Problem with fragment response: ETag mismatch'] * 7
|
||||
+ ['Object returning 503 for []'])
|
||||
# Note the empty list above -- that log line comes out of
|
||||
# best_response but we've already thrown out the "good" responses :-/
|
||||
|
||||
def test_GET_mixed_success_with_range(self):
|
||||
fragment_size = self.policy.fragment_size
|
||||
@ -3926,7 +3991,7 @@ class TestECDuplicationObjController(
|
||||
{'obj': obj1, 'frag': 8},
|
||||
{'obj': obj2, 'frag': 8},
|
||||
]
|
||||
# ... and the rests are 404s which is limited by request_count
|
||||
# ... and the rest are 404s which is limited by request_count
|
||||
# (2 * replicas in default) rather than max_extra_requests limitation
|
||||
# because the retries will be in ResumingGetter if the responses
|
||||
# are 404s
|
||||
@ -3937,7 +4002,7 @@ class TestECDuplicationObjController(
|
||||
with capture_http_requests(fake_response) as log:
|
||||
resp = req.get_response(self.app)
|
||||
|
||||
self.assertEqual(resp.status_int, 404)
|
||||
self.assertEqual(resp.status_int, 503)
|
||||
|
||||
collected_responses = defaultdict(set)
|
||||
for conn in log:
|
||||
@ -4267,9 +4332,9 @@ class TestECDuplicationObjController(
|
||||
with capture_http_requests(fake_response) as log:
|
||||
resp = req.get_response(self.app)
|
||||
# read body to provoke any EC decode errors
|
||||
self.assertFalse(resp.body)
|
||||
self.assertTrue(resp.body)
|
||||
|
||||
self.assertEqual(resp.status_int, 404)
|
||||
self.assertEqual(resp.status_int, 503)
|
||||
self.assertEqual(len(log), self.replicas() * 2)
|
||||
collected_etags = set()
|
||||
for conn in log:
|
||||
@ -4278,7 +4343,8 @@ class TestECDuplicationObjController(
|
||||
self.assertEqual({obj1['etag'], obj2['etag'], None}, collected_etags)
|
||||
log_lines = self.app.logger.get_lines_for_level('error')
|
||||
self.assertEqual(log_lines,
|
||||
['Problem with fragment response: ETag mismatch'] * 7)
|
||||
['Problem with fragment response: ETag mismatch'] * 7
|
||||
+ ['Object returning 503 for []'])
|
||||
|
||||
def _test_determine_chunk_destinations_prioritize(
|
||||
self, missing_two, missing_one):
|
||||
|
@ -3975,6 +3975,13 @@ class TestReplicatedObjectController(
|
||||
test_status_map((200, 200, 404, 404, 500), 404)
|
||||
test_status_map((200, 200, 500, 500, 500), 503)
|
||||
|
||||
POLICIES.default.object_ring.max_more_nodes = 3
|
||||
test_status_map(
|
||||
(200, 200,
|
||||
Timeout(), Timeout(), Timeout(), # Can't reach primaries
|
||||
404, 404, 404), # No surprise: handoffs know nothing
|
||||
503)
|
||||
|
||||
def test_HEAD_newest(self):
|
||||
with save_globals():
|
||||
def test_status_map(statuses, expected, timestamps,
|
||||
@ -7527,7 +7534,7 @@ class TestECGets(unittest.TestCase):
|
||||
}
|
||||
|
||||
resp = self._setup_nodes_and_do_GET(objs, node_state)
|
||||
self.assertEqual(resp.status_int, 503)
|
||||
self.assertEqual(resp.status_int, 404)
|
||||
|
||||
def test_GET_with_multiple_frags_per_node(self):
|
||||
# verify object GET behavior when multiple fragments are on same node
|
||||
@ -7632,13 +7639,26 @@ class TestECGets(unittest.TestCase):
|
||||
# will be sent frag prefs that exclude frag_index 1)
|
||||
node_state = {
|
||||
0: [dict(ref='obj1a', frag_index=1, durable=False)],
|
||||
1: [dict(ref='obj1b', frag_index=1, durable=True)],
|
||||
1: [dict(ref='obj1b', frag_index=1, durable=False)],
|
||||
2: [dict(ref='obj1c', frag_index=1, durable=False)]
|
||||
}
|
||||
|
||||
resp = self._setup_nodes_and_do_GET(objs, node_state)
|
||||
self.assertEqual(resp.status_int, 404)
|
||||
|
||||
# if we know it should be durable, we can be more specific.
|
||||
# note that we need to set *both* of those first ones durable
|
||||
# to avoid a flaky test -- in the future we can be smarter and
|
||||
# let the durability bubble up, even from a duplicate frag
|
||||
node_state = {
|
||||
0: [dict(ref='obj1a', frag_index=1, durable=True)],
|
||||
1: [dict(ref='obj1b', frag_index=1, durable=True)],
|
||||
2: [dict(ref='obj1c', frag_index=1, durable=False)]
|
||||
}
|
||||
|
||||
resp = self._setup_nodes_and_do_GET(objs, node_state)
|
||||
self.assertEqual(resp.status_int, 503)
|
||||
|
||||
|
||||
class TestObjectDisconnectCleanup(unittest.TestCase):
|
||||
|
||||
|
Loading…
x
Reference in New Issue
Block a user