|
| 1 | +/* |
| 2 | + * Licensed to Elasticsearch under one or more contributor |
| 3 | + * license agreements. See the NOTICE file distributed with |
| 4 | + * this work for additional information regarding copyright |
| 5 | + * ownership. Elasticsearch licenses this file to you under |
| 6 | + * the Apache License, Version 2.0 (the "License"); you may |
| 7 | + * not use this file except in compliance with the License. |
| 8 | + * You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, |
| 13 | + * software distributed under the License is distributed on an |
| 14 | + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY |
| 15 | + * KIND, either express or implied. See the License for the |
| 16 | + * specific language governing permissions and limitations |
| 17 | + * under the License. |
| 18 | + */ |
| 19 | +package org.elasticsearch.benchmark.search.aggregations; |
| 20 | + |
| 21 | +import org.apache.lucene.search.ScoreDoc; |
| 22 | +import org.apache.lucene.search.TopDocs; |
| 23 | +import org.apache.lucene.search.TotalHits; |
| 24 | +import org.apache.lucene.util.BytesRef; |
| 25 | +import org.elasticsearch.action.OriginalIndices; |
| 26 | +import org.elasticsearch.action.search.QueryPhaseResultConsumer; |
| 27 | +import org.elasticsearch.action.search.SearchPhaseController; |
| 28 | +import org.elasticsearch.action.search.SearchProgressListener; |
| 29 | +import org.elasticsearch.action.search.SearchRequest; |
| 30 | +import org.elasticsearch.common.breaker.CircuitBreaker; |
| 31 | +import org.elasticsearch.common.breaker.NoopCircuitBreaker; |
| 32 | +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; |
| 33 | +import org.elasticsearch.common.lucene.search.TopDocsAndMaxScore; |
| 34 | +import org.elasticsearch.common.settings.Settings; |
| 35 | +import org.elasticsearch.index.Index; |
| 36 | +import org.elasticsearch.index.shard.ShardId; |
| 37 | +import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; |
| 38 | +import org.elasticsearch.search.DocValueFormat; |
| 39 | +import org.elasticsearch.search.SearchModule; |
| 40 | +import org.elasticsearch.search.SearchShardTarget; |
| 41 | +import org.elasticsearch.search.aggregations.AggregationBuilders; |
| 42 | +import org.elasticsearch.search.aggregations.BucketOrder; |
| 43 | +import org.elasticsearch.search.aggregations.InternalAggregation; |
| 44 | +import org.elasticsearch.search.aggregations.InternalAggregations; |
| 45 | +import org.elasticsearch.search.aggregations.MultiBucketConsumerService; |
| 46 | +import org.elasticsearch.search.aggregations.bucket.terms.StringTerms; |
| 47 | +import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; |
| 48 | +import org.elasticsearch.search.builder.SearchSourceBuilder; |
| 49 | +import org.elasticsearch.search.query.QuerySearchResult; |
| 50 | +import org.openjdk.jmh.annotations.Benchmark; |
| 51 | +import org.openjdk.jmh.annotations.BenchmarkMode; |
| 52 | +import org.openjdk.jmh.annotations.Fork; |
| 53 | +import org.openjdk.jmh.annotations.Measurement; |
| 54 | +import org.openjdk.jmh.annotations.Mode; |
| 55 | +import org.openjdk.jmh.annotations.OutputTimeUnit; |
| 56 | +import org.openjdk.jmh.annotations.Param; |
| 57 | +import org.openjdk.jmh.annotations.Scope; |
| 58 | +import org.openjdk.jmh.annotations.Setup; |
| 59 | +import org.openjdk.jmh.annotations.State; |
| 60 | +import org.openjdk.jmh.annotations.Warmup; |
| 61 | + |
| 62 | +import java.util.AbstractList; |
| 63 | +import java.util.ArrayList; |
| 64 | +import java.util.Collections; |
| 65 | +import java.util.HashSet; |
| 66 | +import java.util.List; |
| 67 | +import java.util.Random; |
| 68 | +import java.util.Set; |
| 69 | +import java.util.concurrent.CountDownLatch; |
| 70 | +import java.util.concurrent.ExecutorService; |
| 71 | +import java.util.concurrent.Executors; |
| 72 | +import java.util.concurrent.TimeUnit; |
| 73 | + |
| 74 | +import static java.util.Collections.emptyList; |
| 75 | + |
| 76 | +@Warmup(iterations = 5) |
| 77 | +@Measurement(iterations = 7) |
| 78 | +@BenchmarkMode(Mode.AverageTime) |
| 79 | +@OutputTimeUnit(TimeUnit.MILLISECONDS) |
| 80 | +@State(Scope.Thread) |
| 81 | +@Fork(value = 1) |
| 82 | +public class TermsReduceBenchmark { |
| 83 | + private final SearchModule searchModule = new SearchModule(Settings.EMPTY, emptyList()); |
| 84 | + private final NamedWriteableRegistry namedWriteableRegistry = new NamedWriteableRegistry(searchModule.getNamedWriteables()); |
| 85 | + private final SearchPhaseController controller = new SearchPhaseController( |
| 86 | + namedWriteableRegistry, |
| 87 | + req -> new InternalAggregation.ReduceContextBuilder() { |
| 88 | + @Override |
| 89 | + public InternalAggregation.ReduceContext forPartialReduction() { |
| 90 | + return InternalAggregation.ReduceContext.forPartialReduction(null, null, () -> PipelineAggregator.PipelineTree.EMPTY); |
| 91 | + } |
| 92 | + |
| 93 | + @Override |
| 94 | + public InternalAggregation.ReduceContext forFinalReduction() { |
| 95 | + final MultiBucketConsumerService.MultiBucketConsumer bucketConsumer = new MultiBucketConsumerService.MultiBucketConsumer( |
| 96 | + Integer.MAX_VALUE, |
| 97 | + new NoneCircuitBreakerService().getBreaker(CircuitBreaker.REQUEST) |
| 98 | + ); |
| 99 | + return InternalAggregation.ReduceContext.forFinalReduction( |
| 100 | + null, |
| 101 | + null, |
| 102 | + bucketConsumer, |
| 103 | + PipelineAggregator.PipelineTree.EMPTY |
| 104 | + ); |
| 105 | + } |
| 106 | + } |
| 107 | + ); |
| 108 | + |
| 109 | + @State(Scope.Benchmark) |
| 110 | + public static class TermsList extends AbstractList<InternalAggregations> { |
| 111 | + @Param({ "1600172297" }) |
| 112 | + long seed; |
| 113 | + |
| 114 | + @Param({ "64", "128", "512" }) |
| 115 | + int numShards; |
| 116 | + |
| 117 | + @Param({ "100" }) |
| 118 | + int topNSize; |
| 119 | + |
| 120 | + @Param({ "1", "10", "100" }) |
| 121 | + int cardinalityFactor; |
| 122 | + |
| 123 | + List<InternalAggregations> aggsList; |
| 124 | + |
| 125 | + @Setup |
| 126 | + public void setup() { |
| 127 | + this.aggsList = new ArrayList<>(); |
| 128 | + Random rand = new Random(seed); |
| 129 | + int cardinality = cardinalityFactor * topNSize; |
| 130 | + BytesRef[] dict = new BytesRef[cardinality]; |
| 131 | + for (int i = 0; i < dict.length; i++) { |
| 132 | + dict[i] = new BytesRef(Long.toString(rand.nextLong())); |
| 133 | + } |
| 134 | + for (int i = 0; i < numShards; i++) { |
| 135 | + aggsList.add(InternalAggregations.from(Collections.singletonList(newTerms(rand, dict, true)))); |
| 136 | + } |
| 137 | + } |
| 138 | + |
| 139 | + private StringTerms newTerms(Random rand, BytesRef[] dict, boolean withNested) { |
| 140 | + Set<BytesRef> randomTerms = new HashSet<>(); |
| 141 | + for (int i = 0; i < topNSize; i++) { |
| 142 | + randomTerms.add(dict[rand.nextInt(dict.length)]); |
| 143 | + } |
| 144 | + List<StringTerms.Bucket> buckets = new ArrayList<>(); |
| 145 | + for (BytesRef term : randomTerms) { |
| 146 | + InternalAggregations subAggs; |
| 147 | + if (withNested) { |
| 148 | + subAggs = InternalAggregations.from(Collections.singletonList(newTerms(rand, dict, false))); |
| 149 | + } else { |
| 150 | + subAggs = InternalAggregations.EMPTY; |
| 151 | + } |
| 152 | + buckets.add(new StringTerms.Bucket(term, rand.nextInt(10000), subAggs, true, 0L, DocValueFormat.RAW)); |
| 153 | + } |
| 154 | + |
| 155 | + Collections.sort(buckets, (a, b) -> a.compareKey(b)); |
| 156 | + return new StringTerms( |
| 157 | + "terms", |
| 158 | + BucketOrder.key(true), |
| 159 | + BucketOrder.count(false), |
| 160 | + topNSize, |
| 161 | + 1, |
| 162 | + Collections.emptyMap(), |
| 163 | + DocValueFormat.RAW, |
| 164 | + numShards, |
| 165 | + true, |
| 166 | + 0, |
| 167 | + buckets, |
| 168 | + 0 |
| 169 | + ); |
| 170 | + } |
| 171 | + |
| 172 | + @Override |
| 173 | + public InternalAggregations get(int index) { |
| 174 | + return aggsList.get(index); |
| 175 | + } |
| 176 | + |
| 177 | + @Override |
| 178 | + public int size() { |
| 179 | + return aggsList.size(); |
| 180 | + } |
| 181 | + } |
| 182 | + |
| 183 | + @Param({ "32", "512" }) |
| 184 | + private int bufferSize; |
| 185 | + |
| 186 | + @Benchmark |
| 187 | + public SearchPhaseController.ReducedQueryPhase reduceAggs(TermsList candidateList) throws Exception { |
| 188 | + List<QuerySearchResult> shards = new ArrayList<>(); |
| 189 | + for (int i = 0; i < candidateList.size(); i++) { |
| 190 | + QuerySearchResult result = new QuerySearchResult(); |
| 191 | + result.setShardIndex(i); |
| 192 | + result.from(0); |
| 193 | + result.size(0); |
| 194 | + result.topDocs( |
| 195 | + new TopDocsAndMaxScore( |
| 196 | + new TopDocs(new TotalHits(1000, TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO), new ScoreDoc[0]), |
| 197 | + Float.NaN |
| 198 | + ), |
| 199 | + new DocValueFormat[] { DocValueFormat.RAW } |
| 200 | + ); |
| 201 | + result.aggregations(candidateList.get(i)); |
| 202 | + result.setSearchShardTarget( |
| 203 | + new SearchShardTarget("node", new ShardId(new Index("index", "index"), i), null, OriginalIndices.NONE) |
| 204 | + ); |
| 205 | + shards.add(result); |
| 206 | + } |
| 207 | + SearchRequest request = new SearchRequest(); |
| 208 | + request.source(new SearchSourceBuilder().size(0).aggregation(AggregationBuilders.terms("test"))); |
| 209 | + request.setBatchedReduceSize(bufferSize); |
| 210 | + ExecutorService executor = Executors.newFixedThreadPool(1); |
| 211 | + QueryPhaseResultConsumer consumer = new QueryPhaseResultConsumer( |
| 212 | + request, |
| 213 | + executor, |
| 214 | + new NoopCircuitBreaker(CircuitBreaker.REQUEST), |
| 215 | + controller, |
| 216 | + SearchProgressListener.NOOP, |
| 217 | + namedWriteableRegistry, |
| 218 | + shards.size(), |
| 219 | + exc -> {} |
| 220 | + ); |
| 221 | + CountDownLatch latch = new CountDownLatch(shards.size()); |
| 222 | + for (int i = 0; i < shards.size(); i++) { |
| 223 | + consumer.consumeResult(shards.get(i), () -> latch.countDown()); |
| 224 | + } |
| 225 | + latch.await(); |
| 226 | + SearchPhaseController.ReducedQueryPhase phase = consumer.reduce(); |
| 227 | + executor.shutdownNow(); |
| 228 | + return phase; |
| 229 | + } |
| 230 | +} |
0 commit comments