Browse Source

Simplify InternalClusterInfoService (#103370)

Ievgen Degtiarenko 1 year ago
parent
commit
997fbb372c
21 changed files with 59 additions and 108 deletions
  1. 1 1
      server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/stats/ClusterStatsIT.java
  2. 4 5
      server/src/internalClusterTest/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java
  3. 1 1
      server/src/internalClusterTest/java/org/elasticsearch/snapshots/RepositoryThrottlingStatsIT.java
  4. 1 1
      server/src/main/java/org/elasticsearch/action/admin/indices/shrink/ResizeNumberOfShardsCalculator.java
  5. 2 7
      server/src/main/java/org/elasticsearch/cluster/InternalClusterInfoService.java
  6. 0 27
      server/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java
  7. 1 1
      server/src/main/java/org/elasticsearch/index/shard/DocsStats.java
  8. 13 21
      server/src/main/java/org/elasticsearch/index/store/StoreStats.java
  9. 1 1
      server/src/main/java/org/elasticsearch/rest/action/cat/RestAllocationAction.java
  10. 2 2
      server/src/main/java/org/elasticsearch/rest/action/cat/RestShardsAction.java
  11. 1 9
      server/src/test/java/org/elasticsearch/cluster/DiskUsageTests.java
  12. 11 11
      server/src/test/java/org/elasticsearch/index/store/StoreTests.java
  13. 1 1
      test/framework/src/main/java/org/elasticsearch/cluster/MockInternalClusterInfoService.java
  14. 2 2
      x-pack/plugin/autoscaling/src/internalClusterTest/java/org/elasticsearch/xpack/autoscaling/storage/FrozenStorageDeciderIT.java
  15. 1 1
      x-pack/plugin/autoscaling/src/internalClusterTest/java/org/elasticsearch/xpack/autoscaling/storage/ProactiveStorageIT.java
  16. 4 4
      x-pack/plugin/autoscaling/src/internalClusterTest/java/org/elasticsearch/xpack/autoscaling/storage/ReactiveStorageIT.java
  17. 2 2
      x-pack/plugin/ccr/src/internalClusterTest/java/org/elasticsearch/xpack/ccr/CcrRepositoryIT.java
  18. 1 1
      x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java
  19. 1 1
      x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportPutTrainedModelAction.java
  20. 8 8
      x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/FrozenSearchableSnapshotsIntegTests.java
  21. 1 1
      x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsIntegTests.java

+ 1 - 1
server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/stats/ClusterStatsIT.java

@@ -207,7 +207,7 @@ public class ClusterStatsIT extends ESIntegTestCase {
         ClusterStatsResponse response = clusterAdmin().prepareClusterStats().get();
         String msg = response.toString();
         assertThat(msg, response.getTimestamp(), greaterThan(946681200000L)); // 1 Jan 2000
-        assertThat(msg, response.indicesStats.getStore().getSizeInBytes(), greaterThan(0L));
+        assertThat(msg, response.indicesStats.getStore().sizeInBytes(), greaterThan(0L));
 
         assertThat(msg, response.nodesStats.getFs().getTotal().getBytes(), greaterThan(0L));
         assertThat(msg, response.nodesStats.getJvm().getVersions().size(), greaterThan(0));

+ 4 - 5
server/src/internalClusterTest/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java

@@ -1745,12 +1745,12 @@ public class IndexRecoveryIT extends AbstractIndexRecoveryIntegTestCase {
                         .getNodes()
                         .get(0)
                         .getIndices();
-                    assertThat(nodeIndicesStats.getStore().getReservedSize().getBytes(), equalTo(0L));
+                    assertThat(nodeIndicesStats.getStore().reservedSizeInBytes(), equalTo(0L));
                     assertThat(
                         nodeIndicesStats.getShardStats(clusterState.metadata().index(indexName).getIndex())
                             .stream()
                             .flatMap(s -> Arrays.stream(s.getShards()))
-                            .map(s -> s.getStats().getStore().getReservedSize().getBytes())
+                            .map(s -> s.getStats().getStore().reservedSizeInBytes())
                             .toList(),
                         everyItem(equalTo(StoreStats.UNKNOWN_RESERVED_BYTES))
                     );
@@ -1766,8 +1766,7 @@ public class IndexRecoveryIT extends AbstractIndexRecoveryIntegTestCase {
                             .get(0)
                             .getIndices()
                             .getStore()
-                            .getReservedSize()
-                            .getBytes(),
+                            .reservedSizeInBytes(),
                         greaterThan(0L)
                     );
                 }
@@ -1785,7 +1784,7 @@ public class IndexRecoveryIT extends AbstractIndexRecoveryIntegTestCase {
                 .get()
                 .getNodes()
                 .stream()
-                .mapToLong(n -> n.getIndices().getStore().getReservedSize().getBytes())
+                .mapToLong(n -> n.getIndices().getStore().reservedSizeInBytes())
                 .sum(),
             equalTo(0L)
         );

+ 1 - 1
server/src/internalClusterTest/java/org/elasticsearch/snapshots/RepositoryThrottlingStatsIT.java

@@ -37,7 +37,7 @@ public class RepositoryThrottlingStatsIT extends AbstractSnapshotIntegTestCase {
         IndexStats indexStats = indicesStats.getIndex("test-idx");
         long totalSizeInBytes = 0;
         for (ShardStats shard : indexStats.getShards()) {
-            totalSizeInBytes += shard.getStats().getStore().getSizeInBytes();
+            totalSizeInBytes += shard.getStats().getStore().sizeInBytes();
         }
         logger.info("--> total shards size: {} bytes", totalSizeInBytes);
 

+ 1 - 1
server/src/main/java/org/elasticsearch/action/admin/indices/shrink/ResizeNumberOfShardsCalculator.java

@@ -69,7 +69,7 @@ public interface ResizeNumberOfShardsCalculator {
                 }
             } else if (maxPrimaryShardSize != null) {
                 int sourceIndexShardsNum = sourceMetadata.getNumberOfShards();
-                long sourceIndexStorageBytes = indexStoreStats.getSizeInBytes();
+                long sourceIndexStorageBytes = indexStoreStats.sizeInBytes();
                 long maxPrimaryShardSizeBytes = maxPrimaryShardSize.getBytes();
                 long minShardsNum = sourceIndexStorageBytes / maxPrimaryShardSizeBytes;
                 if (minShardsNum * maxPrimaryShardSizeBytes < sourceIndexStorageBytes) {

+ 2 - 7
server/src/main/java/org/elasticsearch/cluster/InternalClusterInfoService.java

@@ -26,7 +26,6 @@ import org.elasticsearch.action.support.ThreadedActionListener;
 import org.elasticsearch.client.internal.Client;
 import org.elasticsearch.cluster.block.ClusterBlockException;
 import org.elasticsearch.cluster.node.DiscoveryNode;
-import org.elasticsearch.cluster.routing.RoutingTable;
 import org.elasticsearch.cluster.routing.ShardRouting;
 import org.elasticsearch.cluster.routing.allocation.DiskThresholdSettings;
 import org.elasticsearch.cluster.service.ClusterService;
@@ -97,7 +96,6 @@ public class InternalClusterInfoService implements ClusterInfoService, ClusterSt
     private final Object mutex = new Object();
     private final List<ActionListener<ClusterInfo>> nextRefreshListeners = new ArrayList<>();
 
-    private final ClusterService clusterService;
     private AsyncRefresh currentRefresh;
     private RefreshScheduler refreshScheduler;
 
@@ -108,7 +106,6 @@ public class InternalClusterInfoService implements ClusterInfoService, ClusterSt
         this.indicesStatsSummary = IndicesStatsSummary.EMPTY;
         this.threadPool = threadPool;
         this.client = client;
-        this.clusterService = clusterService;
         this.updateFrequency = INTERNAL_CLUSTER_INFO_UPDATE_INTERVAL_SETTING.get(settings);
         this.fetchTimeout = INTERNAL_CLUSTER_INFO_TIMEOUT_SETTING.get(settings);
         this.enabled = DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.get(settings);
@@ -250,7 +247,6 @@ public class InternalClusterInfoService implements ClusterInfoService, ClusterSt
                                 final Map<ClusterInfo.NodeAndPath, ClusterInfo.ReservedSpace.Builder> reservedSpaceBuilders =
                                     new HashMap<>();
                                 buildShardLevelInfo(
-                                    clusterService.state().routingTable(),
                                     adjustShardStats(stats),
                                     shardSizeByIdentifierBuilder,
                                     shardDataSetSizeBuilder,
@@ -445,7 +441,6 @@ public class InternalClusterInfoService implements ClusterInfoService, ClusterSt
     }
 
     static void buildShardLevelInfo(
-        RoutingTable routingTable,
         ShardStats[] stats,
         Map<String, Long> shardSizes,
         Map<ShardId, Long> shardDataSetSizeBuilder,
@@ -453,7 +448,7 @@ public class InternalClusterInfoService implements ClusterInfoService, ClusterSt
         Map<ClusterInfo.NodeAndPath, ClusterInfo.ReservedSpace.Builder> reservedSpaceByShard
     ) {
         for (ShardStats s : stats) {
-            final ShardRouting shardRouting = routingTable.deduplicate(s.getShardRouting());
+            final ShardRouting shardRouting = s.getShardRouting();
             dataPathByShard.put(ClusterInfo.NodeAndShard.from(shardRouting), s.getDataPath());
 
             final StoreStats storeStats = s.getStats().getStore();
@@ -462,7 +457,7 @@ public class InternalClusterInfoService implements ClusterInfoService, ClusterSt
             }
             final long size = storeStats.sizeInBytes();
             final long dataSetSize = storeStats.totalDataSetSizeInBytes();
-            final long reserved = storeStats.getReservedSize().getBytes();
+            final long reserved = storeStats.reservedSizeInBytes();
 
             final String shardIdentifier = ClusterInfo.shardIdentifierFromRouting(shardRouting);
             logger.trace("shard: {} size: {} reserved: {}", shardIdentifier, size, reserved);

+ 0 - 27
server/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java

@@ -148,33 +148,6 @@ public class RoutingTable implements Iterable<IndexRoutingTable>, Diffable<Routi
         return shard;
     }
 
-    /**
-     * Try to deduplicate the given shard routing with an equal instance found in this routing table. This is used by the logic of the
-     * {@link org.elasticsearch.cluster.routing.allocation.decider.DiskThresholdDecider} and
-     * {@link org.elasticsearch.cluster.InternalClusterInfoService} to deduplicate instances created by a master node and those read from
-     * the network to speed up the use of {@link ShardRouting} as a map key in {@link org.elasticsearch.cluster.ClusterInfo#getDataPath}.
-     *
-     * @param shardRouting shard routing to deduplicate
-     * @return deduplicated shard routing from this routing table if an equivalent shard routing was found or the given instance otherwise
-     */
-    public ShardRouting deduplicate(ShardRouting shardRouting) {
-        final IndexRoutingTable indexShardRoutingTable = indicesRouting.get(shardRouting.index().getName());
-        if (indexShardRoutingTable == null) {
-            return shardRouting;
-        }
-        final IndexShardRoutingTable shardRoutingTable = indexShardRoutingTable.shard(shardRouting.id());
-        if (shardRoutingTable == null) {
-            return shardRouting;
-        }
-        for (int i = 0; i < shardRoutingTable.size(); i++) {
-            ShardRouting found = shardRoutingTable.shard(i);
-            if (shardRouting.equals(found)) {
-                return found;
-            }
-        }
-        return shardRouting;
-    }
-
     @Nullable
     public ShardRouting getByAllocationId(ShardId shardId, String allocationId) {
         final IndexRoutingTable indexRoutingTable = index(shardId.getIndex());

+ 1 - 1
server/src/main/java/org/elasticsearch/index/shard/DocsStats.java

@@ -63,7 +63,7 @@ public class DocsStats implements Writeable, ToXContentFragment {
 
     /**
      * Returns the total size in bytes of all documents in this stats.
-     * This value may be more reliable than {@link StoreStats#getSizeInBytes()} in estimating the index size.
+     * This value may be more reliable than {@link StoreStats#sizeInBytes()} in estimating the index size.
      */
     public long getTotalSizeInBytes() {
         return totalSizeInBytes;

+ 13 - 21
server/src/main/java/org/elasticsearch/index/store/StoreStats.java

@@ -33,7 +33,7 @@ public class StoreStats implements Writeable, ToXContentFragment {
 
     private long sizeInBytes;
     private long totalDataSetSizeInBytes;
-    private long reservedSize;
+    private long reservedSizeInBytes;
 
     public StoreStats() {
 
@@ -47,9 +47,9 @@ public class StoreStats implements Writeable, ToXContentFragment {
             totalDataSetSizeInBytes = sizeInBytes;
         }
         if (in.getTransportVersion().onOrAfter(RESERVED_BYTES_VERSION)) {
-            reservedSize = in.readZLong();
+            reservedSizeInBytes = in.readZLong();
         } else {
-            reservedSize = UNKNOWN_RESERVED_BYTES;
+            reservedSizeInBytes = UNKNOWN_RESERVED_BYTES;
         }
     }
 
@@ -63,7 +63,7 @@ public class StoreStats implements Writeable, ToXContentFragment {
         assert reservedSize == UNKNOWN_RESERVED_BYTES || reservedSize >= 0 : reservedSize;
         this.sizeInBytes = sizeInBytes;
         this.totalDataSetSizeInBytes = totalDataSetSizeInBytes;
-        this.reservedSize = reservedSize;
+        this.reservedSizeInBytes = reservedSize;
     }
 
     public void add(StoreStats stats) {
@@ -72,7 +72,7 @@ public class StoreStats implements Writeable, ToXContentFragment {
         }
         sizeInBytes += stats.sizeInBytes;
         totalDataSetSizeInBytes += stats.totalDataSetSizeInBytes;
-        reservedSize = ignoreIfUnknown(reservedSize) + ignoreIfUnknown(stats.reservedSize);
+        reservedSizeInBytes = ignoreIfUnknown(reservedSizeInBytes) + ignoreIfUnknown(stats.reservedSizeInBytes);
     }
 
     private static long ignoreIfUnknown(long reservedSize) {
@@ -83,28 +83,20 @@ public class StoreStats implements Writeable, ToXContentFragment {
         return sizeInBytes;
     }
 
-    public long getSizeInBytes() {
-        return sizeInBytes;
-    }
-
     public ByteSizeValue size() {
         return ByteSizeValue.ofBytes(sizeInBytes);
     }
 
-    public ByteSizeValue getSize() {
-        return size();
+    public long totalDataSetSizeInBytes() {
+        return totalDataSetSizeInBytes;
     }
 
     public ByteSizeValue totalDataSetSize() {
         return ByteSizeValue.ofBytes(totalDataSetSizeInBytes);
     }
 
-    public ByteSizeValue getTotalDataSetSize() {
-        return totalDataSetSize();
-    }
-
-    public long totalDataSetSizeInBytes() {
-        return totalDataSetSizeInBytes;
+    public long reservedSizeInBytes() {
+        return reservedSizeInBytes;
     }
 
     /**
@@ -113,7 +105,7 @@ public class StoreStats implements Writeable, ToXContentFragment {
      * the reserved size is unknown.
      */
     public ByteSizeValue getReservedSize() {
-        return ByteSizeValue.ofBytes(reservedSize);
+        return ByteSizeValue.ofBytes(reservedSizeInBytes);
     }
 
     @Override
@@ -123,7 +115,7 @@ public class StoreStats implements Writeable, ToXContentFragment {
             out.writeVLong(totalDataSetSizeInBytes);
         }
         if (out.getTransportVersion().onOrAfter(RESERVED_BYTES_VERSION)) {
-            out.writeZLong(reservedSize);
+            out.writeZLong(reservedSizeInBytes);
         }
     }
 
@@ -144,12 +136,12 @@ public class StoreStats implements Writeable, ToXContentFragment {
         StoreStats that = (StoreStats) o;
         return sizeInBytes == that.sizeInBytes
             && totalDataSetSizeInBytes == that.totalDataSetSizeInBytes
-            && reservedSize == that.reservedSize;
+            && reservedSizeInBytes == that.reservedSizeInBytes;
     }
 
     @Override
     public int hashCode() {
-        return Objects.hash(sizeInBytes, totalDataSetSizeInBytes, reservedSize);
+        return Objects.hash(sizeInBytes, totalDataSetSizeInBytes, reservedSizeInBytes);
     }
 
     static final class Fields {

+ 1 - 1
server/src/main/java/org/elasticsearch/rest/action/cat/RestAllocationAction.java

@@ -130,7 +130,7 @@ public class RestAllocationAction extends AbstractCatAction {
 
             table.startRow();
             table.addCell(shardCount);
-            table.addCell(nodeStats.getIndices().getStore().getSize());
+            table.addCell(nodeStats.getIndices().getStore().size());
             table.addCell(used < 0 ? null : ByteSizeValue.ofBytes(used));
             table.addCell(avail.getBytes() < 0 ? null : avail);
             table.addCell(total.getBytes() < 0 ? null : total);

+ 2 - 2
server/src/main/java/org/elasticsearch/rest/action/cat/RestShardsAction.java

@@ -293,8 +293,8 @@ public class RestShardsAction extends AbstractCatAction {
             }
             table.addCell(shard.state());
             table.addCell(getOrNull(commonStats, CommonStats::getDocs, DocsStats::getCount));
-            table.addCell(getOrNull(commonStats, CommonStats::getStore, StoreStats::getSize));
-            table.addCell(getOrNull(commonStats, CommonStats::getStore, StoreStats::getTotalDataSetSize));
+            table.addCell(getOrNull(commonStats, CommonStats::getStore, StoreStats::size));
+            table.addCell(getOrNull(commonStats, CommonStats::getStore, StoreStats::totalDataSetSize));
             if (shard.assignedToNode()) {
                 String ip = state.getState().nodes().get(shard.currentNodeId()).getHostAddress();
                 String nodeId = shard.currentNodeId();

+ 1 - 9
server/src/test/java/org/elasticsearch/cluster/DiskUsageTests.java

@@ -13,7 +13,6 @@ import org.elasticsearch.action.admin.indices.stats.CommonStats;
 import org.elasticsearch.action.admin.indices.stats.ShardStats;
 import org.elasticsearch.cluster.node.DiscoveryNodeUtils;
 import org.elasticsearch.cluster.routing.RecoverySource.PeerRecoverySource;
-import org.elasticsearch.cluster.routing.RoutingTable;
 import org.elasticsearch.cluster.routing.ShardRouting;
 import org.elasticsearch.cluster.routing.ShardRoutingHelper;
 import org.elasticsearch.cluster.routing.UnassignedInfo;
@@ -137,14 +136,7 @@ public class DiskUsageTests extends ESTestCase {
         Map<String, Long> shardSizes = new HashMap<>();
         Map<ShardId, Long> shardDataSetSizes = new HashMap<>();
         Map<ClusterInfo.NodeAndShard, String> routingToPath = new HashMap<>();
-        InternalClusterInfoService.buildShardLevelInfo(
-            RoutingTable.EMPTY_ROUTING_TABLE,
-            stats,
-            shardSizes,
-            shardDataSetSizes,
-            routingToPath,
-            new HashMap<>()
-        );
+        InternalClusterInfoService.buildShardLevelInfo(stats, shardSizes, shardDataSetSizes, routingToPath, new HashMap<>());
 
         assertThat(
             shardSizes,

+ 11 - 11
server/src/test/java/org/elasticsearch/index/store/StoreTests.java

@@ -775,22 +775,22 @@ public class StoreTests extends ESTestCase {
         final long localStoreSizeDelta = randomLongBetween(-initialStoreSize, initialStoreSize);
         final long reservedBytes = randomBoolean() ? StoreStats.UNKNOWN_RESERVED_BYTES : randomLongBetween(0L, Integer.MAX_VALUE);
         StoreStats stats = store.stats(reservedBytes, size -> size + localStoreSizeDelta);
-        assertEquals(initialStoreSize, stats.totalDataSetSize().getBytes());
-        assertEquals(initialStoreSize + localStoreSizeDelta, stats.getSize().getBytes());
-        assertEquals(reservedBytes, stats.getReservedSize().getBytes());
+        assertEquals(initialStoreSize, stats.totalDataSetSizeInBytes());
+        assertEquals(initialStoreSize + localStoreSizeDelta, stats.sizeInBytes());
+        assertEquals(reservedBytes, stats.reservedSizeInBytes());
 
         stats.add(null);
-        assertEquals(initialStoreSize, stats.totalDataSetSize().getBytes());
-        assertEquals(initialStoreSize + localStoreSizeDelta, stats.getSize().getBytes());
-        assertEquals(reservedBytes, stats.getReservedSize().getBytes());
+        assertEquals(initialStoreSize, stats.totalDataSetSizeInBytes());
+        assertEquals(initialStoreSize + localStoreSizeDelta, stats.sizeInBytes());
+        assertEquals(reservedBytes, stats.reservedSizeInBytes());
 
         final long otherStatsDataSetBytes = randomLongBetween(0L, Integer.MAX_VALUE);
         final long otherStatsLocalBytes = randomLongBetween(0L, Integer.MAX_VALUE);
         final long otherStatsReservedBytes = randomBoolean() ? StoreStats.UNKNOWN_RESERVED_BYTES : randomLongBetween(0L, Integer.MAX_VALUE);
         stats.add(new StoreStats(otherStatsLocalBytes, otherStatsDataSetBytes, otherStatsReservedBytes));
-        assertEquals(initialStoreSize + otherStatsDataSetBytes, stats.totalDataSetSize().getBytes());
-        assertEquals(initialStoreSize + otherStatsLocalBytes + localStoreSizeDelta, stats.getSize().getBytes());
-        assertEquals(Math.max(reservedBytes, 0L) + Math.max(otherStatsReservedBytes, 0L), stats.getReservedSize().getBytes());
+        assertEquals(initialStoreSize + otherStatsDataSetBytes, stats.totalDataSetSizeInBytes());
+        assertEquals(initialStoreSize + otherStatsLocalBytes + localStoreSizeDelta, stats.sizeInBytes());
+        assertEquals(Math.max(reservedBytes, 0L) + Math.max(otherStatsReservedBytes, 0L), stats.reservedSizeInBytes());
 
         Directory dir = store.directory();
         final long length;
@@ -805,8 +805,8 @@ public class StoreTests extends ESTestCase {
 
         assertTrue(numNonExtraFiles(store) > 0);
         stats = store.stats(0L, size -> size + localStoreSizeDelta);
-        assertEquals(initialStoreSize + length, stats.totalDataSetSize().getBytes());
-        assertEquals(initialStoreSize + localStoreSizeDelta + length, stats.getSizeInBytes());
+        assertEquals(initialStoreSize + length, stats.totalDataSetSizeInBytes());
+        assertEquals(initialStoreSize + localStoreSizeDelta + length, stats.sizeInBytes());
 
         deleteContent(store.directory());
         IOUtils.close(store);

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

@@ -107,7 +107,7 @@ public class MockInternalClusterInfoService extends InternalClusterInfoService {
             var storeStats = new StoreStats(
                 shardSizeFunctionCopy.apply(shardRouting),
                 shardSizeFunctionCopy.apply(shardRouting),
-                shardStats.getStats().store.getReservedSize().getBytes()
+                shardStats.getStats().store.reservedSizeInBytes()
             );
             var commonStats = new CommonStats(new CommonStatsFlags(CommonStatsFlags.Flag.Store));
             commonStats.store = storeStats;

+ 2 - 2
x-pack/plugin/autoscaling/src/internalClusterTest/java/org/elasticsearch/xpack/autoscaling/storage/FrozenStorageDeciderIT.java

@@ -31,8 +31,8 @@ public class FrozenStorageDeciderIT extends AbstractFrozenAutoscalingIntegTestCa
             capacity().results().get("frozen").requiredCapacity().total().storage(),
             equalTo(
                 ByteSizeValue.ofBytes(
-                    (long) (statsResponse.getPrimaries().store.totalDataSetSize().getBytes()
-                        * FrozenStorageDeciderService.DEFAULT_PERCENTAGE) / 100
+                    (long) (statsResponse.getPrimaries().store.totalDataSetSizeInBytes() * FrozenStorageDeciderService.DEFAULT_PERCENTAGE)
+                        / 100
                 )
             )
         );

+ 1 - 1
x-pack/plugin/autoscaling/src/internalClusterTest/java/org/elasticsearch/xpack/autoscaling/storage/ProactiveStorageIT.java

@@ -67,7 +67,7 @@ public class ProactiveStorageIT extends AutoscalingStorageIntegTestCase {
         capacity();
 
         IndicesStatsResponse stats = indicesAdmin().prepareStats(dsName).clear().setStore(true).get();
-        long used = stats.getTotal().getStore().getSizeInBytes();
+        long used = stats.getTotal().getStore().sizeInBytes();
         long maxShardSize = Arrays.stream(stats.getShards()).mapToLong(s -> s.getStats().getStore().sizeInBytes()).max().orElseThrow();
         // As long as usage is above low watermark, we will trigger a proactive scale up, since the simulated shards have an in-sync
         // set and therefore allocating these do not skip the low watermark check in the disk threshold decider.

+ 4 - 4
x-pack/plugin/autoscaling/src/internalClusterTest/java/org/elasticsearch/xpack/autoscaling/storage/ReactiveStorageIT.java

@@ -78,7 +78,7 @@ public class ReactiveStorageIT extends AutoscalingStorageIntegTestCase {
         capacity();
 
         IndicesStatsResponse stats = indicesAdmin().prepareStats(indexName).clear().setStore(true).get();
-        long used = stats.getTotal().getStore().getSizeInBytes();
+        long used = stats.getTotal().getStore().sizeInBytes();
         long minShardSize = Arrays.stream(stats.getShards()).mapToLong(s -> s.getStats().getStore().sizeInBytes()).min().orElseThrow();
         long maxShardSize = Arrays.stream(stats.getShards()).mapToLong(s -> s.getStats().getStore().sizeInBytes()).max().orElseThrow();
         long enoughSpace = used + HIGH_WATERMARK_BYTES + 1;
@@ -274,14 +274,14 @@ public class ReactiveStorageIT extends AutoscalingStorageIntegTestCase {
         refresh();
 
         IndicesStatsResponse stats = indicesAdmin().prepareStats(indexName).clear().setStore(true).get();
-        long used = stats.getTotal().getStore().getSizeInBytes();
+        long used = stats.getTotal().getStore().sizeInBytes();
         long maxShardSize = Arrays.stream(stats.getShards()).mapToLong(s -> s.getStats().getStore().sizeInBytes()).max().orElseThrow();
 
         Map<String, Long> byNode = Arrays.stream(stats.getShards())
             .collect(
                 Collectors.groupingBy(
                     s -> s.getShardRouting().currentNodeId(),
-                    Collectors.summingLong(s -> s.getStats().getStore().getSizeInBytes())
+                    Collectors.summingLong(s -> s.getStats().getStore().sizeInBytes())
                 )
             );
 
@@ -427,7 +427,7 @@ public class ReactiveStorageIT extends AutoscalingStorageIntegTestCase {
         refresh();
 
         IndicesStatsResponse stats = indicesAdmin().prepareStats(indexName).clear().setStore(true).get();
-        long used = stats.getTotal().getStore().getSizeInBytes();
+        long used = stats.getTotal().getStore().sizeInBytes();
 
         long enoughSpace = used + HIGH_WATERMARK_BYTES + 1;
 

+ 2 - 2
x-pack/plugin/ccr/src/internalClusterTest/java/org/elasticsearch/xpack/ccr/CcrRepositoryIT.java

@@ -537,7 +537,7 @@ public class CcrRepositoryIT extends CcrIntegTestCase {
             assertThat(indexShardSnapshotStatus.getStage(), is(IndexShardSnapshotStatus.Stage.DONE));
             assertThat(
                 indexShardSnapshotStatus.getTotalSize(),
-                equalTo(indexStats.getIndexShards().get(shardId).getPrimary().getStore().getSizeInBytes())
+                equalTo(indexStats.getIndexShards().get(shardId).getPrimary().getStore().sizeInBytes())
             );
         }
 
@@ -594,7 +594,7 @@ public class CcrRepositoryIT extends CcrIntegTestCase {
             assertThat(
                 "Snapshot shard size fetched for follower shard [" + shardId + "] does not match leader store size",
                 fetchedSnapshotShardSizes.get(shardId),
-                equalTo(indexStats.getIndexShards().get(shardId).getPrimary().getStore().getSizeInBytes())
+                equalTo(indexStats.getIndexShards().get(shardId).getPrimary().getStore().sizeInBytes())
             );
         }
 

+ 1 - 1
x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java

@@ -517,7 +517,7 @@ public class CcrRepository extends AbstractLifecycleComponent implements Reposit
             final ShardRouting shardRouting = shardStats.getShardRouting();
             if (shardRouting.shardId().id() == shardId.getId() && shardRouting.primary() && shardRouting.active()) {
                 // we only care about the shard size here for shard allocation, populate the rest with dummy values
-                final long totalSize = shardStats.getStats().getStore().getSizeInBytes();
+                final long totalSize = shardStats.getStats().getStore().sizeInBytes();
                 return IndexShardSnapshotStatus.newDone(0L, 0L, 1, 1, totalSize, totalSize, DUMMY_GENERATION);
             }
         }

+ 1 - 1
x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportPutTrainedModelAction.java

@@ -289,7 +289,7 @@ public class TransportPutTrainedModelAction extends TransportMasterNodeAction<Re
                     .execute(ActionListener.wrap(stats -> {
                         IndexStats indexStats = stats.getIndices().get(InferenceIndexConstants.nativeDefinitionStore());
                         if (indexStats != null
-                            && indexStats.getTotal().getStore().getSizeInBytes() > MAX_NATIVE_DEFINITION_INDEX_SIZE.getBytes()) {
+                            && indexStats.getTotal().getStore().sizeInBytes() > MAX_NATIVE_DEFINITION_INDEX_SIZE.getBytes()) {
                             finalResponseListener.onFailure(
                                 new ElasticsearchStatusException(
                                     "Native model store has exceeded the maximum acceptable size of {}, "

+ 8 - 8
x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/FrozenSearchableSnapshotsIntegTests.java

@@ -208,12 +208,12 @@ public class FrozenSearchableSnapshotsIntegTests extends BaseFrozenSearchableSna
 
                 for (ShardStats shardStats : indicesStatsResponse.getShards()) {
                     StoreStats store = shardStats.getStats().getStore();
-                    assertThat(shardStats.getShardRouting().toString(), store.getReservedSize().getBytes(), equalTo(0L));
-                    assertThat(shardStats.getShardRouting().toString(), store.getSize().getBytes(), equalTo(0L));
+                    assertThat(shardStats.getShardRouting().toString(), store.reservedSizeInBytes(), equalTo(0L));
+                    assertThat(shardStats.getShardRouting().toString(), store.sizeInBytes(), equalTo(0L));
                 }
                 if (indicesStatsResponse.getShards().length > 0) {
-                    assertThat(indicesStatsResponse.getTotal().getStore().getReservedSize().getBytes(), equalTo(0L));
-                    assertThat(indicesStatsResponse.getTotal().getStore().getSize().getBytes(), equalTo(0L));
+                    assertThat(indicesStatsResponse.getTotal().getStore().reservedSizeInBytes(), equalTo(0L));
+                    assertThat(indicesStatsResponse.getTotal().getStore().sizeInBytes(), equalTo(0L));
                 }
             }
         }, "test-stats-watcher");
@@ -251,8 +251,8 @@ public class FrozenSearchableSnapshotsIntegTests extends BaseFrozenSearchableSna
             StoreStats store = shardStats.getStats().getStore();
 
             final ShardRouting shardRouting = shardStats.getShardRouting();
-            assertThat(shardRouting.toString(), store.getReservedSize().getBytes(), equalTo(0L));
-            assertThat(shardRouting.toString(), store.getSize().getBytes(), equalTo(0L));
+            assertThat(shardRouting.toString(), store.reservedSizeInBytes(), equalTo(0L));
+            assertThat(shardRouting.toString(), store.sizeInBytes(), equalTo(0L));
 
             // the original shard size from the snapshot
             final long originalSize = snapshotShards.get(shardRouting.getId()).getStats().getTotalSize();
@@ -273,11 +273,11 @@ public class FrozenSearchableSnapshotsIntegTests extends BaseFrozenSearchableSna
             final ByteBuffersDirectory inMemoryDir = (ByteBuffersDirectory) unwrappedDir;
             assertThat(inMemoryDir.listAll(), arrayWithSize(1));
 
-            assertThat(shardRouting.toString(), store.getTotalDataSetSize().getBytes(), equalTo(originalSize));
+            assertThat(shardRouting.toString(), store.totalDataSetSizeInBytes(), equalTo(originalSize));
         }
 
         final StoreStats store = indicesStatsResponse.getTotal().getStore();
-        assertThat(store.getTotalDataSetSize().getBytes(), equalTo(totalExpectedSize));
+        assertThat(store.totalDataSetSizeInBytes(), equalTo(totalExpectedSize));
 
         statsWatcherRunning.set(false);
         statsWatcher.join();

+ 1 - 1
x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsIntegTests.java

@@ -498,7 +498,7 @@ public class SearchableSnapshotsIntegTests extends BaseSearchableSnapshotsIntegT
         IndexStats indexStats = indicesStats.getIndex(indexName);
         Map<String, Long> maxShardSizeByNode = new HashMap<>();
         for (ShardStats shard : indexStats.getShards()) {
-            long sizeInBytes = shard.getStats().getStore().getSizeInBytes();
+            long sizeInBytes = shard.getStats().getStore().sizeInBytes();
             if (sizeInBytes > 0) {
                 maxShardSizeByNode.compute(
                     shard.getShardRouting().currentNodeId(),