Implementation of Segment disk stats aggregating sizes by index file extension.
Use 'includeSegmentFileSizes' as the flag name to report disk usage. Added test that verifies reported segment disk usage is growing accordingly after adding a document. Documentation: Reference the new parameter as part of indices stats.
This commit is contained in:
parent
ea2792e278
commit
356364810c
|
@ -166,7 +166,7 @@ public class CommonStats implements Streamable, ToXContent {
|
|||
completion = indexShard.completionStats(flags.completionDataFields());
|
||||
break;
|
||||
case Segments:
|
||||
segments = indexShard.segmentStats();
|
||||
segments = indexShard.segmentStats(flags.includeSegmentFileSizes());
|
||||
break;
|
||||
case Percolate:
|
||||
percolate = indexShard.percolateStats();
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
|
||||
package org.elasticsearch.action.admin.indices.stats;
|
||||
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.io.stream.Streamable;
|
||||
|
@ -38,6 +39,7 @@ public class CommonStatsFlags implements Streamable, Cloneable {
|
|||
private String[] groups = null;
|
||||
private String[] fieldDataFields = null;
|
||||
private String[] completionDataFields = null;
|
||||
private boolean includeSegmentFileSizes = false;
|
||||
|
||||
|
||||
/**
|
||||
|
@ -62,6 +64,7 @@ public class CommonStatsFlags implements Streamable, Cloneable {
|
|||
groups = null;
|
||||
fieldDataFields = null;
|
||||
completionDataFields = null;
|
||||
includeSegmentFileSizes = false;
|
||||
return this;
|
||||
}
|
||||
|
||||
|
@ -74,6 +77,7 @@ public class CommonStatsFlags implements Streamable, Cloneable {
|
|||
groups = null;
|
||||
fieldDataFields = null;
|
||||
completionDataFields = null;
|
||||
includeSegmentFileSizes = false;
|
||||
return this;
|
||||
}
|
||||
|
||||
|
@ -137,6 +141,15 @@ public class CommonStatsFlags implements Streamable, Cloneable {
|
|||
return this.completionDataFields;
|
||||
}
|
||||
|
||||
public CommonStatsFlags includeSegmentFileSizes(boolean includeSegmentFileSizes) {
|
||||
this.includeSegmentFileSizes = includeSegmentFileSizes;
|
||||
return this;
|
||||
}
|
||||
|
||||
public boolean includeSegmentFileSizes() {
|
||||
return this.includeSegmentFileSizes;
|
||||
}
|
||||
|
||||
public boolean isSet(Flag flag) {
|
||||
return flags.contains(flag);
|
||||
}
|
||||
|
@ -177,6 +190,9 @@ public class CommonStatsFlags implements Streamable, Cloneable {
|
|||
out.writeStringArrayNullable(groups);
|
||||
out.writeStringArrayNullable(fieldDataFields);
|
||||
out.writeStringArrayNullable(completionDataFields);
|
||||
if (out.getVersion().onOrAfter(Version.V_5_0_0)) {
|
||||
out.writeBoolean(includeSegmentFileSizes);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -192,6 +208,11 @@ public class CommonStatsFlags implements Streamable, Cloneable {
|
|||
groups = in.readStringArray();
|
||||
fieldDataFields = in.readStringArray();
|
||||
completionDataFields = in.readStringArray();
|
||||
if (in.getVersion().onOrAfter(Version.V_5_0_0)) {
|
||||
includeSegmentFileSizes = in.readBoolean();
|
||||
} else {
|
||||
includeSegmentFileSizes = false;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -265,6 +265,15 @@ public class IndicesStatsRequest extends BroadcastRequest<IndicesStatsRequest> {
|
|||
return flags.isSet(Flag.Recovery);
|
||||
}
|
||||
|
||||
public boolean includeSegmentFileSizes() {
|
||||
return flags.includeSegmentFileSizes();
|
||||
}
|
||||
|
||||
public IndicesStatsRequest includeSegmentFileSizes(boolean includeSegmentFileSizes) {
|
||||
flags.includeSegmentFileSizes(includeSegmentFileSizes);
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
super.writeTo(out);
|
||||
|
|
|
@ -166,4 +166,9 @@ public class IndicesStatsRequestBuilder extends BroadcastOperationRequestBuilder
|
|||
request.recovery(recovery);
|
||||
return this;
|
||||
}
|
||||
|
||||
public IndicesStatsRequestBuilder setIncludeSegmentFileSizes(boolean includeSegmentFileSizes) {
|
||||
request.includeSegmentFileSizes(includeSegmentFileSizes);
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -144,6 +144,7 @@ public class TransportIndicesStatsAction extends TransportBroadcastByNodeAction<
|
|||
}
|
||||
if (request.segments()) {
|
||||
flags.set(CommonStatsFlags.Flag.Segments);
|
||||
flags.includeSegmentFileSizes(request.includeSegmentFileSizes());
|
||||
}
|
||||
if (request.completion()) {
|
||||
flags.set(CommonStatsFlags.Flag.Completion);
|
||||
|
|
|
@ -22,11 +22,13 @@ package org.elasticsearch.index.engine;
|
|||
import org.apache.lucene.index.DirectoryReader;
|
||||
import org.apache.lucene.index.FilterLeafReader;
|
||||
import org.apache.lucene.index.IndexCommit;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexWriter;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.SegmentCommitInfo;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.SegmentInfos;
|
||||
import org.apache.lucene.index.SegmentReader;
|
||||
import org.apache.lucene.index.SnapshotDeletionPolicy;
|
||||
|
@ -36,12 +38,15 @@ import org.apache.lucene.search.Query;
|
|||
import org.apache.lucene.search.SearcherManager;
|
||||
import org.apache.lucene.search.join.BitSetProducer;
|
||||
import org.apache.lucene.store.AlreadyClosedException;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.Accountable;
|
||||
import org.apache.lucene.util.Accountables;
|
||||
import org.elasticsearch.ExceptionsHelper;
|
||||
import org.elasticsearch.common.Base64;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.bytes.BytesReference;
|
||||
import org.elasticsearch.common.collect.ImmutableOpenMap;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.io.stream.Writeable;
|
||||
|
@ -64,8 +69,11 @@ import org.elasticsearch.index.store.Store;
|
|||
import org.elasticsearch.index.translog.Translog;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.nio.file.NoSuchFileException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
|
@ -406,7 +414,7 @@ public abstract class Engine implements Closeable {
|
|||
/**
|
||||
* Global stats on segments.
|
||||
*/
|
||||
public final SegmentsStats segmentsStats() {
|
||||
public final SegmentsStats segmentsStats(boolean includeSegmentFileSizes) {
|
||||
ensureOpen();
|
||||
try (final Searcher searcher = acquireSearcher("segments_stats")) {
|
||||
SegmentsStats stats = new SegmentsStats();
|
||||
|
@ -418,12 +426,81 @@ public abstract class Engine implements Closeable {
|
|||
stats.addTermVectorsMemoryInBytes(guardedRamBytesUsed(segmentReader.getTermVectorsReader()));
|
||||
stats.addNormsMemoryInBytes(guardedRamBytesUsed(segmentReader.getNormsReader()));
|
||||
stats.addDocValuesMemoryInBytes(guardedRamBytesUsed(segmentReader.getDocValuesReader()));
|
||||
|
||||
if (includeSegmentFileSizes) {
|
||||
// TODO: consider moving this to StoreStats
|
||||
stats.addFileSizes(getSegmentFileSizes(segmentReader));
|
||||
}
|
||||
}
|
||||
writerSegmentStats(stats);
|
||||
return stats;
|
||||
}
|
||||
}
|
||||
|
||||
private ImmutableOpenMap<String, Long> getSegmentFileSizes(SegmentReader segmentReader) {
|
||||
Directory directory = null;
|
||||
SegmentCommitInfo segmentCommitInfo = segmentReader.getSegmentInfo();
|
||||
boolean useCompoundFile = segmentCommitInfo.info.getUseCompoundFile();
|
||||
if (useCompoundFile) {
|
||||
try {
|
||||
directory = engineConfig.getCodec().compoundFormat().getCompoundReader(segmentReader.directory(), segmentCommitInfo.info, IOContext.READ);
|
||||
} catch (IOException e) {
|
||||
logger.warn("Error when opening compound reader for Directory [{}] and SegmentCommitInfo [{}]", e,
|
||||
segmentReader.directory(), segmentCommitInfo);
|
||||
|
||||
return ImmutableOpenMap.of();
|
||||
}
|
||||
} else {
|
||||
directory = segmentReader.directory();
|
||||
}
|
||||
|
||||
assert directory != null;
|
||||
|
||||
String[] files;
|
||||
if (useCompoundFile) {
|
||||
try {
|
||||
files = directory.listAll();
|
||||
} catch (IOException e) {
|
||||
logger.warn("Couldn't list Compound Reader Directory [{}]", e, directory);
|
||||
return ImmutableOpenMap.of();
|
||||
}
|
||||
} else {
|
||||
try {
|
||||
files = segmentReader.getSegmentInfo().files().toArray(new String[]{});
|
||||
} catch (IOException e) {
|
||||
logger.warn("Couldn't list Directory from SegmentReader [{}] and SegmentInfo [{}]", e, segmentReader, segmentReader.getSegmentInfo());
|
||||
return ImmutableOpenMap.of();
|
||||
}
|
||||
}
|
||||
|
||||
ImmutableOpenMap.Builder<String, Long> map = ImmutableOpenMap.builder();
|
||||
for (String file : files) {
|
||||
String extension = IndexFileNames.getExtension(file);
|
||||
long length = 0L;
|
||||
try {
|
||||
length = directory.fileLength(file);
|
||||
} catch (NoSuchFileException | FileNotFoundException e) {
|
||||
logger.warn("Tried to query fileLength but file is gone [{}] [{}]", e, directory, file);
|
||||
} catch (IOException e) {
|
||||
logger.warn("Error when trying to query fileLength [{}] [{}]", e, directory, file);
|
||||
}
|
||||
if (length == 0L) {
|
||||
continue;
|
||||
}
|
||||
map.put(extension, length);
|
||||
}
|
||||
|
||||
if (useCompoundFile && directory != null) {
|
||||
try {
|
||||
directory.close();
|
||||
} catch (IOException e) {
|
||||
logger.warn("Error when closing compound reader on Directory [{}]", e, directory);
|
||||
}
|
||||
}
|
||||
|
||||
return map.build();
|
||||
}
|
||||
|
||||
protected void writerSegmentStats(SegmentsStats stats) {
|
||||
// by default we don't have a writer here... subclasses can override this
|
||||
stats.addVersionMapMemoryInBytes(0);
|
||||
|
|
|
@ -19,6 +19,10 @@
|
|||
|
||||
package org.elasticsearch.index.engine;
|
||||
|
||||
import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
|
||||
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.common.collect.ImmutableOpenMap;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.io.stream.Streamable;
|
||||
|
@ -28,6 +32,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
|
|||
import org.elasticsearch.common.xcontent.XContentBuilderString;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Iterator;
|
||||
|
||||
public class SegmentsStats implements Streamable, ToXContent {
|
||||
|
||||
|
@ -42,6 +47,33 @@ public class SegmentsStats implements Streamable, ToXContent {
|
|||
private long indexWriterMaxMemoryInBytes;
|
||||
private long versionMapMemoryInBytes;
|
||||
private long bitsetMemoryInBytes;
|
||||
private ImmutableOpenMap<String, Long> fileSizes = ImmutableOpenMap.of();
|
||||
|
||||
/*
|
||||
* A map to provide a best-effort approach describing Lucene index files.
|
||||
*
|
||||
* Ideally this should be in sync to what the current version of Lucene is using, but it's harmless to leave extensions out,
|
||||
* they'll just miss a proper description in the stats
|
||||
*/
|
||||
private static ImmutableOpenMap<String, String> fileDescriptions = ImmutableOpenMap.<String, String>builder()
|
||||
.fPut("si", "Segment Info")
|
||||
.fPut("fnm", "Fields")
|
||||
.fPut("fdx", "Field Index")
|
||||
.fPut("fdt", "Field Data")
|
||||
.fPut("tim", "Term Dictionary")
|
||||
.fPut("tip", "Term Index")
|
||||
.fPut("doc", "Frequencies")
|
||||
.fPut("pos", "Positions")
|
||||
.fPut("pay", "Payloads")
|
||||
.fPut("nvd", "Norms")
|
||||
.fPut("nvm", "Norms")
|
||||
.fPut("dvd", "DocValues")
|
||||
.fPut("dvm", "DocValues")
|
||||
.fPut("tvx", "Term Vector Index")
|
||||
.fPut("tvd", "Term Vector Documents")
|
||||
.fPut("tvf", "Term Vector Fields")
|
||||
.fPut("liv", "Live Documents")
|
||||
.build();
|
||||
|
||||
public SegmentsStats() {}
|
||||
|
||||
|
@ -86,6 +118,22 @@ public class SegmentsStats implements Streamable, ToXContent {
|
|||
this.bitsetMemoryInBytes += bitsetMemoryInBytes;
|
||||
}
|
||||
|
||||
public void addFileSizes(ImmutableOpenMap<String, Long> fileSizes) {
|
||||
ImmutableOpenMap.Builder<String, Long> map = ImmutableOpenMap.builder(this.fileSizes);
|
||||
|
||||
for (Iterator<ObjectObjectCursor<String, Long>> it = fileSizes.iterator(); it.hasNext();) {
|
||||
ObjectObjectCursor<String, Long> entry = it.next();
|
||||
if (map.containsKey(entry.key)) {
|
||||
Long oldValue = map.get(entry.key);
|
||||
map.put(entry.key, oldValue + entry.value);
|
||||
} else {
|
||||
map.put(entry.key, entry.value);
|
||||
}
|
||||
}
|
||||
|
||||
this.fileSizes = map.build();
|
||||
}
|
||||
|
||||
public void add(SegmentsStats mergeStats) {
|
||||
if (mergeStats == null) {
|
||||
return;
|
||||
|
@ -100,6 +148,7 @@ public class SegmentsStats implements Streamable, ToXContent {
|
|||
addIndexWriterMaxMemoryInBytes(mergeStats.indexWriterMaxMemoryInBytes);
|
||||
addVersionMapMemoryInBytes(mergeStats.versionMapMemoryInBytes);
|
||||
addBitsetMemoryInBytes(mergeStats.bitsetMemoryInBytes);
|
||||
addFileSizes(mergeStats.fileSizes);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -219,6 +268,10 @@ public class SegmentsStats implements Streamable, ToXContent {
|
|||
return new ByteSizeValue(bitsetMemoryInBytes);
|
||||
}
|
||||
|
||||
public ImmutableOpenMap<String, Long> getFileSizes() {
|
||||
return fileSizes;
|
||||
}
|
||||
|
||||
public static SegmentsStats readSegmentsStats(StreamInput in) throws IOException {
|
||||
SegmentsStats stats = new SegmentsStats();
|
||||
stats.readFrom(in);
|
||||
|
@ -239,6 +292,15 @@ public class SegmentsStats implements Streamable, ToXContent {
|
|||
builder.byteSizeField(Fields.INDEX_WRITER_MAX_MEMORY_IN_BYTES, Fields.INDEX_WRITER_MAX_MEMORY, indexWriterMaxMemoryInBytes);
|
||||
builder.byteSizeField(Fields.VERSION_MAP_MEMORY_IN_BYTES, Fields.VERSION_MAP_MEMORY, versionMapMemoryInBytes);
|
||||
builder.byteSizeField(Fields.FIXED_BIT_SET_MEMORY_IN_BYTES, Fields.FIXED_BIT_SET, bitsetMemoryInBytes);
|
||||
builder.startObject(Fields.FILE_SIZES);
|
||||
for (Iterator<ObjectObjectCursor<String, Long>> it = fileSizes.iterator(); it.hasNext();) {
|
||||
ObjectObjectCursor<String, Long> entry = it.next();
|
||||
builder.startObject(entry.key);
|
||||
builder.byteSizeField(Fields.SIZE_IN_BYTES, Fields.SIZE, entry.value);
|
||||
builder.field(Fields.DESCRIPTION, fileDescriptions.getOrDefault(entry.key, "Others"));
|
||||
builder.endObject();
|
||||
}
|
||||
builder.endObject();
|
||||
builder.endObject();
|
||||
return builder;
|
||||
}
|
||||
|
@ -266,6 +328,10 @@ public class SegmentsStats implements Streamable, ToXContent {
|
|||
static final XContentBuilderString VERSION_MAP_MEMORY_IN_BYTES = new XContentBuilderString("version_map_memory_in_bytes");
|
||||
static final XContentBuilderString FIXED_BIT_SET = new XContentBuilderString("fixed_bit_set");
|
||||
static final XContentBuilderString FIXED_BIT_SET_MEMORY_IN_BYTES = new XContentBuilderString("fixed_bit_set_memory_in_bytes");
|
||||
static final XContentBuilderString FILE_SIZES = new XContentBuilderString("file_sizes");
|
||||
static final XContentBuilderString SIZE = new XContentBuilderString("size");
|
||||
static final XContentBuilderString SIZE_IN_BYTES = new XContentBuilderString("size_in_bytes");
|
||||
static final XContentBuilderString DESCRIPTION = new XContentBuilderString("description");
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -281,6 +347,19 @@ public class SegmentsStats implements Streamable, ToXContent {
|
|||
versionMapMemoryInBytes = in.readLong();
|
||||
indexWriterMaxMemoryInBytes = in.readLong();
|
||||
bitsetMemoryInBytes = in.readLong();
|
||||
|
||||
if (in.getVersion().onOrAfter(Version.V_5_0_0)) {
|
||||
int size = in.readVInt();
|
||||
ImmutableOpenMap.Builder<String, Long> map = ImmutableOpenMap.builder(size);
|
||||
for (int i = 0; i < size; i++) {
|
||||
String key = in.readString();
|
||||
Long value = in.readLong();
|
||||
map.put(key, value);
|
||||
}
|
||||
fileSizes = map.build();
|
||||
} else {
|
||||
fileSizes = ImmutableOpenMap.of();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -296,5 +375,14 @@ public class SegmentsStats implements Streamable, ToXContent {
|
|||
out.writeLong(versionMapMemoryInBytes);
|
||||
out.writeLong(indexWriterMaxMemoryInBytes);
|
||||
out.writeLong(bitsetMemoryInBytes);
|
||||
|
||||
if (out.getVersion().onOrAfter(Version.V_5_0_0)) {
|
||||
out.writeVInt(fileSizes.size());
|
||||
for (Iterator<ObjectObjectCursor<String, Long>> it = fileSizes.iterator(); it.hasNext();) {
|
||||
ObjectObjectCursor<String, Long> entry = it.next();
|
||||
out.writeString(entry.key);
|
||||
out.writeLong(entry.value);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -630,8 +630,8 @@ public class IndexShard extends AbstractIndexShardComponent {
|
|||
return engine.getMergeStats();
|
||||
}
|
||||
|
||||
public SegmentsStats segmentStats() {
|
||||
SegmentsStats segmentsStats = getEngine().segmentsStats();
|
||||
public SegmentsStats segmentStats(boolean includeSegmentFileSizes) {
|
||||
SegmentsStats segmentsStats = getEngine().segmentsStats(includeSegmentFileSizes);
|
||||
segmentsStats.addBitsetMemoryInBytes(shardBitsetFilterCache.getMemorySizeInBytes());
|
||||
return segmentsStats;
|
||||
}
|
||||
|
|
|
@ -109,6 +109,9 @@ public class RestNodesStatsAction extends BaseRestHandler {
|
|||
if (nodesStatsRequest.indices().isSet(Flag.Indexing) && (request.hasParam("types"))) {
|
||||
nodesStatsRequest.indices().types(request.paramAsStringArray("types", null));
|
||||
}
|
||||
if (nodesStatsRequest.indices().isSet(Flag.Segments) && (request.hasParam("include_segment_file_sizes"))) {
|
||||
nodesStatsRequest.indices().includeSegmentFileSizes(true);
|
||||
}
|
||||
|
||||
client.admin().cluster().nodesStats(nodesStatsRequest, new RestToXContentListener<NodesStatsResponse>(channel));
|
||||
}
|
||||
|
|
|
@ -104,6 +104,10 @@ public class RestIndicesStatsAction extends BaseRestHandler {
|
|||
indicesStatsRequest.fieldDataFields(request.paramAsStringArray("fielddata_fields", request.paramAsStringArray("fields", Strings.EMPTY_ARRAY)));
|
||||
}
|
||||
|
||||
if (indicesStatsRequest.segments() && request.hasParam("include_segment_file_sizes")) {
|
||||
indicesStatsRequest.includeSegmentFileSizes(true);
|
||||
}
|
||||
|
||||
client.admin().indices().stats(indicesStatsRequest, new RestBuilderListener<IndicesStatsResponse>(channel) {
|
||||
@Override
|
||||
public RestResponse buildResponse(IndicesStatsResponse response, XContentBuilder builder) throws Exception {
|
||||
|
|
|
@ -19,6 +19,8 @@
|
|||
|
||||
package org.elasticsearch.index.engine;
|
||||
|
||||
import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
|
||||
|
||||
import org.apache.log4j.AppenderSkeleton;
|
||||
import org.apache.log4j.Level;
|
||||
import org.apache.log4j.LogManager;
|
||||
|
@ -126,6 +128,7 @@ import static java.util.Collections.emptyMap;
|
|||
import static org.elasticsearch.index.engine.Engine.Operation.Origin.PRIMARY;
|
||||
import static org.elasticsearch.index.engine.Engine.Operation.Origin.REPLICA;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.everyItem;
|
||||
import static org.hamcrest.Matchers.greaterThan;
|
||||
import static org.hamcrest.Matchers.hasKey;
|
||||
import static org.hamcrest.Matchers.not;
|
||||
|
@ -293,8 +296,8 @@ public class InternalEngineTests extends ESTestCase {
|
|||
Engine engine = createEngine(defaultSettings, store, createTempDir(), NoMergePolicy.INSTANCE)) {
|
||||
List<Segment> segments = engine.segments(false);
|
||||
assertThat(segments.isEmpty(), equalTo(true));
|
||||
assertThat(engine.segmentsStats().getCount(), equalTo(0L));
|
||||
assertThat(engine.segmentsStats().getMemoryInBytes(), equalTo(0L));
|
||||
assertThat(engine.segmentsStats(false).getCount(), equalTo(0L));
|
||||
assertThat(engine.segmentsStats(false).getMemoryInBytes(), equalTo(0L));
|
||||
|
||||
// create a doc and refresh
|
||||
ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, null);
|
||||
|
@ -306,7 +309,7 @@ public class InternalEngineTests extends ESTestCase {
|
|||
|
||||
segments = engine.segments(false);
|
||||
assertThat(segments.size(), equalTo(1));
|
||||
SegmentsStats stats = engine.segmentsStats();
|
||||
SegmentsStats stats = engine.segmentsStats(false);
|
||||
assertThat(stats.getCount(), equalTo(1L));
|
||||
assertThat(stats.getTermsMemoryInBytes(), greaterThan(0L));
|
||||
assertThat(stats.getStoredFieldsMemoryInBytes(), greaterThan(0L));
|
||||
|
@ -324,7 +327,7 @@ public class InternalEngineTests extends ESTestCase {
|
|||
|
||||
segments = engine.segments(false);
|
||||
assertThat(segments.size(), equalTo(1));
|
||||
assertThat(engine.segmentsStats().getCount(), equalTo(1L));
|
||||
assertThat(engine.segmentsStats(false).getCount(), equalTo(1L));
|
||||
assertThat(segments.get(0).isCommitted(), equalTo(true));
|
||||
assertThat(segments.get(0).isSearch(), equalTo(true));
|
||||
assertThat(segments.get(0).getNumDocs(), equalTo(2));
|
||||
|
@ -337,12 +340,12 @@ public class InternalEngineTests extends ESTestCase {
|
|||
|
||||
segments = engine.segments(false);
|
||||
assertThat(segments.size(), equalTo(2));
|
||||
assertThat(engine.segmentsStats().getCount(), equalTo(2L));
|
||||
assertThat(engine.segmentsStats().getTermsMemoryInBytes(), greaterThan(stats.getTermsMemoryInBytes()));
|
||||
assertThat(engine.segmentsStats().getStoredFieldsMemoryInBytes(), greaterThan(stats.getStoredFieldsMemoryInBytes()));
|
||||
assertThat(engine.segmentsStats().getTermVectorsMemoryInBytes(), equalTo(0L));
|
||||
assertThat(engine.segmentsStats().getNormsMemoryInBytes(), greaterThan(stats.getNormsMemoryInBytes()));
|
||||
assertThat(engine.segmentsStats().getDocValuesMemoryInBytes(), greaterThan(stats.getDocValuesMemoryInBytes()));
|
||||
assertThat(engine.segmentsStats(false).getCount(), equalTo(2L));
|
||||
assertThat(engine.segmentsStats(false).getTermsMemoryInBytes(), greaterThan(stats.getTermsMemoryInBytes()));
|
||||
assertThat(engine.segmentsStats(false).getStoredFieldsMemoryInBytes(), greaterThan(stats.getStoredFieldsMemoryInBytes()));
|
||||
assertThat(engine.segmentsStats(false).getTermVectorsMemoryInBytes(), equalTo(0L));
|
||||
assertThat(engine.segmentsStats(false).getNormsMemoryInBytes(), greaterThan(stats.getNormsMemoryInBytes()));
|
||||
assertThat(engine.segmentsStats(false).getDocValuesMemoryInBytes(), greaterThan(stats.getDocValuesMemoryInBytes()));
|
||||
assertThat(segments.get(0).getGeneration() < segments.get(1).getGeneration(), equalTo(true));
|
||||
assertThat(segments.get(0).isCommitted(), equalTo(true));
|
||||
assertThat(segments.get(0).isSearch(), equalTo(true));
|
||||
|
@ -363,7 +366,7 @@ public class InternalEngineTests extends ESTestCase {
|
|||
|
||||
segments = engine.segments(false);
|
||||
assertThat(segments.size(), equalTo(2));
|
||||
assertThat(engine.segmentsStats().getCount(), equalTo(2L));
|
||||
assertThat(engine.segmentsStats(false).getCount(), equalTo(2L));
|
||||
assertThat(segments.get(0).getGeneration() < segments.get(1).getGeneration(), equalTo(true));
|
||||
assertThat(segments.get(0).isCommitted(), equalTo(true));
|
||||
assertThat(segments.get(0).isSearch(), equalTo(true));
|
||||
|
@ -384,7 +387,7 @@ public class InternalEngineTests extends ESTestCase {
|
|||
|
||||
segments = engine.segments(false);
|
||||
assertThat(segments.size(), equalTo(3));
|
||||
assertThat(engine.segmentsStats().getCount(), equalTo(3L));
|
||||
assertThat(engine.segmentsStats(false).getCount(), equalTo(3L));
|
||||
assertThat(segments.get(0).getGeneration() < segments.get(1).getGeneration(), equalTo(true));
|
||||
assertThat(segments.get(0).isCommitted(), equalTo(true));
|
||||
assertThat(segments.get(0).isSearch(), equalTo(true));
|
||||
|
@ -487,6 +490,29 @@ public class InternalEngineTests extends ESTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
public void testSegmentsStatsIncludingFileSizes() throws Exception {
|
||||
try (Store store = createStore();
|
||||
Engine engine = createEngine(defaultSettings, store, createTempDir(), NoMergePolicy.INSTANCE)) {
|
||||
assertThat(engine.segmentsStats(true).getFileSizes().size(), equalTo(0));
|
||||
|
||||
ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, null);
|
||||
engine.index(new Engine.Index(newUid("1"), doc));
|
||||
engine.refresh("test");
|
||||
|
||||
SegmentsStats stats = engine.segmentsStats(true);
|
||||
assertThat(stats.getFileSizes().size(), greaterThan(0));
|
||||
assertThat((Iterable<Long>) () -> stats.getFileSizes().valuesIt(), everyItem(greaterThan(0L)));
|
||||
|
||||
ObjectObjectCursor<String, Long> firstEntry = stats.getFileSizes().iterator().next();
|
||||
|
||||
ParsedDocument doc2 = testParsedDocument("2", "2", "test", null, -1, -1, testDocumentWithTextField(), B_2, null);
|
||||
engine.index(new Engine.Index(newUid("2"), doc2));
|
||||
engine.refresh("test");
|
||||
|
||||
assertThat(engine.segmentsStats(true).getFileSizes().get(firstEntry.key), greaterThan(firstEntry.value));
|
||||
}
|
||||
}
|
||||
|
||||
public void testCommitStats() {
|
||||
Document document = testDocumentWithTextField();
|
||||
document.add(new Field(SourceFieldMapper.NAME, B_1.toBytes(), SourceFieldMapper.Defaults.FIELD_TYPE));
|
||||
|
|
|
@ -276,8 +276,8 @@ public class ShadowEngineTests extends ESTestCase {
|
|||
primaryEngine = createInternalEngine(defaultSettings, store, createTempDir(), NoMergePolicy.INSTANCE);
|
||||
List<Segment> segments = primaryEngine.segments(false);
|
||||
assertThat(segments.isEmpty(), equalTo(true));
|
||||
assertThat(primaryEngine.segmentsStats().getCount(), equalTo(0L));
|
||||
assertThat(primaryEngine.segmentsStats().getMemoryInBytes(), equalTo(0L));
|
||||
assertThat(primaryEngine.segmentsStats(false).getCount(), equalTo(0L));
|
||||
assertThat(primaryEngine.segmentsStats(false).getMemoryInBytes(), equalTo(0L));
|
||||
|
||||
// create a doc and refresh
|
||||
ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, null);
|
||||
|
@ -289,7 +289,7 @@ public class ShadowEngineTests extends ESTestCase {
|
|||
|
||||
segments = primaryEngine.segments(false);
|
||||
assertThat(segments.size(), equalTo(1));
|
||||
SegmentsStats stats = primaryEngine.segmentsStats();
|
||||
SegmentsStats stats = primaryEngine.segmentsStats(false);
|
||||
assertThat(stats.getCount(), equalTo(1L));
|
||||
assertThat(stats.getTermsMemoryInBytes(), greaterThan(0L));
|
||||
assertThat(stats.getStoredFieldsMemoryInBytes(), greaterThan(0L));
|
||||
|
@ -306,7 +306,7 @@ public class ShadowEngineTests extends ESTestCase {
|
|||
// Check that the replica sees nothing
|
||||
segments = replicaEngine.segments(false);
|
||||
assertThat(segments.size(), equalTo(0));
|
||||
stats = replicaEngine.segmentsStats();
|
||||
stats = replicaEngine.segmentsStats(false);
|
||||
assertThat(stats.getCount(), equalTo(0L));
|
||||
assertThat(stats.getTermsMemoryInBytes(), equalTo(0L));
|
||||
assertThat(stats.getStoredFieldsMemoryInBytes(), equalTo(0L));
|
||||
|
@ -323,7 +323,7 @@ public class ShadowEngineTests extends ESTestCase {
|
|||
// Check that the primary AND replica sees segments now
|
||||
segments = primaryEngine.segments(false);
|
||||
assertThat(segments.size(), equalTo(1));
|
||||
assertThat(primaryEngine.segmentsStats().getCount(), equalTo(1L));
|
||||
assertThat(primaryEngine.segmentsStats(false).getCount(), equalTo(1L));
|
||||
assertThat(segments.get(0).isCommitted(), equalTo(true));
|
||||
assertThat(segments.get(0).isSearch(), equalTo(true));
|
||||
assertThat(segments.get(0).getNumDocs(), equalTo(2));
|
||||
|
@ -332,7 +332,7 @@ public class ShadowEngineTests extends ESTestCase {
|
|||
|
||||
segments = replicaEngine.segments(false);
|
||||
assertThat(segments.size(), equalTo(1));
|
||||
assertThat(replicaEngine.segmentsStats().getCount(), equalTo(1L));
|
||||
assertThat(replicaEngine.segmentsStats(false).getCount(), equalTo(1L));
|
||||
assertThat(segments.get(0).isCommitted(), equalTo(true));
|
||||
assertThat(segments.get(0).isSearch(), equalTo(true));
|
||||
assertThat(segments.get(0).getNumDocs(), equalTo(2));
|
||||
|
@ -346,12 +346,12 @@ public class ShadowEngineTests extends ESTestCase {
|
|||
|
||||
segments = primaryEngine.segments(false);
|
||||
assertThat(segments.size(), equalTo(2));
|
||||
assertThat(primaryEngine.segmentsStats().getCount(), equalTo(2L));
|
||||
assertThat(primaryEngine.segmentsStats().getTermsMemoryInBytes(), greaterThan(stats.getTermsMemoryInBytes()));
|
||||
assertThat(primaryEngine.segmentsStats().getStoredFieldsMemoryInBytes(), greaterThan(stats.getStoredFieldsMemoryInBytes()));
|
||||
assertThat(primaryEngine.segmentsStats().getTermVectorsMemoryInBytes(), equalTo(0L));
|
||||
assertThat(primaryEngine.segmentsStats().getNormsMemoryInBytes(), greaterThan(stats.getNormsMemoryInBytes()));
|
||||
assertThat(primaryEngine.segmentsStats().getDocValuesMemoryInBytes(), greaterThan(stats.getDocValuesMemoryInBytes()));
|
||||
assertThat(primaryEngine.segmentsStats(false).getCount(), equalTo(2L));
|
||||
assertThat(primaryEngine.segmentsStats(false).getTermsMemoryInBytes(), greaterThan(stats.getTermsMemoryInBytes()));
|
||||
assertThat(primaryEngine.segmentsStats(false).getStoredFieldsMemoryInBytes(), greaterThan(stats.getStoredFieldsMemoryInBytes()));
|
||||
assertThat(primaryEngine.segmentsStats(false).getTermVectorsMemoryInBytes(), equalTo(0L));
|
||||
assertThat(primaryEngine.segmentsStats(false).getNormsMemoryInBytes(), greaterThan(stats.getNormsMemoryInBytes()));
|
||||
assertThat(primaryEngine.segmentsStats(false).getDocValuesMemoryInBytes(), greaterThan(stats.getDocValuesMemoryInBytes()));
|
||||
assertThat(segments.get(0).getGeneration() < segments.get(1).getGeneration(), equalTo(true));
|
||||
assertThat(segments.get(0).isCommitted(), equalTo(true));
|
||||
assertThat(segments.get(0).isSearch(), equalTo(true));
|
||||
|
@ -370,12 +370,12 @@ public class ShadowEngineTests extends ESTestCase {
|
|||
|
||||
segments = replicaEngine.segments(false);
|
||||
assertThat(segments.size(), equalTo(2));
|
||||
assertThat(replicaEngine.segmentsStats().getCount(), equalTo(2L));
|
||||
assertThat(replicaEngine.segmentsStats().getTermsMemoryInBytes(), greaterThan(stats.getTermsMemoryInBytes()));
|
||||
assertThat(replicaEngine.segmentsStats().getStoredFieldsMemoryInBytes(), greaterThan(stats.getStoredFieldsMemoryInBytes()));
|
||||
assertThat(replicaEngine.segmentsStats().getTermVectorsMemoryInBytes(), equalTo(0L));
|
||||
assertThat(replicaEngine.segmentsStats().getNormsMemoryInBytes(), greaterThan(stats.getNormsMemoryInBytes()));
|
||||
assertThat(replicaEngine.segmentsStats().getDocValuesMemoryInBytes(), greaterThan(stats.getDocValuesMemoryInBytes()));
|
||||
assertThat(replicaEngine.segmentsStats(false).getCount(), equalTo(2L));
|
||||
assertThat(replicaEngine.segmentsStats(false).getTermsMemoryInBytes(), greaterThan(stats.getTermsMemoryInBytes()));
|
||||
assertThat(replicaEngine.segmentsStats(false).getStoredFieldsMemoryInBytes(), greaterThan(stats.getStoredFieldsMemoryInBytes()));
|
||||
assertThat(replicaEngine.segmentsStats(false).getTermVectorsMemoryInBytes(), equalTo(0L));
|
||||
assertThat(replicaEngine.segmentsStats(false).getNormsMemoryInBytes(), greaterThan(stats.getNormsMemoryInBytes()));
|
||||
assertThat(replicaEngine.segmentsStats(false).getDocValuesMemoryInBytes(), greaterThan(stats.getDocValuesMemoryInBytes()));
|
||||
assertThat(segments.get(0).getGeneration() < segments.get(1).getGeneration(), equalTo(true));
|
||||
assertThat(segments.get(0).isCommitted(), equalTo(true));
|
||||
assertThat(segments.get(0).isSearch(), equalTo(true));
|
||||
|
@ -393,7 +393,7 @@ public class ShadowEngineTests extends ESTestCase {
|
|||
|
||||
segments = primaryEngine.segments(false);
|
||||
assertThat(segments.size(), equalTo(2));
|
||||
assertThat(primaryEngine.segmentsStats().getCount(), equalTo(2L));
|
||||
assertThat(primaryEngine.segmentsStats(false).getCount(), equalTo(2L));
|
||||
assertThat(segments.get(0).getGeneration() < segments.get(1).getGeneration(), equalTo(true));
|
||||
assertThat(segments.get(0).isCommitted(), equalTo(true));
|
||||
assertThat(segments.get(0).isSearch(), equalTo(true));
|
||||
|
@ -416,7 +416,7 @@ public class ShadowEngineTests extends ESTestCase {
|
|||
|
||||
segments = primaryEngine.segments(false);
|
||||
assertThat(segments.size(), equalTo(3));
|
||||
assertThat(primaryEngine.segmentsStats().getCount(), equalTo(3L));
|
||||
assertThat(primaryEngine.segmentsStats(false).getCount(), equalTo(3L));
|
||||
assertThat(segments.get(0).getGeneration() < segments.get(1).getGeneration(), equalTo(true));
|
||||
assertThat(segments.get(0).isCommitted(), equalTo(true));
|
||||
assertThat(segments.get(0).isSearch(), equalTo(true));
|
||||
|
|
|
@ -39,6 +39,8 @@ specified as well in the URI. Those stats can be any of:
|
|||
groups). The `groups` parameter accepts a comma separated list of group names.
|
||||
Use `_all` to return statistics for all groups.
|
||||
|
||||
`segments`:: Retrieve the memory use of the open segments. Optionally, setting the `include_segment_file_sizes` flag, report the aggregated disk usage of each one of the Lucene index files.
|
||||
|
||||
`completion`:: Completion suggest statistics.
|
||||
`fielddata`:: Fielddata statistics.
|
||||
`flush`:: Flush statistics.
|
||||
|
|
Loading…
Reference in New Issue