Cleanup random stats serialization code (#22223)

Some of our stats serialization code duplicates complicated seriazliation logic
or could use existing building blocks from StreamOutput/Input. This commit
cleans up some of the serialization code.
This commit is contained in:
Simon Willnauer 2016-12-17 11:45:55 +01:00 committed by GitHub
parent efe5a75d26
commit 0b338bf523
29 changed files with 488 additions and 477 deletions

View File

@ -46,6 +46,9 @@ import org.elasticsearch.indices.IndicesQueryCache;
import org.elasticsearch.search.suggest.completion.CompletionStats;
import java.io.IOException;
import java.util.Arrays;
import java.util.Objects;
import java.util.stream.Stream;
public class CommonStats implements Writeable, ToXContent {
@ -225,45 +228,19 @@ public class CommonStats implements Writeable, ToXContent {
}
public CommonStats(StreamInput in) throws IOException {
if (in.readBoolean()) {
docs = DocsStats.readDocStats(in);
}
if (in.readBoolean()) {
store = StoreStats.readStoreStats(in);
}
if (in.readBoolean()) {
indexing = IndexingStats.readIndexingStats(in);
}
if (in.readBoolean()) {
get = GetStats.readGetStats(in);
}
if (in.readBoolean()) {
search = SearchStats.readSearchStats(in);
}
if (in.readBoolean()) {
merge = MergeStats.readMergeStats(in);
}
if (in.readBoolean()) {
refresh = RefreshStats.readRefreshStats(in);
}
if (in.readBoolean()) {
flush = FlushStats.readFlushStats(in);
}
if (in.readBoolean()) {
warmer = WarmerStats.readWarmerStats(in);
}
if (in.readBoolean()) {
queryCache = QueryCacheStats.readQueryCacheStats(in);
}
if (in.readBoolean()) {
fieldData = FieldDataStats.readFieldDataStats(in);
}
if (in.readBoolean()) {
completion = CompletionStats.readCompletionStats(in);
}
if (in.readBoolean()) {
segments = SegmentsStats.readSegmentsStats(in);
}
docs = in.readOptionalStreamable(DocsStats::new);
store = in.readOptionalStreamable(StoreStats::new);
indexing = in.readOptionalStreamable(IndexingStats::new);
get = in.readOptionalStreamable(GetStats::new);
search = in.readOptionalStreamable(SearchStats::new);
merge = in.readOptionalStreamable(MergeStats::new);
refresh = in.readOptionalStreamable(RefreshStats::new);
flush = in.readOptionalStreamable(FlushStats::new);
warmer = in.readOptionalStreamable(WarmerStats::new);
queryCache = in.readOptionalStreamable(QueryCacheStats::new);
fieldData = in.readOptionalStreamable(FieldDataStats::new);
completion = in.readOptionalStreamable(CompletionStats::new);
segments = in.readOptionalStreamable(SegmentsStats::new);
translog = in.readOptionalStreamable(TranslogStats::new);
requestCache = in.readOptionalStreamable(RequestCacheStats::new);
recoveryStats = in.readOptionalStreamable(RecoveryStats::new);
@ -271,84 +248,19 @@ public class CommonStats implements Writeable, ToXContent {
@Override
public void writeTo(StreamOutput out) throws IOException {
if (docs == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
docs.writeTo(out);
}
if (store == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
store.writeTo(out);
}
if (indexing == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
indexing.writeTo(out);
}
if (get == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
get.writeTo(out);
}
if (search == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
search.writeTo(out);
}
if (merge == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
merge.writeTo(out);
}
if (refresh == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
refresh.writeTo(out);
}
if (flush == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
flush.writeTo(out);
}
if (warmer == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
warmer.writeTo(out);
}
if (queryCache == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
queryCache.writeTo(out);
}
if (fieldData == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
fieldData.writeTo(out);
}
if (completion == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
completion.writeTo(out);
}
if (segments == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
segments.writeTo(out);
}
out.writeOptionalStreamable(docs);
out.writeOptionalStreamable(store);
out.writeOptionalStreamable(indexing);
out.writeOptionalStreamable(get);
out.writeOptionalStreamable(search);
out.writeOptionalStreamable(merge);
out.writeOptionalStreamable(refresh);
out.writeOptionalStreamable(flush);
out.writeOptionalStreamable(warmer);
out.writeOptionalStreamable(queryCache);
out.writeOptionalStreamable(fieldData);
out.writeOptionalStreamable(completion);
out.writeOptionalStreamable(segments);
out.writeOptionalStreamable(translog);
out.writeOptionalStreamable(requestCache);
out.writeOptionalStreamable(recoveryStats);
@ -590,53 +502,12 @@ public class CommonStats implements Writeable, ToXContent {
// note, requires a wrapping object
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
if (docs != null) {
docs.toXContent(builder, params);
}
if (store != null) {
store.toXContent(builder, params);
}
if (indexing != null) {
indexing.toXContent(builder, params);
}
if (get != null) {
get.toXContent(builder, params);
}
if (search != null) {
search.toXContent(builder, params);
}
if (merge != null) {
merge.toXContent(builder, params);
}
if (refresh != null) {
refresh.toXContent(builder, params);
}
if (flush != null) {
flush.toXContent(builder, params);
}
if (warmer != null) {
warmer.toXContent(builder, params);
}
if (queryCache != null) {
queryCache.toXContent(builder, params);
}
if (fieldData != null) {
fieldData.toXContent(builder, params);
}
if (completion != null) {
completion.toXContent(builder, params);
}
if (segments != null) {
segments.toXContent(builder, params);
}
if (translog != null) {
translog.toXContent(builder, params);
}
if (requestCache != null) {
requestCache.toXContent(builder, params);
}
if (recoveryStats != null) {
recoveryStats.toXContent(builder, params);
final Stream<ToXContent> stream = Arrays.stream(new ToXContent[] {
docs, store, indexing, get, search, merge, refresh, flush, warmer, queryCache,
fieldData, completion, segments, translog, requestCache, recoveryStats})
.filter(Objects::nonNull);
for (ToXContent toXContent : ((Iterable<ToXContent>)stream::iterator)) {
toXContent.toXContent(builder, params);
}
return builder;
}

View File

@ -135,19 +135,13 @@ public class IndicesStatsResponse extends BroadcastResponse implements ToXConten
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
shards = new ShardStats[in.readVInt()];
for (int i = 0; i < shards.length; i++) {
shards[i] = ShardStats.readShardStats(in);
}
shards = in.readArray(ShardStats::readShardStats, (size) -> new ShardStats[size]);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeVInt(shards.length);
for (ShardStats shard : shards) {
shard.writeTo(out);
}
out.writeArray(shards);
}
@Override

View File

@ -24,6 +24,7 @@ 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.io.stream.Writeable;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.engine.CommitStats;
@ -32,7 +33,7 @@ import org.elasticsearch.index.shard.ShardPath;
import java.io.IOException;
public class ShardStats implements Streamable, ToXContent {
public class ShardStats implements Streamable, Writeable, ToXContent {
private ShardRouting shardRouting;
private CommonStats commonStats;
@Nullable

View File

@ -505,11 +505,7 @@ public abstract class TransportBroadcastByNodeAction<Request extends BroadcastRe
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
indicesLevelRequest = readRequestFrom(in);
int size = in.readVInt();
shards = new ArrayList<>(size);
for (int i = 0; i < size; i++) {
shards.add(new ShardRouting(in));
}
shards = in.readList(ShardRouting::new);
nodeId = in.readString();
}
@ -517,11 +513,7 @@ public abstract class TransportBroadcastByNodeAction<Request extends BroadcastRe
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
indicesLevelRequest.writeTo(out);
int size = shards.size();
out.writeVInt(size);
for (int i = 0; i < size; i++) {
shards.get(i).writeTo(out);
}
out.writeList(shards);
out.writeString(nodeId);
}
}
@ -566,18 +558,9 @@ public abstract class TransportBroadcastByNodeAction<Request extends BroadcastRe
super.readFrom(in);
nodeId = in.readString();
totalShards = in.readVInt();
int resultsSize = in.readVInt();
results = new ArrayList<>(resultsSize);
for (; resultsSize > 0; resultsSize--) {
final ShardOperationResult result = in.readBoolean() ? readShardResult(in) : null;
results.add(result);
}
results = in.readList((stream) -> stream.readBoolean() ? readShardResult(stream) : null);
if (in.readBoolean()) {
int failureShards = in.readVInt();
exceptions = new ArrayList<>(failureShards);
for (int i = 0; i < failureShards; i++) {
exceptions.add(new BroadcastShardOperationFailedException(in));
}
exceptions = in.readList(BroadcastShardOperationFailedException::new);
} else {
exceptions = null;
}
@ -594,11 +577,7 @@ public abstract class TransportBroadcastByNodeAction<Request extends BroadcastRe
}
out.writeBoolean(exceptions != null);
if (exceptions != null) {
int failureShards = exceptions.size();
out.writeVInt(failureShards);
for (int i = 0; i < failureShards; i++) {
exceptions.get(i).writeTo(out);
}
out.writeList(exceptions);
}
}
}

View File

@ -0,0 +1,132 @@
/*
* 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.common;
import com.carrotsearch.hppc.ObjectLongHashMap;
import com.carrotsearch.hppc.cursors.ObjectLongCursor;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.xcontent.XContentBuilder;
import java.io.IOException;
import java.util.Iterator;
import java.util.Objects;
/**
* A reusable class to encode <tt>field -&gt; memory size</tt> mappings
*/
public final class FieldMemoryStats implements Writeable, Iterable<ObjectLongCursor<String>>{
private final ObjectLongHashMap<String> stats;
/**
* Creates a new FieldMemoryStats instance
*/
public FieldMemoryStats(ObjectLongHashMap<String> stats) {
this.stats = Objects.requireNonNull(stats, "status must be non-null");
assert !stats.containsKey(null);
}
/**
* Creates a new FieldMemoryStats instance from a stream
*/
public FieldMemoryStats(StreamInput input) throws IOException {
int size = input.readVInt();
stats = new ObjectLongHashMap<>(size);
for (int i = 0; i < size; i++) {
stats.put(input.readString(), input.readVLong());
}
}
/**
* Adds / merges the given field memory stats into this stats instance
*/
public void add(FieldMemoryStats fieldMemoryStats) {
for (ObjectLongCursor<String> entry : fieldMemoryStats.stats) {
stats.addTo(entry.key, entry.value);
}
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeVInt(stats.size());
for (ObjectLongCursor<String> entry : stats) {
out.writeString(entry.key);
out.writeVLong(entry.value);
}
}
/**
* Generates x-content into the given builder for each of the fields in this stats instance
* @param builder the builder to generated on
* @param key the top level key for this stats object
* @param rawKey the raw byte key for each of the fields byte sizes
* @param readableKey the readable key for each of the fields byte sizes
*/
public void toXContent(XContentBuilder builder, String key, String rawKey, String readableKey) throws IOException {
builder.startObject(key);
for (ObjectLongCursor<String> entry : stats) {
builder.startObject(entry.key);
builder.byteSizeField(rawKey, readableKey, entry.value);
builder.endObject();
}
builder.endObject();
}
/**
* Creates a deep copy of this stats instance
*/
public FieldMemoryStats copy() {
return new FieldMemoryStats(stats.clone());
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
FieldMemoryStats that = (FieldMemoryStats) o;
return Objects.equals(stats, that.stats);
}
@Override
public int hashCode() {
return Objects.hash(stats);
}
@Override
public Iterator<ObjectLongCursor<String>> iterator() {
return stats.iterator();
}
/**
* Returns the fields value in bytes or <code>0</code> if it's not present in the stats
*/
public long get(String field) {
return stats.get(field);
}
/**
* Returns <code>true</code> iff the given field is in the stats
*/
public boolean containsField(String field) {
return stats.containsKey(field);
}
}

View File

@ -467,16 +467,32 @@ public abstract class StreamOutput extends OutputStream {
* @param keyWriter The key writer
* @param valueWriter The value writer
*/
public <K, V> void writeMapOfLists(final Map<K, List<V>> map, final Writer<K> keyWriter, final Writer<V> valueWriter)
public final <K, V> void writeMapOfLists(final Map<K, List<V>> map, final Writer<K> keyWriter, final Writer<V> valueWriter)
throws IOException {
writeVInt(map.size());
for (final Map.Entry<K, List<V>> entry : map.entrySet()) {
keyWriter.write(this, entry.getKey());
writeVInt(entry.getValue().size());
for (final V value : entry.getValue()) {
writeMap(map, keyWriter, (stream, list) -> {
writeVInt(list.size());
for (final V value : list) {
valueWriter.write(this, value);
}
});
}
/**
* Write a {@link Map} of {@code K}-type keys to {@code V}-type.
* <pre><code>
* Map&lt;String, String&gt; map = ...;
* out.writeMap(map, StreamOutput::writeString, StreamOutput::writeString);
* </code></pre>
*
* @param keyWriter The key writer
* @param valueWriter The value writer
*/
public final <K, V> void writeMap(final Map<K, V> map, final Writer<K> keyWriter, final Writer<V> valueWriter)
throws IOException {
writeVInt(map.size());
for (final Map.Entry<K, V> entry : map.entrySet()) {
keyWriter.write(this, entry.getKey());
valueWriter.write(this, entry.getValue());
}
}

View File

@ -106,13 +106,6 @@ public class QueryCacheStats implements Streamable, ToXContent {
return cacheCount - cacheSize;
}
public static QueryCacheStats readQueryCacheStats(StreamInput in) throws IOException {
QueryCacheStats stats = new QueryCacheStats();
stats.readFrom(in);
return stats;
}
@Override
public void readFrom(StreamInput in) throws IOException {
ramBytesUsed = in.readLong();

View File

@ -49,13 +49,6 @@ public final class CommitStats implements Streamable, ToXContent {
}
private CommitStats() {
}
public static CommitStats readCommitStatsFrom(StreamInput in) throws IOException {
CommitStats commitStats = new CommitStats();
commitStats.readFrom(in);
return commitStats;
}
public static CommitStats readOptionalCommitStatsFrom(StreamInput in) throws IOException {

View File

@ -286,12 +286,6 @@ public class SegmentsStats implements Streamable, ToXContent {
return maxUnsafeAutoIdTimestamp;
}
public static SegmentsStats readSegmentsStats(StreamInput in) throws IOException {
SegmentsStats stats = new SegmentsStats();
stats.readFrom(in);
return stats;
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject(Fields.SEGMENTS);
@ -391,10 +385,9 @@ public class SegmentsStats implements Streamable, ToXContent {
out.writeLong(maxUnsafeAutoIdTimestamp);
out.writeVInt(fileSizes.size());
for (Iterator<ObjectObjectCursor<String, Long>> it = fileSizes.iterator(); it.hasNext();) {
ObjectObjectCursor<String, Long> entry = it.next();
for (ObjectObjectCursor<String, Long> entry : fileSizes) {
out.writeString(entry.key);
out.writeLong(entry.value);
out.writeLong(entry.value.longValue());
}
}
}

View File

@ -19,7 +19,7 @@
package org.elasticsearch.index.fielddata;
import com.carrotsearch.hppc.ObjectLongHashMap;
import org.elasticsearch.common.FieldMemoryStats;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
@ -29,19 +29,25 @@ import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import java.io.IOException;
import java.util.Objects;
public class FieldDataStats implements Streamable, ToXContent {
private static final String FIELDDATA = "fielddata";
private static final String MEMORY_SIZE = "memory_size";
private static final String MEMORY_SIZE_IN_BYTES = "memory_size_in_bytes";
private static final String EVICTIONS = "evictions";
private static final String FIELDS = "fields";
long memorySize;
long evictions;
@Nullable
ObjectLongHashMap<String> fields;
FieldMemoryStats fields;
public FieldDataStats() {
}
public FieldDataStats(long memorySize, long evictions, @Nullable ObjectLongHashMap<String> fields) {
public FieldDataStats(long memorySize, long evictions, @Nullable FieldMemoryStats fields) {
this.memorySize = memorySize;
this.evictions = evictions;
this.fields = fields;
@ -52,16 +58,9 @@ public class FieldDataStats implements Streamable, ToXContent {
this.evictions += stats.evictions;
if (stats.fields != null) {
if (fields == null) {
fields = stats.fields.clone();
fields = stats.fields.copy();
} else {
assert !stats.fields.containsKey(null);
final Object[] keys = stats.fields.keys;
final long[] values = stats.fields.values;
for (int i = 0; i < keys.length; i++) {
if (keys[i] != null) {
fields.addTo((String) keys[i], values[i]);
}
}
fields.add(stats.fields);
}
}
}
@ -79,78 +78,48 @@ public class FieldDataStats implements Streamable, ToXContent {
}
@Nullable
public ObjectLongHashMap<String> getFields() {
public FieldMemoryStats getFields() {
return fields;
}
public static FieldDataStats readFieldDataStats(StreamInput in) throws IOException {
FieldDataStats stats = new FieldDataStats();
stats.readFrom(in);
return stats;
}
@Override
public void readFrom(StreamInput in) throws IOException {
memorySize = in.readVLong();
evictions = in.readVLong();
if (in.readBoolean()) {
int size = in.readVInt();
fields = new ObjectLongHashMap<>(size);
for (int i = 0; i < size; i++) {
fields.put(in.readString(), in.readVLong());
}
}
fields = in.readOptionalWriteable(FieldMemoryStats::new);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeVLong(memorySize);
out.writeVLong(evictions);
if (fields == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
out.writeVInt(fields.size());
assert !fields.containsKey(null);
final Object[] keys = fields.keys;
final long[] values = fields.values;
for (int i = 0; i < keys.length; i++) {
if (keys[i] != null) {
out.writeString((String) keys[i]);
out.writeVLong(values[i]);
}
}
}
out.writeOptionalWriteable(fields);
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject(Fields.FIELDDATA);
builder.byteSizeField(Fields.MEMORY_SIZE_IN_BYTES, Fields.MEMORY_SIZE, memorySize);
builder.field(Fields.EVICTIONS, getEvictions());
builder.startObject(FIELDDATA);
builder.byteSizeField(MEMORY_SIZE_IN_BYTES, MEMORY_SIZE, memorySize);
builder.field(EVICTIONS, getEvictions());
if (fields != null) {
builder.startObject(Fields.FIELDS);
assert !fields.containsKey(null);
final Object[] keys = fields.keys;
final long[] values = fields.values;
for (int i = 0; i < keys.length; i++) {
if (keys[i] != null) {
builder.startObject((String) keys[i]);
builder.byteSizeField(Fields.MEMORY_SIZE_IN_BYTES, Fields.MEMORY_SIZE, values[i]);
builder.endObject();
}
}
builder.endObject();
fields.toXContent(builder, FIELDS, MEMORY_SIZE_IN_BYTES, MEMORY_SIZE);
}
builder.endObject();
return builder;
}
static final class Fields {
static final String FIELDDATA = "fielddata";
static final String MEMORY_SIZE = "memory_size";
static final String MEMORY_SIZE_IN_BYTES = "memory_size_in_bytes";
static final String EVICTIONS = "evictions";
static final String FIELDS = "fields";
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
FieldDataStats that = (FieldDataStats) o;
return memorySize == that.memorySize &&
evictions == that.evictions &&
Objects.equals(fields, that.fields);
}
@Override
public int hashCode() {
return Objects.hash(memorySize, evictions, fields);
}
}

View File

@ -21,6 +21,7 @@ package org.elasticsearch.index.fielddata;
import com.carrotsearch.hppc.ObjectLongHashMap;
import org.apache.lucene.util.Accountable;
import org.elasticsearch.common.FieldMemoryStats;
import org.elasticsearch.common.metrics.CounterMetric;
import org.elasticsearch.common.regex.Regex;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
@ -45,7 +46,8 @@ public class ShardFieldData implements IndexFieldDataCache.Listener {
}
}
}
return new FieldDataStats(totalMetric.count(), evictionsMetric.count(), fieldTotals);
return new FieldDataStats(totalMetric.count(), evictionsMetric.count(), fieldTotals == null ? null :
new FieldMemoryStats(fieldTotals));
}
@Override

View File

@ -81,12 +81,6 @@ public class FlushStats implements Streamable, ToXContent {
return new TimeValue(totalTimeInMillis);
}
public static FlushStats readFlushStats(StreamInput in) throws IOException {
FlushStats flushStats = new FlushStats();
flushStats.readFrom(in);
return flushStats;
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject(Fields.FLUSH);

View File

@ -134,12 +134,6 @@ public class GetStats implements Streamable, ToXContent {
static final String CURRENT = "current";
}
public static GetStats readGetStats(StreamInput in) throws IOException {
GetStats stats = new GetStats();
stats.readFrom(in);
return stats;
}
@Override
public void readFrom(StreamInput in) throws IOException {
existsCount = in.readVLong();

View File

@ -182,12 +182,6 @@ public class MergeStats implements Streamable, ToXContent {
return new ByteSizeValue(currentSizeInBytes);
}
public static MergeStats readMergeStats(StreamInput in) throws IOException {
MergeStats stats = new MergeStats();
stats.readFrom(in);
return stats;
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject(Fields.MERGES);

View File

@ -81,12 +81,6 @@ public class RefreshStats implements Streamable, ToXContent {
return new TimeValue(totalTimeInMillis);
}
public static RefreshStats readRefreshStats(StreamInput in) throws IOException {
RefreshStats refreshStats = new RefreshStats();
refreshStats.readFrom(in);
return refreshStats;
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject(Fields.REFRESH);

View File

@ -19,6 +19,7 @@
package org.elasticsearch.index.search.stats;
import org.elasticsearch.action.support.ToXContentToBytes;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
@ -32,7 +33,7 @@ import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
public class SearchStats implements Streamable, ToXContent {
public class SearchStats extends ToXContentToBytes implements Streamable {
public static class Stats implements Streamable, ToXContent {
@ -338,22 +339,12 @@ public class SearchStats implements Streamable, ToXContent {
static final String SUGGEST_CURRENT = "suggest_current";
}
public static SearchStats readSearchStats(StreamInput in) throws IOException {
SearchStats searchStats = new SearchStats();
searchStats.readFrom(in);
return searchStats;
}
@Override
public void readFrom(StreamInput in) throws IOException {
totalStats = Stats.readStats(in);
openContexts = in.readVLong();
if (in.readBoolean()) {
int size = in.readVInt();
groupStats = new HashMap<>(size);
for (int i = 0; i < size; i++) {
groupStats.put(in.readString(), Stats.readStats(in));
}
groupStats = in.readMap(StreamInput::readString, Stats::readStats);
}
}
@ -365,24 +356,7 @@ public class SearchStats implements Streamable, ToXContent {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
out.writeVInt(groupStats.size());
for (Map.Entry<String, Stats> entry : groupStats.entrySet()) {
out.writeString(entry.getKey());
entry.getValue().writeTo(out);
}
}
}
@Override
public String toString() {
try {
XContentBuilder builder = XContentFactory.jsonBuilder().prettyPrint();
builder.startObject();
toXContent(builder, EMPTY_PARAMS);
builder.endObject();
return builder.string();
} catch (IOException e) {
return "{ \"error\" : \"" + e.getMessage() + "\"}";
out.writeMap(groupStats, StreamOutput::writeString, (stream, stats) -> stats.writeTo(stream));
}
}
}

View File

@ -57,12 +57,6 @@ public class DocsStats implements Streamable, ToXContent {
return this.deleted;
}
public static DocsStats readDocStats(StreamInput in) throws IOException {
DocsStats docsStats = new DocsStats();
docsStats.readFrom(in);
return docsStats;
}
@Override
public void readFrom(StreamInput in) throws IOException {
count = in.readVLong();

View File

@ -143,11 +143,7 @@ public class IndexingStats implements Streamable, ToXContent {
indexCount = in.readVLong();
indexTimeInMillis = in.readVLong();
indexCurrent = in.readVLong();
if(in.getVersion().onOrAfter(Version.V_2_1_0)){
indexFailedCount = in.readVLong();
}
indexFailedCount = in.readVLong();
deleteCount = in.readVLong();
deleteTimeInMillis = in.readVLong();
deleteCurrent = in.readVLong();
@ -161,11 +157,7 @@ public class IndexingStats implements Streamable, ToXContent {
out.writeVLong(indexCount);
out.writeVLong(indexTimeInMillis);
out.writeVLong(indexCurrent);
if(out.getVersion().onOrAfter(Version.V_2_1_0)) {
out.writeVLong(indexFailedCount);
}
out.writeVLong(indexFailedCount);
out.writeVLong(deleteCount);
out.writeVLong(deleteTimeInMillis);
out.writeVLong(deleteCurrent);
@ -283,21 +275,11 @@ public class IndexingStats implements Streamable, ToXContent {
static final String THROTTLED_TIME = "throttle_time";
}
public static IndexingStats readIndexingStats(StreamInput in) throws IOException {
IndexingStats indexingStats = new IndexingStats();
indexingStats.readFrom(in);
return indexingStats;
}
@Override
public void readFrom(StreamInput in) throws IOException {
totalStats = Stats.readStats(in);
if (in.readBoolean()) {
int size = in.readVInt();
typeStats = new HashMap<>(size);
for (int i = 0; i < size; i++) {
typeStats.put(in.readString(), Stats.readStats(in));
}
typeStats = in.readMap(StreamInput::readString, Stats::readStats);
}
}
@ -308,11 +290,7 @@ public class IndexingStats implements Streamable, ToXContent {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
out.writeVInt(typeStats.size());
for (Map.Entry<String, Stats> entry : typeStats.entrySet()) {
out.writeString(entry.getKey());
entry.getValue().writeTo(out);
}
out.writeMap(typeStats, StreamOutput::writeString, (stream, stats) -> stats.writeTo(stream));
}
}
}

View File

@ -65,12 +65,6 @@ public class StoreStats implements Streamable, ToXContent {
return size();
}
public static StoreStats readStoreStats(StreamInput in) throws IOException {
StoreStats store = new StoreStats();
store.readFrom(in);
return store;
}
@Override
public void readFrom(StreamInput in) throws IOException {
sizeInBytes = in.readVLong();

View File

@ -86,12 +86,6 @@ public class WarmerStats implements Streamable, ToXContent {
return new TimeValue(totalTimeInMillis);
}
public static WarmerStats readWarmerStats(StreamInput in) throws IOException {
WarmerStats refreshStats = new WarmerStats();
refreshStats.readFrom(in);
return refreshStats;
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject(Fields.WARMER);

View File

@ -54,7 +54,7 @@ public class IngestStats implements Writeable, ToXContent {
@Override
public void writeTo(StreamOutput out) throws IOException {
totalStats.writeTo(out);
out.writeVLong(statsPerPipeline.size());
out.writeVInt(statsPerPipeline.size());
for (Map.Entry<String, Stats> entry : statsPerPipeline.entrySet()) {
out.writeString(entry.getKey());
entry.getValue().writeTo(out);

View File

@ -27,6 +27,7 @@ import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.Terms;
import org.apache.lucene.search.suggest.document.CompletionTerms;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.common.FieldMemoryStats;
import org.elasticsearch.common.regex.Regex;
import java.io.IOException;
@ -64,6 +65,6 @@ public class CompletionFieldStats {
throw new ElasticsearchException(ioe);
}
}
return new CompletionStats(sizeInBytes, completionFields);
return new CompletionStats(sizeInBytes, completionFields == null ? null : new FieldMemoryStats(completionFields));
}
}

View File

@ -18,7 +18,7 @@
*/
package org.elasticsearch.search.suggest.completion;
import com.carrotsearch.hppc.ObjectLongHashMap;
import org.elasticsearch.common.FieldMemoryStats;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
@ -31,15 +31,19 @@ import java.io.IOException;
public class CompletionStats implements Streamable, ToXContent {
private long sizeInBytes;
private static final String COMPLETION = "completion";
private static final String SIZE_IN_BYTES = "size_in_bytes";
private static final String SIZE = "size";
private static final String FIELDS = "fields";
private long sizeInBytes;
@Nullable
private ObjectLongHashMap<String> fields;
private FieldMemoryStats fields;
public CompletionStats() {
}
public CompletionStats(long size, @Nullable ObjectLongHashMap<String> fields) {
public CompletionStats(long size, @Nullable FieldMemoryStats fields) {
this.sizeInBytes = size;
this.fields = fields;
}
@ -52,98 +56,43 @@ public class CompletionStats implements Streamable, ToXContent {
return new ByteSizeValue(sizeInBytes);
}
public ObjectLongHashMap<String> getFields() {
public FieldMemoryStats getFields() {
return fields;
}
@Override
public void readFrom(StreamInput in) throws IOException {
sizeInBytes = in.readVLong();
if (in.readBoolean()) {
int size = in.readVInt();
fields = new ObjectLongHashMap<>(size);
for (int i = 0; i < size; i++) {
fields.put(in.readString(), in.readVLong());
}
}
fields = in.readOptionalWriteable(FieldMemoryStats::new);
}
@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());
assert !fields.containsKey(null);
final Object[] keys = fields.keys;
final long[] values = fields.values;
for (int i = 0; i < keys.length; i++) {
if (keys[i] != null) {
out.writeString((String) keys[i]);
out.writeVLong(values[i]);
}
}
}
out.writeOptionalWriteable(fields);
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject(Fields.COMPLETION);
builder.byteSizeField(Fields.SIZE_IN_BYTES, Fields.SIZE, sizeInBytes);
builder.startObject(COMPLETION);
builder.byteSizeField(SIZE_IN_BYTES, SIZE, sizeInBytes);
if (fields != null) {
builder.startObject(Fields.FIELDS);
assert !fields.containsKey(null);
final Object[] keys = fields.keys;
final long[] values = fields.values;
for (int i = 0; i < keys.length; i++) {
if (keys[i] != null) {
builder.startObject((String) keys[i]);
builder.byteSizeField(Fields.SIZE_IN_BYTES, Fields.SIZE, values[i]);
builder.endObject();
}
}
builder.endObject();
fields.toXContent(builder, FIELDS, SIZE_IN_BYTES, SIZE);
}
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 String COMPLETION = "completion";
static final String SIZE_IN_BYTES = "size_in_bytes";
static final String SIZE = "size";
static final String FIELDS = "fields";
}
public void add(CompletionStats completion) {
if (completion == null) {
return;
}
sizeInBytes += completion.getSizeInBytes();
if (completion.fields != null) {
if (fields == null) {
fields = completion.fields.clone();
fields = completion.fields.copy();
} else {
assert !completion.fields.containsKey(null);
final Object[] keys = completion.fields.keys;
final long[] values = completion.fields.values;
for (int i = 0; i < keys.length; i++) {
if (keys[i] != null) {
fields.addTo((String) keys[i], values[i]);
}
}
fields.add(completion.fields);
}
}
}

View File

@ -0,0 +1,102 @@
/*
* 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.common;
import com.carrotsearch.hppc.ObjectLongHashMap;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.test.ESTestCase;
import java.io.IOException;
public class FieldMemoryStatsTests extends ESTestCase {
public void testSerialize() throws IOException {
FieldMemoryStats stats = randomFieldMemoryStats();
BytesStreamOutput out = new BytesStreamOutput();
stats.writeTo(out);
StreamInput input = out.bytes().streamInput();
FieldMemoryStats read = new FieldMemoryStats(input);
assertEquals(-1, input.read());
assertEquals(stats, read);
}
public void testHashCodeEquals() {
FieldMemoryStats stats = randomFieldMemoryStats();
assertEquals(stats, stats);
assertEquals(stats.hashCode(), stats.hashCode());
ObjectLongHashMap<String> map1 = new ObjectLongHashMap<>();
map1.put("bar", 1);
FieldMemoryStats stats1 = new FieldMemoryStats(map1);
ObjectLongHashMap<String> map2 = new ObjectLongHashMap<>();
map2.put("foo", 2);
FieldMemoryStats stats2 = new FieldMemoryStats(map2);
ObjectLongHashMap<String> map3 = new ObjectLongHashMap<>();
map3.put("foo", 2);
map3.put("bar", 1);
FieldMemoryStats stats3 = new FieldMemoryStats(map3);
ObjectLongHashMap<String> map4 = new ObjectLongHashMap<>();
map4.put("foo", 2);
map4.put("bar", 1);
FieldMemoryStats stats4 = new FieldMemoryStats(map4);
assertNotEquals(stats1, stats2);
assertNotEquals(stats1, stats3);
assertNotEquals(stats2, stats3);
assertEquals(stats4, stats3);
stats1.add(stats2);
assertEquals(stats1, stats3);
assertEquals(stats1, stats4);
assertEquals(stats1.hashCode(), stats3.hashCode());
}
public void testAdd() {
ObjectLongHashMap<String> map1 = new ObjectLongHashMap<>();
map1.put("bar", 1);
FieldMemoryStats stats1 = new FieldMemoryStats(map1);
ObjectLongHashMap<String> map2 = new ObjectLongHashMap<>();
map2.put("foo", 2);
FieldMemoryStats stats2 = new FieldMemoryStats(map2);
ObjectLongHashMap<String> map3 = new ObjectLongHashMap<>();
map3.put("bar", 1);
FieldMemoryStats stats3 = new FieldMemoryStats(map3);
stats3.add(stats1);
ObjectLongHashMap<String> map4 = new ObjectLongHashMap<>();
map4.put("foo", 2);
map4.put("bar", 2);
FieldMemoryStats stats4 = new FieldMemoryStats(map4);
assertNotEquals(stats3, stats4);
stats3.add(stats2);
assertEquals(stats3, stats4);
}
public static FieldMemoryStats randomFieldMemoryStats() {
ObjectLongHashMap<String> map = new ObjectLongHashMap<>();
int keys = randomIntBetween(1, 1000);
for (int i = 0; i < keys; i++) {
map.put(randomRealisticUnicodeOfCodepointLengthBetween(1, 10), randomPositiveLong());
}
return new FieldMemoryStats(map);
}
}

View File

@ -456,6 +456,22 @@ public class BytesStreamsTests extends ESTestCase {
out.close();
}
public void testWriteMap() throws IOException {
final int size = randomIntBetween(0, 100);
final Map<String, String> expected = new HashMap<>(randomIntBetween(0, 100));
for (int i = 0; i < size; ++i) {
expected.put(randomAsciiOfLength(2), randomAsciiOfLength(5));
}
final BytesStreamOutput out = new BytesStreamOutput();
out.writeMap(expected, StreamOutput::writeString, StreamOutput::writeString);
final StreamInput in = StreamInput.wrap(BytesReference.toBytes(out.bytes()));
final Map<String, String> loaded = in.readMap(StreamInput::readString, StreamInput::readString);
assertThat(loaded.size(), equalTo(expected.size()));
assertThat(expected, equalTo(loaded));
}
public void testWriteMapOfLists() throws IOException {
final int size = randomIntBetween(0, 5);
final Map<String, List<String>> expected = new HashMap<>(size);

View File

@ -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.index.fielddata;
import org.elasticsearch.common.FieldMemoryStats;
import org.elasticsearch.common.FieldMemoryStatsTests;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.test.ESTestCase;
import java.io.IOException;
public class FieldDataStatsTests extends ESTestCase {
public void testSerialize() throws IOException {
FieldMemoryStats map = randomBoolean() ? null : FieldMemoryStatsTests.randomFieldMemoryStats();
FieldDataStats stats = new FieldDataStats(randomPositiveLong(), randomPositiveLong(), map == null ? null :
map);
BytesStreamOutput out = new BytesStreamOutput();
stats.writeTo(out);
FieldDataStats read = new FieldDataStats();
StreamInput input = out.bytes().streamInput();
read.readFrom(input);
assertEquals(-1, input.read());
assertEquals(stats.evictions, read.evictions);
assertEquals(stats.memorySize, read.memorySize);
assertEquals(stats.getFields(), read.getFields());
}
}

View File

@ -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.index.suggest.stats;
import org.elasticsearch.common.FieldMemoryStats;
import org.elasticsearch.common.FieldMemoryStatsTests;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.search.suggest.completion.CompletionStats;
import org.elasticsearch.test.ESTestCase;
import java.io.IOException;
public class CompletionsStatsTests extends ESTestCase {
public void testSerialize() throws IOException {
FieldMemoryStats map = randomBoolean() ? null : FieldMemoryStatsTests.randomFieldMemoryStats();
CompletionStats stats = new CompletionStats(randomPositiveLong(), map == null ? null :
map);
BytesStreamOutput out = new BytesStreamOutput();
stats.writeTo(out);
CompletionStats read = new CompletionStats();
StreamInput input = out.bytes().streamInput();
read.readFrom(input);
assertEquals(-1, input.read());
assertEquals(stats.getSizeInBytes(), read.getSizeInBytes());
assertEquals(stats.getFields(), read.getFields());
}
}

View File

@ -44,7 +44,6 @@ import org.elasticsearch.index.VersionType;
import org.elasticsearch.index.cache.query.QueryCacheStats;
import org.elasticsearch.index.engine.VersionConflictEngineException;
import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.index.store.IndexStore;
import org.elasticsearch.index.translog.Translog;
import org.elasticsearch.indices.IndicesQueryCache;
import org.elasticsearch.indices.IndicesRequestCache;
@ -737,29 +736,29 @@ public class IndexStatsIT extends ESIntegTestCase {
stats = builder.setFieldDataFields("bar").execute().actionGet();
assertThat(stats.getTotal().fieldData.getMemorySizeInBytes(), greaterThan(0L));
assertThat(stats.getTotal().fieldData.getFields().containsKey("bar"), is(true));
assertThat(stats.getTotal().fieldData.getFields().containsField("bar"), is(true));
assertThat(stats.getTotal().fieldData.getFields().get("bar"), greaterThan(0L));
assertThat(stats.getTotal().fieldData.getFields().containsKey("baz"), is(false));
assertThat(stats.getTotal().fieldData.getFields().containsField("baz"), is(false));
stats = builder.setFieldDataFields("bar", "baz").execute().actionGet();
assertThat(stats.getTotal().fieldData.getMemorySizeInBytes(), greaterThan(0L));
assertThat(stats.getTotal().fieldData.getFields().containsKey("bar"), is(true));
assertThat(stats.getTotal().fieldData.getFields().containsField("bar"), is(true));
assertThat(stats.getTotal().fieldData.getFields().get("bar"), greaterThan(0L));
assertThat(stats.getTotal().fieldData.getFields().containsKey("baz"), is(true));
assertThat(stats.getTotal().fieldData.getFields().containsField("baz"), is(true));
assertThat(stats.getTotal().fieldData.getFields().get("baz"), greaterThan(0L));
stats = builder.setFieldDataFields("*").execute().actionGet();
assertThat(stats.getTotal().fieldData.getMemorySizeInBytes(), greaterThan(0L));
assertThat(stats.getTotal().fieldData.getFields().containsKey("bar"), is(true));
assertThat(stats.getTotal().fieldData.getFields().containsField("bar"), is(true));
assertThat(stats.getTotal().fieldData.getFields().get("bar"), greaterThan(0L));
assertThat(stats.getTotal().fieldData.getFields().containsKey("baz"), is(true));
assertThat(stats.getTotal().fieldData.getFields().containsField("baz"), is(true));
assertThat(stats.getTotal().fieldData.getFields().get("baz"), greaterThan(0L));
stats = builder.setFieldDataFields("*r").execute().actionGet();
assertThat(stats.getTotal().fieldData.getMemorySizeInBytes(), greaterThan(0L));
assertThat(stats.getTotal().fieldData.getFields().containsKey("bar"), is(true));
assertThat(stats.getTotal().fieldData.getFields().containsField("bar"), is(true));
assertThat(stats.getTotal().fieldData.getFields().get("bar"), greaterThan(0L));
assertThat(stats.getTotal().fieldData.getFields().containsKey("baz"), is(false));
assertThat(stats.getTotal().fieldData.getFields().containsField("baz"), is(false));
}
@ -782,29 +781,29 @@ public class IndexStatsIT extends ESIntegTestCase {
stats = builder.setCompletionFields("bar.completion").execute().actionGet();
assertThat(stats.getTotal().completion.getSizeInBytes(), greaterThan(0L));
assertThat(stats.getTotal().completion.getFields().containsKey("bar.completion"), is(true));
assertThat(stats.getTotal().completion.getFields().containsField("bar.completion"), is(true));
assertThat(stats.getTotal().completion.getFields().get("bar.completion"), greaterThan(0L));
assertThat(stats.getTotal().completion.getFields().containsKey("baz.completion"), is(false));
assertThat(stats.getTotal().completion.getFields().containsField("baz.completion"), is(false));
stats = builder.setCompletionFields("bar.completion", "baz.completion").execute().actionGet();
assertThat(stats.getTotal().completion.getSizeInBytes(), greaterThan(0L));
assertThat(stats.getTotal().completion.getFields().containsKey("bar.completion"), is(true));
assertThat(stats.getTotal().completion.getFields().containsField("bar.completion"), is(true));
assertThat(stats.getTotal().completion.getFields().get("bar.completion"), greaterThan(0L));
assertThat(stats.getTotal().completion.getFields().containsKey("baz.completion"), is(true));
assertThat(stats.getTotal().completion.getFields().containsField("baz.completion"), is(true));
assertThat(stats.getTotal().completion.getFields().get("baz.completion"), greaterThan(0L));
stats = builder.setCompletionFields("*").execute().actionGet();
assertThat(stats.getTotal().completion.getSizeInBytes(), greaterThan(0L));
assertThat(stats.getTotal().completion.getFields().containsKey("bar.completion"), is(true));
assertThat(stats.getTotal().completion.getFields().containsField("bar.completion"), is(true));
assertThat(stats.getTotal().completion.getFields().get("bar.completion"), greaterThan(0L));
assertThat(stats.getTotal().completion.getFields().containsKey("baz.completion"), is(true));
assertThat(stats.getTotal().completion.getFields().containsField("baz.completion"), is(true));
assertThat(stats.getTotal().completion.getFields().get("baz.completion"), greaterThan(0L));
stats = builder.setCompletionFields("*r*").execute().actionGet();
assertThat(stats.getTotal().completion.getSizeInBytes(), greaterThan(0L));
assertThat(stats.getTotal().completion.getFields().containsKey("bar.completion"), is(true));
assertThat(stats.getTotal().completion.getFields().containsField("bar.completion"), is(true));
assertThat(stats.getTotal().completion.getFields().get("bar.completion"), greaterThan(0L));
assertThat(stats.getTotal().completion.getFields().containsKey("baz.completion"), is(false));
assertThat(stats.getTotal().completion.getFields().containsField("baz.completion"), is(false));
}

View File

@ -32,6 +32,7 @@ import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse;
import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.action.search.SearchPhaseExecutionException;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.common.FieldMemoryStats;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.Fuzziness;
import org.elasticsearch.common.xcontent.XContentBuilder;
@ -750,7 +751,7 @@ public class CompletionSuggestSearchIT extends ESIntegTestCase {
// regexes
IndicesStatsResponse regexFieldStats = client().admin().indices().prepareStats(INDEX).setIndices(INDEX).setCompletion(true).setCompletionFields("*").get();
ObjectLongHashMap<String> fields = regexFieldStats.getIndex(INDEX).getPrimaries().completion.getFields();
FieldMemoryStats fields = regexFieldStats.getIndex(INDEX).getPrimaries().completion.getFields();
long regexSizeInBytes = fields.get(FIELD) + fields.get(otherField);
assertThat(regexSizeInBytes, is(totalSizeInBytes));
}