From 0c7408d90a67e4139b0c5fabc4a488cb2033b901 Mon Sep 17 00:00:00 2001 From: Ashish Singh Date: Tue, 18 Apr 2023 16:25:19 +0530 Subject: [PATCH 01/17] Add remote refresh segment pressure service, settings and tracker Signed-off-by: Ashish Singh --- .../RemoteRefreshSegmentPressureService.java | 198 ++++++++++ .../RemoteRefreshSegmentPressureSettings.java | 220 +++++++++++ .../RemoteRefreshSegmentPressureTracker.java | 353 ++++++++++++++++++ 3 files changed, 771 insertions(+) create mode 100644 server/src/main/java/org/opensearch/index/RemoteRefreshSegmentPressureService.java create mode 100644 server/src/main/java/org/opensearch/index/RemoteRefreshSegmentPressureSettings.java create mode 100644 server/src/main/java/org/opensearch/index/RemoteRefreshSegmentPressureTracker.java diff --git a/server/src/main/java/org/opensearch/index/RemoteRefreshSegmentPressureService.java b/server/src/main/java/org/opensearch/index/RemoteRefreshSegmentPressureService.java new file mode 100644 index 0000000000000..ab3584f3b8178 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/RemoteRefreshSegmentPressureService.java @@ -0,0 +1,198 @@ +/* + * 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; + +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.Settings; +import org.opensearch.common.util.concurrent.ConcurrentCollections; +import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; +import org.opensearch.index.shard.IndexEventListener; +import org.opensearch.index.shard.IndexShard; +import org.opensearch.index.shard.ShardId; + +import java.util.Locale; +import java.util.Map; +import java.util.function.BiConsumer; + +/** + * Service used to validate if the incoming indexing request should be rejected based on the {@link RemoteRefreshSegmentPressureTracker}. + */ +public class RemoteRefreshSegmentPressureService implements IndexEventListener { + private static final Logger logger = LogManager.getLogger(RemoteRefreshSegmentPressureService.class); + + /** + * Keeps map of remote-backed index shards and their corresponding backpressure tracker. + */ + private final Map trackerMap = ConcurrentCollections.newConcurrentMap(); + + /** + * Remote refresh segment pressure settings which is used for creation of the backpressure tracker and as well as rejection. + */ + private final RemoteRefreshSegmentPressureSettings pressureSettings; + + @Inject + public RemoteRefreshSegmentPressureService(ClusterService clusterService, Settings settings) { + pressureSettings = new RemoteRefreshSegmentPressureSettings(clusterService, settings, this); + } + + /** + * Get {@code RemoteRefreshSegmentPressureTracker} only if the underlying Index has remote segments integration enabled. + * + * @param shardId shard id + * @return the tracker if index is remote-backed, else null. + */ + public RemoteRefreshSegmentPressureTracker getStatsTracker(ShardId shardId) { + return trackerMap.get(shardId); + } + + @Override + public void afterIndexShardCreated(IndexShard indexShard) { + if (indexShard.indexSettings().isRemoteStoreEnabled() == false) { + return; + } + ShardId shardId = indexShard.shardId(); + trackerMap.put(shardId, new RemoteRefreshSegmentPressureTracker(shardId, pressureSettings)); + logger.trace("Created tracker for shardId={}", shardId); + } + + @Override + public void beforeIndexShardClosed(ShardId shardId, IndexShard indexShard, Settings indexSettings) { + if (indexShard.indexSettings().isRemoteStoreEnabled() == false) { + return; + } + trackerMap.remove(shardId); + logger.trace("Deleted tracker for shardId={}", shardId); + } + + /** + * Check if remote refresh segments backpressure is enabled. This is backed by a cluster level setting. + * + * @return true if enabled, else false. + */ + public boolean isSegmentsUploadBackpressureEnabled() { + return pressureSettings.isRemoteRefreshSegmentPressureEnabled(); + } + + public void validateSegmentsUploadLag(ShardId shardId) { + RemoteRefreshSegmentPressureTracker pressureTracker = getStatsTracker(shardId); + // Check if refresh checkpoint (a.k.a. seq number) lag is 2 or below - this is to handle segment merges that can + // increase the bytes to upload almost suddenly. + if (pressureTracker.getSeqNoLag() <= 2) { + return; + } + + // Check if the remote store seq no lag is above the min seq no lag limit + validateSeqNoLag(pressureTracker, shardId); + + // Check if the remote store is lagging more than the upload bytes average multiplied by a variance factor + validateBytesLag(pressureTracker, shardId); + + // Check if the remote store is lagging more than the upload time average multiplied by a variance factor + validateTimeLag(pressureTracker, shardId); + + // Check if consecutive failure limit has been breached + validateConsecutiveFailureLimitBreached(pressureTracker, shardId); + } + + private void validateSeqNoLag(RemoteRefreshSegmentPressureTracker pressureTracker, ShardId shardId) { + // Check if the remote store seq no lag is above the min seq no lag limit + if (pressureTracker.getSeqNoLag() > pressureSettings.getMinSeqNoLagLimit()) { + pressureTracker.incrementRejectionCount(); + throw new OpenSearchRejectedExecutionException( + String.format( + Locale.ROOT, + "rejected execution on primary shard:%s due to remote segments lagging behind local segments." + + "remote_refresh_seq_no:%s local_refresh_seq_no:%s", + shardId, + pressureTracker.getRemoteRefreshSeqNo(), + pressureTracker.getLocalRefreshSeqNo() + ) + ); + } + } + + private void validateBytesLag(RemoteRefreshSegmentPressureTracker pressureTracker, ShardId shardId) { + if (pressureTracker.isUploadBytesAverageReady() == false) { + return; + } + double dynamicBytesLagThreshold = pressureTracker.getUploadBytesAverage() * pressureSettings.getBytesLagVarianceThreshold(); + long bytesLag = pressureTracker.getBytesLag(); + if (bytesLag > dynamicBytesLagThreshold) { + pressureTracker.incrementRejectionCount(); + throw new OpenSearchRejectedExecutionException( + String.format( + Locale.ROOT, + "rejected execution on primary shard:%s due to remote segments lagging behind local segments." + + "bytes_lag:%s dynamic_bytes_lag_threshold:%s", + shardId, + bytesLag, + dynamicBytesLagThreshold + ) + ); + } + } + + private void validateTimeLag(RemoteRefreshSegmentPressureTracker pressureTracker, ShardId shardId) { + if (pressureTracker.isUploadTimeMsAverageReady() == false) { + return; + } + long timeLag = pressureTracker.getTimeMsLag(); + double dynamicTimeLagThreshold = pressureTracker.getUploadTimeMsAverage() * pressureSettings.getTimeLagVarianceThreshold(); + if (timeLag > dynamicTimeLagThreshold) { + pressureTracker.incrementRejectionCount(); + throw new OpenSearchRejectedExecutionException( + String.format( + Locale.ROOT, + "rejected execution on primary shard:%s due to remote segments lagging behind local segments." + + "time_lag:%s ns dynamic_time_lag_threshold:%s ns", + shardId, + timeLag, + dynamicTimeLagThreshold + ) + ); + } + } + + private void validateConsecutiveFailureLimitBreached(RemoteRefreshSegmentPressureTracker pressureTracker, ShardId shardId) { + int failureStreakCount = pressureTracker.getConsecutiveFailureCount(); + int minConsecutiveFailureThreshold = pressureSettings.getMinConsecutiveFailuresLimit(); + if (failureStreakCount > minConsecutiveFailureThreshold) { + pressureTracker.incrementRejectionCount(); + throw new OpenSearchRejectedExecutionException( + String.format( + Locale.ROOT, + "rejected execution on primary shard:%s due to remote segments lagging behind local segments." + + "failure_streak_count:%s min_consecutive_failure_threshold:%s", + shardId, + failureStreakCount, + minConsecutiveFailureThreshold + ) + ); + } + } + + void updateUploadBytesMovingAverageWindowSize(int updatedSize) { + updateMovingAverageWindowSize(RemoteRefreshSegmentPressureTracker::updateUploadBytesMovingAverageWindowSize, updatedSize); + } + + void updateUploadBytesPerSecMovingAverageWindowSize(int updatedSize) { + updateMovingAverageWindowSize(RemoteRefreshSegmentPressureTracker::updateUploadBytesPerSecMovingAverageWindowSize, updatedSize); + } + + void updateUploadTimeMsMovingAverageWindowSize(int updatedSize) { + updateMovingAverageWindowSize(RemoteRefreshSegmentPressureTracker::updateUploadTimeMsMovingAverageWindowSize, updatedSize); + } + + void updateMovingAverageWindowSize(BiConsumer biConsumer, int updatedSize) { + trackerMap.values().forEach(tracker -> biConsumer.accept(tracker, updatedSize)); + } +} diff --git a/server/src/main/java/org/opensearch/index/RemoteRefreshSegmentPressureSettings.java b/server/src/main/java/org/opensearch/index/RemoteRefreshSegmentPressureSettings.java new file mode 100644 index 0000000000000..3039c7cae9413 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/RemoteRefreshSegmentPressureSettings.java @@ -0,0 +1,220 @@ +/* + * 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; + +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Setting; +import org.opensearch.common.settings.Settings; + +/** + * Settings related to back pressure on account of segments upload failures / lags. + */ +public class RemoteRefreshSegmentPressureSettings { + + private static class Defaults { + private static final long MIN_SEQ_NO_LAG_LIMIT = 5; + private static final double BYTES_LAG_VARIANCE_THRESHOLD = 2.0; + private static final double TIME_LAG_VARIANCE_THRESHOLD = 2.0; + private static final int MIN_CONSECUTIVE_FAILURES_LIMIT = 10; + private static final int UPLOAD_BYTES_MOVING_AVERAGE_WINDOW_SIZE = 20; + private static final int UPLOAD_BYTES_PER_SEC_MOVING_AVERAGE_WINDOW_SIZE = 20; + private static final int UPLOAD_TIME_MOVING_AVERAGE_WINDOW_SIZE = 20; + } + + public static final Setting REMOTE_REFRESH_SEGMENT_PRESSURE_ENABLED = Setting.boolSetting( + "remote_store.segment.pressure.enabled", + false, + Setting.Property.Dynamic, + Setting.Property.NodeScope + ); + + public static final Setting MIN_SEQ_NO_LAG_LIMIT = Setting.longSetting( + "remote_store.segment.pressure.seq_no_lag.limit", + Defaults.MIN_SEQ_NO_LAG_LIMIT, + 2L, + Setting.Property.Dynamic, + Setting.Property.NodeScope + ); + + public static final Setting BYTES_LAG_VARIANCE_THRESHOLD = Setting.doubleSetting( + "remote_store.segment.pressure.bytes_lag.variance", + Defaults.BYTES_LAG_VARIANCE_THRESHOLD, + 0.0, + Setting.Property.Dynamic, + Setting.Property.NodeScope + ); + + public static final Setting TIME_LAG_VARIANCE_THRESHOLD = Setting.doubleSetting( + "remote_store.segment.pressure.time_lag.variance", + Defaults.TIME_LAG_VARIANCE_THRESHOLD, + 0.0, + Setting.Property.Dynamic, + Setting.Property.NodeScope + ); + + public static final Setting MIN_CONSECUTIVE_FAILURES_LIMIT = Setting.intSetting( + "remote_store.segment.pressure.consecutive_failures.limit", + Defaults.MIN_CONSECUTIVE_FAILURES_LIMIT, + 1, + Setting.Property.Dynamic, + Setting.Property.NodeScope + ); + + public static final Setting UPLOAD_BYTES_MOVING_AVERAGE_WINDOW_SIZE = Setting.intSetting( + "remote_store.segment.pressure.upload_bytes_moving_average_window_size", + Defaults.UPLOAD_BYTES_MOVING_AVERAGE_WINDOW_SIZE, + 0, + Setting.Property.Dynamic, + Setting.Property.NodeScope + ); + + public static final Setting UPLOAD_BYTES_PER_SEC_MOVING_AVERAGE_WINDOW_SIZE = Setting.intSetting( + "remote_store.segment.pressure.upload_bytes_per_sec_moving_average_window_size", + Defaults.UPLOAD_BYTES_PER_SEC_MOVING_AVERAGE_WINDOW_SIZE, + 0, + Setting.Property.Dynamic, + Setting.Property.NodeScope + ); + + public static final Setting UPLOAD_TIME_MOVING_AVERAGE_WINDOW_SIZE = Setting.intSetting( + "remote_store.segment.pressure.upload_time_moving_average_window_size", + Defaults.UPLOAD_TIME_MOVING_AVERAGE_WINDOW_SIZE, + 0, + Setting.Property.Dynamic, + Setting.Property.NodeScope + ); + + private volatile boolean remoteRefreshSegmentPressureEnabled; + + private volatile long minSeqNoLagLimit; + + private volatile double bytesLagVarianceThreshold; + + private volatile double timeLagVarianceThreshold; + + private volatile int minConsecutiveFailuresLimit; + + private volatile int uploadBytesMovingAverageWindowSize; + + private volatile int uploadBytesPerSecMovingAverageWindowSize; + + private volatile int uploadTimeMovingAverageWindowSize; + + public RemoteRefreshSegmentPressureSettings( + ClusterService clusterService, + Settings settings, + RemoteRefreshSegmentPressureService remoteUploadPressureService + ) { + ClusterSettings clusterSettings = clusterService.getClusterSettings(); + + this.remoteRefreshSegmentPressureEnabled = REMOTE_REFRESH_SEGMENT_PRESSURE_ENABLED.get(settings); + clusterSettings.addSettingsUpdateConsumer(REMOTE_REFRESH_SEGMENT_PRESSURE_ENABLED, this::setRemoteRefreshSegmentPressureEnabled); + + this.minSeqNoLagLimit = MIN_SEQ_NO_LAG_LIMIT.get(settings); + clusterSettings.addSettingsUpdateConsumer(MIN_SEQ_NO_LAG_LIMIT, this::setMinSeqNoLagLimit); + + this.bytesLagVarianceThreshold = BYTES_LAG_VARIANCE_THRESHOLD.get(settings); + clusterSettings.addSettingsUpdateConsumer(BYTES_LAG_VARIANCE_THRESHOLD, this::setBytesLagVarianceThreshold); + + this.timeLagVarianceThreshold = TIME_LAG_VARIANCE_THRESHOLD.get(settings); + clusterSettings.addSettingsUpdateConsumer(TIME_LAG_VARIANCE_THRESHOLD, this::setTimeLagVarianceThreshold); + + this.minConsecutiveFailuresLimit = MIN_CONSECUTIVE_FAILURES_LIMIT.get(settings); + clusterSettings.addSettingsUpdateConsumer(MIN_CONSECUTIVE_FAILURES_LIMIT, this::setMinConsecutiveFailuresLimit); + + this.uploadBytesMovingAverageWindowSize = UPLOAD_BYTES_MOVING_AVERAGE_WINDOW_SIZE.get(settings); + clusterSettings.addSettingsUpdateConsumer( + UPLOAD_BYTES_MOVING_AVERAGE_WINDOW_SIZE, + remoteUploadPressureService::updateUploadBytesMovingAverageWindowSize + ); + clusterSettings.addSettingsUpdateConsumer(UPLOAD_BYTES_MOVING_AVERAGE_WINDOW_SIZE, this::setUploadBytesMovingAverageWindowSize); + + this.uploadBytesPerSecMovingAverageWindowSize = UPLOAD_BYTES_PER_SEC_MOVING_AVERAGE_WINDOW_SIZE.get(settings); + clusterSettings.addSettingsUpdateConsumer( + UPLOAD_BYTES_PER_SEC_MOVING_AVERAGE_WINDOW_SIZE, + remoteUploadPressureService::updateUploadBytesPerSecMovingAverageWindowSize + ); + clusterSettings.addSettingsUpdateConsumer( + UPLOAD_BYTES_PER_SEC_MOVING_AVERAGE_WINDOW_SIZE, + this::setUploadBytesPerSecMovingAverageWindowSize + ); + + this.uploadTimeMovingAverageWindowSize = UPLOAD_TIME_MOVING_AVERAGE_WINDOW_SIZE.get(settings); + clusterSettings.addSettingsUpdateConsumer( + UPLOAD_TIME_MOVING_AVERAGE_WINDOW_SIZE, + remoteUploadPressureService::updateUploadTimeMsMovingAverageWindowSize + ); + clusterSettings.addSettingsUpdateConsumer(UPLOAD_TIME_MOVING_AVERAGE_WINDOW_SIZE, this::setUploadTimeMovingAverageWindowSize); + } + + public boolean isRemoteRefreshSegmentPressureEnabled() { + return remoteRefreshSegmentPressureEnabled; + } + + public void setRemoteRefreshSegmentPressureEnabled(boolean remoteRefreshSegmentPressureEnabled) { + this.remoteRefreshSegmentPressureEnabled = remoteRefreshSegmentPressureEnabled; + } + + public long getMinSeqNoLagLimit() { + return minSeqNoLagLimit; + } + + public void setMinSeqNoLagLimit(long minSeqNoLagLimit) { + this.minSeqNoLagLimit = minSeqNoLagLimit; + } + + public double getBytesLagVarianceThreshold() { + return bytesLagVarianceThreshold; + } + + public void setBytesLagVarianceThreshold(double bytesLagVarianceThreshold) { + this.bytesLagVarianceThreshold = bytesLagVarianceThreshold; + } + + public double getTimeLagVarianceThreshold() { + return timeLagVarianceThreshold; + } + + public void setTimeLagVarianceThreshold(double timeLagVarianceThreshold) { + this.timeLagVarianceThreshold = timeLagVarianceThreshold; + } + + public int getMinConsecutiveFailuresLimit() { + return minConsecutiveFailuresLimit; + } + + public void setMinConsecutiveFailuresLimit(int minConsecutiveFailuresLimit) { + this.minConsecutiveFailuresLimit = minConsecutiveFailuresLimit; + } + + public int getUploadBytesMovingAverageWindowSize() { + return uploadBytesMovingAverageWindowSize; + } + + public void setUploadBytesMovingAverageWindowSize(int uploadBytesMovingAverageWindowSize) { + this.uploadBytesMovingAverageWindowSize = uploadBytesMovingAverageWindowSize; + } + + public int getUploadBytesPerSecMovingAverageWindowSize() { + return uploadBytesPerSecMovingAverageWindowSize; + } + + public void setUploadBytesPerSecMovingAverageWindowSize(int uploadBytesPerSecMovingAverageWindowSize) { + this.uploadBytesPerSecMovingAverageWindowSize = uploadBytesPerSecMovingAverageWindowSize; + } + + public int getUploadTimeMovingAverageWindowSize() { + return uploadTimeMovingAverageWindowSize; + } + + public void setUploadTimeMovingAverageWindowSize(int uploadTimeMovingAverageWindowSize) { + this.uploadTimeMovingAverageWindowSize = uploadTimeMovingAverageWindowSize; + } +} diff --git a/server/src/main/java/org/opensearch/index/RemoteRefreshSegmentPressureTracker.java b/server/src/main/java/org/opensearch/index/RemoteRefreshSegmentPressureTracker.java new file mode 100644 index 0000000000000..3fc6e6ce41e29 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/RemoteRefreshSegmentPressureTracker.java @@ -0,0 +1,353 @@ +/* + * 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; + +import org.opensearch.common.util.MovingAverage; +import org.opensearch.common.util.Streak; +import org.opensearch.index.shard.ShardId; + +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; + +/** + * Keeps track of remote refresh which happens in {@link org.opensearch.index.shard.RemoteStoreRefreshListener}. This consist of multiple critical metrics. + */ +public class RemoteRefreshSegmentPressureTracker { + + RemoteRefreshSegmentPressureTracker(ShardId shardId, RemoteRefreshSegmentPressureSettings remoteUploadPressureSettings) { + 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 currentTimeMs = System.nanoTime() / 1_000_000L; + localRefreshTimeMs.set(currentTimeMs); + remoteRefreshTimeMs.set(currentTimeMs); + uploadBytesMovingAverageReference = new AtomicReference<>( + new MovingAverage(remoteUploadPressureSettings.getUploadBytesMovingAverageWindowSize()) + ); + uploadBytesPerSecMovingAverageReference = new AtomicReference<>( + new MovingAverage(remoteUploadPressureSettings.getUploadBytesPerSecMovingAverageWindowSize()) + ); + uploadTimeMsMovingAverageReference = new AtomicReference<>( + new MovingAverage(remoteUploadPressureSettings.getUploadTimeMovingAverageWindowSize()) + ); + } + + /** + * 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. + */ + private final AtomicLong localRefreshSeqNo = new AtomicLong(); + + /** + * The refresh time of the most recent refresh. + */ + private final AtomicLong localRefreshTimeMs = new AtomicLong(); + + /** + * Sequence number of the most recent remote refresh. + */ + private final AtomicLong remoteRefreshSeqNo = new AtomicLong(); + + /** + * The refresh time of most recent remote refresh. + */ + private final AtomicLong remoteRefreshTimeMs = new AtomicLong(); + + /** + * Keeps the seq no lag computed so that we do not compute it for every request. + */ + private final AtomicLong seqNoLag = new AtomicLong(); + + /** + * Keeps the time (ms) lag computed so that we do not compute it for every request. + */ + private final AtomicLong timeMsLag = new AtomicLong(); + + /** + * 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 final AtomicLong totalUploadsStarted = new AtomicLong(); + + /** + * Cumulative sum of count of remote refreshes that have failed. + */ + private final AtomicLong totalUploadsFailed = new AtomicLong(); + + /** + * Cumulative sum of count of remote refreshes that have succeeded. + */ + private final AtomicLong totalUploadsSucceeded = new AtomicLong(); + + /** + * Cumulative sum of rejection counts for this shard. + */ + private final AtomicLong rejectionCount = new AtomicLong(); + + /** + * Map of name to size of the segment files created as part of the most recent refresh. + */ + private volatile Map latestLocalFileNameLengthMap; + + /** + * Set of names of segment files that were uploaded as part of the most recent remote refresh. + */ + private volatile Set latestUploadFiles; + + /** + * Keeps the bytes lag computed so that we do not compute it for every request. + */ + private final AtomicLong bytesLag = new AtomicLong(); + + /** + * Holds count of consecutive failures until last success. Gets reset to zero if there is a success. + */ + 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; + + /** + * 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; + + /** + * 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; + + ShardId getShardId() { + return shardId; + } + + AtomicLong getLocalRefreshSeqNo() { + return localRefreshSeqNo; + } + + void updateLocalRefreshSeqNo(long localRefreshSeqNo) { + this.localRefreshSeqNo.set(localRefreshSeqNo); + computeSeqNoLag(); + } + + AtomicLong getLocalRefreshTimeMs() { + return localRefreshTimeMs; + } + + void updateLocalRefreshTimeMs(long localRefreshTimeMs) { + this.localRefreshTimeMs.set(localRefreshTimeMs); + computeTimeMsLag(); + } + + AtomicLong getRemoteRefreshSeqNo() { + return remoteRefreshSeqNo; + } + + void updateRemoteRefreshSeqNo(long remoteRefreshSeqNo) { + this.remoteRefreshSeqNo.set(remoteRefreshSeqNo); + computeSeqNoLag(); + } + + AtomicLong getRemoteRefreshTimeMs() { + return remoteRefreshTimeMs; + } + + void updateRemoteRefreshTimeMs(long remoteRefreshTimeMs) { + this.remoteRefreshTimeMs.set(remoteRefreshTimeMs); + computeTimeMsLag(); + } + + private void computeSeqNoLag() { + seqNoLag.set(localRefreshSeqNo.get() - remoteRefreshSeqNo.get()); + } + + long getSeqNoLag() { + return seqNoLag.get(); + } + + private void computeTimeMsLag() { + timeMsLag.set(localRefreshTimeMs.get() - remoteRefreshTimeMs.get()); + } + + long getTimeMsLag() { + return timeMsLag.get(); + } + + long getBytesLag() { + return bytesLag.get(); + } + + AtomicLong getUploadBytesStarted() { + return uploadBytesStarted; + } + + void addUploadBytesStarted(long size) { + uploadBytesStarted.addAndGet(size); + } + + AtomicLong getUploadBytesFailed() { + return uploadBytesFailed; + } + + void addUploadBytesFailed(long size) { + uploadBytesFailed.addAndGet(size); + } + + AtomicLong getUploadBytesSucceeded() { + return uploadBytesSucceeded; + } + + void addUploadBytesSucceeded(long size) { + uploadBytesSucceeded.addAndGet(size); + } + + long getInflightUploadBytes() { + return uploadBytesStarted.get() - uploadBytesFailed.get() - uploadBytesSucceeded.get(); + } + + AtomicLong getTotalUploadsStarted() { + return totalUploadsStarted; + } + + void incrementTotalUploadsStarted() { + totalUploadsStarted.incrementAndGet(); + } + + public AtomicLong getTotalUploadsFailed() { + return totalUploadsFailed; + } + + void incrementTotalUploadsFailed() { + totalUploadsFailed.incrementAndGet(); + failures.record(true); + } + + AtomicLong getTotalUploadsSucceeded() { + return totalUploadsSucceeded; + } + + void incrementTotalUploadSucceeded() { + totalUploadsSucceeded.incrementAndGet(); + failures.record(false); + } + + long getInflightUploads() { + return totalUploadsStarted.get() - totalUploadsFailed.get() - totalUploadsSucceeded.get(); + } + + AtomicLong getRejectionCount() { + return rejectionCount; + } + + void incrementRejectionCount() { + rejectionCount.incrementAndGet(); + } + + Map getLatestLocalFileNameLengthMap() { + return latestLocalFileNameLengthMap; + } + + void setLatestLocalFileNameLengthMap(Map latestLocalFileNameLengthMap) { + this.latestLocalFileNameLengthMap = latestLocalFileNameLengthMap; + computeBytesLag(); + } + + void addToLatestUploadFiles(String file) { + this.latestUploadFiles.add(file); + computeBytesLag(); + } + + private void computeBytesLag() { + if (latestLocalFileNameLengthMap == null || latestLocalFileNameLengthMap.isEmpty()) { + return; + } + Set filesNotYetUploaded = latestLocalFileNameLengthMap.keySet() + .stream() + .filter(f -> latestUploadFiles == null || latestUploadFiles.contains(f) == false) + .collect(Collectors.toSet()); + long bytesLag = filesNotYetUploaded.stream().map(latestLocalFileNameLengthMap::get).mapToLong(Long::longValue).sum(); + this.bytesLag.set(bytesLag); + } + + int getConsecutiveFailureCount() { + return failures.length(); + } + + boolean isUploadBytesAverageReady() { + return uploadBytesMovingAverageReference.get().isReady(); + } + + double getUploadBytesAverage() { + return uploadBytesMovingAverageReference.get().getAverage(); + } + + /** + * Updates the window size for data collection of upload bytes. This also resets any data collected so far. + * + * @param updatedSize the updated size + */ + void updateUploadBytesMovingAverageWindowSize(int updatedSize) { + this.uploadBytesMovingAverageReference.set(new MovingAverage(updatedSize)); + } + + double getUploadBytesPerSecAverage() { + return uploadBytesPerSecMovingAverageReference.get().getAverage(); + } + + /** + * Updates the window size for data collection of upload bytes per second. This also resets any data collected so far. + * + * @param updatedSize the updated size + */ + void updateUploadBytesPerSecMovingAverageWindowSize(int updatedSize) { + this.uploadBytesPerSecMovingAverageReference.set(new MovingAverage(updatedSize)); + } + + boolean isUploadTimeMsAverageReady() { + return uploadTimeMsMovingAverageReference.get().isReady(); + } + + double getUploadTimeMsAverage() { + return uploadTimeMsMovingAverageReference.get().getAverage(); + } + + /** + * Updates the window size for data collection of upload time (ms). This also resets any data collected so far. + * + * @param updatedSize the updated size + */ + void updateUploadTimeMsMovingAverageWindowSize(int updatedSize) { + this.uploadTimeMsMovingAverageReference.set(new MovingAverage(updatedSize)); + } +} From b7ba0495ba99aed02c56d19c1e79c439c967709e Mon Sep 17 00:00:00 2001 From: Ashish Singh Date: Wed, 19 Apr 2023 00:01:49 +0530 Subject: [PATCH 02/17] Register settings and add unit tests Signed-off-by: Ashish Singh --- .../common/settings/ClusterSettings.java | 13 +- ...oteRefreshSegmentPressureSettingsTest.java | 268 ++++++++++++++++++ 2 files changed, 280 insertions(+), 1 deletion(-) create mode 100644 server/src/test/java/org/opensearch/index/RemoteRefreshSegmentPressureSettingsTest.java diff --git a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java index de3fd1c9380b9..4c3272e53379c 100644 --- a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java @@ -39,6 +39,7 @@ import org.opensearch.index.IndexModule; import org.opensearch.index.IndexSettings; import org.opensearch.index.IndexingPressure; +import org.opensearch.index.RemoteRefreshSegmentPressureSettings; import org.opensearch.index.SegmentReplicationPressureService; import org.opensearch.index.ShardIndexingPressureMemoryManager; import org.opensearch.index.ShardIndexingPressureSettings; @@ -635,7 +636,17 @@ public void apply(Settings value, Settings current, Settings previous) { SegmentReplicationPressureService.MAX_ALLOWED_STALE_SHARDS, // Settings related to Searchable Snapshots - Node.NODE_SEARCH_CACHE_SIZE_SETTING + Node.NODE_SEARCH_CACHE_SIZE_SETTING, + + // Settings related to Remote Refresh Segment Pressure + RemoteRefreshSegmentPressureSettings.REMOTE_REFRESH_SEGMENT_PRESSURE_ENABLED, + RemoteRefreshSegmentPressureSettings.MIN_SEQ_NO_LAG_LIMIT, + RemoteRefreshSegmentPressureSettings.BYTES_LAG_VARIANCE_THRESHOLD, + RemoteRefreshSegmentPressureSettings.TIME_LAG_VARIANCE_THRESHOLD, + RemoteRefreshSegmentPressureSettings.MIN_CONSECUTIVE_FAILURES_LIMIT, + RemoteRefreshSegmentPressureSettings.UPLOAD_BYTES_MOVING_AVERAGE_WINDOW_SIZE, + RemoteRefreshSegmentPressureSettings.UPLOAD_BYTES_PER_SEC_MOVING_AVERAGE_WINDOW_SIZE, + RemoteRefreshSegmentPressureSettings.UPLOAD_TIME_MOVING_AVERAGE_WINDOW_SIZE ) ) ); diff --git a/server/src/test/java/org/opensearch/index/RemoteRefreshSegmentPressureSettingsTest.java b/server/src/test/java/org/opensearch/index/RemoteRefreshSegmentPressureSettingsTest.java new file mode 100644 index 0000000000000..2f58403584a27 --- /dev/null +++ b/server/src/test/java/org/opensearch/index/RemoteRefreshSegmentPressureSettingsTest.java @@ -0,0 +1,268 @@ +/* + * 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; + +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; +import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.threadpool.TestThreadPool; +import org.opensearch.threadpool.ThreadPool; + +import java.util.concurrent.atomic.AtomicInteger; + +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; + +public class RemoteRefreshSegmentPressureSettingsTest extends OpenSearchTestCase { + + private ClusterService clusterService; + + private ThreadPool threadPool; + + @Override + public void setUp() throws Exception { + super.setUp(); + threadPool = new TestThreadPool("remote_refresh_segment_pressure_settings_test"); + clusterService = new ClusterService( + Settings.EMPTY, + new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), + threadPool + ); + } + + @Override + public void tearDown() throws Exception { + super.tearDown(); + threadPool.shutdownNow(); + } + + public void testGetDefaultSettings() { + RemoteRefreshSegmentPressureSettings pressureSettings = new RemoteRefreshSegmentPressureSettings( + clusterService, + Settings.EMPTY, + mock(RemoteRefreshSegmentPressureService.class) + ); + + // Check remote refresh segment pressure enabled is false + assertFalse(pressureSettings.isRemoteRefreshSegmentPressureEnabled()); + + // Check min sequence number lag limit default value + assertEquals(5L, pressureSettings.getMinSeqNoLagLimit()); + + // Check bytes lag variance threshold default value + assertEquals(2.0, pressureSettings.getBytesLagVarianceThreshold(), 0.0d); + + // Check time lag variance threshold default value + assertEquals(2.0, pressureSettings.getTimeLagVarianceThreshold(), 0.0d); + + // Check minimum consecutive failures limit default value + assertEquals(10, pressureSettings.getMinConsecutiveFailuresLimit()); + + // Check upload bytes moving average window size default value + assertEquals(20, pressureSettings.getUploadBytesMovingAverageWindowSize()); + + // Check upload bytes per sec moving average window size default value + assertEquals(20, pressureSettings.getUploadBytesPerSecMovingAverageWindowSize()); + + // Check upload time moving average window size default value + assertEquals(20, pressureSettings.getUploadTimeMovingAverageWindowSize()); + } + + public void testGetConfiguredSettings() { + Settings settings = Settings.builder() + .put(RemoteRefreshSegmentPressureSettings.REMOTE_REFRESH_SEGMENT_PRESSURE_ENABLED.getKey(), true) + .put(RemoteRefreshSegmentPressureSettings.MIN_SEQ_NO_LAG_LIMIT.getKey(), 100) + .put(RemoteRefreshSegmentPressureSettings.BYTES_LAG_VARIANCE_THRESHOLD.getKey(), 50.0) + .put(RemoteRefreshSegmentPressureSettings.TIME_LAG_VARIANCE_THRESHOLD.getKey(), 60.0) + .put(RemoteRefreshSegmentPressureSettings.MIN_CONSECUTIVE_FAILURES_LIMIT.getKey(), 121) + .put(RemoteRefreshSegmentPressureSettings.UPLOAD_BYTES_MOVING_AVERAGE_WINDOW_SIZE.getKey(), 102) + .put(RemoteRefreshSegmentPressureSettings.UPLOAD_BYTES_PER_SEC_MOVING_AVERAGE_WINDOW_SIZE.getKey(), 103) + .put(RemoteRefreshSegmentPressureSettings.UPLOAD_TIME_MOVING_AVERAGE_WINDOW_SIZE.getKey(), 104) + .build(); + RemoteRefreshSegmentPressureSettings pressureSettings = new RemoteRefreshSegmentPressureSettings( + clusterService, + settings, + mock(RemoteRefreshSegmentPressureService.class) + ); + + // Check remote refresh segment pressure enabled is true + assertTrue(pressureSettings.isRemoteRefreshSegmentPressureEnabled()); + + // Check min sequence number lag limit configured value + assertEquals(100L, pressureSettings.getMinSeqNoLagLimit()); + + // Check bytes lag variance threshold configured value + assertEquals(50.0, pressureSettings.getBytesLagVarianceThreshold(), 0.0d); + + // Check time lag variance threshold configured value + assertEquals(60.0, pressureSettings.getTimeLagVarianceThreshold(), 0.0d); + + // Check minimum consecutive failures limit configured value + assertEquals(121, pressureSettings.getMinConsecutiveFailuresLimit()); + + // Check upload bytes moving average window size configured value + assertEquals(102, pressureSettings.getUploadBytesMovingAverageWindowSize()); + + // Check upload bytes per sec moving average window size configured value + assertEquals(103, pressureSettings.getUploadBytesPerSecMovingAverageWindowSize()); + + // Check upload time moving average window size configured value + assertEquals(104, pressureSettings.getUploadTimeMovingAverageWindowSize()); + } + + public void testUpdateAfterGetDefaultSettings() { + RemoteRefreshSegmentPressureSettings pressureSettings = new RemoteRefreshSegmentPressureSettings( + clusterService, + Settings.EMPTY, + mock(RemoteRefreshSegmentPressureService.class) + ); + + Settings newSettings = Settings.builder() + .put(RemoteRefreshSegmentPressureSettings.REMOTE_REFRESH_SEGMENT_PRESSURE_ENABLED.getKey(), true) + .put(RemoteRefreshSegmentPressureSettings.MIN_SEQ_NO_LAG_LIMIT.getKey(), 100) + .put(RemoteRefreshSegmentPressureSettings.BYTES_LAG_VARIANCE_THRESHOLD.getKey(), 50.0) + .put(RemoteRefreshSegmentPressureSettings.TIME_LAG_VARIANCE_THRESHOLD.getKey(), 60.0) + .put(RemoteRefreshSegmentPressureSettings.MIN_CONSECUTIVE_FAILURES_LIMIT.getKey(), 121) + .put(RemoteRefreshSegmentPressureSettings.UPLOAD_BYTES_MOVING_AVERAGE_WINDOW_SIZE.getKey(), 102) + .put(RemoteRefreshSegmentPressureSettings.UPLOAD_BYTES_PER_SEC_MOVING_AVERAGE_WINDOW_SIZE.getKey(), 103) + .put(RemoteRefreshSegmentPressureSettings.UPLOAD_TIME_MOVING_AVERAGE_WINDOW_SIZE.getKey(), 104) + .build(); + clusterService.getClusterSettings().applySettings(newSettings); + + // Check updated remote refresh segment pressure enabled is false + assertTrue(pressureSettings.isRemoteRefreshSegmentPressureEnabled()); + + // Check min sequence number lag limit + assertEquals(100L, pressureSettings.getMinSeqNoLagLimit()); + + // Check bytes lag variance threshold updated + assertEquals(50.0, pressureSettings.getBytesLagVarianceThreshold(), 0.0d); + + // Check time lag variance threshold updated + assertEquals(60.0, pressureSettings.getTimeLagVarianceThreshold(), 0.0d); + + // Check minimum consecutive failures limit updated + assertEquals(121, pressureSettings.getMinConsecutiveFailuresLimit()); + + // Check upload bytes moving average window size updated + assertEquals(102, pressureSettings.getUploadBytesMovingAverageWindowSize()); + + // Check upload bytes per sec moving average window size updated + assertEquals(103, pressureSettings.getUploadBytesPerSecMovingAverageWindowSize()); + + // Check upload time moving average window size updated + assertEquals(104, pressureSettings.getUploadTimeMovingAverageWindowSize()); + } + + public void testUpdateAfterGetConfiguredSettings() { + Settings settings = Settings.builder() + .put(RemoteRefreshSegmentPressureSettings.REMOTE_REFRESH_SEGMENT_PRESSURE_ENABLED.getKey(), true) + .put(RemoteRefreshSegmentPressureSettings.MIN_SEQ_NO_LAG_LIMIT.getKey(), 100) + .put(RemoteRefreshSegmentPressureSettings.BYTES_LAG_VARIANCE_THRESHOLD.getKey(), 50.0) + .put(RemoteRefreshSegmentPressureSettings.TIME_LAG_VARIANCE_THRESHOLD.getKey(), 60.0) + .put(RemoteRefreshSegmentPressureSettings.MIN_CONSECUTIVE_FAILURES_LIMIT.getKey(), 121) + .put(RemoteRefreshSegmentPressureSettings.UPLOAD_BYTES_MOVING_AVERAGE_WINDOW_SIZE.getKey(), 102) + .put(RemoteRefreshSegmentPressureSettings.UPLOAD_BYTES_PER_SEC_MOVING_AVERAGE_WINDOW_SIZE.getKey(), 103) + .put(RemoteRefreshSegmentPressureSettings.UPLOAD_TIME_MOVING_AVERAGE_WINDOW_SIZE.getKey(), 104) + .build(); + RemoteRefreshSegmentPressureSettings pressureSettings = new RemoteRefreshSegmentPressureSettings( + clusterService, + settings, + mock(RemoteRefreshSegmentPressureService.class) + ); + + Settings newSettings = Settings.builder() + .put(RemoteRefreshSegmentPressureSettings.MIN_SEQ_NO_LAG_LIMIT.getKey(), 80) + .put(RemoteRefreshSegmentPressureSettings.BYTES_LAG_VARIANCE_THRESHOLD.getKey(), 40.0) + .put(RemoteRefreshSegmentPressureSettings.TIME_LAG_VARIANCE_THRESHOLD.getKey(), 50.0) + .put(RemoteRefreshSegmentPressureSettings.MIN_CONSECUTIVE_FAILURES_LIMIT.getKey(), 111) + .put(RemoteRefreshSegmentPressureSettings.UPLOAD_BYTES_MOVING_AVERAGE_WINDOW_SIZE.getKey(), 112) + .put(RemoteRefreshSegmentPressureSettings.UPLOAD_BYTES_PER_SEC_MOVING_AVERAGE_WINDOW_SIZE.getKey(), 113) + .put(RemoteRefreshSegmentPressureSettings.UPLOAD_TIME_MOVING_AVERAGE_WINDOW_SIZE.getKey(), 114) + .build(); + + clusterService.getClusterSettings().applySettings(newSettings); + + // Check updated remote refresh segment pressure enabled is true + assertTrue(pressureSettings.isRemoteRefreshSegmentPressureEnabled()); + + // Check min sequence number lag limit + assertEquals(80L, pressureSettings.getMinSeqNoLagLimit()); + + // Check bytes lag variance threshold updated + assertEquals(40.0, pressureSettings.getBytesLagVarianceThreshold(), 0.0d); + + // Check time lag variance threshold updated + assertEquals(50.0, pressureSettings.getTimeLagVarianceThreshold(), 0.0d); + + // Check minimum consecutive failures limit updated + assertEquals(111, pressureSettings.getMinConsecutiveFailuresLimit()); + + // Check upload bytes moving average window size updated + assertEquals(112, pressureSettings.getUploadBytesMovingAverageWindowSize()); + + // Check upload bytes per sec moving average window size updated + assertEquals(113, pressureSettings.getUploadBytesPerSecMovingAverageWindowSize()); + + // Check upload time moving average window size updated + assertEquals(114, pressureSettings.getUploadTimeMovingAverageWindowSize()); + } + + public void testUpdateTriggeredInRemotePressureServiceOnUpdateSettings() { + + int toUpdateVal1 = 1121, toUpdateVal2 = 1123, toUpdateVal3 = 1125; + + AtomicInteger updatedUploadBytesWindowSize = new AtomicInteger(); + AtomicInteger updatedUploadBytesPerSecWindowSize = new AtomicInteger(); + AtomicInteger updatedUploadTimeWindowSize = new AtomicInteger(); + + + RemoteRefreshSegmentPressureService pressureService = mock(RemoteRefreshSegmentPressureService.class); + + // Upload bytes + doAnswer(invocation -> { + updatedUploadBytesWindowSize.set(invocation.getArgument(0)); + return null; + }).when(pressureService).updateUploadBytesMovingAverageWindowSize(anyInt()); + + // Upload bytes per sec + doAnswer(invocation -> { + updatedUploadBytesPerSecWindowSize.set(invocation.getArgument(0)); + return null; + }).when(pressureService).updateUploadBytesPerSecMovingAverageWindowSize(anyInt()); + + // Upload time + doAnswer(invocation -> { + updatedUploadTimeWindowSize.set(invocation.getArgument(0)); + return null; + }).when(pressureService).updateUploadTimeMsMovingAverageWindowSize(anyInt()); + + RemoteRefreshSegmentPressureSettings pressureSettings = new RemoteRefreshSegmentPressureSettings( + clusterService, + Settings.EMPTY, + pressureService + ); + Settings newSettings = Settings.builder() + .put(RemoteRefreshSegmentPressureSettings.UPLOAD_BYTES_MOVING_AVERAGE_WINDOW_SIZE.getKey(), toUpdateVal1) + .put(RemoteRefreshSegmentPressureSettings.UPLOAD_BYTES_PER_SEC_MOVING_AVERAGE_WINDOW_SIZE.getKey(), toUpdateVal2) + .put(RemoteRefreshSegmentPressureSettings.UPLOAD_TIME_MOVING_AVERAGE_WINDOW_SIZE.getKey(), toUpdateVal3) + .build(); + clusterService.getClusterSettings().applySettings(newSettings); + + // Assertions + assertEquals(toUpdateVal1, pressureSettings.getUploadBytesMovingAverageWindowSize()); + assertEquals(toUpdateVal1, updatedUploadBytesWindowSize.get()); + assertEquals(toUpdateVal2, pressureSettings.getUploadBytesPerSecMovingAverageWindowSize()); + assertEquals(toUpdateVal2, updatedUploadBytesPerSecWindowSize.get()); + assertEquals(toUpdateVal3, pressureSettings.getUploadTimeMovingAverageWindowSize()); + assertEquals(toUpdateVal3, updatedUploadTimeWindowSize.get()); + } +} From 8a76d1be7c00a7c8f0a9ea5470739b1626957d19 Mon Sep 17 00:00:00 2001 From: Ashish Singh Date: Wed, 19 Apr 2023 01:25:10 +0530 Subject: [PATCH 03/17] Add pressure tracker tests Signed-off-by: Ashish Singh --- .../RemoteRefreshSegmentPressureService.java | 1 + .../RemoteRefreshSegmentPressureTracker.java | 41 ++++++-- ...eRefreshSegmentPressureSettingsTests.java} | 3 +- ...oteRefreshSegmentPressureTrackerTests.java | 98 +++++++++++++++++++ 4 files changed, 131 insertions(+), 12 deletions(-) rename server/src/test/java/org/opensearch/index/{RemoteRefreshSegmentPressureSettingsTest.java => RemoteRefreshSegmentPressureSettingsTests.java} (99%) create mode 100644 server/src/test/java/org/opensearch/index/RemoteRefreshSegmentPressureTrackerTests.java diff --git a/server/src/main/java/org/opensearch/index/RemoteRefreshSegmentPressureService.java b/server/src/main/java/org/opensearch/index/RemoteRefreshSegmentPressureService.java index ab3584f3b8178..d1c75519831e0 100644 --- a/server/src/main/java/org/opensearch/index/RemoteRefreshSegmentPressureService.java +++ b/server/src/main/java/org/opensearch/index/RemoteRefreshSegmentPressureService.java @@ -27,6 +27,7 @@ * Service used to validate if the incoming indexing request should be rejected based on the {@link RemoteRefreshSegmentPressureTracker}. */ public class RemoteRefreshSegmentPressureService implements IndexEventListener { + private static final Logger logger = LogManager.getLogger(RemoteRefreshSegmentPressureService.class); /** diff --git a/server/src/main/java/org/opensearch/index/RemoteRefreshSegmentPressureTracker.java b/server/src/main/java/org/opensearch/index/RemoteRefreshSegmentPressureTracker.java index 3fc6e6ce41e29..7a272a63b790d 100644 --- a/server/src/main/java/org/opensearch/index/RemoteRefreshSegmentPressureTracker.java +++ b/server/src/main/java/org/opensearch/index/RemoteRefreshSegmentPressureTracker.java @@ -10,6 +10,7 @@ import org.opensearch.common.util.MovingAverage; import org.opensearch.common.util.Streak; +import org.opensearch.common.util.concurrent.ConcurrentCollections; import org.opensearch.index.shard.ShardId; import java.util.Map; @@ -118,7 +119,7 @@ public class RemoteRefreshSegmentPressureTracker { /** * Set of names of segment files that were uploaded as part of the most recent remote refresh. */ - private volatile Set latestUploadFiles; + private final Set latestUploadFiles = ConcurrentCollections.newConcurrentSet(); /** * Keeps the bytes lag computed so that we do not compute it for every request. @@ -152,38 +153,58 @@ ShardId getShardId() { return shardId; } - AtomicLong getLocalRefreshSeqNo() { - return localRefreshSeqNo; + long getLocalRefreshSeqNo() { + return localRefreshSeqNo.get(); } void updateLocalRefreshSeqNo(long localRefreshSeqNo) { + assert localRefreshSeqNo > this.localRefreshSeqNo.get() : "newLocalRefreshSeqNo=" + + localRefreshSeqNo + + ">=" + + "currentLocalRefreshSeqNo=" + + this.localRefreshSeqNo.get(); this.localRefreshSeqNo.set(localRefreshSeqNo); computeSeqNoLag(); } - AtomicLong getLocalRefreshTimeMs() { - return localRefreshTimeMs; + long getLocalRefreshTimeMs() { + return localRefreshTimeMs.get(); } void updateLocalRefreshTimeMs(long localRefreshTimeMs) { + assert localRefreshTimeMs > this.localRefreshTimeMs.get() : "newLocalRefreshTimeMs=" + + localRefreshTimeMs + + ">=" + + "currentLocalRefreshTimeMs=" + + this.localRefreshTimeMs.get(); this.localRefreshTimeMs.set(localRefreshTimeMs); computeTimeMsLag(); } - AtomicLong getRemoteRefreshSeqNo() { - return remoteRefreshSeqNo; + long getRemoteRefreshSeqNo() { + return remoteRefreshSeqNo.get(); } void updateRemoteRefreshSeqNo(long remoteRefreshSeqNo) { + assert remoteRefreshSeqNo > this.remoteRefreshSeqNo.get() : "newRemoteRefreshSeqNo=" + + remoteRefreshSeqNo + + ">=" + + "currentRemoteRefreshSeqNo=" + + this.remoteRefreshSeqNo.get(); this.remoteRefreshSeqNo.set(remoteRefreshSeqNo); computeSeqNoLag(); } - AtomicLong getRemoteRefreshTimeMs() { - return remoteRefreshTimeMs; + long getRemoteRefreshTimeMs() { + return remoteRefreshTimeMs.get(); } void updateRemoteRefreshTimeMs(long remoteRefreshTimeMs) { + assert remoteRefreshTimeMs > this.remoteRefreshTimeMs.get() : "newRemoteRefreshTimeMs=" + + remoteRefreshTimeMs + + ">=" + + "currentRemoteRefreshTimeMs=" + + this.remoteRefreshTimeMs.get(); this.remoteRefreshTimeMs.set(remoteRefreshTimeMs); computeTimeMsLag(); } @@ -294,7 +315,7 @@ private void computeBytesLag() { } Set filesNotYetUploaded = latestLocalFileNameLengthMap.keySet() .stream() - .filter(f -> latestUploadFiles == null || latestUploadFiles.contains(f) == false) + .filter(f -> !latestUploadFiles.contains(f)) .collect(Collectors.toSet()); long bytesLag = filesNotYetUploaded.stream().map(latestLocalFileNameLengthMap::get).mapToLong(Long::longValue).sum(); this.bytesLag.set(bytesLag); diff --git a/server/src/test/java/org/opensearch/index/RemoteRefreshSegmentPressureSettingsTest.java b/server/src/test/java/org/opensearch/index/RemoteRefreshSegmentPressureSettingsTests.java similarity index 99% rename from server/src/test/java/org/opensearch/index/RemoteRefreshSegmentPressureSettingsTest.java rename to server/src/test/java/org/opensearch/index/RemoteRefreshSegmentPressureSettingsTests.java index 2f58403584a27..3d7a82e6148ac 100644 --- a/server/src/test/java/org/opensearch/index/RemoteRefreshSegmentPressureSettingsTest.java +++ b/server/src/test/java/org/opensearch/index/RemoteRefreshSegmentPressureSettingsTests.java @@ -21,7 +21,7 @@ import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; -public class RemoteRefreshSegmentPressureSettingsTest extends OpenSearchTestCase { +public class RemoteRefreshSegmentPressureSettingsTests extends OpenSearchTestCase { private ClusterService clusterService; @@ -224,7 +224,6 @@ public void testUpdateTriggeredInRemotePressureServiceOnUpdateSettings() { AtomicInteger updatedUploadBytesPerSecWindowSize = new AtomicInteger(); AtomicInteger updatedUploadTimeWindowSize = new AtomicInteger(); - RemoteRefreshSegmentPressureService pressureService = mock(RemoteRefreshSegmentPressureService.class); // Upload bytes diff --git a/server/src/test/java/org/opensearch/index/RemoteRefreshSegmentPressureTrackerTests.java b/server/src/test/java/org/opensearch/index/RemoteRefreshSegmentPressureTrackerTests.java new file mode 100644 index 0000000000000..694541c94834b --- /dev/null +++ b/server/src/test/java/org/opensearch/index/RemoteRefreshSegmentPressureTrackerTests.java @@ -0,0 +1,98 @@ +/* + * 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; + +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; +import org.opensearch.index.shard.ShardId; +import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.threadpool.TestThreadPool; +import org.opensearch.threadpool.ThreadPool; + +import static org.mockito.Mockito.mock; + +public class RemoteRefreshSegmentPressureTrackerTests extends OpenSearchTestCase { + + private RemoteRefreshSegmentPressureSettings pressureSettings; + + private ClusterService clusterService; + + private ThreadPool threadPool; + + private ShardId shardId; + + private RemoteRefreshSegmentPressureTracker pressureTracker; + + @Override + public void setUp() throws Exception { + super.setUp(); + threadPool = new TestThreadPool("remote_refresh_segment_pressure_settings_test"); + clusterService = new ClusterService( + Settings.EMPTY, + new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), + threadPool + ); + pressureSettings = new RemoteRefreshSegmentPressureSettings( + clusterService, + Settings.EMPTY, + mock(RemoteRefreshSegmentPressureService.class) + ); + shardId = new ShardId("index", "uuid", 0); + } + + @Override + public void tearDown() throws Exception { + super.tearDown(); + threadPool.shutdownNow(); + } + + public void testGetShardId() { + pressureTracker = new RemoteRefreshSegmentPressureTracker(shardId, pressureSettings); + assertEquals(shardId, pressureTracker.getShardId()); + } + + public void testUpdateLocalRefreshSeqNo() { + pressureTracker = new RemoteRefreshSegmentPressureTracker(shardId, pressureSettings); + long refreshSeqNo = 2; + pressureTracker.updateLocalRefreshSeqNo(refreshSeqNo); + assertEquals(refreshSeqNo, pressureTracker.getLocalRefreshSeqNo()); + } + + public void testUpdateRemoteRefreshSeqNo() { + pressureTracker = new RemoteRefreshSegmentPressureTracker(shardId, pressureSettings); + long refreshSeqNo = 4; + pressureTracker.updateRemoteRefreshSeqNo(refreshSeqNo); + assertEquals(refreshSeqNo, pressureTracker.getRemoteRefreshSeqNo()); + } + + public void testUpdateLocalRefreshTimeMs() { + pressureTracker = new RemoteRefreshSegmentPressureTracker(shardId, pressureSettings); + long refreshTimeMs = System.nanoTime() / 1_000_000L + randomIntBetween(10, 100); + pressureTracker.updateLocalRefreshTimeMs(refreshTimeMs); + assertEquals(refreshTimeMs, pressureTracker.getLocalRefreshTimeMs()); + } + + public void testUpdateRemoteRefreshTimeMs() { + pressureTracker = new RemoteRefreshSegmentPressureTracker(shardId, pressureSettings); + long refreshTimeMs = System.nanoTime() / 1_000_000 + randomIntBetween(10, 100); + pressureTracker.updateRemoteRefreshTimeMs(refreshTimeMs); + assertEquals(refreshTimeMs, pressureTracker.getRemoteRefreshTimeMs()); + } + + public void testComputeSeqNoLagOnUpdate() { + pressureTracker = new RemoteRefreshSegmentPressureTracker(shardId, pressureSettings); + int localRefreshSeqNo = randomIntBetween(50, 100); + int remoteRefreshSeqNo = randomIntBetween(20, 50); + pressureTracker.updateLocalRefreshSeqNo(localRefreshSeqNo); + assertEquals(localRefreshSeqNo, pressureTracker.getSeqNoLag()); + pressureTracker.updateRemoteRefreshSeqNo(remoteRefreshSeqNo); + assertEquals(localRefreshSeqNo - remoteRefreshSeqNo, pressureTracker.getSeqNoLag()); + } +} From 57bd7342f5721deb3b26dc4eab0acc4905594f0c Mon Sep 17 00:00:00 2001 From: Ashish Singh Date: Wed, 19 Apr 2023 02:24:28 +0530 Subject: [PATCH 04/17] Add more pressure tracker tests Signed-off-by: Ashish Singh --- .../RemoteRefreshSegmentPressureTracker.java | 44 ++-- ...oteRefreshSegmentPressureTrackerTests.java | 201 ++++++++++++++++++ 2 files changed, 231 insertions(+), 14 deletions(-) diff --git a/server/src/main/java/org/opensearch/index/RemoteRefreshSegmentPressureTracker.java b/server/src/main/java/org/opensearch/index/RemoteRefreshSegmentPressureTracker.java index 7a272a63b790d..6b62cabd9be52 100644 --- a/server/src/main/java/org/opensearch/index/RemoteRefreshSegmentPressureTracker.java +++ b/server/src/main/java/org/opensearch/index/RemoteRefreshSegmentPressureTracker.java @@ -229,24 +229,24 @@ long getBytesLag() { return bytesLag.get(); } - AtomicLong getUploadBytesStarted() { - return uploadBytesStarted; + long getUploadBytesStarted() { + return uploadBytesStarted.get(); } void addUploadBytesStarted(long size) { uploadBytesStarted.addAndGet(size); } - AtomicLong getUploadBytesFailed() { - return uploadBytesFailed; + long getUploadBytesFailed() { + return uploadBytesFailed.get(); } void addUploadBytesFailed(long size) { uploadBytesFailed.addAndGet(size); } - AtomicLong getUploadBytesSucceeded() { - return uploadBytesSucceeded; + long getUploadBytesSucceeded() { + return uploadBytesSucceeded.get(); } void addUploadBytesSucceeded(long size) { @@ -257,16 +257,16 @@ long getInflightUploadBytes() { return uploadBytesStarted.get() - uploadBytesFailed.get() - uploadBytesSucceeded.get(); } - AtomicLong getTotalUploadsStarted() { - return totalUploadsStarted; + long getTotalUploadsStarted() { + return totalUploadsStarted.get(); } void incrementTotalUploadsStarted() { totalUploadsStarted.incrementAndGet(); } - public AtomicLong getTotalUploadsFailed() { - return totalUploadsFailed; + long getTotalUploadsFailed() { + return totalUploadsFailed.get(); } void incrementTotalUploadsFailed() { @@ -274,8 +274,8 @@ void incrementTotalUploadsFailed() { failures.record(true); } - AtomicLong getTotalUploadsSucceeded() { - return totalUploadsSucceeded; + long getTotalUploadsSucceeded() { + return totalUploadsSucceeded.get(); } void incrementTotalUploadSucceeded() { @@ -287,8 +287,8 @@ long getInflightUploads() { return totalUploadsStarted.get() - totalUploadsFailed.get() - totalUploadsSucceeded.get(); } - AtomicLong getRejectionCount() { - return rejectionCount; + long getRejectionCount() { + return rejectionCount.get(); } void incrementRejectionCount() { @@ -333,6 +333,10 @@ boolean isUploadBytesAverageReady() { return uploadBytesMovingAverageReference.get().getAverage(); } + void addUploadBytes(long size) { + this.uploadBytesMovingAverageReference.get().record(size); + } + /** * Updates the window size for data collection of upload bytes. This also resets any data collected so far. * @@ -342,10 +346,18 @@ void updateUploadBytesMovingAverageWindowSize(int updatedSize) { this.uploadBytesMovingAverageReference.set(new MovingAverage(updatedSize)); } + boolean isUploadBytesPerSecAverageReady() { + return uploadBytesPerSecMovingAverageReference.get().isReady(); + } + double getUploadBytesPerSecAverage() { return uploadBytesPerSecMovingAverageReference.get().getAverage(); } + void addUploadBytesPerSec(long bytesPerSec) { + this.uploadBytesPerSecMovingAverageReference.get().record(bytesPerSec); + } + /** * Updates the window size for data collection of upload bytes per second. This also resets any data collected so far. * @@ -363,6 +375,10 @@ boolean isUploadTimeMsAverageReady() { return uploadTimeMsMovingAverageReference.get().getAverage(); } + void addUploadTimeMs(long timeMs) { + this.uploadTimeMsMovingAverageReference.get().record(timeMs); + } + /** * Updates the window size for data collection of upload time (ms). This also resets any data collected so far. * diff --git a/server/src/test/java/org/opensearch/index/RemoteRefreshSegmentPressureTrackerTests.java b/server/src/test/java/org/opensearch/index/RemoteRefreshSegmentPressureTrackerTests.java index 694541c94834b..d5ff0201a2472 100644 --- a/server/src/test/java/org/opensearch/index/RemoteRefreshSegmentPressureTrackerTests.java +++ b/server/src/test/java/org/opensearch/index/RemoteRefreshSegmentPressureTrackerTests.java @@ -16,6 +16,9 @@ import org.opensearch.threadpool.TestThreadPool; import org.opensearch.threadpool.ThreadPool; +import java.util.HashMap; +import java.util.Map; + import static org.mockito.Mockito.mock; public class RemoteRefreshSegmentPressureTrackerTests extends OpenSearchTestCase { @@ -95,4 +98,202 @@ public void testComputeSeqNoLagOnUpdate() { pressureTracker.updateRemoteRefreshSeqNo(remoteRefreshSeqNo); assertEquals(localRefreshSeqNo - remoteRefreshSeqNo, pressureTracker.getSeqNoLag()); } + + public void testComputeTimeLagOnUpdate() { + pressureTracker = new RemoteRefreshSegmentPressureTracker(shardId, pressureSettings); + long currentLocalRefreshTimeMs = pressureTracker.getLocalRefreshTimeMs(); + long currentTimeMs = System.nanoTime() / 1_000_000L; + long localRefreshTimeMs = currentTimeMs + randomIntBetween(100, 500); + long remoteRefreshTimeMs = currentTimeMs + randomIntBetween(50, 99); + pressureTracker.updateLocalRefreshTimeMs(localRefreshTimeMs); + assertEquals(localRefreshTimeMs - currentLocalRefreshTimeMs, pressureTracker.getTimeMsLag()); + pressureTracker.updateRemoteRefreshTimeMs(remoteRefreshTimeMs); + assertEquals(localRefreshTimeMs - remoteRefreshTimeMs, pressureTracker.getTimeMsLag()); + } + + public void testAddUploadBytesStarted() { + pressureTracker = new RemoteRefreshSegmentPressureTracker(shardId, pressureSettings); + long bytesToAdd = randomLongBetween(1000, 1000000); + pressureTracker.addUploadBytesStarted(bytesToAdd); + assertEquals(bytesToAdd, pressureTracker.getUploadBytesStarted()); + long moreBytesToAdd = randomLongBetween(1000, 10000); + pressureTracker.addUploadBytesStarted(moreBytesToAdd); + assertEquals(bytesToAdd + moreBytesToAdd, pressureTracker.getUploadBytesStarted()); + } + + public void testAddUploadBytesFailed() { + pressureTracker = new RemoteRefreshSegmentPressureTracker(shardId, pressureSettings); + long bytesToAdd = randomLongBetween(1000, 1000000); + pressureTracker.addUploadBytesFailed(bytesToAdd); + assertEquals(bytesToAdd, pressureTracker.getUploadBytesFailed()); + long moreBytesToAdd = randomLongBetween(1000, 10000); + pressureTracker.addUploadBytesFailed(moreBytesToAdd); + assertEquals(bytesToAdd + moreBytesToAdd, pressureTracker.getUploadBytesFailed()); + } + + public void testAddUploadBytesSucceeded() { + pressureTracker = new RemoteRefreshSegmentPressureTracker(shardId, pressureSettings); + long bytesToAdd = randomLongBetween(1000, 1000000); + pressureTracker.addUploadBytesSucceeded(bytesToAdd); + assertEquals(bytesToAdd, pressureTracker.getUploadBytesSucceeded()); + long moreBytesToAdd = randomLongBetween(1000, 10000); + pressureTracker.addUploadBytesSucceeded(moreBytesToAdd); + assertEquals(bytesToAdd + moreBytesToAdd, pressureTracker.getUploadBytesSucceeded()); + } + + public void testGetInflightUploadBytes() { + pressureTracker = new RemoteRefreshSegmentPressureTracker(shardId, pressureSettings); + long bytesStarted = randomLongBetween(10000, 100000); + long bytesSucceeded = randomLongBetween(1000, 10000); + long bytesFailed = randomLongBetween(100, 1000); + pressureTracker.addUploadBytesStarted(bytesStarted); + pressureTracker.addUploadBytesSucceeded(bytesSucceeded); + pressureTracker.addUploadBytesFailed(bytesFailed); + assertEquals(bytesStarted - bytesSucceeded - bytesFailed, pressureTracker.getInflightUploadBytes()); + } + + public void testIncrementTotalUploadsStarted() { + pressureTracker = new RemoteRefreshSegmentPressureTracker(shardId, pressureSettings); + pressureTracker.incrementTotalUploadsStarted(); + assertEquals(1, pressureTracker.getTotalUploadsStarted()); + pressureTracker.incrementTotalUploadsStarted(); + assertEquals(2, pressureTracker.getTotalUploadsStarted()); + } + + public void testIncrementTotalUploadsFailed() { + pressureTracker = new RemoteRefreshSegmentPressureTracker(shardId, pressureSettings); + pressureTracker.incrementTotalUploadsFailed(); + assertEquals(1, pressureTracker.getTotalUploadsFailed()); + pressureTracker.incrementTotalUploadsFailed(); + assertEquals(2, pressureTracker.getTotalUploadsFailed()); + } + + public void testIncrementTotalUploadSucceeded() { + pressureTracker = new RemoteRefreshSegmentPressureTracker(shardId, pressureSettings); + pressureTracker.incrementTotalUploadSucceeded(); + assertEquals(1, pressureTracker.getTotalUploadsSucceeded()); + pressureTracker.incrementTotalUploadSucceeded(); + assertEquals(2, pressureTracker.getTotalUploadsSucceeded()); + } + + public void testGetInflightUploads() { + pressureTracker = new RemoteRefreshSegmentPressureTracker(shardId, pressureSettings); + pressureTracker.incrementTotalUploadsStarted(); + assertEquals(1, pressureTracker.getInflightUploads()); + pressureTracker.incrementTotalUploadsStarted(); + assertEquals(2, pressureTracker.getInflightUploads()); + pressureTracker.incrementTotalUploadSucceeded(); + assertEquals(1, pressureTracker.getInflightUploads()); + pressureTracker.incrementTotalUploadsFailed(); + assertEquals(0, pressureTracker.getInflightUploads()); + } + + public void testIncrementRejectionCount() { + pressureTracker = new RemoteRefreshSegmentPressureTracker(shardId, pressureSettings); + pressureTracker.incrementRejectionCount(); + assertEquals(1, pressureTracker.getRejectionCount()); + pressureTracker.incrementRejectionCount(); + assertEquals(2, pressureTracker.getRejectionCount()); + } + + public void testGetConsecutiveFailureCount() { + pressureTracker = new RemoteRefreshSegmentPressureTracker(shardId, pressureSettings); + pressureTracker.incrementTotalUploadsFailed(); + assertEquals(1, pressureTracker.getConsecutiveFailureCount()); + pressureTracker.incrementTotalUploadsFailed(); + assertEquals(2, pressureTracker.getConsecutiveFailureCount()); + pressureTracker.incrementTotalUploadSucceeded(); + assertEquals(0, pressureTracker.getConsecutiveFailureCount()); + } + + public void testComputeBytesLag() { + pressureTracker = new RemoteRefreshSegmentPressureTracker(shardId, pressureSettings); + + // Create local file size map + Map fileSizeMap = new HashMap<>(); + fileSizeMap.put("a", 100L); + fileSizeMap.put("b", 105L); + pressureTracker.setLatestLocalFileNameLengthMap(fileSizeMap); + assertEquals(205L, pressureTracker.getBytesLag()); + + pressureTracker.addToLatestUploadFiles("a"); + assertEquals(105L, pressureTracker.getBytesLag()); + + fileSizeMap.put("c", 115L); + pressureTracker.setLatestLocalFileNameLengthMap(fileSizeMap); + assertEquals(220L, pressureTracker.getBytesLag()); + + pressureTracker.addToLatestUploadFiles("b"); + assertEquals(115L, pressureTracker.getBytesLag()); + + pressureTracker.addToLatestUploadFiles("c"); + assertEquals(0L, pressureTracker.getBytesLag()); + } + + public void testIsUploadBytesAverageReady() { + pressureTracker = new RemoteRefreshSegmentPressureTracker(shardId, pressureSettings); + assertFalse(pressureTracker.isUploadBytesAverageReady()); + + long sum = 0; + for (int i = 1; i < 20; i++) { + pressureTracker.addUploadBytes(i); + sum += i; + assertFalse(pressureTracker.isUploadBytesAverageReady()); + assertEquals((double) sum / i, pressureTracker.getUploadBytesAverage(), 0.0d); + } + + pressureTracker.addUploadBytes(20); + sum += 20; + assertTrue(pressureTracker.isUploadBytesAverageReady()); + assertEquals((double) sum / 20, pressureTracker.getUploadBytesAverage(), 0.0d); + + pressureTracker.addUploadBytes(100); + sum = sum + 100 - 1; + assertEquals((double) sum / 20, pressureTracker.getUploadBytesAverage(), 0.0d); + } + + public void testIsUploadBytesPerSecAverageReady() { + pressureTracker = new RemoteRefreshSegmentPressureTracker(shardId, pressureSettings); + assertFalse(pressureTracker.isUploadBytesPerSecAverageReady()); + + long sum = 0; + for (int i = 1; i < 20; i++) { + pressureTracker.addUploadBytesPerSec(i); + sum += i; + assertFalse(pressureTracker.isUploadBytesPerSecAverageReady()); + assertEquals((double) sum / i, pressureTracker.getUploadBytesPerSecAverage(), 0.0d); + } + + pressureTracker.addUploadBytesPerSec(20); + sum += 20; + assertTrue(pressureTracker.isUploadBytesPerSecAverageReady()); + assertEquals((double) sum / 20, pressureTracker.getUploadBytesPerSecAverage(), 0.0d); + + pressureTracker.addUploadBytesPerSec(100); + sum = sum + 100 - 1; + assertEquals((double) sum / 20, pressureTracker.getUploadBytesPerSecAverage(), 0.0d); + } + + public void testIsUploadTimeMsAverageReady() { + pressureTracker = new RemoteRefreshSegmentPressureTracker(shardId, pressureSettings); + assertFalse(pressureTracker.isUploadTimeMsAverageReady()); + + long sum = 0; + for (int i = 1; i < 20; i++) { + pressureTracker.addUploadTimeMs(i); + sum += i; + assertFalse(pressureTracker.isUploadTimeMsAverageReady()); + assertEquals((double) sum / i, pressureTracker.getUploadTimeMsAverage(), 0.0d); + } + + pressureTracker.addUploadTimeMs(20); + sum += 20; + assertTrue(pressureTracker.isUploadTimeMsAverageReady()); + assertEquals((double) sum / 20, pressureTracker.getUploadTimeMsAverage(), 0.0d); + + pressureTracker.addUploadTimeMs(100); + sum = sum + 100 - 1; + assertEquals((double) sum / 20, pressureTracker.getUploadTimeMsAverage(), 0.0d); + } + } From 4c0399e78cf8939132c8b73c7331265a9ac6d133 Mon Sep 17 00:00:00 2001 From: Ashish Singh Date: Wed, 19 Apr 2023 16:15:43 +0530 Subject: [PATCH 05/17] Add remote refresh segment pressure service tests Signed-off-by: Ashish Singh --- .../RemoteRefreshSegmentPressureService.java | 4 +- ...oteRefreshSegmentPressureServiceTests.java | 104 ++++++++++++++++++ 2 files changed, 106 insertions(+), 2 deletions(-) create mode 100644 server/src/test/java/org/opensearch/index/RemoteRefreshSegmentPressureServiceTests.java diff --git a/server/src/main/java/org/opensearch/index/RemoteRefreshSegmentPressureService.java b/server/src/main/java/org/opensearch/index/RemoteRefreshSegmentPressureService.java index d1c75519831e0..5e9028c2117f8 100644 --- a/server/src/main/java/org/opensearch/index/RemoteRefreshSegmentPressureService.java +++ b/server/src/main/java/org/opensearch/index/RemoteRefreshSegmentPressureService.java @@ -51,7 +51,7 @@ public RemoteRefreshSegmentPressureService(ClusterService clusterService, Settin * @param shardId shard id * @return the tracker if index is remote-backed, else null. */ - public RemoteRefreshSegmentPressureTracker getStatsTracker(ShardId shardId) { + public RemoteRefreshSegmentPressureTracker getPressureTracker(ShardId shardId) { return trackerMap.get(shardId); } @@ -84,7 +84,7 @@ public boolean isSegmentsUploadBackpressureEnabled() { } public void validateSegmentsUploadLag(ShardId shardId) { - RemoteRefreshSegmentPressureTracker pressureTracker = getStatsTracker(shardId); + RemoteRefreshSegmentPressureTracker pressureTracker = getPressureTracker(shardId); // Check if refresh checkpoint (a.k.a. seq number) lag is 2 or below - this is to handle segment merges that can // increase the bytes to upload almost suddenly. if (pressureTracker.getSeqNoLag() <= 2) { diff --git a/server/src/test/java/org/opensearch/index/RemoteRefreshSegmentPressureServiceTests.java b/server/src/test/java/org/opensearch/index/RemoteRefreshSegmentPressureServiceTests.java new file mode 100644 index 0000000000000..16a89f0a06e2f --- /dev/null +++ b/server/src/test/java/org/opensearch/index/RemoteRefreshSegmentPressureServiceTests.java @@ -0,0 +1,104 @@ +/* + * 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; + +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; +import org.opensearch.index.shard.IndexShard; +import org.opensearch.index.shard.ShardId; +import org.opensearch.test.IndexSettingsModule; +import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.threadpool.TestThreadPool; +import org.opensearch.threadpool.ThreadPool; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class RemoteRefreshSegmentPressureServiceTests extends OpenSearchTestCase { + + private ClusterService clusterService; + + private ThreadPool threadPool; + + private ShardId shardId; + + private RemoteRefreshSegmentPressureService pressureService; + + @Override + public void setUp() throws Exception { + super.setUp(); + threadPool = new TestThreadPool("remote_refresh_segment_pressure_settings_test"); + clusterService = new ClusterService( + Settings.EMPTY, + new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), + threadPool + ); + shardId = new ShardId("index", "uuid", 0); + } + + @Override + public void tearDown() throws Exception { + super.tearDown(); + threadPool.shutdownNow(); + } + + public void testIsSegmentsUploadBackpressureEnabled() { + pressureService = new RemoteRefreshSegmentPressureService(clusterService, Settings.EMPTY); + assertFalse(pressureService.isSegmentsUploadBackpressureEnabled()); + + Settings newSettings = Settings.builder() + .put(RemoteRefreshSegmentPressureSettings.REMOTE_REFRESH_SEGMENT_PRESSURE_ENABLED.getKey(), "true") + .build(); + clusterService.getClusterSettings().applySettings(newSettings); + + assertTrue(pressureService.isSegmentsUploadBackpressureEnabled()); + } + + public void testAfterIndexShardCreatedForRemoteBackedIndex() { + Settings settings = Settings.builder().put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, "true").build(); + IndexSettings indexSettings = IndexSettingsModule.newIndexSettings("test_index", settings); + ShardId testShardId = new ShardId("index", "uuid", 0); + IndexShard indexShard = mock(IndexShard.class); + when(indexShard.indexSettings()).thenReturn(indexSettings); + when(indexShard.shardId()).thenReturn(testShardId); + pressureService = new RemoteRefreshSegmentPressureService(clusterService, Settings.EMPTY); + pressureService.afterIndexShardCreated(indexShard); + assertNotNull(pressureService.getPressureTracker(testShardId)); + } + + public void testAfterIndexShardCreatedForNonRemoteBackedIndex() { + Settings settings = Settings.builder().put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, "false").build(); + IndexSettings indexSettings = IndexSettingsModule.newIndexSettings("test_index", settings); + ShardId testShardId = new ShardId("index", "uuid", 0); + IndexShard indexShard = mock(IndexShard.class); + when(indexShard.indexSettings()).thenReturn(indexSettings); + when(indexShard.shardId()).thenReturn(testShardId); + pressureService = new RemoteRefreshSegmentPressureService(clusterService, Settings.EMPTY); + pressureService.afterIndexShardCreated(indexShard); + assertNull(pressureService.getPressureTracker(testShardId)); + } + + public void testBeforeIndexShardClosed() { + Settings settings = Settings.builder().put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, "true").build(); + IndexSettings indexSettings = IndexSettingsModule.newIndexSettings("test_index", settings); + ShardId testShardId = new ShardId("index", "uuid", 0); + IndexShard indexShard = mock(IndexShard.class); + when(indexShard.indexSettings()).thenReturn(indexSettings); + when(indexShard.shardId()).thenReturn(testShardId); + pressureService = new RemoteRefreshSegmentPressureService(clusterService, Settings.EMPTY); + pressureService.afterIndexShardCreated(indexShard); + assertNotNull(pressureService.getPressureTracker(testShardId)); + + pressureService.beforeIndexShardClosed(testShardId, indexShard, settings); + assertNull(pressureService.getPressureTracker(testShardId)); + } + +} From 5c9d353ac155334645e8988e0de90e902df3564d Mon Sep 17 00:00:00 2001 From: Ashish Singh Date: Wed, 19 Apr 2023 16:53:35 +0530 Subject: [PATCH 06/17] Add more remote refresh segment pressure service tests Signed-off-by: Ashish Singh --- ...oteRefreshSegmentPressureServiceTests.java | 63 +++++++++++++++++++ 1 file changed, 63 insertions(+) diff --git a/server/src/test/java/org/opensearch/index/RemoteRefreshSegmentPressureServiceTests.java b/server/src/test/java/org/opensearch/index/RemoteRefreshSegmentPressureServiceTests.java index 16a89f0a06e2f..0f417c1b97fe2 100644 --- a/server/src/test/java/org/opensearch/index/RemoteRefreshSegmentPressureServiceTests.java +++ b/server/src/test/java/org/opensearch/index/RemoteRefreshSegmentPressureServiceTests.java @@ -12,6 +12,7 @@ import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; +import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; import org.opensearch.index.shard.IndexShard; import org.opensearch.index.shard.ShardId; import org.opensearch.test.IndexSettingsModule; @@ -19,6 +20,11 @@ import org.opensearch.threadpool.TestThreadPool; import org.opensearch.threadpool.ThreadPool; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.IntStream; + import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -101,4 +107,61 @@ public void testBeforeIndexShardClosed() { assertNull(pressureService.getPressureTracker(testShardId)); } + public void testValidateSegmentUploadLag() { + // Create the pressure tracker + Settings settings = Settings.builder().put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, "true").build(); + IndexSettings indexSettings = IndexSettingsModule.newIndexSettings("test_index", settings); + ShardId testShardId = new ShardId("index", "uuid", 0); + IndexShard indexShard = mock(IndexShard.class); + when(indexShard.indexSettings()).thenReturn(indexSettings); + when(indexShard.shardId()).thenReturn(testShardId); + pressureService = new RemoteRefreshSegmentPressureService(clusterService, Settings.EMPTY); + pressureService.afterIndexShardCreated(indexShard); + + // 1. Seq no - add data points to the pressure tracker + RemoteRefreshSegmentPressureTracker pressureTracker = pressureService.getPressureTracker(testShardId); + pressureTracker.updateLocalRefreshSeqNo(6); + assertThrows(OpenSearchRejectedExecutionException.class, () -> pressureService.validateSegmentsUploadLag(testShardId)); + + // 2. time lag more than dynamic threshold + pressureTracker.updateRemoteRefreshSeqNo(3); + AtomicLong sum = new AtomicLong(); + IntStream.range(0, 20).forEach(i -> { + pressureTracker.addUploadTimeMs(i); + sum.addAndGet(i); + }); + double avg = (double) sum.get() / 20; + long currentMs = System.nanoTime() / 1_000_000; + pressureTracker.updateLocalRefreshTimeMs((long) (currentMs + 4 * avg)); + pressureTracker.updateRemoteRefreshTimeMs(currentMs); + assertThrows(OpenSearchRejectedExecutionException.class, () -> pressureService.validateSegmentsUploadLag(testShardId)); + + pressureTracker.updateRemoteRefreshTimeMs((long) (currentMs + 2 * avg)); + pressureService.validateSegmentsUploadLag(testShardId); + + // 3. bytes lag more than dynamic threshold + sum.set(0); + IntStream.range(0, 20).forEach(i -> { + pressureTracker.addUploadBytes(i); + sum.addAndGet(i); + }); + avg = (double) sum.get() / 20; + Map nameSizeMap = new HashMap<>(); + nameSizeMap.put("a", (long) (4 * avg)); + pressureTracker.setLatestLocalFileNameLengthMap(nameSizeMap); + assertThrows(OpenSearchRejectedExecutionException.class, () -> pressureService.validateSegmentsUploadLag(testShardId)); + + nameSizeMap.put("a", (long) (2 * avg)); + pressureTracker.setLatestLocalFileNameLengthMap(nameSizeMap); + pressureService.validateSegmentsUploadLag(testShardId); + + // 4. Consecutive failures more than the limit + IntStream.range(0, 10).forEach(ignore -> pressureTracker.incrementTotalUploadsFailed()); + pressureService.validateSegmentsUploadLag(testShardId); + pressureTracker.incrementTotalUploadsFailed(); + assertThrows(OpenSearchRejectedExecutionException.class, () -> pressureService.validateSegmentsUploadLag(testShardId)); + pressureTracker.incrementTotalUploadSucceeded(); + pressureService.validateSegmentsUploadLag(testShardId); + } + } From 0e6aa4dc943b2d0817fc06d2751626c9a5a65c59 Mon Sep 17 00:00:00 2001 From: Ashish Singh Date: Wed, 19 Apr 2023 17:02:23 +0530 Subject: [PATCH 07/17] Add message assertion in remote refresh segment pressure service tests Signed-off-by: Ashish Singh --- .../RemoteRefreshSegmentPressureService.java | 2 +- ...oteRefreshSegmentPressureServiceTests.java | 19 +++++++++++++++---- 2 files changed, 16 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/org/opensearch/index/RemoteRefreshSegmentPressureService.java b/server/src/main/java/org/opensearch/index/RemoteRefreshSegmentPressureService.java index 5e9028c2117f8..a0fdf8eb75e8b 100644 --- a/server/src/main/java/org/opensearch/index/RemoteRefreshSegmentPressureService.java +++ b/server/src/main/java/org/opensearch/index/RemoteRefreshSegmentPressureService.java @@ -154,7 +154,7 @@ private void validateTimeLag(RemoteRefreshSegmentPressureTracker pressureTracker String.format( Locale.ROOT, "rejected execution on primary shard:%s due to remote segments lagging behind local segments." - + "time_lag:%s ns dynamic_time_lag_threshold:%s ns", + + "time_lag:%s ms dynamic_time_lag_threshold:%s ms", shardId, timeLag, dynamicTimeLagThreshold diff --git a/server/src/test/java/org/opensearch/index/RemoteRefreshSegmentPressureServiceTests.java b/server/src/test/java/org/opensearch/index/RemoteRefreshSegmentPressureServiceTests.java index 0f417c1b97fe2..fbe865c1c9ec3 100644 --- a/server/src/test/java/org/opensearch/index/RemoteRefreshSegmentPressureServiceTests.java +++ b/server/src/test/java/org/opensearch/index/RemoteRefreshSegmentPressureServiceTests.java @@ -121,7 +121,12 @@ public void testValidateSegmentUploadLag() { // 1. Seq no - add data points to the pressure tracker RemoteRefreshSegmentPressureTracker pressureTracker = pressureService.getPressureTracker(testShardId); pressureTracker.updateLocalRefreshSeqNo(6); - assertThrows(OpenSearchRejectedExecutionException.class, () -> pressureService.validateSegmentsUploadLag(testShardId)); + Exception e = assertThrows( + OpenSearchRejectedExecutionException.class, + () -> pressureService.validateSegmentsUploadLag(testShardId) + ); + assertTrue(e.getMessage().contains("due to remote segments lagging behind local segments")); + assertTrue(e.getMessage().contains("remote_refresh_seq_no:0 local_refresh_seq_no:6")); // 2. time lag more than dynamic threshold pressureTracker.updateRemoteRefreshSeqNo(3); @@ -134,7 +139,9 @@ public void testValidateSegmentUploadLag() { long currentMs = System.nanoTime() / 1_000_000; pressureTracker.updateLocalRefreshTimeMs((long) (currentMs + 4 * avg)); pressureTracker.updateRemoteRefreshTimeMs(currentMs); - assertThrows(OpenSearchRejectedExecutionException.class, () -> pressureService.validateSegmentsUploadLag(testShardId)); + e = assertThrows(OpenSearchRejectedExecutionException.class, () -> pressureService.validateSegmentsUploadLag(testShardId)); + assertTrue(e.getMessage().contains("due to remote segments lagging behind local segments")); + assertTrue(e.getMessage().contains("time_lag:38 ms dynamic_time_lag_threshold:19.0 ms")); pressureTracker.updateRemoteRefreshTimeMs((long) (currentMs + 2 * avg)); pressureService.validateSegmentsUploadLag(testShardId); @@ -149,7 +156,9 @@ public void testValidateSegmentUploadLag() { Map nameSizeMap = new HashMap<>(); nameSizeMap.put("a", (long) (4 * avg)); pressureTracker.setLatestLocalFileNameLengthMap(nameSizeMap); - assertThrows(OpenSearchRejectedExecutionException.class, () -> pressureService.validateSegmentsUploadLag(testShardId)); + e = assertThrows(OpenSearchRejectedExecutionException.class, () -> pressureService.validateSegmentsUploadLag(testShardId)); + assertTrue(e.getMessage().contains("due to remote segments lagging behind local segments")); + assertTrue(e.getMessage().contains("bytes_lag:38 dynamic_bytes_lag_threshold:19.0")); nameSizeMap.put("a", (long) (2 * avg)); pressureTracker.setLatestLocalFileNameLengthMap(nameSizeMap); @@ -159,7 +168,9 @@ public void testValidateSegmentUploadLag() { IntStream.range(0, 10).forEach(ignore -> pressureTracker.incrementTotalUploadsFailed()); pressureService.validateSegmentsUploadLag(testShardId); pressureTracker.incrementTotalUploadsFailed(); - assertThrows(OpenSearchRejectedExecutionException.class, () -> pressureService.validateSegmentsUploadLag(testShardId)); + e = assertThrows(OpenSearchRejectedExecutionException.class, () -> pressureService.validateSegmentsUploadLag(testShardId)); + assertTrue(e.getMessage().contains("due to remote segments lagging behind local segments")); + assertTrue(e.getMessage().contains("failure_streak_count:11 min_consecutive_failure_threshold:10")); pressureTracker.incrementTotalUploadSucceeded(); pressureService.validateSegmentsUploadLag(testShardId); } From b44ab48b0cf52b4881c3502df9c63ac7be4c98f1 Mon Sep 17 00:00:00 2001 From: Ashish Singh Date: Tue, 25 Apr 2023 15:14:51 +0530 Subject: [PATCH 08/17] Incorporate PR review feedback Signed-off-by: Ashish Singh --- .../common/settings/ClusterSettings.java | 6 +- .../RemoteRefreshSegmentPressureService.java | 44 +++-- .../RemoteRefreshSegmentPressureSettings.java | 64 ++++---- .../RemoteRefreshSegmentTracker.java} | 25 ++- .../opensearch/index/remote/package-info.java | 10 ++ ...oteRefreshSegmentPressureServiceTests.java | 5 +- ...teRefreshSegmentPressureSettingsTests.java | 34 ++-- .../RemoteRefreshSegmentTrackerTests.java} | 153 +++++++++++++++--- 8 files changed, 235 insertions(+), 106 deletions(-) rename server/src/main/java/org/opensearch/index/{ => remote}/RemoteRefreshSegmentPressureService.java (79%) rename server/src/main/java/org/opensearch/index/{ => remote}/RemoteRefreshSegmentPressureSettings.java (77%) rename server/src/main/java/org/opensearch/index/{RemoteRefreshSegmentPressureTracker.java => remote/RemoteRefreshSegmentTracker.java} (94%) create mode 100644 server/src/main/java/org/opensearch/index/remote/package-info.java rename server/src/test/java/org/opensearch/index/{ => remote}/RemoteRefreshSegmentPressureServiceTests.java (97%) rename server/src/test/java/org/opensearch/index/{ => remote}/RemoteRefreshSegmentPressureSettingsTests.java (90%) rename server/src/test/java/org/opensearch/index/{RemoteRefreshSegmentPressureTrackerTests.java => remote/RemoteRefreshSegmentTrackerTests.java} (63%) diff --git a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java index 083929c18c8a0..0f50f8ae5fef2 100644 --- a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java @@ -39,7 +39,7 @@ import org.opensearch.index.IndexModule; import org.opensearch.index.IndexSettings; import org.opensearch.index.IndexingPressure; -import org.opensearch.index.RemoteRefreshSegmentPressureSettings; +import org.opensearch.index.remote.RemoteRefreshSegmentPressureSettings; import org.opensearch.index.SegmentReplicationPressureService; import org.opensearch.index.ShardIndexingPressureMemoryManager; import org.opensearch.index.ShardIndexingPressureSettings; @@ -644,8 +644,8 @@ public void apply(Settings value, Settings current, Settings previous) { // Settings related to Remote Refresh Segment Pressure RemoteRefreshSegmentPressureSettings.REMOTE_REFRESH_SEGMENT_PRESSURE_ENABLED, RemoteRefreshSegmentPressureSettings.MIN_SEQ_NO_LAG_LIMIT, - RemoteRefreshSegmentPressureSettings.BYTES_LAG_VARIANCE_THRESHOLD, - RemoteRefreshSegmentPressureSettings.TIME_LAG_VARIANCE_THRESHOLD, + RemoteRefreshSegmentPressureSettings.BYTES_LAG_VARIANCE_FACTOR, + RemoteRefreshSegmentPressureSettings.UPLOAD_TIME_LAG_VARIANCE_FACTOR, RemoteRefreshSegmentPressureSettings.MIN_CONSECUTIVE_FAILURES_LIMIT, RemoteRefreshSegmentPressureSettings.UPLOAD_BYTES_MOVING_AVERAGE_WINDOW_SIZE, RemoteRefreshSegmentPressureSettings.UPLOAD_BYTES_PER_SEC_MOVING_AVERAGE_WINDOW_SIZE, diff --git a/server/src/main/java/org/opensearch/index/RemoteRefreshSegmentPressureService.java b/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureService.java similarity index 79% rename from server/src/main/java/org/opensearch/index/RemoteRefreshSegmentPressureService.java rename to server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureService.java index a0fdf8eb75e8b..38f79e264ef14 100644 --- a/server/src/main/java/org/opensearch/index/RemoteRefreshSegmentPressureService.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureService.java @@ -6,7 +6,7 @@ * compatible open source license. */ -package org.opensearch.index; +package org.opensearch.index.remote; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -24,7 +24,7 @@ import java.util.function.BiConsumer; /** - * Service used to validate if the incoming indexing request should be rejected based on the {@link RemoteRefreshSegmentPressureTracker}. + * Service used to validate if the incoming indexing request should be rejected based on the {@link RemoteRefreshSegmentTracker}. */ public class RemoteRefreshSegmentPressureService implements IndexEventListener { @@ -33,7 +33,7 @@ public class RemoteRefreshSegmentPressureService implements IndexEventListener { /** * Keeps map of remote-backed index shards and their corresponding backpressure tracker. */ - private final Map trackerMap = ConcurrentCollections.newConcurrentMap(); + private final Map trackerMap = ConcurrentCollections.newConcurrentMap(); /** * Remote refresh segment pressure settings which is used for creation of the backpressure tracker and as well as rejection. @@ -46,12 +46,12 @@ public RemoteRefreshSegmentPressureService(ClusterService clusterService, Settin } /** - * Get {@code RemoteRefreshSegmentPressureTracker} only if the underlying Index has remote segments integration enabled. + * Get {@code RemoteRefreshSegmentTracker} only if the underlying Index has remote segments integration enabled. * * @param shardId shard id * @return the tracker if index is remote-backed, else null. */ - public RemoteRefreshSegmentPressureTracker getPressureTracker(ShardId shardId) { + public RemoteRefreshSegmentTracker getPressureTracker(ShardId shardId) { return trackerMap.get(shardId); } @@ -61,7 +61,15 @@ public void afterIndexShardCreated(IndexShard indexShard) { return; } ShardId shardId = indexShard.shardId(); - trackerMap.put(shardId, new RemoteRefreshSegmentPressureTracker(shardId, pressureSettings)); + trackerMap.put( + shardId, + new RemoteRefreshSegmentTracker( + shardId, + pressureSettings.getUploadBytesMovingAverageWindowSize(), + pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), + pressureSettings.getUploadTimeMovingAverageWindowSize() + ) + ); logger.trace("Created tracker for shardId={}", shardId); } @@ -84,7 +92,7 @@ public boolean isSegmentsUploadBackpressureEnabled() { } public void validateSegmentsUploadLag(ShardId shardId) { - RemoteRefreshSegmentPressureTracker pressureTracker = getPressureTracker(shardId); + RemoteRefreshSegmentTracker pressureTracker = getPressureTracker(shardId); // Check if refresh checkpoint (a.k.a. seq number) lag is 2 or below - this is to handle segment merges that can // increase the bytes to upload almost suddenly. if (pressureTracker.getSeqNoLag() <= 2) { @@ -104,7 +112,7 @@ public void validateSegmentsUploadLag(ShardId shardId) { validateConsecutiveFailureLimitBreached(pressureTracker, shardId); } - private void validateSeqNoLag(RemoteRefreshSegmentPressureTracker pressureTracker, ShardId shardId) { + private void validateSeqNoLag(RemoteRefreshSegmentTracker pressureTracker, ShardId shardId) { // Check if the remote store seq no lag is above the min seq no lag limit if (pressureTracker.getSeqNoLag() > pressureSettings.getMinSeqNoLagLimit()) { pressureTracker.incrementRejectionCount(); @@ -121,11 +129,12 @@ private void validateSeqNoLag(RemoteRefreshSegmentPressureTracker pressureTracke } } - private void validateBytesLag(RemoteRefreshSegmentPressureTracker pressureTracker, ShardId shardId) { + private void validateBytesLag(RemoteRefreshSegmentTracker pressureTracker, ShardId shardId) { if (pressureTracker.isUploadBytesAverageReady() == false) { + logger.trace("upload bytes moving average is not ready"); return; } - double dynamicBytesLagThreshold = pressureTracker.getUploadBytesAverage() * pressureSettings.getBytesLagVarianceThreshold(); + double dynamicBytesLagThreshold = pressureTracker.getUploadBytesAverage() * pressureSettings.getBytesLagVarianceFactor(); long bytesLag = pressureTracker.getBytesLag(); if (bytesLag > dynamicBytesLagThreshold) { pressureTracker.incrementRejectionCount(); @@ -142,12 +151,13 @@ private void validateBytesLag(RemoteRefreshSegmentPressureTracker pressureTracke } } - private void validateTimeLag(RemoteRefreshSegmentPressureTracker pressureTracker, ShardId shardId) { + private void validateTimeLag(RemoteRefreshSegmentTracker pressureTracker, ShardId shardId) { if (pressureTracker.isUploadTimeMsAverageReady() == false) { + logger.trace("upload time moving average is not ready"); return; } long timeLag = pressureTracker.getTimeMsLag(); - double dynamicTimeLagThreshold = pressureTracker.getUploadTimeMsAverage() * pressureSettings.getTimeLagVarianceThreshold(); + double dynamicTimeLagThreshold = pressureTracker.getUploadTimeMsAverage() * pressureSettings.getUploadTimeLagVarianceFactor(); if (timeLag > dynamicTimeLagThreshold) { pressureTracker.incrementRejectionCount(); throw new OpenSearchRejectedExecutionException( @@ -163,7 +173,7 @@ private void validateTimeLag(RemoteRefreshSegmentPressureTracker pressureTracker } } - private void validateConsecutiveFailureLimitBreached(RemoteRefreshSegmentPressureTracker pressureTracker, ShardId shardId) { + private void validateConsecutiveFailureLimitBreached(RemoteRefreshSegmentTracker pressureTracker, ShardId shardId) { int failureStreakCount = pressureTracker.getConsecutiveFailureCount(); int minConsecutiveFailureThreshold = pressureSettings.getMinConsecutiveFailuresLimit(); if (failureStreakCount > minConsecutiveFailureThreshold) { @@ -182,18 +192,18 @@ private void validateConsecutiveFailureLimitBreached(RemoteRefreshSegmentPressur } void updateUploadBytesMovingAverageWindowSize(int updatedSize) { - updateMovingAverageWindowSize(RemoteRefreshSegmentPressureTracker::updateUploadBytesMovingAverageWindowSize, updatedSize); + updateMovingAverageWindowSize(RemoteRefreshSegmentTracker::updateUploadBytesMovingAverageWindowSize, updatedSize); } void updateUploadBytesPerSecMovingAverageWindowSize(int updatedSize) { - updateMovingAverageWindowSize(RemoteRefreshSegmentPressureTracker::updateUploadBytesPerSecMovingAverageWindowSize, updatedSize); + updateMovingAverageWindowSize(RemoteRefreshSegmentTracker::updateUploadBytesPerSecMovingAverageWindowSize, updatedSize); } void updateUploadTimeMsMovingAverageWindowSize(int updatedSize) { - updateMovingAverageWindowSize(RemoteRefreshSegmentPressureTracker::updateUploadTimeMsMovingAverageWindowSize, updatedSize); + updateMovingAverageWindowSize(RemoteRefreshSegmentTracker::updateUploadTimeMsMovingAverageWindowSize, updatedSize); } - void updateMovingAverageWindowSize(BiConsumer biConsumer, int updatedSize) { + void updateMovingAverageWindowSize(BiConsumer biConsumer, int updatedSize) { trackerMap.values().forEach(tracker -> biConsumer.accept(tracker, updatedSize)); } } diff --git a/server/src/main/java/org/opensearch/index/RemoteRefreshSegmentPressureSettings.java b/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureSettings.java similarity index 77% rename from server/src/main/java/org/opensearch/index/RemoteRefreshSegmentPressureSettings.java rename to server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureSettings.java index 3039c7cae9413..b7f467d5b9d6a 100644 --- a/server/src/main/java/org/opensearch/index/RemoteRefreshSegmentPressureSettings.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureSettings.java @@ -6,7 +6,7 @@ * compatible open source license. */ -package org.opensearch.index; +package org.opensearch.index.remote; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.settings.ClusterSettings; @@ -20,12 +20,16 @@ public class RemoteRefreshSegmentPressureSettings { private static class Defaults { private static final long MIN_SEQ_NO_LAG_LIMIT = 5; - private static final double BYTES_LAG_VARIANCE_THRESHOLD = 2.0; - private static final double TIME_LAG_VARIANCE_THRESHOLD = 2.0; + private static final long MIN_SEQ_NO_LAG_LIMIT_MIN_VALUE = 2; + private static final double BYTES_LAG_VARIANCE_FACTOR = 2.0; + private static final double UPLOAD_TIME_LAG_VARIANCE_FACTOR = 2.0; + private static final double VARIANCE_FACTOR_MIN_VALUE = 1.0; private static final int MIN_CONSECUTIVE_FAILURES_LIMIT = 10; + private static final int MIN_CONSECUTIVE_FAILURES_LIMIT_MIN_VALUE = 1; private static final int UPLOAD_BYTES_MOVING_AVERAGE_WINDOW_SIZE = 20; private static final int UPLOAD_BYTES_PER_SEC_MOVING_AVERAGE_WINDOW_SIZE = 20; private static final int UPLOAD_TIME_MOVING_AVERAGE_WINDOW_SIZE = 20; + private static final int MOVING_AVERAGE_WINDOW_SIZE_MIN_VALUE = 5; } public static final Setting REMOTE_REFRESH_SEGMENT_PRESSURE_ENABLED = Setting.boolSetting( @@ -38,23 +42,23 @@ private static class Defaults { public static final Setting MIN_SEQ_NO_LAG_LIMIT = Setting.longSetting( "remote_store.segment.pressure.seq_no_lag.limit", Defaults.MIN_SEQ_NO_LAG_LIMIT, - 2L, + Defaults.MIN_SEQ_NO_LAG_LIMIT_MIN_VALUE, Setting.Property.Dynamic, Setting.Property.NodeScope ); - public static final Setting BYTES_LAG_VARIANCE_THRESHOLD = Setting.doubleSetting( - "remote_store.segment.pressure.bytes_lag.variance", - Defaults.BYTES_LAG_VARIANCE_THRESHOLD, - 0.0, + public static final Setting BYTES_LAG_VARIANCE_FACTOR = Setting.doubleSetting( + "remote_store.segment.pressure.bytes_lag.variance_factor", + Defaults.BYTES_LAG_VARIANCE_FACTOR, + Defaults.VARIANCE_FACTOR_MIN_VALUE, Setting.Property.Dynamic, Setting.Property.NodeScope ); - public static final Setting TIME_LAG_VARIANCE_THRESHOLD = Setting.doubleSetting( - "remote_store.segment.pressure.time_lag.variance", - Defaults.TIME_LAG_VARIANCE_THRESHOLD, - 0.0, + public static final Setting UPLOAD_TIME_LAG_VARIANCE_FACTOR = Setting.doubleSetting( + "remote_store.segment.pressure.time_lag.variance_factor", + Defaults.UPLOAD_TIME_LAG_VARIANCE_FACTOR, + Defaults.VARIANCE_FACTOR_MIN_VALUE, Setting.Property.Dynamic, Setting.Property.NodeScope ); @@ -62,7 +66,7 @@ private static class Defaults { public static final Setting MIN_CONSECUTIVE_FAILURES_LIMIT = Setting.intSetting( "remote_store.segment.pressure.consecutive_failures.limit", Defaults.MIN_CONSECUTIVE_FAILURES_LIMIT, - 1, + Defaults.MIN_CONSECUTIVE_FAILURES_LIMIT_MIN_VALUE, Setting.Property.Dynamic, Setting.Property.NodeScope ); @@ -70,7 +74,7 @@ private static class Defaults { public static final Setting UPLOAD_BYTES_MOVING_AVERAGE_WINDOW_SIZE = Setting.intSetting( "remote_store.segment.pressure.upload_bytes_moving_average_window_size", Defaults.UPLOAD_BYTES_MOVING_AVERAGE_WINDOW_SIZE, - 0, + Defaults.MOVING_AVERAGE_WINDOW_SIZE_MIN_VALUE, Setting.Property.Dynamic, Setting.Property.NodeScope ); @@ -78,7 +82,7 @@ private static class Defaults { public static final Setting UPLOAD_BYTES_PER_SEC_MOVING_AVERAGE_WINDOW_SIZE = Setting.intSetting( "remote_store.segment.pressure.upload_bytes_per_sec_moving_average_window_size", Defaults.UPLOAD_BYTES_PER_SEC_MOVING_AVERAGE_WINDOW_SIZE, - 0, + Defaults.MOVING_AVERAGE_WINDOW_SIZE_MIN_VALUE, Setting.Property.Dynamic, Setting.Property.NodeScope ); @@ -86,7 +90,7 @@ private static class Defaults { public static final Setting UPLOAD_TIME_MOVING_AVERAGE_WINDOW_SIZE = Setting.intSetting( "remote_store.segment.pressure.upload_time_moving_average_window_size", Defaults.UPLOAD_TIME_MOVING_AVERAGE_WINDOW_SIZE, - 0, + Defaults.MOVING_AVERAGE_WINDOW_SIZE_MIN_VALUE, Setting.Property.Dynamic, Setting.Property.NodeScope ); @@ -95,9 +99,9 @@ private static class Defaults { private volatile long minSeqNoLagLimit; - private volatile double bytesLagVarianceThreshold; + private volatile double bytesLagVarianceFactor; - private volatile double timeLagVarianceThreshold; + private volatile double uploadTimeLagVarianceFactor; private volatile int minConsecutiveFailuresLimit; @@ -120,11 +124,11 @@ public RemoteRefreshSegmentPressureSettings( this.minSeqNoLagLimit = MIN_SEQ_NO_LAG_LIMIT.get(settings); clusterSettings.addSettingsUpdateConsumer(MIN_SEQ_NO_LAG_LIMIT, this::setMinSeqNoLagLimit); - this.bytesLagVarianceThreshold = BYTES_LAG_VARIANCE_THRESHOLD.get(settings); - clusterSettings.addSettingsUpdateConsumer(BYTES_LAG_VARIANCE_THRESHOLD, this::setBytesLagVarianceThreshold); + this.bytesLagVarianceFactor = BYTES_LAG_VARIANCE_FACTOR.get(settings); + clusterSettings.addSettingsUpdateConsumer(BYTES_LAG_VARIANCE_FACTOR, this::setBytesLagVarianceFactor); - this.timeLagVarianceThreshold = TIME_LAG_VARIANCE_THRESHOLD.get(settings); - clusterSettings.addSettingsUpdateConsumer(TIME_LAG_VARIANCE_THRESHOLD, this::setTimeLagVarianceThreshold); + this.uploadTimeLagVarianceFactor = UPLOAD_TIME_LAG_VARIANCE_FACTOR.get(settings); + clusterSettings.addSettingsUpdateConsumer(UPLOAD_TIME_LAG_VARIANCE_FACTOR, this::setUploadTimeLagVarianceFactor); this.minConsecutiveFailuresLimit = MIN_CONSECUTIVE_FAILURES_LIMIT.get(settings); clusterSettings.addSettingsUpdateConsumer(MIN_CONSECUTIVE_FAILURES_LIMIT, this::setMinConsecutiveFailuresLimit); @@ -170,20 +174,20 @@ public void setMinSeqNoLagLimit(long minSeqNoLagLimit) { this.minSeqNoLagLimit = minSeqNoLagLimit; } - public double getBytesLagVarianceThreshold() { - return bytesLagVarianceThreshold; + public double getBytesLagVarianceFactor() { + return bytesLagVarianceFactor; } - public void setBytesLagVarianceThreshold(double bytesLagVarianceThreshold) { - this.bytesLagVarianceThreshold = bytesLagVarianceThreshold; + public void setBytesLagVarianceFactor(double bytesLagVarianceFactor) { + this.bytesLagVarianceFactor = bytesLagVarianceFactor; } - public double getTimeLagVarianceThreshold() { - return timeLagVarianceThreshold; + public double getUploadTimeLagVarianceFactor() { + return uploadTimeLagVarianceFactor; } - public void setTimeLagVarianceThreshold(double timeLagVarianceThreshold) { - this.timeLagVarianceThreshold = timeLagVarianceThreshold; + public void setUploadTimeLagVarianceFactor(double uploadTimeLagVarianceFactor) { + this.uploadTimeLagVarianceFactor = uploadTimeLagVarianceFactor; } public int getMinConsecutiveFailuresLimit() { diff --git a/server/src/main/java/org/opensearch/index/RemoteRefreshSegmentPressureTracker.java b/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentTracker.java similarity index 94% rename from server/src/main/java/org/opensearch/index/RemoteRefreshSegmentPressureTracker.java rename to server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentTracker.java index 6b62cabd9be52..839f0d7423218 100644 --- a/server/src/main/java/org/opensearch/index/RemoteRefreshSegmentPressureTracker.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentTracker.java @@ -6,7 +6,7 @@ * compatible open source license. */ -package org.opensearch.index; +package org.opensearch.index.remote; import org.opensearch.common.util.MovingAverage; import org.opensearch.common.util.Streak; @@ -22,23 +22,22 @@ /** * Keeps track of remote refresh which happens in {@link org.opensearch.index.shard.RemoteStoreRefreshListener}. This consist of multiple critical metrics. */ -public class RemoteRefreshSegmentPressureTracker { - - RemoteRefreshSegmentPressureTracker(ShardId shardId, RemoteRefreshSegmentPressureSettings remoteUploadPressureSettings) { +public class RemoteRefreshSegmentTracker { + + public RemoteRefreshSegmentTracker( + ShardId shardId, + int uploadBytesMovingAverageWindowSize, + int uploadBytesPerSecMovingAverageWindowSize, + int uploadTimeMsMovingAverageWindowSize + ) { 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 currentTimeMs = System.nanoTime() / 1_000_000L; localRefreshTimeMs.set(currentTimeMs); remoteRefreshTimeMs.set(currentTimeMs); - uploadBytesMovingAverageReference = new AtomicReference<>( - new MovingAverage(remoteUploadPressureSettings.getUploadBytesMovingAverageWindowSize()) - ); - uploadBytesPerSecMovingAverageReference = new AtomicReference<>( - new MovingAverage(remoteUploadPressureSettings.getUploadBytesPerSecMovingAverageWindowSize()) - ); - uploadTimeMsMovingAverageReference = new AtomicReference<>( - new MovingAverage(remoteUploadPressureSettings.getUploadTimeMovingAverageWindowSize()) - ); + this.uploadBytesMovingAverageReference = new AtomicReference<>(new MovingAverage(uploadBytesMovingAverageWindowSize)); + this.uploadBytesPerSecMovingAverageReference = new AtomicReference<>(new MovingAverage(uploadBytesPerSecMovingAverageWindowSize)); + this.uploadTimeMsMovingAverageReference = new AtomicReference<>(new MovingAverage(uploadTimeMsMovingAverageWindowSize)); } /** diff --git a/server/src/main/java/org/opensearch/index/remote/package-info.java b/server/src/main/java/org/opensearch/index/remote/package-info.java new file mode 100644 index 0000000000000..a3aa969316372 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/remote/package-info.java @@ -0,0 +1,10 @@ +/* + * 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. + */ + +/** Core classes related to remote segments and translogs */ +package org.opensearch.index.remote; diff --git a/server/src/test/java/org/opensearch/index/RemoteRefreshSegmentPressureServiceTests.java b/server/src/test/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureServiceTests.java similarity index 97% rename from server/src/test/java/org/opensearch/index/RemoteRefreshSegmentPressureServiceTests.java rename to server/src/test/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureServiceTests.java index fbe865c1c9ec3..2a8bdf54b01b2 100644 --- a/server/src/test/java/org/opensearch/index/RemoteRefreshSegmentPressureServiceTests.java +++ b/server/src/test/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureServiceTests.java @@ -6,13 +6,14 @@ * compatible open source license. */ -package org.opensearch.index; +package org.opensearch.index.remote; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; +import org.opensearch.index.IndexSettings; import org.opensearch.index.shard.IndexShard; import org.opensearch.index.shard.ShardId; import org.opensearch.test.IndexSettingsModule; @@ -119,7 +120,7 @@ public void testValidateSegmentUploadLag() { pressureService.afterIndexShardCreated(indexShard); // 1. Seq no - add data points to the pressure tracker - RemoteRefreshSegmentPressureTracker pressureTracker = pressureService.getPressureTracker(testShardId); + RemoteRefreshSegmentTracker pressureTracker = pressureService.getPressureTracker(testShardId); pressureTracker.updateLocalRefreshSeqNo(6); Exception e = assertThrows( OpenSearchRejectedExecutionException.class, diff --git a/server/src/test/java/org/opensearch/index/RemoteRefreshSegmentPressureSettingsTests.java b/server/src/test/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureSettingsTests.java similarity index 90% rename from server/src/test/java/org/opensearch/index/RemoteRefreshSegmentPressureSettingsTests.java rename to server/src/test/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureSettingsTests.java index 3d7a82e6148ac..ed88f91ebd062 100644 --- a/server/src/test/java/org/opensearch/index/RemoteRefreshSegmentPressureSettingsTests.java +++ b/server/src/test/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureSettingsTests.java @@ -6,7 +6,7 @@ * compatible open source license. */ -package org.opensearch.index; +package org.opensearch.index.remote; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.settings.ClusterSettings; @@ -58,10 +58,10 @@ public void testGetDefaultSettings() { assertEquals(5L, pressureSettings.getMinSeqNoLagLimit()); // Check bytes lag variance threshold default value - assertEquals(2.0, pressureSettings.getBytesLagVarianceThreshold(), 0.0d); + assertEquals(2.0, pressureSettings.getBytesLagVarianceFactor(), 0.0d); // Check time lag variance threshold default value - assertEquals(2.0, pressureSettings.getTimeLagVarianceThreshold(), 0.0d); + assertEquals(2.0, pressureSettings.getUploadTimeLagVarianceFactor(), 0.0d); // Check minimum consecutive failures limit default value assertEquals(10, pressureSettings.getMinConsecutiveFailuresLimit()); @@ -80,8 +80,8 @@ public void testGetConfiguredSettings() { Settings settings = Settings.builder() .put(RemoteRefreshSegmentPressureSettings.REMOTE_REFRESH_SEGMENT_PRESSURE_ENABLED.getKey(), true) .put(RemoteRefreshSegmentPressureSettings.MIN_SEQ_NO_LAG_LIMIT.getKey(), 100) - .put(RemoteRefreshSegmentPressureSettings.BYTES_LAG_VARIANCE_THRESHOLD.getKey(), 50.0) - .put(RemoteRefreshSegmentPressureSettings.TIME_LAG_VARIANCE_THRESHOLD.getKey(), 60.0) + .put(RemoteRefreshSegmentPressureSettings.BYTES_LAG_VARIANCE_FACTOR.getKey(), 50.0) + .put(RemoteRefreshSegmentPressureSettings.UPLOAD_TIME_LAG_VARIANCE_FACTOR.getKey(), 60.0) .put(RemoteRefreshSegmentPressureSettings.MIN_CONSECUTIVE_FAILURES_LIMIT.getKey(), 121) .put(RemoteRefreshSegmentPressureSettings.UPLOAD_BYTES_MOVING_AVERAGE_WINDOW_SIZE.getKey(), 102) .put(RemoteRefreshSegmentPressureSettings.UPLOAD_BYTES_PER_SEC_MOVING_AVERAGE_WINDOW_SIZE.getKey(), 103) @@ -100,10 +100,10 @@ public void testGetConfiguredSettings() { assertEquals(100L, pressureSettings.getMinSeqNoLagLimit()); // Check bytes lag variance threshold configured value - assertEquals(50.0, pressureSettings.getBytesLagVarianceThreshold(), 0.0d); + assertEquals(50.0, pressureSettings.getBytesLagVarianceFactor(), 0.0d); // Check time lag variance threshold configured value - assertEquals(60.0, pressureSettings.getTimeLagVarianceThreshold(), 0.0d); + assertEquals(60.0, pressureSettings.getUploadTimeLagVarianceFactor(), 0.0d); // Check minimum consecutive failures limit configured value assertEquals(121, pressureSettings.getMinConsecutiveFailuresLimit()); @@ -128,8 +128,8 @@ public void testUpdateAfterGetDefaultSettings() { Settings newSettings = Settings.builder() .put(RemoteRefreshSegmentPressureSettings.REMOTE_REFRESH_SEGMENT_PRESSURE_ENABLED.getKey(), true) .put(RemoteRefreshSegmentPressureSettings.MIN_SEQ_NO_LAG_LIMIT.getKey(), 100) - .put(RemoteRefreshSegmentPressureSettings.BYTES_LAG_VARIANCE_THRESHOLD.getKey(), 50.0) - .put(RemoteRefreshSegmentPressureSettings.TIME_LAG_VARIANCE_THRESHOLD.getKey(), 60.0) + .put(RemoteRefreshSegmentPressureSettings.BYTES_LAG_VARIANCE_FACTOR.getKey(), 50.0) + .put(RemoteRefreshSegmentPressureSettings.UPLOAD_TIME_LAG_VARIANCE_FACTOR.getKey(), 60.0) .put(RemoteRefreshSegmentPressureSettings.MIN_CONSECUTIVE_FAILURES_LIMIT.getKey(), 121) .put(RemoteRefreshSegmentPressureSettings.UPLOAD_BYTES_MOVING_AVERAGE_WINDOW_SIZE.getKey(), 102) .put(RemoteRefreshSegmentPressureSettings.UPLOAD_BYTES_PER_SEC_MOVING_AVERAGE_WINDOW_SIZE.getKey(), 103) @@ -144,10 +144,10 @@ public void testUpdateAfterGetDefaultSettings() { assertEquals(100L, pressureSettings.getMinSeqNoLagLimit()); // Check bytes lag variance threshold updated - assertEquals(50.0, pressureSettings.getBytesLagVarianceThreshold(), 0.0d); + assertEquals(50.0, pressureSettings.getBytesLagVarianceFactor(), 0.0d); // Check time lag variance threshold updated - assertEquals(60.0, pressureSettings.getTimeLagVarianceThreshold(), 0.0d); + assertEquals(60.0, pressureSettings.getUploadTimeLagVarianceFactor(), 0.0d); // Check minimum consecutive failures limit updated assertEquals(121, pressureSettings.getMinConsecutiveFailuresLimit()); @@ -166,8 +166,8 @@ public void testUpdateAfterGetConfiguredSettings() { Settings settings = Settings.builder() .put(RemoteRefreshSegmentPressureSettings.REMOTE_REFRESH_SEGMENT_PRESSURE_ENABLED.getKey(), true) .put(RemoteRefreshSegmentPressureSettings.MIN_SEQ_NO_LAG_LIMIT.getKey(), 100) - .put(RemoteRefreshSegmentPressureSettings.BYTES_LAG_VARIANCE_THRESHOLD.getKey(), 50.0) - .put(RemoteRefreshSegmentPressureSettings.TIME_LAG_VARIANCE_THRESHOLD.getKey(), 60.0) + .put(RemoteRefreshSegmentPressureSettings.BYTES_LAG_VARIANCE_FACTOR.getKey(), 50.0) + .put(RemoteRefreshSegmentPressureSettings.UPLOAD_TIME_LAG_VARIANCE_FACTOR.getKey(), 60.0) .put(RemoteRefreshSegmentPressureSettings.MIN_CONSECUTIVE_FAILURES_LIMIT.getKey(), 121) .put(RemoteRefreshSegmentPressureSettings.UPLOAD_BYTES_MOVING_AVERAGE_WINDOW_SIZE.getKey(), 102) .put(RemoteRefreshSegmentPressureSettings.UPLOAD_BYTES_PER_SEC_MOVING_AVERAGE_WINDOW_SIZE.getKey(), 103) @@ -181,8 +181,8 @@ public void testUpdateAfterGetConfiguredSettings() { Settings newSettings = Settings.builder() .put(RemoteRefreshSegmentPressureSettings.MIN_SEQ_NO_LAG_LIMIT.getKey(), 80) - .put(RemoteRefreshSegmentPressureSettings.BYTES_LAG_VARIANCE_THRESHOLD.getKey(), 40.0) - .put(RemoteRefreshSegmentPressureSettings.TIME_LAG_VARIANCE_THRESHOLD.getKey(), 50.0) + .put(RemoteRefreshSegmentPressureSettings.BYTES_LAG_VARIANCE_FACTOR.getKey(), 40.0) + .put(RemoteRefreshSegmentPressureSettings.UPLOAD_TIME_LAG_VARIANCE_FACTOR.getKey(), 50.0) .put(RemoteRefreshSegmentPressureSettings.MIN_CONSECUTIVE_FAILURES_LIMIT.getKey(), 111) .put(RemoteRefreshSegmentPressureSettings.UPLOAD_BYTES_MOVING_AVERAGE_WINDOW_SIZE.getKey(), 112) .put(RemoteRefreshSegmentPressureSettings.UPLOAD_BYTES_PER_SEC_MOVING_AVERAGE_WINDOW_SIZE.getKey(), 113) @@ -198,10 +198,10 @@ public void testUpdateAfterGetConfiguredSettings() { assertEquals(80L, pressureSettings.getMinSeqNoLagLimit()); // Check bytes lag variance threshold updated - assertEquals(40.0, pressureSettings.getBytesLagVarianceThreshold(), 0.0d); + assertEquals(40.0, pressureSettings.getBytesLagVarianceFactor(), 0.0d); // Check time lag variance threshold updated - assertEquals(50.0, pressureSettings.getTimeLagVarianceThreshold(), 0.0d); + assertEquals(50.0, pressureSettings.getUploadTimeLagVarianceFactor(), 0.0d); // Check minimum consecutive failures limit updated assertEquals(111, pressureSettings.getMinConsecutiveFailuresLimit()); diff --git a/server/src/test/java/org/opensearch/index/RemoteRefreshSegmentPressureTrackerTests.java b/server/src/test/java/org/opensearch/index/remote/RemoteRefreshSegmentTrackerTests.java similarity index 63% rename from server/src/test/java/org/opensearch/index/RemoteRefreshSegmentPressureTrackerTests.java rename to server/src/test/java/org/opensearch/index/remote/RemoteRefreshSegmentTrackerTests.java index d5ff0201a2472..ffa591c3c886b 100644 --- a/server/src/test/java/org/opensearch/index/RemoteRefreshSegmentPressureTrackerTests.java +++ b/server/src/test/java/org/opensearch/index/remote/RemoteRefreshSegmentTrackerTests.java @@ -6,7 +6,7 @@ * compatible open source license. */ -package org.opensearch.index; +package org.opensearch.index.remote; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.settings.ClusterSettings; @@ -21,7 +21,7 @@ import static org.mockito.Mockito.mock; -public class RemoteRefreshSegmentPressureTrackerTests extends OpenSearchTestCase { +public class RemoteRefreshSegmentTrackerTests extends OpenSearchTestCase { private RemoteRefreshSegmentPressureSettings pressureSettings; @@ -31,7 +31,7 @@ public class RemoteRefreshSegmentPressureTrackerTests extends OpenSearchTestCase private ShardId shardId; - private RemoteRefreshSegmentPressureTracker pressureTracker; + private RemoteRefreshSegmentTracker pressureTracker; @Override public void setUp() throws Exception { @@ -57,40 +57,70 @@ public void tearDown() throws Exception { } public void testGetShardId() { - pressureTracker = new RemoteRefreshSegmentPressureTracker(shardId, pressureSettings); + pressureTracker = new RemoteRefreshSegmentTracker( + shardId, + pressureSettings.getUploadBytesMovingAverageWindowSize(), + pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), + pressureSettings.getUploadTimeMovingAverageWindowSize() + ); assertEquals(shardId, pressureTracker.getShardId()); } public void testUpdateLocalRefreshSeqNo() { - pressureTracker = new RemoteRefreshSegmentPressureTracker(shardId, pressureSettings); + pressureTracker = new RemoteRefreshSegmentTracker( + shardId, + pressureSettings.getUploadBytesMovingAverageWindowSize(), + pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), + pressureSettings.getUploadTimeMovingAverageWindowSize() + ); long refreshSeqNo = 2; pressureTracker.updateLocalRefreshSeqNo(refreshSeqNo); assertEquals(refreshSeqNo, pressureTracker.getLocalRefreshSeqNo()); } public void testUpdateRemoteRefreshSeqNo() { - pressureTracker = new RemoteRefreshSegmentPressureTracker(shardId, pressureSettings); + pressureTracker = new RemoteRefreshSegmentTracker( + shardId, + pressureSettings.getUploadBytesMovingAverageWindowSize(), + pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), + pressureSettings.getUploadTimeMovingAverageWindowSize() + ); long refreshSeqNo = 4; pressureTracker.updateRemoteRefreshSeqNo(refreshSeqNo); assertEquals(refreshSeqNo, pressureTracker.getRemoteRefreshSeqNo()); } public void testUpdateLocalRefreshTimeMs() { - pressureTracker = new RemoteRefreshSegmentPressureTracker(shardId, pressureSettings); + pressureTracker = new RemoteRefreshSegmentTracker( + shardId, + pressureSettings.getUploadBytesMovingAverageWindowSize(), + pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), + pressureSettings.getUploadTimeMovingAverageWindowSize() + ); long refreshTimeMs = System.nanoTime() / 1_000_000L + randomIntBetween(10, 100); pressureTracker.updateLocalRefreshTimeMs(refreshTimeMs); assertEquals(refreshTimeMs, pressureTracker.getLocalRefreshTimeMs()); } public void testUpdateRemoteRefreshTimeMs() { - pressureTracker = new RemoteRefreshSegmentPressureTracker(shardId, pressureSettings); + pressureTracker = new RemoteRefreshSegmentTracker( + shardId, + pressureSettings.getUploadBytesMovingAverageWindowSize(), + pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), + pressureSettings.getUploadTimeMovingAverageWindowSize() + ); long refreshTimeMs = System.nanoTime() / 1_000_000 + randomIntBetween(10, 100); pressureTracker.updateRemoteRefreshTimeMs(refreshTimeMs); assertEquals(refreshTimeMs, pressureTracker.getRemoteRefreshTimeMs()); } public void testComputeSeqNoLagOnUpdate() { - pressureTracker = new RemoteRefreshSegmentPressureTracker(shardId, pressureSettings); + pressureTracker = new RemoteRefreshSegmentTracker( + shardId, + pressureSettings.getUploadBytesMovingAverageWindowSize(), + pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), + pressureSettings.getUploadTimeMovingAverageWindowSize() + ); int localRefreshSeqNo = randomIntBetween(50, 100); int remoteRefreshSeqNo = randomIntBetween(20, 50); pressureTracker.updateLocalRefreshSeqNo(localRefreshSeqNo); @@ -100,7 +130,12 @@ public void testComputeSeqNoLagOnUpdate() { } public void testComputeTimeLagOnUpdate() { - pressureTracker = new RemoteRefreshSegmentPressureTracker(shardId, pressureSettings); + pressureTracker = new RemoteRefreshSegmentTracker( + shardId, + pressureSettings.getUploadBytesMovingAverageWindowSize(), + pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), + pressureSettings.getUploadTimeMovingAverageWindowSize() + ); long currentLocalRefreshTimeMs = pressureTracker.getLocalRefreshTimeMs(); long currentTimeMs = System.nanoTime() / 1_000_000L; long localRefreshTimeMs = currentTimeMs + randomIntBetween(100, 500); @@ -112,7 +147,12 @@ public void testComputeTimeLagOnUpdate() { } public void testAddUploadBytesStarted() { - pressureTracker = new RemoteRefreshSegmentPressureTracker(shardId, pressureSettings); + pressureTracker = new RemoteRefreshSegmentTracker( + shardId, + pressureSettings.getUploadBytesMovingAverageWindowSize(), + pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), + pressureSettings.getUploadTimeMovingAverageWindowSize() + ); long bytesToAdd = randomLongBetween(1000, 1000000); pressureTracker.addUploadBytesStarted(bytesToAdd); assertEquals(bytesToAdd, pressureTracker.getUploadBytesStarted()); @@ -122,7 +162,12 @@ public void testAddUploadBytesStarted() { } public void testAddUploadBytesFailed() { - pressureTracker = new RemoteRefreshSegmentPressureTracker(shardId, pressureSettings); + pressureTracker = new RemoteRefreshSegmentTracker( + shardId, + pressureSettings.getUploadBytesMovingAverageWindowSize(), + pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), + pressureSettings.getUploadTimeMovingAverageWindowSize() + ); long bytesToAdd = randomLongBetween(1000, 1000000); pressureTracker.addUploadBytesFailed(bytesToAdd); assertEquals(bytesToAdd, pressureTracker.getUploadBytesFailed()); @@ -132,7 +177,12 @@ public void testAddUploadBytesFailed() { } public void testAddUploadBytesSucceeded() { - pressureTracker = new RemoteRefreshSegmentPressureTracker(shardId, pressureSettings); + pressureTracker = new RemoteRefreshSegmentTracker( + shardId, + pressureSettings.getUploadBytesMovingAverageWindowSize(), + pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), + pressureSettings.getUploadTimeMovingAverageWindowSize() + ); long bytesToAdd = randomLongBetween(1000, 1000000); pressureTracker.addUploadBytesSucceeded(bytesToAdd); assertEquals(bytesToAdd, pressureTracker.getUploadBytesSucceeded()); @@ -142,7 +192,12 @@ public void testAddUploadBytesSucceeded() { } public void testGetInflightUploadBytes() { - pressureTracker = new RemoteRefreshSegmentPressureTracker(shardId, pressureSettings); + pressureTracker = new RemoteRefreshSegmentTracker( + shardId, + pressureSettings.getUploadBytesMovingAverageWindowSize(), + pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), + pressureSettings.getUploadTimeMovingAverageWindowSize() + ); long bytesStarted = randomLongBetween(10000, 100000); long bytesSucceeded = randomLongBetween(1000, 10000); long bytesFailed = randomLongBetween(100, 1000); @@ -153,7 +208,12 @@ public void testGetInflightUploadBytes() { } public void testIncrementTotalUploadsStarted() { - pressureTracker = new RemoteRefreshSegmentPressureTracker(shardId, pressureSettings); + pressureTracker = new RemoteRefreshSegmentTracker( + shardId, + pressureSettings.getUploadBytesMovingAverageWindowSize(), + pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), + pressureSettings.getUploadTimeMovingAverageWindowSize() + ); pressureTracker.incrementTotalUploadsStarted(); assertEquals(1, pressureTracker.getTotalUploadsStarted()); pressureTracker.incrementTotalUploadsStarted(); @@ -161,7 +221,12 @@ public void testIncrementTotalUploadsStarted() { } public void testIncrementTotalUploadsFailed() { - pressureTracker = new RemoteRefreshSegmentPressureTracker(shardId, pressureSettings); + pressureTracker = new RemoteRefreshSegmentTracker( + shardId, + pressureSettings.getUploadBytesMovingAverageWindowSize(), + pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), + pressureSettings.getUploadTimeMovingAverageWindowSize() + ); pressureTracker.incrementTotalUploadsFailed(); assertEquals(1, pressureTracker.getTotalUploadsFailed()); pressureTracker.incrementTotalUploadsFailed(); @@ -169,7 +234,12 @@ public void testIncrementTotalUploadsFailed() { } public void testIncrementTotalUploadSucceeded() { - pressureTracker = new RemoteRefreshSegmentPressureTracker(shardId, pressureSettings); + pressureTracker = new RemoteRefreshSegmentTracker( + shardId, + pressureSettings.getUploadBytesMovingAverageWindowSize(), + pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), + pressureSettings.getUploadTimeMovingAverageWindowSize() + ); pressureTracker.incrementTotalUploadSucceeded(); assertEquals(1, pressureTracker.getTotalUploadsSucceeded()); pressureTracker.incrementTotalUploadSucceeded(); @@ -177,7 +247,12 @@ public void testIncrementTotalUploadSucceeded() { } public void testGetInflightUploads() { - pressureTracker = new RemoteRefreshSegmentPressureTracker(shardId, pressureSettings); + pressureTracker = new RemoteRefreshSegmentTracker( + shardId, + pressureSettings.getUploadBytesMovingAverageWindowSize(), + pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), + pressureSettings.getUploadTimeMovingAverageWindowSize() + ); pressureTracker.incrementTotalUploadsStarted(); assertEquals(1, pressureTracker.getInflightUploads()); pressureTracker.incrementTotalUploadsStarted(); @@ -189,7 +264,12 @@ public void testGetInflightUploads() { } public void testIncrementRejectionCount() { - pressureTracker = new RemoteRefreshSegmentPressureTracker(shardId, pressureSettings); + pressureTracker = new RemoteRefreshSegmentTracker( + shardId, + pressureSettings.getUploadBytesMovingAverageWindowSize(), + pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), + pressureSettings.getUploadTimeMovingAverageWindowSize() + ); pressureTracker.incrementRejectionCount(); assertEquals(1, pressureTracker.getRejectionCount()); pressureTracker.incrementRejectionCount(); @@ -197,7 +277,12 @@ public void testIncrementRejectionCount() { } public void testGetConsecutiveFailureCount() { - pressureTracker = new RemoteRefreshSegmentPressureTracker(shardId, pressureSettings); + pressureTracker = new RemoteRefreshSegmentTracker( + shardId, + pressureSettings.getUploadBytesMovingAverageWindowSize(), + pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), + pressureSettings.getUploadTimeMovingAverageWindowSize() + ); pressureTracker.incrementTotalUploadsFailed(); assertEquals(1, pressureTracker.getConsecutiveFailureCount()); pressureTracker.incrementTotalUploadsFailed(); @@ -207,7 +292,12 @@ public void testGetConsecutiveFailureCount() { } public void testComputeBytesLag() { - pressureTracker = new RemoteRefreshSegmentPressureTracker(shardId, pressureSettings); + pressureTracker = new RemoteRefreshSegmentTracker( + shardId, + pressureSettings.getUploadBytesMovingAverageWindowSize(), + pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), + pressureSettings.getUploadTimeMovingAverageWindowSize() + ); // Create local file size map Map fileSizeMap = new HashMap<>(); @@ -231,7 +321,12 @@ public void testComputeBytesLag() { } public void testIsUploadBytesAverageReady() { - pressureTracker = new RemoteRefreshSegmentPressureTracker(shardId, pressureSettings); + pressureTracker = new RemoteRefreshSegmentTracker( + shardId, + pressureSettings.getUploadBytesMovingAverageWindowSize(), + pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), + pressureSettings.getUploadTimeMovingAverageWindowSize() + ); assertFalse(pressureTracker.isUploadBytesAverageReady()); long sum = 0; @@ -253,7 +348,12 @@ public void testIsUploadBytesAverageReady() { } public void testIsUploadBytesPerSecAverageReady() { - pressureTracker = new RemoteRefreshSegmentPressureTracker(shardId, pressureSettings); + pressureTracker = new RemoteRefreshSegmentTracker( + shardId, + pressureSettings.getUploadBytesMovingAverageWindowSize(), + pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), + pressureSettings.getUploadTimeMovingAverageWindowSize() + ); assertFalse(pressureTracker.isUploadBytesPerSecAverageReady()); long sum = 0; @@ -275,7 +375,12 @@ public void testIsUploadBytesPerSecAverageReady() { } public void testIsUploadTimeMsAverageReady() { - pressureTracker = new RemoteRefreshSegmentPressureTracker(shardId, pressureSettings); + pressureTracker = new RemoteRefreshSegmentTracker( + shardId, + pressureSettings.getUploadBytesMovingAverageWindowSize(), + pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), + pressureSettings.getUploadTimeMovingAverageWindowSize() + ); assertFalse(pressureTracker.isUploadTimeMsAverageReady()); long sum = 0; From fe098252c5cd8c8ac827b2dcf34debbb5cfaf5b8 Mon Sep 17 00:00:00 2001 From: Ashish Singh Date: Tue, 25 Apr 2023 19:17:26 +0530 Subject: [PATCH 09/17] Incorporate PR review feedback Signed-off-by: Ashish Singh --- .../opensearch/common/util/MovingAverage.java | 32 +++++++++++++-- .../remote/RemoteRefreshSegmentTracker.java | 35 +++++++++++++--- .../common/util/MovingAverageTests.java | 40 +++++++++++++++++++ 3 files changed, 97 insertions(+), 10 deletions(-) diff --git a/server/src/main/java/org/opensearch/common/util/MovingAverage.java b/server/src/main/java/org/opensearch/common/util/MovingAverage.java index 650ba62ecd8c8..79c762f20a293 100644 --- a/server/src/main/java/org/opensearch/common/util/MovingAverage.java +++ b/server/src/main/java/org/opensearch/common/util/MovingAverage.java @@ -22,14 +22,38 @@ public class MovingAverage { private double average = 0; public MovingAverage(int windowSize) { - if (windowSize <= 0) { - throw new IllegalArgumentException("window size must be greater than zero"); - } - + checkWindowSize(windowSize); this.windowSize = windowSize; this.observations = new long[windowSize]; } + public MovingAverage(int newWindowSize, MovingAverage oldMovingAverage) { + checkWindowSize(newWindowSize); + windowSize = newWindowSize; + observations = new long[newWindowSize]; + + // Start is inclusive, but end is exclusive + long start, end = oldMovingAverage.count; + if (oldMovingAverage.isReady() == false) { + start = 0; + } else { + start = end - oldMovingAverage.windowSize; + } + // If the newWindow Size is smaller than the elements eligible to be copied over, then we adjust the start value + if (end - start > newWindowSize) { + start = end - newWindowSize; + } + for (int i = (int) start; i < end; i++) { + record(oldMovingAverage.observations[i % oldMovingAverage.observations.length]); + } + } + + private void checkWindowSize(int size) { + if (size <= 0) { + throw new IllegalArgumentException("window size must be greater than zero"); + } + } + /** * Records a new observation and evicts the n-th last observation. */ diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentTracker.java b/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentTracker.java index 839f0d7423218..1c90cda566797 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentTracker.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentTracker.java @@ -136,18 +136,27 @@ public RemoteRefreshSegmentTracker( */ 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(); + ShardId getShardId() { return shardId; } @@ -333,7 +342,9 @@ boolean isUploadBytesAverageReady() { } void addUploadBytes(long size) { - this.uploadBytesMovingAverageReference.get().record(size); + synchronized (uploadBytesMutex) { + this.uploadBytesMovingAverageReference.get().record(size); + } } /** @@ -342,7 +353,9 @@ void addUploadBytes(long size) { * @param updatedSize the updated size */ void updateUploadBytesMovingAverageWindowSize(int updatedSize) { - this.uploadBytesMovingAverageReference.set(new MovingAverage(updatedSize)); + synchronized (uploadBytesMutex) { + this.uploadBytesMovingAverageReference.set(new MovingAverage(updatedSize, this.uploadBytesMovingAverageReference.get())); + } } boolean isUploadBytesPerSecAverageReady() { @@ -354,7 +367,9 @@ boolean isUploadBytesPerSecAverageReady() { } void addUploadBytesPerSec(long bytesPerSec) { - this.uploadBytesPerSecMovingAverageReference.get().record(bytesPerSec); + synchronized (uploadBytesPerSecMutex) { + this.uploadBytesPerSecMovingAverageReference.get().record(bytesPerSec); + } } /** @@ -363,7 +378,11 @@ void addUploadBytesPerSec(long bytesPerSec) { * @param updatedSize the updated size */ void updateUploadBytesPerSecMovingAverageWindowSize(int updatedSize) { - this.uploadBytesPerSecMovingAverageReference.set(new MovingAverage(updatedSize)); + synchronized (uploadBytesPerSecMutex) { + this.uploadBytesPerSecMovingAverageReference.set( + new MovingAverage(updatedSize, this.uploadBytesPerSecMovingAverageReference.get()) + ); + } } boolean isUploadTimeMsAverageReady() { @@ -375,7 +394,9 @@ boolean isUploadTimeMsAverageReady() { } void addUploadTimeMs(long timeMs) { - this.uploadTimeMsMovingAverageReference.get().record(timeMs); + synchronized (uploadTimeMsMutex) { + this.uploadTimeMsMovingAverageReference.get().record(timeMs); + } } /** @@ -384,6 +405,8 @@ void addUploadTimeMs(long timeMs) { * @param updatedSize the updated size */ void updateUploadTimeMsMovingAverageWindowSize(int updatedSize) { - this.uploadTimeMsMovingAverageReference.set(new MovingAverage(updatedSize)); + synchronized (uploadTimeMsMutex) { + this.uploadTimeMsMovingAverageReference.set(new MovingAverage(updatedSize, this.uploadTimeMsMovingAverageReference.get())); + } } } diff --git a/server/src/test/java/org/opensearch/common/util/MovingAverageTests.java b/server/src/test/java/org/opensearch/common/util/MovingAverageTests.java index 415058992e081..935176405a57b 100644 --- a/server/src/test/java/org/opensearch/common/util/MovingAverageTests.java +++ b/server/src/test/java/org/opensearch/common/util/MovingAverageTests.java @@ -46,4 +46,44 @@ public void testMovingAverageWithZeroSize() { fail("exception should have been thrown"); } + + public void testUpdateMovingAverageWindowSize() { + MovingAverage ma = new MovingAverage(5); + ma.record(1); + ma.record(2); + ma.record(3); + double avg = ma.getAverage(); + + // Test case 1 - Not ready and increasing size + MovingAverage newMa = new MovingAverage(10, ma); + assertEquals(avg, newMa.getAverage(), 0.0d); + + // Test case 2 - Not ready and decreasing size + newMa = new MovingAverage(2, ma); + assertEquals(2.5, newMa.getAverage(), 0.0d); + + // Test case 3 - Ready and increasing size + ma.record(4); + ma.record(5); + ma.record(6); + assertEquals(4.0, ma.getAverage(), 0.0d); + + newMa = new MovingAverage(10, ma); + assertEquals(ma.getAverage(), newMa.getAverage(), 0.0d); + + // Test case 4 - Ready and decreasing size + newMa = new MovingAverage(3, ma); + assertEquals(5.0, newMa.getAverage(), 0.0d); + + // Test case 5 - Ready, array overwritten and increasing size + for (int i = 7; i < 20; i++) { + ma.record(i); + } + newMa = new MovingAverage(6, ma); + assertEquals(ma.getAverage(), newMa.getAverage(), 0.0d); + + // Test case 6 - Ready, array overwritten and decreasing size + newMa = new MovingAverage(3, ma); + assertEquals(18.0, newMa.getAverage(), 0.0d); + } } From 9973468d5ef859bf34874e399d4f557f01a07901 Mon Sep 17 00:00:00 2001 From: Ashish Singh Date: Wed, 26 Apr 2023 01:05:45 +0530 Subject: [PATCH 10/17] Tracker deletion on hook beforeIndexShardClosed Signed-off-by: Ashish Singh --- .../index/remote/RemoteRefreshSegmentPressureService.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureService.java b/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureService.java index 38f79e264ef14..43c3f468269b5 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureService.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureService.java @@ -74,7 +74,7 @@ public void afterIndexShardCreated(IndexShard indexShard) { } @Override - public void beforeIndexShardClosed(ShardId shardId, IndexShard indexShard, Settings indexSettings) { + public void afterIndexShardClosed(ShardId shardId, IndexShard indexShard, Settings indexSettings) { if (indexShard.indexSettings().isRemoteStoreEnabled() == false) { return; } From 91945a7a72c116556b3ef5afcc8716d97dd89481 Mon Sep 17 00:00:00 2001 From: Ashish Singh Date: Wed, 26 Apr 2023 10:10:42 +0530 Subject: [PATCH 11/17] Tracker deletion on hook beforeIndexShardClosed - Fix test Signed-off-by: Ashish Singh --- .../remote/RemoteRefreshSegmentPressureServiceTests.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/server/src/test/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureServiceTests.java b/server/src/test/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureServiceTests.java index 2a8bdf54b01b2..b4cf6d14c28f5 100644 --- a/server/src/test/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureServiceTests.java +++ b/server/src/test/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureServiceTests.java @@ -93,7 +93,7 @@ public void testAfterIndexShardCreatedForNonRemoteBackedIndex() { assertNull(pressureService.getPressureTracker(testShardId)); } - public void testBeforeIndexShardClosed() { + public void testAfterIndexShardClosed() { Settings settings = Settings.builder().put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, "true").build(); IndexSettings indexSettings = IndexSettingsModule.newIndexSettings("test_index", settings); ShardId testShardId = new ShardId("index", "uuid", 0); @@ -104,7 +104,7 @@ public void testBeforeIndexShardClosed() { pressureService.afterIndexShardCreated(indexShard); assertNotNull(pressureService.getPressureTracker(testShardId)); - pressureService.beforeIndexShardClosed(testShardId, indexShard, settings); + pressureService.afterIndexShardClosed(testShardId, indexShard, settings); assertNull(pressureService.getPressureTracker(testShardId)); } From 2839cf7bea9e4ea511417dfe359adf183db57890 Mon Sep 17 00:00:00 2001 From: Ashish Singh Date: Thu, 27 Apr 2023 13:35:24 +0530 Subject: [PATCH 12/17] Introduce abstract class LagValidator and rejection count per type Signed-off-by: Ashish Singh --- .../RemoteRefreshSegmentPressureService.java | 251 ++++++++++++------ .../remote/RemoteRefreshSegmentTracker.java | 13 + ...oteRefreshSegmentPressureServiceTests.java | 10 +- 3 files changed, 185 insertions(+), 89 deletions(-) diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureService.java b/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureService.java index 43c3f468269b5..df12c598b049b 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureService.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureService.java @@ -19,6 +19,8 @@ import org.opensearch.index.shard.IndexShard; import org.opensearch.index.shard.ShardId; +import java.util.Arrays; +import java.util.List; import java.util.Locale; import java.util.Map; import java.util.function.BiConsumer; @@ -40,9 +42,17 @@ public class RemoteRefreshSegmentPressureService implements IndexEventListener { */ private final RemoteRefreshSegmentPressureSettings pressureSettings; + private final List lagValidators; + @Inject public RemoteRefreshSegmentPressureService(ClusterService clusterService, Settings settings) { pressureSettings = new RemoteRefreshSegmentPressureSettings(clusterService, settings, this); + lagValidators = Arrays.asList( + new RefreshSeqNoLagValidator(pressureSettings), + new BytesLagValidator(pressureSettings), + new TimeLagValidator(pressureSettings), + new ConsecutiveFailureValidator(pressureSettings) + ); } /** @@ -51,7 +61,7 @@ public RemoteRefreshSegmentPressureService(ClusterService clusterService, Settin * @param shardId shard id * @return the tracker if index is remote-backed, else null. */ - public RemoteRefreshSegmentTracker getPressureTracker(ShardId shardId) { + public RemoteRefreshSegmentTracker getRemoteRefreshSegmentTracker(ShardId shardId) { return trackerMap.get(shardId); } @@ -92,118 +102,191 @@ public boolean isSegmentsUploadBackpressureEnabled() { } public void validateSegmentsUploadLag(ShardId shardId) { - RemoteRefreshSegmentTracker pressureTracker = getPressureTracker(shardId); + RemoteRefreshSegmentTracker remoteRefreshSegmentTracker = getRemoteRefreshSegmentTracker(shardId); // Check if refresh checkpoint (a.k.a. seq number) lag is 2 or below - this is to handle segment merges that can // increase the bytes to upload almost suddenly. - if (pressureTracker.getSeqNoLag() <= 2) { + if (remoteRefreshSegmentTracker.getSeqNoLag() <= 1) { return; } - // Check if the remote store seq no lag is above the min seq no lag limit - validateSeqNoLag(pressureTracker, shardId); + for (LagValidator lagValidator : lagValidators) { + if (lagValidator.validate(remoteRefreshSegmentTracker, shardId) == false) { + remoteRefreshSegmentTracker.incrementRejectionCount(lagValidator.name()); + throw new OpenSearchRejectedExecutionException(lagValidator.rejectionMessage(remoteRefreshSegmentTracker, shardId)); + } + } + } - // Check if the remote store is lagging more than the upload bytes average multiplied by a variance factor - validateBytesLag(pressureTracker, shardId); + void updateUploadBytesMovingAverageWindowSize(int updatedSize) { + updateMovingAverageWindowSize(RemoteRefreshSegmentTracker::updateUploadBytesMovingAverageWindowSize, updatedSize); + } - // Check if the remote store is lagging more than the upload time average multiplied by a variance factor - validateTimeLag(pressureTracker, shardId); + void updateUploadBytesPerSecMovingAverageWindowSize(int updatedSize) { + updateMovingAverageWindowSize(RemoteRefreshSegmentTracker::updateUploadBytesPerSecMovingAverageWindowSize, updatedSize); + } - // Check if consecutive failure limit has been breached - validateConsecutiveFailureLimitBreached(pressureTracker, shardId); + void updateUploadTimeMsMovingAverageWindowSize(int updatedSize) { + updateMovingAverageWindowSize(RemoteRefreshSegmentTracker::updateUploadTimeMsMovingAverageWindowSize, updatedSize); } - private void validateSeqNoLag(RemoteRefreshSegmentTracker pressureTracker, ShardId shardId) { - // Check if the remote store seq no lag is above the min seq no lag limit - if (pressureTracker.getSeqNoLag() > pressureSettings.getMinSeqNoLagLimit()) { - pressureTracker.incrementRejectionCount(); - throw new OpenSearchRejectedExecutionException( - String.format( - Locale.ROOT, - "rejected execution on primary shard:%s due to remote segments lagging behind local segments." - + "remote_refresh_seq_no:%s local_refresh_seq_no:%s", - shardId, - pressureTracker.getRemoteRefreshSeqNo(), - pressureTracker.getLocalRefreshSeqNo() - ) - ); + void updateMovingAverageWindowSize(BiConsumer biConsumer, int updatedSize) { + trackerMap.values().forEach(tracker -> biConsumer.accept(tracker, updatedSize)); + } + + /** + * Abstract class for validating if lag is acceptable or not. + */ + private static abstract class LagValidator { + + final RemoteRefreshSegmentPressureSettings pressureSettings; + + private LagValidator(RemoteRefreshSegmentPressureSettings pressureSettings) { + this.pressureSettings = pressureSettings; } + + /** + * Validates the lag and returns value accordingly. + * + * @param pressureTracker tracker which holds information about the shard. + * @param shardId shard id of the {@code IndexShard} currently being validated. + * @return true if successfully validated that lag is acceptable. + */ + abstract boolean validate(RemoteRefreshSegmentTracker pressureTracker, ShardId shardId); + + /** + * Returns the name of the lag validator. + * + * @return the name using class name. + */ + final String name() { + return this.getClass().getSimpleName(); + } + + abstract String rejectionMessage(RemoteRefreshSegmentTracker pressureTracker, ShardId shardId); } - private void validateBytesLag(RemoteRefreshSegmentTracker pressureTracker, ShardId shardId) { - if (pressureTracker.isUploadBytesAverageReady() == false) { - logger.trace("upload bytes moving average is not ready"); - return; + /** + * Check if the remote store seq no lag is above the min seq no lag limit + */ + private static class RefreshSeqNoLagValidator extends LagValidator { + + private RefreshSeqNoLagValidator(RemoteRefreshSegmentPressureSettings pressureSettings) { + super(pressureSettings); + } + + @Override + public boolean validate(RemoteRefreshSegmentTracker pressureTracker, ShardId shardId) { + // Check if the remote store seq no lag is above the min seq no lag limit + return pressureTracker.getSeqNoLag() <= pressureSettings.getMinSeqNoLagLimit(); } - double dynamicBytesLagThreshold = pressureTracker.getUploadBytesAverage() * pressureSettings.getBytesLagVarianceFactor(); - long bytesLag = pressureTracker.getBytesLag(); - if (bytesLag > dynamicBytesLagThreshold) { - pressureTracker.incrementRejectionCount(); - throw new OpenSearchRejectedExecutionException( - String.format( - Locale.ROOT, - "rejected execution on primary shard:%s due to remote segments lagging behind local segments." - + "bytes_lag:%s dynamic_bytes_lag_threshold:%s", - shardId, - bytesLag, - dynamicBytesLagThreshold - ) + + @Override + String rejectionMessage(RemoteRefreshSegmentTracker pressureTracker, ShardId shardId) { + return String.format( + Locale.ROOT, + "rejected execution on primary shard:%s due to remote segments lagging behind local segments." + + "remote_refresh_seq_no:%s local_refresh_seq_no:%s", + shardId, + pressureTracker.getRemoteRefreshSeqNo(), + pressureTracker.getLocalRefreshSeqNo() ); } } - private void validateTimeLag(RemoteRefreshSegmentTracker pressureTracker, ShardId shardId) { - if (pressureTracker.isUploadTimeMsAverageReady() == false) { - logger.trace("upload time moving average is not ready"); - return; + /** + * Check if the remote store is lagging more than the upload bytes average multiplied by a variance factor + */ + private static class BytesLagValidator extends LagValidator { + + private BytesLagValidator(RemoteRefreshSegmentPressureSettings pressureSettings) { + super(pressureSettings); + } + + @Override + public boolean validate(RemoteRefreshSegmentTracker pressureTracker, ShardId shardId) { + if (pressureTracker.isUploadBytesAverageReady() == false) { + logger.trace("upload bytes moving average is not ready"); + return true; + } + double dynamicBytesLagThreshold = pressureTracker.getUploadBytesAverage() * pressureSettings.getBytesLagVarianceFactor(); + long bytesLag = pressureTracker.getBytesLag(); + return bytesLag <= dynamicBytesLagThreshold; } - long timeLag = pressureTracker.getTimeMsLag(); - double dynamicTimeLagThreshold = pressureTracker.getUploadTimeMsAverage() * pressureSettings.getUploadTimeLagVarianceFactor(); - if (timeLag > dynamicTimeLagThreshold) { - pressureTracker.incrementRejectionCount(); - throw new OpenSearchRejectedExecutionException( - String.format( - Locale.ROOT, - "rejected execution on primary shard:%s due to remote segments lagging behind local segments." - + "time_lag:%s ms dynamic_time_lag_threshold:%s ms", - shardId, - timeLag, - dynamicTimeLagThreshold - ) + + @Override + public String rejectionMessage(RemoteRefreshSegmentTracker pressureTracker, ShardId shardId) { + double dynamicBytesLagThreshold = pressureTracker.getUploadBytesAverage() * pressureSettings.getBytesLagVarianceFactor(); + return String.format( + Locale.ROOT, + "rejected execution on primary shard:%s due to remote segments lagging behind local segments." + + "bytes_lag:%s dynamic_bytes_lag_threshold:%s", + shardId, + pressureTracker.getBytesLag(), + dynamicBytesLagThreshold ); } } - private void validateConsecutiveFailureLimitBreached(RemoteRefreshSegmentTracker pressureTracker, ShardId shardId) { - int failureStreakCount = pressureTracker.getConsecutiveFailureCount(); - int minConsecutiveFailureThreshold = pressureSettings.getMinConsecutiveFailuresLimit(); - if (failureStreakCount > minConsecutiveFailureThreshold) { - pressureTracker.incrementRejectionCount(); - throw new OpenSearchRejectedExecutionException( - String.format( - Locale.ROOT, - "rejected execution on primary shard:%s due to remote segments lagging behind local segments." - + "failure_streak_count:%s min_consecutive_failure_threshold:%s", - shardId, - failureStreakCount, - minConsecutiveFailureThreshold - ) + /** + * Check if the remote store is lagging more than the upload time average multiplied by a variance factor + */ + private static class TimeLagValidator extends LagValidator { + + private TimeLagValidator(RemoteRefreshSegmentPressureSettings pressureSettings) { + super(pressureSettings); + } + + @Override + public boolean validate(RemoteRefreshSegmentTracker pressureTracker, ShardId shardId) { + if (pressureTracker.isUploadTimeMsAverageReady() == false) { + logger.trace("upload time moving average is not ready"); + return true; + } + long timeLag = pressureTracker.getTimeMsLag(); + double dynamicTimeLagThreshold = pressureTracker.getUploadTimeMsAverage() * pressureSettings.getUploadTimeLagVarianceFactor(); + return timeLag <= dynamicTimeLagThreshold; + } + + @Override + public String rejectionMessage(RemoteRefreshSegmentTracker pressureTracker, ShardId shardId) { + double dynamicTimeLagThreshold = pressureTracker.getUploadTimeMsAverage() * pressureSettings.getUploadTimeLagVarianceFactor(); + return String.format( + Locale.ROOT, + "rejected execution on primary shard:%s due to remote segments lagging behind local segments." + + "time_lag:%s ms dynamic_time_lag_threshold:%s ms", + shardId, + pressureTracker.getTimeMsLag(), + dynamicTimeLagThreshold ); } } - void updateUploadBytesMovingAverageWindowSize(int updatedSize) { - updateMovingAverageWindowSize(RemoteRefreshSegmentTracker::updateUploadBytesMovingAverageWindowSize, updatedSize); - } + /** + * Check if consecutive failure limit has been breached + */ + private static class ConsecutiveFailureValidator extends LagValidator { - void updateUploadBytesPerSecMovingAverageWindowSize(int updatedSize) { - updateMovingAverageWindowSize(RemoteRefreshSegmentTracker::updateUploadBytesPerSecMovingAverageWindowSize, updatedSize); - } + private ConsecutiveFailureValidator(RemoteRefreshSegmentPressureSettings pressureSettings) { + super(pressureSettings); + } - void updateUploadTimeMsMovingAverageWindowSize(int updatedSize) { - updateMovingAverageWindowSize(RemoteRefreshSegmentTracker::updateUploadTimeMsMovingAverageWindowSize, updatedSize); - } + @Override + public boolean validate(RemoteRefreshSegmentTracker pressureTracker, ShardId shardId) { + int failureStreakCount = pressureTracker.getConsecutiveFailureCount(); + int minConsecutiveFailureThreshold = pressureSettings.getMinConsecutiveFailuresLimit(); + return failureStreakCount <= minConsecutiveFailureThreshold; + } - void updateMovingAverageWindowSize(BiConsumer biConsumer, int updatedSize) { - trackerMap.values().forEach(tracker -> biConsumer.accept(tracker, updatedSize)); + @Override + public String rejectionMessage(RemoteRefreshSegmentTracker pressureTracker, ShardId shardId) { + return String.format( + Locale.ROOT, + "rejected execution on primary shard:%s due to remote segments lagging behind local segments." + + "failure_streak_count:%s min_consecutive_failure_threshold:%s", + shardId, + pressureTracker.getConsecutiveFailureCount(), + pressureSettings.getMinConsecutiveFailuresLimit() + ); + } } } diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentTracker.java b/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentTracker.java index 1c90cda566797..cbffda355d1aa 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentTracker.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentTracker.java @@ -110,6 +110,11 @@ public RemoteRefreshSegmentTracker( */ private final AtomicLong rejectionCount = new AtomicLong(); + /** + * Keeps track of rejection count with each rejection reason. + */ + private final Map rejectionCountMap = ConcurrentCollections.newConcurrentMap(); + /** * Map of name to size of the segment files created as part of the most recent refresh. */ @@ -303,6 +308,14 @@ void incrementRejectionCount() { rejectionCount.incrementAndGet(); } + void incrementRejectionCount(String rejectionReason) { + rejectionCountMap.computeIfAbsent(rejectionReason, k -> new AtomicLong()).incrementAndGet(); + } + + long getRejectionCount(String rejectionReason) { + return rejectionCountMap.get(rejectionReason).get(); + } + Map getLatestLocalFileNameLengthMap() { return latestLocalFileNameLengthMap; } diff --git a/server/src/test/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureServiceTests.java b/server/src/test/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureServiceTests.java index b4cf6d14c28f5..39a49100ec755 100644 --- a/server/src/test/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureServiceTests.java +++ b/server/src/test/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureServiceTests.java @@ -78,7 +78,7 @@ public void testAfterIndexShardCreatedForRemoteBackedIndex() { when(indexShard.shardId()).thenReturn(testShardId); pressureService = new RemoteRefreshSegmentPressureService(clusterService, Settings.EMPTY); pressureService.afterIndexShardCreated(indexShard); - assertNotNull(pressureService.getPressureTracker(testShardId)); + assertNotNull(pressureService.getRemoteRefreshSegmentTracker(testShardId)); } public void testAfterIndexShardCreatedForNonRemoteBackedIndex() { @@ -90,7 +90,7 @@ public void testAfterIndexShardCreatedForNonRemoteBackedIndex() { when(indexShard.shardId()).thenReturn(testShardId); pressureService = new RemoteRefreshSegmentPressureService(clusterService, Settings.EMPTY); pressureService.afterIndexShardCreated(indexShard); - assertNull(pressureService.getPressureTracker(testShardId)); + assertNull(pressureService.getRemoteRefreshSegmentTracker(testShardId)); } public void testAfterIndexShardClosed() { @@ -102,10 +102,10 @@ public void testAfterIndexShardClosed() { when(indexShard.shardId()).thenReturn(testShardId); pressureService = new RemoteRefreshSegmentPressureService(clusterService, Settings.EMPTY); pressureService.afterIndexShardCreated(indexShard); - assertNotNull(pressureService.getPressureTracker(testShardId)); + assertNotNull(pressureService.getRemoteRefreshSegmentTracker(testShardId)); pressureService.afterIndexShardClosed(testShardId, indexShard, settings); - assertNull(pressureService.getPressureTracker(testShardId)); + assertNull(pressureService.getRemoteRefreshSegmentTracker(testShardId)); } public void testValidateSegmentUploadLag() { @@ -120,7 +120,7 @@ public void testValidateSegmentUploadLag() { pressureService.afterIndexShardCreated(indexShard); // 1. Seq no - add data points to the pressure tracker - RemoteRefreshSegmentTracker pressureTracker = pressureService.getPressureTracker(testShardId); + RemoteRefreshSegmentTracker pressureTracker = pressureService.getRemoteRefreshSegmentTracker(testShardId); pressureTracker.updateLocalRefreshSeqNo(6); Exception e = assertThrows( OpenSearchRejectedExecutionException.class, From cdfb8e0089dbd467971141a6a8e627774d41cdc4 Mon Sep 17 00:00:00 2001 From: Ashish Singh Date: Thu, 27 Apr 2023 22:12:44 +0530 Subject: [PATCH 13/17] Incorporate PR review feedback Signed-off-by: Ashish Singh --- .../opensearch/common/util/MovingAverage.java | 22 ++++++---- .../RemoteRefreshSegmentPressureService.java | 12 ++++++ .../RemoteRefreshSegmentPressureSettings.java | 2 + .../remote/RemoteRefreshSegmentTracker.java | 42 +++++++++---------- .../common/util/MovingAverageTests.java | 12 +++--- 5 files changed, 54 insertions(+), 36 deletions(-) diff --git a/server/src/main/java/org/opensearch/common/util/MovingAverage.java b/server/src/main/java/org/opensearch/common/util/MovingAverage.java index 79c762f20a293..0271c20cd155d 100644 --- a/server/src/main/java/org/opensearch/common/util/MovingAverage.java +++ b/server/src/main/java/org/opensearch/common/util/MovingAverage.java @@ -27,25 +27,29 @@ public MovingAverage(int windowSize) { this.observations = new long[windowSize]; } - public MovingAverage(int newWindowSize, MovingAverage oldMovingAverage) { - checkWindowSize(newWindowSize); - windowSize = newWindowSize; - observations = new long[newWindowSize]; - + /** + * Used for changing the window size of {@code MovingAverage}. + * + * @param newWindowSize new window size. + * @return copy of original object with updated size. + */ + public MovingAverage copyWithSize(int newWindowSize) { + MovingAverage copy = new MovingAverage(newWindowSize); // Start is inclusive, but end is exclusive - long start, end = oldMovingAverage.count; - if (oldMovingAverage.isReady() == false) { + long start, end = count; + if (isReady() == false) { start = 0; } else { - start = end - oldMovingAverage.windowSize; + start = end - windowSize; } // If the newWindow Size is smaller than the elements eligible to be copied over, then we adjust the start value if (end - start > newWindowSize) { start = end - newWindowSize; } for (int i = (int) start; i < end; i++) { - record(oldMovingAverage.observations[i % oldMovingAverage.observations.length]); + copy.record(observations[i % observations.length]); } + return copy; } private void checkWindowSize(int size) { diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureService.java b/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureService.java index df12c598b049b..02b8aacd184ee 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureService.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureService.java @@ -27,6 +27,8 @@ /** * Service used to validate if the incoming indexing request should be rejected based on the {@link RemoteRefreshSegmentTracker}. + * + * @opensearch.internal */ public class RemoteRefreshSegmentPressureService implements IndexEventListener { @@ -135,6 +137,8 @@ void updateMovingAverageWindowSize(BiConsumer(new MovingAverage(uploadBytesMovingAverageWindowSize)); - this.uploadBytesPerSecMovingAverageReference = new AtomicReference<>(new MovingAverage(uploadBytesPerSecMovingAverageWindowSize)); - this.uploadTimeMsMovingAverageReference = new AtomicReference<>(new MovingAverage(uploadTimeMsMovingAverageWindowSize)); - } - /** * ShardId for which this instance tracks the remote segment upload metadata. */ @@ -162,6 +148,22 @@ public RemoteRefreshSegmentTracker( private final Object uploadTimeMsMutex = new Object(); + public RemoteRefreshSegmentTracker( + ShardId shardId, + int uploadBytesMovingAverageWindowSize, + int uploadBytesPerSecMovingAverageWindowSize, + int uploadTimeMsMovingAverageWindowSize + ) { + 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 currentTimeMs = System.nanoTime() / 1_000_000L; + localRefreshTimeMs.set(currentTimeMs); + remoteRefreshTimeMs.set(currentTimeMs); + this.uploadBytesMovingAverageReference = new AtomicReference<>(new MovingAverage(uploadBytesMovingAverageWindowSize)); + this.uploadBytesPerSecMovingAverageReference = new AtomicReference<>(new MovingAverage(uploadBytesPerSecMovingAverageWindowSize)); + this.uploadTimeMsMovingAverageReference = new AtomicReference<>(new MovingAverage(uploadTimeMsMovingAverageWindowSize)); + } + ShardId getShardId() { return shardId; } @@ -367,7 +369,7 @@ void addUploadBytes(long size) { */ void updateUploadBytesMovingAverageWindowSize(int updatedSize) { synchronized (uploadBytesMutex) { - this.uploadBytesMovingAverageReference.set(new MovingAverage(updatedSize, this.uploadBytesMovingAverageReference.get())); + this.uploadBytesMovingAverageReference.set(this.uploadBytesMovingAverageReference.get().copyWithSize(updatedSize)); } } @@ -392,9 +394,7 @@ void addUploadBytesPerSec(long bytesPerSec) { */ void updateUploadBytesPerSecMovingAverageWindowSize(int updatedSize) { synchronized (uploadBytesPerSecMutex) { - this.uploadBytesPerSecMovingAverageReference.set( - new MovingAverage(updatedSize, this.uploadBytesPerSecMovingAverageReference.get()) - ); + this.uploadBytesPerSecMovingAverageReference.set(this.uploadBytesPerSecMovingAverageReference.get().copyWithSize(updatedSize)); } } @@ -419,7 +419,7 @@ void addUploadTimeMs(long timeMs) { */ void updateUploadTimeMsMovingAverageWindowSize(int updatedSize) { synchronized (uploadTimeMsMutex) { - this.uploadTimeMsMovingAverageReference.set(new MovingAverage(updatedSize, this.uploadTimeMsMovingAverageReference.get())); + this.uploadTimeMsMovingAverageReference.set(this.uploadTimeMsMovingAverageReference.get().copyWithSize(updatedSize)); } } } diff --git a/server/src/test/java/org/opensearch/common/util/MovingAverageTests.java b/server/src/test/java/org/opensearch/common/util/MovingAverageTests.java index 935176405a57b..4f6b0ac4f7c88 100644 --- a/server/src/test/java/org/opensearch/common/util/MovingAverageTests.java +++ b/server/src/test/java/org/opensearch/common/util/MovingAverageTests.java @@ -55,11 +55,11 @@ public void testUpdateMovingAverageWindowSize() { double avg = ma.getAverage(); // Test case 1 - Not ready and increasing size - MovingAverage newMa = new MovingAverage(10, ma); + MovingAverage newMa = ma.copyWithSize(10); assertEquals(avg, newMa.getAverage(), 0.0d); // Test case 2 - Not ready and decreasing size - newMa = new MovingAverage(2, ma); + newMa = ma.copyWithSize(2); assertEquals(2.5, newMa.getAverage(), 0.0d); // Test case 3 - Ready and increasing size @@ -68,22 +68,22 @@ public void testUpdateMovingAverageWindowSize() { ma.record(6); assertEquals(4.0, ma.getAverage(), 0.0d); - newMa = new MovingAverage(10, ma); + newMa = ma.copyWithSize(10); assertEquals(ma.getAverage(), newMa.getAverage(), 0.0d); // Test case 4 - Ready and decreasing size - newMa = new MovingAverage(3, ma); + newMa = ma.copyWithSize(3); assertEquals(5.0, newMa.getAverage(), 0.0d); // Test case 5 - Ready, array overwritten and increasing size for (int i = 7; i < 20; i++) { ma.record(i); } - newMa = new MovingAverage(6, ma); + newMa = ma.copyWithSize(6); assertEquals(ma.getAverage(), newMa.getAverage(), 0.0d); // Test case 6 - Ready, array overwritten and decreasing size - newMa = new MovingAverage(3, ma); + newMa = ma.copyWithSize(3); assertEquals(18.0, newMa.getAverage(), 0.0d); } } From 327cce89c5b630d7275448c6792cbee201af1476 Mon Sep 17 00:00:00 2001 From: Ashish Singh Date: Thu, 27 Apr 2023 22:43:42 +0530 Subject: [PATCH 14/17] Incorporate PR review feedback Signed-off-by: Ashish Singh --- .../remote/RemoteRefreshSegmentTracker.java | 112 +++++++++--------- 1 file changed, 56 insertions(+), 56 deletions(-) diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentTracker.java b/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentTracker.java index a331046a71983..50a9524291b4a 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentTracker.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentTracker.java @@ -13,6 +13,7 @@ import org.opensearch.common.util.concurrent.ConcurrentCollections; import org.opensearch.index.shard.ShardId; +import java.util.HashSet; import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicLong; @@ -34,62 +35,62 @@ public class RemoteRefreshSegmentTracker { /** * Every refresh is assigned a sequence number. This is the sequence number of the most recent refresh. */ - private final AtomicLong localRefreshSeqNo = new AtomicLong(); + private volatile long localRefreshSeqNo; /** * The refresh time of the most recent refresh. */ - private final AtomicLong localRefreshTimeMs = new AtomicLong(); + private volatile long localRefreshTimeMs; /** * Sequence number of the most recent remote refresh. */ - private final AtomicLong remoteRefreshSeqNo = new AtomicLong(); + private volatile long remoteRefreshSeqNo; /** * The refresh time of most recent remote refresh. */ - private final AtomicLong remoteRefreshTimeMs = new AtomicLong(); + private volatile long remoteRefreshTimeMs; /** * Keeps the seq no lag computed so that we do not compute it for every request. */ - private final AtomicLong seqNoLag = new AtomicLong(); + private volatile long seqNoLag; /** * Keeps the time (ms) lag computed so that we do not compute it for every request. */ - private final AtomicLong timeMsLag = new AtomicLong(); + private volatile long timeMsLag; /** * Cumulative sum of size in bytes of segment files for which upload has started during remote refresh. */ - private final AtomicLong uploadBytesStarted = new AtomicLong(); + private volatile long uploadBytesStarted; /** * Cumulative sum of size in bytes of segment files for which upload has failed during remote refresh. */ - private final AtomicLong uploadBytesFailed = new AtomicLong(); + private volatile long uploadBytesFailed; /** * Cumulative sum of size in bytes of segment files for which upload has succeeded during remote refresh. */ - private final AtomicLong uploadBytesSucceeded = new AtomicLong(); + private volatile long uploadBytesSucceeded; /** * Cumulative sum of count of remote refreshes that have started. */ - private final AtomicLong totalUploadsStarted = new AtomicLong(); + private volatile long totalUploadsStarted; /** * Cumulative sum of count of remote refreshes that have failed. */ - private final AtomicLong totalUploadsFailed = new AtomicLong(); + private volatile long totalUploadsFailed; /** * Cumulative sum of count of remote refreshes that have succeeded. */ - private final AtomicLong totalUploadsSucceeded = new AtomicLong(); + private volatile long totalUploadsSucceeded; /** * Cumulative sum of rejection counts for this shard. @@ -109,12 +110,12 @@ public class RemoteRefreshSegmentTracker { /** * Set of names of segment files that were uploaded as part of the most recent remote refresh. */ - private final Set latestUploadFiles = ConcurrentCollections.newConcurrentSet(); + private final Set latestUploadFiles = new HashSet<>(); /** * Keeps the bytes lag computed so that we do not compute it for every request. */ - private final AtomicLong bytesLag = new AtomicLong(); + private volatile long bytesLag; /** * Holds count of consecutive failures until last success. Gets reset to zero if there is a success. @@ -157,11 +158,11 @@ public RemoteRefreshSegmentTracker( 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 currentTimeMs = System.nanoTime() / 1_000_000L; - localRefreshTimeMs.set(currentTimeMs); - remoteRefreshTimeMs.set(currentTimeMs); - this.uploadBytesMovingAverageReference = new AtomicReference<>(new MovingAverage(uploadBytesMovingAverageWindowSize)); - this.uploadBytesPerSecMovingAverageReference = new AtomicReference<>(new MovingAverage(uploadBytesPerSecMovingAverageWindowSize)); - this.uploadTimeMsMovingAverageReference = new AtomicReference<>(new MovingAverage(uploadTimeMsMovingAverageWindowSize)); + localRefreshTimeMs = currentTimeMs; + remoteRefreshTimeMs = currentTimeMs; + uploadBytesMovingAverageReference = new AtomicReference<>(new MovingAverage(uploadBytesMovingAverageWindowSize)); + uploadBytesPerSecMovingAverageReference = new AtomicReference<>(new MovingAverage(uploadBytesPerSecMovingAverageWindowSize)); + uploadTimeMsMovingAverageReference = new AtomicReference<>(new MovingAverage(uploadTimeMsMovingAverageWindowSize)); } ShardId getShardId() { @@ -169,137 +170,137 @@ ShardId getShardId() { } long getLocalRefreshSeqNo() { - return localRefreshSeqNo.get(); + return localRefreshSeqNo; } void updateLocalRefreshSeqNo(long localRefreshSeqNo) { - assert localRefreshSeqNo > this.localRefreshSeqNo.get() : "newLocalRefreshSeqNo=" + assert localRefreshSeqNo > this.localRefreshSeqNo : "newLocalRefreshSeqNo=" + localRefreshSeqNo + ">=" + "currentLocalRefreshSeqNo=" - + this.localRefreshSeqNo.get(); - this.localRefreshSeqNo.set(localRefreshSeqNo); + + this.localRefreshSeqNo; + this.localRefreshSeqNo = localRefreshSeqNo; computeSeqNoLag(); } long getLocalRefreshTimeMs() { - return localRefreshTimeMs.get(); + return localRefreshTimeMs; } void updateLocalRefreshTimeMs(long localRefreshTimeMs) { - assert localRefreshTimeMs > this.localRefreshTimeMs.get() : "newLocalRefreshTimeMs=" + assert localRefreshTimeMs > this.localRefreshTimeMs : "newLocalRefreshTimeMs=" + localRefreshTimeMs + ">=" + "currentLocalRefreshTimeMs=" - + this.localRefreshTimeMs.get(); - this.localRefreshTimeMs.set(localRefreshTimeMs); + + this.localRefreshTimeMs; + this.localRefreshTimeMs = localRefreshTimeMs; computeTimeMsLag(); } long getRemoteRefreshSeqNo() { - return remoteRefreshSeqNo.get(); + return remoteRefreshSeqNo; } void updateRemoteRefreshSeqNo(long remoteRefreshSeqNo) { - assert remoteRefreshSeqNo > this.remoteRefreshSeqNo.get() : "newRemoteRefreshSeqNo=" + assert remoteRefreshSeqNo > this.remoteRefreshSeqNo : "newRemoteRefreshSeqNo=" + remoteRefreshSeqNo + ">=" + "currentRemoteRefreshSeqNo=" - + this.remoteRefreshSeqNo.get(); - this.remoteRefreshSeqNo.set(remoteRefreshSeqNo); + + this.remoteRefreshSeqNo; + this.remoteRefreshSeqNo = remoteRefreshSeqNo; computeSeqNoLag(); } long getRemoteRefreshTimeMs() { - return remoteRefreshTimeMs.get(); + return remoteRefreshTimeMs; } void updateRemoteRefreshTimeMs(long remoteRefreshTimeMs) { - assert remoteRefreshTimeMs > this.remoteRefreshTimeMs.get() : "newRemoteRefreshTimeMs=" + assert remoteRefreshTimeMs > this.remoteRefreshTimeMs : "newRemoteRefreshTimeMs=" + remoteRefreshTimeMs + ">=" + "currentRemoteRefreshTimeMs=" - + this.remoteRefreshTimeMs.get(); - this.remoteRefreshTimeMs.set(remoteRefreshTimeMs); + + this.remoteRefreshTimeMs; + this.remoteRefreshTimeMs = remoteRefreshTimeMs; computeTimeMsLag(); } private void computeSeqNoLag() { - seqNoLag.set(localRefreshSeqNo.get() - remoteRefreshSeqNo.get()); + seqNoLag = localRefreshSeqNo - remoteRefreshSeqNo; } long getSeqNoLag() { - return seqNoLag.get(); + return seqNoLag; } private void computeTimeMsLag() { - timeMsLag.set(localRefreshTimeMs.get() - remoteRefreshTimeMs.get()); + timeMsLag = localRefreshTimeMs - remoteRefreshTimeMs; } long getTimeMsLag() { - return timeMsLag.get(); + return timeMsLag; } long getBytesLag() { - return bytesLag.get(); + return bytesLag; } long getUploadBytesStarted() { - return uploadBytesStarted.get(); + return uploadBytesStarted; } void addUploadBytesStarted(long size) { - uploadBytesStarted.addAndGet(size); + uploadBytesStarted += size; } long getUploadBytesFailed() { - return uploadBytesFailed.get(); + return uploadBytesFailed; } void addUploadBytesFailed(long size) { - uploadBytesFailed.addAndGet(size); + uploadBytesFailed += size; } long getUploadBytesSucceeded() { - return uploadBytesSucceeded.get(); + return uploadBytesSucceeded; } void addUploadBytesSucceeded(long size) { - uploadBytesSucceeded.addAndGet(size); + uploadBytesSucceeded += size; } long getInflightUploadBytes() { - return uploadBytesStarted.get() - uploadBytesFailed.get() - uploadBytesSucceeded.get(); + return uploadBytesStarted - uploadBytesFailed - uploadBytesSucceeded; } long getTotalUploadsStarted() { - return totalUploadsStarted.get(); + return totalUploadsStarted; } void incrementTotalUploadsStarted() { - totalUploadsStarted.incrementAndGet(); + totalUploadsStarted += 1; } long getTotalUploadsFailed() { - return totalUploadsFailed.get(); + return totalUploadsFailed; } void incrementTotalUploadsFailed() { - totalUploadsFailed.incrementAndGet(); + totalUploadsFailed += 1; failures.record(true); } long getTotalUploadsSucceeded() { - return totalUploadsSucceeded.get(); + return totalUploadsSucceeded; } void incrementTotalUploadSucceeded() { - totalUploadsSucceeded.incrementAndGet(); + totalUploadsSucceeded += 1; failures.record(false); } long getInflightUploads() { - return totalUploadsStarted.get() - totalUploadsFailed.get() - totalUploadsSucceeded.get(); + return totalUploadsStarted - totalUploadsFailed - totalUploadsSucceeded; } long getRejectionCount() { @@ -340,8 +341,7 @@ private void computeBytesLag() { .stream() .filter(f -> !latestUploadFiles.contains(f)) .collect(Collectors.toSet()); - long bytesLag = filesNotYetUploaded.stream().map(latestLocalFileNameLengthMap::get).mapToLong(Long::longValue).sum(); - this.bytesLag.set(bytesLag); + this.bytesLag = filesNotYetUploaded.stream().map(latestLocalFileNameLengthMap::get).mapToLong(Long::longValue).sum(); } int getConsecutiveFailureCount() { From 518ef11038deef1fd0781982ce4cbb1d295e7d28 Mon Sep 17 00:00:00 2001 From: Ashish Singh Date: Fri, 28 Apr 2023 00:36:44 +0530 Subject: [PATCH 15/17] Incorporate PR review feedback Signed-off-by: Ashish Singh --- ...oteRefreshSegmentPressureServiceTests.java | 68 ++++++++----------- 1 file changed, 27 insertions(+), 41 deletions(-) diff --git a/server/src/test/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureServiceTests.java b/server/src/test/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureServiceTests.java index 39a49100ec755..c5a6c0323a6f9 100644 --- a/server/src/test/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureServiceTests.java +++ b/server/src/test/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureServiceTests.java @@ -70,62 +70,39 @@ public void testIsSegmentsUploadBackpressureEnabled() { } public void testAfterIndexShardCreatedForRemoteBackedIndex() { - Settings settings = Settings.builder().put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, "true").build(); - IndexSettings indexSettings = IndexSettingsModule.newIndexSettings("test_index", settings); - ShardId testShardId = new ShardId("index", "uuid", 0); - IndexShard indexShard = mock(IndexShard.class); - when(indexShard.indexSettings()).thenReturn(indexSettings); - when(indexShard.shardId()).thenReturn(testShardId); + IndexShard indexShard = createIndexShard(shardId, true); pressureService = new RemoteRefreshSegmentPressureService(clusterService, Settings.EMPTY); pressureService.afterIndexShardCreated(indexShard); - assertNotNull(pressureService.getRemoteRefreshSegmentTracker(testShardId)); + assertNotNull(pressureService.getRemoteRefreshSegmentTracker(indexShard.shardId())); } public void testAfterIndexShardCreatedForNonRemoteBackedIndex() { - Settings settings = Settings.builder().put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, "false").build(); - IndexSettings indexSettings = IndexSettingsModule.newIndexSettings("test_index", settings); - ShardId testShardId = new ShardId("index", "uuid", 0); - IndexShard indexShard = mock(IndexShard.class); - when(indexShard.indexSettings()).thenReturn(indexSettings); - when(indexShard.shardId()).thenReturn(testShardId); + IndexShard indexShard = createIndexShard(shardId, false); pressureService = new RemoteRefreshSegmentPressureService(clusterService, Settings.EMPTY); pressureService.afterIndexShardCreated(indexShard); - assertNull(pressureService.getRemoteRefreshSegmentTracker(testShardId)); + assertNull(pressureService.getRemoteRefreshSegmentTracker(indexShard.shardId())); } public void testAfterIndexShardClosed() { - Settings settings = Settings.builder().put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, "true").build(); - IndexSettings indexSettings = IndexSettingsModule.newIndexSettings("test_index", settings); - ShardId testShardId = new ShardId("index", "uuid", 0); - IndexShard indexShard = mock(IndexShard.class); - when(indexShard.indexSettings()).thenReturn(indexSettings); - when(indexShard.shardId()).thenReturn(testShardId); + IndexShard indexShard = createIndexShard(shardId, true); pressureService = new RemoteRefreshSegmentPressureService(clusterService, Settings.EMPTY); pressureService.afterIndexShardCreated(indexShard); - assertNotNull(pressureService.getRemoteRefreshSegmentTracker(testShardId)); + assertNotNull(pressureService.getRemoteRefreshSegmentTracker(shardId)); - pressureService.afterIndexShardClosed(testShardId, indexShard, settings); - assertNull(pressureService.getRemoteRefreshSegmentTracker(testShardId)); + pressureService.afterIndexShardClosed(shardId, indexShard, indexShard.indexSettings().getSettings()); + assertNull(pressureService.getRemoteRefreshSegmentTracker(shardId)); } public void testValidateSegmentUploadLag() { // Create the pressure tracker - Settings settings = Settings.builder().put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, "true").build(); - IndexSettings indexSettings = IndexSettingsModule.newIndexSettings("test_index", settings); - ShardId testShardId = new ShardId("index", "uuid", 0); - IndexShard indexShard = mock(IndexShard.class); - when(indexShard.indexSettings()).thenReturn(indexSettings); - when(indexShard.shardId()).thenReturn(testShardId); + IndexShard indexShard = createIndexShard(shardId, true); pressureService = new RemoteRefreshSegmentPressureService(clusterService, Settings.EMPTY); pressureService.afterIndexShardCreated(indexShard); // 1. Seq no - add data points to the pressure tracker - RemoteRefreshSegmentTracker pressureTracker = pressureService.getRemoteRefreshSegmentTracker(testShardId); + RemoteRefreshSegmentTracker pressureTracker = pressureService.getRemoteRefreshSegmentTracker(shardId); pressureTracker.updateLocalRefreshSeqNo(6); - Exception e = assertThrows( - OpenSearchRejectedExecutionException.class, - () -> pressureService.validateSegmentsUploadLag(testShardId) - ); + Exception e = assertThrows(OpenSearchRejectedExecutionException.class, () -> pressureService.validateSegmentsUploadLag(shardId)); assertTrue(e.getMessage().contains("due to remote segments lagging behind local segments")); assertTrue(e.getMessage().contains("remote_refresh_seq_no:0 local_refresh_seq_no:6")); @@ -140,12 +117,12 @@ public void testValidateSegmentUploadLag() { long currentMs = System.nanoTime() / 1_000_000; pressureTracker.updateLocalRefreshTimeMs((long) (currentMs + 4 * avg)); pressureTracker.updateRemoteRefreshTimeMs(currentMs); - e = assertThrows(OpenSearchRejectedExecutionException.class, () -> pressureService.validateSegmentsUploadLag(testShardId)); + e = assertThrows(OpenSearchRejectedExecutionException.class, () -> pressureService.validateSegmentsUploadLag(shardId)); assertTrue(e.getMessage().contains("due to remote segments lagging behind local segments")); assertTrue(e.getMessage().contains("time_lag:38 ms dynamic_time_lag_threshold:19.0 ms")); pressureTracker.updateRemoteRefreshTimeMs((long) (currentMs + 2 * avg)); - pressureService.validateSegmentsUploadLag(testShardId); + pressureService.validateSegmentsUploadLag(shardId); // 3. bytes lag more than dynamic threshold sum.set(0); @@ -157,23 +134,32 @@ public void testValidateSegmentUploadLag() { Map nameSizeMap = new HashMap<>(); nameSizeMap.put("a", (long) (4 * avg)); pressureTracker.setLatestLocalFileNameLengthMap(nameSizeMap); - e = assertThrows(OpenSearchRejectedExecutionException.class, () -> pressureService.validateSegmentsUploadLag(testShardId)); + e = assertThrows(OpenSearchRejectedExecutionException.class, () -> pressureService.validateSegmentsUploadLag(shardId)); assertTrue(e.getMessage().contains("due to remote segments lagging behind local segments")); assertTrue(e.getMessage().contains("bytes_lag:38 dynamic_bytes_lag_threshold:19.0")); nameSizeMap.put("a", (long) (2 * avg)); pressureTracker.setLatestLocalFileNameLengthMap(nameSizeMap); - pressureService.validateSegmentsUploadLag(testShardId); + pressureService.validateSegmentsUploadLag(shardId); // 4. Consecutive failures more than the limit IntStream.range(0, 10).forEach(ignore -> pressureTracker.incrementTotalUploadsFailed()); - pressureService.validateSegmentsUploadLag(testShardId); + pressureService.validateSegmentsUploadLag(shardId); pressureTracker.incrementTotalUploadsFailed(); - e = assertThrows(OpenSearchRejectedExecutionException.class, () -> pressureService.validateSegmentsUploadLag(testShardId)); + 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:11 min_consecutive_failure_threshold:10")); pressureTracker.incrementTotalUploadSucceeded(); - pressureService.validateSegmentsUploadLag(testShardId); + pressureService.validateSegmentsUploadLag(shardId); + } + + private static IndexShard createIndexShard(ShardId shardId, boolean remoteStoreEnabled) { + Settings settings = Settings.builder().put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, String.valueOf(remoteStoreEnabled)).build(); + IndexSettings indexSettings = IndexSettingsModule.newIndexSettings("test_index", settings); + IndexShard indexShard = mock(IndexShard.class); + when(indexShard.indexSettings()).thenReturn(indexSettings); + when(indexShard.shardId()).thenReturn(shardId); + return indexShard; } } From 008a0fafac3766d36e8782acfc65ee95dd1b486d Mon Sep 17 00:00:00 2001 From: Ashish Singh Date: Fri, 28 Apr 2023 10:12:45 +0530 Subject: [PATCH 16/17] Incorporate PR review feedback Signed-off-by: Ashish Singh --- .../main/java/org/opensearch/common/util/MovingAverage.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/opensearch/common/util/MovingAverage.java b/server/src/main/java/org/opensearch/common/util/MovingAverage.java index 0271c20cd155d..50d863709d489 100644 --- a/server/src/main/java/org/opensearch/common/util/MovingAverage.java +++ b/server/src/main/java/org/opensearch/common/util/MovingAverage.java @@ -17,9 +17,9 @@ public class MovingAverage { private final int windowSize; private final long[] observations; - private long count = 0; - private long sum = 0; - private double average = 0; + private volatile long count = 0; + private volatile long sum = 0; + private volatile double average = 0; public MovingAverage(int windowSize) { checkWindowSize(windowSize); From c82ffb775b582a53b6879f57e3c5da3d6e95113f Mon Sep 17 00:00:00 2001 From: Ashish Singh Date: Tue, 2 May 2023 16:31:07 +0530 Subject: [PATCH 17/17] Incorporate PR review feedback Signed-off-by: Ashish Singh --- .../RemoteRefreshSegmentPressureService.java | 36 ++++++++++++++++--- .../RemoteRefreshSegmentPressureSettings.java | 14 ++++---- .../remote/RemoteRefreshSegmentTracker.java | 14 ++++---- ...teRefreshSegmentPressureSettingsTests.java | 8 ++--- .../RemoteRefreshSegmentTrackerTests.java | 4 +-- 5 files changed, 51 insertions(+), 25 deletions(-) diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureService.java b/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureService.java index 02b8aacd184ee..37935cc0eb29d 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureService.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureService.java @@ -107,7 +107,7 @@ public void validateSegmentsUploadLag(ShardId shardId) { RemoteRefreshSegmentTracker remoteRefreshSegmentTracker = getRemoteRefreshSegmentTracker(shardId); // Check if refresh checkpoint (a.k.a. seq number) lag is 2 or below - this is to handle segment merges that can // increase the bytes to upload almost suddenly. - if (remoteRefreshSegmentTracker.getSeqNoLag() <= 1) { + if (remoteRefreshSegmentTracker.getRefreshSeqNoLag() <= 1) { return; } @@ -162,9 +162,7 @@ private LagValidator(RemoteRefreshSegmentPressureSettings pressureSettings) { * * @return the name using class name. */ - final String name() { - return this.getClass().getSimpleName(); - } + abstract String name(); abstract String rejectionMessage(RemoteRefreshSegmentTracker pressureTracker, ShardId shardId); } @@ -176,6 +174,8 @@ final String name() { */ private static class RefreshSeqNoLagValidator extends LagValidator { + private static final String NAME = "refresh_seq_no_lag"; + private RefreshSeqNoLagValidator(RemoteRefreshSegmentPressureSettings pressureSettings) { super(pressureSettings); } @@ -183,7 +183,7 @@ private RefreshSeqNoLagValidator(RemoteRefreshSegmentPressureSettings pressureSe @Override public boolean validate(RemoteRefreshSegmentTracker pressureTracker, ShardId shardId) { // Check if the remote store seq no lag is above the min seq no lag limit - return pressureTracker.getSeqNoLag() <= pressureSettings.getMinSeqNoLagLimit(); + return pressureTracker.getRefreshSeqNoLag() <= pressureSettings.getMinRefreshSeqNoLagLimit(); } @Override @@ -197,6 +197,11 @@ String rejectionMessage(RemoteRefreshSegmentTracker pressureTracker, ShardId sha pressureTracker.getLocalRefreshSeqNo() ); } + + @Override + String name() { + return NAME; + } } /** @@ -206,6 +211,8 @@ String rejectionMessage(RemoteRefreshSegmentTracker pressureTracker, ShardId sha */ private static class BytesLagValidator extends LagValidator { + private static final String NAME = "bytes_lag"; + private BytesLagValidator(RemoteRefreshSegmentPressureSettings pressureSettings) { super(pressureSettings); } @@ -233,6 +240,11 @@ public String rejectionMessage(RemoteRefreshSegmentTracker pressureTracker, Shar dynamicBytesLagThreshold ); } + + @Override + String name() { + return NAME; + } } /** @@ -242,6 +254,8 @@ public String rejectionMessage(RemoteRefreshSegmentTracker pressureTracker, Shar */ private static class TimeLagValidator extends LagValidator { + private static final String NAME = "time_lag"; + private TimeLagValidator(RemoteRefreshSegmentPressureSettings pressureSettings) { super(pressureSettings); } @@ -269,6 +283,11 @@ public String rejectionMessage(RemoteRefreshSegmentTracker pressureTracker, Shar dynamicTimeLagThreshold ); } + + @Override + String name() { + return NAME; + } } /** @@ -278,6 +297,8 @@ public String rejectionMessage(RemoteRefreshSegmentTracker pressureTracker, Shar */ private static class ConsecutiveFailureValidator extends LagValidator { + private static final String NAME = "consecutive_failures_lag"; + private ConsecutiveFailureValidator(RemoteRefreshSegmentPressureSettings pressureSettings) { super(pressureSettings); } @@ -300,5 +321,10 @@ public String rejectionMessage(RemoteRefreshSegmentTracker pressureTracker, Shar pressureSettings.getMinConsecutiveFailuresLimit() ); } + + @Override + String name() { + return NAME; + } } } diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureSettings.java b/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureSettings.java index 95bd64d6935a6..6cb0d1d07e78b 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureSettings.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureSettings.java @@ -99,7 +99,7 @@ private static class Defaults { private volatile boolean remoteRefreshSegmentPressureEnabled; - private volatile long minSeqNoLagLimit; + private volatile long minRefreshSeqNoLagLimit; private volatile double bytesLagVarianceFactor; @@ -123,8 +123,8 @@ public RemoteRefreshSegmentPressureSettings( this.remoteRefreshSegmentPressureEnabled = REMOTE_REFRESH_SEGMENT_PRESSURE_ENABLED.get(settings); clusterSettings.addSettingsUpdateConsumer(REMOTE_REFRESH_SEGMENT_PRESSURE_ENABLED, this::setRemoteRefreshSegmentPressureEnabled); - this.minSeqNoLagLimit = MIN_SEQ_NO_LAG_LIMIT.get(settings); - clusterSettings.addSettingsUpdateConsumer(MIN_SEQ_NO_LAG_LIMIT, this::setMinSeqNoLagLimit); + this.minRefreshSeqNoLagLimit = MIN_SEQ_NO_LAG_LIMIT.get(settings); + clusterSettings.addSettingsUpdateConsumer(MIN_SEQ_NO_LAG_LIMIT, this::setMinRefreshSeqNoLagLimit); this.bytesLagVarianceFactor = BYTES_LAG_VARIANCE_FACTOR.get(settings); clusterSettings.addSettingsUpdateConsumer(BYTES_LAG_VARIANCE_FACTOR, this::setBytesLagVarianceFactor); @@ -168,12 +168,12 @@ public void setRemoteRefreshSegmentPressureEnabled(boolean remoteRefreshSegmentP this.remoteRefreshSegmentPressureEnabled = remoteRefreshSegmentPressureEnabled; } - public long getMinSeqNoLagLimit() { - return minSeqNoLagLimit; + public long getMinRefreshSeqNoLagLimit() { + return minRefreshSeqNoLagLimit; } - public void setMinSeqNoLagLimit(long minSeqNoLagLimit) { - this.minSeqNoLagLimit = minSeqNoLagLimit; + public void setMinRefreshSeqNoLagLimit(long minRefreshSeqNoLagLimit) { + this.minRefreshSeqNoLagLimit = minRefreshSeqNoLagLimit; } public double getBytesLagVarianceFactor() { diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentTracker.java b/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentTracker.java index 50a9524291b4a..109eadf34509b 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentTracker.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentTracker.java @@ -55,7 +55,7 @@ public class RemoteRefreshSegmentTracker { /** * Keeps the seq no lag computed so that we do not compute it for every request. */ - private volatile long seqNoLag; + private volatile long refreshSeqNoLag; /** * Keeps the time (ms) lag computed so that we do not compute it for every request. @@ -180,7 +180,7 @@ void updateLocalRefreshSeqNo(long localRefreshSeqNo) { + "currentLocalRefreshSeqNo=" + this.localRefreshSeqNo; this.localRefreshSeqNo = localRefreshSeqNo; - computeSeqNoLag(); + computeRefreshSeqNoLag(); } long getLocalRefreshTimeMs() { @@ -208,7 +208,7 @@ void updateRemoteRefreshSeqNo(long remoteRefreshSeqNo) { + "currentRemoteRefreshSeqNo=" + this.remoteRefreshSeqNo; this.remoteRefreshSeqNo = remoteRefreshSeqNo; - computeSeqNoLag(); + computeRefreshSeqNoLag(); } long getRemoteRefreshTimeMs() { @@ -225,12 +225,12 @@ void updateRemoteRefreshTimeMs(long remoteRefreshTimeMs) { computeTimeMsLag(); } - private void computeSeqNoLag() { - seqNoLag = localRefreshSeqNo - remoteRefreshSeqNo; + private void computeRefreshSeqNoLag() { + refreshSeqNoLag = localRefreshSeqNo - remoteRefreshSeqNo; } - long getSeqNoLag() { - return seqNoLag; + long getRefreshSeqNoLag() { + return refreshSeqNoLag; } private void computeTimeMsLag() { diff --git a/server/src/test/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureSettingsTests.java b/server/src/test/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureSettingsTests.java index ed88f91ebd062..66b5d6c4c19d8 100644 --- a/server/src/test/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureSettingsTests.java +++ b/server/src/test/java/org/opensearch/index/remote/RemoteRefreshSegmentPressureSettingsTests.java @@ -55,7 +55,7 @@ public void testGetDefaultSettings() { assertFalse(pressureSettings.isRemoteRefreshSegmentPressureEnabled()); // Check min sequence number lag limit default value - assertEquals(5L, pressureSettings.getMinSeqNoLagLimit()); + assertEquals(5L, pressureSettings.getMinRefreshSeqNoLagLimit()); // Check bytes lag variance threshold default value assertEquals(2.0, pressureSettings.getBytesLagVarianceFactor(), 0.0d); @@ -97,7 +97,7 @@ public void testGetConfiguredSettings() { assertTrue(pressureSettings.isRemoteRefreshSegmentPressureEnabled()); // Check min sequence number lag limit configured value - assertEquals(100L, pressureSettings.getMinSeqNoLagLimit()); + assertEquals(100L, pressureSettings.getMinRefreshSeqNoLagLimit()); // Check bytes lag variance threshold configured value assertEquals(50.0, pressureSettings.getBytesLagVarianceFactor(), 0.0d); @@ -141,7 +141,7 @@ public void testUpdateAfterGetDefaultSettings() { assertTrue(pressureSettings.isRemoteRefreshSegmentPressureEnabled()); // Check min sequence number lag limit - assertEquals(100L, pressureSettings.getMinSeqNoLagLimit()); + assertEquals(100L, pressureSettings.getMinRefreshSeqNoLagLimit()); // Check bytes lag variance threshold updated assertEquals(50.0, pressureSettings.getBytesLagVarianceFactor(), 0.0d); @@ -195,7 +195,7 @@ public void testUpdateAfterGetConfiguredSettings() { assertTrue(pressureSettings.isRemoteRefreshSegmentPressureEnabled()); // Check min sequence number lag limit - assertEquals(80L, pressureSettings.getMinSeqNoLagLimit()); + assertEquals(80L, pressureSettings.getMinRefreshSeqNoLagLimit()); // Check bytes lag variance threshold updated assertEquals(40.0, pressureSettings.getBytesLagVarianceFactor(), 0.0d); diff --git a/server/src/test/java/org/opensearch/index/remote/RemoteRefreshSegmentTrackerTests.java b/server/src/test/java/org/opensearch/index/remote/RemoteRefreshSegmentTrackerTests.java index ffa591c3c886b..48bc28e3a497d 100644 --- a/server/src/test/java/org/opensearch/index/remote/RemoteRefreshSegmentTrackerTests.java +++ b/server/src/test/java/org/opensearch/index/remote/RemoteRefreshSegmentTrackerTests.java @@ -124,9 +124,9 @@ public void testComputeSeqNoLagOnUpdate() { int localRefreshSeqNo = randomIntBetween(50, 100); int remoteRefreshSeqNo = randomIntBetween(20, 50); pressureTracker.updateLocalRefreshSeqNo(localRefreshSeqNo); - assertEquals(localRefreshSeqNo, pressureTracker.getSeqNoLag()); + assertEquals(localRefreshSeqNo, pressureTracker.getRefreshSeqNoLag()); pressureTracker.updateRemoteRefreshSeqNo(remoteRefreshSeqNo); - assertEquals(localRefreshSeqNo - remoteRefreshSeqNo, pressureTracker.getSeqNoLag()); + assertEquals(localRefreshSeqNo - remoteRefreshSeqNo, pressureTracker.getRefreshSeqNoLag()); } public void testComputeTimeLagOnUpdate() {