Browse Source

Expose reconciliation metrics via APM (#102244)

Ievgen Degtiarenko 1 year ago
parent
commit
4474bbd153
18 changed files with 156 additions and 38 deletions
  1. 5 0
      docs/changelog/102244.yaml
  2. 1 1
      qa/smoke-test-multinode/src/yamlRestTest/resources/rest-api-spec/test/smoke_test_multinode/30_desired_balance.yml
  3. 1 1
      rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/cluster.desired_balance/10_basic.yml
  4. 9 4
      server/src/main/java/org/elasticsearch/cluster/ClusterModule.java
  5. 38 5
      server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceReconciler.java
  6. 12 4
      server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceShardsAllocator.java
  7. 2 2
      server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceStats.java
  8. 2 1
      server/src/main/java/org/elasticsearch/node/NodeConstruction.java
  9. 3 0
      server/src/main/java/org/elasticsearch/telemetry/metric/DoubleWithAttributes.java
  10. 33 0
      server/src/main/java/org/elasticsearch/telemetry/metric/LongGaugeMetric.java
  11. 3 0
      server/src/main/java/org/elasticsearch/telemetry/metric/LongWithAttributes.java
  12. 9 1
      server/src/test/java/org/elasticsearch/action/admin/cluster/allocation/TransportDeleteDesiredBalanceActionTests.java
  13. 10 6
      server/src/test/java/org/elasticsearch/cluster/ClusterModuleTests.java
  14. 3 1
      server/src/test/java/org/elasticsearch/cluster/routing/allocation/allocator/ClusterAllocationSimulationTests.java
  15. 7 3
      server/src/test/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceReconcilerTests.java
  16. 13 6
      server/src/test/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceShardsAllocatorTests.java
  17. 2 2
      server/src/test/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceStatsTests.java
  18. 3 1
      test/framework/src/main/java/org/elasticsearch/cluster/ESAllocationTestCase.java

+ 5 - 0
docs/changelog/102244.yaml

@@ -0,0 +1,5 @@
+pr: 102244
+summary: Expose reconciliation metrics via APM
+area: Allocation
+type: enhancement
+issues: []

+ 1 - 1
qa/smoke-test-multinode/src/yamlRestTest/resources/rest-api-spec/test/smoke_test_multinode/30_desired_balance.yml

@@ -200,4 +200,4 @@ setup:
   - gte: { 'stats.unassigned_shards' : 0 }
   - gte: { 'stats.total_allocations' : 0 }
   - gte: { 'stats.undesired_allocations' : 0 }
-  - gte: { 'stats.undesired_allocations_fraction' : 0.0 }
+  - gte: { 'stats.undesired_allocations_ratio' : 0.0 }

+ 1 - 1
rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/cluster.desired_balance/10_basic.yml

@@ -235,4 +235,4 @@ setup:
   - gte: { 'stats.unassigned_shards' : 0 }
   - gte: { 'stats.total_allocations' : 0 }
   - gte: { 'stats.undesired_allocations' : 0 }
-  - gte: { 'stats.undesired_allocations_fraction' : 0.0 }
+  - gte: { 'stats.undesired_allocations_ratio' : 0.0 }

+ 9 - 4
server/src/main/java/org/elasticsearch/cluster/ClusterModule.java

@@ -77,6 +77,7 @@ import org.elasticsearch.script.ScriptMetadata;
 import org.elasticsearch.snapshots.SnapshotsInfoService;
 import org.elasticsearch.tasks.Task;
 import org.elasticsearch.tasks.TaskResultsService;
+import org.elasticsearch.telemetry.TelemetryProvider;
 import org.elasticsearch.threadpool.ThreadPool;
 import org.elasticsearch.upgrades.FeatureMigrationResults;
 import org.elasticsearch.xcontent.NamedXContentRegistry;
@@ -126,7 +127,8 @@ public class ClusterModule extends AbstractModule {
         SnapshotsInfoService snapshotsInfoService,
         ThreadPool threadPool,
         SystemIndices systemIndices,
-        WriteLoadForecaster writeLoadForecaster
+        WriteLoadForecaster writeLoadForecaster,
+        TelemetryProvider telemetryProvider
     ) {
         this.clusterPlugins = clusterPlugins;
         this.deciderList = createAllocationDeciders(settings, clusterService.getClusterSettings(), clusterPlugins);
@@ -138,7 +140,8 @@ public class ClusterModule extends AbstractModule {
             clusterPlugins,
             clusterService,
             this::reconcile,
-            writeLoadForecaster
+            writeLoadForecaster,
+            telemetryProvider
         );
         this.clusterService = clusterService;
         this.indexNameExpressionResolver = new IndexNameExpressionResolver(threadPool.getThreadContext(), systemIndices);
@@ -381,7 +384,8 @@ public class ClusterModule extends AbstractModule {
         List<ClusterPlugin> clusterPlugins,
         ClusterService clusterService,
         DesiredBalanceReconcilerAction reconciler,
-        WriteLoadForecaster writeLoadForecaster
+        WriteLoadForecaster writeLoadForecaster,
+        TelemetryProvider telemetryProvider
     ) {
         Map<String, Supplier<ShardsAllocator>> allocators = new HashMap<>();
         allocators.put(BALANCED_ALLOCATOR, () -> new BalancedShardsAllocator(clusterSettings, writeLoadForecaster));
@@ -392,7 +396,8 @@ public class ClusterModule extends AbstractModule {
                 new BalancedShardsAllocator(clusterSettings, writeLoadForecaster),
                 threadPool,
                 clusterService,
-                reconciler
+                reconciler,
+                telemetryProvider
             )
         );
 

+ 38 - 5
server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceReconciler.java

@@ -29,12 +29,15 @@ import org.elasticsearch.core.Tuple;
 import org.elasticsearch.gateway.PriorityComparator;
 import org.elasticsearch.index.IndexVersions;
 import org.elasticsearch.index.shard.ShardId;
+import org.elasticsearch.telemetry.metric.DoubleGauge;
+import org.elasticsearch.telemetry.metric.DoubleWithAttributes;
+import org.elasticsearch.telemetry.metric.LongGaugeMetric;
+import org.elasticsearch.telemetry.metric.MeterRegistry;
 import org.elasticsearch.threadpool.ThreadPool;
 
 import java.util.Comparator;
 import java.util.Iterator;
 import java.util.Set;
-import java.util.concurrent.atomic.AtomicLong;
 import java.util.function.BiFunction;
 import java.util.stream.Collectors;
 import java.util.stream.IntStream;
@@ -74,23 +77,53 @@ public class DesiredBalanceReconciler {
     /**
      * Number of unassigned shards during last reconciliation
      */
-    protected final AtomicLong unassignedShards = new AtomicLong();
+    protected final LongGaugeMetric unassignedShards;
     /**
      * Total number of assigned shards during last reconciliation
      */
-    protected final AtomicLong totalAllocations = new AtomicLong();
+    protected final LongGaugeMetric totalAllocations;
     /**
      * Number of assigned shards during last reconciliation that are not allocated on desired node and need to be moved
      */
-    protected final AtomicLong undesiredAllocations = new AtomicLong();
+    protected final LongGaugeMetric undesiredAllocations;
+    private final DoubleGauge undesiredAllocationsRatio;
 
-    public DesiredBalanceReconciler(ClusterSettings clusterSettings, ThreadPool threadPool) {
+    public DesiredBalanceReconciler(ClusterSettings clusterSettings, ThreadPool threadPool, MeterRegistry meterRegistry) {
         this.undesiredAllocationLogInterval = new FrequencyCappedAction(threadPool);
         clusterSettings.initializeAndWatch(UNDESIRED_ALLOCATIONS_LOG_INTERVAL_SETTING, this.undesiredAllocationLogInterval::setMinInterval);
         clusterSettings.initializeAndWatch(
             UNDESIRED_ALLOCATIONS_LOG_THRESHOLD_SETTING,
             value -> this.undesiredAllocationsLogThreshold = value
         );
+
+        unassignedShards = LongGaugeMetric.create(
+            meterRegistry,
+            "es.allocator.desired_balance.unassigned_shards",
+            "Current unassigned shards count",
+            "count"
+        );
+        totalAllocations = LongGaugeMetric.create(
+            meterRegistry,
+            "es.allocator.desired_balance.total_allocations",
+            "Current total shards count in cluster",
+            "count"
+        );
+        undesiredAllocations = LongGaugeMetric.create(
+            meterRegistry,
+            "es.allocator.desired_balance.undesired_allocations",
+            "Current number of shards allocated on undesired nodes",
+            "count"
+        );
+        undesiredAllocationsRatio = meterRegistry.registerDoubleGauge(
+            "es.allocator.desired_balance.undesired_allocations.ratio",
+            "Current undesired_allocations / allocations ratio",
+            "count",
+            () -> {
+                var total = totalAllocations.get();
+                var undesired = undesiredAllocations.get();
+                return new DoubleWithAttributes(total != 0 ? (double) undesired / total : 0.0);
+            }
+        );
     }
 
     public void reconcile(DesiredBalance desiredBalance, RoutingAllocation allocation) {

+ 12 - 4
server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceShardsAllocator.java

@@ -29,6 +29,7 @@ import org.elasticsearch.common.Priority;
 import org.elasticsearch.common.metrics.CounterMetric;
 import org.elasticsearch.common.metrics.MeanMetric;
 import org.elasticsearch.common.settings.ClusterSettings;
+import org.elasticsearch.telemetry.TelemetryProvider;
 import org.elasticsearch.threadpool.ThreadPool;
 
 import java.util.ArrayList;
@@ -77,14 +78,16 @@ public class DesiredBalanceShardsAllocator implements ShardsAllocator {
         ShardsAllocator delegateAllocator,
         ThreadPool threadPool,
         ClusterService clusterService,
-        DesiredBalanceReconcilerAction reconciler
+        DesiredBalanceReconcilerAction reconciler,
+        TelemetryProvider telemetryProvider
     ) {
         this(
             delegateAllocator,
             threadPool,
             clusterService,
             new DesiredBalanceComputer(clusterSettings, threadPool, delegateAllocator),
-            reconciler
+            reconciler,
+            telemetryProvider
         );
     }
 
@@ -93,13 +96,18 @@ public class DesiredBalanceShardsAllocator implements ShardsAllocator {
         ThreadPool threadPool,
         ClusterService clusterService,
         DesiredBalanceComputer desiredBalanceComputer,
-        DesiredBalanceReconcilerAction reconciler
+        DesiredBalanceReconcilerAction reconciler,
+        TelemetryProvider telemetryProvider
     ) {
         this.delegateAllocator = delegateAllocator;
         this.threadPool = threadPool;
         this.reconciler = reconciler;
         this.desiredBalanceComputer = desiredBalanceComputer;
-        this.desiredBalanceReconciler = new DesiredBalanceReconciler(clusterService.getClusterSettings(), threadPool);
+        this.desiredBalanceReconciler = new DesiredBalanceReconciler(
+            clusterService.getClusterSettings(),
+            threadPool,
+            telemetryProvider.getMeterRegistry()
+        );
         this.desiredBalanceComputation = new ContinuousComputation<>(threadPool.generic()) {
 
             @Override

+ 2 - 2
server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceStats.java

@@ -97,12 +97,12 @@ public record DesiredBalanceStats(
         builder.field("unassigned_shards", unassignedShards);
         builder.field("total_allocations", totalAllocations);
         builder.field("undesired_allocations", undesiredAllocations);
-        builder.field("undesired_allocations_fraction", undesiredAllocationsFraction());
+        builder.field("undesired_allocations_ratio", undesiredAllocationsRatio());
         builder.endObject();
         return builder;
     }
 
-    public double undesiredAllocationsFraction() {
+    public double undesiredAllocationsRatio() {
         if (unassignedShards == -1 || totalAllocations == -1 || undesiredAllocations == -1) {
             return -1.0;
         } else if (totalAllocations == 0) {

+ 2 - 1
server/src/main/java/org/elasticsearch/node/NodeConstruction.java

@@ -665,7 +665,8 @@ class NodeConstruction {
             snapshotsInfoService,
             threadPool,
             systemIndices,
-            writeLoadForecaster
+            writeLoadForecaster,
+            telemetryProvider
         );
         modules.add(clusterModule);
         IndicesModule indicesModule = new IndicesModule(pluginsService.filterPlugins(MapperPlugin.class).toList());

+ 3 - 0
server/src/main/java/org/elasticsearch/telemetry/metric/DoubleWithAttributes.java

@@ -12,4 +12,7 @@ import java.util.Map;
 
 public record DoubleWithAttributes(double value, Map<String, Object> attributes) {
 
+    public DoubleWithAttributes(double value) {
+        this(value, Map.of());
+    }
 }

+ 33 - 0
server/src/main/java/org/elasticsearch/telemetry/metric/LongGaugeMetric.java

@@ -0,0 +1,33 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0 and the Server Side Public License, v 1; you may not use this file except
+ * in compliance with, at your election, the Elastic License 2.0 or the Server
+ * Side Public License, v 1.
+ */
+
+package org.elasticsearch.telemetry.metric;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This wrapper allow us to record metric with APM (via {@link LongGauge}) while also access its current state via {@link AtomicLong}
+ */
+public record LongGaugeMetric(AtomicLong value, LongGauge gauge) {
+
+    public static LongGaugeMetric create(MeterRegistry meterRegistry, String name, String description, String unit) {
+        final AtomicLong value = new AtomicLong();
+        return new LongGaugeMetric(
+            value,
+            meterRegistry.registerLongGauge(name, description, unit, () -> new LongWithAttributes(value.get()))
+        );
+    }
+
+    public void set(long l) {
+        value.set(l);
+    }
+
+    public long get() {
+        return value.get();
+    }
+}

+ 3 - 0
server/src/main/java/org/elasticsearch/telemetry/metric/LongWithAttributes.java

@@ -12,4 +12,7 @@ import java.util.Map;
 
 public record LongWithAttributes(long value, Map<String, Object> attributes) {
 
+    public LongWithAttributes(long value) {
+        this(value, Map.of());
+    }
 }

+ 9 - 1
server/src/test/java/org/elasticsearch/action/admin/cluster/allocation/TransportDeleteDesiredBalanceActionTests.java

@@ -37,6 +37,7 @@ import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.index.IndexVersion;
 import org.elasticsearch.index.shard.ShardId;
 import org.elasticsearch.tasks.Task;
+import org.elasticsearch.telemetry.TelemetryProvider;
 import org.elasticsearch.test.ClusterServiceUtils;
 import org.elasticsearch.test.MockUtils;
 import org.elasticsearch.test.gateway.TestGatewayAllocator;
@@ -114,7 +115,14 @@ public class TransportDeleteDesiredBalanceActionTests extends ESAllocationTestCa
                 return super.compute(previousDesiredBalance, desiredBalanceInput, pendingDesiredBalanceMoves, isFresh);
             }
         };
-        var allocator = new DesiredBalanceShardsAllocator(delegate, threadPool, clusterService, computer, (state, action) -> state);
+        var allocator = new DesiredBalanceShardsAllocator(
+            delegate,
+            threadPool,
+            clusterService,
+            computer,
+            (state, action) -> state,
+            TelemetryProvider.NOOP
+        );
         var allocationService = new MockAllocationService(
             randomAllocationDeciders(settings, clusterSettings),
             new TestGatewayAllocator(),

+ 10 - 6
server/src/test/java/org/elasticsearch/cluster/ClusterModuleTests.java

@@ -45,6 +45,7 @@ import org.elasticsearch.gateway.GatewayAllocator;
 import org.elasticsearch.indices.EmptySystemIndices;
 import org.elasticsearch.plugins.ClusterPlugin;
 import org.elasticsearch.tasks.TaskManager;
+import org.elasticsearch.telemetry.TelemetryProvider;
 import org.elasticsearch.test.gateway.TestGatewayAllocator;
 import org.elasticsearch.threadpool.TestThreadPool;
 import org.elasticsearch.threadpool.ThreadPool;
@@ -155,7 +156,7 @@ public class ClusterModuleTests extends ModuleTestCase {
                 public Collection<AllocationDecider> createAllocationDeciders(Settings settings, ClusterSettings clusterSettings) {
                     return Collections.singletonList(new EnableAllocationDecider(clusterSettings));
                 }
-            }), clusterInfoService, null, threadPool, EmptySystemIndices.INSTANCE, WriteLoadForecaster.DEFAULT)
+            }), clusterInfoService, null, threadPool, EmptySystemIndices.INSTANCE, WriteLoadForecaster.DEFAULT, TelemetryProvider.NOOP)
         );
         assertEquals(e.getMessage(), "Cannot specify allocation decider [" + EnableAllocationDecider.class.getName() + "] twice");
     }
@@ -166,7 +167,7 @@ public class ClusterModuleTests extends ModuleTestCase {
             public Collection<AllocationDecider> createAllocationDeciders(Settings settings, ClusterSettings clusterSettings) {
                 return Collections.singletonList(new FakeAllocationDecider());
             }
-        }), clusterInfoService, null, threadPool, EmptySystemIndices.INSTANCE, WriteLoadForecaster.DEFAULT);
+        }), clusterInfoService, null, threadPool, EmptySystemIndices.INSTANCE, WriteLoadForecaster.DEFAULT, TelemetryProvider.NOOP);
         assertTrue(module.deciderList.stream().anyMatch(d -> d.getClass().equals(FakeAllocationDecider.class)));
     }
 
@@ -176,7 +177,7 @@ public class ClusterModuleTests extends ModuleTestCase {
             public Map<String, Supplier<ShardsAllocator>> getShardsAllocators(Settings settings, ClusterSettings clusterSettings) {
                 return Collections.singletonMap(name, supplier);
             }
-        }), clusterInfoService, null, threadPool, EmptySystemIndices.INSTANCE, WriteLoadForecaster.DEFAULT);
+        }), clusterInfoService, null, threadPool, EmptySystemIndices.INSTANCE, WriteLoadForecaster.DEFAULT, TelemetryProvider.NOOP);
     }
 
     public void testRegisterShardsAllocator() {
@@ -208,7 +209,8 @@ public class ClusterModuleTests extends ModuleTestCase {
                 null,
                 threadPool,
                 EmptySystemIndices.INSTANCE,
-                WriteLoadForecaster.DEFAULT
+                WriteLoadForecaster.DEFAULT,
+                TelemetryProvider.NOOP
             )
         );
         assertEquals("Unknown ShardsAllocator [dne]", e.getMessage());
@@ -272,7 +274,8 @@ public class ClusterModuleTests extends ModuleTestCase {
             null,
             threadPool,
             EmptySystemIndices.INSTANCE,
-            WriteLoadForecaster.DEFAULT
+            WriteLoadForecaster.DEFAULT,
+            TelemetryProvider.NOOP
         );
         expectThrows(IllegalArgumentException.class, () -> clusterModule.setExistingShardsAllocators(new TestGatewayAllocator()));
     }
@@ -286,7 +289,8 @@ public class ClusterModuleTests extends ModuleTestCase {
             null,
             threadPool,
             EmptySystemIndices.INSTANCE,
-            WriteLoadForecaster.DEFAULT
+            WriteLoadForecaster.DEFAULT,
+            TelemetryProvider.NOOP
         );
         expectThrows(IllegalArgumentException.class, () -> clusterModule.setExistingShardsAllocators(new TestGatewayAllocator()));
     }

+ 3 - 1
server/src/test/java/org/elasticsearch/cluster/routing/allocation/allocator/ClusterAllocationSimulationTests.java

@@ -47,6 +47,7 @@ import org.elasticsearch.core.TimeValue;
 import org.elasticsearch.index.IndexVersion;
 import org.elasticsearch.snapshots.SnapshotShardSizeInfo;
 import org.elasticsearch.tasks.TaskManager;
+import org.elasticsearch.telemetry.TelemetryProvider;
 import org.elasticsearch.test.ClusterServiceUtils;
 import org.elasticsearch.test.gateway.TestGatewayAllocator;
 import org.elasticsearch.threadpool.ThreadPool;
@@ -487,7 +488,8 @@ public class ClusterAllocationSimulationTests extends ESAllocationTestCase {
             threadPool,
             clusterService,
             (clusterState, routingAllocationAction) -> strategyRef.get()
-                .executeWithRoutingAllocation(clusterState, "reconcile-desired-balance", routingAllocationAction)
+                .executeWithRoutingAllocation(clusterState, "reconcile-desired-balance", routingAllocationAction),
+            TelemetryProvider.NOOP
         ) {
             @Override
             public void allocate(RoutingAllocation allocation, ActionListener<Void> listener) {

+ 7 - 3
server/src/test/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceReconcilerTests.java

@@ -63,6 +63,7 @@ import org.elasticsearch.snapshots.Snapshot;
 import org.elasticsearch.snapshots.SnapshotId;
 import org.elasticsearch.snapshots.SnapshotShardSizeInfo;
 import org.elasticsearch.snapshots.SnapshotsInfoService;
+import org.elasticsearch.telemetry.metric.MeterRegistry;
 import org.elasticsearch.test.ESTestCase;
 import org.elasticsearch.test.MockLogAppender;
 import org.elasticsearch.threadpool.ThreadPool;
@@ -1206,7 +1207,7 @@ public class DesiredBalanceReconcilerTests extends ESAllocationTestCase {
             new ConcurrentRebalanceAllocationDecider(clusterSettings),
             new ThrottlingAllocationDecider(clusterSettings) };
 
-        var reconciler = new DesiredBalanceReconciler(clusterSettings, mock(ThreadPool.class));
+        var reconciler = new DesiredBalanceReconciler(clusterSettings, mock(ThreadPool.class), mock(MeterRegistry.class));
 
         var totalOutgoingMoves = new HashMap<String, AtomicInteger>();
         for (int i = 0; i < numberOfNodes; i++) {
@@ -1275,7 +1276,7 @@ public class DesiredBalanceReconcilerTests extends ESAllocationTestCase {
         var threadPool = mock(ThreadPool.class);
         when(threadPool.relativeTimeInMillis()).thenReturn(1L).thenReturn(2L).thenReturn(3L);
 
-        var reconciler = new DesiredBalanceReconciler(createBuiltInClusterSettings(), threadPool);
+        var reconciler = new DesiredBalanceReconciler(createBuiltInClusterSettings(), threadPool, mock(MeterRegistry.class));
 
         var expectedWarningMessage = "[100%] of assigned shards ("
             + shardCount
@@ -1315,7 +1316,10 @@ public class DesiredBalanceReconcilerTests extends ESAllocationTestCase {
     }
 
     private static void reconcile(RoutingAllocation routingAllocation, DesiredBalance desiredBalance) {
-        new DesiredBalanceReconciler(createBuiltInClusterSettings(), mock(ThreadPool.class)).reconcile(desiredBalance, routingAllocation);
+        new DesiredBalanceReconciler(createBuiltInClusterSettings(), mock(ThreadPool.class), mock(MeterRegistry.class)).reconcile(
+            desiredBalance,
+            routingAllocation
+        );
     }
 
     private static boolean isReconciled(RoutingNode node, DesiredBalance balance) {

+ 13 - 6
server/src/test/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceShardsAllocatorTests.java

@@ -47,6 +47,7 @@ import org.elasticsearch.gateway.GatewayAllocator;
 import org.elasticsearch.index.IndexVersion;
 import org.elasticsearch.index.shard.ShardId;
 import org.elasticsearch.snapshots.SnapshotShardSizeInfo;
+import org.elasticsearch.telemetry.TelemetryProvider;
 import org.elasticsearch.test.ClusterServiceUtils;
 import org.elasticsearch.threadpool.TestThreadPool;
 
@@ -157,7 +158,8 @@ public class DesiredBalanceShardsAllocatorTests extends ESAllocationTestCase {
             createShardsAllocator(),
             threadPool,
             clusterService,
-            reconcileAction
+            reconcileAction,
+            TelemetryProvider.NOOP
         );
         assertValidStats(desiredBalanceShardsAllocator.getStats());
         var allocationService = createAllocationService(desiredBalanceShardsAllocator, createGatewayAllocator(allocateUnassigned));
@@ -277,7 +279,8 @@ public class DesiredBalanceShardsAllocatorTests extends ESAllocationTestCase {
             createShardsAllocator(),
             threadPool,
             clusterService,
-            reconcileAction
+            reconcileAction,
+            TelemetryProvider.NOOP
         );
         var allocationService = new AllocationService(
             new AllocationDeciders(List.of()),
@@ -369,7 +372,8 @@ public class DesiredBalanceShardsAllocatorTests extends ESAllocationTestCase {
                     return super.compute(previousDesiredBalance, desiredBalanceInput, pendingDesiredBalanceMoves, isFresh);
                 }
             },
-            reconcileAction
+            reconcileAction,
+            TelemetryProvider.NOOP
         );
         var allocationService = createAllocationService(desiredBalanceShardsAllocator, gatewayAllocator);
         allocationServiceRef.set(allocationService);
@@ -471,7 +475,8 @@ public class DesiredBalanceShardsAllocatorTests extends ESAllocationTestCase {
                     return super.compute(previousDesiredBalance, desiredBalanceInput, pendingDesiredBalanceMoves, isFresh);
                 }
             },
-            reconcileAction
+            reconcileAction,
+            TelemetryProvider.NOOP
         );
 
         var allocationService = createAllocationService(desiredBalanceShardsAllocator, gatewayAllocator);
@@ -561,7 +566,8 @@ public class DesiredBalanceShardsAllocatorTests extends ESAllocationTestCase {
             threadPool,
             clusterService,
             desiredBalanceComputer,
-            (reconcilerClusterState, rerouteStrategy) -> reconcilerClusterState
+            (reconcilerClusterState, rerouteStrategy) -> reconcilerClusterState,
+            TelemetryProvider.NOOP
         );
 
         var service = createAllocationService(desiredBalanceShardsAllocator, createGatewayAllocator());
@@ -613,7 +619,8 @@ public class DesiredBalanceShardsAllocatorTests extends ESAllocationTestCase {
             threadPool,
             clusterService,
             desiredBalanceComputer,
-            (reconcilerClusterState, rerouteStrategy) -> reconcilerClusterState
+            (reconcilerClusterState, rerouteStrategy) -> reconcilerClusterState,
+            TelemetryProvider.NOOP
         );
 
         var service = createAllocationService(desiredBalanceShardsAllocator, createGatewayAllocator());

+ 2 - 2
server/src/test/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceStatsTests.java

@@ -68,7 +68,7 @@ public class DesiredBalanceStatsTests extends AbstractWireSerializingTestCase<De
                           "unassigned_shards" : %d,
                           "total_allocations" : %d,
                           "undesired_allocations" : %d,
-                          "undesired_allocations_fraction" : %s
+                          "undesired_allocations_ratio" : %s
                         }""",
                     instance.lastConvergedIndex(),
                     instance.computationActive(),
@@ -82,7 +82,7 @@ public class DesiredBalanceStatsTests extends AbstractWireSerializingTestCase<De
                     instance.unassignedShards(),
                     instance.totalAllocations(),
                     instance.undesiredAllocations(),
-                    Double.toString(instance.undesiredAllocationsFraction())
+                    Double.toString(instance.undesiredAllocationsRatio())
                 )
             )
         );

+ 3 - 1
test/framework/src/main/java/org/elasticsearch/cluster/ESAllocationTestCase.java

@@ -43,6 +43,7 @@ import org.elasticsearch.index.IndexVersion;
 import org.elasticsearch.index.IndexVersions;
 import org.elasticsearch.snapshots.SnapshotShardSizeInfo;
 import org.elasticsearch.snapshots.SnapshotsInfoService;
+import org.elasticsearch.telemetry.TelemetryProvider;
 import org.elasticsearch.test.ClusterServiceUtils;
 import org.elasticsearch.test.ESTestCase;
 import org.elasticsearch.test.gateway.TestGatewayAllocator;
@@ -161,7 +162,8 @@ public abstract class ESAllocationTestCase extends ESTestCase {
             new BalancedShardsAllocator(settings),
             queue.getThreadPool(),
             clusterService,
-            null
+            null,
+            TelemetryProvider.NOOP
         ) {
             private RoutingAllocation lastAllocation;