Skip to content

Commit 345cc59

Browse files
committed
Add snapshot APM metrics
1 parent adc149a commit 345cc59

File tree

4 files changed

+60
-2
lines changed

4 files changed

+60
-2
lines changed

server/src/internalClusterTest/java/org/elasticsearch/repositories/SnapshotMetricsIT.java

Lines changed: 39 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -14,11 +14,13 @@
1414
import org.elasticsearch.action.admin.indices.stats.ShardStats;
1515
import org.elasticsearch.common.unit.ByteSizeValue;
1616
import org.elasticsearch.common.util.CollectionUtils;
17+
import org.elasticsearch.core.TimeValue;
1718
import org.elasticsearch.plugins.Plugin;
1819
import org.elasticsearch.plugins.PluginsService;
1920
import org.elasticsearch.snapshots.AbstractSnapshotIntegTestCase;
2021
import org.elasticsearch.telemetry.Measurement;
2122
import org.elasticsearch.telemetry.TestTelemetryPlugin;
23+
import org.elasticsearch.threadpool.ThreadPool;
2224
import org.hamcrest.Matcher;
2325

2426
import java.util.Collection;
@@ -31,6 +33,7 @@
3133
import static org.hamcrest.Matchers.everyItem;
3234
import static org.hamcrest.Matchers.greaterThan;
3335
import static org.hamcrest.Matchers.hasItem;
36+
import static org.hamcrest.Matchers.hasSize;
3437
import static org.hamcrest.Matchers.lessThan;
3538

3639
public class SnapshotMetricsIT extends AbstractSnapshotIntegTestCase {
@@ -82,6 +85,10 @@ public void testSnapshotAPMMetrics() throws Exception {
8285
waitForBlockOnAnyDataNode(repositoryName);
8386
collectMetrics();
8487
assertShardsInProgressMetricIs(hasItem(greaterThan(0L)));
88+
assertThat(getTotalClusterLongCounterValue(SnapshotMetrics.SNAPSHOT_STARTED), equalTo(1L));
89+
assertThat(getTotalClusterLongCounterValue(SnapshotMetrics.SNAPSHOT_COMPLETED), equalTo(0L));
90+
assertThat(getTotalClusterLongCounterValue(SnapshotMetrics.SNAPSHOT_SHARDS_STARTED), greaterThan(0L));
91+
assertThat(getTotalClusterLongCounterValue(SnapshotMetrics.SNAPSHOT_SHARDS_COMPLETED), equalTo(0L));
8592
} finally {
8693
unblockAllDataNodes(repositoryName);
8794
}
@@ -96,9 +103,31 @@ public void testSnapshotAPMMetrics() throws Exception {
96103
assertThat(getTotalClusterLongCounterValue(SnapshotMetrics.SNAPSHOT_BYTES_UPLOADED), greaterThan(0L));
97104
assertThat(getTotalClusterLongCounterValue(SnapshotMetrics.SNAPSHOT_CREATE_THROTTLE_DURATION), greaterThan(0L));
98105
assertThat(getTotalClusterLongCounterValue(SnapshotMetrics.SNAPSHOT_RESTORE_THROTTLE_DURATION), equalTo(0L));
106+
assertThat(getTotalClusterLongCounterValue(SnapshotMetrics.SNAPSHOT_STARTED), equalTo(1L));
107+
assertThat(getTotalClusterLongCounterValue(SnapshotMetrics.SNAPSHOT_COMPLETED), equalTo(1L));
108+
109+
// Sanity check shard duration observations
110+
assertDoubleHistogramMetrics(SnapshotMetrics.SNAPSHOT_SHARDS_DURATION, hasSize(numShards));
111+
assertDoubleHistogramMetrics(
112+
SnapshotMetrics.SNAPSHOT_SHARDS_DURATION,
113+
everyItem(lessThan(TimeValue.timeValueNanos(snapshotElapsedTimeNanos).secondsFrac()))
114+
);
115+
116+
// Sanity check snapshot observations
117+
assertDoubleHistogramMetrics(SnapshotMetrics.SNAPSHOT_DURATION, hasSize(1));
118+
assertDoubleHistogramMetrics(
119+
SnapshotMetrics.SNAPSHOT_DURATION,
120+
everyItem(lessThan(TimeValue.timeValueNanos(snapshotElapsedTimeNanos).secondsFrac()))
121+
);
122+
123+
// Work out the maximum amount of concurrency per node
124+
final ThreadPool tp = internalCluster().getDataNodeInstance(ThreadPool.class);
125+
int snapshotThreadPoolSize = tp.info(ThreadPool.Names.SNAPSHOT).getMax();
126+
int maximumPerNodeConcurrency = Math.max(snapshotThreadPoolSize, numShards);
99127

100128
// sanity check duration values
101-
final long upperBoundTimeSpentOnSnapshotThingsNanos = internalCluster().numDataNodes() * snapshotElapsedTimeNanos;
129+
final long upperBoundTimeSpentOnSnapshotThingsNanos = internalCluster().numDataNodes() * maximumPerNodeConcurrency
130+
* snapshotElapsedTimeNanos;
102131
assertThat(
103132
getTotalClusterLongCounterValue(SnapshotMetrics.SNAPSHOT_UPLOAD_DURATION),
104133
allOf(greaterThan(0L), lessThan(upperBoundTimeSpentOnSnapshotThingsNanos))
@@ -126,6 +155,14 @@ public void testSnapshotAPMMetrics() throws Exception {
126155
assertThat(getTotalClusterLongCounterValue(SnapshotMetrics.SNAPSHOT_RESTORE_THROTTLE_DURATION), greaterThan(0L));
127156
}
128157

158+
private static void assertDoubleHistogramMetrics(String metricName, Matcher<? super List<Double>> matcher) {
159+
final List<Double> values = allTestTelemetryPlugins().flatMap(testTelemetryPlugin -> {
160+
final List<Measurement> doubleHistogramMeasurement = testTelemetryPlugin.getDoubleHistogramMeasurement(metricName);
161+
return doubleHistogramMeasurement.stream().map(Measurement::getDouble);
162+
}).toList();
163+
assertThat(values, matcher);
164+
}
165+
129166
private static void assertShardsInProgressMetricIs(Matcher<? super List<Long>> matcher) {
130167
final List<Long> values = allTestTelemetryPlugins().map(testTelemetryPlugin -> {
131168
final List<Measurement> longGaugeMeasurement = testTelemetryPlugin.getLongGaugeMeasurement(
@@ -147,7 +184,7 @@ private long getTotalClusterLongCounterValue(String metricName) {
147184
}
148185

149186
private static Stream<TestTelemetryPlugin> allTestTelemetryPlugins() {
150-
return StreamSupport.stream(internalCluster().getDataNodeInstances(PluginsService.class).spliterator(), false)
187+
return StreamSupport.stream(internalCluster().getDataOrMasterNodeInstances(PluginsService.class).spliterator(), false)
151188
.flatMap(pluginsService -> pluginsService.filterPlugins(TestTelemetryPlugin.class));
152189
}
153190
}

server/src/main/java/org/elasticsearch/repositories/RepositoriesService.java

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1269,6 +1269,10 @@ public RepositoryUsageStats getUsageStats() {
12691269
);
12701270
}
12711271

1272+
public SnapshotMetrics getSnapshotMetrics() {
1273+
return snapshotMetrics;
1274+
}
1275+
12721276
@Override
12731277
protected void doStart() {}
12741278

server/src/main/java/org/elasticsearch/repositories/SnapshotMetrics.java

Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,9 @@
2222
import java.util.function.Supplier;
2323

2424
public record SnapshotMetrics(
25+
LongCounter snapshotsStartedCounter,
26+
LongCounter snapshotsCompletedCounter,
27+
DoubleHistogram snapshotsDurationHistogram,
2528
LongCounter snapshotsShardsStartedCounter,
2629
LongCounter snapshotsShardsCompletedCounter,
2730
LongGauge snapshotShardsInProgressGauge,
@@ -36,6 +39,9 @@ public record SnapshotMetrics(
3639

3740
public static final SnapshotMetrics NOOP = new SnapshotMetrics(MeterRegistry.NOOP, List::of);
3841

42+
public static final String SNAPSHOT_STARTED = "es.repositories.snapshots.started.total";
43+
public static final String SNAPSHOT_COMPLETED = "es.repositories.snapshots.completed.total";
44+
public static final String SNAPSHOT_DURATION = "es.repositories.snapshots.duration.histogram";
3945
public static final String SNAPSHOT_SHARDS_STARTED = "es.repositories.snapshots.shards.started.total";
4046
public static final String SNAPSHOT_SHARDS_COMPLETED = "es.repositories.snapshots.shards.completed.total";
4147
public static final String SNAPSHOT_SHARDS_IN_PROGRESS = "es.repositories.snapshots.shards.current";
@@ -49,6 +55,9 @@ public record SnapshotMetrics(
4955

5056
public SnapshotMetrics(MeterRegistry meterRegistry, Supplier<Collection<LongWithAttributes>> shardSnapshotsInProgressObserver) {
5157
this(
58+
meterRegistry.registerLongCounter(SNAPSHOT_STARTED, "snapshots started", "unit"),
59+
meterRegistry.registerLongCounter(SNAPSHOT_COMPLETED, "snapshots completed", "unit"),
60+
meterRegistry.registerDoubleHistogram(SNAPSHOT_DURATION, "snapshots duration", "s"),
5261
meterRegistry.registerLongCounter(SNAPSHOT_SHARDS_STARTED, "shard snapshots started", "unit"),
5362
meterRegistry.registerLongCounter(SNAPSHOT_SHARDS_COMPLETED, "shard snapshots completed", "unit"),
5463
meterRegistry.registerLongsGauge(

server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -101,6 +101,7 @@
101101
import org.elasticsearch.repositories.ShardGeneration;
102102
import org.elasticsearch.repositories.ShardGenerations;
103103
import org.elasticsearch.repositories.ShardSnapshotResult;
104+
import org.elasticsearch.repositories.SnapshotMetrics;
104105
import org.elasticsearch.tasks.Task;
105106
import org.elasticsearch.threadpool.ThreadPool;
106107
import org.elasticsearch.transport.TransportService;
@@ -1596,6 +1597,11 @@ protected void doRun() {
15961597
@Override
15971598
public void onResponse(List<ActionListener<SnapshotInfo>> actionListeners) {
15981599
completeListenersIgnoringException(actionListeners, snapshotInfo);
1600+
final Map<String, Object> attributes = SnapshotMetrics.createAttributesMap(repo.getMetadata());
1601+
final SnapshotMetrics snapshotMetrics = repositoriesService.getSnapshotMetrics();
1602+
snapshotMetrics.snapshotsCompletedCounter().incrementBy(1, attributes);
1603+
snapshotMetrics.snapshotsDurationHistogram()
1604+
.record((snapshotInfo.endTime() - snapshotInfo.startTime()) / 1_000.0, attributes);
15991605
logger.info("snapshot [{}] completed with state [{}]", snapshot, snapshotInfo.state());
16001606
}
16011607

@@ -4323,6 +4329,8 @@ private SnapshotsInProgress createSnapshot(
43234329
final var res = snapshotsInProgress.withAddedEntry(newEntry);
43244330
taskContext.success(() -> {
43254331
logger.info("snapshot [{}] started", snapshot);
4332+
final Map<String, Object> attributes = SnapshotMetrics.createAttributesMap(repository.getMetadata());
4333+
repositoriesService.getSnapshotMetrics().snapshotsStartedCounter().incrementBy(1, attributes);
43264334
createSnapshotTask.listener.onResponse(snapshot);
43274335
if (newEntry.state().completed()) {
43284336
endSnapshot(newEntry, currentState.metadata(), createSnapshotTask.repositoryData);

0 commit comments

Comments
 (0)