Merge suggest stats into search stats

This commit is contained in:
Areek Zillur 2016-03-18 14:45:49 -04:00
parent ed49ec437f
commit 91dd9b3301
17 changed files with 145 additions and 334 deletions

View File

@ -42,7 +42,6 @@ import org.elasticsearch.index.search.stats.SearchStats;
import org.elasticsearch.index.shard.DocsStats;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.store.StoreStats;
import org.elasticsearch.index.suggest.stats.SuggestStats;
import org.elasticsearch.index.translog.TranslogStats;
import org.elasticsearch.index.warmer.WarmerStats;
import org.elasticsearch.indices.IndicesQueryCache;
@ -109,7 +108,7 @@ public class CommonStats implements Streamable, ToXContent {
translog = new TranslogStats();
break;
case Suggest:
suggest = new SuggestStats();
// skip
break;
case RequestCache:
requestCache = new RequestCacheStats();
@ -177,7 +176,7 @@ public class CommonStats implements Streamable, ToXContent {
translog = indexShard.translogStats();
break;
case Suggest:
suggest = indexShard.suggestStats();
// skip
break;
case RequestCache:
requestCache = indexShard.requestCache().stats();
@ -236,9 +235,6 @@ public class CommonStats implements Streamable, ToXContent {
@Nullable
public TranslogStats translog;
@Nullable
public SuggestStats suggest;
@Nullable
public RequestCacheStats requestCache;
@ -367,14 +363,6 @@ public class CommonStats implements Streamable, ToXContent {
} else {
translog.add(stats.getTranslog());
}
if (suggest == null) {
if (stats.getSuggest() != null) {
suggest = new SuggestStats();
suggest.add(stats.getSuggest());
}
} else {
suggest.add(stats.getSuggest());
}
if (requestCache == null) {
if (stats.getRequestCache() != null) {
requestCache = new RequestCacheStats();
@ -468,11 +456,6 @@ public class CommonStats implements Streamable, ToXContent {
return translog;
}
@Nullable
public SuggestStats getSuggest() {
return suggest;
}
@Nullable
public RequestCacheStats getRequestCache() {
return requestCache;
@ -555,7 +538,6 @@ public class CommonStats implements Streamable, ToXContent {
segments = SegmentsStats.readSegmentsStats(in);
}
translog = in.readOptionalStreamable(TranslogStats::new);
suggest = in.readOptionalStreamable(SuggestStats::new);
requestCache = in.readOptionalStreamable(RequestCacheStats::new);
recoveryStats = in.readOptionalStreamable(RecoveryStats::new);
}
@ -647,7 +629,6 @@ public class CommonStats implements Streamable, ToXContent {
segments.writeTo(out);
}
out.writeOptionalStreamable(translog);
out.writeOptionalStreamable(suggest);
out.writeOptionalStreamable(requestCache);
out.writeOptionalStreamable(recoveryStats);
}
@ -700,9 +681,6 @@ public class CommonStats implements Streamable, ToXContent {
if (translog != null) {
translog.toXContent(builder, params);
}
if (suggest != null) {
suggest.toXContent(builder, params);
}
if (requestCache != null) {
requestCache.toXContent(builder, params);
}

View File

@ -244,7 +244,7 @@ public class CommonStatsFlags implements Streamable, Cloneable {
Completion("completion"),
Segments("segments"),
Translog("translog"),
Suggest("suggest"),
Suggest("suggest"), // unused
RequestCache("request_cache"),
Recovery("recovery");

View File

@ -152,11 +152,6 @@ public class IndicesStatsRequestBuilder extends BroadcastOperationRequestBuilder
return this;
}
public IndicesStatsRequestBuilder setSuggest(boolean suggest) {
request.suggest(suggest);
return this;
}
public IndicesStatsRequestBuilder setRequestCache(boolean requestCache) {
request.requestCache(requestCache);
return this;

View File

@ -51,6 +51,10 @@ public class SearchStats implements Streamable, ToXContent {
private long scrollTimeInMillis;
private long scrollCurrent;
private long suggestCount;
private long suggestTimeInMillis;
private long suggestCurrent;
Stats() {
}
@ -58,7 +62,8 @@ public class SearchStats implements Streamable, ToXContent {
public Stats(
long queryCount, long queryTimeInMillis, long queryCurrent,
long fetchCount, long fetchTimeInMillis, long fetchCurrent,
long scrollCount, long scrollTimeInMillis, long scrollCurrent
long scrollCount, long scrollTimeInMillis, long scrollCurrent,
long suggestCount, long suggestTimeInMillis, long suggestCurrent
) {
this.queryCount = queryCount;
this.queryTimeInMillis = queryTimeInMillis;
@ -71,13 +76,19 @@ public class SearchStats implements Streamable, ToXContent {
this.scrollCount = scrollCount;
this.scrollTimeInMillis = scrollTimeInMillis;
this.scrollCurrent = scrollCurrent;
this.suggestCount = suggestCount;
this.suggestTimeInMillis = suggestTimeInMillis;
this.suggestCurrent = suggestCurrent;
}
public Stats(Stats stats) {
this(
stats.queryCount, stats.queryTimeInMillis, stats.queryCurrent,
stats.fetchCount, stats.fetchTimeInMillis, stats.fetchCurrent,
stats.scrollCount, stats.scrollTimeInMillis, stats.scrollCurrent
stats.scrollCount, stats.scrollTimeInMillis, stats.scrollCurrent,
stats.suggestCount, stats.suggestTimeInMillis, stats.suggestCurrent
);
}
@ -93,6 +104,10 @@ public class SearchStats implements Streamable, ToXContent {
scrollCount += stats.scrollCount;
scrollTimeInMillis += stats.scrollTimeInMillis;
scrollCurrent += stats.scrollCurrent;
suggestCount += stats.suggestCount;
suggestTimeInMillis += stats.suggestTimeInMillis;
suggestCurrent += stats.suggestCurrent;
}
public long getQueryCount() {
@ -143,6 +158,22 @@ public class SearchStats implements Streamable, ToXContent {
return scrollCurrent;
}
public long getSuggestCount() {
return suggestCount;
}
public long getSuggestTimeInMillis() {
return suggestTimeInMillis;
}
public TimeValue getSuggestTime() {
return new TimeValue(suggestTimeInMillis);
}
public long getSuggestCurrent() {
return suggestCurrent;
}
public static Stats readStats(StreamInput in) throws IOException {
Stats stats = new Stats();
stats.readFrom(in);
@ -162,6 +193,10 @@ public class SearchStats implements Streamable, ToXContent {
scrollCount = in.readVLong();
scrollTimeInMillis = in.readVLong();
scrollCurrent = in.readVLong();
suggestCount = in.readVLong();
suggestTimeInMillis = in.readVLong();
suggestCurrent = in.readVLong();
}
@Override
@ -177,6 +212,10 @@ public class SearchStats implements Streamable, ToXContent {
out.writeVLong(scrollCount);
out.writeVLong(scrollTimeInMillis);
out.writeVLong(scrollCurrent);
out.writeVLong(suggestCount);
out.writeVLong(suggestTimeInMillis);
out.writeVLong(suggestCurrent);
}
@Override
@ -193,6 +232,10 @@ public class SearchStats implements Streamable, ToXContent {
builder.timeValueField(Fields.SCROLL_TIME_IN_MILLIS, Fields.SCROLL_TIME, scrollTimeInMillis);
builder.field(Fields.SCROLL_CURRENT, scrollCurrent);
builder.field(Fields.SUGGEST_TOTAL, suggestCount);
builder.timeValueField(Fields.SUGGEST_TIME_IN_MILLIS, Fields.SUGGEST_TIME, suggestTimeInMillis);
builder.field(Fields.SUGGEST_CURRENT, suggestCurrent);
return builder;
}
}
@ -292,6 +335,10 @@ public class SearchStats implements Streamable, ToXContent {
static final XContentBuilderString SCROLL_TIME = new XContentBuilderString("scroll_time");
static final XContentBuilderString SCROLL_TIME_IN_MILLIS = new XContentBuilderString("scroll_time_in_millis");
static final XContentBuilderString SCROLL_CURRENT = new XContentBuilderString("scroll_current");
static final XContentBuilderString SUGGEST_TOTAL = new XContentBuilderString("suggest_total");
static final XContentBuilderString SUGGEST_TIME = new XContentBuilderString("suggest_time");
static final XContentBuilderString SUGGEST_TIME_IN_MILLIS = new XContentBuilderString("suggest_time_in_millis");
static final XContentBuilderString SUGGEST_CURRENT = new XContentBuilderString("suggest_current");
}
public static SearchStats readSearchStats(StreamInput in) throws IOException {

View File

@ -23,13 +23,13 @@ import org.elasticsearch.common.collect.MapBuilder;
import org.elasticsearch.common.metrics.CounterMetric;
import org.elasticsearch.common.metrics.MeanMetric;
import org.elasticsearch.common.regex.Regex;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.SearchSlowLog;
import org.elasticsearch.search.internal.SearchContext;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import java.util.function.Consumer;
import static java.util.Collections.emptyMap;
@ -72,64 +72,51 @@ public final class ShardSearchStats {
}
public void onPreQueryPhase(SearchContext searchContext) {
totalStats.queryCurrent.inc();
if (searchContext.groupStats() != null) {
for (int i = 0; i < searchContext.groupStats().size(); i++) {
groupStats(searchContext.groupStats().get(i)).queryCurrent.inc();
computeStats(searchContext, statsHolder -> {
if (searchContext.hasOnlySuggest()) {
statsHolder.suggestCurrent.inc();
} else {
statsHolder.queryCurrent.inc();
}
}
});
}
public void onFailedQueryPhase(SearchContext searchContext) {
totalStats.queryCurrent.dec();
if (searchContext.groupStats() != null) {
for (int i = 0; i < searchContext.groupStats().size(); i++) {
groupStats(searchContext.groupStats().get(i)).queryCurrent.dec();
computeStats(searchContext, statsHolder -> {
if (searchContext.hasOnlySuggest()) {
statsHolder.suggestCurrent.dec();
} else {
statsHolder.queryCurrent.dec();
}
}
});
}
public void onQueryPhase(SearchContext searchContext, long tookInNanos) {
totalStats.queryMetric.inc(tookInNanos);
totalStats.queryCurrent.dec();
if (searchContext.groupStats() != null) {
for (int i = 0; i < searchContext.groupStats().size(); i++) {
StatsHolder statsHolder = groupStats(searchContext.groupStats().get(i));
computeStats(searchContext, statsHolder -> {
if (searchContext.hasOnlySuggest()) {
statsHolder.suggestMetric.inc(tookInNanos);
statsHolder.suggestCurrent.dec();
} else {
statsHolder.queryMetric.inc(tookInNanos);
statsHolder.queryCurrent.dec();
}
}
});
slowLogSearchService.onQueryPhase(searchContext, tookInNanos);
}
public void onPreFetchPhase(SearchContext searchContext) {
totalStats.fetchCurrent.inc();
if (searchContext.groupStats() != null) {
for (int i = 0; i < searchContext.groupStats().size(); i++) {
groupStats(searchContext.groupStats().get(i)).fetchCurrent.inc();
}
}
computeStats(searchContext, statsHolder -> statsHolder.fetchCurrent.inc());
}
public void onFailedFetchPhase(SearchContext searchContext) {
totalStats.fetchCurrent.dec();
if (searchContext.groupStats() != null) {
for (int i = 0; i < searchContext.groupStats().size(); i++) {
groupStats(searchContext.groupStats().get(i)).fetchCurrent.dec();
}
}
computeStats(searchContext, statsHolder -> statsHolder.fetchCurrent.dec());
}
public void onFetchPhase(SearchContext searchContext, long tookInNanos) {
totalStats.fetchMetric.inc(tookInNanos);
totalStats.fetchCurrent.dec();
if (searchContext.groupStats() != null) {
for (int i = 0; i < searchContext.groupStats().size(); i++) {
StatsHolder statsHolder = groupStats(searchContext.groupStats().get(i));
statsHolder.fetchMetric.inc(tookInNanos);
statsHolder.fetchCurrent.dec();
}
}
computeStats(searchContext, statsHolder -> {
statsHolder.fetchMetric.inc(tookInNanos);
statsHolder.fetchCurrent.dec();
});
slowLogSearchService.onFetchPhase(searchContext, tookInNanos);
}
@ -149,6 +136,15 @@ public final class ShardSearchStats {
}
}
private void computeStats(SearchContext searchContext, Consumer<StatsHolder> consumer) {
consumer.accept(totalStats);
if (searchContext.groupStats() != null) {
for (int i = 0; i < searchContext.groupStats().size(); i++) {
consumer.accept(groupStats(searchContext.groupStats().get(i)));
}
}
}
private StatsHolder groupStats(String group) {
StatsHolder stats = groupsStats.get(group);
if (stats == null) {
@ -184,26 +180,30 @@ public final class ShardSearchStats {
public final MeanMetric queryMetric = new MeanMetric();
public final MeanMetric fetchMetric = new MeanMetric();
public final MeanMetric scrollMetric = new MeanMetric();
public final MeanMetric suggestMetric = new MeanMetric();
public final CounterMetric queryCurrent = new CounterMetric();
public final CounterMetric fetchCurrent = new CounterMetric();
public final CounterMetric scrollCurrent = new CounterMetric();
public final CounterMetric suggestCurrent = new CounterMetric();
public SearchStats.Stats stats() {
return new SearchStats.Stats(
queryMetric.count(), TimeUnit.NANOSECONDS.toMillis(queryMetric.sum()), queryCurrent.count(),
fetchMetric.count(), TimeUnit.NANOSECONDS.toMillis(fetchMetric.sum()), fetchCurrent.count(),
scrollMetric.count(), TimeUnit.NANOSECONDS.toMillis(scrollMetric.sum()), scrollCurrent.count()
scrollMetric.count(), TimeUnit.NANOSECONDS.toMillis(scrollMetric.sum()), scrollCurrent.count(),
suggestMetric.count(), TimeUnit.NANOSECONDS.toMillis(suggestMetric.sum()), suggestCurrent.count()
);
}
public long totalCurrent() {
return queryCurrent.count() + fetchCurrent.count() + scrollCurrent.count();
return queryCurrent.count() + fetchCurrent.count() + scrollCurrent.count() + suggestCurrent.count();
}
public void clear() {
queryMetric.clear();
fetchMetric.clear();
scrollMetric.clear();
suggestMetric.clear();
}
}
}

View File

@ -92,8 +92,6 @@ import org.elasticsearch.index.store.Store;
import org.elasticsearch.index.store.Store.MetadataSnapshot;
import org.elasticsearch.index.store.StoreFileMetaData;
import org.elasticsearch.index.store.StoreStats;
import org.elasticsearch.index.suggest.stats.ShardSuggestMetric;
import org.elasticsearch.index.suggest.stats.SuggestStats;
import org.elasticsearch.index.translog.Translog;
import org.elasticsearch.index.translog.TranslogConfig;
import org.elasticsearch.index.translog.TranslogStats;
@ -135,7 +133,6 @@ public class IndexShard extends AbstractIndexShardComponent {
private final ShardRequestCache shardQueryCache;
private final ShardFieldData shardFieldData;
private final IndexFieldDataService indexFieldDataService;
private final ShardSuggestMetric shardSuggestMetric = new ShardSuggestMetric();
private final ShardBitsetFilterCache shardBitsetFilterCache;
private final Object mutex = new Object();
private final String checkIndexOnStartup;
@ -254,10 +251,6 @@ public class IndexShard extends AbstractIndexShardComponent {
return this.getService;
}
public ShardSuggestMetric getSuggestMetric() {
return shardSuggestMetric;
}
public ShardBitsetFilterCache shardBitsetFilterCache() {
return shardBitsetFilterCache;
}
@ -631,10 +624,6 @@ public class IndexShard extends AbstractIndexShardComponent {
return getEngine().getTranslog().stats();
}
public SuggestStats suggestStats() {
return shardSuggestMetric.stats();
}
public CompletionStats completionStats(String... fields) {
CompletionStats completionStats = new CompletionStats();
try (final Engine.Searcher currentSearcher = acquireSearcher("completion_stats")) {

View File

@ -1,56 +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.index.suggest.stats;
import org.elasticsearch.common.metrics.CounterMetric;
import org.elasticsearch.common.metrics.MeanMetric;
import java.util.concurrent.TimeUnit;
/**
*
*/
public final class ShardSuggestMetric {
private final MeanMetric suggestMetric = new MeanMetric();
private final CounterMetric currentMetric = new CounterMetric();
/**
* Called before suggest
*/
public void preSuggest() {
currentMetric.inc();
}
/**
* Called after suggest
* @param tookInNanos time of suggest used in nanos
*/
public void postSuggest(long tookInNanos) {
currentMetric.dec();
suggestMetric.inc(tookInNanos);
}
/**
* @return The current stats
*/
public SuggestStats stats() {
return new SuggestStats(suggestMetric.count(), TimeUnit.NANOSECONDS.toMillis(suggestMetric.sum()), currentMetric.count());
}
}

View File

@ -1,124 +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.index.suggest.stats;
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.unit.TimeValue;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentBuilderString;
import java.io.IOException;
/**
* Exposes suggest related statistics.
*/
public class SuggestStats implements Streamable, ToXContent {
private long suggestCount;
private long suggestTimeInMillis;
private long current;
public SuggestStats() {
}
SuggestStats(long suggestCount, long suggestTimeInMillis, long current) {
this.suggestCount = suggestCount;
this.suggestTimeInMillis = suggestTimeInMillis;
this.current = current;
}
/**
* @return The number of times the suggest api has been invoked.
*/
public long getCount() {
return suggestCount;
}
/**
* @return The total amount of time spend in the suggest api
*/
public long getTimeInMillis() {
return suggestTimeInMillis;
}
/**
* @return The total amount of time spend in the suggest api
*/
public TimeValue getTime() {
return new TimeValue(getTimeInMillis());
}
/**
* @return The total amount of active suggest api invocations.
*/
public long getCurrent() {
return current;
}
public void add(SuggestStats suggestStats) {
if (suggestStats != null) {
suggestCount += suggestStats.getCount();
suggestTimeInMillis += suggestStats.getTimeInMillis();
current += suggestStats.getCurrent();
}
}
public static SuggestStats readSuggestStats(StreamInput in) throws IOException {
SuggestStats stats = new SuggestStats();
stats.readFrom(in);
return stats;
}
static final class Fields {
static final XContentBuilderString SUGGEST = new XContentBuilderString("suggest");
static final XContentBuilderString TOTAL = new XContentBuilderString("total");
static final XContentBuilderString TIME = new XContentBuilderString("time");
static final XContentBuilderString TIME_IN_MILLIS = new XContentBuilderString("time_in_millis");
static final XContentBuilderString CURRENT = new XContentBuilderString("current");
}
@Override
public void readFrom(StreamInput in) throws IOException {
suggestCount = in.readVLong();
suggestTimeInMillis = in.readVLong();
current = in.readVLong();
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeVLong(suggestCount);
out.writeVLong(suggestTimeInMillis);
out.writeVLong(current);
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject(Fields.SUGGEST);
builder.field(Fields.TOTAL, suggestCount);
builder.timeValueField(Fields.TIME_IN_MILLIS, Fields.TIME, suggestTimeInMillis);
builder.field(Fields.CURRENT, current);
builder.endObject();
return builder;
}
}

View File

@ -44,7 +44,6 @@ import org.elasticsearch.index.refresh.RefreshStats;
import org.elasticsearch.index.search.stats.SearchStats;
import org.elasticsearch.index.shard.DocsStats;
import org.elasticsearch.index.store.StoreStats;
import org.elasticsearch.index.suggest.stats.SuggestStats;
import org.elasticsearch.search.suggest.completion.CompletionStats;
import java.io.IOException;
@ -149,11 +148,6 @@ public class NodeIndicesStats implements Streamable, ToXContent {
return stats.getSegments();
}
@Nullable
public SuggestStats getSuggest() {
return stats.getSuggest();
}
@Nullable
public RecoveryStats getRecoveryStats() {
return stats.getRecoveryStats();

View File

@ -71,7 +71,7 @@ public class RestIndicesStatsAction extends BaseRestHandler {
indicesStatsRequest.docs(metrics.contains("docs"));
indicesStatsRequest.store(metrics.contains("store"));
indicesStatsRequest.indexing(metrics.contains("indexing"));
indicesStatsRequest.search(metrics.contains("search"));
indicesStatsRequest.search(metrics.contains("search") || metrics.contains("suggest"));
indicesStatsRequest.get(metrics.contains("get"));
indicesStatsRequest.merge(metrics.contains("merge"));
indicesStatsRequest.refresh(metrics.contains("refresh"));
@ -82,7 +82,6 @@ public class RestIndicesStatsAction extends BaseRestHandler {
indicesStatsRequest.segments(metrics.contains("segments"));
indicesStatsRequest.fieldData(metrics.contains("fielddata"));
indicesStatsRequest.completion(metrics.contains("completion"));
indicesStatsRequest.suggest(metrics.contains("suggest"));
indicesStatsRequest.requestCache(metrics.contains("request_cache"));
indicesStatsRequest.recovery(metrics.contains("recovery"));
indicesStatsRequest.translog(metrics.contains("translog"));

View File

@ -490,14 +490,14 @@ public class RestIndicesAction extends AbstractCatAction {
table.addCell(indexStats == null ? null : indexStats.getTotal().getWarmer().totalTime());
table.addCell(indexStats == null ? null : indexStats.getPrimaries().getWarmer().totalTime());
table.addCell(indexStats == null ? null : indexStats.getTotal().getSuggest().getCurrent());
table.addCell(indexStats == null ? null : indexStats.getPrimaries().getSuggest().getCurrent());
table.addCell(indexStats == null ? null : indexStats.getTotal().getSearch().getTotal().getSuggestCurrent());
table.addCell(indexStats == null ? null : indexStats.getPrimaries().getSearch().getTotal().getSuggestCurrent());
table.addCell(indexStats == null ? null : indexStats.getTotal().getSuggest().getTime());
table.addCell(indexStats == null ? null : indexStats.getPrimaries().getSuggest().getTime());
table.addCell(indexStats == null ? null : indexStats.getTotal().getSearch().getTotal().getSuggestTime());
table.addCell(indexStats == null ? null : indexStats.getPrimaries().getSearch().getTotal().getSuggestTime());
table.addCell(indexStats == null ? null : indexStats.getTotal().getSuggest().getCount());
table.addCell(indexStats == null ? null : indexStats.getPrimaries().getSuggest().getCount());
table.addCell(indexStats == null ? null : indexStats.getTotal().getSearch().getTotal().getSuggestCount());
table.addCell(indexStats == null ? null : indexStats.getPrimaries().getSearch().getTotal().getSuggestCount());
table.addCell(indexStats == null ? null : indexStats.getTotal().getTotalMemory());
table.addCell(indexStats == null ? null : indexStats.getPrimaries().getTotalMemory());

View File

@ -49,7 +49,6 @@ import org.elasticsearch.index.merge.MergeStats;
import org.elasticsearch.index.percolator.PercolatorQueryCacheStats;
import org.elasticsearch.index.refresh.RefreshStats;
import org.elasticsearch.index.search.stats.SearchStats;
import org.elasticsearch.index.suggest.stats.SuggestStats;
import org.elasticsearch.indices.NodeIndicesStats;
import org.elasticsearch.monitor.fs.FsInfo;
import org.elasticsearch.monitor.jvm.JvmInfo;
@ -362,10 +361,9 @@ public class RestNodesAction extends AbstractCatAction {
table.addCell(segmentsStats == null ? null : segmentsStats.getVersionMapMemory());
table.addCell(segmentsStats == null ? null : segmentsStats.getBitsetMemory());
SuggestStats suggestStats = indicesStats == null ? null : indicesStats.getSuggest();
table.addCell(suggestStats == null ? null : suggestStats.getCurrent());
table.addCell(suggestStats == null ? null : suggestStats.getTime());
table.addCell(suggestStats == null ? null : suggestStats.getCount());
table.addCell(searchStats == null ? null : searchStats.getTotal().getSuggestCurrent());
table.addCell(searchStats == null ? null : searchStats.getTotal().getSuggestTime());
table.addCell(searchStats == null ? null : searchStats.getTotal().getSuggestCount());
table.endRow();
}

View File

@ -22,13 +22,15 @@ package org.elasticsearch.index.suggest.stats;
import org.elasticsearch.action.admin.cluster.node.stats.NodeStats;
import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse;
import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse;
import org.elasticsearch.action.suggest.SuggestRequestBuilder;
import org.elasticsearch.action.suggest.SuggestResponse;
import org.elasticsearch.action.search.SearchRequestBuilder;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.routing.GroupShardsIterator;
import org.elasticsearch.cluster.routing.ShardIterator;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.search.stats.SearchStats;
import org.elasticsearch.search.suggest.SuggestBuilder;
import org.elasticsearch.search.suggest.phrase.PhraseSuggestionBuilder;
import org.elasticsearch.search.suggest.term.TermSuggestionBuilder;
import org.elasticsearch.test.ESIntegTestCase;
@ -86,49 +88,50 @@ public class SuggestStatsIT extends ESIntegTestCase {
long startTime = System.currentTimeMillis();
for (int i = 0; i < suggestAllIdx; i++) {
SuggestResponse suggestResponse = addSuggestions(internalCluster().clientNodeClient().prepareSuggest(), i).get();
SearchResponse suggestResponse = addSuggestions(internalCluster().clientNodeClient().prepareSearch(), i).get();
assertAllSuccessful(suggestResponse);
}
for (int i = 0; i < suggestIdx1; i++) {
SuggestResponse suggestResponse = addSuggestions(internalCluster().clientNodeClient().prepareSuggest("test1"), i).get();
SearchResponse suggestResponse = addSuggestions(internalCluster().clientNodeClient().prepareSearch("test1"), i).get();
assertAllSuccessful(suggestResponse);
}
for (int i = 0; i < suggestIdx2; i++) {
SuggestResponse suggestResponse = addSuggestions(internalCluster().clientNodeClient().prepareSuggest("test2"), i).get();
SearchResponse suggestResponse = addSuggestions(internalCluster().clientNodeClient().prepareSearch("test2"), i).get();
assertAllSuccessful(suggestResponse);
}
long endTime = System.currentTimeMillis();
IndicesStatsResponse indicesStats = client().admin().indices().prepareStats().execute().actionGet();
final SearchStats.Stats suggest = indicesStats.getTotal().getSearch().getTotal();
// check current
assertThat(indicesStats.getTotal().getSuggest().getCurrent(), equalTo(0L));
assertThat(suggest.getSuggestCurrent(), equalTo(0L));
// check suggest count
assertThat(indicesStats.getTotal().getSuggest().getCount(), equalTo((long) (suggestAllIdx * totalShards + suggestIdx1 * shardsIdx1 + suggestIdx2 * shardsIdx2)));
assertThat(indicesStats.getIndices().get("test1").getTotal().getSuggest().getCount(), equalTo((long) ((suggestAllIdx + suggestIdx1) * shardsIdx1)));
assertThat(indicesStats.getIndices().get("test2").getTotal().getSuggest().getCount(), equalTo((long) ((suggestAllIdx + suggestIdx2) * shardsIdx2)));
assertThat(suggest.getSuggestCount(), equalTo((long) (suggestAllIdx * totalShards + suggestIdx1 * shardsIdx1 + suggestIdx2 * shardsIdx2)));
assertThat(indicesStats.getIndices().get("test1").getTotal().getSearch().getTotal().getSuggestCount(), equalTo((long) ((suggestAllIdx + suggestIdx1) * shardsIdx1)));
assertThat(indicesStats.getIndices().get("test2").getTotal().getSearch().getTotal().getSuggestCount(), equalTo((long) ((suggestAllIdx + suggestIdx2) * shardsIdx2)));
logger.info("iter {}, iter1 {}, iter2 {}, {}", suggestAllIdx, suggestIdx1, suggestIdx2, endTime - startTime);
// check suggest time
assertThat(indicesStats.getTotal().getSuggest().getTimeInMillis(), greaterThan(0L));
assertThat(suggest.getSuggestTimeInMillis(), greaterThan(0L));
// the upperbound is num shards * total time since we do searches in parallel
assertThat(indicesStats.getTotal().getSuggest().getTimeInMillis(), lessThanOrEqualTo(totalShards * (endTime - startTime)));
assertThat(suggest.getSuggestTimeInMillis(), lessThanOrEqualTo(totalShards * (endTime - startTime)));
NodesStatsResponse nodeStats = client().admin().cluster().prepareNodesStats().execute().actionGet();
NodeStats[] nodes = nodeStats.getNodes();
Set<String> nodeIdsWithIndex = nodeIdsWithIndex("test1", "test2");
int num = 0;
for (NodeStats stat : nodes) {
SuggestStats suggestStats = stat.getIndices().getSuggest();
SearchStats.Stats suggestStats = stat.getIndices().getSearch().getTotal();
logger.info("evaluating {}", stat.getNode());
if (nodeIdsWithIndex.contains(stat.getNode().getId())) {
assertThat(suggestStats.getCount(), greaterThan(0L));
assertThat(suggestStats.getTimeInMillis(), greaterThan(0L));
assertThat(suggestStats.getSuggestCount(), greaterThan(0L));
assertThat(suggestStats.getSuggestTimeInMillis(), greaterThan(0L));
num++;
} else {
assertThat(suggestStats.getCount(), equalTo(0L));
assertThat(suggestStats.getTimeInMillis(), equalTo(0L));
assertThat(suggestStats.getSuggestCount(), equalTo(0L));
assertThat(suggestStats.getSuggestTimeInMillis(), equalTo(0L));
}
}
@ -136,15 +139,16 @@ public class SuggestStatsIT extends ESIntegTestCase {
}
private SuggestRequestBuilder addSuggestions(SuggestRequestBuilder request, int i) {
private SearchRequestBuilder addSuggestions(SearchRequestBuilder request, int i) {
final SuggestBuilder suggestBuilder = new SuggestBuilder();
for (int s = 0; s < randomIntBetween(2, 10); s++) {
if (randomBoolean()) {
request.addSuggestion("s" + s, new PhraseSuggestionBuilder("f").text("test" + i + " test" + (i - 1)));
suggestBuilder.addSuggestion("s" + s, new PhraseSuggestionBuilder("f").text("test" + i + " test" + (i - 1)));
} else {
request.addSuggestion("s" + s, new TermSuggestionBuilder("f").text("test" + i));
suggestBuilder.addSuggestion("s" + s, new TermSuggestionBuilder("f").text("test" + i));
}
}
return request;
return request.suggest(suggestBuilder);
}
private Set<String> nodeIdsWithIndex(String... indices) {

View File

@ -44,7 +44,6 @@ import org.elasticsearch.action.search.MultiSearchRequestBuilder;
import org.elasticsearch.action.search.MultiSearchResponse;
import org.elasticsearch.action.search.SearchRequestBuilder;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.suggest.SuggestRequestBuilder;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.settings.Setting;
@ -53,6 +52,7 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.settings.SettingsModule;
import org.elasticsearch.index.IndexNotFoundException;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.search.suggest.SuggestBuilder;
import org.elasticsearch.search.suggest.SuggestBuilders;
import org.elasticsearch.test.ESIntegTestCase;
@ -94,7 +94,6 @@ public class IndicesOptionsIntegrationIT extends ESIntegTestCase {
verify(typesExists("test1", "test2"), true);
verify(percolate("test1", "test2"), true);
verify(mpercolate(null, "test1", "test2"), false);
verify(suggest("test1", "test2"), true);
verify(getAliases("test1", "test2"), true);
verify(getFieldMapping("test1", "test2"), true);
verify(getMapping("test1", "test2"), true);
@ -114,7 +113,6 @@ public class IndicesOptionsIntegrationIT extends ESIntegTestCase {
verify(typesExists("test1", "test2").setIndicesOptions(options), true);
verify(percolate("test1", "test2").setIndicesOptions(options), true);
verify(mpercolate(options, "test1", "test2").setIndicesOptions(options), false);
verify(suggest("test1", "test2").setIndicesOptions(options), true);
verify(getAliases("test1", "test2").setIndicesOptions(options), true);
verify(getFieldMapping("test1", "test2").setIndicesOptions(options), true);
verify(getMapping("test1", "test2").setIndicesOptions(options), true);
@ -134,7 +132,6 @@ public class IndicesOptionsIntegrationIT extends ESIntegTestCase {
verify(typesExists("test1", "test2").setIndicesOptions(options), false);
verify(percolate("test1", "test2").setIndicesOptions(options), false);
verify(mpercolate(options, "test1", "test2").setIndicesOptions(options), false);
verify(suggest("test1", "test2").setIndicesOptions(options), false);
verify(getAliases("test1", "test2").setIndicesOptions(options), false);
verify(getFieldMapping("test1", "test2").setIndicesOptions(options), false);
verify(getMapping("test1", "test2").setIndicesOptions(options), false);
@ -156,7 +153,6 @@ public class IndicesOptionsIntegrationIT extends ESIntegTestCase {
verify(typesExists("test1", "test2").setIndicesOptions(options), false);
verify(percolate("test1", "test2").setIndicesOptions(options), false);
verify(mpercolate(options, "test1", "test2").setIndicesOptions(options), false);
verify(suggest("test1", "test2").setIndicesOptions(options), false);
verify(getAliases("test1", "test2").setIndicesOptions(options), false);
verify(getFieldMapping("test1", "test2").setIndicesOptions(options), false);
verify(getMapping("test1", "test2").setIndicesOptions(options), false);
@ -186,7 +182,6 @@ public class IndicesOptionsIntegrationIT extends ESIntegTestCase {
verify(typesExists("test1").setIndicesOptions(options), true);
verify(percolate("test1").setIndicesOptions(options), true);
verify(mpercolate(options, "test1").setIndicesOptions(options), true);
verify(suggest("test1").setIndicesOptions(options), true);
verify(getAliases("test1").setIndicesOptions(options), true);
verify(getFieldMapping("test1").setIndicesOptions(options), true);
verify(getMapping("test1").setIndicesOptions(options), true);
@ -206,7 +201,6 @@ public class IndicesOptionsIntegrationIT extends ESIntegTestCase {
verify(typesExists("test1").setIndicesOptions(options), false);
verify(percolate("test1").setIndicesOptions(options), false);
verify(mpercolate(options, "test1").setIndicesOptions(options), false);
verify(suggest("test1").setIndicesOptions(options), false);
verify(getAliases("test1").setIndicesOptions(options), false);
verify(getFieldMapping("test1").setIndicesOptions(options), false);
verify(getMapping("test1").setIndicesOptions(options), false);
@ -229,7 +223,6 @@ public class IndicesOptionsIntegrationIT extends ESIntegTestCase {
verify(typesExists("test1").setIndicesOptions(options), false);
verify(percolate("test1").setIndicesOptions(options), false);
verify(mpercolate(options, "test1").setIndicesOptions(options), false);
verify(suggest("test1").setIndicesOptions(options), false);
verify(getAliases("test1").setIndicesOptions(options), false);
verify(getFieldMapping("test1").setIndicesOptions(options), false);
verify(getMapping("test1").setIndicesOptions(options), false);
@ -250,7 +243,6 @@ public class IndicesOptionsIntegrationIT extends ESIntegTestCase {
verify(aliasExists("test1").setIndicesOptions(options), true);
verify(typesExists("test1").setIndicesOptions(options), true);
verify(percolate("test1").setIndicesOptions(options), true);
verify(suggest("test1").setIndicesOptions(options), true);
verify(getAliases("test1").setIndicesOptions(options), true);
verify(getFieldMapping("test1").setIndicesOptions(options), true);
verify(getMapping("test1").setIndicesOptions(options), true);
@ -269,7 +261,6 @@ public class IndicesOptionsIntegrationIT extends ESIntegTestCase {
verify(aliasExists("test1").setIndicesOptions(options), false);
verify(typesExists("test1").setIndicesOptions(options), false);
verify(percolate("test1").setIndicesOptions(options), false);
verify(suggest("test1").setIndicesOptions(options), false);
verify(getAliases("test1").setIndicesOptions(options), false);
verify(getFieldMapping("test1").setIndicesOptions(options), false);
verify(getMapping("test1").setIndicesOptions(options), false);
@ -291,7 +282,6 @@ public class IndicesOptionsIntegrationIT extends ESIntegTestCase {
verify(aliasExists("test1").setIndicesOptions(options), false);
verify(typesExists("test1").setIndicesOptions(options), false);
verify(percolate("test1").setIndicesOptions(options), false);
verify(suggest("test1").setIndicesOptions(options), false);
verify(getAliases("test1").setIndicesOptions(options), false);
verify(getFieldMapping("test1").setIndicesOptions(options), false);
verify(getMapping("test1").setIndicesOptions(options), false);
@ -344,7 +334,6 @@ public class IndicesOptionsIntegrationIT extends ESIntegTestCase {
verify(typesExists(indices), false);
verify(percolate(indices), false);
verify(mpercolate(null, indices), false);
verify(suggest(indices), false);
verify(getAliases(indices), false);
verify(getFieldMapping(indices), false);
verify(getMapping(indices), false);
@ -365,7 +354,6 @@ public class IndicesOptionsIntegrationIT extends ESIntegTestCase {
verify(typesExists(indices).setIndicesOptions(options), false);
verify(percolate(indices).setIndicesOptions(options), false);
verify(mpercolate(options, indices), false);
verify(suggest(indices).setIndicesOptions(options), false);
verify(getAliases(indices).setIndicesOptions(options), false);
verify(getFieldMapping(indices).setIndicesOptions(options), false);
verify(getMapping(indices).setIndicesOptions(options), false);
@ -389,7 +377,6 @@ public class IndicesOptionsIntegrationIT extends ESIntegTestCase {
verify(typesExists(indices), false);
verify(percolate(indices), false);
verify(mpercolate(null, indices), false);
verify(suggest(indices), false);
verify(getAliases(indices), false);
verify(getFieldMapping(indices), false);
verify(getMapping(indices), false);
@ -410,7 +397,6 @@ public class IndicesOptionsIntegrationIT extends ESIntegTestCase {
verify(typesExists(indices), false);
verify(percolate(indices), false);
verify(mpercolate(null, indices), false);
verify(suggest(indices), false);
verify(getAliases(indices), false);
verify(getFieldMapping(indices), false);
verify(getMapping(indices), false);
@ -431,7 +417,6 @@ public class IndicesOptionsIntegrationIT extends ESIntegTestCase {
verify(typesExists(indices).setIndicesOptions(options), false);
verify(percolate(indices).setIndicesOptions(options), false);
verify(mpercolate(options, indices), false);
verify(suggest(indices).setIndicesOptions(options), false);
verify(getAliases(indices).setIndicesOptions(options), false);
verify(getFieldMapping(indices).setIndicesOptions(options), false);
verify(getMapping(indices).setIndicesOptions(options), false);
@ -755,10 +740,6 @@ public class IndicesOptionsIntegrationIT extends ESIntegTestCase {
return builder.add(percolate(indices));
}
private static SuggestRequestBuilder suggest(String... indices) {
return client().prepareSuggest(indices).addSuggestion("name", SuggestBuilders.termSuggestion("a"));
}
private static GetAliasesRequestBuilder getAliases(String... indices) {
return client().admin().indices().prepareGetAliases("dummy").addIndices(indices);
}

View File

@ -61,6 +61,7 @@ import java.util.Random;
import static org.elasticsearch.cluster.metadata.IndexMetaData.PROTO;
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS;
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
import static org.elasticsearch.common.xcontent.XContentFactory.contentBuilder;
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAllSuccessful;
@ -575,6 +576,10 @@ public class IndexStatsIT extends ESIntegTestCase {
IndicesStatsResponse stats = builder.execute().actionGet();
for (Flag flag : values) {
if (flag == Flag.Suggest) {
// suggest flag is unused
continue;
}
assertThat(isSet(flag, stats.getPrimaries()), equalTo(false));
assertThat(isSet(flag, stats.getTotal()), equalTo(false));
}
@ -610,6 +615,10 @@ public class IndexStatsIT extends ESIntegTestCase {
}
for (Flag flag : EnumSet.complementOf(flags)) { // check the complement
if (flag == Flag.Suggest) {
// suggest flag is unused
continue;
}
assertThat(isSet(flag, stats.getPrimaries()), equalTo(false));
assertThat(isSet(flag, stats.getTotal()), equalTo(false));
}
@ -914,8 +923,7 @@ public class IndexStatsIT extends ESIntegTestCase {
case Translog:
builder.setTranslog(set);
break;
case Suggest:
builder.setSuggest(set);
case Suggest: // unused
break;
case RequestCache:
builder.setRequestCache(set);
@ -961,8 +969,8 @@ public class IndexStatsIT extends ESIntegTestCase {
return response.getSegments() != null;
case Translog:
return response.getTranslog() != null;
case Suggest:
return response.getSuggest() != null;
case Suggest: // unused
return true;
case RequestCache:
return response.getRequestCache() != null;
case Recovery:

View File

@ -32,9 +32,9 @@ public class SearchStatsUnitTests extends ESTestCase {
// let's create two dummy search stats with groups
Map<String, Stats> groupStats1 = new HashMap<>();
Map<String, Stats> groupStats2 = new HashMap<>();
groupStats2.put("group1", new Stats(1, 1, 1, 1, 1, 1, 1, 1, 1));
SearchStats searchStats1 = new SearchStats(new Stats(1, 1, 1, 1, 1, 1, 1, 1, 1), 0, groupStats1);
SearchStats searchStats2 = new SearchStats(new Stats(1, 1, 1, 1, 1, 1, 1, 1, 1), 0, groupStats2);
groupStats2.put("group1", new Stats(1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1));
SearchStats searchStats1 = new SearchStats(new Stats(1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1), 0, groupStats1);
SearchStats searchStats2 = new SearchStats(new Stats(1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1), 0, groupStats2);
// adding these two search stats and checking group stats are correct
searchStats1.add(searchStats2);
@ -62,5 +62,8 @@ public class SearchStatsUnitTests extends ESTestCase {
assertEquals(equalTo, stats.getScrollCount());
assertEquals(equalTo, stats.getScrollTimeInMillis());
assertEquals(equalTo, stats.getScrollCurrent());
assertEquals(equalTo, stats.getSuggestCount());
assertEquals(equalTo, stats.getSuggestTimeInMillis());
assertEquals(equalTo, stats.getSuggestCurrent());
}
}

View File

@ -34,7 +34,6 @@ setup:
- is_true: _all.total.completion
- is_true: _all.total.segments
- is_true: _all.total.translog
- is_true: _all.total.suggest
- is_true: _all.total.recovery
---
@ -56,7 +55,6 @@ setup:
- is_true: _all.total.completion
- is_true: _all.total.segments
- is_true: _all.total.translog
- is_true: _all.total.suggest
- is_true: _all.total.recovery
---
@ -78,7 +76,6 @@ setup:
- is_false: _all.total.completion
- is_false: _all.total.segments
- is_false: _all.total.translog
- is_false: _all.total.suggest
- is_false: _all.total.recovery
---
@ -100,7 +97,6 @@ setup:
- is_false: _all.total.completion
- is_false: _all.total.segments
- is_false: _all.total.translog
- is_false: _all.total.suggest
- is_false: _all.total.recovery
@ -123,6 +119,5 @@ setup:
- is_false: _all.total.completion
- is_false: _all.total.segments
- is_false: _all.total.translog
- is_false: _all.total.suggest
- is_true: _all.total.recovery