From 997fbb372cdec82eb803d32de5147dbccec21e64 Mon Sep 17 00:00:00 2001 From: Ievgen Degtiarenko Date: Mon, 18 Dec 2023 11:53:27 +0100 Subject: [PATCH] Simplify InternalClusterInfoService (#103370) --- .../admin/cluster/stats/ClusterStatsIT.java | 2 +- .../indices/recovery/IndexRecoveryIT.java | 9 +++-- .../RepositoryThrottlingStatsIT.java | 2 +- .../ResizeNumberOfShardsCalculator.java | 2 +- .../cluster/InternalClusterInfoService.java | 9 ++--- .../cluster/routing/RoutingTable.java | 27 --------------- .../elasticsearch/index/shard/DocsStats.java | 2 +- .../elasticsearch/index/store/StoreStats.java | 34 +++++++------------ .../rest/action/cat/RestAllocationAction.java | 2 +- .../rest/action/cat/RestShardsAction.java | 4 +-- .../elasticsearch/cluster/DiskUsageTests.java | 10 +----- .../elasticsearch/index/store/StoreTests.java | 22 ++++++------ .../MockInternalClusterInfoService.java | 2 +- .../storage/FrozenStorageDeciderIT.java | 4 +-- .../storage/ProactiveStorageIT.java | 2 +- .../storage/ReactiveStorageIT.java | 8 ++--- .../xpack/ccr/CcrRepositoryIT.java | 4 +-- .../xpack/ccr/repository/CcrRepository.java | 2 +- .../TransportPutTrainedModelAction.java | 2 +- .../FrozenSearchableSnapshotsIntegTests.java | 16 ++++----- .../SearchableSnapshotsIntegTests.java | 2 +- 21 files changed, 59 insertions(+), 108 deletions(-) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/stats/ClusterStatsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/stats/ClusterStatsIT.java index 93fc17a9a02eb..1fda9c67a0beb 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/stats/ClusterStatsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/stats/ClusterStatsIT.java @@ -207,7 +207,7 @@ public void testValuesSmokeScreen() throws IOException, ExecutionException, Inte 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)); diff --git a/server/src/internalClusterTest/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java b/server/src/internalClusterTest/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java index d40d2e02415b1..5dcd8b5b0e34f 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java @@ -1745,12 +1745,12 @@ public void testReservesBytesDuringPeerRecoveryPhaseOne() throws Exception { .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 void testReservesBytesDuringPeerRecoveryPhaseOne() throws Exception { .get(0) .getIndices() .getStore() - .getReservedSize() - .getBytes(), + .reservedSizeInBytes(), greaterThan(0L) ); } @@ -1785,7 +1784,7 @@ public void testReservesBytesDuringPeerRecoveryPhaseOne() throws Exception { .get() .getNodes() .stream() - .mapToLong(n -> n.getIndices().getStore().getReservedSize().getBytes()) + .mapToLong(n -> n.getIndices().getStore().reservedSizeInBytes()) .sum(), equalTo(0L) ); diff --git a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/RepositoryThrottlingStatsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/RepositoryThrottlingStatsIT.java index 0f0858982b4ad..d8bc9327a2edd 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/RepositoryThrottlingStatsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/RepositoryThrottlingStatsIT.java @@ -37,7 +37,7 @@ public void testRepositoryThrottlingStats() throws Exception { 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); diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/shrink/ResizeNumberOfShardsCalculator.java b/server/src/main/java/org/elasticsearch/action/admin/indices/shrink/ResizeNumberOfShardsCalculator.java index 31807919fd9d9..076841e3efadc 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/shrink/ResizeNumberOfShardsCalculator.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/shrink/ResizeNumberOfShardsCalculator.java @@ -69,7 +69,7 @@ public int calculate(Integer numberOfShards, ByteSizeValue maxPrimaryShardSize, } } 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) { diff --git a/server/src/main/java/org/elasticsearch/cluster/InternalClusterInfoService.java b/server/src/main/java/org/elasticsearch/cluster/InternalClusterInfoService.java index 1744bcc91b834..26c453d419f4c 100644 --- a/server/src/main/java/org/elasticsearch/cluster/InternalClusterInfoService.java +++ b/server/src/main/java/org/elasticsearch/cluster/InternalClusterInfoService.java @@ -26,7 +26,6 @@ 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> nextRefreshListeners = new ArrayList<>(); - private final ClusterService clusterService; private AsyncRefresh currentRefresh; private RefreshScheduler refreshScheduler; @@ -108,7 +106,6 @@ public InternalClusterInfoService(Settings settings, ClusterService clusterServi 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 void onResponse(IndicesStatsResponse indicesStatsResponse) { final Map reservedSpaceBuilders = new HashMap<>(); buildShardLevelInfo( - clusterService.state().routingTable(), adjustShardStats(stats), shardSizeByIdentifierBuilder, shardDataSetSizeBuilder, @@ -445,7 +441,6 @@ public void addListener(Consumer clusterInfoConsumer) { } static void buildShardLevelInfo( - RoutingTable routingTable, ShardStats[] stats, Map shardSizes, Map shardDataSetSizeBuilder, @@ -453,7 +448,7 @@ static void buildShardLevelInfo( Map 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 @@ static void buildShardLevelInfo( } 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); diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java b/server/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java index 986a6bd0385e8..723d65fbc2a3f 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java @@ -148,33 +148,6 @@ public IndexShardRoutingTable shardRoutingTable(ShardId shardId) { 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()); diff --git a/server/src/main/java/org/elasticsearch/index/shard/DocsStats.java b/server/src/main/java/org/elasticsearch/index/shard/DocsStats.java index 1b4b6405df71c..0d8fc52cddacf 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/DocsStats.java +++ b/server/src/main/java/org/elasticsearch/index/shard/DocsStats.java @@ -63,7 +63,7 @@ public long getDeleted() { /** * 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; diff --git a/server/src/main/java/org/elasticsearch/index/store/StoreStats.java b/server/src/main/java/org/elasticsearch/index/store/StoreStats.java index cda87a421bd32..f0df51d4cb78b 100644 --- a/server/src/main/java/org/elasticsearch/index/store/StoreStats.java +++ b/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 StoreStats(StreamInput in) throws IOException { 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 StoreStats(long sizeInBytes, long totalDataSetSizeInBytes, long reservedS 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 void add(StoreStats stats) { } 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 long sizeInBytes() { 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 long totalDataSetSizeInBytes() { * the reserved size is unknown. */ public ByteSizeValue getReservedSize() { - return ByteSizeValue.ofBytes(reservedSize); + return ByteSizeValue.ofBytes(reservedSizeInBytes); } @Override @@ -123,7 +115,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeVLong(totalDataSetSizeInBytes); } if (out.getTransportVersion().onOrAfter(RESERVED_BYTES_VERSION)) { - out.writeZLong(reservedSize); + out.writeZLong(reservedSizeInBytes); } } @@ -144,12 +136,12 @@ public boolean equals(Object o) { 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 { diff --git a/server/src/main/java/org/elasticsearch/rest/action/cat/RestAllocationAction.java b/server/src/main/java/org/elasticsearch/rest/action/cat/RestAllocationAction.java index 5e9b2c8452579..068c809554631 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/cat/RestAllocationAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/cat/RestAllocationAction.java @@ -130,7 +130,7 @@ private Table buildTable(RestRequest request, final ClusterStateResponse state, 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); diff --git a/server/src/main/java/org/elasticsearch/rest/action/cat/RestShardsAction.java b/server/src/main/java/org/elasticsearch/rest/action/cat/RestShardsAction.java index 94fffd6582155..a57d45e07fd15 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/cat/RestShardsAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/cat/RestShardsAction.java @@ -293,8 +293,8 @@ Table buildTable(RestRequest request, ClusterStateResponse state, IndicesStatsRe } 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(); diff --git a/server/src/test/java/org/elasticsearch/cluster/DiskUsageTests.java b/server/src/test/java/org/elasticsearch/cluster/DiskUsageTests.java index 7b2795abfd62d..75439578448a4 100644 --- a/server/src/test/java/org/elasticsearch/cluster/DiskUsageTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/DiskUsageTests.java @@ -13,7 +13,6 @@ 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 void testFillShardLevelInfo() { Map shardSizes = new HashMap<>(); Map shardDataSetSizes = new HashMap<>(); Map 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, diff --git a/server/src/test/java/org/elasticsearch/index/store/StoreTests.java b/server/src/test/java/org/elasticsearch/index/store/StoreTests.java index dfbfb737c9ab2..3eb4675d37e97 100644 --- a/server/src/test/java/org/elasticsearch/index/store/StoreTests.java +++ b/server/src/test/java/org/elasticsearch/index/store/StoreTests.java @@ -775,22 +775,22 @@ public void testStoreStats() throws IOException { 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 void testStoreStats() throws IOException { 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); diff --git a/test/framework/src/main/java/org/elasticsearch/cluster/MockInternalClusterInfoService.java b/test/framework/src/main/java/org/elasticsearch/cluster/MockInternalClusterInfoService.java index bb86dab60b0eb..1004ea5b50119 100644 --- a/test/framework/src/main/java/org/elasticsearch/cluster/MockInternalClusterInfoService.java +++ b/test/framework/src/main/java/org/elasticsearch/cluster/MockInternalClusterInfoService.java @@ -107,7 +107,7 @@ ShardStats[] adjustShardStats(ShardStats[] shardsStats) { 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; diff --git a/x-pack/plugin/autoscaling/src/internalClusterTest/java/org/elasticsearch/xpack/autoscaling/storage/FrozenStorageDeciderIT.java b/x-pack/plugin/autoscaling/src/internalClusterTest/java/org/elasticsearch/xpack/autoscaling/storage/FrozenStorageDeciderIT.java index 13056ed2e4d5e..13e7d3aca1501 100644 --- a/x-pack/plugin/autoscaling/src/internalClusterTest/java/org/elasticsearch/xpack/autoscaling/storage/FrozenStorageDeciderIT.java +++ b/x-pack/plugin/autoscaling/src/internalClusterTest/java/org/elasticsearch/xpack/autoscaling/storage/FrozenStorageDeciderIT.java @@ -31,8 +31,8 @@ public void testScale() throws Exception { 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 ) ) ); diff --git a/x-pack/plugin/autoscaling/src/internalClusterTest/java/org/elasticsearch/xpack/autoscaling/storage/ProactiveStorageIT.java b/x-pack/plugin/autoscaling/src/internalClusterTest/java/org/elasticsearch/xpack/autoscaling/storage/ProactiveStorageIT.java index c5e062df5e77c..d84c5977cba93 100644 --- a/x-pack/plugin/autoscaling/src/internalClusterTest/java/org/elasticsearch/xpack/autoscaling/storage/ProactiveStorageIT.java +++ b/x-pack/plugin/autoscaling/src/internalClusterTest/java/org/elasticsearch/xpack/autoscaling/storage/ProactiveStorageIT.java @@ -67,7 +67,7 @@ public void testScaleUp() throws IOException, InterruptedException { 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. diff --git a/x-pack/plugin/autoscaling/src/internalClusterTest/java/org/elasticsearch/xpack/autoscaling/storage/ReactiveStorageIT.java b/x-pack/plugin/autoscaling/src/internalClusterTest/java/org/elasticsearch/xpack/autoscaling/storage/ReactiveStorageIT.java index 5c097cdc24ed1..5f724509ec98a 100644 --- a/x-pack/plugin/autoscaling/src/internalClusterTest/java/org/elasticsearch/xpack/autoscaling/storage/ReactiveStorageIT.java +++ b/x-pack/plugin/autoscaling/src/internalClusterTest/java/org/elasticsearch/xpack/autoscaling/storage/ReactiveStorageIT.java @@ -78,7 +78,7 @@ public void testScaleUp() throws InterruptedException { 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 void testScaleWhileShrinking() throws Exception { 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 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 void testScaleDuringSplitOrClone() throws Exception { 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; diff --git a/x-pack/plugin/ccr/src/internalClusterTest/java/org/elasticsearch/xpack/ccr/CcrRepositoryIT.java b/x-pack/plugin/ccr/src/internalClusterTest/java/org/elasticsearch/xpack/ccr/CcrRepositoryIT.java index f7baafa8402d0..dff3ff935595f 100644 --- a/x-pack/plugin/ccr/src/internalClusterTest/java/org/elasticsearch/xpack/ccr/CcrRepositoryIT.java +++ b/x-pack/plugin/ccr/src/internalClusterTest/java/org/elasticsearch/xpack/ccr/CcrRepositoryIT.java @@ -537,7 +537,7 @@ public void testCcrRepositoryFetchesSnapshotShardSizeFromIndexShardStoreStats() 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 void testCcrRepositoryFetchesSnapshotShardSizeFromIndexShardStoreStats() 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()) ); } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java index b90b203e2d29f..c99726803e00e 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java @@ -517,7 +517,7 @@ public IndexShardSnapshotStatus.Copy getShardSnapshotStatus(SnapshotId snapshotI 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); } } diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportPutTrainedModelAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportPutTrainedModelAction.java index 7462b6cd918aa..af9ade4f97995 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportPutTrainedModelAction.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportPutTrainedModelAction.java @@ -289,7 +289,7 @@ protected void masterOperation( .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 {}, " diff --git a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/FrozenSearchableSnapshotsIntegTests.java b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/FrozenSearchableSnapshotsIntegTests.java index 5ef524f8211c1..18b4e6ed7cb31 100644 --- a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/FrozenSearchableSnapshotsIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/FrozenSearchableSnapshotsIntegTests.java @@ -208,12 +208,12 @@ public void testCreateAndRestorePartialSearchableSnapshot() throws Exception { 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 void testCreateAndRestorePartialSearchableSnapshot() throws Exception { 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 void testCreateAndRestorePartialSearchableSnapshot() throws Exception { 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(); diff --git a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsIntegTests.java b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsIntegTests.java index 876ff9ebdb86f..38222f64b282b 100644 --- a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsIntegTests.java @@ -498,7 +498,7 @@ private Map getMaxShardSizeByNodeInBytes(String indexName) { IndexStats indexStats = indicesStats.getIndex(indexName); Map 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(),