diff --git a/CHANGELOG.md b/CHANGELOG.md index 53ff23e0a1a78..c2a25b4b9c249 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -176,6 +176,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Redefine telemetry context restoration and propagation ([#9617](https://github.com/opensearch-project/OpenSearch/pull/9617)) - Use non-concurrent path for sort request on timeseries index and field([#9562](https://github.com/opensearch-project/OpenSearch/pull/9562)) - Added sampler based on `Blanket Probabilistic Sampling rate` and `Override for on demand` ([#9621](https://github.com/opensearch-project/OpenSearch/issues/9621)) +- [Remote Store] Add support for Remote Translog Store stats in `_remotestore/stats/` API ([#9263](https://github.com/opensearch-project/OpenSearch/pull/9263)) ### Deprecated diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreStatsIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreStatsIT.java index ae4f52f0f4e18..2b9fb9c497cb6 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreStatsIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreStatsIT.java @@ -22,6 +22,7 @@ import org.opensearch.common.settings.Settings; import org.opensearch.index.IndexSettings; import org.opensearch.index.remote.RemoteSegmentTransferTracker; +import org.opensearch.index.remote.RemoteTranslogTransferTracker; import org.opensearch.test.InternalTestCluster; import org.opensearch.test.OpenSearchIntegTestCase; import org.junit.Before; @@ -72,9 +73,14 @@ public void testStatsResponseFromAllNodes() { .filter(stat -> indexShardId.equals(stat.getSegmentStats().shardId.toString())) .collect(Collectors.toList()); assertEquals(1, matches.size()); - RemoteSegmentTransferTracker.Stats stats = matches.get(0).getSegmentStats(); - validateSegmentUploadStats(stats); - assertEquals(0, stats.directoryFileTransferTrackerStats.transferredBytesStarted); + + RemoteSegmentTransferTracker.Stats segmentStats = matches.get(0).getSegmentStats(); + validateSegmentUploadStats(segmentStats); + assertEquals(0, segmentStats.directoryFileTransferTrackerStats.transferredBytesStarted); + + RemoteTranslogTransferTracker.Stats translogStats = matches.get(0).getTranslogStats(); + assertNonZeroTranslogUploadStatsNoFailures(translogStats); + assertZeroTranslogDownloadStats(translogStats); } // Step 3 - Enable replicas on the existing indices and ensure that download @@ -92,13 +98,20 @@ public void testStatsResponseFromAllNodes() { for (RemoteStoreStats stat : matches) { ShardRouting routing = stat.getShardRouting(); validateShardRouting(routing); - RemoteSegmentTransferTracker.Stats stats = stat.getSegmentStats(); + RemoteSegmentTransferTracker.Stats segmentStats = stat.getSegmentStats(); + RemoteTranslogTransferTracker.Stats translogStats = stat.getTranslogStats(); if (routing.primary()) { - validateSegmentUploadStats(stats); - assertEquals(0, stats.directoryFileTransferTrackerStats.transferredBytesStarted); + validateSegmentUploadStats(segmentStats); + assertEquals(0, segmentStats.directoryFileTransferTrackerStats.transferredBytesStarted); + + assertNonZeroTranslogUploadStatsNoFailures(translogStats); + assertZeroTranslogDownloadStats(translogStats); } else { - validateSegmentDownloadStats(stats); - assertEquals(0, stats.totalUploadsStarted); + validateSegmentDownloadStats(segmentStats); + assertEquals(0, segmentStats.totalUploadsStarted); + + assertZeroTranslogUploadStats(translogStats); + assertZeroTranslogDownloadStats(translogStats); } } } @@ -124,10 +137,15 @@ public void testStatsResponseAllShards() { RemoteStoreStatsResponse response = remoteStoreStatsRequestBuilder.get(); assertEquals(3, response.getSuccessfulShards()); assertTrue(response.getRemoteStoreStats() != null && response.getRemoteStoreStats().length == 3); + RemoteSegmentTransferTracker.Stats segmentStats = response.getRemoteStoreStats()[0].getSegmentStats(); validateSegmentUploadStats(segmentStats); assertEquals(0, segmentStats.directoryFileTransferTrackerStats.transferredBytesStarted); + RemoteTranslogTransferTracker.Stats translogStats = response.getRemoteStoreStats()[0].getTranslogStats(); + assertNonZeroTranslogUploadStatsNoFailures(translogStats); + assertZeroTranslogDownloadStats(translogStats); + // Step 3 - Enable replicas on the existing indices and ensure that download // stats are being populated as well changeReplicaCountAndEnsureGreen(1); @@ -138,12 +156,19 @@ public void testStatsResponseAllShards() { ShardRouting routing = stat.getShardRouting(); validateShardRouting(routing); segmentStats = stat.getSegmentStats(); + translogStats = stat.getTranslogStats(); if (routing.primary()) { validateSegmentUploadStats(segmentStats); assertEquals(0, segmentStats.directoryFileTransferTrackerStats.transferredBytesStarted); + + assertNonZeroTranslogUploadStatsNoFailures(translogStats); + assertZeroTranslogDownloadStats(translogStats); } else { validateSegmentDownloadStats(segmentStats); assertEquals(0, segmentStats.totalUploadsStarted); + + assertZeroTranslogUploadStats(translogStats); + assertZeroTranslogDownloadStats(translogStats); } } @@ -174,6 +199,10 @@ public void testStatsResponseFromLocalNode() { RemoteSegmentTransferTracker.Stats segmentStats = response.getRemoteStoreStats()[0].getSegmentStats(); validateSegmentUploadStats(segmentStats); assertEquals(0, segmentStats.directoryFileTransferTrackerStats.transferredBytesStarted); + + RemoteTranslogTransferTracker.Stats translogStats = response.getRemoteStoreStats()[0].getTranslogStats(); + assertNonZeroTranslogUploadStatsNoFailures(translogStats); + assertZeroTranslogDownloadStats(translogStats); } changeReplicaCountAndEnsureGreen(1); for (String node : nodes) { @@ -187,13 +216,20 @@ public void testStatsResponseFromLocalNode() { for (RemoteStoreStats stat : response.getRemoteStoreStats()) { ShardRouting routing = stat.getShardRouting(); validateShardRouting(routing); - RemoteSegmentTransferTracker.Stats stats = stat.getSegmentStats(); + RemoteSegmentTransferTracker.Stats segmentStats = stat.getSegmentStats(); + RemoteTranslogTransferTracker.Stats translogStats = stat.getTranslogStats(); if (routing.primary()) { - validateSegmentUploadStats(stats); - assertEquals(0, stats.directoryFileTransferTrackerStats.transferredBytesStarted); + validateSegmentUploadStats(segmentStats); + assertEquals(0, segmentStats.directoryFileTransferTrackerStats.transferredBytesStarted); + + assertNonZeroTranslogUploadStatsNoFailures(translogStats); + assertZeroTranslogDownloadStats(translogStats); } else { - validateSegmentDownloadStats(stats); - assertEquals(0, stats.totalUploadsStarted); + validateSegmentDownloadStats(segmentStats); + assertEquals(0, segmentStats.totalUploadsStarted); + + assertZeroTranslogUploadStats(translogStats); + assertZeroTranslogDownloadStats(translogStats); } } } @@ -491,15 +527,19 @@ public void testStatsOnRemoteStoreRestore() throws IOException { RemoteStoreStatsResponse remoteStoreStatsResponse = client().admin().cluster().prepareRemoteStoreStats(INDEX_NAME, "0").get(); Arrays.stream(remoteStoreStatsResponse.getRemoteStoreStats()).forEach(statObject -> { - RemoteSegmentTransferTracker.Stats segmentTracker = statObject.getSegmentStats(); + RemoteSegmentTransferTracker.Stats segmentStats = statObject.getSegmentStats(); // Assert that we have both upload and download stats for the index assertTrue( - segmentTracker.totalUploadsStarted > 0 && segmentTracker.totalUploadsSucceeded > 0 && segmentTracker.totalUploadsFailed == 0 + segmentStats.totalUploadsStarted > 0 && segmentStats.totalUploadsSucceeded > 0 && segmentStats.totalUploadsFailed == 0 ); assertTrue( - segmentTracker.directoryFileTransferTrackerStats.transferredBytesStarted > 0 - && segmentTracker.directoryFileTransferTrackerStats.transferredBytesSucceeded > 0 + segmentStats.directoryFileTransferTrackerStats.transferredBytesStarted > 0 + && segmentStats.directoryFileTransferTrackerStats.transferredBytesSucceeded > 0 ); + + RemoteTranslogTransferTracker.Stats translogStats = statObject.getTranslogStats(); + assertNonZeroTranslogUploadStatsNoFailures(translogStats); + assertNonZeroTranslogDownloadStats(translogStats); }); } @@ -520,19 +560,23 @@ public void testNonZeroPrimaryStatsOnNewlyCreatedIndexWithZeroDocs() throws Exce .get() .getRemoteStoreStats(); Arrays.stream(remoteStoreStats).forEach(statObject -> { - RemoteSegmentTransferTracker.Stats segmentTracker = statObject.getSegmentStats(); + RemoteSegmentTransferTracker.Stats segmentStats = statObject.getSegmentStats(); if (statObject.getShardRouting().primary()) { assertTrue( - segmentTracker.totalUploadsSucceeded == 1 - && segmentTracker.totalUploadsStarted == segmentTracker.totalUploadsSucceeded - && segmentTracker.totalUploadsFailed == 0 + segmentStats.totalUploadsSucceeded == 1 + && segmentStats.totalUploadsStarted == segmentStats.totalUploadsSucceeded + && segmentStats.totalUploadsFailed == 0 ); } else { assertTrue( - segmentTracker.directoryFileTransferTrackerStats.transferredBytesStarted == 0 - && segmentTracker.directoryFileTransferTrackerStats.transferredBytesSucceeded == 0 + segmentStats.directoryFileTransferTrackerStats.transferredBytesStarted == 0 + && segmentStats.directoryFileTransferTrackerStats.transferredBytesSucceeded == 0 ); } + + RemoteTranslogTransferTracker.Stats translogStats = statObject.getTranslogStats(); + assertZeroTranslogUploadStats(translogStats); + assertZeroTranslogDownloadStats(translogStats); }); }, 5, TimeUnit.SECONDS); } @@ -545,9 +589,7 @@ private void indexDocs() { refresh(INDEX_NAME); } int numberOfOperations = randomIntBetween(10, 30); - for (int j = 0; j < numberOfOperations; j++) { - indexSingleDoc(INDEX_NAME); - } + indexBulk(INDEX_NAME, numberOfOperations); } } @@ -594,6 +636,43 @@ private void validateSegmentDownloadStats(RemoteSegmentTransferTracker.Stats sta assertTrue(stats.directoryFileTransferTrackerStats.transferredBytesPerSecMovingAverage > 0); } + private void assertNonZeroTranslogUploadStatsNoFailures(RemoteTranslogTransferTracker.Stats stats) { + assertTrue(stats.uploadBytesStarted > 0); + assertTrue(stats.totalUploadsStarted > 0); + assertEquals(0, stats.uploadBytesFailed); + assertEquals(0, stats.totalUploadsFailed); + assertTrue(stats.uploadBytesSucceeded > 0); + assertTrue(stats.totalUploadsSucceeded > 0); + assertTrue(stats.totalUploadTimeInMillis > 0); + assertTrue(stats.lastSuccessfulUploadTimestamp > 0); + } + + private void assertZeroTranslogUploadStats(RemoteTranslogTransferTracker.Stats stats) { + assertEquals(0, stats.uploadBytesStarted); + assertEquals(0, stats.totalUploadsStarted); + assertEquals(0, stats.uploadBytesFailed); + assertEquals(0, stats.totalUploadsFailed); + assertEquals(0, stats.uploadBytesSucceeded); + assertEquals(0, stats.totalUploadsSucceeded); + assertEquals(0, stats.totalUploadTimeInMillis); + assertEquals(0, stats.lastSuccessfulUploadTimestamp); + } + + private void assertNonZeroTranslogDownloadStats(RemoteTranslogTransferTracker.Stats stats) { + assertTrue(stats.downloadBytesSucceeded > 0); + assertTrue(stats.totalDownloadsSucceeded > 0); + // TODO: Need to simulate a delay for this assertion to avoid flakiness + // assertTrue(stats.totalDownloadTimeInMillis > 0); + assertTrue(stats.lastSuccessfulDownloadTimestamp > 0); + } + + private void assertZeroTranslogDownloadStats(RemoteTranslogTransferTracker.Stats stats) { + assertEquals(0, stats.downloadBytesSucceeded); + assertEquals(0, stats.totalDownloadsSucceeded); + assertEquals(0, stats.totalDownloadTimeInMillis); + assertEquals(0, stats.lastSuccessfulDownloadTimestamp); + } + // Validate if the shardRouting obtained from cluster state contains the exact same routing object // parameters as obtained from the remote store stats API private void validateShardRouting(ShardRouting routing) { diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/remotestore/stats/RemoteStoreStats.java b/server/src/main/java/org/opensearch/action/admin/cluster/remotestore/stats/RemoteStoreStats.java index aa8a8ec098ead..c1cbeac2b4216 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/remotestore/stats/RemoteStoreStats.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/remotestore/stats/RemoteStoreStats.java @@ -15,6 +15,7 @@ import org.opensearch.core.xcontent.ToXContentFragment; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.index.remote.RemoteSegmentTransferTracker; +import org.opensearch.index.remote.RemoteTranslogTransferTracker; import java.io.IOException; @@ -29,15 +30,25 @@ public class RemoteStoreStats implements Writeable, ToXContentFragment { */ private final RemoteSegmentTransferTracker.Stats remoteSegmentShardStats; + /** + * Stats related to Remote Translog Store operations + */ + private final RemoteTranslogTransferTracker.Stats remoteTranslogShardStats; private final ShardRouting shardRouting; - public RemoteStoreStats(RemoteSegmentTransferTracker.Stats remoteSegmentUploadShardStats, ShardRouting shardRouting) { + RemoteStoreStats( + RemoteSegmentTransferTracker.Stats remoteSegmentUploadShardStats, + RemoteTranslogTransferTracker.Stats remoteTranslogShardStats, + ShardRouting shardRouting + ) { this.remoteSegmentShardStats = remoteSegmentUploadShardStats; + this.remoteTranslogShardStats = remoteTranslogShardStats; this.shardRouting = shardRouting; } - public RemoteStoreStats(StreamInput in) throws IOException { - this.remoteSegmentShardStats = in.readOptionalWriteable(RemoteSegmentTransferTracker.Stats::new); + RemoteStoreStats(StreamInput in) throws IOException { + remoteSegmentShardStats = in.readOptionalWriteable(RemoteSegmentTransferTracker.Stats::new); + remoteTranslogShardStats = in.readOptionalWriteable(RemoteTranslogTransferTracker.Stats::new); this.shardRouting = new ShardRouting(in); } @@ -49,10 +60,15 @@ public ShardRouting getShardRouting() { return shardRouting; } + public RemoteTranslogTransferTracker.Stats getTranslogStats() { + return remoteTranslogShardStats; + } + @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(); buildShardRouting(builder); + builder.startObject(Fields.SEGMENT); builder.startObject(SubFields.DOWNLOAD); // Ensuring that we are not showing 0 metrics to the user @@ -67,15 +83,88 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws } builder.endObject(); // segment.upload builder.endObject(); // segment + + builder.startObject(Fields.TRANSLOG); + builder.startObject(SubFields.UPLOAD); + // Ensuring that we are not showing 0 metrics to the user + if (remoteTranslogShardStats.totalUploadsStarted > 0) { + buildTranslogUploadStats(builder); + } + builder.endObject(); // translog.upload + builder.startObject(SubFields.DOWNLOAD); + // Ensuring that we are not showing 0 metrics to the user + if (remoteTranslogShardStats.totalDownloadsSucceeded > 0) { + buildTranslogDownloadStats(builder); + } + builder.endObject(); // translog.download + builder.endObject(); // translog + return builder.endObject(); } @Override public void writeTo(StreamOutput out) throws IOException { out.writeOptionalWriteable(remoteSegmentShardStats); + out.writeOptionalWriteable(remoteTranslogShardStats); shardRouting.writeTo(out); } + private void buildTranslogUploadStats(XContentBuilder builder) throws IOException { + builder.field(UploadStatsFields.LAST_SUCCESSFUL_UPLOAD_TIMESTAMP, remoteTranslogShardStats.lastSuccessfulUploadTimestamp); + + builder.startObject(UploadStatsFields.TOTAL_UPLOADS); + builder.field(SubFields.STARTED, remoteTranslogShardStats.totalUploadsStarted) + .field(SubFields.FAILED, remoteTranslogShardStats.totalUploadsFailed) + .field(SubFields.SUCCEEDED, remoteTranslogShardStats.totalUploadsSucceeded); + builder.endObject(); + + builder.startObject(UploadStatsFields.TOTAL_UPLOADS_IN_BYTES); + builder.field(SubFields.STARTED, remoteTranslogShardStats.uploadBytesStarted) + .field(SubFields.FAILED, remoteTranslogShardStats.uploadBytesFailed) + .field(SubFields.SUCCEEDED, remoteTranslogShardStats.uploadBytesSucceeded); + builder.endObject(); + + builder.field(UploadStatsFields.TOTAL_UPLOAD_TIME_IN_MILLIS, remoteTranslogShardStats.totalUploadTimeInMillis); + + builder.startObject(UploadStatsFields.UPLOAD_SIZE_IN_BYTES); + builder.field(SubFields.MOVING_AVG, remoteTranslogShardStats.uploadBytesMovingAverage); + builder.endObject(); + + builder.startObject(UploadStatsFields.UPLOAD_SPEED_IN_BYTES_PER_SEC); + builder.field(SubFields.MOVING_AVG, remoteTranslogShardStats.uploadBytesPerSecMovingAverage); + builder.endObject(); + + builder.startObject(UploadStatsFields.UPLOAD_TIME_IN_MILLIS); + builder.field(SubFields.MOVING_AVG, remoteTranslogShardStats.uploadTimeMovingAverage); + builder.endObject(); + } + + private void buildTranslogDownloadStats(XContentBuilder builder) throws IOException { + builder.field(DownloadStatsFields.LAST_SUCCESSFUL_DOWNLOAD_TIMESTAMP, remoteTranslogShardStats.lastSuccessfulDownloadTimestamp); + + builder.startObject(DownloadStatsFields.TOTAL_DOWNLOADS); + builder.field(SubFields.SUCCEEDED, remoteTranslogShardStats.totalDownloadsSucceeded); + builder.endObject(); + + builder.startObject(DownloadStatsFields.TOTAL_DOWNLOADS_IN_BYTES); + builder.field(SubFields.SUCCEEDED, remoteTranslogShardStats.downloadBytesSucceeded); + builder.endObject(); + + builder.field(DownloadStatsFields.TOTAL_DOWNLOAD_TIME_IN_MILLIS, remoteTranslogShardStats.totalDownloadTimeInMillis); + + builder.startObject(DownloadStatsFields.DOWNLOAD_SIZE_IN_BYTES); + builder.field(SubFields.MOVING_AVG, remoteTranslogShardStats.downloadBytesMovingAverage); + builder.endObject(); + + builder.startObject(DownloadStatsFields.DOWNLOAD_SPEED_IN_BYTES_PER_SEC); + builder.field(SubFields.MOVING_AVG, remoteTranslogShardStats.downloadBytesPerSecMovingAverage); + builder.endObject(); + + builder.startObject(DownloadStatsFields.DOWNLOAD_TIME_IN_MILLIS); + builder.field(SubFields.MOVING_AVG, remoteTranslogShardStats.downloadTimeMovingAverage); + builder.endObject(); + } + private void buildSegmentUploadStats(XContentBuilder builder) throws IOException { builder.field(UploadStatsFields.LOCAL_REFRESH_TIMESTAMP, remoteSegmentShardStats.localRefreshClockTimeMs) .field(UploadStatsFields.REMOTE_REFRESH_TIMESTAMP, remoteSegmentShardStats.remoteRefreshClockTimeMs) @@ -98,7 +187,7 @@ private void buildSegmentUploadStats(XContentBuilder builder) throws IOException .field(SubFields.LAST_SUCCESSFUL, remoteSegmentShardStats.lastSuccessfulRemoteRefreshBytes) .field(SubFields.MOVING_AVG, remoteSegmentShardStats.uploadBytesMovingAverage); builder.endObject(); - builder.startObject(UploadStatsFields.UPLOAD_LATENCY_IN_BYTES_PER_SEC) + builder.startObject(UploadStatsFields.UPLOAD_SPEED_IN_BYTES_PER_SEC) .field(SubFields.MOVING_AVG, remoteSegmentShardStats.uploadBytesPerSecMovingAverage); builder.endObject(); builder.startObject(UploadStatsFields.REMOTE_REFRESH_LATENCY_IN_MILLIS) @@ -133,6 +222,9 @@ private void buildShardRouting(XContentBuilder builder) throws IOException { builder.endObject(); } + /** + * Fields for remote store stats response + */ static final class Fields { static final String ROUTING = "routing"; static final String SEGMENT = "segment"; @@ -189,47 +281,92 @@ static final class UploadStatsFields { */ static final String TOTAL_SYNCS_TO_REMOTE = "total_syncs_to_remote"; + /** + * Represents the size of new data to be uploaded as part of a refresh + */ + static final String REMOTE_REFRESH_SIZE_IN_BYTES = "remote_refresh_size_in_bytes"; + + /** + * Time taken by a single remote refresh + */ + static final String REMOTE_REFRESH_LATENCY_IN_MILLIS = "remote_refresh_latency_in_millis"; + + /** + * Timestamp of last successful remote store upload + */ + static final String LAST_SUCCESSFUL_UPLOAD_TIMESTAMP = "last_successful_upload_timestamp"; + + /** + * Count of files uploaded to remote store + */ + static final String TOTAL_UPLOADS = "total_uploads"; + /** * Represents the total uploads to remote store in bytes */ static final String TOTAL_UPLOADS_IN_BYTES = "total_uploads_in_bytes"; /** - * Represents the size of new data to be uploaded as part of a refresh + * Total time spent on remote store uploads */ - static final String REMOTE_REFRESH_SIZE_IN_BYTES = "remote_refresh_size_in_bytes"; + static final String TOTAL_UPLOAD_TIME_IN_MILLIS = "total_upload_time_in_millis"; + + /** + * Represents the size of new data to be transferred as part of a remote store upload + */ + static final String UPLOAD_SIZE_IN_BYTES = "upload_size_in_bytes"; /** * Represents the speed of remote store uploads in bytes per sec */ - static final String UPLOAD_LATENCY_IN_BYTES_PER_SEC = "upload_latency_in_bytes_per_sec"; + static final String UPLOAD_SPEED_IN_BYTES_PER_SEC = "upload_speed_in_bytes_per_sec"; /** - * Time taken by a single remote refresh + * Time taken by a remote store upload */ - static final String REMOTE_REFRESH_LATENCY_IN_MILLIS = "remote_refresh_latency_in_millis"; + static final String UPLOAD_TIME_IN_MILLIS = "upload_time_in_millis"; } static final class DownloadStatsFields { + /** + * Epoch timestamp of the last successful download + */ + public static final String LAST_SUCCESSFUL_DOWNLOAD_TIMESTAMP = "last_successful_download_timestamp"; + /** * Last successful sync from remote in milliseconds */ static final String LAST_SYNC_TIMESTAMP = "last_sync_timestamp"; /** - * Total bytes of segment files downloaded from the remote store for a specific shard + * Count of files downloaded from remote store + */ + public static final String TOTAL_DOWNLOADS = "total_downloads"; + + /** + * Total time spent in downloads from remote store + */ + public static final String TOTAL_DOWNLOAD_TIME_IN_MILLIS = "total_download_time_in_millis"; + + /** + * Total bytes of files downloaded from the remote store */ static final String TOTAL_DOWNLOADS_IN_BYTES = "total_downloads_in_bytes"; /** - * Size of each segment file downloaded from the remote store + * Average size of a file downloaded from the remote store */ static final String DOWNLOAD_SIZE_IN_BYTES = "download_size_in_bytes"; /** - * Speed (in bytes/sec) for segment file downloads + * Average speed (in bytes/sec) of a remote store download */ static final String DOWNLOAD_SPEED_IN_BYTES_PER_SEC = "download_speed_in_bytes_per_sec"; + + /** + * Average time spent on a remote store download + */ + public static final String DOWNLOAD_TIME_IN_MILLIS = "download_time_in_millis"; } /** diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/remotestore/stats/TransportRemoteStoreStatsAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/remotestore/stats/TransportRemoteStoreStatsAction.java index 2595f783b4cf3..bd8db4a160bf6 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/remotestore/stats/TransportRemoteStoreStatsAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/remotestore/stats/TransportRemoteStoreStatsAction.java @@ -25,6 +25,7 @@ import org.opensearch.index.IndexService; import org.opensearch.index.remote.RemoteSegmentTransferTracker; import org.opensearch.index.remote.RemoteStoreStatsTrackerFactory; +import org.opensearch.index.remote.RemoteTranslogTransferTracker; import org.opensearch.index.shard.IndexShard; import org.opensearch.index.shard.ShardNotFoundException; import org.opensearch.indices.IndicesService; @@ -157,6 +158,11 @@ protected RemoteStoreStats shardOperation(RemoteStoreStatsRequest request, Shard indexShard.shardId() ); assert Objects.nonNull(remoteSegmentTransferTracker); - return new RemoteStoreStats(remoteSegmentTransferTracker.stats(), indexShard.routingEntry()); + RemoteTranslogTransferTracker remoteTranslogTransferTracker = remoteStoreStatsTrackerFactory.getRemoteTranslogTransferTracker( + indexShard.shardId() + ); + assert Objects.nonNull(remoteTranslogTransferTracker); + + return new RemoteStoreStats(remoteSegmentTransferTracker.stats(), remoteTranslogTransferTracker.stats(), indexShard.routingEntry()); } } diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteSegmentTransferTracker.java b/server/src/main/java/org/opensearch/index/remote/RemoteSegmentTransferTracker.java index 564fdfbc0761d..d18462cd65fc1 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteSegmentTransferTracker.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteSegmentTransferTracker.java @@ -12,7 +12,6 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.opensearch.common.CheckedFunction; import org.opensearch.common.logging.Loggers; -import org.opensearch.common.util.MovingAverage; import org.opensearch.common.util.Streak; import org.opensearch.common.util.concurrent.ConcurrentCollections; import org.opensearch.core.common.io.stream.StreamInput; @@ -29,7 +28,6 @@ import java.util.Objects; import java.util.Set; import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; import static org.opensearch.index.shard.RemoteStoreRefreshListener.EXCLUDE_FILES; @@ -39,15 +37,10 @@ * * @opensearch.internal */ -public class RemoteSegmentTransferTracker { +public class RemoteSegmentTransferTracker extends RemoteTransferTracker { private final Logger logger; - /** - * ShardId for which this instance tracks the remote segment upload metadata. - */ - private final ShardId shardId; - /** * Every refresh is assigned a sequence number. This is the sequence number of the most recent refresh. */ @@ -93,41 +86,6 @@ public class RemoteSegmentTransferTracker { */ private volatile long lastSuccessfulRemoteRefreshBytes; - /** - * Cumulative sum of size in bytes of segment files for which upload has started during remote refresh. - */ - private final AtomicLong uploadBytesStarted = new AtomicLong(); - - /** - * Cumulative sum of size in bytes of segment files for which upload has failed during remote refresh. - */ - private final AtomicLong uploadBytesFailed = new AtomicLong(); - - /** - * Cumulative sum of size in bytes of segment files for which upload has succeeded during remote refresh. - */ - private final AtomicLong uploadBytesSucceeded = new AtomicLong(); - - /** - * Cumulative sum of count of remote refreshes that have started. - */ - private volatile long totalUploadsStarted; - - /** - * Cumulative sum of count of remote refreshes that have failed. - */ - private volatile long totalUploadsFailed; - - /** - * Cumulative sum of count of remote refreshes that have succeeded. - */ - private volatile long totalUploadsSucceeded; - - /** - * Cumulative sum of time taken in remote refresh (in milliseconds) [Tracked per file] - */ - private AtomicLong totalUploadTimeInMs = new AtomicLong(); - /** * Cumulative sum of rejection counts for this shard. */ @@ -159,33 +117,6 @@ public class RemoteSegmentTransferTracker { */ private final Streak failures = new Streak(); - /** - * Provides moving average over the last N total size in bytes of segment files uploaded as part of remote refresh. - * N is window size. Wrapped with {@code AtomicReference} for dynamic changes in window size. - */ - private final AtomicReference uploadBytesMovingAverageReference; - - /** - * This lock object is used for making sure we do not miss any data - */ - private final Object uploadBytesMutex = new Object(); - - /** - * Provides moving average over the last N upload speed (in bytes/s) of segment files uploaded as part of remote refresh. - * N is window size. Wrapped with {@code AtomicReference} for dynamic changes in window size. - */ - private final AtomicReference uploadBytesPerSecMovingAverageReference; - - private final Object uploadBytesPerSecMutex = new Object(); - - /** - * Provides moving average over the last N overall upload time (in nanos) as part of remote refresh.N is window size. - * Wrapped with {@code AtomicReference} for dynamic changes in window size. - */ - private final AtomicReference uploadTimeMsMovingAverageReference; - - private final Object uploadTimeMsMutex = new Object(); - /** * {@link org.opensearch.index.store.Store.StoreDirectory} level file transfer tracker, used to show download stats */ @@ -196,8 +127,9 @@ public RemoteSegmentTransferTracker( DirectoryFileTransferTracker directoryFileTransferTracker, int movingAverageWindowSize ) { + super(shardId, movingAverageWindowSize); + logger = Loggers.getLogger(getClass(), shardId); - this.shardId = shardId; // Both the local refresh time and remote refresh time are set with current time to give consistent view of time lag when it arises. long currentClockTimeMs = System.currentTimeMillis(); long currentTimeMs = System.nanoTime() / 1_000_000L; @@ -205,14 +137,19 @@ public RemoteSegmentTransferTracker( remoteRefreshTimeMs = currentTimeMs; localRefreshClockTimeMs = currentClockTimeMs; remoteRefreshClockTimeMs = currentClockTimeMs; - uploadBytesMovingAverageReference = new AtomicReference<>(new MovingAverage(movingAverageWindowSize)); - uploadBytesPerSecMovingAverageReference = new AtomicReference<>(new MovingAverage(movingAverageWindowSize)); - uploadTimeMsMovingAverageReference = new AtomicReference<>(new MovingAverage(movingAverageWindowSize)); this.directoryFileTransferTracker = directoryFileTransferTracker; } - ShardId getShardId() { - return shardId; + @Override + public void incrementTotalUploadsFailed() { + super.incrementTotalUploadsFailed(); + failures.record(true); + } + + @Override + public void incrementTotalUploadsSucceeded() { + super.incrementTotalUploadsSucceeded(); + failures.record(false); } public long getLocalRefreshSeqNo() { @@ -318,62 +255,12 @@ public long getBytesLag() { return bytesLag; } - public long getUploadBytesStarted() { - return uploadBytesStarted.get(); - } - - public void addUploadBytesStarted(long size) { - uploadBytesStarted.getAndAdd(size); - } - - public long getUploadBytesFailed() { - return uploadBytesFailed.get(); - } - - public void addUploadBytesFailed(long size) { - uploadBytesFailed.getAndAdd(size); - } - - public long getUploadBytesSucceeded() { - return uploadBytesSucceeded.get(); - } - - public void addUploadBytesSucceeded(long size) { - uploadBytesSucceeded.getAndAdd(size); - } - public long getInflightUploadBytes() { return uploadBytesStarted.get() - uploadBytesFailed.get() - uploadBytesSucceeded.get(); } - public long getTotalUploadsStarted() { - return totalUploadsStarted; - } - - public void incrementTotalUploadsStarted() { - totalUploadsStarted += 1; - } - - public long getTotalUploadsFailed() { - return totalUploadsFailed; - } - - public void incrementTotalUploadsFailed() { - totalUploadsFailed += 1; - failures.record(true); - } - - public long getTotalUploadsSucceeded() { - return totalUploadsSucceeded; - } - - public void incrementTotalUploadsSucceeded() { - totalUploadsSucceeded += 1; - failures.record(false); - } - public long getInflightUploads() { - return totalUploadsStarted - totalUploadsFailed - totalUploadsSucceeded; + return totalUploadsStarted.get() - totalUploadsFailed.get() - totalUploadsSucceeded.get(); } public long getRejectionCount() { @@ -452,76 +339,6 @@ int getConsecutiveFailureCount() { return failures.length(); } - boolean isUploadBytesAverageReady() { - return uploadBytesMovingAverageReference.get().isReady(); - } - - double getUploadBytesAverage() { - return uploadBytesMovingAverageReference.get().getAverage(); - } - - public void addUploadBytes(long size) { - lastSuccessfulRemoteRefreshBytes = size; - synchronized (uploadBytesMutex) { - this.uploadBytesMovingAverageReference.get().record(size); - } - } - - /** - * Updates the window size for data collection. This also resets any data collected so far. - * - * @param updatedSize the updated size - */ - void updateMovingAverageWindowSize(int updatedSize) { - synchronized (uploadBytesMutex) { - this.uploadBytesMovingAverageReference.set(this.uploadBytesMovingAverageReference.get().copyWithSize(updatedSize)); - } - - synchronized (uploadBytesPerSecMutex) { - this.uploadBytesPerSecMovingAverageReference.set(this.uploadBytesPerSecMovingAverageReference.get().copyWithSize(updatedSize)); - } - - synchronized (uploadTimeMsMutex) { - this.uploadTimeMsMovingAverageReference.set(this.uploadTimeMsMovingAverageReference.get().copyWithSize(updatedSize)); - } - } - - boolean isUploadBytesPerSecAverageReady() { - return uploadBytesPerSecMovingAverageReference.get().isReady(); - } - - double getUploadBytesPerSecAverage() { - return uploadBytesPerSecMovingAverageReference.get().getAverage(); - } - - public void addUploadBytesPerSec(long bytesPerSec) { - synchronized (uploadBytesPerSecMutex) { - this.uploadBytesPerSecMovingAverageReference.get().record(bytesPerSec); - } - } - - boolean isUploadTimeMsAverageReady() { - return uploadTimeMsMovingAverageReference.get().isReady(); - } - - double getUploadTimeMsAverage() { - return uploadTimeMsMovingAverageReference.get().getAverage(); - } - - public void addTimeForCompletedUploadSync(long timeMs) { - synchronized (uploadTimeMsMutex) { - this.uploadTimeMsMovingAverageReference.get().record(timeMs); - } - } - - public void addTotalUploadTimeInMs(long fileUploadTimeInMs) { - this.totalUploadTimeInMs.addAndGet(fileUploadTimeInMs); - } - - public long getTotalUploadTimeInMs() { - return totalUploadTimeInMs.get(); - } - public DirectoryFileTransferTracker getDirectoryFileTransferTracker() { return directoryFileTransferTracker; } @@ -537,9 +354,9 @@ public RemoteSegmentTransferTracker.Stats stats() { uploadBytesStarted.get(), uploadBytesSucceeded.get(), uploadBytesFailed.get(), - totalUploadsStarted, - totalUploadsSucceeded, - totalUploadsFailed, + totalUploadsStarted.get(), + totalUploadsSucceeded.get(), + totalUploadsFailed.get(), rejectionCount.get(), failures.length(), lastSuccessfulRemoteRefreshBytes, @@ -547,7 +364,7 @@ public RemoteSegmentTransferTracker.Stats stats() { uploadBytesPerSecMovingAverageReference.get().getAverage(), uploadTimeMsMovingAverageReference.get().getAverage(), getBytesLag(), - totalUploadTimeInMs.get(), + totalUploadTimeInMillis.get(), directoryFileTransferTracker.stats() ); } diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteStorePressureService.java b/server/src/main/java/org/opensearch/index/remote/RemoteStorePressureService.java index 3722be7dadbfb..2920b33921869 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteStorePressureService.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteStorePressureService.java @@ -133,18 +133,18 @@ public boolean validate(RemoteSegmentTransferTracker pressureTracker, ShardId sh if (pressureTracker.getRefreshSeqNoLag() <= 1) { return true; } - if (pressureTracker.isUploadBytesAverageReady() == false) { + if (pressureTracker.isUploadBytesMovingAverageReady() == false) { logger.trace("upload bytes moving average is not ready"); return true; } - double dynamicBytesLagThreshold = pressureTracker.getUploadBytesAverage() * pressureSettings.getBytesLagVarianceFactor(); + double dynamicBytesLagThreshold = pressureTracker.getUploadBytesMovingAverage() * pressureSettings.getBytesLagVarianceFactor(); long bytesLag = pressureTracker.getBytesLag(); return bytesLag <= dynamicBytesLagThreshold; } @Override public String rejectionMessage(RemoteSegmentTransferTracker pressureTracker, ShardId shardId) { - double dynamicBytesLagThreshold = pressureTracker.getUploadBytesAverage() * pressureSettings.getBytesLagVarianceFactor(); + double dynamicBytesLagThreshold = pressureTracker.getUploadBytesMovingAverage() * pressureSettings.getBytesLagVarianceFactor(); return String.format( Locale.ROOT, "rejected execution on primary shard:%s due to remote segments lagging behind local segments." @@ -179,18 +179,20 @@ public boolean validate(RemoteSegmentTransferTracker pressureTracker, ShardId sh if (pressureTracker.getRefreshSeqNoLag() <= 1) { return true; } - if (pressureTracker.isUploadTimeMsAverageReady() == false) { + if (pressureTracker.isUploadTimeMovingAverageReady() == false) { logger.trace("upload time moving average is not ready"); return true; } long timeLag = pressureTracker.getTimeMsLag(); - double dynamicTimeLagThreshold = pressureTracker.getUploadTimeMsAverage() * pressureSettings.getUploadTimeLagVarianceFactor(); + double dynamicTimeLagThreshold = pressureTracker.getUploadTimeMovingAverage() * pressureSettings + .getUploadTimeLagVarianceFactor(); return timeLag <= dynamicTimeLagThreshold; } @Override public String rejectionMessage(RemoteSegmentTransferTracker pressureTracker, ShardId shardId) { - double dynamicTimeLagThreshold = pressureTracker.getUploadTimeMsAverage() * pressureSettings.getUploadTimeLagVarianceFactor(); + double dynamicTimeLagThreshold = pressureTracker.getUploadTimeMovingAverage() * pressureSettings + .getUploadTimeLagVarianceFactor(); return String.format( Locale.ROOT, "rejected execution on primary shard:%s due to remote segments lagging behind local segments." diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteStorePressureSettings.java b/server/src/main/java/org/opensearch/index/remote/RemoteStorePressureSettings.java index af2e453f8107d..864fe24c282a2 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteStorePressureSettings.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteStorePressureSettings.java @@ -89,43 +89,43 @@ public RemoteStorePressureSettings( clusterSettings.addSettingsUpdateConsumer(MIN_CONSECUTIVE_FAILURES_LIMIT, this::setMinConsecutiveFailuresLimit); } - public boolean isRemoteRefreshSegmentPressureEnabled() { + boolean isRemoteRefreshSegmentPressureEnabled() { return remoteRefreshSegmentPressureEnabled; } - public void setRemoteRefreshSegmentPressureEnabled(boolean remoteRefreshSegmentPressureEnabled) { + private void setRemoteRefreshSegmentPressureEnabled(boolean remoteRefreshSegmentPressureEnabled) { this.remoteRefreshSegmentPressureEnabled = remoteRefreshSegmentPressureEnabled; } - public long getMinRefreshSeqNoLagLimit() { + long getMinRefreshSeqNoLagLimit() { return minRefreshSeqNoLagLimit; } - public void setMinRefreshSeqNoLagLimit(long minRefreshSeqNoLagLimit) { + private void setMinRefreshSeqNoLagLimit(long minRefreshSeqNoLagLimit) { this.minRefreshSeqNoLagLimit = minRefreshSeqNoLagLimit; } - public double getBytesLagVarianceFactor() { + double getBytesLagVarianceFactor() { return bytesLagVarianceFactor; } - public void setBytesLagVarianceFactor(double bytesLagVarianceFactor) { + private void setBytesLagVarianceFactor(double bytesLagVarianceFactor) { this.bytesLagVarianceFactor = bytesLagVarianceFactor; } - public double getUploadTimeLagVarianceFactor() { + double getUploadTimeLagVarianceFactor() { return uploadTimeLagVarianceFactor; } - public void setUploadTimeLagVarianceFactor(double uploadTimeLagVarianceFactor) { + private void setUploadTimeLagVarianceFactor(double uploadTimeLagVarianceFactor) { this.uploadTimeLagVarianceFactor = uploadTimeLagVarianceFactor; } - public int getMinConsecutiveFailuresLimit() { + int getMinConsecutiveFailuresLimit() { return minConsecutiveFailuresLimit; } - public void setMinConsecutiveFailuresLimit(int minConsecutiveFailuresLimit) { + private void setMinConsecutiveFailuresLimit(int minConsecutiveFailuresLimit) { this.minConsecutiveFailuresLimit = minConsecutiveFailuresLimit; } } diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteStoreStatsTrackerFactory.java b/server/src/main/java/org/opensearch/index/remote/RemoteStoreStatsTrackerFactory.java index 783f4195be156..6058fcc7345d2 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteStoreStatsTrackerFactory.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteStoreStatsTrackerFactory.java @@ -11,7 +11,6 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.opensearch.cluster.service.ClusterService; -import org.opensearch.common.inject.Inject; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Settings; @@ -53,7 +52,11 @@ static class Defaults { */ private final Map remoteSegmentTrackerMap = ConcurrentCollections.newConcurrentMap(); - @Inject + /** + * Keeps map of remote-backed index shards and their corresponding stats tracker. + */ + private final Map remoteTranslogTrackerMap = ConcurrentCollections.newConcurrentMap(); + public RemoteStoreStatsTrackerFactory(ClusterService clusterService, Settings settings) { ClusterSettings clusterSettings = clusterService.getClusterSettings(); @@ -72,6 +75,8 @@ public void afterIndexShardCreated(IndexShard indexShard) { new RemoteSegmentTransferTracker(shardId, indexShard.store().getDirectoryFileTransferTracker(), movingAverageWindowSize) ); logger.trace("Created RemoteSegmentTransferTracker for shardId={}", shardId); + remoteTranslogTrackerMap.put(shardId, new RemoteTranslogTransferTracker(shardId, movingAverageWindowSize)); + logger.trace("Created RemoteTranslogTransferTracker for shardId={}", shardId); } @Override @@ -80,10 +85,16 @@ public void afterIndexShardClosed(ShardId shardId, IndexShard indexShard, Settin if (remoteSegmentTransferTracker != null) { logger.trace("Deleted RemoteSegmentTransferTracker for shardId={}", shardId); } + + RemoteTranslogTransferTracker remoteTranslogTransferTracker = remoteTranslogTrackerMap.remove(shardId); + if (remoteTranslogTransferTracker != null) { + logger.trace("Deleted RemoteTranslogTransferTracker for shardId={}", shardId); + } } - void updateMovingAverageWindowSize(int updatedSize) { + private void updateMovingAverageWindowSize(int updatedSize) { remoteSegmentTrackerMap.values().forEach(tracker -> tracker.updateMovingAverageWindowSize(updatedSize)); + remoteTranslogTrackerMap.values().forEach(tracker -> tracker.updateMovingAverageWindowSize(updatedSize)); // Update movingAverageWindowSize only if the trackers were successfully updated movingAverageWindowSize = updatedSize; @@ -93,6 +104,10 @@ public RemoteSegmentTransferTracker getRemoteSegmentTransferTracker(ShardId shar return remoteSegmentTrackerMap.get(shardId); } + public RemoteTranslogTransferTracker getRemoteTranslogTransferTracker(ShardId shardId) { + return remoteTranslogTrackerMap.get(shardId); + } + // visible for testing int getMovingAverageWindowSize() { return movingAverageWindowSize; diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteTransferTracker.java b/server/src/main/java/org/opensearch/index/remote/RemoteTransferTracker.java new file mode 100644 index 0000000000000..cbae4931b7001 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/remote/RemoteTransferTracker.java @@ -0,0 +1,269 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.index.remote; + +import org.opensearch.common.util.MovingAverage; +import org.opensearch.core.index.shard.ShardId; + +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; + +/** + * Base class for remote store stats trackers + * + * @opensearch.internal + */ +public abstract class RemoteTransferTracker { + /** + * The shard that this tracker is associated with + */ + protected final ShardId shardId; + + /** + * Total time spent on Remote Store uploads. + */ + protected final AtomicLong totalUploadTimeInMillis; + + /** + * Total number of Remote Store uploads that have been started. + */ + protected final AtomicLong totalUploadsStarted; + + /** + * Total number of Remote Store uploads that have failed. + */ + protected final AtomicLong totalUploadsFailed; + + /** + * Total number of Remote Store that have been successful. + */ + protected final AtomicLong totalUploadsSucceeded; + + /** + * Total number of byte uploads to Remote Store that have been started. + */ + protected final AtomicLong uploadBytesStarted; + + /** + * Total number of byte uploads to Remote Store that have failed. + */ + protected final AtomicLong uploadBytesFailed; + + /** + * Total number of byte uploads to Remote Store that have been successful. + */ + protected final AtomicLong uploadBytesSucceeded; + + /** + * Provides moving average over the last N total size in bytes of files uploaded as part of Remote Store upload. + * N is window size. Wrapped with {@code AtomicReference} for dynamic changes in window size. + */ + protected final AtomicReference uploadBytesMovingAverageReference; + + /** + * This lock object is used for making sure we do not miss any data. + */ + protected final Object uploadBytesMutex; + + /** + * Provides moving average over the last N upload speed (in bytes/s) of files uploaded as part of Remote Store upload. + * N is window size. Wrapped with {@code AtomicReference} for dynamic changes in window size. + */ + protected final AtomicReference uploadBytesPerSecMovingAverageReference; + + /** + * This lock object is used for making sure we do not miss any data. + */ + protected final Object uploadBytesPerSecMutex; + + /** + * Provides moving average over the last N overall upload time (in nanos) as part of Remote Store upload. N is window size. + * Wrapped with {@code AtomicReference} for dynamic changes in window size. + */ + protected final AtomicReference uploadTimeMsMovingAverageReference; + + /** + * This lock object is used for making sure we do not miss any data. + */ + protected final Object uploadTimeMsMutex; + + public RemoteTransferTracker(ShardId shardId, int movingAverageWindowSize) { + this.shardId = shardId; + totalUploadTimeInMillis = new AtomicLong(0); + totalUploadsStarted = new AtomicLong(0); + totalUploadsFailed = new AtomicLong(0); + totalUploadsSucceeded = new AtomicLong(0); + uploadBytesStarted = new AtomicLong(0); + uploadBytesFailed = new AtomicLong(0); + uploadBytesSucceeded = new AtomicLong(0); + uploadBytesMutex = new Object(); + uploadBytesMovingAverageReference = new AtomicReference<>(new MovingAverage(movingAverageWindowSize)); + uploadBytesPerSecMutex = new Object(); + uploadBytesPerSecMovingAverageReference = new AtomicReference<>(new MovingAverage(movingAverageWindowSize)); + uploadTimeMsMutex = new Object(); + uploadTimeMsMovingAverageReference = new AtomicReference<>(new MovingAverage(movingAverageWindowSize)); + } + + ShardId getShardId() { + return shardId; + } + + public long getTotalUploadTimeInMillis() { + return totalUploadTimeInMillis.get(); + } + + public void addUploadTimeInMillis(long duration) { + totalUploadTimeInMillis.addAndGet(duration); + } + + public long getTotalUploadsStarted() { + return totalUploadsStarted.get(); + } + + public long getTotalUploadsFailed() { + return totalUploadsFailed.get(); + } + + public long getTotalUploadsSucceeded() { + return totalUploadsSucceeded.get(); + } + + public void incrementTotalUploadsStarted() { + totalUploadsStarted.addAndGet(1); + } + + public void incrementTotalUploadsFailed() { + checkTotal(totalUploadsStarted.get(), totalUploadsFailed.get(), totalUploadsSucceeded.get(), 1); + totalUploadsFailed.addAndGet(1); + } + + public void incrementTotalUploadsSucceeded() { + checkTotal(totalUploadsStarted.get(), totalUploadsFailed.get(), totalUploadsSucceeded.get(), 1); + totalUploadsSucceeded.addAndGet(1); + } + + public long getUploadBytesStarted() { + return uploadBytesStarted.get(); + } + + public long getUploadBytesFailed() { + return uploadBytesFailed.get(); + } + + public long getUploadBytesSucceeded() { + return uploadBytesSucceeded.get(); + } + + public void addUploadBytesStarted(long count) { + uploadBytesStarted.addAndGet(count); + } + + public void addUploadBytesFailed(long count) { + checkTotal(uploadBytesStarted.get(), uploadBytesFailed.get(), uploadBytesSucceeded.get(), count); + uploadBytesFailed.addAndGet(count); + } + + public void addUploadBytesSucceeded(long count) { + checkTotal(uploadBytesStarted.get(), uploadBytesFailed.get(), uploadBytesSucceeded.get(), count); + uploadBytesSucceeded.addAndGet(count); + } + + boolean isUploadBytesMovingAverageReady() { + return uploadBytesMovingAverageReference.get().isReady(); + } + + double getUploadBytesMovingAverage() { + return uploadBytesMovingAverageReference.get().getAverage(); + } + + public void updateUploadBytesMovingAverage(long count) { + updateMovingAverage(count, uploadBytesMutex, uploadBytesMovingAverageReference); + } + + boolean isUploadBytesPerSecMovingAverageReady() { + return uploadBytesPerSecMovingAverageReference.get().isReady(); + } + + double getUploadBytesPerSecMovingAverage() { + return uploadBytesPerSecMovingAverageReference.get().getAverage(); + } + + public void updateUploadBytesPerSecMovingAverage(long speed) { + updateMovingAverage(speed, uploadBytesPerSecMutex, uploadBytesPerSecMovingAverageReference); + } + + boolean isUploadTimeMovingAverageReady() { + return uploadTimeMsMovingAverageReference.get().isReady(); + } + + double getUploadTimeMovingAverage() { + return uploadTimeMsMovingAverageReference.get().getAverage(); + } + + public void updateUploadTimeMovingAverage(long duration) { + updateMovingAverage(duration, uploadTimeMsMutex, uploadTimeMsMovingAverageReference); + } + + /** + * Records a new data point for a moving average stat + * + * @param value The new data point to be added + * @param mutex The mutex to use for the update + * @param movingAverageReference The atomic reference to be updated + */ + protected void updateMovingAverage(long value, Object mutex, AtomicReference movingAverageReference) { + synchronized (mutex) { + movingAverageReference.get().record(value); + } + } + + /** + * Updates the window size for data collection. This also resets any data collected so far. + * + * @param updatedSize The updated size + */ + void updateMovingAverageWindowSize(int updatedSize) { + updateMovingAverageWindowSize(updatedSize, uploadBytesMutex, uploadBytesMovingAverageReference); + updateMovingAverageWindowSize(updatedSize, uploadBytesPerSecMutex, uploadBytesPerSecMovingAverageReference); + updateMovingAverageWindowSize(updatedSize, uploadTimeMsMutex, uploadTimeMsMovingAverageReference); + } + + /** + * Updates the window size for data collection. This also resets any data collected so far. + * + * @param updatedSize The updated size + * @param mutex The mutex to use for the update + * @param movingAverageReference The atomic reference to be updated + */ + protected void updateMovingAverageWindowSize(int updatedSize, Object mutex, AtomicReference movingAverageReference) { + synchronized (mutex) { + movingAverageReference.set(movingAverageReference.get().copyWithSize(updatedSize)); + } + } + + /** + * Validates that the sum of successful operations, failed operations, and the number of operations to add (irrespective of failed/successful) does not exceed the number of operations originally started + * @param startedCount Number of operations started + * @param failedCount Number of operations failed + * @param succeededCount Number of operations successful + * @param countToAdd Number of operations to add + */ + private void checkTotal(long startedCount, long failedCount, long succeededCount, long countToAdd) { + long delta = startedCount - (failedCount + succeededCount + countToAdd); + assert delta >= 0 : "Sum of failure count (" + + failedCount + + "), success count (" + + succeededCount + + "), and count to add (" + + countToAdd + + ") cannot exceed started count (" + + startedCount + + ")"; + } +} diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteTranslogTransferTracker.java b/server/src/main/java/org/opensearch/index/remote/RemoteTranslogTransferTracker.java new file mode 100644 index 0000000000000..1a9896540212e --- /dev/null +++ b/server/src/main/java/org/opensearch/index/remote/RemoteTranslogTransferTracker.java @@ -0,0 +1,488 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.index.remote; + +import org.opensearch.common.util.MovingAverage; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.common.io.stream.StreamOutput; +import org.opensearch.core.common.io.stream.Writeable; +import org.opensearch.core.index.shard.ShardId; + +import java.io.IOException; +import java.util.Objects; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; + +/** + * Stores Remote Translog Store-related stats for a given IndexShard. + * + * @opensearch.internal + */ +public class RemoteTranslogTransferTracker extends RemoteTransferTracker { + /** + * Epoch timestamp of the last successful Remote Translog Store upload. + */ + private final AtomicLong lastSuccessfulUploadTimestamp; + + /** + * Epoch timestamp of the last successful Remote Translog Store download. + */ + private final AtomicLong lastSuccessfulDownloadTimestamp; + + /** + * Total number of Remote Translog Store downloads that have been successful. + */ + private final AtomicLong totalDownloadsSucceeded; + + /** + * Total number of byte downloads to Remote Translog Store that have been successful. + */ + private final AtomicLong downloadBytesSucceeded; + + /** + * Total time spent on Remote Translog Store downloads. + */ + private final AtomicLong totalDownloadTimeInMillis; + + /** + * Provides moving average over the last N total size in bytes of translog files downloaded as part of Remote Translog Store download. + * N is window size. Wrapped with {@code AtomicReference} for dynamic changes in window size. + */ + private final AtomicReference downloadBytesMovingAverageReference; + + /** + * This lock object is used for making sure we do not miss any data. + */ + private final Object downloadBytesMutex; + + /** + * Provides moving average over the last N download speed (in bytes/s) of translog files downloaded as part of Remote Translog Store download. + * N is window size. Wrapped with {@code AtomicReference} for dynamic changes in window size. + */ + private final AtomicReference downloadBytesPerSecMovingAverageReference; + + /** + * This lock object is used for making sure we do not miss any data. + */ + private final Object downloadBytesPerSecMutex; + + /** + * Provides moving average over the last N overall download time (in nanos) as part of Remote Translog Store download. N is window size. + * Wrapped with {@code AtomicReference} for dynamic changes in window size. + */ + private final AtomicReference downloadTimeMsMovingAverageReference; + + /** + * This lock object is used for making sure we do not miss any data. + */ + private final Object downloadTimeMsMutex; + + public RemoteTranslogTransferTracker(ShardId shardId, int movingAverageWindowSize) { + super(shardId, movingAverageWindowSize); + + lastSuccessfulUploadTimestamp = new AtomicLong(0); + lastSuccessfulDownloadTimestamp = new AtomicLong(0); + totalDownloadsSucceeded = new AtomicLong(0); + downloadBytesSucceeded = new AtomicLong(0); + totalDownloadTimeInMillis = new AtomicLong(0); + downloadBytesMutex = new Object(); + downloadBytesMovingAverageReference = new AtomicReference<>(new MovingAverage(movingAverageWindowSize)); + downloadBytesPerSecMutex = new Object(); + downloadBytesPerSecMovingAverageReference = new AtomicReference<>(new MovingAverage(movingAverageWindowSize)); + downloadTimeMsMutex = new Object(); + downloadTimeMsMovingAverageReference = new AtomicReference<>(new MovingAverage(movingAverageWindowSize)); + } + + public long getLastSuccessfulUploadTimestamp() { + return lastSuccessfulUploadTimestamp.get(); + } + + public void setLastSuccessfulUploadTimestamp(long lastSuccessfulUploadTimestamp) { + this.lastSuccessfulUploadTimestamp.set(lastSuccessfulUploadTimestamp); + } + + /** + * Updates the window size for data collection. This also resets any data collected so far. + * + * @param updatedSize the updated size + */ + void updateMovingAverageWindowSize(int updatedSize) { + super.updateMovingAverageWindowSize(updatedSize); + updateMovingAverageWindowSize(updatedSize, downloadBytesMutex, downloadBytesMovingAverageReference); + updateMovingAverageWindowSize(updatedSize, downloadBytesPerSecMutex, downloadBytesPerSecMovingAverageReference); + updateMovingAverageWindowSize(updatedSize, downloadTimeMsMutex, downloadTimeMsMovingAverageReference); + } + + public long getTotalDownloadsSucceeded() { + return totalDownloadsSucceeded.get(); + } + + void incrementDownloadsSucceeded() { + totalDownloadsSucceeded.addAndGet(1); + } + + public long getDownloadBytesSucceeded() { + return downloadBytesSucceeded.get(); + } + + public void addDownloadBytesSucceeded(long count) { + downloadBytesSucceeded.addAndGet(count); + } + + public long getTotalDownloadTimeInMillis() { + return totalDownloadTimeInMillis.get(); + } + + public void addDownloadTimeInMillis(long duration) { + totalDownloadTimeInMillis.addAndGet(duration); + } + + public long getLastSuccessfulDownloadTimestamp() { + return lastSuccessfulDownloadTimestamp.get(); + } + + void setLastSuccessfulDownloadTimestamp(long lastSuccessfulDownloadTimestamp) { + this.lastSuccessfulDownloadTimestamp.set(lastSuccessfulDownloadTimestamp); + } + + boolean isDownloadBytesMovingAverageReady() { + return downloadBytesMovingAverageReference.get().isReady(); + } + + double getDownloadBytesMovingAverage() { + return downloadBytesMovingAverageReference.get().getAverage(); + } + + void updateDownloadBytesMovingAverage(long count) { + updateMovingAverage(count, downloadBytesMutex, downloadBytesMovingAverageReference); + } + + boolean isDownloadBytesPerSecMovingAverageReady() { + return downloadBytesPerSecMovingAverageReference.get().isReady(); + } + + double getDownloadBytesPerSecMovingAverage() { + return downloadBytesPerSecMovingAverageReference.get().getAverage(); + } + + void updateDownloadBytesPerSecMovingAverage(long speed) { + updateMovingAverage(speed, downloadBytesPerSecMutex, downloadBytesPerSecMovingAverageReference); + } + + boolean isDownloadTimeMovingAverageReady() { + return downloadTimeMsMovingAverageReference.get().isReady(); + } + + double getDownloadTimeMovingAverage() { + return downloadTimeMsMovingAverageReference.get().getAverage(); + } + + void updateDownloadTimeMovingAverage(long duration) { + updateMovingAverage(duration, downloadTimeMsMutex, downloadTimeMsMovingAverageReference); + } + + /** + * Record stats related to a download from Remote Translog Store + * @param prevDownloadBytesSucceeded Number of downloadBytesSucceeded in this tracker before the download was started + * @param prevDownloadTimeInMillis Amount of downloadTimeInMillis in this tracker before the download was started + */ + public void recordDownloadStats(long prevDownloadBytesSucceeded, long prevDownloadTimeInMillis) { + setLastSuccessfulDownloadTimestamp(System.currentTimeMillis()); + incrementDownloadsSucceeded(); + long bytesDownloaded = getDownloadBytesSucceeded() - prevDownloadBytesSucceeded; + updateDownloadBytesMovingAverage(bytesDownloaded); + long durationInMillis = getTotalDownloadTimeInMillis() - prevDownloadTimeInMillis; + updateDownloadTimeMovingAverage(durationInMillis); + if (durationInMillis > 0) { + updateDownloadBytesPerSecMovingAverage(bytesDownloaded * 1_000L / durationInMillis); + } + } + + /** + * Gets the tracker's state as seen in the stats API + * @return Stats object with the tracker's stats + */ + public RemoteTranslogTransferTracker.Stats stats() { + return new RemoteTranslogTransferTracker.Stats( + shardId, + lastSuccessfulUploadTimestamp.get(), + totalUploadsStarted.get(), + totalUploadsSucceeded.get(), + totalUploadsFailed.get(), + uploadBytesStarted.get(), + uploadBytesSucceeded.get(), + uploadBytesFailed.get(), + totalUploadTimeInMillis.get(), + uploadBytesMovingAverageReference.get().getAverage(), + uploadBytesPerSecMovingAverageReference.get().getAverage(), + uploadTimeMsMovingAverageReference.get().getAverage(), + lastSuccessfulDownloadTimestamp.get(), + totalDownloadsSucceeded.get(), + downloadBytesSucceeded.get(), + totalDownloadTimeInMillis.get(), + downloadBytesMovingAverageReference.get().getAverage(), + downloadBytesPerSecMovingAverageReference.get().getAverage(), + downloadTimeMsMovingAverageReference.get().getAverage() + ); + } + + /** + * Represents the tracker's state as seen in the stats API. + * + * @opensearch.internal + */ + public static class Stats implements Writeable { + + final ShardId shardId; + + /** + * Epoch timestamp of the last successful Remote Translog Store upload. + */ + public final long lastSuccessfulUploadTimestamp; + + /** + * Total number of Remote Translog Store uploads that have been started. + */ + public final long totalUploadsStarted; + + /** + * Total number of Remote Translog Store uploads that have failed. + */ + public final long totalUploadsFailed; + + /** + * Total number of Remote Translog Store that have been successful. + */ + public final long totalUploadsSucceeded; + + /** + * Total number of byte uploads to Remote Translog Store that have been started. + */ + public final long uploadBytesStarted; + + /** + * Total number of byte uploads to Remote Translog Store that have failed. + */ + public final long uploadBytesFailed; + + /** + * Total number of byte uploads to Remote Translog Store that have been successful. + */ + public final long uploadBytesSucceeded; + + /** + * Total time spent on Remote Translog Store uploads. + */ + public final long totalUploadTimeInMillis; + + /** + * Size of a Remote Translog Store upload in bytes. + */ + public final double uploadBytesMovingAverage; + + /** + * Speed of a Remote Translog Store upload in bytes-per-second. + */ + public final double uploadBytesPerSecMovingAverage; + + /** + * Time taken by a Remote Translog Store upload. + */ + public final double uploadTimeMovingAverage; + + /** + * Epoch timestamp of the last successful Remote Translog Store download. + */ + public final long lastSuccessfulDownloadTimestamp; + + /** + * Total number of Remote Translog Store downloads that have been successful. + */ + public final long totalDownloadsSucceeded; + + /** + * Total number of byte downloads from Remote Translog Store that have been successful. + */ + public final long downloadBytesSucceeded; + + /** + * Total time spent on Remote Translog Store downloads. + */ + public final long totalDownloadTimeInMillis; + + /** + * Size of a Remote Translog Store download in bytes. + */ + public final double downloadBytesMovingAverage; + + /** + * Speed of a Remote Translog Store download in bytes-per-second. + */ + public final double downloadBytesPerSecMovingAverage; + + /** + * Time taken by a Remote Translog Store download. + */ + public final double downloadTimeMovingAverage; + + public Stats( + ShardId shardId, + long lastSuccessfulUploadTimestamp, + long totalUploadsStarted, + long totalUploadsSucceeded, + long totalUploadsFailed, + long uploadBytesStarted, + long uploadBytesSucceeded, + long uploadBytesFailed, + long totalUploadTimeInMillis, + double uploadBytesMovingAverage, + double uploadBytesPerSecMovingAverage, + double uploadTimeMovingAverage, + long lastSuccessfulDownloadTimestamp, + long totalDownloadsSucceeded, + long downloadBytesSucceeded, + long totalDownloadTimeInMillis, + double downloadBytesMovingAverage, + double downloadBytesPerSecMovingAverage, + double downloadTimeMovingAverage + ) { + this.shardId = shardId; + + this.lastSuccessfulUploadTimestamp = lastSuccessfulUploadTimestamp; + this.totalUploadsStarted = totalUploadsStarted; + this.totalUploadsFailed = totalUploadsFailed; + this.totalUploadsSucceeded = totalUploadsSucceeded; + this.uploadBytesStarted = uploadBytesStarted; + this.uploadBytesFailed = uploadBytesFailed; + this.uploadBytesSucceeded = uploadBytesSucceeded; + this.totalUploadTimeInMillis = totalUploadTimeInMillis; + this.uploadBytesMovingAverage = uploadBytesMovingAverage; + this.uploadBytesPerSecMovingAverage = uploadBytesPerSecMovingAverage; + this.uploadTimeMovingAverage = uploadTimeMovingAverage; + + this.lastSuccessfulDownloadTimestamp = lastSuccessfulDownloadTimestamp; + this.totalDownloadsSucceeded = totalDownloadsSucceeded; + this.downloadBytesSucceeded = downloadBytesSucceeded; + this.totalDownloadTimeInMillis = totalDownloadTimeInMillis; + this.downloadBytesMovingAverage = downloadBytesMovingAverage; + this.downloadBytesPerSecMovingAverage = downloadBytesPerSecMovingAverage; + this.downloadTimeMovingAverage = downloadTimeMovingAverage; + } + + public Stats(StreamInput in) throws IOException { + this.shardId = new ShardId(in); + + this.lastSuccessfulUploadTimestamp = in.readVLong(); + this.totalUploadsStarted = in.readVLong(); + this.totalUploadsFailed = in.readVLong(); + this.totalUploadsSucceeded = in.readVLong(); + this.uploadBytesStarted = in.readVLong(); + this.uploadBytesFailed = in.readVLong(); + this.uploadBytesSucceeded = in.readVLong(); + this.totalUploadTimeInMillis = in.readVLong(); + this.uploadBytesMovingAverage = in.readDouble(); + this.uploadBytesPerSecMovingAverage = in.readDouble(); + this.uploadTimeMovingAverage = in.readDouble(); + + this.lastSuccessfulDownloadTimestamp = in.readVLong(); + this.totalDownloadsSucceeded = in.readVLong(); + this.downloadBytesSucceeded = in.readVLong(); + this.totalDownloadTimeInMillis = in.readVLong(); + this.downloadBytesMovingAverage = in.readDouble(); + this.downloadBytesPerSecMovingAverage = in.readDouble(); + this.downloadTimeMovingAverage = in.readDouble(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + shardId.writeTo(out); + + out.writeVLong(lastSuccessfulUploadTimestamp); + out.writeVLong(totalUploadsStarted); + out.writeVLong(totalUploadsFailed); + out.writeVLong(totalUploadsSucceeded); + out.writeVLong(uploadBytesStarted); + out.writeVLong(uploadBytesFailed); + out.writeVLong(uploadBytesSucceeded); + out.writeVLong(totalUploadTimeInMillis); + out.writeDouble(uploadBytesMovingAverage); + out.writeDouble(uploadBytesPerSecMovingAverage); + out.writeDouble(uploadTimeMovingAverage); + + out.writeVLong(lastSuccessfulDownloadTimestamp); + out.writeVLong(totalDownloadsSucceeded); + out.writeVLong(downloadBytesSucceeded); + out.writeVLong(totalDownloadTimeInMillis); + out.writeDouble(downloadBytesMovingAverage); + out.writeDouble(downloadBytesPerSecMovingAverage); + out.writeDouble(downloadTimeMovingAverage); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + RemoteTranslogTransferTracker.Stats other = (RemoteTranslogTransferTracker.Stats) obj; + + return this.shardId.equals(other.shardId) + && this.lastSuccessfulUploadTimestamp == other.lastSuccessfulUploadTimestamp + && this.totalUploadsStarted == other.totalUploadsStarted + && this.totalUploadsFailed == other.totalUploadsFailed + && this.totalUploadsSucceeded == other.totalUploadsSucceeded + && this.uploadBytesStarted == other.uploadBytesStarted + && this.uploadBytesFailed == other.uploadBytesFailed + && this.uploadBytesSucceeded == other.uploadBytesSucceeded + && this.totalUploadTimeInMillis == other.totalUploadTimeInMillis + && Double.compare(this.uploadBytesMovingAverage, other.uploadBytesMovingAverage) == 0 + && Double.compare(this.uploadBytesPerSecMovingAverage, other.uploadBytesPerSecMovingAverage) == 0 + && Double.compare(this.uploadTimeMovingAverage, other.uploadTimeMovingAverage) == 0 + && this.lastSuccessfulDownloadTimestamp == other.lastSuccessfulDownloadTimestamp + && this.totalDownloadsSucceeded == other.totalDownloadsSucceeded + && this.downloadBytesSucceeded == other.downloadBytesSucceeded + && this.totalDownloadTimeInMillis == other.totalDownloadTimeInMillis + && Double.compare(this.downloadBytesMovingAverage, other.downloadBytesMovingAverage) == 0 + && Double.compare(this.downloadBytesPerSecMovingAverage, other.downloadBytesPerSecMovingAverage) == 0 + && Double.compare(this.downloadTimeMovingAverage, other.downloadTimeMovingAverage) == 0; + } + + @Override + public int hashCode() { + return Objects.hash( + shardId.toString(), + lastSuccessfulUploadTimestamp, + totalUploadsStarted, + totalUploadsFailed, + totalUploadsSucceeded, + uploadBytesStarted, + uploadBytesFailed, + uploadBytesSucceeded, + totalUploadTimeInMillis, + uploadBytesMovingAverage, + uploadBytesPerSecMovingAverage, + uploadTimeMovingAverage, + lastSuccessfulDownloadTimestamp, + totalDownloadsSucceeded, + downloadBytesSucceeded, + totalDownloadTimeInMillis, + downloadBytesMovingAverage, + downloadBytesPerSecMovingAverage, + downloadTimeMovingAverage + ); + } + } + + /** + * Validates if the stats in this tracker and the stats contained in the given stats object are same or not + * @param other Stats object to compare this tracker against + * @return true if stats are same and false otherwise + */ + boolean hasSameStatsAs(RemoteTranslogTransferTracker.Stats other) { + return this.stats().equals(other); + } +} diff --git a/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java b/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java index bbb8fd1b3d08e..4e035af9bbc5e 100644 --- a/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java +++ b/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java @@ -430,9 +430,9 @@ private void updateFinalStatusInSegmentTracker(boolean uploadStatus, long bytesB long bytesUploaded = segmentTracker.getUploadBytesSucceeded() - bytesBeforeUpload; long timeTakenInMS = TimeValue.nsecToMSec(System.nanoTime() - startTimeInNS); segmentTracker.incrementTotalUploadsSucceeded(); - segmentTracker.addUploadBytes(bytesUploaded); - segmentTracker.addUploadBytesPerSec((bytesUploaded * 1_000L) / Math.max(1, timeTakenInMS)); - segmentTracker.addTimeForCompletedUploadSync(timeTakenInMS); + segmentTracker.updateUploadBytesMovingAverage(bytesUploaded); + segmentTracker.updateUploadBytesPerSecMovingAverage((bytesUploaded * 1_000L) / Math.max(1, timeTakenInMS)); + segmentTracker.updateUploadTimeMovingAverage(timeTakenInMS); } else { segmentTracker.incrementTotalUploadsFailed(); } @@ -457,14 +457,14 @@ public void onSuccess(String file) { // Track upload success segmentTracker.addUploadBytesSucceeded(segmentTracker.getLatestLocalFileNameLengthMap().get(file)); segmentTracker.addToLatestUploadedFiles(file); - segmentTracker.addTotalUploadTimeInMs(Math.max(1, System.currentTimeMillis() - uploadStartTime)); + segmentTracker.addUploadTimeInMillis(Math.max(1, System.currentTimeMillis() - uploadStartTime)); } @Override public void onFailure(String file) { // Track upload failure segmentTracker.addUploadBytesFailed(segmentTracker.getLatestLocalFileNameLengthMap().get(file)); - segmentTracker.addTotalUploadTimeInMs(Math.max(1, System.currentTimeMillis() - uploadStartTime)); + segmentTracker.addUploadTimeInMillis(Math.max(1, System.currentTimeMillis() - uploadStartTime)); } }; } diff --git a/server/src/main/java/org/opensearch/index/translog/RemoteBlobStoreInternalTranslogFactory.java b/server/src/main/java/org/opensearch/index/translog/RemoteBlobStoreInternalTranslogFactory.java index 339e16db6f360..1e2cb388e690e 100644 --- a/server/src/main/java/org/opensearch/index/translog/RemoteBlobStoreInternalTranslogFactory.java +++ b/server/src/main/java/org/opensearch/index/translog/RemoteBlobStoreInternalTranslogFactory.java @@ -8,6 +8,7 @@ package org.opensearch.index.translog; +import org.opensearch.index.remote.RemoteTranslogTransferTracker; import org.opensearch.repositories.RepositoriesService; import org.opensearch.repositories.Repository; import org.opensearch.repositories.RepositoryMissingException; @@ -31,10 +32,13 @@ public class RemoteBlobStoreInternalTranslogFactory implements TranslogFactory { private final ThreadPool threadPool; + private final RemoteTranslogTransferTracker remoteTranslogTransferTracker; + public RemoteBlobStoreInternalTranslogFactory( Supplier repositoriesServiceSupplier, ThreadPool threadPool, - String repositoryName + String repositoryName, + RemoteTranslogTransferTracker remoteTranslogTransferTracker ) { Repository repository; try { @@ -44,6 +48,7 @@ public RemoteBlobStoreInternalTranslogFactory( } this.repository = repository; this.threadPool = threadPool; + this.remoteTranslogTransferTracker = remoteTranslogTransferTracker; } @Override @@ -68,7 +73,8 @@ public Translog newTranslog( persistedSequenceNumberConsumer, blobStoreRepository, threadPool, - primaryModeSupplier + primaryModeSupplier, + remoteTranslogTransferTracker ); } diff --git a/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java b/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java index b23374a2cce3b..43b6f615e7002 100644 --- a/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java +++ b/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java @@ -17,6 +17,7 @@ import org.opensearch.common.util.io.IOUtils; import org.opensearch.core.index.shard.ShardId; import org.opensearch.core.util.FileSystemUtils; +import org.opensearch.index.remote.RemoteTranslogTransferTracker; import org.opensearch.index.translog.transfer.BlobStoreTransferService; import org.opensearch.index.translog.transfer.FileTransferTracker; import org.opensearch.index.translog.transfer.TransferSnapshot; @@ -55,6 +56,7 @@ public class RemoteFsTranslog extends Translog { private final TranslogTransferManager translogTransferManager; private final FileTransferTracker fileTransferTracker; private final BooleanSupplier primaryModeSupplier; + private final RemoteTranslogTransferTracker remoteTranslogTransferTracker; private volatile long maxRemoteTranslogGenerationUploaded; private volatile long minSeqNoToKeep; @@ -80,14 +82,22 @@ public RemoteFsTranslog( LongConsumer persistedSequenceNumberConsumer, BlobStoreRepository blobStoreRepository, ThreadPool threadPool, - BooleanSupplier primaryModeSupplier + BooleanSupplier primaryModeSupplier, + RemoteTranslogTransferTracker remoteTranslogTransferTracker ) throws IOException { super(config, translogUUID, deletionPolicy, globalCheckpointSupplier, primaryTermSupplier, persistedSequenceNumberConsumer); logger = Loggers.getLogger(getClass(), shardId); this.blobStoreRepository = blobStoreRepository; this.primaryModeSupplier = primaryModeSupplier; - fileTransferTracker = new FileTransferTracker(shardId); - this.translogTransferManager = buildTranslogTransferManager(blobStoreRepository, threadPool, shardId, fileTransferTracker); + this.remoteTranslogTransferTracker = remoteTranslogTransferTracker; + fileTransferTracker = new FileTransferTracker(shardId, remoteTranslogTransferTracker); + this.translogTransferManager = buildTranslogTransferManager( + blobStoreRepository, + threadPool, + shardId, + fileTransferTracker, + remoteTranslogTransferTracker + ); try { download(translogTransferManager, location, logger); Checkpoint checkpoint = readCheckpoint(location); @@ -124,6 +134,11 @@ public RemoteFsTranslog( } } + // visible for testing + RemoteTranslogTransferTracker getRemoteTranslogTracker() { + return remoteTranslogTransferTracker; + } + public static void download(Repository repository, ShardId shardId, ThreadPool threadPool, Path location, Logger logger) throws IOException { assert repository instanceof BlobStoreRepository : String.format( @@ -132,32 +147,44 @@ public static void download(Repository repository, ShardId shardId, ThreadPool t shardId ); BlobStoreRepository blobStoreRepository = (BlobStoreRepository) repository; - FileTransferTracker fileTransferTracker = new FileTransferTracker(shardId); + // We use a dummy stats tracker to ensure the flow doesn't break. + // TODO: To be revisited as part of https://github.com/opensearch-project/OpenSearch/issues/7567 + RemoteTranslogTransferTracker remoteTranslogTransferTracker = new RemoteTranslogTransferTracker(shardId, 1000); + FileTransferTracker fileTransferTracker = new FileTransferTracker(shardId, remoteTranslogTransferTracker); TranslogTransferManager translogTransferManager = buildTranslogTransferManager( blobStoreRepository, threadPool, shardId, - fileTransferTracker + fileTransferTracker, + remoteTranslogTransferTracker ); RemoteFsTranslog.download(translogTransferManager, location, logger); } public static void download(TranslogTransferManager translogTransferManager, Path location, Logger logger) throws IOException { logger.trace("Downloading translog files from remote"); + RemoteTranslogTransferTracker statsTracker = translogTransferManager.getRemoteTranslogTransferTracker(); + long prevDownloadBytesSucceeded = statsTracker.getDownloadBytesSucceeded(); + long prevDownloadTimeInMillis = statsTracker.getTotalDownloadTimeInMillis(); TranslogTransferMetadata translogMetadata = translogTransferManager.readMetadata(); if (translogMetadata != null) { if (Files.notExists(location)) { Files.createDirectories(location); } + // Delete translog files on local before downloading from remote for (Path file : FileSystemUtils.files(location)) { Files.delete(file); } + Map generationToPrimaryTermMapper = translogMetadata.getGenerationToPrimaryTermMapper(); for (long i = translogMetadata.getGeneration(); i >= translogMetadata.getMinTranslogGeneration(); i--) { String generation = Long.toString(i); translogTransferManager.downloadTranslog(generationToPrimaryTermMapper.get(generation), generation, location); } + + statsTracker.recordDownloadStats(prevDownloadBytesSucceeded, prevDownloadTimeInMillis); + // We copy the latest generation .ckp file to translog.ckp so that flows that depend on // existence of translog.ckp file work in the same way Files.copy( @@ -172,13 +199,15 @@ public static TranslogTransferManager buildTranslogTransferManager( BlobStoreRepository blobStoreRepository, ThreadPool threadPool, ShardId shardId, - FileTransferTracker fileTransferTracker + FileTransferTracker fileTransferTracker, + RemoteTranslogTransferTracker remoteTranslogTransferTracker ) { return new TranslogTransferManager( shardId, new BlobStoreTransferService(blobStoreRepository.blobStore(), threadPool), blobStoreRepository.basePath().add(shardId.getIndex().getUUID()).add(String.valueOf(shardId.id())).add(TRANSLOG), - fileTransferTracker + fileTransferTracker, + remoteTranslogTransferTracker ); } @@ -265,28 +294,10 @@ private boolean upload(Long primaryTerm, Long generation) throws IOException { ).build() ) { Releasable transferReleasable = Releasables.wrap(deletionPolicy.acquireTranslogGen(getMinFileGeneration())); - return translogTransferManager.transferSnapshot(transferSnapshotProvider, new TranslogTransferListener() { - @Override - - public void onUploadComplete(TransferSnapshot transferSnapshot) throws IOException { - transferReleasable.close(); - closeFilesIfNoPendingRetentionLocks(); - maxRemoteTranslogGenerationUploaded = generation; - minRemoteGenReferenced = getMinFileGeneration(); - logger.trace("uploaded translog for {} {} ", primaryTerm, generation); - } - - @Override - public void onUploadFailed(TransferSnapshot transferSnapshot, Exception ex) throws IOException { - transferReleasable.close(); - closeFilesIfNoPendingRetentionLocks(); - if (ex instanceof IOException) { - throw (IOException) ex; - } else { - throw (RuntimeException) ex; - } - } - }); + return translogTransferManager.transferSnapshot( + transferSnapshotProvider, + new RemoteFsTranslogTransferListener(transferReleasable, generation, primaryTerm) + ); } } @@ -439,12 +450,16 @@ private void deleteStaleRemotePrimaryTerms() { public static void cleanup(Repository repository, ShardId shardId, ThreadPool threadPool) throws IOException { assert repository instanceof BlobStoreRepository : "repository should be instance of BlobStoreRepository"; BlobStoreRepository blobStoreRepository = (BlobStoreRepository) repository; - FileTransferTracker fileTransferTracker = new FileTransferTracker(shardId); + // We use a dummy stats tracker to ensure the flow doesn't break. + // TODO: To be revisited as part of https://github.com/opensearch-project/OpenSearch/issues/7567 + RemoteTranslogTransferTracker remoteTranslogTransferTracker = new RemoteTranslogTransferTracker(shardId, 1000); + FileTransferTracker fileTransferTracker = new FileTransferTracker(shardId, remoteTranslogTransferTracker); TranslogTransferManager translogTransferManager = buildTranslogTransferManager( blobStoreRepository, threadPool, shardId, - fileTransferTracker + fileTransferTracker, + remoteTranslogTransferTracker ); // clean up all remote translog files translogTransferManager.deleteTranslogFiles(); @@ -459,4 +474,52 @@ protected void onDelete() { // clean up all remote translog files translogTransferManager.delete(); } + + /** + * TranslogTransferListener implementation for RemoteFsTranslog + * + * @opensearch.internal + */ + private class RemoteFsTranslogTransferListener implements TranslogTransferListener { + /** + * Releasable instance for the translog + */ + private final Releasable transferReleasable; + + /** + * Generation for the translog + */ + private final Long generation; + + /** + * Primary Term for the translog + */ + private final Long primaryTerm; + + RemoteFsTranslogTransferListener(Releasable transferReleasable, Long generation, Long primaryTerm) { + this.transferReleasable = transferReleasable; + this.generation = generation; + this.primaryTerm = primaryTerm; + } + + @Override + public void onUploadComplete(TransferSnapshot transferSnapshot) throws IOException { + transferReleasable.close(); + closeFilesIfNoPendingRetentionLocks(); + maxRemoteTranslogGenerationUploaded = generation; + minRemoteGenReferenced = getMinFileGeneration(); + logger.trace("uploaded translog for {} {} ", primaryTerm, generation); + } + + @Override + public void onUploadFailed(TransferSnapshot transferSnapshot, Exception ex) throws IOException { + transferReleasable.close(); + closeFilesIfNoPendingRetentionLocks(); + if (ex instanceof IOException) { + throw (IOException) ex; + } else { + throw (RuntimeException) ex; + } + } + } } diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/FileTransferTracker.java b/server/src/main/java/org/opensearch/index/translog/transfer/FileTransferTracker.java index 406533561a798..9c2304f809f46 100644 --- a/server/src/main/java/org/opensearch/index/translog/transfer/FileTransferTracker.java +++ b/server/src/main/java/org/opensearch/index/translog/transfer/FileTransferTracker.java @@ -9,11 +9,15 @@ package org.opensearch.index.translog.transfer; import org.opensearch.core.index.shard.ShardId; +import org.opensearch.index.remote.RemoteTranslogTransferTracker; import org.opensearch.index.translog.transfer.FileSnapshot.TransferFileSnapshot; import org.opensearch.index.translog.transfer.listener.FileTransferListener; +import java.io.IOException; +import java.util.HashMap; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; @@ -26,14 +30,43 @@ public class FileTransferTracker implements FileTransferListener { private final ConcurrentHashMap fileTransferTracker; private final ShardId shardId; + private final RemoteTranslogTransferTracker remoteTranslogTransferTracker; + private Map bytesForTlogCkpFileToUpload; + private long fileTransferStartTime = -1; - public FileTransferTracker(ShardId shardId) { + public FileTransferTracker(ShardId shardId, RemoteTranslogTransferTracker remoteTranslogTransferTracker) { this.shardId = shardId; this.fileTransferTracker = new ConcurrentHashMap<>(); + this.remoteTranslogTransferTracker = remoteTranslogTransferTracker; + } + + void recordFileTransferStartTime(long uploadStartTime) { + // Recording the start time more than once for a sync is invalid + if (fileTransferStartTime == -1) { + fileTransferStartTime = uploadStartTime; + } + } + + void recordBytesForFiles(Set toUpload) { + bytesForTlogCkpFileToUpload = new HashMap<>(); + toUpload.forEach(file -> { + try { + bytesForTlogCkpFileToUpload.put(file.getName(), file.getContentLength()); + } catch (IOException ignored) { + bytesForTlogCkpFileToUpload.put(file.getName(), 0L); + } + }); + } + + long getTotalBytesToUpload() { + return bytesForTlogCkpFileToUpload.values().stream().reduce(0L, Long::sum); } @Override public void onSuccess(TransferFileSnapshot fileSnapshot) { + long durationInMillis = (System.nanoTime() - fileTransferStartTime) / 1_000_000L; + remoteTranslogTransferTracker.addUploadTimeInMillis(durationInMillis); + remoteTranslogTransferTracker.addUploadBytesSucceeded(bytesForTlogCkpFileToUpload.get(fileSnapshot.getName())); add(fileSnapshot.getName(), TransferState.SUCCESS); } @@ -53,6 +86,9 @@ private void add(String file, TransferState targetState) { @Override public void onFailure(TransferFileSnapshot fileSnapshot, Exception e) { + long durationInMillis = (System.nanoTime() - fileTransferStartTime) / 1_000_000L; + remoteTranslogTransferTracker.addUploadTimeInMillis(durationInMillis); + remoteTranslogTransferTracker.addUploadBytesFailed(bytesForTlogCkpFileToUpload.get(fileSnapshot.getName())); add(fileSnapshot.getName(), TransferState.FAILED); } diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferManager.java b/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferManager.java index 1d42e8a546858..fd4936603671c 100644 --- a/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferManager.java +++ b/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferManager.java @@ -24,6 +24,7 @@ import org.opensearch.core.action.ActionListener; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.core.index.shard.ShardId; +import org.opensearch.index.remote.RemoteTranslogTransferTracker; import org.opensearch.index.translog.Translog; import org.opensearch.index.translog.transfer.listener.TranslogTransferListener; import org.opensearch.threadpool.ThreadPool; @@ -59,6 +60,7 @@ public class TranslogTransferManager { private final BlobPath remoteMetadataTransferPath; private final BlobPath remoteBaseTransferPath; private final FileTransferTracker fileTransferTracker; + private final RemoteTranslogTransferTracker remoteTranslogTransferTracker; private static final long TRANSFER_TIMEOUT_IN_MILLIS = 30000; @@ -76,7 +78,8 @@ public TranslogTransferManager( ShardId shardId, TransferService transferService, BlobPath remoteBaseTransferPath, - FileTransferTracker fileTransferTracker + FileTransferTracker fileTransferTracker, + RemoteTranslogTransferTracker remoteTranslogTransferTracker ) { this.shardId = shardId; this.transferService = transferService; @@ -85,6 +88,11 @@ public TranslogTransferManager( this.remoteMetadataTransferPath = remoteBaseTransferPath.add(METADATA_DIR); this.fileTransferTracker = fileTransferTracker; this.logger = Loggers.getLogger(getClass(), shardId); + this.remoteTranslogTransferTracker = remoteTranslogTransferTracker; + } + + public RemoteTranslogTransferTracker getRemoteTranslogTransferTracker() { + return remoteTranslogTransferTracker; } public ShardId getShardId() { @@ -95,14 +103,22 @@ public boolean transferSnapshot(TransferSnapshot transferSnapshot, TranslogTrans throws IOException { List exceptionList = new ArrayList<>(transferSnapshot.getTranslogTransferMetadata().getCount()); Set toUpload = new HashSet<>(transferSnapshot.getTranslogTransferMetadata().getCount()); + long metadataBytesToUpload; + long metadataUploadStartTime; + long uploadStartTime; + long prevUploadBytesSucceeded = remoteTranslogTransferTracker.getUploadBytesSucceeded(); + long prevUploadTimeInMillis = remoteTranslogTransferTracker.getTotalUploadTimeInMillis(); + try { toUpload.addAll(fileTransferTracker.exclusionFilter(transferSnapshot.getTranslogFileSnapshots())); toUpload.addAll(fileTransferTracker.exclusionFilter((transferSnapshot.getCheckpointFileSnapshots()))); if (toUpload.isEmpty()) { logger.trace("Nothing to upload for transfer"); - translogTransferListener.onUploadComplete(transferSnapshot); return true; } + + fileTransferTracker.recordBytesForFiles(toUpload); + captureStatsBeforeUpload(); final CountDownLatch latch = new CountDownLatch(toUpload.size()); LatchedActionListener latchedActionListener = new LatchedActionListener<>( ActionListener.wrap(fileTransferTracker::onSuccess, ex -> { @@ -115,7 +131,8 @@ public boolean transferSnapshot(TransferSnapshot transferSnapshot, TranslogTrans ex ); FileTransferException e = (FileTransferException) ex; - fileTransferTracker.onFailure(e.getFileSnapshot(), ex); + TransferFileSnapshot file = e.getFileSnapshot(); + fileTransferTracker.onFailure(file, ex); exceptionList.add(ex); }), latch @@ -128,6 +145,10 @@ public boolean transferSnapshot(TransferSnapshot transferSnapshot, TranslogTrans ) ); + uploadStartTime = System.nanoTime(); + // TODO: Ideally each file's upload start time should be when it is actually picked for upload + // https://github.com/opensearch-project/OpenSearch/issues/9729 + fileTransferTracker.recordFileTransferStartTime(uploadStartTime); transferService.uploadBlobs(toUpload, blobPathMap, latchedActionListener, WritePriority.HIGH); try { @@ -142,7 +163,22 @@ public boolean transferSnapshot(TransferSnapshot transferSnapshot, TranslogTrans throw ex; } if (exceptionList.isEmpty()) { - transferService.uploadBlob(prepareMetadata(transferSnapshot), remoteMetadataTransferPath, WritePriority.HIGH); + TransferFileSnapshot tlogMetadata = prepareMetadata(transferSnapshot); + metadataBytesToUpload = tlogMetadata.getContentLength(); + remoteTranslogTransferTracker.addUploadBytesStarted(metadataBytesToUpload); + metadataUploadStartTime = System.nanoTime(); + try { + transferService.uploadBlob(tlogMetadata, remoteMetadataTransferPath, WritePriority.HIGH); + } catch (Exception exception) { + remoteTranslogTransferTracker.addUploadTimeInMillis((System.nanoTime() - metadataUploadStartTime) / 1_000_000L); + remoteTranslogTransferTracker.addUploadBytesFailed(metadataBytesToUpload); + // outer catch handles capturing stats on upload failure + throw exception; + } + + remoteTranslogTransferTracker.addUploadTimeInMillis((System.nanoTime() - metadataUploadStartTime) / 1_000_000L); + remoteTranslogTransferTracker.addUploadBytesSucceeded(metadataBytesToUpload); + captureStatsOnUploadSuccess(prevUploadBytesSucceeded, prevUploadTimeInMillis); translogTransferListener.onUploadComplete(transferSnapshot); return true; } else { @@ -152,11 +188,44 @@ public boolean transferSnapshot(TransferSnapshot transferSnapshot, TranslogTrans } } catch (Exception ex) { logger.error(() -> new ParameterizedMessage("Transfer failed for snapshot {}", transferSnapshot), ex); + captureStatsOnUploadFailure(); translogTransferListener.onUploadFailed(transferSnapshot, ex); return false; } } + /** + * Adds relevant stats to the tracker when an upload is started + */ + private void captureStatsBeforeUpload() { + remoteTranslogTransferTracker.incrementTotalUploadsStarted(); + // TODO: Ideally each file's byte uploads started should be when it is actually picked for upload + // https://github.com/opensearch-project/OpenSearch/issues/9729 + remoteTranslogTransferTracker.addUploadBytesStarted(fileTransferTracker.getTotalBytesToUpload()); + } + + /** + * Adds relevant stats to the tracker when an upload is successfully completed + */ + private void captureStatsOnUploadSuccess(long prevUploadBytesSucceeded, long prevUploadTimeInMillis) { + remoteTranslogTransferTracker.setLastSuccessfulUploadTimestamp(System.currentTimeMillis()); + remoteTranslogTransferTracker.incrementTotalUploadsSucceeded(); + long totalUploadedBytes = remoteTranslogTransferTracker.getUploadBytesSucceeded() - prevUploadBytesSucceeded; + remoteTranslogTransferTracker.updateUploadBytesMovingAverage(totalUploadedBytes); + long uploadDurationInMillis = remoteTranslogTransferTracker.getTotalUploadTimeInMillis() - prevUploadTimeInMillis; + remoteTranslogTransferTracker.updateUploadTimeMovingAverage(uploadDurationInMillis); + if (uploadDurationInMillis > 0) { + remoteTranslogTransferTracker.updateUploadBytesPerSecMovingAverage((totalUploadedBytes * 1_000L) / uploadDurationInMillis); + } + } + + /** + * Adds relevant stats to the tracker when an upload has failed + */ + private void captureStatsOnUploadFailure() { + remoteTranslogTransferTracker.incrementTotalUploadsFailed(); + } + public boolean downloadTranslog(String primaryTerm, String generation, Path location) throws IOException { logger.info( "Downloading translog files with: Primary Term = {}, Generation = {}, Location = {}", @@ -180,9 +249,21 @@ private void downloadToFS(String fileName, Path location, String primaryTerm) th if (Files.exists(filePath)) { Files.delete(filePath); } + + boolean downloadStatus = false; + long bytesToRead = 0, downloadStartTime = System.nanoTime(); try (InputStream inputStream = transferService.downloadBlob(remoteDataTransferPath.add(primaryTerm), fileName)) { + // Capture number of bytes for stats before reading + bytesToRead = inputStream.available(); Files.copy(inputStream, filePath); + downloadStatus = true; + } finally { + remoteTranslogTransferTracker.addDownloadTimeInMillis((System.nanoTime() - downloadStartTime) / 1_000_000L); + if (downloadStatus) { + remoteTranslogTransferTracker.addDownloadBytesSucceeded(bytesToRead); + } } + // Mark in FileTransferTracker so that the same files are not uploaded at the time of translog sync fileTransferTracker.add(fileName, true); } @@ -195,12 +276,22 @@ public TranslogTransferMetadata readMetadata() throws IOException { ActionListener.wrap(blobMetadataList -> { if (blobMetadataList.isEmpty()) return; String filename = blobMetadataList.get(0).name(); + boolean downloadStatus = false; + long downloadStartTime = System.nanoTime(), bytesToRead = 0; try (InputStream inputStream = transferService.downloadBlob(remoteMetadataTransferPath, filename)) { + // Capture number of bytes for stats before reading + bytesToRead = inputStream.available(); IndexInput indexInput = new ByteArrayIndexInput("metadata file", inputStream.readAllBytes()); metadataSetOnce.set(metadataStreamWrapper.readStream(indexInput)); + downloadStatus = true; } catch (IOException e) { logger.error(() -> new ParameterizedMessage("Exception while reading metadata file: {}", filename), e); exceptionSetOnce.set(e); + } finally { + remoteTranslogTransferTracker.addDownloadTimeInMillis((System.nanoTime() - downloadStartTime) / 1_000_000L); + if (downloadStatus) { + remoteTranslogTransferTracker.addDownloadBytesSucceeded(bytesToRead); + } } }, e -> { logger.error(() -> new ParameterizedMessage("Exception while listing metadata files"), e); diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/listener/TranslogTransferListener.java b/server/src/main/java/org/opensearch/index/translog/transfer/listener/TranslogTransferListener.java index c09fd8798e505..132d1adf916da 100644 --- a/server/src/main/java/org/opensearch/index/translog/transfer/listener/TranslogTransferListener.java +++ b/server/src/main/java/org/opensearch/index/translog/transfer/listener/TranslogTransferListener.java @@ -18,7 +18,6 @@ * @opensearch.internal */ public interface TranslogTransferListener { - /** * Invoked when the transfer of {@link TransferSnapshot} succeeds * @param transferSnapshot the transfer snapshot diff --git a/server/src/main/java/org/opensearch/indices/IndicesModule.java b/server/src/main/java/org/opensearch/indices/IndicesModule.java index 72b9af934d0ae..008386feaa6f9 100644 --- a/server/src/main/java/org/opensearch/indices/IndicesModule.java +++ b/server/src/main/java/org/opensearch/indices/IndicesModule.java @@ -71,7 +71,6 @@ import org.opensearch.index.mapper.TextFieldMapper; import org.opensearch.index.mapper.VersionFieldMapper; import org.opensearch.index.remote.RemoteStorePressureService; -import org.opensearch.index.remote.RemoteStoreStatsTrackerFactory; import org.opensearch.index.seqno.GlobalCheckpointSyncAction; import org.opensearch.index.seqno.RetentionLeaseBackgroundSyncAction; import org.opensearch.index.seqno.RetentionLeaseSyncAction; @@ -292,7 +291,6 @@ protected void configure() { bind(SegmentReplicationCheckpointPublisher.class).asEagerSingleton(); bind(SegmentReplicationPressureService.class).asEagerSingleton(); if (FeatureFlags.isEnabled(FeatureFlags.REMOTE_STORE)) { - bind(RemoteStoreStatsTrackerFactory.class).asEagerSingleton(); bind(RemoteStorePressureService.class).asEagerSingleton(); } } diff --git a/server/src/main/java/org/opensearch/indices/IndicesService.java b/server/src/main/java/org/opensearch/indices/IndicesService.java index c31acd5a0f966..8defaef1c844b 100644 --- a/server/src/main/java/org/opensearch/indices/IndicesService.java +++ b/server/src/main/java/org/opensearch/indices/IndicesService.java @@ -393,7 +393,8 @@ public IndicesService( Map recoveryStateFactories, IndexStorePlugin.DirectoryFactory remoteDirectoryFactory, Supplier repositoriesServiceSupplier, - FileCacheCleaner fileCacheCleaner + FileCacheCleaner fileCacheCleaner, + @Nullable RemoteStoreStatsTrackerFactory remoteStoreStatsTrackerFactory ) { this.settings = settings; this.threadPool = threadPool; @@ -482,7 +483,7 @@ protected void closeInternal() { this.allowExpensiveQueries = ALLOW_EXPENSIVE_QUERIES.get(clusterService.getSettings()); clusterService.getClusterSettings().addSettingsUpdateConsumer(ALLOW_EXPENSIVE_QUERIES, this::setAllowExpensiveQueries); this.remoteDirectoryFactory = remoteDirectoryFactory; - this.translogFactorySupplier = getTranslogFactorySupplier(repositoriesServiceSupplier, threadPool); + this.translogFactorySupplier = getTranslogFactorySupplier(repositoriesServiceSupplier, threadPool, remoteStoreStatsTrackerFactory); this.clusterDefaultRefreshInterval = CLUSTER_DEFAULT_INDEX_REFRESH_INTERVAL_SETTING.get(clusterService.getSettings()); clusterService.getClusterSettings() .addSettingsUpdateConsumer(CLUSTER_DEFAULT_INDEX_REFRESH_INTERVAL_SETTING, this::onRefreshIntervalUpdate); @@ -512,14 +513,16 @@ private void onRefreshIntervalUpdate(TimeValue clusterDefaultRefreshInterval) { private static BiFunction getTranslogFactorySupplier( Supplier repositoriesServiceSupplier, - ThreadPool threadPool + ThreadPool threadPool, + RemoteStoreStatsTrackerFactory remoteStoreStatsTrackerFactory ) { return (indexSettings, shardRouting) -> { if (indexSettings.isRemoteTranslogStoreEnabled() && shardRouting.primary()) { return new RemoteBlobStoreInternalTranslogFactory( repositoriesServiceSupplier, threadPool, - indexSettings.getRemoteStoreTranslogRepository() + indexSettings.getRemoteStoreTranslogRepository(), + remoteStoreStatsTrackerFactory.getRemoteTranslogTransferTracker(shardRouting.shardId()) ); } return new InternalTranslogFactory(); diff --git a/server/src/main/java/org/opensearch/node/Node.java b/server/src/main/java/org/opensearch/node/Node.java index 46f023910427c..fdf8b616ccb6c 100644 --- a/server/src/main/java/org/opensearch/node/Node.java +++ b/server/src/main/java/org/opensearch/node/Node.java @@ -138,6 +138,7 @@ import org.opensearch.index.analysis.AnalysisRegistry; import org.opensearch.index.engine.EngineFactory; import org.opensearch.index.recovery.RemoteStoreRestoreService; +import org.opensearch.index.remote.RemoteStoreStatsTrackerFactory; import org.opensearch.index.store.RemoteSegmentStoreDirectoryFactory; import org.opensearch.index.store.remote.filecache.FileCache; import org.opensearch.index.store.remote.filecache.FileCacheCleaner; @@ -389,6 +390,7 @@ public static class DiscoverySettings { final NamedWriteableRegistry namedWriteableRegistry; private final AtomicReference runnableTaskListener; private FileCache fileCache; + private final RemoteStoreStatsTrackerFactory remoteStoreStatsTrackerFactory; public Node(Environment environment) { this(environment, Collections.emptyList(), true); @@ -739,6 +741,7 @@ protected Node( threadPool ); + remoteStoreStatsTrackerFactory = new RemoteStoreStatsTrackerFactory(clusterService, settings); final IndicesService indicesService = new IndicesService( settings, pluginsService, @@ -762,7 +765,8 @@ protected Node( recoveryStateFactories, remoteDirectoryFactory, repositoriesServiceReference::get, - fileCacheCleaner + fileCacheCleaner, + remoteStoreStatsTrackerFactory ); final AliasValidator aliasValidator = new AliasValidator(); @@ -1120,6 +1124,7 @@ protected Node( b.bind(MetaStateService.class).toInstance(metaStateService); b.bind(PersistedClusterStateService.class).toInstance(lucenePersistedStateFactory); b.bind(IndicesService.class).toInstance(indicesService); + b.bind(RemoteStoreStatsTrackerFactory.class).toInstance(remoteStoreStatsTrackerFactory); b.bind(AliasValidator.class).toInstance(aliasValidator); b.bind(MetadataCreateIndexService.class).toInstance(metadataCreateIndexService); b.bind(AwarenessReplicaBalance.class).toInstance(awarenessReplicaBalance); diff --git a/server/src/test/java/org/opensearch/action/admin/cluster/remotestore/stats/RemoteStoreStatsResponseTests.java b/server/src/test/java/org/opensearch/action/admin/cluster/remotestore/stats/RemoteStoreStatsResponseTests.java index a4787a9beda8b..8cc0982c86233 100644 --- a/server/src/test/java/org/opensearch/action/admin/cluster/remotestore/stats/RemoteStoreStatsResponseTests.java +++ b/server/src/test/java/org/opensearch/action/admin/cluster/remotestore/stats/RemoteStoreStatsResponseTests.java @@ -16,6 +16,7 @@ import org.opensearch.core.index.shard.ShardId; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.index.remote.RemoteSegmentTransferTracker; +import org.opensearch.index.remote.RemoteTranslogTransferTracker; import org.opensearch.test.OpenSearchTestCase; import org.opensearch.threadpool.TestThreadPool; import org.opensearch.threadpool.ThreadPool; @@ -24,10 +25,12 @@ import java.util.Map; import static org.opensearch.action.admin.cluster.remotestore.stats.RemoteStoreStatsTestHelper.compareStatsResponse; +import static org.opensearch.action.admin.cluster.remotestore.stats.RemoteStoreStatsTestHelper.createEmptyTranslogStats; import static org.opensearch.action.admin.cluster.remotestore.stats.RemoteStoreStatsTestHelper.createShardRouting; import static org.opensearch.action.admin.cluster.remotestore.stats.RemoteStoreStatsTestHelper.createStatsForNewPrimary; import static org.opensearch.action.admin.cluster.remotestore.stats.RemoteStoreStatsTestHelper.createStatsForNewReplica; import static org.opensearch.action.admin.cluster.remotestore.stats.RemoteStoreStatsTestHelper.createStatsForRemoteStoreRestoredPrimary; +import static org.opensearch.action.admin.cluster.remotestore.stats.RemoteStoreStatsTestHelper.createTranslogStats; import static org.opensearch.core.xcontent.ToXContent.EMPTY_PARAMS; public class RemoteStoreStatsResponseTests extends OpenSearchTestCase { @@ -48,9 +51,10 @@ public void tearDown() throws Exception { } public void testSerializationForPrimary() throws Exception { - RemoteSegmentTransferTracker.Stats mockPrimaryTrackerStats = createStatsForNewPrimary(shardId); + RemoteSegmentTransferTracker.Stats mockSegmentTrackerStats = createStatsForNewPrimary(shardId); + RemoteTranslogTransferTracker.Stats mockTranslogTrackerStats = createTranslogStats(shardId); ShardRouting primaryShardRouting = createShardRouting(shardId, true); - RemoteStoreStats primaryShardStats = new RemoteStoreStats(mockPrimaryTrackerStats, primaryShardRouting); + RemoteStoreStats primaryShardStats = new RemoteStoreStats(mockSegmentTrackerStats, mockTranslogTrackerStats, primaryShardRouting); RemoteStoreStatsResponse statsResponse = new RemoteStoreStatsResponse( new RemoteStoreStats[] { primaryShardStats }, 1, @@ -73,16 +77,26 @@ public void testSerializationForPrimary() throws Exception { ArrayList> perShardNumberObject = (ArrayList>) shardsObject.get("0"); assertEquals(perShardNumberObject.size(), 1); Map perShardCopyObject = perShardNumberObject.get(0); - compareStatsResponse(perShardCopyObject, mockPrimaryTrackerStats, primaryShardRouting); + compareStatsResponse(perShardCopyObject, mockSegmentTrackerStats, mockTranslogTrackerStats, primaryShardRouting); } public void testSerializationForBothPrimaryAndReplica() throws Exception { - RemoteSegmentTransferTracker.Stats mockPrimaryTrackerStats = createStatsForNewPrimary(shardId); - RemoteSegmentTransferTracker.Stats mockReplicaTrackerStats = createStatsForNewReplica(shardId); + RemoteSegmentTransferTracker.Stats mockPrimarySegmentTrackerStats = createStatsForNewPrimary(shardId); + RemoteSegmentTransferTracker.Stats mockReplicaSegmentTrackerStats = createStatsForNewReplica(shardId); + RemoteTranslogTransferTracker.Stats mockPrimaryTranslogTrackerStats = createTranslogStats(shardId); + RemoteTranslogTransferTracker.Stats mockReplicaTranslogTrackerStats = createEmptyTranslogStats(shardId); ShardRouting primaryShardRouting = createShardRouting(shardId, true); ShardRouting replicaShardRouting = createShardRouting(shardId, false); - RemoteStoreStats primaryShardStats = new RemoteStoreStats(mockPrimaryTrackerStats, primaryShardRouting); - RemoteStoreStats replicaShardStats = new RemoteStoreStats(mockReplicaTrackerStats, replicaShardRouting); + RemoteStoreStats primaryShardStats = new RemoteStoreStats( + mockPrimarySegmentTrackerStats, + mockPrimaryTranslogTrackerStats, + primaryShardRouting + ); + RemoteStoreStats replicaShardStats = new RemoteStoreStats( + mockReplicaSegmentTrackerStats, + mockReplicaTranslogTrackerStats, + replicaShardRouting + ); RemoteStoreStatsResponse statsResponse = new RemoteStoreStatsResponse( new RemoteStoreStats[] { primaryShardStats, replicaShardStats }, 2, @@ -109,20 +123,30 @@ public void testSerializationForBothPrimaryAndReplica() throws Exception { RemoteStoreStats.RoutingFields.PRIMARY ); if (isPrimary) { - compareStatsResponse(shardObject, mockPrimaryTrackerStats, primaryShardRouting); + compareStatsResponse(shardObject, mockPrimarySegmentTrackerStats, mockPrimaryTranslogTrackerStats, primaryShardRouting); } else { - compareStatsResponse(shardObject, mockReplicaTrackerStats, replicaShardRouting); + compareStatsResponse(shardObject, mockReplicaSegmentTrackerStats, mockReplicaTranslogTrackerStats, replicaShardRouting); } }); } public void testSerializationForBothRemoteStoreRestoredPrimaryAndReplica() throws Exception { - RemoteSegmentTransferTracker.Stats mockPrimaryTrackerStats = createStatsForRemoteStoreRestoredPrimary(shardId); - RemoteSegmentTransferTracker.Stats mockReplicaTrackerStats = createStatsForNewReplica(shardId); + RemoteSegmentTransferTracker.Stats mockPrimarySegmentTrackerStats = createStatsForRemoteStoreRestoredPrimary(shardId); + RemoteSegmentTransferTracker.Stats mockReplicaSegmentTrackerStats = createStatsForNewReplica(shardId); + RemoteTranslogTransferTracker.Stats mockPrimaryTranslogTrackerStats = createTranslogStats(shardId); + RemoteTranslogTransferTracker.Stats mockReplicaTranslogTrackerStats = createEmptyTranslogStats(shardId); ShardRouting primaryShardRouting = createShardRouting(shardId, true); ShardRouting replicaShardRouting = createShardRouting(shardId, false); - RemoteStoreStats primaryShardStats = new RemoteStoreStats(mockPrimaryTrackerStats, primaryShardRouting); - RemoteStoreStats replicaShardStats = new RemoteStoreStats(mockReplicaTrackerStats, replicaShardRouting); + RemoteStoreStats primaryShardStats = new RemoteStoreStats( + mockPrimarySegmentTrackerStats, + mockPrimaryTranslogTrackerStats, + primaryShardRouting + ); + RemoteStoreStats replicaShardStats = new RemoteStoreStats( + mockReplicaSegmentTrackerStats, + mockReplicaTranslogTrackerStats, + replicaShardRouting + ); RemoteStoreStatsResponse statsResponse = new RemoteStoreStatsResponse( new RemoteStoreStats[] { primaryShardStats, replicaShardStats }, 2, @@ -149,9 +173,9 @@ public void testSerializationForBothRemoteStoreRestoredPrimaryAndReplica() throw RemoteStoreStats.RoutingFields.PRIMARY ); if (isPrimary) { - compareStatsResponse(shardObject, mockPrimaryTrackerStats, primaryShardRouting); + compareStatsResponse(shardObject, mockPrimarySegmentTrackerStats, mockPrimaryTranslogTrackerStats, primaryShardRouting); } else { - compareStatsResponse(shardObject, mockReplicaTrackerStats, replicaShardRouting); + compareStatsResponse(shardObject, mockReplicaSegmentTrackerStats, mockReplicaTranslogTrackerStats, replicaShardRouting); } }); } diff --git a/server/src/test/java/org/opensearch/action/admin/cluster/remotestore/stats/RemoteStoreStatsTestHelper.java b/server/src/test/java/org/opensearch/action/admin/cluster/remotestore/stats/RemoteStoreStatsTestHelper.java index e2a0209503976..8dfe8d768eda3 100644 --- a/server/src/test/java/org/opensearch/action/admin/cluster/remotestore/stats/RemoteStoreStatsTestHelper.java +++ b/server/src/test/java/org/opensearch/action/admin/cluster/remotestore/stats/RemoteStoreStatsTestHelper.java @@ -13,12 +13,14 @@ import org.opensearch.cluster.routing.TestShardRouting; import org.opensearch.core.index.shard.ShardId; import org.opensearch.index.remote.RemoteSegmentTransferTracker; +import org.opensearch.index.remote.RemoteTranslogTransferTracker; import org.opensearch.index.store.DirectoryFileTransferTracker; import java.util.Map; import static org.opensearch.test.OpenSearchTestCase.assertEquals; import static org.opensearch.test.OpenSearchTestCase.randomAlphaOfLength; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; /** @@ -115,11 +117,21 @@ static ShardRouting createShardRouting(ShardId shardId, boolean isPrimary) { return TestShardRouting.newShardRouting(shardId, randomAlphaOfLength(4), isPrimary, ShardRoutingState.STARTED); } + static RemoteTranslogTransferTracker.Stats createTranslogStats(ShardId shardId) { + return new RemoteTranslogTransferTracker.Stats(shardId, 1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9D, 10D, 11D, 1L, 2L, 3L, 4L, 9D, 10D, 11D); + } + + static RemoteTranslogTransferTracker.Stats createEmptyTranslogStats(ShardId shardId) { + return new RemoteTranslogTransferTracker.Stats(shardId, 0L, 0L, 0L, 0L, 0L, 0L, 0L, 0L, 0D, 0D, 0D, 0L, 0L, 0L, 0L, 0D, 0D, 0D); + } + static void compareStatsResponse( Map statsObject, - RemoteSegmentTransferTracker.Stats segmentStatsTracker, + RemoteSegmentTransferTracker.Stats segmentTransferStats, + RemoteTranslogTransferTracker.Stats translogTransferStats, ShardRouting routing ) { + // Compare Remote Segment Store stats assertEquals( ((Map) statsObject.get(RemoteStoreStats.Fields.ROUTING)).get(RemoteStoreStats.RoutingFields.NODE_ID), routing.currentNodeId() @@ -137,138 +149,273 @@ static void compareStatsResponse( Map segmentDownloads = ((Map) segment.get(RemoteStoreStats.SubFields.DOWNLOAD)); Map segmentUploads = ((Map) segment.get(RemoteStoreStats.SubFields.UPLOAD)); - if (segmentStatsTracker.directoryFileTransferTrackerStats.transferredBytesStarted != 0) { + if (segmentTransferStats.directoryFileTransferTrackerStats.transferredBytesStarted != 0) { assertEquals( segmentDownloads.get(RemoteStoreStats.DownloadStatsFields.LAST_SYNC_TIMESTAMP), - (int) segmentStatsTracker.directoryFileTransferTrackerStats.lastTransferTimestampMs + (int) segmentTransferStats.directoryFileTransferTrackerStats.lastTransferTimestampMs ); assertEquals( ((Map) segmentDownloads.get(RemoteStoreStats.DownloadStatsFields.TOTAL_DOWNLOADS_IN_BYTES)).get( RemoteStoreStats.SubFields.STARTED ), - (int) segmentStatsTracker.directoryFileTransferTrackerStats.transferredBytesStarted + (int) segmentTransferStats.directoryFileTransferTrackerStats.transferredBytesStarted ); assertEquals( ((Map) segmentDownloads.get(RemoteStoreStats.DownloadStatsFields.TOTAL_DOWNLOADS_IN_BYTES)).get( RemoteStoreStats.SubFields.SUCCEEDED ), - (int) segmentStatsTracker.directoryFileTransferTrackerStats.transferredBytesSucceeded + (int) segmentTransferStats.directoryFileTransferTrackerStats.transferredBytesSucceeded ); assertEquals( ((Map) segmentDownloads.get(RemoteStoreStats.DownloadStatsFields.TOTAL_DOWNLOADS_IN_BYTES)).get( RemoteStoreStats.SubFields.FAILED ), - (int) segmentStatsTracker.directoryFileTransferTrackerStats.transferredBytesFailed + (int) segmentTransferStats.directoryFileTransferTrackerStats.transferredBytesFailed ); assertEquals( ((Map) segmentDownloads.get(RemoteStoreStats.DownloadStatsFields.DOWNLOAD_SIZE_IN_BYTES)).get( RemoteStoreStats.SubFields.LAST_SUCCESSFUL ), - (int) segmentStatsTracker.directoryFileTransferTrackerStats.lastSuccessfulTransferInBytes + (int) segmentTransferStats.directoryFileTransferTrackerStats.lastSuccessfulTransferInBytes ); assertEquals( ((Map) segmentDownloads.get(RemoteStoreStats.DownloadStatsFields.DOWNLOAD_SIZE_IN_BYTES)).get( RemoteStoreStats.SubFields.MOVING_AVG ), - segmentStatsTracker.directoryFileTransferTrackerStats.transferredBytesMovingAverage + segmentTransferStats.directoryFileTransferTrackerStats.transferredBytesMovingAverage ); assertEquals( ((Map) segmentDownloads.get(RemoteStoreStats.DownloadStatsFields.DOWNLOAD_SPEED_IN_BYTES_PER_SEC)).get( RemoteStoreStats.SubFields.MOVING_AVG ), - segmentStatsTracker.directoryFileTransferTrackerStats.transferredBytesPerSecMovingAverage + segmentTransferStats.directoryFileTransferTrackerStats.transferredBytesPerSecMovingAverage ); } else { assertTrue(segmentDownloads.isEmpty()); } - if (segmentStatsTracker.totalUploadsStarted != 0) { + if (segmentTransferStats.totalUploadsStarted != 0) { assertEquals( segmentUploads.get(RemoteStoreStats.UploadStatsFields.LOCAL_REFRESH_TIMESTAMP), - (int) segmentStatsTracker.localRefreshClockTimeMs + (int) segmentTransferStats.localRefreshClockTimeMs ); assertEquals( segmentUploads.get(RemoteStoreStats.UploadStatsFields.REMOTE_REFRESH_TIMESTAMP), - (int) segmentStatsTracker.remoteRefreshClockTimeMs + (int) segmentTransferStats.remoteRefreshClockTimeMs ); assertEquals( segmentUploads.get(RemoteStoreStats.UploadStatsFields.REFRESH_TIME_LAG_IN_MILLIS), - (int) segmentStatsTracker.refreshTimeLagMs + (int) segmentTransferStats.refreshTimeLagMs ); assertEquals( segmentUploads.get(RemoteStoreStats.UploadStatsFields.REFRESH_LAG), - (int) (segmentStatsTracker.localRefreshNumber - segmentStatsTracker.remoteRefreshNumber) + (int) (segmentTransferStats.localRefreshNumber - segmentTransferStats.remoteRefreshNumber) ); - assertEquals(segmentUploads.get(RemoteStoreStats.UploadStatsFields.BYTES_LAG), (int) segmentStatsTracker.bytesLag); + assertEquals(segmentUploads.get(RemoteStoreStats.UploadStatsFields.BYTES_LAG), (int) segmentTransferStats.bytesLag); assertEquals( segmentUploads.get(RemoteStoreStats.UploadStatsFields.BACKPRESSURE_REJECTION_COUNT), - (int) segmentStatsTracker.rejectionCount + (int) segmentTransferStats.rejectionCount ); assertEquals( segmentUploads.get(RemoteStoreStats.UploadStatsFields.CONSECUTIVE_FAILURE_COUNT), - (int) segmentStatsTracker.consecutiveFailuresCount + (int) segmentTransferStats.consecutiveFailuresCount ); assertEquals( ((Map) segmentUploads.get(RemoteStoreStats.UploadStatsFields.TOTAL_UPLOADS_IN_BYTES)).get( RemoteStoreStats.SubFields.STARTED ), - (int) segmentStatsTracker.uploadBytesStarted + (int) segmentTransferStats.uploadBytesStarted ); assertEquals( ((Map) segmentUploads.get(RemoteStoreStats.UploadStatsFields.TOTAL_UPLOADS_IN_BYTES)).get( RemoteStoreStats.SubFields.SUCCEEDED ), - (int) segmentStatsTracker.uploadBytesSucceeded + (int) segmentTransferStats.uploadBytesSucceeded ); assertEquals( ((Map) segmentUploads.get(RemoteStoreStats.UploadStatsFields.TOTAL_UPLOADS_IN_BYTES)).get( RemoteStoreStats.SubFields.FAILED ), - (int) segmentStatsTracker.uploadBytesFailed + (int) segmentTransferStats.uploadBytesFailed ); assertEquals( ((Map) segmentUploads.get(RemoteStoreStats.UploadStatsFields.REMOTE_REFRESH_SIZE_IN_BYTES)).get( RemoteStoreStats.SubFields.MOVING_AVG ), - segmentStatsTracker.uploadBytesMovingAverage + segmentTransferStats.uploadBytesMovingAverage ); assertEquals( ((Map) segmentUploads.get(RemoteStoreStats.UploadStatsFields.REMOTE_REFRESH_SIZE_IN_BYTES)).get( RemoteStoreStats.SubFields.LAST_SUCCESSFUL ), - (int) segmentStatsTracker.lastSuccessfulRemoteRefreshBytes + (int) segmentTransferStats.lastSuccessfulRemoteRefreshBytes ); assertEquals( - ((Map) segmentUploads.get(RemoteStoreStats.UploadStatsFields.UPLOAD_LATENCY_IN_BYTES_PER_SEC)).get( + ((Map) segmentUploads.get(RemoteStoreStats.UploadStatsFields.UPLOAD_SPEED_IN_BYTES_PER_SEC)).get( RemoteStoreStats.SubFields.MOVING_AVG ), - segmentStatsTracker.uploadBytesPerSecMovingAverage + segmentTransferStats.uploadBytesPerSecMovingAverage ); assertEquals( ((Map) segmentUploads.get(RemoteStoreStats.UploadStatsFields.TOTAL_SYNCS_TO_REMOTE)).get( RemoteStoreStats.SubFields.STARTED ), - (int) segmentStatsTracker.totalUploadsStarted + (int) segmentTransferStats.totalUploadsStarted ); assertEquals( ((Map) segmentUploads.get(RemoteStoreStats.UploadStatsFields.TOTAL_SYNCS_TO_REMOTE)).get( RemoteStoreStats.SubFields.SUCCEEDED ), - (int) segmentStatsTracker.totalUploadsSucceeded + (int) segmentTransferStats.totalUploadsSucceeded ); assertEquals( ((Map) segmentUploads.get(RemoteStoreStats.UploadStatsFields.TOTAL_SYNCS_TO_REMOTE)).get(RemoteStoreStats.SubFields.FAILED), - (int) segmentStatsTracker.totalUploadsFailed + (int) segmentTransferStats.totalUploadsFailed ); assertEquals( ((Map) segmentUploads.get(RemoteStoreStats.UploadStatsFields.REMOTE_REFRESH_LATENCY_IN_MILLIS)).get( RemoteStoreStats.SubFields.MOVING_AVG ), - segmentStatsTracker.uploadTimeMovingAverage + segmentTransferStats.uploadTimeMovingAverage ); } else { assertTrue(segmentUploads.isEmpty()); } + + // Compare Remote Translog Store stats + Map tlogStatsObj = (Map) statsObject.get(RemoteStoreStats.Fields.TRANSLOG); + Map tlogUploadStatsObj = (Map) tlogStatsObj.get(RemoteStoreStats.SubFields.UPLOAD); + if (translogTransferStats.totalUploadsStarted > 0) { + assertEquals( + translogTransferStats.lastSuccessfulUploadTimestamp, + Long.parseLong(tlogUploadStatsObj.get(RemoteStoreStats.UploadStatsFields.LAST_SUCCESSFUL_UPLOAD_TIMESTAMP).toString()) + ); + + assertEquals( + translogTransferStats.totalUploadsStarted, + Long.parseLong( + ((Map) tlogUploadStatsObj.get(RemoteStoreStats.UploadStatsFields.TOTAL_UPLOADS)).get( + RemoteStoreStats.SubFields.STARTED + ).toString() + ) + ); + assertEquals( + translogTransferStats.totalUploadsSucceeded, + Long.parseLong( + ((Map) tlogUploadStatsObj.get(RemoteStoreStats.UploadStatsFields.TOTAL_UPLOADS)).get( + RemoteStoreStats.SubFields.SUCCEEDED + ).toString() + ) + ); + assertEquals( + translogTransferStats.totalUploadsFailed, + Long.parseLong( + ((Map) tlogUploadStatsObj.get(RemoteStoreStats.UploadStatsFields.TOTAL_UPLOADS)).get( + RemoteStoreStats.SubFields.FAILED + ).toString() + ) + ); + + assertEquals( + translogTransferStats.uploadBytesStarted, + Long.parseLong( + ((Map) tlogUploadStatsObj.get(RemoteStoreStats.UploadStatsFields.TOTAL_UPLOADS_IN_BYTES)).get( + RemoteStoreStats.SubFields.STARTED + ).toString() + ) + ); + assertEquals( + translogTransferStats.uploadBytesSucceeded, + Long.parseLong( + ((Map) tlogUploadStatsObj.get(RemoteStoreStats.UploadStatsFields.TOTAL_UPLOADS_IN_BYTES)).get( + RemoteStoreStats.SubFields.SUCCEEDED + ).toString() + ) + ); + assertEquals( + translogTransferStats.uploadBytesFailed, + Long.parseLong( + ((Map) tlogUploadStatsObj.get(RemoteStoreStats.UploadStatsFields.TOTAL_UPLOADS_IN_BYTES)).get( + RemoteStoreStats.SubFields.FAILED + ).toString() + ) + ); + + assertEquals( + translogTransferStats.totalUploadTimeInMillis, + Long.parseLong(tlogUploadStatsObj.get(RemoteStoreStats.UploadStatsFields.TOTAL_UPLOAD_TIME_IN_MILLIS).toString()) + ); + + assertEquals( + translogTransferStats.uploadBytesMovingAverage, + ((Map) tlogUploadStatsObj.get(RemoteStoreStats.UploadStatsFields.UPLOAD_SIZE_IN_BYTES)).get( + RemoteStoreStats.SubFields.MOVING_AVG + ) + ); + assertEquals( + translogTransferStats.uploadBytesPerSecMovingAverage, + ((Map) tlogUploadStatsObj.get(RemoteStoreStats.UploadStatsFields.UPLOAD_SPEED_IN_BYTES_PER_SEC)).get( + RemoteStoreStats.SubFields.MOVING_AVG + ) + ); + assertEquals( + translogTransferStats.uploadTimeMovingAverage, + ((Map) tlogUploadStatsObj.get(RemoteStoreStats.UploadStatsFields.UPLOAD_TIME_IN_MILLIS)).get( + RemoteStoreStats.SubFields.MOVING_AVG + ) + ); + } else { + assertNull(tlogUploadStatsObj.get(RemoteStoreStats.UploadStatsFields.TOTAL_UPLOADS)); + } + + Map tlogDownloadStatsObj = (Map) tlogStatsObj.get(RemoteStoreStats.SubFields.DOWNLOAD); + if (translogTransferStats.totalDownloadsSucceeded > 0) { + assertEquals( + translogTransferStats.lastSuccessfulDownloadTimestamp, + Long.parseLong(tlogDownloadStatsObj.get(RemoteStoreStats.DownloadStatsFields.LAST_SUCCESSFUL_DOWNLOAD_TIMESTAMP).toString()) + ); + assertEquals( + translogTransferStats.totalDownloadsSucceeded, + Long.parseLong( + ((Map) tlogDownloadStatsObj.get(RemoteStoreStats.DownloadStatsFields.TOTAL_DOWNLOADS)).get( + RemoteStoreStats.SubFields.SUCCEEDED + ).toString() + ) + ); + assertEquals( + translogTransferStats.downloadBytesSucceeded, + Long.parseLong( + ((Map) tlogDownloadStatsObj.get(RemoteStoreStats.DownloadStatsFields.TOTAL_DOWNLOADS_IN_BYTES)).get( + RemoteStoreStats.SubFields.SUCCEEDED + ).toString() + ) + ); + assertEquals( + translogTransferStats.totalDownloadTimeInMillis, + Long.parseLong(tlogDownloadStatsObj.get(RemoteStoreStats.DownloadStatsFields.TOTAL_DOWNLOAD_TIME_IN_MILLIS).toString()) + ); + + assertEquals( + translogTransferStats.downloadBytesMovingAverage, + ((Map) tlogDownloadStatsObj.get(RemoteStoreStats.DownloadStatsFields.DOWNLOAD_SIZE_IN_BYTES)).get( + RemoteStoreStats.SubFields.MOVING_AVG + ) + ); + assertEquals( + translogTransferStats.downloadBytesPerSecMovingAverage, + ((Map) tlogDownloadStatsObj.get(RemoteStoreStats.DownloadStatsFields.DOWNLOAD_SPEED_IN_BYTES_PER_SEC)).get( + RemoteStoreStats.SubFields.MOVING_AVG + ) + ); + assertEquals( + translogTransferStats.downloadTimeMovingAverage, + ((Map) tlogDownloadStatsObj.get(RemoteStoreStats.DownloadStatsFields.DOWNLOAD_TIME_IN_MILLIS)).get( + RemoteStoreStats.SubFields.MOVING_AVG + ) + ); + } else { + assertNull(tlogDownloadStatsObj.get(RemoteStoreStats.DownloadStatsFields.TOTAL_DOWNLOADS_IN_BYTES)); + } } } diff --git a/server/src/test/java/org/opensearch/action/admin/cluster/remotestore/stats/RemoteStoreStatsTests.java b/server/src/test/java/org/opensearch/action/admin/cluster/remotestore/stats/RemoteStoreStatsTests.java index 53a3b8e26a902..1c78539a00a07 100644 --- a/server/src/test/java/org/opensearch/action/admin/cluster/remotestore/stats/RemoteStoreStatsTests.java +++ b/server/src/test/java/org/opensearch/action/admin/cluster/remotestore/stats/RemoteStoreStatsTests.java @@ -17,6 +17,7 @@ import org.opensearch.core.index.shard.ShardId; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.index.remote.RemoteSegmentTransferTracker; +import org.opensearch.index.remote.RemoteTranslogTransferTracker; import org.opensearch.test.OpenSearchTestCase; import org.opensearch.threadpool.TestThreadPool; import org.opensearch.threadpool.ThreadPool; @@ -25,10 +26,12 @@ import java.util.Map; import static org.opensearch.action.admin.cluster.remotestore.stats.RemoteStoreStatsTestHelper.compareStatsResponse; +import static org.opensearch.action.admin.cluster.remotestore.stats.RemoteStoreStatsTestHelper.createEmptyTranslogStats; import static org.opensearch.action.admin.cluster.remotestore.stats.RemoteStoreStatsTestHelper.createShardRouting; import static org.opensearch.action.admin.cluster.remotestore.stats.RemoteStoreStatsTestHelper.createStatsForNewPrimary; import static org.opensearch.action.admin.cluster.remotestore.stats.RemoteStoreStatsTestHelper.createStatsForNewReplica; import static org.opensearch.action.admin.cluster.remotestore.stats.RemoteStoreStatsTestHelper.createStatsForRemoteStoreRestoredPrimary; +import static org.opensearch.action.admin.cluster.remotestore.stats.RemoteStoreStatsTestHelper.createTranslogStats; import static org.opensearch.core.xcontent.ToXContent.EMPTY_PARAMS; public class RemoteStoreStatsTests extends OpenSearchTestCase { @@ -49,70 +52,79 @@ public void tearDown() throws Exception { } public void testXContentBuilderWithPrimaryShard() throws IOException { - RemoteSegmentTransferTracker.Stats segmentStats = createStatsForNewPrimary(shardId); + RemoteSegmentTransferTracker.Stats segmentTransferStats = createStatsForNewPrimary(shardId); + RemoteTranslogTransferTracker.Stats translogTransferStats = createTranslogStats(shardId); ShardRouting routing = createShardRouting(shardId, true); - RemoteStoreStats stats = new RemoteStoreStats(segmentStats, routing); + RemoteStoreStats stats = new RemoteStoreStats(segmentTransferStats, translogTransferStats, routing); XContentBuilder builder = XContentFactory.jsonBuilder(); stats.toXContent(builder, EMPTY_PARAMS); Map jsonObject = XContentHelper.convertToMap(BytesReference.bytes(builder), false, builder.contentType()).v2(); - compareStatsResponse(jsonObject, segmentStats, routing); + compareStatsResponse(jsonObject, segmentTransferStats, translogTransferStats, routing); } public void testXContentBuilderWithReplicaShard() throws IOException { - RemoteSegmentTransferTracker.Stats segmentStats = createStatsForNewReplica(shardId); + RemoteSegmentTransferTracker.Stats segmentTransferStats = createStatsForNewReplica(shardId); + RemoteTranslogTransferTracker.Stats translogTransferStats = createEmptyTranslogStats(shardId); ShardRouting routing = createShardRouting(shardId, false); - RemoteStoreStats stats = new RemoteStoreStats(segmentStats, routing); + RemoteStoreStats stats = new RemoteStoreStats(segmentTransferStats, translogTransferStats, routing); XContentBuilder builder = XContentFactory.jsonBuilder(); stats.toXContent(builder, EMPTY_PARAMS); Map jsonObject = XContentHelper.convertToMap(BytesReference.bytes(builder), false, builder.contentType()).v2(); - compareStatsResponse(jsonObject, segmentStats, routing); + compareStatsResponse(jsonObject, segmentTransferStats, translogTransferStats, routing); } public void testXContentBuilderWithRemoteStoreRestoredShard() throws IOException { - RemoteSegmentTransferTracker.Stats segmentStats = createStatsForRemoteStoreRestoredPrimary(shardId); + RemoteSegmentTransferTracker.Stats segmentTransferStats = createStatsForRemoteStoreRestoredPrimary(shardId); + RemoteTranslogTransferTracker.Stats translogTransferStats = createTranslogStats(shardId); ShardRouting routing = createShardRouting(shardId, true); - RemoteStoreStats stats = new RemoteStoreStats(segmentStats, routing); + RemoteStoreStats stats = new RemoteStoreStats(segmentTransferStats, translogTransferStats, routing); XContentBuilder builder = XContentFactory.jsonBuilder(); stats.toXContent(builder, EMPTY_PARAMS); Map jsonObject = XContentHelper.convertToMap(BytesReference.bytes(builder), false, builder.contentType()).v2(); - compareStatsResponse(jsonObject, segmentStats, routing); + compareStatsResponse(jsonObject, segmentTransferStats, translogTransferStats, routing); } public void testSerializationForPrimaryShard() throws Exception { - RemoteSegmentTransferTracker.Stats segmentStats = createStatsForNewPrimary(shardId); - RemoteStoreStats stats = new RemoteStoreStats(segmentStats, createShardRouting(shardId, true)); + RemoteSegmentTransferTracker.Stats segmentTransferStats = createStatsForNewPrimary(shardId); + RemoteTranslogTransferTracker.Stats translogTransferStats = createTranslogStats(shardId); + RemoteStoreStats stats = new RemoteStoreStats(segmentTransferStats, translogTransferStats, createShardRouting(shardId, true)); try (BytesStreamOutput out = new BytesStreamOutput()) { stats.writeTo(out); try (StreamInput in = out.bytes().streamInput()) { RemoteStoreStats deserializedStats = new RemoteStoreStats(in); assertEquals(stats.getSegmentStats(), deserializedStats.getSegmentStats()); + assertEquals(stats.getTranslogStats(), deserializedStats.getTranslogStats()); } } } public void testSerializationForReplicaShard() throws Exception { RemoteSegmentTransferTracker.Stats replicaShardStats = createStatsForNewReplica(shardId); - RemoteStoreStats stats = new RemoteStoreStats(replicaShardStats, createShardRouting(shardId, false)); + RemoteTranslogTransferTracker.Stats translogTransferStats = createEmptyTranslogStats(shardId); + RemoteStoreStats stats = new RemoteStoreStats(replicaShardStats, translogTransferStats, createShardRouting(shardId, false)); try (BytesStreamOutput out = new BytesStreamOutput()) { stats.writeTo(out); try (StreamInput in = out.bytes().streamInput()) { RemoteStoreStats deserializedStats = new RemoteStoreStats(in); assertEquals(stats.getSegmentStats(), deserializedStats.getSegmentStats()); + assertEquals(stats.getTranslogStats(), deserializedStats.getTranslogStats()); } } } public void testSerializationForRemoteStoreRestoredPrimaryShard() throws Exception { RemoteSegmentTransferTracker.Stats primaryShardStats = createStatsForRemoteStoreRestoredPrimary(shardId); - RemoteStoreStats stats = new RemoteStoreStats(primaryShardStats, createShardRouting(shardId, true)); + RemoteTranslogTransferTracker.Stats translogTransferStats = createTranslogStats(shardId); + RemoteStoreStats stats = new RemoteStoreStats(primaryShardStats, translogTransferStats, createShardRouting(shardId, true)); try (BytesStreamOutput out = new BytesStreamOutput()) { stats.writeTo(out); try (StreamInput in = out.bytes().streamInput()) { RemoteStoreStats deserializedStats = new RemoteStoreStats(in); assertEquals(stats.getSegmentStats(), deserializedStats.getSegmentStats()); + assertEquals(stats.getTranslogStats(), deserializedStats.getTranslogStats()); } } } diff --git a/server/src/test/java/org/opensearch/index/IndexModuleTests.java b/server/src/test/java/org/opensearch/index/IndexModuleTests.java index fadce99416cbc..99673d4b01690 100644 --- a/server/src/test/java/org/opensearch/index/IndexModuleTests.java +++ b/server/src/test/java/org/opensearch/index/IndexModuleTests.java @@ -87,6 +87,7 @@ import org.opensearch.index.fielddata.IndexFieldDataCache; import org.opensearch.index.mapper.ParsedDocument; import org.opensearch.index.mapper.Uid; +import org.opensearch.index.remote.RemoteTranslogTransferTracker; import org.opensearch.index.shard.IndexEventListener; import org.opensearch.index.shard.IndexingOperationListener; import org.opensearch.index.shard.SearchOperationListener; @@ -231,7 +232,8 @@ private IndexService newIndexService(IndexModule module) throws IOException { return new RemoteBlobStoreInternalTranslogFactory( repositoriesServiceReference::get, threadPool, - indexSettings.getRemoteStoreTranslogRepository() + indexSettings.getRemoteStoreTranslogRepository(), + new RemoteTranslogTransferTracker(shardRouting.shardId(), 10) ); } return new InternalTranslogFactory(); diff --git a/server/src/test/java/org/opensearch/index/remote/RemoteSegmentTransferTrackerTests.java b/server/src/test/java/org/opensearch/index/remote/RemoteSegmentTransferTrackerTests.java index e0a05e8d6b49e..0bf00f9e48137 100644 --- a/server/src/test/java/org/opensearch/index/remote/RemoteSegmentTransferTrackerTests.java +++ b/server/src/test/java/org/opensearch/index/remote/RemoteSegmentTransferTrackerTests.java @@ -170,9 +170,10 @@ public void testAddUploadBytesFailed() { remoteStoreStatsTrackerFactory.getMovingAverageWindowSize() ); long bytesToAdd = randomLongBetween(1000, 1000000); + long moreBytesToAdd = randomLongBetween(1000, 10000); + transferTracker.addUploadBytesStarted(bytesToAdd + moreBytesToAdd); transferTracker.addUploadBytesFailed(bytesToAdd); assertEquals(bytesToAdd, transferTracker.getUploadBytesFailed()); - long moreBytesToAdd = randomLongBetween(1000, 10000); transferTracker.addUploadBytesFailed(moreBytesToAdd); assertEquals(bytesToAdd + moreBytesToAdd, transferTracker.getUploadBytesFailed()); } @@ -184,9 +185,10 @@ public void testAddUploadBytesSucceeded() { remoteStoreStatsTrackerFactory.getMovingAverageWindowSize() ); long bytesToAdd = randomLongBetween(1000, 1000000); + long moreBytesToAdd = randomLongBetween(1000, 10000); + transferTracker.addUploadBytesStarted(bytesToAdd + moreBytesToAdd); transferTracker.addUploadBytesSucceeded(bytesToAdd); assertEquals(bytesToAdd, transferTracker.getUploadBytesSucceeded()); - long moreBytesToAdd = randomLongBetween(1000, 10000); transferTracker.addUploadBytesSucceeded(moreBytesToAdd); assertEquals(bytesToAdd + moreBytesToAdd, transferTracker.getUploadBytesSucceeded()); } @@ -266,6 +268,8 @@ public void testIncrementTotalUploadsFailed() { directoryFileTransferTracker, remoteStoreStatsTrackerFactory.getMovingAverageWindowSize() ); + transferTracker.incrementTotalUploadsStarted(); + transferTracker.incrementTotalUploadsStarted(); transferTracker.incrementTotalUploadsFailed(); assertEquals(1, transferTracker.getTotalUploadsFailed()); transferTracker.incrementTotalUploadsFailed(); @@ -278,6 +282,8 @@ public void testIncrementTotalUploadSucceeded() { directoryFileTransferTracker, remoteStoreStatsTrackerFactory.getMovingAverageWindowSize() ); + transferTracker.incrementTotalUploadsStarted(); + transferTracker.incrementTotalUploadsStarted(); transferTracker.incrementTotalUploadsSucceeded(); assertEquals(1, transferTracker.getTotalUploadsSucceeded()); transferTracker.incrementTotalUploadsSucceeded(); @@ -318,6 +324,9 @@ public void testGetConsecutiveFailureCount() { directoryFileTransferTracker, remoteStoreStatsTrackerFactory.getMovingAverageWindowSize() ); + transferTracker.incrementTotalUploadsStarted(); + transferTracker.incrementTotalUploadsStarted(); + transferTracker.incrementTotalUploadsStarted(); transferTracker.incrementTotalUploadsFailed(); assertEquals(1, transferTracker.getConsecutiveFailureCount()); transferTracker.incrementTotalUploadsFailed(); @@ -354,73 +363,73 @@ public void testComputeBytesLag() { assertEquals(0L, transferTracker.getBytesLag()); } - public void testIsUploadBytesAverageReady() { + public void testisUploadBytesMovingAverageReady() { int movingAverageWindowSize = remoteStoreStatsTrackerFactory.getMovingAverageWindowSize(); transferTracker = new RemoteSegmentTransferTracker(shardId, directoryFileTransferTracker, movingAverageWindowSize); - assertFalse(transferTracker.isUploadBytesAverageReady()); + assertFalse(transferTracker.isUploadBytesMovingAverageReady()); long sum = 0; for (int i = 1; i < movingAverageWindowSize; i++) { - transferTracker.addUploadBytes(i); + transferTracker.updateUploadBytesMovingAverage(i); sum += i; - assertFalse(transferTracker.isUploadBytesAverageReady()); - assertEquals((double) sum / i, transferTracker.getUploadBytesAverage(), 0.0d); + assertFalse(transferTracker.isUploadBytesMovingAverageReady()); + assertEquals((double) sum / i, transferTracker.getUploadBytesMovingAverage(), 0.0d); } - transferTracker.addUploadBytes(movingAverageWindowSize); + transferTracker.updateUploadBytesMovingAverage(movingAverageWindowSize); sum += movingAverageWindowSize; - assertTrue(transferTracker.isUploadBytesAverageReady()); - assertEquals((double) sum / movingAverageWindowSize, transferTracker.getUploadBytesAverage(), 0.0d); + assertTrue(transferTracker.isUploadBytesMovingAverageReady()); + assertEquals((double) sum / movingAverageWindowSize, transferTracker.getUploadBytesMovingAverage(), 0.0d); - transferTracker.addUploadBytes(100); + transferTracker.updateUploadBytesMovingAverage(100); sum = sum + 100 - 1; - assertEquals((double) sum / movingAverageWindowSize, transferTracker.getUploadBytesAverage(), 0.0d); + assertEquals((double) sum / movingAverageWindowSize, transferTracker.getUploadBytesMovingAverage(), 0.0d); } public void testIsUploadBytesPerSecAverageReady() { int movingAverageWindowSize = remoteStoreStatsTrackerFactory.getMovingAverageWindowSize(); transferTracker = new RemoteSegmentTransferTracker(shardId, directoryFileTransferTracker, movingAverageWindowSize); - assertFalse(transferTracker.isUploadBytesPerSecAverageReady()); + assertFalse(transferTracker.isUploadBytesPerSecMovingAverageReady()); long sum = 0; for (int i = 1; i < movingAverageWindowSize; i++) { - transferTracker.addUploadBytesPerSec(i); + transferTracker.updateUploadBytesPerSecMovingAverage(i); sum += i; - assertFalse(transferTracker.isUploadBytesPerSecAverageReady()); - assertEquals((double) sum / i, transferTracker.getUploadBytesPerSecAverage(), 0.0d); + assertFalse(transferTracker.isUploadBytesPerSecMovingAverageReady()); + assertEquals((double) sum / i, transferTracker.getUploadBytesPerSecMovingAverage(), 0.0d); } - transferTracker.addUploadBytesPerSec(movingAverageWindowSize); + transferTracker.updateUploadBytesPerSecMovingAverage(movingAverageWindowSize); sum += movingAverageWindowSize; - assertTrue(transferTracker.isUploadBytesPerSecAverageReady()); - assertEquals((double) sum / movingAverageWindowSize, transferTracker.getUploadBytesPerSecAverage(), 0.0d); + assertTrue(transferTracker.isUploadBytesPerSecMovingAverageReady()); + assertEquals((double) sum / movingAverageWindowSize, transferTracker.getUploadBytesPerSecMovingAverage(), 0.0d); - transferTracker.addUploadBytesPerSec(100); + transferTracker.updateUploadBytesPerSecMovingAverage(100); sum = sum + 100 - 1; - assertEquals((double) sum / movingAverageWindowSize, transferTracker.getUploadBytesPerSecAverage(), 0.0d); + assertEquals((double) sum / movingAverageWindowSize, transferTracker.getUploadBytesPerSecMovingAverage(), 0.0d); } public void testIsUploadTimeMsAverageReady() { int movingAverageWindowSize = remoteStoreStatsTrackerFactory.getMovingAverageWindowSize(); transferTracker = new RemoteSegmentTransferTracker(shardId, directoryFileTransferTracker, movingAverageWindowSize); - assertFalse(transferTracker.isUploadTimeMsAverageReady()); + assertFalse(transferTracker.isUploadTimeMovingAverageReady()); long sum = 0; for (int i = 1; i < movingAverageWindowSize; i++) { - transferTracker.addTimeForCompletedUploadSync(i); + transferTracker.updateUploadTimeMovingAverage(i); sum += i; - assertFalse(transferTracker.isUploadTimeMsAverageReady()); - assertEquals((double) sum / i, transferTracker.getUploadTimeMsAverage(), 0.0d); + assertFalse(transferTracker.isUploadTimeMovingAverageReady()); + assertEquals((double) sum / i, transferTracker.getUploadTimeMovingAverage(), 0.0d); } - transferTracker.addTimeForCompletedUploadSync(movingAverageWindowSize); + transferTracker.updateUploadTimeMovingAverage(movingAverageWindowSize); sum += movingAverageWindowSize; - assertTrue(transferTracker.isUploadTimeMsAverageReady()); - assertEquals((double) sum / movingAverageWindowSize, transferTracker.getUploadTimeMsAverage(), 0.0d); + assertTrue(transferTracker.isUploadTimeMovingAverageReady()); + assertEquals((double) sum / movingAverageWindowSize, transferTracker.getUploadTimeMovingAverage(), 0.0d); - transferTracker.addTimeForCompletedUploadSync(100); + transferTracker.updateUploadTimeMovingAverage(100); sum = sum + 100 - 1; - assertEquals((double) sum / movingAverageWindowSize, transferTracker.getUploadTimeMsAverage(), 0.0d); + assertEquals((double) sum / movingAverageWindowSize, transferTracker.getUploadTimeMovingAverage(), 0.0d); } public void testIsDownloadBytesAverageReady() { @@ -482,11 +491,11 @@ public void testAddTotalUploadTimeInMs() { remoteStoreStatsTrackerFactory.getMovingAverageWindowSize() ); long timeToAdd = randomLongBetween(100, 200); - transferTracker.addTotalUploadTimeInMs(timeToAdd); - assertEquals(timeToAdd, transferTracker.getTotalUploadTimeInMs()); + transferTracker.addUploadTimeInMillis(timeToAdd); + assertEquals(timeToAdd, transferTracker.getTotalUploadTimeInMillis()); long moreTimeToAdd = randomLongBetween(100, 200); - transferTracker.addTotalUploadTimeInMs(moreTimeToAdd); - assertEquals(timeToAdd + moreTimeToAdd, transferTracker.getTotalUploadTimeInMs()); + transferTracker.addUploadTimeInMillis(moreTimeToAdd); + assertEquals(timeToAdd + moreTimeToAdd, transferTracker.getTotalUploadTimeInMillis()); } public void testAddTotalTransferTimeMs() { @@ -519,9 +528,9 @@ public void testStatsObjectCreation() { assertEquals(transferTracker.getUploadBytesStarted(), (int) transferTrackerStats.uploadBytesStarted); assertEquals(transferTracker.getUploadBytesSucceeded(), (int) transferTrackerStats.uploadBytesSucceeded); assertEquals(transferTracker.getUploadBytesFailed(), (int) transferTrackerStats.uploadBytesFailed); - assertEquals(transferTracker.getUploadBytesAverage(), transferTrackerStats.uploadBytesMovingAverage, 0); - assertEquals(transferTracker.getUploadBytesPerSecAverage(), transferTrackerStats.uploadBytesPerSecMovingAverage, 0); - assertEquals(transferTracker.getUploadTimeMsAverage(), transferTrackerStats.uploadTimeMovingAverage, 0); + assertEquals(transferTracker.getUploadBytesMovingAverage(), transferTrackerStats.uploadBytesMovingAverage, 0); + assertEquals(transferTracker.getUploadBytesPerSecMovingAverage(), transferTrackerStats.uploadBytesPerSecMovingAverage, 0); + assertEquals(transferTracker.getUploadTimeMovingAverage(), transferTrackerStats.uploadTimeMovingAverage, 0); assertEquals(transferTracker.getTotalUploadsStarted(), (int) transferTrackerStats.totalUploadsStarted); assertEquals(transferTracker.getTotalUploadsSucceeded(), (int) transferTrackerStats.totalUploadsSucceeded); assertEquals(transferTracker.getTotalUploadsFailed(), (int) transferTrackerStats.totalUploadsFailed); @@ -580,9 +589,10 @@ private RemoteSegmentTransferTracker constructTracker() { new DirectoryFileTransferTracker(), remoteStoreStatsTrackerFactory.getMovingAverageWindowSize() ); + transferTracker.incrementTotalUploadsStarted(); transferTracker.incrementTotalUploadsFailed(); - transferTracker.addTimeForCompletedUploadSync(System.nanoTime() / 1_000_000L + randomIntBetween(10, 100)); - transferTracker.addUploadBytes(99); + transferTracker.updateUploadTimeMovingAverage(System.nanoTime() / 1_000_000L + randomIntBetween(10, 100)); + transferTracker.updateUploadBytesMovingAverage(99); transferTracker.updateRemoteRefreshTimeMs(System.nanoTime() / 1_000_000L + randomIntBetween(10, 100)); transferTracker.incrementRejectionCount(); transferTracker.getDirectoryFileTransferTracker().addTransferredBytesStarted(10); diff --git a/server/src/test/java/org/opensearch/index/remote/RemoteStorePressureServiceTests.java b/server/src/test/java/org/opensearch/index/remote/RemoteStorePressureServiceTests.java index 355333e74f826..ada8d9983aa3d 100644 --- a/server/src/test/java/org/opensearch/index/remote/RemoteStorePressureServiceTests.java +++ b/server/src/test/java/org/opensearch/index/remote/RemoteStorePressureServiceTests.java @@ -82,7 +82,7 @@ public void testValidateSegmentUploadLag() { pressureTracker.updateRemoteRefreshSeqNo(3); AtomicLong sum = new AtomicLong(); IntStream.range(0, 20).forEach(i -> { - pressureTracker.addTimeForCompletedUploadSync(i); + pressureTracker.updateUploadTimeMovingAverage(i); sum.addAndGet(i); }); double avg = (double) sum.get() / 20; @@ -99,7 +99,7 @@ public void testValidateSegmentUploadLag() { // 2. bytes lag more than dynamic threshold sum.set(0); IntStream.range(0, 20).forEach(i -> { - pressureTracker.addUploadBytes(i); + pressureTracker.updateUploadBytesMovingAverage(i); sum.addAndGet(i); }); avg = (double) sum.get() / 20; @@ -116,12 +116,15 @@ public void testValidateSegmentUploadLag() { pressureService.validateSegmentsUploadLag(shardId); // 3. Consecutive failures more than the limit + IntStream.range(0, 5).forEach(ignore -> pressureTracker.incrementTotalUploadsStarted()); IntStream.range(0, 5).forEach(ignore -> pressureTracker.incrementTotalUploadsFailed()); pressureService.validateSegmentsUploadLag(shardId); + pressureTracker.incrementTotalUploadsStarted(); pressureTracker.incrementTotalUploadsFailed(); e = assertThrows(OpenSearchRejectedExecutionException.class, () -> pressureService.validateSegmentsUploadLag(shardId)); assertTrue(e.getMessage().contains("due to remote segments lagging behind local segments")); assertTrue(e.getMessage().contains("failure_streak_count:6 min_consecutive_failure_threshold:5")); + pressureTracker.incrementTotalUploadsStarted(); pressureTracker.incrementTotalUploadsSucceeded(); pressureService.validateSegmentsUploadLag(shardId); } diff --git a/server/src/test/java/org/opensearch/index/remote/RemoteTranslogTransferTrackerTests.java b/server/src/test/java/org/opensearch/index/remote/RemoteTranslogTransferTrackerTests.java new file mode 100644 index 0000000000000..6b6d388f725f7 --- /dev/null +++ b/server/src/test/java/org/opensearch/index/remote/RemoteTranslogTransferTrackerTests.java @@ -0,0 +1,383 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.index.remote; + +import org.opensearch.common.io.stream.BytesStreamOutput; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.test.OpenSearchTestCase; +import org.junit.Before; + +import java.io.IOException; + +public class RemoteTranslogTransferTrackerTests extends OpenSearchTestCase { + private ShardId shardId; + private RemoteTranslogTransferTracker tracker; + + @Override + public void setUp() throws Exception { + super.setUp(); + shardId = new ShardId("index", "uuid", 0); + } + + @Before + public void initTracker() { + tracker = new RemoteTranslogTransferTracker(shardId, RemoteStoreStatsTrackerFactory.Defaults.MOVING_AVERAGE_WINDOW_SIZE_MIN_VALUE); + } + + public void testGetShardId() { + assertEquals(shardId, tracker.getShardId()); + } + + public void testAddUploadsStarted() { + populateUploadsStarted(); + } + + public void testAddUploadsFailed() { + populateUploadsStarted(); + assertEquals(0L, tracker.getTotalUploadsFailed()); + tracker.incrementTotalUploadsFailed(); + assertEquals(1L, tracker.getTotalUploadsFailed()); + tracker.incrementTotalUploadsFailed(); + assertEquals(2L, tracker.getTotalUploadsFailed()); + } + + public void testInvalidAddUploadsFailed() { + populateUploadsStarted(); + for (int i = 0; i < tracker.getTotalUploadsStarted(); i++) { + tracker.incrementTotalUploadsSucceeded(); + } + + AssertionError error = assertThrows(AssertionError.class, () -> tracker.incrementTotalUploadsFailed()); + assertTrue(error.getMessage().contains("Sum of failure count (")); + } + + public void testAddUploadsSucceeded() { + populateUploadsStarted(); + assertEquals(0L, tracker.getTotalUploadsSucceeded()); + tracker.incrementTotalUploadsSucceeded(); + assertEquals(1L, tracker.getTotalUploadsSucceeded()); + tracker.incrementTotalUploadsSucceeded(); + assertEquals(2L, tracker.getTotalUploadsSucceeded()); + } + + public void testInvalidAddUploadsSucceeded() { + populateUploadsStarted(); + for (int i = 0; i < tracker.getTotalUploadsStarted(); i++) { + tracker.incrementTotalUploadsFailed(); + } + + AssertionError error = assertThrows(AssertionError.class, () -> tracker.incrementTotalUploadsSucceeded()); + assertTrue(error.getMessage().contains("Sum of failure count (")); + } + + public void testAddUploadBytesStarted() { + populateUploadBytesStarted(); + } + + public void testAddUploadBytesFailed() { + populateUploadBytesStarted(); + assertEquals(0L, tracker.getUploadBytesFailed()); + long count1 = randomIntBetween(1, (int) tracker.getUploadBytesStarted() / 4); + tracker.addUploadBytesFailed(count1); + assertEquals(count1, tracker.getUploadBytesFailed()); + long count2 = randomIntBetween(1, (int) tracker.getUploadBytesStarted() / 4); + tracker.addUploadBytesFailed(count2); + assertEquals(count1 + count2, tracker.getUploadBytesFailed()); + } + + public void testInvalidAddUploadBytesFailed() { + populateUploadBytesStarted(); + tracker.addUploadBytesSucceeded(tracker.getUploadBytesStarted()); + AssertionError error = assertThrows(AssertionError.class, () -> tracker.addUploadBytesFailed(1L)); + assertTrue(error.getMessage().contains("Sum of failure count (")); + } + + public void testAddUploadBytesSucceeded() { + populateUploadBytesStarted(); + assertEquals(0L, tracker.getUploadBytesSucceeded()); + long count1 = randomIntBetween(1, (int) tracker.getUploadBytesStarted() / 4); + tracker.addUploadBytesSucceeded(count1); + assertEquals(count1, tracker.getUploadBytesSucceeded()); + long count2 = randomIntBetween(1, (int) tracker.getUploadBytesStarted() / 4); + tracker.addUploadBytesSucceeded(count2); + assertEquals(count1 + count2, tracker.getUploadBytesSucceeded()); + } + + public void testInvalidAddUploadBytesSucceeded() { + populateUploadBytesStarted(); + tracker.addUploadBytesFailed(tracker.getUploadBytesStarted()); + AssertionError error = assertThrows(AssertionError.class, () -> tracker.addUploadBytesSucceeded(1L)); + assertTrue(error.getMessage().contains("Sum of failure count (")); + } + + public void testAddUploadTimeInMillis() { + assertEquals(0L, tracker.getTotalUploadTimeInMillis()); + int duration1 = randomIntBetween(10, 50); + tracker.addUploadTimeInMillis(duration1); + assertEquals(duration1, tracker.getTotalUploadTimeInMillis()); + int duration2 = randomIntBetween(10, 50); + tracker.addUploadTimeInMillis(duration2); + assertEquals(duration1 + duration2, tracker.getTotalUploadTimeInMillis()); + } + + public void testSetLastSuccessfulUploadTimestamp() { + assertEquals(0, tracker.getLastSuccessfulUploadTimestamp()); + long lastUploadTimestamp = System.currentTimeMillis() + randomIntBetween(10, 100); + tracker.setLastSuccessfulUploadTimestamp(lastUploadTimestamp); + assertEquals(lastUploadTimestamp, tracker.getLastSuccessfulUploadTimestamp()); + } + + public void testUpdateUploadBytesMovingAverage() { + int movingAverageWindowSize = randomIntBetween( + RemoteStoreStatsTrackerFactory.Defaults.MOVING_AVERAGE_WINDOW_SIZE_MIN_VALUE, + RemoteStoreStatsTrackerFactory.Defaults.MOVING_AVERAGE_WINDOW_SIZE + 5 + ); + tracker = new RemoteTranslogTransferTracker(shardId, movingAverageWindowSize); + assertFalse(tracker.isUploadBytesMovingAverageReady()); + + long sum = 0; + for (int i = 1; i < movingAverageWindowSize; i++) { + tracker.updateUploadBytesMovingAverage(i); + sum += i; + assertFalse(tracker.isUploadBytesMovingAverageReady()); + assertEquals((double) sum / i, tracker.getUploadBytesMovingAverage(), 0.0d); + } + + tracker.updateUploadBytesMovingAverage(movingAverageWindowSize); + sum += movingAverageWindowSize; + assertTrue(tracker.isUploadBytesMovingAverageReady()); + assertEquals((double) sum / movingAverageWindowSize, tracker.getUploadBytesMovingAverage(), 0.0d); + + tracker.updateUploadBytesMovingAverage(100); + sum = sum + 100 - 1; + assertEquals((double) sum / movingAverageWindowSize, tracker.getUploadBytesMovingAverage(), 0.0d); + } + + public void testUpdateUploadBytesPerSecMovingAverage() { + int movingAverageWindowSize = randomIntBetween( + RemoteStoreStatsTrackerFactory.Defaults.MOVING_AVERAGE_WINDOW_SIZE_MIN_VALUE, + RemoteStoreStatsTrackerFactory.Defaults.MOVING_AVERAGE_WINDOW_SIZE + 5 + ); + tracker = new RemoteTranslogTransferTracker(shardId, movingAverageWindowSize); + assertFalse(tracker.isUploadBytesPerSecMovingAverageReady()); + + long sum = 0; + for (int i = 1; i < movingAverageWindowSize; i++) { + tracker.updateUploadBytesPerSecMovingAverage(i); + sum += i; + assertFalse(tracker.isUploadBytesPerSecMovingAverageReady()); + assertEquals((double) sum / i, tracker.getUploadBytesPerSecMovingAverage(), 0.0d); + } + + tracker.updateUploadBytesPerSecMovingAverage(movingAverageWindowSize); + sum += movingAverageWindowSize; + assertTrue(tracker.isUploadBytesPerSecMovingAverageReady()); + assertEquals((double) sum / movingAverageWindowSize, tracker.getUploadBytesPerSecMovingAverage(), 0.0d); + + tracker.updateUploadBytesPerSecMovingAverage(100); + sum = sum + 100 - 1; + assertEquals((double) sum / movingAverageWindowSize, tracker.getUploadBytesPerSecMovingAverage(), 0.0d); + } + + public void testUpdateUploadTimeMovingAverage() { + int movingAverageWindowSize = randomIntBetween( + RemoteStoreStatsTrackerFactory.Defaults.MOVING_AVERAGE_WINDOW_SIZE_MIN_VALUE, + RemoteStoreStatsTrackerFactory.Defaults.MOVING_AVERAGE_WINDOW_SIZE + 5 + ); + tracker = new RemoteTranslogTransferTracker(shardId, movingAverageWindowSize); + assertFalse(tracker.isUploadTimeMovingAverageReady()); + + long sum = 0; + for (int i = 1; i < movingAverageWindowSize; i++) { + tracker.updateUploadTimeMovingAverage(i); + sum += i; + assertFalse(tracker.isUploadTimeMovingAverageReady()); + assertEquals((double) sum / i, tracker.getUploadTimeMovingAverage(), 0.0d); + } + + tracker.updateUploadTimeMovingAverage(movingAverageWindowSize); + sum += movingAverageWindowSize; + assertTrue(tracker.isUploadTimeMovingAverageReady()); + assertEquals((double) sum / movingAverageWindowSize, tracker.getUploadTimeMovingAverage(), 0.0d); + + tracker.updateUploadTimeMovingAverage(100); + sum = sum + 100 - 1; + assertEquals((double) sum / movingAverageWindowSize, tracker.getUploadTimeMovingAverage(), 0.0d); + } + + public void testAddDownloadsSucceeded() { + assertEquals(0L, tracker.getTotalDownloadsSucceeded()); + tracker.incrementDownloadsSucceeded(); + assertEquals(1L, tracker.getTotalDownloadsSucceeded()); + tracker.incrementDownloadsSucceeded(); + assertEquals(2L, tracker.getTotalDownloadsSucceeded()); + } + + public void testAddDownloadBytesSucceeded() { + assertEquals(0L, tracker.getDownloadBytesSucceeded()); + long count1 = randomIntBetween(1, 500); + tracker.addDownloadBytesSucceeded(count1); + assertEquals(count1, tracker.getDownloadBytesSucceeded()); + long count2 = randomIntBetween(1, 500); + tracker.addDownloadBytesSucceeded(count2); + assertEquals(count1 + count2, tracker.getDownloadBytesSucceeded()); + } + + public void testAddDownloadTimeInMillis() { + assertEquals(0L, tracker.getTotalDownloadTimeInMillis()); + int duration1 = randomIntBetween(10, 50); + tracker.addDownloadTimeInMillis(duration1); + assertEquals(duration1, tracker.getTotalDownloadTimeInMillis()); + int duration2 = randomIntBetween(10, 50); + tracker.addDownloadTimeInMillis(duration2); + assertEquals(duration1 + duration2, tracker.getTotalDownloadTimeInMillis()); + } + + public void testSetLastSuccessfulDownloadTimestamp() { + assertEquals(0, tracker.getLastSuccessfulDownloadTimestamp()); + long lastSuccessfulDownloadTimestamp = System.currentTimeMillis() + randomIntBetween(10, 100); + tracker.setLastSuccessfulDownloadTimestamp(lastSuccessfulDownloadTimestamp); + assertEquals(lastSuccessfulDownloadTimestamp, tracker.getLastSuccessfulDownloadTimestamp()); + } + + public void testUpdateDowmloadBytesMovingAverage() { + int movingAverageWindowSize = randomIntBetween( + RemoteStoreStatsTrackerFactory.Defaults.MOVING_AVERAGE_WINDOW_SIZE_MIN_VALUE, + RemoteStoreStatsTrackerFactory.Defaults.MOVING_AVERAGE_WINDOW_SIZE + 5 + ); + tracker = new RemoteTranslogTransferTracker(shardId, movingAverageWindowSize); + assertFalse(tracker.isDownloadBytesMovingAverageReady()); + + long sum = 0; + for (int i = 1; i < movingAverageWindowSize; i++) { + tracker.updateDownloadBytesMovingAverage(i); + sum += i; + assertFalse(tracker.isDownloadBytesMovingAverageReady()); + assertEquals((double) sum / i, tracker.getDownloadBytesMovingAverage(), 0.0d); + } + + tracker.updateDownloadBytesMovingAverage(movingAverageWindowSize); + sum += movingAverageWindowSize; + assertTrue(tracker.isDownloadBytesMovingAverageReady()); + assertEquals((double) sum / movingAverageWindowSize, tracker.getDownloadBytesMovingAverage(), 0.0d); + + tracker.updateDownloadBytesMovingAverage(100); + sum = sum + 100 - 1; + assertEquals((double) sum / movingAverageWindowSize, tracker.getDownloadBytesMovingAverage(), 0.0d); + } + + public void testUpdateDownloadBytesPerSecMovingAverage() { + int movingAverageWindowSize = randomIntBetween( + RemoteStoreStatsTrackerFactory.Defaults.MOVING_AVERAGE_WINDOW_SIZE_MIN_VALUE, + RemoteStoreStatsTrackerFactory.Defaults.MOVING_AVERAGE_WINDOW_SIZE + 5 + ); + tracker = new RemoteTranslogTransferTracker(shardId, movingAverageWindowSize); + assertFalse(tracker.isDownloadBytesPerSecMovingAverageReady()); + + long sum = 0; + for (int i = 1; i < movingAverageWindowSize; i++) { + tracker.updateDownloadBytesPerSecMovingAverage(i); + sum += i; + assertFalse(tracker.isDownloadBytesPerSecMovingAverageReady()); + assertEquals((double) sum / i, tracker.getDownloadBytesPerSecMovingAverage(), 0.0d); + } + + tracker.updateDownloadBytesPerSecMovingAverage(movingAverageWindowSize); + sum += movingAverageWindowSize; + assertTrue(tracker.isDownloadBytesPerSecMovingAverageReady()); + assertEquals((double) sum / movingAverageWindowSize, tracker.getDownloadBytesPerSecMovingAverage(), 0.0d); + + tracker.updateDownloadBytesPerSecMovingAverage(100); + sum = sum + 100 - 1; + assertEquals((double) sum / movingAverageWindowSize, tracker.getDownloadBytesPerSecMovingAverage(), 0.0d); + } + + public void testUpdateDownloadTimeMovingAverage() { + int movingAverageWindowSize = randomIntBetween( + RemoteStoreStatsTrackerFactory.Defaults.MOVING_AVERAGE_WINDOW_SIZE_MIN_VALUE, + RemoteStoreStatsTrackerFactory.Defaults.MOVING_AVERAGE_WINDOW_SIZE + 5 + ); + tracker = new RemoteTranslogTransferTracker(shardId, movingAverageWindowSize); + assertFalse(tracker.isDownloadTimeMovingAverageReady()); + + long sum = 0; + for (int i = 1; i < movingAverageWindowSize; i++) { + tracker.updateDownloadTimeMovingAverage(i); + sum += i; + assertFalse(tracker.isDownloadTimeMovingAverageReady()); + assertEquals((double) sum / i, tracker.getDownloadTimeMovingAverage(), 0.0d); + } + + tracker.updateDownloadTimeMovingAverage(movingAverageWindowSize); + sum += movingAverageWindowSize; + assertTrue(tracker.isDownloadTimeMovingAverageReady()); + assertEquals((double) sum / movingAverageWindowSize, tracker.getDownloadTimeMovingAverage(), 0.0d); + + tracker.updateDownloadTimeMovingAverage(100); + sum = sum + 100 - 1; + assertEquals((double) sum / movingAverageWindowSize, tracker.getDownloadTimeMovingAverage(), 0.0d); + } + + public void testStatsObjectCreation() { + populateDummyStats(); + RemoteTranslogTransferTracker.Stats actualStats = tracker.stats(); + assertTrue(tracker.hasSameStatsAs(actualStats)); + } + + public void testStatsObjectCreationViaStream() throws IOException { + populateDummyStats(); + RemoteTranslogTransferTracker.Stats expectedStats = tracker.stats(); + try (BytesStreamOutput out = new BytesStreamOutput()) { + expectedStats.writeTo(out); + try (StreamInput in = out.bytes().streamInput()) { + RemoteTranslogTransferTracker.Stats deserializedStats = new RemoteTranslogTransferTracker.Stats(in); + assertTrue(tracker.hasSameStatsAs(deserializedStats)); + } + } + } + + private void populateUploadsStarted() { + assertEquals(0L, tracker.getTotalUploadsStarted()); + tracker.incrementTotalUploadsStarted(); + assertEquals(1L, tracker.getTotalUploadsStarted()); + tracker.incrementTotalUploadsStarted(); + assertEquals(2L, tracker.getTotalUploadsStarted()); + } + + private void populateUploadBytesStarted() { + assertEquals(0L, tracker.getUploadBytesStarted()); + long count1 = randomIntBetween(500, 1000); + tracker.addUploadBytesStarted(count1); + assertEquals(count1, tracker.getUploadBytesStarted()); + long count2 = randomIntBetween(500, 1000); + tracker.addUploadBytesStarted(count2); + assertEquals(count1 + count2, tracker.getUploadBytesStarted()); + } + + private void populateDummyStats() { + int startedBytesUpload = randomIntBetween(10, 100); + tracker.addUploadBytesStarted(startedBytesUpload); + tracker.addUploadBytesFailed(randomIntBetween(1, startedBytesUpload / 2)); + tracker.addUploadBytesSucceeded(randomIntBetween(1, startedBytesUpload / 2)); + + tracker.addUploadTimeInMillis(randomIntBetween(10, 100)); + tracker.setLastSuccessfulUploadTimestamp(System.currentTimeMillis() + randomIntBetween(10, 100)); + + tracker.incrementTotalUploadsStarted(); + tracker.incrementTotalUploadsStarted(); + tracker.incrementTotalUploadsFailed(); + tracker.incrementTotalUploadsSucceeded(); + + tracker.addDownloadBytesSucceeded(randomIntBetween(10, 100)); + tracker.addDownloadTimeInMillis(randomIntBetween(10, 100)); + tracker.setLastSuccessfulDownloadTimestamp(System.currentTimeMillis() + randomIntBetween(10, 100)); + tracker.incrementDownloadsSucceeded(); + } +} diff --git a/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java index 2281c086db5d8..140ea0fade4b4 100644 --- a/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java @@ -4902,6 +4902,7 @@ public void testRecordsForceMerges() throws IOException { } private void populateSampleRemoteStoreStats(RemoteSegmentTransferTracker tracker) { + tracker.addUploadBytesStarted(10L); tracker.addUploadBytesStarted(10L); tracker.addUploadBytesSucceeded(10L); tracker.addUploadBytesFailed(10L); diff --git a/server/src/test/java/org/opensearch/index/translog/RemoteFSTranslogTests.java b/server/src/test/java/org/opensearch/index/translog/RemoteFsTranslogTests.java similarity index 93% rename from server/src/test/java/org/opensearch/index/translog/RemoteFSTranslogTests.java rename to server/src/test/java/org/opensearch/index/translog/RemoteFsTranslogTests.java index e13c0be93c6fe..b69d065f56de0 100644 --- a/server/src/test/java/org/opensearch/index/translog/RemoteFSTranslogTests.java +++ b/server/src/test/java/org/opensearch/index/translog/RemoteFsTranslogTests.java @@ -40,6 +40,7 @@ import org.opensearch.env.TestEnvironment; import org.opensearch.index.IndexSettings; import org.opensearch.index.engine.MissingHistoryOperationsException; +import org.opensearch.index.remote.RemoteTranslogTransferTracker; import org.opensearch.index.seqno.LocalCheckpointTracker; import org.opensearch.index.seqno.LocalCheckpointTrackerTests; import org.opensearch.index.seqno.SequenceNumbers; @@ -100,7 +101,7 @@ @LuceneTestCase.SuppressFileSystems("ExtrasFS") -public class RemoteFSTranslogTests extends OpenSearchTestCase { +public class RemoteFsTranslogTests extends OpenSearchTestCase { protected final ShardId shardId = new ShardId("index", "_na_", 1); @@ -172,7 +173,8 @@ private RemoteFsTranslog create(Path path, BlobStoreRepository repository, Strin getPersistedSeqNoConsumer(), repository, threadPool, - primaryMode::get + primaryMode::get, + new RemoteTranslogTransferTracker(shardId, 10) ); } @@ -256,6 +258,43 @@ private Translog.Location addToTranslogAndListAndUpload(Translog translog, List< return loc; } + private static void assertUploadStatsNoFailures(RemoteTranslogTransferTracker statsTracker) { + assertTrue(statsTracker.getUploadBytesStarted() > 0); + assertTrue(statsTracker.getTotalUploadsStarted() > 0); + assertEquals(0, statsTracker.getUploadBytesFailed()); + assertEquals(0, statsTracker.getTotalUploadsFailed()); + assertTrue(statsTracker.getUploadBytesSucceeded() > 0); + assertTrue(statsTracker.getTotalUploadsSucceeded() > 0); + assertTrue(statsTracker.getTotalUploadTimeInMillis() > 0); + assertTrue(statsTracker.getLastSuccessfulUploadTimestamp() > 0); + } + + private static void assertUploadStatsNoUploads(RemoteTranslogTransferTracker statsTracker) { + assertEquals(0, statsTracker.getUploadBytesStarted()); + assertEquals(0, statsTracker.getUploadBytesFailed()); + assertEquals(0, statsTracker.getUploadBytesSucceeded()); + assertEquals(0, statsTracker.getTotalUploadsStarted()); + assertEquals(0, statsTracker.getTotalUploadsFailed()); + assertEquals(0, statsTracker.getTotalUploadsSucceeded()); + assertEquals(0, statsTracker.getTotalUploadTimeInMillis()); + assertEquals(0, statsTracker.getLastSuccessfulUploadTimestamp()); + } + + private static void assertDownloadStatsPopulated(RemoteTranslogTransferTracker statsTracker) { + assertTrue(statsTracker.getDownloadBytesSucceeded() > 0); + assertTrue(statsTracker.getTotalDownloadsSucceeded() > 0); + // TODO: Need to simulate a delay for this assertion to avoid flakiness + // assertTrue(statsTracker.getTotalDownloadTimeInMillis() > 0); + assertTrue(statsTracker.getLastSuccessfulDownloadTimestamp() > 0); + } + + private static void assertDownloadStatsNoDownloads(RemoteTranslogTransferTracker statsTracker) { + assertEquals(0, statsTracker.getDownloadBytesSucceeded()); + assertEquals(0, statsTracker.getTotalDownloadsSucceeded()); + assertEquals(0, statsTracker.getTotalDownloadTimeInMillis()); + assertEquals(0, statsTracker.getLastSuccessfulDownloadTimestamp()); + } + public void testUploadWithPrimaryModeFalse() { // Test setup primaryMode.set(false); @@ -269,6 +308,9 @@ public void testUploadWithPrimaryModeFalse() { throw new RuntimeException(e); } assertTrue(translog.syncNeeded()); + RemoteTranslogTransferTracker statsTracker = translog.getRemoteTranslogTracker(); + assertUploadStatsNoUploads(statsTracker); + assertDownloadStatsNoDownloads(statsTracker); } public void testUploadWithPrimaryModeTrue() { @@ -281,6 +323,9 @@ public void testUploadWithPrimaryModeTrue() { throw new RuntimeException(e); } assertFalse(translog.syncNeeded()); + RemoteTranslogTransferTracker statsTracker = translog.getRemoteTranslogTracker(); + assertUploadStatsNoFailures(statsTracker); + assertDownloadStatsNoDownloads(statsTracker); } public void testSimpleOperations() throws IOException { @@ -330,6 +375,9 @@ public void testReadLocation() throws IOException { assertEquals(op, translog.readOperation(locs.get(i++))); } assertNull(translog.readOperation(new Translog.Location(100, 0, 0))); + RemoteTranslogTransferTracker statsTracker = translog.getRemoteTranslogTracker(); + assertUploadStatsNoFailures(statsTracker); + assertDownloadStatsNoDownloads(statsTracker); } public void testReadLocationDownload() throws IOException { @@ -338,12 +386,17 @@ public void testReadLocationDownload() throws IOException { locs.add(addToTranslogAndListAndUpload(translog, ops, new Translog.Index("1", 0, primaryTerm.get(), new byte[] { 1 }))); locs.add(addToTranslogAndListAndUpload(translog, ops, new Translog.Index("2", 1, primaryTerm.get(), new byte[] { 1 }))); locs.add(addToTranslogAndListAndUpload(translog, ops, new Translog.Index("3", 2, primaryTerm.get(), new byte[] { 1 }))); + translog.sync(); int i = 0; for (Translog.Operation op : ops) { assertEquals(op, translog.readOperation(locs.get(i++))); } + RemoteTranslogTransferTracker statsTracker = translog.getRemoteTranslogTracker(); + assertUploadStatsNoFailures(statsTracker); + assertDownloadStatsNoDownloads(statsTracker); + String translogUUID = translog.translogUUID; try { translog.getDeletionPolicy().assertNoOpenTranslogRefs(); @@ -358,11 +411,16 @@ public void testReadLocationDownload() throws IOException { } // Creating RemoteFsTranslog with the same location - Translog newTranslog = create(translogDir, repository, translogUUID); + RemoteFsTranslog newTranslog = create(translogDir, repository, translogUUID); i = 0; for (Translog.Operation op : ops) { assertEquals(op, newTranslog.readOperation(locs.get(i++))); } + + statsTracker = newTranslog.getRemoteTranslogTracker(); + assertUploadStatsNoUploads(statsTracker); + assertDownloadStatsPopulated(statsTracker); + try { newTranslog.close(); } catch (Exception e) { @@ -976,6 +1034,9 @@ public void testSyncUpTo() throws IOException { if (randomBoolean()) { translog.sync(); assertFalse("translog has been synced already", translog.ensureSynced(location)); + RemoteTranslogTransferTracker statsTracker = translog.getRemoteTranslogTracker(); + assertUploadStatsNoFailures(statsTracker); + assertDownloadStatsNoDownloads(statsTracker); } } } @@ -985,12 +1046,13 @@ public void testSyncUpFailure() throws IOException { int count = 0; fail.failAlways(); ArrayList locations = new ArrayList<>(); + boolean shouldFailAlways = randomBoolean(); for (int op = 0; op < translogOperations; op++) { int seqNo = ++count; final Translog.Location location = translog.add( new Translog.Index("" + op, seqNo, primaryTerm.get(), Integer.toString(seqNo).getBytes(Charset.forName("UTF-8"))) ); - if (randomBoolean()) { + if (shouldFailAlways) { fail.failAlways(); try { translog.ensureSynced(location); @@ -1016,6 +1078,19 @@ public void testSyncUpFailure() throws IOException { assertFalse("all of the locations should be synced: " + location, translog.ensureSynced(location)); } + RemoteTranslogTransferTracker statsTracker = translog.getRemoteTranslogTracker(); + assertTrue(statsTracker.getUploadBytesStarted() > 0); + assertTrue(statsTracker.getTotalUploadsStarted() > 0); + + if (shouldFailAlways) { + assertTrue(statsTracker.getTotalUploadsFailed() > 0); + } else { + assertEquals(0, statsTracker.getTotalUploadsFailed()); + } + + assertTrue(statsTracker.getTotalUploadsSucceeded() > 0); + assertTrue(statsTracker.getLastSuccessfulUploadTimestamp() > 0); + assertDownloadStatsNoDownloads(statsTracker); } public void testSyncUpToStream() throws IOException { @@ -1048,6 +1123,11 @@ public void testSyncUpToStream() throws IOException { translog.sync(); assertFalse("translog has been synced already", translog.ensureSynced(locations.stream())); } + + RemoteTranslogTransferTracker statsTracker = translog.getRemoteTranslogTracker(); + assertUploadStatsNoFailures(statsTracker); + assertDownloadStatsNoDownloads(statsTracker); + for (Translog.Location location : locations) { assertFalse("all of the locations should be synced: " + location, translog.ensureSynced(location)); } @@ -1223,7 +1303,8 @@ public int write(ByteBuffer src) throws IOException { persistedSeqNos::add, repository, threadPool, - () -> Boolean.TRUE + () -> Boolean.TRUE, + new RemoteTranslogTransferTracker(shardId, 10) ) { @Override ChannelFactory getChannelFactory() { @@ -1329,7 +1410,8 @@ public void force(boolean metaData) throws IOException { persistedSeqNos::add, repository, threadPool, - () -> Boolean.TRUE + () -> Boolean.TRUE, + new RemoteTranslogTransferTracker(shardId, 10) ) { @Override ChannelFactory getChannelFactory() { diff --git a/server/src/test/java/org/opensearch/index/translog/transfer/FileTransferTrackerTests.java b/server/src/test/java/org/opensearch/index/translog/transfer/FileTransferTrackerTests.java index 1914790ac58d2..b96ada1f6bbff 100644 --- a/server/src/test/java/org/opensearch/index/translog/transfer/FileTransferTrackerTests.java +++ b/server/src/test/java/org/opensearch/index/translog/transfer/FileTransferTrackerTests.java @@ -9,27 +9,33 @@ package org.opensearch.index.translog.transfer; import org.opensearch.core.index.shard.ShardId; +import org.opensearch.index.remote.RemoteTranslogTransferTracker; import org.opensearch.test.OpenSearchTestCase; import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.StandardOpenOption; +import java.util.HashSet; import java.util.List; +import java.util.Set; public class FileTransferTrackerTests extends OpenSearchTestCase { protected final ShardId shardId = new ShardId("index", "_na_", 1); FileTransferTracker fileTransferTracker; + RemoteTranslogTransferTracker remoteTranslogTransferTracker; @Override public void setUp() throws Exception { super.setUp(); + remoteTranslogTransferTracker = new RemoteTranslogTransferTracker(shardId, 20); + fileTransferTracker = new FileTransferTracker(shardId, remoteTranslogTransferTracker); } public void testOnSuccess() throws IOException { - fileTransferTracker = new FileTransferTracker(shardId); Path testFile = createTempFile(); + int fileSize = 128; Files.write(testFile, randomByteArrayOfLength(128), StandardOpenOption.APPEND); try ( FileSnapshot.TransferFileSnapshot transferFileSnapshot = new FileSnapshot.TransferFileSnapshot( @@ -38,11 +44,17 @@ public void testOnSuccess() throws IOException { null ) ) { + Set toUpload = new HashSet<>(2); + toUpload.add(transferFileSnapshot); + fileTransferTracker.recordBytesForFiles(toUpload); + remoteTranslogTransferTracker.addUploadBytesStarted(fileSize); fileTransferTracker.onSuccess(transferFileSnapshot); // idempotent + remoteTranslogTransferTracker.addUploadBytesStarted(fileSize); fileTransferTracker.onSuccess(transferFileSnapshot); assertEquals(fileTransferTracker.allUploaded().size(), 1); try { + remoteTranslogTransferTracker.addUploadBytesStarted(fileSize); fileTransferTracker.onFailure(transferFileSnapshot, new IOException("random exception")); fail("failure after succcess invalid"); } catch (IllegalStateException ex) { @@ -52,10 +64,10 @@ public void testOnSuccess() throws IOException { } public void testOnFailure() throws IOException { - fileTransferTracker = new FileTransferTracker(shardId); Path testFile = createTempFile(); Path testFile2 = createTempFile(); - Files.write(testFile, randomByteArrayOfLength(128), StandardOpenOption.APPEND); + int fileSize = 128; + Files.write(testFile, randomByteArrayOfLength(fileSize), StandardOpenOption.APPEND); try ( FileSnapshot.TransferFileSnapshot transferFileSnapshot = new FileSnapshot.TransferFileSnapshot( testFile, @@ -66,30 +78,37 @@ public void testOnFailure() throws IOException { testFile2, randomNonNegativeLong(), null - ) + ); ) { - + Set toUpload = new HashSet<>(2); + toUpload.add(transferFileSnapshot); + toUpload.add(transferFileSnapshot2); + fileTransferTracker.recordBytesForFiles(toUpload); + remoteTranslogTransferTracker.addUploadBytesStarted(fileSize); fileTransferTracker.onFailure(transferFileSnapshot, new IOException("random exception")); fileTransferTracker.onSuccess(transferFileSnapshot2); assertEquals(fileTransferTracker.allUploaded().size(), 1); - + remoteTranslogTransferTracker.addUploadBytesStarted(fileSize); fileTransferTracker.onSuccess(transferFileSnapshot); assertEquals(fileTransferTracker.allUploaded().size(), 2); } } public void testUploaded() throws IOException { - fileTransferTracker = new FileTransferTracker(shardId); Path testFile = createTempFile(); - Files.write(testFile, randomByteArrayOfLength(128), StandardOpenOption.APPEND); + int fileSize = 128; + Files.write(testFile, randomByteArrayOfLength(fileSize), StandardOpenOption.APPEND); try ( FileSnapshot.TransferFileSnapshot transferFileSnapshot = new FileSnapshot.TransferFileSnapshot( testFile, randomNonNegativeLong(), null ); - ) { + Set toUpload = new HashSet<>(2); + toUpload.add(transferFileSnapshot); + fileTransferTracker.recordBytesForFiles(toUpload); + remoteTranslogTransferTracker.addUploadBytesStarted(fileSize); fileTransferTracker.onSuccess(transferFileSnapshot); String fileName = String.valueOf(testFile.getFileName()); assertTrue(fileTransferTracker.uploaded(fileName)); diff --git a/server/src/test/java/org/opensearch/index/translog/transfer/TranslogTransferManagerTests.java b/server/src/test/java/org/opensearch/index/translog/transfer/TranslogTransferManagerTests.java index a49cbe6d5eec4..6fc4557a75675 100644 --- a/server/src/test/java/org/opensearch/index/translog/transfer/TranslogTransferManagerTests.java +++ b/server/src/test/java/org/opensearch/index/translog/transfer/TranslogTransferManagerTests.java @@ -19,6 +19,7 @@ import org.opensearch.core.action.ActionListener; import org.opensearch.core.index.Index; import org.opensearch.core.index.shard.ShardId; +import org.opensearch.index.remote.RemoteTranslogTransferTracker; import org.opensearch.index.translog.Translog; import org.opensearch.index.translog.transfer.FileSnapshot.CheckpointFileSnapshot; import org.opensearch.index.translog.transfer.FileSnapshot.TransferFileSnapshot; @@ -63,6 +64,12 @@ public class TranslogTransferManagerTests extends OpenSearchTestCase { private long primaryTerm; private long generation; private long minTranslogGeneration; + private RemoteTranslogTransferTracker remoteTranslogTransferTracker; + byte[] tlogBytes; + byte[] ckpBytes; + FileTransferTracker tracker; + TranslogTransferManager translogTransferManager; + long delayForBlobDownload; @Override public void setUp() throws Exception { @@ -75,6 +82,28 @@ public void setUp() throws Exception { remoteBaseTransferPath = new BlobPath().add("base_path"); transferService = mock(TransferService.class); threadPool = new TestThreadPool(getClass().getName()); + remoteTranslogTransferTracker = new RemoteTranslogTransferTracker(shardId, 20); + tlogBytes = "Hello Translog".getBytes(StandardCharsets.UTF_8); + ckpBytes = "Hello Checkpoint".getBytes(StandardCharsets.UTF_8); + tracker = new FileTransferTracker(new ShardId("index", "indexUuid", 0), remoteTranslogTransferTracker); + translogTransferManager = new TranslogTransferManager( + shardId, + transferService, + remoteBaseTransferPath, + tracker, + remoteTranslogTransferTracker + ); + + delayForBlobDownload = 1; + when(transferService.downloadBlob(any(BlobPath.class), eq("translog-23.tlog"))).thenAnswer(invocation -> { + Thread.sleep(delayForBlobDownload); + return new ByteArrayInputStream(tlogBytes); + }); + + when(transferService.downloadBlob(any(BlobPath.class), eq("translog-23.ckp"))).thenAnswer(invocation -> { + Thread.sleep(delayForBlobDownload); + return new ByteArrayInputStream(ckpBytes); + }); } @Override @@ -103,7 +132,10 @@ public void testTransferSnapshot() throws Exception { return null; }).when(transferService).uploadBlobs(anySet(), anyMap(), any(ActionListener.class), any(WritePriority.class)); - FileTransferTracker fileTransferTracker = new FileTransferTracker(new ShardId("index", "indexUUid", 0)) { + FileTransferTracker fileTransferTracker = new FileTransferTracker( + new ShardId("index", "indexUUid", 0), + remoteTranslogTransferTracker + ) { @Override public void onSuccess(TransferFileSnapshot fileSnapshot) { fileTransferSucceeded.incrementAndGet(); @@ -122,7 +154,8 @@ public void onFailure(TransferFileSnapshot fileSnapshot, Exception e) { shardId, transferService, remoteBaseTransferPath, - fileTransferTracker + fileTransferTracker, + remoteTranslogTransferTracker ); assertTrue(translogTransferManager.transferSnapshot(createTransferSnapshot(), new TranslogTransferListener() { @@ -203,7 +236,8 @@ public void testReadMetadataNoFile() throws IOException { shardId, transferService, remoteBaseTransferPath, - null + null, + remoteTranslogTransferTracker ); doAnswer(invocation -> { LatchedActionListener> latchedActionListener = invocation.getArgument(3); @@ -214,6 +248,7 @@ public void testReadMetadataNoFile() throws IOException { .listAllInSortedOrder(any(BlobPath.class), eq(TranslogTransferMetadata.METADATA_PREFIX), anyInt(), any(ActionListener.class)); assertNull(translogTransferManager.readMetadata()); + assertNoDownloadStats(false); } // This should happen most of the time - Just a single metadata file @@ -222,7 +257,8 @@ public void testReadMetadataSingleFile() throws IOException { shardId, transferService, remoteBaseTransferPath, - null + null, + remoteTranslogTransferTracker ); TranslogTransferMetadata tm = new TranslogTransferMetadata(1, 1, 1, 2); String mdFilename = tm.getFileName(); @@ -236,11 +272,16 @@ public void testReadMetadataSingleFile() throws IOException { .listAllInSortedOrder(any(BlobPath.class), eq(TranslogTransferMetadata.METADATA_PREFIX), anyInt(), any(ActionListener.class)); TranslogTransferMetadata metadata = createTransferSnapshot().getTranslogTransferMetadata(); - when(transferService.downloadBlob(any(BlobPath.class), eq(mdFilename))).thenReturn( - new ByteArrayInputStream(translogTransferManager.getMetadataBytes(metadata)) - ); + long delayForMdDownload = 1; + when(transferService.downloadBlob(any(BlobPath.class), eq(mdFilename))).thenAnswer(invocation -> { + Thread.sleep(delayForMdDownload); + return new ByteArrayInputStream(translogTransferManager.getMetadataBytes(metadata)); + }); assertEquals(metadata, translogTransferManager.readMetadata()); + + assertEquals(translogTransferManager.getMetadataBytes(metadata).length, remoteTranslogTransferTracker.getDownloadBytesSucceeded()); + assertTrue(remoteTranslogTransferTracker.getTotalDownloadTimeInMillis() >= delayForMdDownload); } public void testReadMetadataReadException() throws IOException { @@ -248,9 +289,9 @@ public void testReadMetadataReadException() throws IOException { shardId, transferService, remoteBaseTransferPath, - null + null, + remoteTranslogTransferTracker ); - TranslogTransferMetadata tm = new TranslogTransferMetadata(1, 1, 1, 2); String mdFilename = tm.getFileName(); @@ -266,6 +307,7 @@ public void testReadMetadataReadException() throws IOException { when(transferService.downloadBlob(any(BlobPath.class), eq(mdFilename))).thenThrow(new IOException("Something went wrong")); assertThrows(IOException.class, translogTransferManager::readMetadata); + assertNoDownloadStats(true); } public void testMetadataFileNameOrder() throws IOException { @@ -283,7 +325,8 @@ public void testReadMetadataListException() throws IOException { shardId, transferService, remoteBaseTransferPath, - null + null, + remoteTranslogTransferTracker ); doAnswer(invocation -> { @@ -296,81 +339,39 @@ public void testReadMetadataListException() throws IOException { when(transferService.downloadBlob(any(BlobPath.class), any(String.class))).thenThrow(new IOException("Something went wrong")); assertThrows(IOException.class, translogTransferManager::readMetadata); + assertNoDownloadStats(false); } public void testDownloadTranslog() throws IOException { Path location = createTempDir(); - TranslogTransferManager translogTransferManager = new TranslogTransferManager( - shardId, - transferService, - remoteBaseTransferPath, - new FileTransferTracker(new ShardId("index", "indexUuid", 0)) - ); - - when(transferService.downloadBlob(any(BlobPath.class), eq("translog-23.tlog"))).thenReturn( - new ByteArrayInputStream("Hello Translog".getBytes(StandardCharsets.UTF_8)) - ); - - when(transferService.downloadBlob(any(BlobPath.class), eq("translog-23.ckp"))).thenReturn( - new ByteArrayInputStream("Hello Checkpoint".getBytes(StandardCharsets.UTF_8)) - ); - assertFalse(Files.exists(location.resolve("translog-23.tlog"))); assertFalse(Files.exists(location.resolve("translog-23.ckp"))); translogTransferManager.downloadTranslog("12", "23", location); assertTrue(Files.exists(location.resolve("translog-23.tlog"))); assertTrue(Files.exists(location.resolve("translog-23.ckp"))); + assertTlogCkpDownloadStats(); } public void testDownloadTranslogAlreadyExists() throws IOException { - FileTransferTracker tracker = new FileTransferTracker(new ShardId("index", "indexUuid", 0)); Path location = createTempDir(); Files.createFile(location.resolve("translog-23.tlog")); Files.createFile(location.resolve("translog-23.ckp")); - TranslogTransferManager translogTransferManager = new TranslogTransferManager( - shardId, - transferService, - remoteBaseTransferPath, - tracker - ); - - when(transferService.downloadBlob(any(BlobPath.class), eq("translog-23.tlog"))).thenReturn( - new ByteArrayInputStream("Hello Translog".getBytes(StandardCharsets.UTF_8)) - ); - when(transferService.downloadBlob(any(BlobPath.class), eq("translog-23.ckp"))).thenReturn( - new ByteArrayInputStream("Hello Checkpoint".getBytes(StandardCharsets.UTF_8)) - ); - translogTransferManager.downloadTranslog("12", "23", location); verify(transferService).downloadBlob(any(BlobPath.class), eq("translog-23.tlog")); verify(transferService).downloadBlob(any(BlobPath.class), eq("translog-23.ckp")); assertTrue(Files.exists(location.resolve("translog-23.tlog"))); assertTrue(Files.exists(location.resolve("translog-23.ckp"))); + assertTlogCkpDownloadStats(); } public void testDownloadTranslogWithTrackerUpdated() throws IOException { - FileTransferTracker tracker = new FileTransferTracker(new ShardId("index", "indexUuid", 0)); Path location = createTempDir(); String translogFile = "translog-23.tlog", checkpointFile = "translog-23.ckp"; Files.createFile(location.resolve(translogFile)); Files.createFile(location.resolve(checkpointFile)); - TranslogTransferManager translogTransferManager = new TranslogTransferManager( - shardId, - transferService, - remoteBaseTransferPath, - tracker - ); - - when(transferService.downloadBlob(any(BlobPath.class), eq(translogFile))).thenReturn( - new ByteArrayInputStream("Hello Translog".getBytes(StandardCharsets.UTF_8)) - ); - when(transferService.downloadBlob(any(BlobPath.class), eq(checkpointFile))).thenReturn( - new ByteArrayInputStream("Hello Checkpoint".getBytes(StandardCharsets.UTF_8)) - ); - translogTransferManager.downloadTranslog("12", "23", location); verify(transferService).downloadBlob(any(BlobPath.class), eq(translogFile)); @@ -385,10 +386,10 @@ public void testDownloadTranslogWithTrackerUpdated() throws IOException { // Since the tracker already holds the files with success state, adding them with success state is allowed tracker.add(translogFile, true); tracker.add(checkpointFile, true); + assertTlogCkpDownloadStats(); } public void testDeleteTranslogSuccess() throws Exception { - FileTransferTracker tracker = new FileTransferTracker(new ShardId("index", "indexUuid", 0)); BlobStore blobStore = mock(BlobStore.class); BlobContainer blobContainer = mock(BlobContainer.class); when(blobStore.blobContainer(any(BlobPath.class))).thenReturn(blobContainer); @@ -397,7 +398,8 @@ public void testDeleteTranslogSuccess() throws Exception { shardId, blobStoreTransferService, remoteBaseTransferPath, - tracker + tracker, + remoteTranslogTransferTracker ); String translogFile = "translog-19.tlog", checkpointFile = "translog-19.ckp"; tracker.add(translogFile, true); @@ -415,7 +417,8 @@ public void testDeleteStaleTranslogMetadata() { shardId, transferService, remoteBaseTransferPath, - null + null, + remoteTranslogTransferTracker ); String tm1 = new TranslogTransferMetadata(1, 1, 1, 2).getFileName(); String tm2 = new TranslogTransferMetadata(1, 2, 1, 2).getFileName(); @@ -455,7 +458,7 @@ public void testDeleteStaleTranslogMetadata() { } public void testDeleteTranslogFailure() throws Exception { - FileTransferTracker tracker = new FileTransferTracker(new ShardId("index", "indexUuid", 0)); + FileTransferTracker tracker = new FileTransferTracker(new ShardId("index", "indexUuid", 0), remoteTranslogTransferTracker); BlobStore blobStore = mock(BlobStore.class); BlobContainer blobContainer = mock(BlobContainer.class); doAnswer(invocation -> { throw new IOException("test exception"); }).when(blobStore).blobContainer(any(BlobPath.class)); @@ -465,7 +468,8 @@ public void testDeleteTranslogFailure() throws Exception { shardId, blobStoreTransferService, remoteBaseTransferPath, - tracker + tracker, + remoteTranslogTransferTracker ); String translogFile = "translog-19.tlog", checkpointFile = "translog-19.ckp"; tracker.add(translogFile, true); @@ -475,4 +479,21 @@ public void testDeleteTranslogFailure() throws Exception { translogTransferManager.deleteGenerationAsync(primaryTerm, Set.of(19L), () -> {}); assertEquals(2, tracker.allUploaded().size()); } + + private void assertNoDownloadStats(boolean nonZeroUploadTime) { + assertEquals(0, remoteTranslogTransferTracker.getDownloadBytesSucceeded()); + assertEquals(0, remoteTranslogTransferTracker.getTotalDownloadsSucceeded()); + assertEquals(0, remoteTranslogTransferTracker.getLastSuccessfulDownloadTimestamp()); + if (nonZeroUploadTime) { + assertNotEquals(0, remoteTranslogTransferTracker.getTotalDownloadTimeInMillis()); + } else { + assertEquals(0, remoteTranslogTransferTracker.getTotalDownloadTimeInMillis()); + } + } + + private void assertTlogCkpDownloadStats() { + assertEquals(tlogBytes.length + ckpBytes.length, remoteTranslogTransferTracker.getDownloadBytesSucceeded()); + // Expect delay for both tlog and ckp file + assertTrue(remoteTranslogTransferTracker.getTotalDownloadTimeInMillis() >= 2 * delayForBlobDownload); + } } diff --git a/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java b/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java index 81e3b74c40104..37ea8a28bd4b4 100644 --- a/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java +++ b/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java @@ -2064,7 +2064,8 @@ public void onFailure(final Exception e) { emptyMap(), new RemoteSegmentStoreDirectoryFactory(() -> repositoriesService, threadPool), repositoriesServiceReference::get, - fileCacheCleaner + fileCacheCleaner, + new RemoteStoreStatsTrackerFactory(clusterService, settings) ); final RecoverySettings recoverySettings = new RecoverySettings(settings, clusterSettings); snapshotShardsService = new SnapshotShardsService( diff --git a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java index 474acc764620d..43f2cce668e81 100644 --- a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java @@ -98,6 +98,7 @@ import org.opensearch.index.mapper.MapperService; import org.opensearch.index.mapper.SourceToParse; import org.opensearch.index.remote.RemoteStoreStatsTrackerFactory; +import org.opensearch.index.remote.RemoteTranslogTransferTracker; import org.opensearch.index.replication.TestReplicationSource; import org.opensearch.index.seqno.ReplicationTracker; import org.opensearch.index.seqno.RetentionLeaseSyncer; @@ -662,11 +663,11 @@ protected IndexShard newShard( final BiFunction translogFactorySupplier = (settings, shardRouting) -> { if (settings.isRemoteTranslogStoreEnabled() && shardRouting.primary()) { - return new RemoteBlobStoreInternalTranslogFactory( () -> mockRepoSvc, threadPool, - settings.getRemoteStoreTranslogRepository() + settings.getRemoteStoreTranslogRepository(), + new RemoteTranslogTransferTracker(shardRouting.shardId(), 20) ); } return new InternalTranslogFactory();