X-Git-Url: https://git.arvados.org/arvados.git/blobdiff_plain/d65b1921a9aeffd6906b95aa927a07a48f013b32..5b3187552676947ee74e4b652e7a04d3d9b9a3a4:/sdk/python/arvados/keep.py diff --git a/sdk/python/arvados/keep.py b/sdk/python/arvados/keep.py index 46bd1cb464..b2700ae5ba 100644 --- a/sdk/python/arvados/keep.py +++ b/sdk/python/arvados/keep.py @@ -1,29 +1,28 @@ +import bz2 +import datetime +import fcntl +import functools import gflags -import httplib -import httplib2 +import hashlib +import json import logging import os import pprint -import sys -import types -import subprocess -import json -import UserDict +import pycurl +import Queue import re -import hashlib +import socket +import ssl import string -import bz2 -import zlib -import fcntl -import time +import cStringIO +import subprocess +import sys import threading +import time import timer -import datetime -import ssl -import socket - -_logger = logging.getLogger('arvados.keep') -global_client_object = None +import types +import UserDict +import zlib import arvados import arvados.config as config @@ -31,6 +30,10 @@ import arvados.errors import arvados.retry as retry import arvados.util +_logger = logging.getLogger('arvados.keep') +global_client_object = None + + class KeepLocator(object): EPOCH_DATETIME = datetime.datetime.utcfromtimestamp(0) HINT_RE = re.compile(r'^[A-Z][A-Za-z0-9@_-]+$') @@ -47,7 +50,7 @@ class KeepLocator(object): self.size = None for hint in pieces: if self.HINT_RE.match(hint) is None: - raise ValueError("unrecognized hint data {}".format(hint)) + raise ValueError("invalid hint format: {}".format(hint)) elif hint.startswith('A'): self.parse_permission_hint(hint) else: @@ -59,6 +62,12 @@ class KeepLocator(object): self.permission_hint()] + self.hints if s is not None) + def stripped(self): + if self.size is not None: + return "%s+%i" % (self.md5sum, self.size) + else: + return self.md5sum + def _make_hex_prop(name, length): # Build and return a new property with the given name that # must be a hex string of the given length. @@ -67,7 +76,7 @@ class KeepLocator(object): return getattr(self, data_name) def setter(self, hex_str): if not arvados.util.is_hex(hex_str, length): - raise ValueError("{} must be a {}-digit hex string: {}". + raise ValueError("{} is not a {}-digit hex string: {}". format(name, length, hex_str)) setattr(self, data_name, hex_str) return property(getter, setter) @@ -172,8 +181,7 @@ class KeepBlockCache(object): def cap_cache(self): '''Cap the cache size to self.cache_max''' - self._cache_lock.acquire() - try: + with self._cache_lock: # Select all slots except those where ready.is_set() and content is # None (that means there was an error reading the block). self._cache = [c for c in self._cache if not (c.ready.is_set() and c.content is None)] @@ -184,32 +192,45 @@ class KeepBlockCache(object): del self._cache[i] break sm = sum([slot.size() for slot in self._cache]) - finally: - self._cache_lock.release() + + def _get(self, locator): + # Test if the locator is already in the cache + for i in xrange(0, len(self._cache)): + if self._cache[i].locator == locator: + n = self._cache[i] + if i != 0: + # move it to the front + del self._cache[i] + self._cache.insert(0, n) + return n + return None + + def get(self, locator): + with self._cache_lock: + return self._get(locator) def reserve_cache(self, locator): '''Reserve a cache slot for the specified locator, or return the existing slot.''' - self._cache_lock.acquire() - try: - # Test if the locator is already in the cache - for i in xrange(0, len(self._cache)): - if self._cache[i].locator == locator: - n = self._cache[i] - if i != 0: - # move it to the front - del self._cache[i] - self._cache.insert(0, n) - return n, False - - # Add a new cache slot for the locator - n = KeepBlockCache.CacheSlot(locator) - self._cache.insert(0, n) - return n, True - finally: - self._cache_lock.release() + with self._cache_lock: + n = self._get(locator) + if n: + return n, False + else: + # Add a new cache slot for the locator + n = KeepBlockCache.CacheSlot(locator) + self._cache.insert(0, n) + return n, True class KeepClient(object): + + # Default Keep server connection timeout: 2 seconds + # Default Keep server read timeout: 300 seconds + # Default Keep proxy connection timeout: 20 seconds + # Default Keep proxy read timeout: 300 seconds + DEFAULT_TIMEOUT = (2, 300) + DEFAULT_PROXY_TIMEOUT = (20, 300) + class ThreadLimiter(object): """ Limit the number of threads running at a given time to @@ -268,72 +289,216 @@ class KeepClient(object): class KeepService(object): - # Make requests to a single Keep service, and track results. - HTTP_ERRORS = (httplib2.HttpLib2Error, httplib.HTTPException, - socket.error, ssl.SSLError) + """Make requests to a single Keep service, and track results. + + A KeepService is intended to last long enough to perform one + transaction (GET or PUT) against one Keep service. This can + involve calling either get() or put() multiple times in order + to retry after transient failures. However, calling both get() + and put() on a single instance -- or using the same instance + to access two different Keep services -- will not produce + sensible behavior. + """ - def __init__(self, root, **headers): + HTTP_ERRORS = ( + socket.error, + ssl.SSLError, + arvados.errors.HttpError, + ) + + def __init__(self, root, user_agent_pool=Queue.LifoQueue(), **headers): self.root = root - self.last_result = None - self.success_flag = None + self._user_agent_pool = user_agent_pool + self._result = {'error': None} + self._usable = True + self._session = None self.get_headers = {'Accept': 'application/octet-stream'} self.get_headers.update(headers) self.put_headers = headers def usable(self): - return self.success_flag is not False + """Is it worth attempting a request?""" + return self._usable def finished(self): - return self.success_flag is not None + """Did the request succeed or encounter permanent failure?""" + return self._result['error'] == False or not self._usable + + def last_result(self): + return self._result - def last_status(self): + def _get_user_agent(self): try: - return int(self.last_result[0].status) - except (AttributeError, IndexError, ValueError): - return None + return self._user_agent_pool.get(False) + except Queue.Empty: + return pycurl.Curl() - def get(self, http, locator): - # http is an httplib2.Http object. + def _put_user_agent(self, ua): + try: + ua.reset() + self._user_agent_pool.put(ua, False) + except: + ua.close() + + @staticmethod + def _socket_open(family, socktype, protocol, address=None): + """Because pycurl doesn't have CURLOPT_TCP_KEEPALIVE""" + s = socket.socket(family, socktype, protocol) + s.setsockopt(socket.SOL_SOCKET, socket.SO_KEEPALIVE, 1) + s.setsockopt(socket.IPPROTO_TCP, socket.TCP_KEEPIDLE, 75) + s.setsockopt(socket.IPPROTO_TCP, socket.TCP_KEEPINTVL, 75) + return s + + def get(self, locator, timeout=None): # locator is a KeepLocator object. url = self.root + str(locator) _logger.debug("Request: GET %s", url) + curl = self._get_user_agent() try: with timer.Timer() as t: - result = http.request(url.encode('utf-8'), 'GET', - headers=self.get_headers) + self._headers = {} + response_body = cStringIO.StringIO() + curl.setopt(pycurl.NOSIGNAL, 1) + curl.setopt(pycurl.OPENSOCKETFUNCTION, self._socket_open) + curl.setopt(pycurl.URL, url.encode('utf-8')) + curl.setopt(pycurl.HTTPHEADER, [ + '{}: {}'.format(k,v) for k,v in self.get_headers.iteritems()]) + curl.setopt(pycurl.WRITEFUNCTION, response_body.write) + curl.setopt(pycurl.HEADERFUNCTION, self._headerfunction) + self._setcurltimeouts(curl, timeout) + try: + curl.perform() + except Exception as e: + raise arvados.errors.HttpError(0, str(e)) + self._result = { + 'status_code': curl.getinfo(pycurl.RESPONSE_CODE), + 'body': response_body.getvalue(), + 'headers': self._headers, + 'error': False, + } + ok = retry.check_http_response_success(self._result['status_code']) + if not ok: + self._result['error'] = arvados.errors.HttpError( + self._result['status_code'], + self._headers.get('x-status-line', 'Error')) except self.HTTP_ERRORS as e: - _logger.debug("Request fail: GET %s => %s: %s", - url, type(e), str(e)) - self.last_result = e + self._result = { + 'error': e, + } + ok = False + self._usable = ok != False + if self._result.get('status_code', None): + # The client worked well enough to get an HTTP status + # code, so presumably any problems are just on the + # server side and it's OK to reuse the client. + self._put_user_agent(curl) else: - self.last_result = result - self.success_flag = retry.check_http_response_success(result) - content = result[1] - _logger.info("%s response: %s bytes in %s msec (%.3f MiB/sec)", - self.last_status(), len(content), t.msecs, - (len(content)/(1024.0*1024))/t.secs) - if self.success_flag: - resp_md5 = hashlib.md5(content).hexdigest() - if resp_md5 == locator.md5sum: - return content - _logger.warning("Checksum fail: md5(%s) = %s", - url, resp_md5) - return None + # Don't return this client to the pool, in case it's + # broken. + curl.close() + if not ok: + _logger.debug("Request fail: GET %s => %s: %s", + url, type(self._result['error']), str(self._result['error'])) + return None + _logger.info("%s response: %s bytes in %s msec (%.3f MiB/sec)", + self._result['status_code'], + len(self._result['body']), + t.msecs, + (len(self._result['body'])/(1024.0*1024))/t.secs if t.secs > 0 else 0) + resp_md5 = hashlib.md5(self._result['body']).hexdigest() + if resp_md5 != locator.md5sum: + _logger.warning("Checksum fail: md5(%s) = %s", + url, resp_md5) + self._result['error'] = arvados.errors.HttpError( + 0, 'Checksum fail') + return None + return self._result['body'] - def put(self, http, hash_s, body): + def put(self, hash_s, body, timeout=None): url = self.root + hash_s _logger.debug("Request: PUT %s", url) + curl = self._get_user_agent() try: - result = http.request(url.encode('utf-8'), 'PUT', - headers=self.put_headers, body=body) + self._headers = {} + body_reader = cStringIO.StringIO(body) + response_body = cStringIO.StringIO() + curl.setopt(pycurl.NOSIGNAL, 1) + curl.setopt(pycurl.OPENSOCKETFUNCTION, self._socket_open) + curl.setopt(pycurl.URL, url.encode('utf-8')) + # Using UPLOAD tells cURL to wait for a "go ahead" from the + # Keep server (in the form of a HTTP/1.1 "100 Continue" + # response) instead of sending the request body immediately. + # This allows the server to reject the request if the request + # is invalid or the server is read-only, without waiting for + # the client to send the entire block. + curl.setopt(pycurl.UPLOAD, True) + curl.setopt(pycurl.INFILESIZE, len(body)) + curl.setopt(pycurl.READFUNCTION, body_reader.read) + curl.setopt(pycurl.HTTPHEADER, [ + '{}: {}'.format(k,v) for k,v in self.put_headers.iteritems()]) + curl.setopt(pycurl.WRITEFUNCTION, response_body.write) + curl.setopt(pycurl.HEADERFUNCTION, self._headerfunction) + self._setcurltimeouts(curl, timeout) + try: + curl.perform() + except Exception as e: + raise arvados.errors.HttpError(0, str(e)) + self._result = { + 'status_code': curl.getinfo(pycurl.RESPONSE_CODE), + 'body': response_body.getvalue(), + 'headers': self._headers, + 'error': False, + } + ok = retry.check_http_response_success(self._result['status_code']) + if not ok: + self._result['error'] = arvados.errors.HttpError( + self._result['status_code'], + self._headers.get('x-status-line', 'Error')) except self.HTTP_ERRORS as e: + self._result = { + 'error': e, + } + ok = False + self._usable = ok != False # still usable if ok is True or None + if self._result.get('status_code', None): + # Client is functional. See comment in get(). + self._put_user_agent(curl) + else: + curl.close() + if not ok: _logger.debug("Request fail: PUT %s => %s: %s", - url, type(e), str(e)) - self.last_result = e + url, type(self._result['error']), str(self._result['error'])) + return False + return True + + def _setcurltimeouts(self, curl, timeouts): + if not timeouts: + return + elif isinstance(timeouts, tuple): + conn_t, xfer_t = timeouts else: - self.last_result = result - self.success_flag = retry.check_http_response_success(result) - return self.success_flag + conn_t, xfer_t = (timeouts, timeouts) + curl.setopt(pycurl.CONNECTTIMEOUT_MS, int(conn_t*1000)) + curl.setopt(pycurl.TIMEOUT_MS, int(xfer_t*1000)) + + def _headerfunction(self, header_line): + header_line = header_line.decode('iso-8859-1') + if ':' in header_line: + name, value = header_line.split(':', 1) + name = name.strip().lower() + value = value.strip() + elif self._headers: + name = self._lastheadername + value = self._headers[name] + ' ' + header_line.strip() + elif header_line.startswith('HTTP/'): + name = 'x-status-line' + value = header_line + else: + _logger.error("Unexpected header line: %s", header_line) + return + self._lastheadername = name + self._headers[name] = value + # Returning None implies all bytes were written class KeepWriterThread(threading.Thread): @@ -362,62 +527,90 @@ class KeepClient(object): def run_with_limiter(self, limiter): if self.service.finished(): return - _logger.debug("KeepWriterThread %s proceeding %s %s", + _logger.debug("KeepWriterThread %s proceeding %s+%i %s", str(threading.current_thread()), self.args['data_hash'], + len(self.args['data']), self.args['service_root']) - h = httplib2.Http(timeout=self.args.get('timeout', None)) self._success = bool(self.service.put( - h, self.args['data_hash'], self.args['data'])) - status = self.service.last_status() + self.args['data_hash'], + self.args['data'], + timeout=self.args.get('timeout', None))) + result = self.service.last_result() if self._success: - resp, body = self.service.last_result - _logger.debug("KeepWriterThread %s succeeded %s %s", + _logger.debug("KeepWriterThread %s succeeded %s+%i %s", str(threading.current_thread()), self.args['data_hash'], + len(self.args['data']), self.args['service_root']) # Tick the 'done' counter for the number of replica # reported stored by the server, for the case that # we're talking to a proxy or other backend that # stores to multiple copies for us. try: - replicas_stored = int(resp['x-keep-replicas-stored']) + replicas_stored = int(result['headers']['x-keep-replicas-stored']) except (KeyError, ValueError): replicas_stored = 1 - limiter.save_response(body.strip(), replicas_stored) - elif status is not None: + limiter.save_response(result['body'].strip(), replicas_stored) + elif result.get('status_code', None): _logger.debug("Request fail: PUT %s => %s %s", - self.args['data_hash'], status, - self.service.last_result[1]) + self.args['data_hash'], + result['status_code'], + result['body']) - def __init__(self, api_client=None, proxy=None, timeout=300, + def __init__(self, api_client=None, proxy=None, + timeout=DEFAULT_TIMEOUT, proxy_timeout=DEFAULT_PROXY_TIMEOUT, api_token=None, local_store=None, block_cache=None, - num_retries=0): + num_retries=0, session=None): """Initialize a new KeepClient. Arguments: - * api_client: The API client to use to find Keep services. If not + :api_client: + The API client to use to find Keep services. If not provided, KeepClient will build one from available Arvados configuration. - * proxy: If specified, this KeepClient will send requests to this - Keep proxy. Otherwise, KeepClient will fall back to the setting - of the ARVADOS_KEEP_PROXY configuration setting. If you want to - ensure KeepClient does not use a proxy, pass in an empty string. - * timeout: The timeout for all HTTP requests, in seconds. Default - 300. - * api_token: If you're not using an API client, but only talking + + :proxy: + If specified, this KeepClient will send requests to this Keep + proxy. Otherwise, KeepClient will fall back to the setting of the + ARVADOS_KEEP_PROXY configuration setting. If you want to ensure + KeepClient does not use a proxy, pass in an empty string. + + :timeout: + The initial timeout (in seconds) for HTTP requests to Keep + non-proxy servers. A tuple of two floats is interpreted as + (connection_timeout, read_timeout): see + http://docs.python-requests.org/en/latest/user/advanced/#timeouts. + Because timeouts are often a result of transient server load, the + actual connection timeout will be increased by a factor of two on + each retry. + Default: (2, 300). + + :proxy_timeout: + The initial timeout (in seconds) for HTTP requests to + Keep proxies. A tuple of two floats is interpreted as + (connection_timeout, read_timeout). The behavior described + above for adjusting connection timeouts on retry also applies. + Default: (20, 300). + + :api_token: + If you're not using an API client, but only talking directly to a Keep proxy, this parameter specifies an API token to authenticate Keep requests. It is an error to specify both api_client and api_token. If you specify neither, KeepClient will use one available from the Arvados configuration. - * local_store: If specified, this KeepClient will bypass Keep + + :local_store: + If specified, this KeepClient will bypass Keep services, and save data to the named directory. If unspecified, KeepClient will fall back to the setting of the $KEEP_LOCAL_STORE environment variable. If you want to ensure KeepClient does not use local storage, pass in an empty string. This is primarily intended to mock a server for testing. - * num_retries: The default number of times to retry failed requests. + + :num_retries: + The default number of times to retry failed requests. This will be used as the default num_retries value when get() and put() are called. Default 0. """ @@ -436,21 +629,28 @@ class KeepClient(object): local_store = os.environ.get('KEEP_LOCAL_STORE') self.block_cache = block_cache if block_cache else KeepBlockCache() + self.timeout = timeout + self.proxy_timeout = proxy_timeout + self._user_agent_pool = Queue.LifoQueue() if local_store: self.local_store = local_store self.get = self.local_store_get self.put = self.local_store_put else: - self.timeout = timeout self.num_retries = num_retries if proxy: if not proxy.endswith('/'): proxy += '/' self.api_token = api_token - self.service_roots = [proxy] + self._gateway_services = {} + self._keep_services = [{ + 'uuid': 'proxy', + '_service_root': proxy, + }] + self._writable_services = self._keep_services self.using_proxy = True - self.static_service_roots = True + self._static_services_list = True else: # It's important to avoid instantiating an API client # unless we actually need one, for testing's sake. @@ -458,13 +658,30 @@ class KeepClient(object): api_client = arvados.api('v1') self.api_client = api_client self.api_token = api_client.api_token - self.service_roots = None + self._gateway_services = {} + self._keep_services = None + self._writable_services = None self.using_proxy = None - self.static_service_roots = False + self._static_services_list = False - def build_service_roots(self, force_rebuild=False): - if (self.static_service_roots or - (self.service_roots and not force_rebuild)): + def current_timeout(self, attempt_number): + """Return the appropriate timeout to use for this client. + + The proxy timeout setting if the backend service is currently a proxy, + the regular timeout setting otherwise. The `attempt_number` indicates + how many times the operation has been tried already (starting from 0 + for the first try), and scales the connection timeout portion of the + return value accordingly. + + """ + # TODO(twp): the timeout should be a property of a + # KeepService, not a KeepClient. See #4488. + t = self.proxy_timeout if self.using_proxy else self.timeout + return (t[0] * (1 << attempt_number), t[1]) + + def build_services_list(self, force_rebuild=False): + if (self._static_services_list or + (self._keep_services and not force_rebuild)): return with self.lock: try: @@ -472,79 +689,94 @@ class KeepClient(object): except Exception: # API server predates Keep services. keep_services = self.api_client.keep_disks().list() - keep_services = keep_services.execute().get('items') - if not keep_services: + accessible = keep_services.execute().get('items') + if not accessible: raise arvados.errors.NoKeepServersError() + # Precompute the base URI for each service. + for r in accessible: + host = r['service_host'] + if not host.startswith('[') and host.find(':') >= 0: + # IPv6 URIs must be formatted like http://[::1]:80/... + host = '[' + host + ']' + r['_service_root'] = "{}://{}:{:d}/".format( + 'https' if r['service_ssl_flag'] else 'http', + host, + r['service_port']) + + # Gateway services are only used when specified by UUID, + # so there's nothing to gain by filtering them by + # service_type. + self._gateway_services = {ks.get('uuid'): ks for ks in accessible} + _logger.debug(str(self._gateway_services)) + + self._keep_services = [ + ks for ks in accessible + if ks.get('service_type') in ['disk', 'proxy']] + self._writable_services = [ + ks for ks in accessible + if (ks.get('service_type') in ['disk', 'proxy']) and (True != ks.get('read_only'))] + _logger.debug(str(self._keep_services)) + self.using_proxy = any(ks.get('service_type') == 'proxy' - for ks in keep_services) - - roots = ("{}://[{}]:{:d}/".format( - 'https' if ks['service_ssl_flag'] else 'http', - ks['service_host'], - ks['service_port']) - for ks in keep_services) - self.service_roots = sorted(set(roots)) - _logger.debug(str(self.service_roots)) - - def shuffled_service_roots(self, hash, force_rebuild=False): - self.build_service_roots(force_rebuild) - - # Build an ordering with which to query the Keep servers based on the - # contents of the hash. - # "hash" is a hex-encoded number at least 8 digits - # (32 bits) long - - # seed used to calculate the next keep server from 'pool' - # to be added to 'pseq' - seed = hash - - # Keep servers still to be added to the ordering - pool = self.service_roots[:] - - # output probe sequence - pseq = [] - - # iterate while there are servers left to be assigned - while len(pool) > 0: - if len(seed) < 8: - # ran out of digits in the seed - if len(pseq) < len(hash) / 4: - # the number of servers added to the probe sequence is less - # than the number of 4-digit slices in 'hash' so refill the - # seed with the last 4 digits and then append the contents - # of 'hash'. - seed = hash[-4:] + hash - else: - # refill the seed with the contents of 'hash' - seed += hash - - # Take the next 8 digits (32 bytes) and interpret as an integer, - # then modulus with the size of the remaining pool to get the next - # selected server. - probe = int(seed[0:8], 16) % len(pool) - - # Append the selected server to the probe sequence and remove it - # from the pool. - pseq += [pool[probe]] - pool = pool[:probe] + pool[probe+1:] - - # Remove the digits just used from the seed - seed = seed[8:] - _logger.debug(str(pseq)) - return pseq - - - def map_new_services(self, roots_map, md5_s, force_rebuild, **headers): + for ks in self._keep_services) + + def _service_weight(self, data_hash, service_uuid): + """Compute the weight of a Keep service endpoint for a data + block with a known hash. + + The weight is md5(h + u) where u is the last 15 characters of + the service endpoint's UUID. + """ + return hashlib.md5(data_hash + service_uuid[-15:]).hexdigest() + + def weighted_service_roots(self, locator, force_rebuild=False, need_writable=False): + """Return an array of Keep service endpoints, in the order in + which they should be probed when reading or writing data with + the given hash+hints. + """ + self.build_services_list(force_rebuild) + + sorted_roots = [] + + # Use the services indicated by the given +K@... remote + # service hints, if any are present and can be resolved to a + # URI. + for hint in locator.hints: + if hint.startswith('K@'): + if len(hint) == 7: + sorted_roots.append( + "https://keep.{}.arvadosapi.com/".format(hint[2:])) + elif len(hint) == 29: + svc = self._gateway_services.get(hint[2:]) + if svc: + sorted_roots.append(svc['_service_root']) + + # Sort the available local services by weight (heaviest first) + # for this locator, and return their service_roots (base URIs) + # in that order. + use_services = self._keep_services + if need_writable: + use_services = self._writable_services + sorted_roots.extend([ + svc['_service_root'] for svc in sorted( + use_services, + reverse=True, + key=lambda svc: self._service_weight(locator.md5sum, svc['uuid']))]) + _logger.debug("{}: {}".format(locator, sorted_roots)) + return sorted_roots + + def map_new_services(self, roots_map, locator, force_rebuild, need_writable, **headers): # roots_map is a dictionary, mapping Keep service root strings # to KeepService objects. Poll for Keep services, and add any # new ones to roots_map. Return the current list of local # root strings. headers.setdefault('Authorization', "OAuth2 %s" % (self.api_token,)) - local_roots = self.shuffled_service_roots(md5_s, force_rebuild) + local_roots = self.weighted_service_roots(locator, force_rebuild, need_writable) for root in local_roots: if root not in roots_map: - roots_map[root] = self.KeepService(root, **headers) + roots_map[root] = self.KeepService( + root, self._user_agent_pool, **headers) return local_roots @staticmethod @@ -565,6 +797,14 @@ class KeepClient(object): else: return None + def get_from_cache(self, loc): + """Fetch a block only if is in the cache, otherwise return None.""" + slot = self.block_cache.get(loc) + if slot is not None and slot.ready.is_set(): + return slot.get() + else: + return None + @retry.retry_method def get(self, loc_s, num_retries=None): """Get data from Keep. @@ -589,30 +829,45 @@ class KeepClient(object): if ',' in loc_s: return ''.join(self.get(x) for x in loc_s.split(',')) locator = KeepLocator(loc_s) - expect_hash = locator.md5sum - - slot, first = self.block_cache.reserve_cache(expect_hash) + slot, first = self.block_cache.reserve_cache(locator.md5sum) if not first: v = slot.get() return v + # If the locator has hints specifying a prefix (indicating a + # remote keepproxy) or the UUID of a local gateway service, + # read data from the indicated service(s) instead of the usual + # list of local disk services. + hint_roots = ['http://keep.{}.arvadosapi.com/'.format(hint[2:]) + for hint in locator.hints if hint.startswith('K@') and len(hint) == 7] + hint_roots.extend([self._gateway_services[hint[2:]]['_service_root'] + for hint in locator.hints if ( + hint.startswith('K@') and + len(hint) == 29 and + self._gateway_services.get(hint[2:]) + )]) + # Map root URLs to their KeepService objects. + roots_map = { + root: self.KeepService(root, self._user_agent_pool) + for root in hint_roots + } + # See #3147 for a discussion of the loop implementation. Highlights: # * Refresh the list of Keep services after each failure, in case # it's being updated. # * Retry until we succeed, we're out of retries, or every available # service has returned permanent failure. - hint_roots = ['http://keep.{}.arvadosapi.com/'.format(hint[2:]) - for hint in locator.hints if hint.startswith('K@')] - # Map root URLs their KeepService objects. - roots_map = {root: self.KeepService(root) for root in hint_roots} + sorted_roots = [] + roots_map = {} blob = None loop = retry.RetryLoop(num_retries, self._check_loop_result, backoff_start=2) for tries_left in loop: try: - local_roots = self.map_new_services( - roots_map, expect_hash, - force_rebuild=(tries_left < num_retries)) + sorted_roots = self.map_new_services( + roots_map, locator, + force_rebuild=(tries_left < num_retries), + need_writable=False) except Exception as error: loop.save_result(error) continue @@ -620,11 +875,10 @@ class KeepClient(object): # Query KeepService objects that haven't returned # permanent failure, in our specified shuffle order. services_to_try = [roots_map[root] - for root in (local_roots + hint_roots) + for root in sorted_roots if roots_map[root].usable()] - http = httplib2.Http(timeout=self.timeout) for keep_service in services_to_try: - blob = keep_service.get(http, locator) + blob = keep_service.get(locator, timeout=self.current_timeout(num_retries-tries_left)) if blob is not None: break loop.save_result((blob, len(services_to_try))) @@ -635,17 +889,22 @@ class KeepClient(object): if loop.success(): return blob - # No servers fulfilled the request. Count how many responded - # "not found;" if the ratio is high enough (currently 75%), report - # Not Found; otherwise a generic error. # Q: Including 403 is necessary for the Keep tests to continue # passing, but maybe they should expect KeepReadError instead? - not_founds = sum(1 for ks in roots_map.values() - if ks.last_status() in set([403, 404, 410])) - if roots_map and ((float(not_founds) / len(roots_map)) >= .75): - raise arvados.errors.NotFoundError(loc_s) + not_founds = sum(1 for key in sorted_roots + if roots_map[key].last_result().get('status_code', None) in {403, 404, 410}) + service_errors = ((key, roots_map[key].last_result()['error']) + for key in sorted_roots) + if not roots_map: + raise arvados.errors.KeepReadError( + "failed to read {}: no Keep services available ({})".format( + loc_s, loop.last_result())) + elif not_founds == len(sorted_roots): + raise arvados.errors.NotFoundError( + "{} not found".format(loc_s), service_errors) else: - raise arvados.errors.KeepReadError(loc_s) + raise arvados.errors.KeepReadError( + "failed to read {}".format(loc_s), service_errors, label="service") @retry.retry_method def put(self, data, copies=2, num_retries=None): @@ -666,9 +925,16 @@ class KeepClient(object): exponential backoff. The default value is set when the KeepClient is initialized. """ + + if isinstance(data, unicode): + data = data.encode("ascii") + elif not isinstance(data, str): + raise arvados.errors.ArgumentError("Argument 'data' to KeepClient.put is not type 'str'") + data_hash = hashlib.md5(data).hexdigest() if copies < 1: return data_hash + locator = KeepLocator(data_hash + '+' + str(len(data))) headers = {} if self.using_proxy: @@ -681,8 +947,8 @@ class KeepClient(object): for tries_left in loop: try: local_roots = self.map_new_services( - roots_map, data_hash, - force_rebuild=(tries_left < num_retries), **headers) + roots_map, locator, + force_rebuild=(tries_left < num_retries), need_writable=True, **headers) except Exception as error: loop.save_result(error) continue @@ -697,7 +963,7 @@ class KeepClient(object): data_hash=data_hash, service_root=service_root, thread_limiter=thread_limiter, - timeout=self.timeout) + timeout=self.current_timeout(num_retries-tries_left)) t.start() threads.append(t) for t in threads: @@ -706,14 +972,30 @@ class KeepClient(object): if loop.success(): return thread_limiter.response() - raise arvados.errors.KeepWriteError( - "Write fail for %s: wanted %d but wrote %d" % - (data_hash, copies, thread_limiter.done())) - - # Local storage methods need no-op num_retries arguments to keep - # integration tests happy. With better isolation they could - # probably be removed again. - def local_store_put(self, data, num_retries=0): + if not roots_map: + raise arvados.errors.KeepWriteError( + "failed to write {}: no Keep services available ({})".format( + data_hash, loop.last_result())) + else: + service_errors = ((key, roots_map[key].last_result()['error']) + for key in local_roots + if roots_map[key].last_result()['error']) + raise arvados.errors.KeepWriteError( + "failed to write {} (wanted {} copies but wrote {})".format( + data_hash, copies, thread_limiter.done()), service_errors, label="service") + + def local_store_put(self, data, copies=1, num_retries=None): + """A stub for put(). + + This method is used in place of the real put() method when + using local storage (see constructor's local_store argument). + + copies and num_retries arguments are ignored: they are here + only for the sake of offering the same call signature as + put(). + + Data stored this way can be retrieved via local_store_get(). + """ md5 = hashlib.md5(data).hexdigest() locator = '%s+%d' % (md5, len(data)) with open(os.path.join(self.local_store, md5 + '.tmp'), 'w') as f: @@ -722,7 +1004,8 @@ class KeepClient(object): os.path.join(self.local_store, md5)) return locator - def local_store_get(self, loc_s, num_retries=0): + def local_store_get(self, loc_s, num_retries=None): + """Companion to local_store_put().""" try: locator = KeepLocator(loc_s) except ValueError: @@ -732,3 +1015,6 @@ class KeepClient(object): return '' with open(os.path.join(self.local_store, locator.md5sum), 'r') as f: return f.read() + + def is_cached(self, locator): + return self.block_cache.reserve_cache(expect_hash)