mirror of
https://github.com/honeymoose/OpenSearch.git
synced 2025-03-06 19:09:14 +00:00
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:
parent
efe5a75d26
commit
0b338bf523
@ -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;
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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 -> 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);
|
||||
}
|
||||
}
|
@ -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<String, String> 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());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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();
|
||||
|
@ -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 {
|
||||
|
@ -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());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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);
|
||||
|
@ -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();
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
|
@ -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));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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();
|
||||
|
@ -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));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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();
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
|
@ -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));
|
||||
}
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
@ -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);
|
||||
|
@ -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());
|
||||
}
|
||||
}
|
@ -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());
|
||||
}
|
||||
}
|
@ -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));
|
||||
|
||||
}
|
||||
|
||||
|
@ -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));
|
||||
}
|
||||
|
Loading…
x
Reference in New Issue
Block a user