Skip to content

Commit

Permalink
[7.x] Include min/max/average file size in Searchable Snapshots Stats…
Browse files Browse the repository at this point in the history
… API (#70327)

This commit adds the minimum, maximum and average length
of files in the Searchable Snapshots Stats API.

Backport of #70294 for 7.x
  • Loading branch information
tlrx authored Mar 12, 2021
1 parent 3e13074 commit 6bc7c6c
Show file tree
Hide file tree
Showing 13 changed files with 192 additions and 41 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@
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.unit.ByteSizeValue;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.ToXContentObject;
import org.elasticsearch.common.xcontent.XContentBuilder;
Expand Down Expand Up @@ -125,7 +126,9 @@ public static class CacheIndexInputStats implements Writeable, ToXContentObject

private final String fileExt;
private final long numFiles;
private final long totalSize;
private final ByteSizeValue totalSize;
private final ByteSizeValue minSize;
private final ByteSizeValue maxSize;

private final long openCount;
private final long closeCount;
Expand All @@ -144,7 +147,8 @@ public static class CacheIndexInputStats implements Writeable, ToXContentObject
private final Counter blobStoreBytesRequested;
private final long currentIndexCacheFills;

public CacheIndexInputStats(String fileExt, long numFiles, long totalSize, long openCount, long closeCount,
public CacheIndexInputStats(String fileExt, long numFiles, ByteSizeValue totalSize, ByteSizeValue minSize, ByteSizeValue maxSize,
long openCount, long closeCount,
Counter forwardSmallSeeks, Counter backwardSmallSeeks,
Counter forwardLargeSeeks, Counter backwardLargeSeeks,
Counter contiguousReads, Counter nonContiguousReads,
Expand All @@ -154,6 +158,8 @@ public CacheIndexInputStats(String fileExt, long numFiles, long totalSize, long
this.fileExt = fileExt;
this.numFiles = numFiles;
this.totalSize = totalSize;
this.minSize = minSize;
this.maxSize = maxSize;
this.openCount = openCount;
this.closeCount = closeCount;
this.forwardSmallSeeks = forwardSmallSeeks;
Expand All @@ -180,7 +186,15 @@ public CacheIndexInputStats(String fileExt, long numFiles, long totalSize, long
}
this.fileExt = in.readString();
this.numFiles = in.readVLong();
this.totalSize = in.readVLong();
if (in.getVersion().before(Version.V_7_13_0)) {
this.totalSize = new ByteSizeValue(in.readVLong());
this.minSize = ByteSizeValue.ZERO;
this.maxSize = ByteSizeValue.ZERO;
} else {
this.totalSize = new ByteSizeValue(in);
this.minSize = new ByteSizeValue(in);
this.maxSize = new ByteSizeValue(in);
}
this.openCount = in.readVLong();
this.closeCount = in.readVLong();
this.forwardSmallSeeks = new Counter(in);
Expand Down Expand Up @@ -216,7 +230,9 @@ public static CacheIndexInputStats combine(CacheIndexInputStats cis1, CacheIndex
return new CacheIndexInputStats(
cis1.fileExt,
cis1.numFiles + cis2.numFiles,
cis1.totalSize + cis2.totalSize,
new ByteSizeValue(Math.addExact(cis1.totalSize.getBytes(), cis2.totalSize.getBytes())),
new ByteSizeValue(Math.min(cis1.minSize.getBytes(), cis2.minSize.getBytes())),
new ByteSizeValue(Math.max(cis1.maxSize.getBytes(), cis2.maxSize.getBytes())),
cis1.openCount + cis2.openCount,
cis1.closeCount + cis2.closeCount,
cis1.forwardSmallSeeks.add(cis2.forwardSmallSeeks),
Expand Down Expand Up @@ -245,7 +261,13 @@ public void writeTo(StreamOutput out) throws IOException {
}
out.writeString(fileExt);
out.writeVLong(numFiles);
out.writeVLong(totalSize);
if (out.getVersion().before(Version.V_7_13_0)) {
out.writeVLong(totalSize.getBytes());
} else {
totalSize.writeTo(out);
minSize.writeTo(out);
maxSize.writeTo(out);
}
out.writeVLong(openCount);
out.writeVLong(closeCount);

Expand Down Expand Up @@ -276,10 +298,23 @@ public long getNumFiles() {
return numFiles;
}

public long getTotalSize() {
public ByteSizeValue getTotalSize() {
return totalSize;
}

public ByteSizeValue getMinSize() {
return minSize;
}

public ByteSizeValue getMaxSize() {
return maxSize;
}

public ByteSizeValue getAverageSize() {
final double average = (double) totalSize.getBytes()/ (double) numFiles;
return new ByteSizeValue(Math.round(average));
}

public long getOpenCount() {
return openCount;
}
Expand Down Expand Up @@ -346,9 +381,16 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws
{
builder.field("file_ext", getFileExt());
builder.field("num_files", getNumFiles());
builder.field("total_size", getTotalSize());
builder.field("open_count", getOpenCount());
builder.field("close_count", getCloseCount());
{
builder.startObject("size");
builder.humanReadableField("total_in_bytes", "total", getTotalSize());
builder.humanReadableField("min_in_bytes", "min", getMinSize());
builder.humanReadableField("max_in_bytes", "max", getMaxSize());
builder.humanReadableField("average_in_bytes", "average", getAverageSize());
builder.endObject();
}
builder.field("contiguous_bytes_read", getContiguousReads(), params);
builder.field("non_contiguous_bytes_read", getNonContiguousReads(), params);
builder.field("cached_bytes_read", getCachedBytesRead(), params);
Expand Down Expand Up @@ -384,10 +426,12 @@ public boolean equals(Object other) {
}
CacheIndexInputStats stats = (CacheIndexInputStats) other;
return numFiles == stats.numFiles
&& totalSize == stats.totalSize
&& openCount == stats.openCount
&& closeCount == stats.closeCount
&& Objects.equals(fileExt, stats.fileExt)
&& Objects.equals(totalSize, stats.totalSize)
&& Objects.equals(minSize, stats.minSize)
&& Objects.equals(maxSize, stats.maxSize)
&& Objects.equals(forwardSmallSeeks, stats.forwardSmallSeeks)
&& Objects.equals(backwardSmallSeeks, stats.backwardSmallSeeks)
&& Objects.equals(forwardLargeSeeks, stats.forwardLargeSeeks)
Expand All @@ -405,7 +449,9 @@ public boolean equals(Object other) {

@Override
public int hashCode() {
return Objects.hash(fileExt, numFiles, totalSize, openCount, closeCount,
return Objects.hash(fileExt, numFiles, totalSize,
minSize, maxSize,
openCount, closeCount,
forwardSmallSeeks, backwardSmallSeeks,
forwardLargeSeeks, backwardLargeSeeks,
contiguousReads, nonContiguousReads,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,7 @@
import org.elasticsearch.cluster.routing.ShardRoutingState;
import org.elasticsearch.cluster.routing.TestShardRouting;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.repositories.IndexId;
import org.elasticsearch.snapshots.SnapshotId;
import org.elasticsearch.test.AbstractWireSerializingTestCase;
Expand Down Expand Up @@ -42,7 +43,8 @@ protected SearchableSnapshotShardStats createTestInstance() {
}

private CacheIndexInputStats randomCacheIndexInputStats() {
return new CacheIndexInputStats(randomAlphaOfLength(10), randomNonNegativeLong(), randomNonNegativeLong(),
return new CacheIndexInputStats(randomAlphaOfLength(10), randomNonNegativeLong(), new ByteSizeValue(randomNonNegativeLong()),
new ByteSizeValue(randomNonNegativeLong()), new ByteSizeValue(randomNonNegativeLong()),
randomNonNegativeLong(), randomNonNegativeLong(),
randomCounter(), randomCounter(),
randomCounter(), randomCounter(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -66,8 +66,8 @@ teardown:
---
"Tests searchable snapshots stats":
- skip:
version: " - 7.7.99"
reason: searchable snapshots introduced in 7.8.0
version: " - 7.12.99"
reason: searchable snapshots response format changed in 7.13.0

- do:
catch: missing
Expand Down Expand Up @@ -139,10 +139,14 @@ teardown:
- is_true: total
- is_true: total.0.file_ext
- gt: { total.0.num_files: 0 }
- gt: { total.0.total_size: 0 }
- gt: { total.0.open_count: 0 }
- gt: { total.0.close_count: 0 }

- gt: { total.0.size.total_in_bytes: 0 }
- gt: { total.0.size.min_in_bytes: 0 }
- gt: { total.0.size.max_in_bytes: 0 }
- gt: { total.0.size.average_in_bytes: 0 }

- gte: { total.0.contiguous_bytes_read.count: 0 }
- gte: { total.0.contiguous_bytes_read.sum: 0 }
- gte: { total.0.contiguous_bytes_read.min: 0 }
Expand Down Expand Up @@ -215,10 +219,14 @@ teardown:

- is_true: indices.docs.total.0.file_ext
- gt: { indices.docs.total.0.num_files: 0 }
- gt: { indices.docs.total.0.total_size: 0 }
- gt: { indices.docs.total.0.open_count: 0 }
- gt: { indices.docs.total.0.close_count: 0 }

- gt: { indices.docs.total.0.size.total_in_bytes: 0 }
- gt: { indices.docs.total.0.size.min_in_bytes: 0 }
- gt: { indices.docs.total.0.size.max_in_bytes: 0 }
- gt: { indices.docs.total.0.size.average_in_bytes: 0 }

- gte: { indices.docs.total.0.contiguous_bytes_read.count: 0 }
- gte: { indices.docs.total.0.contiguous_bytes_read.sum: 0 }
- gte: { indices.docs.total.0.contiguous_bytes_read.min: 0 }
Expand Down Expand Up @@ -295,10 +303,14 @@ teardown:

- is_true: indices.docs.shards.0.0.files.0.file_ext
- gt: { indices.docs.shards.0.0.files.0.num_files: 0 }
- gt: { indices.docs.shards.0.0.files.0.total_size: 0 }
- gt: { indices.docs.shards.0.0.files.0.open_count: 0 }
- gt: { indices.docs.shards.0.0.files.0.close_count: 0 }

- gt: { indices.docs.shards.0.0.files.0.size.total_in_bytes: 0 }
- gt: { indices.docs.shards.0.0.files.0.size.min_in_bytes: 0 }
- gt: { indices.docs.shards.0.0.files.0.size.max_in_bytes: 0 }
- gt: { indices.docs.shards.0.0.files.0.size.average_in_bytes: 0 }

- gte: { indices.docs.shards.0.0.files.0.contiguous_bytes_read.count: 0 }
- gte: { indices.docs.shards.0.0.files.0.contiguous_bytes_read.sum: 0 }
- gte: { indices.docs.shards.0.0.files.0.contiguous_bytes_read.min: 0 }
Expand Down Expand Up @@ -373,6 +385,10 @@ teardown:

- is_true: indices.docs.shards.0.0.files.0.cached_bytes_written.time
- is_true: indices.docs.shards.0.0.files.0.direct_bytes_read.time
- is_true: indices.docs.shards.0.0.files.0.size.total
- is_true: indices.docs.shards.0.0.files.0.size.min
- is_true: indices.docs.shards.0.0.files.0.size.max
- is_true: indices.docs.shards.0.0.files.0.size.average

- do:
searchable_snapshots.stats:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1446,7 +1446,7 @@ private void assertSearchableSnapshotStats(String indexName, boolean cacheEnable
final long totalSize = statsResponse.getStats()
.stream()
.flatMap(s -> s.getStats().stream())
.mapToLong(SearchableSnapshotShardStats.CacheIndexInputStats::getTotalSize)
.mapToLong(stat -> stat.getTotalSize().getBytes())
.sum();
final Set<String> nodeIdsWithLargeEnoughCache = new HashSet<>();
for (ObjectCursor<DiscoveryNode> nodeCursor : client().admin()
Expand Down Expand Up @@ -1484,7 +1484,22 @@ private void assertSearchableSnapshotStats(String indexName, boolean cacheEnable
);
assertThat(
"Unexpected file length for " + fileExt + " of shard " + shardRouting,
indexInputStats.getTotalSize(),
indexInputStats.getTotalSize().getBytes(),
greaterThan(0L)
);
assertThat(
"Unexpected min. file length for " + fileExt + " of shard " + shardRouting,
indexInputStats.getMinSize().getBytes(),
greaterThan(0L)
);
assertThat(
"Unexpected max. file length for " + fileExt + " of shard " + shardRouting,
indexInputStats.getMaxSize().getBytes(),
greaterThan(0L)
);
assertThat(
"Unexpected average file length for " + fileExt + " of shard " + shardRouting,
indexInputStats.getAverageSize().getBytes(),
greaterThan(0L)
);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,8 +25,11 @@ public class IndexInputStats {
/* A threshold beyond which an index input seeking is counted as "large" */
static final ByteSizeValue SEEKING_THRESHOLD = new ByteSizeValue(8, ByteSizeUnit.MB);

private final int numFiles;
private final long numFiles;
private final long totalSize;
private final long minSize;
private final long maxSize;

private final long seekingThreshold;
private final LongSupplier currentTimeNanos;

Expand All @@ -52,13 +55,22 @@ public class IndexInputStats {
private final Counter blobStoreBytesRequested = new Counter();
private final AtomicLong currentIndexCacheFills = new AtomicLong();

public IndexInputStats(int numFiles, long totalSize, LongSupplier currentTimeNanos) {
this(numFiles, totalSize, SEEKING_THRESHOLD.getBytes(), currentTimeNanos);
public IndexInputStats(long numFiles, long totalSize, long minSize, long maxSize, LongSupplier currentTimeNanos) {
this(numFiles, totalSize, minSize, maxSize, SEEKING_THRESHOLD.getBytes(), currentTimeNanos);
}

public IndexInputStats(int numFiles, long totalSize, long seekingThreshold, LongSupplier currentTimeNanos) {
public IndexInputStats(
long numFiles,
long totalSize,
long minSize,
long maxSize,
long seekingThreshold,
LongSupplier currentTimeNanos
) {
this.numFiles = numFiles;
this.totalSize = totalSize;
this.minSize = minSize;
this.maxSize = maxSize;
this.seekingThreshold = seekingThreshold;
this.currentTimeNanos = currentTimeNanos;
}
Expand Down Expand Up @@ -137,14 +149,22 @@ public Releasable addIndexCacheFill() {
};
}

public int getNumFiles() {
public long getNumFiles() {
return numFiles;
}

public long getTotalSize() {
return totalSize;
}

public long getMinSize() {
return minSize;
}

public long getMaxSize() {
return maxSize;
}

public LongAdder getOpened() {
return opened;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -360,8 +360,8 @@ public void clearCache() {
}
}

protected IndexInputStats createIndexInputStats(final int numFiles, final long totalSize) {
return new IndexInputStats(numFiles, totalSize, statsCurrentTimeNanosSupplier);
protected IndexInputStats createIndexInputStats(long numFiles, long totalSize, long minSize, long maxSize) {
return new IndexInputStats(numFiles, totalSize, minSize, maxSize, statsCurrentTimeNanosSupplier);
}

public CacheKey createCacheKey(String fileName) {
Expand Down Expand Up @@ -395,13 +395,13 @@ public IndexInput openInput(final String name, final IOContext context) throws I

final String ext = getNonNullFileExt(name);
final IndexInputStats inputStats = stats.computeIfAbsent(ext, n -> {
// get all fileInfo with same extension
final Tuple<Integer, Long> fileExtCompoundStats = files().stream()
.filter(fi -> ext.equals(getNonNullFileExt(fi.physicalName())))
.map(fi -> Tuple.tuple(1, fi.length()))
.reduce((t1, t2) -> Tuple.tuple(t1.v1() + t2.v1(), t1.v2() + t2.v2()))
.get();
return createIndexInputStats(fileExtCompoundStats.v1(), fileExtCompoundStats.v2());
final IndexInputStats.Counter counter = new IndexInputStats.Counter();
for (BlobStoreIndexShardSnapshot.FileInfo file : files()) {
if (ext.equals(getNonNullFileExt(file.physicalName()))) {
counter.add(file.length());
}
}
return createIndexInputStats(counter.count(), counter.total(), counter.min(), counter.max());
});
if (useCache && isExcludedFromCache(name) == false) {
if (partial) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -65,6 +65,7 @@ private static List<CacheIndexInputStats> computeCompound(Stream<CacheIndexInput
.stream()
.filter(Optional::isPresent)
.map(Optional::get)
.sorted(Comparator.comparing(CacheIndexInputStats::getFileExt))
.collect(toList());
}

Expand Down
Loading

0 comments on commit 6bc7c6c

Please sign in to comment.