X-Git-Url: https://git.arvados.org/arvados.git/blobdiff_plain/8b097ea832516b3f5104c62a40a6d9cf4826232a..2d6c425e78bc5712c63b4ebecb05077b0e30da1f:/sdk/python/arvados/keep.py diff --git a/sdk/python/arvados/keep.py b/sdk/python/arvados/keep.py index 3c0ad6f7a9..94104586de 100644 --- a/sdk/python/arvados/keep.py +++ b/sdk/python/arvados/keep.py @@ -1,16 +1,38 @@ -import cStringIO +# Copyright (C) The Arvados Authors. All rights reserved. +# +# SPDX-License-Identifier: Apache-2.0 + +from __future__ import absolute_import +from __future__ import division +import copy +from future import standard_library +from future.utils import native_str +standard_library.install_aliases() +from builtins import next +from builtins import str +from builtins import range +from builtins import object +import collections import datetime import hashlib +import io import logging import math import os import pycurl -import Queue +import queue import re import socket import ssl +import sys import threading -import timer +from . import timer +import urllib.parse + +if sys.version_info >= (3, 0): + from io import BytesIO +else: + from cStringIO import StringIO as BytesIO import arvados import arvados.config as config @@ -22,6 +44,17 @@ _logger = logging.getLogger('arvados.keep') global_client_object = None +# Monkey patch TCP constants when not available (apple). Values sourced from: +# http://www.opensource.apple.com/source/xnu/xnu-2422.115.4/bsd/netinet/tcp.h +if sys.platform == 'darwin': + if not hasattr(socket, 'TCP_KEEPALIVE'): + socket.TCP_KEEPALIVE = 0x010 + if not hasattr(socket, 'TCP_KEEPINTVL'): + socket.TCP_KEEPINTVL = 0x101 + if not hasattr(socket, 'TCP_KEEPCNT'): + socket.TCP_KEEPCNT = 0x102 + + class KeepLocator(object): EPOCH_DATETIME = datetime.datetime.utcfromtimestamp(0) HINT_RE = re.compile(r'^[A-Z][A-Za-z0-9@_-]+$') @@ -46,8 +79,9 @@ class KeepLocator(object): def __str__(self): return '+'.join( - str(s) for s in [self.md5sum, self.size, - self.permission_hint()] + self.hints + native_str(s) + for s in [self.md5sum, self.size, + self.permission_hint()] + self.hints if s is not None) def stripped(self): @@ -64,7 +98,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("{} is not a {}-digit hex string: {}". + raise ValueError("{} is not a {}-digit hex string: {!r}". format(name, length, hex_str)) setattr(self, data_name, hex_str) return property(getter, setter) @@ -177,7 +211,7 @@ class KeepBlockCache(object): self._cache = [c for c in self._cache if not (c.ready.is_set() and c.content is None)] sm = sum([slot.size() for slot in self._cache]) while len(self._cache) > 0 and sm > self.cache_max: - for i in xrange(len(self._cache)-1, -1, -1): + for i in range(len(self._cache)-1, -1, -1): if self._cache[i].ready.is_set(): del self._cache[i] break @@ -185,7 +219,7 @@ class KeepBlockCache(object): def _get(self, locator): # Test if the locator is already in the cache - for i in xrange(0, len(self._cache)): + for i in range(0, len(self._cache)): if self._cache[i].locator == locator: n = self._cache[i] if i != 0: @@ -237,76 +271,6 @@ class KeepClient(object): DEFAULT_TIMEOUT = (2, 256, 32768) DEFAULT_PROXY_TIMEOUT = (20, 256, 32768) - class ThreadLimiter(object): - """Limit the number of threads writing to Keep at once. - - This ensures that only a number of writer threads that could - potentially achieve the desired replication level run at once. - Once the desired replication level is achieved, queued threads - are instructed not to run. - - Should be used in a "with" block. - """ - def __init__(self, want_copies, max_service_replicas): - self._started = 0 - self._want_copies = want_copies - self._done = 0 - self._response = None - self._start_lock = threading.Condition() - if (not max_service_replicas) or (max_service_replicas >= want_copies): - max_threads = 1 - else: - max_threads = math.ceil(float(want_copies) / max_service_replicas) - _logger.debug("Limiter max threads is %d", max_threads) - self._todo_lock = threading.Semaphore(max_threads) - self._done_lock = threading.Lock() - self._local = threading.local() - - def __enter__(self): - self._start_lock.acquire() - if getattr(self._local, 'sequence', None) is not None: - # If the calling thread has used set_sequence(N), then - # we wait here until N other threads have started. - while self._started < self._local.sequence: - self._start_lock.wait() - self._todo_lock.acquire() - self._started += 1 - self._start_lock.notifyAll() - self._start_lock.release() - return self - - def __exit__(self, type, value, traceback): - self._todo_lock.release() - - def set_sequence(self, sequence): - self._local.sequence = sequence - - def shall_i_proceed(self): - """ - Return true if the current thread should write to Keep. - Return false otherwise. - """ - with self._done_lock: - return (self._done < self._want_copies) - - def save_response(self, response_body, replicas_stored): - """ - Records a response body (a locator, possibly signed) returned by - the Keep server, and the number of replicas it stored. - """ - with self._done_lock: - self._done += replicas_stored - self._response = response_body - - def response(self): - """Return the body from the response to a PUT request.""" - with self._done_lock: - return self._response - - def done(self): - """Return the total number of replicas successfully stored.""" - with self._done_lock: - return self._done class KeepService(object): """Make requests to a single Keep service, and track results. @@ -326,19 +290,23 @@ class KeepClient(object): arvados.errors.HttpError, ) - def __init__(self, root, user_agent_pool=Queue.LifoQueue(), + def __init__(self, root, user_agent_pool=queue.LifoQueue(), upload_counter=None, - download_counter=None, **headers): + download_counter=None, + headers={}, + insecure=False): self.root = root self._user_agent_pool = user_agent_pool self._result = {'error': None} self._usable = True self._session = None + self._socket = None self.get_headers = {'Accept': 'application/octet-stream'} self.get_headers.update(headers) self.put_headers = headers self.upload_counter = upload_counter self.download_counter = download_counter + self.insecure = insecure def usable(self): """Is it worth attempting a request?""" @@ -353,24 +321,39 @@ class KeepClient(object): def _get_user_agent(self): try: - return self._user_agent_pool.get(False) - except Queue.Empty: + return self._user_agent_pool.get(block=False) + except queue.Empty: return pycurl.Curl() def _put_user_agent(self, ua): try: ua.reset() - self._user_agent_pool.put(ua, False) + self._user_agent_pool.put(ua, block=False) except: ua.close() - @staticmethod - def _socket_open(family, socktype, protocol, address=None): + def _socket_open(self, *args, **kwargs): + if len(args) + len(kwargs) == 2: + return self._socket_open_pycurl_7_21_5(*args, **kwargs) + else: + return self._socket_open_pycurl_7_19_3(*args, **kwargs) + + def _socket_open_pycurl_7_19_3(self, family, socktype, protocol, address=None): + return self._socket_open_pycurl_7_21_5( + purpose=None, + address=collections.namedtuple( + 'Address', ['family', 'socktype', 'protocol', 'addr'], + )(family, socktype, protocol, address)) + + def _socket_open_pycurl_7_21_5(self, purpose, address): """Because pycurl doesn't have CURLOPT_TCP_KEEPALIVE""" - s = socket.socket(family, socktype, protocol) + s = socket.socket(address.family, address.socktype, address.protocol) s.setsockopt(socket.SOL_SOCKET, socket.SO_KEEPALIVE, 1) - s.setsockopt(socket.IPPROTO_TCP, socket.TCP_KEEPIDLE, 75) + # Will throw invalid protocol error on mac. This test prevents that. + if hasattr(socket, 'TCP_KEEPIDLE'): + s.setsockopt(socket.IPPROTO_TCP, socket.TCP_KEEPIDLE, 75) s.setsockopt(socket.IPPROTO_TCP, socket.TCP_KEEPINTVL, 75) + self._socket = s return s def get(self, locator, method="GET", timeout=None): @@ -382,22 +365,32 @@ class KeepClient(object): try: with timer.Timer() as t: self._headers = {} - response_body = cStringIO.StringIO() + response_body = BytesIO() curl.setopt(pycurl.NOSIGNAL, 1) - curl.setopt(pycurl.OPENSOCKETFUNCTION, self._socket_open) + curl.setopt(pycurl.OPENSOCKETFUNCTION, + lambda *args, **kwargs: self._socket_open(*args, **kwargs)) curl.setopt(pycurl.URL, url.encode('utf-8')) curl.setopt(pycurl.HTTPHEADER, [ - '{}: {}'.format(k,v) for k,v in self.get_headers.iteritems()]) + '{}: {}'.format(k,v) for k,v in self.get_headers.items()]) curl.setopt(pycurl.WRITEFUNCTION, response_body.write) curl.setopt(pycurl.HEADERFUNCTION, self._headerfunction) + if self.insecure: + curl.setopt(pycurl.SSL_VERIFYPEER, 0) + curl.setopt(pycurl.SSL_VERIFYHOST, 0) + else: + curl.setopt(pycurl.CAINFO, arvados.util.ca_certs_path()) if method == "HEAD": curl.setopt(pycurl.NOBODY, True) - self._setcurltimeouts(curl, timeout) + self._setcurltimeouts(curl, timeout, method=="HEAD") try: curl.perform() except Exception as e: raise arvados.errors.HttpError(0, str(e)) + finally: + if self._socket: + self._socket.close() + self._socket = None self._result = { 'status_code': curl.getinfo(pycurl.RESPONSE_CODE), 'body': response_body.getvalue(), @@ -432,13 +425,17 @@ class KeepClient(object): _logger.info("HEAD %s: %s bytes", self._result['status_code'], self._result.get('content-length')) + if self._result['headers'].get('x-keep-locator'): + # This is a response to a remote block copy request, return + # the local copy block locator. + return self._result['headers'].get('x-keep-locator') return True _logger.info("GET %s: %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) + 1.0*len(self._result['body'])/2**20/t.secs if t.secs > 0 else 0) if self.download_counter: self.download_counter.add(len(self._result['body'])) @@ -451,7 +448,9 @@ class KeepClient(object): return None return self._result['body'] - def put(self, hash_s, body, timeout=None): + def put(self, hash_s, body, timeout=None, headers={}): + put_headers = copy.copy(self.put_headers) + put_headers.update(headers) url = self.root + hash_s _logger.debug("Request: PUT %s", url) curl = self._get_user_agent() @@ -459,10 +458,11 @@ class KeepClient(object): try: with timer.Timer() as t: self._headers = {} - body_reader = cStringIO.StringIO(body) - response_body = cStringIO.StringIO() + body_reader = BytesIO(body) + response_body = BytesIO() curl.setopt(pycurl.NOSIGNAL, 1) - curl.setopt(pycurl.OPENSOCKETFUNCTION, self._socket_open) + curl.setopt(pycurl.OPENSOCKETFUNCTION, + lambda *args, **kwargs: self._socket_open(*args, **kwargs)) 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" @@ -474,17 +474,26 @@ class KeepClient(object): 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()]) + '{}: {}'.format(k,v) for k,v in put_headers.items()]) curl.setopt(pycurl.WRITEFUNCTION, response_body.write) curl.setopt(pycurl.HEADERFUNCTION, self._headerfunction) + if self.insecure: + curl.setopt(pycurl.SSL_VERIFYPEER, 0) + curl.setopt(pycurl.SSL_VERIFYHOST, 0) + else: + curl.setopt(pycurl.CAINFO, arvados.util.ca_certs_path()) self._setcurltimeouts(curl, timeout) try: curl.perform() except Exception as e: raise arvados.errors.HttpError(0, str(e)) + finally: + if self._socket: + self._socket.close() + self._socket = None self._result = { 'status_code': curl.getinfo(pycurl.RESPONSE_CODE), - 'body': response_body.getvalue(), + 'body': response_body.getvalue().decode('utf-8'), 'headers': self._headers, 'error': False, } @@ -511,12 +520,12 @@ class KeepClient(object): self._result['status_code'], len(body), t.msecs, - (len(body)/(1024.0*1024))/t.secs if t.secs > 0 else 0) + 1.0*len(body)/2**20/t.secs if t.secs > 0 else 0) if self.upload_counter: self.upload_counter.add(len(body)) return True - def _setcurltimeouts(self, curl, timeouts): + def _setcurltimeouts(self, curl, timeouts, ignore_bandwidth=False): if not timeouts: return elif isinstance(timeouts, tuple): @@ -529,11 +538,13 @@ class KeepClient(object): conn_t, xfer_t = (timeouts, timeouts) bandwidth_bps = KeepClient.DEFAULT_TIMEOUT[2] curl.setopt(pycurl.CONNECTTIMEOUT_MS, int(conn_t*1000)) - curl.setopt(pycurl.LOW_SPEED_TIME, int(math.ceil(xfer_t))) - curl.setopt(pycurl.LOW_SPEED_LIMIT, int(math.ceil(bandwidth_bps))) + if not ignore_bandwidth: + curl.setopt(pycurl.LOW_SPEED_TIME, int(math.ceil(xfer_t))) + curl.setopt(pycurl.LOW_SPEED_LIMIT, int(math.ceil(bandwidth_bps))) def _headerfunction(self, header_line): - header_line = header_line.decode('iso-8859-1') + if isinstance(header_line, bytes): + header_line = header_line.decode('iso-8859-1') if ':' in header_line: name, value = header_line.split(':', 1) name = name.strip().lower() @@ -552,66 +563,194 @@ class KeepClient(object): # Returning None implies all bytes were written + class KeepWriterQueue(queue.Queue): + def __init__(self, copies, classes=[]): + queue.Queue.__init__(self) # Old-style superclass + self.wanted_copies = copies + self.wanted_storage_classes = classes + self.successful_copies = 0 + self.confirmed_storage_classes = {} + self.response = None + self.storage_classes_tracking = True + self.queue_data_lock = threading.RLock() + self.pending_tries = max(copies, len(classes)) + self.pending_tries_notification = threading.Condition() + + def write_success(self, response, replicas_nr, classes_confirmed): + with self.queue_data_lock: + self.successful_copies += replicas_nr + if classes_confirmed is None: + self.storage_classes_tracking = False + elif self.storage_classes_tracking: + for st_class, st_copies in classes_confirmed.items(): + try: + self.confirmed_storage_classes[st_class] += st_copies + except KeyError: + self.confirmed_storage_classes[st_class] = st_copies + self.pending_tries = max(self.wanted_copies - self.successful_copies, len(self.pending_classes())) + self.response = response + with self.pending_tries_notification: + self.pending_tries_notification.notify_all() + + def write_fail(self, ks): + with self.pending_tries_notification: + self.pending_tries += 1 + self.pending_tries_notification.notify() + + def pending_copies(self): + with self.queue_data_lock: + return self.wanted_copies - self.successful_copies + + def satisfied_classes(self): + with self.queue_data_lock: + if not self.storage_classes_tracking: + # Notifies disabled storage classes expectation to + # the outer loop. + return None + return list(set(self.wanted_storage_classes) - set(self.pending_classes())) + + def pending_classes(self): + with self.queue_data_lock: + if (not self.storage_classes_tracking) or (self.wanted_storage_classes is None): + return [] + unsatisfied_classes = copy.copy(self.wanted_storage_classes) + for st_class, st_copies in self.confirmed_storage_classes.items(): + if st_class in unsatisfied_classes and st_copies >= self.wanted_copies: + unsatisfied_classes.remove(st_class) + return unsatisfied_classes + + def get_next_task(self): + with self.pending_tries_notification: + while True: + if self.pending_copies() < 1 and len(self.pending_classes()) == 0: + # This notify_all() is unnecessary -- + # write_success() already called notify_all() + # when pending<1 became true, so it's not + # possible for any other thread to be in + # wait() now -- but it's cheap insurance + # against deadlock so we do it anyway: + self.pending_tries_notification.notify_all() + # Drain the queue and then raise Queue.Empty + while True: + self.get_nowait() + self.task_done() + elif self.pending_tries > 0: + service, service_root = self.get_nowait() + if service.finished(): + self.task_done() + continue + self.pending_tries -= 1 + return service, service_root + elif self.empty(): + self.pending_tries_notification.notify_all() + raise queue.Empty + else: + self.pending_tries_notification.wait() + + + class KeepWriterThreadPool(object): + def __init__(self, data, data_hash, copies, max_service_replicas, timeout=None, classes=[]): + self.total_task_nr = 0 + if (not max_service_replicas) or (max_service_replicas >= copies): + num_threads = 1 + else: + num_threads = int(math.ceil(1.0*copies/max_service_replicas)) + _logger.debug("Pool max threads is %d", num_threads) + self.workers = [] + self.queue = KeepClient.KeepWriterQueue(copies, classes) + # Create workers + for _ in range(num_threads): + w = KeepClient.KeepWriterThread(self.queue, data, data_hash, timeout) + self.workers.append(w) + + def add_task(self, ks, service_root): + self.queue.put((ks, service_root)) + self.total_task_nr += 1 + + def done(self): + return self.queue.successful_copies, self.queue.satisfied_classes() + + def join(self): + # Start workers + for worker in self.workers: + worker.start() + # Wait for finished work + self.queue.join() + + def response(self): + return self.queue.response + + class KeepWriterThread(threading.Thread): - """ - Write a blob of data to the given Keep server. On success, call - save_response() of the given ThreadLimiter to save the returned - locator. - """ - def __init__(self, keep_service, **kwargs): - super(KeepClient.KeepWriterThread, self).__init__() - self.service = keep_service - self.args = kwargs - self._success = False + class TaskFailed(RuntimeError): pass - def success(self): - return self._success + def __init__(self, queue, data, data_hash, timeout=None): + super(KeepClient.KeepWriterThread, self).__init__() + self.timeout = timeout + self.queue = queue + self.data = data + self.data_hash = data_hash + self.daemon = True def run(self): - limiter = self.args['thread_limiter'] - sequence = self.args['thread_sequence'] - if sequence is not None: - limiter.set_sequence(sequence) - with limiter: - if not limiter.shall_i_proceed(): - # My turn arrived, but the job has been done without - # me. + while True: + try: + service, service_root = self.queue.get_next_task() + except queue.Empty: return - self.run_with_limiter(limiter) - - def run_with_limiter(self, limiter): - if self.service.finished(): - return - _logger.debug("KeepWriterThread %s proceeding %s+%i %s", - str(threading.current_thread()), - self.args['data_hash'], - len(self.args['data']), - self.args['service_root']) - self._success = bool(self.service.put( - self.args['data_hash'], - self.args['data'], - timeout=self.args.get('timeout', None))) - result = self.service.last_result() - if self._success: - _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(result['headers']['x-keep-replicas-stored']) - except (KeyError, ValueError): - replicas_stored = 1 - 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'], - result['status_code'], - result['body']) + locator, copies, classes = self.do_task(service, service_root) + except Exception as e: + if not isinstance(e, self.TaskFailed): + _logger.exception("Exception in KeepWriterThread") + self.queue.write_fail(service) + else: + self.queue.write_success(locator, copies, classes) + finally: + self.queue.task_done() + + def do_task(self, service, service_root): + classes = self.queue.pending_classes() + headers = {} + if len(classes) > 0: + classes.sort() + headers['X-Keep-Storage-Classes'] = ', '.join(classes) + success = bool(service.put(self.data_hash, + self.data, + timeout=self.timeout, + headers=headers)) + result = service.last_result() + + if not success: + if result.get('status_code'): + _logger.debug("Request fail: PUT %s => %s %s", + self.data_hash, + result.get('status_code'), + result.get('body')) + raise self.TaskFailed() + + _logger.debug("KeepWriterThread %s succeeded %s+%i %s", + str(threading.current_thread()), + self.data_hash, + len(self.data), + service_root) + try: + replicas_stored = int(result['headers']['x-keep-replicas-stored']) + except (KeyError, ValueError): + replicas_stored = 1 + + classes_confirmed = {} + try: + scch = result['headers']['x-keep-storage-classes-confirmed'] + for confirmation in scch.replace(' ', '').split(','): + if '=' in confirmation: + stored_class, stored_copies = confirmation.split('=')[:2] + classes_confirmed[stored_class] = int(stored_copies) + except (KeyError, ValueError): + # Storage classes confirmed header missing or corrupt + classes_confirmed = None + + return result['body'].strip(), replicas_stored, classes_confirmed def __init__(self, api_client=None, proxy=None, @@ -629,8 +768,9 @@ class KeepClient(object): :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. + ARVADOS_KEEP_SERVICES or ARVADOS_KEEP_PROXY configuration settings. + If you want to KeepClient does not use a proxy, pass in an empty + string. :timeout: The initial timeout (in seconds) for HTTP requests to Keep @@ -673,7 +813,10 @@ class KeepClient(object): """ self.lock = threading.Lock() if proxy is None: - proxy = config.get('ARVADOS_KEEP_PROXY') + if config.get('ARVADOS_KEEP_SERVICES'): + proxy = config.get('ARVADOS_KEEP_SERVICES') + else: + proxy = config.get('ARVADOS_KEEP_PROXY') if api_token is None: if api_client is None: api_token = config.get('ARVADOS_API_TOKEN') @@ -685,34 +828,48 @@ class KeepClient(object): if local_store is None: local_store = os.environ.get('KEEP_LOCAL_STORE') + if api_client is None: + self.insecure = config.flag_is_true('ARVADOS_API_HOST_INSECURE') + else: + self.insecure = api_client.insecure + self.block_cache = block_cache if block_cache else KeepBlockCache() self.timeout = timeout self.proxy_timeout = proxy_timeout - self._user_agent_pool = Queue.LifoQueue() + self._user_agent_pool = queue.LifoQueue() self.upload_counter = Counter() self.download_counter = Counter() self.put_counter = Counter() self.get_counter = Counter() self.hits_counter = Counter() self.misses_counter = Counter() + self._storage_classes_unsupported_warning = False + self._default_classes = [] if local_store: self.local_store = local_store + self.head = self.local_store_head self.get = self.local_store_get self.put = self.local_store_put else: self.num_retries = num_retries self.max_replicas_per_service = None if proxy: - if not proxy.endswith('/'): - proxy += '/' + proxy_uris = proxy.split() + for i in range(len(proxy_uris)): + if not proxy_uris[i].endswith('/'): + proxy_uris[i] += '/' + # URL validation + url = urllib.parse.urlparse(proxy_uris[i]) + if not (url.scheme and url.netloc): + raise arvados.errors.ArgumentError("Invalid proxy URI: {}".format(proxy_uris[i])) self.api_token = api_token self._gateway_services = {} self._keep_services = [{ - 'uuid': 'proxy', + 'uuid': "00000-bi6l4-%015d" % idx, 'service_type': 'proxy', - '_service_root': proxy, - }] + '_service_root': uri, + } for idx, uri in enumerate(proxy_uris)] self._writable_services = self._keep_services self.using_proxy = True self._static_services_list = True @@ -728,6 +885,12 @@ class KeepClient(object): self._writable_services = None self.using_proxy = None self._static_services_list = False + try: + self._default_classes = [ + k for k, v in self.api_client.config()['StorageClasses'].items() if v['Default']] + except KeyError: + # We're talking to an old cluster + pass def current_timeout(self, attempt_number): """Return the appropriate timeout to use for this client. @@ -769,7 +932,7 @@ class KeepClient(object): raise arvados.errors.NoKeepServersError() # Precompute the base URI for each service. - for r in self._gateway_services.itervalues(): + for r in self._gateway_services.values(): host = r['service_host'] if not host.startswith('[') and host.find(':') >= 0: # IPv6 URIs must be formatted like http://[::1]:80/... @@ -781,7 +944,7 @@ class KeepClient(object): _logger.debug(str(self._gateway_services)) self._keep_services = [ - ks for ks in self._gateway_services.itervalues() + ks for ks in self._gateway_services.values() if not ks.get('service_type', '').startswith('gateway:')] self._writable_services = [ks for ks in self._keep_services if not ks.get('read_only')] @@ -800,7 +963,7 @@ class KeepClient(object): 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() + return hashlib.md5((data_hash + service_uuid[-15:]).encode()).hexdigest() def weighted_service_roots(self, locator, force_rebuild=False, need_writable=False): """Return an array of Keep service endpoints, in the order in @@ -838,7 +1001,7 @@ class KeepClient(object): _logger.debug("{}: {}".format(locator, sorted_roots)) return sorted_roots - def map_new_services(self, roots_map, locator, force_rebuild, need_writable, **headers): + 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 @@ -851,7 +1014,8 @@ class KeepClient(object): root, self._user_agent_pool, upload_counter=self.upload_counter, download_counter=self.download_counter, - **headers) + headers=headers, + insecure=self.insecure) return local_roots @staticmethod @@ -872,23 +1036,33 @@ class KeepClient(object): else: return None - def get_from_cache(self, loc): + def get_from_cache(self, loc_s): """Fetch a block only if is in the cache, otherwise return None.""" - slot = self.block_cache.get(loc) + locator = KeepLocator(loc_s) + slot = self.block_cache.get(locator.md5sum) if slot is not None and slot.ready.is_set(): return slot.get() else: return None + def has_cache_slot(self, loc_s): + locator = KeepLocator(loc_s) + return self.block_cache.get(locator.md5sum) is not None + + def refresh_signature(self, loc): + """Ask Keep to get the remote block and return its local signature""" + now = datetime.datetime.utcnow().isoformat("T") + 'Z' + return self.head(loc, headers={'X-Keep-Signature': 'local, {}'.format(now)}) + @retry.retry_method - def head(self, loc_s, num_retries=None): - return self._get_or_head(loc_s, method="HEAD", num_retries=num_retries) + def head(self, loc_s, **kwargs): + return self._get_or_head(loc_s, method="HEAD", **kwargs) @retry.retry_method - def get(self, loc_s, num_retries=None): - return self._get_or_head(loc_s, method="GET", num_retries=num_retries) + def get(self, loc_s, **kwargs): + return self._get_or_head(loc_s, method="GET", **kwargs) - def _get_or_head(self, loc_s, method="GET", num_retries=None): + def _get_or_head(self, loc_s, method="GET", num_retries=None, request_id=None, headers=None): """Get data from Keep. This method fetches one or more blocks of data from Keep. It @@ -913,76 +1087,89 @@ class KeepClient(object): self.get_counter.add(1) - locator = KeepLocator(loc_s) - if method == "GET": - slot, first = self.block_cache.reserve_cache(locator.md5sum) - if not first: - self.hits_counter.add(1) - v = slot.get() - return v - - self.misses_counter.add(1) - - # 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, - upload_counter=self.upload_counter, - download_counter=self.download_counter) - for root in hint_roots - } + request_id = (request_id or + (hasattr(self, 'api_client') and self.api_client.request_id) or + arvados.util.new_request_id()) + if headers is None: + headers = {} + headers['X-Request-Id'] = request_id - # 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. - sorted_roots = [] - roots_map = {} + slot = None blob = None - loop = retry.RetryLoop(num_retries, self._check_loop_result, - backoff_start=2) - for tries_left in loop: - try: - 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 + try: + locator = KeepLocator(loc_s) + if method == "GET": + slot, first = self.block_cache.reserve_cache(locator.md5sum) + if not first: + self.hits_counter.add(1) + blob = slot.get() + if blob is None: + raise arvados.errors.KeepReadError( + "failed to read {}".format(loc_s)) + return blob + + self.misses_counter.add(1) + + # 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, + upload_counter=self.upload_counter, + download_counter=self.download_counter, + headers=headers, + insecure=self.insecure) + 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. + sorted_roots = [] + roots_map = {} + loop = retry.RetryLoop(num_retries, self._check_loop_result, + backoff_start=2) + for tries_left in loop: + try: + sorted_roots = self.map_new_services( + roots_map, locator, + force_rebuild=(tries_left < num_retries), + need_writable=False, + headers=headers) + except Exception as error: + loop.save_result(error) + continue - # Query KeepService objects that haven't returned - # permanent failure, in our specified shuffle order. - services_to_try = [roots_map[root] - for root in sorted_roots - if roots_map[root].usable()] - for keep_service in services_to_try: - blob = keep_service.get(locator, method=method, timeout=self.current_timeout(num_retries-tries_left)) - if blob is not None: - break - loop.save_result((blob, len(services_to_try))) - - # Always cache the result, then return it if we succeeded. - if method == "GET": - slot.set(blob) - self.block_cache.cap_cache() - if loop.success(): - if method == "HEAD": - return True - else: + # Query KeepService objects that haven't returned + # permanent failure, in our specified shuffle order. + services_to_try = [roots_map[root] + for root in sorted_roots + if roots_map[root].usable()] + for keep_service in services_to_try: + blob = keep_service.get(locator, method=method, timeout=self.current_timeout(num_retries-tries_left)) + if blob is not None: + break + loop.save_result((blob, len(services_to_try))) + + # Always cache the result, then return it if we succeeded. + if loop.success(): return blob + finally: + if slot is not None: + slot.set(blob) + self.block_cache.cap_cache() # Q: Including 403 is necessary for the Keep tests to continue # passing, but maybe they should expect KeepReadError instead? @@ -992,17 +1179,17 @@ class KeepClient(object): 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())) + "[{}] failed to read {}: no Keep services available ({})".format( + request_id, loc_s, loop.last_result())) elif not_founds == len(sorted_roots): raise arvados.errors.NotFoundError( - "{} not found".format(loc_s), service_errors) + "[{}] {} not found".format(request_id, loc_s), service_errors) else: raise arvados.errors.KeepReadError( - "failed to read {}".format(loc_s), service_errors, label="service") + "[{}] failed to read {} after {}".format(request_id, loc_s, loop.attempts_str()), service_errors, label="service") @retry.retry_method - def put(self, data, copies=2, num_retries=None): + def put(self, data, copies=2, num_retries=None, request_id=None, classes=None): """Save data in Keep. This method will get a list of Keep services from the API server, and @@ -1019,12 +1206,14 @@ class KeepClient(object): *each* Keep server if it returns temporary failures, with exponential backoff. The default value is set when the KeepClient is initialized. + * classes: An optional list of storage class names where copies should + be written. """ - 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'") + classes = classes or self._default_classes + + if not isinstance(data, bytes): + data = data.encode() self.put_counter.add(1) @@ -1034,59 +1223,78 @@ class KeepClient(object): return loc_s locator = KeepLocator(loc_s) - headers = {} - # Tell the proxy how many copies we want it to store - headers['X-Keep-Desired-Replication'] = str(copies) + request_id = (request_id or + (hasattr(self, 'api_client') and self.api_client.request_id) or + arvados.util.new_request_id()) + headers = { + 'X-Request-Id': request_id, + 'X-Keep-Desired-Replicas': str(copies), + } roots_map = {} loop = retry.RetryLoop(num_retries, self._check_loop_result, backoff_start=2) - done = 0 + done_copies = 0 + done_classes = [] for tries_left in loop: try: sorted_roots = self.map_new_services( roots_map, locator, - force_rebuild=(tries_left < num_retries), need_writable=True, **headers) + force_rebuild=(tries_left < num_retries), + need_writable=True, + headers=headers) except Exception as error: loop.save_result(error) continue - thread_limiter = KeepClient.ThreadLimiter( - copies - done, self.max_replicas_per_service) - threads = [] + pending_classes = [] + if done_classes is not None: + pending_classes = list(set(classes) - set(done_classes)) + writer_pool = KeepClient.KeepWriterThreadPool(data=data, + data_hash=data_hash, + copies=copies - done_copies, + max_service_replicas=self.max_replicas_per_service, + timeout=self.current_timeout(num_retries - tries_left), + classes=pending_classes) for service_root, ks in [(root, roots_map[root]) for root in sorted_roots]: if ks.finished(): continue - t = KeepClient.KeepWriterThread( - ks, - data=data, - data_hash=data_hash, - service_root=service_root, - thread_limiter=thread_limiter, - timeout=self.current_timeout(num_retries-tries_left), - thread_sequence=len(threads)) - t.start() - threads.append(t) - for t in threads: - t.join() - done += thread_limiter.done() - loop.save_result((done >= copies, len(threads))) + writer_pool.add_task(ks, service_root) + writer_pool.join() + pool_copies, pool_classes = writer_pool.done() + done_copies += pool_copies + if (done_classes is not None) and (pool_classes is not None): + done_classes += pool_classes + loop.save_result( + (done_copies >= copies and set(done_classes) == set(classes), + writer_pool.total_task_nr)) + else: + # Old keepstore contacted without storage classes support: + # success is determined only by successful copies. + # + # Disable storage classes tracking from this point forward. + if not self._storage_classes_unsupported_warning: + self._storage_classes_unsupported_warning = True + _logger.warning("X-Keep-Storage-Classes header not supported by the cluster") + done_classes = None + loop.save_result( + (done_copies >= copies, writer_pool.total_task_nr)) if loop.success(): - return thread_limiter.response() + return writer_pool.response() if not roots_map: raise arvados.errors.KeepWriteError( - "failed to write {}: no Keep services available ({})".format( - data_hash, loop.last_result())) + "[{}] failed to write {}: no Keep services available ({})".format( + request_id, data_hash, loop.last_result())) else: service_errors = ((key, roots_map[key].last_result()['error']) for key in sorted_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") + "[{}] failed to write {} after {} (wanted {} copies but wrote {})".format( + request_id, data_hash, loop.attempts_str(), (copies, classes), writer_pool.done()), service_errors, label="service") - def local_store_put(self, data, copies=1, num_retries=None): + def local_store_put(self, data, copies=1, num_retries=None, classes=[]): """A stub for put(). This method is used in place of the real put() method when @@ -1100,7 +1308,7 @@ class KeepClient(object): """ md5 = hashlib.md5(data).hexdigest() locator = '%s+%d' % (md5, len(data)) - with open(os.path.join(self.local_store, md5 + '.tmp'), 'w') as f: + with open(os.path.join(self.local_store, md5 + '.tmp'), 'wb') as f: f.write(data) os.rename(os.path.join(self.local_store, md5 + '.tmp'), os.path.join(self.local_store, md5)) @@ -1114,9 +1322,18 @@ class KeepClient(object): raise arvados.errors.NotFoundError( "Invalid data locator: '%s'" % loc_s) if locator.md5sum == config.EMPTY_BLOCK_LOCATOR.split('+')[0]: - return '' - with open(os.path.join(self.local_store, locator.md5sum), 'r') as f: + return b'' + with open(os.path.join(self.local_store, locator.md5sum), 'rb') as f: return f.read() - def is_cached(self, locator): - return self.block_cache.reserve_cache(expect_hash) + def local_store_head(self, loc_s, num_retries=None): + """Companion to local_store_put().""" + try: + locator = KeepLocator(loc_s) + except ValueError: + raise arvados.errors.NotFoundError( + "Invalid data locator: '%s'" % loc_s) + if locator.md5sum == config.EMPTY_BLOCK_LOCATOR.split('+')[0]: + return True + if os.path.exists(os.path.join(self.local_store, locator.md5sum)): + return True