diff --git a/rest-api-spec/api/abort_benchmark.json b/rest-api-spec/api/abort_benchmark.json deleted file mode 100644 index 6134199f99823..0000000000000 --- a/rest-api-spec/api/abort_benchmark.json +++ /dev/null @@ -1,20 +0,0 @@ -{ - "abort_benchmark" : { - "documentation": "http://www.elasticsearch.org/guide/en/elasticsearch/reference/master/search-benchmark.html", - "methods": ["POST"], - "url": { - "path": "/_bench/abort/{name}", - "paths": [ - "/_bench/abort/{name}" - ], - "parts": { - "name": { - "type" : "string", - "description" : "A benchmark name" - } - }, - "params": {} - }, - "body": null - } -} diff --git a/rest-api-spec/api/benchmark.abort.json b/rest-api-spec/api/benchmark.abort.json new file mode 100644 index 0000000000000..dd97d880879b2 --- /dev/null +++ b/rest-api-spec/api/benchmark.abort.json @@ -0,0 +1,21 @@ +{ + "benchmark.abort": { + "documentation": "http://www.elasticsearch.org/guide/en/elasticsearch/reference/master/search-benchmark.html", + "methods": ["POST"], + "url": { + "path": "/_bench/abort/{id}", + "paths": [ + "/_bench/abort/{id}" + ], + "parts": { }, + "params": { + "id": { + "required": true, + "type": "boolean", + "description": "The unique ID of the benchmark to abort" + } + } + }, + "body": null + } +} diff --git a/rest-api-spec/api/benchmark.pause.json b/rest-api-spec/api/benchmark.pause.json new file mode 100644 index 0000000000000..23b63de08007f --- /dev/null +++ b/rest-api-spec/api/benchmark.pause.json @@ -0,0 +1,21 @@ +{ + "benchmark.pause": { + "documentation": "http://www.elasticsearch.org/guide/en/elasticsearch/reference/master/search-benchmark.html", + "methods": ["POST"], + "url": { + "path": "/_bench/pause/{id}", + "paths": [ + "/_bench/pause/{id}" + ], + "parts": { }, + "params": { + "id": { + "required": true, + "type": "boolean", + "description": "The unique ID of the benchmark to pause" + } + } + }, + "body": null + } +} diff --git a/rest-api-spec/api/benchmark.resume.json b/rest-api-spec/api/benchmark.resume.json new file mode 100644 index 0000000000000..492ac54bfdf18 --- /dev/null +++ b/rest-api-spec/api/benchmark.resume.json @@ -0,0 +1,21 @@ +{ + "benchmark.resume": { + "documentation": "http://www.elasticsearch.org/guide/en/elasticsearch/reference/master/search-benchmark.html", + "methods": ["POST"], + "url": { + "path": "/_bench/resume/{id}", + "paths": [ + "/_bench/resume/{id}" + ], + "parts": { }, + "params": { + "id": { + "required": true, + "type": "boolean", + "description": "The unique ID of the benchmark to resume" + } + } + }, + "body": null + } +} diff --git a/rest-api-spec/api/benchmark.status.json b/rest-api-spec/api/benchmark.status.json new file mode 100644 index 0000000000000..aee3d8c907288 --- /dev/null +++ b/rest-api-spec/api/benchmark.status.json @@ -0,0 +1,22 @@ +{ + "benchmark.status": { + "documentation": "http://www.elasticsearch.org/guide/en/elasticsearch/reference/master/search-benchmark.html", + "methods": ["GET"], + "url": { + "path": "/_bench/status/{id}", + "paths": [ + "/_bench/status", + "/_bench/status/{id}" + ], + "parts": { }, + "params": { + "id": { + "required": false, + "type": "list", + "description": "A list of comma-separated ID's to get the status of" + } + } + }, + "body": null + } +} diff --git a/rest-api-spec/api/benchmark.json b/rest-api-spec/api/benchmark.submit.json similarity index 51% rename from rest-api-spec/api/benchmark.json rename to rest-api-spec/api/benchmark.submit.json index 4bbed21bd23c0..9067facaa81b9 100644 --- a/rest-api-spec/api/benchmark.json +++ b/rest-api-spec/api/benchmark.submit.json @@ -1,22 +1,24 @@ { - "benchmark" : { + "benchmark.submit": { "documentation": "http://www.elasticsearch.org/guide/en/elasticsearch/reference/master/search-benchmark.html", "methods": ["PUT"], "url": { - "path": "/_bench", + "path": "/_bench/submit", "paths": [ - "/_bench", - "/{index}/_bench", - "/{index}/{type}/_bench" + "/_bench/submit", + "/{index}/_bench/submit", + "/{index}/{types}/_bench/submit" ], "parts": { "index": { - "type" : "list", - "description" : "A comma-separated list of index names; use `_all` or empty string to perform the operation on all indices" + "required": false, + "type": "list", + "description": "A comma-separated list of index names; use `_all` or empty string to perform the operation on all indices" }, - "type": { - "type" : "string", - "description" : "The name of the document type" + "types": { + "required": false, + "type": "list", + "description": "A comma-separated list of document types" } }, "params": { @@ -27,7 +29,8 @@ } }, "body": { - "description": "The search definition using the Query DSL" + "description": "The search definition using the Query DSL", + "required": true } } } diff --git a/rest-api-spec/api/list_benchmarks.json b/rest-api-spec/api/list_benchmarks.json deleted file mode 100644 index 7e3cfb6e182d3..0000000000000 --- a/rest-api-spec/api/list_benchmarks.json +++ /dev/null @@ -1,26 +0,0 @@ -{ - "list_benchmarks" : { - "documentation": "http://www.elasticsearch.org/guide/en/elasticsearch/reference/master/search-benchmark.html", - "methods": ["GET"], - "url": { - "path": "/_bench", - "paths": [ - "/_bench", - "/{index}/_bench", - "/{index}/{type}/_bench" - ], - "parts": { - "index": { - "type" : "list", - "description" : "A comma-separated list of index names; use `_all` or empty string to perform the operation on all indices" - }, - "type": { - "type" : "string", - "description" : "The name of the document type" - } - }, - "params": {} - }, - "body": null - } -} diff --git a/rest-api-spec/test/abort_benchmark/10_basic.yaml b/rest-api-spec/test/abort_benchmark/10_basic.yaml deleted file mode 100644 index a065378bf45bc..0000000000000 --- a/rest-api-spec/test/abort_benchmark/10_basic.yaml +++ /dev/null @@ -1,11 +0,0 @@ ---- -"Test benchmark abort": - - - skip: - features: "benchmark" - - - do: - abort_benchmark: - name: my_benchmark - catch: missing - diff --git a/rest-api-spec/test/benchmark.submit/10_basic.yaml b/rest-api-spec/test/benchmark.submit/10_basic.yaml new file mode 100644 index 0000000000000..50fefc550a432 --- /dev/null +++ b/rest-api-spec/test/benchmark.submit/10_basic.yaml @@ -0,0 +1,111 @@ +--- +setup: + + - skip: + features: "benchmark" + + - do: + index: + index: test_1 + type: type_1 + id: 1 + body: { a: x, b: y, c: z } + + - do: + index: + index: test_1 + type: type_1 + id: 2 + body: { a: xx, b: yy, c: zz } + + - do: + index: + index: test_1 + type: type_1 + id: 3 + body: { a: xxx, b: yyy, c: zzz } + + - do: + indices.refresh: {} + + - do: + cluster.health: + wait_for_status: yellow + +--- +"Submit a benchmark": + + - do: + benchmark.submit: + body: { + name: test_benchmark, + verbose: true, + num_executor_nodes: 1, + iterations: 3, + multiplier: 5, + concurrency: 2, + num_slowest: 1, + competitors: [ { + name: test_competitor_1, + concurrency: 3, + requests: [ { + query: { term: { a: x } } + }, { + query: { term: { b: yy } } + } ] + }, + { + name: test_competitor_2, + concurrency: 1, + requests: [ { + query: { match: { _all: a } } + }, { + query: { match: { _all: c } } + } ] + }] + } + + - match: { benchmark.id: "test_benchmark" } + - match: { benchmark.status: "COMPLETED" } + + - match: { benchmark.competitors.test_competitor_1.summary.concurrency: 3 } + - match: { benchmark.competitors.test_competitor_1.summary.total_requested_iterations: 3 } + - match: { benchmark.competitors.test_competitor_1.summary.total_completed_iterations: 3 } + - match: { benchmark.competitors.test_competitor_1.summary.multiplier: 5 } + - gte: { benchmark.competitors.test_competitor_1.summary.avg_warmup_time: 0.0 } + - gte: { benchmark.competitors.test_competitor_1.summary.statistics.min: 0 } + - gte: { benchmark.competitors.test_competitor_1.summary.statistics.max: 0 } + - gte: { benchmark.competitors.test_competitor_1.summary.statistics.mean: 0.0 } + - gte: { benchmark.competitors.test_competitor_1.summary.statistics.qps: 0.0 } + - gte: { benchmark.competitors.test_competitor_1.summary.statistics.millis_per_hit: 0.0 } + - gte: { benchmark.competitors.test_competitor_1.summary.statistics.percentile_10: 0.0 } + - gte: { benchmark.competitors.test_competitor_1.summary.statistics.percentile_25: 0.0 } + - gte: { benchmark.competitors.test_competitor_1.summary.statistics.percentile_50: 0.0 } + - gte: { benchmark.competitors.test_competitor_1.summary.statistics.percentile_75: 0.0 } + - gte: { benchmark.competitors.test_competitor_1.summary.statistics.percentile_90: 0.0 } + - gte: { benchmark.competitors.test_competitor_1.summary.statistics.percentile_99: 0.0 } + - length: { benchmark.competitors.test_competitor_1.summary.slowest: 1 } + - length: { benchmark.competitors.test_competitor_1.iterations: 3 } + + - match: { benchmark.competitors.test_competitor_2.summary.concurrency: 1 } + - match: { benchmark.competitors.test_competitor_2.summary.total_requested_iterations: 3 } + - match: { benchmark.competitors.test_competitor_2.summary.total_completed_iterations: 3 } + - match: { benchmark.competitors.test_competitor_2.summary.multiplier: 5 } + - gte: { benchmark.competitors.test_competitor_2.summary.avg_warmup_time: 0.0 } + - gte: { benchmark.competitors.test_competitor_2.summary.statistics.min: 0 } + - gte: { benchmark.competitors.test_competitor_2.summary.statistics.max: 0 } + - gte: { benchmark.competitors.test_competitor_1.summary.statistics.mean: 0.0 } + - gte: { benchmark.competitors.test_competitor_2.summary.statistics.qps: 0.0 } + - gte: { benchmark.competitors.test_competitor_2.summary.statistics.millis_per_hit: 0.0 } + - gte: { benchmark.competitors.test_competitor_2.summary.statistics.percentile_10: 0.0 } + - gte: { benchmark.competitors.test_competitor_2.summary.statistics.percentile_25: 0.0 } + - gte: { benchmark.competitors.test_competitor_2.summary.statistics.percentile_50: 0.0 } + - gte: { benchmark.competitors.test_competitor_2.summary.statistics.percentile_75: 0.0 } + - gte: { benchmark.competitors.test_competitor_2.summary.statistics.percentile_90: 0.0 } + - gte: { benchmark.competitors.test_competitor_2.summary.statistics.percentile_99: 0.0 } + - length: { benchmark.competitors.test_competitor_2.summary.slowest: 1 } + - length: { benchmark.competitors.test_competitor_2.iterations: 3 } + + - length: { benchmark.errors: 0 } + - length: { benchmark.competitors: 2 } + diff --git a/rest-api-spec/test/benchmark/10_basic.yaml b/rest-api-spec/test/benchmark/10_basic.yaml deleted file mode 100644 index 9340b26b4d8d7..0000000000000 --- a/rest-api-spec/test/benchmark/10_basic.yaml +++ /dev/null @@ -1,33 +0,0 @@ ---- -"Test benchmark submit": - - - skip: - features: "benchmark" - - - do: - indices.create: - index: test_1 - body: - settings: - index: - number_of_replicas: 0 - - - do: - cluster.health: - wait_for_status: yellow - - - do: - benchmark: - index: test_1 - body: - "name": "my_benchmark" - "competitors": - - - "name": "my_competitor" - "requests": - - - "query": - "match": { "_all": "*" } - - - match: { status: COMPLETE } - diff --git a/rest-api-spec/test/list_benchmarks/10_basic.yaml b/rest-api-spec/test/list_benchmarks/10_basic.yaml deleted file mode 100644 index 7ce1b7ac42942..0000000000000 --- a/rest-api-spec/test/list_benchmarks/10_basic.yaml +++ /dev/null @@ -1,9 +0,0 @@ ---- -"Test benchmark list": - - - skip: - features: "benchmark" - - - do: - list_benchmarks: {} - diff --git a/src/main/java/org/elasticsearch/action/ActionModule.java b/src/main/java/org/elasticsearch/action/ActionModule.java index 8917b16c0614a..a5c49a201e903 100644 --- a/src/main/java/org/elasticsearch/action/ActionModule.java +++ b/src/main/java/org/elasticsearch/action/ActionModule.java @@ -118,7 +118,16 @@ import org.elasticsearch.action.admin.indices.warmer.get.TransportGetWarmersAction; import org.elasticsearch.action.admin.indices.warmer.put.PutWarmerAction; import org.elasticsearch.action.admin.indices.warmer.put.TransportPutWarmerAction; -import org.elasticsearch.action.bench.*; +import org.elasticsearch.action.benchmark.abort.BenchmarkAbortAction; +import org.elasticsearch.action.benchmark.abort.TransportBenchmarkAbortAction; +import org.elasticsearch.action.benchmark.pause.BenchmarkPauseAction; +import org.elasticsearch.action.benchmark.pause.TransportBenchmarkPauseAction; +import org.elasticsearch.action.benchmark.resume.BenchmarkResumeAction; +import org.elasticsearch.action.benchmark.resume.TransportBenchmarkResumeAction; +import org.elasticsearch.action.benchmark.start.BenchmarkStartAction; +import org.elasticsearch.action.benchmark.start.TransportBenchmarkStartAction; +import org.elasticsearch.action.benchmark.status.BenchmarkStatusAction; +import org.elasticsearch.action.benchmark.status.TransportBenchmarkStatusAction; import org.elasticsearch.action.bulk.BulkAction; import org.elasticsearch.action.bulk.TransportBulkAction; import org.elasticsearch.action.bulk.TransportShardBulkAction; @@ -308,9 +317,11 @@ protected void configure() { registerAction(ExplainAction.INSTANCE, TransportExplainAction.class); registerAction(ClearScrollAction.INSTANCE, TransportClearScrollAction.class); registerAction(RecoveryAction.INSTANCE, TransportRecoveryAction.class); - registerAction(BenchmarkAction.INSTANCE, TransportBenchmarkAction.class); - registerAction(AbortBenchmarkAction.INSTANCE, TransportAbortBenchmarkAction.class); + registerAction(BenchmarkStartAction.INSTANCE, TransportBenchmarkStartAction.class); + registerAction(BenchmarkAbortAction.INSTANCE, TransportBenchmarkAbortAction.class); registerAction(BenchmarkStatusAction.INSTANCE, TransportBenchmarkStatusAction.class); + registerAction(BenchmarkResumeAction.INSTANCE, TransportBenchmarkResumeAction.class); + registerAction(BenchmarkPauseAction.INSTANCE, TransportBenchmarkPauseAction.class); //Indexed scripts registerAction(PutIndexedScriptAction.INSTANCE, TransportPutIndexedScriptAction.class); diff --git a/src/main/java/org/elasticsearch/action/bench/BenchmarkExecutor.java b/src/main/java/org/elasticsearch/action/bench/BenchmarkExecutor.java deleted file mode 100644 index 1b63ab9ae62bf..0000000000000 --- a/src/main/java/org/elasticsearch/action/bench/BenchmarkExecutor.java +++ /dev/null @@ -1,447 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch 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. - */ -package org.elasticsearch.action.bench; - -import com.google.common.collect.UnmodifiableIterator; -import org.apache.lucene.util.PriorityQueue; -import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.ExceptionsHelper; -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.search.SearchRequest; -import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.client.Client; -import org.elasticsearch.cluster.ClusterService; -import org.elasticsearch.common.collect.ImmutableOpenMap; -import org.elasticsearch.common.logging.ESLogger; -import org.elasticsearch.common.logging.Loggers; - -import java.util.*; -import java.util.concurrent.*; - -/** - * Handles execution, listing, and aborting of benchmarks - */ -public class BenchmarkExecutor { - - private static final ESLogger logger = Loggers.getLogger(BenchmarkExecutor.class); - - private final Client client; - private String nodeName; - private final ClusterService clusterService; - private volatile ImmutableOpenMap activeBenchmarks = ImmutableOpenMap.of(); - - private final Object activeStateLock = new Object(); - - public BenchmarkExecutor(Client client, ClusterService clusterService) { - this.client = client; - this.clusterService = clusterService; - } - - private static class BenchmarkState { - final String id; - final StoppableSemaphore semaphore; - final BenchmarkResponse response; - - BenchmarkState(BenchmarkRequest request, BenchmarkResponse response, StoppableSemaphore semaphore) { - this.id = request.benchmarkName(); - this.response = response; - this.semaphore = semaphore; - } - } - - /** - * Aborts benchmark(s) matching the given wildcard patterns - * - * @param names the benchmark names to abort - */ - public AbortBenchmarkResponse abortBenchmark(String[] names) { - synchronized (activeStateLock) { - for (String name : names) { - try { - final BenchmarkState state = activeBenchmarks.get(name); - if (state == null) { - continue; - } - state.semaphore.stop(); - activeBenchmarks = ImmutableOpenMap.builder(activeBenchmarks).fRemove(name).build(); - logger.debug("Aborted benchmark [{}] on [{}]", name, nodeName()); - } catch (Throwable e) { - logger.warn("Error while aborting [{}]", name, e); - } - } - } - return new AbortBenchmarkResponse(true); - } - - /** - * Reports status of all active benchmarks - * - * @return Benchmark status response - */ - public BenchmarkStatusNodeResponse benchmarkStatus() { - - BenchmarkStatusNodeResponse response = new BenchmarkStatusNodeResponse(); - final ImmutableOpenMap activeBenchmarks = this.activeBenchmarks; - UnmodifiableIterator iter = activeBenchmarks.keysIt(); - while (iter.hasNext()) { - String id = iter.next(); - BenchmarkState state = activeBenchmarks.get(id); - response.addBenchResponse(state.response); - } - - logger.debug("Reporting [{}] active benchmarks on [{}]", response.activeBenchmarks(), nodeName()); - return response; - } - - /** - * Submits a search benchmark for execution - * - * @param request A benchmark request - * @return Summary response of executed benchmark - * @throws ElasticsearchException - */ - public BenchmarkResponse benchmark(BenchmarkRequest request) throws ElasticsearchException { - - final StoppableSemaphore semaphore = new StoppableSemaphore(1); - final Map competitionResults = new HashMap(); - final BenchmarkResponse benchmarkResponse = new BenchmarkResponse(request.benchmarkName(), competitionResults); - - synchronized (activeStateLock) { - if (activeBenchmarks.containsKey(request.benchmarkName())) { - throw new ElasticsearchException("Benchmark [" + request.benchmarkName() + "] is already running on [" + nodeName() + "]"); - } - - activeBenchmarks = ImmutableOpenMap.builder(activeBenchmarks).fPut( - request.benchmarkName(), new BenchmarkState(request, benchmarkResponse, semaphore)).build(); - } - - try { - for (BenchmarkCompetitor competitor : request.competitors()) { - - final BenchmarkSettings settings = competitor.settings(); - final int iterations = settings.iterations(); - logger.debug("Executing [iterations: {}] [multiplier: {}] for [{}] on [{}]", - iterations, settings.multiplier(), request.benchmarkName(), nodeName()); - - final List competitionIterations = new ArrayList<>(iterations); - final CompetitionResult competitionResult = - new CompetitionResult(competitor.name(), settings.concurrency(), settings.multiplier(), request.percentiles()); - final CompetitionNodeResult competitionNodeResult = - new CompetitionNodeResult(competitor.name(), nodeName(), iterations, competitionIterations); - - competitionResult.addCompetitionNodeResult(competitionNodeResult); - benchmarkResponse.competitionResults.put(competitor.name(), competitionResult); - - final List searchRequests = competitor.settings().searchRequests(); - - if (settings.warmup()) { - final long beforeWarmup = System.nanoTime(); - final List warmUpErrors = warmUp(competitor, searchRequests, semaphore); - final long afterWarmup = System.nanoTime(); - competitionNodeResult.warmUpTime(TimeUnit.MILLISECONDS.convert(afterWarmup - beforeWarmup, TimeUnit.NANOSECONDS)); - if (!warmUpErrors.isEmpty()) { - throw new BenchmarkExecutionException("Failed to execute warmup phase", warmUpErrors); - } - } - - final int numMeasurements = settings.multiplier() * searchRequests.size(); - final long[] timeBuckets = new long[numMeasurements]; - final long[] docBuckets = new long[numMeasurements]; - - for (int i = 0; i < iterations; i++) { - if (settings.allowCacheClearing() && settings.clearCaches() != null) { - try { - client.admin().indices().clearCache(settings.clearCaches()).get(); - } catch (ExecutionException e) { - throw new BenchmarkExecutionException("Failed to clear caches", e); - } - } - - // Run the iteration - CompetitionIteration ci = - runIteration(competitor, searchRequests, timeBuckets, docBuckets, semaphore); - ci.percentiles(request.percentiles()); - competitionIterations.add(ci); - competitionNodeResult.incrementCompletedIterations(); - } - - competitionNodeResult.totalExecutedQueries(settings.multiplier() * searchRequests.size() * iterations); - } - - benchmarkResponse.state(BenchmarkResponse.State.COMPLETE); - - } catch (BenchmarkExecutionException e) { - benchmarkResponse.state(BenchmarkResponse.State.FAILED); - benchmarkResponse.errors(e.errorMessages().toArray(new String[e.errorMessages().size()])); - } catch (InterruptedException ex) { - Thread.currentThread().interrupt(); - benchmarkResponse.state(BenchmarkResponse.State.ABORTED); - } catch (Throwable ex) { - logger.debug("Unexpected exception during benchmark", ex); - benchmarkResponse.state(BenchmarkResponse.State.FAILED); - benchmarkResponse.errors(ex.getMessage()); - } finally { - synchronized (activeStateLock) { - semaphore.stop(); - activeBenchmarks = ImmutableOpenMap.builder(activeBenchmarks).fRemove(request.benchmarkName()).build(); - } - } - - return benchmarkResponse; - } - - private List warmUp(BenchmarkCompetitor competitor, List searchRequests, StoppableSemaphore stoppableSemaphore) - throws InterruptedException { - final StoppableSemaphore semaphore = stoppableSemaphore.reset(competitor.settings().concurrency()); - final CountDownLatch totalCount = new CountDownLatch(searchRequests.size()); - final CopyOnWriteArrayList errorMessages = new CopyOnWriteArrayList<>(); - - for (SearchRequest searchRequest : searchRequests) { - semaphore.acquire(); - client.search(searchRequest, new BoundsManagingActionListener(semaphore, totalCount, errorMessages) { } ); - } - totalCount.await(); - return errorMessages; - } - - private CompetitionIteration runIteration(BenchmarkCompetitor competitor, List searchRequests, - final long[] timeBuckets, final long[] docBuckets, - StoppableSemaphore stoppableSemaphore) throws InterruptedException { - - assert timeBuckets.length == competitor.settings().multiplier() * searchRequests.size(); - assert docBuckets.length == competitor.settings().multiplier() * searchRequests.size(); - - final StoppableSemaphore semaphore = stoppableSemaphore.reset(competitor.settings().concurrency()); - - Arrays.fill(timeBuckets, -1); // wipe CPU cache ;) - Arrays.fill(docBuckets, -1); // wipe CPU cache ;) - - int id = 0; - final CountDownLatch totalCount = new CountDownLatch(timeBuckets.length); - final CopyOnWriteArrayList errorMessages = new CopyOnWriteArrayList<>(); - final long beforeRun = System.nanoTime(); - - for (int i = 0; i < competitor.settings().multiplier(); i++) { - for (SearchRequest searchRequest : searchRequests) { - StatisticCollectionActionListener statsListener = - new StatisticCollectionActionListener(semaphore, timeBuckets, docBuckets, id++, totalCount, errorMessages); - semaphore.acquire(); - client.search(searchRequest, statsListener); - } - } - totalCount.await(); - assert id == timeBuckets.length; - final long afterRun = System.nanoTime(); - if (!errorMessages.isEmpty()) { - throw new BenchmarkExecutionException("Too many execution failures", errorMessages); - } - - final long totalTime = TimeUnit.MILLISECONDS.convert(afterRun - beforeRun, TimeUnit.NANOSECONDS); - - CompetitionIterationData iterationData = new CompetitionIterationData(timeBuckets); - long sumDocs = new CompetitionIterationData(docBuckets).sum(); - - // Don't track slowest request if there is only one request as that is redundant - CompetitionIteration.SlowRequest[] topN = null; - if ((competitor.settings().numSlowest() > 0) && (searchRequests.size() > 1)) { - topN = getTopN(timeBuckets, searchRequests, competitor.settings().multiplier(), competitor.settings().numSlowest()); - } - - CompetitionIteration round = - new CompetitionIteration(topN, totalTime, timeBuckets.length, sumDocs, iterationData); - return round; - } - - private CompetitionIteration.SlowRequest[] getTopN(long[] buckets, List requests, int multiplier, int topN) { - - final int numRequests = requests.size(); - // collect the top N - final PriorityQueue topNQueue = new PriorityQueue(topN) { - @Override - protected boolean lessThan(IndexAndTime a, IndexAndTime b) { - return a.avgTime < b.avgTime; - } - }; - assert multiplier > 0; - for (int i = 0; i < numRequests; i++) { - long sum = 0; - long max = Long.MIN_VALUE; - for (int j = 0; j < multiplier; j++) { - final int base = (numRequests * j); - sum += buckets[i + base]; - max = Math.max(buckets[i + base], max); - } - final long avg = sum / multiplier; - if (topNQueue.size() < topN) { - topNQueue.add(new IndexAndTime(i, max, avg)); - } else if (topNQueue.top().avgTime < max) { - topNQueue.top().update(i, max, avg); - topNQueue.updateTop(); - - } - } - - final CompetitionIteration.SlowRequest[] slowRequests = new CompetitionIteration.SlowRequest[topNQueue.size()]; - int i = topNQueue.size() - 1; - - while (topNQueue.size() > 0) { - IndexAndTime pop = topNQueue.pop(); - CompetitionIteration.SlowRequest slow = - new CompetitionIteration.SlowRequest(pop.avgTime, pop.maxTime, requests.get(pop.index)); - slowRequests[i--] = slow; - } - - return slowRequests; - } - - private static class IndexAndTime { - int index; - long maxTime; - long avgTime; - - public IndexAndTime(int index, long maxTime, long avgTime) { - this.index = index; - this.maxTime = maxTime; - this.avgTime = avgTime; - } - - public void update(int index, long maxTime, long avgTime) { - this.index = index; - this.maxTime = maxTime; - this.avgTime = avgTime; - } - } - - private static abstract class BoundsManagingActionListener implements ActionListener { - - private final StoppableSemaphore semaphore; - private final CountDownLatch latch; - private final CopyOnWriteArrayList errorMessages; - - public BoundsManagingActionListener(StoppableSemaphore semaphore, CountDownLatch latch, CopyOnWriteArrayList errorMessages) { - this.semaphore = semaphore; - this.latch = latch; - this.errorMessages = errorMessages; - } - - private void manage() { - try { - semaphore.release(); - } finally { - latch.countDown(); - } - } - - public void onResponse(Response response) { - manage(); - } - - public void onFailure(Throwable e) { - try { - if (errorMessages.size() < 5) { - logger.debug("Failed to execute benchmark [{}]", e.getMessage(), e); - e = ExceptionsHelper.unwrapCause(e); - errorMessages.add(e.getLocalizedMessage()); - } - } finally { - manage(); // first add the msg then call the count down on the latch otherwise we might iss one error - } - - } - } - - private static class StatisticCollectionActionListener extends BoundsManagingActionListener { - - private final long[] timeBuckets; - private final int bucketId; - private final long[] docBuckets; - - public StatisticCollectionActionListener(StoppableSemaphore semaphore, long[] timeBuckets, long[] docs, - int bucketId, CountDownLatch totalCount, - CopyOnWriteArrayList errorMessages) { - super(semaphore, totalCount, errorMessages); - this.bucketId = bucketId; - this.timeBuckets = timeBuckets; - this.docBuckets = docs; - } - - @Override - public void onResponse(SearchResponse searchResponse) { - super.onResponse(searchResponse); - timeBuckets[bucketId] = searchResponse.getTookInMillis(); - if (searchResponse.getHits() != null) { - docBuckets[bucketId] = searchResponse.getHits().getTotalHits(); - } - } - - @Override - public void onFailure(Throwable e) { - try { - timeBuckets[bucketId] = -1; - docBuckets[bucketId] = -1; - } finally { - super.onFailure(e); - } - - } - } - - private final static class StoppableSemaphore { - private Semaphore semaphore; - private volatile boolean stopped = false; - - public StoppableSemaphore(int concurrency) { - semaphore = new Semaphore(concurrency); - } - - public StoppableSemaphore reset(int concurrency) { - semaphore = new Semaphore(concurrency); - return this; - } - - public void acquire() throws InterruptedException { - if (stopped) { - throw new InterruptedException("Benchmark Interrupted"); - } - semaphore.acquire(); - } - - public void release() { - semaphore.release(); - } - - public void stop() { - stopped = true; - } - } - - private String nodeName() { - if (nodeName == null) { - nodeName = clusterService.localNode().name(); - } - return nodeName; - } - - private final boolean assertBuckets(long[] buckets) { - for (int i = 0; i < buckets.length; i++) { - assert buckets[i] >= 0 : "Bucket value was negative: " + buckets[i] + " bucket id: " + i; - } - return true; - } -} diff --git a/src/main/java/org/elasticsearch/action/bench/BenchmarkService.java b/src/main/java/org/elasticsearch/action/bench/BenchmarkService.java deleted file mode 100644 index 5868aa12b5a28..0000000000000 --- a/src/main/java/org/elasticsearch/action/bench/BenchmarkService.java +++ /dev/null @@ -1,771 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch 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. - */ -package org.elasticsearch.action.bench; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.ElasticsearchIllegalStateException; -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.support.master.MasterNodeOperationRequest; -import org.elasticsearch.client.Client; -import org.elasticsearch.cluster.ClusterService; -import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.ProcessedClusterStateUpdateTask; -import org.elasticsearch.cluster.TimeoutClusterStateUpdateTask; -import org.elasticsearch.cluster.metadata.BenchmarkMetaData; -import org.elasticsearch.cluster.metadata.MetaData; -import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.cluster.node.DiscoveryNodes; -import org.elasticsearch.common.collect.ImmutableOpenMap; -import org.elasticsearch.common.component.AbstractLifecycleComponent; -import org.elasticsearch.common.inject.Inject; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.regex.Regex; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.common.util.concurrent.CountDown; -import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.transport.*; - -import java.io.IOException; -import java.util.*; -import java.util.concurrent.CopyOnWriteArrayList; - -/** - * Service component for running benchmarks - */ -public class BenchmarkService extends AbstractLifecycleComponent { - - private final ThreadPool threadPool; - private final ClusterService clusterService; - private final TransportService transportService; - protected final BenchmarkExecutor executor; - - public static final String ABORT_ACTION_NAME = "indices:data/benchmark/executor/abort"; - public static final String STATUS_ACTION_NAME = "indices:data/benchmark/executor/status"; - public static final String START_ACTION_NAME = "indices:data/benchmark/executor/start"; - - /** - * Constructs a service component for running benchmarks - * - * @param settings Settings - * @param clusterService Cluster service - * @param threadPool Thread pool - * @param client Client - * @param transportService Transport service - */ - @Inject - public BenchmarkService(Settings settings, ClusterService clusterService, ThreadPool threadPool, - Client client, TransportService transportService) { - super(settings); - this.threadPool = threadPool; - this.executor = new BenchmarkExecutor(client, clusterService); - this.clusterService = clusterService; - this.transportService = transportService; - transportService.registerHandler(START_ACTION_NAME, new BenchExecutionHandler()); - transportService.registerHandler(ABORT_ACTION_NAME, new AbortExecutionHandler()); - transportService.registerHandler(STATUS_ACTION_NAME, new StatusExecutionHandler()); - } - - @Override - protected void doStart() throws ElasticsearchException { - } - - @Override - protected void doStop() throws ElasticsearchException { - } - - @Override - protected void doClose() throws ElasticsearchException { - } - - /** - * Lists actively running benchmarks on the cluster - * - * @param request Status request - * @param listener Response listener - */ - public void listBenchmarks(final BenchmarkStatusRequest request, final ActionListener listener) { - - final List nodes = availableBenchmarkNodes(); - if (nodes.size() == 0) { - listener.onResponse(new BenchmarkStatusResponse()); - } else { - BenchmarkStatusAsyncHandler async = new BenchmarkStatusAsyncHandler(nodes.size(), request, listener); - for (DiscoveryNode node : nodes) { - assert isBenchmarkNode(node); - transportService.sendRequest(node, STATUS_ACTION_NAME, new NodeStatusRequest(request), async); - } - } - } - - /** - * Aborts actively running benchmarks on the cluster - * - * @param benchmarkNames Benchmark name(s) to abort - * @param listener Response listener - */ - public void abortBenchmark(final String[] benchmarkNames, final ActionListener listener) { - - final List nodes = availableBenchmarkNodes(); - if (nodes.size() == 0) { - listener.onFailure(new BenchmarkNodeMissingException("No available nodes for executing benchmarks")); - } else { - BenchmarkStateListener benchmarkStateListener = new BenchmarkStateListener() { - @Override - public void onResponse(final ClusterState newState, final List changed) { - if (!changed.isEmpty()) { - threadPool.executor(ThreadPool.Names.GENERIC).execute(new Runnable() { - @Override - public void run() { - Set names = new HashSet<>(); - Set nodeNames = new HashSet<>(); - final ImmutableOpenMap nodes = newState.nodes().nodes(); - - for (BenchmarkMetaData.Entry e : changed) { - names.add(e.benchmarkId()); - nodeNames.addAll(Arrays.asList(e.nodes())); - } - BenchmarkAbortAsyncHandler asyncHandler = new BenchmarkAbortAsyncHandler(nodeNames.size(), listener); - String[] benchmarkNames = names.toArray(new String[names.size()]); - for (String nodeId : nodeNames) { - final DiscoveryNode node = nodes.get(nodeId); - if (node != null) { - transportService.sendRequest(node, ABORT_ACTION_NAME, new NodeAbortRequest(benchmarkNames), asyncHandler); - } else { - asyncHandler.countDown.countDown(); - logger.debug("Node for ID [" + nodeId + "] not found in cluster state - skipping"); - } - } - } - }); - } else { - listener.onFailure(new BenchmarkMissingException("No benchmarks found for " + Arrays.toString(benchmarkNames))); - } - } - - @Override - public void onFailure(Throwable t) { - listener.onFailure(t); - } - }; - clusterService.submitStateUpdateTask("abort_benchmark", new AbortBenchmarkTask(benchmarkNames, benchmarkStateListener)); - } - } - - /** - * Executes benchmarks on the cluster - * - * @param request Benchmark request - * @param listener Response listener - */ - public void startBenchmark(final BenchmarkRequest request, final ActionListener listener) { - - final List nodes = availableBenchmarkNodes(); - if (nodes.size() == 0) { - listener.onFailure(new BenchmarkNodeMissingException("No available nodes for executing benchmark [" + - request.benchmarkName() + "]")); - } else { - final BenchmarkStateListener benchListener = new BenchmarkStateListener() { - @Override - public void onResponse(final ClusterState newState, final List entries) { - threadPool.executor(ThreadPool.Names.GENERIC).execute(new Runnable() { - @Override - public void run() { - assert entries.size() == 1; - BenchmarkMetaData.Entry entry = entries.get(0); - final ImmutableOpenMap nodes = newState.nodes().nodes(); - final BenchmarkSearchAsyncHandler async = new BenchmarkSearchAsyncHandler(entry.nodes().length, request, listener); - for (String nodeId : entry.nodes()) { - final DiscoveryNode node = nodes.get(nodeId); - if (node == null) { - async.handleExceptionInternal( - new ElasticsearchIllegalStateException("Node for ID [" + nodeId + "] not found in cluster state - skipping")); - } else { - logger.debug("Starting benchmark [{}] node [{}]", request.benchmarkName(), node.name()); - transportService.sendRequest(node, START_ACTION_NAME, new NodeBenchRequest(request), async); - } - } - } - }); - } - - @Override - public void onFailure(Throwable t) { - listener.onFailure(t); - } - }; - - clusterService.submitStateUpdateTask("start_benchmark", new StartBenchmarkTask(request, benchListener)); - } - } - - private void finishBenchmark(final BenchmarkResponse benchmarkResponse, final String benchmarkId, final ActionListener listener) { - - clusterService.submitStateUpdateTask("finish_benchmark", new FinishBenchmarkTask("finish_benchmark", benchmarkId, new BenchmarkStateListener() { - @Override - public void onResponse(ClusterState newClusterState, List changed) { - listener.onResponse(benchmarkResponse); - } - - @Override - public void onFailure(Throwable t) { - listener.onFailure(t); - } - }, (benchmarkResponse.state() != BenchmarkResponse.State.ABORTED) && - (benchmarkResponse.state() != BenchmarkResponse.State.FAILED))); - } - - private final boolean isBenchmarkNode(DiscoveryNode node) { - ImmutableMap attributes = node.getAttributes(); - if (attributes.containsKey("bench")) { - String bench = attributes.get("bench"); - return Boolean.parseBoolean(bench); - } - return false; - } - - private List findNodes(BenchmarkRequest request) { - final int numNodes = request.numExecutorNodes(); - final DiscoveryNodes nodes = clusterService.state().nodes(); - DiscoveryNode localNode = nodes.localNode(); - List benchmarkNodes = new ArrayList(); - if (isBenchmarkNode(localNode)) { - benchmarkNodes.add(localNode); - } - for (DiscoveryNode node : nodes) { - if (benchmarkNodes.size() >= numNodes) { - return benchmarkNodes; - } - if (node != localNode && isBenchmarkNode(node)) { - benchmarkNodes.add(node); - } - } - return benchmarkNodes; - } - - private class BenchExecutionHandler extends BaseTransportRequestHandler { - - @Override - public NodeBenchRequest newInstance() { - return new NodeBenchRequest(); - } - - @Override - public void messageReceived(NodeBenchRequest request, TransportChannel channel) throws Exception { - BenchmarkResponse response = executor.benchmark(request.request); - channel.sendResponse(response); - } - - @Override - public String executor() { - return ThreadPool.Names.BENCH; - } - } - - private class StatusExecutionHandler extends BaseTransportRequestHandler { - - @Override - public NodeStatusRequest newInstance() { - return new NodeStatusRequest(); - } - - @Override - public void messageReceived(NodeStatusRequest request, TransportChannel channel) throws Exception { - BenchmarkStatusNodeResponse nodeResponse = executor.benchmarkStatus(); - nodeResponse.nodeName(clusterService.localNode().name()); - channel.sendResponse(nodeResponse); - } - - @Override - public String executor() { - // Perform management tasks on GENERIC so as not to block pending acquisition of a thread from BENCH. - return ThreadPool.Names.GENERIC; - } - } - - private class AbortExecutionHandler extends BaseTransportRequestHandler { - - @Override - public NodeAbortRequest newInstance() { - return new NodeAbortRequest(); - } - - @Override - public void messageReceived(NodeAbortRequest request, TransportChannel channel) throws Exception { - AbortBenchmarkResponse nodeResponse = executor.abortBenchmark(request.benchmarkNames); - channel.sendResponse(nodeResponse); - } - - @Override - public String executor() { - // Perform management tasks on GENERIC so as not to block pending acquisition of a thread from BENCH. - return ThreadPool.Names.GENERIC; - } - } - - public static class NodeAbortRequest extends TransportRequest { - private String[] benchmarkNames; - - public NodeAbortRequest(String[] benchmarkNames) { - this.benchmarkNames = benchmarkNames; - } - - public NodeAbortRequest() { - this(null); - } - - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - benchmarkNames = in.readStringArray(); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - out.writeStringArray(benchmarkNames); - } - } - - public static class NodeStatusRequest extends TransportRequest { - - final BenchmarkStatusRequest request; - - public NodeStatusRequest(BenchmarkStatusRequest request) { - this.request = request; - } - - public NodeStatusRequest() { - this(new BenchmarkStatusRequest()); - } - - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - request.readFrom(in); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - request.writeTo(out); - } - } - - public static class NodeBenchRequest extends TransportRequest { - final BenchmarkRequest request; - - public NodeBenchRequest(BenchmarkRequest request) { - this.request = request; - } - - public NodeBenchRequest() { - this(new BenchmarkRequest()); - } - - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - request.readFrom(in); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - request.writeTo(out); - } - } - - private abstract class CountDownAsyncHandler implements TransportResponseHandler { - - protected final CountDown countDown; - protected final CopyOnWriteArrayList responses = new CopyOnWriteArrayList(); - protected final CopyOnWriteArrayList failures = new CopyOnWriteArrayList(); - - protected CountDownAsyncHandler(int size) { - countDown = new CountDown(size); - } - - public abstract T newInstance(); - - protected abstract void sendResponse(); - - @Override - public void handleResponse(T t) { - responses.add(t); - if (countDown.countDown()) { - sendResponse(); - } - } - - @Override - public void handleException(TransportException t) { - failures.add(t); - logger.error(t.getMessage(), t); - if (countDown.countDown()) { - sendResponse(); - } - } - - public String executor() { - return ThreadPool.Names.SAME; - } - } - - private class BenchmarkAbortAsyncHandler extends CountDownAsyncHandler { - - private final ActionListener listener; - - public BenchmarkAbortAsyncHandler(int size, ActionListener listener) { - super(size); - this.listener = listener; - } - - @Override - public AbortBenchmarkResponse newInstance() { - return new AbortBenchmarkResponse(); - } - - @Override - protected void sendResponse() { - boolean acked = true; - for (AbortBenchmarkResponse nodeResponse : responses) { - if (!nodeResponse.isAcknowledged()) { - acked = false; - break; - } - } - listener.onResponse(new AbortBenchmarkResponse(acked)); - } - } - - private class BenchmarkStatusAsyncHandler extends CountDownAsyncHandler { - - private final BenchmarkStatusRequest request; - private final ActionListener listener; - - public BenchmarkStatusAsyncHandler(int nodeCount, final BenchmarkStatusRequest request, ActionListener listener) { - super(nodeCount); - this.request = request; - this.listener = listener; - } - - @Override - public BenchmarkStatusNodeResponse newInstance() { - return new BenchmarkStatusNodeResponse(); - } - - @Override - protected void sendResponse() { - int activeBenchmarks = 0; - BenchmarkStatusResponse consolidatedResponse = new BenchmarkStatusResponse(); - Map> nameNodeResponseMap = new HashMap<>(); - - // Group node responses by benchmark name - for (BenchmarkStatusNodeResponse nodeResponse : responses) { - for (BenchmarkResponse benchmarkResponse : nodeResponse.benchResponses()) { - List benchmarkResponses = nameNodeResponseMap.get(benchmarkResponse.benchmarkName()); - if (benchmarkResponses == null) { - benchmarkResponses = new ArrayList<>(); - nameNodeResponseMap.put(benchmarkResponse.benchmarkName(), benchmarkResponses); - } - benchmarkResponses.add(benchmarkResponse); - } - activeBenchmarks += nodeResponse.activeBenchmarks(); - } - - for (Map.Entry> entry : nameNodeResponseMap.entrySet()) { - BenchmarkResponse consolidated = consolidateBenchmarkResponses(entry.getValue()); - consolidatedResponse.addBenchResponse(consolidated); - } - - consolidatedResponse.totalActiveBenchmarks(activeBenchmarks); - listener.onResponse(consolidatedResponse); - } - } - - private BenchmarkResponse consolidateBenchmarkResponses(List responses) { - BenchmarkResponse response = new BenchmarkResponse(); - - // Merge node responses into a single consolidated response - List errors = new ArrayList<>(); - for (BenchmarkResponse r : responses) { - for (Map.Entry entry : r.competitionResults.entrySet()) { - if (!response.competitionResults.containsKey(entry.getKey())) { - response.competitionResults.put(entry.getKey(), - new CompetitionResult( - entry.getKey(), entry.getValue().concurrency(), entry.getValue().multiplier(), - false, entry.getValue().percentiles())); - } - CompetitionResult cr = response.competitionResults.get(entry.getKey()); - cr.nodeResults().addAll(entry.getValue().nodeResults()); - } - if (r.hasErrors()) { - for (String error : r.errors()) { - errors.add(error); - } - } - - if (response.benchmarkName() == null) { - response.benchmarkName(r.benchmarkName()); - } - assert response.benchmarkName().equals(r.benchmarkName()); - if (!errors.isEmpty()) { - response.errors(errors.toArray(new String[errors.size()])); - } - response.mergeState(r.state()); - assert errors.isEmpty() || response.state() != BenchmarkResponse.State.COMPLETE : "Response can't be complete since it has errors"; - } - - return response; - } - - private class BenchmarkSearchAsyncHandler extends CountDownAsyncHandler { - - private final ActionListener listener; - private final BenchmarkRequest request; - - public BenchmarkSearchAsyncHandler(int size, BenchmarkRequest request, ActionListener listener) { - super(size); - this.listener = listener; - this.request = request; - } - - @Override - public BenchmarkResponse newInstance() { - return new BenchmarkResponse(); - } - - @Override - protected void sendResponse() { - BenchmarkResponse response = consolidateBenchmarkResponses(responses); - response.benchmarkName(request.benchmarkName()); - response.verbose(request.verbose()); - finishBenchmark(response, request.benchmarkName(), listener); - } - - public void handleExceptionInternal(Throwable t) { - failures.add(t); - if (countDown.countDown()) { - sendResponse(); - } - } - } - - public static interface BenchmarkStateListener { - - void onResponse(ClusterState newClusterState, List changed); - - void onFailure(Throwable t); - } - - public final class StartBenchmarkTask extends BenchmarkStateChangeAction { - - private final BenchmarkStateListener stateListener; - private List newBenchmark = new ArrayList<>(); - - public StartBenchmarkTask(BenchmarkRequest request, BenchmarkStateListener stateListener) { - super(request); - this.stateListener = stateListener; - } - - @Override - public ClusterState execute(ClusterState currentState) { - MetaData metaData = currentState.getMetaData(); - BenchmarkMetaData bmd = metaData.custom(BenchmarkMetaData.TYPE); - MetaData.Builder mdBuilder = MetaData.builder(metaData); - ImmutableList.Builder builder = ImmutableList.builder(); - - if (bmd != null) { - for (BenchmarkMetaData.Entry entry : bmd.entries()) { - if (request.benchmarkName().equals(entry.benchmarkId())) { - if (entry.state() != BenchmarkMetaData.State.SUCCESS && entry.state() != BenchmarkMetaData.State.FAILED) { - throw new ElasticsearchException("A benchmark with ID [" + request.benchmarkName() + "] is already running in state [" + entry.state() + "]"); - } - // just drop the entry it it has finished successfully or it failed! - } else { - builder.add(entry); - } - } - } - List nodes = findNodes(request); - String[] nodeIds = new String[nodes.size()]; - int i = 0; - for (DiscoveryNode node : nodes) { - nodeIds[i++] = node.getId(); - } - BenchmarkMetaData.Entry entry = new BenchmarkMetaData.Entry(request.benchmarkName(), BenchmarkMetaData.State.STARTED, nodeIds); - newBenchmark.add(entry); - bmd = new BenchmarkMetaData(builder.add(entry).build()); - mdBuilder.putCustom(BenchmarkMetaData.TYPE, bmd); - return ClusterState.builder(currentState).metaData(mdBuilder).build(); - } - - @Override - public void onFailure(String source, Throwable t) { - logger.warn("Failed to start benchmark: [{}]", t, request.benchmarkName()); - newBenchmark = null; - stateListener.onFailure(t); - } - - @Override - public void clusterStateProcessed(String source, ClusterState oldState, final ClusterState newState) { - if (newBenchmark != null) { - stateListener.onResponse(newState, newBenchmark); - } - } - - @Override - public TimeValue timeout() { - return request.masterNodeTimeout(); - } - } - - public final class FinishBenchmarkTask extends UpdateBenchmarkStateTask { - - private final boolean success; - - public FinishBenchmarkTask(String reason, String benchmarkId, BenchmarkStateListener listener, boolean success) { - super(reason, benchmarkId, listener); - this.success = success; - } - - @Override - protected BenchmarkMetaData.Entry process(BenchmarkMetaData.Entry entry) { - BenchmarkMetaData.State state = entry.state(); - assert state == BenchmarkMetaData.State.STARTED || state == BenchmarkMetaData.State.ABORTED : "Expected state: STARTED or ABORTED but was: " + entry.state(); - if (success) { - return new BenchmarkMetaData.Entry(entry, BenchmarkMetaData.State.SUCCESS); - } else { - return new BenchmarkMetaData.Entry(entry, BenchmarkMetaData.State.FAILED); - } - } - } - - public final class AbortBenchmarkTask extends UpdateBenchmarkStateTask { - private final String[] patterns; - - public AbortBenchmarkTask(String[] patterns, BenchmarkStateListener listener) { - super("abort_benchmark", null, listener); - this.patterns = patterns; - } - - protected boolean match(BenchmarkMetaData.Entry entry) { - return entry.state() == BenchmarkMetaData.State.STARTED && Regex.simpleMatch(this.patterns, benchmarkId); - } - - @Override - protected BenchmarkMetaData.Entry process(BenchmarkMetaData.Entry entry) { - return new BenchmarkMetaData.Entry(entry, BenchmarkMetaData.State.ABORTED); - } - } - - public abstract class UpdateBenchmarkStateTask extends ProcessedClusterStateUpdateTask { - - private final String reason; - protected final String benchmarkId; - private final BenchmarkStateListener listener; - private final List instances = new ArrayList<>(); - - - protected UpdateBenchmarkStateTask(String reason, String benchmarkId, BenchmarkStateListener listener) { - this.reason = reason; - this.listener = listener; - this.benchmarkId = benchmarkId; - } - - protected boolean match(BenchmarkMetaData.Entry entry) { - return entry.benchmarkId().equals(this.benchmarkId); - } - - @Override - public ClusterState execute(ClusterState currentState) { - MetaData metaData = currentState.getMetaData(); - BenchmarkMetaData bmd = metaData.custom(BenchmarkMetaData.TYPE); - MetaData.Builder mdBuilder = MetaData.builder(metaData); - if (bmd != null && !bmd.entries().isEmpty()) { - ImmutableList.Builder builder = new ImmutableList.Builder(); - for (BenchmarkMetaData.Entry e : bmd.entries()) { - if (benchmarkId == null || match(e)) { - e = process(e); - instances.add(e); - } - // Don't keep finished benchmarks around in cluster state - if (e != null && (e.state() != BenchmarkMetaData.State.SUCCESS && - e.state() != BenchmarkMetaData.State.ABORTED && - e.state() != BenchmarkMetaData.State.FAILED)) { - builder.add(e); - } - } - if (instances.isEmpty()) { - throw new ElasticsearchException("No Benchmark found for id: [" + benchmarkId + "]"); - } - bmd = new BenchmarkMetaData(builder.build()); - } - if (bmd != null) { - mdBuilder.putCustom(BenchmarkMetaData.TYPE, bmd); - } - return ClusterState.builder(currentState).metaData(mdBuilder).build(); - } - - protected abstract BenchmarkMetaData.Entry process(BenchmarkMetaData.Entry entry); - - @Override - public void onFailure(String source, Throwable t) { - logger.warn("Failed updating benchmark state for ID [{}] triggered by: [{}]", t, benchmarkId, reason); - listener.onFailure(t); - } - - @Override - public void clusterStateProcessed(String source, ClusterState oldState, final ClusterState newState) { - listener.onResponse(newState, instances); - } - - public String reason() { - return reason; - } - } - - public abstract class BenchmarkStateChangeAction extends TimeoutClusterStateUpdateTask { - protected final R request; - - public BenchmarkStateChangeAction(R request) { - this.request = request; - } - - @Override - public TimeValue timeout() { - return request.masterNodeTimeout(); - } - } - - private List availableBenchmarkNodes() { - DiscoveryNodes nodes = clusterService.state().nodes(); - List benchmarkNodes = new ArrayList<>(nodes.size()); - for (DiscoveryNode node : nodes) { - if (isBenchmarkNode(node)) { - benchmarkNodes.add(node); - } - } - return benchmarkNodes; - } -} diff --git a/src/main/java/org/elasticsearch/action/bench/BenchmarkStatusNodeResponse.java b/src/main/java/org/elasticsearch/action/bench/BenchmarkStatusNodeResponse.java deleted file mode 100644 index 21056d66c36de..0000000000000 --- a/src/main/java/org/elasticsearch/action/bench/BenchmarkStatusNodeResponse.java +++ /dev/null @@ -1,111 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch 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. - */ - -package org.elasticsearch.action.bench; - -import org.elasticsearch.action.ActionResponse; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.io.stream.Streamable; -import org.elasticsearch.common.xcontent.ToXContent; -import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.common.xcontent.XContentFactory; - -import java.io.IOException; -import java.util.List; -import java.util.ArrayList; - -/** - * Node-level response for the status of an on-going benchmark - */ -public class BenchmarkStatusNodeResponse extends ActionResponse implements Streamable, ToXContent { - - private String nodeName; - private List benchmarkResponses; - - public BenchmarkStatusNodeResponse() { - benchmarkResponses = new ArrayList<>(); - } - - public void nodeName(String nodeName) { - this.nodeName = nodeName; - } - - public String nodeName() { - return nodeName; - } - - public void addBenchResponse(BenchmarkResponse benchmarkResponse) { - benchmarkResponses.add(benchmarkResponse); - } - - public List benchResponses() { - return benchmarkResponses; - } - - public int activeBenchmarks() { - return benchResponses().size(); - } - - @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - builder.field("node", nodeName); - builder.startArray("responses"); - for (BenchmarkResponse benchmarkResponse : benchmarkResponses) { - benchmarkResponse.toXContent(builder, params); - } - builder.endArray(); - return builder; - } - - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - nodeName = in.readString(); - int size = in.readVInt(); - for (int i = 0; i < size; i++) { - BenchmarkResponse br = new BenchmarkResponse(); - br.readFrom(in); - benchmarkResponses.add(br); - } - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - out.writeString(nodeName); - out.writeVInt(benchmarkResponses.size()); - for (BenchmarkResponse br : benchmarkResponses) { - br.writeTo(out); - } - } - - @Override - public String toString() { - try { - XContentBuilder builder = XContentFactory.jsonBuilder().prettyPrint(); - builder.startObject(); - toXContent(builder, EMPTY_PARAMS); - builder.endObject(); - return builder.string(); - } catch (IOException e) { - return "{ \"error\" : \"" + e.getMessage() + "\"}"; - } - } -} diff --git a/src/main/java/org/elasticsearch/action/bench/BenchmarkStatusResponse.java b/src/main/java/org/elasticsearch/action/bench/BenchmarkStatusResponse.java deleted file mode 100644 index eeb19c0d79c88..0000000000000 --- a/src/main/java/org/elasticsearch/action/bench/BenchmarkStatusResponse.java +++ /dev/null @@ -1,96 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch 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. - */ -package org.elasticsearch.action.bench; - -import org.elasticsearch.action.ActionResponse; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.io.stream.Streamable; -import org.elasticsearch.common.xcontent.ToXContent; -import org.elasticsearch.common.xcontent.XContentBuilder; - -import java.io.IOException; -import java.util.Arrays; -import java.util.List; -import java.util.ArrayList; - -/** - * Benchmark status response - */ -public class BenchmarkStatusResponse extends ActionResponse implements Streamable, ToXContent { - - private int totalActiveBenchmarks = 0; - private final List benchmarkResponses = new ArrayList<>(); - - public BenchmarkStatusResponse() { } - - public void addBenchResponse(BenchmarkResponse response) { - benchmarkResponses.add(response); - } - - public List benchmarkResponses() { - return benchmarkResponses; - } - - public void totalActiveBenchmarks(int totalActiveBenchmarks) { - this.totalActiveBenchmarks = totalActiveBenchmarks; - } - - public int totalActiveBenchmarks() { - return totalActiveBenchmarks; - } - - @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - - if (benchmarkResponses.size() > 0) { - builder.startObject("active_benchmarks"); - for (BenchmarkResponse benchmarkResponse : benchmarkResponses) { - builder.startObject(benchmarkResponse.benchmarkName()); - benchmarkResponse.toXContent(builder, params); - builder.endObject(); - } - builder.endObject(); - } - - return builder; - } - - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - totalActiveBenchmarks = in.readVInt(); - int size = in.readVInt(); - for (int i = 0; i < size; i++) { - BenchmarkResponse br = new BenchmarkResponse(); - br.readFrom(in); - benchmarkResponses.add(br); - } - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - out.writeVInt(totalActiveBenchmarks); - out.writeVInt(benchmarkResponses.size()); - for (BenchmarkResponse br : benchmarkResponses) { - br.writeTo(out); - } - } -} diff --git a/src/main/java/org/elasticsearch/action/benchmark/AbstractBenchmarkService.java b/src/main/java/org/elasticsearch/action/benchmark/AbstractBenchmarkService.java new file mode 100644 index 0000000000000..a6e9964e64b0a --- /dev/null +++ b/src/main/java/org/elasticsearch/action/benchmark/AbstractBenchmarkService.java @@ -0,0 +1,234 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch 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. + */ + +package org.elasticsearch.action.benchmark; + +import com.google.common.collect.ImmutableList; + +import org.elasticsearch.action.benchmark.start.BenchmarkStartRequest; +import org.elasticsearch.cluster.ClusterService; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ClusterStateListener; +import org.elasticsearch.cluster.ProcessedClusterStateUpdateTask; +import org.elasticsearch.cluster.metadata.BenchmarkMetaData; +import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.common.component.AbstractComponent; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.*; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +/** + * Abstract base class for benchmark coordinator and executor services. + */ +public abstract class AbstractBenchmarkService extends AbstractComponent implements ClusterStateListener { + + protected final ClusterService clusterService; + protected final TransportService transportService; + protected final ThreadPool threadPool; + + protected AbstractBenchmarkService(Settings settings, ClusterService clusterService, TransportService transportService, + ThreadPool threadPool) { + + super(settings); + + this.clusterService = clusterService; + this.transportService = transportService; + this.threadPool = threadPool; + + clusterService.add(this); + } + + /** + * Request class for fetching benchmark definition from master to executor nodes + */ + public class BenchmarkDefinitionTransportRequest extends BaseNodeTransportRequest { + + public BenchmarkDefinitionTransportRequest() { } + + public BenchmarkDefinitionTransportRequest(String benchmarkId, String nodeId) { + super(benchmarkId, nodeId); + } + } + + /** + * Response class for transmitting benchmark definitions to executor nodes + */ + public class BenchmarkDefinitionActionResponse extends BaseNodeActionResponse { + + protected BenchmarkStartRequest benchmarkStartRequest; + + public BenchmarkDefinitionActionResponse() { } + + public BenchmarkDefinitionActionResponse(BenchmarkStartRequest benchmarkStartRequest, String nodeId) { + super(benchmarkStartRequest.benchmarkId(), nodeId); + this.benchmarkStartRequest = benchmarkStartRequest; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + benchmarkStartRequest = new BenchmarkStartRequest(); + benchmarkStartRequest.readFrom(in); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + benchmarkStartRequest.writeTo(out); + } + } + + /** + * Request class for informing master of node state changes + */ + public class NodeStateUpdateTransportRequest extends BaseNodeTransportRequest { + + protected BenchmarkMetaData.Entry.NodeState state; + + public NodeStateUpdateTransportRequest() { } + + public NodeStateUpdateTransportRequest(String benchmarkId, String nodeId, BenchmarkMetaData.Entry.NodeState state) { + super(benchmarkId, nodeId); + this.state = state; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + state = BenchmarkMetaData.Entry.NodeState.fromId(in.readByte()); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeByte(state.id()); + } + } + + /** + * Response class for acknowledging status update requests + */ + public class NodeStateUpdateActionResponse extends BaseNodeActionResponse { + + public NodeStateUpdateActionResponse() { } + + public NodeStateUpdateActionResponse(String benchmarkId, String nodeId) { + super(benchmarkId, nodeId); + } + } + + /* ** Utilities ** */ + + protected boolean isMasterNode() { + return clusterService.state().nodes().localNodeMaster(); + } + + protected boolean isBenchmarkNode() { + return BenchmarkUtility.isBenchmarkNode(clusterService.localNode()); + } + + protected String nodeId() { + return clusterService.localNode().id(); + } + + protected DiscoveryNode master() { + return clusterService.state().nodes().masterNode(); + } + + public class NodeStateUpdateRequestHandler extends BaseTransportRequestHandler { + + public static final String ACTION = "indices:data/benchmark/node/state/update"; + + @Override + public NodeStateUpdateTransportRequest newInstance() { + return new NodeStateUpdateTransportRequest(); + } + + @Override + public void messageReceived(final NodeStateUpdateTransportRequest request, final TransportChannel channel) throws Exception { + + logger.debug("benchmark [{}]: state update request [{}] from [{}]", request.benchmarkId, request.state, request.nodeId); + + final String cause = "benchmark-update-node-state (" + request.benchmarkId + ":" + request.nodeId + ":" + request.state + ")"; + + clusterService.submitStateUpdateTask(cause, new ProcessedClusterStateUpdateTask() { + + @Override + public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { + try { + channel.sendResponse(new NodeStateUpdateActionResponse(request.benchmarkId, request.nodeId)); + } + catch (Throwable t) { + onFailure(source, t); + } + } + + @Override + public ClusterState execute(ClusterState state) throws Exception { + + final BenchmarkMetaData meta = state.metaData().custom(BenchmarkMetaData.TYPE); + final ImmutableList.Builder builder = ImmutableList.builder(); + + for (BenchmarkMetaData.Entry entry : meta.entries()) { + if (!entry.benchmarkId().equals(request.benchmarkId)) { + builder.add(entry); + } else { + final Map map = new HashMap<>(); + for (Map.Entry mapEntry : entry.nodeStateMap().entrySet()) { + if (mapEntry.getKey().equals(request.nodeId)) { + map.put(mapEntry.getKey(), request.state); // Update node state with requested state + } else { + map.put(mapEntry.getKey(), mapEntry.getValue()); + } + } + builder.add(new BenchmarkMetaData.Entry(request.benchmarkId, entry.masterNodeId(), entry.state(), map)); + } + } + + final MetaData.Builder metabuilder = MetaData.builder(state.metaData()); + metabuilder.putCustom(BenchmarkMetaData.TYPE, new BenchmarkMetaData(builder.build())); + + return ClusterState.builder(state).metaData(metabuilder).build(); + } + + @Override + public void onFailure(String source, Throwable t) { + try { + logger.error(source, t); + channel.sendResponse(t); + } catch (Throwable e) { + logger.error("Unable to send failure response: {}", source, e); + } + } + }); + } + + @Override + public String executor() { + return ThreadPool.Names.SAME; + } + } +} diff --git a/src/main/java/org/elasticsearch/action/benchmark/BaseNodeActionResponse.java b/src/main/java/org/elasticsearch/action/benchmark/BaseNodeActionResponse.java new file mode 100644 index 0000000000000..c3db3c761c1fc --- /dev/null +++ b/src/main/java/org/elasticsearch/action/benchmark/BaseNodeActionResponse.java @@ -0,0 +1,88 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch 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. + */ + +package org.elasticsearch.action.benchmark; + +import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentBuilderString; + +import java.io.IOException; + +/** + * Base class for action response payloads. + */ +public abstract class BaseNodeActionResponse extends ActionResponse implements ToXContent { + + public String nodeId; + public String benchmarkId; + public String error; + + public BaseNodeActionResponse() { } + + public BaseNodeActionResponse(final String benchmarkId, final String nodeId) { + this.benchmarkId = benchmarkId; + this.nodeId = nodeId; + } + + public void error(String error) { + this.error = error; + } + + public String error() { + return error; + } + + public boolean hasErrors() { + return !(error == null || error.isEmpty()); + } + + @Override + public void readFrom(StreamInput in) throws IOException { + benchmarkId = in.readString(); + nodeId = in.readString(); + error = in.readOptionalString(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(benchmarkId); + out.writeString(nodeId); + out.writeOptionalString(error); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.field(Fields.BENCHMARK_ID, benchmarkId); + builder.field(Fields.NODE_ID, nodeId); + if (error != null && !error.isEmpty()) { + builder.field(Fields.ERROR, error); + } + return builder; + } + + static final class Fields { + static final XContentBuilderString NODE_ID = new XContentBuilderString("node_id"); + static final XContentBuilderString BENCHMARK_ID = new XContentBuilderString("benchmark_id"); + static final XContentBuilderString ERROR = new XContentBuilderString("error"); + } +} diff --git a/src/main/java/org/elasticsearch/action/bench/AbortBenchmarkResponse.java b/src/main/java/org/elasticsearch/action/benchmark/BaseNodeTransportRequest.java similarity index 65% rename from src/main/java/org/elasticsearch/action/bench/AbortBenchmarkResponse.java rename to src/main/java/org/elasticsearch/action/benchmark/BaseNodeTransportRequest.java index f8bc65adb67ef..c5387d93082e8 100644 --- a/src/main/java/org/elasticsearch/action/bench/AbortBenchmarkResponse.java +++ b/src/main/java/org/elasticsearch/action/benchmark/BaseNodeTransportRequest.java @@ -16,35 +16,41 @@ * specific language governing permissions and limitations * under the License. */ -package org.elasticsearch.action.bench; -import org.elasticsearch.action.support.master.AcknowledgedResponse; +package org.elasticsearch.action.benchmark; + import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.transport.TransportRequest; import java.io.IOException; /** - * Response for a benchmark abort request + * Base class for transport request payloads. */ -public class AbortBenchmarkResponse extends AcknowledgedResponse { - public AbortBenchmarkResponse() { - super(); - } +public abstract class BaseNodeTransportRequest extends TransportRequest { + + public String benchmarkId; + public String nodeId; + + public BaseNodeTransportRequest() { } - public AbortBenchmarkResponse(boolean acknowledged) { - super(acknowledged); + public BaseNodeTransportRequest(final String benchmarkId, final String nodeId) { + this.benchmarkId = benchmarkId; + this.nodeId = nodeId; } @Override public void readFrom(StreamInput in) throws IOException { super.readFrom(in); - readAcknowledged(in); + benchmarkId = in.readString(); + nodeId = in.readString(); } @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - writeAcknowledged(out); + out.writeString(benchmarkId); + out.writeString(nodeId); } } diff --git a/src/main/java/org/elasticsearch/action/benchmark/BatchedResponse.java b/src/main/java/org/elasticsearch/action/benchmark/BatchedResponse.java new file mode 100644 index 0000000000000..65b5798127904 --- /dev/null +++ b/src/main/java/org/elasticsearch/action/benchmark/BatchedResponse.java @@ -0,0 +1,165 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch 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. + */ + +package org.elasticsearch.action.benchmark; + +import com.google.common.collect.ImmutableMap; +import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.cluster.metadata.BenchmarkMetaData; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Streamable; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentBuilderString; + +import java.io.IOException; +import java.util.Collection; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +/** + * Common response format for commands that operate on potentially more than + * a single benchmark at a time, e.g. pause/resume/abort + */ +public abstract class BatchedResponse extends ActionResponse implements ToXContent { + + private Map responses = new ConcurrentHashMap<>(); + + public void addNodeResponse(final String benchmarkId, final String nodeId, final BenchmarkMetaData.Entry.NodeState nodeState) { + BenchmarkResponse response = responses.get(benchmarkId); + if (response == null) { + response = new BenchmarkResponse(benchmarkId); + responses.put(benchmarkId, response); + } + response.addNodeResponse(nodeId, nodeState); + } + + public BenchmarkResponse getResponse(final String benchmarkId) { + return responses.get(benchmarkId); + } + + public Collection getResponses() { + return responses.values(); + } + + public static class BenchmarkResponse implements ToXContent, Streamable { + + private String benchmarkId; + private Map nodeResponses; + + public BenchmarkResponse() { + this(null); + } + + public BenchmarkResponse(final String benchmarkId) { + this.benchmarkId = benchmarkId; + this.nodeResponses = new ConcurrentHashMap<>(); + } + + public void addNodeResponse(final String nodeId, final BenchmarkMetaData.Entry.NodeState nodeState) { + nodeResponses.put(nodeId, nodeState); + } + + public Map nodeResponses() { + return ImmutableMap.copyOf(nodeResponses); + } + + @Override + public void readFrom(StreamInput in) throws IOException { + benchmarkId = in.readString(); + final int size = in.readVInt(); + nodeResponses = new ConcurrentHashMap<>(size); + for (int i = 0; i < size; i++) { + final String s = in.readString(); + final BenchmarkMetaData.Entry.NodeState ns = BenchmarkMetaData.Entry.NodeState.fromId(in.readByte()); + nodeResponses.put(s, ns); + } + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(benchmarkId); + out.writeVInt(nodeResponses.size()); + for (Map.Entry ns : nodeResponses.entrySet()) { + out.writeString(ns.getKey()); + out.writeByte(ns.getValue().id()); + } + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(Fields.BENCHMARK); + builder.field(Fields.ID, benchmarkId); + builder.startArray(Fields.NODES); + for (Map.Entry ns : nodeResponses.entrySet()) { + builder.startObject(); + builder.field(Fields.NODE, ns.getKey()); + builder.field(Fields.STATE, ns.getValue()); + builder.endObject(); + } + builder.endArray(); + builder.endObject(); + return builder; + } + + static final class Fields { + static final XContentBuilderString ID = new XContentBuilderString("id"); + static final XContentBuilderString BENCHMARK = new XContentBuilderString("benchmark"); + static final XContentBuilderString NODE = new XContentBuilderString("node"); + static final XContentBuilderString NODES = new XContentBuilderString("nodes"); + static final XContentBuilderString STATE = new XContentBuilderString("state"); + } + } + + @Override + public void readFrom(StreamInput in) throws IOException { + responses = new ConcurrentHashMap<>(); + final int size = in.readVInt(); + for (int i = 0; i < size; i++) { + final String benchmarkId = in.readString(); + final BenchmarkResponse response = new BenchmarkResponse(); + response.readFrom(in); + responses.put(benchmarkId, response); + } + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVInt(responses.size()); + for (final Map.Entry entry : responses.entrySet()) { + out.writeString(entry.getKey()); + entry.getValue().writeTo(out); + } + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(Fields.BENCHMARKS); + for (final BenchmarkResponse response : responses.values()) { + response.toXContent(builder, params); + } + builder.endObject(); + return builder; + } + + static final class Fields { + static final XContentBuilderString BENCHMARKS = new XContentBuilderString("benchmarks"); + } +} diff --git a/src/main/java/org/elasticsearch/action/bench/BenchmarkCompetitor.java b/src/main/java/org/elasticsearch/action/benchmark/BenchmarkCompetitor.java similarity index 94% rename from src/main/java/org/elasticsearch/action/bench/BenchmarkCompetitor.java rename to src/main/java/org/elasticsearch/action/benchmark/BenchmarkCompetitor.java index dc45d4a7e41b3..66c80ab971bd6 100644 --- a/src/main/java/org/elasticsearch/action/bench/BenchmarkCompetitor.java +++ b/src/main/java/org/elasticsearch/action/benchmark/BenchmarkCompetitor.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.elasticsearch.action.bench; +package org.elasticsearch.action.benchmark; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ValidateActions; @@ -47,16 +47,16 @@ public class BenchmarkCompetitor implements Streamable { * Constructs a competition across the given indices * @param indices Indices */ - BenchmarkCompetitor(String... indices) { + public BenchmarkCompetitor(String... indices) { settings.indices(indices); } /** * Constructs a competition */ - BenchmarkCompetitor() { } + public BenchmarkCompetitor() { } - ActionRequestValidationException validate(ActionRequestValidationException validationException) { + public ActionRequestValidationException validate(ActionRequestValidationException validationException) { if (name == null) { validationException = ValidateActions.addValidationError("name must not be null", validationException); } diff --git a/src/main/java/org/elasticsearch/action/bench/BenchmarkCompetitorBuilder.java b/src/main/java/org/elasticsearch/action/benchmark/BenchmarkCompetitorBuilder.java similarity index 97% rename from src/main/java/org/elasticsearch/action/bench/BenchmarkCompetitorBuilder.java rename to src/main/java/org/elasticsearch/action/benchmark/BenchmarkCompetitorBuilder.java index df12be0988e23..d974d70bd4953 100644 --- a/src/main/java/org/elasticsearch/action/bench/BenchmarkCompetitorBuilder.java +++ b/src/main/java/org/elasticsearch/action/benchmark/BenchmarkCompetitorBuilder.java @@ -16,11 +16,11 @@ * specific language governing permissions and limitations * under the License. */ -package org.elasticsearch.action.bench; +package org.elasticsearch.action.benchmark; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchType; -import org.elasticsearch.action.bench.BenchmarkSettings.ClearCachesSettings; +import org.elasticsearch.action.benchmark.BenchmarkSettings.ClearCachesSettings; /** * Builder for a benchmark competitor diff --git a/src/main/java/org/elasticsearch/action/benchmark/BenchmarkCoordinatorService.java b/src/main/java/org/elasticsearch/action/benchmark/BenchmarkCoordinatorService.java new file mode 100644 index 0000000000000..c5632540a0c88 --- /dev/null +++ b/src/main/java/org/elasticsearch/action/benchmark/BenchmarkCoordinatorService.java @@ -0,0 +1,776 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch 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. + */ +package org.elasticsearch.action.benchmark; + +import com.google.common.base.Joiner; +import com.google.common.collect.UnmodifiableIterator; + +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.ElasticsearchIllegalStateException; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.benchmark.abort.*; +import org.elasticsearch.action.benchmark.pause.*; +import org.elasticsearch.action.benchmark.resume.*; +import org.elasticsearch.action.benchmark.start.*; +import org.elasticsearch.action.benchmark.status.*; +import org.elasticsearch.action.benchmark.exception.*; +import org.elasticsearch.cluster.*; +import org.elasticsearch.cluster.metadata.*; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.node.DiscoveryNodes; +import org.elasticsearch.common.collect.ImmutableOpenMap; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.regex.Regex; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.CountDown; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.*; + +import java.util.*; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * Coordinates execution of benchmarks. + * + * This class is responsible for coordinating the cluster metadata associated with each benchmark. It + * listens for cluster change events via + * {@link org.elasticsearch.cluster.ClusterStateListener#clusterChanged(org.elasticsearch.cluster.ClusterChangedEvent)} + * and responds by initiating the appropriate sequence of actions necessary to service each event. Since this class + * is primarily focused on the management of cluster state, it checks to see if it is executing on the master and, if not, + * simply does nothing. + * + * There is a related class, {@link org.elasticsearch.action.benchmark.BenchmarkExecutorService} which communicates + * with, and is coordinated by, this class. It's role is to manage the actual execution of benchmarks on the assigned + * nodes. + * + * The typical lifecycle of a benchmark is as follows: + * + * 1. Client submits a benchmark, thereby creating a metadata entry with state INITIALIZING. + * 2. Executor nodes notice the metadata change via + * {@link org.elasticsearch.action.benchmark.BenchmarkExecutorService#clusterChanged(org.elasticsearch.cluster.ClusterChangedEvent)} + * and initialize themselves. + * 3. Executor nodes call back to the coordinator (using the transport service) to get description of the benchmark. Once received, + * executors update their node state to READY. + * 4. Once the coordinator receives READY from all executors, it updates the metadata state to RUNNING. + * 5. Executor nodes notice the metadata change to state RUNNING and start actually executing the benchmark. + * 6. As each executor finishes, it updates its state to COMPLETE. + * 7. Once the coordinator receives COMPLETE from all executors, it requests the results (using the transport service) from + * each executor. + * 8. After all results have been received, the coordinator updates the benchmark state to COMPLETE. + * 9. On receipt of the COMPLETE state, the coordinator removes the benchmark from the cluster metadata and + * returns the results back to the client. + */ +public class BenchmarkCoordinatorService extends AbstractBenchmarkService { + + protected final BenchmarkStateManager manager; + protected final Map benchmarks = new ConcurrentHashMap<>(); + + /** + * Constructs a service component for running benchmarks + */ + @Inject + public BenchmarkCoordinatorService(Settings settings, ClusterService clusterService, ThreadPool threadPool, + TransportService transportService, BenchmarkStateManager manager) { + + super(settings, clusterService, transportService, threadPool); + this.manager = manager; + transportService.registerHandler(NodeStateUpdateRequestHandler.ACTION, new NodeStateUpdateRequestHandler()); + transportService.registerHandler(BenchmarkDefinitionRequestHandler.ACTION, new BenchmarkDefinitionRequestHandler()); + } + + /** + * Listens for and responds to state transitions. + * + * @param event Cluster change event + */ + @Override + public void clusterChanged(ClusterChangedEvent event) { + + if (isMasterNode() && event.nodesDelta().removed()) { + updateNodeLiveness(event.nodesDelta()); + processOrphanedBenchmarks(); + } + + final BenchmarkMetaData meta = event.state().metaData().custom(BenchmarkMetaData.TYPE); + final BenchmarkMetaData prev = event.previousState().metaData().custom(BenchmarkMetaData.TYPE); + + if (!isMasterNode() || !event.metaDataChanged() || meta == null || meta.entries().size() == 0) { + return; + } + + for (final BenchmarkMetaData.Entry entry : BenchmarkMetaData.addedOrChanged(prev, meta)) { + + logger.info(entry.toString()); + final State state = benchmarks.get(entry.benchmarkId()); + if (state == null) { + // Remove any unknown benchmark state from the cluster metadata + logger.warn("benchmark [{}]: unknown benchmark in cluster metadata", entry.benchmarkId()); + manager.clear(entry.benchmarkId(), new ActionListener() { + @Override + public void onResponse(Object o) { /* no-op */ } + @Override + public void onFailure(Throwable e) { + logger.error("benchmark [{}]: failed to remove unknown benchmark from metadata", e, entry.benchmarkId()); + } + }); + continue; + } + + if (allNodesFailed(entry)) { + logger.error("benchmark [{}]: all nodes failed", entry.benchmarkId()); + state.failed(new ElasticsearchException("All nodes failed")); + continue; + } + + switch (entry.state()) { + case INITIALIZING: + if (allNodesReady(entry) && state.canStartRunning()) { + // Once all executors have initialized and reported 'ready', we can update the benchmark's + // top-level state, thereby signalling to the executors that it's okay to begin execution. + state.ready(); + } + break; + case RUNNING: + if (allNodesFinished(entry) && state.canStopRunning()) { + // Once all executors have completed, successfully or otherwise, we can fetch the benchmark's + // results from each executor node, merge them into a single top-level result, and update + // the benchmark's top-level state. + state.finished(entry); + } + break; + case RESUMING: + if (allNodesRunning(entry) && state.canResumeRunning()) { + assert state.batchedResponder != null; + state.batchedResponder.resumed(entry); + } + break; + case PAUSED: + if (allNodesPaused(entry) && state.canPauseRunning()) { + assert state.batchedResponder != null; + state.batchedResponder.paused(entry); + } + break; + case COMPLETED: + if (state.canComplete()) { + state.completed(); + } + break; + case FAILED: + state.failed(new ElasticsearchException("benchmark [" + entry.benchmarkId() + "]: failed")); + break; + case ABORTED: + if (allNodesAborted(entry)) { + assert state.batchedResponder != null; + state.batchedResponder.aborted(entry); + } + break; + default: + throw new ElasticsearchIllegalStateException("benchmark [" + entry.benchmarkId() + "]: illegal state [" + entry.state() + "]"); + } + } + } + + /* ** Public API Methods ** */ + + /** + * Starts a benchmark. Sets top-level and per-node state to INITIALIZING. + * @param request Benchmark request + * @param listener Response listener + */ + public void startBenchmark(final BenchmarkStartRequest request, + final ActionListener listener) { + + preconditions(request.numExecutorNodes()); + + manager.start(request, new ActionListener() { + + @Override + public void onResponse(BenchmarkStateManager.BenchmarkCreationStatus creationStatus) { + if (creationStatus.created()) { + assert null == benchmarks.get(request.benchmarkId()); + benchmarks.put(request.benchmarkId(), new State(request, creationStatus.nodeIds(), listener)); + } else { + onFailure(new ElasticsearchIllegalStateException("benchmark [" + request.benchmarkId() + "]: aborted due to master failure")); + } + } + + @Override + public void onFailure(Throwable e) { + listener.onFailure(e); + } + }); + } + + /** + * Reports on the status of running benchmarks + * @param request Status request + * @param listener Response listener + */ + public void listBenchmarks(final BenchmarkStatusRequest request, + final ActionListener listener) { + + final BenchmarkMetaData meta = clusterService.state().metaData().custom(BenchmarkMetaData.TYPE); + + if (BenchmarkUtility.executors(clusterService.state().nodes(), 1).size() == 0 || meta == null || meta.entries().size() == 0) { + listener.onResponse(new BenchmarkStatusResponses()); + return; + } + + final BenchmarkStatusResponses responses = new BenchmarkStatusResponses(); + + for (final BenchmarkMetaData.Entry entry : meta.entries()) { + + if (request.benchmarkIdPatterns() == null || + request.benchmarkIdPatterns().length == 0 || + Regex.simpleMatch(request.benchmarkIdPatterns(), entry.benchmarkId())) { + + try { + responses.add(manager.status(entry)); + } catch (Throwable t) { + logger.error("benchmark [{}]: failed to read status", t, entry.benchmarkId()); + listener.onFailure(t); + return; + } + } + } + + listener.onResponse(responses); + } + + /** + * Pauses a benchmark(s) + * @param request Pause request + * @param listener Response listener + */ + public void pauseBenchmark(final BenchmarkPauseRequest request, + final ActionListener listener) { + + preconditions(1); + + manager.pause(request, new ActionListener() { + + @Override + public void onResponse(final String[] benchmarkIds) { + if (benchmarkIds == null || benchmarkIds.length == 0) { + listener.onFailure(new BenchmarkMissingException( + "No benchmarks found matching: [" + Joiner.on(",").join(request.benchmarkIdPatterns()) + "]")); + } else { + final BatchedResponder responder = + new BatchedResponder<>(new BenchmarkPauseResponse(), listener, new CountDown(benchmarkIds.length)); + + for (final String benchmarkId : benchmarkIds) { + final State state = benchmarks.get(benchmarkId); + if (state == null) { + throw new ElasticsearchIllegalStateException("benchmark [" + benchmarkId + "]: missing internal state"); + } + + state.batchedResponder = responder; + } + } + } + + @Override + public void onFailure(Throwable t) { + listener.onFailure(t); + } + }); + } + + /** + * Resumes a previously paused benchmark(s) + * @param request Resume request + * @param listener Response listener + */ + public void resumeBenchmark(final BenchmarkResumeRequest request, + final ActionListener listener) { + + preconditions(1); + + manager.resume(request, new ActionListener() { + + @Override + public void onResponse(final String[] benchmarkIds) { + if (benchmarkIds == null || benchmarkIds.length == 0) { + listener.onFailure(new BenchmarkMissingException( + "No benchmarks found matching: [" + Joiner.on(",").join(request.benchmarkIdPatterns()) + "]")); + } else { + final BatchedResponder responder = + new BatchedResponder<>(new BenchmarkResumeResponse(), listener, new CountDown(benchmarkIds.length)); + + for (final String benchmarkId : benchmarkIds) { + final State state = benchmarks.get(benchmarkId); + if (state == null) { + throw new ElasticsearchIllegalStateException("benchmark [" + benchmarkId + "]: missing internal state"); + } + + state.batchedResponder = responder; + } + } + } + + @Override + public void onFailure(Throwable t) { + listener.onFailure(t); + } + }); + } + + /** + * Aborts a running benchmark(s) + * @param request Abort request + * @param listener Response listener + */ + public void abortBenchmark(final BenchmarkAbortRequest request, + final ActionListener listener) { + + preconditions(1); + + manager.abort(request, new ActionListener() { + + @Override + public void onResponse(final String[] benchmarkIds) { + if (benchmarkIds == null || benchmarkIds.length == 0) { + listener.onFailure(new BenchmarkMissingException( + "No benchmarks found matching: [" + Joiner.on(",").join(request.benchmarkIdPatterns()) + "]")); + } else { + final BatchedResponder responder = + new BatchedResponder<>(new BenchmarkAbortResponse(), listener, new CountDown(benchmarkIds.length)); + + for (final String benchmarkId : benchmarkIds) { + final State state = benchmarks.get(benchmarkId); + if (state == null) { + throw new ElasticsearchIllegalStateException("benchmark [" + benchmarkId + "]: missing internal state"); + } + + state.batchedResponder = responder; + } + } + } + + @Override + public void onFailure(Throwable t) { + listener.onFailure(t); + } + }); + } + + /* ** State Change Listeners ** */ + + /** + * Certain client requests may operate on multiple benchmarks in a single request by + * passing wildcard patterns. In such cases this class is used to wait on events from + * all matching benchmarks and respond to the caller only after all have completed the + * requested action. + * + * @param The type of the response payload + */ + private final class BatchedResponder { + + final T response; + final CountDown countDown; + final ActionListener listener; + + BatchedResponder(final T response, final ActionListener listener, final CountDown countDown) { + this.response = response; + this.listener = listener; + this.countDown = countDown; + } + + /** + * Called when top-level state has been reported as {@link org.elasticsearch.cluster.metadata.BenchmarkMetaData.State#ABORTED} + * and all executor nodes have reported {@link org.elasticsearch.cluster.metadata.BenchmarkMetaData.Entry.NodeState#ABORTED}. + */ + void aborted(final BenchmarkMetaData.Entry entry) { + try { + addResponse(entry); + // Initiate completion sequence; send partial results back to original caller + final State state = benchmarks.get(entry.benchmarkId()); + state.finished(entry); + } finally { + if (countDown.countDown()) { + listener.onResponse(response); + } + } + } + + /** + * Called when top-level state has been reported as {@link org.elasticsearch.cluster.metadata.BenchmarkMetaData.State#RESUMING}. + */ + void resumed(final BenchmarkMetaData.Entry entry) { + try { + addResponse(entry); + final State state = benchmarks.get(entry.benchmarkId()); + manager.update(entry.benchmarkId(), BenchmarkMetaData.State.RUNNING, BenchmarkMetaData.Entry.NodeState.RUNNING, + state.liveness, + new ActionListener() { + @Override + public void onResponse(Object o) { /* no-op */ } + @Override + public void onFailure(Throwable e) { + state.failed(e); + } + }); + } finally { + if (countDown.countDown()) { + listener.onResponse(response); + } + } + } + + /** + * Called when top-level state has been reported as {@link org.elasticsearch.cluster.metadata.BenchmarkMetaData.State#PAUSED} + * and all executor nodes have reported {@link org.elasticsearch.cluster.metadata.BenchmarkMetaData.Entry.NodeState#PAUSED}. + */ + void paused(final BenchmarkMetaData.Entry entry) { + try { + addResponse(entry); + } finally { + if (countDown.countDown()) { + listener.onResponse(response); + } + } + } + + void addResponse(final BenchmarkMetaData.Entry entry) { + for (Map.Entry e : entry.nodeStateMap().entrySet()) { + response.addNodeResponse(entry.benchmarkId(), e.getKey(), e.getValue()); + } + } + } + + /* ** Utilities ** */ + + protected static final class Liveness { + + private final AtomicBoolean liveness; + + Liveness() { + liveness = new AtomicBoolean(true); + } + + public boolean alive() { + return liveness.get(); + } + + public boolean set(boolean expected, boolean updated) { + return liveness.compareAndSet(expected, updated); + } + } + + private final class State { + + final String benchmarkId; + final BenchmarkStartRequest request; + final ImmutableOpenMap liveness; + final List errorMessages = new ArrayList<>(); + BenchmarkStartResponse response; + + AtomicBoolean running = new AtomicBoolean(false); + AtomicBoolean complete = new AtomicBoolean(false); + AtomicBoolean paused = new AtomicBoolean(false); + + ActionListener listener; + BatchedResponder batchedResponder; + + State(final BenchmarkStartRequest request, final List nodeIds, final ActionListener listener) { + + this.benchmarkId = request.benchmarkId(); + this.request = request; + this.listener = listener; + + ImmutableOpenMap.Builder builder = ImmutableOpenMap.builder(); + for (final String nodeId : nodeIds) { + builder.put(nodeId, new Liveness()); + } + liveness = builder.build(); + } + + /** + * Called when all executor nodes have reported {@link org.elasticsearch.cluster.metadata.BenchmarkMetaData.Entry.NodeState#READY}. + * Sets top-level state to {@link org.elasticsearch.cluster.metadata.BenchmarkMetaData.State#RUNNING} and + * per-node state to {@link org.elasticsearch.cluster.metadata.BenchmarkMetaData.Entry.NodeState#RUNNING}. + */ + void ready() { + + manager.update(benchmarkId, BenchmarkMetaData.State.RUNNING, BenchmarkMetaData.Entry.NodeState.RUNNING, + liveness, + new ActionListener() { + @Override + public void onResponse(Object o) { /* no-op */ } + + @Override + public void onFailure(Throwable e) { + failed(e); + } + }); + } + + /** + * Called when all executor nodes have completed, successfully or otherwise. + * Fetches benchmark response and sets state. + * Sets top-level state to {@link org.elasticsearch.cluster.metadata.BenchmarkMetaData.State#COMPLETED} and + * per-node state to {@link org.elasticsearch.cluster.metadata.BenchmarkMetaData.Entry.NodeState#COMPLETED}. + */ + void finished(final BenchmarkMetaData.Entry entry) { + + try { + response = manager.status(entry); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + logger.error("benchmark [{}]: failed to read status", e, entry.benchmarkId()); + failed(e); + return; + } + + manager.update(entry.benchmarkId(), BenchmarkMetaData.State.COMPLETED, BenchmarkMetaData.Entry.NodeState.COMPLETED, + liveness, + new ActionListener() { + @Override + public void onResponse(Object o) { /* no-op */ } + + @Override + public void onFailure(Throwable e) { + failed(e); + } + }); + } + + /** + * Called when top-level state has been reported as {@link org.elasticsearch.cluster.metadata.BenchmarkMetaData.State#COMPLETED}. + */ + void completed() { + + manager.clear(benchmarkId, new ActionListener() { + @Override + public void onResponse(Object o) { + benchmarks.remove(benchmarkId); + sendResponse(); + } + + @Override + public void onFailure(Throwable e) { + benchmarks.remove(benchmarkId); + listener.onFailure(e); + } + }); + } + + void failed(final Throwable cause) { + + manager.clear(benchmarkId, new ActionListener() { + @Override + public void onResponse(Object o) { + internalFailed(cause); + } + + @Override + public void onFailure(Throwable e) { + internalFailed(cause); + } + }); + } + + private void internalFailed(final Throwable cause) { + + benchmarks.remove(benchmarkId); + if (response == null) { + response = new BenchmarkStartResponse(benchmarkId); + } + + response.state(BenchmarkStartResponse.State.FAILED); + if (cause != null) { + response.errors(cause.getMessage()); + } + + sendResponse(); + } + + private void sendResponse() { + if (response == null) { + listener.onFailure(new ElasticsearchIllegalStateException("benchmark [" + benchmarkId + "]: missing response")); + } else { + response.errors(errorMessages); + listener.onResponse(response); + } + } + + boolean isNodeAlive(final String nodeId) { + return liveness.containsKey(nodeId) && liveness.get(nodeId).alive(); + } + + boolean canStartRunning() { + return running.compareAndSet(false, true); + } + + boolean canStopRunning() { + return running.compareAndSet(true, false); + } + + boolean canPauseRunning() { + return paused.compareAndSet(false, true); + } + + boolean canResumeRunning() { + return paused.compareAndSet(true, false); + } + + boolean canComplete() { + return complete.compareAndSet(false, true); + } + } + + /** + * Record nodes that dropped out of the cluster + * @param delta Delta of discovery nodes from last cluster state update + */ + private void updateNodeLiveness(final DiscoveryNodes.Delta delta) { + + for (final DiscoveryNode node : delta.removedNodes()) { + for (Map.Entry entry : benchmarks.entrySet()) { + if (entry.getValue().isNodeAlive(node.id())) { + final Liveness liveness = entry.getValue().liveness.get(node.id()); + if (liveness != null) { + if (liveness.set(true, false)) { + logger.warn("benchmark [{}]: marked node [{}] as not live", entry.getKey(), node.id()); + entry.getValue().errorMessages.add("node: [" + node.id() + "] dropped out of cluster"); + } + } + } + } + } + } + + /** + * Find benchmarks for which all executor nodes have dropped from cluster state + * and forcefully fail them. + */ + private void processOrphanedBenchmarks() { + + for (Map.Entry entry : benchmarks.entrySet()) { + boolean hasLiveNodes = false; + UnmodifiableIterator iter = entry.getValue().liveness.keysIt(); + while (iter.hasNext()) { + if (entry.getValue().isNodeAlive(iter.next())) { + hasLiveNodes = true; + } + } + + if (!hasLiveNodes) { + logger.warn("benchmark [{}]: has no live nodes; manually killing it", entry.getKey()); + entry.getValue().failed(new ElasticsearchException( + "benchmark [" + entry.getKey() + "]: all executor nodes dropped out of cluster")); + } + } + } + + protected void preconditions(int num) { + final int n = BenchmarkUtility.executors(clusterService.state().nodes(), num).size(); + if (n < num) { + throw new BenchmarkNodeMissingException( + "Insufficient executor nodes in cluster: require at least [" + num + "] found [" + n + "]"); + } + } + + private boolean allNodesFailed(final BenchmarkMetaData.Entry entry) { + return checkAllNodeStates(entry, BenchmarkMetaData.Entry.NodeState.FAILED); + } + + private boolean allNodesReady(final BenchmarkMetaData.Entry entry) { + return checkAllNodeStates(entry, BenchmarkMetaData.Entry.NodeState.READY); + } + + private boolean allNodesRunning(final BenchmarkMetaData.Entry entry) { + return checkAllNodeStates(entry, BenchmarkMetaData.Entry.NodeState.RUNNING); + } + + private boolean allNodesAborted(final BenchmarkMetaData.Entry entry) { + return checkAllNodeStates(entry, BenchmarkMetaData.Entry.NodeState.ABORTED); + } + + private boolean allNodesPaused(final BenchmarkMetaData.Entry entry) { + return checkAllNodeStates(entry, BenchmarkMetaData.Entry.NodeState.PAUSED); + } + + private boolean checkAllNodeStates(final BenchmarkMetaData.Entry entry, final BenchmarkMetaData.Entry.NodeState nodeState) { + for (Map.Entry e : entry.nodeStateMap().entrySet()) { + if (e.getValue() == BenchmarkMetaData.Entry.NodeState.FAILED) { + continue; // Failed nodes don't factor in + } + final State state = benchmarks.get(entry.benchmarkId()); + if (state != null && !state.isNodeAlive(e.getKey())) { + continue; // Dead nodes don't factor in + } + if (e.getValue() != nodeState) { + return false; + } + } + return true; + } + + private static final EnumSet NOT_FINISHED = EnumSet.of( + BenchmarkMetaData.Entry.NodeState.INITIALIZING, BenchmarkMetaData.Entry.NodeState.READY, + BenchmarkMetaData.Entry.NodeState.RUNNING, BenchmarkMetaData.Entry.NodeState.PAUSED); + + private boolean allNodesFinished(final BenchmarkMetaData.Entry entry) { + for (Map.Entry e : entry.nodeStateMap().entrySet()) { + + final State state = benchmarks.get(entry.benchmarkId()); + if (state != null && !state.isNodeAlive(e.getKey())) { + continue; // Dead nodes don't factor in + } + + if (NOT_FINISHED.contains(e.getValue())) { + return false; + } + } + return true; + } + + /* ** Request Handlers ** */ + + /** + * Responds to requests from the executor nodes to transmit the definition for the given benchmark. + */ + public class BenchmarkDefinitionRequestHandler extends BaseTransportRequestHandler { + + public static final String ACTION = "indices:data/benchmark/node/definition"; + + @Override + public BenchmarkDefinitionTransportRequest newInstance() { + return new BenchmarkDefinitionTransportRequest(); + } + + @Override + public void messageReceived(BenchmarkDefinitionTransportRequest request, TransportChannel channel) throws Exception { + if (benchmarks.get(request.benchmarkId) != null) { + final BenchmarkDefinitionActionResponse response = + new BenchmarkDefinitionActionResponse(benchmarks.get(request.benchmarkId).request, request.nodeId); + channel.sendResponse(response); + } else { + channel.sendResponse(new ElasticsearchIllegalStateException("benchmark [" + request.benchmarkId + "]: missing internal state")); + } + } + + @Override + public String executor() { + return ThreadPool.Names.SAME; + } + } +} diff --git a/src/main/java/org/elasticsearch/action/benchmark/BenchmarkExecutor.java b/src/main/java/org/elasticsearch/action/benchmark/BenchmarkExecutor.java new file mode 100644 index 0000000000000..de20734c7c5ee --- /dev/null +++ b/src/main/java/org/elasticsearch/action/benchmark/BenchmarkExecutor.java @@ -0,0 +1,366 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch 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. + */ +package org.elasticsearch.action.benchmark; + +import org.apache.lucene.util.PriorityQueue; + +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.benchmark.start.*; +import org.elasticsearch.action.benchmark.status.*; +import org.elasticsearch.action.benchmark.competition.*; +import org.elasticsearch.action.benchmark.exception.*; +import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.client.Client; +import org.elasticsearch.cluster.ClusterService; +import org.elasticsearch.common.logging.ESLogger; +import org.elasticsearch.common.logging.Loggers; + +import java.util.*; +import java.util.concurrent.*; + +/** + * Handles execution, listing, pausing, and aborting of benchmarks + */ +public class BenchmarkExecutor { + + private static final ESLogger logger = Loggers.getLogger(BenchmarkExecutor.class); + + private final Client client; + protected final ClusterService clusterService; + + public BenchmarkExecutor(Client client, ClusterService clusterService) { + this.client = client; + this.clusterService = clusterService; + } + + public BenchmarkStatusNodeActionResponse start(BenchmarkStartRequest request, BenchmarkStartResponse benchmarkStartResponse, + BenchmarkExecutorService.BenchmarkSemaphores benchmarkSemaphores) throws ElasticsearchException { + try { + for (BenchmarkCompetitor competitor : request.competitors()) { + + final BenchmarkSettings settings = competitor.settings(); + logger.debug("Executing [iterations: {}] [multiplier: {}] for [{}] on [{}]", + settings.iterations(), settings.multiplier(), request.benchmarkId(), nodeName()); + + final List competitionIterations = new ArrayList<>(settings.iterations()); + final CompetitionResult competitionResult = + new CompetitionResult(competitor.name(), settings.concurrency(), settings.multiplier(), request.verbose(), request.percentiles()); + final CompetitionNodeResult competitionNodeResult = + new CompetitionNodeResult(competitor.name(), nodeName(), settings.iterations(), competitionIterations); + + competitionResult.addCompetitionNodeResult(competitionNodeResult); + benchmarkStartResponse.competitionResults().put(competitor.name(), competitionResult); + + // Make sure headers and context are passed through to all searches + final List searchRequests = bindOriginalRequest(competitor.settings().searchRequests(), request); + + // Perform warmup if requested + if (settings.warmup()) { + final long start = System.nanoTime(); + final List errors = warmup(searchRequests, benchmarkSemaphores.competitorSemaphore(competitor.name())); + competitionNodeResult.warmUpTime(TimeUnit.MILLISECONDS.convert(System.nanoTime() - start, TimeUnit.NANOSECONDS)); + if (!errors.isEmpty()) { + throw new BenchmarkExecutionException("Failed to execute warmup phase", errors); + } + } + + for (int i = 0; i < settings.iterations(); i++) { + + final Measurements measurements = new Measurements(settings.multiplier() * searchRequests.size()); + + // Run the iteration + final CompetitionIteration ci = + iterate(competitor, searchRequests, measurements, benchmarkSemaphores.competitorSemaphore(competitor.name())); + + ci.percentiles(request.percentiles()); + competitionIterations.add(ci); + competitionNodeResult.incrementCompletedIterations(); + } + + competitionNodeResult.totalExecutedQueries(settings.multiplier() * searchRequests.size() * settings.iterations()); + } + + benchmarkStartResponse.state(BenchmarkStartResponse.State.COMPLETED); + + } catch (BenchmarkExecutionException e) { + benchmarkStartResponse.state(BenchmarkStartResponse.State.FAILED); + benchmarkStartResponse.errors(e.errorMessages().toArray(new String[e.errorMessages().size()])); + } catch (InterruptedException ex) { + Thread.currentThread().interrupt(); + benchmarkStartResponse.state(BenchmarkStartResponse.State.ABORTED); + } catch (Throwable ex) { + logger.debug("Unexpected exception during benchmark", ex); + benchmarkStartResponse.state(BenchmarkStartResponse.State.FAILED); + benchmarkStartResponse.errors(ex.getMessage()); + } finally { + benchmarkSemaphores.stopAllCompetitors(); + } + + final BenchmarkStatusNodeActionResponse response = new BenchmarkStatusNodeActionResponse(request.benchmarkId(), nodeId()); + response.response(benchmarkStartResponse); + return response; + } + + protected List warmup(List searchRequests, StoppableSemaphore semaphore) throws InterruptedException { + + final CountDownLatch totalCount = new CountDownLatch(searchRequests.size()); + final CopyOnWriteArrayList errorMessages = new CopyOnWriteArrayList<>(); + + for (SearchRequest searchRequest : searchRequests) { + semaphore.acquire(); + client.search(searchRequest, new BoundsManagingActionListener(semaphore, totalCount, errorMessages) { } ); + } + totalCount.await(); + return errorMessages; + } + + protected CompetitionIteration iterate(final BenchmarkCompetitor competitor, + final List searchRequests, + final Measurements measurements, + final StoppableSemaphore semaphore) throws InterruptedException { + + assert measurements.timeBuckets().length == competitor.settings().multiplier() * searchRequests.size(); + assert measurements.docBuckets().length == competitor.settings().multiplier() * searchRequests.size(); + + int id = 0; + final CountDownLatch totalCount = new CountDownLatch(measurements.size()); + final CopyOnWriteArrayList errorMessages = new CopyOnWriteArrayList<>(); + final long start = System.nanoTime(); + + for (int i = 0; i < competitor.settings().multiplier(); i++) { + for (SearchRequest searchRequest : searchRequests) { + StatisticCollectionActionListener statsListener = + new StatisticCollectionActionListener(semaphore, measurements, id++, totalCount, errorMessages); + semaphore.acquire(); + client.search(searchRequest, statsListener); + } + } + + totalCount.await(); + assert id == measurements.timeBuckets().length; + final long totalTime = TimeUnit.MILLISECONDS.convert(System.nanoTime() - start, TimeUnit.NANOSECONDS); + + if (!errorMessages.isEmpty()) { + throw new BenchmarkExecutionException("Too many execution failures", errorMessages); + } + + final CompetitionIterationData iterationData = new CompetitionIterationData(measurements.timeBuckets()); + final long sumDocs = new CompetitionIterationData(measurements.docBuckets()).sum(); + + // Don't track slowest request if there is only one request as that is redundant + CompetitionIteration.SlowRequest[] topN = null; + if ((competitor.settings().numSlowest() > 0) && (searchRequests.size() > 1)) { + topN = getTopN(measurements.timeBuckets(), searchRequests, competitor.settings().multiplier(), competitor.settings().numSlowest()); + } + + return new CompetitionIteration(topN, totalTime, measurements.timeBuckets().length, sumDocs, iterationData); + } + + private CompetitionIteration.SlowRequest[] getTopN(long[] buckets, List requests, int multiplier, int topN) { + + final int numRequests = requests.size(); + // collect the top N + final PriorityQueue topNQueue = new PriorityQueue(topN) { + @Override + protected boolean lessThan(IndexAndTime a, IndexAndTime b) { + return a.avgTime < b.avgTime; + } + }; + assert multiplier > 0; + for (int i = 0; i < numRequests; i++) { + long sum = 0; + long max = Long.MIN_VALUE; + for (int j = 0; j < multiplier; j++) { + final int base = (numRequests * j); + sum += buckets[i + base]; + max = Math.max(buckets[i + base], max); + } + final long avg = sum / multiplier; + if (topNQueue.size() < topN) { + topNQueue.add(new IndexAndTime(i, max, avg)); + } else if (topNQueue.top().avgTime < avg) { + topNQueue.top().update(i, max, avg); + topNQueue.updateTop(); + } + } + + final CompetitionIteration.SlowRequest[] slowRequests = new CompetitionIteration.SlowRequest[topNQueue.size()]; + int i = topNQueue.size() - 1; + + while (topNQueue.size() > 0) { + IndexAndTime pop = topNQueue.pop(); + CompetitionIteration.SlowRequest slow = + new CompetitionIteration.SlowRequest(pop.avgTime, pop.maxTime, requests.get(pop.index)); + slowRequests[i--] = slow; + } + + return slowRequests; + } + + private static class IndexAndTime { + int index; + long maxTime; + long avgTime; + + public IndexAndTime(int index, long maxTime, long avgTime) { + this.index = index; + this.maxTime = maxTime; + this.avgTime = avgTime; + } + + public void update(int index, long maxTime, long avgTime) { + this.index = index; + this.maxTime = maxTime; + this.avgTime = avgTime; + } + } + + protected static class Measurements { + + private final long[] timeBuckets; + private final long[] docBuckets; + + public Measurements(final int size) { + timeBuckets = new long[size]; + docBuckets = new long[size]; + Arrays.fill(timeBuckets, -1); // wipe CPU cache ;) + Arrays.fill(docBuckets, -1); // wipe CPU cache ;) + } + + public long[] timeBuckets() { + return timeBuckets; + } + + public long[] docBuckets() { + return docBuckets; + } + + public int size() { + return timeBuckets.length; + } + + public void invalidate(final int bucketId) { + timeBuckets[bucketId] = -1; + docBuckets[bucketId] = -1; + } + } + + private static class BoundsManagingActionListener implements ActionListener { + + private final StoppableSemaphore semaphore; + private final CountDownLatch latch; + private final CopyOnWriteArrayList errorMessages; + + public BoundsManagingActionListener(StoppableSemaphore semaphore, CountDownLatch latch, CopyOnWriteArrayList errorMessages) { + this.semaphore = semaphore; + this.latch = latch; + this.errorMessages = errorMessages; + } + + private void manage() { + try { + semaphore.release(); + } finally { + latch.countDown(); + } + } + + public void onResponse(Response response) { + manage(); + } + + public void onFailure(Throwable e) { + try { + if (errorMessages.size() < 5) { + logger.debug("Failed to execute benchmark [{}]", e.getMessage(), e); + e = ExceptionsHelper.unwrapCause(e); + errorMessages.add(e.getLocalizedMessage()); + } + } finally { + manage(); + } + } + } + + private static class StatisticCollectionActionListener extends BoundsManagingActionListener { + + private final int bucketId; + private final Measurements measurements; + + public StatisticCollectionActionListener(StoppableSemaphore semaphore, Measurements measurements, + int bucketId, CountDownLatch totalCount, + CopyOnWriteArrayList errorMessages) { + super(semaphore, totalCount, errorMessages); + this.bucketId = bucketId; + this.measurements = measurements; + } + + @Override + public void onResponse(SearchResponse searchResponse) { + super.onResponse(searchResponse); + measurements.timeBuckets()[bucketId] = searchResponse.getTookInMillis(); + if (searchResponse.getHits() != null) { + measurements.docBuckets()[bucketId] = searchResponse.getHits().getTotalHits(); + } + } + + @Override + public void onFailure(Throwable e) { + try { + measurements.invalidate(bucketId); + } finally { + super.onFailure(e); + } + } + } + + protected String nodeName() { + return clusterService.localNode().name(); + } + + protected String nodeId() { + return clusterService.localNode().id(); + } + + private final boolean assertBuckets(long[] buckets) { + for (int i = 0; i < buckets.length; i++) { + assert buckets[i] >= 0 : "Bucket value was negative: " + buckets[i] + " bucket id: " + i; + } + return true; + } + + /** + * Bind the originating request to the benchmark's search requests so that the headers and context are passed through. + * + * @param searchRequests Benchmark search requests + * @param originalRequest Originating action request + * @return Benchmark search requests re-constructed to use headers and context of originating request + */ + private List bindOriginalRequest(final List searchRequests, final ActionRequest originalRequest) { + final List newSearchRequests = new ArrayList<>(searchRequests.size()); + for (final SearchRequest searchRequest : searchRequests) { + newSearchRequests.add(new SearchRequest(searchRequest, originalRequest)); + } + return newSearchRequests; + } +} diff --git a/src/main/java/org/elasticsearch/action/benchmark/BenchmarkExecutorService.java b/src/main/java/org/elasticsearch/action/benchmark/BenchmarkExecutorService.java new file mode 100644 index 0000000000000..10d8f4533741b --- /dev/null +++ b/src/main/java/org/elasticsearch/action/benchmark/BenchmarkExecutorService.java @@ -0,0 +1,519 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch 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. + */ + +package org.elasticsearch.action.benchmark; + +import com.carrotsearch.hppc.cursors.ObjectObjectCursor; +import org.elasticsearch.ElasticsearchIllegalStateException; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.benchmark.competition.CompetitionResult; +import org.elasticsearch.action.benchmark.start.*; +import org.elasticsearch.action.benchmark.status.*; +import org.elasticsearch.cluster.metadata.BenchmarkMetaData; +import org.elasticsearch.client.Client; +import org.elasticsearch.cluster.*; +import org.elasticsearch.common.collect.ImmutableOpenMap; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.logging.ESLogger; +import org.elasticsearch.common.logging.Loggers; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.*; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.TimeUnit; + + +/** + * Manages the execution of benchmarks. + * + * See {@link org.elasticsearch.action.benchmark.BenchmarkCoordinatorService} for a description of how + * benchmark communication works. + */ +public class BenchmarkExecutorService extends AbstractBenchmarkService { + + protected final BenchmarkExecutor executor; + protected final Map states = new ConcurrentHashMap<>(); + + private static final long TIMEOUT = 60; + private static final TimeUnit TIMEUNIT = TimeUnit.SECONDS; + + @Inject + public BenchmarkExecutorService(Settings settings, ClusterService clusterService, ThreadPool threadPool, + Client client, TransportService transportService) { + + this(settings, clusterService, threadPool, transportService, new BenchmarkExecutor(client, clusterService)); + } + + protected BenchmarkExecutorService(Settings settings, ClusterService clusterService, ThreadPool threadPool, + TransportService transportService, BenchmarkExecutor executor) { + + super(settings, clusterService, transportService, threadPool); + this.executor = executor; + transportService.registerHandler(BenchmarkStatusRequestHandler.ACTION, new BenchmarkStatusRequestHandler()); + } + + @Override + public void clusterChanged(ClusterChangedEvent event) { + + final BenchmarkMetaData meta = event.state().metaData().custom(BenchmarkMetaData.TYPE); + final BenchmarkMetaData prev = event.previousState().metaData().custom(BenchmarkMetaData.TYPE); + + if (!isBenchmarkNode() || !event.metaDataChanged() || meta == null || meta.entries().size() == 0) { + return; + } + + for (final BenchmarkMetaData.Entry entry : BenchmarkMetaData.addedOrChanged(prev, meta)) { + + if (entry.nodeStateMap().get(nodeId()) == null) { // Benchmark not assigned to this node. Skip it. + continue; + } + + if (states.get(entry.benchmarkId()) == null) { + if (entry.state() == BenchmarkMetaData.State.INITIALIZING && + entry.nodeStateMap().get(nodeId()) == BenchmarkMetaData.Entry.NodeState.INITIALIZING) { + states.put(entry.benchmarkId(), new State(entry.benchmarkId())); + } else { + logger.error("benchmark [{}]: missing internal state", entry.benchmarkId()); + continue; + } + } + + final State state = states.get(entry.benchmarkId()); + + switch (entry.state()) { + case INITIALIZING: + if (entry.nodeStateMap().get(nodeId()) != BenchmarkMetaData.Entry.NodeState.INITIALIZING) { + break; // Benchmark has already been initialized on this node + } + + if (state.initialize()) { + // Fetch benchmark definition from master + logger.debug("benchmark [{}]: fetching definition", entry.benchmarkId()); + final BenchmarkDefinitionResponseHandler handler = new BenchmarkDefinitionResponseHandler(entry.benchmarkId()); + + threadPool.executor(ThreadPool.Names.GENERIC).execute(new Runnable() { + @Override + public void run() { + transportService.sendRequest( + master(), + BenchmarkCoordinatorService.BenchmarkDefinitionRequestHandler.ACTION, + new BenchmarkDefinitionTransportRequest(entry.benchmarkId(), nodeId()), + handler); + } + }); + } + break; + case RUNNING: + if (entry.nodeStateMap().get(nodeId()) != BenchmarkMetaData.Entry.NodeState.RUNNING) { + break; + } + + if (state.start()) { + final ActionListener listener = new ActionListener() { + @Override + public void onResponse(BenchmarkStartResponse response) { + logger.debug("benchmark [{}]: completed [{}]", response.benchmarkId(), response.state()); + updateNodeState(response.benchmarkId(), nodeId(), convertToNodeState(response.state())); + } + + @Override + public void onFailure(Throwable t) { + logger.error(t.getMessage(), t); + updateNodeState(entry.benchmarkId(), nodeId(), BenchmarkMetaData.Entry.NodeState.FAILED); + } + }; + + logger.debug("benchmark [{}]: starting execution", entry.benchmarkId()); + + threadPool.executor(ThreadPool.Names.BENCH).execute(new Runnable() { + @Override + public void run() { + try { + final BenchmarkStatusNodeActionResponse response = + executor.start(state.request, state.response, state.benchmarkSemaphores); + state.response = response.response(); + state.complete = true; + listener.onResponse(state.response); + } catch (Throwable t) { + listener.onFailure(t); + } + } + }); + } + break; + case RESUMING: + if (entry.nodeStateMap().get(nodeId()) != BenchmarkMetaData.Entry.NodeState.PAUSED) { + break; + } + + try { + if (state.resume()) { + logger.debug("benchmark [{}]: resuming execution", entry.benchmarkId()); + updateNodeState(entry.benchmarkId(), nodeId(), BenchmarkMetaData.Entry.NodeState.RUNNING); + } + } catch (Exception e) { + logger.error("benchmark [{}]: failed to resume", e, entry.benchmarkId()); + updateNodeState(entry.benchmarkId(), nodeId(), BenchmarkMetaData.Entry.NodeState.FAILED); + } + break; + case PAUSED: + if (entry.nodeStateMap().get(nodeId()) != BenchmarkMetaData.Entry.NodeState.RUNNING && + entry.nodeStateMap().get(nodeId()) != BenchmarkMetaData.Entry.NodeState.READY) { + break; + } + + try { + if (state.pause()) { + logger.debug("benchmark [{}]: pausing execution", entry.benchmarkId()); + updateNodeState(entry.benchmarkId(), nodeId(), BenchmarkMetaData.Entry.NodeState.PAUSED); + } + } catch (Exception e) { + logger.error("benchmark [{}]: failed to pause", e, entry.benchmarkId()); + updateNodeState(entry.benchmarkId(), nodeId(), BenchmarkMetaData.Entry.NodeState.FAILED); + } + break; + case ABORTED: + if (entry.nodeStateMap().get(nodeId()) == BenchmarkMetaData.Entry.NodeState.COMPLETED && + entry.nodeStateMap().get(nodeId()) == BenchmarkMetaData.Entry.NodeState.ABORTED && + entry.nodeStateMap().get(nodeId()) == BenchmarkMetaData.Entry.NodeState.FAILED) { + break; + } + + try { + if (state.abort()) { + logger.debug("benchmark [{}]: aborting execution", entry.benchmarkId()); + updateNodeState(entry.benchmarkId(), nodeId(), BenchmarkMetaData.Entry.NodeState.ABORTED); + } + } catch (Exception e) { + logger.error("benchmark [{}]: failed to abort", e, entry.benchmarkId()); + updateNodeState(entry.benchmarkId(), nodeId(), BenchmarkMetaData.Entry.NodeState.FAILED); + } + break; + case COMPLETED: + states.remove(entry.benchmarkId()); + logger.debug("benchmark [{}]: completed", entry.benchmarkId()); + break; + default: + throw new ElasticsearchIllegalStateException("benchmark [" + entry.benchmarkId() + "]: illegal state [" + entry.state() + "]"); + } + } + } + + /** + * Response handler for benchmark definition requests. The payload is the benchmark definition which + * is transmitted to us from the master. We use this to know how to execute the benchmark. + */ + public class BenchmarkDefinitionResponseHandler implements TransportResponseHandler { + + final String benchmarkId; + + public BenchmarkDefinitionResponseHandler(String benchmarkId) { + this.benchmarkId = benchmarkId; + } + + @Override + public BenchmarkDefinitionActionResponse newInstance() { + return new BenchmarkDefinitionActionResponse(); + } + + @Override + public void handleResponse(BenchmarkDefinitionActionResponse response) { + + // We have received the benchmark definition from the master. + logger.debug("benchmark [{}]: received definition", response.benchmarkId); + + // Update our internal bookkeeping + final State state = states.get(response.benchmarkId); + if (state == null) { + throw new ElasticsearchIllegalStateException("benchmark [" + response.benchmarkId + "]: missing internal state"); + } + + state.request = response.benchmarkStartRequest; + BenchmarkMetaData.Entry.NodeState newNodeState = BenchmarkMetaData.Entry.NodeState.READY; + state.benchmarkSemaphores = new BenchmarkSemaphores(state.request.competitors()); + + try { + // Initialize the benchmark response payload + final BenchmarkStartResponse bsr = new BenchmarkStartResponse(state.benchmarkId, new HashMap()); + bsr.state(BenchmarkStartResponse.State.RUNNING); + bsr.verbose(state.request.verbose()); + state.response = bsr; + } catch (Throwable t) { + logger.error("benchmark [{}]: failed to create", t, response.benchmarkId); + states.remove(benchmarkId); + newNodeState = BenchmarkMetaData.Entry.NodeState.FAILED; + } + + // Notify the master that either we are ready to start executing or that we have failed to initialize + final NodeStateUpdateResponseHandler handler = new NodeStateUpdateResponseHandler(benchmarkId); + final NodeStateUpdateTransportRequest update = new NodeStateUpdateTransportRequest(response.benchmarkId, response.nodeId, newNodeState); + + transportService.sendRequest(master(), BenchmarkCoordinatorService.NodeStateUpdateRequestHandler.ACTION, update, handler); + } + + @Override + public void handleException(TransportException e) { + logger.error("benchmark [{}]: failed to receive definition - cannot execute", e, benchmarkId); + states.remove(benchmarkId); + } + + @Override + public String executor() { + return ThreadPool.Names.SAME; + } + } + + /** + * Response handler for node state update requests. + */ + public class NodeStateUpdateResponseHandler implements TransportResponseHandler { + + final String benchmarkId; + + public NodeStateUpdateResponseHandler(String benchmarkId) { + this.benchmarkId = benchmarkId; + } + + @Override + public NodeStateUpdateActionResponse newInstance() { + return new NodeStateUpdateActionResponse(); + } + + @Override + public void handleResponse(NodeStateUpdateActionResponse response) { /* no-op */ } + + @Override + public void handleException(TransportException e) { + logger.error("benchmark [{}]: failed to receive state change acknowledgement - cannot execute", e, benchmarkId); + states.remove(benchmarkId); + } + + @Override + public String executor() { + return ThreadPool.Names.SAME; + } + } + + /** + * Responds to requests from the master to transmit the results of the given benchmark from executors back to master. + */ + public class BenchmarkStatusRequestHandler extends BaseTransportRequestHandler { + + public static final String ACTION = "indices:data/benchmark/node/status"; + + @Override + public BenchmarkStatusTransportRequest newInstance() { + return new BenchmarkStatusTransportRequest(); + } + + @Override + public void messageReceived(final BenchmarkStatusTransportRequest request, final TransportChannel channel) throws Exception { + + final State state = states.get(request.benchmarkId); + if (state == null) { + channel.sendResponse(new ElasticsearchIllegalStateException("benchmark [" + request.benchmarkId + "]: missing internal state")); + return; + } + + try { + if (state.complete()) { + channel.sendResponse(new BenchmarkStatusNodeActionResponse(state.benchmarkId, nodeId(), state.response)); + } else { + final BenchmarkStatusNodeActionResponse status = new BenchmarkStatusNodeActionResponse(state.benchmarkId, nodeId()); + status.response(state.response); + if (!status.hasErrors()) { + channel.sendResponse(status); + } else { + channel.sendResponse(new ElasticsearchIllegalStateException("benchmark [" + request.benchmarkId + "]: " + status.error())); + } + } + } catch (Throwable t) { + try { + logger.error("benchmark [{}]: failed to send results", t, request.benchmarkId); + channel.sendResponse(t); + } catch (Throwable e) { + logger.error("unable to send failure response", e); + } + } + } + + @Override + public String executor() { + return ThreadPool.Names.SAME; + } + } + + /** + * Update node state in cluster meta data by sending a request to the master. + */ + protected void updateNodeState(String benchmarkId, String nodeId, BenchmarkMetaData.Entry.NodeState nodeState) { + + final NodeStateUpdateResponseHandler handler = new NodeStateUpdateResponseHandler(benchmarkId); + final NodeStateUpdateTransportRequest request = new NodeStateUpdateTransportRequest(benchmarkId, nodeId, nodeState); + + threadPool.executor(ThreadPool.Names.GENERIC).execute(new Runnable() { + @Override + public void run() { + transportService.sendRequest( + master(), + BenchmarkCoordinatorService.NodeStateUpdateRequestHandler.ACTION, + request, handler); + } + }); + } + + public static class BenchmarkSemaphores { + + private final ImmutableOpenMap semaphores; + + BenchmarkSemaphores(List competitors) { + ImmutableOpenMap.Builder builder = ImmutableOpenMap.builder(); + for (BenchmarkCompetitor competitor : competitors) { + builder.put(competitor.name(), new StoppableSemaphore(competitor.settings().concurrency())); + } + semaphores = builder.build(); + } + + void stopAllCompetitors() { + for (ObjectObjectCursor entry : semaphores) { + entry.value.stop(); + } + } + + void abortAllCompetitors() { + for (ObjectObjectCursor entry : semaphores) { + entry.value.stop(); + } + } + + void pauseAllCompetitors() throws InterruptedException { + for (ObjectObjectCursor entry : semaphores) { + entry.value.tryAcquireAll(TIMEOUT, TIMEUNIT); + } + } + + void resumeAllCompetitors() { + for (ObjectObjectCursor entry : semaphores) { + entry.value.releaseAll(); + } + } + + StoppableSemaphore competitorSemaphore(String name) { + return semaphores.get(name); + } + } + + protected static final class State { + + private final String benchmarkId; + private BenchmarkStartRequest request; + private BenchmarkStartResponse response; + private BenchmarkSemaphores benchmarkSemaphores; + + private volatile boolean initialized = false; + private volatile boolean started = false; + private volatile boolean paused = false; + private volatile boolean complete = false; + + State(final String benchmarkId) { + this.benchmarkId = benchmarkId; + } + + public boolean initialize() { + if (!initialized) { + initialized = true; + return true; + } + return false; + } + + public boolean start() { + if (initialized && !started) { + started = true; + return true; + } + return false; + } + + public boolean pause() throws InterruptedException { + if (initialized && started && !paused) { + if (response == null || benchmarkSemaphores == null) { + throw new ElasticsearchIllegalStateException("benchmark [" + benchmarkId + "]: not properly initialized"); + } + benchmarkSemaphores.pauseAllCompetitors(); + response.state(BenchmarkStartResponse.State.PAUSED); + paused = true; + return true; + } + return false; + } + + public boolean resume() { + if (initialized && started && paused) { + if (response == null || benchmarkSemaphores == null) { + throw new ElasticsearchIllegalStateException("benchmark [" + benchmarkId + "]: not properly initialized"); + } + benchmarkSemaphores.resumeAllCompetitors(); + response.state(BenchmarkStartResponse.State.RUNNING); + paused = false; + return true; + } + return false; + } + + public boolean abort() { + if (initialized) { + if (response == null || benchmarkSemaphores == null) { + throw new ElasticsearchIllegalStateException("benchmark [" + benchmarkId + "]: not properly initialized"); + } + benchmarkSemaphores.abortAllCompetitors(); + response.state(BenchmarkStartResponse.State.ABORTED); + return true; + } + return false; + } + + public boolean complete() { + return complete; + } + } + + protected BenchmarkMetaData.Entry.NodeState convertToNodeState(BenchmarkStartResponse.State state) { + switch (state) { + case INITIALIZING: + return BenchmarkMetaData.Entry.NodeState.INITIALIZING; + case RUNNING: + return BenchmarkMetaData.Entry.NodeState.RUNNING; + case PAUSED: + return BenchmarkMetaData.Entry.NodeState.PAUSED; + case COMPLETED: + return BenchmarkMetaData.Entry.NodeState.COMPLETED; + case FAILED: + return BenchmarkMetaData.Entry.NodeState.FAILED; + case ABORTED: + return BenchmarkMetaData.Entry.NodeState.ABORTED; + default: + throw new ElasticsearchIllegalStateException("unhandled benchmark response state: " + state); + } + } +} diff --git a/src/main/java/org/elasticsearch/action/bench/BenchmarkModule.java b/src/main/java/org/elasticsearch/action/benchmark/BenchmarkModule.java similarity index 53% rename from src/main/java/org/elasticsearch/action/bench/BenchmarkModule.java rename to src/main/java/org/elasticsearch/action/benchmark/BenchmarkModule.java index 667c8e2116c4b..2fd11baee2af3 100644 --- a/src/main/java/org/elasticsearch/action/bench/BenchmarkModule.java +++ b/src/main/java/org/elasticsearch/action/benchmark/BenchmarkModule.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.elasticsearch.action.bench; +package org.elasticsearch.action.benchmark; import org.elasticsearch.common.inject.AbstractModule; import org.elasticsearch.common.settings.Settings; @@ -28,7 +28,8 @@ public class BenchmarkModule extends AbstractModule { private final Settings settings; - public static final String BENCHMARK_SERVICE_KEY = "benchmark.service.impl"; + public static final String BENCHMARK_COORDINATOR_SERVICE_KEY = "benchmark.service.coordinator.impl"; + public static final String BENCHMARK_EXECUTOR_SERVICE_KEY = "benchmark.service.executor.impl"; public BenchmarkModule(Settings settings) { this.settings = settings; @@ -37,12 +38,18 @@ public BenchmarkModule(Settings settings) { @Override protected void configure() { - final Class service = settings.getAsClass(BENCHMARK_SERVICE_KEY, BenchmarkService.class); + final Class coordinator = settings.getAsClass(BENCHMARK_COORDINATOR_SERVICE_KEY, BenchmarkCoordinatorService.class); + if (!BenchmarkCoordinatorService.class.equals(coordinator)) { + bind(BenchmarkCoordinatorService.class).to(coordinator).asEagerSingleton(); + } else { + bind(BenchmarkCoordinatorService.class).asEagerSingleton(); + } - if (!BenchmarkService.class.equals(service)) { - bind(BenchmarkService.class).to(service).asEagerSingleton(); + final Class executor = settings.getAsClass(BENCHMARK_EXECUTOR_SERVICE_KEY, BenchmarkExecutorService.class); + if (!BenchmarkExecutorService.class.equals(executor)) { + bind(BenchmarkExecutorService.class).to(executor).asEagerSingleton(); } else { - bind(BenchmarkService.class).asEagerSingleton(); + bind(BenchmarkExecutorService.class).asEagerSingleton(); } } } diff --git a/src/main/java/org/elasticsearch/action/bench/BenchmarkSettings.java b/src/main/java/org/elasticsearch/action/benchmark/BenchmarkSettings.java similarity index 99% rename from src/main/java/org/elasticsearch/action/bench/BenchmarkSettings.java rename to src/main/java/org/elasticsearch/action/benchmark/BenchmarkSettings.java index e0efa3cf5cb32..e5ee3db09d257 100644 --- a/src/main/java/org/elasticsearch/action/bench/BenchmarkSettings.java +++ b/src/main/java/org/elasticsearch/action/benchmark/BenchmarkSettings.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.action.bench; +package org.elasticsearch.action.benchmark; import org.elasticsearch.action.admin.indices.cache.clear.ClearIndicesCacheRequest; import org.elasticsearch.action.search.SearchRequest; diff --git a/src/main/java/org/elasticsearch/action/benchmark/BenchmarkStateManager.java b/src/main/java/org/elasticsearch/action/benchmark/BenchmarkStateManager.java new file mode 100644 index 0000000000000..b759202be040b --- /dev/null +++ b/src/main/java/org/elasticsearch/action/benchmark/BenchmarkStateManager.java @@ -0,0 +1,464 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch 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. + */ + +package org.elasticsearch.action.benchmark; + +import com.google.common.base.Joiner; +import com.google.common.collect.ImmutableList; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.benchmark.abort.*; +import org.elasticsearch.action.benchmark.exception.BenchmarkIdConflictException; +import org.elasticsearch.action.benchmark.pause.BenchmarkPauseRequest; +import org.elasticsearch.action.benchmark.resume.BenchmarkResumeRequest; +import org.elasticsearch.action.benchmark.start.BenchmarkStartRequest; +import org.elasticsearch.action.benchmark.start.BenchmarkStartResponse; +import org.elasticsearch.action.benchmark.status.BenchmarkStatusResponseHandler; +import org.elasticsearch.action.benchmark.status.BenchmarkStatusResponseListener; +import org.elasticsearch.action.benchmark.status.BenchmarkStatusTransportRequest; +import org.elasticsearch.action.benchmark.BenchmarkCoordinatorService.Liveness; +import org.elasticsearch.action.support.master.MasterNodeOperationRequest; +import org.elasticsearch.cluster.ClusterService; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.TimeoutClusterStateUpdateTask; +import org.elasticsearch.cluster.metadata.BenchmarkMetaData; +import org.elasticsearch.cluster.metadata.BenchmarkMetaData.Entry; +import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.common.collect.ImmutableOpenMap; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.logging.ESLogger; +import org.elasticsearch.common.logging.ESLoggerFactory; +import org.elasticsearch.common.regex.Regex; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportService; + +import java.util.*; + +/** + * Manages cluster metadata state for benchmarks + */ +public class BenchmarkStateManager { + + private static final ESLogger logger = ESLoggerFactory.getLogger(BenchmarkStateManager.class.getName()); + + private final ClusterService clusterService; + private final ThreadPool threadPool; + private final TransportService transportService; + + private static EnumSet PAUSE_ELIGIBLE = EnumSet.of(BenchmarkMetaData.State.RUNNING); + + private static final EnumSet RESUME_ELIGIBLE = EnumSet.of(BenchmarkMetaData.State.PAUSED); + + private static final EnumSet ABORT_ELIGIBLE = EnumSet.of(BenchmarkMetaData.State.RUNNING, + BenchmarkMetaData.State.PAUSED, BenchmarkMetaData.State.RESUMING); + + // Consider node states COMPLETED, ABORTED, or FAILED to be final and not subject to updates + private static final EnumSet UPDATE_ELIGIBLE = EnumSet.complementOf(EnumSet.of(Entry.NodeState.COMPLETED, + Entry.NodeState.ABORTED, Entry.NodeState.FAILED)); + + @Inject + public BenchmarkStateManager(final ClusterService clusterService, final ThreadPool threadPool, final TransportService transportService) { + this.clusterService = clusterService; + this.threadPool = threadPool; + this.transportService = transportService; + } + + public static class BenchmarkCreationStatus { + + private final String benchmarkId; + private final boolean created; + private final List nodeIds; + + public BenchmarkCreationStatus(final String benchmarkId, final boolean created, final List nodeIds) { + this.benchmarkId = benchmarkId; + this.created = created; + this.nodeIds = nodeIds; + } + + public String benchmarkId() { + return benchmarkId; + } + + public boolean created() { + return created; + } + + public List nodeIds() { + return ImmutableList.copyOf(nodeIds); + } + } + + /** + * Initiates the lifecycle of a benchmark by creating an entry in the cluster metadata. + * + * @param request Benchmark request and definition + * @param listener Response listener + */ + public void start(final BenchmarkStartRequest request, final ActionListener listener) { + + final String cause = "benchmark-start-request (" + request.benchmarkId() + ")"; + + clusterService.submitStateUpdateTask(cause, new TimeoutClusterStateUpdateTask() { + + @Override + public TimeValue timeout() { + return request.masterNodeTimeout(); + } + + @Override + public void clusterStateProcessed(final String source, final ClusterState oldState, final ClusterState newState) { } + + @Override + public ClusterState execute(ClusterState state) throws Exception { + + final BenchmarkMetaData meta = state.metaData().custom(BenchmarkMetaData.TYPE); + final Entry existing = meta == null ? null : BenchmarkUtility.exists(request.benchmarkId(), meta); + BenchmarkCreationStatus creation; + + if (existing != null) { + // If a benchmark was submitted with the same ID from the same master node, then it + // is a user error of double submission and we throw an ID conflict exception. + // Otherwise, if the existing benchmark's master node ID differs from the current master node ID, + // this is an automatic re-submission due to master failure. + // This case is not supported, so we respond back with creation failed message. + if (state.nodes().masterNodeId().equals(existing.masterNodeId())) { + throw new BenchmarkIdConflictException("benchmark [" + request.benchmarkId() + "]: already exists"); + } + } + + final ImmutableList.Builder builder = ImmutableList.builder(); + if (meta != null) { + for (BenchmarkMetaData.Entry entry : meta.entries()) { + if (existing != null && existing.equals(entry)) { + // Benchmark is a leftover from a master failure; abort it. + builder.add(new BenchmarkMetaData.Entry(existing.benchmarkId(), + existing.masterNodeId(), BenchmarkMetaData.State.ABORTED, entry.nodeStateMap())); + } else { + builder.add(entry); + } + } + } + + if (existing != null) { + creation = new BenchmarkCreationStatus(request.benchmarkId(), false, new ArrayList()); + } else { + // Assign nodes on which to execute the benchmark + final BenchmarkMetaData.Entry entry = new BenchmarkMetaData.Entry(request.benchmarkId(), state.getNodes().masterNodeId()); + final List nodes = BenchmarkUtility.executors(clusterService.state().nodes(), request.numExecutorNodes()); + final List nodeIds = new ArrayList<>(); + + for (DiscoveryNode node : nodes) { + nodeIds.add(node.id()); + entry.nodeStateMap().put(node.id(), BenchmarkMetaData.Entry.NodeState.INITIALIZING); + } + + // Add benchmark to cluster metadata + builder.add(entry); + creation = new BenchmarkCreationStatus(request.benchmarkId(), true, nodeIds); + } + + final MetaData.Builder metabuilder = MetaData.builder(state.metaData()); + metabuilder.putCustom(BenchmarkMetaData.TYPE, new BenchmarkMetaData(builder.build())); + + // Notify caller that everything is OK and send back a list of nodeIds on which the benchmark will execute + listener.onResponse(creation); + + return ClusterState.builder(state).metaData(metabuilder).build(); + } + + @Override + public void onFailure(String source, Throwable t) { + logger.error("failed to initialize benchmark state [{}] ([{}])", t, request.benchmarkId(), source); + listener.onFailure(t); + } + }); + } + + private static class UpdateTask extends TimeoutClusterStateUpdateTask { + + final TimeValue timeValue; + final ActionListener listener; + final String[] patterns; + final EnumSet eligibilitySet; + final BenchmarkMetaData.State target; + final List found = new ArrayList<>(); + + UpdateTask(final String[] patterns, final TimeValue timeValue, final ActionListener listener, + final BenchmarkMetaData.State target, final EnumSet eligibilitySet) { + this.timeValue = timeValue; + this.patterns = patterns; + this.listener = listener; + this.target = target; + this.eligibilitySet = eligibilitySet; + } + + @Override + public TimeValue timeout() { + return timeValue; + } + + @Override + public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { + listener.onResponse(found.toArray(new String[found.size()])); + } + + @Override + public ClusterState execute(ClusterState state) throws Exception { + + final BenchmarkMetaData meta = state.metaData().custom(BenchmarkMetaData.TYPE); + if (meta == null || meta.entries().size() == 0) { + return state; + } + + final ImmutableList.Builder builder = ImmutableList.builder(); + + for (BenchmarkMetaData.Entry entry : meta.entries()) { + + if (!eligibilitySet.contains(entry.state())) { + builder.add(entry); + } else if (Regex.simpleMatch(patterns, entry.benchmarkId())) { + builder.add(new BenchmarkMetaData.Entry(entry.benchmarkId(), entry.masterNodeId(), target, entry.nodeStateMap())); + found.add(entry.benchmarkId()); + } else { + builder.add(entry); + } + } + + if (found.size() > 0) { + final MetaData.Builder metabuilder = MetaData.builder(state.metaData()); + metabuilder.putCustom(BenchmarkMetaData.TYPE, new BenchmarkMetaData(builder.build())); + return ClusterState.builder(state).metaData(metabuilder).build(); + } else { + return state; + } + } + + @Override + public void onFailure(String source, Throwable t) { + logger.error("failed to set benchmark state to {} [{}] ([{}])", t, target, patterns, source); + listener.onFailure(t); + } + } + + /** + * Updates the state of one or more benchmarks to {@link org.elasticsearch.cluster.metadata.BenchmarkMetaData.State#PAUSED}. + * + * @param request Request to pause one or more benchmarks + * @param listener Invoked with array of benchmark ID's that were updated + */ + public void pause(final BenchmarkPauseRequest request, final ActionListener listener) { + + final String cause = "benchmark-pause-request (" + Joiner.on(",").join(request.benchmarkIdPatterns()) + ")"; + + clusterService.submitStateUpdateTask(cause, + new UpdateTask(request.benchmarkIdPatterns(), request.masterNodeTimeout(), listener, + BenchmarkMetaData.State.PAUSED, PAUSE_ELIGIBLE)); + } + + /** + * Updates the state of one or more benchmarks to {@link org.elasticsearch.cluster.metadata.BenchmarkMetaData.State#RESUMING}. + * + * @param request Request to resume one or more benchmarks + * @param listener Invoked with array of benchmark ID's that were updated + */ + public void resume(final BenchmarkResumeRequest request, final ActionListener listener) { + + final String cause = "benchmark-resume-request (" + Joiner.on(",").join(request.benchmarkIdPatterns()) + ")"; + + clusterService.submitStateUpdateTask(cause, + new UpdateTask(request.benchmarkIdPatterns(), request.masterNodeTimeout(), listener, + BenchmarkMetaData.State.RESUMING, RESUME_ELIGIBLE)); + } + + /** + * Updates the state of one or more benchmarks to {@link org.elasticsearch.cluster.metadata.BenchmarkMetaData.State#ABORTED}. + * + * @param request Request to abort one or more benchmarks + * @param listener Invoked with array of benchmark ID's that were updated + */ + public void abort(final BenchmarkAbortRequest request, final ActionListener listener) { + + final String cause = "benchmark-abort-request (" + Joiner.on(",").join(request.benchmarkIdPatterns()) + ")"; + + clusterService.submitStateUpdateTask(cause, + new UpdateTask(request.benchmarkIdPatterns(), request.masterNodeTimeout(), listener, + BenchmarkMetaData.State.ABORTED, ABORT_ELIGIBLE)); + } + + /** + * Updates the state of the benchmark in the cluster metadata. + * + * @param benchmarkId Benchmark to update + * @param benchmarkState New state to apply + * @param nodeState New per-node state to apply + * @param nodeLiveness Per-node liveness state + * @param listener Response listener + */ + public void update(final String benchmarkId, final BenchmarkMetaData.State benchmarkState, + final BenchmarkMetaData.Entry.NodeState nodeState, + final ImmutableOpenMap nodeLiveness, final ActionListener listener) { + + final String cause = "benchmark-update-state (" + benchmarkId + ":" + benchmarkState + ")"; + + clusterService.submitStateUpdateTask(cause, new TimeoutClusterStateUpdateTask() { + @Override + public TimeValue timeout() { + return MasterNodeOperationRequest.DEFAULT_MASTER_NODE_TIMEOUT; + } + + @Override + public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { } + + @Override + public ClusterState execute(ClusterState clusterState) throws Exception { + + final BenchmarkMetaData meta = clusterState.metaData().custom(BenchmarkMetaData.TYPE); + final ImmutableList.Builder builder = ImmutableList.builder(); + + for (BenchmarkMetaData.Entry entry : meta.entries()) { + if (entry.benchmarkId().equals(benchmarkId)) { + Map map = entry.nodeStateMap(); + if (nodeState != null) { + map = new HashMap<>(); + for (Map.Entry e : entry.nodeStateMap().entrySet()) { + final Liveness liveness = nodeLiveness.get(e.getKey()); + if (liveness != null && !liveness.alive()) { + logger.warn("benchmark [{}]: marking dead node [{}] as {}", entry.benchmarkId(), e.getKey(), Entry.NodeState.FAILED); + map.put(e.getKey(), Entry.NodeState.FAILED); + } else if (UPDATE_ELIGIBLE.contains(e.getValue())) { + map.put(e.getKey(), nodeState); + } else { + map.put(e.getKey(), e.getValue()); + } + } + } + builder.add(new BenchmarkMetaData.Entry(entry.benchmarkId(), entry.masterNodeId(), benchmarkState, map)); + } else { + builder.add(entry); + } + } + + final MetaData.Builder metabuilder = MetaData.builder(clusterState.metaData()); + metabuilder.putCustom(BenchmarkMetaData.TYPE, new BenchmarkMetaData(builder.build())); + + return ClusterState.builder(clusterState).metaData(metabuilder).build(); + } + + @Override + public void onFailure(String source, Throwable t) { + logger.error("failed to update benchmark state [{}] ([{}])", t, benchmarkId, source); + listener.onFailure(t); + } + }); + } + + /** + * Retrieves per-node benchmark statuses and merges them into a single top-level status. + * + * @param entry Benchmark metadata entry containing list of nodes which hold benchmark status + * @return A benchmark response + * @throws InterruptedException If interrupted while awaiting node responses + */ + public BenchmarkStartResponse status(final BenchmarkMetaData.Entry entry) throws InterruptedException { + + final BenchmarkStatusResponseListener listener = new BenchmarkStatusResponseListener(entry.nodeStateMap().size()); + final List errors = new ArrayList<>(); + + for (Map.Entry e : entry.nodeStateMap().entrySet()) { + + final DiscoveryNode node = clusterService.state().nodes().get(e.getKey()); + if (node == null) { + logger.warn("benchmark [{}]: node [{}] unavailable", entry.benchmarkId(), e.getKey()); + errors.add("benchmark [" + entry.benchmarkId() + "]: node [" + e.getKey() + "] unavailable"); + listener.countdown(); + continue; + } + + final BenchmarkStatusResponseHandler handler = new BenchmarkStatusResponseHandler(entry.benchmarkId(), node.id(), listener); + + threadPool.executor(ThreadPool.Names.GENERIC).execute(new Runnable() { + @Override + public void run() { + logger.debug("benchmark [{}]: requesting status from [{}]", entry.benchmarkId(), node.id()); + transportService.sendRequest( + node, + BenchmarkExecutorService.BenchmarkStatusRequestHandler.ACTION, + new BenchmarkStatusTransportRequest(entry.benchmarkId(), node.id()), + handler); + } + }); + } + + // Block pending response by all executors + listener.awaitCompletion(); + listener.response().errors(errors); + + return listener.response(); + } + + /** + * Removes the metadata associated with the benchmark from the cluster. + * + * @param benchmarkId Benchmark to clear + * @param listener Response listener + */ + public void clear(final String benchmarkId, final ActionListener listener) { + + final String cause = "benchmark-clear-state (" + benchmarkId + ")"; + + clusterService.submitStateUpdateTask(cause, new TimeoutClusterStateUpdateTask() { + @Override + public TimeValue timeout() { + return MasterNodeOperationRequest.DEFAULT_MASTER_NODE_TIMEOUT; + } + + @Override + public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { + logger.debug("benchmark [{}]: cleared from cluster state [{}]", benchmarkId); + listener.onResponse(null); + } + + @Override + public ClusterState execute(ClusterState state) throws Exception { + + final BenchmarkMetaData meta = state.metaData().custom(BenchmarkMetaData.TYPE); + final ImmutableList.Builder builder = ImmutableList.builder(); + + for (BenchmarkMetaData.Entry entry : meta.entries()) { + if (!entry.benchmarkId().equals(benchmarkId)) { + builder.add(entry); + } + } + + final MetaData.Builder metabuilder = MetaData.builder(state.metaData()); + metabuilder.putCustom(BenchmarkMetaData.TYPE, new BenchmarkMetaData((builder.build()))); + + return ClusterState.builder(state).metaData(metabuilder).build(); + } + + @Override + public void onFailure(String source, Throwable t) { + logger.error("failed to clear benchmark state [{}] ([{}])", t, benchmarkId, source); + listener.onFailure(t); + } + }); + } +} diff --git a/src/main/java/org/elasticsearch/action/benchmark/BenchmarkUtility.java b/src/main/java/org/elasticsearch/action/benchmark/BenchmarkUtility.java new file mode 100644 index 0000000000000..c04667995f18d --- /dev/null +++ b/src/main/java/org/elasticsearch/action/benchmark/BenchmarkUtility.java @@ -0,0 +1,68 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch 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. + */ + +package org.elasticsearch.action.benchmark; + +import com.google.common.collect.ImmutableMap; +import org.elasticsearch.cluster.metadata.BenchmarkMetaData; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.node.DiscoveryNodes; + +import java.util.ArrayList; +import java.util.List; + +/** + * Various benchmark utility methods + */ +public final class BenchmarkUtility { + + public static BenchmarkMetaData.Entry exists(String benchmarkId, BenchmarkMetaData meta) { + if (meta != null) { + for (BenchmarkMetaData.Entry entry : meta.entries()) { + if (entry.benchmarkId().equals(benchmarkId)) { + return entry; + } + } + } + return null; + } + + public static List executors(DiscoveryNodes nodes, int num) { + assert num >= 1; + final List executors = new ArrayList<>(); + for (DiscoveryNode node : nodes) { + if (executors.size() == num) { + break; + } + if (isBenchmarkNode(node)) { + executors.add(node); + } + } + return executors; + } + + public static boolean isBenchmarkNode(DiscoveryNode node) { + ImmutableMap attributes = node.getAttributes(); + if (attributes.containsKey("bench")) { + String bench = attributes.get("bench"); + return Boolean.parseBoolean(bench); + } + return false; + } +} diff --git a/src/main/java/org/elasticsearch/action/bench/SinglePassStatistics.java b/src/main/java/org/elasticsearch/action/benchmark/SinglePassStatistics.java similarity index 98% rename from src/main/java/org/elasticsearch/action/bench/SinglePassStatistics.java rename to src/main/java/org/elasticsearch/action/benchmark/SinglePassStatistics.java index c67e3afccc332..e93d8072c25ae 100644 --- a/src/main/java/org/elasticsearch/action/bench/SinglePassStatistics.java +++ b/src/main/java/org/elasticsearch/action/benchmark/SinglePassStatistics.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.action.bench; +package org.elasticsearch.action.benchmark; import org.elasticsearch.search.aggregations.metrics.percentiles.tdigest.TDigestState; diff --git a/src/main/java/org/elasticsearch/action/benchmark/StoppableSemaphore.java b/src/main/java/org/elasticsearch/action/benchmark/StoppableSemaphore.java new file mode 100644 index 0000000000000..bc6d3924191fd --- /dev/null +++ b/src/main/java/org/elasticsearch/action/benchmark/StoppableSemaphore.java @@ -0,0 +1,64 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch 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. + */ + +package org.elasticsearch.action.benchmark; + +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; + +/** +* Stoppable semaphore for controlling benchmark execution +*/ +final class StoppableSemaphore { + + private final int permits; + private final Semaphore semaphore; + private volatile boolean stopped; + + public StoppableSemaphore(int permits) { + this.permits = permits; + this.semaphore = new Semaphore(permits); + } + + public void acquire() throws InterruptedException { + if (stopped) { + throw new InterruptedException("Benchmark interrupted"); + } + semaphore.acquire(); + } + + public boolean tryAcquireAll(long timeout, TimeUnit unit) throws InterruptedException { + if (stopped) { + throw new InterruptedException("Benchmark interrupted"); + } + return semaphore.tryAcquire(permits, timeout, unit); + } + + public void release() { + semaphore.release(); + } + + public void releaseAll() { + semaphore.release(permits); + } + + public void stop() { + stopped = true; + } +} diff --git a/src/main/java/org/elasticsearch/action/bench/AbortBenchmarkAction.java b/src/main/java/org/elasticsearch/action/benchmark/abort/BenchmarkAbortAction.java similarity index 67% rename from src/main/java/org/elasticsearch/action/bench/AbortBenchmarkAction.java rename to src/main/java/org/elasticsearch/action/benchmark/abort/BenchmarkAbortAction.java index 47abd130bc279..5cfacd9dbea90 100644 --- a/src/main/java/org/elasticsearch/action/bench/AbortBenchmarkAction.java +++ b/src/main/java/org/elasticsearch/action/benchmark/abort/BenchmarkAbortAction.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.action.bench; +package org.elasticsearch.action.benchmark.abort; import org.elasticsearch.action.ClientAction; import org.elasticsearch.client.Client; @@ -25,22 +25,22 @@ /** * Abort benchmark action */ -public class AbortBenchmarkAction extends ClientAction { +public class BenchmarkAbortAction extends ClientAction { - public static final AbortBenchmarkAction INSTANCE = new AbortBenchmarkAction(); public static final String NAME = "indices:data/benchmark/abort"; + public static final BenchmarkAbortAction INSTANCE = new BenchmarkAbortAction(); - private AbortBenchmarkAction() { + private BenchmarkAbortAction() { super(NAME); } @Override - public AbortBenchmarkResponse newResponse() { - return new AbortBenchmarkResponse(); + public BenchmarkAbortResponse newResponse() { + return new BenchmarkAbortResponse(); } @Override - public AbortBenchmarkRequestBuilder newRequestBuilder(Client client) { - return new AbortBenchmarkRequestBuilder(client); + public BenchmarkAbortRequestBuilder newRequestBuilder(Client client) { + return new BenchmarkAbortRequestBuilder(client); } } diff --git a/src/main/java/org/elasticsearch/action/bench/AbortBenchmarkRequest.java b/src/main/java/org/elasticsearch/action/benchmark/abort/BenchmarkAbortRequest.java similarity index 64% rename from src/main/java/org/elasticsearch/action/bench/AbortBenchmarkRequest.java rename to src/main/java/org/elasticsearch/action/benchmark/abort/BenchmarkAbortRequest.java index 17644daba7779..946fe46c81c5f 100644 --- a/src/main/java/org/elasticsearch/action/bench/AbortBenchmarkRequest.java +++ b/src/main/java/org/elasticsearch/action/benchmark/abort/BenchmarkAbortRequest.java @@ -16,11 +16,11 @@ * specific language governing permissions and limitations * under the License. */ -package org.elasticsearch.action.bench; +package org.elasticsearch.action.benchmark.abort; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ValidateActions; -import org.elasticsearch.action.support.master.AcknowledgedRequest; +import org.elasticsearch.action.support.master.MasterNodeOperationRequest; import org.elasticsearch.common.Strings; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; @@ -30,28 +30,28 @@ /** * A request to abort a specified benchmark */ -public class AbortBenchmarkRequest extends AcknowledgedRequest { +public class BenchmarkAbortRequest extends MasterNodeOperationRequest { - private String[] benchmarkNames = Strings.EMPTY_ARRAY; + private String[] benchmarkIdPatterns = Strings.EMPTY_ARRAY; - public AbortBenchmarkRequest() { } + public BenchmarkAbortRequest() { } - public AbortBenchmarkRequest(String... benchmarkNames) { - this.benchmarkNames = benchmarkNames; + public BenchmarkAbortRequest(String... benchmarkIdPatterns) { + this.benchmarkIdPatterns = benchmarkIdPatterns; } - public void benchmarkNames(String... benchmarkNames) { - this.benchmarkNames = benchmarkNames; + public void benchmarkIdPatterns(String... benchmarkNames) { + this.benchmarkIdPatterns = benchmarkNames; } - public String[] benchmarkNames() { - return benchmarkNames; + public String[] benchmarkIdPatterns() { + return benchmarkIdPatterns; } @Override public ActionRequestValidationException validate() { - if (benchmarkNames == null || benchmarkNames.length == 0) { - return ValidateActions.addValidationError("benchmarkNames must not be null or empty", null); + if (benchmarkIdPatterns == null || benchmarkIdPatterns.length == 0) { + return ValidateActions.addValidationError("benchmark names must not be null or empty", null); } return null; } @@ -59,14 +59,12 @@ public ActionRequestValidationException validate() { @Override public void readFrom(StreamInput in) throws IOException { super.readFrom(in); - benchmarkNames = in.readStringArray(); - readTimeout(in); + benchmarkIdPatterns = in.readStringArray(); } @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - out.writeStringArray(benchmarkNames); - writeTimeout(out); + out.writeStringArray(benchmarkIdPatterns); } } diff --git a/src/main/java/org/elasticsearch/action/bench/AbortBenchmarkRequestBuilder.java b/src/main/java/org/elasticsearch/action/benchmark/abort/BenchmarkAbortRequestBuilder.java similarity index 67% rename from src/main/java/org/elasticsearch/action/bench/AbortBenchmarkRequestBuilder.java rename to src/main/java/org/elasticsearch/action/benchmark/abort/BenchmarkAbortRequestBuilder.java index 1e1c6174f5721..cfdef2bdbc2c1 100644 --- a/src/main/java/org/elasticsearch/action/bench/AbortBenchmarkRequestBuilder.java +++ b/src/main/java/org/elasticsearch/action/benchmark/abort/BenchmarkAbortRequestBuilder.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.action.bench; +package org.elasticsearch.action.benchmark.abort; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRequestBuilder; @@ -26,20 +26,19 @@ /** * Request builder for aborting a benchmark */ -public class AbortBenchmarkRequestBuilder extends ActionRequestBuilder { +public class BenchmarkAbortRequestBuilder extends ActionRequestBuilder { - public AbortBenchmarkRequestBuilder(Client client) { - super(client, new AbortBenchmarkRequest()); + public BenchmarkAbortRequestBuilder(Client client) { + super(client, new BenchmarkAbortRequest()); } - public AbortBenchmarkRequestBuilder setBenchmarkNames(String... benchmarkNames) { - request.benchmarkNames(benchmarkNames); + public BenchmarkAbortRequestBuilder setBenchmarkIdPatterns(String... benchmarkIdPatterns) { + request.benchmarkIdPatterns(benchmarkIdPatterns); return this; } @Override - protected void doExecute(ActionListener listener) { + protected void doExecute(ActionListener listener) { client.abortBench(request, listener); } - } diff --git a/src/main/java/org/elasticsearch/action/benchmark/abort/BenchmarkAbortResponse.java b/src/main/java/org/elasticsearch/action/benchmark/abort/BenchmarkAbortResponse.java new file mode 100644 index 0000000000000..333a922689dda --- /dev/null +++ b/src/main/java/org/elasticsearch/action/benchmark/abort/BenchmarkAbortResponse.java @@ -0,0 +1,36 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch 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. + */ + +package org.elasticsearch.action.benchmark.abort; + +import org.elasticsearch.action.benchmark.BatchedResponse; +import org.elasticsearch.cluster.metadata.BenchmarkMetaData; + +/** + * + */ +public class BenchmarkAbortResponse extends BatchedResponse { + + public BenchmarkAbortResponse() { } + + public void addNodeResponse(final String benchmarkId, final String nodeId, + final BenchmarkMetaData.Entry.NodeState nodeState) { + super.addNodeResponse(benchmarkId, nodeId, nodeState); + } +} diff --git a/src/main/java/org/elasticsearch/action/bench/TransportAbortBenchmarkAction.java b/src/main/java/org/elasticsearch/action/benchmark/abort/TransportBenchmarkAbortAction.java similarity index 67% rename from src/main/java/org/elasticsearch/action/bench/TransportAbortBenchmarkAction.java rename to src/main/java/org/elasticsearch/action/benchmark/abort/TransportBenchmarkAbortAction.java index 98e00b56c77c2..ead1bb6e86a77 100644 --- a/src/main/java/org/elasticsearch/action/bench/TransportAbortBenchmarkAction.java +++ b/src/main/java/org/elasticsearch/action/benchmark/abort/TransportBenchmarkAbortAction.java @@ -16,11 +16,12 @@ * specific language governing permissions and limitations * under the License. */ -package org.elasticsearch.action.bench; +package org.elasticsearch.action.benchmark.abort; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.benchmark.BenchmarkCoordinatorService; import org.elasticsearch.action.support.master.TransportMasterNodeOperationAction; import org.elasticsearch.cluster.ClusterService; import org.elasticsearch.cluster.ClusterState; @@ -32,14 +33,14 @@ /** * Transport action for benchmark abort requests */ -public class TransportAbortBenchmarkAction extends TransportMasterNodeOperationAction { +public class TransportBenchmarkAbortAction extends TransportMasterNodeOperationAction { - private final BenchmarkService service; + private final BenchmarkCoordinatorService service; @Inject - public TransportAbortBenchmarkAction(Settings settings, TransportService transportService, ClusterService clusterService, - ThreadPool threadPool, BenchmarkService service, ActionFilters actionFilters) { - super(settings, AbortBenchmarkAction.NAME, transportService, clusterService, threadPool, actionFilters); + public TransportBenchmarkAbortAction(Settings settings, TransportService transportService, ClusterService clusterService, + ThreadPool threadPool, BenchmarkCoordinatorService service, ActionFilters actionFilters) { + super(settings, BenchmarkAbortAction.NAME, transportService, clusterService, threadPool, actionFilters); this.service = service; } @@ -49,17 +50,17 @@ protected String executor() { } @Override - protected AbortBenchmarkRequest newRequest() { - return new AbortBenchmarkRequest(); + protected BenchmarkAbortRequest newRequest() { + return new BenchmarkAbortRequest(); } @Override - protected AbortBenchmarkResponse newResponse() { - return new AbortBenchmarkResponse(); + protected BenchmarkAbortResponse newResponse() { + return new BenchmarkAbortResponse(); } @Override - protected void masterOperation(AbortBenchmarkRequest request, ClusterState state, final ActionListener listener) throws ElasticsearchException { - service.abortBenchmark(request.benchmarkNames(), listener); + protected void masterOperation(BenchmarkAbortRequest request, ClusterState state, final ActionListener listener) throws ElasticsearchException { + service.abortBenchmark(request, listener); } } diff --git a/src/main/java/org/elasticsearch/action/bench/CompetitionDetails.java b/src/main/java/org/elasticsearch/action/benchmark/competition/CompetitionDetails.java similarity index 91% rename from src/main/java/org/elasticsearch/action/bench/CompetitionDetails.java rename to src/main/java/org/elasticsearch/action/benchmark/competition/CompetitionDetails.java index 0a043b2a5dd7c..351d2fe23ecbf 100644 --- a/src/main/java/org/elasticsearch/action/bench/CompetitionDetails.java +++ b/src/main/java/org/elasticsearch/action/benchmark/competition/CompetitionDetails.java @@ -16,15 +16,18 @@ * specific language governing permissions and limitations * under the License. */ -package org.elasticsearch.action.bench; +package org.elasticsearch.action.benchmark.competition; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilderString; import java.io.IOException; +import java.text.NumberFormat; import java.util.List; +import java.util.Locale; import java.util.Map; +import java.text.DecimalFormat; /** * Detailed statistics for each iteration of a benchmark search competition. @@ -48,16 +51,21 @@ public List getNodeResults() { @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + final NumberFormat nf = NumberFormat.getInstance(Locale.ROOT); + final DecimalFormat formatter = (DecimalFormat) nf; + formatter.applyPattern("#######0.00"); + final int highestIteration = highestCompletedIteration(); + computeAllStatistics(); CompetitionIteration prototypical = prototypicalIteration(); - builder.startObject(Fields.ITERATIONS); + builder.startArray(Fields.ITERATIONS); for (int i = 0; i < highestIteration; i++) { - builder.field(Fields.ITERATION, i); builder.startObject(); + builder.field(Fields.ITERATION, i); builder.startObject(Fields.MIN); for (CompetitionNodeResult nodeResult : nodeResults) { @@ -74,7 +82,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.startObject(Fields.MEAN); for (CompetitionNodeResult nodeResult : nodeResults) { CompetitionIteration iteration = nodeResult.iterations().get(i); - builder.field(nodeResult.nodeName(), iteration == null ? Fields.NULL : iteration.mean()); + builder.field(nodeResult.nodeName(), iteration == null ? Fields.NULL : Double.valueOf(formatter.format(iteration.mean()))); } builder.endObject(); builder.startObject(Fields.TOTAL_TIME); @@ -86,7 +94,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.startObject(Fields.QPS); for (CompetitionNodeResult nodeResult : nodeResults) { CompetitionIteration iteration = nodeResult.iterations().get(i); - builder.field(nodeResult.nodeName(), iteration == null ? Fields.NULL : iteration.queriesPerSecond()); + builder.field(nodeResult.nodeName(), iteration == null ? Fields.NULL : Double.valueOf(formatter.format(iteration.queriesPerSecond()))); } builder.endObject(); @@ -100,7 +108,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws CompetitionIteration iteration = nodeResult.iterations().get(i); if (iteration != null) { Double value = iteration.percentileValues().get(entry.getKey()); - builder.field(nodeResult.nodeName(), (value.isNaN()) ? 0.0 : value); + builder.field(nodeResult.nodeName(), (value.isNaN()) ? 0.0 : Double.valueOf(formatter.format(value))); } else { builder.field(nodeResult.nodeName(), Fields.NULL); } @@ -111,6 +119,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.endObject(); } + builder.endArray(); return builder; } diff --git a/src/main/java/org/elasticsearch/action/bench/CompetitionIteration.java b/src/main/java/org/elasticsearch/action/benchmark/competition/CompetitionIteration.java similarity index 98% rename from src/main/java/org/elasticsearch/action/bench/CompetitionIteration.java rename to src/main/java/org/elasticsearch/action/benchmark/competition/CompetitionIteration.java index ce55184189e61..52025f6f31ffc 100644 --- a/src/main/java/org/elasticsearch/action/bench/CompetitionIteration.java +++ b/src/main/java/org/elasticsearch/action/benchmark/competition/CompetitionIteration.java @@ -17,8 +17,9 @@ * under the License. */ -package org.elasticsearch.action.bench; +package org.elasticsearch.action.benchmark.competition; +import org.elasticsearch.action.benchmark.SinglePassStatistics; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; diff --git a/src/main/java/org/elasticsearch/action/bench/CompetitionIterationData.java b/src/main/java/org/elasticsearch/action/benchmark/competition/CompetitionIterationData.java similarity index 97% rename from src/main/java/org/elasticsearch/action/bench/CompetitionIterationData.java rename to src/main/java/org/elasticsearch/action/benchmark/competition/CompetitionIterationData.java index b3e6535807302..1b961a7c7b115 100644 --- a/src/main/java/org/elasticsearch/action/bench/CompetitionIterationData.java +++ b/src/main/java/org/elasticsearch/action/benchmark/competition/CompetitionIterationData.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.action.bench; +package org.elasticsearch.action.benchmark.competition; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; diff --git a/src/main/java/org/elasticsearch/action/bench/CompetitionNodeResult.java b/src/main/java/org/elasticsearch/action/benchmark/competition/CompetitionNodeResult.java similarity index 89% rename from src/main/java/org/elasticsearch/action/bench/CompetitionNodeResult.java rename to src/main/java/org/elasticsearch/action/benchmark/competition/CompetitionNodeResult.java index 36c01f58aacb4..18cdd640cc428 100644 --- a/src/main/java/org/elasticsearch/action/bench/CompetitionNodeResult.java +++ b/src/main/java/org/elasticsearch/action/benchmark/competition/CompetitionNodeResult.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.action.bench; +package org.elasticsearch.action.benchmark.competition; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.common.io.stream.StreamInput; @@ -35,7 +35,7 @@ public class CompetitionNodeResult extends ActionResponse implements Streamable private String competitionName; private String nodeName; - private int totalIterations = 0; + private int requestedIterations = 0; private int completedIterations = 0; private int totalExecutedQueries = 0; private long warmUpTime = 0; @@ -45,11 +45,11 @@ public CompetitionNodeResult() { iterations = new ArrayList<>(); } - public CompetitionNodeResult(String competitionName, String nodeName, int totalIterations, List iterations) { + public CompetitionNodeResult(String competitionName, String nodeName, int requestedIterations, List iterations) { this.competitionName = competitionName; this.nodeName = nodeName; this.iterations = iterations; - this.totalIterations = totalIterations; + this.requestedIterations = requestedIterations; } public String competitionName() { @@ -60,8 +60,8 @@ public String nodeName() { return nodeName; } - public int totalIterations() { - return totalIterations; + public int requestedIterations() { + return requestedIterations; } public int completedIterations() { @@ -97,7 +97,7 @@ public void readFrom(StreamInput in) throws IOException { super.readFrom(in); competitionName = in.readString(); nodeName = in.readString(); - totalIterations = in.readVInt(); + requestedIterations = in.readVInt(); completedIterations = in.readVInt(); totalExecutedQueries = in.readVInt(); warmUpTime = in.readVLong(); @@ -114,7 +114,7 @@ public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); out.writeString(competitionName); out.writeString(nodeName); - out.writeVInt(totalIterations); + out.writeVInt(requestedIterations); out.writeVInt(completedIterations); out.writeVInt(totalExecutedQueries); out.writeVLong(warmUpTime); diff --git a/src/main/java/org/elasticsearch/action/bench/CompetitionResult.java b/src/main/java/org/elasticsearch/action/benchmark/competition/CompetitionResult.java similarity index 91% rename from src/main/java/org/elasticsearch/action/bench/CompetitionResult.java rename to src/main/java/org/elasticsearch/action/benchmark/competition/CompetitionResult.java index bb0c1c70a9809..7d9d250cae614 100644 --- a/src/main/java/org/elasticsearch/action/bench/CompetitionResult.java +++ b/src/main/java/org/elasticsearch/action/benchmark/competition/CompetitionResult.java @@ -17,8 +17,9 @@ * under the License. */ -package org.elasticsearch.action.bench; +package org.elasticsearch.action.benchmark.competition; +import org.elasticsearch.action.benchmark.BenchmarkSettings; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Streamable; @@ -47,17 +48,6 @@ public class CompetitionResult implements Streamable, ToXContent { public CompetitionResult() { } - /** - * Constructs a competition result - * @param competitionName Competition name - * @param concurrency Concurrency - * @param multiplier Internal multiplier; each iteration will run this many times to smooth out measurements - * @param percentiles Which percentiles to report on - */ - public CompetitionResult(String competitionName, int concurrency, int multiplier, double[] percentiles) { - this(competitionName, concurrency, multiplier, false, percentiles); - } - /** * Constructs a competition result * @param competitionName Competition name @@ -168,7 +158,7 @@ public List nodeResults() { public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(competitionName); competitionSummary.toXContent(builder, params); - if (verbose) { + if (verbose && competitionDetails != null) { competitionDetails.toXContent(builder, params); } builder.endObject(); @@ -188,6 +178,7 @@ public void readFrom(StreamInput in) throws IOException { nodeResults.add(result); } percentiles = in.readDoubleArray(); + competitionDetails = new CompetitionDetails(nodeResults); competitionSummary = new CompetitionSummary(nodeResults, concurrency, multiplier, percentiles); } diff --git a/src/main/java/org/elasticsearch/action/bench/CompetitionSummary.java b/src/main/java/org/elasticsearch/action/benchmark/competition/CompetitionSummary.java similarity index 82% rename from src/main/java/org/elasticsearch/action/bench/CompetitionSummary.java rename to src/main/java/org/elasticsearch/action/benchmark/competition/CompetitionSummary.java index 664db99b03f2a..c61be2f766272 100644 --- a/src/main/java/org/elasticsearch/action/bench/CompetitionSummary.java +++ b/src/main/java/org/elasticsearch/action/benchmark/competition/CompetitionSummary.java @@ -17,8 +17,10 @@ * under the License. */ -package org.elasticsearch.action.bench; +package org.elasticsearch.action.benchmark.competition; +import com.google.common.collect.ImmutableList; +import org.elasticsearch.action.benchmark.SinglePassStatistics; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -27,6 +29,8 @@ import org.apache.lucene.util.CollectionUtil; import java.io.IOException; +import java.text.DecimalFormat; +import java.text.NumberFormat; import java.util.*; /** @@ -42,14 +46,13 @@ public class CompetitionSummary implements ToXContent { private List nodeResults; - private boolean computed = false; private long min = 0; private long max = 0; private long totalTime = 0; private long sumTotalHits = 0; - private long totalIterations = 0; - private long completedIterations = 0; + private long totalRequestedIterations = 0; + private long totalCompletedIterations = 0; private long totalQueries = 0; private double avgWarmupTime = 0; private int concurrency = 0; @@ -86,18 +89,16 @@ public List nodeResults() { */ public void computeSummaryStatistics() { - if (computed) { - return; - } - + clear(); long totalWarmupTime = 0; final SinglePassStatistics single = new SinglePassStatistics(); + final List results = ImmutableList.copyOf(nodeResults); - for (CompetitionNodeResult nodeResult : nodeResults) { + for (CompetitionNodeResult nodeResult : results) { totalWarmupTime += nodeResult.warmUpTime(); - totalIterations += nodeResult.totalIterations(); - completedIterations += nodeResult.completedIterations(); + totalRequestedIterations += nodeResult.requestedIterations(); + totalCompletedIterations += nodeResult.completedIterations(); // only calculate statistics for iterations completed thus far for (int i = 0; i < nodeResult.completedIterations(); i++) { @@ -129,7 +130,7 @@ public void computeSummaryStatistics() { max = single.max(); mean = single.mean(); stdDeviation = single.stddev(); - avgWarmupTime = (nodeResults.size() > 0) ? totalWarmupTime / nodeResults.size() : 0.0; + avgWarmupTime = (results.size() > 0) ? totalWarmupTime / results.size() : 0.0; queriesPerSecond = (single.sum() > 0) ? (totalQueries * (1000.0 / (double) single.sum())) : 0.0; millisPerHit = (sumTotalHits > 0) ? (totalTime / (double) sumTotalHits) : 0.0; @@ -143,12 +144,15 @@ public int compare(Tuple o1, Tuple entry : percentileValues.entrySet()) { // Change back to integral value for display purposes builder.field(new XContentBuilderString("percentile_" + entry.getKey().longValue()), - (entry.getValue().isNaN()) ? 0.0 : entry.getValue()); + (entry.getValue().isNaN()) ? 0.0 : Double.valueOf(formatter.format(entry.getValue()))); } builder.endObject(); - if (totalIterations > 0 && slowest.size() > 0) { + if (totalRequestedIterations > 0 && slowest.size() > 0) { builder.startArray(Fields.SLOWEST); - int n = (int) (slowest.size() / totalIterations); + int n = (int) (slowest.size() / totalRequestedIterations); for (int i = 0; i < n; i++) { builder.startObject(); builder.field(Fields.NODE, slowest.get(i).v1()); @@ -241,16 +245,16 @@ public long getSumTotalHits() { * Number of requested iterations * @return Number of requested iterations */ - public long getTotalIterations() { - return totalIterations; + public long getTotalRequestedIterations() { + return totalRequestedIterations; } /** * Number of iterations actually completed * @return Number of iterations actually completed */ - public long getCompletedIterations() { - return completedIterations; + public long getTotalCompletedIterations() { + return totalCompletedIterations; } /** @@ -341,12 +345,22 @@ public List> getSlowest() { return slowest; } + private void clear() { + totalQueries = 0; + totalTime = 0; + sumTotalHits = 0; + totalRequestedIterations = 0; + totalCompletedIterations = 0; + slowest.clear(); + percentileValues.clear(); + } + static final class Fields { static final XContentBuilderString SUMMARY = new XContentBuilderString("summary"); - static final XContentBuilderString NODES = new XContentBuilderString("nodes"); - static final XContentBuilderString TOTAL_ITERATIONS = new XContentBuilderString("total_iterations"); - static final XContentBuilderString COMPLETED_ITERATIONS = new XContentBuilderString("completed_iterations"); - static final XContentBuilderString TOTAL_QUERIES = new XContentBuilderString("total_queries"); + static final XContentBuilderString NODES = new XContentBuilderString("executor_nodes"); + static final XContentBuilderString TOTAL_ITERATIONS = new XContentBuilderString("total_requested_iterations"); + static final XContentBuilderString COMPLETED_ITERATIONS = new XContentBuilderString("total_completed_iterations"); + static final XContentBuilderString TOTAL_QUERIES = new XContentBuilderString("total_completed_queries"); static final XContentBuilderString CONCURRENCY = new XContentBuilderString("concurrency"); static final XContentBuilderString MULTIPLIER = new XContentBuilderString("multiplier"); static final XContentBuilderString AVG_WARMUP_TIME = new XContentBuilderString("avg_warmup_time"); diff --git a/src/main/java/org/elasticsearch/action/bench/BenchmarkExecutionException.java b/src/main/java/org/elasticsearch/action/benchmark/exception/BenchmarkExecutionException.java similarity index 96% rename from src/main/java/org/elasticsearch/action/bench/BenchmarkExecutionException.java rename to src/main/java/org/elasticsearch/action/benchmark/exception/BenchmarkExecutionException.java index 0520036053134..26495ed44b3e4 100644 --- a/src/main/java/org/elasticsearch/action/bench/BenchmarkExecutionException.java +++ b/src/main/java/org/elasticsearch/action/benchmark/exception/BenchmarkExecutionException.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.action.bench; +package org.elasticsearch.action.benchmark.exception; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.rest.RestStatus; diff --git a/src/main/java/org/elasticsearch/action/benchmark/exception/BenchmarkIdConflictException.java b/src/main/java/org/elasticsearch/action/benchmark/exception/BenchmarkIdConflictException.java new file mode 100644 index 0000000000000..d4ee583a4c3b3 --- /dev/null +++ b/src/main/java/org/elasticsearch/action/benchmark/exception/BenchmarkIdConflictException.java @@ -0,0 +1,39 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch 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. + */ + +package org.elasticsearch.action.benchmark.exception; + +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.rest.RestStatus; + +/** + * Indicates that a benchmark cannot be started due to a naming conflict + * with another benchmark. + */ +public class BenchmarkIdConflictException extends ElasticsearchException { + + public BenchmarkIdConflictException(String msg) { + super(msg); + } + + @Override + public RestStatus status() { + return RestStatus.BAD_REQUEST; + } +} \ No newline at end of file diff --git a/src/main/java/org/elasticsearch/action/bench/BenchmarkMissingException.java b/src/main/java/org/elasticsearch/action/benchmark/exception/BenchmarkMissingException.java similarity index 95% rename from src/main/java/org/elasticsearch/action/bench/BenchmarkMissingException.java rename to src/main/java/org/elasticsearch/action/benchmark/exception/BenchmarkMissingException.java index dbd1872213309..8bbb909d2659a 100644 --- a/src/main/java/org/elasticsearch/action/bench/BenchmarkMissingException.java +++ b/src/main/java/org/elasticsearch/action/benchmark/exception/BenchmarkMissingException.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.action.bench; +package org.elasticsearch.action.benchmark.exception; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.rest.RestStatus; diff --git a/src/main/java/org/elasticsearch/action/bench/BenchmarkNodeMissingException.java b/src/main/java/org/elasticsearch/action/benchmark/exception/BenchmarkNodeMissingException.java similarity index 95% rename from src/main/java/org/elasticsearch/action/bench/BenchmarkNodeMissingException.java rename to src/main/java/org/elasticsearch/action/benchmark/exception/BenchmarkNodeMissingException.java index 66f775ea221f0..8fd8065bbc846 100644 --- a/src/main/java/org/elasticsearch/action/bench/BenchmarkNodeMissingException.java +++ b/src/main/java/org/elasticsearch/action/benchmark/exception/BenchmarkNodeMissingException.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.action.bench; +package org.elasticsearch.action.benchmark.exception; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.rest.RestStatus; diff --git a/src/main/java/org/elasticsearch/action/benchmark/exception/BenchmarkPauseTimedOutException.java b/src/main/java/org/elasticsearch/action/benchmark/exception/BenchmarkPauseTimedOutException.java new file mode 100644 index 0000000000000..0df9860f8a693 --- /dev/null +++ b/src/main/java/org/elasticsearch/action/benchmark/exception/BenchmarkPauseTimedOutException.java @@ -0,0 +1,38 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch 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. + */ + +package org.elasticsearch.action.benchmark.exception; + +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.rest.RestStatus; + +/** + * Indicates a pause request was unable to acquire necessary locks within a reasonable amount of time. + */ +public class BenchmarkPauseTimedOutException extends ElasticsearchException { + + public BenchmarkPauseTimedOutException(String msg, Throwable cause) { + super(msg, cause); + } + + @Override + public RestStatus status() { + return RestStatus.REQUEST_TIMEOUT; + } +} diff --git a/src/main/java/org/elasticsearch/action/benchmark/pause/BenchmarkPauseAction.java b/src/main/java/org/elasticsearch/action/benchmark/pause/BenchmarkPauseAction.java new file mode 100644 index 0000000000000..1108e7995f3a7 --- /dev/null +++ b/src/main/java/org/elasticsearch/action/benchmark/pause/BenchmarkPauseAction.java @@ -0,0 +1,47 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch 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. + */ + +package org.elasticsearch.action.benchmark.pause; + +import org.elasticsearch.action.ClientAction; +import org.elasticsearch.client.Client; +import org.elasticsearch.common.Strings; + +/** + * Action for resuming benchmarks + */ +public class BenchmarkPauseAction extends ClientAction { + + public static final BenchmarkPauseAction INSTANCE = new BenchmarkPauseAction(); + public static final String NAME = "indices:data/benchmark/pause"; + + public BenchmarkPauseAction() { + super(NAME); + } + + @Override + public BenchmarkPauseRequestBuilder newRequestBuilder(Client client) { + return new BenchmarkPauseRequestBuilder(client, Strings.EMPTY_ARRAY); + } + + @Override + public BenchmarkPauseResponse newResponse() { + return new BenchmarkPauseResponse(); + } +} diff --git a/src/main/java/org/elasticsearch/action/benchmark/pause/BenchmarkPauseRequest.java b/src/main/java/org/elasticsearch/action/benchmark/pause/BenchmarkPauseRequest.java new file mode 100644 index 0000000000000..cf73274def41e --- /dev/null +++ b/src/main/java/org/elasticsearch/action/benchmark/pause/BenchmarkPauseRequest.java @@ -0,0 +1,70 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch 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. + */ + +package org.elasticsearch.action.benchmark.pause; + +import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.ValidateActions; +import org.elasticsearch.action.support.master.MasterNodeOperationRequest; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; + +import java.io.IOException; + +/** + * Request class for resuming one or more paused benchmarks + */ +public class BenchmarkPauseRequest extends MasterNodeOperationRequest { + + private String[] benchmarkIdPatterns = Strings.EMPTY_ARRAY; + + public BenchmarkPauseRequest(String... benchmarkIdPatterns) { + this.benchmarkIdPatterns = benchmarkIdPatterns; + } + + @Override + public ActionRequestValidationException validate() { + ActionRequestValidationException validationException = null; + if (benchmarkIdPatterns == null || benchmarkIdPatterns.length == 0) { + validationException = ValidateActions.addValidationError("Benchmark ID patterns must not be null", validationException); + } + return validationException; + } + + public String[] benchmarkIdPatterns() { + return benchmarkIdPatterns; + } + + public void benchmarkIdPatterns(String... benchmarkIds) { + this.benchmarkIdPatterns = benchmarkIds; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + benchmarkIdPatterns = in.readStringArray(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeStringArray(benchmarkIdPatterns); + } +} diff --git a/src/main/java/org/elasticsearch/action/benchmark/pause/BenchmarkPauseRequestBuilder.java b/src/main/java/org/elasticsearch/action/benchmark/pause/BenchmarkPauseRequestBuilder.java new file mode 100644 index 0000000000000..636c26d111bc7 --- /dev/null +++ b/src/main/java/org/elasticsearch/action/benchmark/pause/BenchmarkPauseRequestBuilder.java @@ -0,0 +1,48 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch 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. + */ + +package org.elasticsearch.action.benchmark.pause; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ActionRequestBuilder; +import org.elasticsearch.client.Client; + +/** + * + */ +public class BenchmarkPauseRequestBuilder extends ActionRequestBuilder { + + public BenchmarkPauseRequestBuilder(Client client) { + super(client, new BenchmarkPauseRequest()); + } + + public BenchmarkPauseRequestBuilder(Client client, String[] benchmarkIdPatterns) { + super(client, new BenchmarkPauseRequest(benchmarkIdPatterns)); + } + + public BenchmarkPauseRequestBuilder setBenchmarkIdPatterns(String... benchmarkIdPatterns) { + request.benchmarkIdPatterns(benchmarkIdPatterns); + return this; + } + + @Override + protected void doExecute(ActionListener listener) { + client.pauseBenchmark(request, listener); + } +} diff --git a/src/main/java/org/elasticsearch/action/benchmark/pause/BenchmarkPauseResponse.java b/src/main/java/org/elasticsearch/action/benchmark/pause/BenchmarkPauseResponse.java new file mode 100644 index 0000000000000..8f3bea00edba3 --- /dev/null +++ b/src/main/java/org/elasticsearch/action/benchmark/pause/BenchmarkPauseResponse.java @@ -0,0 +1,36 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch 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. + */ + +package org.elasticsearch.action.benchmark.pause; + +import org.elasticsearch.action.benchmark.BatchedResponse; +import org.elasticsearch.cluster.metadata.BenchmarkMetaData; + +/** + * + */ +public class BenchmarkPauseResponse extends BatchedResponse { + + public BenchmarkPauseResponse() { } + + public void addNodeResponse(final String benchmarkId, final String nodeId, + final BenchmarkMetaData.Entry.NodeState nodeState) { + super.addNodeResponse(benchmarkId, nodeId, nodeState); + } +} diff --git a/src/main/java/org/elasticsearch/action/benchmark/pause/TransportBenchmarkPauseAction.java b/src/main/java/org/elasticsearch/action/benchmark/pause/TransportBenchmarkPauseAction.java new file mode 100644 index 0000000000000..36613cb187539 --- /dev/null +++ b/src/main/java/org/elasticsearch/action/benchmark/pause/TransportBenchmarkPauseAction.java @@ -0,0 +1,67 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch 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. + */ + +package org.elasticsearch.action.benchmark.pause; + +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.benchmark.BenchmarkCoordinatorService; +import org.elasticsearch.action.support.master.TransportMasterNodeOperationAction; +import org.elasticsearch.cluster.ClusterService; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportService; + +/** + * Transport action for pause benchmark requests + */ +public class TransportBenchmarkPauseAction extends TransportMasterNodeOperationAction { + + private final BenchmarkCoordinatorService service; + + @Inject + public TransportBenchmarkPauseAction(Settings settings, TransportService transportService, ClusterService clusterService, + ThreadPool threadPool, BenchmarkCoordinatorService service, ActionFilters actionFilters) { + super(settings, BenchmarkPauseAction.NAME, transportService, clusterService, threadPool, actionFilters); + this.service = service; + } + + @Override + protected String executor() { + return ThreadPool.Names.GENERIC; + } + + @Override + protected BenchmarkPauseRequest newRequest() { + return new BenchmarkPauseRequest(); + } + + @Override + protected BenchmarkPauseResponse newResponse() { + return new BenchmarkPauseResponse(); + } + + @Override + protected void masterOperation(BenchmarkPauseRequest request, ClusterState state, ActionListener listener) throws ElasticsearchException { + service.pauseBenchmark(request, listener); + } +} diff --git a/src/main/java/org/elasticsearch/action/benchmark/resume/BenchmarkResumeAction.java b/src/main/java/org/elasticsearch/action/benchmark/resume/BenchmarkResumeAction.java new file mode 100644 index 0000000000000..ca53882c82fee --- /dev/null +++ b/src/main/java/org/elasticsearch/action/benchmark/resume/BenchmarkResumeAction.java @@ -0,0 +1,47 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch 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. + */ + +package org.elasticsearch.action.benchmark.resume; + +import org.elasticsearch.action.ClientAction; +import org.elasticsearch.client.Client; +import org.elasticsearch.common.Strings; + +/** + * Action for resuming benchmarks + */ +public class BenchmarkResumeAction extends ClientAction { + + public static final BenchmarkResumeAction INSTANCE = new BenchmarkResumeAction(); + public static final String NAME = "indices:data/benchmark/resume"; + + public BenchmarkResumeAction() { + super(NAME); + } + + @Override + public BenchmarkResumeRequestBuilder newRequestBuilder(Client client) { + return new BenchmarkResumeRequestBuilder(client, Strings.EMPTY_ARRAY); + } + + @Override + public BenchmarkResumeResponse newResponse() { + return new BenchmarkResumeResponse(); + } +} diff --git a/src/main/java/org/elasticsearch/action/benchmark/resume/BenchmarkResumeNodeActionResponse.java b/src/main/java/org/elasticsearch/action/benchmark/resume/BenchmarkResumeNodeActionResponse.java new file mode 100644 index 0000000000000..e451fe8bd290c --- /dev/null +++ b/src/main/java/org/elasticsearch/action/benchmark/resume/BenchmarkResumeNodeActionResponse.java @@ -0,0 +1,68 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch 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. + */ + +package org.elasticsearch.action.benchmark.resume; + +import org.elasticsearch.action.benchmark.BaseNodeActionResponse; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentBuilderString; + +import java.io.IOException; + +/** + * Node-level response for a resume request + */ +public class BenchmarkResumeNodeActionResponse extends BaseNodeActionResponse implements ToXContent { + + public boolean success = false; + + public BenchmarkResumeNodeActionResponse() { } + + public BenchmarkResumeNodeActionResponse(String benchmarkId, String nodeId) { + super(benchmarkId, nodeId); + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + success = in.readBoolean(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeBoolean(success); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.field(Fields.SUCCESS, success); + super.toXContent(builder, params); + builder.endObject(); + return builder; + } + + static final class Fields { + static final XContentBuilderString SUCCESS = new XContentBuilderString("success"); + } +} diff --git a/src/main/java/org/elasticsearch/action/benchmark/resume/BenchmarkResumeRequest.java b/src/main/java/org/elasticsearch/action/benchmark/resume/BenchmarkResumeRequest.java new file mode 100644 index 0000000000000..f1c5420cb7c95 --- /dev/null +++ b/src/main/java/org/elasticsearch/action/benchmark/resume/BenchmarkResumeRequest.java @@ -0,0 +1,70 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch 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. + */ + +package org.elasticsearch.action.benchmark.resume; + +import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.ValidateActions; +import org.elasticsearch.action.support.master.MasterNodeOperationRequest; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; + +import java.io.IOException; + +/** + * Request class for resuming one or more paused benchmarks + */ +public class BenchmarkResumeRequest extends MasterNodeOperationRequest { + + private String[] benchmarkIdPatterns = Strings.EMPTY_ARRAY; + + public BenchmarkResumeRequest(String... benchmarkIdPatterns) { + this.benchmarkIdPatterns = benchmarkIdPatterns; + } + + @Override + public ActionRequestValidationException validate() { + ActionRequestValidationException validationException = null; + if (benchmarkIdPatterns == null || benchmarkIdPatterns.length == 0) { + validationException = ValidateActions.addValidationError("Benchmark ID patterns must not be null", validationException); + } + return validationException; + } + + public String[] benchmarkIdPatterns() { + return benchmarkIdPatterns; + } + + public void benchmarkIdPatterns(String... benchmarkIds) { + this.benchmarkIdPatterns = benchmarkIds; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + benchmarkIdPatterns = in.readStringArray(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeStringArray(benchmarkIdPatterns); + } +} diff --git a/src/main/java/org/elasticsearch/action/benchmark/resume/BenchmarkResumeRequestBuilder.java b/src/main/java/org/elasticsearch/action/benchmark/resume/BenchmarkResumeRequestBuilder.java new file mode 100644 index 0000000000000..a74dae849604e --- /dev/null +++ b/src/main/java/org/elasticsearch/action/benchmark/resume/BenchmarkResumeRequestBuilder.java @@ -0,0 +1,48 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch 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. + */ + +package org.elasticsearch.action.benchmark.resume; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ActionRequestBuilder; +import org.elasticsearch.client.Client; + +/** + * + */ +public class BenchmarkResumeRequestBuilder extends ActionRequestBuilder { + + public BenchmarkResumeRequestBuilder(Client client) { + super(client, new BenchmarkResumeRequest()); + } + + public BenchmarkResumeRequestBuilder(Client client, String[] benchmarkIdPatterns) { + super(client, new BenchmarkResumeRequest(benchmarkIdPatterns)); + } + + public BenchmarkResumeRequestBuilder setBenchmarkIdPatterns(String... benchmarkIdPatterns) { + request.benchmarkIdPatterns(benchmarkIdPatterns); + return this; + } + + @Override + protected void doExecute(ActionListener listener) { + client.resumeBenchmark(request, listener); + } +} diff --git a/src/main/java/org/elasticsearch/action/benchmark/resume/BenchmarkResumeResponse.java b/src/main/java/org/elasticsearch/action/benchmark/resume/BenchmarkResumeResponse.java new file mode 100644 index 0000000000000..dc93fca27554b --- /dev/null +++ b/src/main/java/org/elasticsearch/action/benchmark/resume/BenchmarkResumeResponse.java @@ -0,0 +1,36 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch 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. + */ + +package org.elasticsearch.action.benchmark.resume; + +import org.elasticsearch.action.benchmark.BatchedResponse; +import org.elasticsearch.cluster.metadata.BenchmarkMetaData; + +/** + * + */ +public class BenchmarkResumeResponse extends BatchedResponse { + + public BenchmarkResumeResponse() { } + + public void addNodeResponse(final String benchmarkId, final String nodeId, + final BenchmarkMetaData.Entry.NodeState nodeState) { + super.addNodeResponse(benchmarkId, nodeId, nodeState); + } +} diff --git a/src/main/java/org/elasticsearch/action/benchmark/resume/TransportBenchmarkResumeAction.java b/src/main/java/org/elasticsearch/action/benchmark/resume/TransportBenchmarkResumeAction.java new file mode 100644 index 0000000000000..9f5c00410a41e --- /dev/null +++ b/src/main/java/org/elasticsearch/action/benchmark/resume/TransportBenchmarkResumeAction.java @@ -0,0 +1,67 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch 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. + */ + +package org.elasticsearch.action.benchmark.resume; + +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.benchmark.BenchmarkCoordinatorService; +import org.elasticsearch.action.support.master.TransportMasterNodeOperationAction; +import org.elasticsearch.cluster.ClusterService; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportService; + +/** + * Transport action for resume benchmark requests + */ +public class TransportBenchmarkResumeAction extends TransportMasterNodeOperationAction { + + private final BenchmarkCoordinatorService service; + + @Inject + public TransportBenchmarkResumeAction(Settings settings, TransportService transportService, ClusterService clusterService, + ThreadPool threadPool, BenchmarkCoordinatorService service, ActionFilters actionFilters) { + super(settings, BenchmarkResumeAction.NAME, transportService, clusterService, threadPool, actionFilters); + this.service = service; + } + + @Override + protected String executor() { + return ThreadPool.Names.GENERIC; + } + + @Override + protected BenchmarkResumeRequest newRequest() { + return new BenchmarkResumeRequest(); + } + + @Override + protected BenchmarkResumeResponse newResponse() { + return new BenchmarkResumeResponse(); + } + + @Override + protected void masterOperation(BenchmarkResumeRequest request, ClusterState state, ActionListener listener) throws ElasticsearchException { + service.resumeBenchmark(request, listener); + } +} diff --git a/src/main/java/org/elasticsearch/action/bench/BenchmarkAction.java b/src/main/java/org/elasticsearch/action/benchmark/start/BenchmarkStartAction.java similarity index 64% rename from src/main/java/org/elasticsearch/action/bench/BenchmarkAction.java rename to src/main/java/org/elasticsearch/action/benchmark/start/BenchmarkStartAction.java index 3dceb7740d9a6..43df0856801cc 100644 --- a/src/main/java/org/elasticsearch/action/bench/BenchmarkAction.java +++ b/src/main/java/org/elasticsearch/action/benchmark/start/BenchmarkStartAction.java @@ -16,31 +16,32 @@ * specific language governing permissions and limitations * under the License. */ -package org.elasticsearch.action.bench; +package org.elasticsearch.action.benchmark.start; import org.elasticsearch.action.ClientAction; import org.elasticsearch.client.Client; import org.elasticsearch.common.Strings; /** - * Benchmark action + * Action for starting benchmarks. */ -public class BenchmarkAction extends ClientAction { +public class BenchmarkStartAction extends ClientAction { - public static final BenchmarkAction INSTANCE = new BenchmarkAction(); public static final String NAME = "indices:data/benchmark/start"; - private BenchmarkAction() { + public static final BenchmarkStartAction INSTANCE = new BenchmarkStartAction(); + + private BenchmarkStartAction() { super(NAME); } @Override - public BenchmarkResponse newResponse() { - return new BenchmarkResponse(); + public BenchmarkStartResponse newResponse() { + return new BenchmarkStartResponse(); } @Override - public BenchmarkRequestBuilder newRequestBuilder(Client client) { - return new BenchmarkRequestBuilder(client, Strings.EMPTY_ARRAY); + public BenchmarkStartRequestBuilder newRequestBuilder(Client client) { + return new BenchmarkStartRequestBuilder(client, Strings.EMPTY_ARRAY); } } diff --git a/src/main/java/org/elasticsearch/action/bench/BenchmarkRequest.java b/src/main/java/org/elasticsearch/action/benchmark/start/BenchmarkStartRequest.java similarity index 86% rename from src/main/java/org/elasticsearch/action/bench/BenchmarkRequest.java rename to src/main/java/org/elasticsearch/action/benchmark/start/BenchmarkStartRequest.java index 85863510c7f94..4ca6127955b63 100644 --- a/src/main/java/org/elasticsearch/action/bench/BenchmarkRequest.java +++ b/src/main/java/org/elasticsearch/action/benchmark/start/BenchmarkStartRequest.java @@ -16,13 +16,15 @@ * specific language governing permissions and limitations * under the License. */ -package org.elasticsearch.action.bench; +package org.elasticsearch.action.benchmark.start; import com.google.common.collect.Lists; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.CompositeIndicesRequest; import org.elasticsearch.action.IndicesRequest; import org.elasticsearch.action.ValidateActions; +import org.elasticsearch.action.benchmark.BenchmarkCompetitor; +import org.elasticsearch.action.benchmark.BenchmarkSettings; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.support.master.MasterNodeOperationRequest; import org.elasticsearch.common.io.stream.StreamInput; @@ -37,26 +39,26 @@ * perform an individual benchmark. Each competitor has its own settings such as concurrency, * number of iterations to perform, and what type of search to perform. */ -public class BenchmarkRequest extends MasterNodeOperationRequest implements CompositeIndicesRequest { +public class BenchmarkStartRequest extends MasterNodeOperationRequest implements CompositeIndicesRequest { - private String benchmarkName; - private boolean verbose; - private int numExecutorNodes = 1; // How many nodes to run the benchmark on + private String benchmarkId; + private boolean verbose; + private int numExecutorNodes = 1; // How many nodes to run the benchmark on private double[] percentiles = BenchmarkSettings.DEFAULT_PERCENTILES; // Global settings which can be overwritten at the competitor level - private BenchmarkSettings settings = new BenchmarkSettings(); + private BenchmarkSettings settings = new BenchmarkSettings(); private List competitors = new ArrayList<>(); /** * Constructs a benchmark request */ - public BenchmarkRequest() { } + public BenchmarkStartRequest() { } /** * Constructs a benchmark request */ - public BenchmarkRequest(String... indices) { + public BenchmarkStartRequest(String... indices) { settings().indices(indices); } @@ -68,14 +70,14 @@ public BenchmarkRequest(String... indices) { @Override public ActionRequestValidationException validate() { ActionRequestValidationException validationException = null; - if (benchmarkName == null) { - validationException = ValidateActions.addValidationError("benchmarkName must not be null", validationException); + if (benchmarkId == null) { + validationException = ValidateActions.addValidationError("Benchmark ID must not be null", validationException); } if (competitors.isEmpty()) { - validationException = ValidateActions.addValidationError("competitors must not be empty", validationException); + validationException = ValidateActions.addValidationError("Benchmark competitors must not be empty", validationException); } if (numExecutorNodes <= 0) { - validationException = ValidateActions.addValidationError("num_executors must not be less than 1", validationException); + validationException = ValidateActions.addValidationError("Benchmark num_executors must be at least 1", validationException); } for (BenchmarkCompetitor competitor : competitors) { validationException = competitor.validate(validationException); @@ -189,19 +191,19 @@ public void numExecutorNodes(int numExecutorNodes) { } /** - * Gets the name of the benchmark - * @return Benchmark name + * Gets the Id of the benchmark + * @return Benchmark Id */ - public String benchmarkName() { - return benchmarkName; + public String benchmarkId() { + return benchmarkId; } /** - * Sets the name of the benchmark - * @param benchmarkId Benchmark name + * Sets the Id of the benchmark + * @param benchmarkId Benchmark Id */ - public void benchmarkName(String benchmarkId) { - this.benchmarkName = benchmarkId; + public void benchmarkId(String benchmarkId) { + this.benchmarkId = benchmarkId; } /** @@ -255,11 +257,11 @@ public void addCompetitor(BenchmarkCompetitor competitor) { @Override public void readFrom(StreamInput in) throws IOException { super.readFrom(in); - benchmarkName = in.readString(); - numExecutorNodes = in.readVInt(); - verbose = in.readBoolean(); - percentiles = in.readDoubleArray(); - int size = in.readVInt(); + benchmarkId = in.readString(); + numExecutorNodes = in.readVInt(); + verbose = in.readBoolean(); + percentiles = in.readDoubleArray(); + int size = in.readVInt(); competitors.clear(); for (int i = 0; i < size; i++) { BenchmarkCompetitor competitor = new BenchmarkCompetitor(); @@ -271,7 +273,7 @@ public void readFrom(StreamInput in) throws IOException { @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - out.writeString(benchmarkName); + out.writeString(benchmarkId); out.writeVInt(numExecutorNodes); out.writeBoolean(verbose); if (percentiles != null) { diff --git a/src/main/java/org/elasticsearch/action/bench/BenchmarkRequestBuilder.java b/src/main/java/org/elasticsearch/action/benchmark/start/BenchmarkStartRequestBuilder.java similarity index 51% rename from src/main/java/org/elasticsearch/action/bench/BenchmarkRequestBuilder.java rename to src/main/java/org/elasticsearch/action/benchmark/start/BenchmarkStartRequestBuilder.java index b367484ca8b48..413be14ca0a93 100644 --- a/src/main/java/org/elasticsearch/action/bench/BenchmarkRequestBuilder.java +++ b/src/main/java/org/elasticsearch/action/benchmark/start/BenchmarkStartRequestBuilder.java @@ -16,103 +16,106 @@ * specific language governing permissions and limitations * under the License. */ -package org.elasticsearch.action.bench; +package org.elasticsearch.action.benchmark.start; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRequestBuilder; +import org.elasticsearch.action.benchmark.BenchmarkCompetitor; +import org.elasticsearch.action.benchmark.BenchmarkCompetitorBuilder; +import org.elasticsearch.action.benchmark.BenchmarkSettings; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchType; import org.elasticsearch.client.Client; /** - * Request builder for benchmarks + * Request builder for starting benchmarks */ -public class BenchmarkRequestBuilder extends ActionRequestBuilder { +public class BenchmarkStartRequestBuilder extends ActionRequestBuilder { - public BenchmarkRequestBuilder(Client client, String[] indices) { - super(client, new BenchmarkRequest(indices)); + public BenchmarkStartRequestBuilder(Client client, String[] indices) { + super(client, new BenchmarkStartRequest(indices)); } - public BenchmarkRequestBuilder(Client client) { - super(client, new BenchmarkRequest()); + public BenchmarkStartRequestBuilder(Client client) { + super(client, new BenchmarkStartRequest()); } - public BenchmarkRequestBuilder setAllowCacheClearing(boolean allowCacheClearing) { + public BenchmarkStartRequestBuilder setAllowCacheClearing(boolean allowCacheClearing) { request.settings().allowCacheClearing(allowCacheClearing); return this; } - public BenchmarkRequestBuilder setClearCachesSettings(BenchmarkSettings.ClearCachesSettings clearCachesSettings) { + public BenchmarkStartRequestBuilder setClearCachesSettings(BenchmarkSettings.ClearCachesSettings clearCachesSettings) { request.settings().clearCachesSettings(clearCachesSettings, false); return this; } - public BenchmarkRequestBuilder addSearchRequest(SearchRequest... searchRequest) { + public BenchmarkStartRequestBuilder addSearchRequest(SearchRequest... searchRequest) { request.settings().addSearchRequest(searchRequest); return this; } - public BenchmarkRequestBuilder addCompetitor(BenchmarkCompetitor competitor) { + public BenchmarkStartRequestBuilder addCompetitor(BenchmarkCompetitor competitor) { request.addCompetitor(competitor); return this; } - public BenchmarkRequestBuilder addCompetitor(BenchmarkCompetitorBuilder competitorBuilder) { + public BenchmarkStartRequestBuilder addCompetitor(BenchmarkCompetitorBuilder competitorBuilder) { return addCompetitor(competitorBuilder.build()); } - public BenchmarkRequestBuilder setNumExecutorNodes(int numExecutorNodes) { + public BenchmarkStartRequestBuilder setNumExecutorNodes(int numExecutorNodes) { request.numExecutorNodes(numExecutorNodes); return this; } - public BenchmarkRequestBuilder setIterations(int iterations) { + public BenchmarkStartRequestBuilder setIterations(int iterations) { request.settings().iterations(iterations, false); return this; } - public BenchmarkRequestBuilder setConcurrency(int concurrency) { + public BenchmarkStartRequestBuilder setConcurrency(int concurrency) { request.settings().concurrency(concurrency, false); return this; } - public BenchmarkRequestBuilder setMultiplier(int multiplier) { + public BenchmarkStartRequestBuilder setMultiplier(int multiplier) { request.settings().multiplier(multiplier, false); return this; } - public BenchmarkRequestBuilder setNumSlowest(int numSlowest) { + public BenchmarkStartRequestBuilder setNumSlowest(int numSlowest) { request.settings().numSlowest(numSlowest, false); return this; } - public BenchmarkRequestBuilder setWarmup(boolean warmup) { + public BenchmarkStartRequestBuilder setWarmup(boolean warmup) { request.settings().warmup(warmup, false); return this; } - public BenchmarkRequestBuilder setBenchmarkId(String benchmarkId) { - request.benchmarkName(benchmarkId); + public BenchmarkStartRequestBuilder setBenchmarkId(String benchmarkId) { + request.benchmarkId(benchmarkId); return this; } - public BenchmarkRequestBuilder setSearchType(SearchType searchType) { + public BenchmarkStartRequestBuilder setSearchType(SearchType searchType) { request.settings().searchType(searchType, false); return this; } - public BenchmarkRequestBuilder setVerbose(boolean verbose) { + public BenchmarkStartRequestBuilder setVerbose(boolean verbose) { request.verbose(verbose); return this; } - public BenchmarkRequestBuilder setPercentiles(double[] percentiles) { + public BenchmarkStartRequestBuilder setPercentiles(double[] percentiles) { request.percentiles(percentiles); return this; } @Override - protected void doExecute(ActionListener listener) { - client.bench(request, listener); + protected void doExecute(ActionListener listener) { + client.startBenchmark(request, listener); } } diff --git a/src/main/java/org/elasticsearch/action/bench/BenchmarkResponse.java b/src/main/java/org/elasticsearch/action/benchmark/start/BenchmarkStartResponse.java similarity index 64% rename from src/main/java/org/elasticsearch/action/bench/BenchmarkResponse.java rename to src/main/java/org/elasticsearch/action/benchmark/start/BenchmarkStartResponse.java index e4c7bf0bb80f7..07fb56275522a 100644 --- a/src/main/java/org/elasticsearch/action/bench/BenchmarkResponse.java +++ b/src/main/java/org/elasticsearch/action/benchmark/start/BenchmarkStartResponse.java @@ -17,11 +17,11 @@ * under the License. */ -package org.elasticsearch.action.bench; +package org.elasticsearch.action.benchmark.start; import org.elasticsearch.ElasticsearchIllegalArgumentException; import org.elasticsearch.action.ActionResponse; -import org.elasticsearch.common.Strings; +import org.elasticsearch.action.benchmark.competition.CompetitionResult; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Streamable; @@ -32,42 +32,47 @@ import java.io.IOException; import java.util.*; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CopyOnWriteArrayList; /** - * Benchmark response. + * Response for a start benchmark request. * * A benchmark response will contain a mapping of names to results for each competition. */ -public class BenchmarkResponse extends ActionResponse implements Streamable, ToXContent { +public class BenchmarkStartResponse extends ActionResponse implements Streamable, ToXContent { - private String benchmarkName; - private State state = State.RUNNING; - private boolean verbose; - private String[] errors = Strings.EMPTY_ARRAY; + private String benchmarkId; + private boolean verbose; + private List errors; + private volatile State state = State.RUNNING; Map competitionResults; - public BenchmarkResponse() { - competitionResults = new HashMap<>(); + public BenchmarkStartResponse() { + this(null); } - public BenchmarkResponse(String benchmarkName, Map competitionResults) { - this.benchmarkName = benchmarkName; - this.competitionResults = competitionResults; + public BenchmarkStartResponse(final String benchmarkId) { + this(benchmarkId, null); + } + + public BenchmarkStartResponse(final String benchmarkId, final Map competitionResults) { + this.benchmarkId = benchmarkId; + this.errors = new CopyOnWriteArrayList<>(); + this.competitionResults = new ConcurrentHashMap<>(); + if (competitionResults != null && competitionResults.size() > 0) { + this.competitionResults.putAll(competitionResults); + } } - /** - * Benchmarks can be in one of: - * RUNNING - executing normally - * COMPLETE - completed normally - * ABORTED - aborted - * FAILED - execution failed - */ public static enum State { - RUNNING((byte) 0), - COMPLETE((byte) 1), - ABORTED((byte) 2), - FAILED((byte) 3); + INITIALIZING((byte) 0), + RUNNING((byte) 1), + PAUSED((byte) 2), + COMPLETED((byte) 3), + ABORTED((byte) 4), + FAILED((byte) 5); private final byte id; private static final State[] STATES = new State[State.values().length]; @@ -96,19 +101,19 @@ public static State fromId(byte id) throws ElasticsearchIllegalArgumentException } /** - * Name of the benchmark - * @return Name of the benchmark + * Id of the benchmark + * @return Id of the benchmark */ - public String benchmarkName() { - return benchmarkName; + public String benchmarkId() { + return benchmarkId; } /** - * Sets the benchmark name - * @param benchmarkName Benchmark name + * Sets the benchmark Id + * @param benchmarkId Benchmark Id */ - public void benchmarkName(String benchmarkName) { - this.benchmarkName = benchmarkName; + public void benchmarkId(String benchmarkId) { + this.benchmarkId = benchmarkId; } /** @@ -168,53 +173,71 @@ public void verbose(boolean verbose) { * @return Whether the benchmark encountered error conditions */ public boolean hasErrors() { - return (errors != null && errors.length > 0); + return (errors != null && errors.size() > 0); } /** * Error messages * @return Error messages */ - public String[] errors() { + public List errors() { return this.errors; } /** - * Sets error messages + * Adds error messages to the response * @param errors Error messages */ public void errors(String... errors) { - this.errors = (errors == null) ? Strings.EMPTY_ARRAY : errors; + for (String e : errors) { + this.errors.add(e); + } + } + + public void errors(List errors) { + this.errors.addAll(errors); } @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(Fields.BENCHMARK); + builder.field(Fields.ID, benchmarkId); builder.field(Fields.STATUS, state.toString()); if (errors != null) { - builder.array(Fields.ERRORS, errors); + builder.array(Fields.ERRORS, errors.toArray(new String[errors.size()])); } builder.startObject(Fields.COMPETITORS); if (competitionResults != null) { for (Map.Entry entry : competitionResults.entrySet()) { - entry.getValue().verbose(verbose); entry.getValue().toXContent(builder, params); } } builder.endObject(); + builder.endObject(); return builder; } @Override public void readFrom(StreamInput in) throws IOException { super.readFrom(in); - benchmarkName = in.readString(); - state = State.fromId(in.readByte()); - errors = in.readStringArray(); - int size = in.readVInt(); + benchmarkId = in.readOptionalString(); + verbose = in.readBoolean(); + state = State.fromId(in.readByte()); + int size = in.readVInt(); + errors = new CopyOnWriteArrayList<>(); + for (int i = 0; i < size; i++) { + final String s = in.readOptionalString(); + if (s != null) { + errors.add(s); + } + } + size = in.readVInt(); + competitionResults = new ConcurrentHashMap<>(size); for (int i = 0; i < size; i++) { String s = in.readString(); CompetitionResult cr = new CompetitionResult(); cr.readFrom(in); + cr.verbose(verbose); competitionResults.put(s, cr); } } @@ -222,9 +245,13 @@ public void readFrom(StreamInput in) throws IOException { @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - out.writeString(benchmarkName); + out.writeOptionalString(benchmarkId); + out.writeBoolean(verbose); out.writeByte(state.id()); - out.writeStringArray(errors); + out.writeVInt(errors.size()); + for (String s : errors) { + out.writeOptionalString(s); + } out.write(competitionResults.size()); for (Map.Entry entry : competitionResults.entrySet()) { out.writeString(entry.getKey()); @@ -246,8 +273,10 @@ public String toString() { } static final class Fields { - static final XContentBuilderString STATUS = new XContentBuilderString("status"); - static final XContentBuilderString ERRORS = new XContentBuilderString("errors"); + static final XContentBuilderString BENCHMARK = new XContentBuilderString("benchmark"); + static final XContentBuilderString ID = new XContentBuilderString("id"); + static final XContentBuilderString STATUS = new XContentBuilderString("status"); + static final XContentBuilderString ERRORS = new XContentBuilderString("errors"); static final XContentBuilderString COMPETITORS = new XContentBuilderString("competitors"); } } diff --git a/src/main/java/org/elasticsearch/action/bench/TransportBenchmarkAction.java b/src/main/java/org/elasticsearch/action/benchmark/start/TransportBenchmarkStartAction.java similarity index 59% rename from src/main/java/org/elasticsearch/action/bench/TransportBenchmarkAction.java rename to src/main/java/org/elasticsearch/action/benchmark/start/TransportBenchmarkStartAction.java index d25f0062f2cca..d19c2303aefe9 100644 --- a/src/main/java/org/elasticsearch/action/bench/TransportBenchmarkAction.java +++ b/src/main/java/org/elasticsearch/action/benchmark/start/TransportBenchmarkStartAction.java @@ -16,11 +16,12 @@ * specific language governing permissions and limitations * under the License. */ -package org.elasticsearch.action.bench; +package org.elasticsearch.action.benchmark.start; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.benchmark.BenchmarkCoordinatorService; import org.elasticsearch.action.support.master.TransportMasterNodeOperationAction; import org.elasticsearch.cluster.ClusterService; import org.elasticsearch.cluster.ClusterState; @@ -31,16 +32,16 @@ /** - * Transport action for benchmarks + * Transport action for starting benchmarks */ -public class TransportBenchmarkAction extends TransportMasterNodeOperationAction { +public class TransportBenchmarkStartAction extends TransportMasterNodeOperationAction { - private final BenchmarkService service; + private final BenchmarkCoordinatorService service; @Inject - public TransportBenchmarkAction(Settings settings, TransportService transportService, ClusterService clusterService, - ThreadPool threadPool, BenchmarkService service, ActionFilters actionFilters) { - super(settings, BenchmarkAction.NAME, transportService, clusterService, threadPool, actionFilters); + public TransportBenchmarkStartAction(Settings settings, TransportService transportService, ClusterService clusterService, + ThreadPool threadPool, BenchmarkCoordinatorService service, ActionFilters actionFilters) { + super(settings, BenchmarkStartAction.NAME, transportService, clusterService, threadPool, actionFilters); this.service = service; } @@ -50,17 +51,17 @@ protected String executor() { } @Override - protected BenchmarkRequest newRequest() { - return new BenchmarkRequest(); + protected BenchmarkStartRequest newRequest() { + return new BenchmarkStartRequest(); } @Override - protected BenchmarkResponse newResponse() { - return new BenchmarkResponse(); + protected BenchmarkStartResponse newResponse() { + return new BenchmarkStartResponse(); } @Override - protected void masterOperation(BenchmarkRequest request, ClusterState state, ActionListener listener) throws ElasticsearchException { + protected void masterOperation(BenchmarkStartRequest request, ClusterState state, ActionListener listener) throws ElasticsearchException { service.startBenchmark(request, listener); } } diff --git a/src/main/java/org/elasticsearch/action/bench/BenchmarkStatusAction.java b/src/main/java/org/elasticsearch/action/benchmark/status/BenchmarkStatusAction.java similarity index 85% rename from src/main/java/org/elasticsearch/action/bench/BenchmarkStatusAction.java rename to src/main/java/org/elasticsearch/action/benchmark/status/BenchmarkStatusAction.java index 1523c93c71756..1fb6d75373f66 100644 --- a/src/main/java/org/elasticsearch/action/bench/BenchmarkStatusAction.java +++ b/src/main/java/org/elasticsearch/action/benchmark/status/BenchmarkStatusAction.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.action.bench; +package org.elasticsearch.action.benchmark.status; import org.elasticsearch.action.ClientAction; import org.elasticsearch.client.Client; @@ -25,7 +25,7 @@ /** * Benchmark status action */ -public class BenchmarkStatusAction extends ClientAction { +public class BenchmarkStatusAction extends ClientAction { public static final BenchmarkStatusAction INSTANCE = new BenchmarkStatusAction(); public static final String NAME = "indices:data/benchmark/status"; @@ -35,8 +35,8 @@ public BenchmarkStatusAction() { } @Override - public BenchmarkStatusResponse newResponse() { - return new BenchmarkStatusResponse(); + public BenchmarkStatusResponses newResponse() { + return new BenchmarkStatusResponses(); } @Override diff --git a/src/main/java/org/elasticsearch/action/benchmark/status/BenchmarkStatusNodeActionResponse.java b/src/main/java/org/elasticsearch/action/benchmark/status/BenchmarkStatusNodeActionResponse.java new file mode 100644 index 0000000000000..7572bc3d0f5ea --- /dev/null +++ b/src/main/java/org/elasticsearch/action/benchmark/status/BenchmarkStatusNodeActionResponse.java @@ -0,0 +1,68 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch 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. + */ + +package org.elasticsearch.action.benchmark.status; + +import org.elasticsearch.action.benchmark.BaseNodeActionResponse; +import org.elasticsearch.action.benchmark.start.BenchmarkStartResponse; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.xcontent.ToXContent; + +import java.io.IOException; + +/** + * Node-level response for a status request + */ +public class BenchmarkStatusNodeActionResponse extends BaseNodeActionResponse implements ToXContent { + + private BenchmarkStartResponse response; + + public BenchmarkStatusNodeActionResponse() { } + + public BenchmarkStatusNodeActionResponse(String benchmarkId, String nodeId) { + super(benchmarkId, nodeId); + } + + public BenchmarkStatusNodeActionResponse(String benchmarkId, String nodeId, BenchmarkStartResponse response) { + super(benchmarkId, nodeId); + this.response = response; + } + + public void response(BenchmarkStartResponse response) { + this.response = response; + } + + public BenchmarkStartResponse response() { + return response; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + response = new BenchmarkStartResponse(); + in.readOptionalStreamable(response); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeOptionalStreamable(response); + } +} diff --git a/src/main/java/org/elasticsearch/action/bench/BenchmarkStatusRequest.java b/src/main/java/org/elasticsearch/action/benchmark/status/BenchmarkStatusRequest.java similarity index 70% rename from src/main/java/org/elasticsearch/action/bench/BenchmarkStatusRequest.java rename to src/main/java/org/elasticsearch/action/benchmark/status/BenchmarkStatusRequest.java index 757e784d7d956..d11162519ea34 100644 --- a/src/main/java/org/elasticsearch/action/bench/BenchmarkStatusRequest.java +++ b/src/main/java/org/elasticsearch/action/benchmark/status/BenchmarkStatusRequest.java @@ -17,12 +17,13 @@ * under the License. */ -package org.elasticsearch.action.bench; +package org.elasticsearch.action.benchmark.status; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.support.master.MasterNodeOperationRequest; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.Strings; import java.io.IOException; @@ -31,6 +32,22 @@ */ public class BenchmarkStatusRequest extends MasterNodeOperationRequest { + private String[] benchmarkIdPatterns = Strings.EMPTY_ARRAY; + + public BenchmarkStatusRequest() { } + + public BenchmarkStatusRequest(String... benchmarkIdPatterns) { + this.benchmarkIdPatterns = benchmarkIdPatterns; + } + + public String[] benchmarkIdPatterns() { + return benchmarkIdPatterns; + } + + public void benchmarkIdPatterns(String... benchmarkIdPatterns) { + this.benchmarkIdPatterns = benchmarkIdPatterns; + } + @Override public ActionRequestValidationException validate() { return null; @@ -39,10 +56,12 @@ public ActionRequestValidationException validate() { @Override public void readFrom(StreamInput in) throws IOException { super.readFrom(in); + benchmarkIdPatterns = in.readStringArray(); } @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); + out.writeStringArray(benchmarkIdPatterns); } } diff --git a/src/main/java/org/elasticsearch/action/bench/BenchmarkStatusRequestBuilder.java b/src/main/java/org/elasticsearch/action/benchmark/status/BenchmarkStatusRequestBuilder.java similarity index 75% rename from src/main/java/org/elasticsearch/action/bench/BenchmarkStatusRequestBuilder.java rename to src/main/java/org/elasticsearch/action/benchmark/status/BenchmarkStatusRequestBuilder.java index b615cab8b5451..c0148b8c5a41a 100644 --- a/src/main/java/org/elasticsearch/action/bench/BenchmarkStatusRequestBuilder.java +++ b/src/main/java/org/elasticsearch/action/benchmark/status/BenchmarkStatusRequestBuilder.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.action.bench; +package org.elasticsearch.action.benchmark.status; import org.elasticsearch.client.Client; import org.elasticsearch.action.ActionListener; @@ -26,14 +26,19 @@ /** * Request builder for benchmark status */ -public class BenchmarkStatusRequestBuilder extends ActionRequestBuilder { +public class BenchmarkStatusRequestBuilder extends ActionRequestBuilder { public BenchmarkStatusRequestBuilder(Client client) { super(client, new BenchmarkStatusRequest()); } + public BenchmarkStatusRequestBuilder setBenchmarkIdPatterns(String... benchmarkIdPatterns) { + request.benchmarkIdPatterns(benchmarkIdPatterns); + return this; + } + @Override - protected void doExecute(ActionListener listener) { - client.benchStatus(request, listener); + protected void doExecute(ActionListener listener) { + client.benchmarkStatus(request, listener); } } diff --git a/src/main/java/org/elasticsearch/action/benchmark/status/BenchmarkStatusResponseHandler.java b/src/main/java/org/elasticsearch/action/benchmark/status/BenchmarkStatusResponseHandler.java new file mode 100644 index 0000000000000..8c61b746fc566 --- /dev/null +++ b/src/main/java/org/elasticsearch/action/benchmark/status/BenchmarkStatusResponseHandler.java @@ -0,0 +1,66 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch 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. + */ + +package org.elasticsearch.action.benchmark.status; + +import org.elasticsearch.common.logging.ESLogger; +import org.elasticsearch.common.logging.ESLoggerFactory; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportException; +import org.elasticsearch.transport.TransportResponseHandler; + +/** + * + */ +public class BenchmarkStatusResponseHandler implements TransportResponseHandler { + + private static final ESLogger logger = ESLoggerFactory.getLogger(BenchmarkStatusResponseHandler.class.getName()); + + final String nodeId; + final String benchmarkId; + final BenchmarkStatusResponseListener listener; + + public BenchmarkStatusResponseHandler(String benchmarkId, String nodeId, BenchmarkStatusResponseListener listener) { + this.benchmarkId = benchmarkId; + this.nodeId = nodeId; + this.listener = listener; + } + + @Override + public BenchmarkStatusNodeActionResponse newInstance() { + return new BenchmarkStatusNodeActionResponse(); + } + + @Override + public void handleResponse(BenchmarkStatusNodeActionResponse response) { + logger.debug("benchmark [{}]: received results from [{}]", benchmarkId, response.nodeId); + listener.onResponse(response.response()); + } + + @Override + public void handleException(TransportException e) { + logger.error("benchmark [{}]: failed to receive results", e, benchmarkId); + listener.onFailure(e); + } + + @Override + public String executor() { + return ThreadPool.Names.SAME; + } +} diff --git a/src/main/java/org/elasticsearch/action/benchmark/status/BenchmarkStatusResponseListener.java b/src/main/java/org/elasticsearch/action/benchmark/status/BenchmarkStatusResponseListener.java new file mode 100644 index 0000000000000..99f6f4f8ee6d2 --- /dev/null +++ b/src/main/java/org/elasticsearch/action/benchmark/status/BenchmarkStatusResponseListener.java @@ -0,0 +1,142 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch 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. + */ + +package org.elasticsearch.action.benchmark.status; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.benchmark.competition.CompetitionResult; +import org.elasticsearch.action.benchmark.start.BenchmarkStartResponse; +import org.elasticsearch.common.logging.ESLogger; +import org.elasticsearch.common.logging.ESLoggerFactory; +import org.elasticsearch.common.util.concurrent.CountDown; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CountDownLatch; + +/** + * Listens for status responses from one or more executor nodes and merges them into a + * single, consolidated benchmark response. + */ +public class BenchmarkStatusResponseListener implements ActionListener { + + private static final ESLogger logger = ESLoggerFactory.getLogger(BenchmarkStatusResponseListener.class.getName()); + + private final CountDown countdown; + private final CountDownLatch complete = new CountDownLatch(1); + private final List responses = Collections.synchronizedList(new ArrayList()); + private BenchmarkStartResponse response; + + public BenchmarkStatusResponseListener(int count) { + this.countdown = new CountDown(count); + } + + @Override + public void onResponse(BenchmarkStartResponse benchmarkStartResponse) { + responses.add(benchmarkStartResponse); + if (countdown.countDown()) { + processResponses(); + } + } + + @Override + public void onFailure(Throwable e) { + logger.debug(e.getMessage(), e); + if (countdown.countDown()) { + processResponses(); + } + } + + private void processResponses() { + try { + synchronized (responses) { + response = merge(responses); + } + } finally { + complete.countDown(); + } + } + + public boolean countdown() { + return countdown.countDown(); + } + + public void awaitCompletion() throws InterruptedException { + complete.await(); + } + + public List responses() { + return responses; + } + + public BenchmarkStartResponse response() { + return response; + } + + /** + * Merge node responses into a single consolidated response + */ + private BenchmarkStartResponse merge(List responses) { + + final BenchmarkStartResponse response = new BenchmarkStartResponse(); + final List errors = new ArrayList<>(); + + for (BenchmarkStartResponse r : responses) { + + if (r.competitionResults() == null) { + continue; + } + + for (Map.Entry entry : r.competitionResults().entrySet()) { + if (!response.competitionResults().containsKey(entry.getKey())) { + response.competitionResults().put(entry.getKey(), + new CompetitionResult( + entry.getKey(), entry.getValue().concurrency(), entry.getValue().multiplier(), + entry.getValue().verbose(), entry.getValue().percentiles()) + ); + } + CompetitionResult cr = response.competitionResults().get(entry.getKey()); + cr.nodeResults().addAll(entry.getValue().nodeResults()); + } + + if (r.hasErrors()) { + for (String error : r.errors()) { + errors.add(error); + } + } + + if (response.benchmarkId() == null) { + response.benchmarkId(r.benchmarkId()); + } + + assert response.benchmarkId().equals(r.benchmarkId()); + if (!errors.isEmpty()) { + response.errors(errors.toArray(new String[errors.size()])); + } + + response.mergeState(r.state()); + response.verbose(r.verbose()); + assert errors.isEmpty() || response.state() != BenchmarkStartResponse.State.COMPLETED : "Response can't be complete since it has errors"; + } + + return response; + } +} diff --git a/src/main/java/org/elasticsearch/action/benchmark/status/BenchmarkStatusResponses.java b/src/main/java/org/elasticsearch/action/benchmark/status/BenchmarkStatusResponses.java new file mode 100644 index 0000000000000..65ddf7817227d --- /dev/null +++ b/src/main/java/org/elasticsearch/action/benchmark/status/BenchmarkStatusResponses.java @@ -0,0 +1,87 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch 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. + */ + +package org.elasticsearch.action.benchmark.status; + +import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.action.benchmark.start.BenchmarkStartResponse; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentBuilderString; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +/** + * + */ +public class BenchmarkStatusResponses extends ActionResponse implements ToXContent { + + private List responses = new ArrayList<>(); + + public BenchmarkStatusResponses() { } + + public BenchmarkStatusResponses(List responses) { + this.responses = responses; + } + + public void add(BenchmarkStartResponse response) { + responses.add(response); + } + + public List responses() { + return responses; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + final int size = in.readVInt(); + responses = new ArrayList<>(size); + for (int i = 0; i < size; i++) { + BenchmarkStartResponse response = new BenchmarkStartResponse(); + response.readFrom(in); + responses.add(response); + } + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVInt(responses.size()); + for (BenchmarkStartResponse response : responses) { + response.writeTo(out); + } + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startArray(Fields.BENCHMARKS); + for (BenchmarkStartResponse response : responses) { + response.toXContent(builder, params); + } + builder.endArray(); + return builder; + } + + static final class Fields { + static final XContentBuilderString BENCHMARKS = new XContentBuilderString("benchmarks"); + } +} diff --git a/src/main/java/org/elasticsearch/action/benchmark/status/BenchmarkStatusTransportRequest.java b/src/main/java/org/elasticsearch/action/benchmark/status/BenchmarkStatusTransportRequest.java new file mode 100644 index 0000000000000..d5f7c168e4f3c --- /dev/null +++ b/src/main/java/org/elasticsearch/action/benchmark/status/BenchmarkStatusTransportRequest.java @@ -0,0 +1,34 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch 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. + */ + +package org.elasticsearch.action.benchmark.status; + +import org.elasticsearch.action.benchmark.BaseNodeTransportRequest; + +/** + * Request class for fetching benchmark results from executors to master + */ +public class BenchmarkStatusTransportRequest extends BaseNodeTransportRequest { + + public BenchmarkStatusTransportRequest() { } + + public BenchmarkStatusTransportRequest(final String benchmarkId, final String nodeId) { + super(benchmarkId, nodeId); + } +} diff --git a/src/main/java/org/elasticsearch/action/bench/TransportBenchmarkStatusAction.java b/src/main/java/org/elasticsearch/action/benchmark/status/TransportBenchmarkStatusAction.java similarity index 84% rename from src/main/java/org/elasticsearch/action/bench/TransportBenchmarkStatusAction.java rename to src/main/java/org/elasticsearch/action/benchmark/status/TransportBenchmarkStatusAction.java index d17cbda7247eb..c0d3aeeb42ea3 100644 --- a/src/main/java/org/elasticsearch/action/bench/TransportBenchmarkStatusAction.java +++ b/src/main/java/org/elasticsearch/action/benchmark/status/TransportBenchmarkStatusAction.java @@ -17,11 +17,12 @@ * under the License. */ -package org.elasticsearch.action.bench; +package org.elasticsearch.action.benchmark.status; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.benchmark.BenchmarkCoordinatorService; import org.elasticsearch.cluster.ClusterService; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.common.inject.Inject; @@ -33,13 +34,13 @@ /** * Transport action for benchmark status requests */ -public class TransportBenchmarkStatusAction extends TransportMasterNodeOperationAction { +public class TransportBenchmarkStatusAction extends TransportMasterNodeOperationAction { - private final BenchmarkService service; + private final BenchmarkCoordinatorService service; @Inject public TransportBenchmarkStatusAction(Settings settings, TransportService transportService, ClusterService clusterService, - ThreadPool threadPool, BenchmarkService service, ActionFilters actionFilters) { + ThreadPool threadPool, BenchmarkCoordinatorService service, ActionFilters actionFilters) { super(settings, BenchmarkStatusAction.NAME, transportService, clusterService, threadPool, actionFilters); this.service = service; } @@ -55,12 +56,12 @@ protected BenchmarkStatusRequest newRequest() { } @Override - protected BenchmarkStatusResponse newResponse() { - return new BenchmarkStatusResponse(); + protected BenchmarkStatusResponses newResponse() { + return new BenchmarkStatusResponses(); } @Override - protected void masterOperation(BenchmarkStatusRequest request, ClusterState state, ActionListener listener) + protected void masterOperation(BenchmarkStatusRequest request, ClusterState state, ActionListener listener) throws ElasticsearchException { service.listBenchmarks(request, listener); } diff --git a/src/main/java/org/elasticsearch/client/Client.java b/src/main/java/org/elasticsearch/client/Client.java index c88d2d77f2318..a295ae6f0cda7 100644 --- a/src/main/java/org/elasticsearch/client/Client.java +++ b/src/main/java/org/elasticsearch/client/Client.java @@ -20,7 +20,11 @@ package org.elasticsearch.client; import org.elasticsearch.action.*; -import org.elasticsearch.action.bench.*; +import org.elasticsearch.action.benchmark.abort.*; +import org.elasticsearch.action.benchmark.pause.*; +import org.elasticsearch.action.benchmark.resume.*; +import org.elasticsearch.action.benchmark.start.*; +import org.elasticsearch.action.benchmark.status.*; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkRequestBuilder; import org.elasticsearch.action.bulk.BulkResponse; @@ -649,37 +653,72 @@ public interface Client extends ElasticsearchClient, Releasable { /** * Runs a benchmark on the server */ - void bench(BenchmarkRequest request, ActionListener listener); + void startBenchmark(BenchmarkStartRequest request, ActionListener listener); /** * Runs a benchmark on the server */ - ActionFuture bench(BenchmarkRequest request); + ActionFuture startBenchmark(BenchmarkStartRequest request); /** * Runs a benchmark on the server */ - BenchmarkRequestBuilder prepareBench(String... indices); + BenchmarkStartRequestBuilder prepareStartBenchmark(String... indices); /** * Aborts a benchmark run on the server */ - void abortBench(AbortBenchmarkRequest request, ActionListener listener); + void abortBench(BenchmarkAbortRequest request, ActionListener listener); /** * Aborts a benchmark run on the server */ - AbortBenchmarkRequestBuilder prepareAbortBench(String... benchmarkNames); + BenchmarkAbortRequestBuilder prepareAbortBench(String... benchmarkIdPatterns); /** * Reports on status of actively running benchmarks */ - void benchStatus(BenchmarkStatusRequest request, ActionListener listener); + void benchmarkStatus(BenchmarkStatusRequest request, ActionListener listener); /** * Reports on status of actively running benchmarks */ - BenchmarkStatusRequestBuilder prepareBenchStatus(); + ActionFuture benchmarkStatus(BenchmarkStatusRequest request); + + /** + * Reports on status of actively running benchmarks + */ + BenchmarkStatusRequestBuilder prepareBenchmarkStatus(String... benchmarkIdPatterns); + + /** + * Resumes a paused benchmark + */ + void resumeBenchmark(BenchmarkResumeRequest request, ActionListener listener); + + /** + * Resumes a paused benchmark + */ + ActionFuture resumeBenchmark(BenchmarkResumeRequest request); + + /** + * Resumes a paused benchmark + */ + BenchmarkResumeRequestBuilder prepareResumeBenchmark(String... benchmarkIdPatterns); + + /** + * Pauses a running benchmark + */ + void pauseBenchmark(BenchmarkPauseRequest request, ActionListener listener); + + /** + * Pauses a running benchmark + */ + ActionFuture pauseBenchmark(BenchmarkPauseRequest request); + + /** + * Pauses a running benchmark + */ + BenchmarkPauseRequestBuilder preparePauseBenchmark(String... benchmarkIdPatterns); /** * Returns this clients settings diff --git a/src/main/java/org/elasticsearch/client/support/AbstractClient.java b/src/main/java/org/elasticsearch/client/support/AbstractClient.java index 22b5e5e99291a..d8fe778e1d5f3 100644 --- a/src/main/java/org/elasticsearch/client/support/AbstractClient.java +++ b/src/main/java/org/elasticsearch/client/support/AbstractClient.java @@ -20,8 +20,11 @@ package org.elasticsearch.client.support; import org.elasticsearch.action.*; -import org.elasticsearch.action.admin.indices.delete.DeleteIndexAction; -import org.elasticsearch.action.bench.*; +import org.elasticsearch.action.benchmark.abort.*; +import org.elasticsearch.action.benchmark.pause.*; +import org.elasticsearch.action.benchmark.resume.*; +import org.elasticsearch.action.benchmark.start.*; +import org.elasticsearch.action.benchmark.status.*; import org.elasticsearch.action.bulk.BulkAction; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkRequestBuilder; @@ -522,37 +525,72 @@ public ClearScrollRequestBuilder prepareClearScroll() { } @Override - public void bench(BenchmarkRequest request, ActionListener listener) { - execute(BenchmarkAction.INSTANCE, request, listener); + public void startBenchmark(BenchmarkStartRequest request, ActionListener listener) { + execute(BenchmarkStartAction.INSTANCE, request, listener); } @Override - public ActionFuture bench(BenchmarkRequest request) { - return execute(BenchmarkAction.INSTANCE, request); + public ActionFuture startBenchmark(BenchmarkStartRequest request) { + return execute(BenchmarkStartAction.INSTANCE, request); } @Override - public BenchmarkRequestBuilder prepareBench(String... indices) { - return new BenchmarkRequestBuilder(this, indices); + public BenchmarkStartRequestBuilder prepareStartBenchmark(String... indices) { + return new BenchmarkStartRequestBuilder(this, indices); } @Override - public void abortBench(AbortBenchmarkRequest request, ActionListener listener) { - execute(AbortBenchmarkAction.INSTANCE, request, listener); + public void abortBench(BenchmarkAbortRequest request, ActionListener listener) { + execute(BenchmarkAbortAction.INSTANCE, request, listener); } @Override - public AbortBenchmarkRequestBuilder prepareAbortBench(String... benchmarkNames) { - return new AbortBenchmarkRequestBuilder(this).setBenchmarkNames(benchmarkNames); + public BenchmarkAbortRequestBuilder prepareAbortBench(String... benchmarkIdPatterns) { + return new BenchmarkAbortRequestBuilder(this).setBenchmarkIdPatterns(benchmarkIdPatterns); } @Override - public void benchStatus(BenchmarkStatusRequest request, ActionListener listener) { + public void benchmarkStatus(BenchmarkStatusRequest request, ActionListener listener) { execute(BenchmarkStatusAction.INSTANCE, request, listener); } @Override - public BenchmarkStatusRequestBuilder prepareBenchStatus() { - return new BenchmarkStatusRequestBuilder(this); + public ActionFuture benchmarkStatus(BenchmarkStatusRequest request) { + return execute(BenchmarkStatusAction.INSTANCE, request); + } + + @Override + public BenchmarkStatusRequestBuilder prepareBenchmarkStatus(String... benchmarkIdPatterns) { + return new BenchmarkStatusRequestBuilder(this).setBenchmarkIdPatterns(benchmarkIdPatterns); + } + + @Override + public void resumeBenchmark(BenchmarkResumeRequest request, ActionListener listener) { + execute(BenchmarkResumeAction.INSTANCE, request, listener); + } + + @Override + public ActionFuture resumeBenchmark(BenchmarkResumeRequest request) { + return execute(BenchmarkResumeAction.INSTANCE, request); + } + + @Override + public BenchmarkResumeRequestBuilder prepareResumeBenchmark(String... benchmarkIds) { + return new BenchmarkResumeRequestBuilder(this).setBenchmarkIdPatterns(benchmarkIds); + } + + @Override + public void pauseBenchmark(BenchmarkPauseRequest request, ActionListener listener) { + execute(BenchmarkPauseAction.INSTANCE, request, listener); + } + + @Override + public ActionFuture pauseBenchmark(BenchmarkPauseRequest request) { + return execute(BenchmarkPauseAction.INSTANCE, request); + } + + @Override + public BenchmarkPauseRequestBuilder preparePauseBenchmark(String... benchmarkIdPatterns) { + return new BenchmarkPauseRequestBuilder(this).setBenchmarkIdPatterns(benchmarkIdPatterns); } } diff --git a/src/main/java/org/elasticsearch/client/transport/TransportClient.java b/src/main/java/org/elasticsearch/client/transport/TransportClient.java index 91d59dd86a67d..72823b860b7a4 100644 --- a/src/main/java/org/elasticsearch/client/transport/TransportClient.java +++ b/src/main/java/org/elasticsearch/client/transport/TransportClient.java @@ -23,9 +23,9 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.Version; import org.elasticsearch.action.*; -import org.elasticsearch.action.bench.BenchmarkRequest; -import org.elasticsearch.action.bench.BenchmarkRequestBuilder; -import org.elasticsearch.action.bench.BenchmarkResponse; +import org.elasticsearch.action.benchmark.start.BenchmarkStartRequest; +import org.elasticsearch.action.benchmark.start.BenchmarkStartRequestBuilder; +import org.elasticsearch.action.benchmark.start.BenchmarkStartResponse; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.count.CountRequest; @@ -497,12 +497,12 @@ public void explain(ExplainRequest request, ActionListener list } @Override - public void bench(BenchmarkRequest request, ActionListener listener) { - internalClient.bench(request, listener); + public void startBenchmark(BenchmarkStartRequest request, ActionListener listener) { + internalClient.startBenchmark(request, listener); } @Override - public BenchmarkRequestBuilder prepareBench(String... indices) { - return internalClient.prepareBench(indices); + public BenchmarkStartRequestBuilder prepareStartBenchmark(String... indices) { + return internalClient.prepareStartBenchmark(indices); } } diff --git a/src/main/java/org/elasticsearch/cluster/metadata/BenchmarkMetaData.java b/src/main/java/org/elasticsearch/cluster/metadata/BenchmarkMetaData.java index bbb06ee364e0b..e779331118c45 100644 --- a/src/main/java/org/elasticsearch/cluster/metadata/BenchmarkMetaData.java +++ b/src/main/java/org/elasticsearch/cluster/metadata/BenchmarkMetaData.java @@ -27,22 +27,24 @@ import org.elasticsearch.common.xcontent.XContentParser; import java.io.IOException; +import java.util.*; /** * Meta data about benchmarks that are currently executing */ public class BenchmarkMetaData implements MetaData.Custom { - public static final String TYPE = "benchmark"; + public static final String TYPE = "benchmark"; public static final Factory FACTORY = new Factory(); + private final ImmutableList entries; + @Override public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; BenchmarkMetaData that = (BenchmarkMetaData) o; - if (!entries.equals(that.entries)) return false; return true; @@ -53,33 +55,84 @@ public int hashCode() { return entries.hashCode(); } + public static ImmutableList addedOrChanged(BenchmarkMetaData prev, BenchmarkMetaData cur) { + + if (cur == null || cur.entries == null || cur.entries.size() == 0) { + return ImmutableList.of(); + } + if (prev == null || prev.entries == null || prev.entries.size() == 0) { + return ImmutableList.copyOf(cur.entries); + } + + final List changed = new ArrayList<>(); + + for (Entry e : cur.entries) { + final Entry matched = find(e, prev.entries); + if (matched == null) { + changed.add(e); + } else { + if (!e.equals(matched)) { + changed.add(e); + } + } + } + + return ImmutableList.copyOf(changed); + } + + private static Entry find(Entry entry, List list) { + for (Entry e : list) { + if (e.benchmarkId().equals(entry.benchmarkId())) { + return e; + } + } + return null; + } + public static class Entry { - private final State state; - private final String benchmarkId; - private final String[] nodeids; - public Entry(Entry e, State state) { - this(e.benchmarkId(), state, e.nodes()); + private final String masterNodeId; + private final String benchmarkId; + private final State state; + private final Map nodeStateMap; + + public Entry(String benchmarkId, String masterNodeId) { + this(benchmarkId, masterNodeId, State.INITIALIZING, new HashMap()); } - public Entry(String benchmarkId, State state, String[] nodeIds) { - this.state = state; - this.benchmarkId = benchmarkId; - this.nodeids = nodeIds; + public Entry(String benchmarkId, String masterNodeId, State state, Map nodeStateMap) { + this.benchmarkId = benchmarkId; + this.state = state; + this.nodeStateMap = nodeStateMap; + this.masterNodeId = masterNodeId; } public String benchmarkId() { - return this.benchmarkId; + return benchmarkId; + } + + public String masterNodeId() { + return masterNodeId; } public State state() { return state; } - public String[] nodes() { - return nodeids; + public Map nodeStateMap() { + return nodeStateMap; } + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append("benchmark metadata state: [").append(benchmarkId).append("] (").append(state()).append(") ["); + for (Map.Entry e : nodeStateMap().entrySet()) { + sb.append(" ").append(e.getKey()).append(":").append(e.getValue()); + } + sb.append(" ]"); + return sb.toString(); + } @Override public boolean equals(Object o) { @@ -89,24 +142,80 @@ public boolean equals(Object o) { Entry entry = (Entry) o; if (!benchmarkId.equals(entry.benchmarkId)) return false; + if (!masterNodeId.equals(entry.masterNodeId)) return false; if (state != entry.state) return false; + if (nodeStateMap().size() != entry.nodeStateMap().size()) return false; + + for (Map.Entry me : nodeStateMap().entrySet()) { + final NodeState ns = entry.nodeStateMap().get(me.getKey()); + if (ns == null) { + return false; + } + if (me.getValue() != ns) { + return false; + } + } + return true; } @Override public int hashCode() { - int result = state.hashCode(); - result = 31 * result + benchmarkId.hashCode(); + int result = benchmarkId.hashCode(); + result = 31 * result + masterNodeId.hashCode(); + result = 31 * result + state.hashCode(); + result = 31 * result + nodeStateMap.hashCode(); return result; } + + public static enum NodeState { + + INITIALIZING((byte) 0), + READY((byte) 1), + RUNNING((byte) 2), + PAUSED((byte) 3), + COMPLETED((byte) 4), + FAILED((byte) 5), + ABORTED((byte) 6); + + private static final NodeState[] NODE_STATES = new NodeState[NodeState.values().length]; + + static { + for (NodeState state : NodeState.values()) { + assert state.id() < NODE_STATES.length && state.id() >= 0; + NODE_STATES[state.id()] = state; + } + } + + private final byte id; + + NodeState(byte id) { + this.id = id; + } + + public byte id() { + return id; + } + + public static NodeState fromId(byte id) { + if (id < 0 || id >= NodeState.values().length) { + throw new ElasticsearchIllegalArgumentException("No benchmark state for value [" + id + "]"); + } + return NODE_STATES[id]; + } + } } public static enum State { - STARTED((byte) 0), - SUCCESS((byte) 1), - FAILED((byte) 2), - ABORTED((byte) 3); + + INITIALIZING((byte) 0), + RUNNING((byte) 1), + PAUSED((byte) 2), + RESUMING((byte) 3), + COMPLETED((byte) 4), + FAILED((byte) 5), + ABORTED((byte) 6); private static final State[] STATES = new State[State.values().length]; @@ -128,7 +237,7 @@ public byte id() { } public boolean completed() { - return this == SUCCESS || this == FAILED; + return this == COMPLETED || this == FAILED; } public static State fromId(byte id) { @@ -139,9 +248,6 @@ public static State fromId(byte id) { } } - private final ImmutableList entries; - - public BenchmarkMetaData(ImmutableList entries) { this.entries = entries; } @@ -154,7 +260,6 @@ public ImmutableList entries() { return this.entries; } - public static class Factory implements MetaData.Custom.Factory { @Override @@ -167,9 +272,14 @@ public BenchmarkMetaData readFrom(StreamInput in) throws IOException { Entry[] entries = new Entry[in.readVInt()]; for (int i = 0; i < entries.length; i++) { String benchmarkId = in.readString(); + String masterNodeId = in.readString(); State state = State.fromId(in.readByte()); - String[] nodes = in.readStringArray(); - entries[i] = new Entry(benchmarkId, state, nodes); + int size = in.readVInt(); + Map map = new HashMap<>(size); + for (int j = 0; j < size; j++) { + map.put(in.readString(), Entry.NodeState.fromId(in.readByte())); + } + entries[i] = new Entry(benchmarkId, masterNodeId, state, map); } return new BenchmarkMetaData(entries); } @@ -179,8 +289,13 @@ public void writeTo(BenchmarkMetaData repositories, StreamOutput out) throws IOE out.writeVInt(repositories.entries().size()); for (Entry entry : repositories.entries()) { out.writeString(entry.benchmarkId()); + out.writeString(entry.masterNodeId()); out.writeByte(entry.state().id()); - out.writeStringArray(entry.nodes()); + out.writeVInt(entry.nodeStateMap.size()); + for (Map.Entry mapEntry : entry.nodeStateMap().entrySet()) { + out.writeString(mapEntry.getKey()); + out.writeByte(mapEntry.getValue().id()); + } } } @@ -202,9 +317,10 @@ public void toXContent(Entry entry, XContentBuilder builder, ToXContent.Params p builder.startObject(); builder.field("id", entry.benchmarkId()); builder.field("state", entry.state()); - builder.startArray("on_nodes"); - for (String nodeid : entry.nodes()) { - builder.value(nodeid); + builder.startArray("node_states"); + for (Map.Entry mapEntry : entry.nodeStateMap().entrySet()) { + builder.field("node_id", mapEntry.getKey()); + builder.field("node_state", mapEntry.getValue()); } builder.endArray(); builder.endObject(); @@ -217,10 +333,19 @@ public boolean isPersistent() { public boolean contains(String benchmarkId) { for (Entry e : entries) { - if (e.benchmarkId.equals(benchmarkId)) { - return true; - } + if (e.benchmarkId.equals(benchmarkId)) { + return true; + } } return false; } + + public Entry get(String benchmarkId) { + for (Entry e : entries) { + if (e.benchmarkId.equals(benchmarkId)) { + return e; + } + } + return null; + } } diff --git a/src/main/java/org/elasticsearch/node/internal/InternalNode.java b/src/main/java/org/elasticsearch/node/internal/InternalNode.java index 747ad1d0a06c3..a7b964ed2859a 100644 --- a/src/main/java/org/elasticsearch/node/internal/InternalNode.java +++ b/src/main/java/org/elasticsearch/node/internal/InternalNode.java @@ -23,7 +23,7 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.Version; import org.elasticsearch.action.ActionModule; -import org.elasticsearch.action.bench.BenchmarkModule; +import org.elasticsearch.action.benchmark.BenchmarkModule; import org.elasticsearch.bulk.udp.BulkUdpModule; import org.elasticsearch.bulk.udp.BulkUdpService; import org.elasticsearch.cache.recycler.PageCacheRecycler; diff --git a/src/main/java/org/elasticsearch/rest/action/RestActionModule.java b/src/main/java/org/elasticsearch/rest/action/RestActionModule.java index 7dc3ab18b7652..a46ccaa0ca9cd 100644 --- a/src/main/java/org/elasticsearch/rest/action/RestActionModule.java +++ b/src/main/java/org/elasticsearch/rest/action/RestActionModule.java @@ -79,8 +79,12 @@ import org.elasticsearch.rest.action.admin.indices.warmer.get.RestGetWarmerAction; import org.elasticsearch.rest.action.admin.indices.warmer.put.RestPutWarmerAction; import org.elasticsearch.rest.action.admin.indices.recovery.RestRecoveryAction; -import org.elasticsearch.rest.action.bench.RestBenchAction; import org.elasticsearch.rest.action.bulk.RestBulkAction; +import org.elasticsearch.rest.action.benchmark.abort.RestBenchmarkAbortAction; +import org.elasticsearch.rest.action.benchmark.pause.RestBenchmarkPauseAction; +import org.elasticsearch.rest.action.benchmark.resume.RestBenchmarkResumeAction; +import org.elasticsearch.rest.action.benchmark.status.RestBenchmarkStatusAction; +import org.elasticsearch.rest.action.benchmark.submit.RestBenchmarkSubmitAction; import org.elasticsearch.rest.action.cat.*; import org.elasticsearch.rest.action.delete.RestDeleteAction; import org.elasticsearch.rest.action.deletebyquery.RestDeleteByQueryAction; @@ -218,7 +222,11 @@ protected void configure() { bind(RestRecoveryAction.class).asEagerSingleton(); // Benchmark API - bind(RestBenchAction.class).asEagerSingleton(); + bind(RestBenchmarkSubmitAction.class).asEagerSingleton(); + bind(RestBenchmarkStatusAction.class).asEagerSingleton(); + bind(RestBenchmarkAbortAction.class).asEagerSingleton(); + bind(RestBenchmarkResumeAction.class).asEagerSingleton(); + bind(RestBenchmarkPauseAction.class).asEagerSingleton(); // Templates API bind(RestGetSearchTemplateAction.class).asEagerSingleton(); diff --git a/src/main/java/org/elasticsearch/rest/action/benchmark/abort/RestBenchmarkAbortAction.java b/src/main/java/org/elasticsearch/rest/action/benchmark/abort/RestBenchmarkAbortAction.java new file mode 100644 index 0000000000000..3a09673e6f644 --- /dev/null +++ b/src/main/java/org/elasticsearch/rest/action/benchmark/abort/RestBenchmarkAbortAction.java @@ -0,0 +1,65 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch 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. + */ + +package org.elasticsearch.rest.action.benchmark.abort; + +import org.elasticsearch.action.benchmark.abort.BenchmarkAbortRequest; +import org.elasticsearch.action.benchmark.abort.BenchmarkAbortResponse; +import org.elasticsearch.client.Client; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.rest.*; +import org.elasticsearch.rest.action.support.RestBuilderListener; + +import static org.elasticsearch.rest.RestRequest.Method.POST; +import static org.elasticsearch.rest.RestStatus.OK; + +/** + * REST handler for benchmark status actions. + */ +public class RestBenchmarkAbortAction extends BaseRestHandler { + + @Inject + public RestBenchmarkAbortAction(Settings settings, Client client, RestController controller) { + super(settings, client); + controller.registerHandler(POST, "/_bench/abort/{name}", this); + } + + /** + * Aborts actively running benchmark(s) + */ + public void handleRequest(final RestRequest request, final RestChannel channel, final Client client) { + + final String[] benchmarkNames = Strings.splitStringByCommaToArray(request.param("name")); + final BenchmarkAbortRequest benchmarkAbortRequest = new BenchmarkAbortRequest(benchmarkNames); + + client.abortBench(benchmarkAbortRequest, new RestBuilderListener(channel) { + + @Override + public RestResponse buildResponse(BenchmarkAbortResponse response, XContentBuilder builder) throws Exception { + builder.startObject(); + response.toXContent(builder, request); + builder.endObject(); + return new BytesRestResponse(OK, builder); + } + }); + } +} diff --git a/src/main/java/org/elasticsearch/rest/action/benchmark/pause/RestBenchmarkPauseAction.java b/src/main/java/org/elasticsearch/rest/action/benchmark/pause/RestBenchmarkPauseAction.java new file mode 100644 index 0000000000000..e240d15caf944 --- /dev/null +++ b/src/main/java/org/elasticsearch/rest/action/benchmark/pause/RestBenchmarkPauseAction.java @@ -0,0 +1,66 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch 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. + */ + +package org.elasticsearch.rest.action.benchmark.pause; + +import org.elasticsearch.action.benchmark.pause.BenchmarkPauseRequest; +import org.elasticsearch.action.benchmark.pause.BenchmarkPauseResponse; +import org.elasticsearch.client.Client; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.rest.*; +import org.elasticsearch.rest.action.support.RestBuilderListener; + +import static org.elasticsearch.rest.RestRequest.Method.POST; +import static org.elasticsearch.rest.RestStatus.OK; + + +/** + * REST handler for benchmark pause actions. + */ +public class RestBenchmarkPauseAction extends BaseRestHandler { + + @Inject + public RestBenchmarkPauseAction(Settings settings, Client client, RestController controller) { + super(settings, client); + controller.registerHandler(POST, "/_bench/pause/{name}", this); + } + + /** + * Pauses active benchmark(s) + */ + @Override + public void handleRequest(final RestRequest request, final RestChannel channel, final Client client) throws Exception { + + final String[] benchmarkNames = Strings.splitStringByCommaToArray(request.param("name")); + final BenchmarkPauseRequest benchmarkPauseRequest = new BenchmarkPauseRequest(benchmarkNames); + + client.pauseBenchmark(benchmarkPauseRequest, new RestBuilderListener(channel) { + @Override + public RestResponse buildResponse(BenchmarkPauseResponse response, XContentBuilder builder) throws Exception { + builder.startObject(); + response.toXContent(builder, request); + builder.endObject(); + return new BytesRestResponse(OK, builder); + } + }); + } +} diff --git a/src/main/java/org/elasticsearch/rest/action/benchmark/resume/RestBenchmarkResumeAction.java b/src/main/java/org/elasticsearch/rest/action/benchmark/resume/RestBenchmarkResumeAction.java new file mode 100644 index 0000000000000..51040641d6595 --- /dev/null +++ b/src/main/java/org/elasticsearch/rest/action/benchmark/resume/RestBenchmarkResumeAction.java @@ -0,0 +1,66 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch 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. + */ + +package org.elasticsearch.rest.action.benchmark.resume; + +import org.elasticsearch.action.benchmark.resume.BenchmarkResumeRequest; +import org.elasticsearch.action.benchmark.resume.BenchmarkResumeResponse; +import org.elasticsearch.client.Client; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.rest.*; +import org.elasticsearch.rest.action.support.RestBuilderListener; + +import static org.elasticsearch.rest.RestRequest.Method.POST; +import static org.elasticsearch.rest.RestStatus.OK; + + +/** + * REST handler for benchmark resume actions. + */ +public class RestBenchmarkResumeAction extends BaseRestHandler { + + @Inject + public RestBenchmarkResumeAction(Settings settings, Client client, RestController controller) { + super(settings, client); + controller.registerHandler(POST, "/_bench/resume/{name}", this); + } + + /** + * Resumes paused benchmark(s) + */ + @Override + public void handleRequest(final RestRequest request, final RestChannel channel, final Client client) throws Exception { + + final String[] benchmarkNames = Strings.splitStringByCommaToArray(request.param("name")); + final BenchmarkResumeRequest benchmarkResumeRequest = new BenchmarkResumeRequest(benchmarkNames); + + client.resumeBenchmark(benchmarkResumeRequest, new RestBuilderListener(channel) { + @Override + public RestResponse buildResponse(final BenchmarkResumeResponse response, final XContentBuilder builder) throws Exception { + builder.startObject(); + response.toXContent(builder, request); + builder.endObject(); + return new BytesRestResponse(OK, builder); + } + }); + } +} diff --git a/src/main/java/org/elasticsearch/rest/action/benchmark/status/RestBenchmarkStatusAction.java b/src/main/java/org/elasticsearch/rest/action/benchmark/status/RestBenchmarkStatusAction.java new file mode 100644 index 0000000000000..0f5577cdfd157 --- /dev/null +++ b/src/main/java/org/elasticsearch/rest/action/benchmark/status/RestBenchmarkStatusAction.java @@ -0,0 +1,66 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch 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. + */ + +package org.elasticsearch.rest.action.benchmark.status; + +import org.elasticsearch.action.benchmark.status.*; +import org.elasticsearch.client.Client; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.rest.*; +import org.elasticsearch.rest.action.support.RestBuilderListener; + +import static org.elasticsearch.rest.RestRequest.Method.*; +import static org.elasticsearch.rest.RestStatus.*; + +/** + * REST handler for benchmark status actions. + */ +public class RestBenchmarkStatusAction extends BaseRestHandler { + + @Inject + public RestBenchmarkStatusAction(Settings settings, Client client, RestController controller) { + super(settings, client); + controller.registerHandler(GET, "/_bench/status", this); + controller.registerHandler(GET, "/_bench/status/{name}", this); + } + + /** + * Reports on the status of all actively running benchmarks + */ + @Override + public void handleRequest(final RestRequest request, final RestChannel channel, final Client client) { + + final String[] benchmarkNames = Strings.splitStringByCommaToArray(request.param("name")); + BenchmarkStatusRequest benchmarkStatusRequest = new BenchmarkStatusRequest(benchmarkNames); + + client.benchmarkStatus(benchmarkStatusRequest, new RestBuilderListener(channel) { + + @Override + public RestResponse buildResponse(BenchmarkStatusResponses response, XContentBuilder builder) throws Exception { + builder.startObject(); + response.toXContent(builder, request); + builder.endObject(); + return new BytesRestResponse(OK, builder); + } + }); + } +} diff --git a/src/main/java/org/elasticsearch/rest/action/bench/RestBenchAction.java b/src/main/java/org/elasticsearch/rest/action/benchmark/submit/RestBenchmarkSubmitAction.java similarity index 79% rename from src/main/java/org/elasticsearch/rest/action/bench/RestBenchAction.java rename to src/main/java/org/elasticsearch/rest/action/benchmark/submit/RestBenchmarkSubmitAction.java index 3256df8ba8a8e..1216e890c0875 100644 --- a/src/main/java/org/elasticsearch/rest/action/bench/RestBenchAction.java +++ b/src/main/java/org/elasticsearch/rest/action/benchmark/submit/RestBenchmarkSubmitAction.java @@ -17,11 +17,15 @@ * under the License. */ -package org.elasticsearch.rest.action.bench; +package org.elasticsearch.rest.action.benchmark.submit; import com.google.common.primitives.Doubles; import org.elasticsearch.ElasticsearchParseException; -import org.elasticsearch.action.bench.*; +import org.elasticsearch.action.benchmark.BenchmarkCompetitorBuilder; +import org.elasticsearch.action.benchmark.BenchmarkSettings; +import org.elasticsearch.action.benchmark.start.BenchmarkStartRequest; +import org.elasticsearch.action.benchmark.start.BenchmarkStartRequestBuilder; +import org.elasticsearch.action.benchmark.start.BenchmarkStartResponse; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchType; import org.elasticsearch.client.Client; @@ -35,7 +39,6 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.rest.*; import org.elasticsearch.rest.action.admin.indices.cache.clear.RestClearIndicesCacheAction; -import org.elasticsearch.rest.action.support.AcknowledgedRestListener; import org.elasticsearch.rest.action.support.RestBuilderListener; import java.io.IOException; @@ -43,101 +46,46 @@ import java.util.List; import static org.elasticsearch.common.xcontent.json.JsonXContent.contentBuilder; -import static org.elasticsearch.rest.RestRequest.Method.*; -import static org.elasticsearch.rest.RestStatus.*; +import static org.elasticsearch.rest.RestRequest.Method.PUT; +import static org.elasticsearch.rest.RestStatus.BAD_REQUEST; +import static org.elasticsearch.rest.RestStatus.OK; /** - * REST handler for benchmark actions. + * REST handler for submitting benchmarks. */ -public class RestBenchAction extends BaseRestHandler { +public class RestBenchmarkSubmitAction extends BaseRestHandler { @Inject - public RestBenchAction(Settings settings, Client client, RestController controller) { + protected RestBenchmarkSubmitAction(Settings settings, Client client, RestController controller) { super(settings, client); - // List active benchmarks - controller.registerHandler(GET, "/_bench", this); - controller.registerHandler(GET, "/{index}/_bench", this); - controller.registerHandler(GET, "/{index}/{type}/_bench", this); - - // Submit benchmark - controller.registerHandler(PUT, "/_bench", this); - controller.registerHandler(PUT, "/{index}/_bench", this); - controller.registerHandler(PUT, "/{index}/{type}/_bench", this); - - // Abort benchmark - controller.registerHandler(POST, "/_bench/abort/{name}", this); + controller.registerHandler(PUT, "/_bench/submit", this); + controller.registerHandler(PUT, "/{index}/_bench/submit", this); + controller.registerHandler(PUT, "/{index}/{type}/_bench/submit", this); } @Override - public void handleRequest(final RestRequest request, final RestChannel channel, final Client client) { - switch (request.method()) { - case POST: - handleAbortRequest(request, channel, client); - break; - case PUT: - handleSubmitRequest(request, channel, client); - break; - case GET: - handleStatusRequest(request, channel, client); - break; - default: - // Politely ignore methods we don't support - channel.sendResponse(new BytesRestResponse(METHOD_NOT_ALLOWED)); - } - } - - /** - * Reports on the status of all actively running benchmarks - */ - private void handleStatusRequest(final RestRequest request, final RestChannel channel, final Client client) { - - BenchmarkStatusRequest benchmarkStatusRequest = new BenchmarkStatusRequest(); - - client.benchStatus(benchmarkStatusRequest, new RestBuilderListener(channel) { - - @Override - public RestResponse buildResponse(BenchmarkStatusResponse response, XContentBuilder builder) throws Exception { - builder.startObject(); - response.toXContent(builder, request); - builder.endObject(); - return new BytesRestResponse(OK, builder); - } - }); - } - - /** - * Aborts an actively running benchmark - */ - private void handleAbortRequest(final RestRequest request, final RestChannel channel, final Client client) { - final String[] benchmarkNames = Strings.splitStringByCommaToArray(request.param("name")); - AbortBenchmarkRequest abortBenchmarkRequest = new AbortBenchmarkRequest(benchmarkNames); - - client.abortBench(abortBenchmarkRequest, new AcknowledgedRestListener(channel)); - } - - /** - * Submits a benchmark for execution - */ - private void handleSubmitRequest(final RestRequest request, final RestChannel channel, final Client client) { + public void handleRequest(final RestRequest request, final RestChannel channel, final Client client) throws Exception { String[] indices = Strings.splitStringByCommaToArray(request.param("index")); String[] types = Strings.splitStringByCommaToArray(request.param("type")); - final BenchmarkRequest benchmarkRequest; + final BenchmarkStartRequest benchmarkStartRequest; try { - BenchmarkRequestBuilder builder = new BenchmarkRequestBuilder(client); - builder.setVerbose(request.paramAsBoolean("verbose", false)); - benchmarkRequest = parse(builder, request.content(), request.contentUnsafe()); - benchmarkRequest.cascadeGlobalSettings(); // Make sure competitors inherit global settings - benchmarkRequest.applyLateBoundSettings(indices, types); // Some settings cannot be applied until after parsing - Exception ex = benchmarkRequest.validate(); + BenchmarkStartRequestBuilder builder = new BenchmarkStartRequestBuilder(client); + benchmarkStartRequest = parse(builder, request.content(), request.contentUnsafe()); + benchmarkStartRequest.cascadeGlobalSettings(); // Make sure competitors inherit global settings + benchmarkStartRequest.applyLateBoundSettings(indices, types); // Some settings cannot be applied until after parsing + if (request.hasParam("verbose")) { + builder.setVerbose(request.paramAsBoolean("verbose", false)); + } + Exception ex = benchmarkStartRequest.validate(); if (ex != null) { throw ex; } - benchmarkRequest.listenerThreaded(false); + benchmarkStartRequest.listenerThreaded(false); } catch (Exception e) { - logger.debug("failed to parse search request parameters", e); + logger.debug("failed to parse search request parameters", e); try { channel.sendResponse(new BytesRestResponse(BAD_REQUEST, contentBuilder().startObject().field("error", e.getMessage()).endObject())); } catch (IOException e1) { @@ -145,10 +93,10 @@ private void handleSubmitRequest(final RestRequest request, final RestChannel ch } return; } - client.bench(benchmarkRequest, new RestBuilderListener(channel) { + client.startBenchmark(benchmarkStartRequest, new RestBuilderListener(channel) { @Override - public RestResponse buildResponse(BenchmarkResponse response, XContentBuilder builder) throws Exception { + public RestResponse buildResponse(BenchmarkStartResponse response, XContentBuilder builder) throws Exception { builder.startObject(); response.toXContent(builder, request); builder.endObject(); @@ -157,7 +105,7 @@ public RestResponse buildResponse(BenchmarkResponse response, XContentBuilder bu }); } - public static BenchmarkRequest parse(BenchmarkRequestBuilder builder, BytesReference data, boolean contentUnsafe) throws Exception { + public static BenchmarkStartRequest parse(BenchmarkStartRequestBuilder builder, BytesReference data, boolean contentUnsafe) throws Exception { XContent xContent = XContentFactory.xContent(data); XContentParser p = xContent.createParser(data); XContentParser.Token token = p.nextToken(); @@ -218,6 +166,8 @@ public static BenchmarkRequest parse(BenchmarkRequestBuilder builder, BytesRefer case VALUE_BOOLEAN: if ("warmup".equals(fieldName)) { builder.setWarmup(p.booleanValue()); + } else if ("verbose".equals(fieldName)) { + builder.setVerbose(p.booleanValue()); } else if ("clear_caches".equals(fieldName)) { if (p.booleanValue()) { throw new ElasticsearchParseException("Failed parsing field [" + fieldName + "] must specify which caches to clear"); @@ -379,3 +329,4 @@ private static void parseClearCaches(XContentParser p, BenchmarkSettings.ClearCa } } } + diff --git a/src/test/java/org/elasticsearch/action/bench/BenchmarkIntegrationTest.java b/src/test/java/org/elasticsearch/action/bench/BenchmarkIntegrationTest.java deleted file mode 100644 index cd3a35e4963a2..0000000000000 --- a/src/test/java/org/elasticsearch/action/bench/BenchmarkIntegrationTest.java +++ /dev/null @@ -1,446 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch 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. - */ -package org.elasticsearch.action.bench; - -import com.google.common.base.Predicate; -import org.apache.lucene.util.English; -import org.elasticsearch.action.ActionFuture; -import org.elasticsearch.action.index.IndexRequestBuilder; -import org.elasticsearch.action.search.SearchRequest; -import org.elasticsearch.common.Strings; -import org.elasticsearch.common.settings.ImmutableSettings; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.index.query.FilterBuilders; -import org.elasticsearch.index.query.functionscore.script.ScriptScoreFunctionBuilder; -import org.elasticsearch.script.groovy.GroovyScriptEngineService; -import org.elasticsearch.test.ElasticsearchIntegrationTest; -import org.junit.After; -import org.junit.Before; -import org.junit.Ignore; -import org.junit.Test; - -import java.util.*; -import java.util.concurrent.CountDownLatch; - -import static org.elasticsearch.client.Requests.searchRequest; -import static org.elasticsearch.index.query.QueryBuilders.functionScoreQuery; -import static org.elasticsearch.index.query.functionscore.ScoreFunctionBuilders.scriptFunction; -import static org.elasticsearch.search.builder.SearchSourceBuilder.searchSource; -import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; -import static org.hamcrest.Matchers.*; - -/** - * Integration tests for benchmark API - */ -@ElasticsearchIntegrationTest.ClusterScope(scope = ElasticsearchIntegrationTest.Scope.SUITE) -@Ignore -public class BenchmarkIntegrationTest extends ElasticsearchIntegrationTest { - - private static final String BENCHMARK_NAME = "test_benchmark"; - private static final String BENCHMARK_NAME_WILDCARD = "test_*"; - private static final String COMPETITOR_PREFIX = "competitor_"; - private static final String INDEX_PREFIX = "test_index_"; - private static final String INDEX_TYPE = "test_type"; - - private int numExecutorNodes = 0; - private Map competitionSettingsMap; - private String[] indices = Strings.EMPTY_ARRAY; - private HashMap benchNodes = new HashMap<>(); - - - - protected synchronized Settings nodeSettings(int nodeOrdinal) { - if (nodeOrdinal == 0) { // at least one - return ImmutableSettings.builder().put(super.nodeSettings(nodeOrdinal)) - .put("node.bench", true).put(GroovyScriptEngineService.GROOVY_SCRIPT_SANDBOX_ENABLED, false).build(); - } else { - if (benchNodes.containsKey(nodeOrdinal)) { - return ImmutableSettings.builder().put(super.nodeSettings(nodeOrdinal)) - .put("node.bench", benchNodes.get(nodeOrdinal)).put(GroovyScriptEngineService.GROOVY_SCRIPT_SANDBOX_ENABLED, false).build(); - } else { - boolean b = randomBoolean(); - benchNodes.put(nodeOrdinal, b); - return ImmutableSettings.builder().put(super.nodeSettings(nodeOrdinal)) - .put("node.bench", b).put(GroovyScriptEngineService.GROOVY_SCRIPT_SANDBOX_ENABLED, false).build(); - } - } - } - - @After - public void afterBenchmarkIntegrationTests() throws Exception { - final BenchmarkStatusResponse statusResponse = client().prepareBenchStatus().execute().actionGet(); - assertThat("Some benchmarks are still running", statusResponse.benchmarkResponses(), is(empty())); - } - - @Before - public void beforeBenchmarkIntegrationTests() throws Exception { - waitForTestLatch = null; - waitForQuery = null; - numExecutorNodes = internalCluster().numBenchNodes(); - competitionSettingsMap = new HashMap<>(); - logger.info("--> indexing random data"); - indices = randomData(); - } - - @Test - public void testSubmitBenchmark() throws Exception { - final int iters = between(1, 3); // we run this more than once to make sure metadata is cleaned up propperly - for (int i = 0; i < iters ; i++) { - final BenchmarkRequest request = - BenchmarkTestUtil.randomRequest(client(), indices, numExecutorNodes, competitionSettingsMap); - logger.info("--> Submitting benchmark - competitors [{}] iterations [{}]", request.competitors().size(), - request.settings().iterations()); - final BenchmarkResponse response = client().bench(request).actionGet(); - - assertThat(response, notNullValue()); - assertThat(response.state(), equalTo(BenchmarkResponse.State.COMPLETE)); - assertFalse(response.hasErrors()); - assertThat(response.benchmarkName(), equalTo(BENCHMARK_NAME)); - assertThat(response.competitionResults().size(), equalTo(request.competitors().size())); - - for (CompetitionResult result : response.competitionResults().values()) { - assertThat(result.nodeResults().size(), equalTo(numExecutorNodes)); - validateCompetitionResult(result, competitionSettingsMap.get(result.competitionName()), true); - } - } - } - - @Test - public void testListBenchmarks() throws Exception { - SearchRequest searchRequest = prepareBlockingScriptQuery(); - final BenchmarkRequest request = - BenchmarkTestUtil.randomRequest(client(), indices, numExecutorNodes, competitionSettingsMap, searchRequest); - logger.info("--> Submitting benchmark - competitors [{}] iterations [{}]", request.competitors().size(), - request.settings().iterations()); - - final ActionFuture future = client().bench(request); - try { - waitForQuery.await(); - final BenchmarkStatusResponse statusResponse = client().prepareBenchStatus().execute().actionGet(); - waitForTestLatch.countDown(); - assertThat(statusResponse.benchmarkResponses().size(), equalTo(1)); - for (BenchmarkResponse benchmarkResponse : statusResponse.benchmarkResponses()) { - assertThat(benchmarkResponse.benchmarkName(), equalTo(BENCHMARK_NAME)); - assertThat(benchmarkResponse.state(), equalTo(BenchmarkResponse.State.RUNNING)); - assertFalse(benchmarkResponse.hasErrors()); - - for (CompetitionResult result : benchmarkResponse.competitionResults().values()) { - assertThat(result.nodeResults().size(), lessThanOrEqualTo(numExecutorNodes)); - validateCompetitionResult(result, competitionSettingsMap.get(result.competitionName()), false); - } - } - - } finally { - if (waitForTestLatch.getCount() == 1) { - waitForTestLatch.countDown(); - } - client().prepareAbortBench(BENCHMARK_NAME).get(); - // Confirm that there are no active benchmarks in the cluster - assertThat(client().prepareBenchStatus().execute().actionGet().totalActiveBenchmarks(), equalTo(0)); - assertThat(waitForTestLatch.getCount(), is(0l)); - } - // Confirm that benchmark was indeed aborted - assertThat(future.get().state(), isOneOf(BenchmarkResponse.State.ABORTED, BenchmarkResponse.State.COMPLETE)); - - } - - public static CountDownLatch waitForTestLatch; - public static CountDownLatch waitForQuery; - - private SearchRequest prepareBlockingScriptQuery() { - /* Chuck Norris back in the house!! - this is super evil but the only way at this - point to ensure we actually call abort / list while a benchmark is executing - without doing busy waiting etc. This Script calls the two static latches above and this test - will not work if somebody messes around with them but it's much faster and less resource intensive / hardware - dependent to run massive benchmarks and do busy waiting. */ - internalCluster(); // mark that we need a JVM local cluster! - waitForQuery = new CountDownLatch(1); - waitForTestLatch = new CountDownLatch(1); - String className = "BenchmarkIntegrationTest"; - ScriptScoreFunctionBuilder scriptFunction = scriptFunction("import " + this.getClass().getName() + "; \n" + - className + ".waitForQuery.countDown(); \n" + className + ".waitForTestLatch.await(); \n return 1.0;"); - SearchRequest searchRequest = searchRequest().source( - searchSource() - .query(functionScoreQuery(FilterBuilders.matchAllFilter(), scriptFunction))); - return searchRequest; - } - - @Test - public void testBenchmarkWithErrors() { - List reqList = new ArrayList<>(); - int numQueries = scaledRandomIntBetween(20, 100); - int numErrors = scaledRandomIntBetween(1, numQueries); - final boolean containsFatal = randomBoolean(); - if (containsFatal) { - ScriptScoreFunctionBuilder scriptFunction = scriptFunction("DOES NOT COMPILE - fails on any shard"); - SearchRequest searchRequest = searchRequest().source( - searchSource() - .query(functionScoreQuery(FilterBuilders.matchAllFilter(), scriptFunction))); - reqList.add(searchRequest); - - } - for (int i = 0; reqList.size() < numErrors; i++) { - ScriptScoreFunctionBuilder scriptFunction = scriptFunction("throw new RuntimeException();"); - SearchRequest searchRequest = searchRequest().source( - searchSource() - .query(functionScoreQuery(FilterBuilders.matchAllFilter(), scriptFunction))); - reqList.add(searchRequest); - } - logger.info("--> run with [{}] errors ", numErrors); - for (int i = 0; reqList.size() < numQueries; i++) { - - reqList.add(BenchmarkTestUtil.randomSearch(client(), indices)); - } - Collections.shuffle(reqList, getRandom()); - - final BenchmarkRequest request = - BenchmarkTestUtil.randomRequest(client(),indices, numExecutorNodes, competitionSettingsMap, reqList.toArray(new SearchRequest[0])); - logger.info("--> Submitting benchmark - competitors [{}] iterations [{}]", request.competitors().size(), - request.settings().iterations()); - final BenchmarkResponse response = client().bench(request).actionGet(); - - assertThat(response, notNullValue()); - if (response.hasErrors() || containsFatal) { - assertThat(response.state(), equalTo(BenchmarkResponse.State.FAILED)); - } else { - assertThat(response.state(), equalTo(BenchmarkResponse.State.COMPLETE)); - for (CompetitionResult result : response.competitionResults().values()) { - assertThat(result.nodeResults().size(), equalTo(numExecutorNodes)); - validateCompetitionResult(result, competitionSettingsMap.get(result.competitionName()), true); - } - } - assertThat(response.benchmarkName(), equalTo(BENCHMARK_NAME)); - } - - @Test - public void testAbortByPattern() throws Exception { - final int iters = between(1, 3); // we run this more than once to make sure metadata is cleaned up propperly - for (int i = 0; i < iters ; i++) { - List requests = new ArrayList<>(); - List> responses = new ArrayList<>(); - - SearchRequest searchRequest = prepareBlockingScriptQuery(); - final int benches = between(1, 3); - String[] names = new String[benches]; - for (int k = 0; k < benches; k++) { - final BenchmarkRequest request = - BenchmarkTestUtil.randomRequest(client(), indices, numExecutorNodes, competitionSettingsMap, searchRequest); - request.settings().iterations(Integer.MAX_VALUE, true); // massive amount of iterations - names[k] = BENCHMARK_NAME + Integer.toString(k); - request.benchmarkName(names[k]); - requests.add(request); - logger.info("--> Submitting benchmark - competitors [{}] iterations [{}]", request.competitors().size(), - request.settings().iterations()); - } - - boolean aborted = false; - for (BenchmarkRequest r : requests) { - final ActionFuture benchmarkResponse = client().bench(r); - responses.add(benchmarkResponse); - } - try { - waitForQuery.await(); - if (benches > 1) { - awaitBusy(new Predicate() { - @Override - public boolean apply(java.lang.Object input) { - return client().prepareBenchStatus().get().benchmarkResponses().size() == benches; - } - }); - } - final String badPatternA = "*z"; - final String badPatternB = "xxx"; - final String[] patterns; - switch (getRandom().nextInt(3)) { - case 0: - patterns = new String [] {"*"}; - break; - case 1: - patterns = new String[] {BENCHMARK_NAME_WILDCARD, badPatternA, badPatternB }; - break; - case 2: - patterns = names; - break; - default: - patterns = new String [] {BENCHMARK_NAME_WILDCARD}; - } - final AbortBenchmarkResponse abortResponse = client().prepareAbortBench(patterns).get(); - aborted = true; - assertAcked(abortResponse); - - // Confirm that there are no active benchmarks in the cluster - final BenchmarkStatusResponse statusResponse = client().prepareBenchStatus().execute().actionGet(); - waitForTestLatch.countDown(); // let the queries go - we already aborted and got the status - assertThat(statusResponse.totalActiveBenchmarks(), equalTo(0)); - - // Confirm that benchmark was indeed aborted - for (ActionFuture r : responses) { - assertThat(r.get().state(), is(BenchmarkResponse.State.ABORTED)); - } - } finally { - if (waitForTestLatch.getCount() == 1) { - waitForTestLatch.countDown(); - } - if (!aborted) { - client().prepareAbortBench(BENCHMARK_NAME).get(); - } - assertThat(waitForTestLatch.getCount(), is(0l)); - } - } - } - - @Test - public void testAbortBenchmark() throws Exception { - final int iters = between(1, 3); // we run this more than once to make sure metadata is cleaned up propperly - for (int i = 0; i < iters ; i++) { - SearchRequest searchRequest = prepareBlockingScriptQuery(); - final BenchmarkRequest request = - BenchmarkTestUtil.randomRequest(client(), indices, numExecutorNodes, competitionSettingsMap, searchRequest); - request.settings().iterations(Integer.MAX_VALUE, true); // massive amount of iterations - logger.info("--> Submitting benchmark - competitors [{}] iterations [{}]", request.competitors().size(), - request.settings().iterations()); - boolean aborted = false; - final ActionFuture benchmarkResponse = client().bench(request); - try { - waitForQuery.await(); - final AbortBenchmarkResponse abortResponse = - client().prepareAbortBench(BENCHMARK_NAME).get(); - aborted = true; - // Confirm that the benchmark was actually aborted and did not finish on its own - assertAcked(abortResponse); - // Confirm that there are no active benchmarks in the cluster - final BenchmarkStatusResponse statusResponse = client().prepareBenchStatus().execute().actionGet(); - waitForTestLatch.countDown(); // let the queries go - we already aborted and got the status - assertThat(statusResponse.totalActiveBenchmarks(), equalTo(0)); - - // Confirm that benchmark was indeed aborted - assertThat(benchmarkResponse.get().state(), is(BenchmarkResponse.State.ABORTED)); - - } finally { - if (waitForTestLatch.getCount() == 1) { - waitForTestLatch.countDown(); - } - if (!aborted) { - client().prepareAbortBench(BENCHMARK_NAME).get(); - } - assertThat(waitForTestLatch.getCount(), is(0l)); - } - } - } - - @Test(expected = BenchmarkMissingException.class) - public void testAbortNoSuchBenchmark() throws Exception { - client().prepareAbortBench(BENCHMARK_NAME).execute().actionGet(); - } - - private void validateCompetitionResult(CompetitionResult result, BenchmarkSettings requestedSettings, boolean strict) { - // Validate settings - assertTrue(result.competitionName().startsWith(COMPETITOR_PREFIX)); - assertThat(result.concurrency(), equalTo(requestedSettings.concurrency())); - assertThat(result.multiplier(), equalTo(requestedSettings.multiplier())); - - // Validate node-level responses - for (CompetitionNodeResult nodeResult : result.nodeResults()) { - - assertThat(nodeResult.nodeName(), notNullValue()); - - assertThat(nodeResult.totalIterations(), equalTo(requestedSettings.iterations())); - if (strict) { - assertThat(nodeResult.completedIterations(), equalTo(requestedSettings.iterations())); - final int expectedQueryCount = requestedSettings.multiplier() * - nodeResult.totalIterations() * requestedSettings.searchRequests().size(); - assertThat(nodeResult.totalExecutedQueries(), equalTo(expectedQueryCount)); - assertThat(nodeResult.iterations().size(), equalTo(requestedSettings.iterations())); - } - - assertThat(nodeResult.warmUpTime(), greaterThanOrEqualTo(0L)); - - for (CompetitionIteration iteration : nodeResult.iterations()) { - // Basic sanity checks - iteration.computeStatistics(); - assertThat(iteration.totalTime(), greaterThanOrEqualTo(0L)); - assertThat(iteration.min(), greaterThanOrEqualTo(0L)); - assertThat(iteration.max(), greaterThanOrEqualTo(iteration.min())); - assertThat(iteration.mean(), greaterThanOrEqualTo((double) iteration.min())); - assertThat(iteration.mean(), lessThanOrEqualTo((double) iteration.max())); - assertThat(iteration.queriesPerSecond(), greaterThanOrEqualTo(0.0)); - assertThat(iteration.millisPerHit(), greaterThanOrEqualTo(0.0)); - validatePercentiles(iteration.percentileValues()); - } - } - - // Validate summary statistics - final CompetitionSummary summary = result.competitionSummary(); - summary.computeSummaryStatistics(); - assertThat(summary, notNullValue()); - assertThat(summary.getMin(), greaterThanOrEqualTo(0L)); - assertThat(summary.getMax(), greaterThanOrEqualTo(summary.getMin())); - assertThat(summary.getMean(), greaterThanOrEqualTo((double) summary.getMin())); - assertThat(summary.getMean(), lessThanOrEqualTo((double) summary.getMax())); - assertThat(summary.getTotalTime(), greaterThanOrEqualTo(0L)); - assertThat(summary.getQueriesPerSecond(), greaterThanOrEqualTo(0.0)); - assertThat(summary.getMillisPerHit(), greaterThanOrEqualTo(0.0)); - assertThat(summary.getAvgWarmupTime(), greaterThanOrEqualTo(0.0)); - if (strict) { - assertThat((int) summary.getTotalIterations(), equalTo(requestedSettings.iterations() * summary.nodeResults().size())); - assertThat((int) summary.getCompletedIterations(), equalTo(requestedSettings.iterations() * summary.nodeResults().size())); - assertThat((int) summary.getTotalQueries(), equalTo(requestedSettings.iterations() * requestedSettings.multiplier() * - requestedSettings.searchRequests().size() * summary.nodeResults().size())); - validatePercentiles(summary.percentileValues); - } - } - - private void validatePercentiles(Map percentiles) { - int i = 0; - double last = Double.NEGATIVE_INFINITY; - for (Map.Entry entry : percentiles.entrySet()) { - assertThat(entry.getKey(), equalTo(BenchmarkSettings.DEFAULT_PERCENTILES[i++])); - // This is a hedge against rounding errors. Sometimes two adjacent percentile values will - // be nearly equivalent except for some insignificant decimal places. In such cases we - // want the two values to compare as equal. - assertThat(entry.getValue(), greaterThanOrEqualTo(last - 1e-6)); - last = entry.getValue(); - } - } - - private String[] randomData() throws Exception { - - final int numIndices = scaledRandomIntBetween(1, 5); - final String[] indices = new String[numIndices]; - - for (int i = 0; i < numIndices; i++) { - indices[i] = INDEX_PREFIX + i; - final int numDocs = scaledRandomIntBetween(1, 100); - final IndexRequestBuilder[] docs = new IndexRequestBuilder[numDocs]; - - for (int j = 0; j < numDocs; j++) { - docs[j] = client().prepareIndex(indices[i], INDEX_TYPE). - setSource(BenchmarkTestUtil.TestIndexField.INT_FIELD.toString(), randomInt(), - BenchmarkTestUtil.TestIndexField.FLOAT_FIELD.toString(), randomFloat(), - BenchmarkTestUtil.TestIndexField.BOOLEAN_FIELD.toString(), randomBoolean(), - BenchmarkTestUtil.TestIndexField.STRING_FIELD.toString(), English.intToEnglish(j)); - } - - indexRandom(true, docs); - } - - flushAndRefresh(); - return indices; - } -} diff --git a/src/test/java/org/elasticsearch/action/benchmark/AbstractBenchmarkTest.java b/src/test/java/org/elasticsearch/action/benchmark/AbstractBenchmarkTest.java new file mode 100644 index 0000000000000..d6385185cfec4 --- /dev/null +++ b/src/test/java/org/elasticsearch/action/benchmark/AbstractBenchmarkTest.java @@ -0,0 +1,159 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch 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. + */ + +package org.elasticsearch.action.benchmark; + +import org.apache.lucene.util.English; + +import org.elasticsearch.action.benchmark.abort.BenchmarkAbortResponse; +import org.elasticsearch.action.benchmark.pause.BenchmarkPauseResponse; +import org.elasticsearch.action.benchmark.resume.BenchmarkResumeResponse; +import org.elasticsearch.action.benchmark.start.BenchmarkStartRequest; +import org.elasticsearch.action.benchmark.start.BenchmarkStartResponse; +import org.elasticsearch.action.benchmark.status.BenchmarkStatusResponses; +import org.elasticsearch.action.index.IndexRequestBuilder; +import org.elasticsearch.cluster.metadata.BenchmarkMetaData; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.collect.Tuple; +import org.elasticsearch.test.ElasticsearchIntegrationTest; +import org.junit.Before; +import org.junit.Ignore; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.Semaphore; + +import static org.elasticsearch.action.benchmark.BenchmarkTestUtil.INDEX_PREFIX; +import static org.elasticsearch.action.benchmark.BenchmarkTestUtil.INDEX_TYPE; +import static org.hamcrest.Matchers.equalTo; + +/** + * Abstract base class for benchmark integration tests. + */ +@Ignore +public class AbstractBenchmarkTest extends ElasticsearchIntegrationTest { + + protected int numExecutorNodes = 0; + protected String[] indices = Strings.EMPTY_ARRAY; + protected Map> competitionSettingsMap; + + @Before + public void baseBefore() throws Exception { + numExecutorNodes = cluster().numBenchNodes(); + } + + protected Iterable mockExecutorServices() { + return internalCluster().getInstances(BenchmarkExecutorService.class); + } + + protected MockBenchmarkCoordinatorService mockCoordinatorService() { + + // Don't use mock service class for getInstances(), otherwise we won't get the singleton. + // Use the base service class instead and cast to the mock service + final Iterable services = internalCluster().getInstances(BenchmarkCoordinatorService.class); + + for (BenchmarkCoordinatorService service : services) { + // The instance on the master node will have all the published meta-data changes + if (((MockBenchmarkCoordinatorService) service).isOnMasterNode()) { + return (MockBenchmarkCoordinatorService) service; + } + } + fail("Unable to find mock benchmark coordinator service on master node"); + return null; + } + + protected String[] randomData() throws Exception { + + final int numIndices = scaledRandomIntBetween(1, 5); + final String[] indices = new String[numIndices]; + + for (int i = 0; i < numIndices; i++) { + indices[i] = INDEX_PREFIX + i; + final int numDocs = scaledRandomIntBetween(1, 100); + final IndexRequestBuilder[] docs = new IndexRequestBuilder[numDocs]; + + for (int j = 0; j < numDocs; j++) { + docs[j] = client().prepareIndex(indices[i], INDEX_TYPE). + setSource(BenchmarkTestUtil.TestIndexField.INT_FIELD.toString(), randomInt(), + BenchmarkTestUtil.TestIndexField.FLOAT_FIELD.toString(), randomFloat(), + BenchmarkTestUtil.TestIndexField.BOOLEAN_FIELD.toString(), randomBoolean(), + BenchmarkTestUtil.TestIndexField.STRING_FIELD.toString(), English.intToEnglish(j)); + } + + indexRandom(true, docs); + } + + flushAndRefresh(); + return indices; + } + + protected void validateStatusRunning(final String benchmarkId) { + + final BenchmarkStatusResponses status = client().prepareBenchmarkStatus(benchmarkId).execute().actionGet(); + assertThat(status.responses().size(), equalTo(1)); + + final BenchmarkStartResponse response = status.responses().get(0); + assertThat(response.benchmarkId(), equalTo(benchmarkId)); + assertThat(response.state(), equalTo(BenchmarkStartResponse.State.RUNNING)); + assertFalse(response.hasErrors()); + } + + protected void validateStatusAborted(final String benchmarkId, final BenchmarkAbortResponse response) { + + validateBatchedResponseHasNodeState(benchmarkId, response, BenchmarkMetaData.Entry.NodeState.ABORTED); + } + + protected void validateStatusPaused(final String benchmarkId, final BenchmarkPauseResponse response) { + + validateBatchedResponseHasNodeState(benchmarkId, response, BenchmarkMetaData.Entry.NodeState.PAUSED); + } + + protected void validateStatusResumed(final String benchmarkId, final BenchmarkResumeResponse response) { + + validateBatchedResponseHasNodeState(benchmarkId, response, BenchmarkMetaData.Entry.NodeState.RUNNING); + } + + protected void validateBatchedResponseHasNodeState(final String benchmarkId, final BatchedResponse response, + final BenchmarkMetaData.Entry.NodeState nodeState) { + + assertNotNull(response.getResponse(benchmarkId)); + + final BatchedResponse.BenchmarkResponse br = response.getResponse(benchmarkId); + final Map nodeResponses = br.nodeResponses(); + assertThat(nodeResponses.size(), equalTo(numExecutorNodes)); + for (Map.Entry entry : nodeResponses.entrySet()) { + assertThat(entry.getValue(), equalTo(nodeState)); + } + } + + protected void control(final CyclicBarrier barrier, final String competition, List semaphores) throws InterruptedException { + + for (BenchmarkExecutorService mock : mockExecutorServices()) { + + final MockBenchmarkExecutorService.MockBenchmarkExecutor executor = ((MockBenchmarkExecutorService) mock).executor(); + final Semaphore semaphore = new Semaphore(1, true); + semaphore.acquire(); + semaphores.add(semaphore); + MockBenchmarkExecutorService.MockBenchmarkExecutor.FlowControl control = new MockBenchmarkExecutorService.MockBenchmarkExecutor.FlowControl(barrier, competition, semaphore); + executor.control(control); + } + } +} diff --git a/src/test/java/org/elasticsearch/action/benchmark/BenchmarkIntegrationTest.java b/src/test/java/org/elasticsearch/action/benchmark/BenchmarkIntegrationTest.java new file mode 100644 index 0000000000000..c4e9d56368a6f --- /dev/null +++ b/src/test/java/org/elasticsearch/action/benchmark/BenchmarkIntegrationTest.java @@ -0,0 +1,405 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch 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. + */ +package org.elasticsearch.action.benchmark; + +import org.elasticsearch.action.ActionFuture; +import org.elasticsearch.action.benchmark.abort.BenchmarkAbortResponse; +import org.elasticsearch.action.benchmark.competition.*; +import org.elasticsearch.action.benchmark.pause.*; +import org.elasticsearch.action.benchmark.resume.*; +import org.elasticsearch.action.benchmark.start.*; +import org.elasticsearch.action.benchmark.status.*; +import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.common.settings.ImmutableSettings; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.query.FilterBuilders; +import org.elasticsearch.index.query.functionscore.script.ScriptScoreFunctionBuilder; +import org.elasticsearch.test.ElasticsearchIntegrationTest; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.util.*; +import java.util.concurrent.Semaphore; +import java.util.concurrent.CyclicBarrier; + +import static org.elasticsearch.action.benchmark.BenchmarkTestUtil.*; +import static org.elasticsearch.client.Requests.searchRequest; +import static org.elasticsearch.index.query.QueryBuilders.functionScoreQuery; +import static org.elasticsearch.index.query.functionscore.ScoreFunctionBuilders.scriptFunction; +import static org.elasticsearch.search.builder.SearchSourceBuilder.searchSource; +import static org.hamcrest.Matchers.*; + +/** + * Integration tests for benchmark API + */ +@ElasticsearchIntegrationTest.ClusterScope(scope = ElasticsearchIntegrationTest.Scope.TEST) +public class BenchmarkIntegrationTest extends AbstractBenchmarkTest { + + protected synchronized Settings nodeSettings(int ordinal) { + return ImmutableSettings.builder().put("node.bench", + ordinal == 0 || randomBoolean()). + put(BenchmarkModule.BENCHMARK_COORDINATOR_SERVICE_KEY, MockBenchmarkCoordinatorService.class). + put(BenchmarkModule.BENCHMARK_EXECUTOR_SERVICE_KEY, MockBenchmarkExecutorService.class). + build(); + } + + @Before + public void pre() throws Exception { + + mockCoordinatorService().clearMockState(); + competitionSettingsMap = new HashMap<>(); + indices = randomData(); + + final Iterable services = mockExecutorServices(); + for (BenchmarkExecutorService service : services) { + ((MockBenchmarkExecutorService) service).clearMockState(); + } + } + + @After + public void post() throws Exception { + final BenchmarkStatusResponses responses = client().prepareBenchmarkStatus().execute().actionGet(); + assertThat("Some benchmarks are still running", responses.responses(), is(empty())); + } + + @Test + public void testStartBenchmark() throws Exception { + + // Submit benchmark and wait for completion + final BenchmarkStartRequest request = BenchmarkTestUtil.randomRequest(client(), indices, numExecutorNodes, competitionSettingsMap); + logger.info("--> Submitting benchmark - competitors [{}] iterations [{}] executors [{}]", + request.competitors().size(), request.settings().iterations(), numExecutorNodes); + final BenchmarkStartResponse response = client().startBenchmark(request).actionGet(); + + // Validate results + assertNotNull(response); + assertThat(response.benchmarkId(), equalTo(BENCHMARK_NAME)); + assertThat(response.state(), equalTo(BenchmarkStartResponse.State.COMPLETED)); + assertFalse(response.hasErrors()); + assertThat(response.competitionResults().size(), equalTo(request.competitors().size())); + + for (CompetitionResult result : response.competitionResults().values()) { + assertThat(result.nodeResults().size(), equalTo(numExecutorNodes)); + Map settingsMap = competitionSettingsMap.get(BENCHMARK_NAME); + validateCompetitionResult(result, settingsMap.get(result.competitionName()), true); + } + + // Confirm that cluster metadata went through proper state transitions + mockCoordinatorService().validateNormalLifecycle(BENCHMARK_NAME, numExecutorNodes); + } + + @Test + public void testPauseBenchmark() throws Exception { + + // Submit benchmark and wait for completion + final BenchmarkStartRequest request = BenchmarkTestUtil.randomRequest(client(), indices, numExecutorNodes, + competitionSettingsMap); + logger.info("--> Submitting benchmark - competitors [{}] iterations [{}] executors [{}]", + request.competitors().size(), request.settings().iterations(), numExecutorNodes); + + // Setup initialization and iteration barriers + final List semaphores = new ArrayList<>(request.numExecutorNodes()); + final CyclicBarrier barrier = new CyclicBarrier(request.numExecutorNodes() + 1); + control(barrier, request.competitors().get(0).name(), semaphores); + + // Start benchmark and block pending initialization + final ActionFuture future = client().startBenchmark(request); + int n = barrier.await(); + logger.info("--> Passed initialization barrier [{}] on node: [{}] (test thread)", n, clusterService().localNode().name()); + + // Check status + validateStatusRunning(BENCHMARK_NAME); + + // Pause benchmark + final BenchmarkPauseResponse pauseResponse = client().preparePauseBenchmark(BENCHMARK_NAME).execute().actionGet(); + validateStatusPaused(BENCHMARK_NAME, pauseResponse); + + // Check status + final BenchmarkStatusResponses statusResponses2 = client().prepareBenchmarkStatus(BENCHMARK_NAME).execute().actionGet(); + assertThat(statusResponses2.responses().size(), equalTo(1)); + final BenchmarkStartResponse statusResponse2 = statusResponses2.responses().get(0); + assertThat(statusResponse2.benchmarkId(), equalTo(BENCHMARK_NAME)); + assertThat(statusResponse2.state(), equalTo(BenchmarkStartResponse.State.PAUSED)); + assertFalse(statusResponse2.hasErrors()); + + // Release iteration semaphores and let executors finish + for (BenchmarkExecutorService mock : mockExecutorServices()) { + final MockBenchmarkExecutorService.MockBenchmarkExecutor executor = ((MockBenchmarkExecutorService) mock).executor(); + executor.control.controlSemaphore.release(); + logger.info("--> Released iteration semaphore: [{}] [{}] (test thread)", executor.control.controlSemaphore, clusterService().localNode().name()); + } + + // Resume benchmark + final BenchmarkResumeResponse resumeResponse = client().prepareResumeBenchmark(BENCHMARK_NAME).execute().actionGet(); + validateStatusResumed(BENCHMARK_NAME, resumeResponse); + + // Validate results + logger.info("--> Waiting for benchmark to complete"); + final BenchmarkStartResponse startResponse = future.get(); + assertNotNull(startResponse); + assertThat(startResponse.benchmarkId(), equalTo(BENCHMARK_NAME)); + assertThat(startResponse.state(), equalTo(BenchmarkStartResponse.State.COMPLETED)); + assertFalse(startResponse.hasErrors()); + assertThat(startResponse.competitionResults().size(), equalTo(request.competitors().size())); + + for (CompetitionResult result : startResponse.competitionResults().values()) { + assertThat(result.nodeResults().size(), equalTo(numExecutorNodes)); + Map settingsMap = competitionSettingsMap.get(BENCHMARK_NAME); + validateCompetitionResult(result, settingsMap.get(result.competitionName()), true); + } + + // Confirm that cluster metadata went through proper state transitions + mockCoordinatorService().validatePausedLifecycle(BENCHMARK_NAME, numExecutorNodes); + } + + @Test + public void testResumeBenchmark() throws Exception { + + // Submit benchmark and wait for completion + final BenchmarkStartRequest request = BenchmarkTestUtil.randomRequest(client(), indices, numExecutorNodes, + competitionSettingsMap); + logger.info("--> Submitting benchmark - competitors [{}] iterations [{}] executors [{}]", + request.competitors().size(), request.settings().iterations(), numExecutorNodes); + + // Setup initialization and iteration barriers + final List semaphores = new ArrayList<>(request.numExecutorNodes()); + final CyclicBarrier barrier = new CyclicBarrier(request.numExecutorNodes() + 1); + control(barrier, request.competitors().get(0).name(), semaphores); + + // Start benchmark + final ActionFuture future = client().startBenchmark(request); + int n = barrier.await(); + logger.info("--> Passed initialization barrier [{}] on node: [{}] (test thread)", n, clusterService().localNode().name()); + + // Check status + validateStatusRunning(BENCHMARK_NAME); + + // Pause benchmark + final BenchmarkPauseResponse pauseResponse = client().preparePauseBenchmark(BENCHMARK_NAME).execute().actionGet(); + validateStatusPaused(BENCHMARK_NAME, pauseResponse); + + // Check status + final BenchmarkStatusResponses statusResponses2 = client().prepareBenchmarkStatus(BENCHMARK_NAME).execute().actionGet(); + assertThat(statusResponses2.responses().size(), equalTo(1)); + final BenchmarkStartResponse statusResponse2 = statusResponses2.responses().get(0); + assertThat(statusResponse2.benchmarkId(), equalTo(BENCHMARK_NAME)); + assertThat(statusResponse2.state(), equalTo(BenchmarkStartResponse.State.PAUSED)); + assertFalse(statusResponse2.hasErrors()); + + // Resume benchmark + final BenchmarkResumeResponse resumeResponse = client().prepareResumeBenchmark(BENCHMARK_NAME).execute().actionGet(); + validateStatusResumed(BENCHMARK_NAME, resumeResponse); + + // Check status + final BenchmarkStatusResponses statusResponses3 = client().prepareBenchmarkStatus(BENCHMARK_NAME).execute().actionGet(); + assertThat(statusResponses3.responses().size(), equalTo(1)); + final BenchmarkStartResponse statusResponse3 = statusResponses3.responses().get(0); + assertThat(statusResponse3.benchmarkId(), equalTo(BENCHMARK_NAME)); + assertThat(statusResponse3.state(), equalTo(BenchmarkStartResponse.State.RUNNING)); + assertFalse(statusResponse3.hasErrors()); + + // Release iteration semaphores and let executors finish + for (BenchmarkExecutorService mock : mockExecutorServices()) { + final MockBenchmarkExecutorService.MockBenchmarkExecutor executor = ((MockBenchmarkExecutorService) mock).executor(); + executor.control.controlSemaphore.release(); + logger.info("--> Released iteration semaphore: [{}] [{}] (test thread)", executor.control.controlSemaphore, clusterService().localNode().name()); + } + + // Validate results + final BenchmarkStartResponse startResponse = future.get(); + assertNotNull(startResponse); + assertThat(startResponse.benchmarkId(), equalTo(BENCHMARK_NAME)); + assertThat(startResponse.state(), equalTo(BenchmarkStartResponse.State.COMPLETED)); + assertFalse(startResponse.hasErrors()); + assertThat(startResponse.competitionResults().size(), equalTo(request.competitors().size())); + + for (CompetitionResult result : startResponse.competitionResults().values()) { + assertThat(result.nodeResults().size(), equalTo(numExecutorNodes)); + Map settingsMap = competitionSettingsMap.get(BENCHMARK_NAME); + validateCompetitionResult(result, settingsMap.get(result.competitionName()), true); + } + + // Confirm that cluster metadata went through proper state transitions + mockCoordinatorService().validateResumedLifecycle(BENCHMARK_NAME, numExecutorNodes); + } + + @Test + public void testAbortBenchmark() throws Exception { + + // Submit benchmark and wait for completion + final BenchmarkStartRequest request = + BenchmarkTestUtil.randomRequest(client(), indices, numExecutorNodes, competitionSettingsMap); + logger.info("--> Submitting benchmark - competitors [{}] iterations [{}] executors [{}]", + request.competitors().size(), request.settings().iterations(), numExecutorNodes); + + // Setup initialization and iteration barriers + final List semaphores = new ArrayList<>(request.numExecutorNodes()); + final CyclicBarrier barrier = new CyclicBarrier(request.numExecutorNodes() + 1); + control(barrier, request.competitors().get(0).name(), semaphores); + + // Start benchmark + final ActionFuture future = client().startBenchmark(request); + int n = barrier.await(); + logger.info("--> Passed initialization barrier [{}] on node: [{}] (test thread)", n, clusterService().localNode().name()); + + // Abort benchmark + final BenchmarkAbortResponse abortResponse = client().prepareAbortBench(BENCHMARK_NAME).execute().actionGet(); + validateStatusAborted(BENCHMARK_NAME, abortResponse); + + // Release iteration semaphores and let executors finish + for (BenchmarkExecutorService mock : mockExecutorServices()) { + final MockBenchmarkExecutorService.MockBenchmarkExecutor executor = ((MockBenchmarkExecutorService) mock).executor(); + executor.control.controlSemaphore.release(); + logger.info("--> Released iteration semaphore: [{}] [{}] (test thread)", executor.control.controlSemaphore, clusterService().localNode().name()); + } + + // Validate results + final BenchmarkStartResponse startResponse = future.actionGet(); + assertNotNull(startResponse); + assertThat(startResponse.benchmarkId(), equalTo(BENCHMARK_NAME)); + assertThat(startResponse.state(), equalTo(BenchmarkStartResponse.State.ABORTED)); + assertFalse(startResponse.hasErrors()); + + // Confirm that cluster metadata went through proper state transitions + mockCoordinatorService().validateAbortedLifecycle(BENCHMARK_NAME, numExecutorNodes); + } + + @Test + public void testBenchmarkWithErrors() { + + List reqList = new ArrayList<>(); + int numQueries = scaledRandomIntBetween(1, 5); + int numErrors = scaledRandomIntBetween(1, numQueries); + final boolean containsFatal = randomBoolean(); + + if (containsFatal) { + ScriptScoreFunctionBuilder scriptFunction = scriptFunction("DOES NOT COMPILE - fails on any shard"); + SearchRequest searchRequest = searchRequest().source( + searchSource() + .query(functionScoreQuery(FilterBuilders.matchAllFilter(), scriptFunction))); + reqList.add(searchRequest); + } + + for (int i = 0; reqList.size() < numErrors; i++) { + ScriptScoreFunctionBuilder scriptFunction = scriptFunction("throw new RuntimeException();"); + SearchRequest searchRequest = searchRequest().source( + searchSource() + .query(functionScoreQuery(FilterBuilders.matchAllFilter(), scriptFunction))); + reqList.add(searchRequest); + } + + logger.info("--> run with [{}] errors ", numErrors); + for (int i = 0; reqList.size() < numQueries; i++) { + reqList.add(BenchmarkTestUtil.randomSearch(client(), indices)); + } + + Collections.shuffle(reqList, getRandom()); + + final BenchmarkStartRequest request = + BenchmarkTestUtil.randomRequest(client(),indices, numExecutorNodes, competitionSettingsMap, reqList.toArray(new SearchRequest[0])); + logger.info("--> Submitting benchmark - competitors [{}] iterations [{}]", request.competitors().size(), + request.settings().iterations()); + final BenchmarkStartResponse response = client().startBenchmark(request).actionGet(); + + assertThat(response, notNullValue()); + if (response.hasErrors() || containsFatal) { + assertThat(response.state(), equalTo(BenchmarkStartResponse.State.FAILED)); + } else { + assertThat(response.state(), equalTo(BenchmarkStartResponse.State.COMPLETED)); + for (CompetitionResult result : response.competitionResults().values()) { + assertThat(result.nodeResults().size(), equalTo(numExecutorNodes)); + Map settingsMap = competitionSettingsMap.get(BENCHMARK_NAME); + validateCompetitionResult(result, settingsMap.get(result.competitionName()), true); + } + } + assertThat(response.benchmarkId(), equalTo(BENCHMARK_NAME)); + } + + private void validateCompetitionResult(CompetitionResult result, BenchmarkSettings requestedSettings, boolean strict) { + // Validate settings + assertTrue(result.competitionName().startsWith(COMPETITOR_PREFIX)); + assertThat(result.concurrency(), equalTo(requestedSettings.concurrency())); + assertThat(result.multiplier(), equalTo(requestedSettings.multiplier())); + + // Validate node-level responses + for (CompetitionNodeResult nodeResult : result.nodeResults()) { + + assertThat(nodeResult.nodeName(), notNullValue()); + + assertThat(nodeResult.requestedIterations(), equalTo(requestedSettings.iterations())); + if (strict) { + assertThat(nodeResult.completedIterations(), equalTo(requestedSettings.iterations())); + final int expectedQueryCount = requestedSettings.multiplier() * + nodeResult.requestedIterations() * requestedSettings.searchRequests().size(); + assertThat(nodeResult.totalExecutedQueries(), equalTo(expectedQueryCount)); + assertThat(nodeResult.iterations().size(), equalTo(requestedSettings.iterations())); + } + + assertThat(nodeResult.warmUpTime(), greaterThanOrEqualTo(0L)); + + for (CompetitionIteration iteration : nodeResult.iterations()) { + // Basic sanity checks + iteration.computeStatistics(); + assertThat(iteration.totalTime(), greaterThanOrEqualTo(0L)); + assertThat(iteration.min(), greaterThanOrEqualTo(0L)); + assertThat(iteration.max(), greaterThanOrEqualTo(iteration.min())); + assertThat(iteration.mean(), greaterThanOrEqualTo((double) iteration.min())); + assertThat(iteration.mean(), lessThanOrEqualTo((double) iteration.max())); + assertThat(iteration.queriesPerSecond(), greaterThanOrEqualTo(0.0)); + assertThat(iteration.millisPerHit(), greaterThanOrEqualTo(0.0)); + validatePercentiles(iteration.percentileValues()); + } + } + + // Validate summary statistics + final CompetitionSummary summary = result.competitionSummary(); + summary.computeSummaryStatistics(); + assertThat(summary, notNullValue()); + assertThat(summary.getMin(), greaterThanOrEqualTo(0L)); + assertThat(summary.getMax(), greaterThanOrEqualTo(summary.getMin())); + assertThat(summary.getMean(), greaterThanOrEqualTo((double) summary.getMin())); + assertThat(summary.getMean(), lessThanOrEqualTo((double) summary.getMax())); + assertThat(summary.getTotalTime(), greaterThanOrEqualTo(0L)); + assertThat(summary.getQueriesPerSecond(), greaterThanOrEqualTo(0.0)); + assertThat(summary.getMillisPerHit(), greaterThanOrEqualTo(0.0)); + assertThat(summary.getAvgWarmupTime(), greaterThanOrEqualTo(0.0)); + if (strict) { + assertThat((int) summary.getTotalRequestedIterations(), equalTo(requestedSettings.iterations() * summary.nodeResults().size())); + assertThat((int) summary.getTotalCompletedIterations(), equalTo(requestedSettings.iterations() * summary.nodeResults().size())); + assertThat((int) summary.getTotalQueries(), equalTo(requestedSettings.iterations() * requestedSettings.multiplier() * + requestedSettings.searchRequests().size() * summary.nodeResults().size())); + validatePercentiles(summary.getPercentileValues()); + } + } + + private void validatePercentiles(Map percentiles) { + int i = 0; + double last = Double.NEGATIVE_INFINITY; + for (Map.Entry entry : percentiles.entrySet()) { + assertThat(entry.getKey(), equalTo(BenchmarkSettings.DEFAULT_PERCENTILES[i++])); + // This is a hedge against rounding errors. Sometimes two adjacent percentile values will + // be nearly equivalent except for some insignificant decimal places. In such cases we + // want the two values to compare as equal. + assertThat(entry.getValue(), greaterThanOrEqualTo(last - 1e-6)); + last = entry.getValue(); + } + } +} diff --git a/src/test/java/org/elasticsearch/action/bench/BenchmarkNegativeTest.java b/src/test/java/org/elasticsearch/action/benchmark/BenchmarkNegativeTest.java similarity index 78% rename from src/test/java/org/elasticsearch/action/bench/BenchmarkNegativeTest.java rename to src/test/java/org/elasticsearch/action/benchmark/BenchmarkNegativeTest.java index c42b7d13ca042..853ab8fe06ece 100644 --- a/src/test/java/org/elasticsearch/action/bench/BenchmarkNegativeTest.java +++ b/src/test/java/org/elasticsearch/action/benchmark/BenchmarkNegativeTest.java @@ -17,8 +17,10 @@ * under the License. */ -package org.elasticsearch.action.bench; +package org.elasticsearch.action.benchmark; +import org.elasticsearch.action.benchmark.exception.BenchmarkNodeMissingException; +import org.elasticsearch.action.benchmark.status.BenchmarkStatusResponses; import org.elasticsearch.common.settings.ImmutableSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.test.ElasticsearchIntegrationTest; @@ -44,14 +46,14 @@ protected Settings nodeSettings(int nodeOrdinal) { @Test(expected = BenchmarkNodeMissingException.class) public void testSubmitBenchmarkNegative() { - client().bench(BenchmarkTestUtil.randomRequest( - client(), new String[] {INDEX_NAME}, internalCluster().size(), null)).actionGet(); + client().startBenchmark(BenchmarkTestUtil.randomRequest( + client(), new String[]{INDEX_NAME}, cluster().size(), null)).actionGet(); } public void testListBenchmarkNegative() { - final BenchmarkStatusResponse response = - client().prepareBenchStatus().execute().actionGet(); - assertThat(response.benchmarkResponses().size(), equalTo(0)); + final BenchmarkStatusResponses response = + client().prepareBenchmarkStatus().execute().actionGet(); + assertThat(response.responses().size(), equalTo(0)); } @Test(expected = BenchmarkNodeMissingException.class) diff --git a/src/test/java/org/elasticsearch/action/benchmark/BenchmarkStateTransitionFailureTest.java b/src/test/java/org/elasticsearch/action/benchmark/BenchmarkStateTransitionFailureTest.java new file mode 100644 index 0000000000000..e7db7321aeec9 --- /dev/null +++ b/src/test/java/org/elasticsearch/action/benchmark/BenchmarkStateTransitionFailureTest.java @@ -0,0 +1,61 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch 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. + */ + +package org.elasticsearch.action.benchmark; + +import org.elasticsearch.action.benchmark.start.BenchmarkStartRequest; +import org.elasticsearch.cluster.metadata.BenchmarkMetaData; +import org.elasticsearch.common.settings.ImmutableSettings; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.test.ElasticsearchIntegrationTest; +import org.junit.Test; + +/** + * Test failures that happen during state transitions + */ +@ElasticsearchIntegrationTest.ClusterScope(scope = ElasticsearchIntegrationTest.Scope.SUITE) +public class BenchmarkStateTransitionFailureTest extends AbstractBenchmarkTest { + + protected synchronized Settings nodeSettings(int ordinal) { + return ImmutableSettings.builder().put("node.bench", + ordinal == 0 || randomBoolean()). + put(BenchmarkModule.BENCHMARK_COORDINATOR_SERVICE_KEY, MockBenchmarkCoordinatorService.class). + put(BenchmarkModule.BENCHMARK_EXECUTOR_SERVICE_KEY, MockBenchmarkExecutorService.class). + build(); + } + + @Test + public void testOnStateUpdateFailure() throws Exception { + + MockBenchmarkCoordinatorService coordinator = mockCoordinatorService(); + coordinator.mockBenchmarkStateManager().forceFailureOnUpdate = true; + + final BenchmarkStartRequest request = BenchmarkTestUtil.randomRequest(client(), indices, numExecutorNodes, competitionSettingsMap); + + try { + client().startBenchmark(request).actionGet(); + fail("Expected to fail"); + } catch (Throwable t) { + final BenchmarkMetaData meta = clusterService().state().metaData().custom(BenchmarkMetaData.TYPE); + if (meta != null) { + assertFalse("Failed to clear metadata", meta.contains(request.benchmarkId())); // Should leave no trace in cluster metadata + } + } + } +} diff --git a/src/test/java/org/elasticsearch/action/bench/BenchmarkTestUtil.java b/src/test/java/org/elasticsearch/action/benchmark/BenchmarkTestUtil.java similarity index 56% rename from src/test/java/org/elasticsearch/action/bench/BenchmarkTestUtil.java rename to src/test/java/org/elasticsearch/action/benchmark/BenchmarkTestUtil.java index cae64dffe1975..b8a9dbc0314fd 100644 --- a/src/test/java/org/elasticsearch/action/bench/BenchmarkTestUtil.java +++ b/src/test/java/org/elasticsearch/action/benchmark/BenchmarkTestUtil.java @@ -17,8 +17,10 @@ * under the License. */ -package org.elasticsearch.action.bench; +package org.elasticsearch.action.benchmark; +import org.elasticsearch.action.benchmark.start.BenchmarkStartRequest; +import org.elasticsearch.action.benchmark.start.BenchmarkStartRequestBuilder; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchType; @@ -26,6 +28,9 @@ import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryBuilders; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; import java.util.Map; import static org.elasticsearch.test.ElasticsearchIntegrationTest.between; @@ -38,10 +43,13 @@ */ public class BenchmarkTestUtil { - - public static final String BENCHMARK_NAME = "test_benchmark"; + public static final String BENCHMARK_NAME = "test_benchmark"; public static final String COMPETITOR_PREFIX = "competitor_"; - public static final String INDEX_TYPE = "test_type"; + public static final String INDEX_PREFIX = "test_index_"; + public static final String INDEX_TYPE = "test_type"; + + public static final int DEFAULT_LOW_INTERVAL_BOUND = 1; + public static final int DEFAULT_HIGH_INTERVAL_BOUND = 5; public static final SearchType[] searchTypes = { SearchType.DFS_QUERY_THEN_FETCH, SearchType.QUERY_THEN_FETCH, @@ -105,39 +113,74 @@ QueryBuilder getQuery() { abstract QueryBuilder getQuery(); } - public static BenchmarkRequest randomRequest(Client client, String[] indices, int numExecutorNodes, - Map competitionSettingsMap, - int lowRandomIntervalBound, int highRandomIntervalBound, SearchRequest... requests) { + public static BenchmarkStartRequest randomRequest(Client client, String[] indices, int numExecutorNodes, + Map> competitionSettingsMap, SearchRequest... requests) { - final BenchmarkRequestBuilder builder = new BenchmarkRequestBuilder(client, indices); - final BenchmarkSettings settings = randomSettings(lowRandomIntervalBound, highRandomIntervalBound); + return randomRequest(client, indices, numExecutorNodes, competitionSettingsMap, + DEFAULT_LOW_INTERVAL_BOUND, DEFAULT_HIGH_INTERVAL_BOUND, requests); + } - builder.setIterations(settings.iterations()); - builder.setConcurrency(settings.concurrency()); - builder.setMultiplier(settings.multiplier()); - builder.setSearchType(settings.searchType()); - builder.setWarmup(settings.warmup()); - builder.setNumExecutorNodes(numExecutorNodes); + public static BenchmarkStartRequest randomRequest(Client client, String[] indices, int numExecutorNodes, + Map> competitionSettingsMap, String benchmarkName, + SearchRequest... requests) { - final int numCompetitors = between(lowRandomIntervalBound, highRandomIntervalBound); - for (int i = 0; i < numCompetitors; i++) { - builder.addCompetitor(randomCompetitor(client, COMPETITOR_PREFIX + i, indices, - competitionSettingsMap, lowRandomIntervalBound, highRandomIntervalBound, requests)); - } + final List benchmarkRequests = randomRequests(1, client, indices, + numExecutorNodes, competitionSettingsMap, DEFAULT_LOW_INTERVAL_BOUND, DEFAULT_HIGH_INTERVAL_BOUND, benchmarkName, requests); + return benchmarkRequests.get(0); + } - final BenchmarkRequest request = builder.request(); - request.benchmarkName(BENCHMARK_NAME); - request.cascadeGlobalSettings(); - request.applyLateBoundSettings(indices, new String[] { INDEX_TYPE }); + public static List randomRequests(int nRequests, Client client, String[] indices, int numExecutorNodes, + Map> competitionSettingsMap, SearchRequest... requests) { - return request; + return randomRequests(nRequests, client, indices, numExecutorNodes, + competitionSettingsMap, DEFAULT_LOW_INTERVAL_BOUND, DEFAULT_HIGH_INTERVAL_BOUND, BENCHMARK_NAME, requests); } - public static BenchmarkRequest randomRequest(Client client, String[] indices, int numExecutorNodes, - Map competitionSettingsMap, SearchRequest... requests) { + public static BenchmarkStartRequest randomRequest(Client client, String[] indices, int numExecutorNodes, + Map> competitionSettingsMap, + int lowRandomIntervalBound, int highRandomIntervalBound, + SearchRequest... requests) { + final List benchmarkRequests = randomRequests(1, client, indices, + numExecutorNodes, competitionSettingsMap, lowRandomIntervalBound, highRandomIntervalBound, BENCHMARK_NAME, requests); + return benchmarkRequests.get(0); + } - return randomRequest(client, indices, numExecutorNodes, - competitionSettingsMap, 1, 3, requests); + public static List randomRequests(int nRequests, Client client, String[] indices, int numExecutorNodes, + Map> competitionSettingsMap, + int lowRandomIntervalBound, int highRandomIntervalBound, + String namePrefix, SearchRequest... requests) { + + final String prefix = namePrefix == null ? BENCHMARK_NAME : namePrefix; + + final List benchmarkRequests = new ArrayList<>(); + + for (int i = 0; i < nRequests; i++) { + + final BenchmarkStartRequestBuilder builder = new BenchmarkStartRequestBuilder(client, indices); + final BenchmarkSettings settings = randomSettings(lowRandomIntervalBound, highRandomIntervalBound); + + builder.setIterations(settings.iterations()); + builder.setConcurrency(settings.concurrency()); + builder.setMultiplier(settings.multiplier()); + builder.setSearchType(settings.searchType()); + builder.setWarmup(settings.warmup()); + builder.setNumExecutorNodes(numExecutorNodes); + + final String benchmarkId = nRequests == 1 ? prefix : prefix + "_" + i; + final int numCompetitors = between(lowRandomIntervalBound, highRandomIntervalBound); + for (int j = 0; j < numCompetitors; j++) { + builder.addCompetitor(randomCompetitor(benchmarkId, client, COMPETITOR_PREFIX + j, indices, + competitionSettingsMap, lowRandomIntervalBound, highRandomIntervalBound, requests)); + } + + final BenchmarkStartRequest request = builder.request(); + request.benchmarkId(benchmarkId); + request.cascadeGlobalSettings(); + request.applyLateBoundSettings(indices, new String[]{INDEX_TYPE}); + + benchmarkRequests.add(request); + } + return benchmarkRequests; } public static SearchRequest randomSearch(Client client, String[] indices) { @@ -149,8 +192,8 @@ public static SearchRequest randomSearch(Client client, String[] indices) { return builder.request(); } - public static BenchmarkCompetitor randomCompetitor(Client client, String name, String[] indices, - Map competitionSettingsMap, + public static BenchmarkCompetitor randomCompetitor(String benchmarkId, Client client, String name, String[] indices, + Map> competitionSettingsMap, int lowRandomIntervalBound, int highRandomIntervalBound, SearchRequest... requests) { final BenchmarkCompetitorBuilder builder = new BenchmarkCompetitorBuilder(); @@ -178,7 +221,12 @@ public static BenchmarkCompetitor randomCompetitor(Client client, String name, S } if (competitionSettingsMap != null) { - competitionSettingsMap.put(name, settings); + Map map = competitionSettingsMap.get(benchmarkId); + if (map == null) { + map = new HashMap<>(); + competitionSettingsMap.put(benchmarkId, map); + } + map.put(name, settings); } return builder.build(); diff --git a/src/test/java/org/elasticsearch/action/benchmark/MockBenchmarkCoordinatorService.java b/src/test/java/org/elasticsearch/action/benchmark/MockBenchmarkCoordinatorService.java new file mode 100644 index 0000000000000..479cadb2aedfe --- /dev/null +++ b/src/test/java/org/elasticsearch/action/benchmark/MockBenchmarkCoordinatorService.java @@ -0,0 +1,222 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch 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. + */ + +package org.elasticsearch.action.benchmark; + +import org.elasticsearch.cluster.*; +import org.elasticsearch.cluster.metadata.BenchmarkMetaData; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportService; + +import java.util.HashMap; +import java.util.Map; + +import static org.hamcrest.Matchers.equalTo; +import static org.junit.Assert.*; + +/** + * Mock benchmark service for testing. + */ +public class MockBenchmarkCoordinatorService extends BenchmarkCoordinatorService { + + private final MockBenchmarkStateManager mockStateManager; + private final Map lifecycles = new HashMap<>(); + + @Inject + public MockBenchmarkCoordinatorService(Settings settings, ClusterService clusterService, ThreadPool threadPool, + TransportService transportService, MockBenchmarkStateManager manager) { + super(settings, clusterService, threadPool, transportService, manager); + this.mockStateManager = manager; + } + + @Override + public void clusterChanged(ClusterChangedEvent event) { + super.clusterChanged(event); + + final BenchmarkMetaData meta = event.state().metaData().custom(BenchmarkMetaData.TYPE); + + if (event.metaDataChanged() && meta != null && meta.entries().size() > 0 && isOnMasterNode()) { + observe(event); + } + } + + private void observe(final ClusterChangedEvent event) { + + final BenchmarkMetaData meta = event.state().metaData().custom(BenchmarkMetaData.TYPE); + final BenchmarkMetaData prev = event.previousState().metaData().custom(BenchmarkMetaData.TYPE); + + logger.info("--> Observing event: [{}]", event.source()); + + for (final BenchmarkMetaData.Entry entry : BenchmarkMetaData.addedOrChanged(prev, meta)) { + + if (lifecycles.get(entry.benchmarkId()) == null) { + lifecycles.put(entry.benchmarkId(), new Lifecycle()); + } + + switch (entry.state()) { + case INITIALIZING: + lifecycles.get(entry.benchmarkId()).didInitialize = true; + lifecycles.get(entry.benchmarkId()).initialMetaDataEntry = entry; + break; + case RUNNING: + lifecycles.get(entry.benchmarkId()).didStartRunning = true; + break; + case COMPLETED: + lifecycles.get(entry.benchmarkId()).didComplete = true; + break; + case PAUSED: + lifecycles.get(entry.benchmarkId()).didPause = true; + break; + case RESUMING: + lifecycles.get(entry.benchmarkId()).didResume = true; + break; + case FAILED: + lifecycles.get(entry.benchmarkId()).didFail = true; + break; + case ABORTED: + lifecycles.get(entry.benchmarkId()).didAbort = true; + break; + default: + lifecycles.get(entry.benchmarkId()).didObserveUnknownState = true; + break; + } + } + } + + public boolean isOnMasterNode() { + final String masterNodeId = clusterService.state().nodes().getMasterNode().id(); + final String thisNodeId = clusterService.localNode().id(); + return masterNodeId.equals(thisNodeId); + } + + public void clearMockState() { + lifecycles.clear(); + mockStateManager.forceFailureOnUpdate = false; + } + + public static class Lifecycle { + boolean didInitialize = false; + boolean didStartRunning = false; + boolean didComplete = false; + boolean didPause = false; + boolean didResume = false; + boolean didFail = false; + boolean didAbort = false; + boolean didObserveUnknownState = false; + + BenchmarkMetaData.Entry initialMetaDataEntry = null; + } + + public void validateNormalLifecycle(final String benchmarkId, final int numExecutors) { + + final Lifecycle lifecycle = lifecycles.get(benchmarkId); + + assertNotNull(lifecycle); + assertTrue(lifecycle.didInitialize); + assertTrue(lifecycle.didStartRunning); + assertTrue(lifecycle.didComplete); + assertFalse(lifecycle.didPause); + assertFalse(lifecycle.didFail); + assertFalse(lifecycle.didAbort); + assertFalse(lifecycle.didObserveUnknownState); + + validateInitialState(benchmarkId, numExecutors, lifecycle.initialMetaDataEntry); + validateClearedMetaData(benchmarkId); + } + + public void validatePausedLifecycle(final String benchmarkId, final int numExecutors) { + + final Lifecycle lifecycle = lifecycles.get(benchmarkId); + + assertNotNull(lifecycle); + assertTrue(lifecycle.didInitialize); + assertTrue(lifecycle.didStartRunning); + assertTrue(lifecycle.didComplete); + assertTrue(lifecycle.didPause); + assertTrue(lifecycle.didResume); + assertFalse(lifecycle.didFail); + assertFalse(lifecycle.didAbort); + assertFalse(lifecycle.didObserveUnknownState); + + validateInitialState(benchmarkId, numExecutors, lifecycle.initialMetaDataEntry); + validateClearedMetaData(benchmarkId); + } + + public void validateResumedLifecycle(final String benchmarkId, final int numExecutors) { + + final Lifecycle lifecycle = lifecycles.get(benchmarkId); + + assertNotNull(lifecycle); + assertTrue(lifecycle.didInitialize); + assertTrue(lifecycle.didStartRunning); + assertTrue(lifecycle.didComplete); + assertTrue(lifecycle.didPause); + assertTrue(lifecycle.didResume); + assertFalse(lifecycle.didFail); + assertFalse(lifecycle.didAbort); + assertFalse(lifecycle.didObserveUnknownState); + + validateInitialState(benchmarkId, numExecutors, lifecycle.initialMetaDataEntry); + validateClearedMetaData(benchmarkId); + } + + public void validateAbortedLifecycle(final String benchmarkId, final int numExecutors) { + + final Lifecycle lifecycle = lifecycles.get(benchmarkId); + + assertNotNull(lifecycle); + assertTrue(lifecycle.didInitialize); + assertTrue(lifecycle.didStartRunning); + assertTrue(lifecycle.didComplete); + assertFalse(lifecycle.didPause); + assertFalse(lifecycle.didResume); + assertFalse(lifecycle.didFail); + assertTrue(lifecycle.didAbort); + assertFalse(lifecycle.didObserveUnknownState); + + validateInitialState(benchmarkId, numExecutors, lifecycle.initialMetaDataEntry); + validateClearedMetaData(benchmarkId); + } + + private void validateInitialState(final String benchmarkId, final int numExecutors, final BenchmarkMetaData.Entry entry) { + + assertNotNull(entry); + assertThat(entry.benchmarkId(), equalTo(benchmarkId)); + assertThat(entry.nodeStateMap().size(), equalTo(numExecutors)); + for (final BenchmarkMetaData.Entry.NodeState ns : entry.nodeStateMap().values()) { + assertThat(ns, equalTo(BenchmarkMetaData.Entry.NodeState.READY)); + } + } + + private void validateClearedMetaData(String benchmarkId) { + + final BenchmarkMetaData meta = clusterService.state().metaData().custom(BenchmarkMetaData.TYPE); + assertNotNull(meta); + + for (BenchmarkMetaData.Entry entry : meta.entries()) { + assertFalse(benchmarkId.equals(entry.benchmarkId())); + } + } + + public MockBenchmarkStateManager mockBenchmarkStateManager() { + return mockStateManager; + } +} diff --git a/src/test/java/org/elasticsearch/action/benchmark/MockBenchmarkExecutorService.java b/src/test/java/org/elasticsearch/action/benchmark/MockBenchmarkExecutorService.java new file mode 100644 index 0000000000000..6dadc2948148f --- /dev/null +++ b/src/test/java/org/elasticsearch/action/benchmark/MockBenchmarkExecutorService.java @@ -0,0 +1,154 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch 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. + */ + +package org.elasticsearch.action.benchmark; + +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.action.benchmark.competition.CompetitionIteration; +import org.elasticsearch.action.benchmark.start.BenchmarkStartRequest; +import org.elasticsearch.action.benchmark.start.BenchmarkStartResponse; +import org.elasticsearch.action.benchmark.status.BenchmarkStatusNodeActionResponse; +import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.client.Client; +import org.elasticsearch.cluster.ClusterService; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.logging.ESLogger; +import org.elasticsearch.common.logging.Loggers; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportService; + +import java.util.List; +import java.util.concurrent.BrokenBarrierException; +import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.Semaphore; + +/** + * Mock benchmark executor for testing + */ +public final class MockBenchmarkExecutorService extends BenchmarkExecutorService { + + private static final ESLogger logger = Loggers.getLogger(MockBenchmarkExecutorService.class.getName()); + + @Inject + public MockBenchmarkExecutorService(Settings settings, ClusterService clusterService, + ThreadPool threadPool, Client client, + TransportService transportService) { + super(settings, clusterService, threadPool, + transportService, new MockBenchmarkExecutor(client, clusterService)); + } + + public MockBenchmarkExecutor executor() { + return (MockBenchmarkExecutor) executor; + } + + public void clearMockState() { + executor().clearMockState(); + } + + /** + * Hook for testing via a mock executor + */ + static final class MockBenchmarkExecutor extends BenchmarkExecutor { + + FlowControl control = null; + + public MockBenchmarkExecutor(final Client client, final ClusterService clusterService) { + super(client, clusterService); + } + + public void control(final FlowControl control) { + this.control = control; + } + + public FlowControl control() { + return control; + } + + public void clear() { + control = null; + } + + static final class FlowControl { + + final CyclicBarrier initializationBarrier; + final Semaphore controlSemaphore; + final String competition; + + FlowControl(final CyclicBarrier initializationBarrier, final String competition, final Semaphore controlSemaphore) { + + this.initializationBarrier = initializationBarrier; + this.competition = competition; + this.controlSemaphore = controlSemaphore; + } + } + + public void clearMockState() { + if (control != null) { + control.initializationBarrier.reset(); + control.controlSemaphore.release(); + control = null; + } + } + + @Override + public BenchmarkStatusNodeActionResponse start(BenchmarkStartRequest request, BenchmarkStartResponse benchmarkStartResponse, + BenchmarkExecutorService.BenchmarkSemaphores benchmarkSemaphores) + throws ElasticsearchException { + + if (control != null) { + try { + control.initializationBarrier.await(); + } catch (InterruptedException | BrokenBarrierException e) { + logger.error("--> Failed to wait on initialization barrier: {}", + e, e.getMessage()); + } + } + + return super.start(request, benchmarkStartResponse, benchmarkSemaphores); + } + + protected CompetitionIteration iterate(final BenchmarkCompetitor competitor, + final List searchRequests, + final Measurements measurements, + final StoppableSemaphore semaphore) + throws InterruptedException { + + CompetitionIteration ci = null; + + try { + if (control != null && control.competition.equals(competitor.name())) { + logger.info("--> Acquiring iteration semaphore: [{}] [{}]", + control.controlSemaphore, clusterService.localNode().name()); + control.controlSemaphore.acquire(); + } + ci = super.iterate(competitor, searchRequests, measurements, semaphore); + } + finally { + if (control != null && control.competition.equals(competitor.name())) { + control.controlSemaphore.release(); + logger.info("--> Released iteration semaphore: [{}] [{}]", + control.controlSemaphore, clusterService.localNode().name()); + } + } + + return ci; + } + } +} diff --git a/src/test/java/org/elasticsearch/action/benchmark/MockBenchmarkStateManager.java b/src/test/java/org/elasticsearch/action/benchmark/MockBenchmarkStateManager.java new file mode 100644 index 0000000000000..d65cd341f1efe --- /dev/null +++ b/src/test/java/org/elasticsearch/action/benchmark/MockBenchmarkStateManager.java @@ -0,0 +1,76 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch 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. + */ + +package org.elasticsearch.action.benchmark; + +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.cluster.ClusterService; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ProcessedClusterStateUpdateTask; +import org.elasticsearch.cluster.metadata.BenchmarkMetaData; +import org.elasticsearch.common.collect.ImmutableOpenMap; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportService; + +import java.util.HashMap; +import java.util.Map; + +/** + * Mock state manager + */ +public class MockBenchmarkStateManager extends BenchmarkStateManager { + + private final ClusterService clusterService; + boolean forceFailureOnUpdate = false; + + @Inject + public MockBenchmarkStateManager(ClusterService clusterService, ThreadPool threadPool, TransportService transportService) { + super(clusterService, threadPool, transportService); + this.clusterService = clusterService; + } + + @Override + public void update(String benchmarkId, BenchmarkMetaData.State benchmarkState, BenchmarkMetaData.Entry.NodeState nodeState, + final ImmutableOpenMap nodeLiveness, final ActionListener listener) { + + if (!forceFailureOnUpdate) { + super.update(benchmarkId, benchmarkState, nodeState, nodeLiveness, listener); + return; + } + + final String cause = "benchmark-update-state-forced-failure (" + benchmarkId + ":" + benchmarkState + ")"; + + clusterService.submitStateUpdateTask(cause, new ProcessedClusterStateUpdateTask() { + @Override + public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { } + + @Override + public ClusterState execute(ClusterState currentState) throws Exception { + throw new ElasticsearchException("Forced failure"); + } + + @Override + public void onFailure(String source, Throwable t) { + listener.onFailure(t); + } + }); + } +} diff --git a/src/test/java/org/elasticsearch/client/AbstractClientHeadersTests.java b/src/test/java/org/elasticsearch/client/AbstractClientHeadersTests.java index 9a7b842a32c12..3a48df626fef8 100644 --- a/src/test/java/org/elasticsearch/client/AbstractClientHeadersTests.java +++ b/src/test/java/org/elasticsearch/client/AbstractClientHeadersTests.java @@ -38,7 +38,12 @@ import org.elasticsearch.action.admin.indices.flush.FlushResponse; import org.elasticsearch.action.admin.indices.stats.IndicesStatsAction; import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; -import org.elasticsearch.action.bench.*; +import org.elasticsearch.action.benchmark.BenchmarkCompetitorBuilder; +import org.elasticsearch.action.benchmark.abort.BenchmarkAbortAction; +import org.elasticsearch.action.benchmark.abort.BenchmarkAbortResponse; +import org.elasticsearch.action.benchmark.start.BenchmarkStartAction; +import org.elasticsearch.action.benchmark.start.BenchmarkStartResponse; +import org.elasticsearch.action.benchmark.status.*; import org.elasticsearch.action.delete.DeleteAction; import org.elasticsearch.action.delete.DeleteResponse; import org.elasticsearch.action.get.GetAction; @@ -77,7 +82,7 @@ public abstract class AbstractClientHeadersTests extends ElasticsearchTestCase { private static final GenericAction[] ACTIONS = new GenericAction[] { // client actions GetAction.INSTANCE, SearchAction.INSTANCE, DeleteAction.INSTANCE, DeleteIndexedScriptAction.INSTANCE, - IndexAction.INSTANCE, AbortBenchmarkAction.INSTANCE, BenchmarkAction.INSTANCE, BenchmarkStatusAction.INSTANCE, + IndexAction.INSTANCE, BenchmarkAbortAction.INSTANCE, BenchmarkStartAction.INSTANCE, BenchmarkStatusAction.INSTANCE, // cluster admin actions ClusterStatsAction.INSTANCE, CreateSnapshotAction.INSTANCE, NodesShutdownAction.INSTANCE, ClusterRerouteAction.INSTANCE, @@ -115,9 +120,9 @@ public void testActions() { client.prepareDelete("idx", "type", "id").execute().addListener(new AssertingActionListener(DeleteAction.NAME)); client.prepareDeleteIndexedScript("lang", "id").execute().addListener(new AssertingActionListener(DeleteIndexedScriptAction.NAME)); client.prepareIndex("idx", "type", "id").setSource("source").execute().addListener(new AssertingActionListener(IndexAction.NAME)); - client.prepareAbortBench("bname").execute().addListener(new AssertingActionListener(AbortBenchmarkAction.NAME)); - client.prepareBench("idx").setBenchmarkId("id").addCompetitor(new BenchmarkCompetitorBuilder().setName("name")).execute().addListener(new AssertingActionListener(BenchmarkAction.NAME)); - client.prepareBenchStatus().execute().addListener(new AssertingActionListener(BenchmarkStatusAction.NAME)); + client.prepareAbortBench("bname").execute().addListener(new AssertingActionListener(BenchmarkAbortAction.NAME)); + client.prepareStartBenchmark("idx").setBenchmarkId("id").addCompetitor(new BenchmarkCompetitorBuilder().setName("name")).execute().addListener(new AssertingActionListener(BenchmarkStartAction.NAME)); + client.prepareBenchmarkStatus().execute().addListener(new AssertingActionListener(BenchmarkStatusAction.NAME)); // choosing arbitrary cluster admin actions to test client.admin().cluster().prepareClusterStats().execute().addListener(new AssertingActionListener(ClusterStatsAction.NAME)); diff --git a/src/test/java/org/elasticsearch/rest/HeadersCopyClientTests.java b/src/test/java/org/elasticsearch/rest/HeadersCopyClientTests.java index b3ec5a02408b5..beca8afe77d4d 100644 --- a/src/test/java/org/elasticsearch/rest/HeadersCopyClientTests.java +++ b/src/test/java/org/elasticsearch/rest/HeadersCopyClientTests.java @@ -208,7 +208,7 @@ public void testCopyHeadersRequestBuilder() { client.prepareIndex(), client.prepareClearScroll(), client.prepareMultiGet(), - client.prepareBenchStatus() + client.prepareBenchmarkStatus() }; for (ActionRequestBuilder requestBuilder : requestBuilders) { diff --git a/src/test/java/org/elasticsearch/transport/ActionNamesBackwardsCompatibilityTest.java b/src/test/java/org/elasticsearch/transport/ActionNamesBackwardsCompatibilityTest.java index 5b80a7982fe37..79ec317209d0b 100644 --- a/src/test/java/org/elasticsearch/transport/ActionNamesBackwardsCompatibilityTest.java +++ b/src/test/java/org/elasticsearch/transport/ActionNamesBackwardsCompatibilityTest.java @@ -21,10 +21,11 @@ import com.google.common.collect.ImmutableMap; import org.elasticsearch.Version; -import org.elasticsearch.action.bench.AbortBenchmarkAction; -import org.elasticsearch.action.bench.BenchmarkAction; -import org.elasticsearch.action.bench.BenchmarkService; -import org.elasticsearch.action.bench.BenchmarkStatusAction; +import org.elasticsearch.action.benchmark.abort.BenchmarkAbortAction; +import org.elasticsearch.action.benchmark.status.BenchmarkStatusAction; +import org.elasticsearch.action.benchmark.pause.BenchmarkPauseAction; +import org.elasticsearch.action.benchmark.resume.BenchmarkResumeAction; +import org.elasticsearch.action.benchmark.start.BenchmarkStartAction; import org.elasticsearch.action.exists.ExistsAction; import org.elasticsearch.action.indexedscripts.delete.DeleteIndexedScriptAction; import org.elasticsearch.action.indexedscripts.get.GetIndexedScriptAction; @@ -117,12 +118,11 @@ private static boolean isActionNotFoundExpected(Version version, String action) private static final Map actionsVersions = new HashMap<>(); static { - actionsVersions.put(BenchmarkService.STATUS_ACTION_NAME, Version.V_1_4_0); - actionsVersions.put(BenchmarkService.START_ACTION_NAME, Version.V_1_4_0); - actionsVersions.put(BenchmarkService.ABORT_ACTION_NAME, Version.V_1_4_0); - actionsVersions.put(BenchmarkAction.NAME, Version.V_1_4_0); actionsVersions.put(BenchmarkStatusAction.NAME, Version.V_1_4_0); - actionsVersions.put(AbortBenchmarkAction.NAME, Version.V_1_4_0); + actionsVersions.put(BenchmarkAbortAction.NAME, Version.V_1_4_0); + actionsVersions.put(BenchmarkPauseAction.NAME, Version.V_1_4_0); + actionsVersions.put(BenchmarkResumeAction.NAME, Version.V_1_4_0); + actionsVersions.put(BenchmarkStartAction.NAME, Version.V_1_4_0); actionsVersions.put(ExistsAction.NAME, Version.V_1_4_0); actionsVersions.put(ExistsAction.NAME + "[s]", Version.V_1_4_0); diff --git a/src/test/java/org/elasticsearch/transport/ActionNamesTests.java b/src/test/java/org/elasticsearch/transport/ActionNamesTests.java index 1ad687bfe0203..769da78092f82 100644 --- a/src/test/java/org/elasticsearch/transport/ActionNamesTests.java +++ b/src/test/java/org/elasticsearch/transport/ActionNamesTests.java @@ -20,10 +20,14 @@ package org.elasticsearch.transport; import org.elasticsearch.Version; -import org.elasticsearch.action.bench.AbortBenchmarkAction; -import org.elasticsearch.action.bench.BenchmarkAction; -import org.elasticsearch.action.bench.BenchmarkService; -import org.elasticsearch.action.bench.BenchmarkStatusAction; +import org.elasticsearch.action.benchmark.AbstractBenchmarkService; +import org.elasticsearch.action.benchmark.BenchmarkCoordinatorService; +import org.elasticsearch.action.benchmark.BenchmarkExecutorService; +import org.elasticsearch.action.benchmark.abort.BenchmarkAbortAction; +import org.elasticsearch.action.benchmark.pause.BenchmarkPauseAction; +import org.elasticsearch.action.benchmark.resume.BenchmarkResumeAction; +import org.elasticsearch.action.benchmark.start.BenchmarkStartAction; +import org.elasticsearch.action.benchmark.status.BenchmarkStatusAction; import org.elasticsearch.action.exists.ExistsAction; import org.elasticsearch.test.ElasticsearchIntegrationTest; import org.junit.Test; @@ -123,12 +127,14 @@ public void testIncomingAction() { static { //add here new actions that don't need a mapping as they weren't available prior to 1.4 - post_1_4_actions.add(BenchmarkService.STATUS_ACTION_NAME); - post_1_4_actions.add(BenchmarkService.START_ACTION_NAME); - post_1_4_actions.add(BenchmarkService.ABORT_ACTION_NAME); - post_1_4_actions.add(BenchmarkAction.NAME); post_1_4_actions.add(BenchmarkStatusAction.NAME); - post_1_4_actions.add(AbortBenchmarkAction.NAME); + post_1_4_actions.add(BenchmarkStartAction.NAME); + post_1_4_actions.add(BenchmarkPauseAction.NAME); + post_1_4_actions.add(BenchmarkResumeAction.NAME); + post_1_4_actions.add(BenchmarkAbortAction.NAME); + post_1_4_actions.add(BenchmarkCoordinatorService.BenchmarkDefinitionRequestHandler.ACTION); + post_1_4_actions.add(AbstractBenchmarkService.NodeStateUpdateRequestHandler.ACTION); + post_1_4_actions.add(BenchmarkExecutorService.BenchmarkStatusRequestHandler.ACTION); post_1_4_actions.add(ExistsAction.NAME); post_1_4_actions.add(ExistsAction.NAME + "[s]"); }