Expose size statistics for completion suggest
In order to determine how many RAM the completion suggest structures will eat up, this data should be exposed. Closes #3522
This commit is contained in:
parent
ac3d5d67be
commit
cdddbb7585
|
@ -19,6 +19,7 @@
|
|||
|
||||
package org.elasticsearch.action.admin.indices.stats;
|
||||
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
|
@ -38,6 +39,7 @@ import org.elasticsearch.index.search.stats.SearchStats;
|
|||
import org.elasticsearch.index.shard.DocsStats;
|
||||
import org.elasticsearch.index.store.StoreStats;
|
||||
import org.elasticsearch.index.warmer.WarmerStats;
|
||||
import org.elasticsearch.search.suggest.completion.CompletionStats;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
|
@ -84,6 +86,9 @@ public class CommonStats implements Streamable, ToXContent {
|
|||
@Nullable
|
||||
public PercolateStats percolate;
|
||||
|
||||
@Nullable
|
||||
public CompletionStats completion;
|
||||
|
||||
public void add(CommonStats stats) {
|
||||
if (docs == null) {
|
||||
if (stats.getDocs() != null) {
|
||||
|
@ -191,6 +196,14 @@ public class CommonStats implements Streamable, ToXContent {
|
|||
} else {
|
||||
percolate.add(stats.getPercolate());
|
||||
}
|
||||
if (completion == null) {
|
||||
if (stats.getCompletion() != null) {
|
||||
completion = new CompletionStats();
|
||||
completion.add(stats.getCompletion());
|
||||
}
|
||||
} else {
|
||||
completion.add(stats.getCompletion());
|
||||
}
|
||||
}
|
||||
|
||||
@Nullable
|
||||
|
@ -258,6 +271,11 @@ public class CommonStats implements Streamable, ToXContent {
|
|||
return percolate;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
public CompletionStats getCompletion() {
|
||||
return completion;
|
||||
}
|
||||
|
||||
public static CommonStats readCommonStats(StreamInput in) throws IOException {
|
||||
CommonStats stats = new CommonStats();
|
||||
stats.readFrom(in);
|
||||
|
@ -305,6 +323,11 @@ public class CommonStats implements Streamable, ToXContent {
|
|||
if (in.readBoolean()) {
|
||||
percolate = PercolateStats.readPercolateStats(in);
|
||||
}
|
||||
if (in.getVersion().onOrAfter(Version.V_0_90_4)) {
|
||||
if (in.readBoolean()) {
|
||||
completion = CompletionStats.readCompletionStats(in);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -387,6 +410,14 @@ public class CommonStats implements Streamable, ToXContent {
|
|||
out.writeBoolean(true);
|
||||
percolate.writeTo(out);
|
||||
}
|
||||
if (out.getVersion().onOrAfter(Version.V_0_90_4)) {
|
||||
if (completion == null) {
|
||||
out.writeBoolean(false);
|
||||
} else {
|
||||
out.writeBoolean(true);
|
||||
completion.writeTo(out);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// note, requires a wrapping object
|
||||
|
@ -431,6 +462,9 @@ public class CommonStats implements Streamable, ToXContent {
|
|||
if (percolate != null) {
|
||||
percolate.toXContent(builder, params);
|
||||
}
|
||||
if (completion != null) {
|
||||
completion.toXContent(builder, params);
|
||||
}
|
||||
return builder;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
|
||||
package org.elasticsearch.action.admin.indices.stats;
|
||||
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.io.stream.Streamable;
|
||||
|
@ -33,6 +34,7 @@ public class CommonStatsFlags implements Streamable, Cloneable {
|
|||
private String[] types = null;
|
||||
private String[] groups = null;
|
||||
private String[] fieldDataFields = null;
|
||||
private String[] completionDataFields = null;
|
||||
|
||||
/**
|
||||
* Sets all flags to return all stats.
|
||||
|
@ -42,6 +44,7 @@ public class CommonStatsFlags implements Streamable, Cloneable {
|
|||
types = null;
|
||||
groups = null;
|
||||
fieldDataFields = null;
|
||||
completionDataFields = null;
|
||||
return this;
|
||||
}
|
||||
|
||||
|
@ -53,6 +56,7 @@ public class CommonStatsFlags implements Streamable, Cloneable {
|
|||
types = null;
|
||||
groups = null;
|
||||
fieldDataFields = null;
|
||||
completionDataFields = null;
|
||||
return this;
|
||||
}
|
||||
|
||||
|
@ -107,6 +111,14 @@ public class CommonStatsFlags implements Streamable, Cloneable {
|
|||
return this.fieldDataFields;
|
||||
}
|
||||
|
||||
public CommonStatsFlags completionDataFields(String... completionDataFields) {
|
||||
this.completionDataFields = completionDataFields;
|
||||
return this;
|
||||
}
|
||||
|
||||
public String[] completionDataFields() {
|
||||
return this.completionDataFields;
|
||||
}
|
||||
|
||||
public boolean isSet(Flag flag) {
|
||||
return flags.contains(flag);
|
||||
|
@ -146,6 +158,9 @@ public class CommonStatsFlags implements Streamable, Cloneable {
|
|||
out.writeStringArrayNullable(types);
|
||||
out.writeStringArrayNullable(groups);
|
||||
out.writeStringArrayNullable(fieldDataFields);
|
||||
if (out.getVersion().onOrAfter(Version.V_0_90_4)) {
|
||||
out.writeStringArrayNullable(completionDataFields);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -160,6 +175,9 @@ public class CommonStatsFlags implements Streamable, Cloneable {
|
|||
types = in.readStringArray();
|
||||
groups = in.readStringArray();
|
||||
fieldDataFields = in.readStringArray();
|
||||
if (in.getVersion().onOrAfter(Version.V_0_90_4)) {
|
||||
completionDataFields = in.readStringArray();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -188,7 +206,8 @@ public class CommonStatsFlags implements Streamable, Cloneable {
|
|||
FieldData("fielddata"),
|
||||
Docs("docs"),
|
||||
Warmer("warmer"),
|
||||
Percolate("percolate");
|
||||
Percolate("percolate"),
|
||||
Completion("completion");
|
||||
|
||||
private final String restName;
|
||||
|
||||
|
|
|
@ -212,6 +212,24 @@ public class IndicesStatsRequest extends BroadcastOperationRequest<IndicesStatsR
|
|||
return flags.fieldDataFields();
|
||||
}
|
||||
|
||||
public IndicesStatsRequest completion(boolean completion) {
|
||||
flags.set(Flag.Completion, completion);
|
||||
return this;
|
||||
}
|
||||
|
||||
public boolean completion() {
|
||||
return flags.isSet(Flag.Completion);
|
||||
}
|
||||
|
||||
public IndicesStatsRequest completionFields(String ... completionDataFields) {
|
||||
flags.completionDataFields(completionDataFields);
|
||||
return this;
|
||||
}
|
||||
|
||||
public String[] completionFields() {
|
||||
return flags.completionDataFields();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
super.writeTo(out);
|
||||
|
|
|
@ -139,6 +139,16 @@ public class IndicesStatsRequestBuilder extends BroadcastOperationRequestBuilder
|
|||
return this;
|
||||
}
|
||||
|
||||
public IndicesStatsRequestBuilder setCompletion(boolean completion) {
|
||||
request.completion(completion);
|
||||
return this;
|
||||
}
|
||||
|
||||
public IndicesStatsRequestBuilder setCompletionFields(String... fields) {
|
||||
request.completionFields(fields);
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doExecute(ActionListener<IndicesStatsResponse> listener) {
|
||||
((IndicesAdminClient) client).stats(request, listener);
|
||||
|
|
|
@ -179,6 +179,9 @@ public class TransportIndicesStatsAction extends TransportBroadcastOperationActi
|
|||
if (request.request.percolate()) {
|
||||
stats.stats.percolate = indexShard.shardPercolateService().stats();
|
||||
}
|
||||
if (request.request.completion()) {
|
||||
stats.stats.completion = indexShard.completionStats(request.request.completionFields());
|
||||
}
|
||||
|
||||
return stats;
|
||||
}
|
||||
|
|
|
@ -50,6 +50,7 @@ import org.elasticsearch.index.shard.IndexShardState;
|
|||
import org.elasticsearch.index.store.StoreStats;
|
||||
import org.elasticsearch.index.warmer.ShardIndexWarmerService;
|
||||
import org.elasticsearch.index.warmer.WarmerStats;
|
||||
import org.elasticsearch.search.suggest.completion.CompletionStats;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -96,6 +97,8 @@ public interface IndexShard extends IndexShardComponent {
|
|||
|
||||
FieldDataStats fieldDataStats(String... fields);
|
||||
|
||||
CompletionStats completionStats(String ... fields);
|
||||
|
||||
PercolatorQueriesRegistry percolateRegistry();
|
||||
|
||||
ShardPercolateService shardPercolateService();
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package org.elasticsearch.index.shard.service;
|
||||
|
||||
import com.google.common.base.Charsets;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.index.CheckIndex;
|
||||
import org.apache.lucene.search.Filter;
|
||||
import org.apache.lucene.search.Query;
|
||||
|
@ -75,6 +76,8 @@ import org.elasticsearch.index.warmer.WarmerStats;
|
|||
import org.elasticsearch.indices.IndicesLifecycle;
|
||||
import org.elasticsearch.indices.InternalIndicesLifecycle;
|
||||
import org.elasticsearch.indices.recovery.RecoveryStatus;
|
||||
import org.elasticsearch.search.suggest.completion.Completion090PostingsFormat;
|
||||
import org.elasticsearch.search.suggest.completion.CompletionStats;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -506,6 +509,25 @@ public class InternalIndexShard extends AbstractIndexShardComponent implements I
|
|||
return shardIdCache.stats();
|
||||
}
|
||||
|
||||
@Override
|
||||
public CompletionStats completionStats(String... fields) {
|
||||
CompletionStats completionStats = new CompletionStats();
|
||||
|
||||
Engine.Searcher searcher = engine().searcher();
|
||||
|
||||
try {
|
||||
PostingsFormat postingsFormat = this.codecService.postingsFormatService().get(Completion090PostingsFormat.CODEC_NAME).get();
|
||||
if (postingsFormat instanceof Completion090PostingsFormat) {
|
||||
Completion090PostingsFormat completionPostingsFormat = (Completion090PostingsFormat) postingsFormat;
|
||||
completionStats.add(completionPostingsFormat.completionStats(searcher().reader(), fields));
|
||||
}
|
||||
} finally {
|
||||
searcher.release();
|
||||
}
|
||||
|
||||
return completionStats;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void flush(Engine.Flush flush) throws ElasticSearchException {
|
||||
// we allows flush while recovering, since we allow for operations to happen
|
||||
|
|
|
@ -76,6 +76,7 @@ import org.elasticsearch.indices.recovery.RecoverySettings;
|
|||
import org.elasticsearch.indices.store.IndicesStore;
|
||||
import org.elasticsearch.plugins.IndexPluginsModule;
|
||||
import org.elasticsearch.plugins.PluginsService;
|
||||
import org.elasticsearch.search.suggest.completion.CompletionStats;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
@ -242,6 +243,9 @@ public class InternalIndicesService extends AbstractLifecycleComponent<IndicesSe
|
|||
case Percolate:
|
||||
stats.percolate = new PercolateStats();
|
||||
break;
|
||||
case Completion:
|
||||
stats.completion = new CompletionStats();
|
||||
break;
|
||||
default:
|
||||
throw new IllegalStateException("Unknown Flag: " + flag);
|
||||
}
|
||||
|
@ -291,6 +295,9 @@ public class InternalIndicesService extends AbstractLifecycleComponent<IndicesSe
|
|||
case Percolate:
|
||||
stats.percolate.add(indexShard.shardPercolateService().stats());
|
||||
break;
|
||||
case Completion:
|
||||
stats.completion.add(indexShard.completionStats(flags.completionDataFields()));
|
||||
break;
|
||||
default:
|
||||
throw new IllegalStateException("Unknown Flag: " + flag);
|
||||
}
|
||||
|
|
|
@ -89,6 +89,11 @@ public class RestIndicesStatsAction extends BaseRestHandler {
|
|||
controller.registerHandler(GET, "/_stats/fielddata/{fields}", new RestFieldDataStatsHandler());
|
||||
controller.registerHandler(GET, "/{index}/_stats/fielddata/{fields}", new RestFieldDataStatsHandler());
|
||||
|
||||
controller.registerHandler(GET, "/_stats/completion", new RestCompletionStatsHandler());
|
||||
controller.registerHandler(GET, "/{index}/_stats/completion", new RestCompletionStatsHandler());
|
||||
controller.registerHandler(GET, "/_stats/completion/{fields}", new RestCompletionStatsHandler());
|
||||
controller.registerHandler(GET, "/{index}/_stats/completion/{fields}", new RestCompletionStatsHandler());
|
||||
|
||||
controller.registerHandler(GET, "/_stats/percolate", new RestPercolateStatsHandler());
|
||||
controller.registerHandler(GET, "/{index}/_stats/percolate", new RestPercolateStatsHandler());
|
||||
}
|
||||
|
@ -585,6 +590,45 @@ public class RestIndicesStatsAction extends BaseRestHandler {
|
|||
}
|
||||
}
|
||||
|
||||
class RestCompletionStatsHandler implements RestHandler {
|
||||
|
||||
@Override
|
||||
public void handleRequest(final RestRequest request, final RestChannel channel) {
|
||||
IndicesStatsRequest indicesStatsRequest = new IndicesStatsRequest();
|
||||
indicesStatsRequest.listenerThreaded(false);
|
||||
indicesStatsRequest.clear().completion(true);
|
||||
indicesStatsRequest.indices(splitIndices(request.param("index")));
|
||||
indicesStatsRequest.types(splitTypes(request.param("types")));
|
||||
indicesStatsRequest.completionFields(request.paramAsStringArray("fields", null));
|
||||
|
||||
client.admin().indices().stats(indicesStatsRequest, new ActionListener<IndicesStatsResponse>() {
|
||||
@Override
|
||||
public void onResponse(IndicesStatsResponse response) {
|
||||
try {
|
||||
XContentBuilder builder = RestXContentBuilder.restContentBuilder(request);
|
||||
builder.startObject();
|
||||
builder.field("ok", true);
|
||||
buildBroadcastShardsHeader(builder, response);
|
||||
response.toXContent(builder, request);
|
||||
builder.endObject();
|
||||
channel.sendResponse(new XContentRestResponse(request, OK, builder));
|
||||
} catch (Throwable e) {
|
||||
onFailure(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(Throwable e) {
|
||||
try {
|
||||
channel.sendResponse(new XContentThrowableRestResponse(request, e));
|
||||
} catch (IOException e1) {
|
||||
logger.error("Failed to send failure response", e1);
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
class RestRefreshStatsHandler implements RestHandler {
|
||||
|
||||
@Override
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
|
||||
package org.elasticsearch.search.suggest.completion;
|
||||
|
||||
import gnu.trove.map.hash.TObjectLongHashMap;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.codecs.*;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
|
@ -32,6 +33,7 @@ import org.apache.lucene.util.IOUtils;
|
|||
import org.apache.lucene.util.IntsRef;
|
||||
import org.apache.lucene.util.fst.*;
|
||||
import org.apache.lucene.util.fst.PairOutputs.Pair;
|
||||
import org.elasticsearch.common.regex.Regex;
|
||||
import org.elasticsearch.index.mapper.FieldMapper;
|
||||
import org.elasticsearch.search.suggest.completion.Completion090PostingsFormat.CompletionLookupProvider;
|
||||
import org.elasticsearch.search.suggest.completion.Completion090PostingsFormat.LookupFactory;
|
||||
|
@ -238,6 +240,31 @@ public class AnalyzingCompletionLookupProvider extends CompletionLookupProvider
|
|||
suggester.setPreservePositionIncrements(analyzingSuggestHolder.preservePositionIncrements);
|
||||
return suggester;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CompletionStats stats(String ... fields) {
|
||||
long sizeInBytes = 0;
|
||||
TObjectLongHashMap<String> completionFields = null;
|
||||
if (fields != null && fields.length > 0) {
|
||||
completionFields = new TObjectLongHashMap<String>(fields.length);
|
||||
}
|
||||
|
||||
for (Map.Entry<String, AnalyzingSuggestHolder> entry: lookupMap.entrySet()) {
|
||||
sizeInBytes += entry.getValue().fst.sizeInBytes();
|
||||
|
||||
if (fields == null || fields.length == 0) continue;
|
||||
|
||||
for (String field : fields) {
|
||||
// support for getting fields by regex as in fielddata
|
||||
if (Regex.simpleMatch(field, entry.getKey())) {
|
||||
long fstSize = entry.getValue().fst.sizeInBytes();
|
||||
completionFields.adjustOrPutValue(field, fstSize, fstSize);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return new CompletionStats(sizeInBytes, completionFields);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
|
|
|
@ -25,10 +25,15 @@ import org.apache.lucene.index.*;
|
|||
import org.apache.lucene.index.FilterAtomicReader.FilterTerms;
|
||||
import org.apache.lucene.search.suggest.Lookup;
|
||||
import org.apache.lucene.store.IOContext.Context;
|
||||
import org.apache.lucene.store.*;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.store.InputStreamDataInput;
|
||||
import org.apache.lucene.store.OutputStreamDataOutput;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.elasticsearch.ElasticSearchIllegalStateException;
|
||||
import org.elasticsearch.common.logging.ESLogger;
|
||||
import org.elasticsearch.common.logging.Loggers;
|
||||
import org.elasticsearch.index.mapper.FieldMapper;
|
||||
import org.elasticsearch.search.suggest.completion.CompletionTokenStream.ToFiniteStrings;
|
||||
|
||||
|
@ -47,7 +52,6 @@ import java.util.Map;
|
|||
* handle all the merge operations. The auxiliary suggest FST data structure is
|
||||
* only loaded if a FieldsProducer is requested for reading, for merging it uses
|
||||
* the low memory delegate postings format.
|
||||
*
|
||||
*/
|
||||
public class Completion090PostingsFormat extends PostingsFormat {
|
||||
|
||||
|
@ -55,10 +59,12 @@ public class Completion090PostingsFormat extends PostingsFormat {
|
|||
public static final int SUGGEST_CODEC_VERSION = 1;
|
||||
public static final String EXTENSION = "cmp";
|
||||
|
||||
private final static ESLogger logger = Loggers.getLogger(Completion090PostingsFormat.class);
|
||||
private PostingsFormat delegatePostingsFormat;
|
||||
private final static Map<String, CompletionLookupProvider> providers;
|
||||
private CompletionLookupProvider writeProvider;
|
||||
|
||||
|
||||
static {
|
||||
final CompletionLookupProvider provider = new AnalyzingCompletionLookupProvider(true, false, true, false);
|
||||
final Builder<String, CompletionLookupProvider> builder = ImmutableMap.builder();
|
||||
|
@ -81,13 +87,13 @@ public class Completion090PostingsFormat extends PostingsFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public SuggestFieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
|
||||
public CompletionFieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
|
||||
if (delegatePostingsFormat == null) {
|
||||
throw new UnsupportedOperationException("Error - " + getClass().getName()
|
||||
+ " has been constructed without a choice of PostingsFormat");
|
||||
}
|
||||
assert writeProvider != null;
|
||||
return new SuggestFieldsConsumer(state);
|
||||
return new CompletionFieldsConsumer(state);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -95,12 +101,12 @@ public class Completion090PostingsFormat extends PostingsFormat {
|
|||
return new CompletionFieldsProducer(state);
|
||||
}
|
||||
|
||||
private class SuggestFieldsConsumer extends FieldsConsumer {
|
||||
private class CompletionFieldsConsumer extends FieldsConsumer {
|
||||
|
||||
private FieldsConsumer delegatesFieldsConsumer;
|
||||
private FieldsConsumer suggestFieldsConsumer;
|
||||
|
||||
public SuggestFieldsConsumer(SegmentWriteState state) throws IOException {
|
||||
public CompletionFieldsConsumer(SegmentWriteState state) throws IOException {
|
||||
this.delegatesFieldsConsumer = delegatePostingsFormat.fieldsConsumer(state);
|
||||
String suggestFSTFile = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, EXTENSION);
|
||||
IndexOutput output = null;
|
||||
|
@ -274,6 +280,10 @@ public class Completion090PostingsFormat extends PostingsFormat {
|
|||
public Lookup getLookup(FieldMapper<?> mapper, CompletionSuggestionContext suggestionContext) {
|
||||
return lookup.getLookup(mapper, suggestionContext);
|
||||
}
|
||||
|
||||
public CompletionStats stats(String ... fields) {
|
||||
return lookup.stats(fields);
|
||||
}
|
||||
}
|
||||
|
||||
public static abstract class CompletionLookupProvider implements PayloadProcessor, ToFiniteStrings {
|
||||
|
@ -326,7 +336,28 @@ public class Completion090PostingsFormat extends PostingsFormat {
|
|||
}
|
||||
}
|
||||
|
||||
public CompletionStats completionStats(IndexReader indexReader, String ... fields) {
|
||||
CompletionStats completionStats = new CompletionStats();
|
||||
for (AtomicReaderContext atomicReaderContext : indexReader.leaves()) {
|
||||
AtomicReader atomicReader = atomicReaderContext.reader();
|
||||
try {
|
||||
for (String fieldName : atomicReader.fields()) {
|
||||
Terms terms = atomicReader.fields().terms(fieldName);
|
||||
if (terms instanceof CompletionTerms) {
|
||||
CompletionTerms completionTerms = (CompletionTerms) terms;
|
||||
completionStats.add(completionTerms.stats(fields));
|
||||
}
|
||||
}
|
||||
} catch (IOException e) {
|
||||
logger.error("Could not get completion stats: {}", e, e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
return completionStats;
|
||||
}
|
||||
|
||||
public static abstract class LookupFactory {
|
||||
public abstract Lookup getLookup(FieldMapper<?> mapper, CompletionSuggestionContext suggestionContext);
|
||||
public abstract CompletionStats stats(String ... fields);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,138 @@
|
|||
/*
|
||||
* Licensed to ElasticSearch and Shay Banon 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.search.suggest.completion;
|
||||
|
||||
import gnu.trove.iterator.TObjectLongIterator;
|
||||
import gnu.trove.map.hash.TObjectLongHashMap;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
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.ByteSizeValue;
|
||||
import org.elasticsearch.common.xcontent.ToXContent;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilderString;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class CompletionStats implements Streamable, ToXContent {
|
||||
|
||||
private long sizeInBytes;
|
||||
|
||||
@Nullable
|
||||
private TObjectLongHashMap<String> fields;
|
||||
|
||||
public CompletionStats() {
|
||||
}
|
||||
|
||||
public CompletionStats(long size, @Nullable TObjectLongHashMap<String> fields) {
|
||||
this.sizeInBytes = size;
|
||||
this.fields = fields;
|
||||
}
|
||||
|
||||
public long getSizeInBytes() {
|
||||
return sizeInBytes;
|
||||
}
|
||||
|
||||
public ByteSizeValue getSize() {
|
||||
return new ByteSizeValue(sizeInBytes);
|
||||
}
|
||||
|
||||
public TObjectLongHashMap<String> getFields() {
|
||||
return fields;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFrom(StreamInput in) throws IOException {
|
||||
sizeInBytes = in.readVLong();
|
||||
if (in.readBoolean()) {
|
||||
int size = in.readVInt();
|
||||
fields = new TObjectLongHashMap<String>(size);
|
||||
for (int i = 0; i < size; i++) {
|
||||
fields.put(in.readString(), in.readVLong());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
out.writeVLong(sizeInBytes);
|
||||
if (fields == null) {
|
||||
out.writeBoolean(false);
|
||||
} else {
|
||||
out.writeBoolean(true);
|
||||
out.writeVInt(fields.size());
|
||||
for (TObjectLongIterator<String> it = fields.iterator(); it.hasNext(); ) {
|
||||
it.advance();
|
||||
out.writeString(it.key());
|
||||
out.writeVLong(it.value());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject(Fields.COMPLETION);
|
||||
builder.byteSizeField(Fields.SIZE, Fields.SIZE_IN_BYTES, sizeInBytes);
|
||||
if (fields != null) {
|
||||
builder.startObject(Fields.FIELDS);
|
||||
for (TObjectLongIterator<String> it = fields.iterator(); it.hasNext(); ) {
|
||||
it.advance();
|
||||
builder.startObject(it.key(), XContentBuilder.FieldCaseConversion.NONE);
|
||||
builder.byteSizeField(Fields.SIZE, Fields.SIZE_IN_BYTES, it.value());
|
||||
builder.endObject();
|
||||
}
|
||||
builder.endObject();
|
||||
}
|
||||
builder.endObject();
|
||||
return builder;
|
||||
}
|
||||
|
||||
public static CompletionStats readCompletionStats(StreamInput in) throws IOException {
|
||||
CompletionStats stats = new CompletionStats();
|
||||
stats.readFrom(in);
|
||||
return stats;
|
||||
}
|
||||
|
||||
static final class Fields {
|
||||
static final XContentBuilderString COMPLETION = new XContentBuilderString("completion");
|
||||
static final XContentBuilderString SIZE_IN_BYTES = new XContentBuilderString("size_in_bytes");
|
||||
static final XContentBuilderString SIZE = new XContentBuilderString("size");
|
||||
static final XContentBuilderString FIELDS = new XContentBuilderString("fields");
|
||||
}
|
||||
|
||||
public void add(CompletionStats completion) {
|
||||
if (completion == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
sizeInBytes += completion.getSizeInBytes();
|
||||
|
||||
if (completion.fields != null) {
|
||||
if (fields == null) fields = new TObjectLongHashMap<String>();
|
||||
for (TObjectLongIterator<String> it = completion.fields.iterator(); it.hasNext(); ) {
|
||||
it.advance();
|
||||
fields.adjustOrPutValue(it.key(), it.value(), it.value());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -309,7 +309,7 @@ public class SimpleIndexStatsTests extends AbstractNodesTests {
|
|||
@Test
|
||||
public void testFlagOrdinalOrder() {
|
||||
Flag[] flags = new Flag[]{Flag.Store, Flag.Indexing, Flag.Get, Flag.Search, Flag.Merge, Flag.Flush, Flag.Refresh,
|
||||
Flag.FilterCache, Flag.IdCache, Flag.FieldData, Flag.Docs, Flag.Warmer, Flag.Percolate};
|
||||
Flag.FilterCache, Flag.IdCache, Flag.FieldData, Flag.Docs, Flag.Warmer, Flag.Percolate, Flag.Completion};
|
||||
|
||||
assertThat(flags.length, equalTo(Flag.values().length));
|
||||
for (int i = 0; i < flags.length; i++) {
|
||||
|
@ -358,6 +358,9 @@ public class SimpleIndexStatsTests extends AbstractNodesTests {
|
|||
case Percolate:
|
||||
builder.setPercolate(set);
|
||||
break;
|
||||
case Completion:
|
||||
builder.setCompletion(set);
|
||||
break;
|
||||
default:
|
||||
assert false : "new flag? " + flag;
|
||||
break;
|
||||
|
@ -392,6 +395,8 @@ public class SimpleIndexStatsTests extends AbstractNodesTests {
|
|||
return response.getWarmer() != null;
|
||||
case Percolate:
|
||||
return response.getPercolate() != null;
|
||||
case Completion:
|
||||
return response.getCompletion() != null;
|
||||
default:
|
||||
assert false : "new flag? " + flag;
|
||||
return false;
|
||||
|
|
|
@ -22,6 +22,7 @@ import com.carrotsearch.randomizedtesting.generators.RandomStrings;
|
|||
import com.google.common.collect.Lists;
|
||||
import org.elasticsearch.action.admin.cluster.health.ClusterHealthStatus;
|
||||
import org.elasticsearch.action.admin.indices.mapping.put.PutMappingResponse;
|
||||
import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse;
|
||||
import org.elasticsearch.action.index.IndexRequestBuilder;
|
||||
import org.elasticsearch.action.suggest.SuggestResponse;
|
||||
import org.elasticsearch.common.settings.ImmutableSettings;
|
||||
|
@ -30,6 +31,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
|
|||
import org.elasticsearch.common.xcontent.json.JsonXContent;
|
||||
import org.elasticsearch.index.mapper.MapperException;
|
||||
import org.elasticsearch.search.suggest.Suggest;
|
||||
import org.elasticsearch.search.suggest.completion.CompletionStats;
|
||||
import org.elasticsearch.search.suggest.completion.CompletionSuggestion;
|
||||
import org.elasticsearch.search.suggest.completion.CompletionSuggestionBuilder;
|
||||
import org.elasticsearch.search.suggest.completion.CompletionSuggestionFuzzyBuilder;
|
||||
|
@ -219,7 +221,7 @@ public class CompletionSuggestSearchTests extends AbstractSharedClusterTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testThatMultipleInputsAreSuppored() throws Exception {
|
||||
public void testThatMultipleInputsAreSupported() throws Exception {
|
||||
createIndexAndMapping("simple", "simple", false, false, true);
|
||||
|
||||
client().prepareIndex(INDEX, TYPE, "1").setSource(jsonBuilder()
|
||||
|
@ -451,6 +453,49 @@ public class CompletionSuggestSearchTests extends AbstractSharedClusterTest {
|
|||
assertSuggestions(suggestResponse, false, "foo", "Nirvana");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testThatStatsAreWorking() throws Exception {
|
||||
String otherField = "testOtherField";
|
||||
|
||||
client().admin().indices().prepareDelete().get();
|
||||
client().admin().indices().prepareCreate(INDEX)
|
||||
.setSettings(createDefaultSettings())
|
||||
.get();
|
||||
|
||||
PutMappingResponse putMappingResponse = client().admin().indices().preparePutMapping(INDEX).setType(TYPE).setSource(jsonBuilder().startObject()
|
||||
.startObject(TYPE).startObject("properties")
|
||||
.startObject(FIELD)
|
||||
.field("type", "completion").field("analyzer", "simple")
|
||||
.endObject()
|
||||
.startObject(otherField)
|
||||
.field("type", "completion").field("analyzer", "simple")
|
||||
.endObject()
|
||||
.endObject().endObject().endObject())
|
||||
.get();
|
||||
assertThat(putMappingResponse.isAcknowledged(), is(true));
|
||||
|
||||
// Index two entities
|
||||
client().prepareIndex(INDEX, TYPE, "1").setRefresh(true).setSource(jsonBuilder().startObject().field(FIELD, "Foo Fighters").field(otherField, "WHATEVER").endObject()).get();
|
||||
client().prepareIndex(INDEX, TYPE, "2").setRefresh(true).setSource(jsonBuilder().startObject().field(FIELD, "Bar Fighters").field(otherField, "WHATEVER2").endObject()).get();
|
||||
|
||||
// Get all stats
|
||||
IndicesStatsResponse indicesStatsResponse = client().admin().indices().prepareStats(INDEX).setIndices(INDEX).setCompletion(true).get();
|
||||
CompletionStats completionStats = indicesStatsResponse.getIndex(INDEX).getPrimaries().completion;
|
||||
assertThat(completionStats, notNullValue());
|
||||
long totalSizeInBytes = completionStats.getSizeInBytes();
|
||||
assertThat(totalSizeInBytes, is(greaterThan(0L)));
|
||||
|
||||
IndicesStatsResponse singleFieldStats = client().admin().indices().prepareStats(INDEX).setIndices(INDEX).setCompletion(true).setCompletionFields(FIELD).get();
|
||||
long singleFieldSizeInBytes = singleFieldStats.getIndex(INDEX).getPrimaries().completion.getFields().get(FIELD);
|
||||
IndicesStatsResponse otherFieldStats = client().admin().indices().prepareStats(INDEX).setIndices(INDEX).setCompletion(true).setCompletionFields(otherField).get();
|
||||
long otherFieldSizeInBytes = otherFieldStats.getIndex(INDEX).getPrimaries().completion.getFields().get(otherField);
|
||||
assertThat(singleFieldSizeInBytes + otherFieldSizeInBytes, is(totalSizeInBytes));
|
||||
|
||||
// regexes
|
||||
IndicesStatsResponse regexFieldStats = client().admin().indices().prepareStats(INDEX).setIndices(INDEX).setCompletion(true).setCompletionFields("test*").get();
|
||||
long regexSizeInBytes = regexFieldStats.getIndex(INDEX).getPrimaries().completion.getFields().get("test*");
|
||||
assertThat(regexSizeInBytes, is(totalSizeInBytes));
|
||||
}
|
||||
|
||||
public void assertSuggestions(String suggestion, String ... suggestions) {
|
||||
String suggestionName = RandomStrings.randomAsciiOfLength(new Random(), 10);
|
||||
|
|
Loading…
Reference in New Issue