浏览代码

Add client benchmark

With this commit we add a benchmark for the new REST client and the
existing transport client.

Closes #19281
Daniel Mitterdorfer 9 年之前
父节点
当前提交
c33f85bc37
共有 17 个文件被更改,包括 1057 次插入0 次删除
  1. 35 0
      client/benchmark/README.md
  2. 48 0
      client/benchmark/build.gradle
  3. 88 0
      client/benchmark/src/main/java/org/elasticsearch/client/benchmark/AbstractBenchmark.java
  4. 84 0
      client/benchmark/src/main/java/org/elasticsearch/client/benchmark/BenchmarkRunner.java
  5. 29 0
      client/benchmark/src/main/java/org/elasticsearch/client/benchmark/BenchmarkTask.java
  6. 45 0
      client/benchmark/src/main/java/org/elasticsearch/client/benchmark/metrics/Metrics.java
  7. 80 0
      client/benchmark/src/main/java/org/elasticsearch/client/benchmark/metrics/MetricsCalculator.java
  8. 54 0
      client/benchmark/src/main/java/org/elasticsearch/client/benchmark/metrics/Sample.java
  9. 51 0
      client/benchmark/src/main/java/org/elasticsearch/client/benchmark/metrics/SampleRecorder.java
  10. 174 0
      client/benchmark/src/main/java/org/elasticsearch/client/benchmark/ops/bulk/BulkBenchmarkTask.java
  11. 25 0
      client/benchmark/src/main/java/org/elasticsearch/client/benchmark/ops/bulk/BulkRequestExecutor.java
  12. 86 0
      client/benchmark/src/main/java/org/elasticsearch/client/benchmark/ops/search/SearchBenchmarkTask.java
  13. 23 0
      client/benchmark/src/main/java/org/elasticsearch/client/benchmark/ops/search/SearchRequestExecutor.java
  14. 108 0
      client/benchmark/src/main/java/org/elasticsearch/client/benchmark/rest/RestClientBenchmark.java
  15. 117 0
      client/benchmark/src/main/java/org/elasticsearch/client/benchmark/transport/TransportClientBenchmark.java
  16. 9 0
      client/benchmark/src/main/resources/log4j.properties
  17. 1 0
      settings.gradle

+ 35 - 0
client/benchmark/README.md

@@ -0,0 +1,35 @@
+Steps to execute the benchmark:
+
+1. Start Elasticsearch on the target host (ideally *not* on the same machine)
+2. Create an empty index with the mapping you want to benchmark
+3. Start either the RestClientBenchmark class or the TransportClientBenchmark
+4. Delete the index
+5. Repeat steps 2. - 4. for multiple iterations. The first iterations are intended as warmup for Elasticsearch itself. Always start the same benchmark in step 3!
+4. After the benchmark: Shutdown Elasticsearch and delete the data directory
+
+Repeat all steps above for the other benchmark candidate.
+
+Example benchmark:
+
+* Download benchmark data from http://benchmarks.elastic.co/corpora/geonames/documents.json.bz2 and decompress
+* Use the mapping file https://github.com/elastic/rally-tracks/blob/master/geonames/mappings.json to create the index
+
+Example command line parameter list:
+
+```
+192.168.2.2 /home/your_user_name/.rally/benchmarks/data/geonames/documents.json geonames type 8647880 5000 "{ \"query\": { \"match_phrase\": { \"name\": \"Sankt Georgen\" } } }\""
+```
+
+The parameters are in order:
+
+* Benchmark target host IP (the host where Elasticsearch is running)
+* full path to the file that should be bulk indexed
+* name of the index
+* name of the (sole) type in the index 
+* number of documents in the file
+* bulk size
+* a search request body (remember to escape double quotes). The `TransportClientBenchmark` uses `QueryBuilders.wrapperQuery()` internally which automatically adds a root key `query`, so it must not be present in the command line parameter. 
+ 
+You should also define a few GC-related settings `-Xms4096M -Xmx4096M  -XX:+UseConcMarkSweepGC -verbose:gc -XX:+PrintGCDetails` and keep an eye on GC activity. You can also define `-XX:+PrintCompilation` to see JIT activity.
+
+

+ 48 - 0
client/benchmark/build.gradle

@@ -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.
+ */
+
+apply plugin: 'elasticsearch.build'
+
+group = 'org.elasticsearch.client'
+
+// never try to invoke tests on the benchmark project - there aren't any
+check.dependsOn.remove(test)
+// explicitly override the test task too in case somebody invokes 'gradle test' so it won't trip
+task test(type: Test, overwrite: true)
+
+dependencies {
+  compile 'org.apache.commons:commons-math3:3.2'
+
+  compile("org.elasticsearch.client:rest:${version}")
+  // for transport client
+  compile("org.elasticsearch:elasticsearch:${version}")
+  compile("org.elasticsearch.client:transport:${version}")
+  compile project(path: ':modules:transport-netty3', configuration: 'runtime')
+  compile project(path: ':modules:transport-netty4', configuration: 'runtime')
+  compile project(path: ':modules:reindex', configuration: 'runtime')
+  compile project(path: ':modules:lang-mustache', configuration: 'runtime')
+  compile project(path: ':modules:percolator', configuration: 'runtime')
+}
+
+// No licenses for our benchmark deps (we don't ship benchmarks)
+dependencyLicenses.enabled = false
+
+extraArchive {
+  javadoc = false
+}

+ 88 - 0
client/benchmark/src/main/java/org/elasticsearch/client/benchmark/AbstractBenchmark.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.client.benchmark;
+
+import org.elasticsearch.client.benchmark.ops.bulk.BulkBenchmarkTask;
+import org.elasticsearch.client.benchmark.ops.bulk.BulkRequestExecutor;
+import org.elasticsearch.client.benchmark.ops.search.SearchBenchmarkTask;
+import org.elasticsearch.client.benchmark.ops.search.SearchRequestExecutor;
+import org.elasticsearch.common.SuppressForbidden;
+
+import java.io.Closeable;
+
+public abstract class AbstractBenchmark<T extends Closeable> {
+    private static final int SEARCH_BENCHMARK_ITERATIONS = 10_000;
+
+    protected abstract T client(String benchmarkTargetHost) throws Exception;
+
+    protected abstract BulkRequestExecutor bulkRequestExecutor(T client, String indexName, String typeName);
+
+    protected abstract SearchRequestExecutor searchRequestExecutor(T client, String indexName);
+
+    @SuppressForbidden(reason = "system out is ok for a command line tool")
+    public final void run(String[] args) throws Exception {
+        if (args.length < 6) {
+            System.err.println(
+                "usage: benchmarkTargetHostIp indexFilePath indexName typeName numberOfDocuments bulkSize [search request body]");
+            System.exit(1);
+        }
+        String benchmarkTargetHost = args[0];
+        String indexFilePath = args[1];
+        String indexName = args[2];
+        String typeName = args[3];
+        int totalDocs = Integer.valueOf(args[4]);
+        int bulkSize = Integer.valueOf(args[5]);
+
+        int totalIterationCount = (int) Math.floor(totalDocs / bulkSize);
+        // consider 40% of all iterations as warmup iterations
+        int warmupIterations = (int) (0.4d * totalIterationCount);
+        int iterations = totalIterationCount - warmupIterations;
+        String searchBody = (args.length == 7) ? args[6] : null;
+
+        T client = client(benchmarkTargetHost);
+
+        BenchmarkRunner benchmark = new BenchmarkRunner(warmupIterations, iterations,
+            new BulkBenchmarkTask(
+                bulkRequestExecutor(client, indexName, typeName), indexFilePath, warmupIterations + iterations, bulkSize));
+
+        try {
+            benchmark.run();
+            if (searchBody != null) {
+                for (int run = 1; run <= 5; run++) {
+                    System.out.println("=============");
+                    System.out.println(" Trial run " + run);
+                    System.out.println("=============");
+
+                    for (int throughput = 100; throughput <= 100_000; throughput *= 10) {
+                        //request a GC between trials to reduce the likelihood of a GC occurring in the middle of a trial.
+                        System.gc();
+
+                        BenchmarkRunner searchBenchmark = new BenchmarkRunner(SEARCH_BENCHMARK_ITERATIONS, SEARCH_BENCHMARK_ITERATIONS,
+                            new SearchBenchmarkTask(
+                                searchRequestExecutor(client, indexName), searchBody, 2 * SEARCH_BENCHMARK_ITERATIONS, throughput));
+                        System.out.printf("Target throughput = %d ops / s%n", throughput);
+                        searchBenchmark.run();
+                    }
+                }
+            }
+        } finally {
+            client.close();
+        }
+    }
+}

+ 84 - 0
client/benchmark/src/main/java/org/elasticsearch/client/benchmark/BenchmarkRunner.java

@@ -0,0 +1,84 @@
+/*
+ * 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.client.benchmark;
+
+import org.elasticsearch.client.benchmark.metrics.Metrics;
+import org.elasticsearch.client.benchmark.metrics.MetricsCalculator;
+import org.elasticsearch.client.benchmark.metrics.Sample;
+import org.elasticsearch.client.benchmark.metrics.SampleRecorder;
+import org.elasticsearch.common.SuppressForbidden;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Locale;
+
+public final class BenchmarkRunner {
+    private final int warmupIterations;
+    private final int iterations;
+    private final BenchmarkTask task;
+
+    public BenchmarkRunner(int warmupIterations, int iterations, BenchmarkTask task) {
+        this.warmupIterations = warmupIterations;
+        this.iterations = iterations;
+        this.task = task;
+    }
+
+    @SuppressForbidden(reason = "system out is ok for a command line tool")
+    public void run() throws Exception {
+        SampleRecorder recorder = new SampleRecorder(warmupIterations, iterations);
+        System.out.printf("Running %s with %d warmup iterations and %d iterations.%n",
+            task.getClass().getSimpleName(), warmupIterations, iterations);
+
+        try {
+            task.setUp(recorder);
+            task.run();
+            task.tearDown();
+        } catch (InterruptedException ex) {
+            Thread.currentThread().interrupt();
+            return;
+        }
+
+        List<Sample> samples = recorder.getSamples();
+        final List<Metrics> summaryMetrics = MetricsCalculator.calculate(samples);
+
+        if (summaryMetrics.isEmpty()) {
+            System.out.println("No results.");
+        }
+
+        for (Metrics metrics : summaryMetrics) {
+            System.out.printf(Locale.ROOT, "Operation: %s%n", metrics.operation);
+            String stats = String.format(Locale.ROOT,
+                "Throughput = %f ops/s, p90 = %f ms, p95 = %f ms, p99 = %f ms, p99.9 = %f ms, p99.99 = %f ms",
+                metrics.throughput,
+                metrics.serviceTimeP90, metrics.serviceTimeP95,
+                metrics.serviceTimeP99, metrics.serviceTimeP999,
+                metrics.serviceTimeP9999);
+            System.out.println(repeat(stats.length(), '-'));
+            System.out.println(stats);
+            System.out.printf("success count = %d, error count = %d%n", metrics.successCount, metrics.errorCount);
+            System.out.println(repeat(stats.length(), '-'));
+        }
+    }
+
+    private String repeat(int times, char character) {
+        char[] characters = new char[times];
+        Arrays.fill(characters, character);
+        return new String(characters);
+    }
+}

+ 29 - 0
client/benchmark/src/main/java/org/elasticsearch/client/benchmark/BenchmarkTask.java

@@ -0,0 +1,29 @@
+/*
+ * 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.client.benchmark;
+
+import org.elasticsearch.client.benchmark.metrics.SampleRecorder;
+
+public interface BenchmarkTask {
+    void setUp(SampleRecorder sampleRecorder) throws Exception;
+
+    void run() throws Exception;
+
+    void tearDown() throws Exception;
+}

+ 45 - 0
client/benchmark/src/main/java/org/elasticsearch/client/benchmark/metrics/Metrics.java

@@ -0,0 +1,45 @@
+/*
+ * 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.client.benchmark.metrics;
+
+public final class Metrics {
+    public final String operation;
+    public final long successCount;
+    public final long errorCount;
+    public final double throughput;
+    public final double serviceTimeP90;
+    public final double serviceTimeP95;
+    public final double serviceTimeP99;
+    public final double serviceTimeP999;
+    public final double serviceTimeP9999;
+
+    public Metrics(String operation, long successCount, long errorCount, double throughput,
+                   double serviceTimeP90, double serviceTimeP95, double serviceTimeP99,
+                   double serviceTimeP999, double serviceTimeP9999) {
+        this.operation = operation;
+        this.successCount = successCount;
+        this.errorCount = errorCount;
+        this.throughput = throughput;
+        this.serviceTimeP90 = serviceTimeP90;
+        this.serviceTimeP95 = serviceTimeP95;
+        this.serviceTimeP99 = serviceTimeP99;
+        this.serviceTimeP999 = serviceTimeP999;
+        this.serviceTimeP9999 = serviceTimeP9999;
+    }
+}

+ 80 - 0
client/benchmark/src/main/java/org/elasticsearch/client/benchmark/metrics/MetricsCalculator.java

@@ -0,0 +1,80 @@
+/*
+ * 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.client.benchmark.metrics;
+
+import org.apache.commons.math3.stat.StatUtils;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+public final class MetricsCalculator {
+    public static List<Metrics> calculate(Collection<Sample> samples) {
+        Map<String, List<Sample>> samplesPerOperation = groupByOperation(samples);
+        return calculateMetricsPerOperation(samplesPerOperation);
+    }
+
+    private static Map<String, List<Sample>> groupByOperation(Collection<Sample> samples) {
+        Map<String, List<Sample>> samplesPerOperation = new HashMap<>();
+
+        for (Sample sample : samples) {
+            if (!samplesPerOperation.containsKey(sample.getOperation())) {
+                samplesPerOperation.put(sample.getOperation(), new ArrayList<>());
+            }
+            samplesPerOperation.get(sample.getOperation()).add(sample);
+        }
+        return samplesPerOperation;
+    }
+
+    private static List<Metrics> calculateMetricsPerOperation(Map<String, List<Sample>> samplesPerOperation) {
+        List<Metrics> metrics = new ArrayList<>();
+        for (Map.Entry<String, List<Sample>> operationAndMetrics : samplesPerOperation.entrySet()) {
+            List<Sample> samples = operationAndMetrics.getValue();
+            double[] serviceTimes = new double[samples.size()];
+            int it = 0;
+            long firstStart = Long.MAX_VALUE;
+            long latestEnd = Long.MIN_VALUE;
+            for (Sample sample : samples) {
+                firstStart = Math.min(sample.getStartTimestamp(), firstStart);
+                latestEnd = Math.max(sample.getStopTimestamp(), latestEnd);
+                serviceTimes[it++] = sample.getServiceTime();
+            }
+
+            metrics.add(new Metrics(operationAndMetrics.getKey(),
+                samples.stream().filter((r) -> r.isSuccess()).count(),
+                samples.stream().filter((r) -> !r.isSuccess()).count(),
+                // throughput calculation is based on the total (Wall clock) time it took to generate all samples
+                calculateThroughput(samples.size(), latestEnd - firstStart),
+                // convert ns -> ms without losing precision
+                StatUtils.percentile(serviceTimes, 90.0d) / TimeUnit.MILLISECONDS.toNanos(1L),
+                StatUtils.percentile(serviceTimes, 95.0d) / TimeUnit.MILLISECONDS.toNanos(1L),
+                StatUtils.percentile(serviceTimes, 99.0d) / TimeUnit.MILLISECONDS.toNanos(1L),
+                StatUtils.percentile(serviceTimes, 99.9d) / TimeUnit.MILLISECONDS.toNanos(1L),
+                StatUtils.percentile(serviceTimes, 99.99d) / TimeUnit.MILLISECONDS.toNanos(1L)));
+        }
+        return metrics;
+    }
+
+    private static double calculateThroughput(int sampleSize, double duration) {
+        return sampleSize * (TimeUnit.SECONDS.toNanos(1L) / duration);
+    }
+}

+ 54 - 0
client/benchmark/src/main/java/org/elasticsearch/client/benchmark/metrics/Sample.java

@@ -0,0 +1,54 @@
+/*
+ * 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.client.benchmark.metrics;
+
+public final class Sample {
+    private final String operation;
+    private final long startTimestamp;
+    private final long stopTimestamp;
+    private final boolean success;
+
+    public Sample(String operation, long startTimestamp, long stopTimestamp, boolean success) {
+        this.operation = operation;
+        this.startTimestamp = startTimestamp;
+        this.stopTimestamp = stopTimestamp;
+        this.success = success;
+    }
+
+    public String getOperation() {
+        return operation;
+    }
+
+    public boolean isSuccess() {
+        return success;
+    }
+
+    public long getStartTimestamp() {
+        return startTimestamp;
+    }
+
+    public long getStopTimestamp() {
+        return stopTimestamp;
+    }
+
+    public long getServiceTime() {
+        // this is *not* latency, we're not including wait time in the queue (on purpose)
+        return stopTimestamp - startTimestamp;
+    }
+}

+ 51 - 0
client/benchmark/src/main/java/org/elasticsearch/client/benchmark/metrics/SampleRecorder.java

@@ -0,0 +1,51 @@
+/*
+ * 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.client.benchmark.metrics;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Stores measurement samples.
+ *
+ * This class is NOT threadsafe.
+ */
+public final class SampleRecorder {
+    private final int warmupIterations;
+    private final List<Sample> samples;
+    private int currentIteration;
+
+    public SampleRecorder(int warmupIterations, int iterations) {
+        this.warmupIterations = warmupIterations;
+        this.samples = new ArrayList<>(iterations);
+    }
+
+    public void addSample(Sample sample) {
+        currentIteration++;
+        // only add samples after warmup
+        if (currentIteration > warmupIterations) {
+            samples.add(sample);
+        }
+    }
+
+    public List<Sample> getSamples() {
+        return Collections.unmodifiableList(samples);
+    }
+}

+ 174 - 0
client/benchmark/src/main/java/org/elasticsearch/client/benchmark/ops/bulk/BulkBenchmarkTask.java

@@ -0,0 +1,174 @@
+/*
+ * 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.client.benchmark.ops.bulk;
+
+import org.elasticsearch.ElasticsearchException;
+import org.elasticsearch.client.benchmark.BenchmarkTask;
+import org.elasticsearch.client.benchmark.metrics.Sample;
+import org.elasticsearch.client.benchmark.metrics.SampleRecorder;
+import org.elasticsearch.common.SuppressForbidden;
+import org.elasticsearch.common.io.PathUtils;
+import org.elasticsearch.common.logging.ESLogger;
+import org.elasticsearch.common.logging.ESLoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+public class BulkBenchmarkTask implements BenchmarkTask {
+    private final BulkRequestExecutor requestExecutor;
+    private final String indexFilePath;
+    private final int totalIterations;
+    private final int bulkSize;
+    private LoadGenerator generator;
+    private ExecutorService executorService;
+
+    public BulkBenchmarkTask(BulkRequestExecutor requestExecutor, String indexFilePath, int totalIterations, int bulkSize) {
+        this.requestExecutor = requestExecutor;
+        this.indexFilePath = indexFilePath;
+        this.totalIterations = totalIterations;
+        this.bulkSize = bulkSize;
+    }
+
+    @Override
+    @SuppressForbidden(reason = "PathUtils#get is fine - we don't have environment here")
+    public void setUp(SampleRecorder sampleRecorder) {
+        BlockingQueue<List<String>> bulkQueue = new ArrayBlockingQueue<>(256);
+
+        BulkIndexer runner = new BulkIndexer(bulkQueue, totalIterations, sampleRecorder, requestExecutor);
+
+        executorService = Executors.newSingleThreadExecutor((r) -> new Thread(r, "bulk-index-runner"));
+        executorService.submit(runner);
+
+        generator = new LoadGenerator(PathUtils.get(indexFilePath), bulkQueue, bulkSize);
+    }
+
+    @Override
+    @SuppressForbidden(reason = "system out is ok for a command line tool")
+    public void run() throws Exception  {
+        generator.execute();
+        // when the generator is done, there are no more data -> shutdown client
+        executorService.shutdown();
+        //We need to wait until the queue is drained
+        final boolean finishedNormally = executorService.awaitTermination(20, TimeUnit.MINUTES);
+        if (finishedNormally == false) {
+            System.err.println("Background tasks are still running after timeout on enclosing pool. Forcing pool shutdown.");
+            executorService.shutdownNow();
+        }
+    }
+
+    @Override
+    public void tearDown() {
+        //no op
+    }
+
+    private static final class LoadGenerator {
+        private final Path bulkDataFile;
+        private final BlockingQueue<List<String>> bulkQueue;
+        private final int bulkSize;
+
+        public LoadGenerator(Path bulkDataFile, BlockingQueue<List<String>> bulkQueue, int bulkSize) {
+            this.bulkDataFile = bulkDataFile;
+            this.bulkQueue = bulkQueue;
+            this.bulkSize = bulkSize;
+        }
+
+        @SuppressForbidden(reason = "Classic I/O is fine in non-production code")
+        public void execute() {
+            try (BufferedReader reader = Files.newBufferedReader(bulkDataFile, StandardCharsets.UTF_8)) {
+                String line;
+                int bulkIndex = 0;
+                List<String> bulkData = new ArrayList<>(bulkSize);
+                while ((line = reader.readLine()) != null) {
+                    if (bulkIndex == bulkSize) {
+                        sendBulk(bulkData);
+                        // reset data structures
+                        bulkData = new ArrayList<>(bulkSize);
+                        bulkIndex = 0;
+                    }
+                    bulkData.add(line);
+                    bulkIndex++;
+                }
+                // also send the last bulk:
+                if (bulkIndex > 0) {
+                    sendBulk(bulkData);
+                }
+            } catch (IOException e) {
+                throw new ElasticsearchException(e);
+            } catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+            }
+        }
+
+        private void sendBulk(List<String> bulkData) throws InterruptedException {
+            bulkQueue.put(bulkData);
+        }
+    }
+
+
+    private static final class BulkIndexer implements Runnable {
+        private static final ESLogger logger = ESLoggerFactory.getLogger(BulkIndexer.class.getName());
+
+        private final BlockingQueue<List<String>> bulkData;
+        private final int totalIterations;
+        private final BulkRequestExecutor bulkRequestExecutor;
+        private final SampleRecorder sampleRecorder;
+
+        public BulkIndexer(BlockingQueue<List<String>> bulkData, int totalIterations, SampleRecorder sampleRecorder,
+                           BulkRequestExecutor bulkRequestExecutor) {
+            this.bulkData = bulkData;
+            this.totalIterations = totalIterations;
+            this.bulkRequestExecutor = bulkRequestExecutor;
+            this.sampleRecorder = sampleRecorder;
+        }
+
+        @Override
+        public void run() {
+            for (int iteration = 0; iteration < totalIterations; iteration++) {
+                boolean success = false;
+                List<String> currentBulk;
+                try {
+                    currentBulk = bulkData.take();
+                } catch (InterruptedException e) {
+                    Thread.currentThread().interrupt();
+                    return;
+                }
+                // Yes, this approach is prone to coordinated omission *but* we have to consider that we want to benchmark a closed system
+                // with backpressure here instead of an open system. So this is actually correct in this case.
+                long start = System.nanoTime();
+                try {
+                    success = bulkRequestExecutor.bulkIndex(currentBulk);
+                } catch (Exception ex) {
+                    logger.warn("Error while executing bulk request", ex);
+                }
+                long stop = System.nanoTime();
+                sampleRecorder.addSample(new Sample("bulk", start, stop, success));
+            }
+        }
+    }
+}

+ 25 - 0
client/benchmark/src/main/java/org/elasticsearch/client/benchmark/ops/bulk/BulkRequestExecutor.java

@@ -0,0 +1,25 @@
+/*
+ * 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.client.benchmark.ops.bulk;
+
+import java.util.List;
+
+public interface BulkRequestExecutor {
+    boolean bulkIndex(List<String> bulkData);
+}

+ 86 - 0
client/benchmark/src/main/java/org/elasticsearch/client/benchmark/ops/search/SearchBenchmarkTask.java

@@ -0,0 +1,86 @@
+/*
+ * 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.client.benchmark.ops.search;
+
+import org.elasticsearch.client.benchmark.BenchmarkTask;
+import org.elasticsearch.client.benchmark.metrics.Sample;
+import org.elasticsearch.client.benchmark.metrics.SampleRecorder;
+
+import java.util.concurrent.TimeUnit;
+
+public class SearchBenchmarkTask implements BenchmarkTask {
+    private static final long MICROS_PER_SEC = TimeUnit.SECONDS.toMicros(1L);
+    private static final long NANOS_PER_MICRO = TimeUnit.MICROSECONDS.toNanos(1L);
+
+    private final SearchRequestExecutor searchRequestExecutor;
+    private final String searchRequestBody;
+    private final int iterations;
+    private final int targetThroughput;
+
+    private SampleRecorder sampleRecorder;
+
+    public SearchBenchmarkTask(SearchRequestExecutor searchRequestExecutor, String body, int iterations, int targetThroughput) {
+        this.searchRequestExecutor = searchRequestExecutor;
+        this.searchRequestBody = body;
+        this.iterations = iterations;
+        this.targetThroughput = targetThroughput;
+    }
+
+    @Override
+    public void setUp(SampleRecorder sampleRecorder) throws Exception {
+        this.sampleRecorder = sampleRecorder;
+    }
+
+    @Override
+    public void run() throws Exception {
+        for (int iteration = 0; iteration < this.iterations; iteration++) {
+            final long start = System.nanoTime();
+            boolean success = searchRequestExecutor.search(searchRequestBody);
+            final long stop = System.nanoTime();
+            sampleRecorder.addSample(new Sample("search", start, stop, success));
+
+            int waitTime = (int) Math.floor(MICROS_PER_SEC / targetThroughput - (stop - start) / NANOS_PER_MICRO);
+            if (waitTime > 0) {
+                // Thread.sleep() time is not very accurate (it's most of the time around 1 - 2 ms off)
+                // so we rather busy spin for the last few microseconds. Still not entirely accurate but way closer
+                waitMicros(waitTime);
+            }
+        }
+    }
+
+    private void waitMicros(int waitTime) throws InterruptedException {
+        int millis = waitTime / 1000;
+        int micros = waitTime % 1000;
+        if (millis > 0) {
+            Thread.sleep(millis);
+        }
+        // busy spin for the rest of the time
+        if (micros > 0) {
+            long end = System.nanoTime() + 1000L * micros;
+            while (end > System.nanoTime()) {
+                // busy spin
+            }
+        }
+    }
+
+    @Override
+    public void tearDown() throws Exception {
+        // no op
+    }
+}

+ 23 - 0
client/benchmark/src/main/java/org/elasticsearch/client/benchmark/ops/search/SearchRequestExecutor.java

@@ -0,0 +1,23 @@
+/*
+ * 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.client.benchmark.ops.search;
+
+public interface SearchRequestExecutor {
+    boolean search(String source);
+}

+ 108 - 0
client/benchmark/src/main/java/org/elasticsearch/client/benchmark/rest/RestClientBenchmark.java

@@ -0,0 +1,108 @@
+/*
+ * 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.client.benchmark.rest;
+
+import org.apache.http.HttpEntity;
+import org.apache.http.HttpHost;
+import org.apache.http.HttpStatus;
+import org.apache.http.entity.ContentType;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.nio.entity.NStringEntity;
+import org.elasticsearch.ElasticsearchException;
+import org.elasticsearch.client.Response;
+import org.elasticsearch.client.RestClient;
+import org.elasticsearch.client.benchmark.AbstractBenchmark;
+import org.elasticsearch.client.benchmark.ops.bulk.BulkRequestExecutor;
+import org.elasticsearch.client.benchmark.ops.search.SearchRequestExecutor;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+
+public final class RestClientBenchmark extends AbstractBenchmark<RestClient> {
+    public static void main(String[] args) throws Exception {
+        RestClientBenchmark b = new RestClientBenchmark();
+        b.run(args);
+    }
+
+    @Override
+    protected RestClient client(String benchmarkTargetHost) {
+        return RestClient.builder(new HttpHost(benchmarkTargetHost, 9200)).build();
+    }
+
+    @Override
+    protected BulkRequestExecutor bulkRequestExecutor(RestClient client, String indexName, String typeName) {
+        return new RestBulkRequestExecutor(client, indexName, typeName);
+    }
+
+    @Override
+    protected SearchRequestExecutor searchRequestExecutor(RestClient client, String indexName) {
+        return new RestSearchRequestExecutor(client, indexName);
+    }
+
+    private static final class RestBulkRequestExecutor implements BulkRequestExecutor {
+        private final RestClient client;
+        private final String actionMetaData;
+
+        public RestBulkRequestExecutor(RestClient client, String index, String type) {
+            this.client = client;
+            this.actionMetaData = String.format(Locale.ROOT, "{ \"index\" : { \"_index\" : \"%s\", \"_type\" : \"%s\" } }%n", index, type);
+        }
+
+        @Override
+        public boolean bulkIndex(List<String> bulkData) {
+            StringBuilder bulkRequestBody = new StringBuilder();
+            for (String bulkItem : bulkData) {
+                bulkRequestBody.append(actionMetaData);
+                bulkRequestBody.append(bulkItem);
+                bulkRequestBody.append("\n");
+            }
+            HttpEntity entity = new NStringEntity(bulkRequestBody.toString(), ContentType.APPLICATION_JSON);
+            try {
+                Response response = client.performRequest("POST", "/geonames/type/_bulk", Collections.emptyMap(), entity);
+                return response.getStatusLine().getStatusCode() == HttpStatus.SC_OK;
+            } catch (Exception e) {
+                throw new ElasticsearchException(e);
+            }
+        }
+    }
+
+    private static final class RestSearchRequestExecutor implements SearchRequestExecutor {
+        private final RestClient client;
+        private final String endpoint;
+
+        private RestSearchRequestExecutor(RestClient client, String indexName) {
+            this.client = client;
+            this.endpoint = "/" + indexName + "/_search";
+        }
+
+        @Override
+        public boolean search(String source) {
+            HttpEntity searchBody = new NStringEntity(source, StandardCharsets.UTF_8);
+            try {
+                Response response = client.performRequest("GET", endpoint, Collections.emptyMap(), searchBody);
+                return response.getStatusLine().getStatusCode() == HttpStatus.SC_OK;
+            } catch (IOException e) {
+                throw new ElasticsearchException(e);
+            }
+        }
+    }
+}

+ 117 - 0
client/benchmark/src/main/java/org/elasticsearch/client/benchmark/transport/TransportClientBenchmark.java

@@ -0,0 +1,117 @@
+/*
+ * 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.client.benchmark.transport;
+
+import org.elasticsearch.ElasticsearchException;
+import org.elasticsearch.action.bulk.BulkRequestBuilder;
+import org.elasticsearch.action.bulk.BulkResponse;
+import org.elasticsearch.action.index.IndexRequest;
+import org.elasticsearch.action.search.SearchResponse;
+import org.elasticsearch.client.benchmark.AbstractBenchmark;
+import org.elasticsearch.client.benchmark.ops.bulk.BulkRequestExecutor;
+import org.elasticsearch.client.benchmark.ops.search.SearchRequestExecutor;
+import org.elasticsearch.client.transport.TransportClient;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.transport.InetSocketTransportAddress;
+import org.elasticsearch.index.query.QueryBuilders;
+import org.elasticsearch.rest.RestStatus;
+import org.elasticsearch.transport.client.PreBuiltTransportClient;
+
+import java.net.InetAddress;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+
+public final class TransportClientBenchmark extends AbstractBenchmark<TransportClient> {
+    public static void main(String[] args) throws Exception {
+        TransportClientBenchmark benchmark = new TransportClientBenchmark();
+        benchmark.run(args);
+    }
+
+    @Override
+    protected TransportClient client(String benchmarkTargetHost) throws Exception {
+        TransportClient client = new PreBuiltTransportClient(Settings.EMPTY);
+        client.addTransportAddress(new InetSocketTransportAddress(InetAddress.getByName(benchmarkTargetHost), 9300));
+        return client;
+    }
+
+    @Override
+    protected BulkRequestExecutor bulkRequestExecutor(TransportClient client, String indexName, String typeName) {
+        return new TransportBulkRequestExecutor(client, indexName, typeName);
+    }
+
+    @Override
+    protected SearchRequestExecutor searchRequestExecutor(TransportClient client, String indexName) {
+        return new TransportSearchRequestExecutor(client, indexName);
+    }
+
+    private static final class TransportBulkRequestExecutor implements BulkRequestExecutor {
+        private final TransportClient client;
+        private final String indexName;
+        private final String typeName;
+
+        public TransportBulkRequestExecutor(TransportClient client, String indexName, String typeName) {
+            this.client = client;
+            this.indexName = indexName;
+            this.typeName = typeName;
+        }
+
+        @Override
+        public boolean bulkIndex(List<String> bulkData) {
+            BulkRequestBuilder builder = client.prepareBulk();
+            for (String bulkItem : bulkData) {
+                builder.add(new IndexRequest(indexName, typeName).source(bulkItem.getBytes(StandardCharsets.UTF_8)));
+            }
+            BulkResponse bulkResponse;
+            try {
+                bulkResponse = builder.execute().get();
+            } catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+                return false;
+            } catch (ExecutionException e) {
+                throw new ElasticsearchException(e);
+            }
+            return !bulkResponse.hasFailures();
+        }
+    }
+
+    private static final class TransportSearchRequestExecutor implements SearchRequestExecutor {
+        private final TransportClient client;
+        private final String indexName;
+
+        private TransportSearchRequestExecutor(TransportClient client, String indexName) {
+            this.client = client;
+            this.indexName = indexName;
+        }
+
+        @Override
+        public boolean search(String source) {
+            final SearchResponse response;
+            try {
+                response = client.prepareSearch(indexName).setQuery(QueryBuilders.wrapperQuery(source)).execute().get();
+                return response.status() == RestStatus.OK;
+            } catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+                return false;
+            } catch (ExecutionException e) {
+                throw new ElasticsearchException(e);
+            }
+        }
+    }
+}

+ 9 - 0
client/benchmark/src/main/resources/log4j.properties

@@ -0,0 +1,9 @@
+es.logger.level=INFO
+log4j.rootLogger=${es.logger.level}, out
+
+log4j.logger.org.apache.http=INFO, out
+log4j.additivity.org.apache.http=false
+
+log4j.appender.out=org.apache.log4j.ConsoleAppender
+log4j.appender.out.layout=org.apache.log4j.PatternLayout
+log4j.appender.out.layout.conversionPattern=[%d{ISO8601}][%-5p][%-25c] %m%n

+ 1 - 0
settings.gradle

@@ -9,6 +9,7 @@ List projects = [
   'client:sniffer',
   'client:transport',
   'client:test',
+  'client:benchmark',
   'benchmarks',
   'distribution:integ-test-zip',
   'distribution:zip',