From a7e8ec28bcaec4e06b700819fff7515299db5c02 Mon Sep 17 00:00:00 2001 From: Daniel Mitterdorfer Date: Wed, 5 Feb 2020 10:30:41 +0100 Subject: [PATCH 01/24] Add async load generator (WIP) With this commit we add an async load generator implementation. This implementation is work in progress, extremely incomplete and hacky. We also implement an async compatibility layer into the previous load generator which allows us to compare both load generator implementations in realistic scenarios. --- esrally/async_connection.py | 128 ++++++++ esrally/client.py | 21 ++ esrally/driver/__init__.py | 3 + esrally/driver/async_driver.py | 574 +++++++++++++++++++++++++++++++++ esrally/driver/driver.py | 64 +++- esrally/driver/runner.py | 304 ++++++++++++++++- esrally/racecontrol.py | 5 + esrally/rally.py | 11 +- esrally/track/params.py | 8 +- esrally/utils/io.py | 40 +++ setup.py | 1 + 11 files changed, 1146 insertions(+), 13 deletions(-) create mode 100644 esrally/async_connection.py create mode 100644 esrally/driver/async_driver.py diff --git a/esrally/async_connection.py b/esrally/async_connection.py new file mode 100644 index 000000000..29a57c10d --- /dev/null +++ b/esrally/async_connection.py @@ -0,0 +1,128 @@ +import asyncio +import ssl +import warnings + +import aiohttp +from aiohttp.client_exceptions import ServerFingerprintMismatch +import async_timeout + +from elasticsearch.exceptions import ConnectionError, ConnectionTimeout, ImproperlyConfigured, SSLError +from elasticsearch.connection import Connection +from elasticsearch.compat import urlencode +from elasticsearch.connection.http_urllib3 import create_ssl_context + + +# This is only needed because https://github.com/elastic/elasticsearch-py-async/pull/68 is not merged yet +# In addition we have raised the connection limit in TCPConnector from 100 to 10000. +class AIOHttpConnection(Connection): + def __init__(self, host='localhost', port=9200, http_auth=None, + use_ssl=False, verify_certs=False, ca_certs=None, client_cert=None, + client_key=None, loop=None, use_dns_cache=True, headers=None, + ssl_context=None, **kwargs): + super().__init__(host=host, port=port, **kwargs) + + self.loop = asyncio.get_event_loop() if loop is None else loop + + if http_auth is not None: + if isinstance(http_auth, str): + http_auth = tuple(http_auth.split(':', 1)) + + if isinstance(http_auth, (tuple, list)): + http_auth = aiohttp.BasicAuth(*http_auth) + + headers = headers or {} + headers.setdefault('content-type', 'application/json') + + # if providing an SSL context, raise error if any other SSL related flag is used + if ssl_context and (verify_certs or ca_certs): + raise ImproperlyConfigured("When using `ssl_context`, `use_ssl`, `verify_certs`, `ca_certs` are not permitted") + + if use_ssl or ssl_context: + cafile = ca_certs + if not cafile and not ssl_context and verify_certs: + # If no ca_certs and no sslcontext passed and asking to verify certs + # raise error + raise ImproperlyConfigured("Root certificates are missing for certificate " + "validation. Either pass them in using the ca_certs parameter or " + "install certifi to use it automatically.") + if verify_certs or ca_certs: + warnings.warn('Use of `verify_certs`, `ca_certs` have been deprecated in favor of using SSLContext`', DeprecationWarning) + + if not ssl_context: + # if SSLContext hasn't been passed in, create one. + # need to skip if sslContext isn't avail + try: + ssl_context = create_ssl_context(cafile=cafile) + except AttributeError: + ssl_context = None + + if not verify_certs and ssl_context is not None: + ssl_context.check_hostname = False + ssl_context.verify_mode = ssl.CERT_NONE + warnings.warn( + 'Connecting to %s using SSL with verify_certs=False is insecure.' % host) + if ssl_context: + verify_certs = True + use_ssl = True + + self.session = aiohttp.ClientSession( + auth=http_auth, + timeout=self.timeout, + connector=aiohttp.TCPConnector( + loop=self.loop, + verify_ssl=verify_certs, + use_dns_cache=use_dns_cache, + ssl_context=ssl_context, + # this has been changed from the default (100) + limit=100000 + ), + headers=headers + ) + + self.base_url = 'http%s://%s:%d%s' % ( + 's' if use_ssl else '', + host, port, self.url_prefix + ) + + @asyncio.coroutine + def close(self): + yield from self.session.close() + + @asyncio.coroutine + def perform_request(self, method, url, params=None, body=None, timeout=None, ignore=(), headers=None): + url_path = url + if params: + url_path = '%s?%s' % (url, urlencode(params or {})) + url = self.base_url + url_path + + start = self.loop.time() + response = None + try: + with async_timeout.timeout(timeout or self.timeout.total, loop=self.loop): + response = yield from self.session.request(method, url, data=body, headers=headers) + raw_data = yield from response.text() + duration = self.loop.time() - start + + except asyncio.CancelledError: + raise + + except Exception as e: + self.log_request_fail(method, url, url_path, body, self.loop.time() - start, exception=e) + if isinstance(e, ServerFingerprintMismatch): + raise SSLError('N/A', str(e), e) + if isinstance(e, asyncio.TimeoutError): + raise ConnectionTimeout('TIMEOUT', str(e), e) + raise ConnectionError('N/A', str(e), e) + + finally: + if response is not None: + yield from response.release() + + # raise errors based on http status codes, let the client handle those if needed + if not (200 <= response.status < 300) and response.status not in ignore: + self.log_request_fail(method, url, url_path, body, duration, status_code=response.status, response=raw_data) + self._raise_error(response.status, raw_data) + + self.log_request_success(method, url, url_path, body, response.status, raw_data, duration) + + return response.status, response.headers, raw_data diff --git a/esrally/client.py b/esrally/client.py index 43bed68ab..ce1aa615b 100644 --- a/esrally/client.py +++ b/esrally/client.py @@ -127,6 +127,27 @@ def create(self): import elasticsearch return elasticsearch.Elasticsearch(hosts=self.hosts, ssl_context=self.ssl_context, **self.client_options) + def create_async(self): + import elasticsearch + import elasticsearch_async + from aiohttp.client import ClientTimeout + import esrally.async_connection + + # needs patching as https://github.com/elastic/elasticsearch-py-async/pull/68 is not merged yet + class RallyAsyncTransport(elasticsearch_async.transport.AsyncTransport): + def __init__(self, hosts, connection_class=esrally.async_connection.AIOHttpConnection, loop=None, + connection_pool_class=elasticsearch_async.connection_pool.AsyncConnectionPool, + sniff_on_start=False, raise_on_sniff_error=True, **kwargs): + super().__init__(hosts, connection_class, loop, connection_pool_class, sniff_on_start, raise_on_sniff_error, **kwargs) + + if "timeout" in self.client_options and not isinstance(self.client_options["timeout"], ClientTimeout): + self.client_options["timeout"] = ClientTimeout(total=self.client_options["timeout"]) + + return elasticsearch_async.AsyncElasticsearch(hosts=self.hosts, + transport_class=RallyAsyncTransport, + ssl_context=self.ssl_context, + **self.client_options) + def wait_for_rest_layer(es, max_attempts=40): """ diff --git a/esrally/driver/__init__.py b/esrally/driver/__init__.py index c438d3c06..00da31a1c 100644 --- a/esrally/driver/__init__.py +++ b/esrally/driver/__init__.py @@ -17,3 +17,6 @@ # expose only the minimum API from .driver import DriverActor, PrepareBenchmark, PreparationComplete, StartBenchmark, BenchmarkComplete, TaskFinished + +# async API +from .async_driver import race diff --git a/esrally/driver/async_driver.py b/esrally/driver/async_driver.py new file mode 100644 index 000000000..8e7f22b36 --- /dev/null +++ b/esrally/driver/async_driver.py @@ -0,0 +1,574 @@ +# Licensed to Elasticsearch B.V. under one or more contributor +# license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright +# ownership. Elasticsearch B.V. licenses this file to you under +# the Apache License, Version 2.0 (the "License"); you may +# not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + + +import logging +from esrally.driver import driver, runner, scheduler +import time +import asyncio +import threading + +from esrally import exceptions, metrics, track, client, PROGRAM_NAME, telemetry +from esrally.utils import console, convert +import concurrent.futures + + +# TODO: Inline this code later +class PseudoActor: + def __init__(self, cfg, current_race): + self.cfg = cfg + self.race = current_race + + def on_cluster_details_retrieved(self, cluster_details): + #self.cluster_details = cluster_details + pass + + def on_benchmark_complete(self, metrics_store): + # TODO: Should we do this in race control instead? + from esrally import reporter + final_results = metrics.calculate_results(metrics_store, self.race) + metrics.results_store(self.cfg).store_results(self.race) + reporter.summarize(final_results, self.cfg) + + +def race(cfg): + logger = logging.getLogger(__name__) + # TODO: Taken from BenchmarkActor#setup() + t = track.load_track(cfg) + track_revision = cfg.opts("track", "repository.revision", mandatory=False) + challenge_name = cfg.opts("track", "challenge.name") + challenge = t.find_challenge_or_default(challenge_name) + if challenge is None: + raise exceptions.SystemSetupError("Track [%s] does not provide challenge [%s]. List the available tracks with %s list tracks." + % (t.name, challenge_name, PROGRAM_NAME)) + if challenge.user_info: + console.info(challenge.user_info) + current_race = metrics.create_race(cfg, t, challenge, track_revision) + + metrics_store = metrics.metrics_store( + cfg, + track=current_race.track_name, + challenge=current_race.challenge_name, + read_only=False + ) + race_store = metrics.race_store(cfg) + + a = PseudoActor(cfg, current_race) + + d = AsyncDriver(a, cfg) + logger.info("Preparing benchmark...") + cluster_info = d.prepare_benchmark(t) + # TODO: ensure we execute the code in after_track_prepared (from the original actor) + # def after_track_prepared(self): + # cluster_version = self.cluster_details["version"] if self.cluster_details else {} + # for child in self.children: + # self.send(child, thespian.actors.ActorExitRequest()) + # self.children = [] + # self.send(self.start_sender, driver.PreparationComplete( + # # older versions (pre 6.3.0) don't expose build_flavor because the only (implicit) flavor was "oss" + # cluster_version.get("build_flavor", "oss"), + # cluster_version.get("number", "Unknown"), + # cluster_version.get("build_hash", "Unknown") + # )) + logger.info("Running benchmark...") + d.start_benchmark() + + +# TODO: Move to time.py +class Timer: + def __init__(self, fn, interval, stop_event): + self.stop_event = stop_event + self.fn = fn + self.interval = interval + # check at least once a second whether we need to exit + self.wakeup_interval = min(self.interval, 1) + + def __call__(self, *args, **kwargs): + while not self.stop_event.is_set(): + self.fn(*args, **kwargs) + # allow early exit even if a longer sleeping period is requested + for _ in range(self.interval): + if self.stop_event.is_set(): + break + time.sleep(self.wakeup_interval) + + +class AsyncDriver: + def __init__(self, target, config, es_client_factory_class=client.EsClientFactory): + """ + Coordinates all workers. It is technology-agnostic, i.e. it does not know anything about actors. To allow us to hook in an actor, + we provide a ``target`` parameter which will be called whenever some event has occurred. The ``target`` can use this to send + appropriate messages. + + :param target: A target that will be notified of important events. + :param config: The current config object. + """ + self.logger = logging.getLogger(__name__) + self.target = target + self.config = config + self.es_client_factory = es_client_factory_class + self.track = None + self.challenge = None + self.metrics_store = None + self.drivers = [] + + self.progress_reporter = console.progress() + self.progress_counter = 0 + self.quiet = False + self.allocations = None + self.raw_samples = [] + self.throughput_calculator = None + self.most_recent_sample_per_client = {} + + self.number_of_steps = 0 + self.currently_completed = 0 + self.clients_completed_current_step = {} + self.current_step = -1 + self.tasks_per_join_point = None + self.complete_current_task_sent = False + self.current_tasks = [] + + self.telemetry = None + self.es_clients = None + + self._finished = False + self.abort_on_error = self.config.opts("driver", "on.error") == "abort" + self.pool = concurrent.futures.ThreadPoolExecutor(max_workers=4) + self.stop_timer_tasks = threading.Event() + self.sampler = None + + def create_es_clients(self, sync=True): + all_hosts = self.config.opts("client", "hosts").all_hosts + es = {} + for cluster_name, cluster_hosts in all_hosts.items(): + all_client_options = self.config.opts("client", "options").all_client_options + cluster_client_options = dict(all_client_options[cluster_name]) + # Use retries to avoid aborts on long living connections for telemetry devices + cluster_client_options["retry-on-timeout"] = True + + client_factory = self.es_client_factory(cluster_hosts, cluster_client_options) + if sync: + es[cluster_name] = client_factory.create() + else: + es[cluster_name] = client_factory.create_async() + return es + + def prepare_telemetry(self): + enabled_devices = self.config.opts("telemetry", "devices") + telemetry_params = self.config.opts("telemetry", "params") + + es = self.es_clients + es_default = self.es_clients["default"] + self.telemetry = telemetry.Telemetry(enabled_devices, devices=[ + telemetry.NodeStats(telemetry_params, es, self.metrics_store), + telemetry.ExternalEnvironmentInfo(es_default, self.metrics_store), + telemetry.ClusterEnvironmentInfo(es_default, self.metrics_store), + telemetry.JvmStatsSummary(es_default, self.metrics_store), + telemetry.IndexStats(es_default, self.metrics_store), + telemetry.MlBucketProcessingTime(es_default, self.metrics_store), + telemetry.CcrStats(telemetry_params, es, self.metrics_store), + telemetry.RecoveryStats(telemetry_params, es, self.metrics_store) + ]) + + def wait_for_rest_api(self): + skip_rest_api_check = self.config.opts("mechanic", "skip.rest.api.check") + if skip_rest_api_check: + self.logger.info("Skipping REST API check.") + else: + es_default = self.es_clients["default"] + self.logger.info("Checking if REST API is available.") + if client.wait_for_rest_layer(es_default, max_attempts=40): + self.logger.info("REST API is available.") + else: + self.logger.error("REST API layer is not yet available. Stopping benchmark.") + raise exceptions.SystemSetupError("Elasticsearch REST API layer is not available.") + + def retrieve_cluster_info(self): + try: + return self.es_clients["default"].info() + except BaseException: + self.logger.exception("Could not retrieve cluster info on benchmark start") + return None + + def prepare_benchmark(self, t): + self.track = t + self.challenge = driver.select_challenge(self.config, self.track) + self.quiet = self.config.opts("system", "quiet.mode", mandatory=False, default_value=False) + self.throughput_calculator = driver.ThroughputCalculator() + self.metrics_store = metrics.metrics_store(cfg=self.config, + track=self.track.name, + challenge=self.challenge.name, + read_only=False) + self.es_clients = self.create_es_clients() + self.wait_for_rest_api() + self.prepare_telemetry() + + if self.track.has_plugins: + track.track_repo(self.config, fetch=True, update=True) + # we also need to load track plugins eagerly as the respective parameter sources could require + track.load_track_plugins(self.config, runner.register_runner, scheduler.register_scheduler) + track.prepare_track(self.track, self.config) + + return self.retrieve_cluster_info() + + def start_benchmark(self): + self.logger.info("Benchmark is about to start.") + # ensure relative time starts when the benchmark starts. + self.reset_relative_time() + self.logger.info("Attaching cluster-level telemetry devices.") + self.telemetry.on_benchmark_start() + self.logger.info("Cluster-level telemetry devices are now attached.") + # TODO: Turn the intervals into constants + self.pool.submit(Timer(fn=self.update_samples, interval=1, stop_event=self.stop_timer_tasks)) + self.pool.submit(Timer(fn=self.post_process_samples, interval=30, stop_event=self.stop_timer_tasks)) + self.pool.submit(Timer(fn=self.update_progress_message, interval=1, stop_event=self.stop_timer_tasks)) + + # needed because a new thread does not have an event loop (see https://stackoverflow.com/questions/48725890/) + loop = asyncio.new_event_loop() + loop.set_debug(True) + asyncio.set_event_loop(loop) + loop.set_exception_handler(debug_exception_handler) + + track.set_absolute_data_path(self.config, self.track) + runner.register_default_runners() + # TODO: I think we can skip this here - it has already been done earlier in prepare_benchmark() + if self.track.has_plugins: + track.load_track_plugins(self.config, runner.register_runner, scheduler.register_scheduler) + success = False + try: + loop.run_until_complete(self.run_benchmark()) + + self._finished = True + self.telemetry.on_benchmark_stop() + self.logger.info("All steps completed.") + success = True + finally: + self.stop_timer_tasks.set() + self.pool.shutdown() + loop.close() + self.progress_reporter.finish() + if success: + self.target.on_benchmark_complete(self.metrics_store) + self.logger.debug("Closing metrics store...") + self.metrics_store.close() + # immediately clear as we don't need it anymore and it can consume a significant amount of memory + del self.metrics_store + + async def run_benchmark(self): + # avoid: aiohttp.internal WARNING The object should be created from async function + es = self.create_es_clients(sync=False) + try: + cancel = threading.Event() + # used to indicate that we want to prematurely consider this completed. This is *not* due to cancellation but a regular event in + # a benchmark and used to model task dependency of parallel tasks. + complete = threading.Event() + + for task in self.challenge.schedule: + for sub_task in task: + self.logger.info("Running task [%s] with [%d] clients...", sub_task.name, sub_task.clients) + #console.println("Running task [{}] with [{}] clients...".format(sub_task.name, sub_task.clients), logger=self.logger.info) + + # TODO: We need to restructure this later on: We could have only one sampler for the whole benchmark but then we need to + # provide the current task to the sampler. This would also simplify #update_samples(). We also need to move the + # join point (done, pending = await asyncio.wait(aws)) below one level out so we can actually run all sub-tasks of + # a task in parallel. At the moment we'd run one after the other (which is plain wrong) + self.current_tasks = [sub_task] + self.sampler = driver.Sampler(None, task, start_timestamp=time.perf_counter()) + aws = [] + # TODO: This is lacking support for one (sub)task being able to complete a complete parallel + # structure. We can probably achieve that by waiting for the task in question and then + # cancelling all other ongoing clients. + for client_id in range(sub_task.clients): + schedule = driver.schedule_for(self.track, sub_task, client_id) + e = AsyncExecutor(client_id, sub_task, schedule, es, self.sampler, cancel, complete, self.abort_on_error) + aws.append(e()) + # join point + done, pending = await asyncio.wait(aws) + self.logger.info("All clients have finished running task [%s]", sub_task.name) + # drain the active samples before we move on to the next task + self.update_samples() + self.post_process_samples() + self.reset_relative_time() + self.update_progress_message(task_finished=True) + + + #for client_index in range(start_client_index, start_client_index + sub_task.clients): + # this is the actual client that will execute the task. It may differ from the logical one in case we over-commit (i.e. + # more tasks than actually available clients) + # physical_client_index = client_index % max_clients + # if sub_task.completes_parent: + # clients_executing_completing_task.append(physical_client_index) + # allocations[physical_client_index].append(TaskAllocation(sub_task, client_index - start_client_index)) + # start_client_index += sub_task.clients + finally: + await asyncio.get_event_loop().shutdown_asyncgens() + await es["default"].transport.close() + + def reset_relative_time(self): + self.logger.debug("Resetting relative time of request metrics store.") + self.metrics_store.reset_relative_time() + + def close(self): + self.progress_reporter.finish() + if self.metrics_store and self.metrics_store.opened: + self.metrics_store.close() + + def update_samples(self): + if self.sampler: + samples = self.sampler.samples + self.logger.info("Adding [%d] new samples.", len(samples)) + if len(samples) > 0: + self.raw_samples += samples + # We need to check all samples, they will be from different clients + for s in samples: + self.most_recent_sample_per_client[s.client_id] = s + self.logger.info("Done adding [%d] new samples.", len(samples)) + else: + self.logger.info("No sampler defined yet. Skipping update of samples.") + + def update_progress_message(self, task_finished=False): + if not self.quiet and len(self.current_tasks) > 0: + tasks = ",".join([t.name for t in self.current_tasks]) + + if task_finished: + total_progress = 1.0 + else: + # we only count clients which actually contribute to progress. If clients are executing tasks eternally in a parallel + # structure, we should not count them. The reason is that progress depends entirely on the client(s) that execute the + # task that is completing the parallel structure. + progress_per_client = [s.percent_completed + for s in self.most_recent_sample_per_client.values() if s.percent_completed is not None] + + num_clients = max(len(progress_per_client), 1) + total_progress = sum(progress_per_client) / num_clients + self.progress_reporter.print("Running %s" % tasks, "[%3d%% done]" % (round(total_progress * 100))) + if task_finished: + self.progress_reporter.finish() + + def post_process_samples(self): + if len(self.raw_samples) == 0: + return + total_start = time.perf_counter() + start = total_start + # we do *not* do this here to avoid concurrent updates (actors are single-threaded) but rather to make it clear that we use + # only a snapshot and that new data will go to a new sample set. + raw_samples = self.raw_samples + self.raw_samples = [] + for sample in raw_samples: + meta_data = self.merge( + self.track.meta_data, + self.challenge.meta_data, + sample.operation.meta_data, + sample.task.meta_data, + sample.request_meta_data) + + self.metrics_store.put_value_cluster_level(name="latency", value=sample.latency_ms, unit="ms", task=sample.task.name, + operation=sample.operation.name, operation_type=sample.operation.type, + sample_type=sample.sample_type, absolute_time=sample.absolute_time, + relative_time=sample.relative_time, meta_data=meta_data) + + self.metrics_store.put_value_cluster_level(name="service_time", value=sample.service_time_ms, unit="ms", task=sample.task.name, + operation=sample.task.name, operation_type=sample.operation.type, + sample_type=sample.sample_type, absolute_time=sample.absolute_time, + relative_time=sample.relative_time, meta_data=meta_data) + + end = time.perf_counter() + self.logger.debug("Storing latency and service time took [%f] seconds.", (end - start)) + start = end + aggregates = self.throughput_calculator.calculate(raw_samples) + end = time.perf_counter() + self.logger.debug("Calculating throughput took [%f] seconds.", (end - start)) + start = end + for task, samples in aggregates.items(): + meta_data = self.merge( + self.track.meta_data, + self.challenge.meta_data, + task.operation.meta_data, + task.meta_data + ) + for absolute_time, relative_time, sample_type, throughput, throughput_unit in samples: + self.metrics_store.put_value_cluster_level(name="throughput", value=throughput, unit=throughput_unit, task=task.name, + operation=task.operation.name, operation_type=task.operation.type, + sample_type=sample_type, absolute_time=absolute_time, + relative_time=relative_time, meta_data=meta_data) + end = time.perf_counter() + self.logger.debug("Storing throughput took [%f] seconds.", (end - start)) + start = end + # this will be a noop for the in-memory metrics store. If we use an ES metrics store however, this will ensure that we already send + # the data and also clear the in-memory buffer. This allows users to see data already while running the benchmark. In cases where + # it does not matter (i.e. in-memory) we will still defer this step until the end. + # + # Don't force refresh here in the interest of short processing times. We don't need to query immediately afterwards so there is + # no need for frequent refreshes. + self.metrics_store.flush(refresh=False) + end = time.perf_counter() + self.logger.debug("Flushing the metrics store took [%f] seconds.", (end - start)) + self.logger.debug("Postprocessing [%d] raw samples took [%f] seconds in total.", len(raw_samples), (end - total_start)) + + def merge(self, *args): + result = {} + for arg in args: + if arg is not None: + result.update(arg) + return result + + +def debug_exception_handler(loop, context): + logging.getLogger(__name__).error("Uncaught exception in event loop!! %s", context) + + +class AsyncFoo: + def __init__(self, loop): + from concurrent.futures import ThreadPoolExecutor + self.io_pool_exc = ThreadPoolExecutor() + self.loop = loop + + async def __call__(self, task): + yield self.loop.run_in_executor(self.io_pool_exc, task) + + +class BoundAsyncFoo: + def __init__(self, e, t): + self.loop = e.loop + self.io_pool_exc = e.io_pool_exc + self.t = t + + async def __call__(self): + yield self.loop.run_in_executor(self.io_pool_exc, self.t) + + +class AsyncExecutor: + def __init__(self, client_id, task, schedule, es, sampler, cancel, complete, abort_on_error=False): + """ + Executes tasks according to the schedule for a given operation. + + :param task: The task that is executed. + :param schedule: The schedule for this task. + :param es: Elasticsearch client that will be used to execute the operation. + :param sampler: A container to store raw samples. + :param cancel: A shared boolean that indicates we need to cancel execution. + :param complete: A shared boolean that indicates we need to prematurely complete execution. + """ + self.client_id = client_id + self.task = task + self.op = task.operation + self.schedule_handle = schedule + self.es = es + self.sampler = sampler + self.cancel = cancel + self.complete = complete + self.abort_on_error = abort_on_error + self.logger = logging.getLogger(__name__) + + async def __call__(self, *args, **kwargs): + total_start = time.perf_counter() + # lazily initialize the schedule + self.logger.debug("Initializing schedule for client id [%s].", self.client_id) + schedule = self.schedule_handle() + self.logger.debug("Entering main loop for client id [%s].", self.client_id) + # noinspection PyBroadException + try: + async for expected_scheduled_time, sample_type, percent_completed, runner, params in schedule: + #self.logger.info("Next iteration in main loop for client id %s (%s %% completed)", self.client_id, percent_completed) + if self.cancel.is_set(): + self.logger.info("User cancelled execution.") + break + absolute_expected_schedule_time = total_start + expected_scheduled_time + throughput_throttled = expected_scheduled_time > 0 + if throughput_throttled: + rest = absolute_expected_schedule_time - time.perf_counter() + if rest > 0: + await asyncio.sleep(rest) + start = time.perf_counter() + total_ops, total_ops_unit, request_meta_data = await execute_single(runner, self.es, params, self.abort_on_error) + stop = time.perf_counter() + service_time = stop - start + # Do not calculate latency separately when we don't throttle throughput. This metric is just confusing then. + latency = stop - absolute_expected_schedule_time if throughput_throttled else service_time + # last sample should bump progress to 100% if externally completed. + completed = self.complete.is_set() or runner.completed + if completed: + progress = 1.0 + elif runner.percent_completed: + progress = runner.percent_completed + else: + progress = percent_completed + self.sampler.add(sample_type, request_meta_data, convert.seconds_to_ms(latency), convert.seconds_to_ms(service_time), + total_ops, total_ops_unit, (stop - total_start), progress, client_id=self.client_id) + + if completed: + self.logger.info("Task is considered completed due to external event.") + break + except BaseException: + self.logger.exception("Could not execute schedule") + raise + finally: + # Actively set it if this task completes its parent + if self.task.completes_parent: + self.complete.set() + + +async def execute_single(runner, es, params, abort_on_error=False): + """ + Invokes the given runner once and provides the runner's return value in a uniform structure. + + :return: a triple of: total number of operations, unit of operations, a dict of request meta data (may be None). + """ + import elasticsearch + try: + # TODO: Make all runners async-aware - Can we run async runners as a "regular" function (to avoid duplicate implementations)? + with runner: + return_value = await runner(es, params) + if isinstance(return_value, tuple) and len(return_value) == 2: + total_ops, total_ops_unit = return_value + request_meta_data = {"success": True} + elif isinstance(return_value, dict): + total_ops = return_value.pop("weight", 1) + total_ops_unit = return_value.pop("unit", "ops") + request_meta_data = return_value + if "success" not in request_meta_data: + request_meta_data["success"] = True + else: + total_ops = 1 + total_ops_unit = "ops" + request_meta_data = {"success": True} + except elasticsearch.TransportError as e: + total_ops = 0 + total_ops_unit = "ops" + request_meta_data = { + "success": False, + "error-type": "transport" + } + # The ES client will sometimes return string like "N/A" or "TIMEOUT" for connection errors. + if isinstance(e.status_code, int): + request_meta_data["http-status"] = e.status_code + if e.info: + request_meta_data["error-description"] = "%s (%s)" % (e.error, e.info) + else: + request_meta_data["error-description"] = e.error + except KeyError as e: + logging.getLogger(__name__).exception("Cannot execute runner [%s]; most likely due to missing parameters.", str(runner)) + msg = "Cannot execute [%s]. Provided parameters are: %s. Error: [%s]." % (str(runner), list(params.keys()), str(e)) + raise exceptions.SystemSetupError(msg) + + if abort_on_error and not request_meta_data["success"]: + msg = "Request returned an error. Error type: %s" % request_meta_data.get("error-type", "Unknown") + description = request_meta_data.get("error-description") + if description: + msg += ", Description: %s" % description + raise exceptions.RallyAssertionError(msg) + return total_ops, total_ops_unit, request_meta_data diff --git a/esrally/driver/driver.py b/esrally/driver/driver.py index 6a0fbcbb0..7e7e78843 100644 --- a/esrally/driver/driver.py +++ b/esrally/driver/driver.py @@ -810,8 +810,8 @@ def drive(self): # Now we need to ensure that we start partitioning parameters correctly in both cases. And that means we need to start # from (client) index 0 in both cases instead of 0 for indexA and 4 for indexB. schedule = schedule_for(self.track, task_allocation.task, task_allocation.client_index_in_task) - - executor = Executor(task, schedule, self.es, self.sampler, self.cancel, self.complete, self.abort_on_error) + executor = AsyncIoAdapter(self.config, self.client_id, task, schedule, self.sampler, self.cancel, self.complete, self.abort_on_error) + #executor = Executor(task, schedule, self.es, self.sampler, self.cancel, self.complete, self.abort_on_error) final_executor = Profiler(executor, self.client_id, task) if profiling_enabled else executor self.executor_future = self.pool.submit(final_executor) @@ -848,9 +848,10 @@ def __init__(self, client_id, task, start_timestamp): self.q = queue.Queue(maxsize=16384) self.logger = logging.getLogger(__name__) - def add(self, sample_type, request_meta_data, latency_ms, service_time_ms, total_ops, total_ops_unit, time_period, percent_completed): + def add(self, sample_type, request_meta_data, latency_ms, service_time_ms, total_ops, total_ops_unit, time_period, percent_completed, client_id=None): try: - self.q.put_nowait(Sample(self.client_id, time.time(), time.perf_counter() - self.start_timestamp, self.task, + c = self.client_id if client_id is None else client_id + self.q.put_nowait(Sample(c, time.time(), time.perf_counter() - self.start_timestamp, self.task, sample_type, request_meta_data, latency_ms, service_time_ms, total_ops, total_ops_unit, time_period, percent_completed)) except queue.Full: @@ -1053,6 +1054,47 @@ def __call__(self, *args, **kwargs): self.profile_logger.info(profile) +class AsyncIoAdapter: + def __init__(self, cfg, client_id, sub_task, schedule, sampler, cancel, complete, abort_on_error): + self.cfg = cfg + self.client_id = client_id + self.sub_task = sub_task + self.schedule = schedule + self.sampler = sampler + self.cancel = cancel + self.complete = complete + self.abort_on_error = abort_on_error + + def __call__(self, *args, **kwargs): + import asyncio + # only possible in Python 3.7+ (has introduced get_running_loop) + # try: + # loop = asyncio.get_running_loop() + # except RuntimeError: + # loop = asyncio.new_event_loop() + # asyncio.set_event_loop(loop) + loop = asyncio.new_event_loop() + loop.set_debug(True) + asyncio.set_event_loop(loop) + try: + loop.run_until_complete(self.run()) + finally: + loop.close() + + async def run(self): + def es_clients(all_hosts, all_client_options): + es = {} + for cluster_name, cluster_hosts in all_hosts.items(): + es[cluster_name] = client.EsClientFactory(cluster_hosts, all_client_options[cluster_name]).create_async() + return es + + from esrally.driver import async_driver + + es = es_clients(self.cfg.opts("client", "hosts").all_hosts, self.cfg.opts("client", "options").all_client_options) + async_executor = async_driver.AsyncExecutor(self.client_id, self.sub_task, self.schedule, es, self.sampler, self.cancel, self.complete, self.abort_on_error) + return await async_executor() + + class Executor: def __init__(self, task, schedule, es, sampler, cancel, complete, abort_on_error=False): """ @@ -1404,10 +1446,16 @@ def __init__(self, task_name, sched, task_progress_control, runner, params): self.runner = runner self.params = params self.logger = logging.getLogger(__name__) - - def __call__(self): + # TODO: Can we offload the parameter source execution to a different thread / process? Is this too heavy-weight? + #from concurrent.futures import ThreadPoolExecutor + #import asyncio + #self.io_pool_exc = ThreadPoolExecutor(max_workers=1) + #self.loop = asyncio.get_event_loop() + + # TODO: This requires Python 3.6+ (see https://www.python.org/dev/peps/pep-0525/) + async def __call__(self): + # def __call__(self): next_scheduled = 0 - if self.task_progress_control.infinite: self.logger.info("Parameter source will determine when the schedule for [%s] terminates.", self.task_name) param_source_knows_progress = hasattr(self.params, "percent_completed") @@ -1416,6 +1464,7 @@ def __call__(self): try: # does not contribute at all to completion. Hence, we cannot define completion. percent_completed = self.params.percent_completed if param_source_knows_progress else None + #current_params = await self.loop.run_in_executor(self.io_pool_exc, self.params.params) yield (next_scheduled, self.task_progress_control.sample_type, percent_completed, self.runner, self.params.params()) next_scheduled = self.sched.next(next_scheduled) @@ -1430,6 +1479,7 @@ def __call__(self): str(self.task_progress_control), self.task_name) while not self.task_progress_control.completed: try: + #current_params = await self.loop.run_in_executor(self.io_pool_exc, self.params.params) yield (next_scheduled, self.task_progress_control.sample_type, self.task_progress_control.percent_completed, diff --git a/esrally/driver/runner.py b/esrally/driver/runner.py index 6fdfe150c..fe0a2ca7b 100644 --- a/esrally/driver/runner.py +++ b/esrally/driver/runner.py @@ -30,7 +30,8 @@ def register_default_runners(): - register_runner(track.OperationType.Bulk.name, BulkIndex()) + register_runner(track.OperationType.Bulk.name, AsyncBulkIndex()) + #register_runner(track.OperationType.Bulk.name, BulkIndex()) register_runner(track.OperationType.ForceMerge.name, ForceMerge()) register_runner(track.OperationType.IndicesStats.name, IndicesStats()) register_runner(track.OperationType.NodesStats.name, NodeStats()) @@ -261,6 +262,307 @@ def mandatory(params, key, op): " parameter source." % (str(op), key)) +class AsyncBulkIndex(Runner): + """ + Bulk indexes the given documents. + """ + + def __init__(self): + super().__init__() + + async def __call__(self, es, params): + """ + Runs one bulk indexing operation. + + :param es: The Elasticsearch client. + :param params: A hash with all parameters. See below for details. + :return: A hash with meta data for this bulk operation. See below for details. + + It expects a parameter dict with the following mandatory keys: + + * ``body``: containing all documents for the current bulk request. + * ``bulk-size``: the number of documents in this bulk. + * ``action_metadata_present``: if ``True``, assume that an action and metadata line is present (meaning only half of the lines + contain actual documents to index) + * ``index``: The name of the affected index in case ``action_metadata_present`` is ``False``. + * ``type``: The name of the affected type in case ``action_metadata_present`` is ``False``. + + The following keys are optional: + + * ``pipeline``: If present, runs the the specified ingest pipeline for this bulk. + * ``detailed-results``: If ``True``, the runner will analyze the response and add detailed meta-data. Defaults to ``False``. Note + that this has a very significant impact on performance and will very likely cause a bottleneck in the benchmark driver so please + be very cautious enabling this feature. Our own measurements have shown a median overhead of several thousand times (execution time + is in the single digit microsecond range when this feature is disabled and in the single digit millisecond range when this feature + is enabled; numbers based on a bulk size of 500 elements and no errors). For details please refer to the respective benchmarks + in ``benchmarks/driver``. + + + Returned meta data + ` + The following meta data are always returned: + + * ``index``: name of the affected index. May be `None` if it could not be derived. + * ``bulk-size``: bulk size, e.g. 5.000. + * ``bulk-request-size-bytes``: size of the full bulk requset in bytes + * ``total-document-size-bytes``: size of all documents contained in the bulk request in bytes + * ``weight``: operation-agnostic representation of the bulk size (used internally by Rally for throughput calculation). + * ``unit``: The unit in which to interpret ``bulk-size`` and ``weight``. Always "docs". + * ``success``: A boolean indicating whether the bulk request has succeeded. + * ``success-count``: Number of successfully processed items for this request (denoted in ``unit``). + * ``error-count``: Number of failed items for this request (denoted in ``unit``). + * ``took``` Value of the the ``took`` property in the bulk response. + + If ``detailed-results`` is ``True`` the following meta data are returned in addition: + + * ``ops``: A hash with the operation name as key (e.g. index, update, delete) and various counts as values. ``item-count`` contains + the total number of items for this key. Additionally, we return a separate counter each result (indicating e.g. the number of + created items, the number of deleted items etc.). + * ``shards_histogram``: An array of hashes where each hash has two keys: ``item-count`` contains the number of items to which a + shard distribution applies and ``shards`` contains another hash with the actual distribution of ``total``, ``successful`` and + ``failed`` shards (see examples below). + * ``bulk-request-size-bytes``: Total size of the bulk request body in bytes. + * ``total-document-size-bytes``: Total size of all documents within the bulk request body in bytes. + + Here are a few examples: + + If ``detailed-results`` is ``False`` a typical return value is:: + + { + "index": "my_index", + "weight": 5000, + "unit": "docs", + "bulk-size": 5000, + "success": True, + "success-count": 5000, + "error-count": 0, + "took": 20 + } + + Whereas the response will look as follow if there are bulk errors:: + + { + "index": "my_index", + "weight": 5000, + "unit": "docs", + "bulk-size": 5000, + "success": False, + "success-count": 4000, + "error-count": 1000, + "took": 20 + } + + If ``detailed-results`` is ``True`` a typical return value is:: + + + { + "index": "my_index", + "weight": 5000, + "unit": "docs", + "bulk-size": 5000, + "bulk-request-size-bytes": 2250000, + "total-document-size-bytes": 2000000, + "success": True, + "success-count": 5000, + "error-count": 0, + "took": 20, + "ops": { + "index": { + "item-count": 5000, + "created": 5000 + } + }, + "shards_histogram": [ + { + "item-count": 5000, + "shards": { + "total": 2, + "successful": 2, + "failed": 0 + } + } + ] + } + + An example error response may look like this:: + + + { + "index": "my_index", + "weight": 5000, + "unit": "docs", + "bulk-size": 5000, + "bulk-request-size-bytes": 2250000, + "total-document-size-bytes": 2000000, + "success": False, + "success-count": 4000, + "error-count": 1000, + "took": 20, + "ops": { + "index": { + "item-count": 5000, + "created": 4000, + "noop": 1000 + } + }, + "shards_histogram": [ + { + "item-count": 4000, + "shards": { + "total": 2, + "successful": 2, + "failed": 0 + } + }, + { + "item-count": 500, + "shards": { + "total": 2, + "successful": 1, + "failed": 1 + } + }, + { + "item-count": 500, + "shards": { + "total": 2, + "successful": 0, + "failed": 2 + } + } + ] + } + """ + detailed_results = params.get("detailed-results", False) + index = params.get("index") + + bulk_params = {} + if "pipeline" in params: + bulk_params["pipeline"] = params["pipeline"] + + with_action_metadata = mandatory(params, "action-metadata-present", self) + bulk_size = mandatory(params, "bulk-size", self) + + if with_action_metadata: + # only half of the lines are documents + response = await es.bulk(body=params["body"], params=bulk_params) + else: + response = await es.bulk(body=params["body"], index=index, doc_type=params.get("type"), params=bulk_params) + + stats = self.detailed_stats(params, bulk_size, response) if detailed_results else self.simple_stats(bulk_size, response) + + meta_data = { + "index": str(index) if index else None, + "weight": bulk_size, + "unit": "docs", + "bulk-size": bulk_size + } + meta_data.update(stats) + if not stats["success"]: + meta_data["error-type"] = "bulk" + return meta_data + + def detailed_stats(self, params, bulk_size, response): + ops = {} + shards_histogram = OrderedDict() + bulk_error_count = 0 + error_details = set() + bulk_request_size_bytes = 0 + total_document_size_bytes = 0 + with_action_metadata = mandatory(params, "action-metadata-present", self) + + for line_number, data in enumerate(params["body"]): + line_size = len(data.encode('utf-8')) + if with_action_metadata: + if line_number % 2 == 1: + total_document_size_bytes += line_size + else: + total_document_size_bytes += line_size + + bulk_request_size_bytes += line_size + + for idx, item in enumerate(response["items"]): + # there is only one (top-level) item + op, data = next(iter(item.items())) + if op not in ops: + ops[op] = Counter() + ops[op]["item-count"] += 1 + if "result" in data: + ops[op][data["result"]] += 1 + + if "_shards" in data: + s = data["_shards"] + sk = "%d-%d-%d" % (s["total"], s["successful"], s["failed"]) + if sk not in shards_histogram: + shards_histogram[sk] = { + "item-count": 0, + "shards": s + } + shards_histogram[sk]["item-count"] += 1 + if data["status"] > 299 or ("_shards" in data and data["_shards"]["failed"] > 0): + bulk_error_count += 1 + self.extract_error_details(error_details, data) + stats = { + "took": response.get("took"), + "success": bulk_error_count == 0, + "success-count": bulk_size - bulk_error_count, + "error-count": bulk_error_count, + "ops": ops, + "shards_histogram": list(shards_histogram.values()), + "bulk-request-size-bytes": bulk_request_size_bytes, + "total-document-size-bytes": total_document_size_bytes + } + if bulk_error_count > 0: + stats["error-type"] = "bulk" + stats["error-description"] = self.error_description(error_details) + if "ingest_took" in response: + stats["ingest_took"] = response["ingest_took"] + + return stats + + def simple_stats(self, bulk_size, response): + bulk_error_count = 0 + error_details = set() + if response["errors"]: + for idx, item in enumerate(response["items"]): + data = next(iter(item.values())) + if data["status"] > 299 or ('_shards' in data and data["_shards"]["failed"] > 0): + bulk_error_count += 1 + self.extract_error_details(error_details, data) + stats = { + "took": response.get("took"), + "success": bulk_error_count == 0, + "success-count": bulk_size - bulk_error_count, + "error-count": bulk_error_count + } + if "ingest_took" in response: + stats["ingest_took"] = response["ingest_took"] + if bulk_error_count > 0: + stats["error-type"] = "bulk" + stats["error-description"] = self.error_description(error_details) + return stats + + def extract_error_details(self, error_details, data): + error_data = data.get("error", {}) + error_reason = error_data.get("reason") if isinstance(error_data, dict) else str(error_data) + if error_data: + error_details.add((data["status"], error_reason)) + else: + error_details.add((data["status"], None)) + + def error_description(self, error_details): + error_description = "" + for status, reason in error_details: + if reason: + error_description += "HTTP status: %s, message: %s" % (str(status), reason) + else: + error_description += "HTTP status: %s" % str(status) + return error_description + + def __repr__(self, *args, **kwargs): + return "bulk-index" + + class BulkIndex(Runner): """ Bulk indexes the given documents. diff --git a/esrally/racecontrol.py b/esrally/racecontrol.py index 15e99eb80..7697389b0 100644 --- a/esrally/racecontrol.py +++ b/esrally/racecontrol.py @@ -368,3 +368,8 @@ def run(cfg): except BaseException: tb = sys.exc_info()[2] raise exceptions.RallyError("This race ended with a fatal crash.").with_traceback(tb) + + +def run_async(cfg): + # TODO: Provide everything that the BenchmarkActor provides + driver.race(cfg) diff --git a/esrally/rally.py b/esrally/rally.py index 55bb29600..5f2f7b7d7 100644 --- a/esrally/rally.py +++ b/esrally/rally.py @@ -67,6 +67,7 @@ def runtime_jdk(v): help="") race_parser = subparsers.add_parser("race", help="Run the benchmarking pipeline. This sub-command should typically be used.") + async_race_parser = subparsers.add_parser("race-async") # change in favor of "list telemetry", "list tracks", "list pipelines" list_parser = subparsers.add_parser("list", help="List configuration options") list_parser.add_argument( @@ -350,7 +351,7 @@ def runtime_jdk(v): default=preserve_install, action="store_true") - for p in [parser, list_parser, race_parser, generate_parser]: + for p in [parser, list_parser, race_parser, async_race_parser, generate_parser]: p.add_argument( "--distribution-version", help="Define the version of the Elasticsearch distribution to download. " @@ -390,7 +391,7 @@ def runtime_jdk(v): default=False, action="store_true") - for p in [parser, race_parser]: + for p in [parser, race_parser, async_race_parser]: p.add_argument( "--race-id", help="Define a unique id for this race.", @@ -521,7 +522,7 @@ def runtime_jdk(v): # The options below are undocumented and can be removed or changed at any time. # ############################################################################### - for p in [parser, race_parser]: + for p in [parser, race_parser, async_race_parser]: # This option is intended to tell Rally to assume a different start date than 'now'. This is effectively just useful for things like # backtesting or a benchmark run across environments (think: comparison of EC2 and bare metal) but never for the typical user. p.add_argument( @@ -543,7 +544,7 @@ def runtime_jdk(v): default=False) for p in [parser, config_parser, list_parser, race_parser, compare_parser, download_parser, install_parser, - start_parser, stop_parser, info_parser]: + start_parser, stop_parser, info_parser, async_race_parser]: # This option is needed to support a separate configuration for the integration tests on the same machine p.add_argument( "--configuration-name", @@ -712,6 +713,8 @@ def dispatch_sub_command(cfg, sub_command): mechanic.stop(cfg) elif sub_command == "race": race(cfg) + elif sub_command == "race-async": + racecontrol.run_async(cfg) elif sub_command == "generate": generate(cfg) elif sub_command == "info": diff --git a/esrally/track/params.py b/esrally/track/params.py index c7c079a14..09e648f86 100644 --- a/esrally/track/params.py +++ b/esrally/track/params.py @@ -563,6 +563,7 @@ def __init__(self, corpora, partition_index, total_partitions, batch_size, bulk_ all_bulks = number_of_bulks(self.corpora, self.partition_index, self.total_partitions, self.bulk_size) self.total_bulks = math.ceil((all_bulks * self.ingest_percentage) / 100) self.infinite = False + self.logger = logging.getLogger(__name__) def partition(self, partition_index, total_partitions): raise exceptions.RallyError("Cannot partition a PartitionBulkIndexParamSource further") @@ -573,7 +574,11 @@ def params(self): if self.current_bulk == self.total_bulks: raise StopIteration self.current_bulk += 1 - return next(self.internal_params) + start = time.perf_counter() + p = next(self.internal_params) + end = time.perf_counter() + self.logger.info("Reading bulk took [%s] seconds.", (end - start)) + return p @property def percent_completed(self): @@ -644,6 +649,7 @@ def chain(*iterables): def create_default_reader(docs, offset, num_lines, num_docs, batch_size, bulk_size, id_conflicts, conflict_probability, on_conflict, recency): + #source = Slice(io.StaticSource, offset, num_lines) source = Slice(io.FileSource, offset, num_lines) if docs.includes_action_and_meta_data: diff --git a/esrally/utils/io.py b/esrally/utils/io.py index 79407f376..efc37b3e3 100644 --- a/esrally/utils/io.py +++ b/esrally/utils/io.py @@ -135,6 +135,46 @@ def __str__(self, *args, **kwargs): return "StringAsFileSource" +class StaticSource: + def __init__(self, contents, mode, encoding="utf-8"): + self.contents = '{"geonameid": 2986043, "name": "Pic de Font Blanca", "asciiname": "Pic de Font Blanca", "alternatenames": "Pic de Font Blanca,Pic du Port", "feature_class": "T", "feature_code": "PK", "country_code": "AD", "admin1_code": "00", "population": 0, "dem": "2860", "timezone": "Europe/Andorra", "location": [1.53335, 42.64991]}' + self.current_index = 0 + self.opened = False + + def open(self): + self.opened = True + return self + + def seek(self, offset): + pass + + def read(self): + return "\n".join(self.contents) + + def readline(self): + return self.contents + + def close(self): + self._assert_opened() + self.contents = None + self.opened = False + + def _assert_opened(self): + assert self.opened + + def __enter__(self): + self.open() + return self + + def __exit__(self, exc_type, exc_val, exc_tb): + self.close() + return False + + def __str__(self, *args, **kwargs): + return "StaticSource" + + + def ensure_dir(directory, mode=0o777): """ Ensure that the provided directory and all of its parent directories exist. diff --git a/setup.py b/setup.py index d9e27ef6f..75da38d17 100644 --- a/setup.py +++ b/setup.py @@ -50,6 +50,7 @@ def str_from_file(name): # License: Apache 2.0 # transitive dependency urllib3: MIT "elasticsearch==7.0.5", + "elasticsearch-async==6.2.0", # License: BSD "psutil==5.6.5", # License: MIT From 25ab4d7c3113147f1a51a96cb7d5e9617ee58cac Mon Sep 17 00:00:00 2001 From: Daniel Mitterdorfer Date: Thu, 6 Feb 2020 10:13:53 +0100 Subject: [PATCH 02/24] Enable async driver profiling --- esrally/driver/async_driver.py | 35 +++++++++++++++++++++++++++++++++- setup.py | 4 +++- 2 files changed, 37 insertions(+), 2 deletions(-) diff --git a/esrally/driver/async_driver.py b/esrally/driver/async_driver.py index 8e7f22b36..ed37ba064 100644 --- a/esrally/driver/async_driver.py +++ b/esrally/driver/async_driver.py @@ -147,6 +147,7 @@ def __init__(self, target, config, es_client_factory_class=client.EsClientFactor self._finished = False self.abort_on_error = self.config.opts("driver", "on.error") == "abort" + self.profiling_enabled = self.config.opts("driver", "profiling") self.pool = concurrent.futures.ThreadPoolExecutor(max_workers=4) self.stop_timer_tasks = threading.Event() self.sampler = None @@ -250,7 +251,8 @@ def start_benchmark(self): track.load_track_plugins(self.config, runner.register_runner, scheduler.register_scheduler) success = False try: - loop.run_until_complete(self.run_benchmark()) + benchmark_runner = AsyncProfiler(self.run_benchmark) if self.profiling_enabled else self.run_benchmark + loop.run_until_complete(benchmark_runner()) self._finished = True self.telemetry.on_benchmark_stop() @@ -451,6 +453,37 @@ async def __call__(self): yield self.loop.run_in_executor(self.io_pool_exc, self.t) +class AsyncProfiler: + def __init__(self, target): + """ + :param target: The actual executor which should be profiled. + """ + self.target = target + self.profile_logger = logging.getLogger("rally.profile") + + async def __call__(self, *args, **kwargs): + import yappi + import io as python_io + yappi.start() + try: + return await self.target(*args, **kwargs) + finally: + yappi.stop() + s = python_io.StringIO() + yappi.get_func_stats().print_all(out=s, columns={ + 0: ("name", 200), + 1: ("ncall", 5), + 2: ("tsub", 8), + 3: ("ttot", 8), + 4: ("tavg", 8) + }) + + profile = "\n=== Profile START ===\n" + profile += s.getvalue() + profile += "=== Profile END ===" + self.profile_logger.info(profile) + + class AsyncExecutor: def __init__(self, client_id, task, schedule, es, sampler, cancel, complete, abort_on_error=False): """ diff --git a/setup.py b/setup.py index 75da38d17..6c24d07d9 100644 --- a/setup.py +++ b/setup.py @@ -74,7 +74,9 @@ def str_from_file(name): # botocore: Apache 2.0 # jmespath: MIT # s3transfer: Apache 2.0 - "boto3==1.10.32" + "boto3==1.10.32", + # TODO dm: Temporarily here - *might* be removed later + "yappi==1.2.3" ] tests_require = [ From 936f232ef0682d3c0915967ff68787497b3baa80 Mon Sep 17 00:00:00 2001 From: Daniel Mitterdorfer Date: Thu, 6 Feb 2020 10:20:56 +0100 Subject: [PATCH 03/24] Add uvloop support --- esrally/driver/async_driver.py | 8 ++++++++ esrally/rally.py | 8 ++++++++ setup.py | 3 ++- 3 files changed, 18 insertions(+), 1 deletion(-) diff --git a/esrally/driver/async_driver.py b/esrally/driver/async_driver.py index ed37ba064..2cd2b32a4 100644 --- a/esrally/driver/async_driver.py +++ b/esrally/driver/async_driver.py @@ -148,6 +148,7 @@ def __init__(self, target, config, es_client_factory_class=client.EsClientFactor self._finished = False self.abort_on_error = self.config.opts("driver", "on.error") == "abort" self.profiling_enabled = self.config.opts("driver", "profiling") + self.use_uvloop = self.config.opts("driver", "uvloop") self.pool = concurrent.futures.ThreadPoolExecutor(max_workers=4) self.stop_timer_tasks = threading.Event() self.sampler = None @@ -239,6 +240,13 @@ def start_benchmark(self): self.pool.submit(Timer(fn=self.update_progress_message, interval=1, stop_event=self.stop_timer_tasks)) # needed because a new thread does not have an event loop (see https://stackoverflow.com/questions/48725890/) + if self.use_uvloop: + self.logger.info("Using uvloop to perform asyncio.") + import uvloop + uvloop.install() + else: + self.logger.info("Using standard library implementation to perform asyncio.") + loop = asyncio.new_event_loop() loop.set_debug(True) asyncio.set_event_loop(loop) diff --git a/esrally/rally.py b/esrally/rally.py index 5f2f7b7d7..8e6ff4834 100644 --- a/esrally/rally.py +++ b/esrally/rally.py @@ -556,6 +556,13 @@ def runtime_jdk(v): default=False, action="store_true") + # TODO dm: Revisit later whether we still want this + async_race_parser.add_argument( + "--use-uvloop", + help=argparse.SUPPRESS, + default=False, + action="store_true") + return parser @@ -861,6 +868,7 @@ def main(): cfg.add(config.Scope.applicationOverride, "generator", "node.count", args.node_count) cfg.add(config.Scope.applicationOverride, "driver", "profiling", args.enable_driver_profiling) + cfg.add(config.Scope.applicationOverride, "driver", "uvloop", args.use_uvloop) cfg.add(config.Scope.applicationOverride, "driver", "on.error", args.on_error) cfg.add(config.Scope.applicationOverride, "driver", "load_driver_hosts", opts.csv_to_list(args.load_driver_hosts)) if sub_command not in ("list", "install", "download"): diff --git a/setup.py b/setup.py index 6c24d07d9..5f61671f1 100644 --- a/setup.py +++ b/setup.py @@ -76,7 +76,8 @@ def str_from_file(name): # s3transfer: Apache 2.0 "boto3==1.10.32", # TODO dm: Temporarily here - *might* be removed later - "yappi==1.2.3" + "yappi==1.2.3", + "uvloop==0.14.0" ] tests_require = [ From 5fc004259bb55f8f6ff56eb430b63493975e924b Mon Sep 17 00:00:00 2001 From: Daniel Mitterdorfer Date: Thu, 6 Feb 2020 10:39:10 +0100 Subject: [PATCH 04/24] Fine-tune profile log format --- esrally/driver/async_driver.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/esrally/driver/async_driver.py b/esrally/driver/async_driver.py index 2cd2b32a4..9d605ad5f 100644 --- a/esrally/driver/async_driver.py +++ b/esrally/driver/async_driver.py @@ -479,8 +479,8 @@ async def __call__(self, *args, **kwargs): yappi.stop() s = python_io.StringIO() yappi.get_func_stats().print_all(out=s, columns={ - 0: ("name", 200), - 1: ("ncall", 5), + 0: ("name", 140), + 1: ("ncall", 8), 2: ("tsub", 8), 3: ("ttot", 8), 4: ("tavg", 8) From a7780d39769d223ab8e92d893efcc118cc99c51d Mon Sep 17 00:00:00 2001 From: Daniel Mitterdorfer Date: Thu, 6 Feb 2020 10:59:46 +0100 Subject: [PATCH 05/24] Use uvloop only if async --- esrally/rally.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/esrally/rally.py b/esrally/rally.py index 8e6ff4834..42cbed81d 100644 --- a/esrally/rally.py +++ b/esrally/rally.py @@ -868,7 +868,8 @@ def main(): cfg.add(config.Scope.applicationOverride, "generator", "node.count", args.node_count) cfg.add(config.Scope.applicationOverride, "driver", "profiling", args.enable_driver_profiling) - cfg.add(config.Scope.applicationOverride, "driver", "uvloop", args.use_uvloop) + if sub_command == "race-async": + cfg.add(config.Scope.applicationOverride, "driver", "uvloop", args.use_uvloop) cfg.add(config.Scope.applicationOverride, "driver", "on.error", args.on_error) cfg.add(config.Scope.applicationOverride, "driver", "load_driver_hosts", opts.csv_to_list(args.load_driver_hosts)) if sub_command not in ("list", "install", "download"): From 7a4e5e500be658b5948cdddc8cec249082aacb7a Mon Sep 17 00:00:00 2001 From: Daniel Mitterdorfer Date: Mon, 17 Feb 2020 11:52:51 +0100 Subject: [PATCH 06/24] Implement readlines for StaticSource --- esrally/utils/io.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/esrally/utils/io.py b/esrally/utils/io.py index 4a2d0fc1b..1617e9515 100644 --- a/esrally/utils/io.py +++ b/esrally/utils/io.py @@ -173,6 +173,9 @@ def read(self): def readline(self): return self.contents + def readlines(self, num_lines): + return [self.contents] * num_lines + def close(self): self._assert_opened() self.contents = None From fe3f7abfb9f49ba1afde7fbca8a43b4e7dd49dc0 Mon Sep 17 00:00:00 2001 From: Daniel Mitterdorfer Date: Mon, 17 Feb 2020 13:18:53 +0100 Subject: [PATCH 07/24] Remove uvloop --- esrally/driver/async_driver.py | 28 ---------------------------- esrally/rally.py | 9 --------- setup.py | 3 +-- 3 files changed, 1 insertion(+), 39 deletions(-) diff --git a/esrally/driver/async_driver.py b/esrally/driver/async_driver.py index 9d605ad5f..9a5f0d17c 100644 --- a/esrally/driver/async_driver.py +++ b/esrally/driver/async_driver.py @@ -148,7 +148,6 @@ def __init__(self, target, config, es_client_factory_class=client.EsClientFactor self._finished = False self.abort_on_error = self.config.opts("driver", "on.error") == "abort" self.profiling_enabled = self.config.opts("driver", "profiling") - self.use_uvloop = self.config.opts("driver", "uvloop") self.pool = concurrent.futures.ThreadPoolExecutor(max_workers=4) self.stop_timer_tasks = threading.Event() self.sampler = None @@ -240,13 +239,6 @@ def start_benchmark(self): self.pool.submit(Timer(fn=self.update_progress_message, interval=1, stop_event=self.stop_timer_tasks)) # needed because a new thread does not have an event loop (see https://stackoverflow.com/questions/48725890/) - if self.use_uvloop: - self.logger.info("Using uvloop to perform asyncio.") - import uvloop - uvloop.install() - else: - self.logger.info("Using standard library implementation to perform asyncio.") - loop = asyncio.new_event_loop() loop.set_debug(True) asyncio.set_event_loop(loop) @@ -441,26 +433,6 @@ def debug_exception_handler(loop, context): logging.getLogger(__name__).error("Uncaught exception in event loop!! %s", context) -class AsyncFoo: - def __init__(self, loop): - from concurrent.futures import ThreadPoolExecutor - self.io_pool_exc = ThreadPoolExecutor() - self.loop = loop - - async def __call__(self, task): - yield self.loop.run_in_executor(self.io_pool_exc, task) - - -class BoundAsyncFoo: - def __init__(self, e, t): - self.loop = e.loop - self.io_pool_exc = e.io_pool_exc - self.t = t - - async def __call__(self): - yield self.loop.run_in_executor(self.io_pool_exc, self.t) - - class AsyncProfiler: def __init__(self, target): """ diff --git a/esrally/rally.py b/esrally/rally.py index bf7ceabb2..4aa56ace4 100644 --- a/esrally/rally.py +++ b/esrally/rally.py @@ -556,13 +556,6 @@ def runtime_jdk(v): default=False, action="store_true") - # TODO dm: Revisit later whether we still want this - async_race_parser.add_argument( - "--use-uvloop", - help=argparse.SUPPRESS, - default=False, - action="store_true") - return parser @@ -868,8 +861,6 @@ def main(): cfg.add(config.Scope.applicationOverride, "generator", "node.count", args.node_count) cfg.add(config.Scope.applicationOverride, "driver", "profiling", args.enable_driver_profiling) - if sub_command == "race-async": - cfg.add(config.Scope.applicationOverride, "driver", "uvloop", args.use_uvloop) cfg.add(config.Scope.applicationOverride, "driver", "on.error", args.on_error) cfg.add(config.Scope.applicationOverride, "driver", "load_driver_hosts", opts.csv_to_list(args.load_driver_hosts)) if sub_command not in ("list", "install", "download"): diff --git a/setup.py b/setup.py index 5f61671f1..6c24d07d9 100644 --- a/setup.py +++ b/setup.py @@ -76,8 +76,7 @@ def str_from_file(name): # s3transfer: Apache 2.0 "boto3==1.10.32", # TODO dm: Temporarily here - *might* be removed later - "yappi==1.2.3", - "uvloop==0.14.0" + "yappi==1.2.3" ] tests_require = [ From dcba70d3978a767c9f22d45a4ff715a0dd891700 Mon Sep 17 00:00:00 2001 From: Daniel Mitterdorfer Date: Mon, 17 Feb 2020 15:28:39 +0100 Subject: [PATCH 08/24] Run sub-tasks in parallel --- esrally/driver/async_driver.py | 53 +++++++++++++--------------------- esrally/driver/driver.py | 25 ++++++++-------- tests/driver/driver_test.py | 50 +++++++++++++++++++++++++------- 3 files changed, 72 insertions(+), 56 deletions(-) diff --git a/esrally/driver/async_driver.py b/esrally/driver/async_driver.py index 9a5f0d17c..9890a7404 100644 --- a/esrally/driver/async_driver.py +++ b/esrally/driver/async_driver.py @@ -199,6 +199,7 @@ def wait_for_rest_api(self): raise exceptions.SystemSetupError("Elasticsearch REST API layer is not available.") def retrieve_cluster_info(self): + # noinspection PyBroadException try: return self.es_clients["default"].info() except BaseException: @@ -238,15 +239,16 @@ def start_benchmark(self): self.pool.submit(Timer(fn=self.post_process_samples, interval=30, stop_event=self.stop_timer_tasks)) self.pool.submit(Timer(fn=self.update_progress_message, interval=1, stop_event=self.stop_timer_tasks)) - # needed because a new thread does not have an event loop (see https://stackoverflow.com/questions/48725890/) + # needed because a new thread (that is not the main thread) does not have an event loop loop = asyncio.new_event_loop() + # TODO: Make this configurable? loop.set_debug(True) asyncio.set_event_loop(loop) loop.set_exception_handler(debug_exception_handler) track.set_absolute_data_path(self.config, self.track) runner.register_default_runners() - # TODO: I think we can skip this here - it has already been done earlier in prepare_benchmark() + # TODO: We can skip this here if we run in the same process; it has already been done in #prepare_benchmark() if self.track.has_plugins: track.load_track_plugins(self.config, runner.register_runner, scheduler.register_scheduler) success = False @@ -268,7 +270,7 @@ def start_benchmark(self): self.logger.debug("Closing metrics store...") self.metrics_store.close() # immediately clear as we don't need it anymore and it can consume a significant amount of memory - del self.metrics_store + self.metrics_store = None async def run_benchmark(self): # avoid: aiohttp.internal WARNING The object should be created from async function @@ -278,19 +280,15 @@ async def run_benchmark(self): # used to indicate that we want to prematurely consider this completed. This is *not* due to cancellation but a regular event in # a benchmark and used to model task dependency of parallel tasks. complete = threading.Event() + # allow to buffer more events than by default as we expect to have way more clients. + self.sampler = driver.Sampler(start_timestamp=time.perf_counter(), buffer_size=65536) for task in self.challenge.schedule: + self.current_tasks = [] + aws = [] for sub_task in task: + self.current_tasks.append(sub_task) self.logger.info("Running task [%s] with [%d] clients...", sub_task.name, sub_task.clients) - #console.println("Running task [{}] with [{}] clients...".format(sub_task.name, sub_task.clients), logger=self.logger.info) - - # TODO: We need to restructure this later on: We could have only one sampler for the whole benchmark but then we need to - # provide the current task to the sampler. This would also simplify #update_samples(). We also need to move the - # join point (done, pending = await asyncio.wait(aws)) below one level out so we can actually run all sub-tasks of - # a task in parallel. At the moment we'd run one after the other (which is plain wrong) - self.current_tasks = [sub_task] - self.sampler = driver.Sampler(None, task, start_timestamp=time.perf_counter()) - aws = [] # TODO: This is lacking support for one (sub)task being able to complete a complete parallel # structure. We can probably achieve that by waiting for the task in question and then # cancelling all other ongoing clients. @@ -298,24 +296,14 @@ async def run_benchmark(self): schedule = driver.schedule_for(self.track, sub_task, client_id) e = AsyncExecutor(client_id, sub_task, schedule, es, self.sampler, cancel, complete, self.abort_on_error) aws.append(e()) - # join point - done, pending = await asyncio.wait(aws) - self.logger.info("All clients have finished running task [%s]", sub_task.name) - # drain the active samples before we move on to the next task - self.update_samples() - self.post_process_samples() - self.reset_relative_time() - self.update_progress_message(task_finished=True) - - - #for client_index in range(start_client_index, start_client_index + sub_task.clients): - # this is the actual client that will execute the task. It may differ from the logical one in case we over-commit (i.e. - # more tasks than actually available clients) - # physical_client_index = client_index % max_clients - # if sub_task.completes_parent: - # clients_executing_completing_task.append(physical_client_index) - # allocations[physical_client_index].append(TaskAllocation(sub_task, client_index - start_client_index)) - # start_client_index += sub_task.clients + # join point + done, pending = await asyncio.wait(aws) + self.logger.info("All clients have finished running task [%s]", task.name) + # drain the active samples before we move on to the next task + self.update_samples() + self.post_process_samples() + self.reset_relative_time() + self.update_progress_message(task_finished=True) finally: await asyncio.get_event_loop().shutdown_asyncgens() await es["default"].transport.close() @@ -496,7 +484,6 @@ async def __call__(self, *args, **kwargs): # noinspection PyBroadException try: async for expected_scheduled_time, sample_type, percent_completed, runner, params in schedule: - #self.logger.info("Next iteration in main loop for client id %s (%s %% completed)", self.client_id, percent_completed) if self.cancel.is_set(): self.logger.info("User cancelled execution.") break @@ -520,8 +507,8 @@ async def __call__(self, *args, **kwargs): progress = runner.percent_completed else: progress = percent_completed - self.sampler.add(sample_type, request_meta_data, convert.seconds_to_ms(latency), convert.seconds_to_ms(service_time), - total_ops, total_ops_unit, (stop - total_start), progress, client_id=self.client_id) + self.sampler.add(self.task, self.client_id, sample_type, request_meta_data, convert.seconds_to_ms(latency), + convert.seconds_to_ms(service_time), total_ops, total_ops_unit, (stop - total_start), progress) if completed: self.logger.info("Task is considered completed due to external event.") diff --git a/esrally/driver/driver.py b/esrally/driver/driver.py index 7e7e78843..84a30b2d9 100644 --- a/esrally/driver/driver.py +++ b/esrally/driver/driver.py @@ -799,7 +799,7 @@ def drive(self): self.client_id, task) else: self.logger.info("LoadGenerator[%d] is executing [%s].", self.client_id, task) - self.sampler = Sampler(self.client_id, task, start_timestamp=time.perf_counter()) + self.sampler = Sampler(start_timestamp=time.perf_counter()) # We cannot use the global client index here because we need to support parallel execution of tasks with multiple clients. # # Consider the following scenario: @@ -811,7 +811,7 @@ def drive(self): # from (client) index 0 in both cases instead of 0 for indexA and 4 for indexB. schedule = schedule_for(self.track, task_allocation.task, task_allocation.client_index_in_task) executor = AsyncIoAdapter(self.config, self.client_id, task, schedule, self.sampler, self.cancel, self.complete, self.abort_on_error) - #executor = Executor(task, schedule, self.es, self.sampler, self.cancel, self.complete, self.abort_on_error) + #executor = Executor(self.client_id, task, schedule, self.es, self.sampler, self.cancel, self.complete, self.abort_on_error) final_executor = Profiler(executor, self.client_id, task) if profiling_enabled else executor self.executor_future = self.pool.submit(final_executor) @@ -841,21 +841,18 @@ class Sampler: Encapsulates management of gathered samples. """ - def __init__(self, client_id, task, start_timestamp): - self.client_id = client_id - self.task = task + def __init__(self, start_timestamp, buffer_size=16384): self.start_timestamp = start_timestamp - self.q = queue.Queue(maxsize=16384) + self.q = queue.Queue(maxsize=buffer_size) self.logger = logging.getLogger(__name__) - def add(self, sample_type, request_meta_data, latency_ms, service_time_ms, total_ops, total_ops_unit, time_period, percent_completed, client_id=None): + def add(self, task, client_id, sample_type, request_meta_data, latency_ms, service_time_ms, total_ops, total_ops_unit, time_period, percent_completed): try: - c = self.client_id if client_id is None else client_id - self.q.put_nowait(Sample(c, time.time(), time.perf_counter() - self.start_timestamp, self.task, + self.q.put_nowait(Sample(client_id, time.time(), time.perf_counter() - self.start_timestamp, task, sample_type, request_meta_data, latency_ms, service_time_ms, total_ops, total_ops_unit, time_period, percent_completed)) except queue.Full: - self.logger.warning("Dropping sample for [%s] due to a full sampling queue.", self.task.operation.name) + self.logger.warning("Dropping sample for [%s] due to a full sampling queue.", task.operation.name) @property def samples(self): @@ -1096,10 +1093,11 @@ def es_clients(all_hosts, all_client_options): class Executor: - def __init__(self, task, schedule, es, sampler, cancel, complete, abort_on_error=False): + def __init__(self, client_id, task, schedule, es, sampler, cancel, complete, abort_on_error=False): """ Executes tasks according to the schedule for a given operation. + :param client_id: Id of the client that executes requests. :param task: The task that is executed. :param schedule: The schedule for this task. :param es: Elasticsearch client that will be used to execute the operation. @@ -1107,6 +1105,7 @@ def __init__(self, task, schedule, es, sampler, cancel, complete, abort_on_error :param cancel: A shared boolean that indicates we need to cancel execution. :param complete: A shared boolean that indicates we need to prematurely complete execution. """ + self.client_id = client_id self.task = task self.op = task.operation self.schedule_handle = schedule @@ -1148,8 +1147,8 @@ def __call__(self, *args, **kwargs): progress = runner.percent_completed else: progress = percent_completed - self.sampler.add(sample_type, request_meta_data, convert.seconds_to_ms(latency), convert.seconds_to_ms(service_time), - total_ops, total_ops_unit, (stop - total_start), progress) + self.sampler.add(self.task, self.client_id, sample_type, request_meta_data, convert.seconds_to_ms(latency), + convert.seconds_to_ms(service_time), total_ops, total_ops_unit, (stop - total_start), progress) if completed: self.logger.info("Task is considered completed due to external event.") diff --git a/tests/driver/driver_test.py b/tests/driver/driver_test.py index 5be335035..efcf1daa0 100644 --- a/tests/driver/driver_test.py +++ b/tests/driver/driver_test.py @@ -718,11 +718,17 @@ def test_execute_schedule_in_throughput_mode(self, es): warmup_time_period=0, clients=4) schedule = driver.schedule_for(test_track, task, 0) - sampler = driver.Sampler(client_id=2, task=task, start_timestamp=time.perf_counter()) + sampler = driver.Sampler(start_timestamp=time.perf_counter()) cancel = threading.Event() complete = threading.Event() - execute_schedule = driver.Executor(task, schedule, es, sampler, cancel, complete) + execute_schedule = driver.Executor(client_id=2, + task=task, + schedule=schedule, + es=es, + sampler=sampler, + cancel=cancel, + complete=complete) execute_schedule() samples = sampler.samples @@ -764,11 +770,17 @@ def test_execute_schedule_with_progress_determined_by_runner(self, es): }, param_source="driver-test-param-source"), warmup_time_period=0, clients=4) schedule = driver.schedule_for(test_track, task, 0) - sampler = driver.Sampler(client_id=2, task=task, start_timestamp=time.perf_counter()) + sampler = driver.Sampler(start_timestamp=time.perf_counter()) cancel = threading.Event() complete = threading.Event() - execute_schedule = driver.Executor(task, schedule, es, sampler, cancel, complete) + execute_schedule = driver.Executor(client_id=2, + task=task, + schedule=schedule, + es=es, + sampler=sampler, + cancel=cancel, + complete=complete) execute_schedule() samples = sampler.samples @@ -815,13 +827,19 @@ def test_execute_schedule_throughput_throttled(self, es): warmup_time_period=0.5, time_period=0.5, clients=4, params={"target-throughput": target_throughput, "clients": 4}, completes_parent=True) - sampler = driver.Sampler(client_id=0, task=task, start_timestamp=0) + sampler = driver.Sampler(start_timestamp=0) cancel = threading.Event() complete = threading.Event() schedule = driver.schedule_for(test_track, task, 0) - execute_schedule = driver.Executor(task, schedule, es, sampler, cancel, complete) + execute_schedule = driver.Executor(client_id=0, + task=task, + schedule=schedule, + es=es, + sampler=sampler, + cancel=cancel, + complete=complete) execute_schedule() samples = sampler.samples @@ -855,11 +873,17 @@ def test_cancel_execute_schedule(self, es): warmup_time_period=0.5, time_period=0.5, clients=4, params={"target-throughput": target_throughput, "clients": 4}) schedule = driver.schedule_for(test_track, task, 0) - sampler = driver.Sampler(client_id=0, task=task, start_timestamp=0) + sampler = driver.Sampler(start_timestamp=0) cancel = threading.Event() complete = threading.Event() - execute_schedule = driver.Executor(task, schedule, es, sampler, cancel, complete) + execute_schedule = driver.Executor(client_id=0, + task=task, + schedule=schedule, + es=es, + sampler=sampler, + cancel=cancel, + complete=complete) cancel.set() execute_schedule() @@ -885,10 +909,16 @@ def schedule_handle(): warmup_time_period=0.5, time_period=0.5, clients=4, params={"clients": 4}) - sampler = driver.Sampler(client_id=0, task=None, start_timestamp=0) + sampler = driver.Sampler(start_timestamp=0) cancel = threading.Event() complete = threading.Event() - execute_schedule = driver.Executor(task, schedule_handle, es, sampler, cancel, complete) + execute_schedule = driver.Executor(client_id=2, + task=task, + schedule=schedule_handle, + es=es, + sampler=sampler, + cancel=cancel, + complete=complete) with self.assertRaises(ExpectedUnitTestException): execute_schedule() From fd7a3d91840dccc7e80991c1eef7537e6f9c378e Mon Sep 17 00:00:00 2001 From: Daniel Mitterdorfer Date: Mon, 17 Feb 2020 15:58:42 +0100 Subject: [PATCH 09/24] Require at least Python 3.6 With this commit we bump the minimum required Python version to Python 3.6 (thus dropping support for Python 3.5). Python 3.5 will be end of life on September 13, 2020 (source: [1]). We also intend to use several features that require at least Python 3.6 in future versions of Rally thus we drop support for Python 3.5 now. [1] https://devguide.python.org/#status-of-python-branches --- Makefile | 8 ++------ docs/developing.rst | 2 +- docs/install.rst | 8 ++++---- docs/quickstart.rst | 2 +- esrally/__init__.py | 4 ++-- make-requirements.txt | 1 - setup.py | 2 +- tox.ini | 2 +- 8 files changed, 12 insertions(+), 17 deletions(-) diff --git a/Makefile b/Makefile index b7e9699a3..db700b16f 100644 --- a/Makefile +++ b/Makefile @@ -31,11 +31,10 @@ PYENV_PREREQ_HELP = "\033[0;31mIMPORTANT\033[0m: please add \033[0;31meval \"\$$ VE_MISSING_HELP = "\033[0;31mIMPORTANT\033[0m: Couldn't find $(PWD)/$(VENV_NAME); have you executed make venv-create?\033[0m\n" prereq: make-requirements.txt - pyenv install --skip-existing $(PY35) pyenv install --skip-existing $(PY36) pyenv install --skip-existing $(PY37) pyenv install --skip-existing $(PY38) - pyenv global system $(PY35) $(PY36) $(PY37) $(PY38) + pyenv global system $(PY36) $(PY37) $(PY38) @# Ensure all Python versions are registered for this project @awk -F'=' '{print $$2}' make-requirements.txt > .python-version -@ printf $(PYENV_PREREQ_HELP) @@ -105,9 +104,6 @@ precommit: lint it: check-venv python-caches-clean tox-env-clean . $(VENV_ACTIVATE_FILE); tox -it35: check-venv python-caches-clean tox-env-clean - . $(VENV_ACTIVATE_FILE); tox -e py35 - it36: check-venv python-caches-clean tox-env-clean . $(VENV_ACTIVATE_FILE); tox -e py36 @@ -131,4 +127,4 @@ release-checks: check-venv release: check-venv release-checks clean docs it . $(VENV_ACTIVATE_FILE); ./release.sh $(release_version) $(next_version) -.PHONY: install clean nondocs-clean docs-clean python-caches-clean tox-env-clean docs serve-docs test it it35 it36 it37 it38 benchmark coverage release release-checks prereq venv-create check-env +.PHONY: install clean nondocs-clean docs-clean python-caches-clean tox-env-clean docs serve-docs test it it36 it37 it38 benchmark coverage release release-checks prereq venv-create check-env diff --git a/docs/developing.rst b/docs/developing.rst index 45a44cae4..beb4c65e8 100644 --- a/docs/developing.rst +++ b/docs/developing.rst @@ -6,7 +6,7 @@ Prerequisites Install the following software packages: -* Python 3.5 or better available as ``python3`` on the path (verify with: ``python3 --version``) +* Python 3.6 or better available as ``python3`` on the path (verify with: ``python3 --version``) * ``pip3`` available on the path (verify with ``pip3 --version``) * ``virtualenv`` available on the path (verify with ``virtualenv --version``) * Pyenv installed and ``eval "$(pyenv init -)"`` is added to the shell configuration file. For more details please refer to the PyEnv `installation instructions `_. diff --git a/docs/install.rst b/docs/install.rst index 1f1acf79d..25d1f5105 100644 --- a/docs/install.rst +++ b/docs/install.rst @@ -16,7 +16,7 @@ Rally does not support Windows and is only actively tested on MacOS and Linux. I Python ~~~~~~ -* Python 3.5 or better available as `python3` on the path. Verify with: ``python3 --version``. +* Python 3.6 or better available as `python3` on the path. Verify with: ``python3 --version``. * Python3 header files (included in the Python3 development package). * ``pip3`` available on the path. Verify with ``pip3 --version``. @@ -24,18 +24,18 @@ Python :: - sudo apt-get install gcc python3-pip python3-dev + sudo apt-get install gcc python3-pip python3.6-dev **RHEL / CentOS 6 and 7** -Please refer to the `installation instructions for Python 3.5 in the Red Hat Software Collections `_. +Please refer to the `installation instructions for Python 3.6 in the Red Hat Software Collections `_. **Amazon Linux** :: - sudo yum install -y gcc python35-pip.noarch python35-devel.x86_64 + sudo yum install -y gcc python3-pip.noarch python3-devel.x86_64 **MacOS** diff --git a/docs/quickstart.rst b/docs/quickstart.rst index 26fd5a151..a8369620c 100644 --- a/docs/quickstart.rst +++ b/docs/quickstart.rst @@ -6,7 +6,7 @@ Rally is developed for Unix and is actively tested on Linux and MacOS. Rally sup Install ------- -Install Python 3.5+ including ``pip3``, git 1.9+ and an `appropriate JDK to run Elasticsearch `_ Be sure that ``JAVA_HOME`` points to that JDK. Then run the following command, optionally prefixed by ``sudo`` if necessary:: +Install Python 3.6+ including ``pip3``, git 1.9+ and an `appropriate JDK to run Elasticsearch `_ Be sure that ``JAVA_HOME`` points to that JDK. Then run the following command, optionally prefixed by ``sudo`` if necessary:: pip3 install esrally diff --git a/esrally/__init__.py b/esrally/__init__.py index 6e8555fe6..51dfac712 100644 --- a/esrally/__init__.py +++ b/esrally/__init__.py @@ -72,8 +72,8 @@ def check_python_version(): - if sys.version_info.major != 3 or sys.version_info.minor < 5: - raise RuntimeError("Rally requires at least Python 3.5 but you are using:\n\nPython %s" % str(sys.version)) + if sys.version_info.major != 3 or sys.version_info.minor < 6: + raise RuntimeError("Rally requires at least Python 3.6 but you are using:\n\nPython %s" % str(sys.version)) def doc_link(path=None): diff --git a/make-requirements.txt b/make-requirements.txt index 3c099cc07..4fbab7cd7 100644 --- a/make-requirements.txt +++ b/make-requirements.txt @@ -1,4 +1,3 @@ -PY35=3.5.8 PY36=3.6.9 PY37=3.7.5 PY38=3.8.0 diff --git a/setup.py b/setup.py index d9e27ef6f..1ec739b04 100644 --- a/setup.py +++ b/setup.py @@ -37,7 +37,7 @@ def str_from_file(name): long_description = str_from_file("README.rst") # tuples of (major, minor) of supported Python versions ordered from lowest to highest -supported_python_versions = [(3, 5), (3, 6), (3, 7), (3, 8)] +supported_python_versions = [(3, 6), (3, 7), (3, 8)] ################################################################################################ # diff --git a/tox.ini b/tox.ini index 30b024230..9a38aa119 100644 --- a/tox.ini +++ b/tox.ini @@ -7,7 +7,7 @@ ############################################################################### [tox] envlist = - docs, py35, py36, py37, py38 + docs, py36, py37, py38 platform = linux|darwin From 7f90c727f1215059898f404e513de721141520c7 Mon Sep 17 00:00:00 2001 From: Daniel Mitterdorfer Date: Thu, 20 Feb 2020 10:55:32 +0100 Subject: [PATCH 10/24] Implement async load generator With this commit we change Rally's internal implementation to always use the async code path so runner implementations stay the same. --- esrally/async_connection.py | 3 + esrally/client.py | 1 - esrally/driver/async_driver.py | 167 +----- esrally/driver/driver.py | 111 ++-- esrally/driver/runner.py | 657 ++++++---------------- esrally/utils/io.py | 2 + integration-test.sh | 4 +- setup.py | 1 - tests/__init__.py | 32 ++ tests/driver/driver_test.py | 361 ++++++------ tests/driver/runner_test.py | 977 ++++++++++++++++++--------------- 11 files changed, 1012 insertions(+), 1304 deletions(-) diff --git a/esrally/async_connection.py b/esrally/async_connection.py index 29a57c10d..a6e91487e 100644 --- a/esrally/async_connection.py +++ b/esrally/async_connection.py @@ -14,6 +14,9 @@ # This is only needed because https://github.com/elastic/elasticsearch-py-async/pull/68 is not merged yet # In addition we have raised the connection limit in TCPConnector from 100 to 10000. + +# We want to keep the diff as small as possible thus suppressing pylint warnings that we would not allow in Rally +# pylint: disable=W0706 class AIOHttpConnection(Connection): def __init__(self, host='localhost', port=9200, http_auth=None, use_ssl=False, verify_certs=False, ca_certs=None, client_cert=None, diff --git a/esrally/client.py b/esrally/client.py index ce1aa615b..9d82e24b3 100644 --- a/esrally/client.py +++ b/esrally/client.py @@ -128,7 +128,6 @@ def create(self): return elasticsearch.Elasticsearch(hosts=self.hosts, ssl_context=self.ssl_context, **self.client_options) def create_async(self): - import elasticsearch import elasticsearch_async from aiohttp.client import ClientTimeout import esrally.async_connection diff --git a/esrally/driver/async_driver.py b/esrally/driver/async_driver.py index 9890a7404..662d1b30e 100644 --- a/esrally/driver/async_driver.py +++ b/esrally/driver/async_driver.py @@ -16,15 +16,15 @@ # under the License. -import logging -from esrally.driver import driver, runner, scheduler -import time import asyncio +import concurrent.futures +import logging import threading +import time from esrally import exceptions, metrics, track, client, PROGRAM_NAME, telemetry -from esrally.utils import console, convert -import concurrent.futures +from esrally.driver import driver, runner, scheduler +from esrally.utils import console # TODO: Inline this code later @@ -253,7 +253,7 @@ def start_benchmark(self): track.load_track_plugins(self.config, runner.register_runner, scheduler.register_scheduler) success = False try: - benchmark_runner = AsyncProfiler(self.run_benchmark) if self.profiling_enabled else self.run_benchmark + benchmark_runner = driver.AsyncProfiler(self.run_benchmark) if self.profiling_enabled else self.run_benchmark loop.run_until_complete(benchmark_runner()) self._finished = True @@ -294,7 +294,7 @@ async def run_benchmark(self): # cancelling all other ongoing clients. for client_id in range(sub_task.clients): schedule = driver.schedule_for(self.track, sub_task, client_id) - e = AsyncExecutor(client_id, sub_task, schedule, es, self.sampler, cancel, complete, self.abort_on_error) + e = driver.AsyncExecutor(client_id, sub_task, schedule, es, self.sampler, cancel, complete, self.abort_on_error) aws.append(e()) # join point done, pending = await asyncio.wait(aws) @@ -419,156 +419,3 @@ def merge(self, *args): def debug_exception_handler(loop, context): logging.getLogger(__name__).error("Uncaught exception in event loop!! %s", context) - - -class AsyncProfiler: - def __init__(self, target): - """ - :param target: The actual executor which should be profiled. - """ - self.target = target - self.profile_logger = logging.getLogger("rally.profile") - - async def __call__(self, *args, **kwargs): - import yappi - import io as python_io - yappi.start() - try: - return await self.target(*args, **kwargs) - finally: - yappi.stop() - s = python_io.StringIO() - yappi.get_func_stats().print_all(out=s, columns={ - 0: ("name", 140), - 1: ("ncall", 8), - 2: ("tsub", 8), - 3: ("ttot", 8), - 4: ("tavg", 8) - }) - - profile = "\n=== Profile START ===\n" - profile += s.getvalue() - profile += "=== Profile END ===" - self.profile_logger.info(profile) - - -class AsyncExecutor: - def __init__(self, client_id, task, schedule, es, sampler, cancel, complete, abort_on_error=False): - """ - Executes tasks according to the schedule for a given operation. - - :param task: The task that is executed. - :param schedule: The schedule for this task. - :param es: Elasticsearch client that will be used to execute the operation. - :param sampler: A container to store raw samples. - :param cancel: A shared boolean that indicates we need to cancel execution. - :param complete: A shared boolean that indicates we need to prematurely complete execution. - """ - self.client_id = client_id - self.task = task - self.op = task.operation - self.schedule_handle = schedule - self.es = es - self.sampler = sampler - self.cancel = cancel - self.complete = complete - self.abort_on_error = abort_on_error - self.logger = logging.getLogger(__name__) - - async def __call__(self, *args, **kwargs): - total_start = time.perf_counter() - # lazily initialize the schedule - self.logger.debug("Initializing schedule for client id [%s].", self.client_id) - schedule = self.schedule_handle() - self.logger.debug("Entering main loop for client id [%s].", self.client_id) - # noinspection PyBroadException - try: - async for expected_scheduled_time, sample_type, percent_completed, runner, params in schedule: - if self.cancel.is_set(): - self.logger.info("User cancelled execution.") - break - absolute_expected_schedule_time = total_start + expected_scheduled_time - throughput_throttled = expected_scheduled_time > 0 - if throughput_throttled: - rest = absolute_expected_schedule_time - time.perf_counter() - if rest > 0: - await asyncio.sleep(rest) - start = time.perf_counter() - total_ops, total_ops_unit, request_meta_data = await execute_single(runner, self.es, params, self.abort_on_error) - stop = time.perf_counter() - service_time = stop - start - # Do not calculate latency separately when we don't throttle throughput. This metric is just confusing then. - latency = stop - absolute_expected_schedule_time if throughput_throttled else service_time - # last sample should bump progress to 100% if externally completed. - completed = self.complete.is_set() or runner.completed - if completed: - progress = 1.0 - elif runner.percent_completed: - progress = runner.percent_completed - else: - progress = percent_completed - self.sampler.add(self.task, self.client_id, sample_type, request_meta_data, convert.seconds_to_ms(latency), - convert.seconds_to_ms(service_time), total_ops, total_ops_unit, (stop - total_start), progress) - - if completed: - self.logger.info("Task is considered completed due to external event.") - break - except BaseException: - self.logger.exception("Could not execute schedule") - raise - finally: - # Actively set it if this task completes its parent - if self.task.completes_parent: - self.complete.set() - - -async def execute_single(runner, es, params, abort_on_error=False): - """ - Invokes the given runner once and provides the runner's return value in a uniform structure. - - :return: a triple of: total number of operations, unit of operations, a dict of request meta data (may be None). - """ - import elasticsearch - try: - # TODO: Make all runners async-aware - Can we run async runners as a "regular" function (to avoid duplicate implementations)? - with runner: - return_value = await runner(es, params) - if isinstance(return_value, tuple) and len(return_value) == 2: - total_ops, total_ops_unit = return_value - request_meta_data = {"success": True} - elif isinstance(return_value, dict): - total_ops = return_value.pop("weight", 1) - total_ops_unit = return_value.pop("unit", "ops") - request_meta_data = return_value - if "success" not in request_meta_data: - request_meta_data["success"] = True - else: - total_ops = 1 - total_ops_unit = "ops" - request_meta_data = {"success": True} - except elasticsearch.TransportError as e: - total_ops = 0 - total_ops_unit = "ops" - request_meta_data = { - "success": False, - "error-type": "transport" - } - # The ES client will sometimes return string like "N/A" or "TIMEOUT" for connection errors. - if isinstance(e.status_code, int): - request_meta_data["http-status"] = e.status_code - if e.info: - request_meta_data["error-description"] = "%s (%s)" % (e.error, e.info) - else: - request_meta_data["error-description"] = e.error - except KeyError as e: - logging.getLogger(__name__).exception("Cannot execute runner [%s]; most likely due to missing parameters.", str(runner)) - msg = "Cannot execute [%s]. Provided parameters are: %s. Error: [%s]." % (str(runner), list(params.keys()), str(e)) - raise exceptions.SystemSetupError(msg) - - if abort_on_error and not request_meta_data["success"]: - msg = "Request returned an error. Error type: %s" % request_meta_data.get("error-type", "Unknown") - description = request_meta_data.get("error-description") - if description: - msg += ", Description: %s" % description - raise exceptions.RallyAssertionError(msg) - return total_ops, total_ops_unit, request_meta_data diff --git a/esrally/driver/driver.py b/esrally/driver/driver.py index 6af47d563..23d75ea0f 100644 --- a/esrally/driver/driver.py +++ b/esrally/driver/driver.py @@ -15,6 +15,7 @@ # specific language governing permissions and limitations # under the License. +import asyncio import concurrent.futures import datetime import logging @@ -772,7 +773,6 @@ def receiveUnrecognizedMessage(self, msg, sender): self.logger.info("LoadGenerator[%d] received unknown message [%s] (ignoring).", self.client_id, str(msg)) def drive(self): - profiling_enabled = self.config.opts("driver", "profiling") task_allocation = self.current_task_and_advance() # skip non-tasks in the task list while task_allocation is None: @@ -810,11 +810,10 @@ def drive(self): # Now we need to ensure that we start partitioning parameters correctly in both cases. And that means we need to start # from (client) index 0 in both cases instead of 0 for indexA and 4 for indexB. schedule = schedule_for(self.track, task_allocation.task, task_allocation.client_index_in_task) - executor = AsyncIoAdapter(self.config, self.client_id, task, schedule, self.sampler, self.cancel, self.complete, self.abort_on_error) - #executor = Executor(self.client_id, task, schedule, self.es, self.sampler, self.cancel, self.complete, self.abort_on_error) - final_executor = Profiler(executor, self.client_id, task) if profiling_enabled else executor + executor = AsyncIoAdapter( + self.config, self.client_id, task, schedule, self.sampler, self.cancel, self.complete, self.abort_on_error) - self.executor_future = self.pool.submit(final_executor) + self.executor_future = self.pool.submit(executor) self.wakeupAfter(datetime.timedelta(seconds=self.wakeup_interval)) else: raise exceptions.RallyAssertionError("Unknown task type [%s]" % type(task_allocation)) @@ -846,11 +845,10 @@ def __init__(self, start_timestamp, buffer_size=16384): self.q = queue.Queue(maxsize=buffer_size) self.logger = logging.getLogger(__name__) - def add(self, task, client_id, sample_type, request_meta_data, latency_ms, service_time_ms, total_ops, total_ops_unit, time_period, percent_completed): + def add(self, task, client_id, sample_type, meta_data, latency, service_time, ops, ops_unit, time_period, percent_completed): try: self.q.put_nowait(Sample(client_id, time.time(), time.perf_counter() - self.start_timestamp, task, - sample_type, request_meta_data, latency_ms, service_time_ms, total_ops, total_ops_unit, time_period, - percent_completed)) + sample_type, meta_data, latency, service_time, ops, ops_unit, time_period, percent_completed)) except queue.Full: self.logger.warning("Dropping sample for [%s] due to a full sampling queue.", task.operation.name) @@ -1018,39 +1016,6 @@ def calculate(self, samples, bucket_interval_secs=1): return global_throughput -class Profiler: - def __init__(self, target, client_id, task): - """ - :param target: The actual executor which should be profiled. - :param client_id: The id of the client that executes the operation. - :param task: The task that is executed. - """ - self.target = target - self.client_id = client_id - self.task = task - self.profile_logger = logging.getLogger("rally.profile") - - def __call__(self, *args, **kwargs): - import cProfile - import pstats - import io as python_io - profiler = cProfile.Profile() - profiler.enable() - try: - return self.target(*args, **kwargs) - finally: - profiler.disable() - s = python_io.StringIO() - sortby = 'cumulative' - ps = pstats.Stats(profiler, stream=s).sort_stats(sortby) - ps.print_stats() - - profile = "\n=== Profile START for client [%s] and task [%s] ===\n" % (str(self.client_id), str(self.task)) - profile += s.getvalue() - profile += "=== Profile END for client [%s] and task [%s] ===" % (str(self.client_id), str(self.task)) - self.profile_logger.info(profile) - - class AsyncIoAdapter: def __init__(self, cfg, client_id, sub_task, schedule, sampler, cancel, complete, abort_on_error): self.cfg = cfg @@ -1061,9 +1026,9 @@ def __init__(self, cfg, client_id, sub_task, schedule, sampler, cancel, complete self.cancel = cancel self.complete = complete self.abort_on_error = abort_on_error + self.profiling_enabled = self.cfg.opts("driver", "profiling") def __call__(self, *args, **kwargs): - import asyncio # only possible in Python 3.7+ (has introduced get_running_loop) # try: # loop = asyncio.get_running_loop() @@ -1085,19 +1050,49 @@ def es_clients(all_hosts, all_client_options): es[cluster_name] = client.EsClientFactory(cluster_hosts, all_client_options[cluster_name]).create_async() return es - from esrally.driver import async_driver - es = es_clients(self.cfg.opts("client", "hosts").all_hosts, self.cfg.opts("client", "options").all_client_options) - async_executor = async_driver.AsyncExecutor(self.client_id, self.sub_task, self.schedule, es, self.sampler, self.cancel, self.complete, self.abort_on_error) - return await async_executor() + async_executor = AsyncExecutor( + self.client_id, self.sub_task, self.schedule, es, self.sampler, self.cancel, self.complete, self.abort_on_error) + final_executor = AsyncProfiler(async_executor) if self.profiling_enabled else async_executor + return await final_executor() + + +class AsyncProfiler: + def __init__(self, target): + """ + :param target: The actual executor which should be profiled. + """ + self.target = target + self.profile_logger = logging.getLogger("rally.profile") + + async def __call__(self, *args, **kwargs): + import yappi + import io as python_io + yappi.start() + try: + return await self.target(*args, **kwargs) + finally: + yappi.stop() + s = python_io.StringIO() + yappi.get_func_stats().print_all(out=s, columns={ + 0: ("name", 140), + 1: ("ncall", 8), + 2: ("tsub", 8), + 3: ("ttot", 8), + 4: ("tavg", 8) + }) + + profile = "\n=== Profile START ===\n" + profile += s.getvalue() + profile += "=== Profile END ===" + self.profile_logger.info(profile) -class Executor: +class AsyncExecutor: def __init__(self, client_id, task, schedule, es, sampler, cancel, complete, abort_on_error=False): """ Executes tasks according to the schedule for a given operation. - :param client_id: Id of the client that executes requests. :param task: The task that is executed. :param schedule: The schedule for this task. :param es: Elasticsearch client that will be used to execute the operation. @@ -1116,13 +1111,15 @@ def __init__(self, client_id, task, schedule, es, sampler, cancel, complete, abo self.abort_on_error = abort_on_error self.logger = logging.getLogger(__name__) - def __call__(self, *args, **kwargs): + async def __call__(self, *args, **kwargs): total_start = time.perf_counter() # lazily initialize the schedule + self.logger.debug("Initializing schedule for client id [%s].", self.client_id) schedule = self.schedule_handle() + self.logger.debug("Entering main loop for client id [%s].", self.client_id) # noinspection PyBroadException try: - for expected_scheduled_time, sample_type, percent_completed, runner, params in schedule: + async for expected_scheduled_time, sample_type, percent_completed, runner, params in schedule: if self.cancel.is_set(): self.logger.info("User cancelled execution.") break @@ -1131,11 +1128,10 @@ def __call__(self, *args, **kwargs): if throughput_throttled: rest = absolute_expected_schedule_time - time.perf_counter() if rest > 0: - time.sleep(rest) + await asyncio.sleep(rest) start = time.perf_counter() - total_ops, total_ops_unit, request_meta_data = execute_single(runner, self.es, params, self.abort_on_error) + total_ops, total_ops_unit, request_meta_data = await execute_single(runner, self.es, params, self.abort_on_error) stop = time.perf_counter() - service_time = stop - start # Do not calculate latency separately when we don't throttle throughput. This metric is just confusing then. latency = stop - absolute_expected_schedule_time if throughput_throttled else service_time @@ -1162,7 +1158,7 @@ def __call__(self, *args, **kwargs): self.complete.set() -def execute_single(runner, es, params, abort_on_error=False): +async def execute_single(runner, es, params, abort_on_error=False): """ Invokes the given runner once and provides the runner's return value in a uniform structure. @@ -1170,8 +1166,8 @@ def execute_single(runner, es, params, abort_on_error=False): """ import elasticsearch try: - with runner: - return_value = runner(es, params) + async with runner: + return_value = await runner(es, params) if isinstance(return_value, tuple) and len(return_value) == 2: total_ops, total_ops_unit = return_value request_meta_data = {"success": True} @@ -1210,7 +1206,6 @@ def execute_single(runner, es, params, abort_on_error=False): if description: msg += ", Description: %s" % description raise exceptions.RallyAssertionError(msg) - return total_ops, total_ops_unit, request_meta_data @@ -1451,9 +1446,7 @@ def __init__(self, task_name, sched, task_progress_control, runner, params): #self.io_pool_exc = ThreadPoolExecutor(max_workers=1) #self.loop = asyncio.get_event_loop() - # TODO: This requires Python 3.6+ (see https://www.python.org/dev/peps/pep-0525/) async def __call__(self): - # def __call__(self): next_scheduled = 0 if self.task_progress_control.infinite: self.logger.info("Parameter source will determine when the schedule for [%s] terminates.", self.task_name) diff --git a/esrally/driver/runner.py b/esrally/driver/runner.py index 925af4dfc..a148d909c 100644 --- a/esrally/driver/runner.py +++ b/esrally/driver/runner.py @@ -15,10 +15,10 @@ # specific language governing permissions and limitations # under the License. +import asyncio import logging import random import sys -import time import types from collections import Counter, OrderedDict from copy import deepcopy @@ -30,38 +30,37 @@ def register_default_runners(): - register_runner(track.OperationType.Bulk.name, AsyncBulkIndex()) - #register_runner(track.OperationType.Bulk.name, BulkIndex()) - register_runner(track.OperationType.ForceMerge.name, ForceMerge()) - register_runner(track.OperationType.IndicesStats.name, IndicesStats()) - register_runner(track.OperationType.NodesStats.name, NodeStats()) - register_runner(track.OperationType.Search.name, Query()) - register_runner(track.OperationType.RawRequest.name, RawRequest()) + register_runner(track.OperationType.Bulk.name, BulkIndex(), async_runner=True) + register_runner(track.OperationType.ForceMerge.name, ForceMerge(), async_runner=True) + register_runner(track.OperationType.IndicesStats.name, IndicesStats(), async_runner=True) + register_runner(track.OperationType.NodesStats.name, NodeStats(), async_runner=True) + register_runner(track.OperationType.Search.name, Query(), async_runner=True) + register_runner(track.OperationType.RawRequest.name, RawRequest(), async_runner=True) # This is an administrative operation but there is no need for a retry here as we don't issue a request - register_runner(track.OperationType.Sleep.name, Sleep()) + register_runner(track.OperationType.Sleep.name, Sleep(), async_runner=True) # these requests should not be retried as they are not idempotent - register_runner(track.OperationType.RestoreSnapshot.name, RestoreSnapshot()) + register_runner(track.OperationType.RestoreSnapshot.name, RestoreSnapshot(), async_runner=True) # We treat the following as administrative commands and thus already start to wrap them in a retry. - register_runner(track.OperationType.ClusterHealth.name, Retry(ClusterHealth())) - register_runner(track.OperationType.PutPipeline.name, Retry(PutPipeline())) - register_runner(track.OperationType.Refresh.name, Retry(Refresh())) - register_runner(track.OperationType.CreateIndex.name, Retry(CreateIndex())) - register_runner(track.OperationType.DeleteIndex.name, Retry(DeleteIndex())) - register_runner(track.OperationType.CreateIndexTemplate.name, Retry(CreateIndexTemplate())) - register_runner(track.OperationType.DeleteIndexTemplate.name, Retry(DeleteIndexTemplate())) - register_runner(track.OperationType.ShrinkIndex.name, Retry(ShrinkIndex())) - register_runner(track.OperationType.CreateMlDatafeed.name, Retry(CreateMlDatafeed())) - register_runner(track.OperationType.DeleteMlDatafeed.name, Retry(DeleteMlDatafeed())) - register_runner(track.OperationType.StartMlDatafeed.name, Retry(StartMlDatafeed())) - register_runner(track.OperationType.StopMlDatafeed.name, Retry(StopMlDatafeed())) - register_runner(track.OperationType.CreateMlJob.name, Retry(CreateMlJob())) - register_runner(track.OperationType.DeleteMlJob.name, Retry(DeleteMlJob())) - register_runner(track.OperationType.OpenMlJob.name, Retry(OpenMlJob())) - register_runner(track.OperationType.CloseMlJob.name, Retry(CloseMlJob())) - register_runner(track.OperationType.DeleteSnapshotRepository.name, Retry(DeleteSnapshotRepository())) - register_runner(track.OperationType.CreateSnapshotRepository.name, Retry(CreateSnapshotRepository())) - register_runner(track.OperationType.WaitForRecovery.name, Retry(IndicesRecovery())) - register_runner(track.OperationType.PutSettings.name, Retry(PutSettings())) + register_runner(track.OperationType.ClusterHealth.name, Retry(ClusterHealth()), async_runner=True) + register_runner(track.OperationType.PutPipeline.name, Retry(PutPipeline()), async_runner=True) + register_runner(track.OperationType.Refresh.name, Retry(Refresh()), async_runner=True) + register_runner(track.OperationType.CreateIndex.name, Retry(CreateIndex()), async_runner=True) + register_runner(track.OperationType.DeleteIndex.name, Retry(DeleteIndex()), async_runner=True) + register_runner(track.OperationType.CreateIndexTemplate.name, Retry(CreateIndexTemplate()), async_runner=True) + register_runner(track.OperationType.DeleteIndexTemplate.name, Retry(DeleteIndexTemplate()), async_runner=True) + register_runner(track.OperationType.ShrinkIndex.name, Retry(ShrinkIndex()), async_runner=True) + register_runner(track.OperationType.CreateMlDatafeed.name, Retry(CreateMlDatafeed()), async_runner=True) + register_runner(track.OperationType.DeleteMlDatafeed.name, Retry(DeleteMlDatafeed()), async_runner=True) + register_runner(track.OperationType.StartMlDatafeed.name, Retry(StartMlDatafeed()), async_runner=True) + register_runner(track.OperationType.StopMlDatafeed.name, Retry(StopMlDatafeed()), async_runner=True) + register_runner(track.OperationType.CreateMlJob.name, Retry(CreateMlJob()), async_runner=True) + register_runner(track.OperationType.DeleteMlJob.name, Retry(DeleteMlJob()), async_runner=True) + register_runner(track.OperationType.OpenMlJob.name, Retry(OpenMlJob()), async_runner=True) + register_runner(track.OperationType.CloseMlJob.name, Retry(CloseMlJob()), async_runner=True) + register_runner(track.OperationType.DeleteSnapshotRepository.name, Retry(DeleteSnapshotRepository()), async_runner=True) + register_runner(track.OperationType.CreateSnapshotRepository.name, Retry(CreateSnapshotRepository()), async_runner=True) + register_runner(track.OperationType.WaitForRecovery.name, Retry(IndicesRecovery()), async_runner=True) + register_runner(track.OperationType.PutSettings.name, Retry(PutSettings()), async_runner=True) def runner_for(operation_type): @@ -71,10 +70,15 @@ def runner_for(operation_type): raise exceptions.RallyError("No runner available for operation type [%s]" % operation_type) -def register_runner(operation_type, runner): +def register_runner(operation_type, runner, **kwargs): logger = logging.getLogger(__name__) + async_runner = kwargs.get("async_runner", False) + if not async_runner: + raise exceptions.RallyAssertionError( + "Runner [{}] must be implemented as async runner and registered with async_runner=True.".format(str(runner))) + if getattr(runner, "multi_cluster", False) == True: - if "__enter__" in dir(runner) and "__exit__" in dir(runner): + if "__aenter__" in dir(runner) and "__aexit__" in dir(runner): if logger.isEnabledFor(logging.DEBUG): logger.debug("Registering runner object [%s] for [%s].", str(runner), str(operation_type)) __RUNNERS[operation_type] = _multi_cluster_runner(runner, str(runner), context_manager_enabled=True) @@ -87,7 +91,7 @@ def register_runner(operation_type, runner): if logger.isEnabledFor(logging.DEBUG): logger.debug("Registering runner function [%s] for [%s].", str(runner), str(operation_type)) __RUNNERS[operation_type] = _single_cluster_runner(runner, runner.__name__) - elif "__enter__" in dir(runner) and "__exit__" in dir(runner): + elif "__aenter__" in dir(runner) and "__aexit__" in dir(runner): if logger.isEnabledFor(logging.DEBUG): logger.debug("Registering context-manager capable runner object [%s] for [%s].", str(runner), str(operation_type)) __RUNNERS[operation_type] = _single_cluster_runner(runner, str(runner), context_manager_enabled=True) @@ -111,10 +115,10 @@ def __init__(self, *args, **kwargs): super(Runner, self).__init__(*args, **kwargs) self.logger = logging.getLogger(__name__) - def __enter__(self): + async def __aenter__(self): return self - def __call__(self, *args): + async def __call__(self, *args): """ Runs the actual method that should be benchmarked. @@ -126,7 +130,7 @@ def __call__(self, *args): """ raise NotImplementedError("abstract operation") - def __exit__(self, exc_type, exc_val, exc_tb): + async def __aexit__(self, exc_type, exc_val, exc_tb): return False @@ -185,18 +189,18 @@ def completed(self): def percent_completed(self): return None - def __call__(self, *args): - return self.delegate(*args) + async def __call__(self, *args): + return await self.delegate(*args) def __repr__(self, *args, **kwargs): return repr(self.delegate) - def __enter__(self): - self.delegate.__enter__() + async def __aenter__(self): + await self.delegate.__aenter__() return self - def __exit__(self, exc_type, exc_val, exc_tb): - return self.delegate.__exit__(exc_type, exc_val, exc_tb) + async def __aexit__(self, exc_type, exc_val, exc_tb): + return await self.delegate.__aexit__(exc_type, exc_val, exc_tb) class WithCompletion(Runner, Delegator): @@ -212,18 +216,18 @@ def completed(self): def percent_completed(self): return self.progressable.percent_completed - def __call__(self, *args): - return self.delegate(*args) + async def __call__(self, *args): + return await self.delegate(*args) def __repr__(self, *args, **kwargs): return repr(self.delegate) - def __enter__(self): - self.delegate.__enter__() + async def __aenter__(self): + await self.delegate.__aenter__() return self - def __exit__(self, exc_type, exc_val, exc_tb): - return self.delegate.__exit__(exc_type, exc_val, exc_tb) + async def __aexit__(self, exc_type, exc_val, exc_tb): + return await self.delegate.__aexit__(exc_type, exc_val, exc_tb) class MultiClientRunner(Runner, Delegator): @@ -233,8 +237,8 @@ def __init__(self, runnable, name, client_extractor, context_manager_enabled=Fal self.client_extractor = client_extractor self.context_manager_enabled = context_manager_enabled - def __call__(self, *args): - return self.delegate(self.client_extractor(args[0]), *args[1:]) + async def __call__(self, *args): + return await self.delegate(self.client_extractor(args[0]), *args[1:]) def __repr__(self, *args, **kwargs): if self.context_manager_enabled: @@ -242,14 +246,14 @@ def __repr__(self, *args, **kwargs): else: return "user-defined runner for [%s]" % self.name - def __enter__(self): + async def __aenter__(self): if self.context_manager_enabled: - self.delegate.__enter__() + await self.delegate.__aenter__() return self - def __exit__(self, exc_type, exc_val, exc_tb): + async def __aexit__(self, exc_type, exc_val, exc_tb): if self.context_manager_enabled: - return self.delegate.__exit__(exc_type, exc_val, exc_tb) + return await self.delegate.__aexit__(exc_type, exc_val, exc_tb) else: return False @@ -262,7 +266,7 @@ def mandatory(params, key, op): " parameter source." % (str(op), key)) -class AsyncBulkIndex(Runner): +class BulkIndex(Runner): """ Bulk indexes the given documents. """ @@ -462,307 +466,6 @@ async def __call__(self, es, params): meta_data["error-type"] = "bulk" return meta_data - def detailed_stats(self, params, bulk_size, response): - ops = {} - shards_histogram = OrderedDict() - bulk_error_count = 0 - error_details = set() - bulk_request_size_bytes = 0 - total_document_size_bytes = 0 - with_action_metadata = mandatory(params, "action-metadata-present", self) - - for line_number, data in enumerate(params["body"]): - line_size = len(data.encode('utf-8')) - if with_action_metadata: - if line_number % 2 == 1: - total_document_size_bytes += line_size - else: - total_document_size_bytes += line_size - - bulk_request_size_bytes += line_size - - for idx, item in enumerate(response["items"]): - # there is only one (top-level) item - op, data = next(iter(item.items())) - if op not in ops: - ops[op] = Counter() - ops[op]["item-count"] += 1 - if "result" in data: - ops[op][data["result"]] += 1 - - if "_shards" in data: - s = data["_shards"] - sk = "%d-%d-%d" % (s["total"], s["successful"], s["failed"]) - if sk not in shards_histogram: - shards_histogram[sk] = { - "item-count": 0, - "shards": s - } - shards_histogram[sk]["item-count"] += 1 - if data["status"] > 299 or ("_shards" in data and data["_shards"]["failed"] > 0): - bulk_error_count += 1 - self.extract_error_details(error_details, data) - stats = { - "took": response.get("took"), - "success": bulk_error_count == 0, - "success-count": bulk_size - bulk_error_count, - "error-count": bulk_error_count, - "ops": ops, - "shards_histogram": list(shards_histogram.values()), - "bulk-request-size-bytes": bulk_request_size_bytes, - "total-document-size-bytes": total_document_size_bytes - } - if bulk_error_count > 0: - stats["error-type"] = "bulk" - stats["error-description"] = self.error_description(error_details) - if "ingest_took" in response: - stats["ingest_took"] = response["ingest_took"] - - return stats - - def simple_stats(self, bulk_size, response): - bulk_error_count = 0 - error_details = set() - if response["errors"]: - for idx, item in enumerate(response["items"]): - data = next(iter(item.values())) - if data["status"] > 299 or ('_shards' in data and data["_shards"]["failed"] > 0): - bulk_error_count += 1 - self.extract_error_details(error_details, data) - stats = { - "took": response.get("took"), - "success": bulk_error_count == 0, - "success-count": bulk_size - bulk_error_count, - "error-count": bulk_error_count - } - if "ingest_took" in response: - stats["ingest_took"] = response["ingest_took"] - if bulk_error_count > 0: - stats["error-type"] = "bulk" - stats["error-description"] = self.error_description(error_details) - return stats - - def extract_error_details(self, error_details, data): - error_data = data.get("error", {}) - error_reason = error_data.get("reason") if isinstance(error_data, dict) else str(error_data) - if error_data: - error_details.add((data["status"], error_reason)) - else: - error_details.add((data["status"], None)) - - def error_description(self, error_details): - error_description = "" - for status, reason in error_details: - if reason: - error_description += "HTTP status: %s, message: %s" % (str(status), reason) - else: - error_description += "HTTP status: %s" % str(status) - return error_description - - def __repr__(self, *args, **kwargs): - return "bulk-index" - - -class BulkIndex(Runner): - """ - Bulk indexes the given documents. - """ - - def __init__(self): - super().__init__() - - def __call__(self, es, params): - """ - Runs one bulk indexing operation. - - :param es: The Elasticsearch client. - :param params: A hash with all parameters. See below for details. - :return: A hash with meta data for this bulk operation. See below for details. - - It expects a parameter dict with the following mandatory keys: - - * ``body``: containing all documents for the current bulk request. - * ``bulk-size``: the number of documents in this bulk. - * ``action_metadata_present``: if ``True``, assume that an action and metadata line is present (meaning only half of the lines - contain actual documents to index) - * ``index``: The name of the affected index in case ``action_metadata_present`` is ``False``. - * ``type``: The name of the affected type in case ``action_metadata_present`` is ``False``. - - The following keys are optional: - - * ``pipeline``: If present, runs the the specified ingest pipeline for this bulk. - * ``detailed-results``: If ``True``, the runner will analyze the response and add detailed meta-data. Defaults to ``False``. Note - that this has a very significant impact on performance and will very likely cause a bottleneck in the benchmark driver so please - be very cautious enabling this feature. Our own measurements have shown a median overhead of several thousand times (execution time - is in the single digit microsecond range when this feature is disabled and in the single digit millisecond range when this feature - is enabled; numbers based on a bulk size of 500 elements and no errors). For details please refer to the respective benchmarks - in ``benchmarks/driver``. - - - Returned meta data - ` - The following meta data are always returned: - - * ``index``: name of the affected index. May be `None` if it could not be derived. - * ``bulk-size``: bulk size, e.g. 5.000. - * ``bulk-request-size-bytes``: size of the full bulk requset in bytes - * ``total-document-size-bytes``: size of all documents contained in the bulk request in bytes - * ``weight``: operation-agnostic representation of the bulk size (used internally by Rally for throughput calculation). - * ``unit``: The unit in which to interpret ``bulk-size`` and ``weight``. Always "docs". - * ``success``: A boolean indicating whether the bulk request has succeeded. - * ``success-count``: Number of successfully processed items for this request (denoted in ``unit``). - * ``error-count``: Number of failed items for this request (denoted in ``unit``). - * ``took``` Value of the the ``took`` property in the bulk response. - - If ``detailed-results`` is ``True`` the following meta data are returned in addition: - - * ``ops``: A hash with the operation name as key (e.g. index, update, delete) and various counts as values. ``item-count`` contains - the total number of items for this key. Additionally, we return a separate counter each result (indicating e.g. the number of - created items, the number of deleted items etc.). - * ``shards_histogram``: An array of hashes where each hash has two keys: ``item-count`` contains the number of items to which a - shard distribution applies and ``shards`` contains another hash with the actual distribution of ``total``, ``successful`` and - ``failed`` shards (see examples below). - * ``bulk-request-size-bytes``: Total size of the bulk request body in bytes. - * ``total-document-size-bytes``: Total size of all documents within the bulk request body in bytes. - - Here are a few examples: - - If ``detailed-results`` is ``False`` a typical return value is:: - - { - "index": "my_index", - "weight": 5000, - "unit": "docs", - "bulk-size": 5000, - "success": True, - "success-count": 5000, - "error-count": 0, - "took": 20 - } - - Whereas the response will look as follow if there are bulk errors:: - - { - "index": "my_index", - "weight": 5000, - "unit": "docs", - "bulk-size": 5000, - "success": False, - "success-count": 4000, - "error-count": 1000, - "took": 20 - } - - If ``detailed-results`` is ``True`` a typical return value is:: - - - { - "index": "my_index", - "weight": 5000, - "unit": "docs", - "bulk-size": 5000, - "bulk-request-size-bytes": 2250000, - "total-document-size-bytes": 2000000, - "success": True, - "success-count": 5000, - "error-count": 0, - "took": 20, - "ops": { - "index": { - "item-count": 5000, - "created": 5000 - } - }, - "shards_histogram": [ - { - "item-count": 5000, - "shards": { - "total": 2, - "successful": 2, - "failed": 0 - } - } - ] - } - - An example error response may look like this:: - - - { - "index": "my_index", - "weight": 5000, - "unit": "docs", - "bulk-size": 5000, - "bulk-request-size-bytes": 2250000, - "total-document-size-bytes": 2000000, - "success": False, - "success-count": 4000, - "error-count": 1000, - "took": 20, - "ops": { - "index": { - "item-count": 5000, - "created": 4000, - "noop": 1000 - } - }, - "shards_histogram": [ - { - "item-count": 4000, - "shards": { - "total": 2, - "successful": 2, - "failed": 0 - } - }, - { - "item-count": 500, - "shards": { - "total": 2, - "successful": 1, - "failed": 1 - } - }, - { - "item-count": 500, - "shards": { - "total": 2, - "successful": 0, - "failed": 2 - } - } - ] - } - """ - detailed_results = params.get("detailed-results", False) - index = params.get("index") - - bulk_params = {} - if "pipeline" in params: - bulk_params["pipeline"] = params["pipeline"] - - with_action_metadata = mandatory(params, "action-metadata-present", self) - bulk_size = mandatory(params, "bulk-size", self) - - if with_action_metadata: - # only half of the lines are documents - response = es.bulk(body=params["body"], params=bulk_params) - else: - response = es.bulk(body=params["body"], index=index, doc_type=params.get("type"), params=bulk_params) - - stats = self.detailed_stats(params, bulk_size, response) if detailed_results else self.simple_stats(bulk_size, response) - - meta_data = { - "index": str(index) if index else None, - "weight": bulk_size, - "unit": "docs", - "bulk-size": bulk_size - } - meta_data.update(stats) - if not stats["success"]: - meta_data["error-type"] = "bulk" - return meta_data - def detailed_stats(self, params, bulk_size, response): ops = {} shards_histogram = OrderedDict() @@ -876,7 +579,7 @@ class ForceMerge(Runner): Runs a force merge operation against Elasticsearch. """ - def __call__(self, es, params): + async def __call__(self, es, params): import elasticsearch max_num_segments = params.get("max-num-segments") # preliminary support for overriding the global request timeout (see #567). As force-merge falls back to @@ -885,18 +588,18 @@ def __call__(self, es, params): request_timeout = params.get("request-timeout") try: if max_num_segments: - es.indices.forcemerge(index=params.get("index"), max_num_segments=max_num_segments, request_timeout=request_timeout) + await es.indices.forcemerge(index=params.get("index"), max_num_segments=max_num_segments, request_timeout=request_timeout) else: - es.indices.forcemerge(index=params.get("index"), request_timeout=request_timeout) + await es.indices.forcemerge(index=params.get("index"), request_timeout=request_timeout) except elasticsearch.TransportError as e: # this is caused by older versions of Elasticsearch (< 2.1), fall back to optimize if e.status_code == 400: params = {"request_timeout": request_timeout} if max_num_segments: - es.transport.perform_request("POST", "/_optimize?max_num_segments={}".format(max_num_segments), - params=params) + await es.transport.perform_request("POST", "/_optimize?max_num_segments={}".format(max_num_segments), + params=params) else: - es.transport.perform_request("POST", "/_optimize", params=params) + await es.transport.perform_request("POST", "/_optimize", params=params) else: raise e @@ -909,8 +612,8 @@ class IndicesStats(Runner): Gather index stats for all indices. """ - def __call__(self, es, params): - es.indices.stats(metric="_all") + async def __call__(self, es, params): + await es.indices.stats(metric="_all") def __repr__(self, *args, **kwargs): return "indices-stats" @@ -921,8 +624,8 @@ class NodeStats(Runner): Gather node stats for all nodes. """ - def __call__(self, es, params): - es.nodes.stats(metric="_all") + async def __call__(self, es, params): + await es.nodes.stats(metric="_all") def __repr__(self, *args, **kwargs): return "node-stats" @@ -976,13 +679,13 @@ def __init__(self): self.scroll_id = None self.es = None - def __call__(self, es, params): + async def __call__(self, es, params): if "pages" in params and "results-per-page" in params: - return self.scroll_query(es, params) + return await self.scroll_query(es, params) else: - return self.request_body_query(es, params) + return await self.request_body_query(es, params) - def request_body_query(self, es, params): + async def request_body_query(self, es, params): request_params = self._default_request_params(params) index = params.get("index", "_all") body = mandatory(params, "body", self) @@ -990,9 +693,9 @@ def request_body_query(self, es, params): params = request_params if doc_type is not None: - r = search_type_fallback(es, doc_type, index, body, params) + r = await search_type_fallback(es, doc_type, index, body, params) else: - r = es.search(index=index, body=body, params=params) + r = await es.search(index=index, body=body, params=params) hits = r["hits"]["total"] if isinstance(hits, dict): hits_total = hits["value"] @@ -1009,7 +712,7 @@ def request_body_query(self, es, params): "took": r["took"] } - def scroll_query(self, es, params): + async def scroll_query(self, es, params): request_params = self._default_request_params(params) hits = 0 retrieved_pages = 0 @@ -1034,11 +737,11 @@ def scroll_query(self, es, params): params["size"] = size r = search_type_fallback(es, doc_type, index, body, params) else: - r = es.search(index=index, body=body, params=params, sort=sort, scroll=scroll, size=size) + r = await es.search(index=index, body=body, params=params, sort=sort, scroll=scroll, size=size) # This should only happen if we concurrently create an index and start searching self.scroll_id = r.get("_scroll_id", None) else: - r = es.scroll(body={"scroll_id": self.scroll_id, "scroll": "10s"}) + r = await es.scroll(body={"scroll_id": self.scroll_id, "scroll": "10s"}) hit_count = len(r["hits"]["hits"]) timed_out = timed_out or r["timed_out"] took += r["took"] @@ -1066,10 +769,10 @@ def _default_request_params(self, params): request_params["request_cache"] = str(cache).lower() return request_params - def __exit__(self, exc_type, exc_val, exc_tb): + async def __aexit__(self, exc_type, exc_val, exc_tb): if self.scroll_id and self.es: try: - self.es.clear_scroll(body={"scroll_id": [self.scroll_id]}) + await self.es.clear_scroll(body={"scroll_id": [self.scroll_id]}) except BaseException: self.logger.exception("Could not clear scroll [%s]. This will lead to excessive resource usage in " "Elasticsearch and will skew your benchmark results.", self.scroll_id) @@ -1086,7 +789,7 @@ class ClusterHealth(Runner): Get cluster health """ - def __call__(self, es, params): + async def __call__(self, es, params): from enum import Enum from functools import total_ordering @@ -1121,7 +824,7 @@ def status(v): # either the user has defined something or we're good with any count of relocating shards. expected_relocating_shards = int(request_params.get("wait_for_relocating_shards", sys.maxsize)) - result = es.cluster.health(index=index, params=request_params) + result = await es.cluster.health(index=index, params=request_params) cluster_status = result["status"] relocating_shards = result["relocating_shards"] @@ -1143,12 +846,12 @@ class PutPipeline(Runner): API is only available from Elasticsearch 5.0 onwards. """ - def __call__(self, es, params): - es.ingest.put_pipeline(id=mandatory(params, "id", self), - body=mandatory(params, "body", self), - master_timeout=params.get("master-timeout"), - timeout=params.get("timeout"), - ) + async def __call__(self, es, params): + await es.ingest.put_pipeline(id=mandatory(params, "id", self), + body=mandatory(params, "body", self), + master_timeout=params.get("master-timeout"), + timeout=params.get("timeout"), + ) def __repr__(self, *args, **kwargs): return "put-pipeline" @@ -1159,8 +862,8 @@ class Refresh(Runner): Execute the `refresh API `_. """ - def __call__(self, es, params): - es.indices.refresh(index=params.get("index", "_all")) + async def __call__(self, es, params): + await es.indices.refresh(index=params.get("index", "_all")) def __repr__(self, *args, **kwargs): return "refresh" @@ -1171,11 +874,11 @@ class CreateIndex(Runner): Execute the `create index API `_. """ - def __call__(self, es, params): + async def __call__(self, es, params): indices = mandatory(params, "indices", self) request_params = params.get("request-params", {}) for index, body in indices: - es.indices.create(index=index, body=body, params=request_params) + await es.indices.create(index=index, body=body, params=request_params) return len(indices), "ops" def __repr__(self, *args, **kwargs): @@ -1187,7 +890,7 @@ class DeleteIndex(Runner): Execute the `delete index API `_. """ - def __call__(self, es, params): + async def __call__(self, es, params): ops = 0 indices = mandatory(params, "indices", self) @@ -1196,11 +899,11 @@ def __call__(self, es, params): for index_name in indices: if not only_if_exists: - es.indices.delete(index=index_name, params=request_params) + await es.indices.delete(index=index_name, params=request_params) ops += 1 - elif only_if_exists and es.indices.exists(index=index_name): + elif only_if_exists and await es.indices.exists(index=index_name): self.logger.info("Index [%s] already exists. Deleting it.", index_name) - es.indices.delete(index=index_name, params=request_params) + await es.indices.delete(index=index_name, params=request_params) ops += 1 return ops, "ops" @@ -1214,13 +917,13 @@ class CreateIndexTemplate(Runner): Execute the `PUT index template API `_. """ - def __call__(self, es, params): + async def __call__(self, es, params): templates = mandatory(params, "templates", self) request_params = params.get("request-params", {}) for template, body in templates: - es.indices.put_template(name=template, - body=body, - params=request_params) + await es.indices.put_template(name=template, + body=body, + params=request_params) return len(templates), "ops" def __repr__(self, *args, **kwargs): @@ -1233,7 +936,7 @@ class DeleteIndexTemplate(Runner): `_. """ - def __call__(self, es, params): + async def __call__(self, es, params): template_names = mandatory(params, "templates", self) only_if_exists = params.get("only-if-exists", False) request_params = params.get("request-params", {}) @@ -1241,15 +944,15 @@ def __call__(self, es, params): for template_name, delete_matching_indices, index_pattern in template_names: if not only_if_exists: - es.indices.delete_template(name=template_name, params=request_params) + await es.indices.delete_template(name=template_name, params=request_params) ops_count += 1 - elif only_if_exists and es.indices.exists_template(template_name): + elif only_if_exists and await es.indices.exists_template(template_name): self.logger.info("Index template [%s] already exists. Deleting it.", template_name) - es.indices.delete_template(name=template_name, params=request_params) + await es.indices.delete_template(name=template_name, params=request_params) ops_count += 1 # ensure that we do not provide an empty index pattern by accident if delete_matching_indices and index_pattern: - es.indices.delete(index=index_pattern) + await es.indices.delete(index=index_pattern) ops_count += 1 return ops_count, "ops" @@ -1269,10 +972,10 @@ def __init__(self): super().__init__() self.cluster_health = Retry(ClusterHealth()) - def _wait_for(self, es, idx, description): + async def _wait_for(self, es, idx, description): # wait a little bit before the first check - time.sleep(3) - result = self.cluster_health(es, params={ + await asyncio.sleep(3) + result = await self.cluster_health(es, params={ "index": idx, "retries": sys.maxsize, "request-params": { @@ -1282,7 +985,7 @@ def _wait_for(self, es, idx, description): if not result["success"]: raise exceptions.RallyAssertionError("Failed to wait for [{}].".format(description)) - def __call__(self, es, params): + async def __call__(self, es, params): source_index = mandatory(params, "source-index", self) target_index = mandatory(params, "target-index", self) # we need to inject additional settings so we better copy the body @@ -1292,7 +995,8 @@ def __call__(self, es, params): if not shrink_node: node_names = [] # choose a random data node - for node in es.nodes.info()["nodes"].values(): + node_info = await es.nodes.info() + for node in node_info["nodes"].values(): if "data" in node["roles"]: node_names.append(node["name"]) if not node_names: @@ -1301,27 +1005,27 @@ def __call__(self, es, params): self.logger.info("Using [%s] as shrink node.", shrink_node) self.logger.info("Preparing [%s] for shrinking.", source_index) # prepare index for shrinking - es.indices.put_settings(index=source_index, - body={ - "settings": { - "index.routing.allocation.require._name": shrink_node, - "index.blocks.write": "true" - } - }, - preserve_existing=True) + await es.indices.put_settings(index=source_index, + body={ + "settings": { + "index.routing.allocation.require._name": shrink_node, + "index.blocks.write": "true" + } + }, + preserve_existing=True) self.logger.info("Waiting for relocation to finish for index [%s]...", source_index) - self._wait_for(es, source_index, "shard relocation for index [{}]".format(source_index)) + await self._wait_for(es, source_index, "shard relocation for index [{}]".format(source_index)) self.logger.info("Shrinking [%s] to [%s].", source_index, target_index) if "settings" not in target_body: target_body["settings"] = {} target_body["settings"]["index.routing.allocation.require._name"] = None target_body["settings"]["index.blocks.write"] = None # kick off the shrink operation - es.indices.shrink(index=source_index, target=target_index, body=target_body) + await es.indices.shrink(index=source_index, target=target_index, body=target_body) self.logger.info("Waiting for shrink to finish for index [%s]...", source_index) - self._wait_for(es, target_index, "shrink for index [{}]".format(target_index)) + await self._wait_for(es, target_index, "shrink for index [{}]".format(target_index)) self.logger.info("Shrinking [%s] to [%s] has finished.", source_index, target_index) # ops_count is not really important for this operation... return 1, "ops" @@ -1335,16 +1039,16 @@ class CreateMlDatafeed(Runner): Execute the `create datafeed API `_. """ - def __call__(self, es, params): + async def __call__(self, es, params): import elasticsearch datafeed_id = mandatory(params, "datafeed-id", self) body = mandatory(params, "body", self) try: - es.xpack.ml.put_datafeed(datafeed_id=datafeed_id, body=body) + await es.xpack.ml.put_datafeed(datafeed_id=datafeed_id, body=body) except elasticsearch.TransportError as e: # fallback to old path if e.status_code == 400: - es.transport.perform_request( + await es.transport.perform_request( "PUT", "/_xpack/ml/datafeeds/%s" % datafeed_id, params=params, @@ -1362,19 +1066,19 @@ class DeleteMlDatafeed(Runner): Execute the `delete datafeed API `_. """ - def __call__(self, es, params): + async def __call__(self, es, params): import elasticsearch datafeed_id = mandatory(params, "datafeed-id", self) force = params.get("force", False) try: # we don't want to fail if a datafeed does not exist, thus we ignore 404s. - es.xpack.ml.delete_datafeed(datafeed_id=datafeed_id, force=force, ignore=[404]) + await es.xpack.ml.delete_datafeed(datafeed_id=datafeed_id, force=force, ignore=[404]) except elasticsearch.TransportError as e: # fallback to old path (ES < 7) if e.status_code == 400: - es.transport.perform_request( + await es.transport.perform_request( "DELETE", - "/_xpack/ml/datafeeds/%s" %datafeed_id, + "/_xpack/ml/datafeeds/%s" % datafeed_id, params=params, ) else: @@ -1389,7 +1093,7 @@ class StartMlDatafeed(Runner): Execute the `start datafeed API `_. """ - def __call__(self, es, params): + async def __call__(self, es, params): import elasticsearch datafeed_id = mandatory(params, "datafeed-id", self) body = params.get("body") @@ -1397,11 +1101,11 @@ def __call__(self, es, params): end = params.get("end") timeout = params.get("timeout") try: - es.xpack.ml.start_datafeed(datafeed_id=datafeed_id, body=body, start=start, end=end, timeout=timeout) + await es.xpack.ml.start_datafeed(datafeed_id=datafeed_id, body=body, start=start, end=end, timeout=timeout) except elasticsearch.TransportError as e: # fallback to old path (ES < 7) if e.status_code == 400: - es.transport.perform_request( + await es.transport.perform_request( "POST", "/_xpack/ml/datafeeds/%s/_start" % datafeed_id, params=params, @@ -1419,17 +1123,17 @@ class StopMlDatafeed(Runner): Execute the `stop datafeed API `_. """ - def __call__(self, es, params): + async def __call__(self, es, params): import elasticsearch datafeed_id = mandatory(params, "datafeed-id", self) force = params.get("force", False) timeout = params.get("timeout") try: - es.xpack.ml.stop_datafeed(datafeed_id=datafeed_id, force=force, timeout=timeout) + await es.xpack.ml.stop_datafeed(datafeed_id=datafeed_id, force=force, timeout=timeout) except elasticsearch.TransportError as e: # fallback to old path (ES < 7) if e.status_code == 400: - es.transport.perform_request( + await es.transport.perform_request( "POST", "/_xpack/ml/datafeeds/%s/_stop" % datafeed_id, params=params @@ -1446,16 +1150,16 @@ class CreateMlJob(Runner): Execute the `create job API `_. """ - def __call__(self, es, params): + async def __call__(self, es, params): import elasticsearch job_id = mandatory(params, "job-id", self) body = mandatory(params, "body", self) try: - es.xpack.ml.put_job(job_id=job_id, body=body) + await es.xpack.ml.put_job(job_id=job_id, body=body) except elasticsearch.TransportError as e: # fallback to old path (ES < 7) if e.status_code == 400: - es.transport.perform_request( + await es.transport.perform_request( "PUT", "/_xpack/ml/anomaly_detectors/%s" % job_id, params=params, @@ -1473,13 +1177,13 @@ class DeleteMlJob(Runner): Execute the `delete job API `_. """ - def __call__(self, es, params): + async def __call__(self, es, params): import elasticsearch job_id = mandatory(params, "job-id", self) force = params.get("force", False) # we don't want to fail if a job does not exist, thus we ignore 404s. try: - es.xpack.ml.delete_job(job_id=job_id, force=force, ignore=[404]) + await es.xpack.ml.delete_job(job_id=job_id, force=force, ignore=[404]) except elasticsearch.TransportError as e: # fallback to old path (ES < 7) if e.status_code == 400: @@ -1500,15 +1204,15 @@ class OpenMlJob(Runner): Execute the `open job API `_. """ - def __call__(self, es, params): + async def __call__(self, es, params): import elasticsearch job_id = mandatory(params, "job-id", self) try: - es.xpack.ml.open_job(job_id=job_id) + await es.xpack.ml.open_job(job_id=job_id) except elasticsearch.TransportError as e: # fallback to old path (ES < 7) if e.status_code == 400: - es.transport.perform_request( + await es.transport.perform_request( "POST", "/_xpack/ml/anomaly_detectors/%s/_open" % job_id, params=params, @@ -1525,17 +1229,17 @@ class CloseMlJob(Runner): Execute the `close job API `_. """ - def __call__(self, es, params): + async def __call__(self, es, params): import elasticsearch job_id = mandatory(params, "job-id", self) force = params.get("force", False) timeout = params.get("timeout") try: - es.xpack.ml.close_job(job_id=job_id, force=force, timeout=timeout) + await es.xpack.ml.close_job(job_id=job_id, force=force, timeout=timeout) except elasticsearch.TransportError as e: # fallback to old path (ES < 7) if e.status_code == 400: - es.transport.perform_request( + await es.transport.perform_request( "POST", "/_xpack/ml/anomaly_detectors/%s/_close" % job_id, params=params, @@ -1548,17 +1252,17 @@ def __repr__(self, *args, **kwargs): class RawRequest(Runner): - def __call__(self, es, params): + async def __call__(self, es, params): request_params = {} if "ignore" in params: request_params["ignore"] = params["ignore"] request_params.update(params.get("request-params", {})) - es.transport.perform_request(method=params.get("method", "GET"), - url=mandatory(params, "path", self), - headers=params.get("headers"), - body=params.get("body"), - params=request_params) + await es.transport.perform_request(method=params.get("method", "GET"), + url=mandatory(params, "path", self), + headers=params.get("headers"), + body=params.get("body"), + params=request_params) def __repr__(self, *args, **kwargs): return "raw-request" @@ -1569,8 +1273,8 @@ class Sleep(Runner): Sleeps for the specified duration not issuing any request. """ - def __call__(self, es, params): - time.sleep(mandatory(params, "duration", "sleep")) + async def __call__(self, es, params): + await asyncio.sleep(mandatory(params, "duration", "sleep")) def __repr__(self, *args, **kwargs): return "sleep" @@ -1580,8 +1284,8 @@ class DeleteSnapshotRepository(Runner): """ Deletes a snapshot repository """ - def __call__(self, es, params): - es.snapshot.delete_repository(repository=mandatory(params, "repository", repr(self))) + async def __call__(self, es, params): + await es.snapshot.delete_repository(repository=mandatory(params, "repository", repr(self))) def __repr__(self, *args, **kwargs): return "delete-snapshot-repository" @@ -1591,12 +1295,11 @@ class CreateSnapshotRepository(Runner): """ Creates a new snapshot repository """ - def __call__(self, es, params): + async def __call__(self, es, params): request_params = params.get("request-params", {}) - es.snapshot.create_repository( - repository=mandatory(params, "repository", repr(self)), - body=mandatory(params, "body", repr(self)), - params=request_params) + await es.snapshot.create_repository(repository=mandatory(params, "repository", repr(self)), + body=mandatory(params, "body", repr(self)), + params=request_params) def __repr__(self, *args, **kwargs): return "create-snapshot-repository" @@ -1606,13 +1309,13 @@ class RestoreSnapshot(Runner): """ Restores a snapshot from an already registered repository """ - def __call__(self, es, params): + async def __call__(self, es, params): request_params = params.get("request-params", {}) - es.snapshot.restore(repository=mandatory(params, "repository", repr(self)), - snapshot=mandatory(params, "snapshot", repr(self)), - body=params.get("body"), - wait_for_completion=params.get("wait-for-completion", False), - params=request_params) + await es.snapshot.restore(repository=mandatory(params, "repository", repr(self)), + snapshot=mandatory(params, "snapshot", repr(self)), + body=params.get("body"), + wait_for_completion=params.get("wait-for-completion", False), + params=request_params) def __repr__(self, *args, **kwargs): return "restore-snapshot" @@ -1633,17 +1336,17 @@ def completed(self): def percent_completed(self): return self._percent_completed - def __call__(self, es, params): + async def __call__(self, es, params): remaining_attempts = params.get("completion-recheck-attempts", 3) wait_period = params.get("completion-recheck-wait-period", 2) response = None while not response and remaining_attempts > 0: - response = es.indices.recovery(active_only=True) + response = await es.indices.recovery(active_only=True) remaining_attempts -= 1 # This might also happen if all recoveries have just finished and we happen to call the API # before the next recovery is scheduled. if not response: - time.sleep(wait_period) + await asyncio.sleep(wait_period) if not response: self._completed = True @@ -1681,8 +1384,8 @@ class PutSettings(Runner): Updates cluster settings with the `cluster settings API _. """ - def __call__(self, es, params): - es.cluster.put_settings(body=mandatory(params, "body", repr(self))) + async def __call__(self, es, params): + await es.cluster.put_settings(body=mandatory(params, "body", repr(self))) def __repr__(self, *args, **kwargs): return "put-settings" @@ -1708,11 +1411,11 @@ class Retry(Runner, Delegator): def __init__(self, delegate): super().__init__(delegate=delegate) - def __enter__(self): - self.delegate.__enter__() + async def __aenter__(self): + await self.delegate.__aenter__() return self - def __call__(self, es, params): + async def __call__(self, es, params): import elasticsearch import socket @@ -1729,7 +1432,7 @@ def __call__(self, es, params): for attempt in range(max_attempts): last_attempt = attempt + 1 == max_attempts try: - return_value = self.delegate(es, params) + return_value = await self.delegate(es, params) if last_attempt or not retry_on_error: return return_value # we can determine success if and only if the runner returns a dict. Otherwise, we have to assume it was fine. @@ -1737,25 +1440,25 @@ def __call__(self, es, params): if return_value.get("success", True): return return_value else: - time.sleep(sleep_time) + await asyncio.sleep(sleep_time) else: return return_value except (socket.timeout, elasticsearch.exceptions.ConnectionError): if last_attempt or not retry_on_timeout: raise else: - time.sleep(sleep_time) + await asyncio.sleep(sleep_time) except elasticsearch.exceptions.TransportError as e: if last_attempt or not retry_on_timeout: raise e elif e.status_code == 408: self.logger.debug("%s has timed out.", repr(self.delegate)) - time.sleep(sleep_time) + await asyncio.sleep(sleep_time) else: raise e - def __exit__(self, exc_type, exc_val, exc_tb): - return self.delegate.__exit__(exc_type, exc_val, exc_tb) + async def __aexit__(self, exc_type, exc_val, exc_tb): + return await self.delegate.__aexit__(exc_type, exc_val, exc_tb) def __repr__(self, *args, **kwargs): return "retryable %s" % repr(self.delegate) diff --git a/esrally/utils/io.py b/esrally/utils/io.py index 1617e9515..8e88df558 100644 --- a/esrally/utils/io.py +++ b/esrally/utils/io.py @@ -154,6 +154,8 @@ def __str__(self, *args, **kwargs): return "StringAsFileSource" +# TODO: Remove before merging +# pylint: disable=C0301 class StaticSource: def __init__(self, contents, mode, encoding="utf-8"): self.contents = '{"geonameid": 2986043, "name": "Pic de Font Blanca", "asciiname": "Pic de Font Blanca", "alternatenames": "Pic de Font Blanca,Pic du Port", "feature_class": "T", "feature_code": "PK", "country_code": "AD", "admin1_code": "00", "population": 0, "dem": "2860", "timezone": "Europe/Andorra", "location": [1.53335, 42.64991]}' diff --git a/integration-test.sh b/integration-test.sh index dad7a381f..698938496 100755 --- a/integration-test.sh +++ b/integration-test.sh @@ -533,11 +533,11 @@ function test_node_management_commands { info "test start [--configuration-name=${cfg}]" esrally start --quiet --configuration-name="${cfg}" --installation-id="${install_id}" --race-id="rally-integration-test" - esrally --target-host="localhost:39200" \ + esrally race-async \ + --target-host="localhost:39200" \ --configuration-name="${cfg}" \ --race-id="rally-integration-test" \ --on-error=abort \ - --pipeline=benchmark-only \ --track=geonames \ --test-mode \ --challenge=append-no-conflicts-index-only diff --git a/setup.py b/setup.py index 33d1bd93d..9cbc95927 100644 --- a/setup.py +++ b/setup.py @@ -75,7 +75,6 @@ def str_from_file(name): # jmespath: MIT # s3transfer: Apache 2.0 "boto3==1.10.32", - # TODO dm: Temporarily here - *might* be removed later "yappi==1.2.3" ] diff --git a/tests/__init__.py b/tests/__init__.py index a2833546a..c0e50348c 100644 --- a/tests/__init__.py +++ b/tests/__init__.py @@ -14,3 +14,35 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. + +import asyncio + + +def run_async(t): + """ + A wrapper that ensures that a test is run in an asyncio context. + + :param t: The test case to wrap. + """ + def async_wrapper(*args, **kwargs): + asyncio.run(t(*args, **kwargs), debug=True) + return async_wrapper + + +def as_future(result=None, exception=None): + """ + + Helper to create a future that completes immediately either with a result or exceptionally. + + :param result: Regular result. + :param exception: Exceptional result. + :return: The corresponding future. + """ + f = asyncio.Future() + if exception and result: + raise AssertionError("Specify a result or an exception but not both") + if exception: + f.set_exception(exception) + else: + f.set_result(result) + return f diff --git a/tests/driver/driver_test.py b/tests/driver/driver_test.py index efcf1daa0..6410d85ad 100644 --- a/tests/driver/driver_test.py +++ b/tests/driver/driver_test.py @@ -15,6 +15,7 @@ # specific language governing permissions and limitations # under the License. +import asyncio import collections import threading import time @@ -26,6 +27,8 @@ from esrally.driver import driver, runner from esrally.track import params +from tests import run_async, as_future + class DriverTestParamSource: def __init__(self, track=None, params=None, **kwargs): @@ -218,25 +221,6 @@ def test_client_reaches_join_point_which_completes_parent(self): self.assertEqual(4, target.drive_at.call_count) -class ScheduleTestCase(TestCase): - def assert_schedule(self, expected_schedule, schedule, infinite_schedule=False): - if not infinite_schedule: - self.assertEqual(len(expected_schedule), len(schedule), - msg="Number of elements in the schedules do not match") - idx = 0 - for invocation_time, sample_type, progress_percent, runner, params in schedule: - exp_invocation_time, exp_sample_type, exp_progress_percent, exp_params = expected_schedule[idx] - self.assertAlmostEqual(exp_invocation_time, invocation_time, msg="Invocation time for sample at index %d does not match" % idx) - self.assertEqual(exp_sample_type, sample_type, "Sample type for sample at index %d does not match" % idx) - self.assertEqual(exp_progress_percent, progress_percent, "Current progress for sample at index %d does not match" % idx) - self.assertIsNotNone(runner, "runner must be defined") - self.assertEqual(exp_params, params, "Parameters do not match") - idx += 1 - # for infinite schedules we only check the first few elements - if infinite_schedule and idx == len(expected_schedule): - break - - def op(name, operation_type): return track.Operation(name, operation_type, param_source="driver-test-param-source") @@ -448,7 +432,7 @@ def calculate_global_throughput(self, samples): return driver.ThroughputCalculator().calculate(samples) -class SchedulerTests(ScheduleTestCase): +class SchedulerTests(TestCase): class RunnerWithProgress: def __init__(self, complete_after=3): self.completed = False @@ -456,7 +440,7 @@ def __init__(self, complete_after=3): self.calls = 0 self.complete_after = complete_after - def __call__(self, *args, **kwargs): + async def __call__(self, *args, **kwargs): self.calls += 1 if not self.completed: self.percent_completed = self.calls / self.complete_after @@ -464,21 +448,37 @@ def __call__(self, *args, **kwargs): else: self.percent_completed = 1.0 + async def assert_schedule(self, expected_schedule, schedule, infinite_schedule=False): + idx = 0 + async for invocation_time, sample_type, progress_percent, runner, params in schedule: + exp_invocation_time, exp_sample_type, exp_progress_percent, exp_params = expected_schedule[idx] + self.assertAlmostEqual(exp_invocation_time, invocation_time, msg="Invocation time for sample at index %d does not match" % idx) + self.assertEqual(exp_sample_type, sample_type, "Sample type for sample at index %d does not match" % idx) + self.assertEqual(exp_progress_percent, progress_percent, "Current progress for sample at index %d does not match" % idx) + self.assertIsNotNone(runner, "runner must be defined") + self.assertEqual(exp_params, params, "Parameters do not match") + idx += 1 + # for infinite schedules we only check the first few elements + if infinite_schedule and idx == len(expected_schedule): + break + if not infinite_schedule: + self.assertEqual(len(expected_schedule), idx, msg="Number of elements in the schedules do not match") + def setUp(self): self.test_track = track.Track(name="unittest") self.runner_with_progress = SchedulerTests.RunnerWithProgress() params.register_param_source_for_name("driver-test-param-source", DriverTestParamSource) runner.register_default_runners() - runner.register_runner("driver-test-runner-with-completion", self.runner_with_progress) + runner.register_runner("driver-test-runner-with-completion", self.runner_with_progress, async_runner=True) def tearDown(self): runner.remove_runner("driver-test-runner-with-completion") - def test_search_task_one_client(self): + @run_async + async def test_search_task_one_client(self): task = track.Task("search", track.Operation("search", track.OperationType.Search.name, param_source="driver-test-param-source"), warmup_iterations=3, iterations=5, clients=1, params={"target-throughput": 10, "clients": 1}) - schedule_handle = driver.schedule_for(self.test_track, task, 0) - schedule = schedule_handle() + schedule = driver.schedule_for(self.test_track, task, 0) expected_schedule = [ (0, metrics.SampleType.Warmup, 1 / 8, {}), @@ -490,13 +490,13 @@ def test_search_task_one_client(self): (0.6, metrics.SampleType.Normal, 7 / 8, {}), (0.7, metrics.SampleType.Normal, 8 / 8, {}), ] - self.assert_schedule(expected_schedule, list(schedule)) + await self.assert_schedule(expected_schedule, schedule()) - def test_search_task_two_clients(self): + @run_async + async def test_search_task_two_clients(self): task = track.Task("search", track.Operation("search", track.OperationType.Search.name, param_source="driver-test-param-source"), warmup_iterations=1, iterations=5, clients=2, params={"target-throughput": 10, "clients": 2}) - schedule_handle = driver.schedule_for(self.test_track, task, 0) - schedule = schedule_handle() + schedule = driver.schedule_for(self.test_track, task, 0) expected_schedule = [ (0, metrics.SampleType.Warmup, 1 / 6, {}), @@ -506,61 +506,61 @@ def test_search_task_two_clients(self): (0.8, metrics.SampleType.Normal, 5 / 6, {}), (1.0, metrics.SampleType.Normal, 6 / 6, {}), ] - self.assert_schedule(expected_schedule, list(schedule)) + await self.assert_schedule(expected_schedule, schedule()) - def test_schedule_param_source_determines_iterations_no_warmup(self): + @run_async + async def test_schedule_param_source_determines_iterations_no_warmup(self): # we neither define any time-period nor any iteration count on the task. task = track.Task("bulk-index", track.Operation("bulk-index", track.OperationType.Bulk.name, params={"body": ["a"], "size": 3}, param_source="driver-test-param-source"), clients=1, params={"target-throughput": 4, "clients": 4}) - schedule_handle = driver.schedule_for(self.test_track, task, 0) - schedule = schedule_handle() + schedule = driver.schedule_for(self.test_track, task, 0) - self.assert_schedule([ + await self.assert_schedule([ (0.0, metrics.SampleType.Normal, 1 / 3, {"body": ["a"], "size": 3}), (1.0, metrics.SampleType.Normal, 2 / 3, {"body": ["a"], "size": 3}), (2.0, metrics.SampleType.Normal, 3 / 3, {"body": ["a"], "size": 3}), - ], list(schedule)) + ], schedule()) - def test_schedule_param_source_determines_iterations_including_warmup(self): + @run_async + async def test_schedule_param_source_determines_iterations_including_warmup(self): task = track.Task("bulk-index", track.Operation("bulk-index", track.OperationType.Bulk.name, params={"body": ["a"], "size": 5}, param_source="driver-test-param-source"), warmup_iterations=2, clients=1, params={"target-throughput": 4, "clients": 4}) - schedule_handle = driver.schedule_for(self.test_track, task, 0) - schedule = schedule_handle() + schedule = driver.schedule_for(self.test_track, task, 0) - self.assert_schedule([ + await self.assert_schedule([ (0.0, metrics.SampleType.Warmup, 1 / 5, {"body": ["a"], "size": 5}), (1.0, metrics.SampleType.Warmup, 2 / 5, {"body": ["a"], "size": 5}), (2.0, metrics.SampleType.Normal, 3 / 5, {"body": ["a"], "size": 5}), (3.0, metrics.SampleType.Normal, 4 / 5, {"body": ["a"], "size": 5}), (4.0, metrics.SampleType.Normal, 5 / 5, {"body": ["a"], "size": 5}), - ], list(schedule)) + ], schedule()) - def test_schedule_defaults_to_iteration_based(self): + @run_async + async def test_schedule_defaults_to_iteration_based(self): # no time-period and no iterations specified on the task. Also, the parameter source does not define a size. task = track.Task("bulk-index", track.Operation("bulk-index", track.OperationType.Bulk.name, params={"body": ["a"]}, param_source="driver-test-param-source"), clients=1, params={"target-throughput": 4, "clients": 4}) - schedule_handle = driver.schedule_for(self.test_track, task, 0) - schedule = schedule_handle() + schedule = driver.schedule_for(self.test_track, task, 0) - self.assert_schedule([ + await self.assert_schedule([ (0.0, metrics.SampleType.Normal, 1 / 1, {"body": ["a"]}), - ], list(schedule)) + ], schedule()) - def test_schedule_for_warmup_time_based(self): + @run_async + async def test_schedule_for_warmup_time_based(self): task = track.Task("time-based", track.Operation("time-based", track.OperationType.Bulk.name, params={"body": ["a"], "size": 11}, param_source="driver-test-param-source"), warmup_time_period=0, clients=4, params={"target-throughput": 4, "clients": 4}) - schedule_handle = driver.schedule_for(self.test_track, task, 0) - schedule = schedule_handle() + schedule = driver.schedule_for(self.test_track, task, 0) - self.assert_schedule([ + await self.assert_schedule([ (0.0, metrics.SampleType.Normal, 1 / 11, {"body": ["a"], "size": 11}), (1.0, metrics.SampleType.Normal, 2 / 11, {"body": ["a"], "size": 11}), (2.0, metrics.SampleType.Normal, 3 / 11, {"body": ["a"], "size": 11}), @@ -572,71 +572,70 @@ def test_schedule_for_warmup_time_based(self): (8.0, metrics.SampleType.Normal, 9 / 11, {"body": ["a"], "size": 11}), (9.0, metrics.SampleType.Normal, 10 / 11, {"body": ["a"], "size": 11}), (10.0, metrics.SampleType.Normal, 11 / 11, {"body": ["a"], "size": 11}), - ], list(schedule)) + ], schedule()) - def test_infinite_schedule_without_progress_indication(self): + @run_async + async def test_infinite_schedule_without_progress_indication(self): task = track.Task("time-based", track.Operation("time-based", track.OperationType.Bulk.name, params={"body": ["a"]}, param_source="driver-test-param-source"), warmup_time_period=0, clients=4, params={"target-throughput": 4, "clients": 4}) - schedule_handle = driver.schedule_for(self.test_track, task, 0) - schedule = schedule_handle() + schedule = driver.schedule_for(self.test_track, task, 0) - self.assert_schedule([ + await self.assert_schedule([ (0.0, metrics.SampleType.Normal, None, {"body": ["a"]}), (1.0, metrics.SampleType.Normal, None, {"body": ["a"]}), (2.0, metrics.SampleType.Normal, None, {"body": ["a"]}), (3.0, metrics.SampleType.Normal, None, {"body": ["a"]}), (4.0, metrics.SampleType.Normal, None, {"body": ["a"]}), - ], schedule, infinite_schedule=True) + ], schedule(), infinite_schedule=True) - def test_finite_schedule_with_progress_indication(self): + @run_async + async def test_finite_schedule_with_progress_indication(self): task = track.Task("time-based", track.Operation("time-based", track.OperationType.Bulk.name, params={"body": ["a"], "size": 5}, param_source="driver-test-param-source"), warmup_time_period=0, clients=4, params={"target-throughput": 4, "clients": 4}) - schedule_handle = driver.schedule_for(self.test_track, task, 0) - schedule = schedule_handle() + schedule = driver.schedule_for(self.test_track, task, 0) - self.assert_schedule([ + await self.assert_schedule([ (0.0, metrics.SampleType.Normal, 1 / 5, {"body": ["a"], "size": 5}), (1.0, metrics.SampleType.Normal, 2 / 5, {"body": ["a"], "size": 5}), (2.0, metrics.SampleType.Normal, 3 / 5, {"body": ["a"], "size": 5}), (3.0, metrics.SampleType.Normal, 4 / 5, {"body": ["a"], "size": 5}), (4.0, metrics.SampleType.Normal, 5 / 5, {"body": ["a"], "size": 5}), - ], list(schedule), infinite_schedule=False) + ], schedule(), infinite_schedule=False) - def test_schedule_with_progress_determined_by_runner(self): + @run_async + async def test_schedule_with_progress_determined_by_runner(self): task = track.Task("time-based", track.Operation("time-based", "driver-test-runner-with-completion", params={"body": ["a"]}, param_source="driver-test-param-source"), clients=1, params={"target-throughput": 1, "clients": 1}) - schedule_handle = driver.schedule_for(self.test_track, task, 0) - schedule = schedule_handle() + schedule = driver.schedule_for(self.test_track, task, 0) - self.assert_schedule([ + await self.assert_schedule([ (0.0, metrics.SampleType.Normal, None, {"body": ["a"]}), (1.0, metrics.SampleType.Normal, None, {"body": ["a"]}), (2.0, metrics.SampleType.Normal, None, {"body": ["a"]}), (3.0, metrics.SampleType.Normal, None, {"body": ["a"]}), (4.0, metrics.SampleType.Normal, None, {"body": ["a"]}), - ], schedule, infinite_schedule=True) + ], schedule(), infinite_schedule=True) - def test_schedule_for_time_based(self): + @run_async + async def test_schedule_for_time_based(self): task = track.Task("time-based", track.Operation("time-based", track.OperationType.Bulk.name, params={"body": ["a"], "size": 11}, param_source="driver-test-param-source"), warmup_time_period=0.1, time_period=0.1, clients=1) schedule_handle = driver.schedule_for(self.test_track, task, 0) - schedule = list(schedule_handle()) - - self.assertTrue(len(schedule) > 0) + schedule = schedule_handle() last_progress = -1 - for invocation_time, sample_type, progress_percent, runner, params in schedule: + async for invocation_time, sample_type, progress_percent, runner, params in schedule: # we're not throughput throttled self.assertEqual(0, invocation_time) if progress_percent <= 0.5: @@ -651,18 +650,18 @@ def test_schedule_for_time_based(self): self.assertEqual({"body": ["a"], "size": 11}, params) -class ExecutorTests(TestCase): +class AsyncExecutorTests(TestCase): class NoopContextManager: def __init__(self, mock): self.mock = mock - def __enter__(self): + async def __aenter__(self): return self - def __call__(self, *args): - return self.mock(*args) + async def __call__(self, *args): + return await self.mock(*args) - def __exit__(self, exc_type, exc_val, exc_tb): + async def __aexit__(self, exc_type, exc_val, exc_tb): return False def __str__(self): @@ -681,7 +680,7 @@ def completed(self): def percent_completed(self): return (self.iterations - self.iterations_left) / self.iterations - def __call__(self, es, params): + async def __call__(self, es, params): self.iterations_left -= 1 def __init__(self, methodName): @@ -689,18 +688,19 @@ def __init__(self, methodName): self.runner_with_progress = None def context_managed(self, mock): - return ExecutorTests.NoopContextManager(mock) + return AsyncExecutorTests.NoopContextManager(mock) def setUp(self): runner.register_default_runners() - self.runner_with_progress = ExecutorTests.RunnerWithProgress() - runner.register_runner("unit-test-recovery", self.runner_with_progress) + self.runner_with_progress = AsyncExecutorTests.RunnerWithProgress() + runner.register_runner("unit-test-recovery", self.runner_with_progress, async_runner=True) @mock.patch("elasticsearch.Elasticsearch") - def test_execute_schedule_in_throughput_mode(self, es): - es.bulk.return_value = { + @run_async + async def test_execute_schedule_in_throughput_mode(self, es): + es.bulk.return_value = as_future({ "errors": False - } + }) params.register_param_source_for_name("driver-test-param-source", DriverTestParamSource) test_track = track.Track(name="unittest", description="unittest track", @@ -722,14 +722,16 @@ def test_execute_schedule_in_throughput_mode(self, es): cancel = threading.Event() complete = threading.Event() - execute_schedule = driver.Executor(client_id=2, - task=task, - schedule=schedule, - es=es, - sampler=sampler, - cancel=cancel, - complete=complete) - execute_schedule() + execute_schedule = driver.AsyncExecutor(client_id=2, + task=task, + schedule=schedule, + es={ + "default": es + }, + sampler=sampler, + cancel=cancel, + complete=complete) + await execute_schedule() samples = sampler.samples @@ -753,10 +755,11 @@ def test_execute_schedule_in_throughput_mode(self, es): self.assertEqual(1, sample.request_meta_data["bulk-size"]) @mock.patch("elasticsearch.Elasticsearch") - def test_execute_schedule_with_progress_determined_by_runner(self, es): - es.bulk.return_value = { + @run_async + async def test_execute_schedule_with_progress_determined_by_runner(self, es): + es.bulk.return_value = as_future({ "errors": False - } + }) params.register_param_source_for_name("driver-test-param-source", DriverTestParamSource) test_track = track.Track(name="unittest", description="unittest track", @@ -774,14 +777,16 @@ def test_execute_schedule_with_progress_determined_by_runner(self, es): cancel = threading.Event() complete = threading.Event() - execute_schedule = driver.Executor(client_id=2, - task=task, - schedule=schedule, - es=es, - sampler=sampler, - cancel=cancel, - complete=complete) - execute_schedule() + execute_schedule = driver.AsyncExecutor(client_id=2, + task=task, + schedule=schedule, + es={ + "default": es + }, + sampler=sampler, + cancel=cancel, + complete=complete) + await execute_schedule() samples = sampler.samples @@ -806,10 +811,11 @@ def test_execute_schedule_with_progress_determined_by_runner(self, es): self.assertEqual("ops", sample.total_ops_unit) @mock.patch("elasticsearch.Elasticsearch") - def test_execute_schedule_throughput_throttled(self, es): - es.bulk.return_value = { + @run_async + async def test_execute_schedule_throughput_throttled(self, es): + es.bulk.return_value = as_future({ "errors": False - } + }) params.register_param_source_for_name("driver-test-param-source", DriverTestParamSource) test_track = track.Track(name="unittest", description="unittest track", @@ -833,14 +839,16 @@ def test_execute_schedule_throughput_throttled(self, es): complete = threading.Event() schedule = driver.schedule_for(test_track, task, 0) - execute_schedule = driver.Executor(client_id=0, - task=task, - schedule=schedule, - es=es, - sampler=sampler, - cancel=cancel, - complete=complete) - execute_schedule() + execute_schedule = driver.AsyncExecutor(client_id=0, + task=task, + schedule=schedule, + es={ + "default": es + }, + sampler=sampler, + cancel=cancel, + complete=complete) + await execute_schedule() samples = sampler.samples @@ -852,10 +860,11 @@ def test_execute_schedule_throughput_throttled(self, es): self.assertTrue(complete.is_set(), "Executor should auto-complete a task that terminates its parent") @mock.patch("elasticsearch.Elasticsearch") - def test_cancel_execute_schedule(self, es): - es.bulk.return_value = { + @run_async + async def test_cancel_execute_schedule(self, es): + es.bulk.return_value = as_future({ "errors": False - } + }) params.register_param_source_for_name("driver-test-param-source", DriverTestParamSource) test_track = track.Track(name="unittest", description="unittest track", @@ -877,16 +886,18 @@ def test_cancel_execute_schedule(self, es): cancel = threading.Event() complete = threading.Event() - execute_schedule = driver.Executor(client_id=0, - task=task, - schedule=schedule, - es=es, - sampler=sampler, - cancel=cancel, - complete=complete) + execute_schedule = driver.AsyncExecutor(client_id=0, + task=task, + schedule=schedule, + es={ + "default": es + }, + sampler=sampler, + cancel=cancel, + complete=complete) cancel.set() - execute_schedule() + await execute_schedule() samples = sampler.samples @@ -894,15 +905,18 @@ def test_cancel_execute_schedule(self, es): self.assertEqual(0, sample_size) @mock.patch("elasticsearch.Elasticsearch") - def test_execute_schedule_aborts_on_error(self, es): + @run_async + async def test_execute_schedule_aborts_on_error(self, es): class ExpectedUnitTestException(Exception): pass def run(*args, **kwargs): raise ExpectedUnitTestException() - def schedule_handle(): - return [(0, metrics.SampleType.Warmup, 0, self.context_managed(run), None)] + async def schedule_handle(): + invocations = [(0, metrics.SampleType.Warmup, 0, self.context_managed(run), None)] + for invocation in invocations: + yield invocation task = track.Task("no-op", track.Operation("no-op", track.OperationType.Bulk.name, params={}, param_source="driver-test-param-source"), @@ -912,74 +926,81 @@ def schedule_handle(): sampler = driver.Sampler(start_timestamp=0) cancel = threading.Event() complete = threading.Event() - execute_schedule = driver.Executor(client_id=2, - task=task, - schedule=schedule_handle, - es=es, - sampler=sampler, - cancel=cancel, - complete=complete) + execute_schedule = driver.AsyncExecutor(client_id=2, + task=task, + schedule=schedule_handle, + es={ + "default": es + }, + sampler=sampler, + cancel=cancel, + complete=complete) with self.assertRaises(ExpectedUnitTestException): - execute_schedule() + await execute_schedule() self.assertEqual(0, es.call_count) - def test_execute_single_no_return_value(self): + @run_async + async def test_execute_single_no_return_value(self): es = None params = None runner = mock.Mock() + runner.return_value = as_future() - total_ops, total_ops_unit, request_meta_data = driver.execute_single(self.context_managed(runner), es, params) + ops, unit, request_meta_data = await driver.execute_single(self.context_managed(runner), es, params) - self.assertEqual(1, total_ops) - self.assertEqual("ops", total_ops_unit) + self.assertEqual(1, ops) + self.assertEqual("ops", unit) self.assertEqual({"success": True}, request_meta_data) - def test_execute_single_tuple(self): + @run_async + async def test_execute_single_tuple(self): es = None params = None runner = mock.Mock() - runner.return_value = (500, "MB") + runner.return_value = as_future(result=(500, "MB")) - total_ops, total_ops_unit, request_meta_data = driver.execute_single(self.context_managed(runner), es, params) + ops, unit, request_meta_data = await driver.execute_single(self.context_managed(runner), es, params) - self.assertEqual(500, total_ops) - self.assertEqual("MB", total_ops_unit) + self.assertEqual(500, ops) + self.assertEqual("MB", unit) self.assertEqual({"success": True}, request_meta_data) - def test_execute_single_dict(self): + @run_async + async def test_execute_single_dict(self): es = None params = None runner = mock.Mock() - runner.return_value = { + runner.return_value = as_future({ "weight": 50, "unit": "docs", "some-custom-meta-data": "valid", "http-status": 200 - } + }) - total_ops, total_ops_unit, request_meta_data = driver.execute_single(self.context_managed(runner), es, params) + ops, unit, request_meta_data = await driver.execute_single(self.context_managed(runner), es, params) - self.assertEqual(50, total_ops) - self.assertEqual("docs", total_ops_unit) + self.assertEqual(50, ops) + self.assertEqual("docs", unit) self.assertEqual({ "some-custom-meta-data": "valid", "http-status": 200, "success": True }, request_meta_data) - def test_execute_single_with_connection_error(self): + @run_async + async def test_execute_single_with_connection_error(self): import elasticsearch es = None params = None # ES client uses pseudo-status "N/A" in this case... - runner = mock.Mock(side_effect=elasticsearch.ConnectionError("N/A", "no route to host", None)) + runner = mock.Mock(side_effect=as_future(exception=elasticsearch.ConnectionError("N/A", "no route to host", None))) - total_ops, total_ops_unit, request_meta_data = driver.execute_single(self.context_managed(runner), es, params) + ops, unit, request_meta_data = await driver.execute_single(self.context_managed(runner), es, params) - self.assertEqual(0, total_ops) - self.assertEqual("ops", total_ops_unit) + self.assertEqual(0, ops) + self.assertEqual("ops", unit) self.assertEqual({ # Look ma: No http-status! "error-description": "no route to host", @@ -987,16 +1008,18 @@ def test_execute_single_with_connection_error(self): "success": False }, request_meta_data) - def test_execute_single_with_http_400(self): + @run_async + async def test_execute_single_with_http_400(self): import elasticsearch es = None params = None - runner = mock.Mock(side_effect=elasticsearch.NotFoundError(404, "not found", "the requested document could not be found")) + runner = mock.Mock(side_effect= + as_future(exception=elasticsearch.NotFoundError(404, "not found", "the requested document could not be found"))) - total_ops, total_ops_unit, request_meta_data = driver.execute_single(self.context_managed(runner), es, params) + ops, unit, request_meta_data = await driver.execute_single(self.context_managed(runner), es, params) - self.assertEqual(0, total_ops) - self.assertEqual("ops", total_ops_unit) + self.assertEqual(0, ops) + self.assertEqual("ops", unit) self.assertEqual({ "http-status": 404, "error-type": "transport", @@ -1004,9 +1027,10 @@ def test_execute_single_with_http_400(self): "success": False }, request_meta_data) - def test_execute_single_with_key_error(self): + @run_async + async def test_execute_single_with_key_error(self): class FailingRunner: - def __call__(self, *args): + async def __call__(self, *args): raise KeyError("bulk-size missing") def __str__(self): @@ -1020,24 +1044,25 @@ def __str__(self): runner = FailingRunner() with self.assertRaises(exceptions.SystemSetupError) as ctx: - driver.execute_single(self.context_managed(runner), es, params) + await driver.execute_single(self.context_managed(runner), es, params) self.assertEqual( "Cannot execute [failing_mock_runner]. Provided parameters are: ['bulk', 'mode']. Error: ['bulk-size missing'].", ctx.exception.args[0]) -class ProfilerTests(TestCase): - def test_profiler_is_a_transparent_wrapper(self): +class AsyncProfilerTests(TestCase): + @run_async + async def test_profiler_is_a_transparent_wrapper(self): import time - def f(x): - time.sleep(x) + async def f(x): + await asyncio.sleep(x) return x * 2 - profiler = driver.Profiler(f, 0, "sleep-operation") + profiler = driver.AsyncProfiler(f) start = time.perf_counter() # this should take roughly 1 second and should return something - return_value = profiler(1) + return_value = await profiler(1) end = time.perf_counter() self.assertEqual(2, return_value) duration = end - start diff --git a/tests/driver/runner_test.py b/tests/driver/runner_test.py index 9e603f925..fd89588fc 100644 --- a/tests/driver/runner_test.py +++ b/tests/driver/runner_test.py @@ -21,18 +21,18 @@ from unittest import TestCase import elasticsearch -import pytest from esrally import exceptions from esrally.driver import runner +from tests import run_async, as_future class BaseUnitTestContextManagerRunner: - def __enter__(self): + async def __aenter__(self): self.fp = io.StringIO("many\nlines\nin\na\nfile") return self - def __exit__(self, exc_type, exc_val, exc_tb): + async def __aexit__(self, exc_type, exc_val, exc_tb): self.fp.close() return False @@ -41,48 +41,53 @@ class RegisterRunnerTests(TestCase): def tearDown(self): runner.remove_runner("unit_test") - def test_runner_function_should_be_wrapped(self): - def runner_function(*args): + @run_async + async def test_runner_function_should_be_wrapped(self): + async def runner_function(*args): return args - runner.register_runner(operation_type="unit_test", runner=runner_function) + runner.register_runner(operation_type="unit_test", runner=runner_function, async_runner=True) returned_runner = runner.runner_for("unit_test") self.assertIsInstance(returned_runner, runner.NoCompletion) self.assertEqual("user-defined runner for [runner_function]", repr(returned_runner)) - self.assertEqual(("default_client", "param"), returned_runner({"default": "default_client", "other": "other_client"}, "param")) + self.assertEqual(("default_client", "param"), + await returned_runner({"default": "default_client", "other": "other_client"}, "param")) - def test_single_cluster_runner_class_with_context_manager_should_be_wrapped_with_context_manager_enabled(self): + @run_async + async def test_single_cluster_runner_class_with_context_manager_should_be_wrapped_with_context_manager_enabled(self): class UnitTestSingleClusterContextManagerRunner(BaseUnitTestContextManagerRunner): - def __call__(self, *args): + async def __call__(self, *args): return args def __str__(self): return "UnitTestSingleClusterContextManagerRunner" test_runner = UnitTestSingleClusterContextManagerRunner() - runner.register_runner(operation_type="unit_test", runner=test_runner) + runner.register_runner(operation_type="unit_test", runner=test_runner, async_runner=True) returned_runner = runner.runner_for("unit_test") self.assertIsInstance(returned_runner, runner.NoCompletion) self.assertEqual("user-defined context-manager enabled runner for [UnitTestSingleClusterContextManagerRunner]", repr(returned_runner)) # test that context_manager functionality gets preserved after wrapping - with returned_runner: - self.assertEqual(("default_client", "param"), returned_runner({"default": "default_client", "other": "other_client"}, "param")) + async with returned_runner: + self.assertEqual(("default_client", "param"), + await returned_runner({"default": "default_client", "other": "other_client"}, "param")) # check that the context manager interface of our inner runner has been respected. self.assertTrue(test_runner.fp.closed) - def test_multi_cluster_runner_class_with_context_manager_should_be_wrapped_with_context_manager_enabled(self): + @run_async + async def test_multi_cluster_runner_class_with_context_manager_should_be_wrapped_with_context_manager_enabled(self): class UnitTestMultiClusterContextManagerRunner(BaseUnitTestContextManagerRunner): multi_cluster = True - def __call__(self, *args): + async def __call__(self, *args): return args def __str__(self): return "UnitTestMultiClusterContextManagerRunner" test_runner = UnitTestMultiClusterContextManagerRunner() - runner.register_runner(operation_type="unit_test", runner=test_runner) + runner.register_runner(operation_type="unit_test", runner=test_runner, async_runner=True) returned_runner = runner.runner_for("unit_test") self.assertIsInstance(returned_runner, runner.NoCompletion) self.assertEqual("user-defined context-manager enabled runner for [UnitTestMultiClusterContextManagerRunner]", @@ -90,48 +95,52 @@ def __str__(self): # test that context_manager functionality gets preserved after wrapping all_clients = {"default": "default_client", "other": "other_client"} - with returned_runner: - self.assertEqual((all_clients, "param1", "param2"), returned_runner(all_clients, "param1", "param2")) + async with returned_runner: + self.assertEqual((all_clients, "param1", "param2"), await returned_runner(all_clients, "param1", "param2")) # check that the context manager interface of our inner runner has been respected. self.assertTrue(test_runner.fp.closed) - def test_single_cluster_runner_class_should_be_wrapped(self): + @run_async + async def test_single_cluster_runner_class_should_be_wrapped(self): class UnitTestSingleClusterRunner: - def __call__(self, *args): + async def __call__(self, *args): return args def __str__(self): return "UnitTestSingleClusterRunner" test_runner = UnitTestSingleClusterRunner() - runner.register_runner(operation_type="unit_test", runner=test_runner) + runner.register_runner(operation_type="unit_test", runner=test_runner, async_runner=True) returned_runner = runner.runner_for("unit_test") self.assertIsInstance(returned_runner, runner.NoCompletion) self.assertEqual("user-defined runner for [UnitTestSingleClusterRunner]", repr(returned_runner)) - self.assertEqual(("default_client", "param"), returned_runner({"default": "default_client", "other": "other_client"}, "param")) + self.assertEqual(("default_client", "param"), + await returned_runner({"default": "default_client", "other": "other_client"}, "param")) - def test_multi_cluster_runner_class_should_be_wrapped(self): + @run_async + async def test_multi_cluster_runner_class_should_be_wrapped(self): class UnitTestMultiClusterRunner: multi_cluster = True - def __call__(self, *args): + async def __call__(self, *args): return args def __str__(self): return "UnitTestMultiClusterRunner" test_runner = UnitTestMultiClusterRunner() - runner.register_runner(operation_type="unit_test", runner=test_runner) + runner.register_runner(operation_type="unit_test", runner=test_runner, async_runner=True) returned_runner = runner.runner_for("unit_test") self.assertIsInstance(returned_runner, runner.NoCompletion) self.assertEqual("user-defined runner for [UnitTestMultiClusterRunner]", repr(returned_runner)) all_clients = {"default": "default_client", "other": "other_client"} - self.assertEqual((all_clients, "some_param"), returned_runner(all_clients, "some_param")) + self.assertEqual((all_clients, "some_param"), await returned_runner(all_clients, "some_param")) class BulkIndexRunnerTests(TestCase): @mock.patch("elasticsearch.Elasticsearch") - def test_bulk_index_missing_params(self, es): + @run_async + async def test_bulk_index_missing_params(self, es): es.bulk.return_value = { "errors": False } @@ -147,15 +156,16 @@ def test_bulk_index_missing_params(self, es): } with self.assertRaises(exceptions.DataError) as ctx: - bulk(es, bulk_params) + await bulk(es, bulk_params) self.assertEqual("Parameter source for operation 'bulk-index' did not provide the mandatory parameter 'action-metadata-present'. " "Please add it to your parameter source.", ctx.exception.args[0]) @mock.patch("elasticsearch.Elasticsearch") - def test_bulk_index_success_with_metadata(self, es): - es.bulk.return_value = { + @run_async + async def test_bulk_index_success_with_metadata(self, es): + es.bulk.return_value = as_future({ "errors": False - } + }) bulk = runner.BulkIndex() bulk_params = { @@ -169,7 +179,7 @@ def test_bulk_index_success_with_metadata(self, es): "bulk-size": 3 } - result = bulk(es, bulk_params) + result = await bulk(es, bulk_params) self.assertIsNone(result["took"]) self.assertIsNone(result["index"]) @@ -183,10 +193,11 @@ def test_bulk_index_success_with_metadata(self, es): es.bulk.assert_called_with(body=bulk_params["body"], params={}) @mock.patch("elasticsearch.Elasticsearch") - def test_bulk_index_success_without_metadata_with_doc_type(self, es): - es.bulk.return_value = { + @run_async + async def test_bulk_index_success_without_metadata_with_doc_type(self, es): + es.bulk.return_value = as_future({ "errors": False - } + }) bulk = runner.BulkIndex() bulk_params = { @@ -199,7 +210,7 @@ def test_bulk_index_success_without_metadata_with_doc_type(self, es): "type": "_doc" } - result = bulk(es, bulk_params) + result = await bulk(es, bulk_params) self.assertIsNone(result["took"]) self.assertEqual("test-index", result["index"]) @@ -213,10 +224,11 @@ def test_bulk_index_success_without_metadata_with_doc_type(self, es): es.bulk.assert_called_with(body=bulk_params["body"], index="test-index", doc_type="_doc", params={}) @mock.patch("elasticsearch.Elasticsearch") - def test_bulk_index_success_without_metadata_and_without_doc_type(self, es): - es.bulk.return_value = { + @run_async + async def test_bulk_index_success_without_metadata_and_without_doc_type(self, es): + es.bulk.return_value = as_future({ "errors": False - } + }) bulk = runner.BulkIndex() bulk_params = { @@ -228,7 +240,7 @@ def test_bulk_index_success_without_metadata_and_without_doc_type(self, es): "index": "test-index" } - result = bulk(es, bulk_params) + result = await bulk(es, bulk_params) self.assertIsNone(result["took"]) self.assertEqual("test-index", result["index"]) @@ -242,8 +254,9 @@ def test_bulk_index_success_without_metadata_and_without_doc_type(self, es): es.bulk.assert_called_with(body=bulk_params["body"], index="test-index", doc_type=None, params={}) @mock.patch("elasticsearch.Elasticsearch") - def test_bulk_index_error(self, es): - es.bulk.return_value = { + @run_async + async def test_bulk_index_error(self, es): + es.bulk.return_value = as_future({ "took": 5, "errors": True, "items": [ @@ -278,7 +291,7 @@ def test_bulk_index_error(self, es): } }, ] - } + }) bulk = runner.BulkIndex() bulk_params = { @@ -293,7 +306,7 @@ def test_bulk_index_error(self, es): "index": "test" } - result = bulk(es, bulk_params) + result = await bulk(es, bulk_params) self.assertEqual("test", result["index"]) self.assertEqual(5, result["took"]) @@ -307,8 +320,9 @@ def test_bulk_index_error(self, es): es.bulk.assert_called_with(body=bulk_params["body"], params={}) @mock.patch("elasticsearch.Elasticsearch") - def test_bulk_index_error_no_shards(self, es): - es.bulk.return_value = { + @run_async + async def test_bulk_index_error_no_shards(self, es): + es.bulk.return_value = as_future({ "took": 20, "errors": True, "items": [ @@ -340,7 +354,7 @@ def test_bulk_index_error_no_shards(self, es): } } ] - } + }) bulk = runner.BulkIndex() bulk_params = { @@ -356,7 +370,7 @@ def test_bulk_index_error_no_shards(self, es): "index": "test" } - result = bulk(es, bulk_params) + result = await bulk(es, bulk_params) self.assertEqual("test", result["index"]) self.assertEqual(20, result["took"]) @@ -370,8 +384,9 @@ def test_bulk_index_error_no_shards(self, es): es.bulk.assert_called_with(body=bulk_params["body"], params={}) @mock.patch("elasticsearch.Elasticsearch") - def test_mixed_bulk_with_simple_stats(self, es): - es.bulk.return_value = { + @run_async + async def test_mixed_bulk_with_simple_stats(self, es): + es.bulk.return_value = as_future({ "took": 30, "ingest_took": 20, "errors": True, @@ -443,7 +458,7 @@ def test_mixed_bulk_with_simple_stats(self, es): } } ] - } + }) bulk = runner.BulkIndex() bulk_params = { @@ -461,7 +476,7 @@ def test_mixed_bulk_with_simple_stats(self, es): "index": "test" } - result = bulk(es, bulk_params) + result = await bulk(es, bulk_params) self.assertEqual("test", result["index"]) self.assertEqual(30, result["took"]) @@ -475,13 +490,14 @@ def test_mixed_bulk_with_simple_stats(self, es): es.bulk.assert_called_with(body=bulk_params["body"], params={}) - es.bulk.return_value.pop("ingest_took") - result = bulk(es, bulk_params) + es.bulk.return_value.result().pop("ingest_took") + result = await bulk(es, bulk_params) self.assertNotIn("ingest_took", result) @mock.patch("elasticsearch.Elasticsearch") - def test_mixed_bulk_with_detailed_stats_body_as_string(self, es): - es.bulk.return_value = { + @run_async + async def test_mixed_bulk_with_detailed_stats_body_as_string(self, es): + es.bulk.return_value = as_future({ "took": 30, "ingest_took": 20, "errors": True, @@ -587,7 +603,7 @@ def test_mixed_bulk_with_detailed_stats_body_as_string(self, es): } } ] - } + }) bulk = runner.BulkIndex() bulk_params = { @@ -609,7 +625,7 @@ def test_mixed_bulk_with_detailed_stats_body_as_string(self, es): "index": "test" } - result = bulk(es, bulk_params) + result = await bulk(es, bulk_params) self.assertEqual("test", result["index"]) self.assertEqual(30, result["took"]) @@ -666,13 +682,14 @@ def test_mixed_bulk_with_detailed_stats_body_as_string(self, es): es.bulk.assert_called_with(body=bulk_params["body"], params={}) - es.bulk.return_value.pop("ingest_took") - result = bulk(es, bulk_params) + es.bulk.return_value.result().pop("ingest_took") + result = await bulk(es, bulk_params) self.assertNotIn("ingest_took", result) @mock.patch("elasticsearch.Elasticsearch") - def test_simple_bulk_with_detailed_stats_body_as_list(self, es): - es.bulk.return_value = { + @run_async + async def test_simple_bulk_with_detailed_stats_body_as_list(self, es): + es.bulk.return_value = as_future({ "took": 30, "ingest_took": 20, "errors": False, @@ -695,7 +712,7 @@ def test_simple_bulk_with_detailed_stats_body_as_list(self, es): } } ] - } + }) bulk = runner.BulkIndex() bulk_params = { @@ -707,7 +724,7 @@ def test_simple_bulk_with_detailed_stats_body_as_list(self, es): "index": "test" } - result = bulk(es, bulk_params) + result = await bulk(es, bulk_params) self.assertEqual("test", result["index"]) self.assertEqual(30, result["took"]) @@ -740,13 +757,14 @@ def test_simple_bulk_with_detailed_stats_body_as_list(self, es): es.bulk.assert_called_with(body=bulk_params["body"], params={}) - es.bulk.return_value.pop("ingest_took") - result = bulk(es, bulk_params) + es.bulk.return_value.result().pop("ingest_took") + result = await bulk(es, bulk_params) self.assertNotIn("ingest_took", result) @mock.patch("elasticsearch.Elasticsearch") - def test_simple_bulk_with_detailed_stats_body_as_unrecognized_type(self, es): - es.bulk.return_value = { + @run_async + async def test_simple_bulk_with_detailed_stats_body_as_unrecognized_type(self, es): + es.bulk.return_value = as_future({ "took": 30, "ingest_took": 20, "errors": False, @@ -769,7 +787,7 @@ def test_simple_bulk_with_detailed_stats_body_as_unrecognized_type(self, es): } } ] - } + }) bulk = runner.BulkIndex() bulk_params = { @@ -784,47 +802,59 @@ def test_simple_bulk_with_detailed_stats_body_as_unrecognized_type(self, es): } with self.assertRaisesRegex(exceptions.DataError, "bulk body is neither string nor list"): - bulk(es, bulk_params) + await bulk(es, bulk_params) es.bulk.assert_called_with(body=bulk_params["body"], params={}) class ForceMergeRunnerTests(TestCase): @mock.patch("elasticsearch.Elasticsearch") - def test_force_merge_with_defaults(self, es): + @run_async + async def test_force_merge_with_defaults(self, es): + es.indices.forcemerge.return_value = as_future() force_merge = runner.ForceMerge() - force_merge(es, params={"index" : "_all"}) + await force_merge(es, params={"index" : "_all"}) es.indices.forcemerge.assert_called_once_with(index="_all", request_timeout=None) @mock.patch("elasticsearch.Elasticsearch") - def test_force_merge_override_request_timeout(self, es): + @run_async + async def test_force_merge_override_request_timeout(self, es): + es.indices.forcemerge.return_value = as_future() + force_merge = runner.ForceMerge() - force_merge(es, params={"index" : "_all", "request-timeout": 50000}) + await force_merge(es, params={"index" : "_all", "request-timeout": 50000}) es.indices.forcemerge.assert_called_once_with(index="_all", request_timeout=50000) @mock.patch("elasticsearch.Elasticsearch") - def test_force_merge_with_params(self, es): + @run_async + async def test_force_merge_with_params(self, es): + es.indices.forcemerge.return_value = as_future() + force_merge = runner.ForceMerge() - force_merge(es, params={"index" : "_all", "max-num-segments": 1, "request-timeout": 50000}) + await force_merge(es, params={"index" : "_all", "max-num-segments": 1, "request-timeout": 50000}) es.indices.forcemerge.assert_called_once_with(index="_all", max_num_segments=1, request_timeout=50000) @mock.patch("elasticsearch.Elasticsearch") - def test_optimize_with_defaults(self, es): - es.indices.forcemerge.side_effect = elasticsearch.TransportError(400, "Bad Request") + @run_async + async def test_optimize_with_defaults(self, es): + es.indices.forcemerge.side_effect = as_future(exception=elasticsearch.TransportError(400, "Bad Request")) + es.transport.perform_request.return_value = as_future() force_merge = runner.ForceMerge() - force_merge(es, params={}) + await force_merge(es, params={}) es.transport.perform_request.assert_called_once_with("POST", "/_optimize", params={"request_timeout": None}) @mock.patch("elasticsearch.Elasticsearch") - def test_optimize_with_params(self, es): - es.indices.forcemerge.side_effect = elasticsearch.TransportError(400, "Bad Request") + @run_async + async def test_optimize_with_params(self, es): + es.indices.forcemerge.side_effect = as_future(exception=elasticsearch.TransportError(400, "Bad Request")) + es.transport.perform_request.return_value = as_future() force_merge = runner.ForceMerge() - force_merge(es, params={"max-num-segments": 3, "request-timeout": 17000}) + await force_merge(es, params={"max-num-segments": 3, "request-timeout": 17000}) es.transport.perform_request.assert_called_once_with("POST", "/_optimize?max_num_segments=3", params={"request_timeout": 17000}) @@ -832,8 +862,9 @@ def test_optimize_with_params(self, es): class QueryRunnerTests(TestCase): @mock.patch("elasticsearch.Elasticsearch") - def test_query_match_only_request_body_defined(self, es): - es.search.return_value = { + @run_async + async def test_query_match_only_request_body_defined(self, es): + es.search.return_value = as_future({ "timed_out": False, "took": 5, "hits": { @@ -850,7 +881,7 @@ def test_query_match_only_request_body_defined(self, es): } ] } - } + }) query_runner = runner.Query() @@ -863,8 +894,8 @@ def test_query_match_only_request_body_defined(self, es): } } - with query_runner: - result = query_runner(es, params) + async with query_runner: + result = await query_runner(es, params) self.assertEqual(1, result["weight"]) self.assertEqual("ops", result["unit"]) @@ -879,10 +910,12 @@ def test_query_match_only_request_body_defined(self, es): body=params["body"], params={"request_cache": "true"} ) + es.clear_scroll.assert_not_called() @mock.patch("elasticsearch.Elasticsearch") - def test_query_match_using_request_params(self, es): - es.search.return_value = { + @run_async + async def test_query_match_using_request_params(self, es): + es.search.return_value = as_future({ "timed_out": False, "took": 62, "hits": { @@ -900,7 +933,7 @@ def test_query_match_using_request_params(self, es): ] } - } + }) query_runner = runner.Query() params = { @@ -911,8 +944,8 @@ def test_query_match_using_request_params(self, es): } } - with query_runner: - result = query_runner(es, params) + async with query_runner: + result = await query_runner(es, params) self.assertEqual(1, result["weight"]) self.assertEqual("ops", result["unit"]) @@ -930,10 +963,12 @@ def test_query_match_using_request_params(self, es): "q": "user:kimchy" } ) + es.clear_scroll.assert_not_called() @mock.patch("elasticsearch.Elasticsearch") - def test_query_hits_total_as_number(self, es): - es.search.return_value = { + @run_async + async def test_query_hits_total_as_number(self, es): + es.search.return_value = as_future({ "timed_out": False, "took": 5, "hits": { @@ -947,7 +982,7 @@ def test_query_hits_total_as_number(self, es): } ] } - } + }) query_runner = runner.Query() @@ -960,8 +995,8 @@ def test_query_hits_total_as_number(self, es): } } - with query_runner: - result = query_runner(es, params) + async with query_runner: + result = await query_runner(es, params) self.assertEqual(1, result["weight"]) self.assertEqual("ops", result["unit"]) @@ -978,10 +1013,12 @@ def test_query_hits_total_as_number(self, es): "request_cache": "true" } ) + es.clear_scroll.assert_not_called() @mock.patch("elasticsearch.Elasticsearch") - def test_query_match_all(self, es): - es.search.return_value = { + @run_async + async def test_query_match_all(self, es): + es.search.return_value = as_future({ "timed_out": False, "took": 5, "hits": { @@ -998,7 +1035,7 @@ def test_query_match_all(self, es): } ] } - } + }) query_runner = runner.Query() @@ -1012,8 +1049,8 @@ def test_query_match_all(self, es): } } - with query_runner: - result = query_runner(es, params) + async with query_runner: + result = await query_runner(es, params) self.assertEqual(1, result["weight"]) self.assertEqual("ops", result["unit"]) @@ -1028,10 +1065,12 @@ def test_query_match_all(self, es): body=params["body"], params={} ) + es.clear_scroll.assert_not_called() @mock.patch("elasticsearch.Elasticsearch") - def test_query_match_all_doc_type_fallback(self, es): - es.transport.perform_request.return_value = { + @run_async + async def test_query_match_all_doc_type_fallback(self, es): + es.transport.perform_request.return_value = as_future({ "timed_out": False, "took": 5, "hits": { @@ -1048,7 +1087,7 @@ def test_query_match_all_doc_type_fallback(self, es): } ] } - } + }) query_runner = runner.Query() @@ -1063,8 +1102,8 @@ def test_query_match_all_doc_type_fallback(self, es): } } - with query_runner: - result = query_runner(es, params) + async with query_runner: + result = await query_runner(es, params) self.assertEqual(1, result["weight"]) self.assertEqual("ops", result["unit"]) @@ -1079,11 +1118,13 @@ def test_query_match_all_doc_type_fallback(self, es): body=params["body"], params={} ) + es.clear_scroll.assert_not_called() @mock.patch("elasticsearch.Elasticsearch") - def test_scroll_query_only_one_page(self, es): + @run_async + async def test_scroll_query_only_one_page(self, es): # page 1 - es.search.return_value = { + es.search.return_value = as_future({ "_scroll_id": "some-scroll-id", "took": 4, "timed_out": False, @@ -1097,13 +1138,10 @@ def test_scroll_query_only_one_page(self, es): } ] } - } - es.transport.perform_request.side_effect = [ - # delete scroll id response - { - "acknowledged": True - } - ] + }) + es.clear_scroll.return_value = as_future({ + "acknowledged": True + }) query_runner = runner.Query() @@ -1119,8 +1157,8 @@ def test_scroll_query_only_one_page(self, es): } } - with query_runner: - results = query_runner(es, params) + async with query_runner: + results = await query_runner(es, params) self.assertEqual(1, results["weight"]) self.assertEqual(1, results["pages"]) @@ -1141,11 +1179,13 @@ def test_scroll_query_only_one_page(self, es): "request_cache": "true" } ) + es.clear_scroll.assert_called_once_with(body={"scroll_id": ["some-scroll-id"]}) @mock.patch("elasticsearch.Elasticsearch") - def test_scroll_query_no_request_cache(self, es): + @run_async + async def test_scroll_query_no_request_cache(self, es): # page 1 - es.search.return_value = { + es.search.return_value = as_future({ "_scroll_id": "some-scroll-id", "took": 4, "timed_out": False, @@ -1159,13 +1199,10 @@ def test_scroll_query_no_request_cache(self, es): } ] } - } - es.transport.perform_request.side_effect = [ - # delete scroll id response - { - "acknowledged": True - } - ] + }) + es.clear_scroll.return_value = as_future({ + "acknowledged": True + }) query_runner = runner.Query() @@ -1180,8 +1217,8 @@ def test_scroll_query_no_request_cache(self, es): } } - with query_runner: - results = query_runner(es, params) + async with query_runner: + results = await query_runner(es, params) self.assertEqual(1, results["weight"]) self.assertEqual(1, results["pages"]) @@ -1200,11 +1237,13 @@ def test_scroll_query_no_request_cache(self, es): sort='_doc', params={} ) + es.clear_scroll.assert_called_once_with(body={"scroll_id": ["some-scroll-id"]}) @mock.patch("elasticsearch.Elasticsearch") - def test_scroll_query_only_one_page_only_request_body_defined(self, es): + @run_async + async def test_scroll_query_only_one_page_only_request_body_defined(self, es): # page 1 - es.search.return_value = { + es.search.return_value = as_future({ "_scroll_id": "some-scroll-id", "took": 4, "timed_out": False, @@ -1218,13 +1257,11 @@ def test_scroll_query_only_one_page_only_request_body_defined(self, es): } ] } - } - es.transport.perform_request.side_effect = [ - # delete scroll id response - { - "acknowledged": True - } - ] + }) + + es.clear_scroll.return_value = as_future({ + "acknowledged": True + }) query_runner = runner.Query() @@ -1238,8 +1275,8 @@ def test_scroll_query_only_one_page_only_request_body_defined(self, es): } } - with query_runner: - results = query_runner(es, params) + async with query_runner: + results = await query_runner(es, params) self.assertEqual(1, results["weight"]) self.assertEqual(1, results["pages"]) @@ -1250,10 +1287,13 @@ def test_scroll_query_only_one_page_only_request_body_defined(self, es): self.assertFalse(results["timed_out"]) self.assertFalse("error-type" in results) + es.clear_scroll.assert_called_once_with(body={"scroll_id": ["some-scroll-id"]}) + @mock.patch("elasticsearch.Elasticsearch") - def test_scroll_query_with_explicit_number_of_pages(self, es): + @run_async + async def test_scroll_query_with_explicit_number_of_pages(self, es): # page 1 - es.search.return_value = { + es.search.return_value = as_future({ "_scroll_id": "some-scroll-id", "timed_out": False, "took": 54, @@ -1267,26 +1307,23 @@ def test_scroll_query_with_explicit_number_of_pages(self, es): } ] } - } - es.scroll.side_effect = [ - # page 2 - { - "_scroll_id": "some-scroll-id", - "timed_out": True, - "took": 25, - "hits": { - "hits": [ - { - "some-doc-3" - } - ] - } - }, - # delete scroll id response - { - "acknowledged": True + }) + # page 2 + es.scroll.return_value = as_future({ + "_scroll_id": "some-scroll-id", + "timed_out": True, + "took": 25, + "hits": { + "hits": [ + { + "some-doc-3" + } + ] } - ] + }) + es.clear_scroll.return_value = as_future({ + "acknowledged": True + }) query_runner = runner.Query() @@ -1302,8 +1339,8 @@ def test_scroll_query_with_explicit_number_of_pages(self, es): } } - with query_runner: - results = query_runner(es, params) + async with query_runner: + results = await query_runner(es, params) self.assertEqual(2, results["weight"]) self.assertEqual(2, results["pages"]) @@ -1314,10 +1351,14 @@ def test_scroll_query_with_explicit_number_of_pages(self, es): self.assertTrue(results["timed_out"]) self.assertFalse("error-type" in results) + es.clear_scroll.assert_called_once_with(body={"scroll_id": ["some-scroll-id"]}) + @mock.patch("elasticsearch.Elasticsearch") - def test_scroll_query_early_termination(self, es): + @run_async + async def test_scroll_query_cannot_clear_scroll(self, es): + import elasticsearch # page 1 - es.search.return_value = { + es.search.return_value = as_future({ "_scroll_id": "some-scroll-id", "timed_out": False, "took": 53, @@ -1328,77 +1369,17 @@ def test_scroll_query_early_termination(self, es): } ] } - } - es.scroll.side_effect = [ - # page 2 has no results - { - "_scroll_id": "some-scroll-id", - "timed_out": False, - "took": 2, - "hits": { - "hits": [] - } - }, - # delete scroll id response - { - "acknowledged": True - } - ] - - query_runner = runner.Query() - - params = { - "pages": 5, - "results-per-page": 100, - "index": "unittest", - "cache": False, - "body": { - "query": { - "match_all": {} - } - } - } - - with query_runner: - results = query_runner(es, params) - - self.assertEqual(2, results["weight"]) - self.assertEqual(2, results["pages"]) - self.assertEqual(1, results["hits"]) - self.assertEqual("eq", results["hits_relation"]) - self.assertEqual("pages", results["unit"]) - self.assertEqual(55, results["took"]) - self.assertFalse("error-type" in results) - - @mock.patch("elasticsearch.Elasticsearch") - def test_scroll_query_cannot_clear_scroll(self, es): - import elasticsearch - # page 1 - es.search.return_value = { + }) + # page 2 has no results + es.scroll.return_value = as_future({ "_scroll_id": "some-scroll-id", "timed_out": False, - "took": 53, + "took": 2, "hits": { - "hits": [ - { - "some-doc-1" - } - ] + "hits": [] } - } - es.scroll.side_effect = [ - # page 2 has no results - { - "_scroll_id": "some-scroll-id", - "timed_out": False, - "took": 2, - "hits": { - "hits": [] - } - }, - # delete scroll id raises an exception - elasticsearch.ConnectionTimeout() - ] + }) + es.clear_scroll.return_value = as_future(exception=elasticsearch.ConnectionTimeout()) query_runner = runner.Query() @@ -1414,8 +1395,8 @@ def test_scroll_query_cannot_clear_scroll(self, es): } } - with query_runner: - results = query_runner(es, params) + async with query_runner: + results = await query_runner(es, params) self.assertEqual(2, results["weight"]) self.assertEqual(2, results["pages"]) @@ -1425,10 +1406,13 @@ def test_scroll_query_cannot_clear_scroll(self, es): self.assertEqual(55, results["took"]) self.assertFalse("error-type" in results) + es.clear_scroll.assert_called_once_with(body={"scroll_id": ["some-scroll-id"]}) + @mock.patch("elasticsearch.Elasticsearch") - def test_scroll_query_request_all_pages(self, es): + @run_async + async def test_scroll_query_request_all_pages(self, es): # page 1 - es.search.return_value = { + es.search.return_value = as_future({ "_scroll_id": "some-scroll-id", "timed_out": False, "took": 876, @@ -1448,22 +1432,19 @@ def test_scroll_query_request_all_pages(self, es): } ] } - } - es.scroll.side_effect = [ - # page 2 has no results - { - "_scroll_id": "some-scroll-id", - "took": 24, - "timed_out": False, - "hits": { - "hits": [] - } - }, - # delete scroll id response - { - "acknowledged": True + }) + # page 2 has no results + es.scroll.return_value = as_future({ + "_scroll_id": "some-scroll-id", + "timed_out": False, + "took": 2, + "hits": { + "hits": [] } - ] + }) + es.clear_scroll.return_value = as_future({ + "acknowledged": True + }) query_runner = runner.Query() @@ -1479,22 +1460,27 @@ def test_scroll_query_request_all_pages(self, es): } } - with query_runner: - results = query_runner(es, params) + async with query_runner: + results = await query_runner(es, params) self.assertEqual(2, results["weight"]) self.assertEqual(2, results["pages"]) self.assertEqual(4, results["hits"]) self.assertEqual("eq", results["hits_relation"]) - self.assertEqual(900, results["took"]) + self.assertEqual(878, results["took"]) self.assertEqual("pages", results["unit"]) self.assertFalse(results["timed_out"]) self.assertFalse("error-type" in results) + es.clear_scroll.assert_called_once_with(body={"scroll_id": ["some-scroll-id"]}) + class PutPipelineRunnerTests(TestCase): @mock.patch("elasticsearch.Elasticsearch") - def test_create_pipeline(self, es): + @run_async + async def test_create_pipeline(self, es): + es.ingest.put_pipeline.return_value = as_future() + r = runner.PutPipeline() params = { @@ -1512,12 +1498,15 @@ def test_create_pipeline(self, es): } } - r(es, params) + await r(es, params) es.ingest.put_pipeline.assert_called_once_with(id="rename", body=params["body"], master_timeout=None, timeout=None) @mock.patch("elasticsearch.Elasticsearch") - def test_param_body_mandatory(self, es): + @run_async + async def test_param_body_mandatory(self, es): + es.ingest.put_pipeline.return_value = as_future() + r = runner.PutPipeline() params = { @@ -1526,12 +1515,15 @@ def test_param_body_mandatory(self, es): with self.assertRaisesRegex(exceptions.DataError, "Parameter source for operation 'put-pipeline' did not provide the mandatory parameter 'body'. " "Please add it to your parameter source."): - r(es, params) + await r(es, params) self.assertEqual(0, es.ingest.put_pipeline.call_count) @mock.patch("elasticsearch.Elasticsearch") - def test_param_id_mandatory(self, es): + @run_async + async def test_param_id_mandatory(self, es): + es.ingest.put_pipeline.return_value = as_future() + r = runner.PutPipeline() params = { @@ -1540,18 +1532,19 @@ def test_param_id_mandatory(self, es): with self.assertRaisesRegex(exceptions.DataError, "Parameter source for operation 'put-pipeline' did not provide the mandatory parameter 'id'. " "Please add it to your parameter source."): - r(es, params) + await r(es, params) self.assertEqual(0, es.ingest.put_pipeline.call_count) class ClusterHealthRunnerTests(TestCase): @mock.patch("elasticsearch.Elasticsearch") - def test_waits_for_expected_cluster_status(self, es): - es.cluster.health.return_value = { + @run_async + async def test_waits_for_expected_cluster_status(self, es): + es.cluster.health.return_value = as_future({ "status": "green", "relocating_shards": 0 - } + }) r = runner.ClusterHealth() params = { @@ -1560,7 +1553,7 @@ def test_waits_for_expected_cluster_status(self, es): } } - result = r(es, params) + result = await r(es, params) self.assertDictEqual({ "weight": 1, @@ -1573,11 +1566,12 @@ def test_waits_for_expected_cluster_status(self, es): es.cluster.health.assert_called_once_with(index=None, params={"wait_for_status": "green"}) @mock.patch("elasticsearch.Elasticsearch") - def test_accepts_better_cluster_status(self, es): - es.cluster.health.return_value = { + @run_async + async def test_accepts_better_cluster_status(self, es): + es.cluster.health.return_value = as_future({ "status": "green", "relocating_shards": 0 - } + }) r = runner.ClusterHealth() params = { @@ -1586,7 +1580,7 @@ def test_accepts_better_cluster_status(self, es): } } - result = r(es, params) + result = await r(es, params) self.assertDictEqual({ "weight": 1, @@ -1599,11 +1593,12 @@ def test_accepts_better_cluster_status(self, es): es.cluster.health.assert_called_once_with(index=None, params={"wait_for_status": "yellow"}) @mock.patch("elasticsearch.Elasticsearch") - def test_rejects_relocating_shards(self, es): - es.cluster.health.return_value = { + @run_async + async def test_rejects_relocating_shards(self, es): + es.cluster.health.return_value = as_future({ "status": "yellow", "relocating_shards": 3 - } + }) r = runner.ClusterHealth() params = { @@ -1614,7 +1609,7 @@ def test_rejects_relocating_shards(self, es): } } - result = r(es, params) + result = await r(es, params) self.assertDictEqual({ "weight": 1, @@ -1628,11 +1623,12 @@ def test_rejects_relocating_shards(self, es): params={"wait_for_status": "red", "wait_for_no_relocating_shards": True}) @mock.patch("elasticsearch.Elasticsearch") - def test_rejects_unknown_cluster_status(self, es): - es.cluster.health.return_value = { + @run_async + async def test_rejects_unknown_cluster_status(self, es): + es.cluster.health.return_value = as_future({ "status": None, "relocating_shards": 0 - } + }) r = runner.ClusterHealth() params = { @@ -1641,7 +1637,7 @@ def test_rejects_unknown_cluster_status(self, es): } } - result = r(es, params) + result = await r(es, params) self.assertDictEqual({ "weight": 1, @@ -1656,7 +1652,10 @@ def test_rejects_unknown_cluster_status(self, es): class CreateIndexRunnerTests(TestCase): @mock.patch("elasticsearch.Elasticsearch") - def test_creates_multiple_indices(self, es): + @run_async + async def test_creates_multiple_indices(self, es): + es.indices.create.return_value = as_future() + r = runner.CreateIndex() request_params = { @@ -1671,7 +1670,7 @@ def test_creates_multiple_indices(self, es): "request-params": request_params } - result = r(es, params) + result = await r(es, params) self.assertEqual((2, "ops"), result) @@ -1681,22 +1680,27 @@ def test_creates_multiple_indices(self, es): ]) @mock.patch("elasticsearch.Elasticsearch") - def test_param_indices_mandatory(self, es): + @run_async + async def test_param_indices_mandatory(self, es): + es.indices.create.return_value = as_future() + r = runner.CreateIndex() params = {} with self.assertRaisesRegex(exceptions.DataError, "Parameter source for operation 'create-index' did not provide the mandatory parameter 'indices'. " "Please add it to your parameter source."): - r(es, params) + await r(es, params) self.assertEqual(0, es.indices.create.call_count) class DeleteIndexRunnerTests(TestCase): @mock.patch("elasticsearch.Elasticsearch") - def test_deletes_existing_indices(self, es): - es.indices.exists.side_effect = [False, True] + @run_async + async def test_deletes_existing_indices(self, es): + es.indices.exists.side_effect = [as_future(False), as_future(True)] + es.indices.delete.return_value = as_future() r = runner.DeleteIndex() @@ -1705,14 +1709,17 @@ def test_deletes_existing_indices(self, es): "only-if-exists": True } - result = r(es, params) + result = await r(es, params) self.assertEqual((1, "ops"), result) es.indices.delete.assert_called_once_with(index="indexB", params={}) @mock.patch("elasticsearch.Elasticsearch") - def test_deletes_all_indices(self, es): + @run_async + async def test_deletes_all_indices(self, es): + es.indices.delete.return_value = as_future() + r = runner.DeleteIndex() params = { @@ -1724,7 +1731,7 @@ def test_deletes_all_indices(self, es): } } - result = r(es, params) + result = await r(es, params) self.assertEqual((2, "ops"), result) @@ -1737,7 +1744,10 @@ def test_deletes_all_indices(self, es): class CreateIndexTemplateRunnerTests(TestCase): @mock.patch("elasticsearch.Elasticsearch") - def test_create_index_templates(self, es): + @run_async + async def test_create_index_templates(self, es): + es.indices.put_template.return_value = as_future() + r = runner.CreateIndexTemplate() params = { @@ -1751,7 +1761,7 @@ def test_create_index_templates(self, es): } } - result = r(es, params) + result = await r(es, params) self.assertEqual((2, "ops"), result) @@ -1761,21 +1771,28 @@ def test_create_index_templates(self, es): ]) @mock.patch("elasticsearch.Elasticsearch") - def test_param_templates_mandatory(self, es): + @run_async + async def test_param_templates_mandatory(self, es): + es.indices.put_template.return_value = as_future() + r = runner.CreateIndexTemplate() params = {} with self.assertRaisesRegex(exceptions.DataError, "Parameter source for operation 'create-index-template' did not provide the mandatory parameter " "'templates'. Please add it to your parameter source."): - r(es, params) + await r(es, params) self.assertEqual(0, es.indices.put_template.call_count) class DeleteIndexTemplateRunnerTests(TestCase): @mock.patch("elasticsearch.Elasticsearch") - def test_deletes_all_index_templates(self, es): + @run_async + async def test_deletes_all_index_templates(self, es): + es.indices.delete_template.return_value = as_future() + es.indices.delete.return_value = as_future() + r = runner.DeleteIndexTemplate() params = { @@ -1787,7 +1804,7 @@ def test_deletes_all_index_templates(self, es): "timeout": 60 } } - result = r(es, params) + result = await r(es, params) # 2 times delete index template, one time delete matching indices self.assertEqual((3, "ops"), result) @@ -1799,8 +1816,10 @@ def test_deletes_all_index_templates(self, es): es.indices.delete.assert_called_once_with(index="logs-*") @mock.patch("elasticsearch.Elasticsearch") - def test_deletes_only_existing_index_templates(self, es): - es.indices.exists_template.side_effect = [False, True] + @run_async + async def test_deletes_only_existing_index_templates(self, es): + es.indices.exists_template.side_effect = [as_future(False), as_future(True)] + es.indices.delete_template.return_value = as_future() r = runner.DeleteIndexTemplate() @@ -1815,7 +1834,7 @@ def test_deletes_only_existing_index_templates(self, es): }, "only-if-exists": True } - result = r(es, params) + result = await r(es, params) # 2 times delete index template, one time delete matching indices self.assertEqual((1, "ops"), result) @@ -1825,21 +1844,25 @@ def test_deletes_only_existing_index_templates(self, es): self.assertEqual(0, es.indices.delete.call_count) @mock.patch("elasticsearch.Elasticsearch") - def test_param_templates_mandatory(self, es): + @run_async + async def test_param_templates_mandatory(self, es): r = runner.DeleteIndexTemplate() params = {} with self.assertRaisesRegex(exceptions.DataError, "Parameter source for operation 'delete-index-template' did not provide the mandatory parameter " "'templates'. Please add it to your parameter source."): - r(es, params) + await r(es, params) self.assertEqual(0, es.indices.delete_template.call_count) class CreateMlDatafeedTests(TestCase): @mock.patch("elasticsearch.Elasticsearch") - def test_create_ml_datafeed(self, es): + @run_async + async def test_create_ml_datafeed(self, es): + es.xpack.ml.put_datafeed.return_value = as_future() + params = { "datafeed-id": "some-data-feed", "body": { @@ -1849,13 +1872,15 @@ def test_create_ml_datafeed(self, es): } r = runner.CreateMlDatafeed() - r(es, params) + await r(es, params) es.xpack.ml.put_datafeed.assert_called_once_with(datafeed_id=params["datafeed-id"], body=params["body"]) @mock.patch("elasticsearch.Elasticsearch") - def test_create_ml_datafeed_fallback(self, es): - es.xpack.ml.put_datafeed.side_effect = elasticsearch.TransportError(400, "Bad Request") + @run_async + async def test_create_ml_datafeed_fallback(self, es): + es.xpack.ml.put_datafeed.side_effect = as_future(exception=elasticsearch.TransportError(400, "Bad Request")) + es.transport.perform_request.return_value = as_future() datafeed_id = "some-data-feed" body = { "job_id": "total-requests", @@ -1867,7 +1892,7 @@ def test_create_ml_datafeed_fallback(self, es): } r = runner.CreateMlDatafeed() - r(es, params) + await r(es, params) es.transport.perform_request.assert_called_once_with("PUT", "/_xpack/ml/datafeeds/%s" % datafeed_id, @@ -1877,27 +1902,32 @@ def test_create_ml_datafeed_fallback(self, es): class DeleteMlDatafeedTests(TestCase): @mock.patch("elasticsearch.Elasticsearch") - def test_delete_ml_datafeed(self, es): + @run_async + async def test_delete_ml_datafeed(self, es): + es.xpack.ml.delete_datafeed.return_value = as_future() + datafeed_id = "some-data-feed" params = { "datafeed-id": datafeed_id } r = runner.DeleteMlDatafeed() - r(es, params) + await r(es, params) es.xpack.ml.delete_datafeed.assert_called_once_with(datafeed_id=datafeed_id, force=False, ignore=[404]) @mock.patch("elasticsearch.Elasticsearch") - def test_delete_ml_datafeed_fallback(self, es): - es.xpack.ml.delete_datafeed.side_effect = elasticsearch.TransportError(400, "Bad Request") + @run_async + async def test_delete_ml_datafeed_fallback(self, es): + es.xpack.ml.delete_datafeed.side_effect = as_future(exception=elasticsearch.TransportError(400, "Bad Request")) + es.transport.perform_request.return_value = as_future() datafeed_id = "some-data-feed" params = { "datafeed-id": datafeed_id, } r = runner.DeleteMlDatafeed() - r(es, params) + await r(es, params) es.transport.perform_request.assert_called_once_with("DELETE", "/_xpack/ml/datafeeds/%s" % datafeed_id, @@ -1906,7 +1936,9 @@ def test_delete_ml_datafeed_fallback(self, es): class StartMlDatafeedTests(TestCase): @mock.patch("elasticsearch.Elasticsearch") - def test_start_ml_datafeed_with_body(self, es): + @run_async + async def test_start_ml_datafeed_with_body(self, es): + es.xpack.ml.start_datafeed.return_value = as_future() params = { "datafeed-id": "some-data-feed", "body": { @@ -1915,7 +1947,7 @@ def test_start_ml_datafeed_with_body(self, es): } r = runner.StartMlDatafeed() - r(es, params) + await r(es, params) es.xpack.ml.start_datafeed.assert_called_once_with(datafeed_id=params["datafeed-id"], body=params["body"], @@ -1924,8 +1956,10 @@ def test_start_ml_datafeed_with_body(self, es): timeout=None) @mock.patch("elasticsearch.Elasticsearch") - def test_start_ml_datafeed_with_body_fallback(self, es): - es.xpack.ml.start_datafeed.side_effect = elasticsearch.TransportError(400, "Bad Request") + @run_async + async def test_start_ml_datafeed_with_body_fallback(self, es): + es.xpack.ml.start_datafeed.side_effect = as_future(exception=elasticsearch.TransportError(400, "Bad Request")) + es.transport.perform_request.return_value = as_future() body = { "end": "now" } @@ -1935,7 +1969,7 @@ def test_start_ml_datafeed_with_body_fallback(self, es): } r = runner.StartMlDatafeed() - r(es, params) + await r(es, params) es.transport.perform_request.assert_called_once_with("POST", "/_xpack/ml/datafeeds/%s/_start" % params["datafeed-id"], @@ -1943,7 +1977,9 @@ def test_start_ml_datafeed_with_body_fallback(self, es): params=params) @mock.patch("elasticsearch.Elasticsearch") - def test_start_ml_datafeed_with_params(self, es): + @run_async + async def test_start_ml_datafeed_with_params(self, es): + es.xpack.ml.start_datafeed.return_value = as_future() params = { "datafeed-id": "some-data-feed", "start": "2017-01-01T01:00:00Z", @@ -1952,7 +1988,7 @@ def test_start_ml_datafeed_with_params(self, es): } r = runner.StartMlDatafeed() - r(es, params) + await r(es, params) es.xpack.ml.start_datafeed.assert_called_once_with(datafeed_id=params["datafeed-id"], body=None, @@ -1961,11 +1997,11 @@ def test_start_ml_datafeed_with_params(self, es): timeout=params["timeout"]) -class StopMlDatafeedTests: +class StopMlDatafeedTests(TestCase): @mock.patch("elasticsearch.Elasticsearch") - @pytest.mark.parametrize("seed", range(20)) - def test_stop_ml_datafeed(self, es, seed): - random.seed(seed) + @run_async + async def test_stop_ml_datafeed(self, es): + es.xpack.ml.stop_datafeed.return_value = as_future() params = { "datafeed-id": "some-data-feed", "force": random.choice([False, True]), @@ -1973,17 +2009,18 @@ def test_stop_ml_datafeed(self, es, seed): } r = runner.StopMlDatafeed() - r(es, params) + await r(es, params) es.xpack.ml.stop_datafeed.assert_called_once_with(datafeed_id=params["datafeed-id"], force=params["force"], timeout=params["timeout"]) @mock.patch("elasticsearch.Elasticsearch") - @pytest.mark.parametrize("seed", range(20)) - def test_stop_ml_datafeed_fallback(self, es, seed): - random.seed(seed) - es.xpack.ml.stop_datafeed.side_effect = elasticsearch.TransportError(400, "Bad Request") + @run_async + async def test_stop_ml_datafeed_fallback(self, es): + es.xpack.ml.stop_datafeed.side_effect = as_future(exception=elasticsearch.TransportError(400, "Bad Request")) + es.transport.perform_request.return_value = as_future() + params = { "datafeed-id": "some-data-feed", "force": random.choice([False, True]), @@ -1991,7 +2028,7 @@ def test_stop_ml_datafeed_fallback(self, es, seed): } r = runner.StopMlDatafeed() - r(es, params) + await r(es, params) es.transport.perform_request.assert_called_once_with("POST", "/_xpack/ml/datafeeds/%s/_stop" % params["datafeed-id"], @@ -2000,7 +2037,10 @@ def test_stop_ml_datafeed_fallback(self, es, seed): class CreateMlJobTests(TestCase): @mock.patch("elasticsearch.Elasticsearch") - def test_create_ml_job(self, es): + @run_async + async def test_create_ml_job(self, es): + es.xpack.ml.put_job.return_value = as_future() + params = { "job-id": "an-ml-job", "body": { @@ -2023,13 +2063,16 @@ def test_create_ml_job(self, es): } r = runner.CreateMlJob() - r(es, params) + await r(es, params) es.xpack.ml.put_job.assert_called_once_with(job_id=params["job-id"], body=params["body"]) @mock.patch("elasticsearch.Elasticsearch") - def test_create_ml_job_fallback(self, es): - es.xpack.ml.put_job.side_effect = elasticsearch.TransportError(400, "Bad Request") + @run_async + async def test_create_ml_job_fallback(self, es): + es.xpack.ml.put_job.side_effect = as_future(exception=elasticsearch.TransportError(400, "Bad Request")) + es.transport.perform_request.return_value = as_future() + body = { "description": "Total sum of requests", "analysis_config": { @@ -2053,7 +2096,7 @@ def test_create_ml_job_fallback(self, es): } r = runner.CreateMlJob() - r(es, params) + await r(es, params) es.transport.perform_request.assert_called_once_with("PUT", "/_xpack/ml/anomaly_detectors/%s" % params["job-id"], @@ -2063,20 +2106,25 @@ def test_create_ml_job_fallback(self, es): class DeleteMlJobTests(TestCase): @mock.patch("elasticsearch.Elasticsearch") - def test_delete_ml_job(self, es): + @run_async + async def test_delete_ml_job(self, es): + es.xpack.ml.delete_job.return_value = as_future() + job_id = "an-ml-job" params = { "job-id": job_id } r = runner.DeleteMlJob() - r(es, params) + await r(es, params) es.xpack.ml.delete_job.assert_called_once_with(job_id=job_id, force=False, ignore=[404]) @mock.patch("elasticsearch.Elasticsearch") - def test_delete_ml_job_fallback(self, es): - es.xpack.ml.delete_job.side_effect = elasticsearch.TransportError(400, "Bad Request") + @run_async + async def test_delete_ml_job_fallback(self, es): + es.xpack.ml.delete_job.side_effect = as_future(exception=elasticsearch.TransportError(400, "Bad Request")) + es.transport.perform_request.return_value = as_future() job_id = "an-ml-job" params = { @@ -2084,7 +2132,7 @@ def test_delete_ml_job_fallback(self, es): } r = runner.DeleteMlJob() - r(es, params) + await r(es, params) es.transport.perform_request.assert_called_once_with("DELETE", "/_xpack/ml/anomaly_detectors/%s" % params["job-id"], @@ -2093,20 +2141,25 @@ def test_delete_ml_job_fallback(self, es): class OpenMlJobTests(TestCase): @mock.patch("elasticsearch.Elasticsearch") - def test_open_ml_job(self, es): + @run_async + async def test_open_ml_job(self, es): + es.xpack.ml.open_job.return_value = as_future() + job_id = "an-ml-job" params = { "job-id": job_id } r = runner.OpenMlJob() - r(es, params) + await r(es, params) es.xpack.ml.open_job.assert_called_once_with(job_id=job_id) @mock.patch("elasticsearch.Elasticsearch") - def test_open_ml_job_fallback(self, es): - es.xpack.ml.open_job.side_effect = elasticsearch.TransportError(400, "Bad Request") + @run_async + async def test_open_ml_job_fallback(self, es): + es.xpack.ml.open_job.side_effect = as_future(exception=elasticsearch.TransportError(400, "Bad Request")) + es.transport.perform_request.return_value = as_future() job_id = "an-ml-job" params = { @@ -2114,18 +2167,18 @@ def test_open_ml_job_fallback(self, es): } r = runner.OpenMlJob() - r(es, params) + await r(es, params) es.transport.perform_request.assert_called_once_with("POST", "/_xpack/ml/anomaly_detectors/%s/_open" % params["job-id"], params=params) -class CloseMlJobTests: +class CloseMlJobTests(TestCase): @mock.patch("elasticsearch.Elasticsearch") - @pytest.mark.parametrize("seed", range(20)) - def test_close_ml_job(self, es, seed): - random.seed(seed) + @run_async + async def test_close_ml_job(self, es): + es.xpack.ml.close_job.return_value = as_future() params = { "job-id": "an-ml-job", "force": random.choice([False, True]), @@ -2133,15 +2186,15 @@ def test_close_ml_job(self, es, seed): } r = runner.CloseMlJob() - r(es, params) + await r(es, params) es.xpack.ml.close_job.assert_called_once_with(job_id=params["job-id"], force=params["force"], timeout=params["timeout"]) @mock.patch("elasticsearch.Elasticsearch") - @pytest.mark.parametrize("seed", range(20)) - def test_close_ml_job_fallback(self, es, seed): - random.seed(seed) - es.xpack.ml.close_job.side_effect = elasticsearch.TransportError(400, "Bad Request") + @run_async + async def test_close_ml_job_fallback(self, es): + es.xpack.ml.close_job.side_effect = as_future(exception=elasticsearch.TransportError(400, "Bad Request")) + es.transport.perform_request.return_value = as_future() params = { "job-id": "an-ml-job", @@ -2150,7 +2203,7 @@ def test_close_ml_job_fallback(self, es, seed): } r = runner.CloseMlJob() - r(es, params) + await r(es, params) es.transport.perform_request.assert_called_once_with("POST", "/_xpack/ml/anomaly_detectors/%s/_close" % params["job-id"], @@ -2159,13 +2212,15 @@ def test_close_ml_job_fallback(self, es, seed): class RawRequestRunnerTests(TestCase): @mock.patch("elasticsearch.Elasticsearch") - def test_issue_request_with_defaults(self, es): + @run_async + async def test_issue_request_with_defaults(self, es): + es.transport.perform_request.return_value = as_future() r = runner.RawRequest() params = { "path": "/_cat/count" } - r(es, params) + await r(es, params) es.transport.perform_request.assert_called_once_with(method="GET", url="/_cat/count", @@ -2174,7 +2229,9 @@ def test_issue_request_with_defaults(self, es): params={}) @mock.patch("elasticsearch.Elasticsearch") - def test_issue_delete_index(self, es): + @run_async + async def test_issue_delete_index(self, es): + es.transport.perform_request.return_value = as_future() r = runner.RawRequest() params = { @@ -2185,7 +2242,7 @@ def test_issue_delete_index(self, es): "pretty": "true" } } - r(es, params) + await r(es, params) es.transport.perform_request.assert_called_once_with(method="DELETE", url="/twitter", @@ -2194,7 +2251,9 @@ def test_issue_delete_index(self, es): params={"ignore": [400, 404], "pretty": "true"}) @mock.patch("elasticsearch.Elasticsearch") - def test_issue_create_index(self, es): + @run_async + async def test_issue_create_index(self, es): + es.transport.perform_request.return_value = as_future() r = runner.RawRequest() params = { @@ -2208,7 +2267,7 @@ def test_issue_create_index(self, es): } } } - r(es, params) + await r(es, params) es.transport.perform_request.assert_called_once_with(method="POST", url="/twitter", @@ -2223,7 +2282,9 @@ def test_issue_create_index(self, es): params={}) @mock.patch("elasticsearch.Elasticsearch") - def test_issue_msearch(self, es): + @run_async + async def test_issue_msearch(self, es): + es.transport.perform_request.return_value = as_future() r = runner.RawRequest() params = { @@ -2238,7 +2299,7 @@ def test_issue_msearch(self, es): {"query": {"match_all": {}}} ] } - r(es, params) + await r(es, params) es.transport.perform_request.assert_called_once_with(method="GET", url="/_msearch", @@ -2255,23 +2316,25 @@ def test_issue_msearch(self, es): class SleepTests(TestCase): @mock.patch("elasticsearch.Elasticsearch") # To avoid real sleeps in unit tests - @mock.patch("time.sleep") - def test_missing_parameter(self, sleep, es): + @mock.patch("asyncio.sleep", return_value=as_future()) + @run_async + async def test_missing_parameter(self, sleep, es): r = runner.Sleep() with self.assertRaisesRegex(exceptions.DataError, "Parameter source for operation 'sleep' did not provide the mandatory parameter " "'duration'. Please add it to your parameter source."): - r(es, params={}) + await r(es, params={}) self.assertEqual(0, es.call_count) self.assertEqual(0, sleep.call_count) @mock.patch("elasticsearch.Elasticsearch") # To avoid real sleeps in unit tests - @mock.patch("time.sleep") - def test_sleep(self, sleep, es): + @mock.patch("asyncio.sleep", return_value=as_future()) + @run_async + async def test_sleep(self, sleep, es): r = runner.Sleep() - r(es, params={"duration": 4.3}) + await r(es, params={"duration": 4.3}) self.assertEqual(0, es.call_count) sleep.assert_called_once_with(4.3) @@ -2279,20 +2342,24 @@ def test_sleep(self, sleep, es): class DeleteSnapshotRepositoryTests(TestCase): @mock.patch("elasticsearch.Elasticsearch") - def test_delete_snapshot_repository(self, es): + @run_async + async def test_delete_snapshot_repository(self, es): + es.snapshot.delete_repository.return_value = as_future() params = { "repository": "backups" } r = runner.DeleteSnapshotRepository() - r(es, params) + await r(es, params) es.snapshot.delete_repository.assert_called_once_with(repository="backups") class CreateSnapshotRepositoryTests(TestCase): @mock.patch("elasticsearch.Elasticsearch") - def test_create_snapshot_repository(self, es): + @run_async + async def test_create_snapshot_repository(self, es): + es.snapshot.create_repository.return_value = as_future() params = { "repository": "backups", "body": { @@ -2304,7 +2371,7 @@ def test_create_snapshot_repository(self, es): } r = runner.CreateSnapshotRepository() - r(es, params) + await r(es, params) es.snapshot.create_repository.assert_called_once_with(repository="backups", body={ @@ -2318,7 +2385,10 @@ def test_create_snapshot_repository(self, es): class RestoreSnapshotTests(TestCase): @mock.patch("elasticsearch.Elasticsearch") - def test_restore_snapshot(self, es): + @run_async + async def test_restore_snapshot(self, es): + es.snapshot.restore.return_value = as_future() + params = { "repository": "backups", "snapshot": "snapshot-001", @@ -2329,7 +2399,7 @@ def test_restore_snapshot(self, es): } r = runner.RestoreSnapshot() - r(es, params) + await r(es, params) es.snapshot.restore.assert_called_once_with(repository="backups", snapshot="snapshot-001", @@ -2338,7 +2408,9 @@ def test_restore_snapshot(self, es): params={"request_timeout": 7200}) @mock.patch("elasticsearch.Elasticsearch") - def test_restore_snapshot_with_body(self, es): + @run_async + async def test_restore_snapshot_with_body(self, es): + es.snapshot.restore.return_value = as_future() params = { "repository": "backups", "snapshot": "snapshot-001", @@ -2356,7 +2428,7 @@ def test_restore_snapshot_with_body(self, es): } r = runner.RestoreSnapshot() - r(es, params) + await r(es, params) es.snapshot.restore.assert_called_once_with(repository="backups", snapshot="snapshot-001", @@ -2373,15 +2445,16 @@ def test_restore_snapshot_with_body(self, es): class IndicesRecoveryTests(TestCase): @mock.patch("elasticsearch.Elasticsearch") - def test_indices_recovery_already_finished(self, es): + @run_async + async def test_indices_recovery_already_finished(self, es): # empty response - es.indices.recovery.return_value = {} + es.indices.recovery.return_value = as_future({}) r = runner.IndicesRecovery() self.assertFalse(r.completed) self.assertEqual(r.percent_completed, 0.0) - r(es, { + await r(es, { "completion-recheck-wait-period": 0 }) @@ -2393,11 +2466,12 @@ def test_indices_recovery_already_finished(self, es): self.assertEqual(3, es.indices.recovery.call_count) @mock.patch("elasticsearch.Elasticsearch") - def test_waits_for_ongoing_indices_recovery(self, es): + @run_async + async def test_waits_for_ongoing_indices_recovery(self, es): # empty response es.indices.recovery.side_effect = [ # active recovery - { + as_future({ "index1": { "shards": [ { @@ -2424,11 +2498,11 @@ def test_waits_for_ongoing_indices_recovery(self, es): } ] } - }, + }), # completed - will be called three times - {}, - {}, - {}, + as_future({}), + as_future({}), + as_future({}), ] r = runner.IndicesRecovery() @@ -2436,7 +2510,7 @@ def test_waits_for_ongoing_indices_recovery(self, es): self.assertEqual(r.percent_completed, 0.0) while not r.completed: - recovered_bytes, unit = r(es, { + recovered_bytes, unit = await r(es, { "completion-recheck-wait-period": 0 }) if r.completed: @@ -2453,13 +2527,16 @@ def test_waits_for_ongoing_indices_recovery(self, es): class ShrinkIndexTests(TestCase): @mock.patch("elasticsearch.Elasticsearch") # To avoid real sleeps in unit tests - @mock.patch("time.sleep") - def test_shrink_index_with_shrink_node(self, sleep, es): + @mock.patch("asyncio.sleep", return_value=as_future()) + @run_async + async def test_shrink_index_with_shrink_node(self, sleep, es): # cluster health API - es.cluster.health.return_value = { + es.cluster.health.return_value = as_future({ "status": "green", "relocating_shards": 0 - } + }) + es.indices.put_settings.return_value = as_future() + es.indices.shrink.return_value = as_future() r = runner.ShrinkIndex() params = { @@ -2474,7 +2551,7 @@ def test_shrink_index_with_shrink_node(self, sleep, es): "shrink-node": "rally-node-0" } - r(es, params) + await r(es, params) es.indices.put_settings.assert_called_once_with(index="src", body={ @@ -2501,14 +2578,15 @@ def test_shrink_index_with_shrink_node(self, sleep, es): @mock.patch("elasticsearch.Elasticsearch") # To avoid real sleeps in unit tests - @mock.patch("time.sleep") - def test_shrink_index_derives_shrink_node(self, sleep, es): + @mock.patch("asyncio.sleep", return_value=as_future()) + @run_async + async def test_shrink_index_derives_shrink_node(self, sleep, es): # cluster health API - es.cluster.health.return_value = { + es.cluster.health.return_value = as_future({ "status": "green", "relocating_shards": 0 - } - es.nodes.info.return_value = { + }) + es.nodes.info.return_value = as_future({ "_nodes": { "total": 3, "successful": 3, @@ -2537,7 +2615,9 @@ def test_shrink_index_derives_shrink_node(self, sleep, es): ] } } - } + }) + es.indices.put_settings.return_value = as_future() + es.indices.shrink.return_value = as_future() r = runner.ShrinkIndex() params = { @@ -2551,7 +2631,7 @@ def test_shrink_index_derives_shrink_node(self, sleep, es): } } - r(es, params) + await r(es, params) es.indices.put_settings.assert_called_once_with(index="src", body={ @@ -2580,7 +2660,9 @@ def test_shrink_index_derives_shrink_node(self, sleep, es): class PutSettingsTests(TestCase): @mock.patch("elasticsearch.Elasticsearch") - def test_put_settings(self, es): + @run_async + async def test_put_settings(self, es): + es.cluster.put_settings.return_value = as_future() params = { "body": { "transient": { @@ -2590,7 +2672,7 @@ def test_put_settings(self, es): } r = runner.PutSettings() - r(es, params) + await r(es, params) es.cluster.put_settings.assert_called_once_with(body={ "transient": { @@ -2600,22 +2682,24 @@ def test_put_settings(self, es): class RetryTests(TestCase): - def test_is_transparent_on_success_when_no_retries(self): - delegate = mock.Mock() + @run_async + async def test_is_transparent_on_success_when_no_retries(self): + delegate = mock.Mock(return_value=as_future()) es = None params = { # no retries } retrier = runner.Retry(delegate) - retrier(es, params) + await retrier(es, params) delegate.assert_called_once_with(es, params) - def test_is_transparent_on_exception_when_no_retries(self): + @run_async + async def test_is_transparent_on_exception_when_no_retries(self): import elasticsearch - delegate = mock.Mock(side_effect=elasticsearch.ConnectionError("N/A", "no route to host")) + delegate = mock.Mock(side_effect=as_future(exception=elasticsearch.ConnectionError("N/A", "no route to host"))) es = None params = { # no retries @@ -2623,27 +2707,29 @@ def test_is_transparent_on_exception_when_no_retries(self): retrier = runner.Retry(delegate) with self.assertRaises(elasticsearch.ConnectionError): - retrier(es, params) + await retrier(es, params) delegate.assert_called_once_with(es, params) - def test_is_transparent_on_application_error_when_no_retries(self): + @run_async + async def test_is_transparent_on_application_error_when_no_retries(self): original_return_value = {"weight": 1, "unit": "ops", "success": False} - delegate = mock.Mock(return_value=original_return_value) + delegate = mock.Mock(return_value=as_future(original_return_value)) es = None params = { # no retries } retrier = runner.Retry(delegate) - result = retrier(es, params) + result = await retrier(es, params) self.assertEqual(original_return_value, result) delegate.assert_called_once_with(es, params) - def test_is_does_not_retry_on_success(self): - delegate = mock.Mock() + @run_async + async def test_is_does_not_retry_on_success(self): + delegate = mock.Mock(return_value=as_future()) es = None params = { "retries": 3, @@ -2653,14 +2739,20 @@ def test_is_does_not_retry_on_success(self): } retrier = runner.Retry(delegate) - retrier(es, params) + await retrier(es, params) delegate.assert_called_once_with(es, params) - def test_retries_on_timeout_if_wanted_and_raises_if_no_recovery(self): + @run_async + async def test_retries_on_timeout_if_wanted_and_raises_if_no_recovery(self): import elasticsearch - delegate = mock.Mock(side_effect=elasticsearch.ConnectionError("N/A", "no route to host")) + delegate = mock.Mock(side_effect=[ + as_future(exception=elasticsearch.ConnectionError("N/A", "no route to host")), + as_future(exception=elasticsearch.ConnectionError("N/A", "no route to host")), + as_future(exception=elasticsearch.ConnectionError("N/A", "no route to host")), + as_future(exception=elasticsearch.ConnectionError("N/A", "no route to host")) + ]) es = None params = { "retries": 3, @@ -2671,7 +2763,7 @@ def test_retries_on_timeout_if_wanted_and_raises_if_no_recovery(self): retrier = runner.Retry(delegate) with self.assertRaises(elasticsearch.ConnectionError): - retrier(es, params) + await retrier(es, params) delegate.assert_has_calls([ mock.call(es, params), @@ -2679,11 +2771,15 @@ def test_retries_on_timeout_if_wanted_and_raises_if_no_recovery(self): mock.call(es, params) ]) - def test_retries_on_timeout_if_wanted_and_returns_first_call(self): + @run_async + async def test_retries_on_timeout_if_wanted_and_returns_first_call(self): import elasticsearch failed_return_value = {"weight": 1, "unit": "ops", "success": False} - delegate = mock.Mock(side_effect=[elasticsearch.ConnectionError("N/A", "no route to host"), failed_return_value]) + delegate = mock.Mock(side_effect=[ + as_future(exception=elasticsearch.ConnectionError("N/A", "no route to host")), + as_future(failed_return_value) + ]) es = None params = { "retries": 3, @@ -2693,7 +2789,7 @@ def test_retries_on_timeout_if_wanted_and_returns_first_call(self): } retrier = runner.Retry(delegate) - result = retrier(es, params) + result = await retrier(es, params) self.assertEqual(failed_return_value, result) delegate.assert_has_calls([ @@ -2703,20 +2799,21 @@ def test_retries_on_timeout_if_wanted_and_returns_first_call(self): mock.call(es, params) ]) - def test_retries_mixed_timeout_and_application_errors(self): + @run_async + async def test_retries_mixed_timeout_and_application_errors(self): import elasticsearch connection_error = elasticsearch.ConnectionError("N/A", "no route to host") failed_return_value = {"weight": 1, "unit": "ops", "success": False} success_return_value = {"weight": 1, "unit": "ops", "success": False} delegate = mock.Mock(side_effect=[ - connection_error, - failed_return_value, - connection_error, - connection_error, - failed_return_value, - success_return_value] - ) + as_future(exception=connection_error), + as_future(failed_return_value), + as_future(exception=connection_error), + as_future(exception=connection_error), + as_future(failed_return_value), + as_future(success_return_value) + ]) es = None params = { # we try exactly as often as there are errors to also test the semantics of "retry". @@ -2727,7 +2824,7 @@ def test_retries_mixed_timeout_and_application_errors(self): } retrier = runner.Retry(delegate) - result = retrier(es, params) + result = await retrier(es, params) self.assertEqual(success_return_value, result) delegate.assert_has_calls([ @@ -2745,10 +2842,11 @@ def test_retries_mixed_timeout_and_application_errors(self): mock.call(es, params) ]) - def test_does_not_retry_on_timeout_if_not_wanted(self): + @run_async + async def test_does_not_retry_on_timeout_if_not_wanted(self): import elasticsearch - delegate = mock.Mock(side_effect=elasticsearch.ConnectionTimeout(408, "timed out")) + delegate = mock.Mock(side_effect=as_future(exception=elasticsearch.ConnectionTimeout(408, "timed out"))) es = None params = { "retries": 3, @@ -2759,15 +2857,19 @@ def test_does_not_retry_on_timeout_if_not_wanted(self): retrier = runner.Retry(delegate) with self.assertRaises(elasticsearch.ConnectionTimeout): - retrier(es, params) + await retrier(es, params) delegate.assert_called_once_with(es, params) - def test_retries_on_application_error_if_wanted(self): + @run_async + async def test_retries_on_application_error_if_wanted(self): failed_return_value = {"weight": 1, "unit": "ops", "success": False} success_return_value = {"weight": 1, "unit": "ops", "success": True} - delegate = mock.Mock(side_effect=[failed_return_value, success_return_value]) + delegate = mock.Mock(side_effect=[ + as_future(failed_return_value), + as_future(success_return_value) + ]) es = None params = { "retries": 3, @@ -2777,7 +2879,7 @@ def test_retries_on_application_error_if_wanted(self): } retrier = runner.Retry(delegate) - result = retrier(es, params) + result = await retrier(es, params) self.assertEqual(success_return_value, result) @@ -2787,10 +2889,11 @@ def test_retries_on_application_error_if_wanted(self): mock.call(es, params) ]) - def test_does_not_retry_on_application_error_if_not_wanted(self): + @run_async + async def test_does_not_retry_on_application_error_if_not_wanted(self): failed_return_value = {"weight": 1, "unit": "ops", "success": False} - delegate = mock.Mock(return_value=failed_return_value) + delegate = mock.Mock(return_value=as_future(failed_return_value)) es = None params = { "retries": 3, @@ -2800,14 +2903,15 @@ def test_does_not_retry_on_application_error_if_not_wanted(self): } retrier = runner.Retry(delegate) - result = retrier(es, params) + result = await retrier(es, params) self.assertEqual(failed_return_value, result) delegate.assert_called_once_with(es, params) - def test_assumes_success_if_runner_returns_non_dict(self): - delegate = mock.Mock(return_value=(1, "ops")) + @run_async + async def test_assumes_success_if_runner_returns_non_dict(self): + delegate = mock.Mock(return_value=as_future(result=(1, "ops"))) es = None params = { "retries": 3, @@ -2817,21 +2921,22 @@ def test_assumes_success_if_runner_returns_non_dict(self): } retrier = runner.Retry(delegate) - result = retrier(es, params) + result = await retrier(es, params) self.assertEqual((1, "ops"), result) delegate.assert_called_once_with(es, params) - def test_retries_until_success(self): + @run_async + async def test_retries_until_success(self): failure_count = 5 failed_return_value = {"weight": 1, "unit": "ops", "success": False} success_return_value = {"weight": 1, "unit": "ops", "success": True} responses = [] - responses += failure_count * [failed_return_value] - responses += [success_return_value] + responses += failure_count * [as_future(failed_return_value)] + responses += [as_future(success_return_value)] delegate = mock.Mock(side_effect=responses) es = None @@ -2841,7 +2946,7 @@ def test_retries_until_success(self): } retrier = runner.Retry(delegate) - result = retrier(es, params) + result = await retrier(es, params) self.assertEqual(success_return_value, result) From 65925e7b6e1bc432ed50cde1d82f5a07f352aefe Mon Sep 17 00:00:00 2001 From: Daniel Mitterdorfer Date: Thu, 20 Feb 2020 11:01:37 +0100 Subject: [PATCH 11/24] Support completion of parallel task structures in async mode --- esrally/driver/async_driver.py | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/esrally/driver/async_driver.py b/esrally/driver/async_driver.py index 662d1b30e..e7e0f2c72 100644 --- a/esrally/driver/async_driver.py +++ b/esrally/driver/async_driver.py @@ -277,9 +277,6 @@ async def run_benchmark(self): es = self.create_es_clients(sync=False) try: cancel = threading.Event() - # used to indicate that we want to prematurely consider this completed. This is *not* due to cancellation but a regular event in - # a benchmark and used to model task dependency of parallel tasks. - complete = threading.Event() # allow to buffer more events than by default as we expect to have way more clients. self.sampler = driver.Sampler(start_timestamp=time.perf_counter(), buffer_size=65536) @@ -289,11 +286,11 @@ async def run_benchmark(self): for sub_task in task: self.current_tasks.append(sub_task) self.logger.info("Running task [%s] with [%d] clients...", sub_task.name, sub_task.clients) - # TODO: This is lacking support for one (sub)task being able to complete a complete parallel - # structure. We can probably achieve that by waiting for the task in question and then - # cancelling all other ongoing clients. for client_id in range(sub_task.clients): schedule = driver.schedule_for(self.track, sub_task, client_id) + # used to indicate that we want to prematurely consider this completed. This is *not* due to + # cancellation but a regular event in a benchmark and used to model task dependency of parallel tasks. + complete = threading.Event() e = driver.AsyncExecutor(client_id, sub_task, schedule, es, self.sampler, cancel, complete, self.abort_on_error) aws.append(e()) # join point From 440c5a32eb23dcd2a2fbe0674ad5fd40dabc3b8a Mon Sep 17 00:00:00 2001 From: Daniel Mitterdorfer Date: Fri, 21 Feb 2020 07:27:26 +0100 Subject: [PATCH 12/24] Expose meta-data that async-runner is required --- esrally/track/loader.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/esrally/track/loader.py b/esrally/track/loader.py index c746f71ca..6a2f4b4b6 100644 --- a/esrally/track/loader.py +++ b/esrally/track/loader.py @@ -950,7 +950,8 @@ def meta_data(self): from esrally import version return { - "rally_version": version.release_version() + "rally_version": version.release_version(), + "async_runner": True } From 7568faee861518d2798884661443e6b9f49f69b1 Mon Sep 17 00:00:00 2001 From: Daniel Mitterdorfer Date: Fri, 21 Feb 2020 10:15:41 +0100 Subject: [PATCH 13/24] async fallback for queries --- esrally/driver/runner.py | 19 ++++++++----------- 1 file changed, 8 insertions(+), 11 deletions(-) diff --git a/esrally/driver/runner.py b/esrally/driver/runner.py index a148d909c..cb5fbf09d 100644 --- a/esrally/driver/runner.py +++ b/esrally/driver/runner.py @@ -631,15 +631,6 @@ def __repr__(self, *args, **kwargs): return "node-stats" -def search_type_fallback(es, doc_type, index, body, params): - if doc_type and not index: - index = "_all" - path = "/%s/%s/_search" % (index, doc_type) - return es.transport.perform_request( - "GET", path, params=params, body=body - ) - - class Query(Runner): """ Runs a request body search against Elasticsearch. @@ -693,7 +684,7 @@ async def request_body_query(self, es, params): params = request_params if doc_type is not None: - r = await search_type_fallback(es, doc_type, index, body, params) + r = await self._search_type_fallback(es, doc_type, index, body, params) else: r = await es.search(index=index, body=body, params=params) hits = r["hits"]["total"] @@ -735,7 +726,7 @@ async def scroll_query(self, es, params): params["sort"] = sort params["scroll"] = scroll params["size"] = size - r = search_type_fallback(es, doc_type, index, body, params) + r = await self._search_type_fallback(es, doc_type, index, body, params) else: r = await es.search(index=index, body=body, params=params, sort=sort, scroll=scroll, size=size) # This should only happen if we concurrently create an index and start searching @@ -762,6 +753,12 @@ async def scroll_query(self, es, params): "took": took } + async def _search_type_fallback(self, es, doc_type, index, body, params): + if doc_type and not index: + index = "_all" + path = "/%s/%s/_search" % (index, doc_type) + return await es.transport.perform_request("GET", path, params=params, body=body) + def _default_request_params(self, params): request_params = params.get("request-params", {}) cache = params.get("cache") From d1023c64386a0392bef17bc6ed871b88b56bb68b Mon Sep 17 00:00:00 2001 From: Daniel Mitterdorfer Date: Fri, 21 Feb 2020 10:16:03 +0100 Subject: [PATCH 14/24] Properly integrate async driver with racecontrol --- esrally/driver/__init__.py | 2 +- esrally/driver/async_driver.py | 104 ++++--------------- esrally/driver/driver.py | 4 + esrally/racecontrol.py | 181 ++++++++++++++++++++------------- 4 files changed, 132 insertions(+), 159 deletions(-) diff --git a/esrally/driver/__init__.py b/esrally/driver/__init__.py index 00da31a1c..0a5fa2a65 100644 --- a/esrally/driver/__init__.py +++ b/esrally/driver/__init__.py @@ -19,4 +19,4 @@ from .driver import DriverActor, PrepareBenchmark, PreparationComplete, StartBenchmark, BenchmarkComplete, TaskFinished # async API -from .async_driver import race +from .async_driver import AsyncDriver diff --git a/esrally/driver/async_driver.py b/esrally/driver/async_driver.py index e7e0f2c72..a1a209e8e 100644 --- a/esrally/driver/async_driver.py +++ b/esrally/driver/async_driver.py @@ -22,72 +22,11 @@ import threading import time -from esrally import exceptions, metrics, track, client, PROGRAM_NAME, telemetry +from esrally import exceptions, metrics, track, client, telemetry from esrally.driver import driver, runner, scheduler from esrally.utils import console -# TODO: Inline this code later -class PseudoActor: - def __init__(self, cfg, current_race): - self.cfg = cfg - self.race = current_race - - def on_cluster_details_retrieved(self, cluster_details): - #self.cluster_details = cluster_details - pass - - def on_benchmark_complete(self, metrics_store): - # TODO: Should we do this in race control instead? - from esrally import reporter - final_results = metrics.calculate_results(metrics_store, self.race) - metrics.results_store(self.cfg).store_results(self.race) - reporter.summarize(final_results, self.cfg) - - -def race(cfg): - logger = logging.getLogger(__name__) - # TODO: Taken from BenchmarkActor#setup() - t = track.load_track(cfg) - track_revision = cfg.opts("track", "repository.revision", mandatory=False) - challenge_name = cfg.opts("track", "challenge.name") - challenge = t.find_challenge_or_default(challenge_name) - if challenge is None: - raise exceptions.SystemSetupError("Track [%s] does not provide challenge [%s]. List the available tracks with %s list tracks." - % (t.name, challenge_name, PROGRAM_NAME)) - if challenge.user_info: - console.info(challenge.user_info) - current_race = metrics.create_race(cfg, t, challenge, track_revision) - - metrics_store = metrics.metrics_store( - cfg, - track=current_race.track_name, - challenge=current_race.challenge_name, - read_only=False - ) - race_store = metrics.race_store(cfg) - - a = PseudoActor(cfg, current_race) - - d = AsyncDriver(a, cfg) - logger.info("Preparing benchmark...") - cluster_info = d.prepare_benchmark(t) - # TODO: ensure we execute the code in after_track_prepared (from the original actor) - # def after_track_prepared(self): - # cluster_version = self.cluster_details["version"] if self.cluster_details else {} - # for child in self.children: - # self.send(child, thespian.actors.ActorExitRequest()) - # self.children = [] - # self.send(self.start_sender, driver.PreparationComplete( - # # older versions (pre 6.3.0) don't expose build_flavor because the only (implicit) flavor was "oss" - # cluster_version.get("build_flavor", "oss"), - # cluster_version.get("number", "Unknown"), - # cluster_version.get("build_hash", "Unknown") - # )) - logger.info("Running benchmark...") - d.start_benchmark() - - # TODO: Move to time.py class Timer: def __init__(self, fn, interval, stop_event): @@ -108,17 +47,15 @@ def __call__(self, *args, **kwargs): class AsyncDriver: - def __init__(self, target, config, es_client_factory_class=client.EsClientFactory): + def __init__(self, config, es_client_factory_class=client.EsClientFactory): """ Coordinates all workers. It is technology-agnostic, i.e. it does not know anything about actors. To allow us to hook in an actor, we provide a ``target`` parameter which will be called whenever some event has occurred. The ``target`` can use this to send appropriate messages. - :param target: A target that will be notified of important events. :param config: The current config object. """ self.logger = logging.getLogger(__name__) - self.target = target self.config = config self.es_client_factory = es_client_factory_class self.track = None @@ -206,11 +143,12 @@ def retrieve_cluster_info(self): self.logger.exception("Could not retrieve cluster info on benchmark start") return None - def prepare_benchmark(self, t): + def setup(self, t): self.track = t self.challenge = driver.select_challenge(self.config, self.track) self.quiet = self.config.opts("system", "quiet.mode", mandatory=False, default_value=False) self.throughput_calculator = driver.ThroughputCalculator() + self.metrics_store = metrics.metrics_store(cfg=self.config, track=self.track.name, challenge=self.challenge.name, @@ -225,9 +163,11 @@ def prepare_benchmark(self, t): track.load_track_plugins(self.config, runner.register_runner, scheduler.register_scheduler) track.prepare_track(self.track, self.config) - return self.retrieve_cluster_info() + cluster_info = self.retrieve_cluster_info() + cluster_version = cluster_info["version"] if cluster_info else {} + return cluster_version.get("build_flavor", "oss"), cluster_version.get("number"), cluster_version.get("build_hash") - def start_benchmark(self): + def run(self): self.logger.info("Benchmark is about to start.") # ensure relative time starts when the benchmark starts. self.reset_relative_time() @@ -244,35 +184,35 @@ def start_benchmark(self): # TODO: Make this configurable? loop.set_debug(True) asyncio.set_event_loop(loop) - loop.set_exception_handler(debug_exception_handler) + loop.set_exception_handler(self._logging_exception_handler) track.set_absolute_data_path(self.config, self.track) runner.register_default_runners() - # TODO: We can skip this here if we run in the same process; it has already been done in #prepare_benchmark() + # TODO: We can skip this here if we run in the same process; it has already been done in #setup() if self.track.has_plugins: track.load_track_plugins(self.config, runner.register_runner, scheduler.register_scheduler) - success = False try: - benchmark_runner = driver.AsyncProfiler(self.run_benchmark) if self.profiling_enabled else self.run_benchmark + benchmark_runner = driver.AsyncProfiler(self._run_benchmark) if self.profiling_enabled else self._run_benchmark loop.run_until_complete(benchmark_runner()) self._finished = True self.telemetry.on_benchmark_stop() self.logger.info("All steps completed.") - success = True + return self.metrics_store.to_externalizable() finally: self.stop_timer_tasks.set() self.pool.shutdown() loop.close() self.progress_reporter.finish() - if success: - self.target.on_benchmark_complete(self.metrics_store) self.logger.debug("Closing metrics store...") self.metrics_store.close() # immediately clear as we don't need it anymore and it can consume a significant amount of memory self.metrics_store = None - async def run_benchmark(self): + def _logging_exception_handler(self, loop, context): + self.logger.error("Uncaught exception in event loop: %s", context) + + async def _run_benchmark(self): # avoid: aiohttp.internal WARNING The object should be created from async function es = self.create_es_clients(sync=False) try: @@ -303,17 +243,13 @@ async def run_benchmark(self): self.update_progress_message(task_finished=True) finally: await asyncio.get_event_loop().shutdown_asyncgens() - await es["default"].transport.close() + for e in es.values(): + await e.transport.close() def reset_relative_time(self): self.logger.debug("Resetting relative time of request metrics store.") self.metrics_store.reset_relative_time() - def close(self): - self.progress_reporter.finish() - if self.metrics_store and self.metrics_store.opened: - self.metrics_store.close() - def update_samples(self): if self.sampler: samples = self.sampler.samples @@ -412,7 +348,3 @@ def merge(self, *args): if arg is not None: result.update(arg) return result - - -def debug_exception_handler(loop, context): - logging.getLogger(__name__).error("Uncaught exception in event loop!! %s", context) diff --git a/esrally/driver/driver.py b/esrally/driver/driver.py index 23d75ea0f..f4872a21b 100644 --- a/esrally/driver/driver.py +++ b/esrally/driver/driver.py @@ -1037,12 +1037,16 @@ def __call__(self, *args, **kwargs): # asyncio.set_event_loop(loop) loop = asyncio.new_event_loop() loop.set_debug(True) + loop.set_exception_handler(self._logging_exception_handler) asyncio.set_event_loop(loop) try: loop.run_until_complete(self.run()) finally: loop.close() + def _logging_exception_handler(self, loop, context): + logging.getLogger(__name__).error("Uncaught exception in event loop: %s", context) + async def run(self): def es_clients(all_hosts, all_client_options): es = {} diff --git a/esrally/racecontrol.py b/esrally/racecontrol.py index 7697389b0..fba57f061 100644 --- a/esrally/racecontrol.py +++ b/esrally/racecontrol.py @@ -97,19 +97,15 @@ class BenchmarkActor(actor.RallyActor): def __init__(self): super().__init__() self.cfg = None - self.race = None - self.metrics_store = None - self.race_store = None - self.cancelled = False - self.error = False self.start_sender = None self.mechanic = None self.main_driver = None - self.track_revision = None + self.coordinator = None def receiveMsg_PoisonMessage(self, msg, sender): self.logger.info("BenchmarkActor got notified of poison message [%s] (forwarding).", (str(msg))) - self.error = True + if self.coordinator: + self.coordinator.error = True self.send(self.start_sender, msg) def receiveUnrecognizedMessage(self, msg, sender): @@ -117,43 +113,46 @@ def receiveUnrecognizedMessage(self, msg, sender): @actor.no_retry("race control") def receiveMsg_Setup(self, msg, sender): - self.setup(msg, sender) + self.start_sender = sender + self.cfg = msg.cfg + self.coordinator = BenchmarkCoordinator(msg.cfg) + self.coordinator.setup(sources=msg.sources) + self.logger.info("Asking mechanic to start the engine.") + cluster_settings = self.coordinator.current_challenge.cluster_settings + self.mechanic = self.createActor(mechanic.MechanicActor, targetActorRequirements={"coordinator": True}) + self.send(self.mechanic, mechanic.StartEngine(self.cfg, + self.coordinator.metrics_store.open_context, + cluster_settings, + msg.sources, + msg.build, + msg.distribution, + msg.external, + msg.docker)) @actor.no_retry("race control") def receiveMsg_EngineStarted(self, msg, sender): self.logger.info("Mechanic has started engine successfully.") - self.race.team_revision = msg.team_revision + self.coordinator.race.team_revision = msg.team_revision self.main_driver = self.createActor(driver.DriverActor, targetActorRequirements={"coordinator": True}) self.logger.info("Telling driver to prepare for benchmarking.") - self.send(self.main_driver, driver.PrepareBenchmark(self.cfg, self.race.track)) + self.send(self.main_driver, driver.PrepareBenchmark(self.cfg, self.coordinator.current_track)) @actor.no_retry("race control") def receiveMsg_PreparationComplete(self, msg, sender): - self.race.distribution_flavor = msg.distribution_flavor - self.race.distribution_version = msg.distribution_version - self.race.revision = msg.revision - # store race initially (without any results) so other components can retrieve full metadata - self.race_store.store_race(self.race) - if self.race.challenge.auto_generated: - console.info("Racing on track [{}] and car {} with version [{}].\n" - .format(self.race.track_name, self.race.car, self.race.distribution_version)) - else: - console.info("Racing on track [{}], challenge [{}] and car {} with version [{}].\n" - .format(self.race.track_name, self.race.challenge_name, self.race.car, self.race.distribution_version)) - self.run() + self.coordinator.on_preparation_complete(msg.distribution_flavor, msg.distribution_version, msg.revision) + self.logger.info("Telling driver to start benchmark.") + self.send(self.main_driver, driver.StartBenchmark()) @actor.no_retry("race control") def receiveMsg_TaskFinished(self, msg, sender): - self.logger.info("Task has finished.") - self.logger.info("Bulk adding request metrics to metrics store.") - self.metrics_store.bulk_add(msg.metrics) + self.coordinator.on_task_finished(msg.metrics) # We choose *NOT* to reset our own metrics store's timer as this one is only used to collect complete metrics records from # other stores (used by driver and mechanic). Hence there is no need to reset the timer in our own metrics store. self.send(self.mechanic, mechanic.ResetRelativeTime(msg.next_task_scheduled_in)) @actor.no_retry("race control") def receiveMsg_BenchmarkCancelled(self, msg, sender): - self.cancelled = True + self.coordinator.cancelled = True # even notify the start sender if it is the originator. The reason is that we call #ask() which waits for a reply. # We also need to ask in order to avoid races between this notification and the following ActorExitRequest. self.send(self.start_sender, msg) @@ -161,53 +160,56 @@ def receiveMsg_BenchmarkCancelled(self, msg, sender): @actor.no_retry("race control") def receiveMsg_BenchmarkFailure(self, msg, sender): self.logger.info("Received a benchmark failure from [%s] and will forward it now.", sender) - self.error = True + self.coordinator.error = True self.send(self.start_sender, msg) @actor.no_retry("race control") def receiveMsg_BenchmarkComplete(self, msg, sender): - self.logger.info("Benchmark is complete.") - self.logger.info("Bulk adding request metrics to metrics store.") - self.metrics_store.bulk_add(msg.metrics) - self.metrics_store.flush() - if not self.cancelled and not self.error: - final_results = metrics.calculate_results(self.metrics_store, self.race) - self.race.add_results(final_results) - self.race_store.store_race(self.race) - metrics.results_store(self.cfg).store_results(self.race) - reporter.summarize(final_results, self.cfg) - else: - self.logger.info("Suppressing output of summary report. Cancelled = [%r], Error = [%r].", self.cancelled, self.error) - self.metrics_store.close() - - self.teardown() + self.coordinator.on_benchmark_complete(msg.metrics) + self.send(self.main_driver, thespian.actors.ActorExitRequest()) + self.main_driver = None + self.logger.info("Asking mechanic to stop the engine.") + self.send(self.mechanic, mechanic.StopEngine()) @actor.no_retry("race control") def receiveMsg_EngineStopped(self, msg, sender): self.logger.info("Mechanic has stopped engine successfully.") self.send(self.start_sender, Success()) - def setup(self, msg, sender): - self.start_sender = sender - self.cfg = msg.cfg - # to load the track we need to know the correct cluster distribution version. Usually, this value should be set but there are rare - # cases (external pipeline and user did not specify the distribution version) where we need to derive it ourselves. For source - # builds we always assume "master" - if not msg.sources and not self.cfg.exists("mechanic", "distribution.version"): + +class BenchmarkCoordinator: + def __init__(self, cfg): + self.logger = logging.getLogger(__name__) + self.cfg = cfg + self.race = None + self.metrics_store = None + self.race_store = None + self.cancelled = False + self.error = False + self.track_revision = None + self.current_track = None + self.current_challenge = None + + def setup(self, sources=False): + # to load the track we need to know the correct cluster distribution version. Usually, this value should be set + # but there are rare cases (external pipeline and user did not specify the distribution version) where we need + # to derive it ourselves. For source builds we always assume "master" + if not sources and not self.cfg.exists("mechanic", "distribution.version"): distribution_version = mechanic.cluster_distribution_version(self.cfg) self.logger.info("Automatically derived distribution version [%s]", distribution_version) self.cfg.add(config.Scope.benchmark, "mechanic", "distribution.version", distribution_version) - t = track.load_track(self.cfg) + self.current_track = track.load_track(self.cfg) self.track_revision = self.cfg.opts("track", "repository.revision", mandatory=False) challenge_name = self.cfg.opts("track", "challenge.name") - challenge = t.find_challenge_or_default(challenge_name) - if challenge is None: - raise exceptions.SystemSetupError("Track [%s] does not provide challenge [%s]. List the available tracks with %s list tracks." - % (t.name, challenge_name, PROGRAM_NAME)) - if challenge.user_info: - console.info(challenge.user_info) - self.race = metrics.create_race(self.cfg, t, challenge, self.track_revision) + self.current_challenge = self.current_track.find_challenge_or_default(challenge_name) + if self.current_challenge is None: + raise exceptions.SystemSetupError( + "Track [{}] does not provide challenge [{}]. List the available tracks with {} list tracks.".format( + self.current_track.name, challenge_name, PROGRAM_NAME)) + if self.current_challenge.user_info: + console.info(self.current_challenge.user_info) + self.race = metrics.create_race(self.cfg, self.current_track, self.current_challenge, self.track_revision) self.metrics_store = metrics.metrics_store( self.cfg, @@ -216,21 +218,39 @@ def setup(self, msg, sender): read_only=False ) self.race_store = metrics.race_store(self.cfg) - self.logger.info("Asking mechanic to start the engine.") - cluster_settings = challenge.cluster_settings - self.mechanic = self.createActor(mechanic.MechanicActor, targetActorRequirements={"coordinator": True}) - self.send(self.mechanic, mechanic.StartEngine(self.cfg, self.metrics_store.open_context, cluster_settings, msg.sources, msg.build, - msg.distribution, msg.external, msg.docker)) - def run(self): - self.logger.info("Telling driver to start benchmark.") - self.send(self.main_driver, driver.StartBenchmark()) + def on_preparation_complete(self, distribution_flavor, distribution_version, revision): + self.race.distribution_flavor = distribution_flavor + self.race.distribution_version = distribution_version + self.race.revision = revision + # store race initially (without any results) so other components can retrieve full metadata + self.race_store.store_race(self.race) + if self.race.challenge.auto_generated: + console.info("Racing on track [{}] and car {} with version [{}].\n" + .format(self.race.track_name, self.race.car, self.race.distribution_version)) + else: + console.info("Racing on track [{}], challenge [{}] and car {} with version [{}].\n" + .format(self.race.track_name, self.race.challenge_name, self.race.car, self.race.distribution_version)) - def teardown(self): - self.send(self.main_driver, thespian.actors.ActorExitRequest()) - self.main_driver = None - self.logger.info("Asking mechanic to stop the engine.") - self.send(self.mechanic, mechanic.StopEngine()) + def on_task_finished(self, new_metrics): + self.logger.info("Task has finished.") + self.logger.info("Bulk adding request metrics to metrics store.") + self.metrics_store.bulk_add(new_metrics) + + def on_benchmark_complete(self, new_metrics): + self.logger.info("Benchmark is complete.") + self.logger.info("Bulk adding request metrics to metrics store.") + self.metrics_store.bulk_add(new_metrics) + self.metrics_store.flush() + if not self.cancelled and not self.error: + final_results = metrics.calculate_results(self.metrics_store, self.race) + self.race.add_results(final_results) + self.race_store.store_race(self.race) + metrics.results_store(self.cfg).store_results(self.race) + reporter.summarize(final_results, self.cfg) + else: + self.logger.info("Suppressing output of summary report. Cancelled = [%r], Error = [%r].", self.cancelled, self.error) + self.metrics_store.close() def race(cfg, sources=False, build=False, distribution=False, external=False, docker=False): @@ -371,5 +391,22 @@ def run(cfg): def run_async(cfg): - # TODO: Provide everything that the BenchmarkActor provides - driver.race(cfg) + console.warn("The race-async command is experimental and thus subject to change.") + logger = logging.getLogger(__name__) + # We'll use a special car name for external benchmarks. + cfg.add(config.Scope.benchmark, "mechanic", "car.names", ["external"]) + coordinator = BenchmarkCoordinator(cfg) + race_driver = driver.AsyncDriver(cfg) + + try: + coordinator.setup() + distribution_flavor, distribution_version, revision = race_driver.setup(coordinator.current_track) + coordinator.on_preparation_complete(distribution_flavor, distribution_version, revision) + + new_metrics = race_driver.run() + coordinator.on_benchmark_complete(new_metrics) + except KeyboardInterrupt: + logger.info("User has cancelled the benchmark.") + except BaseException: + tb = sys.exc_info()[2] + raise exceptions.RallyError("This race ended with a fatal crash.").with_traceback(tb) From 0c9ddd9f614b3e5607328b3a59d6dff42a8e3335 Mon Sep 17 00:00:00 2001 From: Daniel Mitterdorfer Date: Fri, 21 Feb 2020 15:43:14 +0100 Subject: [PATCH 15/24] Temporarily use elasticsearch-py-async master --- create-notice.sh | 6 ++++++ setup.py | 10 +++++++++- 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/create-notice.sh b/create-notice.sh index f21519752..d4d70aed6 100755 --- a/create-notice.sh +++ b/create-notice.sh @@ -43,6 +43,7 @@ function main { printf "The source code can be obtained at https://github.com/certifi/python-certifi\n" >> "${OUTPUT_FILE}" add_license "certifi" "https://raw.githubusercontent.com/certifi/python-certifi/master/LICENSE" add_license "elasticsearch" "https://raw.githubusercontent.com/elastic/elasticsearch-py/master/LICENSE" + add_license "elasticsearch-async" "https://raw.githubusercontent.com/elastic/elasticsearch-py-async/master/LICENSE" add_license "jinja2" "https://raw.githubusercontent.com/pallets/jinja/master/LICENSE.rst" add_license "jsonschema" "https://raw.githubusercontent.com/Julian/jsonschema/master/COPYING" add_license "psutil" "https://raw.githubusercontent.com/giampaolo/psutil/master/LICENSE" @@ -50,12 +51,17 @@ function main { add_license "tabulate" "https://bitbucket.org/astanin/python-tabulate/raw/03182bf9b8a2becbc54d17aa7e3e7dfed072c5f5/LICENSE" add_license "thespian" "https://raw.githubusercontent.com/kquick/Thespian/master/LICENSE.txt" add_license "boto3" "https://raw.githubusercontent.com/boto/boto3/develop/LICENSE" + add_license "yappi" "https://raw.githubusercontent.com/sumerc/yappi/master/LICENSE" # transitive dependencies # Jinja2 -> Markupsafe add_license "Markupsafe" "https://raw.githubusercontent.com/pallets/markupsafe/master/LICENSE.rst" # elasticsearch -> urllib3 add_license "urllib3" "https://raw.githubusercontent.com/shazow/urllib3/master/LICENSE.txt" + #elasticsearch_async -> aiohttp + add_license "aiohttp" "https://raw.githubusercontent.com/aio-libs/aiohttp/master/LICENSE.txt" + #elasticsearch_async -> async_timeout + add_license "async_timeout" "https://raw.githubusercontent.com/aio-libs/async-timeout/master/LICENSE" # boto3 -> s3transfer add_license "s3transfer" "https://raw.githubusercontent.com/boto/s3transfer/develop/LICENSE.txt" # boto3 -> jmespath diff --git a/setup.py b/setup.py index 7d199642e..56aca299d 100644 --- a/setup.py +++ b/setup.py @@ -50,7 +50,14 @@ def str_from_file(name): # License: Apache 2.0 # transitive dependency urllib3: MIT "elasticsearch==7.0.5", - "elasticsearch-async==6.2.0", + # License: Apache 2.0 + # transitive dependencies: + # aiohttp: Apache 2.0 + # async_timeout: Apache 2.0 + # 6.2.0 does not include https://github.com/elastic/elasticsearch-py-async/pull/49 which (also) causes + # the behavior reported in https://github.com/elastic/elasticsearch-py-async/issues/43 + # "elasticsearch-async==6.2.0", + "elasticsearch-async@git+https://github.com/elastic/elasticsearch-py-async.git", # License: BSD "psutil==5.6.5", # License: MIT @@ -75,6 +82,7 @@ def str_from_file(name): # jmespath: MIT # s3transfer: Apache 2.0 "boto3==1.10.32", + # License: Apache 2.0 "yappi==1.2.3" ] From 3cc3861fb607c5050dba08dda00f0c167aef71f3 Mon Sep 17 00:00:00 2001 From: Daniel Mitterdorfer Date: Mon, 24 Feb 2020 07:01:40 +0100 Subject: [PATCH 16/24] Properly shutdown async components in adapter layer --- esrally/driver/driver.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/esrally/driver/driver.py b/esrally/driver/driver.py index f4872a21b..b094dc69e 100644 --- a/esrally/driver/driver.py +++ b/esrally/driver/driver.py @@ -1058,7 +1058,12 @@ def es_clients(all_hosts, all_client_options): async_executor = AsyncExecutor( self.client_id, self.sub_task, self.schedule, es, self.sampler, self.cancel, self.complete, self.abort_on_error) final_executor = AsyncProfiler(async_executor) if self.profiling_enabled else async_executor - return await final_executor() + try: + return await final_executor() + finally: + await asyncio.get_event_loop().shutdown_asyncgens() + for e in es.values(): + await e.transport.close() class AsyncProfiler: From 4e2c951154a08baf958acc06f8da4dc68bc88410 Mon Sep 17 00:00:00 2001 From: Daniel Mitterdorfer Date: Mon, 24 Feb 2020 07:31:20 +0100 Subject: [PATCH 17/24] Expose kwargs in runner registry --- esrally/track/loader.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/esrally/track/loader.py b/esrally/track/loader.py index 6a2f4b4b6..5817c3bd7 100644 --- a/esrally/track/loader.py +++ b/esrally/track/loader.py @@ -939,8 +939,8 @@ def load(self): def register_param_source(self, name, param_source): params.register_param_source_for_name(name, param_source) - def register_runner(self, name, runner): - self.runner_registry(name, runner) + def register_runner(self, name, runner, **kwargs): + self.runner_registry(name, runner, **kwargs) def register_scheduler(self, name, scheduler): self.scheduler_registry(name, scheduler) From 78a4fc294315344f4d662f9f3eefd4a0ae4ec11a Mon Sep 17 00:00:00 2001 From: Daniel Mitterdorfer Date: Mon, 24 Feb 2020 08:06:47 +0100 Subject: [PATCH 18/24] Add docs --- docs/adding_tracks.rst | 46 +++++++++++++++++++++++++++--------------- docs/migrate.rst | 41 +++++++++++++++++++++++++++++++++++++ 2 files changed, 71 insertions(+), 16 deletions(-) diff --git a/docs/adding_tracks.rst b/docs/adding_tracks.rst index d00d2676f..8ab263dfd 100644 --- a/docs/adding_tracks.rst +++ b/docs/adding_tracks.rst @@ -881,17 +881,15 @@ In ``track.json`` set the ``operation-type`` to "percolate" (you can choose this Then create a file ``track.py`` next to ``track.json`` and implement the following two functions:: - def percolate(es, params): - es.percolate( - index="queries", - doc_type="content", - body=params["body"] - ) - + async def percolate(es, params): + await es.percolate( + index="queries", + doc_type="content", + body=params["body"] + ) def register(registry): - registry.register_runner("percolate", percolate) - + registry.register_runner("percolate", percolate, async_runner=True) The function ``percolate`` is the actual runner and takes the following parameters: @@ -906,11 +904,25 @@ This function can return: Similar to a parameter source you also need to bind the name of your operation type to the function within ``register``. +To illustrate how to use custom return values, suppose we want to implement a custom runner that calls the `pending tasks API `_ and returns the number of pending tasks as additional meta-data:: + + async def pending_tasks(es, params): + response = await es.cluster.pending_tasks() + return { + "weight": 1, + "unit": "ops", + "pending-tasks-count": len(response["tasks"]) + } + + def register(registry): + registry.register_runner("pending-tasks", pending_tasks, async_runner=True) + + If you need more control, you can also implement a runner class. The example above, implemented as a class looks as follows:: class PercolateRunner: - def __call__(self, es, params): - es.percolate( + async def __call__(self, es, params): + await es.percolate( index="queries", doc_type="content", body=params["body"] @@ -920,10 +932,12 @@ If you need more control, you can also implement a runner class. The example abo return "percolate" def register(registry): - registry.register_runner("percolate", PercolateRunner()) + registry.register_runner("percolate", PercolateRunner(), async_runner=True) + +The actual runner is implemented in the method ``__call__`` and the same return value conventions apply as for functions. For debugging purposes you should also implement ``__repr__`` and provide a human-readable name for your runner. Finally, you need to register your runner in the ``register`` function. -The actual runner is implemented in the method ``__call__`` and the same return value conventions apply as for functions. For debugging purposes you should also implement ``__repr__`` and provide a human-readable name for your runner. Finally, you need to register your runner in the ``register`` function. Runners also support Python's `context manager `_ interface. Rally uses a new context for each request. Implementing the context manager interface can be handy for cleanup of resources after executing an operation. Rally uses it, for example, to clear open scrolls. +Runners also support Python's `asynchronous context manager `_ interface. Rally uses a new context for each request. Implementing the asynchronous context manager interface can be handy for cleanup of resources after executing an operation. Rally uses it, for example, to clear open scrolls. If you have specified multiple Elasticsearch clusters using :ref:`target-hosts ` you can make Rally pass a dictionary of client connections instead of one for the ``default`` cluster in the ``es`` parameter. @@ -938,14 +952,14 @@ Example (assuming Rally has been invoked specifying ``default`` and ``remote`` i class CreateIndexInRemoteCluster: multi_cluster = True - def __call__(self, es, params): - es['remote'].indices.create(index='remote-index') + async def __call__(self, es, params): + await es["remote"].indices.create(index="remote-index") def __repr__(self, *args, **kwargs): return "create-index-in-remote-cluster" def register(registry): - registry.register_runner("create-index-in-remote-cluster", CreateIndexInRemoteCluster()) + registry.register_runner("create-index-in-remote-cluster", CreateIndexInRemoteCluster(), async_runner=True) .. note:: diff --git a/docs/migrate.rst b/docs/migrate.rst index 7e8de6fc3..dd3019cb6 100644 --- a/docs/migrate.rst +++ b/docs/migrate.rst @@ -1,6 +1,47 @@ Migration Guide =============== +Migrating to Rally 1.5.0 +------------------------ + +Runner API uses asyncio +^^^^^^^^^^^^^^^^^^^^^^^ + +In order to support more concurrent clients in the future, Rally is moving from a synchronous model to an asynchronous model internally. With Rally 1.5.0 all custom runners need to be implemented using async APIs and a new bool argument ``async_runner=True`` needs to be provided upon registration. Below is an example how to migrate a custom runner function. + +A custom runner prior to Rally 1.5.0:: + + def percolate(es, params): + es.percolate( + index="queries", + doc_type="content", + body=params["body"] + ) + + def register(registry): + registry.register_runner("percolate", percolate) + +With Rally 1.5.0, the implementation changes as follows:: + + async def percolate(es, params): + await es.percolate( + index="queries", + doc_type="content", + body=params["body"] + ) + + def register(registry): + registry.register_runner("percolate", percolate, async_runner=True) + +Apply to the following changes for each custom runner: + +* Prefix the function signature with ``async``. +* Add an ``await`` keyword before each Elasticsearch API call. +* Add ``async_runner=True`` as the last argument to the ``register_runner`` function. + +For more details please refer to the updated documentation on :ref:`custom runners `. + + Migrating to Rally 1.4.1 ------------------------ From a7da504f315c8753b04aaa67322fa64bcd0f3247 Mon Sep 17 00:00:00 2001 From: Daniel Mitterdorfer Date: Mon, 24 Feb 2020 14:35:49 +0100 Subject: [PATCH 19/24] Simplifications and more tests --- esrally/driver/__init__.py | 2 +- esrally/driver/async_driver.py | 136 +++++++++++---------- esrally/driver/driver.py | 3 +- esrally/racecontrol.py | 4 +- setup.cfg | 3 + tests/driver/async_driver_test.py | 194 ++++++++++++++++++++++++++++++ 6 files changed, 277 insertions(+), 65 deletions(-) create mode 100644 tests/driver/async_driver_test.py diff --git a/esrally/driver/__init__.py b/esrally/driver/__init__.py index 0a5fa2a65..14047de95 100644 --- a/esrally/driver/__init__.py +++ b/esrally/driver/__init__.py @@ -19,4 +19,4 @@ from .driver import DriverActor, PrepareBenchmark, PreparationComplete, StartBenchmark, BenchmarkComplete, TaskFinished # async API -from .async_driver import AsyncDriver +from .async_driver import AsyncDriver, Timer diff --git a/esrally/driver/async_driver.py b/esrally/driver/async_driver.py index a1a209e8e..62760d02a 100644 --- a/esrally/driver/async_driver.py +++ b/esrally/driver/async_driver.py @@ -27,66 +27,80 @@ from esrally.utils import console -# TODO: Move to time.py class Timer: - def __init__(self, fn, interval, stop_event): - self.stop_event = stop_event - self.fn = fn - self.interval = interval - # check at least once a second whether we need to exit - self.wakeup_interval = min(self.interval, 1) + """ + A general purpose timer that periodically runs tasks that have been added via ``add_task``. Stop the timer via + the ``stop`` method. Note that tasks can be called only once per second at most. + + """ + class Task: + def __init__(self, fn, interval, wakeup_interval): + self.fn = fn + self.interval = interval + self.wakeup_interval = wakeup_interval + self.current = 0 + + def may_run(self): + self.current += self.wakeup_interval + if self.current >= self.interval: + self.current = 0 + self.fn() + + def __repr__(self): + return "timer task for {} firing every {}s.".format(str(self.fn), self.interval) + + def __init__(self, wakeup_interval=1): + """ + :param wakeup_interval: The interval in seconds in which the timer will check whether it has been stopped or + schedule tasks. Default: 1 second. + """ + self.stop_event = threading.Event() + self.tasks = [] + self.wakeup_interval = wakeup_interval + self.logger = logging.getLogger(__name__) + + def add_task(self, fn, interval): + self.tasks.append(Timer.Task(fn, interval, self.wakeup_interval)) + + def stop(self): + self.stop_event.set() def __call__(self, *args, **kwargs): while not self.stop_event.is_set(): - self.fn(*args, **kwargs) + for t in self.tasks: + self.logger.debug("Invoking [%s]", t) + t.may_run() # allow early exit even if a longer sleeping period is requested - for _ in range(self.interval): - if self.stop_event.is_set(): - break - time.sleep(self.wakeup_interval) + if self.stop_event.is_set(): + self.logger.debug("Stopping timer due to external event.") + break + time.sleep(self.wakeup_interval) class AsyncDriver: - def __init__(self, config, es_client_factory_class=client.EsClientFactory): - """ - Coordinates all workers. It is technology-agnostic, i.e. it does not know anything about actors. To allow us to hook in an actor, - we provide a ``target`` parameter which will be called whenever some event has occurred. The ``target`` can use this to send - appropriate messages. - - :param config: The current config object. - """ + def __init__(self, config, track, challenge, es_client_factory_class=client.EsClientFactory): self.logger = logging.getLogger(__name__) self.config = config + self.track = track + self.challenge = challenge self.es_client_factory = es_client_factory_class - self.track = None - self.challenge = None self.metrics_store = None - self.drivers = [] self.progress_reporter = console.progress() - self.progress_counter = 0 - self.quiet = False - self.allocations = None + self.throughput_calculator = driver.ThroughputCalculator() self.raw_samples = [] - self.throughput_calculator = None self.most_recent_sample_per_client = {} - self.number_of_steps = 0 - self.currently_completed = 0 - self.clients_completed_current_step = {} - self.current_step = -1 - self.tasks_per_join_point = None - self.complete_current_task_sent = False self.current_tasks = [] self.telemetry = None self.es_clients = None - self._finished = False + self.quiet = self.config.opts("system", "quiet.mode", mandatory=False, default_value=False) + # TODO: Change the default value to `False` once this implementation becomes the default + self.debug_event_loop = self.config.opts("system", "async.debug", mandatory=False, default_value=True) self.abort_on_error = self.config.opts("driver", "on.error") == "abort" self.profiling_enabled = self.config.opts("driver", "profiling") - self.pool = concurrent.futures.ThreadPoolExecutor(max_workers=4) - self.stop_timer_tasks = threading.Event() self.sampler = None def create_es_clients(self, sync=True): @@ -143,11 +157,13 @@ def retrieve_cluster_info(self): self.logger.exception("Could not retrieve cluster info on benchmark start") return None - def setup(self, t): - self.track = t - self.challenge = driver.select_challenge(self.config, self.track) - self.quiet = self.config.opts("system", "quiet.mode", mandatory=False, default_value=False) - self.throughput_calculator = driver.ThroughputCalculator() + def setup(self): + if self.track.has_plugins: + # no need to fetch the track once more; it has already been updated + track.track_repo(self.config, fetch=False, update=False) + # load track plugins eagerly to initialize the respective parameter sources + track.load_track_plugins(self.config, runner.register_runner, scheduler.register_scheduler) + track.prepare_track(self.track, self.config) self.metrics_store = metrics.metrics_store(cfg=self.config, track=self.track.name, @@ -157,12 +173,6 @@ def setup(self, t): self.wait_for_rest_api() self.prepare_telemetry() - if self.track.has_plugins: - track.track_repo(self.config, fetch=True, update=True) - # we also need to load track plugins eagerly as the respective parameter sources could require - track.load_track_plugins(self.config, runner.register_runner, scheduler.register_scheduler) - track.prepare_track(self.track, self.config) - cluster_info = self.retrieve_cluster_info() cluster_version = cluster_info["version"] if cluster_info else {} return cluster_version.get("build_flavor", "oss"), cluster_version.get("number"), cluster_version.get("build_hash") @@ -174,34 +184,38 @@ def run(self): self.logger.info("Attaching cluster-level telemetry devices.") self.telemetry.on_benchmark_start() self.logger.info("Cluster-level telemetry devices are now attached.") - # TODO: Turn the intervals into constants - self.pool.submit(Timer(fn=self.update_samples, interval=1, stop_event=self.stop_timer_tasks)) - self.pool.submit(Timer(fn=self.post_process_samples, interval=30, stop_event=self.stop_timer_tasks)) - self.pool.submit(Timer(fn=self.update_progress_message, interval=1, stop_event=self.stop_timer_tasks)) + + pool = concurrent.futures.ThreadPoolExecutor(max_workers=1) + + timer = Timer() + timer.add_task(fn=self.update_samples, interval=1) + timer.add_task(fn=self.post_process_samples, interval=30) + timer.add_task(fn=self.update_progress_message, interval=1) + + pool.submit(timer) # needed because a new thread (that is not the main thread) does not have an event loop loop = asyncio.new_event_loop() - # TODO: Make this configurable? - loop.set_debug(True) + loop.set_debug(self.debug_event_loop) asyncio.set_event_loop(loop) loop.set_exception_handler(self._logging_exception_handler) track.set_absolute_data_path(self.config, self.track) runner.register_default_runners() - # TODO: We can skip this here if we run in the same process; it has already been done in #setup() - if self.track.has_plugins: - track.load_track_plugins(self.config, runner.register_runner, scheduler.register_scheduler) + # We can skip this here as long as we run in the same process; it has already been done in #setup() + # if self.track.has_plugins: + # track.load_track_plugins(self.config, runner.register_runner, scheduler.register_scheduler) try: benchmark_runner = driver.AsyncProfiler(self._run_benchmark) if self.profiling_enabled else self._run_benchmark loop.run_until_complete(benchmark_runner()) - - self._finished = True self.telemetry.on_benchmark_stop() self.logger.info("All steps completed.") return self.metrics_store.to_externalizable() finally: - self.stop_timer_tasks.set() - self.pool.shutdown() + self.logger.debug("Stopping timer...") + timer.stop() + pool.shutdown() + self.logger.debug("Closing event loop...") loop.close() self.progress_reporter.finish() self.logger.debug("Closing metrics store...") @@ -287,7 +301,7 @@ def post_process_samples(self): return total_start = time.perf_counter() start = total_start - # we do *not* do this here to avoid concurrent updates (actors are single-threaded) but rather to make it clear that we use + # we do *not* do this here to avoid concurrent updates (we are single-threaded) but rather to make it clear that we use # only a snapshot and that new data will go to a new sample set. raw_samples = self.raw_samples self.raw_samples = [] diff --git a/esrally/driver/driver.py b/esrally/driver/driver.py index b094dc69e..eeaada266 100644 --- a/esrally/driver/driver.py +++ b/esrally/driver/driver.py @@ -1027,6 +1027,7 @@ def __init__(self, cfg, client_id, sub_task, schedule, sampler, cancel, complete self.complete = complete self.abort_on_error = abort_on_error self.profiling_enabled = self.cfg.opts("driver", "profiling") + self.debug_event_loop = self.cfg.opts("system", "async.debug", mandatory=False, default_value=False) def __call__(self, *args, **kwargs): # only possible in Python 3.7+ (has introduced get_running_loop) @@ -1036,7 +1037,7 @@ def __call__(self, *args, **kwargs): # loop = asyncio.new_event_loop() # asyncio.set_event_loop(loop) loop = asyncio.new_event_loop() - loop.set_debug(True) + loop.set_debug(self.debug_event_loop) loop.set_exception_handler(self._logging_exception_handler) asyncio.set_event_loop(loop) try: diff --git a/esrally/racecontrol.py b/esrally/racecontrol.py index fba57f061..f9cfa1a90 100644 --- a/esrally/racecontrol.py +++ b/esrally/racecontrol.py @@ -396,11 +396,11 @@ def run_async(cfg): # We'll use a special car name for external benchmarks. cfg.add(config.Scope.benchmark, "mechanic", "car.names", ["external"]) coordinator = BenchmarkCoordinator(cfg) - race_driver = driver.AsyncDriver(cfg) try: coordinator.setup() - distribution_flavor, distribution_version, revision = race_driver.setup(coordinator.current_track) + race_driver = driver.AsyncDriver(cfg, coordinator.current_track, coordinator.current_challenge) + distribution_flavor, distribution_version, revision = race_driver.setup() coordinator.on_preparation_complete(distribution_flavor, distribution_version, revision) new_metrics = race_driver.run() diff --git a/setup.cfg b/setup.cfg index 145de8d36..b04e162c0 100644 --- a/setup.cfg +++ b/setup.cfg @@ -2,5 +2,8 @@ test=pytest [tool:pytest] +# set to true for more verbose output of tests +log_cli=false +log_level=INFO addopts = --verbose --color=yes testpaths = tests diff --git a/tests/driver/async_driver_test.py b/tests/driver/async_driver_test.py new file mode 100644 index 000000000..8503de0a1 --- /dev/null +++ b/tests/driver/async_driver_test.py @@ -0,0 +1,194 @@ +# Licensed to Elasticsearch B.V. under one or more contributor +# license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright +# ownership. Elasticsearch B.V. licenses this file to you under +# the Apache License, Version 2.0 (the "License"); you may +# not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +import concurrent.futures +import time +from datetime import datetime +from unittest import TestCase, mock + +from esrally import config, metrics +from esrally.driver import async_driver +from esrally.track import track, params + +from tests import as_future + + +class TimerTests(TestCase): + class Counter: + def __init__(self): + self.count = 0 + + def __call__(self): + self.count += 1 + + def test_scheduled_tasks(self): + pool = concurrent.futures.ThreadPoolExecutor(max_workers=1) + + timer = async_driver.Timer(wakeup_interval=0.1) + counter = TimerTests.Counter() + timer.add_task(fn=counter, interval=0.2) + + pool.submit(timer) + + time.sleep(0.45) + timer.stop() + pool.shutdown() + + self.assertEqual(2, counter.count) + + +class StaticClientFactory: + SYNC_PATCHER = None + ASYNC_PATCHER = None + + def __init__(self, *args, **kwargs): + StaticClientFactory.SYNC_PATCHER = mock.patch("elasticsearch.Elasticsearch") + self.es = StaticClientFactory.SYNC_PATCHER.start() + self.es.indices.stats.return_value = {"mocked": True} + self.es.info.return_value = { + "cluster_name": "elasticsearch", + "version": { + "number": "7.3.0", + "build_flavor": "oss", + "build_type": "tar", + "build_hash": "de777fa", + "build_date": "2019-07-24T18:30:11.767338Z", + "build_snapshot": False, + "lucene_version": "8.1.0", + "minimum_wire_compatibility_version": "6.8.0", + "minimum_index_compatibility_version": "6.0.0-beta1" + } + } + + StaticClientFactory.ASYNC_PATCHER = mock.patch("elasticsearch.Elasticsearch") + self.es_async = StaticClientFactory.ASYNC_PATCHER.start() + self.es_async.bulk.return_value = as_future({ + "errors": False, + "took": 5 + }) + self.es_async.transport.close.return_value = as_future() + + def create(self): + return self.es + + def create_async(self): + return self.es_async + + @classmethod + def close(cls): + StaticClientFactory.SYNC_PATCHER.stop() + StaticClientFactory.ASYNC_PATCHER.stop() + + +class AsyncDriverTestParamSource: + def __init__(self, track=None, params=None, **kwargs): + if params is None: + params = {} + self._indices = track.indices + self._params = params + self._current = 1 + self._total = params.get("size") + self.infinite = self._total is None + + def partition(self, partition_index, total_partitions): + return self + + @property + def percent_completed(self): + if self.infinite: + return None + return self._current / self._total + + def params(self): + if not self.infinite and self._current > self._total: + raise StopIteration() + self._current += 1 + return self._params + + +class AsyncDriverTests(TestCase): + class Holder: + def __init__(self, all_hosts=None, all_client_options=None): + self.all_hosts = all_hosts + self.all_client_options = all_client_options + + def test_run_benchmark(self): + cfg = config.Config() + + cfg.add(config.Scope.application, "system", "env.name", "unittest") + cfg.add(config.Scope.application, "system", "time.start", + datetime(year=2017, month=8, day=20, hour=1, minute=0, second=0)) + cfg.add(config.Scope.application, "system", "race.id", "6ebc6e53-ee20-4b0c-99b4-09697987e9f4") + cfg.add(config.Scope.application, "system", "offline.mode", False) + cfg.add(config.Scope.application, "driver", "on.error", "abort") + cfg.add(config.Scope.application, "driver", "profiling", False) + cfg.add(config.Scope.application, "reporting", "datastore.type", "in-memory") + cfg.add(config.Scope.application, "track", "params", {}) + cfg.add(config.Scope.application, "track", "test.mode.enabled", True) + cfg.add(config.Scope.application, "telemetry", "devices", []) + cfg.add(config.Scope.application, "telemetry", "params", {}) + cfg.add(config.Scope.application, "mechanic", "car.names", ["external"]) + cfg.add(config.Scope.application, "mechanic", "skip.rest.api.check", True) + cfg.add(config.Scope.application, "client", "hosts", + AsyncDriverTests.Holder(all_hosts={"default": ["localhost:9200"]})) + cfg.add(config.Scope.application, "client", "options", + AsyncDriverTests.Holder(all_client_options={"default": {}})) + + params.register_param_source_for_name("bulk-param-source", AsyncDriverTestParamSource) + + task = track.Task(name="bulk-index", + operation=track.Operation( + "bulk-index", + track.OperationType.Bulk.name, + params={ + "body": ["action_metadata_line", "index_line"], + "action-metadata-present": True, + "bulk-size": 1, + # we need this because the parameter source does not know that we only have one + # bulk and hence size() returns incorrect results + "size": 1 + }, + param_source="bulk-param-source"), + warmup_iterations=0, + iterations=1, + clients=1) + + current_challenge = track.Challenge(name="default", default=True, schedule=[task]) + current_track = track.Track(name="unit-test", challenges=[current_challenge]) + + driver = async_driver.AsyncDriver(cfg, current_track, current_challenge, + es_client_factory_class=StaticClientFactory) + + distribution_flavor, distribution_version, revision = driver.setup() + self.assertEqual("oss", distribution_flavor) + self.assertEqual("7.3.0", distribution_version) + self.assertEqual("de777fa", revision) + + metrics_store_representation = driver.run() + + metric_store = metrics.metrics_store(cfg, read_only=True, track=current_track, challenge=current_challenge) + metric_store.bulk_add(metrics_store_representation) + + self.assertEqual(5, len(metric_store.docs)) + self.assertIsNotNone(metric_store.get(name="latency", task="bulk-index", sample_type=metrics.SampleType.Normal)) + self.assertIsNotNone(metric_store.get(name="service_time", task="bulk-index", sample_type=metrics.SampleType.Normal)) + self.assertIsNotNone(metric_store.get(name="throughput", task="bulk-index", sample_type=metrics.SampleType.Normal)) + self.assertIsNotNone(metric_store.get(name="node_total_young_gen_gc_time", sample_type=metrics.SampleType.Normal)) + self.assertIsNotNone(metric_store.get(name="node_total_old_gen_gc_time", sample_type=metrics.SampleType.Normal)) + + def tearDown(self): + StaticClientFactory.close() From a0bca34455748359e5c9ccd5b2fd8fd0a1aaf865 Mon Sep 17 00:00:00 2001 From: Daniel Mitterdorfer Date: Mon, 24 Feb 2020 14:57:17 +0100 Subject: [PATCH 20/24] Properly close transport --- esrally/client.py | 22 ++++++++++++++++++---- setup.py | 5 +---- 2 files changed, 19 insertions(+), 8 deletions(-) diff --git a/esrally/client.py b/esrally/client.py index 9d82e24b3..026551062 100644 --- a/esrally/client.py +++ b/esrally/client.py @@ -128,6 +128,8 @@ def create(self): return elasticsearch.Elasticsearch(hosts=self.hosts, ssl_context=self.ssl_context, **self.client_options) def create_async(self): + # keep imports confined as we do some temporary patching to work around unsolved issues in the async ES connector + import elasticsearch import elasticsearch_async from aiohttp.client import ClientTimeout import esrally.async_connection @@ -142,10 +144,22 @@ def __init__(self, hosts, connection_class=esrally.async_connection.AIOHttpConne if "timeout" in self.client_options and not isinstance(self.client_options["timeout"], ClientTimeout): self.client_options["timeout"] = ClientTimeout(total=self.client_options["timeout"]) - return elasticsearch_async.AsyncElasticsearch(hosts=self.hosts, - transport_class=RallyAsyncTransport, - ssl_context=self.ssl_context, - **self.client_options) + # copy of AsyncElasticsearch as https://github.com/elastic/elasticsearch-py-async/pull/49 is not yet released. + # That PR (also) fixes the behavior reported in https://github.com/elastic/elasticsearch-py-async/issues/43. + class RallyAsyncElasticsearch(elasticsearch.Elasticsearch): + def __init__(self, hosts=None, transport_class=RallyAsyncTransport, **kwargs): + super().__init__(hosts, transport_class=transport_class, **kwargs) + + async def __aenter__(self): + return self + + async def __aexit__(self, _exc_type, _exc_val, _exc_tb): + yield self.transport.close() + + return RallyAsyncElasticsearch(hosts=self.hosts, + transport_class=RallyAsyncTransport, + ssl_context=self.ssl_context, + **self.client_options) def wait_for_rest_layer(es, max_attempts=40): diff --git a/setup.py b/setup.py index 56aca299d..52bd80e51 100644 --- a/setup.py +++ b/setup.py @@ -54,10 +54,7 @@ def str_from_file(name): # transitive dependencies: # aiohttp: Apache 2.0 # async_timeout: Apache 2.0 - # 6.2.0 does not include https://github.com/elastic/elasticsearch-py-async/pull/49 which (also) causes - # the behavior reported in https://github.com/elastic/elasticsearch-py-async/issues/43 - # "elasticsearch-async==6.2.0", - "elasticsearch-async@git+https://github.com/elastic/elasticsearch-py-async.git", + "elasticsearch-async==6.2.0", # License: BSD "psutil==5.6.5", # License: MIT From 135b6858ecebe95490e1fa9bb47f2468d635d938 Mon Sep 17 00:00:00 2001 From: Daniel Mitterdorfer Date: Tue, 25 Feb 2020 07:56:00 +0100 Subject: [PATCH 21/24] Improve error handling --- esrally/client.py | 3 +++ esrally/driver/async_driver.py | 2 +- esrally/racecontrol.py | 6 +++--- 3 files changed, 7 insertions(+), 4 deletions(-) diff --git a/esrally/client.py b/esrally/client.py index 026551062..28492135e 100644 --- a/esrally/client.py +++ b/esrally/client.py @@ -143,6 +143,9 @@ def __init__(self, hosts, connection_class=esrally.async_connection.AIOHttpConne if "timeout" in self.client_options and not isinstance(self.client_options["timeout"], ClientTimeout): self.client_options["timeout"] = ClientTimeout(total=self.client_options["timeout"]) + else: + # 10 seconds is the Elasticsearch default, ensure we always set a ClientTimeout object here + self.client_options["timeout"] = ClientTimeout(total=10) # copy of AsyncElasticsearch as https://github.com/elastic/elasticsearch-py-async/pull/49 is not yet released. # That PR (also) fixes the behavior reported in https://github.com/elastic/elasticsearch-py-async/issues/43. diff --git a/esrally/driver/async_driver.py b/esrally/driver/async_driver.py index 62760d02a..0547f8b80 100644 --- a/esrally/driver/async_driver.py +++ b/esrally/driver/async_driver.py @@ -248,7 +248,7 @@ async def _run_benchmark(self): e = driver.AsyncExecutor(client_id, sub_task, schedule, es, self.sampler, cancel, complete, self.abort_on_error) aws.append(e()) # join point - done, pending = await asyncio.wait(aws) + _ = await asyncio.gather(*aws) self.logger.info("All clients have finished running task [%s]", task.name) # drain the active samples before we move on to the next task self.update_samples() diff --git a/esrally/racecontrol.py b/esrally/racecontrol.py index f9cfa1a90..2ec319a10 100644 --- a/esrally/racecontrol.py +++ b/esrally/racecontrol.py @@ -391,7 +391,7 @@ def run(cfg): def run_async(cfg): - console.warn("The race-async command is experimental and thus subject to change.") + console.warn("The race-async command is experimental.") logger = logging.getLogger(__name__) # We'll use a special car name for external benchmarks. cfg.add(config.Scope.benchmark, "mechanic", "car.names", ["external"]) @@ -407,6 +407,6 @@ def run_async(cfg): coordinator.on_benchmark_complete(new_metrics) except KeyboardInterrupt: logger.info("User has cancelled the benchmark.") - except BaseException: + except BaseException as e: tb = sys.exc_info()[2] - raise exceptions.RallyError("This race ended with a fatal crash.").with_traceback(tb) + raise exceptions.RallyError(str(e)).with_traceback(tb) From 419d0d68bb0605f3297b1d771bb231f65c741f7d Mon Sep 17 00:00:00 2001 From: Daniel Mitterdorfer Date: Tue, 25 Feb 2020 08:38:49 +0100 Subject: [PATCH 22/24] More cleanups --- esrally/track/params.py | 8 +------- esrally/utils/io.py | 45 ----------------------------------------- 2 files changed, 1 insertion(+), 52 deletions(-) diff --git a/esrally/track/params.py b/esrally/track/params.py index 23606d7dc..08618630e 100644 --- a/esrally/track/params.py +++ b/esrally/track/params.py @@ -563,7 +563,6 @@ def __init__(self, corpora, partition_index, total_partitions, batch_size, bulk_ all_bulks = number_of_bulks(self.corpora, self.partition_index, self.total_partitions, self.bulk_size) self.total_bulks = math.ceil((all_bulks * self.ingest_percentage) / 100) self.infinite = False - self.logger = logging.getLogger(__name__) def partition(self, partition_index, total_partitions): raise exceptions.RallyError("Cannot partition a PartitionBulkIndexParamSource further") @@ -574,11 +573,7 @@ def params(self): if self.current_bulk == self.total_bulks: raise StopIteration() self.current_bulk += 1 - start = time.perf_counter() - p = next(self.internal_params) - end = time.perf_counter() - self.logger.info("Reading bulk took [%s] seconds.", (end - start)) - return p + return next(self.internal_params) @property def percent_completed(self): @@ -649,7 +644,6 @@ def chain(*iterables): def create_default_reader(docs, offset, num_lines, num_docs, batch_size, bulk_size, id_conflicts, conflict_probability, on_conflict, recency): - #source = Slice(io.StaticSource, offset, num_lines) source = Slice(io.FileSource, offset, num_lines) if docs.includes_action_and_meta_data: diff --git a/esrally/utils/io.py b/esrally/utils/io.py index 8e88df558..ea134126d 100644 --- a/esrally/utils/io.py +++ b/esrally/utils/io.py @@ -154,51 +154,6 @@ def __str__(self, *args, **kwargs): return "StringAsFileSource" -# TODO: Remove before merging -# pylint: disable=C0301 -class StaticSource: - def __init__(self, contents, mode, encoding="utf-8"): - self.contents = '{"geonameid": 2986043, "name": "Pic de Font Blanca", "asciiname": "Pic de Font Blanca", "alternatenames": "Pic de Font Blanca,Pic du Port", "feature_class": "T", "feature_code": "PK", "country_code": "AD", "admin1_code": "00", "population": 0, "dem": "2860", "timezone": "Europe/Andorra", "location": [1.53335, 42.64991]}' - self.current_index = 0 - self.opened = False - - def open(self): - self.opened = True - return self - - def seek(self, offset): - pass - - def read(self): - return "\n".join(self.contents) - - def readline(self): - return self.contents - - def readlines(self, num_lines): - return [self.contents] * num_lines - - def close(self): - self._assert_opened() - self.contents = None - self.opened = False - - def _assert_opened(self): - assert self.opened - - def __enter__(self): - self.open() - return self - - def __exit__(self, exc_type, exc_val, exc_tb): - self.close() - return False - - def __str__(self, *args, **kwargs): - return "StaticSource" - - - def ensure_dir(directory, mode=0o777): """ Ensure that the provided directory and all of its parent directories exist. From 92bee31f7f3a2f3332c13f10034b0d847fbef354 Mon Sep 17 00:00:00 2001 From: Daniel Mitterdorfer Date: Wed, 4 Mar 2020 09:11:44 +0100 Subject: [PATCH 23/24] Special case handling for connection timeouts --- esrally/driver/driver.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/esrally/driver/driver.py b/esrally/driver/driver.py index eeaada266..faee5ad5b 100644 --- a/esrally/driver/driver.py +++ b/esrally/driver/driver.py @@ -1201,7 +1201,10 @@ async def execute_single(runner, es, params, abort_on_error=False): # The ES client will sometimes return string like "N/A" or "TIMEOUT" for connection errors. if isinstance(e.status_code, int): request_meta_data["http-status"] = e.status_code - if e.info: + # connection timeout errors don't provide a helpful description + if isinstance(e, elasticsearch.ConnectionTimeout): + request_meta_data["error-description"] = "network connection timed out" + elif e.info: request_meta_data["error-description"] = "%s (%s)" % (e.error, e.info) else: request_meta_data["error-description"] = e.error From 25fe49a4cdcaf461967752eb71f964995c742900 Mon Sep 17 00:00:00 2001 From: Daniel Mitterdorfer Date: Sun, 8 Mar 2020 18:38:11 +0100 Subject: [PATCH 24/24] Make new test async --- tests/driver/runner_test.py | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/tests/driver/runner_test.py b/tests/driver/runner_test.py index 8f40d4fd9..05d7c03ed 100644 --- a/tests/driver/runner_test.py +++ b/tests/driver/runner_test.py @@ -942,8 +942,9 @@ async def test_indices_stats_with_successful_condition(self, es): es.indices.stats.assert_called_once_with(index="logs-*", metric="_all") @mock.patch("elasticsearch.Elasticsearch") - def test_indices_stats_with_non_existing_path(self, es): - es.indices.stats.return_value = { + @run_async + async def test_indices_stats_with_non_existing_path(self, es): + es.indices.stats.return_value = as_future({ "indices": { "total": { "docs": { @@ -951,11 +952,11 @@ def test_indices_stats_with_non_existing_path(self, es): } } } - } + }) indices_stats = runner.IndicesStats() - result = indices_stats(es, params={ + result = await indices_stats(es, params={ "index": "logs-*", "condition": { # non-existing path