Skip to content

Commit 73d4516

Browse files
authored
Allow inclusion of unloaded segments in stats (#39512)
Today we have no chance to fetch actual segment stats for segments that are currently unloaded. This is relevant in the case of frozen indices. This allows to monitor how much memory a frozen index would use if it was unfrozen.
1 parent b4bdf06 commit 73d4516

File tree

15 files changed

+101
-40
lines changed

15 files changed

+101
-40
lines changed

rest-api-spec/src/main/resources/rest-api-spec/api/cat.indices.json

+5
Original file line numberDiff line numberDiff line change
@@ -57,6 +57,11 @@
5757
"type": "boolean",
5858
"description": "Verbose mode. Display column headers",
5959
"default": false
60+
},
61+
"include_unloaded_segments": {
62+
"type": "boolean",
63+
"description": "If set to true segment stats will include stats for segments that are not currently loaded into memory",
64+
"default": false
6065
}
6166
}
6267
},

rest-api-spec/src/main/resources/rest-api-spec/api/indices.stats.json

+5
Original file line numberDiff line numberDiff line change
@@ -52,6 +52,11 @@
5252
"type": "boolean",
5353
"description": "Whether to report the aggregated disk usage of each one of the Lucene index files (only applies if segment stats are requested)",
5454
"default": false
55+
},
56+
"include_unloaded_segments": {
57+
"type": "boolean",
58+
"description": "If set to true segment stats will include stats for segments that are not currently loaded into memory",
59+
"default": false
5560
}
5661
}
5762
},

server/src/main/java/org/elasticsearch/action/admin/indices/stats/CommonStats.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -207,7 +207,7 @@ public CommonStats(IndicesQueryCache indicesQueryCache, IndexShard indexShard, C
207207
completion = indexShard.completionStats(flags.completionDataFields());
208208
break;
209209
case Segments:
210-
segments = indexShard.segmentStats(flags.includeSegmentFileSizes());
210+
segments = indexShard.segmentStats(flags.includeSegmentFileSizes(), flags.includeUnloadedSegments());
211211
break;
212212
case Translog:
213213
translog = indexShard.translogStats();

server/src/main/java/org/elasticsearch/action/admin/indices/stats/CommonStatsFlags.java

+19
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,7 @@
1919

2020
package org.elasticsearch.action.admin.indices.stats;
2121

22+
import org.elasticsearch.Version;
2223
import org.elasticsearch.common.io.stream.StreamInput;
2324
import org.elasticsearch.common.io.stream.StreamOutput;
2425
import org.elasticsearch.common.io.stream.Writeable;
@@ -38,6 +39,7 @@ public class CommonStatsFlags implements Writeable, Cloneable {
3839
private String[] fieldDataFields = null;
3940
private String[] completionDataFields = null;
4041
private boolean includeSegmentFileSizes = false;
42+
private boolean includeUnloadedSegments = false;
4143

4244
/**
4345
* @param flags flags to set. If no flags are supplied, default flags will be set.
@@ -62,6 +64,9 @@ public CommonStatsFlags(StreamInput in) throws IOException {
6264
fieldDataFields = in.readStringArray();
6365
completionDataFields = in.readStringArray();
6466
includeSegmentFileSizes = in.readBoolean();
67+
if (in.getVersion().onOrAfter(Version.V_8_0_0)) {
68+
includeUnloadedSegments = in.readBoolean();
69+
}
6570
}
6671

6772
@Override
@@ -77,6 +82,9 @@ public void writeTo(StreamOutput out) throws IOException {
7782
out.writeStringArrayNullable(fieldDataFields);
7883
out.writeStringArrayNullable(completionDataFields);
7984
out.writeBoolean(includeSegmentFileSizes);
85+
if (out.getVersion().onOrAfter(Version.V_8_0_0)) {
86+
out.writeBoolean(includeUnloadedSegments);
87+
}
8088
}
8189

8290
/**
@@ -89,6 +97,7 @@ public CommonStatsFlags all() {
8997
fieldDataFields = null;
9098
completionDataFields = null;
9199
includeSegmentFileSizes = false;
100+
includeUnloadedSegments = false;
92101
return this;
93102
}
94103

@@ -102,6 +111,7 @@ public CommonStatsFlags clear() {
102111
fieldDataFields = null;
103112
completionDataFields = null;
104113
includeSegmentFileSizes = false;
114+
includeUnloadedSegments = false;
105115
return this;
106116
}
107117

@@ -170,6 +180,15 @@ public CommonStatsFlags includeSegmentFileSizes(boolean includeSegmentFileSizes)
170180
return this;
171181
}
172182

183+
public CommonStatsFlags includeUnloadedSegments(boolean includeUnloadedSegments) {
184+
this.includeUnloadedSegments = includeUnloadedSegments;
185+
return this;
186+
}
187+
188+
public boolean includeUnloadedSegments() {
189+
return this.includeUnloadedSegments;
190+
}
191+
173192
public boolean includeSegmentFileSizes() {
174193
return this.includeSegmentFileSizes;
175194
}

server/src/main/java/org/elasticsearch/index/engine/Engine.java

+2-2
Original file line numberDiff line numberDiff line change
@@ -821,7 +821,7 @@ public final CommitStats commitStats() {
821821
/**
822822
* Global stats on segments.
823823
*/
824-
public SegmentsStats segmentsStats(boolean includeSegmentFileSizes) {
824+
public SegmentsStats segmentsStats(boolean includeSegmentFileSizes, boolean includeUnloadedSegments) {
825825
ensureOpen();
826826
Set<String> segmentName = new HashSet<>();
827827
SegmentsStats stats = new SegmentsStats();
@@ -845,7 +845,7 @@ public SegmentsStats segmentsStats(boolean includeSegmentFileSizes) {
845845
return stats;
846846
}
847847

848-
private void fillSegmentStats(SegmentReader segmentReader, boolean includeSegmentFileSizes, SegmentsStats stats) {
848+
protected void fillSegmentStats(SegmentReader segmentReader, boolean includeSegmentFileSizes, SegmentsStats stats) {
849849
stats.add(1, segmentReader.ramBytesUsed());
850850
stats.addTermsMemoryInBytes(guardedRamBytesUsed(segmentReader.getPostingsReader()));
851851
stats.addStoredFieldsMemoryInBytes(guardedRamBytesUsed(segmentReader.getFieldsReader()));

server/src/main/java/org/elasticsearch/index/engine/SegmentsStats.java

+4
Original file line numberDiff line numberDiff line change
@@ -391,4 +391,8 @@ public void writeTo(StreamOutput out) throws IOException {
391391
out.writeLong(entry.value.longValue());
392392
}
393393
}
394+
395+
public void clearFileSizes() {
396+
fileSizes = ImmutableOpenMap.of();
397+
}
394398
}

server/src/main/java/org/elasticsearch/index/shard/IndexShard.java

+2-2
Original file line numberDiff line numberDiff line change
@@ -999,8 +999,8 @@ public MergeStats mergeStats() {
999999
return engine.getMergeStats();
10001000
}
10011001

1002-
public SegmentsStats segmentStats(boolean includeSegmentFileSizes) {
1003-
SegmentsStats segmentsStats = getEngine().segmentsStats(includeSegmentFileSizes);
1002+
public SegmentsStats segmentStats(boolean includeSegmentFileSizes, boolean includeUnloadedSegments) {
1003+
SegmentsStats segmentsStats = getEngine().segmentsStats(includeSegmentFileSizes, includeUnloadedSegments);
10041004
segmentsStats.addBitsetMemoryInBytes(shardBitsetFilterCache.getMemorySizeInBytes());
10051005
return segmentsStats;
10061006
}

server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestIndicesStatsAction.java

+1
Original file line numberDiff line numberDiff line change
@@ -121,6 +121,7 @@ public RestChannelConsumer prepareRequest(final RestRequest request, final NodeC
121121

122122
if (indicesStatsRequest.segments()) {
123123
indicesStatsRequest.includeSegmentFileSizes(request.paramAsBoolean("include_segment_file_sizes", false));
124+
indicesStatsRequest.includeSegmentFileSizes(request.paramAsBoolean("include_unloaded_segments", false));
124125
}
125126

126127
return channel -> client.admin().indices().stats(indicesStatsRequest, new RestToXContentListener<>(channel));

server/src/main/java/org/elasticsearch/rest/action/cat/RestIndicesAction.java

+1
Original file line numberDiff line numberDiff line change
@@ -116,6 +116,7 @@ public void processResponse(final ClusterHealthResponse clusterHealthResponse) {
116116
indicesStatsRequest.indices(indices);
117117
indicesStatsRequest.indicesOptions(strictExpandIndicesOptions);
118118
indicesStatsRequest.all();
119+
indicesStatsRequest.includeSegmentFileSizes(request.paramAsBoolean("include_unloaded_segments", false));
119120

120121
client.admin().indices().stats(indicesStatsRequest, new RestResponseListener<IndicesStatsResponse>(channel) {
121122
@Override

server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java

+23-23
Original file line numberDiff line numberDiff line change
@@ -272,8 +272,8 @@ public void testSegments() throws Exception {
272272
InternalEngine engine = createEngine(config(indexSettings, store, createTempDir(), NoMergePolicy.INSTANCE, null))) {
273273
List<Segment> segments = engine.segments(false);
274274
assertThat(segments.isEmpty(), equalTo(true));
275-
assertThat(engine.segmentsStats(false).getCount(), equalTo(0L));
276-
assertThat(engine.segmentsStats(false).getMemoryInBytes(), equalTo(0L));
275+
assertThat(engine.segmentsStats(false, false).getCount(), equalTo(0L));
276+
assertThat(engine.segmentsStats(false, false).getMemoryInBytes(), equalTo(0L));
277277

278278
// create two docs and refresh
279279
ParsedDocument doc = testParsedDocument("1", null, testDocumentWithTextField(), B_1, null);
@@ -287,7 +287,7 @@ public void testSegments() throws Exception {
287287

288288
segments = engine.segments(false);
289289
assertThat(segments.size(), equalTo(1));
290-
SegmentsStats stats = engine.segmentsStats(false);
290+
SegmentsStats stats = engine.segmentsStats(false, false);
291291
assertThat(stats.getCount(), equalTo(1L));
292292
assertThat(stats.getTermsMemoryInBytes(), greaterThan(0L));
293293
assertThat(stats.getStoredFieldsMemoryInBytes(), greaterThan(0L));
@@ -306,7 +306,7 @@ public void testSegments() throws Exception {
306306

307307
segments = engine.segments(false);
308308
assertThat(segments.size(), equalTo(1));
309-
assertThat(engine.segmentsStats(false).getCount(), equalTo(1L));
309+
assertThat(engine.segmentsStats(false, false).getCount(), equalTo(1L));
310310
assertThat(segments.get(0).isCommitted(), equalTo(true));
311311
assertThat(segments.get(0).isSearch(), equalTo(true));
312312
assertThat(segments.get(0).getNumDocs(), equalTo(2));
@@ -319,15 +319,15 @@ public void testSegments() throws Exception {
319319

320320
segments = engine.segments(false);
321321
assertThat(segments.size(), equalTo(2));
322-
assertThat(engine.segmentsStats(false).getCount(), equalTo(2L));
323-
assertThat(engine.segmentsStats(false).getTermsMemoryInBytes(),
322+
assertThat(engine.segmentsStats(false, false).getCount(), equalTo(2L));
323+
assertThat(engine.segmentsStats(false, false).getTermsMemoryInBytes(),
324324
greaterThan(stats.getTermsMemoryInBytes()));
325-
assertThat(engine.segmentsStats(false).getStoredFieldsMemoryInBytes(),
325+
assertThat(engine.segmentsStats(false, false).getStoredFieldsMemoryInBytes(),
326326
greaterThan(stats.getStoredFieldsMemoryInBytes()));
327-
assertThat(engine.segmentsStats(false).getTermVectorsMemoryInBytes(), equalTo(0L));
328-
assertThat(engine.segmentsStats(false).getNormsMemoryInBytes(),
327+
assertThat(engine.segmentsStats(false, false).getTermVectorsMemoryInBytes(), equalTo(0L));
328+
assertThat(engine.segmentsStats(false, false).getNormsMemoryInBytes(),
329329
greaterThan(stats.getNormsMemoryInBytes()));
330-
assertThat(engine.segmentsStats(false).getDocValuesMemoryInBytes(),
330+
assertThat(engine.segmentsStats(false, false).getDocValuesMemoryInBytes(),
331331
greaterThan(stats.getDocValuesMemoryInBytes()));
332332
assertThat(segments.get(0).getGeneration() < segments.get(1).getGeneration(), equalTo(true));
333333
assertThat(segments.get(0).isCommitted(), equalTo(true));
@@ -349,7 +349,7 @@ public void testSegments() throws Exception {
349349

350350
segments = engine.segments(false);
351351
assertThat(segments.size(), equalTo(2));
352-
assertThat(engine.segmentsStats(false).getCount(), equalTo(2L));
352+
assertThat(engine.segmentsStats(false, false).getCount(), equalTo(2L));
353353
assertThat(segments.get(0).getGeneration() < segments.get(1).getGeneration(), equalTo(true));
354354
assertThat(segments.get(0).isCommitted(), equalTo(true));
355355
assertThat(segments.get(0).isSearch(), equalTo(true));
@@ -370,7 +370,7 @@ public void testSegments() throws Exception {
370370

371371
segments = engine.segments(false);
372372
assertThat(segments.size(), equalTo(3));
373-
assertThat(engine.segmentsStats(false).getCount(), equalTo(3L));
373+
assertThat(engine.segmentsStats(false, false).getCount(), equalTo(3L));
374374
assertThat(segments.get(0).getGeneration() < segments.get(1).getGeneration(), equalTo(true));
375375
assertThat(segments.get(0).isCommitted(), equalTo(true));
376376
assertThat(segments.get(0).isSearch(), equalTo(true));
@@ -397,7 +397,7 @@ public void testSegments() throws Exception {
397397

398398
segments = engine.segments(false);
399399
assertThat(segments.size(), equalTo(4));
400-
assertThat(engine.segmentsStats(false).getCount(), equalTo(4L));
400+
assertThat(engine.segmentsStats(false, false).getCount(), equalTo(4L));
401401
assertThat(segments.get(0).getGeneration() < segments.get(1).getGeneration(), equalTo(true));
402402
assertThat(segments.get(0).isCommitted(), equalTo(true));
403403
assertThat(segments.get(0).isSearch(), equalTo(true));
@@ -427,7 +427,7 @@ public void testSegments() throws Exception {
427427
engine.refresh("test");
428428
segments = engine.segments(false);
429429
assertThat(segments.size(), equalTo(4));
430-
assertThat(engine.segmentsStats(false).getCount(), equalTo(4L));
430+
assertThat(engine.segmentsStats(false, false).getCount(), equalTo(4L));
431431
assertThat(segments.get(0).getGeneration() < segments.get(1).getGeneration(), equalTo(true));
432432
assertThat(segments.get(0).isCommitted(), equalTo(true));
433433
assertThat(segments.get(0).isSearch(), equalTo(true));
@@ -572,13 +572,13 @@ public void testSegmentsWithIndexSort() throws Exception {
572572
public void testSegmentsStatsIncludingFileSizes() throws Exception {
573573
try (Store store = createStore();
574574
Engine engine = createEngine(defaultSettings, store, createTempDir(), NoMergePolicy.INSTANCE)) {
575-
assertThat(engine.segmentsStats(true).getFileSizes().size(), equalTo(0));
575+
assertThat(engine.segmentsStats(true, false).getFileSizes().size(), equalTo(0));
576576

577577
ParsedDocument doc = testParsedDocument("1", null, testDocumentWithTextField(), B_1, null);
578578
engine.index(indexForDoc(doc));
579579
engine.refresh("test");
580580

581-
SegmentsStats stats = engine.segmentsStats(true);
581+
SegmentsStats stats = engine.segmentsStats(true, false);
582582
assertThat(stats.getFileSizes().size(), greaterThan(0));
583583
assertThat(() -> stats.getFileSizes().valuesIt(), everyItem(greaterThan(0L)));
584584

@@ -588,7 +588,7 @@ public void testSegmentsStatsIncludingFileSizes() throws Exception {
588588
engine.index(indexForDoc(doc2));
589589
engine.refresh("test");
590590

591-
assertThat(engine.segmentsStats(true).getFileSizes().get(firstEntry.key), greaterThan(firstEntry.value));
591+
assertThat(engine.segmentsStats(true, false).getFileSizes().get(firstEntry.key), greaterThan(firstEntry.value));
592592
}
593593
}
594594

@@ -3698,23 +3698,23 @@ public void testEngineMaxTimestampIsInitialized() throws IOException {
36983698
NoMergePolicy.INSTANCE, null, null, globalCheckpoint::get);
36993699
try (Store store = createStore(newFSDirectory(storeDir)); Engine engine = createEngine(configSupplier.apply(store))) {
37003700
assertEquals(IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP,
3701-
engine.segmentsStats(false).getMaxUnsafeAutoIdTimestamp());
3701+
engine.segmentsStats(false, false).getMaxUnsafeAutoIdTimestamp());
37023702
final ParsedDocument doc = testParsedDocument("1", null, testDocumentWithTextField(),
37033703
new BytesArray("{}".getBytes(Charset.defaultCharset())), null);
37043704
engine.index(appendOnlyPrimary(doc, true, timestamp1));
3705-
assertEquals(timestamp1, engine.segmentsStats(false).getMaxUnsafeAutoIdTimestamp());
3705+
assertEquals(timestamp1, engine.segmentsStats(false, false).getMaxUnsafeAutoIdTimestamp());
37063706
}
37073707
try (Store store = createStore(newFSDirectory(storeDir));
37083708
InternalEngine engine = new InternalEngine(configSupplier.apply(store))) {
37093709
assertEquals(IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP,
3710-
engine.segmentsStats(false).getMaxUnsafeAutoIdTimestamp());
3710+
engine.segmentsStats(false, false).getMaxUnsafeAutoIdTimestamp());
37113711
engine.initializeMaxSeqNoOfUpdatesOrDeletes();
37123712
engine.recoverFromTranslog(translogHandler, Long.MAX_VALUE);
3713-
assertEquals(timestamp1, engine.segmentsStats(false).getMaxUnsafeAutoIdTimestamp());
3713+
assertEquals(timestamp1, engine.segmentsStats(false, false).getMaxUnsafeAutoIdTimestamp());
37143714
final ParsedDocument doc = testParsedDocument("1", null, testDocumentWithTextField(),
37153715
new BytesArray("{}".getBytes(Charset.defaultCharset())), null);
37163716
engine.index(appendOnlyPrimary(doc, true, timestamp2));
3717-
assertEquals(maxTimestamp12, engine.segmentsStats(false).getMaxUnsafeAutoIdTimestamp());
3717+
assertEquals(maxTimestamp12, engine.segmentsStats(false, false).getMaxUnsafeAutoIdTimestamp());
37183718
globalCheckpoint.set(1); // make sure flush cleans up commits for later.
37193719
engine.flush();
37203720
}
@@ -3725,7 +3725,7 @@ public void testEngineMaxTimestampIsInitialized() throws IOException {
37253725
store.associateIndexWithNewTranslog(translogUUID);
37263726
}
37273727
try (Engine engine = new InternalEngine(configSupplier.apply(store))) {
3728-
assertEquals(maxTimestamp12, engine.segmentsStats(false).getMaxUnsafeAutoIdTimestamp());
3728+
assertEquals(maxTimestamp12, engine.segmentsStats(false, false).getMaxUnsafeAutoIdTimestamp());
37293729
}
37303730
}
37313731
}

server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java

+2-2
Original file line numberDiff line numberDiff line change
@@ -229,8 +229,8 @@ public void testInheritMaxValidAutoIDTimestampOnRecovery() throws Exception {
229229
IndexShard replica = shards.addReplica();
230230
shards.recoverReplica(replica);
231231

232-
SegmentsStats segmentsStats = replica.segmentStats(false);
233-
SegmentsStats primarySegmentStats = shards.getPrimary().segmentStats(false);
232+
SegmentsStats segmentsStats = replica.segmentStats(false, false);
233+
SegmentsStats primarySegmentStats = shards.getPrimary().segmentStats(false, false);
234234
assertNotEquals(IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, primarySegmentStats.getMaxUnsafeAutoIdTimestamp());
235235
assertEquals(primarySegmentStats.getMaxUnsafeAutoIdTimestamp(), segmentsStats.getMaxUnsafeAutoIdTimestamp());
236236
assertNotEquals(Long.MAX_VALUE, segmentsStats.getMaxUnsafeAutoIdTimestamp());

0 commit comments

Comments
 (0)