Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -166,7 +166,7 @@ public CommonStats(IndicesQueryCache indicesQueryCache, IndexShard indexShard, C
completion = indexShard.completionStats(flags.completionDataFields());
break;
case Segments:
segments = indexShard.segmentStats();
segments = indexShard.segmentStats(flags.includeSegmentFileSizes());
break;
case Percolate:
percolate = indexShard.percolateStats();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -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;


/**
Expand All @@ -62,6 +64,7 @@ public CommonStatsFlags all() {
groups = null;
fieldDataFields = null;
completionDataFields = null;
includeSegmentFileSizes = false;
return this;
}

Expand All @@ -74,6 +77,7 @@ public CommonStatsFlags clear() {
groups = null;
fieldDataFields = null;
completionDataFields = null;
includeSegmentFileSizes = false;
return this;
}

Expand Down Expand Up @@ -137,6 +141,15 @@ public String[] completionDataFields() {
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);
}
Expand Down Expand Up @@ -177,6 +190,9 @@ public void writeTo(StreamOutput out) throws IOException {
out.writeStringArrayNullable(groups);
out.writeStringArrayNullable(fieldDataFields);
out.writeStringArrayNullable(completionDataFields);
if (out.getVersion().onOrAfter(Version.V_5_0_0)) {
out.writeBoolean(includeSegmentFileSizes);
}
}

@Override
Expand All @@ -192,6 +208,11 @@ public void readFrom(StreamInput in) throws IOException {
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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -265,6 +265,15 @@ public boolean recovery() {
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);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -166,4 +166,9 @@ public IndicesStatsRequestBuilder setRecovery(boolean recovery) {
request.recovery(recovery);
return this;
}

public IndicesStatsRequestBuilder setIncludeSegmentFileSizes(boolean includeSegmentFileSizes) {
request.includeSegmentFileSizes(includeSegmentFileSizes);
return this;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -144,6 +144,7 @@ protected ShardStats shardOperation(IndicesStatsRequest request, ShardRouting sh
}
if (request.segments()) {
flags.set(CommonStatsFlags.Flag.Segments);
flags.includeSegmentFileSizes(request.includeSegmentFileSizes());
}
if (request.completion()) {
flags.set(CommonStatsFlags.Flag.Completion);
Expand Down
79 changes: 78 additions & 1 deletion core/src/main/java/org/elasticsearch/index/engine/Engine.java
Original file line number Diff line number Diff line change
Expand Up @@ -22,11 +22,13 @@
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;
Expand All @@ -36,12 +38,15 @@
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;
Expand All @@ -64,8 +69,11 @@
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;
Expand Down Expand Up @@ -406,7 +414,7 @@ protected static SegmentInfos readLastCommittedSegmentInfos(final SearcherManage
/**
* 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();
Expand All @@ -418,12 +426,81 @@ public final SegmentsStats segmentsStats() {
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);
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Instead of inserting 0L into the map, can we just continue here?

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);
Expand Down
Loading